浏览代码

celery.utils.info's functions moved to other places.

.humanize_seconds -> celery.utils.timeutils
.textindent -> celery.utils
.format_queues -> Celery().amqp.format_queues
.get_broker_info -> Celery().amqp.get_broker_info
.format_broker_info -> Celery().amqp.format_broker_info
Ask Solem 14 年之前
父节点
当前提交
2808bc3e9f

+ 1 - 1
celery/app/__init__.py

@@ -87,8 +87,8 @@ default_app = App()
 
 
 if os.environ.get("CELERY_TRACE_APP"):
-    from multiprocessing import current_process
     def app_or_default(app=None):
+        from multiprocessing import current_process
         if app is None:
             if current_process()._name == "MainProcess":
                 raise Exception("DEFAULT APP")

+ 41 - 2
celery/app/amqp.py

@@ -6,11 +6,16 @@ from carrot import messaging
 
 from celery import routes
 from celery import signals
-from celery.utils import gen_unique_id, mitemgetter
+from celery.utils import gen_unique_id, mitemgetter, textindent
 
 
 MSG_OPTIONS = ("mandatory", "priority", "immediate",
                "routing_key", "serializer", "delivery_mode")
+QUEUE_FORMAT = """
+. %(name)s -> exchange:%(exchange)s (%(exchange_type)s) \
+binding:%(binding_key)s
+"""
+BROKER_FORMAT = "%(carrot_backend)s://%(userid)s@%(host)s%(port)s%(vhost)s"
 
 get_msg_options = mitemgetter(*MSG_OPTIONS)
 extract_msg_options = lambda d: dict(zip(MSG_OPTIONS, get_msg_options(d)))
@@ -20,6 +25,8 @@ _queues_declared = False
 _exchanges_declared = set()
 
 
+
+
 class TaskPublisher(messaging.Publisher):
     auto_declare = False
 
@@ -63,7 +70,7 @@ class TaskPublisher(messaging.Publisher):
         if taskset_id:
             message_data["taskset"] = taskset_id
 
-        # FIXME (carrot Publisher.send needs to accept exchange argument)
+        # custom exchange passed, need to declare it.
         if exchange and exchange not in _exchanges_declared:
             exchange_type = exchange_type or self.exchange_type
             self.backend.exchange_declare(exchange=exchange,
@@ -177,3 +184,35 @@ class AMQP(object):
                                      backend=cset.backend, **queue_options)
             cset.consumers.append(consumer)
         return cset
+
+    def format_queues(self, queues, indent=0):
+        """Format routing table into string for log dumps."""
+        format = lambda **queue: QUEUE_FORMAT.strip() % queue
+        info = "\n".join(format(name=name, **config)
+                                for name, config in queues.items())
+        return textindent(info, indent=indent)
+
+    def get_broker_info(self):
+        broker_connection = self.app.broker_connection()
+        carrot_backend = broker_connection.backend_cls
+        if carrot_backend and not isinstance(carrot_backend, str):
+            carrot_backend = carrot_backend.__name__
+        carrot_backend = carrot_backend or "amqp"
+
+        port = broker_connection.port or \
+                    broker_connection.get_backend_cls().default_port
+        port = port and ":%s" % port or ""
+
+        vhost = broker_connection.virtual_host
+        if not vhost.startswith("/"):
+            vhost = "/" + vhost
+
+        return {"carrot_backend": carrot_backend,
+                "userid": broker_connection.userid,
+                "host": broker_connection.hostname,
+                "port": port,
+                "vhost": vhost}
+
+    def format_broker_info(self, info=None):
+        """Get message broker connection info string for log dumps."""
+        return BROKER_FORMAT % self.get_broker_info()

+ 1 - 1
celery/apps/beat.py

@@ -74,7 +74,7 @@ class Beat(object):
 
     def startup_info(self):
         return STARTUP_INFO_FMT % {
-            "conninfo": info.format_broker_info(app=self.app),
+            "conninfo": self.app.amqp.format_broker_info(),
             "logfile": self.logfile or "@stderr",
             "loglevel": LOG_LEVELS[self.loglevel],
             "schedule": self.schedule,

+ 2 - 2
celery/apps/worker.py

@@ -153,8 +153,8 @@ class Worker(object):
             tasklist = self.tasklist(include_builtins=include_builtins)
 
         return STARTUP_INFO_FMT % {
-            "conninfo": info.format_broker_info(app=self.app),
-            "queues": info.format_queues(self.queues, indent=8),
+            "conninfo": self.app.amqp.format_broker_info(),
+            "queues": self.app.amqp.format_queues(self.queues, indent=8),
             "concurrency": self.concurrency,
             "loglevel": LOG_LEVELS[self.loglevel],
             "logfile": self.logfile or "[stderr]",

+ 1 - 1
celery/beat.py

@@ -15,7 +15,7 @@ from celery.app import app_or_default
 from celery.log import SilenceRepeated
 from celery.schedules import maybe_schedule
 from celery.utils import instantiate
-from celery.utils.info import humanize_seconds
+from celery.utils.timeutils import humanize_seconds
 
 
 class SchedulingError(Exception):

+ 1 - 1
celery/bin/camqadm.py

@@ -342,7 +342,7 @@ class AMQPAdmin(object):
         if conn:
             conn.close()
         self.say("-> connecting to %s." % (
-                    info.format_broker_info(app=self.app)))
+                    self.app.amqp.format_broker_info(), ))
         conn = self.app.broker_connection()
         conn.connect()
         self.say("-> connected.")

+ 8 - 6
celery/tests/test_utils_info.py

@@ -1,6 +1,8 @@
 import unittest2 as unittest
 
-from celery.utils import info
+from celery.app import default_app
+from celery.utils import textindent
+from celery.utils.timeutils import humanize_seconds
 
 RANDTEXT = """\
 The quick brown
@@ -47,16 +49,16 @@ class TestInfo(unittest.TestCase):
              (0, "now"))
 
         for seconds, human in t:
-            self.assertEqual(info.humanize_seconds(seconds), human)
+            self.assertEqual(humanize_seconds(seconds), human)
 
-        self.assertEqual(info.humanize_seconds(4, prefix="about "),
+        self.assertEqual(humanize_seconds(4, prefix="about "),
                           "about 4.00 seconds")
 
     def test_textindent(self):
-        self.assertEqual(info.textindent(RANDTEXT, 4), RANDTEXT_RES)
+        self.assertEqual(textindent(RANDTEXT, 4), RANDTEXT_RES)
 
     def test_format_queues(self):
-        self.assertEqual(info.format_queues(QUEUES), QUEUE_FORMAT)
+        self.assertEqual(default_app.amqp.format_queues(QUEUES), QUEUE_FORMAT)
 
     def test_broker_info(self):
-        info.format_broker_info()
+        default_app.amqp.format_broker_info()

+ 5 - 0
celery/utils/__init__.py

@@ -391,3 +391,8 @@ def isatty(fh):
     # Fixes bug with mod_wsgi:
     #   mod_wsgi.Log object has no attribute isatty.
     return getattr(fh, "isatty", None) and fh.isatty()
+
+
+def textindent(t, indent=0):
+        """Indent text."""
+        return "\n".join(" " * indent + p for p in t.split("\n"))

+ 0 - 67
celery/utils/info.py

@@ -1,67 +0,0 @@
-import math
-
-from celery.app import app_or_default
-
-QUEUE_FORMAT = """
-. %(name)s -> exchange:%(exchange)s (%(exchange_type)s) \
-binding:%(binding_key)s
-"""
-BROKER_FORMAT = "%(carrot_backend)s://%(userid)s@%(host)s%(port)s%(vhost)s"
-
-TIME_UNITS = (("day", 60 * 60 * 24, lambda n: int(math.ceil(n))),
-              ("hour", 60 * 60, lambda n: int(math.ceil(n))),
-              ("minute", 60, lambda n: int(math.ceil(n))),
-              ("second", 1, lambda n: "%.2f" % n))
-
-
-def humanize_seconds(secs, prefix=""):
-    """Show seconds in human form, e.g. 60 is "1 minute", 7200 is "2
-    hours"."""
-    for unit, divider, formatter in TIME_UNITS:
-        if secs >= divider:
-            w = secs / divider
-            punit = w > 1 and unit+"s" or unit
-            return "%s%s %s" % (prefix, formatter(w), punit)
-    return "now"
-
-
-def textindent(t, indent=0):
-    """Indent text."""
-    return "\n".join(" " * indent + p for p in t.split("\n"))
-
-
-def format_queues(queues, indent=0):
-    """Format routing table into string for log dumps."""
-    format = lambda **queue: QUEUE_FORMAT.strip() % queue
-    info = "\n".join(format(name=name, **config)
-                            for name, config in queues.items())
-    return textindent(info, indent=indent)
-
-
-def get_broker_info(app=None):
-    app = app_or_default(app)
-    broker_connection = app.broker_connection()
-
-    carrot_backend = broker_connection.backend_cls
-    if carrot_backend and not isinstance(carrot_backend, str):
-        carrot_backend = carrot_backend.__name__
-    carrot_backend = carrot_backend or "amqp"
-
-    port = broker_connection.port or \
-                broker_connection.get_backend_cls().default_port
-    port = port and ":%s" % port or ""
-
-    vhost = broker_connection.virtual_host
-    if not vhost.startswith("/"):
-        vhost = "/" + vhost
-
-    return {"carrot_backend": carrot_backend,
-            "userid": broker_connection.userid,
-            "host": broker_connection.hostname,
-            "port": port,
-            "vhost": vhost}
-
-
-def format_broker_info(info=None, app=None):
-    """Get message broker connection info string for log dumps."""
-    return BROKER_FORMAT % get_broker_info(app=app)

+ 20 - 0
celery/utils/timeutils.py

@@ -1,3 +1,5 @@
+import math
+
 from datetime import datetime, timedelta
 
 from carrot.utils import partition
@@ -11,6 +13,11 @@ RATE_MODIFIER_MAP = {"s": lambda n: n,
 
 HAVE_TIMEDELTA_TOTAL_SECONDS = hasattr(timedelta, "total_seconds")
 
+TIME_UNITS = (("day", 60 * 60 * 24, lambda n: int(math.ceil(n))),
+              ("hour", 60 * 60, lambda n: int(math.ceil(n))),
+              ("minute", 60, lambda n: int(math.ceil(n))),
+              ("second", 1, lambda n: "%.2f" % n))
+
 
 def timedelta_seconds(delta):
     """Convert :class:`datetime.timedelta` to seconds.
@@ -96,3 +103,16 @@ def weekday(name):
     except KeyError:
         # Show original day name in exception, instead of abbr.
         raise KeyError(name)
+
+
+def humanize_seconds(secs, prefix=""):
+    """Show seconds in human form, e.g. 60 is "1 minute", 7200 is "2
+    hours"."""
+    for unit, divider, formatter in TIME_UNITS:
+        if secs >= divider:
+            w = secs / divider
+            punit = w > 1 and unit+"s" or unit
+            return "%s%s %s" % (prefix, formatter(w), punit)
+    return "now"
+
+