new_cassandra is now cassandra

This commit is contained in:
Ask Solem 2015-11-04 18:57:23 -08:00
Родитель 757678a59a
Коммит 081c78fffe
15 изменённых файлов: 245 добавлений и 644 удалений

Просмотреть файл

@ -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]:

Просмотреть файл

@ -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(

Просмотреть файл

@ -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',

Просмотреть файл

@ -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
"""
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):
"""Highly fault tolerant Cassandra backend.
.. attribute:: servers
List of Cassandra servers with format: ``hostname:port``.
"""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.read_consistency = getattr(
cassandra.ConsistencyLevel, read_cons,
cassandra.ConsistencyLevel.LOCAL_QUORUM,
)
self.write_consistency = getattr(
cassandra.ConsistencyLevel, write_cons,
cassandra.ConsistencyLevel.LOCAL_QUORUM,
)
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._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}
res = self._session.execute(self._read_stmt, (task_id, ))
if not res:
return {'status': states.PENDING, 'result': None}
return self._retry_on_error(_do_get)
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)

Просмотреть файл

@ -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)

Просмотреть файл

@ -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

Просмотреть файл

@ -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

Просмотреть файл

@ -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

Просмотреть файл

@ -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]:

Просмотреть файл

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

Просмотреть файл

@ -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

Просмотреть файл

@ -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 @@
pycassa
cassandra-driver

Просмотреть файл

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

Просмотреть файл

@ -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