浏览代码

CELERYD_LOG_COLOR is now prepared by the Logging class, not post_config_merge

Ask Solem 15 年之前
父节点
当前提交
f250ce053a
共有 7 个文件被更改,包括 34 次插入20 次删除
  1. 1 1
      Changelog
  2. 1 6
      celery/app/base.py
  3. 1 2
      celery/apps/beat.py
  4. 1 2
      celery/apps/worker.py
  5. 21 4
      celery/log.py
  6. 1 2
      celery/tests/test_app.py
  7. 8 3
      celery/tests/test_concurrency_base.py

+ 1 - 1
Changelog

@@ -100,7 +100,7 @@ Important Notes
 
 * The magic keyword arguments are now available as `task.request`
 
-    This is called *the context*.  Thread-local storage is used, and the
+    This is called *the context*.  Using thread-local storage the
     context contains state that is related to the current request.
 
     It is mutable and you can add custom attributes that will only be seen

+ 1 - 6
celery/app/base.py

@@ -16,7 +16,7 @@ from datetime import timedelta
 from celery import routes
 from celery.app.defaults import DEFAULTS
 from celery.datastructures import ConfigurationView
-from celery.utils import instantiate, isatty, cached_property, maybe_promise
+from celery.utils import instantiate, cached_property, maybe_promise
 from celery.utils.functional import wraps
 
 
@@ -218,11 +218,6 @@ class BaseApp(object):
                     "exchange_type": c["CELERY_DEFAULT_EXCHANGE_TYPE"],
                     "binding_key": c["CELERY_DEFAULT_ROUTING_KEY"]}}
         c["CELERY_ROUTES"] = routes.prepare(c.get("CELERY_ROUTES") or {})
-        if c.get("CELERYD_LOG_COLOR") is None:
-            c["CELERYD_LOG_COLOR"] = not c["CELERYD_LOG_FILE"] and \
-                                        isatty(sys.stderr)
-        if self.IS_WINDOWS:  # windows console doesn't support ANSI colors
-            c["CELERYD_LOG_COLOR"] = False
         if isinstance(c["CELERY_TASK_RESULT_EXPIRES"], int):
             c["CELERY_TASK_RESULT_EXPIRES"] = timedelta(
                     seconds=c["CELERY_TASK_RESULT_EXPIRES"])

+ 1 - 2
celery/apps/beat.py

@@ -8,7 +8,6 @@ from celery import beat
 from celery import platforms
 from celery.app import app_or_default
 from celery.utils import get_full_cls_name, LOG_LEVELS
-from celery.utils import term
 from celery.utils.timeutils import humanize_seconds
 
 STARTUP_INFO_FMT = """
@@ -38,7 +37,7 @@ class Beat(object):
         self.scheduler_cls = scheduler_cls or app.conf.CELERYBEAT_SCHEDULER
         self.max_interval = max_interval
         self.socket_timeout = socket_timeout
-        self.colored = term.colored(enabled=app.conf.CELERYD_LOG_COLOR)
+        self.colored = app.log.colored(self.logfile)
         self.redirect_stdouts = (redirect_stdouts or
                                  app.conf.CELERY_REDIRECT_STDOUTS)
         self.redirect_stdouts_level = (redirect_stdouts_level or

+ 1 - 2
celery/apps/worker.py

@@ -14,7 +14,6 @@ from celery import signals
 from celery.app import app_or_default
 from celery.exceptions import ImproperlyConfigured, SystemTerminate
 from celery.utils import get_full_cls_name, LOG_LEVELS, cry
-from celery.utils import term
 from celery.worker import WorkController
 
 
@@ -82,7 +81,7 @@ class Worker(object):
             self.autoscale = [int(max_c), min_c and int(min_c) or 0]
         self._isatty = sys.stdout.isatty()
 
-        self.colored = term.colored(enabled=app.conf.CELERYD_LOG_COLOR)
+        self.colored = app.log.colored(self.logfile)
 
         if isinstance(self.use_queues, basestring):
             self.use_queues = self.use_queues.split(",")

+ 21 - 4
celery/log.py

@@ -10,7 +10,7 @@ from multiprocessing import util as mputil
 
 from celery import signals
 from celery.app import app_or_default
-from celery.utils import LOG_LEVELS
+from celery.utils import LOG_LEVELS, isatty
 from celery.utils.compat import LoggerAdapter
 from celery.utils.patch import ensure_process_aware_logger
 from celery.utils.term import colored
@@ -63,6 +63,20 @@ class Logging(object):
         self.app = app
         self.loglevel = self.app.conf.CELERYD_LOG_LEVEL
         self.format = self.app.conf.CELERYD_LOG_FORMAT
+        self.colorize = self.app.conf.CELERYD_LOG_COLOR
+
+    def supports_color(self, logfile=None):
+        if self.app.IS_WINDOWS:
+            # Windows does not support ANSI color codes.
+            return False
+        if self.colorize is None:
+            # Only use color if there is no active log file
+            # and stderr is an actual terminal.
+            return logfile is None and isatty(sys.stderr)
+        return self.colorize
+
+    def colored(self, logfile=None):
+        return colored(enabled=self.supports_color(logfile))
 
     def get_task_logger(self, loglevel=None, name=None):
         logger = logging.getLogger(name or "celery.task.default")
@@ -74,7 +88,8 @@ class Logging(object):
             format=None, colorize=None, **kwargs):
         loglevel = loglevel or self.loglevel
         format = format or self.format
-        colorize = self.app.either("CELERYD_LOG_COLOR", colorize)
+        if colorize is None:
+            colorize = self.supports_color(logfile)
 
         if self.__class__._setup:
             return
@@ -136,7 +151,8 @@ class Logging(object):
         """
         loglevel = loglevel or self.loglevel
         format = format or self.format
-        colorize = self.app.either("CELERYD_LOG_COLOR", colorize)
+        if colorize is None:
+            colorize = self.supports_color(logfile)
 
         if not root or self.app.conf.CELERYD_HIJACK_ROOT_LOGGER:
             return self._setup_logger(self.get_default_logger(loglevel, name),
@@ -157,7 +173,8 @@ class Logging(object):
         """
         loglevel = loglevel or self.loglevel
         format = format or self.format
-        colorize = self.app.either("CELERYD_LOG_COLOR", colorize)
+        if colorize is None:
+            colorize = self.supports_color(logfile)
 
         if task_kwargs is None:
             task_kwargs = {}

+ 1 - 2
celery/tests/test_app.py

@@ -122,8 +122,7 @@ class test_App(unittest.TestCase):
 
     def test_Windows_log_color_disabled(self):
         self.app.IS_WINDOWS = True
-        self.app.config_from_object(Object(CELERYD_LOG_COLOR=True))
-        self.assertFalse(self.app.conf.CELERYD_LOG_COLOR)
+        self.assertFalse(self.app.log.supports_color())
 
     def test_task_result_expires_converted_to_timedelta(self):
         self.app.config_from_object(Object(CELERY_TASK_RESULT_EXPIRES=100))

+ 8 - 3
celery/tests/test_concurrency_base.py

@@ -1,3 +1,5 @@
+import os
+
 from itertools import count
 
 from celery.concurrency.base import apply_target, BasePool
@@ -24,10 +26,13 @@ class test_BasePool(unittest.TestCase):
                      callback=gen_callback("callback"),
                      accept_callback=gen_callback("accept_callback"))
 
-        self.assertDictEqual(scratch,
-                             {"accept_callback": (0, ()),
+        self.assertDictContainsSubset({
                               "target": (1, (8, 16)),
-                              "callback": (2, (42, ))})
+                              "callback": (2, (42, ))}, scratch)
+        pa1 = scratch["accept_callback"]
+        self.assertEqual(0, pa1[0])
+        self.assertEqual(pa1[1][0], os.getpid())
+        self.assertTrue(pa1[1][1])
 
         # No accept callback
         scratch.clear()