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/04/18 21:23:17 UTC

aurora git commit: Adding a flag to control whether the executor runs health checks as the job role's user

Repository: aurora
Updated Branches:
  refs/heads/master f4a08459c -> a6197316c


Adding a flag to control whether the executor runs health checks as the job role's user

Bugs closed: AURORA-1666

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


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

Branch: refs/heads/master
Commit: a6197316c20f4b759453a6f2015fef7a1956de5f
Parents: f4a0845
Author: Dmitriy Shirchenko <ca...@gmail.com>
Authored: Mon Apr 18 12:23:11 2016 -0700
Committer: Bill Farner <wf...@apache.org>
Committed: Mon Apr 18 12:23:11 2016 -0700

----------------------------------------------------------------------
 RELEASE-NOTES.md                                |  3 ++
 .../executor/bin/thermos_executor_main.py       | 11 +++++-
 .../aurora/executor/common/health_checker.py    | 18 ++++++----
 .../executor/common/test_health_checker.py      | 36 ++++++++++++++++++++
 4 files changed, 61 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/aurora/blob/a6197316/RELEASE-NOTES.md
----------------------------------------------------------------------
diff --git a/RELEASE-NOTES.md b/RELEASE-NOTES.md
index 99d261b..2068d9c 100644
--- a/RELEASE-NOTES.md
+++ b/RELEASE-NOTES.md
@@ -5,6 +5,9 @@
 
 - Added a new optional [Apache Curator](https://curator.apache.org/) backend for performing
   scheduler leader election. You can enable this with the new `-zk_use_curator` scheduler argument.
+- Adding --setuid-health-checks flag to control whether the executor runs health checks as the job's
+  role's user.
+
 
 0.13.0
 ------

http://git-wip-us.apache.org/repos/asf/aurora/blob/a6197316/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
----------------------------------------------------------------------
diff --git a/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py b/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
index 40a0cd6..0b3c38c 100644
--- a/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
+++ b/src/main/python/apache/aurora/executor/bin/thermos_executor_main.py
@@ -145,6 +145,15 @@ app.add_option(
 
 
 app.add_option(
+    '--setuid-health-checks',
+    dest='setuid_health_checks',
+    action="store_true",
+    help='If False, does not run shell health checks as job\'s role\'s user',
+    default=True
+)
+
+
+app.add_option(
      "--preserve_env",
      dest="preserve_env",
      default=False,
@@ -181,7 +190,7 @@ def initialize(options):
 
   # status providers:
   status_providers = [
-      HealthCheckerProvider(),
+      HealthCheckerProvider(setuid_health_checks=options.setuid_health_checks),
       ResourceManagerProvider(checkpoint_root=checkpoint_root)
   ]
 

http://git-wip-us.apache.org/repos/asf/aurora/blob/a6197316/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 d8675be..88b629e 100644
--- a/src/main/python/apache/aurora/executor/common/health_checker.py
+++ b/src/main/python/apache/aurora/executor/common/health_checker.py
@@ -208,6 +208,9 @@ class HealthChecker(StatusChecker):
 
 class HealthCheckerProvider(StatusCheckerProvider):
 
+  def __init__(self, setuid_health_checks=True):
+    self.setuid_health_checks = setuid_health_checks
+
   @staticmethod
   def interpolate_cmd(task, cmd):
     """
@@ -244,14 +247,17 @@ class HealthCheckerProvider(StatusCheckerProvider):
         task=assigned_task,
         cmd=shell_command
       )
-
-      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)
+      # If we do not want user which is job's role to execute the health shell check.
+      if self.setuid_health_checks:
+        pw_entry = pwd.getpwnam(assigned_task.task.job.role)
+        def demote_to_job_role_user():
+          os.setgid(pw_entry.pw_gid)
+          os.setuid(pw_entry.pw_uid)
+      else:
+        demote_to_job_role_user = None
 
       shell_signaler = ShellHealthCheck(cmd=interpolated_command,
-        preexec_fn=demote_to_user,
+        preexec_fn=demote_to_job_role_user,
         timeout_secs=timeout_secs)
       a_health_checker = lambda: shell_signaler()
     else:

http://git-wip-us.apache.org/repos/asf/aurora/blob/a6197316/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 4ab7a2f..ff39e86 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
@@ -282,6 +282,42 @@ class TestHealthCheckerProvider(unittest.TestCase):
     assert hct_max_fail == max_consecutive_failures
     mock_getpwnam.assert_called_once_with(task_config.job.role)
 
+  @mock.patch('pwd.getpwnam')
+  def test_from_assigned_task_shell_no_demotion(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(
+                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,
+                    timeout_secs=timeout_secs,
+                )
+            ).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'
+    health_checker = HealthCheckerProvider(setuid_health_checks=False).from_assigned_task(
+      assigned_task, None)
+    assert health_checker.threaded_health_checker.interval == interval_secs
+    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
+    # Should not be trying to access role's user info.
+    assert not mock_getpwnam.called
+
   def test_interpolate_cmd(self):
     """Making sure thermos.ports[foo] gets correctly substituted with assignedPorts info."""
     interval_secs = 17