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 2016/12/28 14:59:00 UTC

[1/2] ambari git commit: AMBARI-19296. When adding service, custom core-site properties get reverted back (echekanskiy via dlisnichenko)

Repository: ambari
Updated Branches:
  refs/heads/branch-2.5 cf28b98aa -> ebaa0ea79
  refs/heads/trunk 1bfffc0c8 -> d84aa3bcb


AMBARI-19296. When adding service, custom core-site properties get reverted back (echekanskiy via dlisnichenko)


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

Branch: refs/heads/branch-2.5
Commit: ebaa0ea79c20702e2e03e1cdf331f8816515fecc
Parents: cf28b98
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Dec 28 16:57:35 2016 +0200
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Dec 28 16:57:35 2016 +0200

----------------------------------------------------------------------
 .../stacks/HDP/2.0.6/services/stack_advisor.py  | 86 ++++++++++++++------
 .../stacks/HDP/2.1/services/stack_advisor.py    | 20 +----
 .../stacks/HDP/2.3/services/stack_advisor.py    |  2 +-
 .../stacks/HDP/2.5/services/stack_advisor.py    | 20 +----
 .../stacks/2.0.6/common/test_stack_advisor.py   | 10 +--
 5 files changed, 67 insertions(+), 71 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/ebaa0ea7/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 a98be40..9c0b357 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
@@ -200,12 +200,23 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
          ambari_user = ambari_user.split('@')[0]
     return ambari_user
 
-  def get_hosts_for_proxyuser(self, user_name, services):
+  def get_data_for_proxyuser(self, user_name, services, groups=False):
+    """
+    Returns values of proxyuser properties for given user. Properties can be
+    hadoop.proxyuser.username.groups or hadoop.proxyuser.username.hosts
+    :param user_name:
+    :param services:
+    :param groups: if true, will return values for group property, not hosts
+    :return: tuple (wildcard_value, set[values]), where wildcard_value indicates if property value was *
+    """
     if "core-site" in services["configurations"]:
       coreSite = services["configurations"]["core-site"]['properties']
     else:
       coreSite = {}
-    property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
+    if groups:
+      property_name = "hadoop.proxyuser.{0}.groups".format(user_name)
+    else:
+      property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
     if property_name in coreSite:
       property_value = coreSite[property_name]
       if property_value == "*":
@@ -214,15 +225,40 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         return False, set(property_value.split(","))
     return False, set()
 
+  def put_proxyuser_value(self, user_name, value, is_groups=False, services=None, put_function=None):
+    is_wildcard_value, current_value = self.get_data_for_proxyuser(user_name, services, is_groups)
+    result_value = "*"
+    result_values_set = self.merge_proxyusers_values(current_value, value)
+    if len(result_values_set) > 0:
+      result_value = ",".join(sorted([val for val in result_values_set if val]))
+
+    if is_groups:
+      property_name = "hadoop.proxyuser.{0}.groups".format(user_name)
+    else:
+      property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
+
+    put_function(property_name, result_value)
+
+  def merge_proxyusers_values(self, first, second):
+    result = set()
+    def append(data):
+      if isinstance(data, str) or isinstance(data, unicode):
+        if data != "*":
+          result.update(data.split(","))
+      else:
+        result.update(data)
+    append(first)
+    append(second)
+    return result
+
   def recommendAmbariProxyUsersForHDFS(self, services, servicesList, putCoreSiteProperty, putCoreSitePropertyAttribute):
       if "HDFS" in servicesList:
           ambari_user = self.getAmbariUser(services)
           ambariHostName = socket.getfqdn()
-          is_wildcard_value, hosts = self.get_hosts_for_proxyuser(ambari_user, services)
-          if not is_wildcard_value:
-            hosts.add(ambariHostName)
-            putCoreSiteProperty("hadoop.proxyuser.{0}.hosts".format(ambari_user), ",".join(hosts))
-          putCoreSiteProperty("hadoop.proxyuser.{0}.groups".format(ambari_user), "*")
+
+          self.put_proxyuser_value(ambari_user,ambariHostName,services=services,put_function=putCoreSiteProperty)
+          self.put_proxyuser_value(ambari_user, "*", is_groups=True, services=services, put_function=putCoreSiteProperty)
+
           old_ambari_user = self.getOldAmbariUser(services)
           if old_ambari_user is not None:
             putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.hosts".format(old_ambari_user), 'delete', 'true')
@@ -246,7 +282,7 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
       oozie_user = None
       if "oozie-env" in services["configurations"] and "oozie_user" in services["configurations"]["oozie-env"]["properties"]:
         oozie_user = services["configurations"]["oozie-env"]["properties"]["oozie_user"]
-        _, oozieServerHostsNameSet = self.get_hosts_for_proxyuser(oozie_user, services)
+        oozieServerHostsNameSet = set()
         oozieServerHosts = self.getHostsWithComponent("OOZIE", "OOZIE_SERVER", services, hosts)
         if oozieServerHosts is not None:
           for oozieServerHost in oozieServerHosts:
@@ -267,27 +303,27 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         webHcatServerHosts = self.getHostsWithComponent("HIVE", "WEBHCAT_SERVER", services, hosts)
 
         if hiveServerHosts is not None:
-          _, hiveServerHostsNameList = self.get_hosts_for_proxyuser(hive_user, services)
+          hiveServerHostsNameSet = set()
           for hiveServerHost in hiveServerHosts:
-            hiveServerHostsNameList.add(hiveServerHost["Hosts"]["host_name"])
+            hiveServerHostsNameSet.add(hiveServerHost["Hosts"]["host_name"])
           # Append Hive Server Interactive host as well, as it is Hive2/HiveServer2 component.
           if hiveServerInteractiveHosts:
             for hiveServerInteractiveHost in hiveServerInteractiveHosts:
               hiveServerInteractiveHostName = hiveServerInteractiveHost["Hosts"]["host_name"]
-              if hiveServerInteractiveHostName not in hiveServerHostsNameList:
-                hiveServerHostsNameList.add(hiveServerInteractiveHostName)
-                Logger.info("Appended (if not exiting), Hive Server Interactive Host : '{0}', to Hive Server Host List : '{1}'".format(hiveServerInteractiveHostName, hiveServerHostsNameList))
+              if hiveServerInteractiveHostName not in hiveServerHostsNameSet:
+                hiveServerHostsNameSet.add(hiveServerInteractiveHostName)
+                Logger.info("Appended (if not exiting), Hive Server Interactive Host : '{0}', to Hive Server Host List : '{1}'".format(hiveServerInteractiveHostName, hiveServerHostsNameSet))
 
-          hiveServerHostsNames = ",".join(sorted(hiveServerHostsNameList))  # includes Hive Server interactive host also.
-          Logger.info("Hive Server and Hive Server Interactive (if enabled) Host List : {0}".format(hiveServerHostsNameList))
+          hiveServerHostsNames = ",".join(sorted(hiveServerHostsNameSet))  # includes Hive Server interactive host also.
+          Logger.info("Hive Server and Hive Server Interactive (if enabled) Host List : {0}".format(hiveServerHostsNameSet))
           if not hive_user in users and hive_user is not None:
             users[hive_user] = {"propertyHosts" : hiveServerHostsNames,"propertyGroups" : "*", "config" : "hive-env", "propertyName" : "hive_user"}
 
         if webHcatServerHosts is not None:
-          _, webHcatServerHostsNameList = self.get_hosts_for_proxyuser(hive_user, services)
+          webHcatServerHostsNameSet = set()
           for webHcatServerHost in webHcatServerHosts:
-            webHcatServerHostsNameList.add(webHcatServerHost["Hosts"]["host_name"])
-          webHcatServerHostsNames = ",".join(sorted(webHcatServerHostsNameList))
+            webHcatServerHostsNameSet.add(webHcatServerHost["Hosts"]["host_name"])
+          webHcatServerHostsNames = ",".join(sorted(webHcatServerHostsNameSet))
           if not webhcat_user in users and webhcat_user is not None:
             users[webhcat_user] = {"propertyHosts" : webHcatServerHostsNames,"propertyGroups" : "*", "config" : "hive-env", "propertyName" : "webhcat_user"}
 
@@ -298,10 +334,10 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         rmHosts = self.getHostsWithComponent("YARN", "RESOURCEMANAGER", services, hosts)
 
         if len(rmHosts) > 1:
-          _, rmHostsNameList = self.get_hosts_for_proxyuser(hive_user, services)
+          rmHostsNameSet = set()
           for rmHost in rmHosts:
-            rmHostsNameList.add(rmHost["Hosts"]["host_name"])
-          rmHostsNames = ",".join(sorted(rmHostsNameList))
+            rmHostsNameSet.add(rmHost["Hosts"]["host_name"])
+          rmHostsNames = ",".join(sorted(rmHostsNameSet))
           if not yarn_user in users and yarn_user is not None:
             users[yarn_user] = {"propertyHosts" : rmHostsNames, "config" : "yarn-env", "propertyName" : "yarn_user"}
 
@@ -324,16 +360,14 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
     putCoreSitePropertyAttribute = self.putPropertyAttribute(configurations, "core-site")
 
     for user_name, user_properties in users.iteritems():
-      is_wildcard_value, _ = self.get_hosts_for_proxyuser(user_name, services)
       if hive_user and hive_user == user_name:
         if "propertyHosts" in user_properties:
           services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.hosts".format(hive_user)})
       # Add properties "hadoop.proxyuser.*.hosts", "hadoop.proxyuser.*.groups" to core-site for all users
-      if not is_wildcard_value:
-        putCoreSiteProperty("hadoop.proxyuser.{0}.hosts".format(user_name) , user_properties["propertyHosts"])
-        Logger.info("Updated hadoop.proxyuser.{0}.hosts as : {1}".format(hive_user, user_properties["propertyHosts"]))
+      self.put_proxyuser_value(user_name, user_properties["propertyHosts"], services=services, put_function=putCoreSiteProperty)
+      Logger.info("Updated hadoop.proxyuser.{0}.hosts as : {1}".format(hive_user, user_properties["propertyHosts"]))
       if "propertyGroups" in user_properties:
-        putCoreSiteProperty("hadoop.proxyuser.{0}.groups".format(user_name) , user_properties["propertyGroups"])
+        self.put_proxyuser_value(user_name, user_properties["propertyGroups"], is_groups=True, services=services, put_function=putCoreSiteProperty)
 
       # Remove old properties if user was renamed
       userOldValue = getOldValue(self, services, user_properties["config"], user_properties["propertyName"])

http://git-wip-us.apache.org/repos/asf/ambari/blob/ebaa0ea7/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 17225d0..f34f5a1 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
@@ -139,27 +139,11 @@ class HDP21StackAdvisor(HDP206StackAdvisor):
             webHcatSitePropertyAttributes("webhcat.proxyuser.{0}.groups".format(old_ambari_user), 'delete', 'true')
 
     if self.is_secured_cluster(services):
-      appendCoreSiteProperty = self.updateProperty(configurations, "core-site", services)
-
-      def updateCallback(originalValue, newValue):
-        """
-        :type originalValue str
-        :type newValue list
-        """
-        if originalValue and not originalValue.isspace():
-          hosts = originalValue.split(',')
-
-          if newValue:
-            hosts.extend(newValue)
-
-          result = ','.join(set(hosts))
-          return result
-        else:
-          return ','.join(set(newValue))
+      putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
 
       meta = self.get_service_component_meta("HIVE", "WEBHCAT_SERVER", services)
       if "hostnames" in meta:
-        appendCoreSiteProperty('hadoop.proxyuser.HTTP.hosts', meta["hostnames"], updateCallback)
+        self.put_proxyuser_value("HTTP", meta["hostnames"], services=services, put_function=putCoreSiteProperty)
 
   def recommendTezConfigurations(self, configurations, clusterData, services, hosts):
     putTezProperty = self.putProperty(configurations, "tez-site")

http://git-wip-us.apache.org/repos/asf/ambari/blob/ebaa0ea7/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
index cc53ae5..e09ffde 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
@@ -485,7 +485,7 @@ class HDP23StackAdvisor(HDP22StackAdvisor):
     if kmsEnvProperties and self.checkSiteProperties(kmsEnvProperties, 'kms_user') and 'KERBEROS' in servicesList:
       kmsUser = kmsEnvProperties['kms_user']
       kmsUserOld = getOldValue(self, services, 'kms-env', 'kms_user')
-      putCoreSiteProperty('hadoop.proxyuser.{0}.groups'.format(kmsUser), '*')
+      self.put_proxyuser_value(kmsUser, '*', is_groups=True, services=services, put_function=putCoreSiteProperty)
       if kmsUserOld is not None and kmsUser != kmsUserOld:
         putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.groups".format(kmsUserOld), 'delete', 'true')
         services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.groups".format(kmsUserOld)})

http://git-wip-us.apache.org/repos/asf/ambari/blob/ebaa0ea7/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
index a130566..b52d753 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
@@ -618,7 +618,7 @@ class HDP25StackAdvisor(HDP24StackAdvisor):
   def recommendHBASEConfigurations(self, configurations, clusterData, services, hosts):
     super(HDP25StackAdvisor, self).recommendHBASEConfigurations(configurations, clusterData, services, hosts)
     putHbaseSiteProperty = self.putProperty(configurations, "hbase-site", services)
-    appendCoreSiteProperty = self.updateProperty(configurations, "core-site", services)
+    putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
 
     if "cluster-env" in services["configurations"] \
          and "security_enabled" in services["configurations"]["cluster-env"]["properties"] \
@@ -632,24 +632,8 @@ class HDP25StackAdvisor(HDP24StackAdvisor):
         Logger.debug("Attempting to update hadoop.proxyuser.HTTP.hosts with %s" % str(phoenix_query_server_hosts))
         # The PQS hosts we want to ensure are set
         new_value = ','.join(phoenix_query_server_hosts)
-        # Compute the unique set of hosts for the property
-        def updateCallback(originalValue, newValue):
-          Logger.debug("Original hadoop.proxyuser.HTTP.hosts value %s, appending %s" % (originalValue, newValue))
-          # Only update the original value if it's not whitespace only
-          if originalValue and not originalValue.isspace():
-            hosts = originalValue.split(',')
-            # Add in the new hosts if we have some
-            if newValue and not newValue.isspace():
-              hosts.extend(newValue.split(','))
-            # Return the combined (uniqued) list of hosts
-            result = ','.join(set(hosts))
-            Logger.debug("Setting final to %s" % result)
-            return result
-          else:
-            Logger.debug("Setting final value to %s" % newValue)
-            return newValue
         # Update the proxyuser setting, deferring to out callback to merge results together
-        appendCoreSiteProperty('hadoop.proxyuser.HTTP.hosts', new_value, updateCallback)
+        self.put_proxyuser_value("HTTP", new_value, services=services, put_function=putCoreSiteProperty)
       else:
         Logger.debug("No phoenix query server hosts to update")
     else:

http://git-wip-us.apache.org/repos/asf/ambari/blob/ebaa0ea7/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 4fa9e02..c88e929 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
@@ -1318,7 +1318,7 @@ class TestHDP206StackAdvisor(TestCase):
                       'hadoop.proxyuser.ambari_user.hosts': ambariHostName,
                       'hadoop.proxyuser.oozie.groups': '*',
                       'hadoop.proxyuser.hive.groups': '*',
-                      'hadoop.proxyuser.webhcat.hosts': 'c6401.ambari.apache.org,c6402.ambari.apache.org,c6403.ambari.apache.org',
+                      'hadoop.proxyuser.webhcat.hosts': 'c6401.ambari.apache.org,c6402.ambari.apache.org',
                       'hadoop.proxyuser.falcon.hosts': '*',
                       'hadoop.proxyuser.webhcat.groups': '*',
                       'hadoop.proxyuser.hdfs.groups': '*',
@@ -1352,12 +1352,6 @@ class TestHDP206StackAdvisor(TestCase):
     self.stackAdvisor.recommendHDFSConfigurations(configurations, clusterData, services2, hosts)
     self.assertEquals(configurations, expected)
 
-
-
-
-
-
-
     configurations["hadoop-env"]["properties"]['hdfs_user'] = "hdfs1"
 
     changedConfigurations = [{"type":"hadoop-env",
@@ -1448,7 +1442,7 @@ class TestHDP206StackAdvisor(TestCase):
                                  'hadoop.proxyuser.hive.groups': '*',
                                  'hadoop.proxyuser.hdfs1.groups': '*',
                                  'hadoop.proxyuser.hdfs1.hosts': '*',
-                                 'hadoop.proxyuser.webhcat.hosts': 'c6401.ambari.apache.org,c6402.ambari.apache.org,c6403.ambari.apache.org',
+                                 'hadoop.proxyuser.webhcat.hosts': 'c6401.ambari.apache.org,c6402.ambari.apache.org',
                                  'hadoop.proxyuser.falcon.hosts': '*',
                                  'hadoop.proxyuser.webhcat.groups': '*',
                                  'hadoop.proxyuser.hdfs.groups': '*',


[2/2] ambari git commit: AMBARI-19296. When adding service, custom core-site properties get reverted back (echekanskiy via dlisnichenko)

Posted by dm...@apache.org.
AMBARI-19296. When adding service, custom core-site properties get reverted back (echekanskiy via dlisnichenko)


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

Branch: refs/heads/trunk
Commit: d84aa3bcb184db60e8d1a68429e1c7abb3fa3646
Parents: 1bfffc0
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Dec 28 16:58:27 2016 +0200
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Dec 28 16:58:27 2016 +0200

----------------------------------------------------------------------
 .../stacks/HDP/2.0.6/services/stack_advisor.py  | 51 +++++++++++++++-----
 .../stacks/HDP/2.1/services/stack_advisor.py    | 20 +-------
 .../stacks/HDP/2.3/services/stack_advisor.py    |  2 +-
 .../stacks/HDP/2.5/services/stack_advisor.py    | 20 +-------
 4 files changed, 43 insertions(+), 50 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/d84aa3bc/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 a352cdb..77ed527 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
@@ -205,11 +205,8 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
     if "HDFS" in servicesList:
       ambari_user = self.getAmbariUser(services)
       ambariHostName = socket.getfqdn()
-      is_wildcard_value, hosts = self.get_hosts_for_proxyuser(ambari_user, services)
-      if not is_wildcard_value:
-        hosts.add(ambariHostName)
-        putCoreSiteProperty("hadoop.proxyuser.{0}.hosts".format(ambari_user), ",".join(hosts))
-      putCoreSiteProperty("hadoop.proxyuser.{0}.groups".format(ambari_user), "*")
+      self.put_proxyuser_value(ambari_user, ambariHostName, services=services, put_function=putCoreSiteProperty)
+      self.put_proxyuser_value(ambari_user, "*", is_groups=True, services=services, put_function=putCoreSiteProperty)
       old_ambari_user = self.getOldAmbariUser(services)
       if old_ambari_user is not None:
         putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.hosts".format(old_ambari_user), 'delete', 'true')
@@ -254,7 +251,7 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         for proxyPropertyName, hostSelector in hostSelectorMap.iteritems():
           componentHostNamesString = hostSelector if isinstance(hostSelector, basestring) else '*'
           if isinstance(hostSelector, (list, tuple)):
-            _, componentHostNames = self.get_hosts_for_proxyuser(user, services) # preserve old values
+            _, componentHostNames = self.get_data_for_proxyuser(user, services) # preserve old values
             for component in hostSelector:
               componentHosts = self.getHostsWithComponent(serviceName, component, services, hosts)
               if componentHosts is not None:
@@ -288,13 +285,17 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
 
     return users
 
-  def get_hosts_for_proxyuser(self, user_name, services):
+  def get_data_for_proxyuser(self, user_name, services, groups=False):
     if "core-site" in services["configurations"]:
       coreSite = services["configurations"]["core-site"]['properties']
     else:
       coreSite = {}
 
-    property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
+    if groups:
+        property_name = "hadoop.proxyuser.{0}.groups".format(user_name)
+    else:
+      property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
+
     if property_name in coreSite:
       property_value = coreSite[property_name]
       if property_value == "*":
@@ -303,6 +304,32 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         return False, set(property_value.split(","))
     return False, set()
 
+  def put_proxyuser_value(self, user_name, value, is_groups=False, services=None, put_function=None):
+    is_wildcard_value, current_value = self.get_data_for_proxyuser(user_name, services, is_groups)
+    result_value = "*"
+    result_values_set = self.merge_proxyusers_values(current_value, value)
+    if len(result_values_set) > 0:
+      result_value = ",".join(sorted([val for val in result_values_set if val]))
+
+    if is_groups:
+      property_name = "hadoop.proxyuser.{0}.groups".format(user_name)
+    else:
+      property_name = "hadoop.proxyuser.{0}.hosts".format(user_name)
+
+    put_function(property_name, result_value)
+
+  def merge_proxyusers_values(self, first, second):
+    result = set()
+    def append(data):
+      if isinstance(data, str) or isinstance(data, unicode):
+        if data != "*":
+          result.update(data.split(","))
+      else:
+        result.update(data)
+    append(first)
+    append(second)
+    return result
+
   def getServiceHadoopProxyUsersConfigurationDict(self):
     """
     Returns a map that is used by 'getHadoopProxyUsers' to determine service
@@ -365,14 +392,12 @@ class HDP206StackAdvisor(DefaultStackAdvisor):
         services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.hosts".format(hive_user)})
 
     for user_name, user_properties in users.iteritems():
-      is_wildcard_value, _ = self.get_hosts_for_proxyuser(user_name, services)
 
       # Add properties "hadoop.proxyuser.*.hosts", "hadoop.proxyuser.*.groups" to core-site for all users
-      if not is_wildcard_value:
-        putCoreSiteProperty("hadoop.proxyuser.{0}.hosts".format(user_name) , user_properties["propertyHosts"])
-        Logger.info("Updated hadoop.proxyuser.{0}.hosts as : {1}".format(user_name, user_properties["propertyHosts"]))
+      self.put_proxyuser_value(user_name, user_properties["propertyHosts"], services=services, put_function=putCoreSiteProperty)
+      Logger.info("Updated hadoop.proxyuser.{0}.hosts as : {1}".format(user_name, user_properties["propertyHosts"]))
       if "propertyGroups" in user_properties:
-        putCoreSiteProperty("hadoop.proxyuser.{0}.groups".format(user_name) , user_properties["propertyGroups"])
+        self.put_proxyuser_value(user_name, user_properties["propertyGroups"], is_groups=True, services=services, put_function=putCoreSiteProperty)
 
       # Remove old properties if user was renamed
       userOldValue = getOldValue(self, services, user_properties["config"], user_properties["propertyName"])

http://git-wip-us.apache.org/repos/asf/ambari/blob/d84aa3bc/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 17225d0..f34f5a1 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
@@ -139,27 +139,11 @@ class HDP21StackAdvisor(HDP206StackAdvisor):
             webHcatSitePropertyAttributes("webhcat.proxyuser.{0}.groups".format(old_ambari_user), 'delete', 'true')
 
     if self.is_secured_cluster(services):
-      appendCoreSiteProperty = self.updateProperty(configurations, "core-site", services)
-
-      def updateCallback(originalValue, newValue):
-        """
-        :type originalValue str
-        :type newValue list
-        """
-        if originalValue and not originalValue.isspace():
-          hosts = originalValue.split(',')
-
-          if newValue:
-            hosts.extend(newValue)
-
-          result = ','.join(set(hosts))
-          return result
-        else:
-          return ','.join(set(newValue))
+      putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
 
       meta = self.get_service_component_meta("HIVE", "WEBHCAT_SERVER", services)
       if "hostnames" in meta:
-        appendCoreSiteProperty('hadoop.proxyuser.HTTP.hosts', meta["hostnames"], updateCallback)
+        self.put_proxyuser_value("HTTP", meta["hostnames"], services=services, put_function=putCoreSiteProperty)
 
   def recommendTezConfigurations(self, configurations, clusterData, services, hosts):
     putTezProperty = self.putProperty(configurations, "tez-site")

http://git-wip-us.apache.org/repos/asf/ambari/blob/d84aa3bc/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
index d97efe2..eb81655 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.3/services/stack_advisor.py
@@ -485,7 +485,7 @@ class HDP23StackAdvisor(HDP22StackAdvisor):
     if kmsEnvProperties and self.checkSiteProperties(kmsEnvProperties, 'kms_user') and 'KERBEROS' in servicesList:
       kmsUser = kmsEnvProperties['kms_user']
       kmsUserOld = getOldValue(self, services, 'kms-env', 'kms_user')
-      putCoreSiteProperty('hadoop.proxyuser.{0}.groups'.format(kmsUser), '*')
+      self.put_proxyuser_value(kmsUser, '*', is_groups=True, services=services, put_function=putCoreSiteProperty)
       if kmsUserOld is not None and kmsUser != kmsUserOld:
         putCoreSitePropertyAttribute("hadoop.proxyuser.{0}.groups".format(kmsUserOld), 'delete', 'true')
         services["forced-configurations"].append({"type" : "core-site", "name" : "hadoop.proxyuser.{0}.groups".format(kmsUserOld)})

http://git-wip-us.apache.org/repos/asf/ambari/blob/d84aa3bc/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py b/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
index a130566..b52d753 100644
--- a/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
+++ b/ambari-server/src/main/resources/stacks/HDP/2.5/services/stack_advisor.py
@@ -618,7 +618,7 @@ class HDP25StackAdvisor(HDP24StackAdvisor):
   def recommendHBASEConfigurations(self, configurations, clusterData, services, hosts):
     super(HDP25StackAdvisor, self).recommendHBASEConfigurations(configurations, clusterData, services, hosts)
     putHbaseSiteProperty = self.putProperty(configurations, "hbase-site", services)
-    appendCoreSiteProperty = self.updateProperty(configurations, "core-site", services)
+    putCoreSiteProperty = self.putProperty(configurations, "core-site", services)
 
     if "cluster-env" in services["configurations"] \
          and "security_enabled" in services["configurations"]["cluster-env"]["properties"] \
@@ -632,24 +632,8 @@ class HDP25StackAdvisor(HDP24StackAdvisor):
         Logger.debug("Attempting to update hadoop.proxyuser.HTTP.hosts with %s" % str(phoenix_query_server_hosts))
         # The PQS hosts we want to ensure are set
         new_value = ','.join(phoenix_query_server_hosts)
-        # Compute the unique set of hosts for the property
-        def updateCallback(originalValue, newValue):
-          Logger.debug("Original hadoop.proxyuser.HTTP.hosts value %s, appending %s" % (originalValue, newValue))
-          # Only update the original value if it's not whitespace only
-          if originalValue and not originalValue.isspace():
-            hosts = originalValue.split(',')
-            # Add in the new hosts if we have some
-            if newValue and not newValue.isspace():
-              hosts.extend(newValue.split(','))
-            # Return the combined (uniqued) list of hosts
-            result = ','.join(set(hosts))
-            Logger.debug("Setting final to %s" % result)
-            return result
-          else:
-            Logger.debug("Setting final value to %s" % newValue)
-            return newValue
         # Update the proxyuser setting, deferring to out callback to merge results together
-        appendCoreSiteProperty('hadoop.proxyuser.HTTP.hosts', new_value, updateCallback)
+        self.put_proxyuser_value("HTTP", new_value, services=services, put_function=putCoreSiteProperty)
       else:
         Logger.debug("No phoenix query server hosts to update")
     else: