You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ambari.apache.org by dm...@apache.org on 2016/12/14 17:13:30 UTC

[1/2] ambari git commit: AMBARI-17666. Ambari agent can't start when TLSv1 is disabled in Java security (dlysnichenko)

Repository: ambari
Updated Branches:
  refs/heads/branch-2.5 e1f7795ba -> ecc916bbc
  refs/heads/trunk 30511fad8 -> db9dfd60a


AMBARI-17666. Ambari agent can't start when TLSv1 is disabled in Java security (dlysnichenko)


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

Branch: refs/heads/trunk
Commit: db9dfd60a1930c3ee18322179a1ebe3d795263e8
Parents: 30511fa
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Dec 14 19:10:56 2016 +0200
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Dec 14 19:10:56 2016 +0200

----------------------------------------------------------------------
 .../main/python/ambari_agent/AmbariConfig.py    | 31 +++++++++++++-------
 .../ambari_agent/CustomServiceOrchestrator.py   |  4 ++-
 .../src/main/python/ambari_agent/NetUtil.py     |  3 ++
 .../python/ambari_agent/alerts/web_alert.py     |  7 +++--
 .../TestCustomServiceOrchestrator.py            |  4 +--
 .../main/python/ambari_commons/inet_utils.py    |  7 +++--
 .../libraries/script/script.py                  | 15 ++++++++--
 .../HDFS/2.1.0.2.0/package/files/checkWebUI.py  | 15 ++++++----
 .../HDFS/2.1.0.2.0/package/scripts/params.py    |  1 +
 .../2.1.0.2.0/package/scripts/service_check.py  |  3 +-
 .../HDFS/2.1.0.2.0/package/scripts/utils.py     |  5 ++--
 .../0.8/services/HDFS/package/scripts/params.py |  1 +
 .../HDFS/package/scripts/service_check.py       |  2 +-
 13 files changed, 66 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py b/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
index a097f37..64c2643 100644
--- a/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
+++ b/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
@@ -23,7 +23,6 @@ import ConfigParser
 import StringIO
 import hostname
 import ambari_simplejson as json
-from NetUtil import NetUtil
 import os
 
 from ambari_commons import OSConst
@@ -159,7 +158,6 @@ class AmbariConfig:
   def __init__(self):
     global content
     self.config = ConfigParser.RawConfigParser()
-    self.net = NetUtil(self)
     self.config.readfp(StringIO.StringIO(content))
 
   def get(self, section, value, default=None):
@@ -185,13 +183,22 @@ class AmbariConfig:
   def getConfig(self):
     return self.config
 
-  @staticmethod
-  @OsFamilyFuncImpl(OSConst.WINSRV_FAMILY)
-  def getConfigFile(home_dir=""):
-    if 'AMBARI_AGENT_CONF_DIR' in os.environ:
-      return os.path.join(os.environ['AMBARI_AGENT_CONF_DIR'], "ambari-agent.ini")
-    else:
-      return "ambari-agent.ini"
+  @classmethod
+  def get_resolved_config(cls, home_dir=""):
+    if hasattr(cls, "_conf_cache"):
+      return getattr(cls, "_conf_cache")
+    config = cls()
+    configPath = os.path.abspath(cls.getConfigFile(home_dir))
+    try:
+      if os.path.exists(configPath):
+        config.read(configPath)
+      else:
+        raise Exception("No config found at {0}, use default".format(configPath))
+
+    except Exception, err:
+      logger.warn(err)
+    setattr(cls, "_conf_cache", config)
+    return config
 
   @staticmethod
   @OsFamilyFuncImpl(OsFamilyImpl.DEFAULT)
@@ -259,7 +266,8 @@ class AmbariConfig:
     self.config.read(filename)
 
   def getServerOption(self, url, name, default=None):
-    status, response = self.net.checkURL(url)
+    from ambari_agent.NetUtil import NetUtil
+    status, response = NetUtil(self).checkURL(url)
     if status is True:
       try:
         data = json.loads(response)
@@ -305,6 +313,9 @@ class AmbariConfig:
         logger.info("Updating config property (%s) with value (%s)", k, v)
     pass
 
+  def get_force_https_protocol(self):
+    return self.get('security', 'force_https_protocol', default="PROTOCOL_TLSv1")
+
 def isSameHostList(hostlist1, hostlist2):
   is_same = True
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
index 5fd3068..02f4212 100644
--- a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
+++ b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
@@ -78,6 +78,7 @@ class CustomServiceOrchestrator():
   def __init__(self, config, controller):
     self.config = config
     self.tmp_dir = config.get('agent', 'prefix')
+    self.force_https_protocol = config.get_force_https_protocol()
     self.exec_tmp_dir = Constants.AGENT_TMP_DIR
     self.file_cache = FileCache(config)
     self.status_commands_stdout = os.path.join(self.tmp_dir,
@@ -385,7 +386,8 @@ class CustomServiceOrchestrator():
       
       for py_file, current_base_dir in filtered_py_file_list:
         log_info_on_failure = not command_name in self.DONT_DEBUG_FAILURES_FOR_COMMANDS
-        script_params = [command_name, json_path, current_base_dir, tmpstrucoutfile, logger_level, self.exec_tmp_dir]
+        script_params = [command_name, json_path, current_base_dir, tmpstrucoutfile, logger_level, self.exec_tmp_dir,
+                         self.force_https_protocol]
         
         if log_out_files:
           script_params.append("-o")

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-agent/src/main/python/ambari_agent/NetUtil.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/NetUtil.py b/ambari-agent/src/main/python/ambari_agent/NetUtil.py
index c3cd621..9b29633 100644
--- a/ambari-agent/src/main/python/ambari_agent/NetUtil.py
+++ b/ambari-agent/src/main/python/ambari_agent/NetUtil.py
@@ -20,6 +20,8 @@ import httplib
 import sys
 from ssl import SSLError
 from HeartbeatHandlers import HeartbeatStopHandlers
+from ambari_agent.AmbariConfig import AmbariConfig
+from ambari_commons.inet_utils import ensure_ssl_using_protocol
 
 ERROR_SSL_WRONG_VERSION = "SSLError: Failed to connect. Please check openssl library versions. \n" +\
               "Refer to: https://bugzilla.redhat.com/show_bug.cgi?id=1022468 for more details."
@@ -27,6 +29,7 @@ LOG_REQUEST_MESSAGE = "GET %s -> %s, body: %s"
 
 logger = logging.getLogger(__name__)
 
+ensure_ssl_using_protocol(AmbariConfig.get_resolved_config().get_force_https_protocol())
 
 class NetUtil:
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/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 6caf1d0..ef144bb 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
@@ -33,8 +33,9 @@ from resource_management.libraries.functions.get_port_from_url import get_port_f
 from resource_management.libraries.functions.get_path_from_url import get_path_from_url
 from resource_management.libraries.functions.curl_krb_request import curl_krb_request
 from ambari_commons import OSCheck
-from ambari_commons.inet_utils import resolve_address, ensure_ssl_using_tls_v1
+from ambari_commons.inet_utils import resolve_address, ensure_ssl_using_protocol
 from ambari_agent import Constants
+from ambari_agent.AmbariConfig import AmbariConfig
 
 # hashlib is supplied as of Python 2.5 as the replacement interface for md5
 # and other secure hashes.  In 2.6, md5 is deprecated.  Import hashlib if
@@ -54,7 +55,7 @@ DEFAULT_CONNECTION_TIMEOUT = 5
 
 WebResponse = namedtuple('WebResponse', 'status_code time_millis error_msg')
 
-ensure_ssl_using_tls_v1()
+ensure_ssl_using_protocol(AmbariConfig.get_resolved_config().get_force_https_protocol())
 
 class WebAlert(BaseAlert):
 
@@ -250,4 +251,4 @@ class WebAlert(BaseAlert):
     if state == self.RESULT_CRITICAL:
       return 'Connection failed to {1}'
 
-    return 'HTTP {0} response in {2:.4f} seconds'
\ No newline at end of file
+    return 'HTTP {0} response in {2:.4f} seconds'

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py b/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
index 563d250..3985c5a 100644
--- a/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
+++ b/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
@@ -54,8 +54,8 @@ class TestCustomServiceOrchestrator(TestCase):
     # generate sample config
     tmpdir = tempfile.gettempdir()
     exec_tmp_dir = os.path.join(tmpdir, 'tmp')
-    self.config = ConfigParser.RawConfigParser()
-    self.config.get = AmbariConfig().get
+    self.config = AmbariConfig()
+    self.config.config = ConfigParser.RawConfigParser()
     self.config.add_section('agent')
     self.config.set('agent', 'prefix', tmpdir)
     self.config.set('agent', 'cache_dir', "/cachedir")

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-common/src/main/python/ambari_commons/inet_utils.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/ambari_commons/inet_utils.py b/ambari-common/src/main/python/ambari_commons/inet_utils.py
index b5cea75..d44107d 100644
--- a/ambari-common/src/main/python/ambari_commons/inet_utils.py
+++ b/ambari-common/src/main/python/ambari_commons/inet_utils.py
@@ -183,10 +183,11 @@ def resolve_address(address):
       return '127.0.0.1'
   return address
 
-def ensure_ssl_using_tls_v1():
+def ensure_ssl_using_protocol(protocol):
   """
   Monkey patching ssl module to force it use tls_v1. Do this in common module to avoid problems with
   PythonReflectiveExecutor.
+  :param protocol: one of ("PROTOCOL_SSLv2", "PROTOCOL_SSLv3", "PROTOCOL_SSLv23", "PROTOCOL_TLSv1", "PROTOCOL_TLSv1_1", "PROTOCOL_TLSv1_2")
   :return:
   """
   from functools import wraps
@@ -197,8 +198,8 @@ def ensure_ssl_using_tls_v1():
     @wraps(func)
     def bar(*args, **kw):
       import ssl
-      kw['ssl_version'] = ssl.PROTOCOL_TLSv1
+      kw['ssl_version'] = getattr(ssl, protocol)
       return func(*args, **kw)
     bar._ambari_patched = True
     return bar
-  ssl.wrap_socket = sslwrap(ssl.wrap_socket)
\ No newline at end of file
+  ssl.wrap_socket = sslwrap(ssl.wrap_socket)

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-common/src/main/python/resource_management/libraries/script/script.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/script/script.py b/ambari-common/src/main/python/resource_management/libraries/script/script.py
index 584775e..6a4865d 100644
--- a/ambari-common/src/main/python/resource_management/libraries/script/script.py
+++ b/ambari-common/src/main/python/resource_management/libraries/script/script.py
@@ -67,7 +67,7 @@ if OSCheck.is_windows_family():
 else:
   from resource_management.libraries.functions.tar_archive import archive_dir
 
-USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEVEL> <TMP_DIR>
+USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEVEL> <TMP_DIR> [PROTOCOL]
 
 <COMMAND> command type (INSTALL/CONFIGURE/START/STOP/SERVICE_CHECK...)
 <JSON_CONFIG> path to command json file. Ex: /var/lib/ambari-agent/data/command-2.json
@@ -75,6 +75,7 @@ USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEV
 <STROUTPUT> path to file with structured command output (file will be created). Ex:/tmp/my.txt
 <LOGGING_LEVEL> log level for stdout. Ex:DEBUG,INFO
 <TMP_DIR> temporary directory for executable scripts. Ex: /var/lib/ambari-agent/tmp
+[PROTOCOL] optional protocol to use during https connections. Ex: see python ssl.PROTOCOL_<PROTO> variables, default PROTOCOL_TLSv1
 """
 
 _PASSWORD_MAP = {"/configurations/cluster-env/hadoop.user.name":"/configurations/cluster-env/hadoop.user.password"}
@@ -121,7 +122,8 @@ class Script(object):
 
   # Class variable
   tmp_dir = ""
- 
+  force_https_protocol = "PROTOCOL_TLSv1"
+
   def load_structured_out(self):
     Script.structuredOut = {}
     if os.path.exists(self.stroutfile):
@@ -247,7 +249,10 @@ class Script(object):
     self.load_structured_out()
     self.logging_level = sys.argv[5]
     Script.tmp_dir = sys.argv[6]
-    
+    # optional script argument for forcing https protocol
+    if len(sys.argv) >= 8:
+      Script.force_https_protocol = sys.argv[7]
+
     logging_level_str = logging._levelNames[self.logging_level]
     Logger.initialize_logger(__name__, logging_level=logging_level_str)
 
@@ -433,6 +438,10 @@ class Script(object):
     return Script.tmp_dir
 
   @staticmethod
+  def get_force_https_protocol():
+    return Script.force_https_protocol
+
+  @staticmethod
   def get_component_from_role(role_directory_map, default_role):
     """
     Gets the <stack-root>/current/<component> component given an Ambari role,

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
index ddeb116..6e4b028 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
@@ -23,22 +23,23 @@ import httplib
 import socket
 import ssl
 
-class TLS1HTTPSConnection(httplib.HTTPSConnection):
+class ForcedProtocolHTTPSConnection(httplib.HTTPSConnection):
   """
   Some of python implementations does not work correctly with sslv3 but trying to use it, we need to change protocol to
   tls1.
   """
-  def __init__(self, host, port, **kwargs):
+  def __init__(self, host, port, force_protocol, **kwargs):
     httplib.HTTPSConnection.__init__(self, host, port, **kwargs)
+    self.force_protocol = force_protocol
 
   def connect(self):
     sock = socket.create_connection((self.host, self.port), self.timeout)
     if getattr(self, '_tunnel_host', None):
       self.sock = sock
       self._tunnel()
-    self.sock = ssl.wrap_socket(sock, self.key_file, self.cert_file, ssl_version=ssl.PROTOCOL_TLSv1)
+    self.sock = ssl.wrap_socket(sock, self.key_file, self.cert_file, ssl_version=getattr(ssl, self.force_protocol))
 
-def make_connection(host, port, https):
+def make_connection(host, port, https, force_protocol=None):
   try:
     conn = httplib.HTTPConnection(host, port) if not https else httplib.HTTPSConnection(host, port)
     conn.request("GET", "/")
@@ -46,7 +47,7 @@ def make_connection(host, port, https):
   except ssl.SSLError:
     # got ssl error, lets try to use TLS1 protocol, maybe it will work
     try:
-      tls1_conn = TLS1HTTPSConnection(host, port)
+      tls1_conn = ForcedProtocolHTTPSConnection(host, port, force_protocol)
       tls1_conn.request("GET", "/")
       return tls1_conn.getresponse().status
     except Exception as e:
@@ -65,15 +66,17 @@ def main():
   parser.add_option("-m", "--hosts", dest="hosts", help="Comma separated hosts list for WEB UI to check it availability")
   parser.add_option("-p", "--port", dest="port", help="Port of WEB UI to check it availability")
   parser.add_option("-s", "--https", dest="https", help="\"True\" if value of dfs.http.policy is \"HTTPS_ONLY\"")
+  parser.add_option("-o", "--protocol", dest="protocol", help="Protocol to use when executing https request")
 
   (options, args) = parser.parse_args()
   
   hosts = options.hosts.split(',')
   port = options.port
   https = options.https
+  protocol = options.protocol
 
   for host in hosts:
-    httpCode = make_connection(host, port, https.lower() == "true")
+    httpCode = make_connection(host, port, https.lower() == "true", protocol)
 
     if httpCode != 200:
       print "Cannot access WEB UI on: http://" + host + ":" + port if not https.lower() == "true" else "Cannot access WEB UI on: https://" + host + ":" + port

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
index 25231f9..7f282b3 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
@@ -26,3 +26,4 @@ else:
 
 nfsgateway_heapsize = config['configurations']['hadoop-env']['nfsgateway_heapsize']
 retryAble = default("/commandParams/command_retry_enabled", False)
+script_https_protocol = Script.get_force_https_protocol()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
index 981f002..47fc646 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
@@ -92,7 +92,8 @@ class HdfsServiceCheckDefault(HdfsServiceCheck):
         checkWebUIFileName = "checkWebUI.py"
         checkWebUIFilePath = format("{tmp_dir}/{checkWebUIFileName}")
         comma_sep_jn_hosts = ",".join(params.journalnode_hosts)
-        checkWebUICmd = format("ambari-python-wrap {checkWebUIFilePath} -m {comma_sep_jn_hosts} -p {journalnode_port} -s {https_only}")
+
+        checkWebUICmd = format("ambari-python-wrap {checkWebUIFilePath} -m {comma_sep_jn_hosts} -p {journalnode_port} -s {https_only} -o {script_https_protocol}")
         File(checkWebUIFilePath,
              content=StaticFile(checkWebUIFileName),
              mode=0775)

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
index 4577ad2..3270430 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
@@ -32,12 +32,13 @@ from resource_management.core.exceptions import ComponentIsNotRunning
 from resource_management.core.logger import Logger
 from resource_management.libraries.functions.curl_krb_request import curl_krb_request
 from resource_management.core.exceptions import Fail
+from resource_management.libraries.script.script import Script
 from resource_management.libraries.functions.namenode_ha_utils import get_namenode_states
 from resource_management.libraries.functions.show_logs import show_logs
-from ambari_commons.inet_utils import ensure_ssl_using_tls_v1
+from ambari_commons.inet_utils import ensure_ssl_using_protocol
 from zkfc_slave import ZkfcSlaveDefault
 
-ensure_ssl_using_tls_v1()
+ensure_ssl_using_protocol(Script.get_force_https_protocol())
 
 def safe_zkfc_op(action, env):
   """

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
index 49cfa86..f85efb0 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
@@ -241,3 +241,4 @@ ttnode_heapsize = "1024m"
 dtnode_heapsize = config['configurations']['hadoop-env']['dtnode_heapsize']
 mapred_pid_dir_prefix = default("/configurations/mapred-env/mapred_pid_dir_prefix","/var/run/hadoop-mapreduce")
 mapred_log_dir_prefix = default("/configurations/mapred-env/mapred_log_dir_prefix","/var/log/hadoop-mapreduce")
+script_https_protocol = Script.get_force_https_protocol()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/db9dfd60/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
index 81d7ca5..3b54fd0 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
@@ -93,7 +93,7 @@ class HdfsServiceCheck(Script):
       comma_sep_jn_hosts = ",".join(params.journalnode_hosts)
       checkWebUICmd = format(
         "su -s /bin/bash - {smoke_test_user} -c 'python {checkWebUIFilePath} -m "
-        "{comma_sep_jn_hosts} -p {journalnode_port}'")
+        "{comma_sep_jn_hosts} -p {journalnode_port} -o {script_https_protocol}'")
       File(checkWebUIFilePath,
            content=StaticFile(checkWebUIFileName))
 


[2/2] ambari git commit: AMBARI-17666. Ambari agent can't start when TLSv1 is disabled in Java security (dlysnichenko)

Posted by dm...@apache.org.
AMBARI-17666. Ambari agent can't start when TLSv1 is disabled in Java security (dlysnichenko)


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

Branch: refs/heads/branch-2.5
Commit: ecc916bbc1345e0dbc80ee1bd3a42f8545bc52f2
Parents: e1f7795
Author: Lisnichenko Dmitro <dl...@hortonworks.com>
Authored: Wed Dec 14 19:11:52 2016 +0200
Committer: Lisnichenko Dmitro <dl...@hortonworks.com>
Committed: Wed Dec 14 19:11:52 2016 +0200

----------------------------------------------------------------------
 .../main/python/ambari_agent/AmbariConfig.py    | 31 +++++++++++++-------
 .../ambari_agent/CustomServiceOrchestrator.py   |  4 ++-
 .../src/main/python/ambari_agent/NetUtil.py     |  3 ++
 .../python/ambari_agent/alerts/web_alert.py     |  7 +++--
 .../TestCustomServiceOrchestrator.py            |  4 +--
 .../main/python/ambari_commons/inet_utils.py    |  7 +++--
 .../libraries/script/script.py                  | 15 ++++++++--
 .../HDFS/2.1.0.2.0/package/files/checkWebUI.py  | 15 ++++++----
 .../HDFS/2.1.0.2.0/package/scripts/params.py    |  1 +
 .../2.1.0.2.0/package/scripts/service_check.py  |  3 +-
 .../HDFS/2.1.0.2.0/package/scripts/utils.py     |  4 +--
 .../0.8/services/HDFS/package/scripts/params.py |  1 +
 .../HDFS/package/scripts/service_check.py       |  2 +-
 13 files changed, 65 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py b/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
index c3d6c39..26e54ec 100644
--- a/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
+++ b/ambari-agent/src/main/python/ambari_agent/AmbariConfig.py
@@ -23,7 +23,6 @@ import ConfigParser
 import StringIO
 import hostname
 import ambari_simplejson as json
-from NetUtil import NetUtil
 import os
 
 from ambari_commons import OSConst
@@ -156,7 +155,6 @@ class AmbariConfig:
   def __init__(self):
     global content
     self.config = ConfigParser.RawConfigParser()
-    self.net = NetUtil(self)
     self.config.readfp(StringIO.StringIO(content))
 
   def get(self, section, value, default=None):
@@ -182,13 +180,22 @@ class AmbariConfig:
   def getConfig(self):
     return self.config
 
-  @staticmethod
-  @OsFamilyFuncImpl(OSConst.WINSRV_FAMILY)
-  def getConfigFile(home_dir=""):
-    if 'AMBARI_AGENT_CONF_DIR' in os.environ:
-      return os.path.join(os.environ['AMBARI_AGENT_CONF_DIR'], "ambari-agent.ini")
-    else:
-      return "ambari-agent.ini"
+  @classmethod
+  def get_resolved_config(cls, home_dir=""):
+    if hasattr(cls, "_conf_cache"):
+      return getattr(cls, "_conf_cache")
+    config = cls()
+    configPath = os.path.abspath(cls.getConfigFile(home_dir))
+    try:
+      if os.path.exists(configPath):
+        config.read(configPath)
+      else:
+        raise Exception("No config found at {0}, use default".format(configPath))
+
+    except Exception, err:
+      logger.warn(err)
+    setattr(cls, "_conf_cache", config)
+    return config
 
   @staticmethod
   @OsFamilyFuncImpl(OsFamilyImpl.DEFAULT)
@@ -256,7 +263,8 @@ class AmbariConfig:
     self.config.read(filename)
 
   def getServerOption(self, url, name, default=None):
-    status, response = self.net.checkURL(url)
+    from ambari_agent.NetUtil import NetUtil
+    status, response = NetUtil(self).checkURL(url)
     if status is True:
       try:
         data = json.loads(response)
@@ -293,6 +301,9 @@ class AmbariConfig:
         logger.info("Updating config property (%s) with value (%s)", k, v)
     pass
 
+  def get_force_https_protocol(self):
+    return self.get('security', 'force_https_protocol', default="PROTOCOL_TLSv1")
+
 def isSameHostList(hostlist1, hostlist2):
   is_same = True
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
index 5fd3068..02f4212 100644
--- a/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
+++ b/ambari-agent/src/main/python/ambari_agent/CustomServiceOrchestrator.py
@@ -78,6 +78,7 @@ class CustomServiceOrchestrator():
   def __init__(self, config, controller):
     self.config = config
     self.tmp_dir = config.get('agent', 'prefix')
+    self.force_https_protocol = config.get_force_https_protocol()
     self.exec_tmp_dir = Constants.AGENT_TMP_DIR
     self.file_cache = FileCache(config)
     self.status_commands_stdout = os.path.join(self.tmp_dir,
@@ -385,7 +386,8 @@ class CustomServiceOrchestrator():
       
       for py_file, current_base_dir in filtered_py_file_list:
         log_info_on_failure = not command_name in self.DONT_DEBUG_FAILURES_FOR_COMMANDS
-        script_params = [command_name, json_path, current_base_dir, tmpstrucoutfile, logger_level, self.exec_tmp_dir]
+        script_params = [command_name, json_path, current_base_dir, tmpstrucoutfile, logger_level, self.exec_tmp_dir,
+                         self.force_https_protocol]
         
         if log_out_files:
           script_params.append("-o")

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-agent/src/main/python/ambari_agent/NetUtil.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/main/python/ambari_agent/NetUtil.py b/ambari-agent/src/main/python/ambari_agent/NetUtil.py
index c3cd621..9b29633 100644
--- a/ambari-agent/src/main/python/ambari_agent/NetUtil.py
+++ b/ambari-agent/src/main/python/ambari_agent/NetUtil.py
@@ -20,6 +20,8 @@ import httplib
 import sys
 from ssl import SSLError
 from HeartbeatHandlers import HeartbeatStopHandlers
+from ambari_agent.AmbariConfig import AmbariConfig
+from ambari_commons.inet_utils import ensure_ssl_using_protocol
 
 ERROR_SSL_WRONG_VERSION = "SSLError: Failed to connect. Please check openssl library versions. \n" +\
               "Refer to: https://bugzilla.redhat.com/show_bug.cgi?id=1022468 for more details."
@@ -27,6 +29,7 @@ LOG_REQUEST_MESSAGE = "GET %s -> %s, body: %s"
 
 logger = logging.getLogger(__name__)
 
+ensure_ssl_using_protocol(AmbariConfig.get_resolved_config().get_force_https_protocol())
 
 class NetUtil:
 

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/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 6caf1d0..ef144bb 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
@@ -33,8 +33,9 @@ from resource_management.libraries.functions.get_port_from_url import get_port_f
 from resource_management.libraries.functions.get_path_from_url import get_path_from_url
 from resource_management.libraries.functions.curl_krb_request import curl_krb_request
 from ambari_commons import OSCheck
-from ambari_commons.inet_utils import resolve_address, ensure_ssl_using_tls_v1
+from ambari_commons.inet_utils import resolve_address, ensure_ssl_using_protocol
 from ambari_agent import Constants
+from ambari_agent.AmbariConfig import AmbariConfig
 
 # hashlib is supplied as of Python 2.5 as the replacement interface for md5
 # and other secure hashes.  In 2.6, md5 is deprecated.  Import hashlib if
@@ -54,7 +55,7 @@ DEFAULT_CONNECTION_TIMEOUT = 5
 
 WebResponse = namedtuple('WebResponse', 'status_code time_millis error_msg')
 
-ensure_ssl_using_tls_v1()
+ensure_ssl_using_protocol(AmbariConfig.get_resolved_config().get_force_https_protocol())
 
 class WebAlert(BaseAlert):
 
@@ -250,4 +251,4 @@ class WebAlert(BaseAlert):
     if state == self.RESULT_CRITICAL:
       return 'Connection failed to {1}'
 
-    return 'HTTP {0} response in {2:.4f} seconds'
\ No newline at end of file
+    return 'HTTP {0} response in {2:.4f} seconds'

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py b/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
index 563d250..3985c5a 100644
--- a/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
+++ b/ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py
@@ -54,8 +54,8 @@ class TestCustomServiceOrchestrator(TestCase):
     # generate sample config
     tmpdir = tempfile.gettempdir()
     exec_tmp_dir = os.path.join(tmpdir, 'tmp')
-    self.config = ConfigParser.RawConfigParser()
-    self.config.get = AmbariConfig().get
+    self.config = AmbariConfig()
+    self.config.config = ConfigParser.RawConfigParser()
     self.config.add_section('agent')
     self.config.set('agent', 'prefix', tmpdir)
     self.config.set('agent', 'cache_dir', "/cachedir")

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-common/src/main/python/ambari_commons/inet_utils.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/ambari_commons/inet_utils.py b/ambari-common/src/main/python/ambari_commons/inet_utils.py
index b5cea75..d44107d 100644
--- a/ambari-common/src/main/python/ambari_commons/inet_utils.py
+++ b/ambari-common/src/main/python/ambari_commons/inet_utils.py
@@ -183,10 +183,11 @@ def resolve_address(address):
       return '127.0.0.1'
   return address
 
-def ensure_ssl_using_tls_v1():
+def ensure_ssl_using_protocol(protocol):
   """
   Monkey patching ssl module to force it use tls_v1. Do this in common module to avoid problems with
   PythonReflectiveExecutor.
+  :param protocol: one of ("PROTOCOL_SSLv2", "PROTOCOL_SSLv3", "PROTOCOL_SSLv23", "PROTOCOL_TLSv1", "PROTOCOL_TLSv1_1", "PROTOCOL_TLSv1_2")
   :return:
   """
   from functools import wraps
@@ -197,8 +198,8 @@ def ensure_ssl_using_tls_v1():
     @wraps(func)
     def bar(*args, **kw):
       import ssl
-      kw['ssl_version'] = ssl.PROTOCOL_TLSv1
+      kw['ssl_version'] = getattr(ssl, protocol)
       return func(*args, **kw)
     bar._ambari_patched = True
     return bar
-  ssl.wrap_socket = sslwrap(ssl.wrap_socket)
\ No newline at end of file
+  ssl.wrap_socket = sslwrap(ssl.wrap_socket)

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-common/src/main/python/resource_management/libraries/script/script.py
----------------------------------------------------------------------
diff --git a/ambari-common/src/main/python/resource_management/libraries/script/script.py b/ambari-common/src/main/python/resource_management/libraries/script/script.py
index 5375a53..50eaf4f 100644
--- a/ambari-common/src/main/python/resource_management/libraries/script/script.py
+++ b/ambari-common/src/main/python/resource_management/libraries/script/script.py
@@ -66,7 +66,7 @@ if OSCheck.is_windows_family():
 else:
   from resource_management.libraries.functions.tar_archive import archive_dir
 
-USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEVEL> <TMP_DIR>
+USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEVEL> <TMP_DIR> [PROTOCOL]
 
 <COMMAND> command type (INSTALL/CONFIGURE/START/STOP/SERVICE_CHECK...)
 <JSON_CONFIG> path to command json file. Ex: /var/lib/ambari-agent/data/command-2.json
@@ -74,6 +74,7 @@ USAGE = """Usage: {0} <COMMAND> <JSON_CONFIG> <BASEDIR> <STROUTPUT> <LOGGING_LEV
 <STROUTPUT> path to file with structured command output (file will be created). Ex:/tmp/my.txt
 <LOGGING_LEVEL> log level for stdout. Ex:DEBUG,INFO
 <TMP_DIR> temporary directory for executable scripts. Ex: /var/lib/ambari-agent/tmp
+[PROTOCOL] optional protocol to use during https connections. Ex: see python ssl.PROTOCOL_<PROTO> variables, default PROTOCOL_TLSv1
 """
 
 _PASSWORD_MAP = {"/configurations/cluster-env/hadoop.user.name":"/configurations/cluster-env/hadoop.user.password"}
@@ -120,7 +121,8 @@ class Script(object):
 
   # Class variable
   tmp_dir = ""
- 
+  force_https_protocol = "PROTOCOL_TLSv1"
+
   def load_structured_out(self):
     Script.structuredOut = {}
     if os.path.exists(self.stroutfile):
@@ -246,7 +248,10 @@ class Script(object):
     self.load_structured_out()
     self.logging_level = sys.argv[5]
     Script.tmp_dir = sys.argv[6]
-    
+    # optional script argument for forcing https protocol
+    if len(sys.argv) >= 8:
+      Script.force_https_protocol = sys.argv[7]
+
     logging_level_str = logging._levelNames[self.logging_level]
     Logger.initialize_logger(__name__, logging_level=logging_level_str)
 
@@ -408,6 +413,10 @@ class Script(object):
     return Script.tmp_dir
 
   @staticmethod
+  def get_force_https_protocol():
+    return Script.force_https_protocol
+
+  @staticmethod
   def get_component_from_role(role_directory_map, default_role):
     """
     Gets the <stack-root>/current/<component> component given an Ambari role,

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
index ddeb116..6e4b028 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/files/checkWebUI.py
@@ -23,22 +23,23 @@ import httplib
 import socket
 import ssl
 
-class TLS1HTTPSConnection(httplib.HTTPSConnection):
+class ForcedProtocolHTTPSConnection(httplib.HTTPSConnection):
   """
   Some of python implementations does not work correctly with sslv3 but trying to use it, we need to change protocol to
   tls1.
   """
-  def __init__(self, host, port, **kwargs):
+  def __init__(self, host, port, force_protocol, **kwargs):
     httplib.HTTPSConnection.__init__(self, host, port, **kwargs)
+    self.force_protocol = force_protocol
 
   def connect(self):
     sock = socket.create_connection((self.host, self.port), self.timeout)
     if getattr(self, '_tunnel_host', None):
       self.sock = sock
       self._tunnel()
-    self.sock = ssl.wrap_socket(sock, self.key_file, self.cert_file, ssl_version=ssl.PROTOCOL_TLSv1)
+    self.sock = ssl.wrap_socket(sock, self.key_file, self.cert_file, ssl_version=getattr(ssl, self.force_protocol))
 
-def make_connection(host, port, https):
+def make_connection(host, port, https, force_protocol=None):
   try:
     conn = httplib.HTTPConnection(host, port) if not https else httplib.HTTPSConnection(host, port)
     conn.request("GET", "/")
@@ -46,7 +47,7 @@ def make_connection(host, port, https):
   except ssl.SSLError:
     # got ssl error, lets try to use TLS1 protocol, maybe it will work
     try:
-      tls1_conn = TLS1HTTPSConnection(host, port)
+      tls1_conn = ForcedProtocolHTTPSConnection(host, port, force_protocol)
       tls1_conn.request("GET", "/")
       return tls1_conn.getresponse().status
     except Exception as e:
@@ -65,15 +66,17 @@ def main():
   parser.add_option("-m", "--hosts", dest="hosts", help="Comma separated hosts list for WEB UI to check it availability")
   parser.add_option("-p", "--port", dest="port", help="Port of WEB UI to check it availability")
   parser.add_option("-s", "--https", dest="https", help="\"True\" if value of dfs.http.policy is \"HTTPS_ONLY\"")
+  parser.add_option("-o", "--protocol", dest="protocol", help="Protocol to use when executing https request")
 
   (options, args) = parser.parse_args()
   
   hosts = options.hosts.split(',')
   port = options.port
   https = options.https
+  protocol = options.protocol
 
   for host in hosts:
-    httpCode = make_connection(host, port, https.lower() == "true")
+    httpCode = make_connection(host, port, https.lower() == "true", protocol)
 
     if httpCode != 200:
       print "Cannot access WEB UI on: http://" + host + ":" + port if not https.lower() == "true" else "Cannot access WEB UI on: https://" + host + ":" + port

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
index 25231f9..7f282b3 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/params.py
@@ -26,3 +26,4 @@ else:
 
 nfsgateway_heapsize = config['configurations']['hadoop-env']['nfsgateway_heapsize']
 retryAble = default("/commandParams/command_retry_enabled", False)
+script_https_protocol = Script.get_force_https_protocol()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
index 737ae04..dffa077 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/service_check.py
@@ -86,7 +86,8 @@ class HdfsServiceCheckDefault(HdfsServiceCheck):
         checkWebUIFileName = "checkWebUI.py"
         checkWebUIFilePath = format("{tmp_dir}/{checkWebUIFileName}")
         comma_sep_jn_hosts = ",".join(params.journalnode_hosts)
-        checkWebUICmd = format("ambari-python-wrap {checkWebUIFilePath} -m {comma_sep_jn_hosts} -p {journalnode_port} -s {https_only}")
+
+        checkWebUICmd = format("ambari-python-wrap {checkWebUIFilePath} -m {comma_sep_jn_hosts} -p {journalnode_port} -s {https_only} -o {script_https_protocol}")
         File(checkWebUIFilePath,
              content=StaticFile(checkWebUIFileName),
              mode=0775)

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
index 5997011..9ea4fe4 100644
--- a/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
+++ b/ambari-server/src/main/resources/common-services/HDFS/2.1.0.2.0/package/scripts/utils.py
@@ -35,10 +35,10 @@ from resource_management.core.exceptions import Fail
 from resource_management.libraries.functions.namenode_ha_utils import get_namenode_states
 from resource_management.libraries.script.script import Script
 from resource_management.libraries.functions.show_logs import show_logs
-from ambari_commons.inet_utils import ensure_ssl_using_tls_v1
+from ambari_commons.inet_utils import ensure_ssl_using_protocol
 from zkfc_slave import ZkfcSlaveDefault
 
-ensure_ssl_using_tls_v1()
+ensure_ssl_using_protocol(Script.get_force_https_protocol())
 
 def safe_zkfc_op(action, env):
   """

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
index 49cfa86..f85efb0 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/params.py
@@ -241,3 +241,4 @@ ttnode_heapsize = "1024m"
 dtnode_heapsize = config['configurations']['hadoop-env']['dtnode_heapsize']
 mapred_pid_dir_prefix = default("/configurations/mapred-env/mapred_pid_dir_prefix","/var/run/hadoop-mapreduce")
 mapred_log_dir_prefix = default("/configurations/mapred-env/mapred_log_dir_prefix","/var/log/hadoop-mapreduce")
+script_https_protocol = Script.get_force_https_protocol()
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/ambari/blob/ecc916bb/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
----------------------------------------------------------------------
diff --git a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
index 81d7ca5..3b54fd0 100644
--- a/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
+++ b/ambari-server/src/main/resources/stacks/BIGTOP/0.8/services/HDFS/package/scripts/service_check.py
@@ -93,7 +93,7 @@ class HdfsServiceCheck(Script):
       comma_sep_jn_hosts = ",".join(params.journalnode_hosts)
       checkWebUICmd = format(
         "su -s /bin/bash - {smoke_test_user} -c 'python {checkWebUIFilePath} -m "
-        "{comma_sep_jn_hosts} -p {journalnode_port}'")
+        "{comma_sep_jn_hosts} -p {journalnode_port} -o {script_https_protocol}'")
       File(checkWebUIFilePath,
            content=StaticFile(checkWebUIFileName))