Update API documentation

This commit is contained in:
Tyler Hobbs
2014-01-17 14:16:25 -06:00
parent a8dd5f72c9
commit b918383313
8 changed files with 189 additions and 22 deletions

View File

@@ -221,4 +221,9 @@ class AuthenticationFailed(Exception):
class OperationTimedOut(Exception): class OperationTimedOut(Exception):
"""
The operation took longer than the specified (client-side) timeout
to complete. This is not an error generated by Cassandra, only
the driver.
"""
pass pass

View File

@@ -5,6 +5,7 @@ This module houses the main classes you will interact with,
from concurrent.futures import ThreadPoolExecutor from concurrent.futures import ThreadPoolExecutor
import logging import logging
import socket
import sys import sys
import time import time
from threading import Lock, RLock, Thread, Event from threading import Lock, RLock, Thread, Event
@@ -132,6 +133,13 @@ class Cluster(object):
The server-side port to open connections to. Defaults to 9042. The server-side port to open connections to. Defaults to 9042.
""" """
cql_version = None
"""
If a specific version of CQL should be used, this may be set to that
string version. Otherwise, the highest CQL version supported by the
server will be automatically used.
"""
compression = True compression = True
""" """
Whether or not compression should be enabled when possible. Defaults to Whether or not compression should be enabled when possible. Defaults to
@@ -173,7 +181,8 @@ class Cluster(object):
metrics_enabled = False metrics_enabled = False
""" """
Whether or not metric collection is enabled. Whether or not metric collection is enabled. If enabled, :attr:`.metrics`
will be an instance of :class:`.metrics.Metrics`.
""" """
metrics = None metrics = None
@@ -373,18 +382,42 @@ class Cluster(object):
self._max_requests_per_connection[host_distance] = max_requests self._max_requests_per_connection[host_distance] = max_requests
def get_core_connections_per_host(self, host_distance): def get_core_connections_per_host(self, host_distance):
"""
Gets the minimum number of connections that will be opened for each
host with :class:`~.HostDistance` equal to `host_distance`. The default
is 2 for :attr:`~HostDistance.LOCAL` and 1 for
:attr:`~HostDistance.REMOTE`.
"""
return self._core_connections_per_host[host_distance] return self._core_connections_per_host[host_distance]
def set_core_connections_per_host(self, host_distance, core_connections): def set_core_connections_per_host(self, host_distance, core_connections):
"""
Sets the minimum number of connections that will be opened for each
host with :class:`~.HostDistance` equal to `host_distance`. The default
is 2 for :attr:`~HostDistance.LOCAL` and 1 for
:attr:`~HostDistance.REMOTE`.
"""
old = self._core_connections_per_host[host_distance] old = self._core_connections_per_host[host_distance]
self._core_connections_per_host[host_distance] = core_connections self._core_connections_per_host[host_distance] = core_connections
if old < core_connections: if old < core_connections:
self.ensure_core_connections() self._ensure_core_connections()
def get_max_connections_per_host(self, host_distance): def get_max_connections_per_host(self, host_distance):
"""
Gets the maximum number of connections that will be opened for each
host with :class:`~.HostDistance` equal to `host_distance`. The default
is 8 for :attr:`~HostDistance.LOCAL` and 2 for
:attr:`~HostDistance.REMOTE`.
"""
return self._max_connections_per_host[host_distance] return self._max_connections_per_host[host_distance]
def set_max_connections_per_host(self, host_distance, max_connections): def set_max_connections_per_host(self, host_distance, max_connections):
"""
Gets the maximum number of connections that will be opened for each
host with :class:`~.HostDistance` equal to `host_distance`. The default
is 2 for :attr:`~HostDistance.LOCAL` and 1 for
:attr:`~HostDistance.REMOTE`.
"""
self._max_connections_per_host[host_distance] = max_connections self._max_connections_per_host[host_distance] = max_connections
def connection_factory(self, address, *args, **kwargs): def connection_factory(self, address, *args, **kwargs):
@@ -453,6 +486,9 @@ class Cluster(object):
def shutdown(self): def shutdown(self):
""" """
Closes all sessions and connection associated with this Cluster. Closes all sessions and connection associated with this Cluster.
To ensure all connections are properly closed, **you should always
call shutdown() on a Cluster instance when you are done with it**.
Once shutdown, a Cluster should not be used for any purpose. Once shutdown, a Cluster should not be used for any purpose.
""" """
with self._lock: with self._lock:
@@ -756,7 +792,7 @@ class Cluster(object):
with self._listener_lock: with self._listener_lock:
return self._listeners.copy() return self._listeners.copy()
def ensure_core_connections(self): def _ensure_core_connections(self):
""" """
If any host has fewer than the configured number of core connections If any host has fewer than the configured number of core connections
open, attempt to open connections until that number is met. open, attempt to open connections until that number is met.
@@ -810,8 +846,9 @@ class Cluster(object):
log.debug("Done preparing all known prepared statements against host %s", host) log.debug("Done preparing all known prepared statements against host %s", host)
except OperationTimedOut: except OperationTimedOut:
log.warn("Timed out trying to prepare all statements on host %s", host) log.warn("Timed out trying to prepare all statements on host %s", host)
except (ConnectionException, socket.error) as exc:
log.warn("Error trying to prepare all statements on host %s: %r", host, exc)
except Exception: except Exception:
# log and ignore
log.exception("Error trying to prepare all statements on host %s", host) log.exception("Error trying to prepare all statements on host %s", host)
finally: finally:
connection.close() connection.close()
@@ -864,13 +901,13 @@ class Session(object):
A default timeout, measured in seconds, for queries executed through A default timeout, measured in seconds, for queries executed through
:meth:`.execute()` or :meth:`.execute_async()`. This default may be :meth:`.execute()` or :meth:`.execute_async()`. This default may be
overridden with the `timeout` parameter for either of those methods overridden with the `timeout` parameter for either of those methods
or the `timeout` parameter for :meth:`~.ResponseFuture.result()`. or the `timeout` parameter for :meth:`.ResponseFuture.result()`.
Setting this to :const:`None` will cause no timeouts to be set by default. Setting this to :const:`None` will cause no timeouts to be set by default.
*Important*: This timeout currently has no effect on callbacks registered **Important**: This timeout currently has no effect on callbacks registered
on a :class:`~.ResponseFuture` through :meth:`~.ResponseFuture.add_callback` or on a :class:`~.ResponseFuture` through :meth:`.ResponseFuture.add_callback` or
:meth:`~.ResponseFuture.add_errback`; even if a query exceeds this default :meth:`.ResponseFuture.add_errback`; even if a query exceeds this default
timeout, neither the registered callback or errback will be called. timeout, neither the registered callback or errback will be called.
""" """
@@ -951,7 +988,7 @@ class Session(object):
Execute the given query and return a :class:`~.ResponseFuture` object Execute the given query and return a :class:`~.ResponseFuture` object
which callbacks may be attached to for asynchronous response which callbacks may be attached to for asynchronous response
delivery. You may also call :meth:`~.ResponseFuture.result()` delivery. You may also call :meth:`~.ResponseFuture.result()`
on the ``ResponseFuture`` to syncronously block for results at on the :class:`.ResponseFuture` to syncronously block for results at
any time. any time.
If `trace` is set to :const:`True`, you may call If `trace` is set to :const:`True`, you may call
@@ -1018,7 +1055,20 @@ class Session(object):
>>> session = cluster.connect("mykeyspace") >>> session = cluster.connect("mykeyspace")
>>> query = "INSERT INTO users (id, name, age) VALUES (?, ?, ?)" >>> query = "INSERT INTO users (id, name, age) VALUES (?, ?, ?)"
>>> prepared = session.prepare(query) >>> prepared = session.prepare(query)
>>> session.execute(prepared.bind((user.id, user.name, user.age))) >>> session.execute(prepared, (user.id, user.name, user.age))
Or you may bind values to the prepared statement ahead of time::
>>> prepared = session.prepare(query)
>>> bound_stmt = prepared.bind((user.id, user.name, user.age))
>>> session.execute(bound_stmt)
Of course, prepared statements may (and should) be reused::
>>> prepared = session.prepare(query)
>>> for user in users:
... bound = prepared.bind((user.id, user.name, user.age))
... session.execute(bound)
""" """
message = PrepareMessage(query=query) message = PrepareMessage(query=query)
@@ -1723,10 +1773,16 @@ class ResponseFuture(object):
:meth:`.add_callback()`, :meth:`.add_errback()`, and :meth:`.add_callback()`, :meth:`.add_errback()`, and
:meth:`.add_callbacks()`. :meth:`.add_callbacks()`.
""" """
query = None
"""
The :class:`~.Statement` instance that is being executed through this
:class:`.ResponseFuture`.
"""
session = None session = None
row_factory = None row_factory = None
message = None message = None
query = None
default_timeout = None default_timeout = None
_req_id = None _req_id = None
@@ -2071,6 +2127,12 @@ class ResponseFuture(object):
encountered. If the final result or error has not been set encountered. If the final result or error has not been set
yet, this method will block until that time. yet, this method will block until that time.
You may set a timeout (in seconds) with the `timeout` parameter.
By default, the :attr:`~.default_timeout` for the :class:`.Session`
this was created through will be used for the timeout on this
operation. If the timeout is exceeded, an
:exc:`cassandra.OperationTimedOut` will be raised.
Example usage:: Example usage::
>>> future = session.execute_async("SELECT * FROM mycf") >>> future = session.execute_async("SELECT * FROM mycf")
@@ -2128,6 +2190,10 @@ class ResponseFuture(object):
If the final result has already been seen when this method is called, If the final result has already been seen when this method is called,
the callback will be called immediately (before this method returns). the callback will be called immediately (before this method returns).
**Important**: if the callback you attach results in an exception being
raised, **the exception will be ignored**, so please ensure your
callback handles all error cases that you care about.
Usage example:: Usage example::
>>> session = cluster.connect("mykeyspace") >>> session = cluster.connect("mykeyspace")

View File

@@ -402,7 +402,11 @@ class ReplicationStrategy(object):
class SimpleStrategy(ReplicationStrategy): class SimpleStrategy(ReplicationStrategy):
name = "SimpleStrategy" name = "SimpleStrategy"
replication_factor = None replication_factor = None
"""
The replication factor for this keyspace.
"""
def __init__(self, replication_factor): def __init__(self, replication_factor):
self.replication_factor = int(replication_factor) self.replication_factor = int(replication_factor)
@@ -435,7 +439,11 @@ class SimpleStrategy(ReplicationStrategy):
class NetworkTopologyStrategy(ReplicationStrategy): class NetworkTopologyStrategy(ReplicationStrategy):
name = "NetworkTopologyStrategy" name = "NetworkTopologyStrategy"
dc_replication_factors = None dc_replication_factors = None
"""
A map of datacenter names to the replication factor for that DC.
"""
def __init__(self, dc_replication_factors): def __init__(self, dc_replication_factors):
self.dc_replication_factors = dc_replication_factors self.dc_replication_factors = dc_replication_factors

View File

@@ -38,11 +38,16 @@ class Host(object):
conviction_policy = None conviction_policy = None
""" """
A class:`ConvictionPolicy` instance for determining when this node should A :class:`~.ConvictionPolicy` instance for determining when this node should
be marked up or down. be marked up or down.
""" """
is_up = None is_up = None
"""
:const:`True` if the node is considered up, :const:`False` if it is
considered down, and :const:`None` if it is not known if the node is
up or down.
"""
_datacenter = None _datacenter = None
_rack = None _rack = None

View File

@@ -3,6 +3,14 @@
.. module:: cassandra .. module:: cassandra
.. data:: __version_info__
The version of the driver in a tuple format
.. data:: __version__
The version of the driver in a string format
.. autoclass:: ConsistencyLevel .. autoclass:: ConsistencyLevel
:members: :members:
@@ -18,6 +26,9 @@
.. autoexception:: WriteTimeout() .. autoexception:: WriteTimeout()
:members: :members:
.. autoexception:: AlreadyExists()
:members:
.. autoexception:: InvalidRequest() .. autoexception:: InvalidRequest()
:members: :members:
@@ -26,3 +37,6 @@
.. autoexception:: AuthenticationFailed() .. autoexception:: AuthenticationFailed()
:members: :members:
.. autoexception:: OperationTimedOut()
:members:

View File

@@ -4,16 +4,84 @@
.. module:: cassandra.cluster .. module:: cassandra.cluster
.. autoclass:: Cluster ([contact_points=('127.0.0.1',)][, port=9042][, executor_threads=2], **attr_kwargs) .. autoclass:: Cluster ([contact_points=('127.0.0.1',)][, port=9042][, executor_threads=2], **attr_kwargs)
:members:
:exclude-members: on_up, on_down, add_host, remove_host, connection_factory .. autoattribute:: cql_version
.. autoattribute:: port
.. autoattribute:: compression
.. autoattribute:: auth_provider
.. autoattribute:: load_balancing_policy
.. autoattribute:: reconnection_policy
.. autoattribute:: default_retry_policy
.. autoattribute:: conviction_policy_factory
.. autoattribute:: connection_class
.. autoattribute:: metrics_enabled
.. autoattribute:: metrics
.. autoattribute:: metadata
.. autoattribute:: ssl_options
.. autoattribute:: sockopts
.. autoattribute:: max_schema_agreement_wait
.. autoattribute:: control_connection_timeout
.. automethod:: connect
.. automethod:: shutdown
.. automethod:: register_listener
.. automethod:: unregister_listener
.. automethod:: get_core_connections_per_host
.. automethod:: set_core_connections_per_host
.. automethod:: get_max_connections_per_host
.. automethod:: set_max_connections_per_host
.. autoclass:: Session () .. autoclass:: Session ()
:members:
:exclude-members: on_up, on_down, on_add, on_remove, add_host, prepare_on_all_hosts, submit .. autoattribute:: default_timeout
.. autoattribute:: row_factory
.. automethod:: execute(statement[, parameters][, timeout][, trace])
.. automethod:: execute_async(statement[, parameters][, trace])
.. automethod:: prepare(statement)
.. automethod:: shutdown()
.. automethod:: set_keyspace(keyspace)
.. autoclass:: ResponseFuture () .. autoclass:: ResponseFuture ()
:members:
:exclude-members: send_request .. autoattribute:: query
.. automethod:: result([timeout])
.. automethod:: get_query_trace()
.. automethod:: add_callback(fn, *args, **kwargs)
.. automethod:: add_errback(fn, *args, **kwargs)
.. automethod:: add_callbacks(callback, errback, callback_args=(), callback_kwargs=None, errback_args=(), errback_args=None)
.. autoexception:: NoHostAvailable () .. autoexception:: NoHostAvailable ()
:members: :members:

View File

@@ -4,5 +4,6 @@
.. module:: cassandra.connection .. module:: cassandra.connection
.. autoexception:: ConnectionException () .. autoexception:: ConnectionException ()
.. autoexception:: ConnectionShutdown ()
.. autoexception:: ConnectionBusy () .. autoexception:: ConnectionBusy ()
.. autoexception:: ProtocolError () .. autoexception:: ProtocolError ()

View File

@@ -4,8 +4,8 @@
.. automodule:: cassandra.pool .. automodule:: cassandra.pool
.. autoclass:: Host () .. autoclass:: Host ()
:members: :members:
:exclude-members: set_location_info, get_and_set_reconnection_handler :exclude-members: set_location_info, get_and_set_reconnection_handler
.. autoclass:: HealthMonitor () .. autoexception:: NoConnectionsAvailable
:members: :members: