You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by dm...@apache.org on 2015/09/09 14:24:29 UTC

ambari git commit: AMBARI-13045. Handle dependencies for 'hadoop.proxyuser.**' properties in stack advisor (dlysnichenko)

Repository: ambari
Updated Branches:
  refs/heads/trunk c8c6c9ce1 -> 74dc9474b


AMBARI-13045. Handle dependencies for 'hadoop.proxyuser.**' properties in stack advisor (dlysnichenko)


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

Branch: refs/heads/trunk
Commit: 74dc9474b600458a52e7bf038d239283e067708d
Parents: c8c6c9c
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Sep 9 15:23:09 2015 +0300
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Sep 9 15:24:42 2015 +0300

----------------------------------------------------------------------
 .../stackadvisor/StackAdvisorRequest.java       |   9 +-
 .../internal/StackAdvisorResourceProvider.java  |  13 +-
 .../ambari/server/state/ChangedConfigInfo.java  | 101 +++++++++++
 .../stacks/HDP/2.0.6/services/stack_advisor.py  |  79 ++++++++-
 .../stacks/HDP/2.1/services/stack_advisor.py    |  18 ++
 .../stacks/HDP/2.2/services/stack_advisor.py    |   1 +
 .../src/main/resources/stacks/stack_advisor.py  |   5 +-
 .../stacks/2.0.6/common/test_stack_advisor.py   | 176 ++++++++++++++++++-
 .../stacks/2.1/common/test_stack_advisor.py     |  60 ++++++-
 .../stacks/2.2/common/test_stack_advisor.py     |  12 +-
 10 files changed, 451 insertions(+), 23 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/StackAdvisorRequest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/StackAdvisorRequest.java b/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/StackAdvisorRequest.java
index 4db2133..b804c7e 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/StackAdvisorRequest.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/api/services/stackadvisor/StackAdvisorRequest.java
@@ -27,6 +27,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.ambari.server.api.services.stackadvisor.recommendations.RecommendationResponse;
+import org.apache.ambari.server.state.ChangedConfigInfo;
 import org.apache.ambari.server.state.PropertyDependencyInfo;
 import org.apache.commons.lang.StringUtils;
 
@@ -44,7 +45,7 @@ public class StackAdvisorRequest {
   private Map<String, Set<String>> hostComponents = new HashMap<String, Set<String>>();
   private Map<String, Set<String>> hostGroupBindings = new HashMap<String, Set<String>>();
   private Map<String, Map<String, Map<String, String>>> configurations = new HashMap<String, Map<String, Map<String, String>>>();
-  private List<PropertyDependencyInfo> changedConfigurations = new LinkedList<PropertyDependencyInfo>();
+  private List<ChangedConfigInfo> changedConfigurations = new LinkedList<ChangedConfigInfo>();
   private Set<RecommendationResponse.ConfigGroup> configGroups;
 
   public String getStackName() {
@@ -91,11 +92,11 @@ public class StackAdvisorRequest {
     return configurations;
   }
 
-  public List<PropertyDependencyInfo> getChangedConfigurations() {
+  public List<ChangedConfigInfo> getChangedConfigurations() {
     return changedConfigurations;
   }
 
-  public void setChangedConfigurations(List<PropertyDependencyInfo> changedConfigurations) {
+  public void setChangedConfigurations(List<ChangedConfigInfo> changedConfigurations) {
     this.changedConfigurations = changedConfigurations;
   }
 
@@ -162,7 +163,7 @@ public class StackAdvisorRequest {
     }
 
     public StackAdvisorRequestBuilder withChangedConfigurations(
-      List<PropertyDependencyInfo> changedConfigurations) {
+      List<ChangedConfigInfo> changedConfigurations) {
       this.instance.changedConfigurations = changedConfigurations;
       return this;
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackAdvisorResourceProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackAdvisorResourceProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackAdvisorResourceProvider.java
index 9412a2a..fe3d006 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackAdvisorResourceProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/StackAdvisorResourceProvider.java
@@ -41,6 +41,7 @@ import org.apache.ambari.server.controller.spi.Resource.Type;
 import org.apache.ambari.server.controller.utilities.PropertyHelper;
 
 import com.google.inject.Inject;
+import org.apache.ambari.server.state.ChangedConfigInfo;
 import org.apache.ambari.server.state.PropertyDependencyInfo;
 
 /**
@@ -109,9 +110,9 @@ public abstract class StackAdvisorResourceProvider extends ReadOnlyResourceProvi
           hgHostsMap);
       Map<String, Map<String, Map<String, String>>> configurations = calculateConfigurations(request);
 
-      List<PropertyDependencyInfo> changedConfigurations =
+      List<ChangedConfigInfo> changedConfigurations =
         requestType == StackAdvisorRequestType.CONFIGURATION_DEPENDENCIES ?
-          calculateChangedConfigurations(request) : Collections.<PropertyDependencyInfo>emptyList();
+          calculateChangedConfigurations(request) : Collections.<ChangedConfigInfo>emptyList();
 
       Set<RecommendationResponse.ConfigGroup> configGroups = calculateConfigGroups(request);
       return StackAdvisorRequestBuilder.
@@ -193,13 +194,13 @@ public abstract class StackAdvisorResourceProvider extends ReadOnlyResourceProvi
     return map;
   }
 
-  protected List<PropertyDependencyInfo> calculateChangedConfigurations(Request request) {
-    List<PropertyDependencyInfo> configs =
-      new LinkedList<PropertyDependencyInfo>();
+  protected List<ChangedConfigInfo> calculateChangedConfigurations(Request request) {
+    List<ChangedConfigInfo> configs =
+      new LinkedList<ChangedConfigInfo>();
     HashSet<HashMap<String, String>> changedConfigs =
       (HashSet<HashMap<String, String>>) getRequestProperty(request, CHANGED_CONFIGURATIONS_PROPERTY);
     for (HashMap<String, String> props: changedConfigs) {
-      configs.add(new PropertyDependencyInfo(props.get("type"), props.get("name")));
+      configs.add(new ChangedConfigInfo(props.get("type"), props.get("name"), props.get("old_value")));
     }
 
     return configs;

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/java/org/apache/ambari/server/state/ChangedConfigInfo.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/ChangedConfigInfo.java b/ambari-server/src/main/java/org/apache/ambari/server/state/ChangedConfigInfo.java
new file mode 100644
index 0000000..b69061f
--- /dev/null
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/ChangedConfigInfo.java
@@ -0,0 +1,101 @@
+/**
+ * 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.state;
+
+import org.codehaus.jackson.annotate.JsonProperty;
+
+public class ChangedConfigInfo {
+
+  private String type;
+
+  private String name;
+
+  @JsonProperty("old_value")
+  private String oldValue;
+
+  public ChangedConfigInfo() {
+
+  }
+
+  public ChangedConfigInfo(String type, String name) {
+    this(type, name, null);
+  }
+
+  public ChangedConfigInfo(String type, String name, String oldValue) {
+    this.type = type;
+    this.name = name;
+    this.oldValue = oldValue;
+  }
+
+  public String getType() {
+    return type;
+  }
+
+  public void setType(String type) {
+    this.type = type;
+  }
+
+  public String getName() {
+    return name;
+  }
+
+  public String getOldValue() {
+    return oldValue;
+  }
+
+  public void setOldValue(String oldValue) {
+    this.oldValue = oldValue;
+  }
+
+  public void setName(String name) {
+    this.name = name;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+
+    ChangedConfigInfo that = (ChangedConfigInfo) o;
+
+    if (type != null ? !type.equals(that.type) : that.type != null) return false;
+    if (name != null ? !name.equals(that.name) : that.name != null) return false;
+    return !(oldValue != null ? !oldValue.equals(that.oldValue) : that.oldValue != null);
+
+  }
+
+  @Override
+  public int hashCode() {
+    int result = type != null ? type.hashCode() : 0;
+    result = 31 * result + (name != null ? name.hashCode() : 0);
+    result = 31 * result + (oldValue != null ? oldValue.hashCode() : 0);
+    return result;
+  }
+
+  @Override
+  public String toString() {
+    return "ChangedConfigInfo{" +
+        "type='" + type + '\'' +
+        ", name='" + name + '\'' +
+        ", oldValue='" + oldValue + '\'' +
+        '}';
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/stack_advisor.py
index 3d535de..cce0fc5 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.0.6/services/stack_advisor.py
@@ -144,7 +144,72 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
     putMapredProperty('mapreduce.map.java.opts', "-Xmx" + str(int(round(0.8 * clusterData['mapMemory']))) + "m")
     putMapredProperty('mapreduce.reduce.java.opts', "-Xmx" + str(int(round(0.8 * clusterData['reduceMemory']))) + "m")
     putMapredProperty('mapreduce.task.io.sort.mb', min(int(round(0.4 * clusterData['mapMemory'])), 1024))
- 
+
+  def recommendHadoopProxyUsers (self, configurations, services, hosts):
+    servicesList = [service["StackServices"]["service_name"] for service in services["services"]]
+    users = {}
+
+    if 'forced-configurations' not in services:
+      services["forced-configurations"] = []
+
+    if "HDFS" in servicesList:
+      if "hadoop-env" in services["configurations"] and "hdfs_user" in services["configurations"]["hadoop-env"]["properties"]:
+        hdfs_user = services["configurations"]["hadoop-env"]["properties"]["hdfs_user"]
+        if not hdfs_user in users:
+          users[hdfs_user] = {"propertyHosts" : "*","propertyGroups" : "*", "config" : "hadoop-env", "propertyName" : "hdfs_user"}
+
+    if "OOZIE" in servicesList:
+      if "oozie-env" in services["configurations"] and "oozie_user" in services["configurations"]["oozie-env"]["properties"]:
+        oozie_user = services["configurations"]["oozie-env"]["properties"]["oozie_user"]
+        oozieServerrHost = self.getHostWithComponent("OOZIE", "OOZIE_SERVER", services, hosts)
+        if oozieServerrHost is not None:
+          oozieServerHostName = oozieServerrHost["Hosts"]["public_host_name"]
+        if not oozie_user in users:
+          users[oozie_user] = {"propertyHosts" : oozieServerHostName,"propertyGroups" : "*", "config" : "oozie-env", "propertyName" : "oozie_user"}
+
+    if "HIVE" in servicesList:
+      print "HIVE INSIDE"
+      if "hive-env" in services["configurations"] and "hive_user" in services["configurations"]["hive-env"]["properties"] \
+              and "webhcat_user" in services["configurations"]["hive-env"]["properties"]:
+        print "HIVE INSIDE1"
+        hive_user = services["configurations"]["hive-env"]["properties"]["hive_user"]
+        webhcat_user = services["configurations"]["hive-env"]["properties"]["webhcat_user"]
+        if "hadoop-env" in services["configurations"] and "proxyuser_group" in services["configurations"]["hadoop-env"]["properties"]:
+          proxy_user_group = services["configurations"]["hadoop-env"]["properties"]["proxyuser_group"]
+        hiveServerrHost = self.getHostWithComponent("HIVE", "HIVE_SERVER", services, hosts)
+        if hiveServerrHost is not None:
+          hiveServerHostName = hiveServerrHost["Hosts"]["public_host_name"]
+        if not hive_user in users:
+          users[hive_user] = {"propertyHosts" : hiveServerHostName,"propertyGroups" : "*", "config" : "hive-env", "propertyName" : "hive_user"}
+        if not webhcat_user in users:
+          users[webhcat_user] = {"propertyHosts" : hiveServerHostName,"propertyGroups" : proxy_user_group, "config" : "hive-env", "propertyName" : "webhcat_user"}
+
+    if "FALCON" in servicesList:
+      if "falcon-env" in services["configurations"] and "falcon_user" in services["configurations"]["falcon-env"]["properties"]:
+        falconUser = services["configurations"]["falcon-env"]["properties"]["falcon_user"]
+        if "hadoop-env" in services["configurations"] and "proxyuser_group" in services["configurations"]["hadoop-env"]["properties"]:
+          proxy_user_group = services["configurations"]["hadoop-env"]["properties"]["proxyuser_group"]
+        if not falconUser in users:
+          users[falconUser] = {"propertyHosts" : "*","propertyGroups" : proxy_user_group, "config" : "falcon-env", "propertyName" : "falcon_user"}
+
+    putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
+    putCoreSitePropertyAttribute = self.putPropertyAttribute(configurations, "core-site")
+
+    for user_name, user_properties in users.iteritems():
+      # Add properties "hadoop.proxyuser.*.hosts", "hadoop.proxyuser.*.groups" to core-site for all users
+      putCoreSiteProperty("hadoop.proxyuser.{0}.hosts".format(user_name) , user_properties["propertyHosts"])
+      putCoreSiteProperty("hadoop.proxyuser.{0}.groups".format(user_name) , user_properties["propertyGroups"])
+
+      # Remove old properties if user was renamed
+      userOldValue = getOldValue(self, services, user_properties["config"], user_properties["propertyName"])
+      if userOldValue is not None:
+        putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.hosts".format(userOldValue), 'delete', 'true')
+        putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.groups".format(userOldValue), 'delete', 'true')
+        services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.hosts".format(userOldValue)})
+        services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.groups".format(userOldValue)})
+        services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.hosts".format(user_name)})
+        services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.groups".format(user_name)})
+
   def recommendHDFSConfigurations(self, configurations, clusterData, services, hosts):
     putHDFSProperty = self.putProperty(configurations, "hadoop-env", services)
     putHDFSProperty('namenode_heapsize', max(int(clusterData['totalAvailableRam'] / 2), 1024))
@@ -153,6 +218,9 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
     putHDFSProperty = self.putProperty(configurations, "hadoop-env", services)
     putHDFSProperty('namenode_opt_maxnewsize', max(int(clusterData['totalAvailableRam'] / 8), 256))
 
+    # recommendations for "hadoop.proxyuser.*.hosts", "hadoop.proxyuser.*.groups" properties in core-site
+    self.recommendHadoopProxyUsers(configurations, services, hosts)
+
   def recommendHbaseConfigurations(self, configurations, clusterData, services, hosts):
     # recommendations for HBase env config
     putHbaseProperty = self.putProperty(configurations, "hbase-env", services)
@@ -806,6 +874,15 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         parentValidators[service] = {}
       parentValidators[service].update(configsDict)
 
+def getOldValue(self, services, configType, propertyName):
+  if services:
+    if 'changed-configurations' in services.keys():
+      changedConfigs = services["changed-configurations"]
+      for changedConfig in changedConfigs:
+        if changedConfig["type"] == configType and changedConfig["name"]== propertyName and "old_value" in changedConfig:
+          return changedConfig["old_value"]
+  return None
+
 # Validation helper methods
 def getSiteProperties(configurations, siteName):
   siteConfig = configurations.get(siteName)

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
index 1607287..2bb5aad 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.1/services/stack_advisor.py
@@ -31,6 +31,24 @@ class HDP21StackAdvisor(HDP206StackAdvisor):
 
   def recommendOozieConfigurations(self, configurations, clusterData, services, hosts):
     if "FALCON_SERVER" in clusterData["components"]:
+      putOozieSiteProperty = self.putProperty(configurations, "oozie-site", services)
+
+      if "falcon-env" in services["configurations"] and "falcon_user" in services["configurations"]["falcon-env"]["properties"]:
+        falconUser = services["configurations"]["falcon-env"]["properties"]["falcon_user"]
+        putOozieSiteProperty("oozie.service.ProxyUserService.proxyuser.{0}.groups".format(falconUser) , "*")
+        putOozieSiteProperty("oozie.service.ProxyUserService.proxyuser.{0}.hosts".format(falconUser) , "*")
+        falconUserOldValue = getOldValue(self, services, "falcon-env", "falcon_user")
+        if falconUserOldValue is not None:
+          if 'forced-configurations' not in services:
+            services["forced-configurations"] = []
+          putOozieSitePropertyAttribute = self.putPropertyAttribute(configurations, "oozie-site")
+          putOozieSitePropertyAttribute("oozie.service.ProxyUserService.proxyuser.{0}.groups".format(falconUserOldValue), 'delete', 'true')
+          putOozieSitePropertyAttribute("oozie.service.ProxyUserService.proxyuser.{0}.hosts".format(falconUserOldValue), 'delete', 'true')
+          services["forced-configurations"].append({"type" : "oozie-site", "name" : "oozie.service.ProxyUserService.proxyuser.{0}.hosts".format(falconUserOldValue)})
+          services["forced-configurations"].append({"type" : "oozie-site", "name" : "oozie.service.ProxyUserService.proxyuser.{0}.groups".format(falconUserOldValue)})
+          services["forced-configurations"].append({"type" : "oozie-site", "name" : "oozie.service.ProxyUserService.proxyuser.{0}.hosts".format(falconUser)})
+          services["forced-configurations"].append({"type" : "oozie-site", "name" : "oozie.service.ProxyUserService.proxyuser.{0}.groups".format(falconUser)})
+
       putMapredProperty = self.putProperty(configurations, "oozie-site")
       putMapredProperty("oozie.services.ext",
                         "org.apache.oozie.service.JMSAccessorService," +

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
index 9a7d7a0..f09488f 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.2/services/stack_advisor.py
@@ -90,6 +90,7 @@ class HDP22StackAdvisor(HDP21StackAdvisor):
           putYarnPropertyAttribute('yarn.nodemanager.linux-container-executor.cgroups.mount-path', 'delete', 'true')
 
   def recommendHDFSConfigurations(self, configurations, clusterData, services, hosts):
+    super(HDP22StackAdvisor, self).recommendHDFSConfigurations(configurations, clusterData, services, hosts)
     putHdfsSiteProperty = self.putProperty(configurations, "hdfs-site", services)
     putHdfsSitePropertyAttribute = self.putPropertyAttribute(configurations, "hdfs-site")
     putHdfsSiteProperty("dfs.datanode.max.transfer.threads", 16384 if clusterData["hBaseInstalled"] else 4096)

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/main/resources/stacks/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/stack_advisor.py b/ambari-server/src/main/resources/stacks/stack_advisor.py
index 0ef953d..d993feb 100644
--- a/ambari-server/src/main/resources/stacks/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/stack_advisor.py
@@ -730,6 +730,7 @@ class DefaultStackAdvisor(StackAdvisor):
   def getAffectedConfigs(self, services):
     """returns properties dict including changed-configurations and depended-by configs"""
     changedConfigs = services['changed-configurations']
+    changedConfigs = [{"type": entry["type"], "name": entry["name"]} for entry in changedConfigs]
     allDependencies = []
 
     for item in services['services']:
@@ -754,10 +755,12 @@ class DefaultStackAdvisor(StackAdvisor):
             if dependency not in dependencies:
               dependencies.append(dependency)
 
+    if "forced-configurations" in services and services["forced-configurations"] is not None:
+      dependencies.extend(services["forced-configurations"])
     return  dependencies
 
   def versionCompare(self, version1, version2):
     def normalize(v):
       return [int(x) for x in re.sub(r'(\.0+)*$','', v).split(".")]
     return cmp(normalize(version1), normalize(version2))
-  pass
\ No newline at end of file
+  pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/test/python/stacks/2.0.6/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.0.6/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.0.6/common/test_stack_advisor.py
index 887e172..60ca33d 100644
--- a/ambari-server/src/test/python/stacks/2.0.6/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.0.6/common/test_stack_advisor.py
@@ -677,7 +677,102 @@ class TestHDP206StackAdvisor(TestCase):
     self.assertEquals(configurations, expected)
 
   def test_recommendHDFSConfigurations(self):
-    configurations = {}
+    configurations = {
+      "hadoop-env": {
+        "properties": {
+          "hdfs_user": "hdfs",
+          "proxyuser_group": "users"
+        }
+      },
+      "hive-env": {
+        "properties": {
+          "webhcat_user": "webhcat",
+          "hive_user": "hive"
+        }
+      },
+      "oozie-env": {
+        "properties": {
+          "oozie_user": "oozie"
+        }
+      },
+      "falcon-env": {
+        "properties": {
+          "falcon_user": "falcon"
+        }
+      }
+    }
+
+    hosts = {
+      "items": [
+        {
+          "href": "/api/v1/hosts/host1",
+          "Hosts": {
+            "cpu_count": 1,
+            "host_name": "c6401.ambari.apache.org",
+            "os_arch": "x86_64",
+            "os_type": "centos6",
+            "ph_cpu_count": 1,
+            "public_host_name": "c6401.ambari.apache.org",
+            "rack_info": "/default-rack",
+            "total_mem": 2097152
+          }
+        }]}
+
+    services = {
+      "services": [
+        {
+          "StackServices": {
+            "service_name": "HDFS"
+          }, "components": []
+        },
+        {
+          "StackServices": {
+            "service_name": "FALCON"
+          }, "components": []
+        },
+        {
+          "StackServices": {
+            "service_name": "HIVE"
+          }, "components": [{
+          "href": "/api/v1/stacks/HDP/versions/2.0.6/services/HIVE/components/HIVE_SERVER",
+          "StackServiceComponents": {
+            "advertise_version": "true",
+            "cardinality": "1",
+            "component_category": "MASTER",
+            "component_name": "HIVE_SERVER",
+            "custom_commands": [],
+            "display_name": "Hive Server",
+            "is_client": "false",
+            "is_master": "true",
+            "service_name": "HIVE",
+            "stack_name": "HDP",
+            "stack_version": "2.0.6",
+            "hostnames": ["c6401.ambari.apache.org"]
+          }, }]
+        },
+        {
+          "StackServices": {
+            "service_name": "OOZIE"
+          }, "components": [{
+          "href": "/api/v1/stacks/HDP/versions/2.0.6/services/HIVE/components/OOZIE_SERVER",
+          "StackServiceComponents": {
+            "advertise_version": "true",
+            "cardinality": "1",
+            "component_category": "MASTER",
+            "component_name": "OOZIE_SERVER",
+            "custom_commands": [],
+            "display_name": "Oozie Server",
+            "is_client": "false",
+            "is_master": "true",
+            "service_name": "HIVE",
+            "stack_name": "HDP",
+            "stack_version": "2.0.6",
+            "hostnames": ["c6401.ambari.apache.org"]
+          }, }]
+        }],
+      "configurations": configurations
+    }
+
     clusterData = {
       "totalAvailableRam": 2048
     }
@@ -685,13 +780,86 @@ class TestHDP206StackAdvisor(TestCase):
       'hadoop-env': {
         'properties': {
           'namenode_heapsize': '1024',
-          'namenode_opt_newsize' : '256',
-          'namenode_opt_maxnewsize' : '256'
+          'namenode_opt_newsize': '256',
+          'namenode_opt_maxnewsize': '256',
+          'hdfs_user': 'hdfs',
+          "proxyuser_group": "users"
+        }
+      },
+      "core-site": {
+        "properties": {
+          "hadoop.proxyuser.hdfs.hosts": "*",
+          "hadoop.proxyuser.hdfs.groups": "*",
+          "hadoop.proxyuser.hive.hosts": "c6401.ambari.apache.org",
+          "hadoop.proxyuser.hive.groups": "*",
+          "hadoop.proxyuser.webhcat.hosts": "c6401.ambari.apache.org",
+          "hadoop.proxyuser.webhcat.groups": "users",
+          "hadoop.proxyuser.oozie.groups": "*",
+          "hadoop.proxyuser.oozie.hosts": "c6401.ambari.apache.org",
+          "hadoop.proxyuser.falcon.groups": "users",
+          "hadoop.proxyuser.falcon.hosts": "*"
+        }
+      },
+      "hive-env": {
+        "properties": {
+          "hive_user": "hive",
+          "webhcat_user": "webhcat"
+        }
+      },
+      "oozie-env": {
+        "properties": {
+          "oozie_user": "oozie"
+        }
+      },
+      "falcon-env": {
+        "properties": {
+          "falcon_user": "falcon"
         }
       }
     }
 
-    self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, '', '')
+    self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, services, hosts)
+    self.assertEquals(configurations, expected)
+
+    configurations["hadoop-env"]["properties"]['hdfs_user'] = "hdfs1"
+
+    changedConfigurations = [{"type":"hadoop-env",
+                              "name":"hdfs_user",
+                              "old_value":"hdfs"}]
+
+    services["changed-configurations"] = changedConfigurations
+    services['configurations'] = configurations
+
+    core_site = {
+      "properties": {
+        "hadoop.proxyuser.hdfs1.hosts": "*",
+        "hadoop.proxyuser.hdfs1.groups": "*",
+        "hadoop.proxyuser.hdfs.hosts": "*",
+        "hadoop.proxyuser.hdfs.groups": "*",
+        "hadoop.proxyuser.hive.hosts": "c6401.ambari.apache.org",
+        "hadoop.proxyuser.hive.groups": "*",
+        "hadoop.proxyuser.webhcat.hosts": "c6401.ambari.apache.org",
+        "hadoop.proxyuser.webhcat.groups": "users",
+        "hadoop.proxyuser.oozie.groups": "*",
+        "hadoop.proxyuser.oozie.hosts": "c6401.ambari.apache.org",
+        "hadoop.proxyuser.falcon.groups": "users",
+        "hadoop.proxyuser.falcon.hosts": "*"
+      },
+      "property_attributes": {
+        "hadoop.proxyuser.hdfs.hosts": {
+          "delete": "true"
+        },
+        "hadoop.proxyuser.hdfs.groups": {
+          "delete": "true"
+        }
+      }
+
+    }
+
+    expected["core-site"] = core_site
+    expected["hadoop-env"]["properties"]["hdfs_user"] = "hdfs1"
+
+    self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, services, hosts)
     self.assertEquals(configurations, expected)
 
   def test_validateHDFSConfigurationsEnv(self):

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
index 0d5caf6..1fa4e44 100644
--- a/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.1/common/test_stack_advisor.py
@@ -51,7 +51,24 @@ class TestHDP21StackAdvisor(TestCase):
     self.assertEquals(configurations, expected)
 
   def test_recommendOozieConfigurations_withFalconServer(self):
-    configurations = {}
+    configurations = {
+      "falcon-env" : {
+        "properties" : {
+          "falcon_user" : "falcon"
+        }
+      }
+    }
+
+    services = {
+      "services": [
+        {
+          "StackServices": {
+            "service_name": "FALCON"
+          }, "components": []
+        },],
+      "configurations": configurations
+    }
+
     clusterData = {
       "components" : ["FALCON_SERVER"]
     }
@@ -60,12 +77,19 @@ class TestHDP21StackAdvisor(TestCase):
         "properties": {
           "oozie.services.ext": "org.apache.oozie.service.JMSAccessorService," +
                                 "org.apache.oozie.service.PartitionDependencyManagerService," +
-                                "org.apache.oozie.service.HCatAccessorService"
+                                "org.apache.oozie.service.HCatAccessorService",
+          "oozie.service.ProxyUserService.proxyuser.falcon.groups" : "*",
+          "oozie.service.ProxyUserService.proxyuser.falcon.hosts" : "*"
+        }
+      },
+      "falcon-env" : {
+        "properties" : {
+          "falcon_user" : "falcon"
         }
       }
     }
 
-    self.stackAdvisor.recommendOozieConfigurations(configurations, clusterData, None, None)
+    self.stackAdvisor.recommendOozieConfigurations(configurations, clusterData, services, None)
     self.assertEquals(configurations, expected)
 
   def test_recommendHiveConfigurations_mapMemoryLessThan2048(self):
@@ -220,7 +244,24 @@ class TestHDP21StackAdvisor(TestCase):
     self.assertEquals(configurations, expected)
 
   def test_recommendHDFSConfigurations(self):
-    configurations = {}
+    configurations = {
+      "hadoop-env": {
+        "properties": {
+          "hdfs_user": "hdfs"
+        }
+      }
+    }
+
+    services = {
+      "services": [
+        {
+          "StackServices": {
+            "service_name": "HDFS"
+          }, "components": []
+        }],
+      "configurations": configurations
+    }
+
     clusterData = {
       "totalAvailableRam": 2048
     }
@@ -229,12 +270,19 @@ class TestHDP21StackAdvisor(TestCase):
         'properties': {
           'namenode_heapsize': '1024',
           'namenode_opt_newsize' : '256',
-          'namenode_opt_maxnewsize' : '256'
+          'namenode_opt_maxnewsize' : '256',
+          'hdfs_user' : "hdfs"
+        }
+      },
+      "core-site": {
+        "properties": {
+          "hadoop.proxyuser.hdfs.hosts": "*",
+          "hadoop.proxyuser.hdfs.groups": "*",
         }
       }
     }
 
-    self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, '', '')
+    self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, services, '')
     self.assertEquals(configurations, expected)
 
   def test_validateHDFSConfigurationsEnv(self):

http://git-wip-us.apache.org/repos/asf/ambari/blob/74dc9474/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py b/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
index d0260f9..7443b51 100644
--- a/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
+++ b/ambari-server/src/test/python/stacks/2.2/common/test_stack_advisor.py
@@ -2223,6 +2223,9 @@ class TestHDP22StackAdvisor(TestCase):
       },
       'hdfs-site': {
         "properties": {"dfs.datanode.data.dir": "/path/1,/path/2,/path/3,/path/4"}
+      },
+      "hadoop-env": {
+        "properties": {"hdfs_user": "hdfs"}
       }
     }
     clusterData = {
@@ -2236,7 +2239,8 @@ class TestHDP22StackAdvisor(TestCase):
         'properties': {
           'namenode_heapsize': '1024',
           'namenode_opt_newsize' : '128',
-          'namenode_opt_maxnewsize' : '128'
+          'namenode_opt_maxnewsize' : '128',
+          "hdfs_user": "hdfs"
         },
         'property_attributes': {
           'dtnode_heapsize': {'maximum': '2048'},
@@ -2259,6 +2263,12 @@ class TestHDP22StackAdvisor(TestCase):
         'properties': {
           'ranger-hdfs-plugin-enabled': 'Yes'
         }
+      },
+      "core-site": {
+        "properties": {
+          "hadoop.proxyuser.hdfs.hosts": "*",
+          "hadoop.proxyuser.hdfs.groups": "*",
+        }
       }
     }
     services = {"services":