Document reactors, metadata, rename pyev* to libev*

This commit is contained in:
Tyler Hobbs
2013-07-02 16:49:23 -05:00
parent 2e9cc06873
commit 5d95a6b4f7
22 changed files with 317 additions and 58 deletions

View File

@@ -18,10 +18,10 @@ log.addHandler(handler)
supported_reactors = [AsyncoreConnection]
try:
from cassandra.io.pyevreactor import PyevConnection
supported_reactors.append(PyevConnection)
from cassandra.io.libevreactor import LibevConnection
supported_reactors.append(LibevConnection)
except ImportError, exc:
log.warning("Not benchmarking pyev reactor: %s" % (exc,))
log.warning("Not benchmarking libev reactor: %s" % (exc,))
KEYSPACE = "testkeyspace"
TABLE = "testtable"

View File

@@ -152,9 +152,21 @@ class WriteTimeout(Timeout):
class AlreadyExists(Exception):
"""
An attempt was made to create a keyspace or table that already exists.
"""
keyspace = None
"""
The name of the keyspace that already exists, or, if an attempt was
made to create a new table, the keyspace that the table is in.
"""
table = None
"""
The name of the table that already exists, or, if an attempt was
make to create a keyspace, ``None``.
"""
def __init__(self, keyspace=None, table=None):
if table:
@@ -168,4 +180,22 @@ class AlreadyExists(Exception):
class InvalidRequest(Exception):
"""
A query was made that was invalid for some reason, such as trying to set
the keyspace for a connection to a nonexistent keyspace.
"""
pass
class Unauthorized(Exception):
"""
The current user is not authorized to perfom the requested operation.
"""
pass
class AuthenticationFailed(Exception):
"""
Failed to authenticate.
"""
pass

View File

@@ -13,7 +13,7 @@ import weakref
from functools import partial
from itertools import groupby
from cassandra import ConsistencyLevel
from cassandra import ConsistencyLevel, AuthenticationFailed
from cassandra.connection import ConnectionException
from cassandra.decoder import (QueryMessage, ResultMessage,
ErrorMessage, ReadTimeoutErrorMessage,
@@ -30,8 +30,8 @@ from cassandra.policies import (RoundRobinPolicy, SimpleConvictionPolicy,
ExponentialReconnectionPolicy, HostDistance,
RetryPolicy)
from cassandra.query import SimpleStatement, PreparedStatement, BoundStatement, bind_params
from cassandra.pool import (AuthenticationException, _ReconnectionHandler,
_HostReconnectionHandler, HostConnectionPool)
from cassandra.pool import (_ReconnectionHandler, _HostReconnectionHandler,
HostConnectionPool)
log = logging.getLogger(__name__)
@@ -155,11 +155,11 @@ class Cluster(object):
I/O with Cassandra. These are the current options:
* :class:`cassandra.io.asyncorereactor.AsyncoreConnection`
* :class:`cassandra.io.pyevreactor.PyevConnection`
* :class:`cassandra.io.libevreactor.LibevConnection`
By default, ``AsyncoreConnection`` will be used, which uses
the ``asyncore`` module in the Python standard library. The
performance is slightly worse than with ``pyev``, but it is
performance is slightly worse than with ``libev``, but it is
supported on a wider range of systems.
"""
@@ -715,7 +715,7 @@ class Session(object):
else:
try:
new_pool = HostConnectionPool(host, distance, self)
except AuthenticationException, auth_exc:
except AuthenticationFailed, auth_exc:
conn_exc = ConnectionException(str(auth_exc), host=host)
host.monitor.signal_connection_failure(conn_exc)
return self._pools.get(host)
@@ -803,7 +803,7 @@ class _ControlReconnectionHandler(_ReconnectionHandler):
def on_exception(self, exc, next_delay):
# TODO only overridden to add logging, so add logging
if isinstance(exc, AuthenticationException):
if isinstance(exc, AuthenticationFailed):
return False
else:
log.debug("Error trying to reconnect control connection: %r" % (exc,))

View File

@@ -1,10 +1,10 @@
import errno
from functools import wraps
from functools import wraps, partial
import logging
from threading import Event, Lock, RLock
from Queue import Queue
from cassandra import ConsistencyLevel
from cassandra import ConsistencyLevel, AuthenticationFailed
from cassandra.marshal import int8_unpack
from cassandra.decoder import (ReadyMessage, AuthenticateMessage, OptionsMessage,
StartupMessage, ErrorMessage, CredentialsMessage,
@@ -41,6 +41,10 @@ NONBLOCKING = (errno.EAGAIN, errno.EWOULDBLOCK)
class ConnectionException(Exception):
"""
An unrecoverable error was hit when attempting to use a connection,
or the connection was already closed or defunct.
"""
def __init__(self, message, host=None):
Exception.__init__(self, message)
@@ -48,14 +52,17 @@ class ConnectionException(Exception):
class ConnectionBusy(Exception):
pass
class ProgrammingError(Exception):
"""
An attempt was made to send a message through a :class:`.Connection` that
was already at the max number of in-flight operations.
"""
pass
class ProtocolError(Exception):
"""
Communication did not match the protocol that this driver expects.
"""
pass
@@ -214,7 +221,7 @@ class Connection(object):
self.send_msg(sm, cb=self._handle_startup_response)
@defunct_on_error
def _handle_startup_response(self, startup_response):
def _handle_startup_response(self, startup_response, did_authenticate=False):
if self.is_defunct:
return
if isinstance(startup_response, ReadyMessage):
@@ -226,17 +233,23 @@ class Connection(object):
log.debug("Got AuthenticateMessage on new Connection from %s" % self.host)
if self.credentials is None:
raise ProgrammingError('Remote end requires authentication.')
raise AuthenticationFailed('Remote end requires authentication.')
self.authenticator = startup_response.authenticator
cm = CredentialsMessage(creds=self.credentials)
self.send_msg(cm, cb=self._handle_startup_response)
callback = partial(self._handle_startup_response, did_authenticate=True)
self.send_msg(cm, cb=callback)
elif isinstance(startup_response, ErrorMessage):
log.debug("Received ErrorMessage on new Connection from %s: %s"
% (self.host, startup_response.summary_msg()))
raise ConnectionException(
"Failed to initialize new connection to %s: %s"
% (self.host, startup_response.summary_msg()))
if did_authenticate:
raise AuthenticationFailed(
"Failed to authenticate to %s: %s" %
(self.host, startup_response.summary_msg()))
else:
raise ConnectionException(
"Failed to initialize new connection to %s: %s"
% (self.host, startup_response.summary_msg()))
else:
msg = "Unexpected response during Connection setup: %r" % (startup_response,)
log.error(msg)

View File

@@ -25,7 +25,7 @@ except ImportError:
from StringIO import StringIO # ignore flake8 warning: # NOQA
from cassandra import (ConsistencyLevel, Unavailable, WriteTimeout, ReadTimeout,
AlreadyExists, InvalidRequest)
AlreadyExists, InvalidRequest, Unauthorized)
from cassandra.marshal import (int32_pack, int32_unpack, uint16_pack, uint16_unpack,
int8_pack, int8_unpack)
from cassandra.cqltypes import lookup_cqltype
@@ -278,6 +278,9 @@ class UnauthorizedErrorMessage(RequestValidationException):
summary = 'Unauthorized'
error_code = 0x2100
def to_exception(self):
return Unauthorized(self.summary_msg())
class InvalidRequestException(RequestValidationException):
summary = 'Invalid query'

View File

@@ -43,6 +43,10 @@ def _start_loop():
class AsyncoreConnection(Connection, asyncore.dispatcher):
"""
An implementation of :class:`.Connection` that utilizes the ``asyncore``
module in the Python standard library for its event loop.
"""
_buf = ""
_total_reqd_bytes = 0

View File

@@ -70,7 +70,11 @@ def defunct_on_error(f):
return wrapper
class PyevConnection(Connection):
class LibevConnection(Connection):
"""
An implementation of :class:`.Connection` that utilizes libev through
the pyev library for its event loop.
"""
_buf = ""
_total_reqd_bytes = 0

View File

@@ -41,11 +41,11 @@ class Metadata(object):
keyspaces = None
"""
A map from keyspace names to matching :cls:`~.KeyspaceMetadata` instances.
A map from keyspace names to matching :class:`~.KeyspaceMetadata` instances.
"""
token_map = None
""" A :cls:`~.TokenMap` instance. """
""" A :class:`~.TokenMap` instance describing the ring topology. """
def __init__(self, cluster):
# use a weak reference so that the Cluster object can be GC'ed.
@@ -57,9 +57,19 @@ class Metadata(object):
self._hosts_lock = RLock()
def export_schema_as_string(self):
"""
Returns a string that can be executed as a query in order to recreate
the entire schema. The string is formatted to be human readable.
"""
return "\n".join(ks.export_as_string() for ks in self.keyspaces.values())
def rebuild_schema(self, keyspace, table, ks_results, cf_results, col_results):
"""
Rebuild the view of the current schema from a fresh set of rows from
the system schema tables.
For internal use only.
"""
cf_def_rows = defaultdict(list)
col_def_rows = defaultdict(lambda: defaultdict(list))
@@ -227,6 +237,11 @@ class Metadata(object):
return None
def rebuild_token_map(self, partitioner, token_map):
"""
Rebuild our view of the topology from fresh rows from the
system topology tables.
For internal use only.
"""
if partitioner.endswith('RandomPartitioner'):
token_cls = MD5Token
elif partitioner.endswith('Murmur3Partitioner'):
@@ -249,6 +264,10 @@ class Metadata(object):
self.token_map = TokenMap(token_cls, tokens_to_hosts, ring)
def get_replicas(self, key):
"""
Returns a list of :class:`.Host` instances that are replicas for a given
partition key.
"""
t = self.token_map
return t.get_replicas(t.token_cls.from_key(key))
@@ -272,6 +291,9 @@ class Metadata(object):
return self._hosts.get(address)
def all_hosts(self):
"""
Returns a list of all known :class:`.Host` instances in the cluster.
"""
with self._hosts_lock:
return self._hosts.values()
@@ -299,7 +321,7 @@ class KeyspaceMetadata(object):
tables = None
"""
A map from table names to instances of :cls:`~.TableMetadata`.
A map from table names to instances of :class:`~.TableMetadata`.
"""
def __init__(self, name, durable_writes, strategy_class, strategy_options):
@@ -327,16 +349,46 @@ class TableMetadata(object):
"""
keyspace = None
""" An instance of :cls:`~.KeyspaceMetadata` """
""" An instance of :class:`~.KeyspaceMetadata` """
name = None
""" The string name of the table """
# TODO docstrings for these
partition_key = None
"""
A list of :class:`.ColumnMetadata` instances representing the columns in
the partition key for this table. This will always hold at least one
column.
"""
clustering_key = None
"""
A list of :class:`.ColumnMetadata` instances representing the columns
in the clustering key for this table. These are all of the
:attr:`.primary_key` columns that are not in the :attr:`.partition_key`.
Note that a table may have no clustering keys, in which case this will
be an empty list.
"""
@property
def primary_key(self):
"""
A list of :class:`.ColumnMetadata` representing the components of
the primary key for this table.
"""
return self.partition_key + self.clustering_key
columns = None
"""
A dict mapping column names to :class:`.ColumnMetadata` instances.
"""
options = None
"""
A dict mapping table option names to their specific settings for this
table.
"""
recognized_options = (
"comment", "read_repair_chance", # "local_read_repair_chance",
@@ -355,6 +407,11 @@ class TableMetadata(object):
self.comparator = None
def export_as_string(self):
"""
Returns a string of CQL queries that can be used to recreate this table
along with all indexes on it. The returned string is formatted to
be human readable.
"""
ret = self.as_cql_query(formatted=True)
ret += ";"
@@ -365,6 +422,11 @@ class TableMetadata(object):
return ret
def as_cql_query(self, formatted=False):
"""
Returns a CQL query that can be used to recreate this table (index
creations are not included). If `formatted` is set to ``True``,
extra whitespace will be added to make the query human readable.
"""
ret = "CREATE TABLE %s.%s (%s" % (self.keyspace.name, self.name, "\n" if formatted else "")
if formatted:
@@ -476,6 +538,23 @@ class TableMetadata(object):
class ColumnMetadata(object):
"""
A representation of a single column in a table.
"""
table = None
""" The :class:`.TableMetadata` this column belongs to. """
name = None
""" The string name of this column. """
data_type = None
index = None
"""
If an index exists on this column, this is an instance of
:class:`.IndexMetadata`, otherwise ``None``.
"""
def __init__(self, table_metadata, column_name, data_type, index_metadata=None):
self.table = table_metadata
@@ -485,6 +564,10 @@ class ColumnMetadata(object):
@property
def typestring(self):
"""
A string representation of the type for this column, such as "varchar"
or "map<string, int>".
"""
if issubclass(self.data_type, types.ReversedType):
return self.data_type.subtypes[0].cql_parameterized_type()
else:
@@ -495,6 +578,20 @@ class ColumnMetadata(object):
class IndexMetadata(object):
"""
A representation of a secondary index on a column.
"""
column = None
"""
The column (:class:`.ColumnMetadata`) this index is on.
"""
name = None
""" A string name for the index. """
index_type = None
""" A string representing the type of index. """
def __init__(self, column_metadata, index_name=None, index_type=None):
self.column = column_metadata
@@ -502,6 +599,9 @@ class IndexMetadata(object):
self.index_type = index_type
def as_cql_query(self):
"""
Returns a CQL query that can be used to recreate this index.
"""
table = self.column.table
return "CREATE INDEX %s ON %s.%s (%s)" % (self.name, table.keyspace.name, table.name, self.column.name)

View File

@@ -225,7 +225,7 @@ class DCAwareRoundRobinPolicy(LoadBalancingPolicy):
if dc == self.local_dc:
continue
for host in current_dc_hosts[:self.used_hosts_per_remote_dc]:
for host in list(current_dc_hosts)[:self.used_hosts_per_remote_dc]:
yield host
def on_up(self, host):

View File

@@ -1,30 +1,42 @@
"""
Connection pooling and host management.
"""
import logging
import time
from threading import Lock, RLock, Condition
import traceback
import weakref
from connection import MAX_STREAM_PER_CONNECTION, ConnectionException
from cassandra import AuthenticationFailed
from cassandra.connection import MAX_STREAM_PER_CONNECTION, ConnectionException
log = logging.getLogger(__name__)
class BusyConnectionException(Exception):
pass
class AuthenticationException(Exception):
pass
class NoConnectionsAvailable(Exception):
"""
All existing connections to a given host are busy, or there are
no open connections.
"""
pass
class Host(object):
"""
Represents a single Cassandra node.
"""
address = None
"""
The IP address or hostname of the node.
"""
monitor = None
"""
A :class:`.HealthMonitor` instance that tracks whether this node is
up or down.
"""
_datacenter = None
_rack = None
@@ -43,17 +55,28 @@ class Host(object):
@property
def datacenter(self):
""" The datacenter the node is in. """
return self._datacenter
@property
def rack(self):
""" The rack the node is in. """
return self._rack
def set_location_info(self, datacenter, rack):
"""
Sets the datacenter and rack for this node. Intended for internal
use (by the control connection, which periodically checks the
ring topology) only.
"""
self._datacenter = datacenter
self._rack = rack
def get_and_set_reconnection_handler(self, new_handler):
"""
Atomically replaces the reconnection handler for this
host. Intended for internal use only.
"""
with self._reconnection_lock:
old = self._reconnection_handler
self._reconnection_handler = new_handler
@@ -137,9 +160,9 @@ class _ReconnectionHandler(object):
Subclasses should return ``False`` if no more attempts to connection
should be made, ``True`` otherwise. The default behavior is to
always retry unless the error is an AuthenticationException.
always retry unless the error is an :exc:`.AuthenticationFailed`.
"""
if isinstance(exc, AuthenticationException):
if isinstance(exc, AuthenticationFailed):
return False
else:
return True
@@ -159,7 +182,7 @@ class _HostReconnectionHandler(_ReconnectionHandler):
self.host.monitor.reset()
def on_exception(self, exc, next_delay):
if isinstance(exc, AuthenticationException):
if isinstance(exc, AuthenticationFailed):
return False
else:
log.warn("Error attempting to reconnect to %s: %s", self.host, exc)
@@ -168,8 +191,17 @@ class _HostReconnectionHandler(_ReconnectionHandler):
class HealthMonitor(object):
"""
Monitors whether a particular host is marked as up or down.
This class is primarily intended for internal use, although
applications may find it useful to check whether a given node
is up or down.
"""
is_up = True
"""
A boolean representing the current state of the node.
"""
def __init__(self, conviction_policy):
self._conviction_policy = conviction_policy
@@ -352,7 +384,7 @@ class HostConnectionPool(object):
if self.host.monitor.signal_connection_failure(exc):
self.shutdown()
return False
except AuthenticationException:
except AuthenticationFailed:
with self._lock:
self.open_count -= 1
return False

View File

@@ -17,3 +17,12 @@
.. autoexception:: WriteTimeout()
:members:
.. autoexception:: InvalidRequest()
:members:
.. autoexception:: Unauthorized()
:members:
.. autoexception:: AuthenticationFailed()
:members:

View File

@@ -1,5 +1,5 @@
``cassandra.cluster``
=====================
``cassandra.cluster`` - Clusters and Sessions
=============================================
.. module:: cassandra.cluster

View File

@@ -0,0 +1,8 @@
``cassandra.connection`` - Low Level Connection Info
====================================================
.. module:: cassandra.connection
.. autoexception:: ConnectionException ()
.. autoexception:: ConnectionBusy ()
.. autoexception:: ProtocolError ()

View File

@@ -0,0 +1,7 @@
``cassandra.io.asyncorereactor`` - ``asyncore`` Event Loop
==========================================================
.. module:: cassandra.io.asyncorereactor
.. autoclass:: AsyncoreConnection
:members:

View File

@@ -0,0 +1,6 @@
``cassandra.io.libevreactor`` - ``libev`` Event Loop
====================================================
.. module:: cassandra.io.libevreactor
.. autoclass:: LibevConnection

View File

@@ -0,0 +1,35 @@
``cassandra.metadata`` - Schema and Ring Topology
=================================================
.. module:: cassandra.metadata
.. autoclass:: Metadata ()
:members:
:exclude-members: rebuild_schema, rebuild_token_map, add_host, remove_host, get_host
.. autoclass:: KeyspaceMetadata ()
:members:
.. autoclass:: TableMetadata ()
:members:
.. autoclass:: ColumnMetadata ()
:members:
.. autoclass:: IndexMetadata ()
:members:
.. autoclass:: TokenMap
:members:
.. autoclass:: Token
:members:
.. autoclass:: Murmur3Token
:members:
.. autoclass:: MD5Token
:members:
.. autoclass:: BytesToken
:members:

View File

@@ -0,0 +1,11 @@
``cassandra.pool`` - Hosts and Connection Pools
===============================================
.. automodule:: cassandra.pool
.. autoclass:: Host ()
:members:
:exclude-members: set_location_info, get_and_set_reconnection_handler
.. autoclass:: HealthMonitor ()
:members:

View File

@@ -1,5 +1,5 @@
``cassandra.query``
===================
``cassandra.query`` - Prepared Statements and Query Policies
============================================================
.. module:: cassandra.query

View File

@@ -15,4 +15,4 @@ Cassandra Modules
cassandra/pool
cassandra/connection
cassandra/io/asyncorereactor
cassandra/io/pyevreactor
cassandra/io/libevreactor

View File

@@ -5,7 +5,7 @@ from threading import Thread, Event
from cassandra import ConsistencyLevel
from cassandra.decoder import QueryMessage
from cassandra.io.asyncorereactor import AsyncoreConnection
from cassandra.io.pyevreactor import PyevConnection
from cassandra.io.libevreactor import LibevConnection
class ConnectionTest(object):
@@ -164,6 +164,6 @@ class AsyncoreConnectionTest(ConnectionTest, unittest.TestCase):
klass = AsyncoreConnection
class PyevConnectionTest(ConnectionTest, unittest.TestCase):
class LibevConnectionTest(ConnectionTest, unittest.TestCase):
klass = PyevConnection
klass = LibevConnection

View File

@@ -9,7 +9,7 @@ from cassandra.connection import (PROTOCOL_VERSION,
HEADER_DIRECTION_TO_CLIENT,
ProtocolError,
ConnectionException)
from cassandra.io.pyevreactor import PyevConnection
from cassandra.io.libevreactor import LibevConnection
from cassandra.decoder import (write_stringmultimap, write_int, write_string,
SupportedMessage, ReadyMessage, ServerError)
from cassandra.marshal import uint8_pack, uint32_pack
@@ -17,10 +17,10 @@ from cassandra.marshal import uint8_pack, uint32_pack
@patch('socket.socket')
@patch('pyev.Io')
@patch('cassandra.io.asyncorereactor._start_loop')
class PyevConnectionTest(unittest.TestCase):
class LibevConnectionTest(unittest.TestCase):
def make_connection(self):
c = PyevConnection('1.2.3.4')
c = LibevConnection('1.2.3.4')
c._socket = Mock()
c._socket.send.side_effect = lambda x: len(x)
return c

View File

@@ -1,7 +1,7 @@
from StringIO import StringIO
import unittest
from mock import patch, Mock, ANY
from mock import Mock, ANY
from cassandra.connection import (Connection, PROTOCOL_VERSION,
HEADER_DIRECTION_TO_CLIENT,
@@ -10,9 +10,6 @@ from cassandra.decoder import (write_stringmultimap, write_int, write_string,
SupportedMessage)
from cassandra.marshal import uint8_pack, uint32_pack
@patch('socket.socket')
@patch('pyev.Io')
@patch('cassandra.io.asyncorereactor._start_loop')
class ConnectionTest(unittest.TestCase):
def make_connection(self):