Ver código fonte

Cosmetics for Issue #2253

Ask Solem 10 anos atrás
pai
commit
a399bd7713

+ 0 - 4
celery/backends/amqp.py

@@ -80,10 +80,6 @@ class AMQPBackend(BaseBackend):
         )
         )
         self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
         self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
         self.auto_delete = auto_delete
         self.auto_delete = auto_delete
-
-        self.expires = None
-        if 'expires' not in kwargs or kwargs['expires'] is not None:
-            self.expires = self.prepare_expires(kwargs.get('expires'))
         self.queue_arguments = dictfilter({
         self.queue_arguments = dictfilter({
             'x-expires': maybe_s_to_ms(self.expires),
             'x-expires': maybe_s_to_ms(self.expires),
         })
         })

+ 5 - 2
celery/backends/base.py

@@ -91,8 +91,9 @@ class BaseBackend(object):
         'interval_max': 1,
         'interval_max': 1,
     }
     }
 
 
-    def __init__(self, app, serializer=None,
-                 max_cached_results=None, accept=None, **kwargs):
+    def __init__(self, app,
+                 serializer=None, max_cached_results=None, accept=None,
+                 expires=None, expires_type=None, **kwargs):
         self.app = app
         self.app = app
         conf = self.app.conf
         conf = self.app.conf
         self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
         self.serializer = serializer or conf.CELERY_RESULT_SERIALIZER
@@ -101,6 +102,8 @@ class BaseBackend(object):
          self.encoder) = serializer_registry._encoders[self.serializer]
          self.encoder) = serializer_registry._encoders[self.serializer]
         cmax = max_cached_results or conf.CELERY_MAX_CACHED_RESULTS
         cmax = max_cached_results or conf.CELERY_MAX_CACHED_RESULTS
         self._cache = _nulldict() if cmax == -1 else LRUCache(limit=cmax)
         self._cache = _nulldict() if cmax == -1 else LRUCache(limit=cmax)
+
+        self.expires = self.prepare_expires(expires, expires_type)
         self.accept = prepare_accept_content(
         self.accept = prepare_accept_content(
             conf.CELERY_ACCEPT_CONTENT if accept is None else accept,
             conf.CELERY_ACCEPT_CONTENT if accept is None else accept,
         )
         )

+ 7 - 10
celery/backends/cassandra.py

@@ -22,7 +22,6 @@ from celery import states
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
 from celery.five import monotonic
 from celery.five import monotonic
 from celery.utils.log import get_logger
 from celery.utils.log import get_logger
-from celery.utils.timeutils import maybe_timedelta
 
 
 from .base import BaseBackend
 from .base import BaseBackend
 
 
@@ -60,9 +59,6 @@ class CassandraBackend(BaseBackend):
         """
         """
         super(CassandraBackend, self).__init__(**kwargs)
         super(CassandraBackend, self).__init__(**kwargs)
 
 
-        self.expires = kwargs.get('expires') or maybe_timedelta(
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES)
-
         if not pycassa:
         if not pycassa:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(
                 'You need to install the pycassa library to use the '
                 'You need to install the pycassa library to use the '
@@ -140,21 +136,22 @@ class CassandraBackend(BaseBackend):
         """Store return value and status of an executed task."""
         """Store return value and status of an executed task."""
 
 
         def _do_store():
         def _do_store():
+            detailed = self.detailed_mode
             cf = self._get_column_family()
             cf = self._get_column_family()
             date_done = self.app.now()
             date_done = self.app.now()
             meta = {'status': status,
             meta = {'status': status,
                     'date_done': date_done.strftime('%Y-%m-%dT%H:%M:%SZ'),
                     'date_done': date_done.strftime('%Y-%m-%dT%H:%M:%SZ'),
                     'traceback': self.encode(traceback),
                     'traceback': self.encode(traceback),
+                    'result': result if detailed else self.encode(result),
                     'children': self.encode(
                     'children': self.encode(
                         self.current_task_children(request),
                         self.current_task_children(request),
                     )}
                     )}
-            ttl = self.expires and max(self.expires.total_seconds(), 0)
-            if self.detailed_mode:
-                meta['result'] = result
-                cf.insert(task_id, {date_done: self.encode(meta)}, ttl=ttl)
+            if detailed:
+                cf.insert(
+                    task_id, {date_done: self.encode(meta)}, ttl=self.expires,
+                )
             else:
             else:
-                meta['result'] = self.encode(result)
-                cf.insert(task_id, meta, ttl=ttl)
+                cf.insert(task_id, meta, ttl=self.expires)
 
 
         return self._retry_on_error(_do_store)
         return self._retry_on_error(_do_store)
 
 

+ 0 - 4
celery/backends/couchbase.py

@@ -20,7 +20,6 @@ except ImportError:
 from kombu.utils.url import _parse_url
 from kombu.utils.url import _parse_url
 
 
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
-from celery.utils.timeutils import maybe_timedelta
 
 
 from .base import KeyValueStoreBackend
 from .base import KeyValueStoreBackend
 
 
@@ -49,9 +48,6 @@ class CouchBaseBackend(KeyValueStoreBackend):
         """
         """
         super(CouchBaseBackend, self).__init__(*args, **kwargs)
         super(CouchBaseBackend, self).__init__(*args, **kwargs)
 
 
-        self.expires = kwargs.get('expires') or maybe_timedelta(
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES)
-
         if Couchbase is None:
         if Couchbase is None:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(
                 'You need to install the couchbase library to use the '
                 'You need to install the couchbase library to use the '

+ 32 - 53
celery/backends/couchdb.py

@@ -8,8 +8,6 @@
 """
 """
 from __future__ import absolute_import
 from __future__ import absolute_import
 
 
-import logging
-
 try:
 try:
     import pycouchdb
     import pycouchdb
 except ImportError:
 except ImportError:
@@ -18,26 +16,23 @@ except ImportError:
 from kombu.utils.url import _parse_url
 from kombu.utils.url import _parse_url
 
 
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
-from celery.utils.timeutils import maybe_timedelta
 
 
 from .base import KeyValueStoreBackend
 from .base import KeyValueStoreBackend
 
 
-__all__ = ['CouchDBBackend']
+__all__ = ['CouchBackend']
+
+ERR_LIB_MISSING = """\
+You need to install the pycouchdb library to use the CouchDB result backend\
+"""
 
 
 
 
-class CouchDBBackend(KeyValueStoreBackend):
+class CouchBackend(KeyValueStoreBackend):
     container = 'default'
     container = 'default'
     scheme = 'http'
     scheme = 'http'
     host = 'localhost'
     host = 'localhost'
     port = 5984
     port = 5984
     username = None
     username = None
     password = None
     password = None
-    quiet = False
-    conncache = None
-    unlock_gil = True
-    timeout = 2.5
-    transcoder = None
-    # supports_autoexpire = False
 
 
     def __init__(self, url=None, *args, **kwargs):
     def __init__(self, url=None, *args, **kwargs):
         """Initialize CouchDB backend instance.
         """Initialize CouchDB backend instance.
@@ -46,63 +41,47 @@ class CouchDBBackend(KeyValueStoreBackend):
             module :mod:`pycouchdb` is not available.
             module :mod:`pycouchdb` is not available.
 
 
         """
         """
-        super(CouchDBBackend, self).__init__(*args, **kwargs)
-
-        self.expires = kwargs.get('expires') or maybe_timedelta(
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES)
+        super(CouchBackend, self).__init__(*args, **kwargs)
 
 
         if pycouchdb is None:
         if pycouchdb is None:
-            raise ImproperlyConfigured(
-                'You need to install the pycouchdb library to use the '
-                'CouchDB backend.',
-            )
+            raise ImproperlyConfigured(ERR_LIB_MISSING)
 
 
         uscheme = uhost = uport = uname = upass = ucontainer = None
         uscheme = uhost = uport = uname = upass = ucontainer = None
         if url:
         if url:
-            _, uhost, uport, uname, upass, ucontainer , _ = _parse_url(url)  # noqa
+            _, uhost, uport, uname, upass, ucontainer, _ = _parse_url(url)  # noqa
             ucontainer = ucontainer.strip('/') if ucontainer else None
             ucontainer = ucontainer.strip('/') if ucontainer else None
 
 
-        config = self.app.conf.get('CELERY_COUCHDB_BACKEND_SETTINGS', None)
-        if config is not None:
-            if not isinstance(config, dict):
-                raise ImproperlyConfigured(
-                    'CouchDB backend settings should be grouped in a dict',
-                )
-        else:
-            config = {}
-
-        self.scheme = uscheme or config.get('scheme', self.scheme)
-        self.host = uhost or config.get('host', self.host)
-        self.port = int(uport or config.get('port', self.port))
-        self.container = ucontainer or config.get('container', self.container)
-        self.username = uname or config.get('username', self.username)
-        self.password = upass or config.get('password', self.password)
+        self.scheme = uscheme or self.scheme
+        self.host = uhost or self.host
+        self.port = int(uport or self.port)
+        self.container = ucontainer or self.container
+        self.username = uname or self.username
+        self.password = upass or self.password
 
 
         self._connection = None
         self._connection = None
 
 
     def _get_connection(self):
     def _get_connection(self):
         """Connect to the CouchDB server."""
         """Connect to the CouchDB server."""
-        if self._connection is None:
-            if self.username and self.password:
-                conn_string = '%s://%s:%s@%s:%s' % (
-                    self.scheme, self.username, self.password,
-                    self.host, str(self.port))
-                server = pycouchdb.Server(conn_string, authmethod='basic')
-            else:
-                conn_string = '%s://%s:%s' % (
-                    self.scheme, self.host, str(self.port))
-                server = pycouchdb.Server(conn_string)
-
-            logging.debug('couchdb conn string: %s', conn_string)
-            try:
-                self._connection = server.database(self.container)
-            except pycouchdb.exceptions.NotFound:
-                self._connection = server.create(self.container)
-        return self._connection
+        if self.username and self.password:
+            conn_string = '%s://%s:%s@%s:%s' % (
+                self.scheme, self.username, self.password,
+                self.host, str(self.port))
+            server = pycouchdb.Server(conn_string, authmethod='basic')
+        else:
+            conn_string = '%s://%s:%s' % (
+                self.scheme, self.host, str(self.port))
+            server = pycouchdb.Server(conn_string)
+
+        try:
+            return server.database(self.container)
+        except pycouchdb.exceptions.NotFound:
+            return server.create(self.container)
 
 
     @property
     @property
     def connection(self):
     def connection(self):
-        return self._get_connection()
+        if self._connection is None:
+            self._connection = self._get_connection()
+        return self._connection
 
 
     def get(self, key):
     def get(self, key):
         try:
         try:

+ 4 - 4
celery/backends/database/__init__.py

@@ -79,13 +79,13 @@ class DatabaseBackend(BaseBackend):
     # to not bombard the database with queries.
     # to not bombard the database with queries.
     subpolling_interval = 0.5
     subpolling_interval = 0.5
 
 
-    def __init__(self, dburi=None, expires=None,
-                 engine_options=None, url=None, **kwargs):
+    def __init__(self, dburi=None, engine_options=None, url=None, **kwargs):
         # The `url` argument was added later and is used by
         # The `url` argument was added later and is used by
         # the app to set backend by url (celery.backends.get_backend_by_url)
         # the app to set backend by url (celery.backends.get_backend_by_url)
-        super(DatabaseBackend, self).__init__(**kwargs)
+        super(DatabaseBackend, self).__init__(
+            expires_type=maybe_timedelta, **kwargs
+        )
         conf = self.app.conf
         conf = self.app.conf
-        self.expires = maybe_timedelta(self.prepare_expires(expires))
         self.dburi = url or dburi or conf.CELERY_RESULT_DBURI
         self.dburi = url or dburi or conf.CELERY_RESULT_DBURI
         self.engine_options = dict(
         self.engine_options = dict(
             engine_options or {},
             engine_options or {},

+ 2 - 5
celery/backends/mongodb.py

@@ -31,7 +31,6 @@ from kombu.exceptions import EncodeError
 from celery import states
 from celery import states
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
 from celery.five import string_t
 from celery.five import string_t
-from celery.utils.timeutils import maybe_timedelta
 
 
 from .base import BaseBackend
 from .base import BaseBackend
 
 
@@ -60,7 +59,7 @@ class MongoBackend(BaseBackend):
 
 
     _connection = None
     _connection = None
 
 
-    def __init__(self, *args, **kwargs):
+    def __init__(self, **kwargs):
         """Initialize MongoDB backend instance.
         """Initialize MongoDB backend instance.
 
 
         :raises celery.exceptions.ImproperlyConfigured: if
         :raises celery.exceptions.ImproperlyConfigured: if
@@ -69,9 +68,7 @@ class MongoBackend(BaseBackend):
         """
         """
         self.options = {}
         self.options = {}
 
 
-        super(MongoBackend, self).__init__(*args, **kwargs)
-        self.expires = kwargs.get('expires') or maybe_timedelta(
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES)
+        super(MongoBackend, self).__init__(**kwargs)
 
 
         if not pymongo:
         if not pymongo:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(

+ 2 - 3
celery/backends/redis.py

@@ -57,9 +57,9 @@ class RedisBackend(KeyValueStoreBackend):
     implements_incr = True
     implements_incr = True
 
 
     def __init__(self, host=None, port=None, db=None, password=None,
     def __init__(self, host=None, port=None, db=None, password=None,
-                 expires=None, max_connections=None, url=None,
+                 max_connections=None, url=None,
                  connection_pool=None, new_join=False, **kwargs):
                  connection_pool=None, new_join=False, **kwargs):
-        super(RedisBackend, self).__init__(**kwargs)
+        super(RedisBackend, self).__init__(expires_type=int, **kwargs)
         conf = self.app.conf
         conf = self.app.conf
         if self.redis is None:
         if self.redis is None:
             raise ImproperlyConfigured(REDIS_MISSING)
             raise ImproperlyConfigured(REDIS_MISSING)
@@ -90,7 +90,6 @@ class RedisBackend(KeyValueStoreBackend):
         if url:
         if url:
             self.connparams = self._params_from_url(url, self.connparams)
             self.connparams = self._params_from_url(url, self.connparams)
         self.url = url
         self.url = url
-        self.expires = self.prepare_expires(expires, type=int)
 
 
         try:
         try:
             new_join = strtobool(self.connparams.pop('new_join'))
             new_join = strtobool(self.connparams.pop('new_join'))

+ 0 - 4
celery/backends/riak.py

@@ -18,7 +18,6 @@ except ImportError:  # pragma: no cover
 from kombu.utils.url import _parse_url
 from kombu.utils.url import _parse_url
 
 
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
-from celery.utils.timeutils import maybe_timedelta
 
 
 from .base import KeyValueStoreBackend
 from .base import KeyValueStoreBackend
 
 
@@ -60,9 +59,6 @@ class RiakBackend(KeyValueStoreBackend):
         """
         """
         super(RiakBackend, self).__init__(*args, **kwargs)
         super(RiakBackend, self).__init__(*args, **kwargs)
 
 
-        self.expires = kwargs.get('expires') or maybe_timedelta(
-            self.app.conf.CELERY_TASK_RESULT_EXPIRES)
-
         if not riak:
         if not riak:
             raise ImproperlyConfigured(
             raise ImproperlyConfigured(
                 'You need to install the riak library to use the '
                 'You need to install the riak library to use the '

+ 10 - 42
celery/tests/backends/test_couchdb.py

@@ -1,7 +1,7 @@
 from __future__ import absolute_import
 from __future__ import absolute_import
 
 
 from celery.backends import couchdb as module
 from celery.backends import couchdb as module
-from celery.backends.couchdb import CouchDBBackend
+from celery.backends.couchdb import CouchBackend
 from celery.exceptions import ImproperlyConfigured
 from celery.exceptions import ImproperlyConfigured
 from celery import backends
 from celery import backends
 from celery.tests.case import (
 from celery.tests.case import (
@@ -16,33 +16,22 @@ except ImportError:
 COUCHDB_CONTAINER = 'celery_container'
 COUCHDB_CONTAINER = 'celery_container'
 
 
 
 
-class test_CouchDBBackend(AppCase):
+class test_CouchBackend(AppCase):
 
 
     def setup(self):
     def setup(self):
         if pycouchdb is None:
         if pycouchdb is None:
             raise SkipTest('pycouchdb is not installed.')
             raise SkipTest('pycouchdb is not installed.')
-        self.backend = CouchDBBackend(app=self.app)
+        self.backend = CouchBackend(app=self.app)
 
 
     def test_init_no_pycouchdb(self):
     def test_init_no_pycouchdb(self):
         """test init no pycouchdb raises"""
         """test init no pycouchdb raises"""
         prev, module.pycouchdb = module.pycouchdb, None
         prev, module.pycouchdb = module.pycouchdb, None
         try:
         try:
             with self.assertRaises(ImproperlyConfigured):
             with self.assertRaises(ImproperlyConfigured):
-                CouchDBBackend(app=self.app)
+                CouchBackend(app=self.app)
         finally:
         finally:
             module.pycouchdb = prev
             module.pycouchdb = prev
 
 
-    def test_init_no_settings(self):
-        """test init no settings"""
-        self.app.conf.CELERY_COUCHDB_BACKEND_SETTINGS = []
-        with self.assertRaises(ImproperlyConfigured):
-            CouchDBBackend(app=self.app)
-
-    def test_init_settings_is_None(self):
-        """Test init settings is None"""
-        self.app.conf.CELERY_COUCHDB_BACKEND_SETTINGS = None
-        CouchDBBackend(app=self.app)
-
     def test_get_container_exists(self):
     def test_get_container_exists(self):
         with patch('pycouchdb.client.Database') as mock_Connection:
         with patch('pycouchdb.client.Database') as mock_Connection:
             self.backend._connection = sentinel._connection
             self.backend._connection = sentinel._connection
@@ -55,13 +44,12 @@ class test_CouchDBBackend(AppCase):
     def test_get(self):
     def test_get(self):
         """test_get
         """test_get
 
 
-        CouchDBBackend.get should return  and take two params
+        CouchBackend.get should return  and take two params
         db conn to couchdb is mocked.
         db conn to couchdb is mocked.
         TODO Should test on key not exists
         TODO Should test on key not exists
 
 
         """
         """
-        self.app.conf.CELERY_COUCHDB_BACKEND_SETTINGS = {}
-        x = CouchDBBackend(app=self.app)
+        x = CouchBackend(app=self.app)
         x._connection = Mock()
         x._connection = Mock()
         mocked_get = x._connection.get = Mock()
         mocked_get = x._connection.get = Mock()
         mocked_get.return_value = sentinel.retval
         mocked_get.return_value = sentinel.retval
@@ -72,13 +60,12 @@ class test_CouchDBBackend(AppCase):
     def test_delete(self):
     def test_delete(self):
         """test_delete
         """test_delete
 
 
-        CouchDBBackend.delete should return and take two params
+        CouchBackend.delete should return and take two params
         db conn to pycouchdb is mocked.
         db conn to pycouchdb is mocked.
         TODO Should test on key not exists
         TODO Should test on key not exists
 
 
         """
         """
-        self.app.conf.CELERY_COUCHDB_BACKEND_SETTINGS = {}
-        x = CouchDBBackend(app=self.app)
+        x = CouchBackend(app=self.app)
         x._connection = Mock()
         x._connection = Mock()
         mocked_delete = x._connection.delete = Mock()
         mocked_delete = x._connection.delete = Mock()
         mocked_delete.return_value = None
         mocked_delete.return_value = None
@@ -86,29 +73,10 @@ class test_CouchDBBackend(AppCase):
         self.assertIsNone(x.delete('1f3fab'))
         self.assertIsNone(x.delete('1f3fab'))
         x._connection.delete.assert_called_once_with('1f3fab')
         x._connection.delete.assert_called_once_with('1f3fab')
 
 
-    def test_config_params(self):
-        """test_config_params
-
-        celery.conf.CELERY_COUCHDB_BACKEND_SETTINGS is properly set
-        """
-        self.app.conf.CELERY_COUCHDB_BACKEND_SETTINGS = {
-            'container': 'mycoolcontainer',
-            'host': ['here.host.com', 'there.host.com'],
-            'username': 'johndoe',
-            'password': 'mysecret',
-            'port': '1234',
-        }
-        x = CouchDBBackend(app=self.app)
-        self.assertEqual(x.container, 'mycoolcontainer')
-        self.assertEqual(x.host, ['here.host.com', 'there.host.com'],)
-        self.assertEqual(x.username, 'johndoe',)
-        self.assertEqual(x.password, 'mysecret')
-        self.assertEqual(x.port, 1234)
-
     def test_backend_by_url(self, url='couchdb://myhost/mycoolcontainer'):
     def test_backend_by_url(self, url='couchdb://myhost/mycoolcontainer'):
-        from celery.backends.couchdb import CouchDBBackend
+        from celery.backends.couchdb import CouchBackend
         backend, url_ = backends.get_backend_by_url(url, self.app.loader)
         backend, url_ = backends.get_backend_by_url(url, self.app.loader)
-        self.assertIs(backend, CouchDBBackend)
+        self.assertIs(backend, CouchBackend)
         self.assertEqual(url_, url)
         self.assertEqual(url_, url)
 
 
     def test_backend_params_by_url(self):
     def test_backend_params_by_url(self):

+ 6 - 13
docs/configuration.rst

@@ -794,20 +794,18 @@ CouchDB backend settings
         $ pip install pycouchdb
         $ pip install pycouchdb
 
 
 This backend can be configured via the :setting:`CELERY_RESULT_BACKEND`
 This backend can be configured via the :setting:`CELERY_RESULT_BACKEND`
-set to a couchbase URL::
+set to a couchdb URL::
 
 
     CELERY_RESULT_BACKEND = 'couchdb://username:password@host:port/container'
     CELERY_RESULT_BACKEND = 'couchdb://username:password@host:port/container'
 
 
 
 
-.. setting:: CELERY_COUCHDB_BACKEND_SETTINGS
-
-CELERY_COUCHDB_BACKEND_SETTINGS
-~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
+The URL is formed out of the following parts:
 
 
-This is a dict supporting the following keys:
+* username
+    User name to authenticate to the CouchDB server as (optional).
 
 
-* scheme
-    http or https. Defaults to ``http``.
+* password
+    Password to authenticate to the CouchDB server (optional).
 
 
 * host
 * host
     Host name of the CouchDB server. Defaults to ``localhost``.
     Host name of the CouchDB server. Defaults to ``localhost``.
@@ -819,11 +817,6 @@ This is a dict supporting the following keys:
     The default container the CouchDB server is writing to.
     The default container the CouchDB server is writing to.
     Defaults to ``default``.
     Defaults to ``default``.
 
 
-* username
-    User name to authenticate to the CouchDB server as (optional).
-
-* password
-    Password to authenticate to the CouchDB server (optional).
 
 
 .. _conf-messaging:
 .. _conf-messaging: