You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by jo...@apache.org on 2015/03/02 15:38:12 UTC

ambari git commit: AMBARI-9848 - Alerts Need HA Mode Dynamic Property Support For WEB and METRIC (jonathanhurley)

Repository: ambari
Updated Branches:
  refs/heads/trunk 3b6d56da8 -> 7607fc1e5


AMBARI-9848 - Alerts Need HA Mode Dynamic Property Support For WEB and METRIC (jonathanhurley)


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

Branch: refs/heads/trunk
Commit: 7607fc1e5ccea98f068a7e26b188154db0b003ca
Parents: 3b6d56d
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Sat Feb 28 11:26:32 2015 -0500
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Mon Mar 2 08:42:37 2015 -0500

----------------------------------------------------------------------
 .../python/ambari_agent/alerts/base_alert.py    | 167 ++++++++++++++++---
 .../src/test/python/ambari_agent/TestAlerts.py  | 118 +++++++++++++
 .../ambari/server/state/alert/AlertUri.java     | 100 +++++++++++
 .../common-services/HDFS/2.1.0.2.0/alerts.json  |  82 +++++----
 .../stacks/BIGTOP/0.8/services/HDFS/alerts.json | 123 ++++++++++----
 5 files changed, 501 insertions(+), 89 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/7607fc1e/ambari-agent/src/main/python/ambari_agent/alerts/base_alert.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/alerts/base_alert.py b/ambari-agent/src/main/python/ambari_agent/alerts/base_alert.py
index 34aeff6..3ae3c6d 100644
--- a/ambari-agent/src/main/python/ambari_agent/alerts/base_alert.py
+++ b/ambari-agent/src/main/python/ambari_agent/alerts/base_alert.py
@@ -25,19 +25,24 @@ from collections import namedtuple
 
 logger = logging.getLogger()
 
+# create a named tuple to return both the concrete URI and SSL flag
+AlertUri = namedtuple('AlertUri', 'uri is_ssl_enabled')
+
 class BaseAlert(object):
-  RESULT_OK = 'OK'
-  RESULT_WARNING = 'WARNING'
-  RESULT_CRITICAL = 'CRITICAL'
-  RESULT_UNKNOWN = 'UNKNOWN'
-  RESULT_SKIPPED = 'SKIPPED'
-  
+  RESULT_OK = "OK"
+  RESULT_WARNING = "WARNING"
+  RESULT_CRITICAL = "CRITICAL"
+  RESULT_UNKNOWN = "UNKNOWN"
+  RESULT_SKIPPED = "SKIPPED"
+
+  HA_NAMESERVICE_PARAM = "{{ha-nameservice}}"
+  HA_ALIAS_PARAM = "{{alias}}"
+
   def __init__(self, alert_meta, alert_source_meta):
     self.alert_meta = alert_meta
     self.alert_source_meta = alert_source_meta
     self.cluster_name = ''
     self.host_name = ''
-    self._lookup_keys = []
     
     
   def interval(self):
@@ -166,6 +171,9 @@ class BaseAlert(object):
     returned.
     :return:
     """
+    if key is None:
+      return None
+
     # parse {{foo-bar/baz}}
     placeholder_keys = re.findall("{{([\S]+)}}", key)
 
@@ -204,6 +212,10 @@ class BaseAlert(object):
     default_port = None
     kerberos_keytab = None
     kerberos_principal = None
+    ha_nameservice = None
+    ha_alias_key = None
+    ha_http_pattern = None
+    ha_https_pattern = None
     
     if 'http' in uri_structure:
       http_key = uri_structure['http']
@@ -226,13 +238,34 @@ class BaseAlert(object):
     if 'kerberos_principal' in uri_structure:
       kerberos_principal = uri_structure['kerberos_principal']
 
+    if 'high_availability' in uri_structure:
+      ha = uri_structure['high_availability']
+
+      if 'nameservice' in ha:
+        ha_nameservice = ha['nameservice']
+
+      if 'alias_key' in ha:
+        ha_alias_key = ha['alias_key']
+
+      if 'http_pattern' in ha:
+        ha_http_pattern = ha['http_pattern']
+
+      if 'https_pattern' in ha:
+        ha_https_pattern = ha['https_pattern']
+
+
     AlertUriLookupKeys = namedtuple('AlertUriLookupKeys', 
-        'http https https_property https_property_value default_port kerberos_keytab kerberos_principal')
+      'http https https_property https_property_value default_port '
+      'kerberos_keytab kerberos_principal '
+      'ha_nameservice ha_alias_key ha_http_pattern ha_https_pattern')
     
     alert_uri_lookup_keys = AlertUriLookupKeys(http=http_key, https=https_key, 
-        https_property=https_property_key, 
-        https_property_value=https_property_value_key, default_port=default_port,
-        kerberos_keytab=kerberos_keytab, kerberos_principal=kerberos_principal)
+      https_property=https_property_key,
+      https_property_value=https_property_value_key, default_port=default_port,
+      kerberos_keytab=kerberos_keytab, kerberos_principal=kerberos_principal,
+      ha_nameservice=ha_nameservice, ha_alias_key=ha_alias_key,
+      ha_http_pattern=ha_http_pattern, ha_https_pattern=ha_https_pattern
+    )
     
     return alert_uri_lookup_keys
 
@@ -255,12 +288,15 @@ class BaseAlert(object):
     
     http_uri = None
     https_uri = None
-    https_property = None
-    https_property_value = None
 
-    # create a named tuple to return both the concrete URI and SSL flag
-    AlertUri = namedtuple('AlertUri', 'uri is_ssl_enabled')
-    
+    # first thing is first; if there are HA keys then try to dynamically build
+    # the property which is used to get the actual value of the uri
+    # (ie dfs.namenode.http-address.c1ha.nn2)
+    if alert_uri_lookup_keys.ha_nameservice is not None:
+      alert_uri = self._get_uri_from_ha_structure(alert_uri_lookup_keys)
+      if alert_uri is not None:
+        return alert_uri
+
     # attempt to parse and parameterize the various URIs; properties that
     # do not exist int he lookup map are returned as None
     if alert_uri_lookup_keys.http is not None:
@@ -269,12 +305,6 @@ class BaseAlert(object):
     if alert_uri_lookup_keys.https is not None:
       https_uri = self._get_configuration_value(alert_uri_lookup_keys.https)
 
-    if alert_uri_lookup_keys.https_property is not None:
-      https_property = self._get_configuration_value(alert_uri_lookup_keys.https_property)
-
-    if alert_uri_lookup_keys.https_property_value is not None:
-      https_property_value = self._get_configuration_value(alert_uri_lookup_keys.https_property_value)
-
     # without a URI, there's no way to create the structure we need - return
     # the default port if specified, otherwise throw an exception
     if http_uri is None and https_uri is None:
@@ -289,11 +319,11 @@ class BaseAlert(object):
     is_ssl_enabled = False
     
     if https_uri is not None:
-      # https without http implies SSL
+      # https without http implies SSL, otherwise look it up based on the properties
       if http_uri is None:
         is_ssl_enabled = True
         uri = https_uri
-      elif https_property is not None and https_property == https_property_value:
+      elif self._check_uri_ssl_property(alert_uri_lookup_keys):
         is_ssl_enabled = True
         uri = https_uri
     
@@ -301,6 +331,95 @@ class BaseAlert(object):
     return alert_uri
 
 
+  def _get_uri_from_ha_structure(self, alert_uri_lookup_keys):
+    """
+    Attempts to parse the HA URI structure in order to build a dynamic key
+    that represents the correct host URI to check.
+    :param alert_uri_lookup_keys:
+    :return: the AlertUri named tuple if there is a valid HA URL, otherwise None
+    """
+    if alert_uri_lookup_keys is None or alert_uri_lookup_keys.ha_nameservice is None:
+      return None
+
+    logger.debug("[Alert][{0}] HA URI structure detected in definition, attempting to lookup dynamic HA properties".format(self.get_name()))
+
+    ha_nameservice = self._get_configuration_value(alert_uri_lookup_keys.ha_nameservice)
+    ha_alias_key = alert_uri_lookup_keys.ha_alias_key
+    ha_http_pattern = alert_uri_lookup_keys.ha_http_pattern
+    ha_https_pattern = alert_uri_lookup_keys.ha_https_pattern
+
+    # at least one of these keys is needed
+    if ha_nameservice is None and ha_alias_key is None:
+      return None
+
+    # convert dfs.ha.namenodes.{{ha-nameservice}} into
+    # dfs.ha.namenodes.c1ha
+    if ha_nameservice is not None:
+      ha_alias_key = ha_alias_key.replace(self.HA_NAMESERVICE_PARAM, ha_nameservice)
+
+    # grab the alias value which should be like nn1, nn2
+    ha_nameservice_alias = self._get_configuration_value(ha_alias_key)
+    if ha_nameservice_alias is None:
+      logger.warning("[Alert][{0}] HA nameservice value is present but there are no aliases for {1}".format(
+        self.get_name(), ha_alias_key))
+
+      return None
+
+    # determine which pattern to use (http or https)
+    ha_pattern = ha_http_pattern
+    is_ssl_enabled = self._check_uri_ssl_property(alert_uri_lookup_keys)
+    if is_ssl_enabled:
+      ha_pattern = ha_https_pattern
+
+    # no pattern found
+    if ha_pattern is None:
+      logger.warning("[Alert][{0}] There is no matching http(s) pattern for the HA URI".format(
+        self.get_name()))
+
+      return None
+
+    # convert dfs.namenode.http-address.{{ha-nameservice}}.{{alias}} into
+    # dfs.namenode.http-address.c1ha.{{alias}}
+    ha_pattern = ha_pattern.replace(self.HA_NAMESERVICE_PARAM, ha_nameservice)
+
+    # for each alias, grab it and check to see if this host matches
+    for alias in ha_nameservice_alias.split(','):
+      # convert dfs.namenode.http-address.c1ha.{{alias}} into
+      # dfs.namenode.http-address.c1ha.nn1
+      key = ha_pattern.replace(self.HA_ALIAS_PARAM, alias.strip())
+
+      # get the host for dfs.namenode.http-address.c1ha.nn1 and see if it's
+      # this host
+      value = self._get_configuration_value(key)
+      if value is not None and self.host_name in value:
+        return AlertUri(uri=value, is_ssl_enabled=is_ssl_enabled)
+
+    return None
+
+
+  def _check_uri_ssl_property(self, alert_uri_lookup_keys):
+    """
+    Gets whether the SSL property and value on the URI indicate an SSL
+    connection.
+    :param alert_uri_lookup_keys:
+    :return:  True if the SSL check property and value are defined and match
+              otherwise False
+    """
+    https_property = None
+    https_property_value = None
+
+    if alert_uri_lookup_keys.https_property is not None:
+      https_property = self._get_configuration_value(alert_uri_lookup_keys.https_property)
+
+    if alert_uri_lookup_keys.https_property_value is not None:
+      https_property_value = self._get_configuration_value(alert_uri_lookup_keys.https_property_value)
+
+    if https_property is None:
+      return False
+
+    return https_property == https_property_value
+
+
   def _collect(self):
     """
     Low level function to collect alert data.  The result is a tuple as:

http://git-wip-us.apache.org/repos/asf/ambari/blob/7607fc1e/ambari-agent/src/test/python/ambari_agent/TestAlerts.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/test/python/ambari_agent/TestAlerts.py b/ambari-agent/src/test/python/ambari_agent/TestAlerts.py
index 2ee47c5..e9e106d 100644
--- a/ambari-agent/src/test/python/ambari_agent/TestAlerts.py
+++ b/ambari-agent/src/test/python/ambari_agent/TestAlerts.py
@@ -24,6 +24,7 @@ import sys
 
 from ambari_agent.AlertSchedulerHandler import AlertSchedulerHandler
 from ambari_agent.alerts.collector import AlertCollector
+from ambari_agent.alerts.base_alert import BaseAlert
 from ambari_agent.alerts.metric_alert import MetricAlert
 from ambari_agent.alerts.port_alert import PortAlert
 from ambari_agent.alerts.script_alert import ScriptAlert
@@ -893,6 +894,112 @@ class TestAlerts(TestCase):
     self.assertEquals('bar is rendered-bar2, baz is rendered-baz2', alerts[0]['text'])
 
 
+  def test_uri_structure_parsing(self):
+    uri_structure = {
+      "http": "{{hdfs-site/dfs.namenode.http.address}}",
+      "https": "{{hdfs-site/dfs.namenode.https.address}}",
+      "https_property": "{{hdfs-site/dfs.http.policy}}",
+      "https_property_value": "HTTPS_ONLY",
+      "high_availability": {
+        "nameservice": "{{hdfs-site/dfs.nameservices}}",
+        "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+        "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+        "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+      }
+    }
+
+    configuration = {'hdfs-site' :
+      { 'dfs.namenode.http.address' : '1.2.3.4:80',
+        'dfs.namenode.https.address' : '1.2.3.4:443' }
+    }
+
+    collector = AlertCollector()
+    cluster_configuration = self.__get_cluster_configuration()
+    self.__update_cluster_configuration(cluster_configuration, configuration)
+
+    alert = MockAlert()
+    alert.set_helpers(collector, cluster_configuration)
+    alert.set_cluster("c1", "c6401.ambari.apache.org")
+    uri_keys = alert._lookup_uri_property_keys(uri_structure)
+    self.assertFalse(alert._check_uri_ssl_property(uri_keys))
+
+    uri = alert._get_uri_from_structure(uri_keys)
+    self.assertEqual( '1.2.3.4:80', uri.uri )
+    self.assertEqual( False, uri.is_ssl_enabled )
+
+    configuration = {'hdfs-site' :
+      { 'dfs.http.policy' : 'HTTP_ONLY',
+        'dfs.namenode.http.address' : '1.2.3.4:80',
+        'dfs.namenode.https.address' : '1.2.3.4:443' }
+    }
+
+    self.__update_cluster_configuration(cluster_configuration, configuration)
+    uri_keys = alert._lookup_uri_property_keys(uri_structure)
+    self.assertFalse(alert._check_uri_ssl_property(uri_keys))
+
+    uri = alert._get_uri_from_structure(uri_keys)
+    self.assertEqual( '1.2.3.4:80', uri.uri )
+    self.assertEqual( False, uri.is_ssl_enabled )
+
+    # switch to SSL
+    configuration = {'hdfs-site' :
+      { 'dfs.http.policy' : 'HTTPS_ONLY',
+        'dfs.namenode.http.address' : '1.2.3.4:80',
+        'dfs.namenode.https.address' : '1.2.3.4:443' }
+    }
+
+    self.__update_cluster_configuration(cluster_configuration, configuration)
+    uri_keys = alert._lookup_uri_property_keys(uri_structure)
+    self.assertTrue(alert._check_uri_ssl_property(uri_keys))
+
+    uri = alert._get_uri_from_structure(uri_keys)
+    self.assertEqual( '1.2.3.4:443', uri.uri )
+    self.assertEqual( True, uri.is_ssl_enabled )
+
+    # test HA
+    configuration = {'hdfs-site' :
+      { 'dfs.http.policy' : 'HTTP_ONLY',
+        'dfs.namenode.http.address' : '1.2.3.4:80',
+        'dfs.namenode.https.address' : '1.2.3.4:443',
+        'dfs.nameservices' : 'c1ha',
+        'dfs.ha.namenodes.c1ha' : 'nn1, nn2',
+        'dfs.namenode.http-address.c1ha.nn1' : 'c6401.ambari.apache.org:8080',
+        'dfs.namenode.http-address.c1ha.nn2' : 'c6402.ambari.apache.org:8080',
+      }
+    }
+
+    self.__update_cluster_configuration(cluster_configuration, configuration)
+    uri_keys = alert._lookup_uri_property_keys(uri_structure)
+    self.assertFalse(alert._check_uri_ssl_property(uri_keys))
+
+    uri = alert._get_uri_from_structure(uri_keys)
+    self.assertEqual( 'c6401.ambari.apache.org:8080', uri.uri )
+    self.assertEqual( False, uri.is_ssl_enabled )
+
+    # test HA SSL
+    configuration = {'hdfs-site' :
+      { 'dfs.http.policy' : 'HTTPS_ONLY',
+        'dfs.namenode.http.address' : '1.2.3.4:80',
+        'dfs.namenode.https.address' : '1.2.3.4:443',
+        'dfs.nameservices' : 'c1ha',
+        'dfs.ha.namenodes.c1ha' : 'nn1, nn2',
+        'dfs.namenode.http-address.c1ha.nn1' : 'c6401.ambari.apache.org:8080',
+        'dfs.namenode.http-address.c1ha.nn2' : 'c6402.ambari.apache.org:8080',
+        'dfs.namenode.https-address.c1ha.nn1' : 'c6401.ambari.apache.org:8443',
+        'dfs.namenode.https-address.c1ha.nn2' : 'c6402.ambari.apache.org:8443',
+      }
+    }
+
+    self.__update_cluster_configuration(cluster_configuration, configuration)
+    uri_keys = alert._lookup_uri_property_keys(uri_structure)
+    self.assertTrue(alert._check_uri_ssl_property(uri_keys))
+
+    uri = alert._get_uri_from_structure(uri_keys)
+    self.assertEqual( 'c6401.ambari.apache.org:8443', uri.uri )
+    self.assertEqual( True, uri.is_ssl_enabled )
+
+
+
   def __get_cluster_configuration(self):
     """
     Gets an instance of the cluster cache where the file read and write
@@ -922,3 +1029,14 @@ class TestAlerts(TestCase):
       return file_mock
     else:
       return self.original_open(file, mode)
+
+
+class MockAlert(BaseAlert):
+  """
+  Mock class for testing
+  """
+  def __init__(self):
+    super(MockAlert, self).__init__(None, None)
+
+  def get_name(self):
+    return "mock_alert"

http://git-wip-us.apache.org/repos/asf/ambari/blob/7607fc1e/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertUri.java
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertUri.java b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertUri.java
index fe592a8..d4fb416 100644
--- a/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertUri.java
+++ b/ambari-server/src/main/java/org/apache/ambari/server/state/alert/AlertUri.java
@@ -75,6 +75,15 @@ public class AlertUri {
   private int m_port = 0;
 
   /**
+   * If present, then the component supports HA mode and the properties
+   * contained within need to be checked to see if an HA URI is required to be
+   * constructed instead of using {@link #m_httpProperty} and
+   * {@link #m_httpsProperty}.
+   */
+  @SerializedName("high_availability")
+  private HighAvailability m_highAvailability;
+
+  /**
    * Gets the plaintext (HTTP) URI that can be used to retrieve alert
    * information.
    *
@@ -145,6 +154,97 @@ public class AlertUri {
   }
 
   /**
+   * Gets the HA structure to use when determining if the component is in HA
+   * mode and requires the URL to be built dynamically.
+   *
+   * @return the HA structure or {@code null} if the component does not support
+   *         HA mode.
+   */
+  public HighAvailability getHighAvailability() {
+    return m_highAvailability;
+  }
+
+  /**
+   * The {@link HighAvailability} structure is used to hold information about
+   * how HA URIs are constructed if the service supports HA mode. For example
+   *
+   * <pre>
+   * high_availability": {
+   *   "nameservice": "{{hdfs-site/dfs.nameservices}}",
+   *   "alias_key" : "dfs.ha.namenodes.{{ha-nameservice}}",
+   *   "http_pattern" : "dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}",
+   *   "https_pattern" : "dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}"
+   * }
+   * </pre>
+   *
+   * Where the nameservice is {@code c1ha} and the alias key is
+   * {@code dfs.ha.namenodes.c1ha}. In this case the http pattern is defined as
+   * dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}
+   */
+  public class HighAvailability {
+    /**
+     * The key that represents the name service. The alert will use the
+     * existance of this key as the marker for HA mode.
+     */
+    @SerializedName("nameservice")
+    private String m_nameservice;
+
+    /**
+     * The key that will be used to retrieve the aliases for each host.
+     */
+    @SerializedName("alias_key")
+    private String m_aliasKey;
+
+    /**
+     * The parameterized pattern for determining the HTTP URL to use.
+     */
+    @SerializedName("http_pattern")
+    private String m_httpPattern;
+
+    /**
+     * The parameterized pattern for determining the HTTPS URL to use.
+     */
+    @SerializedName("https_pattern")
+    private String m_httpsPattern;
+
+    /**
+     * Gets the nameservice name.
+     *
+     * @return the nameservice
+     */
+    public String getNameservice() {
+      return m_nameservice;
+    }
+
+    /**
+     * Gets the parameterized key to use when retrieving the host aliases.
+     *
+     * @return the alias key
+     */
+    public String getAliasKey() {
+      return m_aliasKey;
+    }
+
+    /**
+     * Get the parameterized HTTP pattern to use.
+     *
+     * @return the httpPattern
+     */
+    public String getHttpPattern() {
+      return m_httpPattern;
+    }
+
+    /**
+     * Get the parameterized HTTPS pattern to use.
+     *
+     * @return the httpsPattern
+     */
+    public String getHttpsPattern() {
+      return m_httpsPattern;
+    }
+  }
+
+  /**
    * {@inheritDoc}
    */
   @Override

http://git-wip-us.apache.org/repos/asf/ambari/blob/7607fc1e/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/alerts.json b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/alerts.json
index 2910f8b..8f55cde 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/alerts.json
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/alerts.json
@@ -93,7 +93,13 @@
             "https_property": "{{hdfs-site/dfs.http.policy}}",
             "https_property_value": "HTTPS_ONLY",
             "kerberos_keytab": "{{hdfs-site/dfs.web.authentication.kerberos.keytab}}",
-            "kerberos_principal": "{{hdfs-site/dfs.web.authentication.kerberos.principal}}"
+            "kerberos_principal": "{{hdfs-site/dfs.web.authentication.kerberos.principal}}",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -121,7 +127,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -159,7 +171,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -197,7 +215,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -235,7 +259,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -273,7 +303,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -310,7 +346,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -337,32 +379,6 @@
         }
       },
       {
-        "name": "namenode_process",
-        "label": "NameNode Process",
-        "description": "This host-level alert is triggered if the NameNode process cannot be confirmed to be up and listening on the network.",
-        "interval": 1,
-        "scope": "ANY",
-        "enabled": true,
-        "source": {
-          "type": "PORT",
-          "uri": "{{hdfs-site/dfs.namenode.http-address}}",
-          "default_port": 50070,
-          "reporting": {
-            "ok": {
-              "text": "TCP OK - {0:.3f}s response on port {1}"
-            },
-            "warning": {
-              "text": "TCP OK - {0:.3f}s response on port {1}",
-              "value": 1.5
-            },
-            "critical": {
-              "text": "Connection failed: {0} to {1}:{2}",
-              "value": 5.0
-            }
-          }
-        }
-      },
-      {
         "name": "namenode_last_checkpoint",
         "label": "NameNode Last Checkpoint",
         "description": "This service-level alert will trigger if the last time that the NameNode performed a checkpoint was too long ago. It will also trigger if the number of uncommitted transactions is beyond a certain threshold.",

http://git-wip-us.apache.org/repos/asf/ambari/blob/7607fc1e/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/alerts.json b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/alerts.json
index d51f0d8..95464b7 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/alerts.json
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/alerts.json
@@ -91,7 +91,15 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "kerberos_keytab": "{{hdfs-site/dfs.web.authentication.kerberos.keytab}}",
+            "kerberos_principal": "{{hdfs-site/dfs.web.authentication.kerberos.principal}}",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -101,7 +109,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}"
+              "text": "Connection failed to {1}: {3}"
             }
           }
         }
@@ -119,7 +127,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -157,7 +171,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -166,7 +186,7 @@
             "warning": {
               "text": "Total Blocks:[{1}], Missing Blocks:[{0}]",
               "value": 1
-            },          
+            },
             "critical": {
               "text": "Total Blocks:[{1}], Missing Blocks:[{0}]",
               "value": 1
@@ -195,7 +215,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -204,7 +230,7 @@
             "warning": {
               "text": "Capacity Used:[{2:d}%, {0}], Capacity Remaining:[{1}]",
               "value": 80
-            },          
+            },
             "critical": {
               "text": "Capacity Used:[{2:d}%, {0}], Capacity Remaining:[{1}]",
               "value": 90
@@ -233,7 +259,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -242,7 +274,7 @@
             "warning": {
               "text": "Average Queue Time:[{0}], Average Processing Time:[{1}]",
               "value": 3000
-            },          
+            },
             "critical": {
               "text": "Average Queue Time:[{0}], Average Processing Time:[{1}]",
               "value": 5000
@@ -271,7 +303,13 @@
             "http": "{{hdfs-site/dfs.namenode.http-address}}",
             "https": "{{hdfs-site/dfs.namenode.https-address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
           },
           "reporting": {
             "ok": {
@@ -280,7 +318,7 @@
             "warning": {
               "text": "Failed directory count: {1}",
               "value": 1
-            },          
+            },
             "critical": {
               "text": "Failed directory count: {1}",
               "value": 1
@@ -296,28 +334,47 @@
         }
       },
       {
-        "name": "namenode_process",
-        "label": "NameNode Process",
-        "description": "This host-level alert is triggered if the NameNode process cannot be confirmed to be up and listening on the network.",
+        "name": "datanode_health_summary",
+        "label": "DataNode Health Summary",
+        "description": "This service-level alert is triggered if there are unhealthy DataNodes",
         "interval": 1,
-        "scope": "ANY",
+        "scope": "SERVICE",
         "enabled": true,
         "source": {
-          "type": "PORT",
-          "uri": "{{hdfs-site/dfs.namenode.http-address}}",
-          "default_port": 50070,
+          "type": "METRIC",
+          "uri": {
+            "http": "{{hdfs-site/dfs.namenode.http-address}}",
+            "https": "{{hdfs-site/dfs.namenode.https-address}}",
+            "https_property": "{{hdfs-site/dfs.http.policy}}",
+            "https_property_value": "HTTPS_ONLY",
+            "high_availability": {
+              "nameservice": "{{hdfs-site/dfs.nameservices}}",
+              "alias_key" : "{{hdfs-site/dfs.ha.namenodes.{{ha-nameservice}}}}",
+              "http_pattern" : "{{hdfs-site/dfs.namenode.http-address.{{ha-nameservice}}.{{alias}}}}",
+              "https_pattern" : "{{hdfs-site/dfs.namenode.https-address.{{ha-nameservice}}.{{alias}}}}"
+            }
+          },
           "reporting": {
             "ok": {
-              "text": "TCP OK - {0:.3f}s response on port {1}"
+              "text": "All {2} DataNode(s) are healthy"
             },
             "warning": {
-              "text": "TCP OK - {0:.3f}s response on port {1}",
-              "value": 1.5
+              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "value": 1
             },
             "critical": {
-              "text": "Connection failed: {0} to {1}:{2}",
-              "value": 5.0
-            }
+              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "value": 1
+            },
+            "units" : "DataNode(s)"
+          },
+          "jmx": {
+            "property_list": [
+              "Hadoop:service=NameNode,name=FSNamesystemState/NumDeadDataNodes",
+              "Hadoop:service=NameNode,name=FSNamesystemState/NumStaleDataNodes",
+              "Hadoop:service=NameNode,name=FSNamesystemState/NumLiveDataNodes"
+            ],
+            "value": "{0} + {1}"
           }
         }
       },
@@ -384,7 +441,7 @@
         "scope": "HOST",
         "enabled": true,
         "source": {
-          "type": "PORT",        
+          "type": "PORT",
           "uri": "{{hdfs-site/dfs.journalnode.http-address}}",
           "default_port": 8480,
           "reporting": {
@@ -402,7 +459,7 @@
           }
         }
       }
-    ],      
+    ],
     "DATANODE": [
       {
         "name": "datanode_process",
@@ -412,7 +469,7 @@
         "scope": "HOST",
         "enabled": true,
         "source": {
-          "type": "PORT",        
+          "type": "PORT",
           "uri": "{{hdfs-site/dfs.datanode.address}}",
           "default_port": 50010,
           "reporting": {
@@ -443,7 +500,9 @@
             "http": "{{hdfs-site/dfs.datanode.http.address}}",
             "https": "{{hdfs-site/dfs.datanode.https.address}}",
             "https_property": "{{hdfs-site/dfs.http.policy}}",
-            "https_property_value": "HTTPS_ONLY"
+            "https_property_value": "HTTPS_ONLY",
+            "kerberos_keytab": "{{hdfs-site/dfs.web.authentication.kerberos.keytab}}",
+            "kerberos_principal": "{{hdfs-site/dfs.web.authentication.kerberos.principal}}"
           },
           "reporting": {
             "ok": {
@@ -453,11 +512,11 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}"
+              "text": "Connection failed to {1}: {3}"
             }
           }
         }
-      },    
+      },
       {
         "name": "datanode_storage",
         "label": "DataNode Storage",
@@ -495,7 +554,7 @@
             "value": "({1} - {0})/{1} * 100"
           }
         }
-      }    
+      }
     ],
     "ZKFC": [
       {
@@ -506,7 +565,7 @@
         "scope": "ANY",
         "enabled": true,
         "source": {
-          "type": "PORT",        
+          "type": "PORT",
           "uri": "{{core-site/ha.zookeeper.quorum}}",
           "default_port": 2181,
           "reporting": {