Browse Source

Optionally re-raise exceptions in tasks to preserve full traceback.

Patch for issue #253 in GitHub:
https://github.com/ask/celery/issues#issue/253
Christopher Peplin 14 years ago
parent
commit
82219f291d
6 changed files with 36 additions and 6 deletions
  1. 1 0
      celery/app/defaults.py
  2. 1 0
      celery/conf.py
  3. 10 5
      celery/execute/trace.py
  4. 6 1
      celery/task/base.py
  5. 7 0
      celery/tests/test_task.py
  6. 11 0
      docs/configuration.rst

+ 1 - 0
celery/app/defaults.py

@@ -69,6 +69,7 @@ NAMESPACES = {
         "DEFAULT_EXCHANGE_TYPE": Option("direct"),
         "DEFAULT_DELIVERY_MODE": Option(2, type="string"),
         "EAGER_PROPAGATES_EXCEPTIONS": Option(False, type="bool"),
+        "EAGER_RERAISES_EXCEPTIONS": Option(False, type="bool"),
         "EVENT_SERIALIZER": Option("json"),
         "IMPORTS": Option((), type="tuple"),
         "IGNORE_RESULT": Option(False, type="bool"),

+ 1 - 0
celery/conf.py

@@ -15,6 +15,7 @@ conf = app_or_default().conf
 
 ALWAYS_EAGER = conf.CELERY_ALWAYS_EAGER
 EAGER_PROPAGATES_EXCEPTIONS = conf.CELERY_EAGER_PROPAGATES_EXCEPTIONS
+EAGER_RERAISES_EXCEPTIONS = conf.CELERY_EAGER_RERAISES_EXCEPTIONS
 RESULT_BACKEND = conf.CELERY_RESULT_BACKEND
 CACHE_BACKEND = conf.CELERY_CACHE_BACKEND
 CACHE_BACKEND_OPTIONS = conf.CELERY_CACHE_BACKEND_OPTIONS

+ 10 - 5
celery/execute/trace.py

@@ -22,7 +22,7 @@ class TraceInfo(object):
             self.strtb = "\n".join(traceback.format_exception(*exc_info))
 
     @classmethod
-    def trace(cls, fun, args, kwargs):
+    def trace(cls, fun, args, kwargs, reraise=False):
         """Trace the execution of a function, calling the appropiate callback
         if the function raises retry, an failure or returned successfully."""
         try:
@@ -32,8 +32,12 @@ class TraceInfo(object):
         except RetryTaskError, exc:
             return cls(states.RETRY, retval=exc, exc_info=sys.exc_info())
         except Exception, exc:
+            if reraise:
+                raise
             return cls(states.FAILURE, retval=exc, exc_info=sys.exc_info())
         except:
+            if reraise:
+                raise
             # For Python2.4 where raising strings are still allowed.
             return cls(states.FAILURE, retval=None, exc_info=sys.exc_info())
 
@@ -57,13 +61,13 @@ class TaskTrace(object):
     def __call__(self):
         return self.execute()
 
-    def execute(self):
+    def execute(self, reraise=False):
         self.task.request.update(self.request, args=self.args,
                                                kwargs=self.kwargs)
         signals.task_prerun.send(sender=self.task, task_id=self.task_id,
                                  task=self.task, args=self.args,
                                  kwargs=self.kwargs)
-        retval = self._trace()
+        retval = self._trace(reraise)
 
         signals.task_postrun.send(sender=self.task, task_id=self.task_id,
                                   task=self.task, args=self.args,
@@ -71,8 +75,9 @@ class TaskTrace(object):
         self.task.request.clear()
         return retval
 
-    def _trace(self):
-        trace = TraceInfo.trace(self.task, self.args, self.kwargs)
+    def _trace(self, reraise=False):
+        trace = TraceInfo.trace(self.task, self.args, self.kwargs,
+                reraise=reraise)
         self.status = trace.status
         self.strtb = trace.strtb
         self.handle_after_return(trace.status, trace.retval,

+ 6 - 1
celery/task/base.py

@@ -554,6 +554,9 @@ class BaseTask(object):
         :keyword throw: Re-raise task exceptions.  Defaults to
                         the :setting:`CELERY_EAGER_PROPAGATES_EXCEPTIONS`
                         setting.
+        :keyword reraise: Re-raise task exceptions with a full traceback. 
+                        Defaults to the
+                        :setting:`CELERY_EAGER_PROPAGATES_EXCEPTIONS` setting.
 
         :rtype :class:`celery.result.EagerResult`:
 
@@ -564,6 +567,8 @@ class BaseTask(object):
         retries = options.get("retries", 0)
         throw = self.app.either("CELERY_EAGER_PROPAGATES_EXCEPTIONS",
                                 options.pop("throw", None))
+        reraise = self.app.either("CELERY_EAGER_RERAISES_EXCEPTIONS",
+                                options.pop("reraise", None))
 
         # Make sure we get the task instance, not class.
         task = tasks[self.name]
@@ -590,7 +595,7 @@ class BaseTask(object):
 
         trace = TaskTrace(task.name, task_id, args, kwargs,
                           task=task, request=request)
-        retval = trace.execute()
+        retval = trace.execute(reraise=reraise)
         if isinstance(retval, ExceptionInfo):
             if throw:
                 raise retval.exception

+ 7 - 0
celery/tests/test_task.py

@@ -440,6 +440,13 @@ class TestTaskApply(unittest.TestCase):
         finally:
             RaisingTask.app.conf.CELERY_EAGER_PROPAGATES_EXCEPTIONS = False
 
+    def test_apply_with_CELERY_EAGER_RERAISES_EXCEPTIONS(self):
+        RaisingTask.app.conf.CELERY_EAGER_RERAISES_EXCEPTIONS = True
+        try:
+            self.assertRaises(KeyError, RaisingTask.apply)
+        finally:
+            RaisingTask.app.conf.CELERY_EAGER_RERAISES_EXCEPTIONS = False
+
     def test_apply(self):
         IncrementCounterTask.count = 0
 

+ 11 - 0
docs/configuration.rst

@@ -622,6 +622,17 @@ If this is :const:`True`, eagerly executed tasks (using `.apply`, or with
 
 It's the same as always running `apply` with `throw=True`.
 
+.. setting:: CELERY_EAGER_RERAISES_EXCEPTIONS
+
+CELERY_EAGER_RERAISES_EXCEPTIONS
+~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+
+If this is :const:`True`, eagerly executed tasks (using `.apply`, or with
+:setting:`CELERY_ALWAYS_EAGER` on), will re-raise exceptions with a full
+application traceback.
+
+It's the same as always running `apply` with `reraise_exceptions=True`.
+
 .. setting:: CELERY_IGNORE_RESULT
 
 CELERY_IGNORE_RESULT