You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@slider.apache.org by go...@apache.org on 2014/10/24 05:50:08 UTC

git commit: SLIDER-535 Slider agent log is very chatty as it dumps on every heartbeat

Repository: incubator-slider
Updated Branches:
  refs/heads/develop 186d3ba57 -> 3111cbad5


SLIDER-535 Slider agent log is very chatty as it dumps on every heartbeat


Project: http://git-wip-us.apache.org/repos/asf/incubator-slider/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-slider/commit/3111cbad
Tree: http://git-wip-us.apache.org/repos/asf/incubator-slider/tree/3111cbad
Diff: http://git-wip-us.apache.org/repos/asf/incubator-slider/diff/3111cbad

Branch: refs/heads/develop
Commit: 3111cbad5dc845c1fde0b572148d713aee23a81d
Parents: 186d3ba
Author: Gour Saha <go...@apache.org>
Authored: Thu Oct 23 20:49:23 2014 -0700
Committer: Gour Saha <go...@apache.org>
Committed: Thu Oct 23 20:49:23 2014 -0700

----------------------------------------------------------------------
 .../src/main/python/agent/ActionQueue.py        | 19 ++++--
 .../src/main/python/agent/AgentToggleLogger.py  | 69 ++++++++++++++++++++
 .../src/main/python/agent/Controller.py         |  6 +-
 .../python/agent/CustomServiceOrchestrator.py   |  4 +-
 slider-agent/src/main/python/agent/Heartbeat.py | 14 ++--
 .../src/main/python/agent/PythonExecutor.py     | 10 +--
 .../src/test/python/agent/TestActionQueue.py    | 14 ++--
 .../test/python/agent/TestAgentToggleLogger.py  | 60 +++++++++++++++++
 .../src/test/python/agent/TestController.py     |  4 +-
 .../agent/TestCustomServiceOrchestrator.py      | 29 ++++----
 .../src/test/python/agent/TestHeartbeat.py      | 19 +++---
 .../src/test/python/agent/TestPythonExecutor.py | 16 +++--
 12 files changed, 209 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/ActionQueue.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/ActionQueue.py b/slider-agent/src/main/python/agent/ActionQueue.py
index c0dbb91..b19f65f 100644
--- a/slider-agent/src/main/python/agent/ActionQueue.py
+++ b/slider-agent/src/main/python/agent/ActionQueue.py
@@ -27,6 +27,7 @@ import os
 import time
 
 from AgentConfig import AgentConfig
+from AgentToggleLogger import AgentToggleLogger
 from CommandStatusDict import CommandStatusDict
 from CustomServiceOrchestrator import CustomServiceOrchestrator
 import Constants
@@ -51,8 +52,10 @@ class ActionQueue(threading.Thread):
   STORE_APPLIED_CONFIG = 'record_config'
   AUTO_RESTART = 'auto_restart'
 
-  def __init__(self, config, controller):
+  def __init__(self, config, controller, agentToggleLogger):
     super(ActionQueue, self).__init__()
+    self.queueOutAgentToggleLogger = agentToggleLogger
+    self.queueInAgentToggleLogger = AgentToggleLogger("info")
     self.commandQueue = Queue.Queue()
     self.commandStatuses = CommandStatusDict(callback_action=
     self.status_update_callback)
@@ -61,7 +64,8 @@ class ActionQueue(threading.Thread):
     self._stop = threading.Event()
     self.tmpdir = config.getResolvedPath(AgentConfig.APP_TASK_DIR)
     self.customServiceOrchestrator = CustomServiceOrchestrator(config,
-                                                               controller)
+                                                               controller,
+                                                               self.queueOutAgentToggleLogger)
 
 
   def stop(self):
@@ -72,9 +76,12 @@ class ActionQueue(threading.Thread):
 
   def put(self, commands):
     for command in commands:
-      logger.info("Adding " + command['commandType'] + " for service " + \
-                  command['serviceName'] + " of cluster " + \
-                  command['clusterName'] + " to the queue.")
+      self.queueInAgentToggleLogger.adjustLogLevelAtStart(command['commandType'])
+      message = "Adding " + command['commandType'] + " for service " + \
+                command['serviceName'] + " of cluster " + \
+                command['clusterName'] + " to the queue."
+      self.queueInAgentToggleLogger.log(message)
+      self.queueInAgentToggleLogger.adjustLogLevelAtEnd(command['commandType'])
       logger.debug(pprint.pformat(command))
       self.commandQueue.put(command)
 
@@ -86,7 +93,9 @@ class ActionQueue(threading.Thread):
     while not self.stopped():
       time.sleep(2)
       command = self.commandQueue.get() # Will block if queue is empty
+      self.queueOutAgentToggleLogger.adjustLogLevelAtStart(command['commandType'])
       self.process_command(command)
+      self.queueOutAgentToggleLogger.adjustLogLevelAtEnd(command['commandType'])
     logger.info("ActionQueue stopped.")
 
 

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/AgentToggleLogger.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/AgentToggleLogger.py b/slider-agent/src/main/python/agent/AgentToggleLogger.py
new file mode 100644
index 0000000..9a0ae3f
--- /dev/null
+++ b/slider-agent/src/main/python/agent/AgentToggleLogger.py
@@ -0,0 +1,69 @@
+#!/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 logging
+
+logger = logging.getLogger()
+
+'''
+Create a singleton instance of this class for every loop that either
+writes to or reads from the action queue, takes action based on the
+commandType and dumps logs along the way. It's target is to keep the
+verbosity level of agent logs to zero during non-interesting heartbeats
+like STATUS_COMMAND, and to ensure that it starts logging at info level
+again, the moment a non-STATUS_COMMAND shows up during its path into or
+out of the action queue.
+'''
+class AgentToggleLogger:
+  def __init__(self, logLevel="info"):
+    self.logLevel = logLevel
+
+  def log(self, message, *args, **kwargs):
+    if self.logLevel == "info":
+      logger.info(message, *args, **kwargs)
+    else:
+      logger.debug(message, *args, **kwargs)
+
+  '''
+  The methods adjustLogLevelAtStart and adjustLogLevelAtEnd work hand
+  in hand to do the following :
+  - STATUS related info should be logged at least once before the agent
+    enters into the STATUS loop
+  - If a non STATUS command shows up in the queue the logger switches
+    to info level
+  - STATUS will be logged at least once every time the log level toggles
+    back to info level when a non STATUS command shows up
+  '''
+
+  # Call this method at the start of the loop over action queue,
+  # right after reading from or writing to the queue
+  def adjustLogLevelAtStart(self, commandType):
+    from ActionQueue import ActionQueue
+    if self.logLevel != "info" and commandType != ActionQueue.STATUS_COMMAND:
+      self.logLevel = "info"
+
+  # Call this method as the last statement in the loop over action queue
+  def adjustLogLevelAtEnd(self, commandType):
+    from ActionQueue import ActionQueue
+    if commandType == ActionQueue.STATUS_COMMAND:
+      self.logLevel = "debug"
+    else:
+      self.logLevel = "info"
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/Controller.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/Controller.py b/slider-agent/src/main/python/agent/Controller.py
index 9788acd..fe7047d 100644
--- a/slider-agent/src/main/python/agent/Controller.py
+++ b/slider-agent/src/main/python/agent/Controller.py
@@ -31,6 +31,7 @@ import math
 from random import randint
 
 from AgentConfig import AgentConfig
+from AgentToggleLogger import AgentToggleLogger
 from Heartbeat import Heartbeat
 from Register import Register
 from ActionQueue import ActionQueue
@@ -515,10 +516,11 @@ class Controller(threading.Thread):
 
 
   def run(self):
-    self.actionQueue = ActionQueue(self.config, controller=self)
+    self.agentToggleLogger = AgentToggleLogger("info")
+    self.actionQueue = ActionQueue(self.config, controller=self, agentToggleLogger=self.agentToggleLogger)
     self.actionQueue.start()
     self.register = Register(self.config)
-    self.heartbeat = Heartbeat(self.actionQueue, self.config)
+    self.heartbeat = Heartbeat(self.actionQueue, self.config, self.agentToggleLogger)
 
     opener = urllib2.build_opener()
     urllib2.install_opener(opener)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/CustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/CustomServiceOrchestrator.py b/slider-agent/src/main/python/agent/CustomServiceOrchestrator.py
index c632293..1a88bb5 100644
--- a/slider-agent/src/main/python/agent/CustomServiceOrchestrator.py
+++ b/slider-agent/src/main/python/agent/CustomServiceOrchestrator.py
@@ -48,10 +48,10 @@ class CustomServiceOrchestrator():
   LIVE_STATUS = "STARTED"
   DEAD_STATUS = "INSTALLED"
 
-  def __init__(self, config, controller):
+  def __init__(self, config, controller, agentToggleLogger):
     self.config = config
     self.tmp_dir = config.getResolvedPath(AgentConfig.APP_TASK_DIR)
-    self.python_executor = PythonExecutor(self.tmp_dir, config)
+    self.python_executor = PythonExecutor(self.tmp_dir, config, agentToggleLogger)
     self.status_commands_stdout = os.path.realpath(posixpath.join(self.tmp_dir,
                                                                   'status_command_stdout.txt'))
     self.status_commands_stderr = os.path.realpath(posixpath.join(self.tmp_dir,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/Heartbeat.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/Heartbeat.py b/slider-agent/src/main/python/agent/Heartbeat.py
index b107d92..e157ce3 100644
--- a/slider-agent/src/main/python/agent/Heartbeat.py
+++ b/slider-agent/src/main/python/agent/Heartbeat.py
@@ -31,16 +31,17 @@ import AgentConfig
 logger = logging.getLogger()
 
 class Heartbeat:
-  def __init__(self, actionQueue, config=None):
+  def __init__(self, actionQueue, config=None, agentToggleLogger=None):
     self.actionQueue = actionQueue
     self.config = config
     self.reports = []
+    self.agentToggleLogger = agentToggleLogger
 
   def build(self, commandResult, id='-1',
             componentsMapped=False):
     timestamp = int(time.time() * 1000)
     queueResult = self.actionQueue.result()
-    logger.info("Queue result: " + pformat(queueResult))
+    self.agentToggleLogger.log("Queue result: " + pformat(queueResult))
 
     nodeStatus = {"status": "HEALTHY",
                   "cause": "NONE"}
@@ -93,10 +94,11 @@ class Heartbeat:
       if len(componentStatuses) > 0:
         heartbeat['componentStatus'] = componentStatuses
 
-    logger.info("Sending heartbeat with response id: " + str(id) + " and "
-                                                                   "timestamp: " + str(timestamp) +
-                ". Command(s) in progress: " + repr(commandsInProgress) +
-                ". Components mapped: " + repr(componentsMapped))
+    self.agentToggleLogger.log(
+                 "Sending heartbeat with response id: " + str(id) + " and "
+                 "timestamp: " + str(timestamp) +
+                 ". Command(s) in progress: " + repr(commandsInProgress) +
+                 ". Components mapped: " + repr(componentsMapped))
     logger.debug("Heartbeat : " + pformat(heartbeat))
 
     return heartbeat

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/main/python/agent/PythonExecutor.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/main/python/agent/PythonExecutor.py b/slider-agent/src/main/python/agent/PythonExecutor.py
index 54ce247..866755f 100644
--- a/slider-agent/src/main/python/agent/PythonExecutor.py
+++ b/slider-agent/src/main/python/agent/PythonExecutor.py
@@ -30,6 +30,7 @@ import shell
 import sys
 import platform
 import Constants
+from AgentToggleLogger import AgentToggleLogger
 
 
 logger = logging.getLogger()
@@ -47,9 +48,10 @@ class PythonExecutor:
   event = threading.Event()
   python_process_has_been_killed = False
 
-  def __init__(self, tmpDir, config):
+  def __init__(self, tmpDir, config, agentToggleLogger):
     self.tmpDir = tmpDir
     self.config = config
+    self.agentToggleLogger = agentToggleLogger
     pass
 
   def run_file(self, script, script_params, tmpoutfile, tmperrfile, timeout,
@@ -81,7 +83,7 @@ class PythonExecutor:
 
     script_params += [tmpstructedoutfile, logger_level]
     pythonCommand = self.python_command(script, script_params)
-    logger.info("Running command " + pprint.pformat(pythonCommand))
+    self.agentToggleLogger.log("Running command " + pprint.pformat(pythonCommand))
     process = self.launch_python_subprocess(pythonCommand, tmpout, tmperr,
                                             environment_vars)
     logger.debug("Launching watchdog thread")
@@ -116,7 +118,7 @@ class PythonExecutor:
       error = str(error) + "\n Python script has been killed due to timeout"
       returncode = 999
     result = self.condenseOutput(out, error, returncode, structured_out)
-    logger.info("Result: %s" % result)
+    self.agentToggleLogger.log("Result: %s" % result)
     return result
 
 
@@ -130,7 +132,7 @@ class PythonExecutor:
     env = os.environ.copy()
     if environment_vars:
       for k, v in environment_vars:
-        logger.info("Setting env: %s to %s", k, v)
+        self.agentToggleLogger.log("Setting env: %s to %s", k, v)
         env[k] = v
     return subprocess.Popen(command,
                             stdout=tmpout,

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestActionQueue.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestActionQueue.py b/slider-agent/src/test/python/agent/TestActionQueue.py
index 8071ee8..1a0b2cf 100644
--- a/slider-agent/src/test/python/agent/TestActionQueue.py
+++ b/slider-agent/src/test/python/agent/TestActionQueue.py
@@ -37,6 +37,7 @@ from mock.mock import patch, MagicMock, call
 from CustomServiceOrchestrator import CustomServiceOrchestrator
 from PythonExecutor import PythonExecutor
 from CommandStatusDict import CommandStatusDict
+from AgentToggleLogger import AgentToggleLogger
 
 
 class TestActionQueue(TestCase):
@@ -45,6 +46,7 @@ class TestActionQueue(TestCase):
     sys.stdout = out
     # save original open() method for later use
     self.original_open = open
+    self.agentToggleLogger = AgentToggleLogger("info")
 
 
   def tearDown(self):
@@ -104,7 +106,7 @@ class TestActionQueue(TestCase):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
     config = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
     actionQueue.start()
     time.sleep(3)
     actionQueue.stop()
@@ -118,7 +120,7 @@ class TestActionQueue(TestCase):
   def test_process_command(self,
                            execute_command_mock, print_exc_mock):
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller)
+    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller, self.agentToggleLogger)
     execution_command = {
       'commandType': ActionQueue.EXECUTION_COMMAND,
     }
@@ -167,7 +169,7 @@ class TestActionQueue(TestCase):
                                   status_update_callback):
     CustomServiceOrchestrator_mock.return_value = None
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller)
+    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller, self.agentToggleLogger)
 
     requestComponentStatus_mock.return_value = {'exitcode': 'dummy report'}
     actionQueue.execute_status_command(self.status_command)
@@ -193,7 +195,7 @@ class TestActionQueue(TestCase):
     csoMocks[0].status_commands_stdout = None
     csoMocks[0].status_commands_stderr = None
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller)
+    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller, self.agentToggleLogger)
 
     runCommand_mock.return_value = {'configurations': {}}
     actionQueue.execute_status_command(self.status_command_with_config)
@@ -212,7 +214,7 @@ class TestActionQueue(TestCase):
   def test_process_command2(self, execute_status_command_mock,
                            execute_command_mock, print_exc_mock):
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller)
+    actionQueue = ActionQueue(AgentConfig("", ""), dummy_controller, self.agentToggleLogger)
     execution_command = {
       'commandType': ActionQueue.EXECUTION_COMMAND,
     }
@@ -294,7 +296,7 @@ class TestActionQueue(TestCase):
     resolve_script_path_mock.return_value = "abc.py"
 
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
     unfreeze_flag = threading.Event()
     python_execution_result_dict = {
       'stdout': 'out',

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestAgentToggleLogger.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestAgentToggleLogger.py b/slider-agent/src/test/python/agent/TestAgentToggleLogger.py
new file mode 100644
index 0000000..fc07f30
--- /dev/null
+++ b/slider-agent/src/test/python/agent/TestAgentToggleLogger.py
@@ -0,0 +1,60 @@
+#!/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 unittest
+from AgentToggleLogger import AgentToggleLogger
+import logging
+
+class TestAgentToggleLogger(unittest.TestCase):
+  def setUp(self):
+    self.agentToggleLogger = AgentToggleLogger("info")
+
+  def test_adjustLogLevel(self):
+    from ActionQueue import ActionQueue
+    # log level is set to info here (during initialization)
+    # run an execution command first
+    self.agentToggleLogger.adjustLogLevelAtStart(ActionQueue.EXECUTION_COMMAND)
+    assert self.agentToggleLogger.logLevel == "info"
+    self.agentToggleLogger.adjustLogLevelAtEnd(ActionQueue.EXECUTION_COMMAND)
+    assert self.agentToggleLogger.logLevel == "info"
+
+    # run a status command now
+    self.agentToggleLogger.adjustLogLevelAtStart(ActionQueue.STATUS_COMMAND)
+    assert self.agentToggleLogger.logLevel == "info"
+    self.agentToggleLogger.adjustLogLevelAtEnd(ActionQueue.STATUS_COMMAND)
+    assert self.agentToggleLogger.logLevel == "debug"
+
+    # run a status command again
+    self.agentToggleLogger.adjustLogLevelAtStart(ActionQueue.STATUS_COMMAND)
+    assert self.agentToggleLogger.logLevel == "debug"
+    self.agentToggleLogger.adjustLogLevelAtEnd(ActionQueue.STATUS_COMMAND)
+    assert self.agentToggleLogger.logLevel == "debug"
+
+    # now an execution command shows up
+    self.agentToggleLogger.adjustLogLevelAtStart(ActionQueue.EXECUTION_COMMAND)
+    assert self.agentToggleLogger.logLevel == "info"
+    self.agentToggleLogger.adjustLogLevelAtEnd(ActionQueue.EXECUTION_COMMAND)
+    assert self.agentToggleLogger.logLevel == "info"
+
+
+if __name__ == "__main__":
+  logging.basicConfig(format='%(asctime)s %(message)s',level=logging.DEBUG)
+  unittest.main()
+

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestController.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestController.py b/slider-agent/src/test/python/agent/TestController.py
index 4070783..2d4a2bb 100644
--- a/slider-agent/src/test/python/agent/TestController.py
+++ b/slider-agent/src/test/python/agent/TestController.py
@@ -32,6 +32,7 @@ from AgentConfig import AgentConfig
 from mock.mock import patch, MagicMock, call, Mock
 import logging
 from threading import Event
+from AgentToggleLogger import AgentToggleLogger
 
 class TestController(unittest.TestCase):
 
@@ -56,7 +57,8 @@ class TestController(unittest.TestCase):
     self.controller = Controller.Controller(config)
     self.controller.netutil.MINIMUM_INTERVAL_BETWEEN_HEARTBEATS = 0.1
     self.controller.netutil.HEARTBEAT_NOT_IDDLE_INTERVAL_SEC = 0.1
-    self.controller.actionQueue = ActionQueue.ActionQueue(config, self.controller)
+    self.agentToggleLogger = AgentToggleLogger("info")
+    self.controller.actionQueue = ActionQueue.ActionQueue(config, self.controller, self.agentToggleLogger)
 
 
   @patch("json.dumps")

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestCustomServiceOrchestrator.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestCustomServiceOrchestrator.py b/slider-agent/src/test/python/agent/TestCustomServiceOrchestrator.py
index 92bc043..7135e5b 100644
--- a/slider-agent/src/test/python/agent/TestCustomServiceOrchestrator.py
+++ b/slider-agent/src/test/python/agent/TestCustomServiceOrchestrator.py
@@ -35,7 +35,7 @@ from mock.mock import MagicMock, patch
 import StringIO
 import sys
 from socket import socket
-
+from AgentToggleLogger import AgentToggleLogger
 
 class TestCustomServiceOrchestrator(TestCase):
 
@@ -45,6 +45,7 @@ class TestCustomServiceOrchestrator(TestCase):
     sys.stdout = out
     # generate sample config
     tmpdir = tempfile.gettempdir()
+    self.agentToggleLogger = AgentToggleLogger("info")
 
 
   @patch("hostname.public_hostname")
@@ -79,7 +80,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     isfile_mock.return_value = True
     self.assertEquals(command['hostname'], "host1")
     # Test dumping EXECUTION_COMMAND
@@ -100,7 +101,7 @@ class TestCustomServiceOrchestrator(TestCase):
     json_file = orchestrator.dump_command_to_json(command, {})
     self.assertTrue(os.path.exists(json_file))
     self.assertTrue(os.path.getsize(json_file) > 0)
-    self.assertEqual(oct(os.stat(json_file).st_mode & 0777), '0600')
+    self.assertEqual(oct(os.stat(json_file).st_mode & 0777), '0644')
     self.assertTrue(json_file.endswith("status_command.json"))
     os.unlink(json_file)
     # Testing side effect of dump_command_to_json
@@ -142,7 +143,7 @@ class TestCustomServiceOrchestrator(TestCase):
 
     resolve_script_path_mock.return_value = "/basedir/scriptpath"
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     # normal run case
     run_file_mock.return_value = {
       'stdout': 'sss',
@@ -228,7 +229,7 @@ class TestCustomServiceOrchestrator(TestCase):
 
     resolve_script_path_mock.return_value = "/basedir/scriptpath"
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     # normal run case
     run_file_mock.return_value = {
       'stdout': 'sss',
@@ -257,7 +258,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     ret = orchestrator.allocate_port(10)
     self.assertEqual(ret, 10)
 
@@ -275,7 +276,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     socket_getsockname_mock.return_value = [100, 101]
     ret = orchestrator.allocate_port(10)
     self.assertEqual(ret, 101)
@@ -291,7 +292,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     socket_getsockname_mock.return_value = [100, 102]
     ret = orchestrator.allocate_port()
     self.assertEqual(ret, 102)
@@ -308,7 +309,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
 
     is_port_available_mock.return_value = False
     allocate_port_mock.side_effect = [101, 102, 103, 104, 105, 106]
@@ -336,7 +337,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
 
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
 
     is_port_available_mock.return_value = True
     ret = orchestrator.allocate_ports("${A.ALLOCATED_PORT}{DEFAULT_1005}", "${A.ALLOCATED_PORT}")
@@ -409,7 +410,7 @@ class TestCustomServiceOrchestrator(TestCase):
 
     resolve_script_path_mock.return_value = "/basedir/scriptpath"
     dummy_controller = MagicMock()
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     # normal run case
     run_file_mock.return_value = {
       'stdout': 'sss',
@@ -459,7 +460,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getWorkRootPath.return_value = tempdir
     config.getLogPath.return_value = tempdir
 
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     # Test alive case
     runCommand_mock.return_value = {
       "exitcode": 0
@@ -486,7 +487,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getLogPath.return_value = tempdir
     mock_allocate_ports.return_value = "10023"
 
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     command = {}
     command['componentName'] = "HBASE_MASTER"
     command['configurations'] = {}
@@ -540,7 +541,7 @@ class TestCustomServiceOrchestrator(TestCase):
     config.getWorkRootPath.return_value = tempWorkDir
     config.getLogPath.return_value = tempdir
 
-    orchestrator = CustomServiceOrchestrator(config, dummy_controller)
+    orchestrator = CustomServiceOrchestrator(config, dummy_controller, self.agentToggleLogger)
     port = orchestrator.allocate_port()
     self.assertFalse(port == -1)
     self.assertTrue(port > 0)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestHeartbeat.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestHeartbeat.py b/slider-agent/src/test/python/agent/TestHeartbeat.py
index b012218..9898444 100644
--- a/slider-agent/src/test/python/agent/TestHeartbeat.py
+++ b/slider-agent/src/test/python/agent/TestHeartbeat.py
@@ -31,12 +31,15 @@ import StringIO
 import sys
 import logging
 from Controller import State
+from AgentToggleLogger import AgentToggleLogger
+
 
 class TestHeartbeat(TestCase):
   def setUp(self):
     # disable stdout
     out = StringIO.StringIO()
     sys.stdout = out
+    self.agentToggleLogger = AgentToggleLogger("info")
 
 
   def tearDown(self):
@@ -48,8 +51,8 @@ class TestHeartbeat(TestCase):
     config = AgentConfig("", "")
     config.set('agent', 'prefix', 'tmp')
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
-    heartbeat = Heartbeat(actionQueue, config)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
+    heartbeat = Heartbeat(actionQueue, config, self.agentToggleLogger)
     result = heartbeat.build({}, 100)
     print "Heartbeat: " + str(result)
     self.assertEquals(result['hostname'] != '', True,
@@ -74,7 +77,7 @@ class TestHeartbeat(TestCase):
     config = AgentConfig("", "")
     config.set('agent', 'prefix', 'tmp')
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
     result_mock.return_value = {
       'reports': [{'status': 'IN_PROGRESS',
                    'stderr': 'Read from /tmp/errors-3.txt',
@@ -144,7 +147,7 @@ class TestHeartbeat(TestCase):
         {'status': 'UNHEALTHY', 'componentName': 'HBASE_MASTER', 'reportResult' : False},
       ],
     }
-    heartbeat = Heartbeat(actionQueue, config)
+    heartbeat = Heartbeat(actionQueue, config, self.agentToggleLogger)
     # State.STARTED results in agentState to be set to 4 (enum order)
     hb = heartbeat.build({}, 10)
     hb['hostname'] = 'hostname'
@@ -182,7 +185,7 @@ class TestHeartbeat(TestCase):
     config = AgentConfig("", "")
     config.set('agent', 'prefix', 'tmp')
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
     result_mock.return_value = {
       'reports': [{'status': 'IN_PROGRESS',
                    'stderr': 'Read from /tmp/errors-3.txt',
@@ -198,7 +201,7 @@ class TestHeartbeat(TestCase):
       ],
       'componentStatus': []
       }
-    heartbeat = Heartbeat(actionQueue, config)
+    heartbeat = Heartbeat(actionQueue, config, self.agentToggleLogger)
 
     commandResult = {}
     hb = heartbeat.build(commandResult, 10)
@@ -219,7 +222,7 @@ class TestHeartbeat(TestCase):
     config = AgentConfig("", "")
     config.set('agent', 'prefix', 'tmp')
     dummy_controller = MagicMock()
-    actionQueue = ActionQueue(config, dummy_controller)
+    actionQueue = ActionQueue(config, dummy_controller, self.agentToggleLogger)
     result_mock.return_value = {
       'reports': [{'status': 'COMPLETED',
                    'stderr': 'Read from /tmp/errors-3.txt',
@@ -235,7 +238,7 @@ class TestHeartbeat(TestCase):
       ],
       'componentStatus': []
     }
-    heartbeat = Heartbeat(actionQueue, config)
+    heartbeat = Heartbeat(actionQueue, config, self.agentToggleLogger)
 
     commandResult = {}
     hb = heartbeat.build(commandResult, 10)

http://git-wip-us.apache.org/repos/asf/incubator-slider/blob/3111cbad/slider-agent/src/test/python/agent/TestPythonExecutor.py
----------------------------------------------------------------------
diff --git a/slider-agent/src/test/python/agent/TestPythonExecutor.py b/slider-agent/src/test/python/agent/TestPythonExecutor.py
index 1b12a0a..b4c0cfc 100644
--- a/slider-agent/src/test/python/agent/TestPythonExecutor.py
+++ b/slider-agent/src/test/python/agent/TestPythonExecutor.py
@@ -30,9 +30,11 @@ import unittest
 from PythonExecutor import PythonExecutor
 from AgentConfig import AgentConfig
 from mock.mock import MagicMock, patch
-
+from AgentToggleLogger import AgentToggleLogger
 
 class TestPythonExecutor(TestCase):
+  def setUp(self):
+    self.agentToggleLogger = AgentToggleLogger("info")
 
   @patch("shell.kill_process_with_children")
   def test_watchdog_1(self, kill_process_with_children_mock):
@@ -40,7 +42,7 @@ class TestPythonExecutor(TestCase):
     Tests whether watchdog works
     """
     subproc_mock = self.Subprocess_mockup()
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
     _, tmpoutfile = tempfile.mkstemp()
     _, tmperrfile = tempfile.mkstemp()
     _, tmpstrucout = tempfile.mkstemp()
@@ -69,7 +71,7 @@ class TestPythonExecutor(TestCase):
     Tries to catch false positive watchdog invocations
     """
     subproc_mock = self.Subprocess_mockup()
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
     _, tmpoutfile = tempfile.mkstemp()
     _, tmperrfile = tempfile.mkstemp()
     _, tmpstrucout = tempfile.mkstemp()
@@ -100,7 +102,7 @@ class TestPythonExecutor(TestCase):
   @patch("os.environ.copy")
   def test_set_env_values(self, os_env_copy_mock, subprocess_mock, open_mock):
     actual_vars = {"someOther" : "value1"}
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
     environment_vars = [("PYTHONPATH", "a:b")]
     os_env_copy_mock.return_value = actual_vars
     executor.run_file("script.pynot", ["a","b"], "", "", 10, "", "INFO", True, environment_vars)
@@ -108,7 +110,7 @@ class TestPythonExecutor(TestCase):
 
   def test_execution_results(self):
     subproc_mock = self.Subprocess_mockup()
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
     _, tmpoutfile = tempfile.mkstemp()
     _, tmperrfile = tempfile.mkstemp()
     _, tmpstroutfile = tempfile.mkstemp()
@@ -130,7 +132,7 @@ class TestPythonExecutor(TestCase):
 
 
   def test_is_successfull(self):
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
 
     executor.python_process_has_been_killed = False
     self.assertTrue(executor.isSuccessfull(0))
@@ -142,7 +144,7 @@ class TestPythonExecutor(TestCase):
 
 
   def test_python_command(self):
-    executor = PythonExecutor("/tmp", AgentConfig("", ""))
+    executor = PythonExecutor("/tmp", AgentConfig("", ""), self.agentToggleLogger)
     command = executor.python_command("script", ["script_param1"])
     self.assertEqual(4, len(command))
     self.assertTrue("python" in command[0].lower(), "Looking for python in %s" % (command[0].lower()))