You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by wf...@apache.org on 2016/03/31 23:48:17 UTC

aurora git commit: Execute shell-based health checks as the task user.

Repository: aurora
Updated Branches:
  refs/heads/master 193f17ee4 -> 17ade117b


Execute shell-based health checks as the task user.

Bugs closed: AURORA-1641

Reviewed at https://reviews.apache.org/r/45506/


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

Branch: refs/heads/master
Commit: 17ade117b8171d4ef4be62fd4b617a024658551d
Parents: 193f17e
Author: Bill Farner <wf...@apache.org>
Authored: Thu Mar 31 14:48:15 2016 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Thu Mar 31 14:48:15 2016 -0700

----------------------------------------------------------------------
 .../apache/aurora/common/health_check/shell.py  | 23 ++++--
 .../aurora/executor/common/health_checker.py    | 14 +++-
 .../aurora/common/health_check/test_shell.py    | 79 +++++++++-----------
 .../executor/common/test_health_checker.py      |  7 +-
 .../http/http_example_bad_healthcheck.aurora    | 21 ++++--
 .../sh/org/apache/aurora/e2e/test_end_to_end.sh |  2 +-
 6 files changed, 79 insertions(+), 67 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/main/python/apache/aurora/common/health_check/shell.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/common/health_check/shell.py b/src/main/python/apache/aurora/common/health_check/shell.py
index 6cb7dfc..3575082 100644
--- a/src/main/python/apache/aurora/common/health_check/shell.py
+++ b/src/main/python/apache/aurora/common/health_check/shell.py
@@ -26,16 +26,19 @@ else:
 
 class ShellHealthCheck(object):
 
-  def __init__(self, cmd, timeout_secs=None):
+  def __init__(self, cmd, preexec_fn=None, timeout_secs=None):
     """
     Initialize with the commmand we would like to call.
     :param cmd: Command to execute that is expected to have a 0 return code on success.
     :type cmd: str
+    :param preexec_fn: Callable to invoke just before the child shell process is executed.
+    :type preexec_fn: callable
     :param timeout_secs: Timeout in seconds.
     :type timeout_secs: int
     """
-    self.cmd = cmd
-    self.timeout_secs = timeout_secs
+    self._cmd = cmd
+    self._preexec_fn = preexec_fn
+    self._timeout_secs = timeout_secs
 
   def __call__(self):
     """
@@ -45,14 +48,18 @@ class ShellHealthCheck(object):
     :rtype tuple:
     """
     try:
-      subprocess.check_call(self.cmd, timeout=self.timeout_secs, shell=True)
+      subprocess.check_call(
+          self._cmd,
+          timeout=self._timeout_secs,
+          shell=True,
+          preexec_fn=self._preexec_fn)
       return True, None
     except subprocess.CalledProcessError as reason:
       # The command didn't return a 0 so provide reason for failure.
       return False, str(reason)
+    except subprocess.TimeoutExpired:
+      return False, 'Health check timed out.'
     except OSError as e:
-      reason = 'OSError: {error}'.format(error=e.strerror)
-      return False, reason
+      return False, 'OSError: %s' % e.strerror
     except ValueError:
-      reason = 'Invalid commmand.'
-      return False, reason
+      return False, 'Invalid commmand.'

http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/main/python/apache/aurora/executor/common/health_checker.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/common/health_checker.py b/src/main/python/apache/aurora/executor/common/health_checker.py
index 28fd3ec..d8675be 100644
--- a/src/main/python/apache/aurora/executor/common/health_checker.py
+++ b/src/main/python/apache/aurora/executor/common/health_checker.py
@@ -13,6 +13,7 @@
 #
 
 import os.path
+import pwd
 import threading
 import time
 import traceback
@@ -243,10 +244,15 @@ class HealthCheckerProvider(StatusCheckerProvider):
         task=assigned_task,
         cmd=shell_command
       )
-      shell_signaler = ShellHealthCheck(
-        cmd=interpolated_command,
-        timeout_secs=timeout_secs,
-      )
+
+      pw_entry = pwd.getpwnam(assigned_task.task.job.role)
+      def demote_to_user():
+        os.setgid(pw_entry.pw_gid)
+        os.setuid(pw_entry.pw_uid)
+
+      shell_signaler = ShellHealthCheck(cmd=interpolated_command,
+        preexec_fn=demote_to_user,
+        timeout_secs=timeout_secs)
       a_health_checker = lambda: shell_signaler()
     else:
       portmap = resolve_ports(mesos_task, assigned_task.assignedPorts)

http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/test/python/apache/aurora/common/health_check/test_shell.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/common/health_check/test_shell.py b/src/test/python/apache/aurora/common/health_check/test_shell.py
index 7026af8..011464c 100644
--- a/src/test/python/apache/aurora/common/health_check/test_shell.py
+++ b/src/test/python/apache/aurora/common/health_check/test_shell.py
@@ -31,65 +31,56 @@ else:
 
 class TestHealthChecker(unittest.TestCase):
 
-  @mock.patch('subprocess32.check_call')
-  def test_health_check_ok(self, mock_sub):
-    timeout = 30
-    cmd = 'success cmd'
-    shell = ShellHealthCheck(cmd, timeout_secs=timeout)
+  @mock.patch('subprocess32.check_call', autospec=True)
+  def test_health_check_ok(self, mock_popen):
+    shell = ShellHealthCheck('cmd', timeout_secs=30)
     success, msg = shell()
     self.assertTrue(success)
     self.assertIsNone(msg)
-    mock_sub.assert_called_once_with(
-      'success cmd',
-      timeout=30,
-      shell=True,
-    )
-
-  @mock.patch('subprocess32.check_call')
-  def test_health_check_failed(self, mock_sub):
-    timeout = 30
+    mock_popen.assert_called_once_with('cmd', shell=True, timeout=30, preexec_fn=mock.ANY)
+
+  @mock.patch('subprocess32.check_call', autospec=True)
+  def test_health_check_failed(self, mock_popen):
     # Fail due to command returning a non-0 exit status.
-    mock_sub.side_effect = subprocess.CalledProcessError(1, 'failed')
-    cmd = 'cmd to fail'
-    shell = ShellHealthCheck(cmd, timeout_secs=timeout)
+    mock_popen.side_effect = subprocess.CalledProcessError(1, 'failed')
+
+    shell = ShellHealthCheck('cmd', timeout_secs=30)
     success, msg = shell()
-    mock_sub.assert_called_once_with(
-      'cmd to fail',
-      timeout=30,
-      shell=True,
-    )
+    mock_popen.assert_called_once_with('cmd', shell=True, timeout=30, preexec_fn=mock.ANY)
+
     self.assertFalse(success)
     self.assertEqual(msg, "Command 'failed' returned non-zero exit status 1")
 
-  @mock.patch('subprocess32.check_call')
-  def test_health_check_os_error(self, mock_sub):
-    timeout = 30
+  @mock.patch('subprocess32.check_call', autospec=True)
+  def test_health_check_timeout(self, mock_popen):
+    # Fail due to command returning a non-0 exit status.
+    mock_popen.side_effect = subprocess.TimeoutExpired('failed', timeout=30)
+
+    shell = ShellHealthCheck('cmd', timeout_secs=30)
+    success, msg = shell()
+    mock_popen.assert_called_once_with('cmd', shell=True, timeout=30, preexec_fn=mock.ANY)
+
+    self.assertFalse(success)
+    self.assertEqual(msg, 'Health check timed out.')
+
+  @mock.patch('subprocess32.check_call', autospec=True)
+  def test_health_check_os_error(self, mock_popen):
     # Fail due to command not existing.
-    mock_sub.side_effect = OSError(1, 'failed')
-    cmd = 'cmd to not exist'
-    shell = ShellHealthCheck(cmd, timeout_secs=timeout)
+    mock_popen.side_effect = OSError(1, 'failed')
+
+    shell = ShellHealthCheck('cmd', timeout_secs=30)
     success, msg = shell()
-    mock_sub.assert_called_once_with(
-      'cmd to not exist',
-      timeout=30,
-      shell=True,
-    )
+    mock_popen.assert_called_once_with('cmd', shell=True, timeout=30, preexec_fn=mock.ANY)
     self.assertFalse(success)
     self.assertEqual(msg, 'OSError: failed')
 
-  @mock.patch('subprocess32.check_call')
-  def test_health_check_value_error(self, mock_sub):
-    timeout = 30
+  @mock.patch('subprocess32.check_call', autospec=True)
+  def test_health_check_value_error(self, mock_popen):
     # Invalid commmand passed in raises ValueError.
-    mock_sub.side_effect = ValueError('Could not read command.')
-    cmd = 'defensive cmd'
+    mock_popen.side_effect = ValueError('Could not read command.')
     timeout = 10
-    shell = ShellHealthCheck(cmd, timeout_secs=timeout)
+    shell = ShellHealthCheck('cmd', timeout_secs=timeout)
     success, msg = shell()
-    mock_sub.assert_called_once_with(
-      'defensive cmd',
-      timeout=10,
-      shell=True,
-    )
+    mock_popen.assert_called_once_with('cmd', shell=True, timeout=10, preexec_fn=mock.ANY)
     self.assertFalse(success)
     self.assertEqual(msg, 'Invalid commmand.')

http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/test/python/apache/aurora/executor/common/test_health_checker.py
----------------------------------------------------------------------
diff --git a/src/test/python/apache/aurora/executor/common/test_health_checker.py b/src/test/python/apache/aurora/executor/common/test_health_checker.py
index 19c4f76..b0ce81a 100644
--- a/src/test/python/apache/aurora/executor/common/test_health_checker.py
+++ b/src/test/python/apache/aurora/executor/common/test_health_checker.py
@@ -39,7 +39,7 @@ from apache.aurora.executor.common.sandbox import SandboxInterface
 
 from .fixtures import HELLO_WORLD, MESOS_JOB
 
-from gen.apache.aurora.api.ttypes import AssignedTask, ExecutorConfig, TaskConfig
+from gen.apache.aurora.api.ttypes import AssignedTask, ExecutorConfig, JobKey, TaskConfig
 
 
 class TestHealthChecker(unittest.TestCase):
@@ -246,13 +246,15 @@ class TestHealthCheckerProvider(unittest.TestCase):
     assert health_checker.threaded_health_checker.interval == interval_secs
     assert health_checker.threaded_health_checker.initial_interval == initial_interval_secs
 
-  def test_from_assigned_task_shell(self):
+  @mock.patch('pwd.getpwnam')
+  def test_from_assigned_task_shell(self, mock_getpwnam):
     interval_secs = 17
     initial_interval_secs = 3
     max_consecutive_failures = 2
     timeout_secs = 5
     shell_config = ShellHealthChecker(shell_command='failed command')
     task_config = TaskConfig(
+        job=JobKey(role='role', environment='env', name='name'),
         executorConfig=ExecutorConfig(
             name='thermos-generic',
             data=MESOS_JOB(
@@ -276,6 +278,7 @@ class TestHealthCheckerProvider(unittest.TestCase):
     assert health_checker.threaded_health_checker.initial_interval == initial_interval_secs
     hct_max_fail = health_checker.threaded_health_checker.max_consecutive_failures
     assert hct_max_fail == max_consecutive_failures
+    mock_getpwnam.assert_called_once_with(task_config.job.role)
 
   def test_interpolate_cmd(self):
     """Making sure thermos.ports[foo] gets correctly substituted with assignedPorts info."""

http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora b/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
index c709b03..0534c9e 100644
--- a/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
+++ b/src/test/sh/org/apache/aurora/e2e/http/http_example_bad_healthcheck.aurora
@@ -18,7 +18,8 @@ DEFAULT_CMD = 'cp /vagrant/src/test/sh/org/apache/aurora/e2e/http_example.py .'
 
 run_server = Process(
   name = 'run_server',
-  cmdline = 'python http_example.py {{thermos.ports[http]}}')
+  cmdline = 'python http_example.py {{thermos.ports[http]}}'
+)
 
 stage_server = Process(
   name = 'stage_server',
@@ -29,18 +30,22 @@ test_task = Task(
   name = 'http_example',
   resources = Resources(cpu=0.5, ram=32*MB, disk=64*MB),
   processes = [stage_server, run_server],
-  constraints = order(stage_server, run_server))
+  constraints = order(stage_server, run_server)
+)
 
 update_config = UpdateConfig(watch_secs=10, batch_size=2)
 # "I am going to fail" config.
 shell_config = ShellHealthChecker(
-    shell_command='grep foo'
-    )
+  # This shell validates two things:
+  # 1. the shell command is not run as a privileged user
+  # 2. a failed shell command results in a health check failure
+  shell_command='rm -f /etc/passwd'
+)
 health_check_config = HealthCheckConfig(
-    health_checker=HealthCheckerConfig(shell=shell_config),
-    initial_interval_secs=5,
-    interval_secs=1,
-    )
+  health_checker=HealthCheckerConfig(shell=shell_config),
+  initial_interval_secs=5,
+  interval_secs=1,
+)
 
 job = Service(
   cluster = 'devcluster',

http://git-wip-us.apache.org/repos/asf/aurora/blob/17ade117/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
----------------------------------------------------------------------
diff --git a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
index e1c12bb..77a4c36 100755
--- a/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
+++ b/src/test/sh/org/apache/aurora/e2e/test_end_to_end.sh
@@ -202,7 +202,7 @@ test_update_fail() {
       | tail -n +2 | awk '{print $2}')
   # || is so that we don't return an EXIT so that `trap collect_result` doesn't get triggered.
   aurora update wait $_jobkey $_update_id || echo $?
-  # MAKING SURE WE ROLLED BACK.
+  # Making sure we rolled back.
   local status=$(aurora update info $_jobkey $_update_id | grep 'Current status' | awk '{print $NF}')
   if [[ $status != "ROLLED_BACK" ]]; then
     echo "Update should have completed in ROLLED_BACK state due to failed healthcheck."