Pārlūkot izejas kodu

Added an option so the default behaviour is to not report running state. It's not needed in most cases

Rune Halvorsen 15 gadi atpakaļ
vecāks
revīzija
2c8f53c452
3 mainītis faili ar 14 papildinājumiem un 1 dzēšanām
  1. 2 0
      celery/conf.py
  2. 10 0
      celery/task/base.py
  3. 2 1
      celery/worker/job.py

+ 2 - 0
celery/conf.py

@@ -62,6 +62,7 @@ _DEFAULTS = {
     "CELERY_EVENT_ROUTING_KEY": "celeryevent",
     "CELERY_RESULT_EXCHANGE": "celeryresults",
     "CELERY_MAX_CACHED_RESULTS": 5000,
+    "CELERY_TRACK_RUNNING": False,
 }
 
 _DEPRECATION_FMT = """
@@ -91,6 +92,7 @@ CELERY_CACHE_BACKEND = _get("CELERY_CACHE_BACKEND")
 TASK_SERIALIZER = _get("CELERY_TASK_SERIALIZER")
 TASK_RESULT_EXPIRES = _get("CELERY_TASK_RESULT_EXPIRES")
 IGNORE_RESULT = _get("CELERY_IGNORE_RESULT")
+TRACK_RUNNING = _get("CELERY_TRACK_RUNNING")
 # Make sure TASK_RESULT_EXPIRES is a timedelta.
 if isinstance(TASK_RESULT_EXPIRES, int):
     TASK_RESULT_EXPIRES = timedelta(seconds=TASK_RESULT_EXPIRES)

+ 10 - 0
celery/task/base.py

@@ -152,6 +152,15 @@ class Task(object):
         If ``True`` the task is automatically registered in the task
         registry, which is the default behaviour.
 
+    .. attribute:: track_running
+        If ``True`` the task will report its status as "running" to the
+        broker when the task is executed by a worker.
+        The default value is ``False`` as the normal behaviour is to not
+        report that level of granularity. Tasks are either pending, finished,
+        or waiting to be retried. Having a "running" status can be useful for
+        when there are long running tasks and there is a need to report which
+        task is currently running.
+
 
     The resulting class is callable, which if called will apply the
     :meth:`run` method.
@@ -178,6 +187,7 @@ class Task(object):
     backend = default_backend
     exchange_type = conf.DEFAULT_EXCHANGE_TYPE
     delivery_mode = conf.DEFAULT_DELIVERY_MODE
+    track_running = conf.TRACK_RUNNING
 
     MaxRetriesExceededError = MaxRetriesExceededError
 

+ 2 - 1
celery/worker/job.py

@@ -100,7 +100,8 @@ class WorkerTaskTrace(TaskTrace):
         """Execute, trace and store the result of the task."""
         self.loader.on_task_init(self.task_id, self.task)
         self.task.backend.process_cleanup()
-        self.task.backend.mark_as_running(self.task_id)
+        if self.task.track_running:
+            self.task.backend.mark_as_running(self.task_id)
         return super(WorkerTaskTrace, self).execute()
 
     def handle_success(self, retval, *args):