You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by nc...@apache.org on 2016/02/02 13:40:47 UTC

[02/11] ambari git commit: AMBARI-14814. Ambari not showing active/standby status for NameNodes when custom ports are used.(vbrodetskyi)

AMBARI-14814. Ambari not showing active/standby status for NameNodes when custom ports are used.(vbrodetskyi)


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

Branch: refs/heads/branch-dev-patch-upgrade
Commit: 5e36bdc922a92405f9d82ccbdff0ba8006903052
Parents: 31477a5
Author: Vitaly Brodetskyi <vb...@hortonworks.com>
Authored: Mon Feb 1 18:56:56 2016 +0200
Committer: Vitaly Brodetskyi <vb...@hortonworks.com>
Committed: Mon Feb 1 18:56:56 2016 +0200

----------------------------------------------------------------------
 .../internal/AbstractProviderModule.java        | 187 +++++++++++++------
 .../server/controller/jmx/JMXHostProvider.java  |   8 +-
 .../controller/jmx/JMXPropertyProvider.java     |  15 +-
 .../internal/JMXHostProviderTest.java           | 130 ++++++++++---
 .../metrics/JMXPropertyProviderTest.java        |   6 +-
 5 files changed, 249 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/5e36bdc9/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AbstractProviderModule.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AbstractProviderModule.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AbstractProviderModule.java
index 5fb2cf0..52d95e0 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AbstractProviderModule.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/internal/AbstractProviderModule.java
@@ -125,8 +125,10 @@ public abstract class AbstractProviderModule implements ProviderModule,
 
     Map<String, String[]> initPropMap = new HashMap<String, String[]>();
     initPropMap.put("NAMENODE", new String[]{"dfs.http.address", "dfs.namenode.http-address"});
-    initPropMap.put("DATANODE", new String[]{"dfs.datanode.http.address"});
     initPropMap.put("NAMENODE-HTTPS", new String[]{"dfs.namenode.https-address", "dfs.https.port"});
+    initPropMap.put("NAMENODE-HA", new String[]{"dfs.namenode.http-address.%s.%s"});
+    initPropMap.put("NAMENODE-HTTPS-HA", new String[]{"dfs.namenode.https-address.%s.%s"});
+    initPropMap.put("DATANODE", new String[]{"dfs.datanode.http.address"});
     initPropMap.put("JOURNALNODE-HTTPS", new String[]{"dfs.journalnode.https-address"});
     initPropMap.put("JOURNALNODE", new String[]{"dfs.journalnode.http-address"});
     serviceDesiredProperties.put(Service.Type.HDFS, initPropMap);
@@ -197,8 +199,8 @@ public abstract class AbstractProviderModule implements ProviderModule,
   /**
    * JMX ports read from the configs
    */
-  private final Map<String, Map<String, String>> jmxPortMap =
-      new HashMap<String, Map<String, String>>();
+  private final Map<String, Map<String, Map<String, String>> >jmxPortMap =
+      new HashMap<String, Map<String, Map<String, String>>>();
 
   private volatile boolean initialized = false;
 
@@ -481,19 +483,19 @@ public abstract class AbstractProviderModule implements ProviderModule,
   // ----- JMXHostProvider ---------------------------------------------------
 
   @Override
-  public String getPort(String clusterName, String componentName) throws SystemException {
-    return getPort(clusterName, componentName, false);
+  public String getPort(String clusterName, String componentName, String hostName) throws SystemException {
+    return getPort(clusterName, componentName, hostName, false);
   }
 
   @Override
-  public String getPort(String clusterName, String componentName, boolean httpsEnabled) throws SystemException {
+  public String getPort(String clusterName, String componentName, String hostName, boolean httpsEnabled) throws SystemException {
     // Parent map need not be synchronized
-    Map<String, String> clusterJmxPorts = jmxPortMap.get(clusterName);
+    Map<String, Map<String, String>> clusterJmxPorts = jmxPortMap.get(clusterName);
     if (clusterJmxPorts == null) {
       synchronized (jmxPortMap) {
         clusterJmxPorts = jmxPortMap.get(clusterName);
         if (clusterJmxPorts == null) {
-          clusterJmxPorts = new ConcurrentHashMap<String, String>();
+          clusterJmxPorts = new ConcurrentHashMap<String, Map<String, String>>();
           jmxPortMap.put(clusterName, clusterJmxPorts);
         }
       }
@@ -511,30 +513,40 @@ public abstract class AbstractProviderModule implements ProviderModule,
         // performance with a ConcurrentHashMap and maybe get default/existing
         // ports for a few calls.
         if (!currVersion.equals(oldVersion) ||
-            !clusterJmxPorts.containsKey(componentName)) {
+            !(clusterJmxPorts.containsKey(hostName) && clusterJmxPorts.get(hostName).containsKey(componentName))) {
 
           serviceConfigVersions.put(configType, currVersion);
 
-          Map<String, String[]> componentPorts = new HashMap<String, String[]>();
-          String[] componentsHttpsPorts;
-
-          if (httpsEnabled) {
-            componentsHttpsPorts = serviceDesiredProperties.get(service).get(componentName + "-HTTPS");
-          } else {
-            componentsHttpsPorts = serviceDesiredProperties.get(service).get(componentName);
-          }
-          componentPorts.put(componentName, componentsHttpsPorts);
+          Map<String, Object> configProperties = getConfigProperties(
+              clusterName,
+              currVersion,
+              serviceConfigTypes.get(service)
+          );
+
+          Map<String, String[]> componentPortsProperties = new HashMap<String, String[]>();
+          componentPortsProperties.put(
+              componentName,
+              getPortProperties(service,
+                  componentName,
+                  hostName,
+                  configProperties,
+                  httpsEnabled
+              )
+          );
 
           Map<String, String> portMap = getDesiredConfigMap(clusterName,
               currVersion, serviceConfigTypes.get(service),
-              componentPorts);
+              componentPortsProperties, configProperties);
 
           for (Entry<String, String> entry : portMap.entrySet()) {
             // portString will be null if the property defined for the component doesn't exist
             // this will trigger using the default port for the component
             String portString = getPortString(entry.getValue());
             if (null != portString) {
-              clusterJmxPorts.put(entry.getKey(), portString);
+              if(!clusterJmxPorts.containsKey(hostName)) {
+                clusterJmxPorts.put(hostName, new ConcurrentHashMap<String, String>());
+              }
+              clusterJmxPorts.get(hostName).put(entry.getKey(), portString);
             }
           }
         }
@@ -544,7 +556,46 @@ public abstract class AbstractProviderModule implements ProviderModule,
     }
 
     LOG.debug("jmxPortMap -> " + jmxPortMap);
-    return clusterJmxPorts.get(componentName);
+    return clusterJmxPorts.get(hostName).get(componentName);
+  }
+
+  /**
+   * Computes properties that contains proper port for {@code componentName} on {@code hostName}. Must contain custom logic
+   * for different configurations(like NAMENODE HA).
+   * @param service service type
+   * @param componentName component name
+   * @param hostName host which contain requested component
+   * @param properties properties which used for special cases(like NAMENODE HA)
+   * @param httpsEnabled indicates if https enabled for component
+   * @return property name that contain port for {@code componentName} on {@code hostName}
+   */
+  String[] getPortProperties(Service.Type service, String componentName, String hostName, Map<String, Object> properties, boolean httpsEnabled) {
+    componentName = httpsEnabled ? componentName + "-HTTPS" : componentName;
+
+    if(componentName.equals("NAMENODE")) {
+      // iterate over nameservices and namenodes, to find out namenode http(s) property for concrete host
+      if(properties.containsKey("dfs.nameservices")) {
+        for(String nameserviceId : ((String)properties.get("dfs.nameservices")).split(",")) {
+          String haComponentName = componentName + "-HA";
+          if(properties.containsKey("dfs.ha.namenodes."+nameserviceId)) {
+            for (String namenodeId : ((String)properties.get("dfs.ha.namenodes." + nameserviceId)).split(",")) {
+              String propertyName = String.format(
+                  serviceDesiredProperties.get(service).get(haComponentName)[0],
+                  nameserviceId,
+                  namenodeId
+              );
+              if (properties.containsKey(propertyName)) {
+                String propertyValue = (String)properties.get(propertyName);
+                if (propertyValue.split(":")[0].equals(hostName)) {
+                  return new String[]{propertyName};
+                }
+              }
+            }
+          }
+        }
+      }
+    }
+    return serviceDesiredProperties.get(service).get(componentName);
   }
 
   /**
@@ -882,9 +933,9 @@ public abstract class AbstractProviderModule implements ProviderModule,
     }
     return versionTag;
   }
-  // TODO get configs using ConfigHelper
-  private Map<String, String> getDesiredConfigMap(String clusterName, String versionTag,
-      String configType, Map<String, String[]> keys) throws NoSuchParentResourceException,
+
+  private Map<String, Object> getConfigProperties(String clusterName, String versionTag, String configType)
+      throws NoSuchParentResourceException,
       UnsupportedPropertyException, SystemException {
     // Get desired configs based on the tag
     ResourceProvider configResourceProvider = getResourceProvider(Resource.Type.Configuration);
@@ -902,53 +953,67 @@ public abstract class AbstractProviderModule implements ProviderModule,
       LOG.info("Resource for the desired config not found. " + e);
       return Collections.emptyMap();
     }
+    for (Resource res : configResources) {
+      return res.getPropertiesMap().get(PROPERTIES_CATEGORY);
+    }
+    return Collections.emptyMap();
+  }
+
+  private Map<String, String> getDesiredConfigMap(String clusterName, String versionTag,
+                                                  String configType, Map<String, String[]> keys) throws NoSuchParentResourceException,
+      UnsupportedPropertyException, SystemException {
+    return getDesiredConfigMap(clusterName, versionTag, configType, keys, null);
+  }
 
+  // TODO get configs using ConfigHelper
+  private Map<String, String> getDesiredConfigMap(String clusterName, String versionTag,
+                                                  String configType, Map<String, String[]> keys, Map<String, Object> configProperties) throws NoSuchParentResourceException,
+      UnsupportedPropertyException, SystemException {
+    if(configProperties == null) {
+      configProperties = getConfigProperties(clusterName, versionTag, configType);
+    }
     Map<String, String> mConfigs = new HashMap<String, String>();
-    if (configResources != null) {
-      for (Resource res : configResources) {
-        Map<String, String> evaluatedProperties = null;
-        for (Entry<String, String[]> entry : keys.entrySet()) {
-          String propName = null;
-          String value = null;
-
-          for (String pname : entry.getValue()) {
-            propName = pname;
-            // For NN HA the property key contains nameservice id
-            Map<String, Object> properties = res.getPropertiesMap().get(PROPERTIES_CATEGORY);
-            if (properties != null) {
-              for (Map.Entry<String, Object> propertyEntry : properties.entrySet()) {
-                if (propertyEntry.getKey().startsWith(pname)) {
-                  value = (String) propertyEntry.getValue();
-                  break;
-                }
-              }
-            }
-            if (null != value) {
+    if (!configProperties.isEmpty()) {
+      Map<String, String> evaluatedProperties = null;
+      for (Entry<String, String[]> entry : keys.entrySet()) {
+        String propName = null;
+        String value = null;
+
+        for (String pname : entry.getValue()) {
+          propName = pname;
+          // For NN HA the property key contains nameservice id
+          for (Map.Entry<String, Object> propertyEntry : configProperties.entrySet()) {
+            if (propertyEntry.getKey().startsWith(pname)) {
+              value = (String) propertyEntry.getValue();
               break;
             }
           }
 
-          if (value != null && value.contains("${")) {
-            if (evaluatedProperties == null) {
-              evaluatedProperties = new HashMap<String, String>();
-              Map<String, Object> properties = res.getPropertiesMap().get(PROPERTIES_CATEGORY);
-              for (Map.Entry<String, Object> subentry : properties.entrySet()) {
-                String keyString = subentry.getKey();
-                Object object = subentry.getValue();
-                String valueString;
-                if (object != null && object instanceof String) {
-                  valueString = (String) object;
-                  evaluatedProperties.put(keyString, valueString);
-                  postProcessPropertyValue(keyString, valueString, evaluatedProperties, null);
-                }
+          if (null != value) {
+            break;
+          }
+        }
+
+        if (value != null && value.contains("${")) {
+          if (evaluatedProperties == null) {
+            evaluatedProperties = new HashMap<String, String>();
+            for (Map.Entry<String, Object> subentry : configProperties.entrySet()) {
+              String keyString = subentry.getKey();
+              Object object = subentry.getValue();
+              String valueString;
+              if (object != null && object instanceof String) {
+                valueString = (String) object;
+                evaluatedProperties.put(keyString, valueString);
+                postProcessPropertyValue(keyString, valueString, evaluatedProperties, null);
               }
             }
           }
-          value = postProcessPropertyValue(propName, value, evaluatedProperties, null);
-          LOG.debug("PROPERTY -> key: " + propName + ", " + "value: " + value);
-
-          mConfigs.put(entry.getKey(), value);
         }
+        value = postProcessPropertyValue(propName, value, evaluatedProperties, null);
+        LOG.debug("PROPERTY -> key: " + propName + ", " + "value: " + value);
+
+        mConfigs.put(entry.getKey(), value);
+
       }
     }
     return mConfigs;

http://git-wip-us.apache.org/repos/asf/ambari/blob/5e36bdc9/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXHostProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXHostProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXHostProvider.java
index ca5b89f..15d44b2 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXHostProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXHostProvider.java
@@ -42,12 +42,13 @@ public interface JMXHostProvider {
    *
    * @param clusterName    the cluster name
    * @param componentName  the component name
+   * @param hostName       the component hostName
    *
    * @return the port for the specified cluster name and component
    *
    * @throws SystemException if unable to get the JMX port
    */
-  public String getPort(String clusterName, String componentName)
+  public String getPort(String clusterName, String componentName, String hostName)
       throws SystemException;
 
   /**
@@ -55,13 +56,14 @@ public interface JMXHostProvider {
    *
    * @param clusterName    the cluster name
    * @param componentName  the component name
-   * @param componentName httpsEnabled https enabled
+   * @param hostName       the component hostName
+   * @param httpsEnabled   https enabled
    *
    * @return the port for the specified cluster name and component
    *
    * @throws SystemException if unable to get the JMX port
    */
-  public String getPort(String clusterName, String componentName, boolean httpsEnabled)
+  public String getPort(String clusterName, String componentName, String hostName,  boolean httpsEnabled)
       throws SystemException;
   
   /**

http://git-wip-us.apache.org/repos/asf/ambari/blob/5e36bdc9/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXPropertyProvider.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXPropertyProvider.java b/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXPropertyProvider.java
index f8215f6..1e7f4af 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXPropertyProvider.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/controller/jmx/JMXPropertyProvider.java
@@ -198,12 +198,6 @@ public class JMXPropertyProvider extends ThreadPoolEnabledPropertyProvider {
       httpsEnabled = true;
     }
 
-    String port = getPort(clusterName, componentName, httpsEnabled);
-    if (port == null) {
-      LOG.warn("Unable to get JMX metrics.  No port value for " + componentName);
-      return resource;
-    }
-
     Set<String> hostNames = getHosts(resource, clusterName, componentName);
     if (hostNames == null || hostNames.isEmpty()) {
       LOG.warn("Unable to get JMX metrics.  No host name for " + componentName);
@@ -216,6 +210,11 @@ public class JMXPropertyProvider extends ThreadPoolEnabledPropertyProvider {
       try {
         for (String hostName : hostNames) {
           try {
+            String port = getPort(clusterName, componentName, hostName, httpsEnabled);
+            if (port == null) {
+              LOG.warn("Unable to get JMX metrics.  No port value for " + componentName);
+              return resource;
+            }
             if (LOG.isDebugEnabled()) {
               LOG.debug("Spec: " + getSpec(protocol, hostName, port, "/jmx"));
             }
@@ -357,11 +356,11 @@ public class JMXPropertyProvider extends ThreadPoolEnabledPropertyProvider {
     }
   }
 
-  private String getPort(String clusterName, String componentName, boolean httpsEnabled) throws SystemException {
+  private String getPort(String clusterName, String componentName, String hostName, boolean httpsEnabled) throws SystemException {
     String portMapKey = String.format("%s-%s-%s", clusterName, componentName, httpsEnabled);
     String port = clusterComponentPortsMap.get(portMapKey);
     if (port==null) {
-      port = jmxHostProvider.getPort(clusterName, componentName, httpsEnabled);
+      port = jmxHostProvider.getPort(clusterName, componentName, hostName, httpsEnabled);
       port = port == null ? DEFAULT_JMX_PORTS.get(componentName) : port;
       clusterComponentPortsMap.put(portMapKey, port);
     }

http://git-wip-us.apache.org/repos/asf/ambari/blob/5e36bdc9/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/JMXHostProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/JMXHostProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/JMXHostProviderTest.java
index 37190ee..187e0a6 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/JMXHostProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/internal/JMXHostProviderTest.java
@@ -311,6 +311,75 @@ public class JMXHostProviderTest {
       ("hdfs-site").getProperties().get(NAMENODE_PORT_V1));
   }
 
+  private void createConfigsNameNodeHa() throws AmbariException, AuthorizationException {
+    String clusterName = "nnha";
+    ClusterRequest r = new ClusterRequest(null, clusterName, "HDP-2.0.6", null);
+    controller.createCluster(r);
+    Cluster cluster = clusters.getCluster(clusterName);
+    cluster.setDesiredStackVersion(new StackId("HDP-2.0.6"));
+    String serviceName = "HDFS";
+    createService(clusterName, serviceName, null);
+    String componentName1 = "NAMENODE";
+    String componentName2 = "DATANODE";
+    String componentName3 = "HDFS_CLIENT";
+
+    createServiceComponent(clusterName, serviceName, componentName1,
+        State.INIT);
+    createServiceComponent(clusterName, serviceName, componentName2,
+        State.INIT);
+    createServiceComponent(clusterName, serviceName, componentName3,
+        State.INIT);
+
+    String host1 = "h1";
+    clusters.addHost(host1);
+    Map<String, String> hostAttributes = new HashMap<String, String>();
+    hostAttributes.put("os_family", "redhat");
+    hostAttributes.put("os_release_version", "5.9");
+    clusters.getHost("h1").setHostAttributes(hostAttributes);
+    clusters.getHost("h1").persist();
+    String host2 = "h2";
+    clusters.addHost(host2);
+    hostAttributes = new HashMap<String, String>();
+    hostAttributes.put("os_family", "redhat");
+    hostAttributes.put("os_release_version", "6.3");
+    clusters.getHost("h2").setHostAttributes(hostAttributes);
+    clusters.getHost("h2").persist();
+    clusters.mapHostToCluster(host1, clusterName);
+    clusters.mapHostToCluster(host2, clusterName);
+
+    createServiceComponentHost(clusterName, serviceName, componentName1,
+        host1, null);
+    createServiceComponentHost(clusterName, serviceName, componentName1,
+        host2, null);
+    createServiceComponentHost(clusterName, serviceName, componentName2,
+        host1, null);
+    createServiceComponentHost(clusterName, serviceName, componentName2,
+        host2, null);
+    createServiceComponentHost(clusterName, serviceName, componentName3,
+        host1, null);
+    createServiceComponentHost(clusterName, serviceName, componentName3,
+        host2, null);
+
+    // Create configs
+    Map<String, String> configs = new HashMap<String, String>();
+    configs.put("dfs.nameservices", "ns");
+    configs.put("dfs.namenode.http-address", "h1:50070");
+    configs.put("dfs.namenode.http-address.ns.nn1", "h1:50071");
+    configs.put("dfs.namenode.http-address.ns.nn2", "h2:50072");
+    configs.put("dfs.namenode.https-address", "h1:50470");
+    configs.put("dfs.namenode.https-address.ns.nn1", "h1:50473");
+    configs.put("dfs.namenode.https-address.ns.nn2", "h2:50474");
+    configs.put("dfs.ha.namenodes.ns", "nn1,nn2");
+
+
+    ConfigurationRequest cr1 = new ConfigurationRequest(clusterName,
+        "hdfs-site", "version1", configs, null);
+
+    ClusterRequest crReq = new ClusterRequest(cluster.getClusterId(), clusterName, null, null);
+    crReq.setDesiredConfig(Collections.singletonList(cr1));
+    controller.updateClusters(Collections.singleton(crReq), null);
+  }
+
 
   @Test
   public void testJMXPortMapInitAtServiceLevelVersion1() throws
@@ -324,12 +393,12 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Service);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     // Non default port addresses
-    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE"));
-    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE"));
+    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE", "localhost"));
+    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE", "localhost"));
     // Default port addresses
-    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER", "localhost"));
   }
 
   @Test
@@ -344,12 +413,29 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Service);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     // Non default port addresses
-    Assert.assertEquals("70071", providerModule.getPort("c1", "NAMENODE"));
-    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE"));
+    Assert.assertEquals("70071", providerModule.getPort("c1", "NAMENODE", "localhost"));
+    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE", "localhost"));
     // Default port addresses
-    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER", "localhost"));
+  }
+
+  @Test
+  public void testJMXPortMapNameNodeHa() throws
+      NoSuchParentResourceException,
+      ResourceAlreadyExistsException, UnsupportedPropertyException,
+      SystemException, AmbariException, NoSuchResourceException {
+
+    createConfigsNameNodeHa();
+
+    JMXHostProviderModule providerModule = new JMXHostProviderModule();
+    providerModule.registerResourceProvider(Resource.Type.Service);
+    providerModule.registerResourceProvider(Resource.Type.Configuration);
+
+
+    Assert.assertEquals("50071", providerModule.getPort("nnha", "NAMENODE", "h1"));
+    Assert.assertEquals("50072", providerModule.getPort("nnha", "NAMENODE", "h2"));
   }
 
   @Test
@@ -364,12 +450,12 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Cluster);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     // Non default port addresses
-    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE"));
-    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE"));
+    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE", "localhost"));
+    Assert.assertEquals("70075", providerModule.getPort("c1", "DATANODE", "localhost"));
     // Default port addresses
-    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "TASKTRACKER", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "HBASE_MASTER", "localhost"));
   }
 
   @Test
@@ -411,7 +497,7 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Cluster);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     Assert.assertEquals("https", providerModule.getJMXProtocol("c1", "RESOURCEMANAGER"));
-    Assert.assertEquals("8090", providerModule.getPort("c1", "RESOURCEMANAGER", true));
+    Assert.assertEquals("8090", providerModule.getPort("c1", "RESOURCEMANAGER", "localhost", true));
 
   }
 
@@ -425,7 +511,7 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Cluster);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     Assert.assertEquals("https", providerModule.getJMXProtocol("c1", "JOURNALNODE"));
-    Assert.assertEquals("8481", providerModule.getPort("c1", "JOURNALNODE", true));
+    Assert.assertEquals("8481", providerModule.getPort("c1", "JOURNALNODE", "localhost", true));
   }
 
   @Test
@@ -440,7 +526,7 @@ public class JMXHostProviderTest {
     providerModule.registerResourceProvider(Resource.Type.Cluster);
     providerModule.registerResourceProvider(Resource.Type.Configuration);
     // Non default port addresses
-    Assert.assertEquals("8088", providerModule.getPort("c1", "RESOURCEMANAGER"));
+    Assert.assertEquals("8088", providerModule.getPort("c1", "RESOURCEMANAGER", "localhost"));
 
     Map<String, String> yarnConfigs = new HashMap<String, String>();
     yarnConfigs.put(RESOURCEMANAGER_PORT, "localhost:50030");
@@ -451,12 +537,12 @@ public class JMXHostProviderTest {
     ClusterRequest crReq = new ClusterRequest(1L, "c1", null, null);
     crReq.setDesiredConfig(Collections.singletonList(cr2));
     controller.updateClusters(Collections.singleton(crReq), null);
-    Assert.assertEquals("50030", providerModule.getPort("c1", "RESOURCEMANAGER"));
-    Assert.assertEquals("11111", providerModule.getPort("c1", "NODEMANAGER"));
+    Assert.assertEquals("50030", providerModule.getPort("c1", "RESOURCEMANAGER", "localhost"));
+    Assert.assertEquals("11111", providerModule.getPort("c1", "NODEMANAGER", "localhost"));
 
     //Unrelated ports
-    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE"));
-    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER"));
+    Assert.assertEquals("70070", providerModule.getPort("c1", "NAMENODE", "localhost"));
+    Assert.assertEquals(null, providerModule.getPort("c1", "JOBTRACKER", "localhost"));
   }
 
   private static class JMXHostProviderModule extends AbstractProviderModule {

http://git-wip-us.apache.org/repos/asf/ambari/blob/5e36bdc9/ambari-server/src/test/java/org/apache/ambari/server/controller/metrics/JMXPropertyProviderTest.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/test/java/org/apache/ambari/server/controller/metrics/JMXPropertyProviderTest.java b/ambari-server/src/test/java/org/apache/ambari/server/controller/metrics/JMXPropertyProviderTest.java
index 2fef7be..6dfced1 100644
--- a/ambari-server/src/test/java/org/apache/ambari/server/controller/metrics/JMXPropertyProviderTest.java
+++ b/ambari-server/src/test/java/org/apache/ambari/server/controller/metrics/JMXPropertyProviderTest.java
@@ -491,12 +491,12 @@ public class JMXPropertyProviderTest {
     }
 
     @Override
-    public String getPort(String clusterName, String componentName, boolean httpsEnabled) throws SystemException {
-      return getPort(clusterName, componentName);
+    public String getPort(String clusterName, String componentName, String hostName, boolean httpsEnabled) throws SystemException {
+      return getPort(clusterName, componentName, hostName);
     }
 
     @Override
-    public String getPort(String clusterName, String componentName) throws
+    public String getPort(String clusterName, String componentName, String hostName) throws
       SystemException {
 
       if (unknownPort) {