You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@aurora.apache.org by se...@apache.org on 2017/04/05 09:25:30 UTC

aurora git commit: Fix Thermos Health Check for MesosContainerizer with `--nosetuid-health-checks`

Repository: aurora
Updated Branches:
  refs/heads/master 72046170d -> 7678d194f


Fix Thermos Health Check for MesosContainerizer with `--nosetuid-health-checks`

With MesosContainerizer, the health check is performed using a "mesos-containerizer
launch" process, but there is actually a code bug in the way of getting the user
under which to run the health check process:
https://github.com/apache/aurora/blob/master/src/main/python/apache/aurora/executor/common/health_checker.py#L370
```
health_check_user = (os.getusername() if self._nosetuid_health_checks
            else assigned_task.task.job.role)
```

If the scheduler is configured with `--nosetuid-health-checks` then "os.getusername()"
is executed, but the "os" python module does not present any "getusername()" function,
which leads the Thermos execution to abort as follow:
```
D0323 01:08:15.453372 16 aurora_executor.py:159] Task started.
E0323 01:08:15.571124 16 aurora_executor.py:121] Traceback (most recent call last):
File "apache/aurora/executor/aurora_executor.py", line 119, in _run
self._start_status_manager(driver, assigned_task)
File "apache/aurora/executor/aurora_executor.py", line 168, in _start_status_manager
status_checker = status_provider.from_assigned_task(assigned_task, self._sandbox)
File "apache/aurora/executor/common/health_checker.py", line 370, in from_assigned_task
health_check_user = (os.getusername() if self._nosetuid_health_checks
AttributeError: 'module' object has no attribute 'getusername'
```

Following the existing unit testing pattern from test_health_checker.py, a test case
was added to cover the `--nosetuid-health-checks` case for MesosContainerizer.

Bugs closed: AURORA-1909

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


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

Branch: refs/heads/master
Commit: 7678d194f918143d5e8d771796e7dfbaabc931e7
Parents: 7204617
Author: Charles Raimbert <cr...@gmail.com>
Authored: Wed Apr 5 11:25:03 2017 +0200
Committer: Stephan Erb <se...@apache.org>
Committed: Wed Apr 5 11:25:03 2017 +0200

----------------------------------------------------------------------
 .../aurora/executor/common/health_checker.py    |   3 +-
 .../executor/common/test_health_checker.py      | 185 ++++++++++++-------
 2 files changed, 120 insertions(+), 68 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/7678d194/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 5bb4768..e9e4129 100644
--- a/src/main/python/apache/aurora/executor/common/health_checker.py
+++ b/src/main/python/apache/aurora/executor/common/health_checker.py
@@ -12,6 +12,7 @@
 # limitations under the License.
 #
 
+import getpass
 import math
 import os
 import pwd
@@ -367,7 +368,7 @@ class HealthCheckerProvider(StatusCheckerProvider):
       # command within a mesos-containerizer invocation so that it's executed within that
       # filesystem.
       if sandbox.is_filesystem_image:
-        health_check_user = (os.getusername() if self._nosetuid_health_checks
+        health_check_user = (getpass.getuser() if self._nosetuid_health_checks
             else assigned_task.task.job.role)
         wrapped_cmd = wrap_with_mesos_containerizer(
             interpolated_command,

http://git-wip-us.apache.org/repos/asf/aurora/blob/7678d194/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 adf3ac0..1036221 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
@@ -46,6 +46,14 @@ from .fixtures import HELLO_WORLD, MESOS_JOB
 from gen.apache.aurora.api.ttypes import AssignedTask, ExecutorConfig, JobKey, TaskConfig
 
 
+FAKE_MESOS_CONTAINERIZER_BINARY = '''#!/bin/sh
+if [[ $# == 1 ]]; then
+  echo "command_info" >&2
+else
+  echo "$@"
+fi'''
+
+
 class TestHealthChecker(unittest.TestCase):
   def setUp(self):
     self._clock = ThreadedClock(0)
@@ -565,80 +573,123 @@ class TestHealthCheckerProvider(unittest.TestCase):
     # Should not be trying to access role's user info.
     assert not mock_getpwnam.called
 
+  def make_fake_mesos_containerizer(self, temp_dir):
+    # We use a fake version of the mesos-containerizer binary that just echoes out its args so
+    # we can assert on them in the process's output. Also imitates a failure when there are not
+    # enough arguments, this is used to find the version of the binary (by checking the failure
+    # message)
+    fake_mesos_containerizer_path = os.path.join(temp_dir, 'fake-mesos-containerizer')
+    with open(fake_mesos_containerizer_path, 'w') as fd:
+      fd.write(FAKE_MESOS_CONTAINERIZER_BINARY)
+    chmod_plus_x(fake_mesos_containerizer_path)
+    return fake_mesos_containerizer_path
+
   @mock.patch.dict(os.environ, {'MESOS_DIRECTORY': '/some/path'})
-  @mock.patch('pwd.getpwnam')
-  def test_from_assigned_task_shell_filesystem_image(self, mock_getpwnam):
-    interval_secs = 17
-    initial_interval_secs = 3
-    max_consecutive_failures = 2
-    min_consecutive_successes = 2
-    timeout_secs = 5
-    shell_config = ShellHealthChecker(shell_command='failed command')
+  @mock.patch('apache.aurora.executor.common.health_checker.ShellHealthCheck')
+  def test_from_assigned_task_shell_filesystem_image(self, mock_shell):
+    shell_config = ShellHealthChecker(shell_command='run health check')
     task_config = TaskConfig(
-            job=JobKey(role='role', environment='env', name='name'),
-            executorConfig=ExecutorConfig(
-                    name='thermos-generic',
-                    data=MESOS_JOB(
-                            task=HELLO_WORLD,
-                            health_check_config=HealthCheckConfig(
-                                    health_checker=HealthCheckerConfig(shell=shell_config),
-                                    interval_secs=interval_secs,
-                                    initial_interval_secs=initial_interval_secs,
-                                    max_consecutive_failures=max_consecutive_failures,
-                                    min_consecutive_successes=min_consecutive_successes,
-                                    timeout_secs=timeout_secs
-                            )
-                    ).json_dumps()
-            )
+      job=JobKey(role='role', environment='env', name='name'),
+      executorConfig=ExecutorConfig(
+        name='thermos-generic',
+        data=MESOS_JOB(
+          task=HELLO_WORLD,
+          health_check_config=HealthCheckConfig(
+            health_checker=HealthCheckerConfig(shell=shell_config),
+            timeout_secs=5
+          )
+        ).json_dumps()
+      )
     )
     assigned_task = AssignedTask(task=task_config, instanceId=1, assignedPorts={'foo': 9001})
     execconfig_data = json.loads(assigned_task.task.executorConfig.data)
-    assert execconfig_data[
-             'health_check_config']['health_checker']['shell']['shell_command'] == 'failed command'
+    assert execconfig_data['health_check_config'][
+             'health_checker']['shell']['shell_command'] == 'run health check'
 
     with temporary_dir() as td:
-      test_isolator_path = os.path.join(td, 'fake-mesos-containerier')
-      with open(test_isolator_path, 'w') as fd:
-        # We use a fake version of the mesos-containerizer binary that just echoes out its args so
-        # we can assert on them in the process's output. Also imitates a failure when there are not
-        # enough arguments, this is used to find the version of the binary (by checking the failure
-        # message)
-        fd.write('\n'.join([
-          '#!/bin/sh',
-          'if [[ $# == 1 ]]; then',
-          '  { echo "command_info" >&2; };',
-          'else',
-          '  echo "$@";',
-          'fi'
-        ]))
-
-        fd.close()
-
-        chmod_plus_x(test_isolator_path)
-
-        mock_sandbox = mock.Mock(spec_set=SandboxInterface)
-        type(mock_sandbox).root = mock.PropertyMock(return_value=td)
-        type(mock_sandbox).is_filesystem_image = mock.PropertyMock(return_value=True)
-
-        with mock.patch('apache.aurora.executor.common.health_checker.ShellHealthCheck') as shell:
-          HealthCheckerProvider(
-              nosetuid_health_checks=False,
-              mesos_containerizer_path=test_isolator_path).from_assigned_task(
-                  assigned_task,
-                  mock_sandbox)
-
-          class NotNone(object):
-            def __eq__(self, other):
-              return other is not None
-
-          assert shell.mock_calls == [
-              mock.call(
-                  raw_cmd='failed command',
-                  wrapped_cmd=NotNone(),
-                  preexec_fn=None,
-                  timeout_secs=5.0
-              )
-          ]
+      fake_mesos_containerizer_path = self.make_fake_mesos_containerizer(td)
+
+      mock_sandbox = mock.Mock(spec_set=SandboxInterface)
+      type(mock_sandbox).root = mock.PropertyMock(return_value=td)
+      type(mock_sandbox).is_filesystem_image = mock.PropertyMock(return_value=True)
+      container_root_path = os.path.join(td, 'container-root')
+      type(mock_sandbox).container_root = mock.PropertyMock(return_value=container_root_path)
+
+      HealthCheckerProvider(
+        nosetuid_health_checks=False,
+        mesos_containerizer_path=fake_mesos_containerizer_path
+      ).from_assigned_task(
+        assigned_task,
+        mock_sandbox
+      )
+
+      class ValidateWrapperCmd(object):
+        def __eq__(self, other):
+          assert "--user=role" in other
+          return True
+
+      assert mock_shell.mock_calls == [
+        mock.call(
+          raw_cmd='run health check',
+          wrapped_cmd=ValidateWrapperCmd(),
+          preexec_fn=None,
+          timeout_secs=5.0
+        )
+      ]
+
+  @mock.patch.dict(os.environ, {'MESOS_DIRECTORY': '/some/path'})
+  @mock.patch('getpass.getuser', return_value='user')
+  @mock.patch('apache.aurora.executor.common.health_checker.ShellHealthCheck')
+  def test_from_assigned_task_shell_filesystem_image_no_demotion(self, mock_shell, mock_getuser):
+    shell_config = ShellHealthChecker(shell_command='run health check')
+    task_config = TaskConfig(
+      job=JobKey(role='role', environment='env', name='name'),
+      executorConfig=ExecutorConfig(
+        name='thermos-generic',
+        data=MESOS_JOB(
+          task=HELLO_WORLD,
+          health_check_config=HealthCheckConfig(
+            health_checker=HealthCheckerConfig(shell=shell_config),
+            timeout_secs=5
+          )
+        ).json_dumps()
+      )
+    )
+    assigned_task = AssignedTask(task=task_config, instanceId=1, assignedPorts={'foo': 9001})
+    execconfig_data = json.loads(assigned_task.task.executorConfig.data)
+    assert execconfig_data['health_check_config'][
+             'health_checker']['shell']['shell_command'] == 'run health check'
+
+    with temporary_dir() as td:
+      fake_mesos_containerizer_path = self.make_fake_mesos_containerizer(td)
+
+      mock_sandbox = mock.Mock(spec_set=SandboxInterface)
+      type(mock_sandbox).root = mock.PropertyMock(return_value=td)
+      type(mock_sandbox).is_filesystem_image = mock.PropertyMock(return_value=True)
+      container_root_path = os.path.join(td, 'container-root')
+      type(mock_sandbox).container_root = mock.PropertyMock(return_value=container_root_path)
+
+      HealthCheckerProvider(
+        nosetuid_health_checks=True,
+        mesos_containerizer_path=fake_mesos_containerizer_path
+      ).from_assigned_task(
+        assigned_task,
+        mock_sandbox
+      )
+
+      class ValidateWrapperCmd(object):
+        def __eq__(self, other):
+          assert "--user=user" in other
+          return True
+
+      assert mock_shell.mock_calls == [
+        mock.call(
+          raw_cmd='run health check',
+          wrapped_cmd=ValidateWrapperCmd(),
+          preexec_fn=None,
+          timeout_secs=5.0
+        )
+      ]
 
   def test_interpolate_cmd(self):
     """Making sure thermos.ports[foo] gets correctly substituted with assignedPorts info."""