Browse Source

Move find_nearest_pickleable_exception + UnpickleableExceptionWrapper to celery.serialization

Ask Solem 15 years ago
parent
commit
156ff94a48

+ 4 - 101
celery/backends/base.py

@@ -1,92 +1,16 @@
 """celery.backends.base"""
 import time
 import operator
-from functools import partial as curry
 from celery.serialization import pickle
+from celery.serialization import get_pickled_exception
+from celery.serialization import get_pickleable_exception
 from celery.exceptions import TimeoutError
 
 
-def find_nearest_pickleable_exception(exc):
-    """With an exception instance, iterate over its super classes (by mro)
-    and find the first super exception that is pickleable. It does
-    not go below :exc:`Exception` (i.e. it skips :exc:`Exception`,
-    :class:`BaseException` and :class:`object`). If that happens
-    you should use :exc:`UnpickleableException` instead.
-
-    :param exc: An exception instance.
-
-    :returns: the nearest exception if it's not :exc:`Exception` or below,
-        if it is it returns ``None``.
-
-    :rtype: :exc:`Exception`
-
-    """
-
-    unwanted = (Exception, BaseException, object)
-    is_unwanted = lambda exc: any(map(curry(operator.is_, exc), unwanted))
-
-    mro_ = getattr(exc.__class__, "mro", lambda: [])
-    for supercls in mro_():
-        if is_unwanted(supercls):
-            # only BaseException and object, from here on down,
-            # we don't care about these.
-            return None
-        try:
-            exc_args = getattr(exc, "args", [])
-            superexc = supercls(*exc_args)
-            pickle.dumps(superexc)
-        except:
-            pass
-        else:
-            return superexc
-    return None
-
-
-class UnpickleableExceptionWrapper(Exception):
-    """Wraps unpickleable exceptions.
-
-    :param exc_module: see :attr:`exc_module`.
-
-    :param exc_cls_name: see :attr:`exc_cls_name`.
-
-    :param exc_args: see :attr:`exc_args`
-
-    .. attribute:: exc_module
-
-        The module of the original exception.
-
-    .. attribute:: exc_cls_name
-
-        The name of the original exception class.
-
-    .. attribute:: exc_args
-
-        The arguments for the original exception.
-
-    Example
-
-        >>> try:
-        ...     something_raising_unpickleable_exc()
-        >>> except Exception, e:
-        ...     exc = UnpickleableException(e.__class__.__module__,
-        ...                                 e.__class__.__name__,
-        ...                                 e.args)
-        ...     pickle.dumps(exc) # Works fine.
-
-    """
-
-    def __init__(self, exc_module, exc_cls_name, exc_args):
-        self.exc_module = exc_module
-        self.exc_cls_name = exc_cls_name
-        self.exc_args = exc_args
-        super(Exception, self).__init__(exc_module, exc_cls_name, exc_args)
-
-
 class BaseBackend(object):
     """The base backend class. All backends should inherit from this."""
 
     capabilities = []
-    UnpickleableExceptionWrapper = UnpickleableExceptionWrapper
     TimeoutError = TimeoutError
 
     def store_result(self, task_id, result, status):
@@ -109,34 +33,13 @@ class BaseBackend(object):
         return self.store_result(task_id, exc, status="RETRY",
                                  traceback=traceback)
 
-    def create_exception_cls(self, name, module, parent=None):
-        """Dynamically create an exception class."""
-        if not parent:
-            parent = Exception
-        return type(name, (parent, ), {"__module__": module})
-
     def prepare_exception(self, exc):
         """Prepare exception for serialization."""
-        nearest = find_nearest_pickleable_exception(exc)
-        if nearest:
-            return nearest
-
-        try:
-            pickle.dumps(exc)
-        except pickle.PickleError:
-            excwrapper = UnpickleableExceptionWrapper(
-                            exc.__class__.__module__,
-                            exc.__class__.__name__,
-                            getattr(exc, "args", []))
-            return excwrapper
+        return get_pickleable_exception(exc)
 
     def exception_to_python(self, exc):
         """Convert serialized exception to Python exception."""
-        if isinstance(exc, UnpickleableExceptionWrapper):
-            exc_cls = self.create_exception_cls(exc.exc_cls_name,
-                                                exc.exc_module)
-            return exc_cls(*exc.exc_args)
-        return exc
+        return get_pickled_exception(exc)
 
     def get_status(self, task_id):
         """Get the status of a task."""

+ 111 - 0
celery/serialization.py

@@ -1,4 +1,115 @@
+from functools import partial as curry
+import operator
 try:
     import cPickle as pickle
 except ImportError:
     import pickle
+
+
+def find_nearest_pickleable_exception(exc):
+    """With an exception instance, iterate over its super classes (by mro)
+    and find the first super exception that is pickleable. It does
+    not go below :exc:`Exception` (i.e. it skips :exc:`Exception`,
+    :class:`BaseException` and :class:`object`). If that happens
+    you should use :exc:`UnpickleableException` instead.
+
+    :param exc: An exception instance.
+
+    :returns: the nearest exception if it's not :exc:`Exception` or below,
+        if it is it returns ``None``.
+
+    :rtype: :exc:`Exception`
+
+    """
+
+    unwanted = (Exception, BaseException, object)
+    is_unwanted = lambda exc: any(map(curry(operator.is_, exc), unwanted))
+
+    mro_ = getattr(exc.__class__, "mro", lambda: [])
+    for supercls in mro_():
+        if is_unwanted(supercls):
+            # only BaseException and object, from here on down,
+            # we don't care about these.
+            return None
+        try:
+            exc_args = getattr(exc, "args", [])
+            superexc = supercls(*exc_args)
+            pickle.dumps(superexc)
+        except:
+            pass
+        else:
+            return superexc
+    return None
+
+
+def create_exception_cls(name, module, parent=None):
+    """Dynamically create an exception class."""
+    if not parent:
+        parent = Exception
+    return type(name, (parent, ), {"__module__": module})
+
+
+class UnpickleableExceptionWrapper(Exception):
+    """Wraps unpickleable exceptions.
+
+    :param exc_module: see :attr:`exc_module`.
+
+    :param exc_cls_name: see :attr:`exc_cls_name`.
+
+    :param exc_args: see :attr:`exc_args`
+
+    .. attribute:: exc_module
+
+        The module of the original exception.
+
+    .. attribute:: exc_cls_name
+
+        The name of the original exception class.
+
+    .. attribute:: exc_args
+
+        The arguments for the original exception.
+
+    Example
+
+        >>> try:
+        ...     something_raising_unpickleable_exc()
+        >>> except Exception, e:
+        ...     exc = UnpickleableException(e.__class__.__module__,
+        ...                                 e.__class__.__name__,
+        ...                                 e.args)
+        ...     pickle.dumps(exc) # Works fine.
+
+    """
+
+    def __init__(self, exc_module, exc_cls_name, exc_args):
+        self.exc_module = exc_module
+        self.exc_cls_name = exc_cls_name
+        self.exc_args = exc_args
+        super(Exception, self).__init__(exc_module, exc_cls_name, exc_args)
+
+
+def get_pickleable_exception(exc):
+    """Make sure exception is pickleable."""
+    nearest = find_nearest_pickleable_exception(exc)
+    if nearest:
+        return nearest
+
+    try:
+        pickle.dumps(exc)
+    except pickle.PickleError:
+        excwrapper = UnpickleableExceptionWrapper(
+                        exc.__class__.__module__,
+                        exc.__class__.__name__,
+                        getattr(exc, "args", []))
+        return excwrapper
+
+
+def get_pickled_exception(exc):
+    """Get original exception from exception pickled using
+    :meth:`get_pickleable_exception`."""
+    if isinstance(exc, UnpickleableExceptionWrapper):
+        exc_cls = create_exception_cls(exc.exc_cls_name,
+                                       exc.exc_module)
+        return exc_cls(*exc.exc_args)
+    return exc

+ 2 - 2
celery/tests/test_backends/test_base.py

@@ -1,8 +1,8 @@
 import unittest
 import types
-from celery.backends.base import find_nearest_pickleable_exception as fnpe
 from celery.backends.base import BaseBackend, KeyValueStoreBackend
-from celery.backends.base import UnpickleableExceptionWrapper
+from celery.serialization import find_nearest_pickleable_exception as fnpe
+from celery.serialization import UnpickleableExceptionWrapper
 from django.db.models.base import subclass_exception
 
 

+ 2 - 1
celery/tests/test_serialization.py

@@ -7,8 +7,9 @@ class TestAAPickle(unittest.TestCase):
 
     def test_no_cpickle(self):
         from celery.tests.utils import mask_modules
-        del(sys.modules["celery.serialization"])
+        prev = sys.modules.pop("celery.serialization")
         with mask_modules("cPickle"):
             from celery.serialization import pickle
             import pickle as orig_pickle
             self.assertTrue(pickle.dumps is orig_pickle.dumps)
+        sys.modules["celery.serialization"] = prev