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/11 19:42:31 UTC

ambari git commit: AMBARI-10021 - Python Does Not Close Alert TCP Connections Reliably

Repository: ambari
Updated Branches:
  refs/heads/trunk f81da87c0 -> 516d718fc


AMBARI-10021 - Python Does Not Close Alert TCP Connections Reliably


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

Branch: refs/heads/trunk
Commit: 516d718fc96625a146a9e276c65a8fd9990a5976
Parents: f81da87
Author: Jonathan Hurley <jh...@hortonworks.com>
Authored: Wed Mar 11 09:47:08 2015 -0400
Committer: Jonathan Hurley <jh...@hortonworks.com>
Committed: Wed Mar 11 14:15:03 2015 -0400

----------------------------------------------------------------------
 .../python/ambari_agent/alerts/metric_alert.py  |  17 ++-
 .../python/ambari_agent/alerts/web_alert.py     | 116 ++++++++++++-------
 .../AMBARI_METRICS/0.1.0/alerts.json            |   3 -
 .../FALCON/0.5.0.2.1/alerts.json                |   2 +-
 .../HBASE/0.96.0.2.0/alerts.json                |   6 -
 .../common-services/HDFS/2.1.0.2.0/alerts.json  |   8 +-
 .../package/alerts/alert_checkpoint_time.py     |  20 +++-
 .../package/alerts/alert_ha_namenode_health.py  |  20 +++-
 .../package/alerts/alert_hive_metastore.py      |   3 -
 .../package/alerts/alert_hive_thrift_port.py    |   3 -
 .../package/alerts/alert_webhcat_server.py      |   9 +-
 .../common-services/OOZIE/4.0.0.2.0/alerts.json |   2 +-
 .../common-services/STORM/0.9.1.2.1/alerts.json |   2 +-
 .../common-services/YARN/2.1.0.2.0/alerts.json  |   8 +-
 .../package/alerts/alert_nodemanager_health.py  |  25 +++-
 .../alerts/alert_nodemanagers_summary.py        |  27 +++--
 .../BIGTOP/0.8/services/HBASE/alerts.json       |   3 -
 .../stacks/BIGTOP/0.8/services/HDFS/alerts.json |   8 +-
 .../HDFS/package/files/alert_checkpoint_time.py |  20 +++-
 .../package/files/alert_ha_namenode_health.py   |  20 +++-
 .../package/files/alert_hive_thrift_port.py     |   5 +-
 .../package/files/alert_webhcat_server.py       |  11 +-
 .../stacks/BIGTOP/0.8/services/YARN/alerts.json |   8 +-
 .../package/files/alert_nodemanager_health.py   |  30 ++++-
 24 files changed, 250 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py b/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
index 83dc54d..8b5f15d 100644
--- a/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
+++ b/ambari-agent/src/main/python/ambari_agent/alerts/metric_alert.py
@@ -154,10 +154,19 @@ class MetricAlert(BaseAlert):
 
       # use a customer header processor that will look for the non-standard
       # "Refresh" header and attempt to follow the redirect
-      url_opener = urllib2.build_opener(RefreshHeaderProcessor())
-      response = url_opener.open(url)
-
-      content = response.read()
+      response = None
+      try:
+        url_opener = urllib2.build_opener(RefreshHeaderProcessor())
+        response = url_opener.open(url)
+        content = response.read()
+      finally:
+        # explicitely close the connection as we've seen python hold onto these
+        if response is not None:
+          try:
+            response.close()
+          except:
+            logger.debug("[Alert][{0}] Unable to close JMX URL connection to {1}".format
+              (self.get_name(), url))
 
       json_response = json.loads(content)
       json_data = json_response['beans'][0]

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-agent/src/main/python/ambari_agent/alerts/web_alert.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/alerts/web_alert.py b/ambari-agent/src/main/python/ambari_agent/alerts/web_alert.py
index 8252781..2db4996 100644
--- a/ambari-agent/src/main/python/ambari_agent/alerts/web_alert.py
+++ b/ambari-agent/src/main/python/ambari_agent/alerts/web_alert.py
@@ -22,7 +22,9 @@ import logging
 import time
 import subprocess
 import os
+import urllib2
 import uuid
+
 from  tempfile import gettempdir
 from alerts.base_alert import BaseAlert
 from collections import namedtuple
@@ -45,10 +47,13 @@ except ImportError:
 
 logger = logging.getLogger()
 
-CURL_CONNECTION_TIMEOUT = '20'
+CONNECTION_TIMEOUT = 10.0
+CURL_CONNECTION_TIMEOUT = "10"
+
+WebResponse = namedtuple('WebResponse', 'status_code time_millis error_msg')
 
 class WebAlert(BaseAlert):
-  
+
   def __init__(self, alert_meta, alert_source_meta, config):
     super(WebAlert, self).__init__(alert_meta, alert_source_meta)
     
@@ -72,6 +77,10 @@ class WebAlert(BaseAlert):
       self.get_name(), alert_uri.uri, str(alert_uri.is_ssl_enabled)))
 
     url = self._build_web_query(alert_uri)
+
+    # substitute 0.0.0.0 in url with actual fqdn
+    url = url.replace('0.0.0.0', self.host_name)
+
     web_response = self._make_web_request(url)
     status_code = web_response.status_code
     time_seconds = web_response.time_millis / 1000
@@ -79,10 +88,10 @@ class WebAlert(BaseAlert):
 
     if status_code == 0:
       return (self.RESULT_CRITICAL, [status_code, url, time_seconds, error_message])
-    
+
     if status_code < 400:
       return (self.RESULT_OK, [status_code, url, time_seconds])
-    
+
     return (self.RESULT_WARNING, [status_code, url, time_seconds])
 
 
@@ -116,9 +125,11 @@ class WebAlert(BaseAlert):
     scheme = 'http'
     if alert_uri.is_ssl_enabled is True:
       scheme = 'https'
+
     if OSCheck.is_windows_family():
       # on windows 0.0.0.0 is invalid address to connect but on linux it resolved to 127.0.0.1
       host = resolve_address(host)
+
     return "{0}://{1}:{2}".format(scheme, host, str(port))
 
 
@@ -127,10 +138,7 @@ class WebAlert(BaseAlert):
     Makes an http(s) request to a web resource and returns the http code. If
     there was an error making the request, return 0 for the status code.
     """    
-    WebResponse = namedtuple('WebResponse', 'status_code time_millis error_msg')
-    
-    time_millis = 0
-    
+
     try:
       kerberos_keytab = None
       kerberos_principal = None
@@ -164,49 +172,79 @@ class WebAlert(BaseAlert):
 
         if os.system("{0} -s {1}".format(klist_path_local, ccache_file_path)) != 0:
           kinit_path_local = get_kinit_path()
-          logger.debug("[Alert][{0}] Enabling Kerberos authentication via GSSAPI using ccache at {1}."
-                       .format(self.get_name(), ccache_file_path))
-          os.system("{0} -l 5m -c {1} -kt {2} {3} > /dev/null".format(kinit_path_local, ccache_file_path, kerberos_keytab, kerberos_principal))
+          logger.debug("[Alert][{0}] Enabling Kerberos authentication via GSSAPI using ccache at {1}.".format(
+            self.get_name(), ccache_file_path))
+
+          os.system("{0} -l 5m -c {1} -kt {2} {3} > /dev/null".format(
+            kinit_path_local, ccache_file_path, kerberos_keytab,
+            kerberos_principal))
         else:
-          logger.debug("[Alert][{0}] Kerberos authentication via GSSAPI already enabled using ccache at {1}."
-                       .format(self.get_name(), ccache_file_path))
-      else:
-        kerberos_env = None
+          logger.debug("[Alert][{0}] Kerberos authentication via GSSAPI already enabled using ccache at {1}.".format(
+            self.get_name(), ccache_file_path))
 
-      # check if cookies dir exists, if not then create it
-      tmp_dir = self.config.get('agent', 'tmp_dir')
-      cookies_dir = os.path.join(tmp_dir, "cookies")
+        # check if cookies dir exists, if not then create it
+        tmp_dir = self.config.get('agent', 'tmp_dir')
+        cookies_dir = os.path.join(tmp_dir, "cookies")
 
-      if not os.path.exists(cookies_dir):
-        os.makedirs(cookies_dir)
+        if not os.path.exists(cookies_dir):
+          os.makedirs(cookies_dir)
 
-      # substitute 0.0.0.0 in url with actual fqdn
-      url = url.replace('0.0.0.0', self.host_name)
-      cookie_file_name = str(uuid.uuid4())
-      cookie_file = os.path.join(cookies_dir, cookie_file_name)
-      start_time = time.time()
-      curl = subprocess.Popen(['curl', '--negotiate', '-u', ':', '-b', cookie_file, '-c', cookie_file, '-sL', '-w',
-        '%{http_code}', url, '--connect-timeout', CURL_CONNECTION_TIMEOUT,
-        '-o', '/dev/null'], stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=kerberos_env)
+        cookie_file_name = str(uuid.uuid4())
+        cookie_file = os.path.join(cookies_dir, cookie_file_name)
 
-      out, err = curl.communicate()
+        start_time = time.time()
 
-      if err != '':
-        raise Exception(err)
+        try:
+          curl = subprocess.Popen(['curl', '--negotiate', '-u', ':', '-b', cookie_file, '-c', cookie_file, '-sL', '-w',
+            '%{http_code}', url, '--connect-timeout', CURL_CONNECTION_TIMEOUT,
+            '-o', '/dev/null'], stdout=subprocess.PIPE, stderr=subprocess.PIPE, env=kerberos_env)
 
-      response_code = int(out)
-      time_millis = time.time() - start_time
-    except Exception, exc:
+          out, err = curl.communicate()
+        finally:
+          if os.path.isfile(cookie_file):
+            os.remove(cookie_file)
+
+        if err != '':
+          raise Exception(err)
+
+        response_code = int(out)
+        time_millis = time.time() - start_time
+      else:
+        # kerberos is not involved; use urllib2
+        response_code, time_millis = self._make_web_request_urllib(url)
+
+      return WebResponse(status_code=response_code, time_millis=time_millis,
+        error_msg=None)
+
+    except Exception, exception:
       if logger.isEnabledFor(logging.DEBUG):
         logger.exception("[Alert][{0}] Unable to make a web request.".format(self.get_name()))
 
-      return WebResponse(status_code=0, time_millis=0, error_msg=str(exc))
+      return WebResponse(status_code=0, time_millis=0, error_msg=str(exception))
 
-    finally:
-      if os.path.isfile(cookie_file):
-        os.remove(cookie_file)
 
-    return WebResponse(status_code=response_code, time_millis=time_millis, error_msg=None)
+  def _make_web_request_urllib(self, url):
+    """
+    Make a web request using urllib2. This function does not handle exceptions.
+    :param url: the URL to request
+    :return: a tuple of the response code and the total time in ms
+    """
+    response = None
+    start_time = time.time()
+
+    try:
+      response = urllib2.urlopen(url, timeout=CONNECTION_TIMEOUT)
+      response_code = response.getcode()
+      time_millis = time.time() - start_time
+
+      return response_code, time_millis
+    finally:
+      if response is not None:
+        try:
+          response.close()
+        except Exception, exception:
+          if logger.isEnabledFor(logging.DEBUG):
+            logger.exception("[Alert][{0}] Unable to close socket connection".format(self.get_name()))
 
 
   def _get_reporting_text(self, state):

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/alerts.json b/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/alerts.json
index 9b29b0b..2f543ef 100644
--- a/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/alerts.json
+++ b/ambari-server/src/main/resources/common-services/AMBARI_METRICS/0.1.0/alerts.json
@@ -90,9 +90,6 @@
           "type": "METRIC",
           "uri": {
             "http": "{{ams-hbase-site/hbase.master.info.port}}",
-            "https": "{{ams-hbase-site/hbase.master.info.port}}",
-            "https_property": "{{cluster-env/security_enabled}}",
-            "https_property_value": "true",
             "default_port": 61310
           },
           "reporting": {

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/alerts.json b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/alerts.json
index 7001122..e028ed5 100644
--- a/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/alerts.json
+++ b/ambari-server/src/main/resources/common-services/FALCON/0.5.0.2.1/alerts.json
@@ -50,7 +50,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/alerts.json b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/alerts.json
index 724d43a..d10897a 100644
--- a/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/alerts.json
+++ b/ambari-server/src/main/resources/common-services/HBASE/0.96.0.2.0/alerts.json
@@ -64,9 +64,6 @@
           "type": "METRIC",
           "uri": {
             "http": "{{hbase-site/hbase.master.info.port}}",
-            "https": "{{hbase-site/hbase.master.info.port}}",
-            "https_property": "{{cluster-env/security_enabled}}",
-            "https_property_value": "true",
             "default_port": 60010
           },
           "reporting": {
@@ -103,9 +100,6 @@
           "type": "METRIC",
           "uri": {
             "http": "{{hbase-site/hbase.master.info.port}}",
-            "https": "{{hbase-site/hbase.master.info.port}}",
-            "https_property": "{{cluster-env/security_enabled}}",
-            "https_property_value": "true",
             "default_port": 60010
           },
           "reporting": {

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/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 da8ebba..d5dccbf 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
@@ -109,7 +109,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -359,11 +359,11 @@
               "text": "All {2} DataNode(s) are healthy"
             },
             "warning": {
-              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "text": "DataNode Health: [Live={2}, Stale={1}, Dead={0}]",
               "value": 1
             },
             "critical": {
-              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "text": "DataNode Health: [Live={2}, Stale={1}, Dead={0}]",
               "value": 1
             },
             "units" : "DNs"
@@ -512,7 +512,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_checkpoint_time.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_checkpoint_time.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_checkpoint_time.py
index 410608f..032310d 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_checkpoint_time.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_checkpoint_time.py
@@ -129,8 +129,18 @@ def get_time(delta):
   return {'h':h, 'm':m}
 
 
-def get_value_from_jmx(qry, property):
-  response = urllib2.urlopen(qry)
-  data=response.read()
-  data_dict = json.loads(data)
-  return data_dict["beans"][0][property]
+def get_value_from_jmx(query, jmx_property):
+  response = None
+  
+  try:
+    response = urllib2.urlopen(query)
+    data = response.read()
+
+    data_dict = json.loads(data)
+    return data_dict["beans"][0][jmx_property]
+  finally:
+    if response is not None:
+      try:
+        response.close()
+      except:
+        pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_ha_namenode_health.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_ha_namenode_health.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_ha_namenode_health.py
index fc1541d..058b7b2 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_ha_namenode_health.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/alerts/alert_ha_namenode_health.py
@@ -159,8 +159,18 @@ def execute(parameters=None, host_name=None):
       return (RESULT_STATE_SKIPPED, ['Another host will report this alert'])
 
 
-def get_value_from_jmx(qry, property):
-  response = urllib2.urlopen(qry)
-  data=response.read()
-  data_dict = json.loads(data)
-  return data_dict["beans"][0][property]
+def get_value_from_jmx(query, jmx_property):
+  response = None
+  
+  try:
+    response = urllib2.urlopen(query)
+    data = response.read()
+
+    data_dict = json.loads(data)
+    return data_dict["beans"][0][jmx_property]
+  finally:
+    if response is not None:
+      try:
+        response.close()
+      except:
+        pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py
index 20d8abe..120c4a0 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_metastore.py
@@ -18,11 +18,8 @@ See the License for the specific language governing permissions and
 limitations under the License.
 """
 
-import json
 import socket
 import time
-import traceback
-import urllib2
 from resource_management.libraries.functions import format
 from resource_management.libraries.functions import get_kinit_path
 from resource_management.core.resources import Execute

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py
index 5e9ab1f..c496717 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_hive_thrift_port.py
@@ -18,11 +18,8 @@ See the License for the specific language governing permissions and
 limitations under the License.
 """
 
-import json
 import socket
 import time
-import traceback
-import urllib2
 from resource_management.libraries.functions import hive_check
 from resource_management.libraries.functions import format
 from resource_management.libraries.functions import get_kinit_path

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py
index 0b7535c..fb6c4c2 100644
--- a/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py
+++ b/ambari-server/src/main/resources/common-services/HIVE/0.12.0.2.0/package/alerts/alert_webhcat_server.py
@@ -96,7 +96,6 @@ def execute(parameters=None, host_name=None):
   if security_enabled:
     if WEBHCAT_KEYTAB_KEY not in parameters or WEBHCAT_PRINCIPAL_KEY not in parameters:
       return (RESULT_CODE_UNKNOWN, [str(parameters)])
-      # return (RESULT_CODE_UNKNOWN, ['The WebHCat keytab and principal are required parameters when security is enabled.'])
 
     try:
       webhcat_keytab = parameters[WEBHCAT_KEYTAB_KEY]
@@ -164,6 +163,8 @@ def execute(parameters=None, host_name=None):
     except Exception, exception:
       return (RESULT_CODE_CRITICAL, [str(exception)])
   else:
+    url_response = None
+    
     try:
       # execute the query for the JSON that includes WebHCat status
       start_time = time.time()
@@ -177,6 +178,12 @@ def execute(parameters=None, host_name=None):
     except:
       label = CRITICAL_CONNECTION_MESSAGE.format(query_url)
       return (RESULT_CODE_CRITICAL, [label])
+    finally:
+      if url_response is not None:
+        try:
+          url_response.close()
+        except:
+          pass
 
 
   # if status is not in the response, we can't do any check; return CRIT

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/alerts.json b/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/alerts.json
index a29d793..01d9148 100644
--- a/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/alerts.json
+++ b/ambari-server/src/main/resources/common-services/OOZIE/4.0.0.2.0/alerts.json
@@ -23,7 +23,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/alerts.json b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/alerts.json
index 8084e1f..002fd07 100644
--- a/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/alerts.json
+++ b/ambari-server/src/main/resources/common-services/STORM/0.9.1.2.1/alerts.json
@@ -73,7 +73,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/alerts.json b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/alerts.json
index 0555d76..fe1ae8a 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/alerts.json
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/alerts.json
@@ -26,7 +26,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -189,7 +189,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -237,7 +237,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -366,7 +366,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanager_health.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanager_health.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanager_health.py
index 12b1bd7..8c72f4c 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanager_health.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanager_health.py
@@ -23,6 +23,7 @@ import socket
 import urllib2
 from ambari_commons import OSCheck
 from ambari_commons.inet_utils import resolve_address
+
 RESULT_CODE_OK = 'OK'
 RESULT_CODE_CRITICAL = 'CRITICAL'
 RESULT_CODE_UNKNOWN = 'UNKNOWN'
@@ -32,7 +33,8 @@ NODEMANAGER_HTTPS_ADDRESS_KEY = '{{yarn-site/yarn.nodemanager.webapp.https.addre
 YARN_HTTP_POLICY_KEY = '{{yarn-site/yarn.http.policy}}'
 
 OK_MESSAGE = 'NodeManager Healthy'
-CRITICAL_CONNECTION_MESSAGE = 'Connection failed to {0}'
+CRITICAL_CONNECTION_MESSAGE = 'Connection failed to {0} ({1})'
+CRITICAL_HTTP_STATUS_MESSAGE = 'HTTP {0} returned from {1} ({2})'
 CRITICAL_NODEMANAGER_STATUS_MESSAGE = 'NodeManager returned an unexpected status of "{0}"'
 CRITICAL_NODEMANAGER_UNKNOWN_JSON_MESSAGE = 'Unable to determine NodeManager health from unexpected JSON response'
 
@@ -93,18 +95,25 @@ def execute(parameters=None, host_name=None):
       host_name = socket.getfqdn()
 
     uri = '{0}:{1}'.format(host_name, NODEMANAGER_DEFAULT_PORT)
+    
   if OSCheck.is_windows_family():
     uri_host, uri_port = uri.split(':')
     # on windows 0.0.0.0 is invalid address to connect but on linux it resolved to 127.0.0.1
     uri_host = resolve_address(uri_host)
     uri = '{0}:{1}'.format(uri_host, uri_port)
+
+  query = "{0}://{1}/ws/v1/node/info".format(scheme,uri)
+
   try:
-    query = "{0}://{1}/ws/v1/node/info".format(scheme,uri)
-    
     # execute the query for the JSON that includes templeton status
     url_response = urllib2.urlopen(query)
-  except:
-    label = CRITICAL_CONNECTION_MESSAGE.format(uri)
+  except urllib2.HTTPError, httpError:
+    label = CRITICAL_HTTP_STATUS_MESSAGE.format(str(httpError.code), query,
+      str(httpError))
+
+    return (RESULT_CODE_CRITICAL, [label])
+  except Exception, exception:
+    label = CRITICAL_CONNECTION_MESSAGE.format(query, str(exception))
     return (RESULT_CODE_CRITICAL, [label])
 
   # URL response received, parse it
@@ -116,6 +125,12 @@ def execute(parameters=None, host_name=None):
     node_healthy = str(node_healthy)
   except:
     return (RESULT_CODE_CRITICAL, [query])
+  finally:
+    if url_response is not None:
+      try:
+        url_response.close()
+      except:
+        pass
 
   # proper JSON received, compare against known value
   if node_healthy.lower() == 'true':

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanagers_summary.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanagers_summary.py b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanagers_summary.py
index 5f7abbd..b297b0c 100644
--- a/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanagers_summary.py
+++ b/ambari-server/src/main/resources/common-services/YARN/2.1.0.2.0/package/alerts/alert_nodemanagers_summary.py
@@ -102,12 +102,21 @@ def execute(parameters=None, host_name=None):
   return ((result_code, [label]))
 
 
-def get_value_from_jmx(url, property):
-  # use a customer header process that will look for the non-standard
-  # "Refresh" header and attempt to follow the redirect
-  url_opener = urllib2.build_opener(RefreshHeaderProcessor())
-  response = url_opener.open(url)
-
-  data=response.read()
-  data_dict = json.loads(data)
-  return data_dict["beans"][0][property]
+def get_value_from_jmx(query, jmx_property):
+  response = None
+  
+  try:
+    # use a customer header process that will look for the non-standard
+    # "Refresh" header and attempt to follow the redirect
+    url_opener = urllib2.build_opener(RefreshHeaderProcessor())
+    response = url_opener.open(query)
+
+    data = response.read()
+    data_dict = json.loads(data)
+    return data_dict["beans"][0][jmx_property]
+  finally:
+    if response is not None:
+      try:
+        response.close()
+      except:
+        pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HBASE/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HBASE/alerts.json b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HBASE/alerts.json
index d3e55b3..07a8cff 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HBASE/alerts.json
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HBASE/alerts.json
@@ -64,9 +64,6 @@
           "type": "METRIC",
           "uri": {
             "http": "{{hbase-site/hbase.master.info.port}}",
-            "https": "{{hbase-site/hbase.master.info.port}}",
-            "https_property": "{{cluster-env/security_enabled}}",
-            "https_property_value": "true",
             "default_port": 60010
           },
           "reporting": {

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/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 a268ad9..ac5ab37 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
@@ -109,7 +109,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -359,11 +359,11 @@
               "text": "All {2} DataNode(s) are healthy"
             },
             "warning": {
-              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "text": "DataNode Health: [Live={2}, Stale={1}, Dead={0}]",
               "value": 1
             },
             "critical": {
-              "text": "Dead/Stale Datanode(s): {0}/{1} out of {2}",
+              "text": "DataNode Health: [Live={2}, Stale={1}, Dead={0}]",
               "value": 1
             },
             "units" : "DNs"
@@ -512,7 +512,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_checkpoint_time.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_checkpoint_time.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_checkpoint_time.py
index 410608f..032310d 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_checkpoint_time.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_checkpoint_time.py
@@ -129,8 +129,18 @@ def get_time(delta):
   return {'h':h, 'm':m}
 
 
-def get_value_from_jmx(qry, property):
-  response = urllib2.urlopen(qry)
-  data=response.read()
-  data_dict = json.loads(data)
-  return data_dict["beans"][0][property]
+def get_value_from_jmx(query, jmx_property):
+  response = None
+  
+  try:
+    response = urllib2.urlopen(query)
+    data = response.read()
+
+    data_dict = json.loads(data)
+    return data_dict["beans"][0][jmx_property]
+  finally:
+    if response is not None:
+      try:
+        response.close()
+      except:
+        pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_ha_namenode_health.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_ha_namenode_health.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_ha_namenode_health.py
index fc1541d..058b7b2 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_ha_namenode_health.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/files/alert_ha_namenode_health.py
@@ -159,8 +159,18 @@ def execute(parameters=None, host_name=None):
       return (RESULT_STATE_SKIPPED, ['Another host will report this alert'])
 
 
-def get_value_from_jmx(qry, property):
-  response = urllib2.urlopen(qry)
-  data=response.read()
-  data_dict = json.loads(data)
-  return data_dict["beans"][0][property]
+def get_value_from_jmx(query, jmx_property):
+  response = None
+  
+  try:
+    response = urllib2.urlopen(query)
+    data = response.read()
+
+    data_dict = json.loads(data)
+    return data_dict["beans"][0][jmx_property]
+  finally:
+    if response is not None:
+      try:
+        response.close()
+      except:
+        pass

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HIVE/package/files/alert_hive_thrift_port.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HIVE/package/files/alert_hive_thrift_port.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HIVE/package/files/alert_hive_thrift_port.py
index bd3f276..2e302f2 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HIVE/package/files/alert_hive_thrift_port.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HIVE/package/files/alert_hive_thrift_port.py
@@ -18,12 +18,9 @@ See the License for the specific language governing permissions and
 limitations under the License.
 """
 
-import json
 import socket
 import time
-import traceback
-import urllib2
-from resource_management.libraries.functions import hive_check 
+from resource_management.libraries.functions import hive_check
 
 OK_MESSAGE = "TCP OK - %.4f response on port %s"
 CRITICAL_MESSAGE = "Connection failed on host {0}:{1}"

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/WEBHCAT/package/files/alert_webhcat_server.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/WEBHCAT/package/files/alert_webhcat_server.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/WEBHCAT/package/files/alert_webhcat_server.py
index c484f0b..fb6c4c2 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/WEBHCAT/package/files/alert_webhcat_server.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/WEBHCAT/package/files/alert_webhcat_server.py
@@ -96,7 +96,6 @@ def execute(parameters=None, host_name=None):
   if security_enabled:
     if WEBHCAT_KEYTAB_KEY not in parameters or WEBHCAT_PRINCIPAL_KEY not in parameters:
       return (RESULT_CODE_UNKNOWN, [str(parameters)])
-      # return (RESULT_CODE_UNKNOWN, ['The WebHCat keytab and principal are required parameters when security is enabled.'])
 
     try:
       webhcat_keytab = parameters[WEBHCAT_KEYTAB_KEY]
@@ -164,6 +163,8 @@ def execute(parameters=None, host_name=None):
     except Exception, exception:
       return (RESULT_CODE_CRITICAL, [str(exception)])
   else:
+    url_response = None
+    
     try:
       # execute the query for the JSON that includes WebHCat status
       start_time = time.time()
@@ -177,6 +178,12 @@ def execute(parameters=None, host_name=None):
     except:
       label = CRITICAL_CONNECTION_MESSAGE.format(query_url)
       return (RESULT_CODE_CRITICAL, [label])
+    finally:
+      if url_response is not None:
+        try:
+          url_response.close()
+        except:
+          pass
 
 
   # if status is not in the response, we can't do any check; return CRIT
@@ -199,4 +206,4 @@ def execute(parameters=None, host_name=None):
     result_code = RESULT_CODE_CRITICAL
     label = CRITICAL_WEBHCAT_STATUS_MESSAGE.format(webhcat_status)
 
-  return (result_code, [label])
\ No newline at end of file
+  return (result_code, [label])

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/alerts.json
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/alerts.json b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/alerts.json
index 886b46d..aeabc93 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/alerts.json
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/alerts.json
@@ -26,7 +26,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -189,7 +189,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -237,7 +237,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }
@@ -354,7 +354,7 @@
               "text": "HTTP {0} response in {2:.3f} seconds"
             },
             "critical": {
-              "text": "Connection failed to {1}: {3}"
+              "text": "Connection failed to {1} ({3})"
             }
           }
         }

http://git-wip-us.apache.org/repos/asf/ambari/blob/516d718f/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/package/files/alert_nodemanager_health.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/package/files/alert_nodemanager_health.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/package/files/alert_nodemanager_health.py
index b1de951..8c72f4c 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/package/files/alert_nodemanager_health.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/YARN/package/files/alert_nodemanager_health.py
@@ -21,6 +21,8 @@ limitations under the License.
 import json
 import socket
 import urllib2
+from ambari_commons import OSCheck
+from ambari_commons.inet_utils import resolve_address
 
 RESULT_CODE_OK = 'OK'
 RESULT_CODE_CRITICAL = 'CRITICAL'
@@ -31,7 +33,8 @@ NODEMANAGER_HTTPS_ADDRESS_KEY = '{{yarn-site/yarn.nodemanager.webapp.https.addre
 YARN_HTTP_POLICY_KEY = '{{yarn-site/yarn.http.policy}}'
 
 OK_MESSAGE = 'NodeManager Healthy'
-CRITICAL_CONNECTION_MESSAGE = 'Connection failed to {0}'
+CRITICAL_CONNECTION_MESSAGE = 'Connection failed to {0} ({1})'
+CRITICAL_HTTP_STATUS_MESSAGE = 'HTTP {0} returned from {1} ({2})'
 CRITICAL_NODEMANAGER_STATUS_MESSAGE = 'NodeManager returned an unexpected status of "{0}"'
 CRITICAL_NODEMANAGER_UNKNOWN_JSON_MESSAGE = 'Unable to determine NodeManager health from unexpected JSON response'
 
@@ -92,14 +95,25 @@ def execute(parameters=None, host_name=None):
       host_name = socket.getfqdn()
 
     uri = '{0}:{1}'.format(host_name, NODEMANAGER_DEFAULT_PORT)
+    
+  if OSCheck.is_windows_family():
+    uri_host, uri_port = uri.split(':')
+    # on windows 0.0.0.0 is invalid address to connect but on linux it resolved to 127.0.0.1
+    uri_host = resolve_address(uri_host)
+    uri = '{0}:{1}'.format(uri_host, uri_port)
+
+  query = "{0}://{1}/ws/v1/node/info".format(scheme,uri)
 
   try:
-    query = "{0}://{1}/ws/v1/node/info".format(scheme,uri)
-    
     # execute the query for the JSON that includes templeton status
     url_response = urllib2.urlopen(query)
-  except:
-    label = CRITICAL_CONNECTION_MESSAGE.format(uri)
+  except urllib2.HTTPError, httpError:
+    label = CRITICAL_HTTP_STATUS_MESSAGE.format(str(httpError.code), query,
+      str(httpError))
+
+    return (RESULT_CODE_CRITICAL, [label])
+  except Exception, exception:
+    label = CRITICAL_CONNECTION_MESSAGE.format(query, str(exception))
     return (RESULT_CODE_CRITICAL, [label])
 
   # URL response received, parse it
@@ -111,6 +125,12 @@ def execute(parameters=None, host_name=None):
     node_healthy = str(node_healthy)
   except:
     return (RESULT_CODE_CRITICAL, [query])
+  finally:
+    if url_response is not None:
+      try:
+        url_response.close()
+      except:
+        pass
 
   # proper JSON received, compare against known value
   if node_healthy.lower() == 'true':