You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by ja...@apache.org on 2015/01/06 02:46:47 UTC

ambari git commit: AMBARI-8935. JobHistoryServer Fails to pass service check in Kerberized cluster. (robert levas via jaimin)

Repository: ambari
Updated Branches:
  refs/heads/trunk 2ed76c80e -> 1d1ebb487


AMBARI-8935. JobHistoryServer Fails to pass service check in Kerberized cluster. (robert levas via jaimin)


Project: http://git-wip-us.apache.org/repos/asf/ambari/repo
Commit: http://git-wip-us.apache.org/repos/asf/ambari/commit/1d1ebb48
Tree: http://git-wip-us.apache.org/repos/asf/ambari/tree/1d1ebb48
Diff: http://git-wip-us.apache.org/repos/asf/ambari/diff/1d1ebb48

Branch: refs/heads/trunk
Commit: 1d1ebb487e506aed9d76b6c44561435488d8797c
Parents: 2ed76c8
Author: Jaimin Jetly <ja...@hortonworks.com>
Authored: Mon Jan 5 17:45:30 2015 -0800
Committer: Jaimin Jetly <ja...@hortonworks.com>
Committed: Mon Jan 5 17:45:39 2015 -0800

----------------------------------------------------------------------
 .../server/controller/AuthToLocalBuilder.java   | 133 +++++++++++++++++++
 .../server/controller/KerberosHelper.java       | 102 +++++++++++---
 .../kerberos/KerberosPrincipalDescriptor.java   |  40 ++++++
 .../HDP/2.2/services/FALCON/kerberos.json       |   3 +-
 .../stacks/HDP/2.2/services/HBASE/kerberos.json |   9 +-
 .../stacks/HDP/2.2/services/HDFS/kerberos.json  |  14 +-
 .../stacks/HDP/2.2/services/OOZIE/kerberos.json |   5 +-
 .../stacks/HDP/2.2/services/YARN/kerberos.json  |  12 +-
 .../controller/AuthToLocalBuilderTest.java      |  88 ++++++++++++
 .../KerberosPrincipalDescriptorTest.java        |   7 +-
 .../resources/stacks/HDP/2.0.8/kerberos.json    |   2 +-
 .../HDP/2.0.8/services/HDFS/kerberos.json       |   9 +-
 12 files changed, 386 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/java/org/apache/ambari/server/controller/AuthToLocalBuilder.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/AuthToLocalBuilder.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/AuthToLocalBuilder.java
new file mode 100644
index 0000000..6017bed
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/AuthToLocalBuilder.java
@@ -0,0 +1,133 @@
+/*
+ * 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.ambari.server.controller;
+
+import java.util.Map;
+import java.util.TreeMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * AuthToLocalBuilder helps to create auth_to_local rules for use in configuration files like
+ * core-site.xml.
+ * <p/>
+ * For each principal appended to the rule set, parse out the primary value and match it to a local
+ * username.  Then when done appending all principals, generate the rules where each entry yields
+ * one of the following rule:
+ * <p/>
+ * Qualified Principal: RULE:[2:$1@$0](PRIMARY@REALM)s/.*\/LOCAL_USERNAME/
+ * <p/>
+ * Unqualified Principal: RULE:[1:$1@$0](PRIMARY@REALM)s/.*\/LOCAL_USERNAME/
+ */
+public class AuthToLocalBuilder {
+
+  /**
+   * A Regular expression declaring a qualified principal such that the principal is in the following format:
+   * primary/instance@REALM
+   */
+  private static final Pattern PATTERN_QUALIFIED_PRINCIPAL = Pattern.compile("(\\w+)/.*@.*");
+
+  /**
+   * A Regular expression declaring an un qualified principal such that the principal is in the following format:
+   * primary@REALM
+   */
+  private static final Pattern PATTERN_UNQUALIFIED_PRINCIPAL = Pattern.compile("(\\w+)@.*");
+
+  /**
+   * A map of qualified principal names (primary/instance@REALM, with instance and @REALM removed).
+   * <p/>
+   * A TreeMap is used to help generate deterministic ordering of rules for testing.
+   */
+  private Map<String, String> qualifiedAuthToLocalMap = new TreeMap<String, String>();
+
+  /**
+   * A map of unqualified principal names (primary@REALM, with @REALM removed).
+   * <p/>
+   * A TreeMap is used to help generate deterministic ordering of rules for testing.
+   */
+  private Map<String, String> unqualifiedAuthToLocalMap = new TreeMap<String, String>();
+
+
+  /**
+   * Appends a principal and local username mapping to the builder.
+   * <p/>
+   * The supplied principal is parsed to determine if it is qualified or unqualified and stored
+   * accordingly so that when the mapping rules are generated the appropriate rule is generated.
+   * <p/>
+   * If a principal is added that yields a duplicate primary principal value (relative to the set of
+   * qualified or unqualified rules), that later entry will overwrite the older entry, allowing for
+   * only one mapping rule.
+   * <p/>
+   * If the principal does not match one of the two expected patterns, it will be ignored.
+   *
+   * @param principal     a String containing the full principal to append
+   * @param localUsername a String declaring that local username to map the principal to
+   */
+  public void append(String principal, String localUsername) {
+    if ((principal != null) && (localUsername != null) && !principal.isEmpty() && !localUsername.isEmpty()) {
+      // Determine if the principal is contains an instance declaration
+      Matcher matcher;
+
+      matcher = PATTERN_QUALIFIED_PRINCIPAL.matcher(principal);
+      if (matcher.matches()) {
+        qualifiedAuthToLocalMap.put(matcher.group(1), localUsername);
+      } else {
+        matcher = PATTERN_UNQUALIFIED_PRINCIPAL.matcher(principal);
+        if (matcher.matches()) {
+          unqualifiedAuthToLocalMap.put(matcher.group(1), localUsername);
+        }
+      }
+    }
+  }
+
+  /**
+   * Generates the auth_to_local rules used by configuration settings such as core-site/auth_to_local.
+   *
+   * @param realm a String declaring the realm to use in rule set
+   *
+   */
+  public String generate(String realm) {
+
+    StringBuilder builder = new StringBuilder();
+
+    for (Map.Entry<String, String> entry : qualifiedAuthToLocalMap.entrySet()) {
+      // RULE:[2:$1@$0](PRIMARY@REALM)s/.*/LOCAL_USERNAME/
+      appendRule(builder, String.format("RULE:[2:$1@$0](%s@%s)s/.*/%s/", entry.getKey(), realm, entry.getValue()));
+    }
+
+    for (Map.Entry<String, String> entry : unqualifiedAuthToLocalMap.entrySet()) {
+      // RULE:[1:$1@$0](PRIMARY@REALM)s/.*/LOCAL_USERNAME/
+      appendRule(builder, String.format("RULE:[1:$1@$0](%s@%s)s/.*/%s/", entry.getKey(), realm, entry.getValue()));
+    }
+
+    // RULE:[1:$1@$0](.*@YOUR.REALM)s/@.*//
+    appendRule(builder, String.format("RULE:[1:$1@$0](.*@%s)s/@.*//", realm));
+
+    appendRule(builder, "DEFAULT");
+
+    return builder.toString();
+  }
+
+  private void appendRule(StringBuilder stringBuilder, String rule) {
+    if (stringBuilder.length() > 0) {
+      stringBuilder.append('\n');
+    }
+    stringBuilder.append(rule);
+  }
+}

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
index c7d08cc..0533228 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/KerberosHelper.java
@@ -62,7 +62,9 @@ import org.slf4j.LoggerFactory;
 import java.io.File;
 import java.io.IOException;
 import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -252,6 +254,7 @@ public class KerberosHelper {
         KerberosActionDataFileBuilder kerberosActionDataFileBuilder = null;
         Map<String, String> kerberosDescriptorProperties = kerberosDescriptor.getProperties();
         Map<String, Map<String, String>> kerberosConfigurations = new HashMap<String, Map<String, String>>();
+        AuthToLocalBuilder authToLocalBuilder = new AuthToLocalBuilder();
 
         // Create a temporary directory to store metadata needed to complete this task.  Information
         // such as which principals and keytabs files to create as well as what configurations need
@@ -269,9 +272,6 @@ public class KerberosHelper {
         // Create the file used to store details about principals and keytabs to create
         indexFile = new File(dataDirectory, KerberosActionDataFile.DATA_FILE_NAME);
 
-        // host names that would be passed in request resource filters while creating stage for pushing keytabs
-        List<String>  updateHosts = new ArrayList<String>();
-
         try {
           // Iterate over the hosts in the cluster to find the components installed in each.  For each
           // component (aka service component host - sch) determine the configuration updates and
@@ -324,9 +324,12 @@ public class KerberosHelper {
 
                   if (serviceDescriptor != null) {
                     KerberosComponentDescriptor componentDescriptor = serviceDescriptor.getComponent(sch.getServiceComponentName());
+                    List<KerberosIdentityDescriptor> serviceIdentities = serviceDescriptor.getIdentities(true);
 
                     if (componentDescriptor != null) {
+                      List<KerberosIdentityDescriptor> componentIdentities = componentDescriptor.getIdentities(true);
                       int identitiesAdded = 0;
+
                       // Test to see if this component should be process by querying the handler
                       if (handler.shouldProcess(desiredSecurityState, sch)) {
                         // Calculate the set of configurations to update and replace any variables
@@ -341,20 +344,23 @@ public class KerberosHelper {
 
                         // Add service-level principals (and keytabs)
                         identitiesAdded += addIdentities(kerberosActionDataFileBuilder,
-                            serviceDescriptor.getIdentities(true), sch, configurations);
+                            serviceIdentities, sch, configurations);
 
                         // Add component-level principals (and keytabs)
                         identitiesAdded += addIdentities(kerberosActionDataFileBuilder,
-                            componentDescriptor.getIdentities(true), sch, configurations);
-
-                        // add host to updateHosts that would be passed in request resource filters 
-                        updateHosts.add(sch.getHostName());
+                            componentIdentities, sch, configurations);
 
                         if (identitiesAdded > 0) {
                           serviceComponentHostsToProcess.add(sch);
                         }
                       }
+
+                      // Add component-level principals to auth_to_local builder
+                      addIdentities(authToLocalBuilder, componentIdentities, configurations);
                     }
+
+                    // Add service-level principals to auth_to_local builder
+                    addIdentities(authToLocalBuilder, serviceIdentities, configurations);
                   }
                 }
               }
@@ -387,6 +393,21 @@ public class KerberosHelper {
             }
             throw new AmbariException("Missing KDC administrator credentials");
           }
+
+          // Determine if the any auth_to_local configurations need to be set dynamically
+          // Lazily create the auth_to_local rules
+          String authToLocal = null;
+          for(Map<String, String> configuration: kerberosConfigurations.values()) {
+            for(Map.Entry<String,String> entry: configuration.entrySet()) {
+              if("_AUTH_TO_LOCAL_RULES".equals(entry.getValue())) {
+                if (authToLocal == null) {
+                  authToLocal = authToLocalBuilder.generate(realm);
+                }
+
+                entry.setValue(authToLocal);
+              }
+            }
+          }
         }
 
         // Always set up the necessary stages to perform the tasks needed to complete the operation.
@@ -415,7 +436,7 @@ public class KerberosHelper {
         // Use the handler implementation to setup the relevant stages.
         int lastStageId = handler.createStages(cluster, hosts, kerberosConfigurations,
             clusterHostInfoJson, hostParamsJson, event, roleCommandOrder, realm, kdcType.toString(),
-            dataDirectory, requestStageContainer, updateHosts);
+            dataDirectory, requestStageContainer, serviceComponentHostsToProcess);
 
         // Add the cleanup stage...
 
@@ -584,8 +605,7 @@ public class KerberosHelper {
    * @throws java.io.IOException if an error occurs while writing a record to the data file
    */
   private int addIdentities(KerberosActionDataFileBuilder kerberosActionDataFileBuilder,
-                            List<KerberosIdentityDescriptor> identities,
-                            ServiceComponentHost sch,
+                            List<KerberosIdentityDescriptor> identities, ServiceComponentHost sch,
                             Map<String, Map<String, String>> configurations) throws IOException {
     int identitiesAdded = 0;
 
@@ -618,6 +638,7 @@ public class KerberosHelper {
             keytabFileConfiguration = KerberosDescriptor.replaceVariables(keytabDescriptor.getConfiguration(), configurations);
           }
 
+          // Append an entry to the action data file builder...
           kerberosActionDataFileBuilder.addRecord(sch.getHostName(),
               sch.getServiceName(),
               sch.getServiceComponentName(),
@@ -639,6 +660,30 @@ public class KerberosHelper {
   }
 
   /**
+   * Adds identities to the AuthToLocalBuilder.
+   *
+   * @param authToLocalBuilder the AuthToLocalBuilder to use to build the auth_to_local mapping
+   * @param identities         a List of KerberosIdentityDescriptors to process
+   * @param configurations     a Map of configurations to use a replacements for variables
+   *                           in identity fields
+   * @throws org.apache.ambari.server.AmbariException
+   */
+  private void addIdentities(AuthToLocalBuilder authToLocalBuilder,
+                             List<KerberosIdentityDescriptor> identities,
+                             Map<String, Map<String, String>> configurations) throws AmbariException {
+    if (identities != null) {
+      for (KerberosIdentityDescriptor identity : identities) {
+        KerberosPrincipalDescriptor principalDescriptor = identity.getPrincipalDescriptor();
+        if (principalDescriptor != null) {
+          authToLocalBuilder.append(
+              KerberosDescriptor.replaceVariables(principalDescriptor.getValue(), configurations),
+              KerberosDescriptor.replaceVariables(principalDescriptor.getLocalUsername(), configurations));
+        }
+      }
+    }
+  }
+
+  /**
    * Calculates the map of configurations relative to the cluster and host.
    * <p/>
    * This was borrowed from {@link org.apache.ambari.server.actionmanager.ExecutionCommandWrapper#getExecutionCommand()}
@@ -803,6 +848,26 @@ public class KerberosHelper {
   }
 
   /**
+   * Given a Collection of ServiceComponentHosts generates a unique list of hosts.
+   *
+   * @param serviceComponentHosts a Collection of ServiceComponentHosts from which to to retrieve host names
+   * @return a List of (unique) host names
+   */
+  private List<String> createUniqueHostList(Collection<ServiceComponentHost> serviceComponentHosts) {
+    Set<String> hostNames = new HashSet<String>();
+
+    if (serviceComponentHosts != null) {
+
+      for (ServiceComponentHost sch : serviceComponentHosts) {
+        hostNames.add(sch.getHostName());
+      }
+    }
+
+    return new ArrayList<String>(hostNames);
+  }
+
+
+  /**
    * Handler is an interface that needs to be implemented by toggle handler classes to do the
    * "right" thing for the task at hand.
    */
@@ -862,7 +927,7 @@ public class KerberosHelper {
      * @param dataDirectory          a File pointing to the (temporary) data directory
      * @param requestStageContainer  a RequestStageContainer to store the new stages in, if null a
      *                               new RequestStageContainer will be created
-     * @param updateHosts  host names that would be passed in request resource filters while creating stage for pushing keytabs
+     * @param serviceComponentHosts  a List of ServiceComponentHosts that needs to be updated as part of this operation
      * @return the last stage id generated, or -1 if no stages were created
      * @throws AmbariException if an error occurs while creating the relevant stages
      */
@@ -873,7 +938,7 @@ public class KerberosHelper {
                      RoleCommandOrder roleCommandOrder,
                      String realm, String kdcType, File dataDirectory,
                      RequestStageContainer requestStageContainer,
-                     List<String> updateHosts)
+                     List<ServiceComponentHost> serviceComponentHosts)
         throws AmbariException;
 
   }
@@ -929,7 +994,8 @@ public class KerberosHelper {
                             ServiceComponentHostServerActionEvent event,
                             RoleCommandOrder roleCommandOrder, String realm, String kdcType,
                             File dataDirectory, RequestStageContainer requestStageContainer,
-                            List<String> updateHosts) throws AmbariException {
+                            List<ServiceComponentHost> serviceComponentHosts)
+        throws AmbariException {
       // If there are principals, keytabs, and configurations to process, setup the following sages:
       //  1) generate principals
       //  2) generate keytab files
@@ -1038,10 +1104,11 @@ public class KerberosHelper {
           StageUtils.getGson().toJson(commandParameters),
           hostParamsJson);
 
-      if (!updateHosts.isEmpty()) {
+      if (!serviceComponentHosts.isEmpty()) {
+        List<String> hostsToUpdate = createUniqueHostList(serviceComponentHosts);
         Map<String, String> requestParams = new HashMap<String, String>();
         List<RequestResourceFilter> requestResourceFilters = new ArrayList<RequestResourceFilter>();
-        RequestResourceFilter reqResFilter = new RequestResourceFilter("KERBEROS", "KERBEROS_CLIENT", updateHosts);
+        RequestResourceFilter reqResFilter = new RequestResourceFilter("KERBEROS", "KERBEROS_CLIENT", hostsToUpdate);
         requestResourceFilters.add(reqResFilter);
 
         ActionExecutionContext actionExecContext = new ActionExecutionContext(
@@ -1079,7 +1146,6 @@ public class KerberosHelper {
 
   }
 
-
   /**
    * DisableKerberosHandler is an implementation of the Handler interface used to disable Kerberos
    * on the relevant cluster
@@ -1130,7 +1196,7 @@ public class KerberosHelper {
                             ServiceComponentHostServerActionEvent event,
                             RoleCommandOrder roleCommandOrder, String realm, String kdcType,
                             File dataDirectory, RequestStageContainer requestStageContainer,
-                            List<String> updateHosts) {
+                            List<ServiceComponentHost> serviceComponentHosts) {
       // TODO (rlevas): If there are principals, keytabs, and configurations to process, setup the following sages:
       //  1) remove principals
       //  2) remove keytab files

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptor.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptor.java b/ambari-server/src/main/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptor.java
index 70bd396..6725c92 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptor.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptor.java
@@ -47,6 +47,10 @@ import java.util.Map;
  *          "description": "The configuration type and property name indicating the property to be
  *                          updated with the generated principal - format: config-type/property.name",
  *          "type": "string"
+ *        },
+ *        "local_username": {
+ *          "description": "The local username this principal maps to - optional if no mapping is needed",
+ *          "type": "string"
  *        }
  *      }
  *   }
@@ -73,6 +77,14 @@ public class KerberosPrincipalDescriptor extends AbstractKerberosDescriptor {
   private String configuration;
 
   /**
+   * a String indicating the local username related to this principal, or null of no local mapping is
+   * needed or available.
+   * <p/>
+   * This value may be using in generating auth_to_local configuration settings.
+   */
+  private String localUsername;
+
+  /**
    * Creates a new KerberosPrincipalDescriptor
    * <p/>
    * See {@link org.apache.ambari.server.state.kerberos.KerberosPrincipalDescriptor} for the JSON
@@ -87,6 +99,8 @@ public class KerberosPrincipalDescriptor extends AbstractKerberosDescriptor {
     setName(getStringValue(data, "value"));
 
     setConfiguration(getStringValue(data, "configuration"));
+
+    setLocalUsername(getStringValue(data, "local_username"));
   }
 
   /**
@@ -143,6 +157,26 @@ public class KerberosPrincipalDescriptor extends AbstractKerberosDescriptor {
   }
 
   /**
+   * Gets the local username associated with this principal
+   *
+   * @return a String indicating the local username related to this principal, or null of not local
+   * mapping is needed/available
+   */
+  public String getLocalUsername() {
+    return localUsername;
+  }
+
+  /**
+   * Sets the local username associated with this principal
+   *
+   * @param localUsername a String indicating the local username related to this principal, or null
+   *                      of no local mapping is needed/available
+   */
+  public void setLocalUsername(String localUsername) {
+    this.localUsername = localUsername;
+  }
+
+  /**
    * Updates this KerberosPrincipalDescriptor with data from another KerberosPrincipalDescriptor
    * <p/>
    * Properties will be updated if the relevant updated values are not null.
@@ -162,6 +196,11 @@ public class KerberosPrincipalDescriptor extends AbstractKerberosDescriptor {
       if (updatedValue != null) {
         setConfiguration(updatedValue);
       }
+
+      updatedValue = updates.getLocalUsername();
+      if (updatedValue != null) {
+        setLocalUsername(updatedValue);
+      }
     }
   }
 
@@ -179,6 +218,7 @@ public class KerberosPrincipalDescriptor extends AbstractKerberosDescriptor {
 
     map.put("value", getValue());
     map.put("configuration", getConfiguration());
+    map.put("local_username", getLocalUsername());
 
     return map;
   }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/resources/stacks/HDP/2.2/services/FALCON/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/FALCON/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.2/services/FALCON/kerberos.json
index 827ce46..7882838 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/FALCON/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/FALCON/kerberos.json
@@ -30,7 +30,8 @@
               "name": "falcon_server",
               "principal": {
                 "value": "falcon/${host}@${realm}",
-                "configuration": "falcon-startup.properties/*.falcon.service.authentication.kerberos.principal"
+                "configuration": "falcon-startup.properties/*.falcon.service.authentication.kerberos.principal",
+                "local_username" : "${falcon-env/falcon_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/falcon.service.keytab",

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/resources/stacks/HDP/2.2/services/HBASE/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HBASE/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HBASE/kerberos.json
index 4b6213e..17d0c2f 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HBASE/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HBASE/kerberos.json
@@ -13,7 +13,8 @@
           "name": "hbase",
           "principal": {
             "value": "${hbase-env/hbase_user}@${realm}",
-            "configuration": "hbase-env/hbase_principal_name"
+            "configuration": "hbase-env/hbase_principal_name",
+            "local_username": "${hbase-env/hbase_user}"
           },
           "keytab": {
             "file": "${keytab_dir}/hbase.headless.keytab",
@@ -48,7 +49,8 @@
               "name": "hbase_master_hbase",
               "principal": {
                 "value": "hbase/_HOST@${realm}",
-                "configuration": "hbase-site/hbase.master.kerberos.principal"
+                "configuration": "hbase-site/hbase.master.kerberos.principal",
+                "local_username": "${hbase-env/hbase_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/hbase.service.keytab",
@@ -72,7 +74,8 @@
               "name": "hbase_regionserver_hbase",
               "principal": {
                 "value": "hbase/_HOST@${realm}",
-                "configuration": "hbase-site/hbase.regionserver.kerberos.principal"
+                "configuration": "hbase-site/hbase.regionserver.kerberos.principal",
+                "local_username": "${hbase-env/hbase_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/hbase.service.keytab",

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/resources/stacks/HDP/2.2/services/HDFS/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HDFS/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HDFS/kerberos.json
index 52c7d37..9f9a4ea 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/HDFS/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/HDFS/kerberos.json
@@ -25,7 +25,7 @@
             "hadoop.security.authentication": "kerberos",
             "hadoop.rpc.protection": "authentication",
             "hadoop.security.authorization": "true",
-            "hadoop.security.auth_to_local": "RULE:[2:$1@$0]([nd]n@.*${realm})s/.*/hdfs/\nRULE:[2:$1@$0]([rn]m@.*${realm})s/.*/yarn/\nRULE:[2:$1@$0](hs@.*${realm})s/.*/mapred/\nRULE:[2:$1@$0](hm@.*${realm})s/.*/hbase/\nRULE:[2:$1@$0](rs@.*${realm})s/.*/hbase/\nDEFAULT"
+            "hadoop.security.auth_to_local": "_AUTH_TO_LOCAL_RULES"
           }
         }
       ],
@@ -37,7 +37,8 @@
               "name": "namenode_nn",
               "principal": {
                 "value": "nn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.namenode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.namenode.kerberos.principal",
+                "local_username" : "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/nn.service.keytab",
@@ -91,7 +92,8 @@
               "name": "datanode_dn",
               "principal": {
                 "value": "dn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.datanode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.datanode.kerberos.principal",
+                "local_username" : "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/dn.service.keytab",
@@ -140,7 +142,8 @@
               "name": "secondary_namenode_nn",
               "principal": {
                 "value": "nn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.secondary.namenode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.secondary.namenode.kerberos.principal",
+                "local_username" : "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/nn.service.keytab",
@@ -187,7 +190,8 @@
               "name": "journalnode_jn",
               "principal": {
                 "value": "jn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.journalnode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.journalnode.kerberos.principal",
+                "local_username" : "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/jn.service.keytab",

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/resources/stacks/HDP/2.2/services/OOZIE/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/OOZIE/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.2/services/OOZIE/kerberos.json
index 9cb24ca..5c4b458 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/OOZIE/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/OOZIE/kerberos.json
@@ -20,7 +20,7 @@
             "oozie.service.AuthorizationService.authorization.enabled": "true",
             "oozie.service.HadoopAccessorService.kerberos.enabled": "true",
             "local.realm": "${realm}",
-            "oozie.authentication.kerberos.name.rules": "RULE:[2:$1@$0]([jt]t@.*${realm})s/.*/mapred/\nRULE:[2:$1@$0]([nd]n@.*${realm})s/.*/hdfs/\nRULE:[2:$1@$0](hm@.*${realm})s/.*/hbase/\nRULE:[2:$1@$0](rs@.*${realm})s/.*/hbase/\nDEFAULT"
+            "oozie.authentication.kerberos.name.rules": "_AUTH_TO_LOCAL_RULES"
           }
         }
       ],
@@ -32,7 +32,8 @@
               "name": "oozie_server",
               "principal": {
                 "value": "oozie/_HOST@${realm}",
-                "configuration": "oozie-site/oozie.service.HadoopAccessorService.kerberos.principal"
+                "configuration": "oozie-site/oozie.service.HadoopAccessorService.kerberos.principal",
+                "local_username" : "${oozie-env/oozie_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/oozie.service.keytab",

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/main/resources/stacks/HDP/2.2/services/YARN/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/YARN/kerberos.json b/ambari-server/src/main/resources/stacks/HDP/2.2/services/YARN/kerberos.json
index 7677a7a..d6aaa6e 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/YARN/kerberos.json
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/YARN/kerberos.json
@@ -30,7 +30,8 @@
               "name": "nodemanager_nm",
               "principal": {
                 "value": "nm/_HOST@${realm}",
-                "configuration": "yarn-site/yarn.nodemanager.principal"
+                "configuration": "yarn-site/yarn.nodemanager.principal",
+                "local_username": "${yarn-env/yarn_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/nm.service.keytab",
@@ -85,7 +86,8 @@
               "name": "resource_manager_rm",
               "principal": {
                 "value": "rm/_HOST@${realm}",
-                "configuration": "yarn-site/yarn.resourcemanager.principal"
+                "configuration": "yarn-site/yarn.resourcemanager.principal",
+                "local_username": "${yarn-env/yarn_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/rm.service.keytab",
@@ -135,7 +137,8 @@
               "name": "app_timeline_server_yarn",
               "principal": {
                 "value": "yarn/_HOST@${realm}",
-                "configuration": "yarn-site/yarn.timeline-service.principal"
+                "configuration": "yarn-site/yarn.timeline-service.principal",
+                "local_username": "${yarn-env/yarn_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/yarn.service.keytab",
@@ -201,7 +204,8 @@
               "name": "history_server_jhs",
               "principal": {
                 "value": "jhs/_HOST@${realm}",
-                "configuration": "mapred-site/mapreduce.jobhistory.principal"
+                "configuration": "mapred-site/mapreduce.jobhistory.principal",
+                "local_username": "${mapred-env/mapred_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/jhs.service.keytab",

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/test/java/org/apache/ambari/server/controller/AuthToLocalBuilderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/AuthToLocalBuilderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/AuthToLocalBuilderTest.java
new file mode 100644
index 0000000..fbb8ba1
--- /dev/null
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/AuthToLocalBuilderTest.java
@@ -0,0 +1,88 @@
+/*
+ * 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.ambari.server.controller;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class AuthToLocalBuilderTest {
+
+  @Test
+  public void testExpectedRules() {
+    AuthToLocalBuilder builder = new AuthToLocalBuilder();
+
+    builder.append("nn/_HOST@EXAMPLE.COM", "hdfs");
+    // Duplicate principal for secondary namenode, should be filtered out...
+    builder.append("nn/_HOST@EXAMPLE.COM", "hdfs");
+    builder.append("dn/_HOST@EXAMPLE.COM", "hdfs");
+    builder.append("jn/_HOST@EXAMPLE.COM", "hdfs");
+    builder.append("rm/_HOST@EXAMPLE.COM", "yarn");
+    builder.append("jhs/_HOST@EXAMPLE.COM", "mapred");
+    builder.append("hm/_HOST@EXAMPLE.COM", "hbase");
+    builder.append("rs/_HOST@EXAMPLE.COM", "hbase");
+
+    builder.append("foobar@EXAMPLE.COM", "hdfs");
+
+    assertEquals("RULE:[2:$1@$0](dn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](hm@EXAMPLE.COM)s/.*/hbase/\n" +
+            "RULE:[2:$1@$0](jhs@EXAMPLE.COM)s/.*/mapred/\n" +
+            "RULE:[2:$1@$0](jn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](nn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](rm@EXAMPLE.COM)s/.*/yarn/\n" +
+            "RULE:[2:$1@$0](rs@EXAMPLE.COM)s/.*/hbase/\n" +
+            "RULE:[1:$1@$0](foobar@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[1:$1@$0](.*@EXAMPLE.COM)s/@.*//\n" +
+            "DEFAULT",
+        builder.generate("EXAMPLE.COM"));
+  }
+
+  public void testUnexpectedRules() {
+    AuthToLocalBuilder builder = new AuthToLocalBuilder();
+
+    builder.append("nn/c6501.ambari.apache.org", "hdfs");
+    // Duplicate principal for secondary namenode, should be filtered out...
+    builder.append("nn/c6502.ambari.apache.org@EXAMPLE.COM", "hdfs");
+    builder.append("dn/c6501.ambari.apache.org@EXAMPLE.COM", "hdfs");
+    builder.append("jn/c6501.ambari.apache.org@EXAMPLE.COM", "hdfs");
+    builder.append("rm/c6501.ambari.apache.org@EXAMPLE.COM", "yarn");
+    builder.append("jhs/c6501.ambari.apache.org@EXAMPLE.COM", "mapred");
+    builder.append("hm/c6501.ambari.apache.org@EXAMPLE.COM", "hbase");
+    builder.append("rs/c6501.ambari.apache.org@EXAMPLE.COM", "hbase");
+
+    builder.append("hdfs@EXAMPLE.COM", "hdfs");
+    builder.append("hdfs/admin@EXAMPLE.COM", "hdfs");
+
+    // This is an unexpected invalid principal format, it should be ignored
+    builder.append("hdfs:admin@EXAMPLE.COM", "hdfs");
+
+    assertEquals("RULE:[2:$1@$0](dn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](hm@EXAMPLE.COM)s/.*/hbase/\n" +
+            "RULE:[2:$1@$0](jhs@EXAMPLE.COM)s/.*/mapred/\n" +
+            "RULE:[2:$1@$0](jn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](nn@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[2:$1@$0](rm@EXAMPLE.COM)s/.*/yarn/\n" +
+            "RULE:[2:$1@$0](rs@EXAMPLE.COM)s/.*/hbase/\n" +
+            "RULE:[1:$1@$0](hdfs@EXAMPLE.COM)s/.*/hdfs/\n" +
+            "RULE:[1:$1@$0](.*@EXAMPLE.COM)s/@.*//\n" +
+            "DEFAULT",
+        builder.generate("EXAMPLE.COM"));
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/test/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptorTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptorTest.java b/ambari-server/src/test/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptorTest.java
index a35bad3..9d56dc2 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptorTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/state/kerberos/KerberosPrincipalDescriptorTest.java
@@ -29,7 +29,8 @@ public class KerberosPrincipalDescriptorTest {
   public static final String JSON_VALUE =
       "{" +
           "\"value\": \"service/_HOST@_REALM\"," +
-          "\"configuration\": \"service-site/service.component.kerberos.principal\"" +
+          "\"configuration\": \"service-site/service.component.kerberos.principal\"," +
+          "\"local_username\": \"localUser\"" +
           "}";
 
   public static final Map<String, Object> MAP_VALUE =
@@ -37,6 +38,7 @@ public class KerberosPrincipalDescriptorTest {
         {
           put("value", "HTTP/_HOST@_REALM");
           put("configuration", "service-site/service.component.kerberos.https.principal");
+          put("local_username", null);
         }
       };
 
@@ -45,6 +47,7 @@ public class KerberosPrincipalDescriptorTest {
     Assert.assertFalse(principalDescriptor.isContainer());
     Assert.assertEquals("service/_HOST@_REALM", principalDescriptor.getValue());
     Assert.assertEquals("service-site/service.component.kerberos.principal", principalDescriptor.getConfiguration());
+    Assert.assertEquals("localUser", principalDescriptor.getLocalUsername());
   }
 
   public static void validateFromMap(KerberosPrincipalDescriptor principalDescriptor) {
@@ -52,12 +55,14 @@ public class KerberosPrincipalDescriptorTest {
     Assert.assertFalse(principalDescriptor.isContainer());
     Assert.assertEquals("HTTP/_HOST@_REALM", principalDescriptor.getValue());
     Assert.assertEquals("service-site/service.component.kerberos.https.principal", principalDescriptor.getConfiguration());
+    Assert.assertNull(principalDescriptor.getLocalUsername());
   }
 
   public static void validateUpdatedData(KerberosPrincipalDescriptor principalDescriptor) {
     Assert.assertNotNull(principalDescriptor);
     Assert.assertEquals("HTTP/_HOST@_REALM", principalDescriptor.getValue());
     Assert.assertEquals("service-site/service.component.kerberos.https.principal", principalDescriptor.getConfiguration());
+    Assert.assertEquals("localUser", principalDescriptor.getLocalUsername());
   }
 
   private static KerberosPrincipalDescriptor createFromJSON() {

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/test/resources/stacks/HDP/2.0.8/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/resources/stacks/HDP/2.0.8/kerberos.json b/ambari-server/src/test/resources/stacks/HDP/2.0.8/kerberos.json
index 64c7a8c..4308d81 100644
--- a/ambari-server/src/test/resources/stacks/HDP/2.0.8/kerberos.json
+++ b/ambari-server/src/test/resources/stacks/HDP/2.0.8/kerberos.json
@@ -28,7 +28,7 @@
         "hadoop.security.authentication": "kerberos",
         "hadoop.rpc.protection": "authentication",
         "hadoop.security.authorization": "true",
-        "hadoop.security.auth_to_local": "RULE:[2:$1@$0]([jt]t@.*${realm})s/.*/mapred/\nRULE:[2:$1@$0]([nd]n@.*${realm})s/.*/hdfs/\nRULE:[2:$1@$0](hm@.*${realm})s/.*/hbase/\nRULE:[2:$1@$0](rs@.*${realm})s/.*/hbase/\nDEFAULT",
+        "hadoop.security.auth_to_local": "_AUTH_TO_LOCAL_RULES",
         "hadoop.proxyuser.superuser.hosts": "",
         "hadoop.proxyuser.superuser.groups": ""
       }

http://git-wip-us.apache.org/repos/asf/ambari/blob/1d1ebb48/ambari-server/src/test/resources/stacks/HDP/2.0.8/services/HDFS/kerberos.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/resources/stacks/HDP/2.0.8/services/HDFS/kerberos.json b/ambari-server/src/test/resources/stacks/HDP/2.0.8/services/HDFS/kerberos.json
index 7271b36..87c080e 100644
--- a/ambari-server/src/test/resources/stacks/HDP/2.0.8/services/HDFS/kerberos.json
+++ b/ambari-server/src/test/resources/stacks/HDP/2.0.8/services/HDFS/kerberos.json
@@ -10,7 +10,8 @@
               "name": "namenode_nn",
               "principal": {
                 "value": "nn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.namenode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.namenode.kerberos.principal",
+                "local_username": "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/nn.service.keytab",
@@ -64,7 +65,8 @@
               "name": "datanode_dn",
               "principal": {
                 "value": "dn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.datanode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.datanode.kerberos.principal",
+                "local_username": "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/dn.service.keytab",
@@ -105,7 +107,8 @@
               "name": "secondary_namenode_nn",
               "principal": {
                 "value": "nn/_HOST@${realm}",
-                "configuration": "hdfs-site/dfs.secondary.namenode.kerberos.principal"
+                "configuration": "hdfs-site/dfs.secondary.namenode.kerberos.principal",
+                "local_username": "${hadoop-env/hdfs_user}"
               },
               "keytab": {
                 "file": "${keytab_dir}/snn.service.keytab",