Ver Fonte

new_cassandra is now cassandra

Ask Solem há 10 anos atrás
pai
commit
081c78fffe

+ 0 - 3
README.rst

@@ -284,9 +284,6 @@ Transports and Backends
     for using memcached as a result backend.
 
 :celery[cassandra]:
-    for using Apache Cassandra as a result backend with pycassa driver.
-
-:celery[new_cassandra]:
     for using Apache Cassandra as a result backend with DataStax driver.
 
 :celery[couchdb]:

+ 1 - 1
celery/app/defaults.py

@@ -123,12 +123,12 @@ NAMESPACES = Namespace(
         backend_options=Option({}, type='dict'),
     ),
     cassandra=Namespace(
-        column_family=Option(type='string'),
         entry_ttl=Option(type="float"),
         keyspace=Option(type='string'),
         port=Option(type="string"),
         read_consistency=Option(type='string'),
         servers=Option(type='list'),
+        table=Option(type='string'),
         write_consistency=Option(type='string'),
     ),
     chord=Namespace(

+ 0 - 1
celery/backends/__init__.py

@@ -30,7 +30,6 @@ BACKEND_ALIASES = {
     'db': 'celery.backends.database:DatabaseBackend',
     'database': 'celery.backends.database:DatabaseBackend',
     'cassandra': 'celery.backends.cassandra:CassandraBackend',
-    'new_cassandra': 'celery.backends.new_cassandra:CassandraBackend',
     'couchbase': 'celery.backends.couchbase:CouchBaseBackend',
     'couchdb': 'celery.backends.couchdb:CouchDBBackend',
     'riak': 'celery.backends.riak:RiakBackend',

+ 167 - 133
celery/backends/cassandra.py

@@ -3,59 +3,83 @@
     celery.backends.cassandra
     ~~~~~~~~~~~~~~~~~~~~~~~~~
 
-    Apache Cassandra result store backend.
+    Apache Cassandra result store backend using DataStax driver
 
 """
 from __future__ import absolute_import
 
+import sys
 try:  # pragma: no cover
-    import pycassa
-    from thrift import Thrift
-    C = pycassa.cassandra.ttypes
+    import cassandra
+    import cassandra.cluster
 except ImportError:  # pragma: no cover
-    pycassa = None   # noqa
-
-import socket
-import time
+    cassandra = None   # noqa
 
 from celery import states
 from celery.exceptions import ImproperlyConfigured
-from celery.five import monotonic
-from celery.utils import deprecated
 from celery.utils.log import get_logger
-
 from .base import BaseBackend
 
 __all__ = ['CassandraBackend']
 
 logger = get_logger(__name__)
 
+E_NO_CASSANDRA = """
+You need to install the cassandra-driver library to
+use the Cassandra backend. See https://github.com/datastax/python-driver
+"""
 
-class CassandraBackend(BaseBackend):
-    """Highly fault tolerant Cassandra backend.
+Q_INSERT_RESULT = """
+INSERT INTO {table} (
+    task_id, status, result, date_done, traceback, children) VALUES (
+        %s, %s, %s, %s, %s, %s) {expires};
+"""
+
+Q_SELECT_RESULT = """
+SELECT status, result, date_done, traceback, children
+FROM {table}
+WHERE task_id=%s
+LIMIT 1
+"""
+
+Q_CREATE_RESULT_TABLE = """
+CREATE TABLE {table} (
+    task_id text,
+    status text,
+    result blob,
+    date_done timestamp,
+    traceback blob,
+    children blob,
+    PRIMARY KEY ((task_id), date_done)
+) WITH CLUSTERING ORDER BY (date_done DESC);
+"""
 
-    .. attribute:: servers
+Q_EXPIRES = """
+    USING TTL {0}
+"""
+
+if sys.version_info[0] == 3:
+    def buf_t(x):
+        return bytes(x, 'utf8')
+else:
+    buf_t = buffer  # noqa
 
-        List of Cassandra servers with format: ``hostname:port``.
+
+class CassandraBackend(BaseBackend):
+    """Cassandra backend utilizing DataStax driver
 
     :raises celery.exceptions.ImproperlyConfigured: if
-        module :mod:`pycassa` is not available.
+        module :mod:`cassandra` is not available.
 
     """
-    servers = []
-    keyspace = None
-    column_family = None
-    detailed_mode = False
-    _retry_timeout = 300
-    _retry_wait = 3
-    supports_autoexpire = True
-
-    @deprecated(description='The old cassandra backend',
-                deprecation='4.0',
-                removal='5.0',
-                alternative='Use the `new_cassandra` result backend instead')
-    def __init__(self, servers=None, keyspace=None, column_family=None,
-                 cassandra_options=None, detailed_mode=False, **kwargs):
+
+    #: List of Cassandra servers with format: ``hostname``.
+    servers = None
+
+    supports_autoexpire = True      # autoexpire supported via entry_ttl
+
+    def __init__(self, servers=None, keyspace=None, table=None, entry_ttl=None,
+                 port=9042, **kwargs):
         """Initialize Cassandra backend.
 
         Raises :class:`celery.exceptions.ImproperlyConfigured` if
@@ -64,129 +88,139 @@ class CassandraBackend(BaseBackend):
         """
         super(CassandraBackend, self).__init__(**kwargs)
 
-        if not pycassa:
-            raise ImproperlyConfigured(
-                'You need to install the pycassa library to use the '
-                'Cassandra backend. See https://github.com/pycassa/pycassa')
+        if not cassandra:
+            raise ImproperlyConfigured(E_NO_CASSANDRA)
 
         conf = self.app.conf
         self.servers = (servers or
-                        conf.get('cassandra_servers') or
-                        self.servers)
+                        conf.get('cassandra_servers', None))
+        self.port = (port or
+                     conf.get('cassandra_port', None))
         self.keyspace = (keyspace or
-                         conf.get('cassandra_keyspace') or
-                         self.keyspace)
-        self.column_family = (column_family or
-                              conf.get('cassandra_column_family') or
-                              self.column_family)
-        self.cassandra_options = dict(conf.get('cassandra_options') or {},
-                                      **cassandra_options or {})
-        self.detailed_mode = (detailed_mode or
-                              conf.get('cassandra_detailed_mode') or
-                              self.detailed_mode)
+                         conf.get('cassandra_keyspace', None))
+        self.table = (table or
+                      conf.get('cassandra_table', None))
+
+        if not self.servers or not self.keyspace or not self.table:
+            raise ImproperlyConfigured('Cassandra backend not configured.')
+
+        expires = (entry_ttl or conf.get('cassandra_entry_ttl', None))
+
+        self.cqlexpires = (Q_EXPIRES.format(expires)
+                           if expires is not None else '')
+
         read_cons = conf.get('cassandra_read_consistency') or 'LOCAL_QUORUM'
         write_cons = conf.get('cassandra_write_consistency') or 'LOCAL_QUORUM'
-        try:
-            self.read_consistency = getattr(pycassa.ConsistencyLevel,
-                                            read_cons)
-        except AttributeError:
-            self.read_consistency = pycassa.ConsistencyLevel.LOCAL_QUORUM
-        try:
-            self.write_consistency = getattr(pycassa.ConsistencyLevel,
-                                             write_cons)
-        except AttributeError:
-            self.write_consistency = pycassa.ConsistencyLevel.LOCAL_QUORUM
-
-        if not self.servers or not self.keyspace or not self.column_family:
-            raise ImproperlyConfigured(
-                'Cassandra backend not configured.')
-
-        self._column_family = None
-
-    def _retry_on_error(self, fun, *args, **kwargs):
-        ts = monotonic() + self._retry_timeout
-        while 1:
-            try:
-                return fun(*args, **kwargs)
-            except (pycassa.InvalidRequestException,
-                    pycassa.TimedOutException,
-                    pycassa.UnavailableException,
-                    pycassa.AllServersUnavailable,
-                    socket.error,
-                    socket.timeout,
-                    Thrift.TException) as exc:
-                if monotonic() > ts:
-                    raise
-                logger.warning('Cassandra error: %r. Retrying...', exc)
-                time.sleep(self._retry_wait)
-
-    def _get_column_family(self):
-        if self._column_family is None:
-            conn = pycassa.ConnectionPool(self.keyspace,
-                                          server_list=self.servers,
-                                          **self.cassandra_options)
-            self._column_family = pycassa.ColumnFamily(
-                conn, self.column_family,
-                read_consistency_level=self.read_consistency,
-                write_consistency_level=self.write_consistency,
-            )
-        return self._column_family
+
+        self.read_consistency = getattr(
+            cassandra.ConsistencyLevel, read_cons,
+            cassandra.ConsistencyLevel.LOCAL_QUORUM,
+        )
+        self.write_consistency = getattr(
+            cassandra.ConsistencyLevel, write_cons,
+            cassandra.ConsistencyLevel.LOCAL_QUORUM,
+        )
+
+        self._connection = None
+        self._session = None
+        self._write_stmt = None
+        self._read_stmt = None
 
     def process_cleanup(self):
-        if self._column_family is not None:
-            self._column_family = None
+        if self._connection is not None:
+            self._connection = None
+        if self._session is not None:
+            self._session.shutdown()
+            self._session = None
+
+    def _get_connection(self, write=False):
+        """Prepare the connection for action
+
+        :param write: bool - are we a writer?
+
+        """
+        if self._connection is None:
+            try:
+                self._connection = cassandra.cluster.Cluster(self.servers,
+                                                             port=self.port)
+                self._session = self._connection.connect(self.keyspace)
+
+                # We are forced to do concatenation below, as formatting would
+                # blow up on superficial %s that will be processed by Cassandra
+                self._write_stmt = cassandra.query.SimpleStatement(
+                    Q_INSERT_RESULT.format(
+                        table=self.table, expires=self.cqlexpires),
+                )
+                self._write_stmt.consistency_level = self.write_consistency
+
+                self._read_stmt = cassandra.query.SimpleStatement(
+                    Q_SELECT_RESULT.format(table=self.table),
+                )
+                self._read_stmt.consistency_level = self.read_consistency
+
+                if write:
+                    # Only possible writers "workers" are allowed to issue
+                    # CREATE TABLE. This is to prevent conflicting situations
+                    # where both task-creator and task-executor would issue it
+                    # at the same time.
+
+                    # Anyway; if you're doing anything critical, you should
+                    # have created this table in advance, in which case
+                    # this query will be a no-op (AlreadyExists)
+                    self._make_stmt = cassandra.query.SimpleStatement(
+                        Q_CREATE_RESULT_TABLE.format(table=self.table),
+                    )
+                    self._make_stmt.consistency_level = self.write_consistency
+                    try:
+                        self._session.execute(self._make_stmt)
+                    except cassandra.AlreadyExists:
+                        pass
+
+            except cassandra.OperationTimedOut:
+                # a heavily loaded or gone Cassandra cluster failed to respond.
+                # leave this class in a consistent state
+                self._connection = None
+                if self._session is not None:
+                    self._session.shutdown()
+
+                raise   # we did fail after all - reraise
 
     def _store_result(self, task_id, result, status,
                       traceback=None, request=None, **kwargs):
         """Store return value and status of an executed task."""
+        self._get_connection(write=True)
 
-        def _do_store():
-            cf = self._get_column_family()
-            date_done = self.app.now()
-            meta = {'status': status,
-                    'date_done': date_done.strftime('%Y-%m-%dT%H:%M:%SZ'),
-                    'traceback': self.encode(traceback),
-                    'result': self.encode(result),
-                    'children': self.encode(
-                        self.current_task_children(request),
-                    )}
-            if self.detailed_mode:
-                cf.insert(
-                    task_id, {date_done: self.encode(meta)}, ttl=self.expires,
-                )
-            else:
-                cf.insert(task_id, meta, ttl=self.expires)
-
-        return self._retry_on_error(_do_store)
+        self._session.execute(self._write_stmt, (
+            task_id,
+            status,
+            buf_t(self.encode(result)),
+            self.app.now(),
+            buf_t(self.encode(traceback)),
+            buf_t(self.encode(self.current_task_children(request)))
+        ))
 
     def _get_task_meta_for(self, task_id):
         """Get task metadata for a task by id."""
+        self._get_connection()
 
-        def _do_get():
-            cf = self._get_column_family()
-            try:
-                if self.detailed_mode:
-                    row = cf.get(task_id, column_reversed=True, column_count=1)
-                    return self.decode(list(row.values())[0])
-                else:
-                    obj = cf.get(task_id)
-                    return self.meta_from_decoded({
-                        'task_id': task_id,
-                        'status': obj['status'],
-                        'result': self.decode(obj['result']),
-                        'date_done': obj['date_done'],
-                        'traceback': self.decode(obj['traceback']),
-                        'children': self.decode(obj['children']),
-                    })
-            except (KeyError, pycassa.NotFoundException):
-                return {'status': states.PENDING, 'result': None}
-
-        return self._retry_on_error(_do_get)
+        res = self._session.execute(self._read_stmt, (task_id, ))
+        if not res:
+            return {'status': states.PENDING, 'result': None}
+
+        status, result, date_done, traceback, children = res[0]
+
+        return self.meta_from_decoded({
+            'task_id': task_id,
+            'status': status,
+            'result': self.decode(result),
+            'date_done': date_done.strftime('%Y-%m-%dT%H:%M:%SZ'),
+            'traceback': self.decode(traceback),
+            'children': self.decode(children),
+        })
 
     def __reduce__(self, args=(), kwargs={}):
         kwargs.update(
             dict(servers=self.servers,
                  keyspace=self.keyspace,
-                 column_family=self.column_family,
-                 cassandra_options=self.cassandra_options))
+                 table=self.table))
         return super(CassandraBackend, self).__reduce__(args, kwargs)

+ 0 - 226
celery/backends/new_cassandra.py

@@ -1,226 +0,0 @@
-# -* coding: utf-8 -*-
-"""
-    celery.backends.new_cassandra
-    ~~~~~~~~~~~~~~~~~~~~~~~~~~~~~
-
-    Apache Cassandra result store backend using DataStax driver
-
-"""
-from __future__ import absolute_import
-
-import sys
-try:  # pragma: no cover
-    import cassandra
-    import cassandra.cluster
-except ImportError:  # pragma: no cover
-    cassandra = None   # noqa
-
-from celery import states
-from celery.exceptions import ImproperlyConfigured
-from celery.utils.log import get_logger
-from .base import BaseBackend
-
-__all__ = ['CassandraBackend']
-
-logger = get_logger(__name__)
-
-E_NO_CASSANDRA = """
-You need to install the cassandra-driver library to
-use the Cassandra backend. See https://github.com/datastax/python-driver
-"""
-
-Q_INSERT_RESULT = """
-INSERT INTO {table} (
-    task_id, status, result, date_done, traceback, children) VALUES (
-        %s, %s, %s, %s, %s, %s) {expires};
-"""
-
-Q_SELECT_RESULT = """
-SELECT status, result, date_done, traceback, children
-FROM {table}
-WHERE task_id=%s
-LIMIT 1
-"""
-
-Q_CREATE_RESULT_TABLE = """
-CREATE TABLE {table} (
-    task_id text,
-    status text,
-    result blob,
-    date_done timestamp,
-    traceback blob,
-    children blob,
-    PRIMARY KEY ((task_id), date_done)
-) WITH CLUSTERING ORDER BY (date_done DESC);
-"""
-
-Q_EXPIRES = """
-    USING TTL {0}
-"""
-
-if sys.version_info[0] == 3:
-    def buf_t(x):
-        return bytes(x, 'utf8')
-else:
-    buf_t = buffer  # noqa
-
-
-class CassandraBackend(BaseBackend):
-    """Cassandra backend utilizing DataStax driver
-
-    :raises celery.exceptions.ImproperlyConfigured: if
-        module :mod:`cassandra` is not available.
-
-    """
-
-    #: List of Cassandra servers with format: ``hostname``.
-    servers = None
-
-    supports_autoexpire = True      # autoexpire supported via entry_ttl
-
-    def __init__(self, servers=None, keyspace=None, table=None, entry_ttl=None,
-                 port=9042, **kwargs):
-        """Initialize Cassandra backend.
-
-        Raises :class:`celery.exceptions.ImproperlyConfigured` if
-        the :setting:`cassandra_servers` setting is not set.
-
-        """
-        super(CassandraBackend, self).__init__(**kwargs)
-
-        if not cassandra:
-            raise ImproperlyConfigured(E_NO_CASSANDRA)
-
-        conf = self.app.conf
-        self.servers = (servers or
-                        conf.get('cassandra_servers', None))
-        self.port = (port or
-                     conf.get('cassandra_port', None))
-        self.keyspace = (keyspace or
-                         conf.get('cassandra_keyspace', None))
-        self.table = (table or
-                      conf.get('cassandra_table', None))
-
-        if not self.servers or not self.keyspace or not self.table:
-            raise ImproperlyConfigured('Cassandra backend not configured.')
-
-        expires = (entry_ttl or conf.get('cassandra_entry_ttl', None))
-
-        self.cqlexpires = (Q_EXPIRES.format(expires)
-                           if expires is not None else '')
-
-        read_cons = conf.get('cassandra_read_consistency') or 'LOCAL_QUORUM'
-        write_cons = conf.get('cassandra_write_consistency') or 'LOCAL_QUORUM'
-
-        self.read_consistency = getattr(
-            cassandra.ConsistencyLevel, read_cons,
-            cassandra.ConsistencyLevel.LOCAL_QUORUM,
-        )
-        self.write_consistency = getattr(
-            cassandra.ConsistencyLevel, write_cons,
-            cassandra.ConsistencyLevel.LOCAL_QUORUM,
-        )
-
-        self._connection = None
-        self._session = None
-        self._write_stmt = None
-        self._read_stmt = None
-
-    def process_cleanup(self):
-        if self._connection is not None:
-            self._connection = None
-        if self._session is not None:
-            self._session.shutdown()
-            self._session = None
-
-    def _get_connection(self, write=False):
-        """Prepare the connection for action
-
-        :param write: bool - are we a writer?
-
-        """
-        if self._connection is None:
-            try:
-                self._connection = cassandra.cluster.Cluster(self.servers,
-                                                             port=self.port)
-                self._session = self._connection.connect(self.keyspace)
-
-                # We are forced to do concatenation below, as formatting would
-                # blow up on superficial %s that will be processed by Cassandra
-                self._write_stmt = cassandra.query.SimpleStatement(
-                    Q_INSERT_RESULT.format(
-                        table=self.table, expires=self.cqlexpires),
-                )
-                self._write_stmt.consistency_level = self.write_consistency
-
-                self._read_stmt = cassandra.query.SimpleStatement(
-                    Q_SELECT_RESULT.format(table=self.table),
-                )
-                self._read_stmt.consistency_level = self.read_consistency
-
-                if write:
-                    # Only possible writers "workers" are allowed to issue
-                    # CREATE TABLE. This is to prevent conflicting situations
-                    # where both task-creator and task-executor would issue it
-                    # at the same time.
-
-                    # Anyway; if you're doing anything critical, you should
-                    # have created this table in advance, in which case
-                    # this query will be a no-op (AlreadyExists)
-                    self._make_stmt = cassandra.query.SimpleStatement(
-                        Q_CREATE_RESULT_TABLE.format(table=self.table),
-                    )
-                    self._make_stmt.consistency_level = self.write_consistency
-                    try:
-                        self._session.execute(self._make_stmt)
-                    except cassandra.AlreadyExists:
-                        pass
-
-            except cassandra.OperationTimedOut:
-                # a heavily loaded or gone Cassandra cluster failed to respond.
-                # leave this class in a consistent state
-                self._connection = None
-                if self._session is not None:
-                    self._session.shutdown()
-
-                raise   # we did fail after all - reraise
-
-    def _store_result(self, task_id, result, status,
-                      traceback=None, request=None, **kwargs):
-        """Store return value and status of an executed task."""
-        self._get_connection(write=True)
-
-        self._session.execute(self._write_stmt, (
-            task_id,
-            status,
-            buf_t(self.encode(result)),
-            self.app.now(),
-            buf_t(self.encode(traceback)),
-            buf_t(self.encode(self.current_task_children(request)))
-        ))
-
-    def _get_task_meta_for(self, task_id):
-        """Get task metadata for a task by id."""
-        self._get_connection()
-
-        res = self._session.execute(self._read_stmt, (task_id, ))
-        if not res:
-            return {'status': states.PENDING, 'result': None}
-
-        status, result, date_done, traceback, children = res[0]
-
-        return self.meta_from_decoded({
-            'task_id': task_id,
-            'status': status,
-            'result': self.decode(result),
-            'date_done': date_done.strftime('%Y-%m-%dT%H:%M:%SZ'),
-            'traceback': self.decode(traceback),
-            'children': self.decode(children),
-        })
-
-    def __reduce__(self, args=(), kwargs={}):
-        kwargs.update(
-            dict(servers=self.servers,
-                 keyspace=self.keyspace,
-                 table=self.table))
-        return super(CassandraBackend, self).__reduce__(args, kwargs)

+ 62 - 117
celery/tests/backends/test_cassandra.py

@@ -1,74 +1,47 @@
 from __future__ import absolute_import
 
-import socket
-
 from pickle import loads, dumps
+from datetime import datetime
 
 from celery import states
 from celery.exceptions import ImproperlyConfigured
 from celery.tests.case import (
-    AppCase, Mock, mock_module, depends_on_current_app,
+    AppCase, Mock, mock_module, depends_on_current_app
 )
 
+CASSANDRA_MODULES = ['cassandra', 'cassandra.cluster']
+
 
 class Object(object):
     pass
 
 
-def install_exceptions(mod):
-    # py3k: cannot catch exceptions not ineheriting from BaseException.
-
-    class NotFoundException(Exception):
-        pass
-
-    class TException(Exception):
-        pass
-
-    class InvalidRequestException(Exception):
-        pass
-
-    class UnavailableException(Exception):
-        pass
-
-    class TimedOutException(Exception):
-        pass
-
-    class AllServersUnavailable(Exception):
-        pass
-
-    mod.NotFoundException = NotFoundException
-    mod.TException = TException
-    mod.InvalidRequestException = InvalidRequestException
-    mod.TimedOutException = TimedOutException
-    mod.UnavailableException = UnavailableException
-    mod.AllServersUnavailable = AllServersUnavailable
-
-
 class test_CassandraBackend(AppCase):
 
     def setup(self):
         self.app.conf.update(
             cassandra_servers=['example.com'],
-            cassandra_keyspace='keyspace',
-            cassandra_column_family='columns',
+            cassandra_keyspace='celery',
+            cassandra_table='task_results',
         )
 
-    def test_init_no_pycassa(self):
-        with mock_module('pycassa'):
+    def test_init_no_cassandra(self):
+        """should raise ImproperlyConfigured when no python-driver
+        installed."""
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
-            prev, mod.pycassa = mod.pycassa, None
+            prev, mod.cassandra = mod.cassandra, None
             try:
                 with self.assertRaises(ImproperlyConfigured):
                     mod.CassandraBackend(app=self.app)
             finally:
-                mod.pycassa = prev
+                mod.cassandra = prev
 
     def test_init_with_and_without_LOCAL_QUROM(self):
-        with mock_module('pycassa'):
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
-            mod.pycassa = Mock()
-            install_exceptions(mod.pycassa)
-            cons = mod.pycassa.ConsistencyLevel = Object()
+            mod.cassandra = Mock()
+            cons = mod.cassandra.ConsistencyLevel = Object()
             cons.LOCAL_QUORUM = 'foo'
 
             self.app.conf.cassandra_read_consistency = 'LOCAL_FOO'
@@ -87,104 +60,76 @@ class test_CassandraBackend(AppCase):
 
     @depends_on_current_app
     def test_reduce(self):
-        with mock_module('pycassa'):
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends.cassandra import CassandraBackend
             self.assertTrue(loads(dumps(CassandraBackend(app=self.app))))
 
     def test_get_task_meta_for(self):
-        with mock_module('pycassa'):
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
-            mod.pycassa = Mock()
-            install_exceptions(mod.pycassa)
-            mod.Thrift = Mock()
-            install_exceptions(mod.Thrift)
+            mod.cassandra = Mock()
             x = mod.CassandraBackend(app=self.app)
-            Get_Column = x._get_column_family = Mock()
-            get_column = Get_Column.return_value = Mock()
-            get = get_column.get
-            META = get.return_value = {
-                'task_id': 'task_id',
-                'status': states.SUCCESS,
-                'result': '1',
-                'date_done': 'date',
-                'traceback': '',
-                'children': None,
-            }
+            x._connection = True
+            session = x._session = Mock()
+            execute = session.execute = Mock()
+            execute.return_value = [
+                [states.SUCCESS, '1', datetime.now(), b'', b'']
+            ]
             x.decode = Mock()
-            x.detailed_mode = False
-            meta = x._get_task_meta_for('task_id')
-            self.assertEqual(meta['status'], states.SUCCESS)
-
-            x.detailed_mode = True
-            row = get.return_value = Mock()
-            row.values.return_value = [Mock()]
-            x.decode.return_value = META
             meta = x._get_task_meta_for('task_id')
             self.assertEqual(meta['status'], states.SUCCESS)
-            x.decode.return_value = Mock()
 
-            x.detailed_mode = False
-            get.side_effect = KeyError()
+            x._session.execute.return_value = []
             meta = x._get_task_meta_for('task_id')
             self.assertEqual(meta['status'], states.PENDING)
 
-            calls = [0]
-            end = [10]
-
-            def work_eventually(*arg):
-                try:
-                    if calls[0] > end[0]:
-                        return META
-                    raise socket.error()
-                finally:
-                    calls[0] += 1
-            get.side_effect = work_eventually
-            x._retry_timeout = 10
-            x._retry_wait = 0.01
-            meta = x._get_task_meta_for('task')
-            self.assertEqual(meta['status'], states.SUCCESS)
-
-            x._retry_timeout = 0.1
-            calls[0], end[0] = 0, 100
-            with self.assertRaises(socket.error):
-                x._get_task_meta_for('task')
-
     def test_store_result(self):
-        with mock_module('pycassa'):
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
-            mod.pycassa = Mock()
-            install_exceptions(mod.pycassa)
-            mod.Thrift = Mock()
-            install_exceptions(mod.Thrift)
-            x = mod.CassandraBackend(app=self.app)
-            Get_Column = x._get_column_family = Mock()
-            cf = Get_Column.return_value = Mock()
-            x.detailed_mode = False
-            x._store_result('task_id', 'result', states.SUCCESS)
-            self.assertTrue(cf.insert.called)
+            mod.cassandra = Mock()
 
-            cf.insert.reset()
-            x.detailed_mode = True
+            x = mod.CassandraBackend(app=self.app)
+            x._connection = True
+            session = x._session = Mock()
+            session.execute = Mock()
             x._store_result('task_id', 'result', states.SUCCESS)
-            self.assertTrue(cf.insert.called)
 
     def test_process_cleanup(self):
-        with mock_module('pycassa'):
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
             x = mod.CassandraBackend(app=self.app)
-            x._column_family = None
             x.process_cleanup()
 
-            x._column_family = True
-            x.process_cleanup()
-            self.assertIsNone(x._column_family)
+            self.assertIsNone(x._connection)
+            self.assertIsNone(x._session)
 
-    def test_get_column_family(self):
-        with mock_module('pycassa'):
+    def test_timeouting_cluster(self):
+        """
+        Tests behaviour when Cluster.connect raises cassandra.OperationTimedOut
+        """
+        with mock_module(*CASSANDRA_MODULES):
             from celery.backends import cassandra as mod
-            mod.pycassa = Mock()
-            install_exceptions(mod.pycassa)
+
+            class OTOExc(Exception):
+                pass
+
+            class VeryFaultyCluster(object):
+                def __init__(self, *args, **kwargs):
+                    pass
+
+                def connect(self, *args, **kwargs):
+                    raise OTOExc()
+
+            mod.cassandra = Mock()
+            mod.cassandra.OperationTimedOut = OTOExc
+            mod.cassandra.cluster = Mock()
+            mod.cassandra.cluster.Cluster = VeryFaultyCluster
+
             x = mod.CassandraBackend(app=self.app)
-            self.assertTrue(x._get_column_family())
-            self.assertIsNotNone(x._column_family)
-            self.assertIs(x._get_column_family(), x._column_family)
+
+            with self.assertRaises(OTOExc):
+                x._store_result('task_id', 'result', states.SUCCESS)
+            self.assertIsNone(x._connection)
+            self.assertIsNone(x._session)
+
+            x.process_cleanup()  # should not raise

+ 0 - 135
celery/tests/backends/test_new_cassandra.py

@@ -1,135 +0,0 @@
-from __future__ import absolute_import
-
-from pickle import loads, dumps
-from datetime import datetime
-
-from celery import states
-from celery.exceptions import ImproperlyConfigured
-from celery.tests.case import (
-    AppCase, Mock, mock_module, depends_on_current_app
-)
-
-CASSANDRA_MODULES = ['cassandra', 'cassandra.cluster']
-
-
-class Object(object):
-    pass
-
-
-class test_CassandraBackend(AppCase):
-
-    def setup(self):
-        self.app.conf.update(
-            cassandra_servers=['example.com'],
-            cassandra_keyspace='celery',
-            cassandra_table='task_results',
-        )
-
-    def test_init_no_cassandra(self):
-        """should raise ImproperlyConfigured when no python-driver
-        installed."""
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-            prev, mod.cassandra = mod.cassandra, None
-            try:
-                with self.assertRaises(ImproperlyConfigured):
-                    mod.CassandraBackend(app=self.app)
-            finally:
-                mod.cassandra = prev
-
-    def test_init_with_and_without_LOCAL_QUROM(self):
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-            mod.cassandra = Mock()
-            cons = mod.cassandra.ConsistencyLevel = Object()
-            cons.LOCAL_QUORUM = 'foo'
-
-            self.app.conf.cassandra_read_consistency = 'LOCAL_FOO'
-            self.app.conf.cassandra_write_consistency = 'LOCAL_FOO'
-
-            mod.CassandraBackend(app=self.app)
-            cons.LOCAL_FOO = 'bar'
-            mod.CassandraBackend(app=self.app)
-
-            # no servers raises ImproperlyConfigured
-            with self.assertRaises(ImproperlyConfigured):
-                self.app.conf.cassandra_servers = None
-                mod.CassandraBackend(
-                    app=self.app, keyspace='b', column_family='c',
-                )
-
-    @depends_on_current_app
-    def test_reduce(self):
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends.new_cassandra import CassandraBackend
-            self.assertTrue(loads(dumps(CassandraBackend(app=self.app))))
-
-    def test_get_task_meta_for(self):
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-            mod.cassandra = Mock()
-            x = mod.CassandraBackend(app=self.app)
-            x._connection = True
-            session = x._session = Mock()
-            execute = session.execute = Mock()
-            execute.return_value = [
-                [states.SUCCESS, '1', datetime.now(), b'', b'']
-            ]
-            x.decode = Mock()
-            meta = x._get_task_meta_for('task_id')
-            self.assertEqual(meta['status'], states.SUCCESS)
-
-            x._session.execute.return_value = []
-            meta = x._get_task_meta_for('task_id')
-            self.assertEqual(meta['status'], states.PENDING)
-
-    def test_store_result(self):
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-            mod.cassandra = Mock()
-
-            x = mod.CassandraBackend(app=self.app)
-            x._connection = True
-            session = x._session = Mock()
-            session.execute = Mock()
-            x._store_result('task_id', 'result', states.SUCCESS)
-
-    def test_process_cleanup(self):
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-            x = mod.CassandraBackend(app=self.app)
-            x.process_cleanup()
-
-            self.assertIsNone(x._connection)
-            self.assertIsNone(x._session)
-
-    def test_timeouting_cluster(self):
-        """
-        Tests behaviour when Cluster.connect raises cassandra.OperationTimedOut
-        """
-        with mock_module(*CASSANDRA_MODULES):
-            from celery.backends import new_cassandra as mod
-
-            class OTOExc(Exception):
-                pass
-
-            class VeryFaultyCluster(object):
-                def __init__(self, *args, **kwargs):
-                    pass
-
-                def connect(self, *args, **kwargs):
-                    raise OTOExc()
-
-            mod.cassandra = Mock()
-            mod.cassandra.OperationTimedOut = OTOExc
-            mod.cassandra.cluster = Mock()
-            mod.cassandra.cluster.Cluster = VeryFaultyCluster
-
-            x = mod.CassandraBackend(app=self.app)
-
-            with self.assertRaises(OTOExc):
-                x._store_result('task_id', 'result', states.SUCCESS)
-            self.assertIsNone(x._connection)
-            self.assertIsNone(x._session)
-
-            x.process_cleanup()  # should not raise

+ 11 - 11
docs/configuration.rst

@@ -374,9 +374,9 @@ Can be one of the following:
     Use `MongoDB`_ to store the results.
     See :ref:`conf-mongodb-result-backend`.
 
-* new_cassandra
-    Use `Cassandra`_ to store the results, using newer database driver than _cassandra_.
-    See :ref:`conf-new_cassandra-result-backend`.
+* cassandra
+    Use `Cassandra`_ to store the results.
+    See :ref:`conf-cassandra-result-backend`.
 
 * ironcache
     Use `IronCache`_ to store the results.
@@ -742,10 +742,10 @@ Example configuration
         'taskmeta_collection': 'my_taskmeta_collection',
     }
 
-.. _conf-new_cassandra-result-backend:
+.. _conf-cassandra-result-backend:
 
-new_cassandra backend settings
-------------------------------
+cassandra backend settings
+--------------------------
 
 .. note::
 
@@ -786,14 +786,14 @@ The keyspace in which to store the results. e.g.::
 
     cassandra_keyspace = 'tasks_keyspace'
 
-.. setting:: cassandra_column_family
+.. setting:: cassandra_table
 
-cassandra_column_family
-~~~~~~~~~~~~~~~~~~~~~~~
+cassandra_table
+~~~~~~~~~~~~~~~
 
 The table (column family) in which to store the results. e.g.::
 
-    cassandra_column_family = 'tasks'
+    cassandra_table = 'tasks'
 
 .. setting:: cassandra_read_consistency
 
@@ -826,7 +826,7 @@ Example configuration
 
     cassandra_servers = ['localhost']
     cassandra_keyspace = 'celery'
-    cassandra_column_family = 'task_results'
+    cassandra_table = 'tasks'
     cassandra_read_consistency = 'ONE'
     cassandra_write_consistency = 'ONE'
     cassandra_entry_ttl = 86400

+ 0 - 3
docs/includes/installation.txt

@@ -78,9 +78,6 @@ Transports and Backends
     for using memcached as a result backend.
 
 :celery[cassandra]:
-    for using Apache Cassandra as a result backend with pycassa driver.
-
-:celery[new_cassandra]:
     for using Apache Cassandra as a result backend with DataStax driver.
 
 :celery[couchdb]:

+ 0 - 11
docs/internals/reference/celery.backends.new_cassandra.rst

@@ -1,11 +0,0 @@
-================================================
- celery.backends.new_cassandra
-================================================
-
-.. contents::
-    :local:
-.. currentmodule:: celery.backends.new_cassandra
-
-.. automodule:: celery.backends.new_cassandra
-    :members:
-    :undoc-members:

+ 0 - 1
docs/internals/reference/index.rst

@@ -32,7 +32,6 @@
     celery.backends.mongodb
     celery.backends.redis
     celery.backends.riak
-    celery.backends.new_cassandra
     celery.backends.cassandra
     celery.backends.couchbase
     celery.app.trace

+ 6 - 2
docs/whatsnew-4.0.rst

@@ -101,8 +101,12 @@ Bla bla
 
 New Cassandra Backend
 =====================
-New Cassandra backend will be called new_cassandra and utilize python-driver.
-Old backend is now deprecated.
+
+The new Cassandra backend utilizes the python-driver library.
+Old backend is deprecated and everyone using cassandra is required to upgrade
+to be using the new driver.
+
+# XXX What changed?
 
 
 Event Batching

+ 1 - 1
requirements/extras/cassandra.txt

@@ -1 +1 @@
-pycassa
+cassandra-driver

+ 0 - 1
requirements/extras/new_cassandra.txt

@@ -1 +0,0 @@
-cassandra-driver

+ 0 - 1
setup.py

@@ -200,7 +200,6 @@ features = set([
     'eventlet', 'gevent', 'msgpack', 'yaml', 'redis',
     'mongodb', 'sqs', 'couchdb', 'riak', 'beanstalk', 'zookeeper',
     'zeromq', 'sqlalchemy', 'librabbitmq', 'pyro', 'slmq',
-    'new_cassandra',
 ])
 extras_require = dict((x, extras(x + '.txt')) for x in features)
 extra['extras_require'] = extras_require