瀏覽代碼

AMBARI-14944. [cleanup] Remove old unused component version/status staff from agent (dlysnichenko)

Lisnichenko Dmitro 9 年之前
父節點
當前提交
41edfc4e93

+ 22 - 24
ambari-agent/src/main/python/ambari_agent/ActionQueue.py

@@ -81,7 +81,7 @@ class ActionQueue(threading.Thread):
     self.customServiceOrchestrator = CustomServiceOrchestrator(config, controller)
     self.parallel_execution = config.get_parallel_exec_option()
     if self.parallel_execution == 1:
-      logger.info("Parallel execution is enabled, will start Agent commands in parallel")
+      logger.info("Parallel execution is enabled, will execute agent commands in parallel")
 
   def stop(self):
     self._stop.set()
@@ -167,9 +167,9 @@ class ActionQueue(threading.Thread):
     while not self.backgroundCommandQueue.empty():
       try:
         command = self.backgroundCommandQueue.get(False)
-        if(command.has_key('__handle') and command['__handle'].status == None):
+        if command.has_key('__handle') and command['__handle'].status == None:
           self.process_command(command)
-      except (Queue.Empty):
+      except Queue.Empty:
         pass
 
   def processStatusCommandQueueSafeEmpty(self):
@@ -177,12 +177,12 @@ class ActionQueue(threading.Thread):
       try:
         command = self.statusCommandQueue.get(False)
         self.process_command(command)
-      except (Queue.Empty):
+      except Queue.Empty:
         pass
 
 
   def createCommandHandle(self, command):
-    if(command.has_key('__handle')):
+    if command.has_key('__handle'):
       raise AgentException("Command already has __handle")
     command['__handle'] = BackgroundCommandExecutionHandle(command, command['commandId'], None, self.on_background_command_complete_callback)
     return command
@@ -293,7 +293,7 @@ class ActionQueue(threading.Thread):
         else:
           status = self.FAILED_STATUS
 
-      if status != self.COMPLETED_STATUS and retryAble == True and retryDuration > 0:
+      if status != self.COMPLETED_STATUS and retryAble and retryDuration > 0:
         delay = self.get_retry_delay(delay)
         if delay > retryDuration:
           delay = retryDuration
@@ -364,20 +364,19 @@ class ActionQueue(threading.Thread):
       if command.has_key('configurationTags'):
         configHandler.write_actual(command['configurationTags'])
         roleResult['configurationTags'] = command['configurationTags']
-      component = {'serviceName':command['serviceName'],'componentName':command['role']}
-      if command.has_key('roleCommand') and \
-        (command['roleCommand'] == self.ROLE_COMMAND_START or \
-        (command['roleCommand'] == self.ROLE_COMMAND_INSTALL \
-        and component in LiveStatus.CLIENT_COMPONENTS) or \
-        (command['roleCommand'] == self.ROLE_COMMAND_CUSTOM_COMMAND and \
-        command['hostLevelParams'].has_key('custom_command') and \
-        command['hostLevelParams']['custom_command'] == self.CUSTOM_COMMAND_RESTART)):
-        configHandler.write_actual_component(command['role'], command['configurationTags'])
-        if command['hostLevelParams'].has_key('clientsToUpdateConfigs') and \
-          command['hostLevelParams']['clientsToUpdateConfigs']:
-          configHandler.write_client_components(command['serviceName'], command['configurationTags'],
+      if 'roleCommand' in command and \
+          (command['roleCommand'] == self.ROLE_COMMAND_START or
+               (command['roleCommand'] == self.ROLE_COMMAND_CUSTOM_COMMAND and
+                  'custom_command' in command['hostLevelParams'] and
+                      command['hostLevelParams']['custom_command'] == self.CUSTOM_COMMAND_RESTART)):
+        configHandler.write_actual_component(command['role'],
+                                             command['configurationTags'])
+        if 'clientsToUpdateConfigs' in command['hostLevelParams'] and command['hostLevelParams']['clientsToUpdateConfigs']:
+          configHandler.write_client_components(command['serviceName'],
+                                                command['configurationTags'],
                                                 command['hostLevelParams']['clientsToUpdateConfigs'])
-        roleResult['configurationTags'] = configHandler.read_actual_component(command['role'])
+        roleResult['configurationTags'] = configHandler.read_actual_component(
+            command['role'])
 
     self.commandStatuses.put_command_status(command, roleResult)
 
@@ -434,7 +433,6 @@ class ActionQueue(threading.Thread):
                               globalConfig, self.config, self.configTags)
 
       component_extra = None
-      request_execution_cmd = False
 
       # For custom services, responsibility to determine service status is
       # delegated to python scripts
@@ -444,20 +442,20 @@ class ActionQueue(threading.Thread):
       if component_status_result['exitcode'] == 0:
         component_status = LiveStatus.LIVE_STATUS
         if self.controller.recovery_manager.enabled() \
-            and self.controller.recovery_manager.configured_for_recovery(component):
+          and self.controller.recovery_manager.configured_for_recovery(component):
           self.controller.recovery_manager.update_current_status(component, component_status)
       else:
         component_status = LiveStatus.DEAD_STATUS
         if self.controller.recovery_manager.enabled() \
-            and self.controller.recovery_manager.configured_for_recovery(component):
+          and self.controller.recovery_manager.configured_for_recovery(component):
           self.controller.recovery_manager.update_current_status(component, component_status)
       request_execution_cmd = self.controller.recovery_manager.requires_recovery(component) and \
                                 not self.controller.recovery_manager.command_exists(component, ActionQueue.EXECUTION_COMMAND)
 
-      if component_status_result.has_key('structuredOut'):
+      if 'structuredOut' in component_status_result:
         component_extra = component_status_result['structuredOut']
 
-      result = livestatus.build(forced_component_status= component_status)
+      result = livestatus.build(component_status=component_status)
       if self.controller.recovery_manager.enabled():
         result['sendExecCmdDet'] = str(request_execution_cmd)
 

+ 19 - 55
ambari-agent/src/main/python/ambari_agent/LiveStatus.py

@@ -18,21 +18,13 @@ See the License for the specific language governing permissions and
 limitations under the License.
 '''
 
-import ambari_simplejson as json
 import logging
-from StatusCheck import StatusCheck
-import AmbariConfig
-from StackVersionsFileHandler import StackVersionsFileHandler
 from ActualConfigHandler import ActualConfigHandler
 
 logger = logging.getLogger()
 
-class LiveStatus:
-
-  SERVICES = []
-  CLIENT_COMPONENTS = []
-  COMPONENTS = []
 
+class LiveStatus:
   LIVE_STATUS = "STARTED"
   DEAD_STATUS = "INSTALLED"
 
@@ -42,57 +34,29 @@ class LiveStatus:
     self.service = service
     self.component = component
     self.globalConfig = globalConfig
-    versionsFileDir = config.get('agent', 'prefix')
-    self.versionsHandler = StackVersionsFileHandler(versionsFileDir)
     self.configTags = configTags
     self.actualConfigHandler = ActualConfigHandler(config, configTags)
 
-  def belongsToService(self, component):
-    #TODO: Should also check belonging of server to cluster
-    return component['serviceName'] == self.service
-
-  def build(self, forced_component_status = None):
+  def build(self, component_status):
     """
-    If forced_component_status is explicitly defined, than StatusCheck methods are
-    not used. This feature has been added to support custom (ver 2.0) services.
+    :param component_status: component status to include into report
+    :return: populated livestatus dict
     """
-    global SERVICES, CLIENT_COMPONENTS, COMPONENTS, LIVE_STATUS, DEAD_STATUS
-
-    component = {"serviceName" : self.service, "componentName" : self.component}
-    if forced_component_status: # If already determined
-      status = forced_component_status  # Nothing to do
-    elif component in self.CLIENT_COMPONENTS:
-      status = self.DEAD_STATUS # CLIENT components can't have status STARTED
-    elif component in self.COMPONENTS:
-      statusCheck = StatusCheck(AmbariConfig.servicesToPidNames,
-                                AmbariConfig.pidPathVars, self.globalConfig,
-                                AmbariConfig.servicesToLinuxUser)
-      serviceStatus = statusCheck.getStatus(self.component)
-      if serviceStatus is None:
-        logger.warn("There is no service to pid mapping for " + self.component)
-      status = self.LIVE_STATUS if serviceStatus else self.DEAD_STATUS
-
-    livestatus = {"componentName" : self.component,
-                 "msg" : "",
-                 "status" : status,
-                 "clusterName" : self.cluster,
-                 "serviceName" : self.service,
-                 "stackVersion": self.versionsHandler.
-                 read_stack_version(self.component)
-    }
-    
-    active_config = self.actualConfigHandler.read_actual_component(self.component)
-    if not active_config is None:
+    global LIVE_STATUS, DEAD_STATUS
+
+    livestatus = {"componentName": self.component,
+                  "msg": "",
+                  "status": component_status,
+                  "clusterName": self.cluster,
+                  "serviceName": self.service,
+                  "stackVersion": ""  # TODO: populate ?
+                  }
+
+    active_config = self.actualConfigHandler.read_actual_component(
+      self.component)
+    if active_config is not None:
       livestatus['configurationTags'] = active_config
 
-    logger.debug("The live status for component " + str(self.component) +\
-                " of service " + str(self.service) + " is " + str(livestatus))
+    logger.debug("The live status for component " + str(self.component) +
+                 " of service " + str(self.service) + " is " + str(livestatus))
     return livestatus
-
-def main(argv=None):
-  for service in SERVICES:
-    livestatus = LiveStatus('', service)
-    print json.dumps(livestatus.build())
-
-if __name__ == '__main__':
-  main()

+ 0 - 110
ambari-agent/src/main/python/ambari_agent/StackVersionsFileHandler.py

@@ -1,110 +0,0 @@
-#!/usr/bin/env python
-
-'''
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-'''
-
-import os.path
-import logging
-import threading
-import traceback
-import shutil
-
-logger = logging.getLogger(__name__)
-
-class StackVersionsFileHandler:
-
-  VER_FILE = "current-stack"
-  DEFAULT_VER = ""
-
-  def __init__(self, versionsFileDir):
-    self.versionsFileDir = versionsFileDir
-    self.versionsFilePath = os.path.join(versionsFileDir, self.VER_FILE)
-    self._lock = threading.RLock()
-
-  def read_stack_version(self, component):
-    try :
-      self.touch_file()
-      for line in open(self.versionsFilePath):
-        comp, ver = self.extract(line)
-        if comp == component:
-          return ver
-      return self.DEFAULT_VER
-    except Exception, err:
-      logger.error("Can't read versions file: %s " % err.message)
-      traceback.print_exc()
-      return self.DEFAULT_VER
-
-
-  def read_all_stack_versions(self):
-    result = {}
-    try :
-      self.touch_file()
-      for line in open(self.versionsFilePath):
-        comp, ver = self.extract(line)
-        if comp != self.DEFAULT_VER:
-          result[comp] = ver
-      return result
-    except Exception, err:
-      logger.error("Can't read stack versions file: %s " % err.message)
-      traceback.print_exc()
-      return {}
-
-
-  def write_stack_version(self, component, newVersion):
-    self._lock.acquire()
-    try:
-      values = self.read_all_stack_versions()
-      values[component] = newVersion
-      logger.info("Backing up old stack versions file")
-      backup = os.path.join(self.versionsFileDir, self.VER_FILE + ".bak")
-      shutil.move(self.versionsFilePath, backup)
-      logger.info("Writing new stack versions file")
-      with open (self.versionsFilePath, 'w') as f:
-        for key in values:
-          f.write ("%s\t%s\n" % (key, values[key]))
-
-    except Exception, err:
-      logger.error("Can't write new stack version (%s %s) :%s " % (component,
-            newVersion, err.message))
-      traceback.print_exc()
-    finally:
-      self._lock.release()
-
-
-  def extract(self, statement):
-    '''
-    Extracts <Component>, <HDPstack version> values from lines like
-    GANGLIA	StackVersion-1.3.0
-    '''
-    parts = statement.strip().split()
-    if len(parts) != 2:
-      logger.warn("Wrong stack versions file statement format: %s" % statement)
-      return self.DEFAULT_VER, self.DEFAULT_VER
-    else:
-      return parts[0], parts[1]
-
-
-  def touch_file(self):
-    '''
-     Called to create file when it does not exist
-    '''
-    if not os.path.isfile(self.versionsFilePath):
-      logger.info("Creating stacks versions file at %s" % self.versionsFilePath)
-      open(self.versionsFilePath, 'w').close()
-
-

+ 9 - 19
ambari-agent/src/test/python/ambari_agent/TestActionQueue.py

@@ -29,7 +29,6 @@ from threading import Thread
 import copy
 
 from mock.mock import patch, MagicMock, call
-from ambari_agent.StackVersionsFileHandler import StackVersionsFileHandler
 from ambari_agent.CustomServiceOrchestrator import CustomServiceOrchestrator
 from ambari_agent.PythonExecutor import PythonExecutor
 from ambari_agent.ActualConfigHandler import ActualConfigHandler
@@ -649,7 +648,6 @@ class TestActionQueue(TestCase):
 
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
   @patch.object(ActionQueue, "status_update_callback")
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "requestComponentStatus")
   @patch.object(CustomServiceOrchestrator, "requestComponentSecurityState")
   @patch.object(ActionQueue, "execute_command")
@@ -657,7 +655,7 @@ class TestActionQueue(TestCase):
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_status_command(self, CustomServiceOrchestrator_mock,
                                   build_mock, execute_command_mock, requestComponentSecurityState_mock,
-                                  requestComponentStatus_mock, read_stack_version_mock,
+                                  requestComponentStatus_mock,
                                   status_update_callback):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -686,7 +684,6 @@ class TestActionQueue(TestCase):
   @patch.object(RecoveryManager, "requires_recovery")
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
   @patch.object(ActionQueue, "status_update_callback")
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "requestComponentStatus")
   @patch.object(CustomServiceOrchestrator, "requestComponentSecurityState")
   @patch.object(ActionQueue, "execute_command")
@@ -694,7 +691,7 @@ class TestActionQueue(TestCase):
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_status_command_recovery(self, CustomServiceOrchestrator_mock,
                                   build_mock, execute_command_mock, requestComponentSecurityState_mock,
-                                  requestComponentStatus_mock, read_stack_version_mock,
+                                  requestComponentStatus_mock,
                                   status_update_callback, requires_recovery_mock,
                                   command_exists_mock):
     CustomServiceOrchestrator_mock.return_value = None
@@ -743,7 +740,6 @@ class TestActionQueue(TestCase):
 
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
   @patch.object(ActionQueue, "status_update_callback")
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "requestComponentStatus")
   @patch.object(CustomServiceOrchestrator, "requestComponentSecurityState")
   @patch.object(ActionQueue, "execute_command")
@@ -752,7 +748,7 @@ class TestActionQueue(TestCase):
   def test_execute_status_command_with_alerts(self, CustomServiceOrchestrator_mock,
                                               requestComponentSecurityState_mock,
                                   build_mock, execute_command_mock,
-                                  requestComponentStatus_mock, read_stack_version_mock,
+                                  requestComponentStatus_mock,
                                   status_update_callback):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -848,10 +844,9 @@ class TestActionQueue(TestCase):
   @not_for_platform(PLATFORM_LINUX)
   @patch("time.sleep")
   @patch.object(OSCheck, "os_distribution", new=MagicMock(return_value=os_distro_value))
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_retryable_command(self, CustomServiceOrchestrator_mock,
-                                     read_stack_version_mock, sleep_mock
+                                     sleep_mock
   ):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -889,10 +884,9 @@ class TestActionQueue(TestCase):
   @patch("time.time")
   @patch("time.sleep")
   @patch.object(OSCheck, "os_distribution", new=MagicMock(return_value=os_distro_value))
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_retryable_command_with_time_lapse(self, CustomServiceOrchestrator_mock,
-                                     read_stack_version_mock, sleep_mock, time_mock
+                                     sleep_mock, time_mock
   ):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -933,10 +927,9 @@ class TestActionQueue(TestCase):
   @not_for_platform(PLATFORM_LINUX)
   @patch("time.sleep")
   @patch.object(OSCheck, "os_distribution", new=MagicMock(return_value=os_distro_value))
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_retryable_command_fail_and_succeed(self, CustomServiceOrchestrator_mock,
-                                                      read_stack_version_mock, sleep_mock
+                                                      sleep_mock
   ):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -970,10 +963,9 @@ class TestActionQueue(TestCase):
   @not_for_platform(PLATFORM_LINUX)
   @patch("time.sleep")
   @patch.object(OSCheck, "os_distribution", new=MagicMock(return_value=os_distro_value))
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_retryable_command_succeed(self, CustomServiceOrchestrator_mock,
-                                             read_stack_version_mock, sleep_mock
+                                             sleep_mock
   ):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
@@ -997,11 +989,10 @@ class TestActionQueue(TestCase):
     self.assertEqual(1, runCommand_mock.call_count)
 
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
   @patch.object(CustomServiceOrchestrator, "runCommand")
   @patch.object(CustomServiceOrchestrator, "__init__")
   def test_execute_background_command(self, CustomServiceOrchestrator_mock,
-                                  runCommand_mock, read_stack_version_mock
+                                  runCommand_mock,
                                   ):
     CustomServiceOrchestrator_mock.return_value = None
     CustomServiceOrchestrator.runCommand.return_value = {'exitcode' : 0,
@@ -1027,8 +1018,7 @@ class TestActionQueue(TestCase):
 
   @patch.object(CustomServiceOrchestrator, "get_py_executor")
   @patch.object(CustomServiceOrchestrator, "resolve_script_path")
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
-  def test_execute_python_executor(self, read_stack_version_mock, resolve_script_path_mock,
+  def test_execute_python_executor(self, resolve_script_path_mock,
                                    get_py_executor_mock):
     
     dummy_controller = MagicMock()

+ 1 - 4
ambari-agent/src/test/python/ambari_agent/TestCustomServiceOrchestrator.py

@@ -381,16 +381,13 @@ class TestCustomServiceOrchestrator(TestCase):
     except:
       pass
 
-  from ambari_agent.StackVersionsFileHandler import StackVersionsFileHandler
-
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
   @patch.object(CustomServiceOrchestrator, "get_py_executor")
   @patch("ambari_commons.shell.kill_process_with_children")
   @patch.object(FileCache, "__init__")
   @patch.object(CustomServiceOrchestrator, "resolve_script_path")
   @patch.object(CustomServiceOrchestrator, "resolve_hook_script_path")
-  @patch.object(StackVersionsFileHandler, "read_stack_version")
-  def test_cancel_backgound_command(self, read_stack_version_mock, resolve_hook_script_path_mock,
+  def test_cancel_backgound_command(self, resolve_hook_script_path_mock,
                                     resolve_script_path_mock, FileCache_mock, kill_process_with_children_mock,
                                     get_py_executor_mock):
     FileCache_mock.return_value = None

+ 0 - 1
ambari-agent/src/test/python/ambari_agent/TestHeartbeat.py

@@ -33,7 +33,6 @@ with patch("platform.linux_distribution", return_value = ('Suse','11','Final')):
   from ambari_agent.ActionQueue import ActionQueue
   from ambari_agent.LiveStatus import LiveStatus
   from ambari_agent import AmbariConfig
-  from ambari_agent.StackVersionsFileHandler import StackVersionsFileHandler
   from ambari_agent.HostInfo import HostInfoLinux
 
 from only_for_platform import not_for_platform, PLATFORM_WINDOWS

+ 1 - 141
ambari-agent/src/test/python/ambari_agent/TestLiveStatus.py

@@ -36,149 +36,11 @@ class TestLiveStatus(TestCase):
     # disable stdout
     out = StringIO.StringIO()
     sys.stdout = out
-    LiveStatus.SERVICES = [
-      "HDFS", "MAPREDUCE", "GANGLIA", "HBASE",
-      "ZOOKEEPER", "OOZIE",
-      "KERBEROS", "TEMPLETON", "HIVE",
-      "YARN", "MAPREDUCE2", "FLUME", "TEZ",
-      "FALCON", "STORM"
-    ]
-    LiveStatus.CLIENT_COMPONENTS = [
-      {"serviceName" : "HBASE",
-       "componentName" : "HBASE_CLIENT"},
-      {"serviceName" : "HDFS",
-       "componentName" : "HDFS_CLIENT"},
-      {"serviceName" : "MAPREDUCE",
-       "componentName" : "MAPREDUCE_CLIENT"},
-      {"serviceName" : "ZOOKEEPER",
-       "componentName" : "ZOOKEEPER_CLIENT"},
-      {"serviceName" : "OOZIE",
-       "componentName" : "OOZIE_CLIENT"},
-      {"serviceName" : "HCATALOG",
-       "componentName" : "HCAT"},
-      {"serviceName" : "HIVE",
-       "componentName" : "HIVE_CLIENT"},
-      {"serviceName" : "YARN",
-       "componentName" : "YARN_CLIENT"},
-      {"serviceName" : "MAPREDUCE2",
-       "componentName" : "MAPREDUCE2_CLIENT"},
-      {"serviceName" : "PIG",
-       "componentName" : "PIG"},
-      {"serviceName" : "SQOOP",
-       "componentName" : "SQOOP"},
-      {"serviceName" : "TEZ",
-       "componentName" : "TEZ_CLIENT"},
-      {"serviceName" : "FALCON",
-       "componentName" : "FALCON_CLIENT"}
-    ]
-    LiveStatus.COMPONENTS = [
-      {"serviceName" : "HDFS",
-       "componentName" : "DATANODE"},
-      {"serviceName" : "HDFS",
-       "componentName" : "NAMENODE"},
-      {"serviceName" : "HDFS",
-       "componentName" : "SECONDARY_NAMENODE"},
-      {"serviceName" : "HDFS",
-       "componentName" : "JOURNALNODE"},
-      {"serviceName" : "HDFS",
-       "componentName" : "ZKFC"},
-      {"serviceName" : "MAPREDUCE",
-       "componentName" : "JOBTRACKER"},
-      {"serviceName" : "MAPREDUCE",
-       "componentName" : "TASKTRACKER"},
-      {"serviceName" : "GANGLIA",
-       "componentName" : "GANGLIA_SERVER"},
-      {"serviceName" : "GANGLIA",
-       "componentName" : "GANGLIA_MONITOR"},
-      {"serviceName" : "HBASE",
-       "componentName" : "HBASE_MASTER"},
-      {"serviceName" : "HBASE",
-       "componentName" : "HBASE_REGIONSERVER"},
-      {"serviceName" : "FLUME",
-       "componentName" : "FLUME_SERVER"},
-      {"serviceName" : "ZOOKEEPER",
-       "componentName" : "ZOOKEEPER_SERVER"},
-      {"serviceName" : "OOZIE",
-       "componentName" : "OOZIE_SERVER"},
-      {"serviceName" : "HCATALOG",
-       "componentName" : "HCATALOG_SERVER"},
-      {"serviceName" : "KERBEROS",
-       "componentName" : "KERBEROS_SERVER"},
-      {"serviceName" : "HIVE",
-       "componentName" : "HIVE_SERVER"},
-      {"serviceName" : "HIVE",
-       "componentName" : "HIVE_METASTORE"},
-      {"serviceName" : "HIVE",
-       "componentName" : "MYSQL_SERVER"},
-      {"serviceName" : "WEBHCAT",
-       "componentName" : "WEBHCAT_SERVER"},
-      {"serviceName" : "YARN",
-       "componentName" : "RESOURCEMANAGER"},
-      {"serviceName" : "YARN",
-       "componentName" : "NODEMANAGER"},
-      {"serviceName" : "YARN",
-       "componentName" : "APP_TIMELINE_SERVER"},
-      {"serviceName" : "MAPREDUCE2",
-       "componentName" : "HISTORYSERVER"},
-      {"serviceName" : "FALCON",
-       "componentName" : "FALCON_SERVER"},
-      {"serviceName" : "STORM",
-       "componentName" : "NIMBUS"},
-      {"serviceName" : "STORM",
-       "componentName" : "STORM_REST_API"},
-      {"serviceName" : "STORM",
-       "componentName" : "SUPERVISOR"},
-      {"serviceName" : "STORM",
-       "componentName" : "STORM_UI_SERVER"},
-      {"serviceName" : "STORM",
-       "componentName" : "DRPC_SERVER"}
-    ]
-
 
   def tearDown(self):
     # enable stdout
     sys.stdout = sys.__stdout__
 
-  @patch("os.path.isdir")
-  @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
-  @patch.object(ActualConfigHandler.ActualConfigHandler, "read_actual_component")
-  def test_build(self, read_actual_component_mock, isdir_mock):
-    isdir_mock.return_value = False
-    for component in LiveStatus.COMPONENTS:
-      config = AmbariConfig().getConfig()
-      config.set('agent', 'prefix', "ambari_agent" + os.sep + "dummy_files")
-      livestatus = LiveStatus('', component['serviceName'], component['componentName'], {}, config, {})
-      livestatus.versionsHandler.versionsFilePath = "ambari_agent" + os.sep + "dummy_files" + os.sep + "dummy_current_stack"
-      result = livestatus.build()
-      print "LiveStatus of {0}: {1}".format(component['serviceName'], str(result))
-      self.assertEquals(len(result) > 0, True, 'Livestatus should not be empty')
-      if component['componentName'] == 'GANGLIA_SERVER':
-        self.assertEquals(result['stackVersion'],'{"stackName":"HDP","stackVersion":"1.2.2"}',
-                      'Livestatus should contain component stack version')
-
-    # Test build status for CLIENT component (in LiveStatus.CLIENT_COMPONENTS)
-    read_actual_component_mock.return_value = "some tags"
-    livestatus = LiveStatus('c1', 'HDFS', 'HDFS_CLIENT', { }, config, {})
-    result = livestatus.build()
-    self.assertTrue(len(result) > 0, 'Livestatus should not be empty')
-    self.assertTrue(result.has_key('configurationTags'))
-    # Test build status with forced_component_status
-    ## Alive
-    livestatus = LiveStatus('c1', 'HDFS', 'HDFS_CLIENT', { }, config, {})
-    result = livestatus.build(forced_component_status = LiveStatus.LIVE_STATUS)
-    self.assertTrue(len(result) > 0, 'Livestatus should not be empty')
-    self.assertTrue(result['status'], LiveStatus.LIVE_STATUS)
-    ## Dead
-    livestatus = LiveStatus('c1', 'HDFS', 'HDFS_CLIENT', { }, config, {})
-    result = livestatus.build(forced_component_status = LiveStatus.DEAD_STATUS)
-    self.assertTrue(len(result) > 0, 'Livestatus should not be empty')
-    self.assertTrue(result['status'], LiveStatus.DEAD_STATUS)
-
-    livestatus = LiveStatus('c1', 'TEZ', 'TEZ_CLIENT', { }, config, {})
-    result = livestatus.build(forced_component_status = LiveStatus.LIVE_STATUS)
-    self.assertTrue(len(result) > 0, 'Livestatus should not be empty')
-    self.assertTrue(result['status'], LiveStatus.LIVE_STATUS)
-
   @patch.object(OSCheck, "os_distribution", new = MagicMock(return_value = os_distro_value))
   @patch.object(ActualConfigHandler.ActualConfigHandler, "read_actual_component")
   @patch.object(StatusCheck.StatusCheck, "getStatus")
@@ -192,9 +54,7 @@ class TestLiveStatus(TestCase):
     config.set('agent', 'prefix', "ambari_agent" + os.sep + "dummy_files")
     livestatus = LiveStatus('', 'SOME_UNKNOWN_SERVICE',
                             'SOME_UNKNOWN_COMPONENT', {}, config, {})
-    livestatus.versionsHandler.versionsFilePath = "ambari_agent" + \
-                      os.sep + "dummy_files" + os.sep + "dummy_current_stack"
-    result = livestatus.build(forced_component_status = "STARTED")
+    result = livestatus.build(component_status="STARTED")
     result_str = pprint.pformat(result)
     self.assertEqual(result_str,
                      "{'clusterName': '',\n "

+ 0 - 112
ambari-agent/src/test/python/ambari_agent/TestStackVersionsFileHandler.py

@@ -1,112 +0,0 @@
-#!/usr/bin/env python
-
-'''
-Licensed to the Apache Software Foundation (ASF) under one
-or more contributor license agreements.  See the NOTICE file
-distributed with this work for additional information
-regarding copyright ownership.  The ASF licenses this file
-to you under the Apache License, Version 2.0 (the
-"License"); you may not use this file except in compliance
-with the License.  You may obtain a copy of the License at
-
-    http://www.apache.org/licenses/LICENSE-2.0
-
-Unless required by applicable law or agreed to in writing, software
-distributed under the License is distributed on an "AS IS" BASIS,
-WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-See the License for the specific language governing permissions and
-limitations under the License.
-'''
-
-from unittest import TestCase
-import unittest
-import StringIO
-import socket
-import os, sys
-from mock.mock import patch
-from mock.mock import MagicMock
-from mock.mock import create_autospec
-import os, errno, tempfile
-from ambari_agent import StackVersionsFileHandler
-import logging
-
-stackVersionsFileHandler = \
-      StackVersionsFileHandler.StackVersionsFileHandler("/tmp")
-dummyVersionsFile = os.path.dirname(os.path.abspath(__file__))+ os.sep +'dummy_files' + os.sep + 'dummy_current_stack'
-
-class TestStackVersionsFileHandler(TestCase):
-
-  logger = logging.getLogger()
-
-  @patch.object(stackVersionsFileHandler, 'touch_file')
-  def test_read_stack_version(self, touch_method):
-    stackVersionsFileHandler.versionsFilePath = dummyVersionsFile
-    result = stackVersionsFileHandler.read_stack_version("GANGLIA_SERVER")
-    self.assertEquals(result, '{"stackName":"HDP","stackVersion":"1.2.2"}')
-    result = stackVersionsFileHandler.read_stack_version("NOTEXISTING")
-    self.assertEquals(result, stackVersionsFileHandler.DEFAULT_VER)
-    self.assertTrue(touch_method.called)
-
-
-  @patch.object(stackVersionsFileHandler, 'touch_file')
-  def test_read_all_stack_versions(self, touch_method):
-    stackVersionsFileHandler.versionsFilePath = dummyVersionsFile
-    result = stackVersionsFileHandler.read_all_stack_versions()
-    self.assertEquals(len(result.keys()), 3)
-    self.assertEquals(result["HCATALOG"],
-          '{"stackName":"HDP","stackVersion":"1.2.2"}')
-    self.assertTrue(touch_method.called)
-
-
-  def test_extract(self):
-    s = '   GANGLIA_SERVER	\t  {"stackName":"HDP","stackVersion":"1.3.0"}  '
-    comp, ver = stackVersionsFileHandler.extract(s)
-    self.assertEqual(comp, "GANGLIA_SERVER")
-    self.assertEqual(ver, '{"stackName":"HDP","stackVersion":"1.3.0"}')
-    # testing wrong value
-    s = "   GANGLIA_SERVER	"
-    comp, ver = stackVersionsFileHandler.extract(s)
-    self.assertEqual(comp, stackVersionsFileHandler.DEFAULT_VER)
-    self.assertEqual(ver, stackVersionsFileHandler.DEFAULT_VER)
-
-
-  def test_touch_file(self):
-    tmpfile = tempfile.mktemp()
-    stackVersionsFileHandler.versionsFilePath = tmpfile
-    stackVersionsFileHandler.touch_file()
-    result = os.path.isfile(tmpfile)
-    self.assertEqual(result, True)
-
-
-  def test_write_stack_version(self):
-    #saving old values
-    oldFilePathValue = stackVersionsFileHandler.versionsFilePath
-    oldversionsFileDir = stackVersionsFileHandler.versionsFileDir
-    oldVerFile = stackVersionsFileHandler.VER_FILE
-    #preparations and invocation
-    tmpfile = tempfile.mktemp()
-    stackVersionsFileHandler.versionsFilePath = tmpfile
-    stackVersionsFileHandler.VER_FILE = \
-      os.path.basename(tmpfile)
-    stackVersionsFileHandler.versionsFileDir = \
-      os.path.dirname(tmpfile)
-    stackVersionsFileHandler.touch_file()
-    stackVersionsFileHandler.write_stack_version(
-      "GANGLIA_SERVER", '"stackVersion":"1.3.0"')
-    # Checking if backup file exists
-    expectedBackupFile = tmpfile + ".bak"
-    self.assertTrue(os.path.isfile(expectedBackupFile))
-    os.remove(expectedBackupFile)
-    # Checking content of created file
-    content = stackVersionsFileHandler.read_all_stack_versions()
-    self.assertEquals(len(content), 1)
-    self.assertEqual(content['GANGLIA_SERVER'], '"stackVersion":"1.3.0"')
-    self.assertTrue(os.path.isfile(tmpfile))
-    os.remove(tmpfile)
-    # Restoring old values
-    stackVersionsFileHandler.versionsFilePath = oldFilePathValue
-    stackVersionsFileHandler.versionsFileDir = oldversionsFileDir
-    stackVersionsFileHandler.VER_FILE = oldVerFile
-
-if __name__ == "__main__":
-  unittest.main(verbosity=2)