You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by rn...@apache.org on 2015/05/18 19:55:51 UTC

ambari git commit: AMBARI-10411. Yarn ResourceManager HA config properties not processed properly by Blueprints. (rnettleton)

Repository: ambari
Updated Branches:
  refs/heads/trunk 3c4e9111e -> ef40b86f5


AMBARI-10411. Yarn ResourceManager HA config properties not processed properly by Blueprints. (rnettleton)


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

Branch: refs/heads/trunk
Commit: ef40b86f5588a0c7f21e4c2830d000e25163267f
Parents: 3c4e911
Author: Bob Nettleton <rn...@hortonworks.com>
Authored: Mon May 18 13:54:59 2015 -0400
Committer: Bob Nettleton <rn...@hortonworks.com>
Committed: Mon May 18 13:55:40 2015 -0400

----------------------------------------------------------------------
 .../BlueprintConfigurationProcessor.java        | 131 ++++++++++++++++---
 .../ambari/server/topology/ClusterTopology.java |   7 +
 .../server/topology/ClusterTopologyImpl.java    |  16 +++
 .../BlueprintConfigurationProcessorTest.java    | 102 +++++++++++++++
 4 files changed, 239 insertions(+), 17 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/ef40b86f/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessor.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessor.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessor.java
index d7ff543..6984881 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessor.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessor.java
@@ -226,6 +226,11 @@ public class BlueprintConfigurationProcessor {
     if (clusterTopology.isNameNodeHAEnabled()) {
       doNameNodeHAUpdate();
     }
+
+    if (clusterTopology.isYarnResourceManagerHAEnabled()) {
+      doYarnResourceManagerHAUpdate();
+    }
+
     Collection<Map<String, Map<String, String>>> allConfigs = new ArrayList<Map<String, Map<String, String>>>();
     allConfigs.add(clusterTopology.getConfiguration().getFullProperties());
     for (HostGroupInfo groupInfo : clusterTopology.getHostGroupInfo().values()) {
@@ -278,14 +283,28 @@ public class BlueprintConfigurationProcessor {
 
   /**
    * Creates a Collection of PropertyUpdater maps that will handle the configuration
-   *   update for this cluster.  If NameNode HA is enabled, then updater
-   *   instances will be added to the collection, in addition to the default list
-   *   of Updaters that are statically defined.
+   *   update for this cluster.
+   *
+   *   If NameNode HA is enabled, then updater instances will be added to the
+   *   collection, in addition to the default list of Updaters that are statically defined.
+   *
+   *   Similarly, if Yarn ResourceManager HA is enabled, then updater instances specific
+   *   to Yarn HA will be added to the default list of Updaters that are statically defined.
    *
    * @return Collection of PropertyUpdater maps used to handle cluster config update
    */
   private Collection<Map<String, Map<String, PropertyUpdater>>> createCollectionOfUpdaters() {
-    return (clusterTopology.isNameNodeHAEnabled()) ? addHAUpdaters(allUpdaters) : allUpdaters;
+    Collection<Map<String, Map<String, PropertyUpdater>>> updaters = allUpdaters;
+
+    if (clusterTopology.isNameNodeHAEnabled()) {
+      updaters = addNameNodeHAUpdaters(updaters);
+    }
+
+    if (clusterTopology.isYarnResourceManagerHAEnabled()) {
+      updaters = addYarnResourceManagerHAUpdaters(updaters);
+    }
+
+    return updaters;
   }
 
   /**
@@ -299,7 +318,34 @@ public class BlueprintConfigurationProcessor {
    *                   this cluster config update
    * @return A Collection of PropertyUpdater maps to handle the cluster config update
    */
-  private Collection<Map<String, Map<String, PropertyUpdater>>> addHAUpdaters(Collection<Map<String, Map<String, PropertyUpdater>>> updaters) {
+  private Collection<Map<String, Map<String, PropertyUpdater>>> addNameNodeHAUpdaters(Collection<Map<String, Map<String, PropertyUpdater>>> updaters) {
+    Collection<Map<String, Map<String, PropertyUpdater>>> highAvailabilityUpdaters =
+      new LinkedList<Map<String, Map<String, PropertyUpdater>>>();
+
+    // always add the statically-defined list of updaters to the list to use
+    // in processing cluster configuration
+    highAvailabilityUpdaters.addAll(updaters);
+
+    // add the updaters for the dynamic HA properties, based on the HA config in hdfs-site
+    highAvailabilityUpdaters.add(createMapOfNameNodeHAUpdaters());
+
+    return highAvailabilityUpdaters;
+  }
+
+  /**
+   * Creates a Collection of PropertyUpdater maps that include the Yarn ResourceManager HA properties, and
+   *   adds these to the list of updaters used to process the cluster configuration.  The HA
+   *   properties are based on the names of the Resource Manager instances defined in
+   *   yarn-site, and so must be registered at runtime, rather than in the static list.
+   *
+   *   This new Collection includes the statically-defined updaters,
+   *   in addition to the HA-related updaters.
+   *
+   * @param updaters a Collection of updater maps to be included in the list of updaters for
+   *                   this cluster config update
+   * @return A Collection of PropertyUpdater maps to handle the cluster config update
+   */
+  private Collection<Map<String, Map<String, PropertyUpdater>>> addYarnResourceManagerHAUpdaters(Collection<Map<String, Map<String, PropertyUpdater>>> updaters) {
     Collection<Map<String, Map<String, PropertyUpdater>>> highAvailabilityUpdaters =
       new LinkedList<Map<String, Map<String, PropertyUpdater>>>();
 
@@ -308,7 +354,7 @@ public class BlueprintConfigurationProcessor {
     highAvailabilityUpdaters.addAll(updaters);
 
     // add the updaters for the dynamic HA properties, based on the HA config in hdfs-site
-    highAvailabilityUpdaters.add(createMapOfHAUpdaters());
+    highAvailabilityUpdaters.add(createMapOfYarnResourceManagerHAUpdaters());
 
     return highAvailabilityUpdaters;
   }
@@ -348,7 +394,7 @@ public class BlueprintConfigurationProcessor {
    *
    */
   public void doNameNodeHAUpdate() {
-    Map<String, Map<String, PropertyUpdater>> highAvailabilityUpdaters = createMapOfHAUpdaters();
+    Map<String, Map<String, PropertyUpdater>> highAvailabilityUpdaters = createMapOfNameNodeHAUpdaters();
 
     // perform a single host update on these dynamically generated property names
     if (highAvailabilityUpdaters.get("hdfs-site").size() > 0) {
@@ -357,6 +403,23 @@ public class BlueprintConfigurationProcessor {
   }
 
   /**
+   * Perform export update processing for HA configuration for Yarn ResourceManagers.  The HA ResourceManager
+   * property names are based on the ResourceManager names defined when HA is enabled via the Ambari UI, so this method
+   * dynamically determines the property names, and registers PropertyUpdaters to handle the masking of
+   * host names in these configuration items.
+   *
+   */
+  public void doYarnResourceManagerHAUpdate() {
+    Map<String, Map<String, PropertyUpdater>> highAvailabilityUpdaters = createMapOfYarnResourceManagerHAUpdaters();
+
+    // perform a single host update on these dynamically generated property names
+    if (highAvailabilityUpdaters.get("yarn-site").size() > 0) {
+      doSingleHostExportUpdate(highAvailabilityUpdaters, clusterTopology.getConfiguration().getFullProperties());
+    }
+  }
+
+
+  /**
    * Creates map of PropertyUpdater instances that are associated with
    *   NameNode High Availability (HA).  The HA configuration property
    *   names are dynamic, and based on other HA config elements in
@@ -365,7 +428,7 @@ public class BlueprintConfigurationProcessor {
    *
    * @return a Map of registered PropertyUpdaters for handling HA properties in hdfs-site
    */
-  private Map<String, Map<String, PropertyUpdater>> createMapOfHAUpdaters() {
+  private Map<String, Map<String, PropertyUpdater>> createMapOfNameNodeHAUpdaters() {
     Map<String, Map<String, PropertyUpdater>> highAvailabilityUpdaters = new HashMap<String, Map<String, PropertyUpdater>>();
     Map<String, PropertyUpdater> hdfsSiteUpdatersForAvailability = new HashMap<String, PropertyUpdater>();
     highAvailabilityUpdaters.put("hdfs-site", hdfsSiteUpdatersForAvailability);
@@ -386,15 +449,35 @@ public class BlueprintConfigurationProcessor {
     return highAvailabilityUpdaters;
   }
 
+
   /**
-   * Static convenience function to determine if Yarn ResourceManager HA is enabled
-   * @param configProperties configuration properties for this cluster
-   * @return true if Yarn ResourceManager HA is enabled
-   *         false if Yarn ResourceManager HA is not enabled
+   * Creates map of PropertyUpdater instances that are associated with
+   *   Yarn ResourceManager High Availability (HA).  The HA configuration property
+   *   names are dynamic, and based on other HA config elements in
+   *   yarn-site.  This method registers updaters for the required
+   *   properties associated with each ResourceManager.
+   *
+   * @return a Map of registered PropertyUpdaters for handling HA properties in yarn-site
    */
-  static boolean isYarnResourceManagerHAEnabled(Map<String, Map<String, String>> configProperties) {
-    return configProperties.containsKey("yarn-site") && configProperties.get("yarn-site").containsKey("yarn.resourcemanager.ha.enabled")
-      && configProperties.get("yarn-site").get("yarn.resourcemanager.ha.enabled").equals("true");
+  private Map<String, Map<String, PropertyUpdater>> createMapOfYarnResourceManagerHAUpdaters() {
+    Map<String, Map<String, PropertyUpdater>> highAvailabilityUpdaters = new HashMap<String, Map<String, PropertyUpdater>>();
+    Map<String, PropertyUpdater> yarnSiteUpdatersForAvailability = new HashMap<String, PropertyUpdater>();
+    highAvailabilityUpdaters.put("yarn-site", yarnSiteUpdatersForAvailability);
+
+    Map<String, String> yarnSiteConfig = clusterTopology.getConfiguration().getFullProperties().get("yarn-site");
+    // generate the property names based on the current HA config for the ResourceManager deployments
+    for (String resourceManager : parseResourceManagers(yarnSiteConfig)) {
+      final String rmHostPropertyName = "yarn.resourcemanager.hostname." + resourceManager;
+      yarnSiteUpdatersForAvailability.put(rmHostPropertyName, new SingleHostTopologyUpdater("RESOURCEMANAGER"));
+
+      final String rmHTTPAddress = "yarn.resourcemanager.webapp.address." + resourceManager;
+      yarnSiteUpdatersForAvailability.put(rmHTTPAddress, new SingleHostTopologyUpdater("RESOURCEMANAGER"));
+
+      final String rmHTTPSAddress = "yarn.resourcemanager.webapp.https.address." + resourceManager;
+      yarnSiteUpdatersForAvailability.put(rmHTTPSAddress, new SingleHostTopologyUpdater("RESOURCEMANAGER"));
+    }
+
+    return highAvailabilityUpdaters;
   }
 
   /**
@@ -460,6 +543,18 @@ public class BlueprintConfigurationProcessor {
   }
 
   /**
+   * Parses out the list of resource managers associated with this yarn-site configuration.
+   *
+   * @param properties config properties for this cluster
+   *
+   * @return array of Strings that indicate the ResourceManager names for this HA cluster
+   */
+  static String[] parseResourceManagers(Map<String, String> properties) {
+    final String resourceManagerNames = properties.get("yarn.resourcemanager.ha.rm-ids");
+    return splitAndTrimStrings(resourceManagerNames);
+  }
+
+  /**
    * Parses out the list of name nodes associated with a given HDFS
    *   NameService, based on a given HDFS configuration.
    *
@@ -808,7 +903,7 @@ public class BlueprintConfigurationProcessor {
               return origValue;
             }
 
-            if (isYarnResourceManagerHAEnabled(properties) && isComponentResourceManager() && (matchingGroupCount == 2)) {
+            if (topology.isYarnResourceManagerHAEnabled() && isComponentResourceManager() && (matchingGroupCount == 2)) {
               if (!origValue.contains("localhost")) {
                 // if this Yarn property is a FQDN, then simply return it
                 return origValue;
@@ -897,7 +992,7 @@ public class BlueprintConfigurationProcessor {
               return Collections.emptySet();
             }
 
-            if (isYarnResourceManagerHAEnabled(properties) && isComponentResourceManager() && (matchingGroupCount == 2)) {
+            if (topology.isYarnResourceManagerHAEnabled() && isComponentResourceManager() && (matchingGroupCount == 2)) {
               if (!origValue.contains("localhost")) {
                 // if this Yarn property is a FQDN, then simply return it
                 return Collections.emptySet();
@@ -1705,6 +1800,7 @@ public class BlueprintConfigurationProcessor {
     yarnSiteMap.put("yarn.resourcemanager.scheduler.address", new SingleHostTopologyUpdater("RESOURCEMANAGER"));
     yarnSiteMap.put("yarn.resourcemanager.address", new SingleHostTopologyUpdater("RESOURCEMANAGER"));
     yarnSiteMap.put("yarn.resourcemanager.admin.address", new SingleHostTopologyUpdater("RESOURCEMANAGER"));
+    yarnSiteMap.put("yarn.resourcemanager.webapp.https.address", new SingleHostTopologyUpdater("RESOURCEMANAGER"));
 
     // APP_TIMELINE_SERVER
     yarnSiteMap.put("yarn.timeline-service.address", new SingleHostTopologyUpdater("APP_TIMELINE_SERVER"));
@@ -1741,6 +1837,7 @@ public class BlueprintConfigurationProcessor {
     multiWebhcatSiteMap.put("templeton.zookeeper.hosts", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
     multiCoreSiteMap.put("ha.zookeeper.quorum", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
     multiYarnSiteMap.put("hadoop.registry.zk.quorum", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
+    multiYarnSiteMap.put("yarn.resourcemanager.zk-address", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
     multiSliderClientMap.put("slider.zookeeper.quorum", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
     multiKafkaBrokerMap.put("zookeeper.connect", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));
     multiAccumuloSiteMap.put("instance.zookeeper.host", new MultipleHostTopologyUpdater("ZOOKEEPER_SERVER"));

http://git-wip-us.apache.org/repos/asf/ambari/blob/ef40b86f/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopology.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopology.java b/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopology.java
index 6a7a79b..9a9929b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopology.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopology.java
@@ -118,6 +118,13 @@ public interface ClusterTopology {
   public boolean isNameNodeHAEnabled();
 
   /**
+   * Determine if Yarn ResourceManager HA is enabled.
+   *
+   * @return true if Yarn ResourceManager HA is enabled; false otherwise
+   */
+  public boolean isYarnResourceManagerHAEnabled();
+
+  /**
    * Determine if the cluster is kerberos enabled.
    *
    * @return true if the cluster is kerberos enabled; false otherwise

http://git-wip-us.apache.org/repos/asf/ambari/blob/ef40b86f/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopologyImpl.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopologyImpl.java b/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopologyImpl.java
index e0e79b3..6bf563b 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopologyImpl.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/topology/ClusterTopologyImpl.java
@@ -171,6 +171,22 @@ public class ClusterTopologyImpl implements ClusterTopology {
         configurationProperties.get("hdfs-site").containsKey("dfs.nameservices");
   }
 
+  @Override
+  public boolean isYarnResourceManagerHAEnabled() {
+    return isYarnResourceManagerHAEnabled(configuration.getFullProperties());
+  }
+
+  /**
+   * Static convenience function to determine if Yarn ResourceManager HA is enabled
+   * @param configProperties configuration properties for this cluster
+   * @return true if Yarn ResourceManager HA is enabled
+   *         false if Yarn ResourceManager HA is not enabled
+   */
+  static boolean isYarnResourceManagerHAEnabled(Map<String, Map<String, String>> configProperties) {
+    return configProperties.containsKey("yarn-site") && configProperties.get("yarn-site").containsKey("yarn.resourcemanager.ha.enabled")
+      && configProperties.get("yarn-site").get("yarn.resourcemanager.ha.enabled").equals("true");
+  }
+
   private void validateTopology(List<TopologyValidator> validators)
       throws InvalidTopologyException {
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/ef40b86f/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessorTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessorTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessorTest.java
index bebc884..6849b45 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessorTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/BlueprintConfigurationProcessorTest.java
@@ -2245,6 +2245,7 @@ public class BlueprintConfigurationProcessorTest {
     yarnSiteProperties.put("yarn.timeline-service.webapp.address", expectedHostName + ":" + expectedPortNum);
     yarnSiteProperties.put("yarn.timeline-service.webapp.https.address", expectedHostName + ":" + expectedPortNum);
     yarnSiteProperties.put("yarn.resourcemanager.ha.enabled", "true");
+    yarnSiteProperties.put("yarn.resourcemanager.ha.rm-ids", "rm1, rm2");
 
     Configuration clusterConfig = new Configuration(configProperties, Collections.<String, Map<String, Map<String, String>>>emptyMap());
     Collection<String> hgComponents = new HashSet<String>();
@@ -2291,6 +2292,107 @@ public class BlueprintConfigurationProcessorTest {
   }
 
   @Test
+  public void testYarnHighAvailabilityExport() throws Exception {
+    final String expectedHostName = "c6401.apache.ambari.org";
+    final String expectedHostNameTwo = "c6402.apache.ambari.org";
+    final String expectedPortNum = "808080";
+    final String expectedHostGroupName = "host_group_1";
+    final String expectedHostGroupNameTwo = "host_group_2";
+
+    Map<String, Map<String, String>> configProperties = new HashMap<String, Map<String, String>>();
+    Map<String, String> yarnSiteProperties = new HashMap<String, String>();
+    configProperties.put("yarn-site", yarnSiteProperties);
+
+    // setup properties that include host information
+    yarnSiteProperties.put("yarn.log.server.url", "http://" + expectedHostName +":19888/jobhistory/logs");
+    yarnSiteProperties.put("yarn.resourcemanager.hostname", expectedHostName);
+    yarnSiteProperties.put("yarn.resourcemanager.resource-tracker.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.scheduler.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.admin.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.timeline-service.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.timeline-service.webapp.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.timeline-service.webapp.https.address", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.ha.enabled", "true");
+    yarnSiteProperties.put("yarn.resourcemanager.ha.rm-ids", "rm1, rm2");
+    yarnSiteProperties.put("yarn.resourcemanager.hostname.rm1", expectedHostName);
+    yarnSiteProperties.put("yarn.resourcemanager.hostname.rm2", expectedHostNameTwo);
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.address.rm1", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.address.rm2", expectedHostNameTwo + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.https.address.rm1", expectedHostName + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.https.address.rm2", expectedHostNameTwo + ":" + expectedPortNum);
+    yarnSiteProperties.put("yarn.resourcemanager.zk-address", expectedHostName + ":" + "2181" + "," + expectedHostNameTwo + ":" + "2181");
+    yarnSiteProperties.put("yarn.resourcemanager.webapp.https.address", expectedHostName + ":" + "8080");
+
+
+    Configuration clusterConfig = new Configuration(configProperties, Collections.<String, Map<String, Map<String, String>>>emptyMap());
+    Collection<String> hgComponents = new HashSet<String>();
+    hgComponents.add("RESOURCEMANAGER");
+    hgComponents.add("APP_TIMELINE_SERVER");
+    hgComponents.add("HISTORYSERVER");
+    TestHostGroup group1 = new TestHostGroup(expectedHostGroupName, hgComponents, Collections.singleton(expectedHostName));
+
+    Collection<String> hgComponents2 = new HashSet<String>();
+    hgComponents2.add("RESOURCEMANAGER");
+    TestHostGroup group2 = new TestHostGroup(expectedHostGroupNameTwo, hgComponents2, Collections.singleton(expectedHostNameTwo));
+
+    Collection<TestHostGroup> hostGroups = new HashSet<TestHostGroup>();
+    hostGroups.add(group1);
+    hostGroups.add(group2);
+
+    expect(stack.getCardinality("RESOURCEMANAGER")).andReturn(new Cardinality("1-2")).anyTimes();
+
+    ClusterTopology topology = createClusterTopology(bp, clusterConfig, hostGroups);
+    BlueprintConfigurationProcessor updater = new BlueprintConfigurationProcessor(topology);
+    updater.doUpdateForBlueprintExport();
+
+    // verify that the properties with hostname information was correctly preserved
+    assertEquals("Yarn Log Server URL was incorrectly updated",
+      "http://" + createExportedAddress("19888", expectedHostGroupName) + "/jobhistory/logs", yarnSiteProperties.get("yarn.log.server.url"));
+    assertEquals("Yarn ResourceManager hostname was incorrectly updated",
+      createExportedHostName(expectedHostGroupName), yarnSiteProperties.get("yarn.resourcemanager.hostname"));
+    assertEquals("Yarn ResourceManager tracker address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.resource-tracker.address"));
+    assertEquals("Yarn ResourceManager webapp address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.webapp.address"));
+    assertEquals("Yarn ResourceManager scheduler address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.scheduler.address"));
+    assertEquals("Yarn ResourceManager address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.address"));
+    assertEquals("Yarn ResourceManager admin address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.admin.address"));
+    assertEquals("Yarn ResourceManager timeline-service address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.timeline-service.address"));
+    assertEquals("Yarn ResourceManager timeline webapp address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.timeline-service.webapp.address"));
+    assertEquals("Yarn ResourceManager timeline webapp HTTPS address was incorrectly updated",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.timeline-service.webapp.https.address"));
+
+    // verify that dynamically-named RM HA properties are exported as expected
+    assertEquals("Yarn ResourceManager rm1 hostname not exported properly",
+      createExportedHostName(expectedHostGroupName), yarnSiteProperties.get("yarn.resourcemanager.hostname.rm1"));
+    assertEquals("Yarn ResourceManager rm2 hostname not exported properly",
+      createExportedHostName(expectedHostGroupNameTwo), yarnSiteProperties.get("yarn.resourcemanager.hostname.rm2"));
+    assertEquals("Yarn ResourceManager rm1 web address not exported properly",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.webapp.address.rm1"));
+    assertEquals("Yarn ResourceManager rm2 web address not exported properly",
+      createExportedHostName(expectedHostGroupNameTwo, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.webapp.address.rm2"));
+    assertEquals("Yarn ResourceManager rm1 HTTPS address not exported properly",
+      createExportedHostName(expectedHostGroupName, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.webapp.https.address.rm1"));
+    assertEquals("Yarn ResourceManager rm2 HTTPS address not exported properly",
+      createExportedHostName(expectedHostGroupNameTwo, expectedPortNum), yarnSiteProperties.get("yarn.resourcemanager.webapp.https.address.rm2"));
+
+    assertEquals("Yarn Zookeeper address property not exported properly",
+      createExportedHostName(expectedHostGroupName, "2181") + "," + createExportedHostName(expectedHostGroupNameTwo, "2181"),
+      yarnSiteProperties.get("yarn.resourcemanager.zk-address"));
+
+    assertEquals("Yarn RM webapp address not exported properly",
+      createExportedHostName(expectedHostGroupName, "8080"), yarnSiteProperties.get("yarn.resourcemanager.webapp.https.address"));
+
+  }
+
+  @Test
   public void testHDFSConfigClusterUpdateQuorumJournalURLSpecifyingHostNamesDirectly() throws Exception {
     final String expectedHostNameOne = "c6401.apache.ambari.org";
     final String expectedHostNameTwo = "c6402.apache.ambari.org";