Pārlūkot izejas kodu

worker/beat: --uid + --gid now works even without --detach

Ask Solem 11 gadi atpakaļ
vecāks
revīzija
6c1aac8b9f

+ 3 - 26
celery/apps/worker.py

@@ -24,10 +24,11 @@ from billiard import current_process
 from kombu.utils.encoding import safe_str
 
 from celery import VERSION_BANNER, platforms, signals
+from celery.app import trace
 from celery.exceptions import CDeprecationWarning, SystemTerminate
 from celery.five import string, string_t
 from celery.loaders.app import AppLoader
-from celery.app import trace
+from celery.platforms import check_privileges
 from celery.utils import cry, isatty
 from celery.utils.imports import qualname
 from celery.utils.log import get_logger, in_sighandler, set_in_sighandler
@@ -40,23 +41,6 @@ logger = get_logger(__name__)
 is_jython = sys.platform.startswith('java')
 is_pypy = hasattr(sys, 'pypy_version_info')
 
-C_FORCE_ROOT = os.environ.get('C_FORCE_ROOT', False)
-
-ROOT_DISALLOWED = """\
-Running a worker with superuser privileges when the
-worker accepts messages serialized with pickle is a very bad idea!
-
-If you really want to continue then you have to set the C_FORCE_ROOT
-environment variable (but please think about this before you do).
-"""
-
-ROOT_DISCOURAGED = """\
-You are running the worker with superuser privileges, which is
-absolutely not recommended!
-
-Please specify a different user using the -u option.
-"""
-
 W_PICKLE_DEPRECATED = """
 Starting from version 3.2 Celery will refuse to accept pickle by default.
 
@@ -133,6 +117,7 @@ class Worker(WorkController):
             sender=self.hostname, instance=self,
             conf=self.app.conf, options=kwargs,
         )
+        check_privileges(self.app.conf.CELERY_ACCEPT_CONTENT)
 
     def on_after_init(self, purge=False, no_color=None,
                       redirect_stdouts=None, redirect_stdouts_level=None,
@@ -170,14 +155,6 @@ class Worker(WorkController):
             sender=self.hostname, instance=self, conf=self.app.conf,
         )
 
-        if getattr(os, 'getuid', None) and os.getuid() == 0:
-            accept_encoding = self.app.conf.CELERY_ACCEPT_CONTENT
-            if ('pickle' in accept_encoding or
-                    'application/x-python-serialize' in accept_encoding):
-                if not C_FORCE_ROOT:
-                    raise RuntimeError(ROOT_DISALLOWED)
-            warnings.warn(RuntimeWarning(ROOT_DISCOURAGED))
-
         if not self.app.conf.value_set_for('CELERY_ACCEPT_CONTENT'):
             warnings.warn(CDeprecationWarning(W_PICKLE_DEPRECATED))
 

+ 2 - 1
celery/bin/beat.py

@@ -42,7 +42,7 @@ from __future__ import absolute_import
 
 from functools import partial
 
-from celery.platforms import detached
+from celery.platforms import detached, maybe_drop_privileges
 
 from celery.bin.base import Command, Option, daemon_options
 
@@ -65,6 +65,7 @@ class beat(Command):
 
     def run(self, detach=False, logfile=None, pidfile=None, uid=None,
             gid=None, umask=None, working_directory=None, **kwargs):
+        maybe_drop_privileges(uid=uid, gid=gid)
         workdir = working_directory
         kwargs.pop('app', None)
         beat = partial(self.app.Beat,

+ 3 - 1
celery/bin/worker.py

@@ -138,6 +138,7 @@ from celery import concurrency
 from celery.bin.base import Command, Option, daemon_options
 from celery.bin.celeryd_detach import detached_celeryd
 from celery.five import string_t
+from celery.platforms import maybe_drop_privileges
 from celery.utils.log import LOG_LEVELS, mlevel
 
 __all__ = ['worker', 'main']
@@ -180,7 +181,8 @@ class worker(Command):
             raise SystemExit(0)
 
     def run(self, hostname=None, pool_cls=None, loglevel=None,
-            app=None, **kwargs):
+            app=None, uid=None, gid=None, **kwargs):
+        maybe_drop_privileges(uid=uid, gid=gid)
         # Pools like eventlet/gevent needs to patch libs as early
         # as possible.
         pool_cls = (concurrency.get_implementation(pool_cls) or

+ 58 - 4
celery/platforms.py

@@ -16,6 +16,7 @@ import os
 import platform as _platform
 import signal as _signal
 import sys
+import warnings
 
 from collections import namedtuple
 
@@ -64,6 +65,27 @@ Seems we're already running? (pid: {1})"""
 
 _range = namedtuple('_range', ('start', 'stop'))
 
+C_FORCE_ROOT = os.environ.get('C_FORCE_ROOT', False)
+
+ROOT_DISALLOWED = """\
+Running a worker with superuser privileges when the
+worker accepts messages serialized with pickle is a very bad idea!
+
+If you really want to continue then you have to set the C_FORCE_ROOT
+environment variable (but please think about this before you do).
+
+User information: uid={uid} euid={euid} gid={gid} egid={egid}
+"""
+
+ROOT_DISCOURAGED = """\
+You are running the worker with superuser privileges, which is
+absolutely not recommended!
+
+Please specify a different user using the -u option.
+
+User information: uid={uid} euid={euid} gid={gid} egid={egid}
+"""
+
 
 def pyimplementation():
     """Return string identifying the current Python implementation."""
@@ -354,9 +376,7 @@ def detached(logfile=None, pidfile=None, uid=None, gid=None, umask=0,
     workdir = os.getcwd() if workdir is None else workdir
 
     signals.reset('SIGCLD')  # Make sure SIGCLD is using the default handler.
-    if not os.geteuid():
-        # no point trying to setuid unless we're root.
-        maybe_drop_privileges(uid=uid, gid=gid)
+    maybe_drop_privileges(uid=uid, gid=gid)
 
     def after_chdir_do():
         # Since without stderr any errors will be silently suppressed,
@@ -473,6 +493,10 @@ def maybe_drop_privileges(uid=None, gid=None):
     If only GID is specified, only the group is changed.
 
     """
+    if os.geteuid():
+        # no point trying to setuid unless we're root.
+        if not os.getuid():
+            raise AssertionError('contact support')
     uid = uid and parse_uid(uid)
     gid = gid and parse_gid(gid)
 
@@ -501,10 +525,14 @@ def maybe_drop_privileges(uid=None, gid=None):
         else:
             raise RuntimeError(
                 'non-root user able to restore privileges after setuid.')
-
     else:
         gid and setgid(gid)
 
+    if uid and (not os.getuid()) and not (os.geteuid()):
+        raise AssertionError('Still root uid after drop privileges!')
+    if gid and (not os.getgid()) and not (os.getegid()):
+        raise AssertionError('Still root gid after drop privileges!')
+
 
 class Signals(object):
     """Convenience interface to :mod:`signals`.
@@ -701,3 +729,29 @@ def ignore_errno(*errnos, **kwargs):
             raise
         if exc.errno not in errnos:
             raise
+
+
+def check_privileges(accept_content):
+    uid = os.getuid() if hasattr(os, 'getuid') else 65535
+    gid = os.getgid() if hasattr(os, 'getgid') else 65535
+    euid = os.geteuid() if hasattr(os, 'geteuid') else 65535
+    egid = os.getegid() if hasattr(os, 'getegid') else 65535
+
+    if hasattr(os, 'fchown'):
+        if not all(hasattr(os, attr)
+                   for attr in ['getuid', 'getgid', 'geteuid', 'getegid']):
+            raise AssertionError('suspicious platform, contact support')
+
+    if not uid or not gid or not euid or not egid:
+        if ('pickle' in accept_content or
+                'application/x-python-serialize' in accept_content):
+            if not C_FORCE_ROOT:
+                try:
+                    print(ROOT_DISALLOWED.format(
+                        uid=uid, euid=euid, gid=gid, egid=egid,
+                    ), file=sys.stderr)
+                finally:
+                    os._exit(1)
+        warnings.warn(RuntimeWarning(ROOT_DISCOURAGED.format(
+            uid=uid, euid=euid, gid=gid, egid=egid,
+        )))

+ 8 - 6
celery/tests/bin/test_worker.py

@@ -272,7 +272,8 @@ class test_Worker(WorkerAppCase):
         self.assertEqual(worker1.loglevel, 0xFFFF)
 
     @disable_stdouts
-    def test_warns_if_running_as_privileged_user(self):
+    @patch('os._exit')
+    def test_warns_if_running_as_privileged_user(self, _exit):
         app = self.app
         if app.IS_WINDOWS:
             raise SkipTest('Not applicable on Windows')
@@ -280,10 +281,11 @@ class test_Worker(WorkerAppCase):
         with patch('os.getuid') as getuid:
             getuid.return_value = 0
             self.app.conf.CELERY_ACCEPT_CONTENT = ['pickle']
-            with self.assertRaises(RuntimeError):
-                worker = self.Worker(app=self.app)
-                worker.on_start()
-            cd.C_FORCE_ROOT = True
+            worker = self.Worker(app=self.app)
+            worker.on_start()
+            _exit.assert_called_with(1)
+            from celery import platforms
+            platforms.C_FORCE_ROOT = True
             try:
                 with self.assertWarnsRegex(
                         RuntimeWarning,
@@ -291,7 +293,7 @@ class test_Worker(WorkerAppCase):
                     worker = self.Worker(app=self.app)
                     worker.on_start()
             finally:
-                cd.C_FORCE_ROOT = False
+                platforms.C_FORCE_ROOT = False
             self.app.conf.CELERY_ACCEPT_CONTENT = ['json']
             with self.assertWarnsRegex(
                     RuntimeWarning,