Browse Source

Merge branch '3.0'

Conflicts:
	Changelog
Ask Solem 12 years ago
parent
commit
c17856827b
6 changed files with 72 additions and 10 deletions
  1. 59 0
      Changelog
  2. 2 1
      celery/backends/base.py
  3. 5 1
      celery/bin/base.py
  4. 1 0
      celery/result.py
  5. 2 7
      celery/utils/serialization.py
  6. 3 1
      celery/worker/job.py

+ 59 - 0
Changelog

@@ -20,6 +20,65 @@ If you're looking for versions prior to 3.x you should see :ref:`history`.
 - `Task.apply_async` now supports timeout and soft_timeout arguments (Issue #802)
 - `App.control.Inspect.conf` can be used for inspecting worker configuration
 
+.. _version-3.0.6:
+
+3.0.6
+=====
+:release-date: 2012-09-XX XX:XX X.M BST
+
+- Now depends on billiard 2.7.3.12
+
+- Adds ``task_name`` attribute to ``EagerResult`` which is always
+  :const:`None` (Issue #907).
+
+- Old Task class in :mod:`celery.task` no longer accepts magic kwargs by
+  default (Issue #918).
+
+    A regression long ago disabled magic kwargs for these, and since
+    no one has complained about it we don't have any incentive to fix it now.
+
+- The ``inspect reserved`` control command did not work properly.
+
+- Should now play better with static analyzation tools by explicitly
+  specifying dynamically created attributes in the :mod:`celery` and
+  :mod:`celery.task` modules.
+
+- Terminating a task now results in
+  :exc:`~celery.exceptions.RevokedTaskError` instead of a ``WorkerLostError``.
+
+- ``AsyncResult.revoke`` now accepts ``terminate`` and ``signal`` arguments.
+
+- The ``task-revoked`` event now includes new fields: ``terminated``,
+  ``signum``, and ``expired``.
+
+- The argument to :class:`~celery.exceptions.TaskRevokedError` is now one
+  of the reasons ``revoked``, ``expired`` or ``terminated``.
+
+- Redis: Celery now tries to restore messages whenever there are no messages
+  in the queue.
+
+- Old Task class does no longer use classmethods for push_request and
+  pop_request  (Issue #912).
+
+- ``GroupResult`` now supports the ``children`` attribute (Issue #916).
+
+- ``AsyncResult.collect`` now respects the ``intermediate`` argument
+  (Issue #917).
+
+- Fixes example task in documentation (Issue #902).
+
+- Eventlet fixed so that the environment is patched as soon as possible.
+
+- eventlet: Now warns if celery related modules that depends on threads
+  are imported before eventlet is patched.
+
+- Improved event and camera examples in the monitoring guide.
+
+- Disables celery command setuptools entrypoints if the command can't be
+  loaded.
+
+- Fixed broken ``dump_request`` example in the tasks guide.
+
 .. _version-3.0.5:
 
 3.0.5

+ 2 - 1
celery/backends/base.py

@@ -168,7 +168,8 @@ class BaseBackend(object):
     def store_result(self, task_id, result, status, traceback=None, **kwargs):
         """Update task state and result."""
         result = self.encode_result(result, status)
-        return self._store_result(task_id, result, status, traceback, **kwargs)
+        self._store_result(task_id, result, status, traceback, **kwargs)
+        return result
 
     def forget(self, task_id):
         self._cache.pop(task_id, None)

+ 5 - 1
celery/bin/base.py

@@ -295,7 +295,11 @@ class Command(object):
         return argv
 
     def find_app(self, app):
-        sym = self.symbol_by_name(app)
+        try:
+            sym = self.symbol_by_name(app)
+        except AttributeError:
+            # last part was not an attribute, but a module
+            sym = import_from_cwd(app)
         if isinstance(sym, ModuleType):
             if getattr(sym, '__path__', None):
                 return self.find_app('{0}.celery:'.format(

+ 1 - 0
celery/result.py

@@ -653,6 +653,7 @@ class TaskSetResult(GroupResult):
 
 class EagerResult(AsyncResult):
     """Result that we know has already been executed."""
+    task_name = None
 
     def __init__(self, id, ret_value, state, traceback=None):
         self.id = id

+ 2 - 7
celery/utils/serialization.py

@@ -60,7 +60,7 @@ def find_nearest_pickleable_exception(exc):
         try:
             exc_args = getattr(exc, 'args', [])
             superexc = supercls(*exc_args)
-            pickle.dumps(superexc)
+            pickle.loads(pickle.dumps(superexc))
         except:
             pass
         else:
@@ -138,12 +138,7 @@ def get_pickleable_exception(exc):
     nearest = find_nearest_pickleable_exception(exc)
     if nearest:
         return nearest
-
-    try:
-        pickle.dumps(exc)
-    except Exception:
-        return UnpickleableExceptionWrapper.from_exception(exc)
-    return exc
+    return UnpickleableExceptionWrapper.from_exception(exc)
 
 
 def get_pickled_exception(exc):

+ 3 - 1
celery/worker/job.py

@@ -32,6 +32,7 @@ from celery.task.trace import (
 from celery.utils import fun_takes_kwargs
 from celery.utils.functional import noop
 from celery.utils.log import get_logger
+from celery.utils.serialization import get_pickled_exception
 from celery.utils.text import truncate
 from celery.utils.timeutils import maybe_iso8601, timezone
 
@@ -364,8 +365,9 @@ class Request(object):
         self._log_error(exc_info)
 
     def _log_error(self, einfo):
+        einfo.exception = get_pickled_exception(einfo.exception)
         exception, traceback, exc_info, internal, sargs, skwargs = (
-            safe_repr(einfo.exception),
+            einfo.exception,
             safe_str(einfo.traceback),
             einfo.exc_info,
             einfo.internal,