Merge branch '3.0'

Conflicts:
	cassandra/metadata.py
	tests/integration/__init__.py
	tests/integration/standard/test_metadata.py
This commit is contained in:
Adam Holmberg
2015-05-15 14:56:59 -05:00
20 changed files with 1213 additions and 102 deletions

View File

@@ -88,7 +88,6 @@ Bug Fixes
---------
* Make execute_concurrent compatible with Python 2.6 (PYTHON-159)
* Handle Unauthorized message on schema_triggers query (PYTHON-155)
* Make execute_concurrent compatible with Python 2.6 (github-197)
* Pure Python sorted set in support of UDTs nested in collections (PYTON-167)
* Support CUSTOM index metadata and string export (PYTHON-165)

View File

@@ -126,6 +126,58 @@ def consistency_value_to_name(value):
return ConsistencyLevel.value_to_name[value] if value is not None else "Not Set"
class SignatureDescriptor(object):
def __init__(self, name, type_signature):
self.name = name
self.type_signature = type_signature
@property
def signature(self):
"""
function signatue string in the form 'name([type0[,type1[...]]])'
can be used to uniquely identify overloaded function names within a keyspace
"""
return self.format_signature(self.name, self.type_signature)
@staticmethod
def format_signature(name, type_signature):
return "%s(%s)" % (name, ','.join(t for t in type_signature))
class UserFunctionDescriptor(SignatureDescriptor):
"""
Describes a User function by name and argument signature
"""
name = None
"""
name of the function
"""
type_signature = None
"""
Ordered list of CQL argument type name comprising the type signature
"""
class UserAggregateDescriptor(SignatureDescriptor):
"""
Describes a User aggregate function by name and argument signature
"""
name = None
"""
name of the aggregate
"""
type_signature = None
"""
Ordered list of CQL argument type name comprising the type signature
"""
class Unavailable(Exception):
"""
There were not enough live replicas to satisfy the requested consistency
@@ -221,6 +273,103 @@ class WriteTimeout(Timeout):
self.write_type = write_type
class CoordinationFailure(Exception):
"""
Replicas sent a failure to the coordinator.
"""
consistency = None
""" The requested :class:`ConsistencyLevel` """
required_responses = None
""" The number of required replica responses """
received_responses = None
"""
The number of replicas that responded before the coordinator timed out
the operation
"""
failures = None
"""
The number of replicas that sent a failure message
"""
def __init__(self, summary_message, consistency=None, required_responses=None, received_responses=None, failures=None):
self.consistency = consistency
self.required_responses = required_responses
self.received_responses = received_responses
self.failures = failures
Exception.__init__(self, summary_message + ' info=' +
repr({'consistency': consistency_value_to_name(consistency),
'required_responses': required_responses,
'received_responses': received_responses,
'failures': failures}))
class ReadFailure(CoordinationFailure):
"""
A subclass of :exc:`CoordinationFailure` for read operations.
This indicates that the replicas sent a failure message to the coordinator.
"""
data_retrieved = None
"""
A boolean indicating whether the requested data was retrieved
by the coordinator from any replicas before it timed out the
operation
"""
def __init__(self, message, data_retrieved=None, **kwargs):
CoordinationFailure.__init__(self, message, **kwargs)
self.data_retrieved = data_retrieved
class WriteFailure(CoordinationFailure):
"""
A subclass of :exc:`CoordinationFailure` for write operations.
This indicates that the replicas sent a failure message to the coordinator.
"""
write_type = None
"""
The type of write operation, enum on :class:`~cassandra.policies.WriteType`
"""
def __init__(self, message, write_type=None, **kwargs):
CoordinationFailure.__init__(self, message, **kwargs)
self.write_type = write_type
class FunctionFailure(Exception):
"""
User Defined Function failed during execution
"""
keyspace = None
"""
Keyspace of the function
"""
function = None
"""
Name of the function
"""
arg_types = None
"""
List of argument type names of the function
"""
def __init__(self, summary_message, keyspace, function, arg_types):
self.keyspace = keyspace
self.function = function
self.arg_types = arg_types
Exception.__init__(self, summary_message)
class AlreadyExists(Exception):
"""
An attempt was made to create a keyspace or table that already exists.

View File

@@ -15,6 +15,7 @@ try:
except ImportError:
SASLClient = None
class AuthProvider(object):
"""
An abstract class that defines the interface that will be used for
@@ -158,6 +159,7 @@ class SaslAuthProvider(AuthProvider):
def new_authenticator(self, host):
return SaslAuthenticator(host, **self.sasl_kwargs)
class SaslAuthenticator(Authenticator):
"""
A pass-through :class:`~.Authenticator` using the third party package

View File

@@ -1127,9 +1127,29 @@ class Cluster(object):
for pool in session._pools.values():
pool.ensure_core_connections()
def refresh_schema(self, keyspace=None, table=None, usertype=None, max_schema_agreement_wait=None):
def _validate_refresh_schema(self, keyspace, table, usertype, function, aggregate):
if any((table, usertype, function, aggregate)):
if not keyspace:
raise ValueError("keyspace is required to refresh specific sub-entity {table, usertype, function, aggregate}")
if sum(1 for e in (table, usertype, function) if e) > 1:
raise ValueError("{table, usertype, function, aggregate} are mutually exclusive")
def refresh_schema(self, keyspace=None, table=None, usertype=None, function=None, aggregate=None, max_schema_agreement_wait=None):
"""
Synchronously refresh the schema metadata.
Synchronously refresh schema metadata.
{keyspace, table, usertype} are string names of the respective entities.
``function`` is a :class:`cassandra.UserFunctionDescriptor`.
``aggregate`` is a :class:`cassandra.UserAggregateDescriptor`.
If none of ``{keyspace, table, usertype, function, aggregate}`` are specified, the entire schema is refreshed.
If any of ``{keyspace, table, usertype, function, aggregate}`` are specified, ``keyspace`` is required.
If only ``keyspace`` is specified, just the top-level keyspace metadata is refreshed (e.g. replication).
The remaining arguments ``{table, usertype, function, aggregate}``
are mutually exclusive -- only one may be specified.
By default, the timeout for this operation is governed by :attr:`~.Cluster.max_schema_agreement_wait`
and :attr:`~.Cluster.control_connection_timeout`.
@@ -1140,17 +1160,19 @@ class Cluster(object):
An Exception is raised if schema refresh fails for any reason.
"""
if not self.control_connection.refresh_schema(keyspace, table, usertype, max_schema_agreement_wait):
self._validate_refresh_schema(keyspace, table, usertype, function, aggregate)
if not self.control_connection.refresh_schema(keyspace, table, usertype, function,
aggregate, max_schema_agreement_wait):
raise Exception("Schema was not refreshed. See log for details.")
def submit_schema_refresh(self, keyspace=None, table=None, usertype=None):
def submit_schema_refresh(self, keyspace=None, table=None, usertype=None, function=None, aggregate=None):
"""
Schedule a refresh of the internal representation of the current
schema for this cluster. If `keyspace` is specified, only that
keyspace will be refreshed, and likewise for `table`.
schema for this cluster. See :meth:`~.refresh_schema` for description of parameters.
"""
self._validate_refresh_schema(keyspace, table, usertype, function, aggregate)
return self.executor.submit(
self.control_connection.refresh_schema, keyspace, table, usertype)
self.control_connection.refresh_schema, keyspace, table, usertype, function, aggregate)
def refresh_nodes(self):
"""
@@ -1372,7 +1394,7 @@ class Session(object):
for future in futures:
future.result()
def execute(self, query, parameters=None, timeout=_NOT_SET, trace=False):
def execute(self, query, parameters=None, timeout=_NOT_SET, trace=False, custom_payload=None):
"""
Execute the given query and synchronously wait for the response.
@@ -1400,6 +1422,10 @@ class Session(object):
instance and not just a string. If there is an error fetching the
trace details, the :attr:`~.Statement.trace` attribute will be left as
:const:`None`.
`custom_payload` is a :ref:`custom_payload` dict to be passed to the server.
If `query` is a Statement with its own custom_payload. The message payload
will be a union of the two, with the values specified here taking precedence.
"""
if timeout is _NOT_SET:
timeout = self.default_timeout
@@ -1409,7 +1435,7 @@ class Session(object):
"The query argument must be an instance of a subclass of "
"cassandra.query.Statement when trace=True")
future = self.execute_async(query, parameters, trace)
future = self.execute_async(query, parameters, trace, custom_payload)
try:
result = future.result(timeout)
finally:
@@ -1421,7 +1447,7 @@ class Session(object):
return result
def execute_async(self, query, parameters=None, trace=False):
def execute_async(self, query, parameters=None, trace=False, custom_payload=None):
"""
Execute the given query and return a :class:`~.ResponseFuture` object
which callbacks may be attached to for asynchronous response
@@ -1433,6 +1459,14 @@ class Session(object):
:meth:`.ResponseFuture.get_query_trace()` after the request
completes to retrieve a :class:`.QueryTrace` instance.
`custom_payload` is a :ref:`custom_payload` dict to be passed to the server.
If `query` is a Statement with its own custom_payload. The message payload
will be a union of the two, with the values specified here taking precedence.
If the server sends a custom payload in the response message,
the dict can be obtained following :meth:`.ResponseFuture.result` via
:attr:`.ResponseFuture.custom_payload`
Example usage::
>>> session = cluster.connect()
@@ -1458,11 +1492,11 @@ class Session(object):
... log.exception("Operation failed:")
"""
future = self._create_response_future(query, parameters, trace)
future = self._create_response_future(query, parameters, trace, custom_payload)
future.send_request()
return future
def _create_response_future(self, query, parameters, trace):
def _create_response_future(self, query, parameters, trace, custom_payload):
""" Returns the ResponseFuture before calling send_request() on it """
prepared_statement = None
@@ -1512,13 +1546,16 @@ class Session(object):
if trace:
message.tracing = True
message.update_custom_payload(query.custom_payload)
message.update_custom_payload(custom_payload)
return ResponseFuture(
self, message, query, self.default_timeout, metrics=self._metrics,
prepared_statement=prepared_statement)
def prepare(self, query):
def prepare(self, query, custom_payload=None):
"""
Prepares a query string, returing a :class:`~cassandra.query.PreparedStatement`
Prepares a query string, returning a :class:`~cassandra.query.PreparedStatement`
instance which can be used as follows::
>>> session = cluster.connect("mykeyspace")
@@ -1541,19 +1578,24 @@ class Session(object):
**Important**: PreparedStatements should be prepared only once.
Preparing the same query more than once will likely affect performance.
`custom_payload` is a key value map to be passed along with the prepare
message. See :ref:`custom_payload`.
"""
message = PrepareMessage(query=query)
message.custom_payload = custom_payload
future = ResponseFuture(self, message, query=None)
try:
future.send_request()
query_id, column_metadata = future.result(self.default_timeout)
query_id, column_metadata, pk_indexes = future.result(self.default_timeout)
except Exception:
log.exception("Error preparing query:")
raise
prepared_statement = PreparedStatement.from_message(
query_id, column_metadata, self.cluster.metadata, query, self.keyspace,
query_id, column_metadata, pk_indexes, self.cluster.metadata, query, self.keyspace,
self._protocol_version)
prepared_statement.custom_payload = future.custom_payload
host = future._current_host
try:
@@ -1831,6 +1873,8 @@ class ControlConnection(object):
_SELECT_COLUMN_FAMILIES = "SELECT * FROM system.schema_columnfamilies"
_SELECT_COLUMNS = "SELECT * FROM system.schema_columns"
_SELECT_USERTYPES = "SELECT * FROM system.schema_usertypes"
_SELECT_FUNCTIONS = "SELECT * FROM system.schema_functions"
_SELECT_AGGREGATES = "SELECT * FROM system.schema_aggregates"
_SELECT_TRIGGERS = "SELECT * FROM system.schema_triggers"
_SELECT_PEERS = "SELECT peer, data_center, rack, tokens, rpc_address, schema_version FROM system.peers"
@@ -2018,16 +2062,16 @@ class ControlConnection(object):
self._connection.close()
del self._connection
def refresh_schema(self, keyspace=None, table=None, usertype=None,
schema_agreement_wait=None):
def refresh_schema(self, keyspace=None, table=None, usertype=None, function=None,
aggregate=None, schema_agreement_wait=None):
if not self._meta_refresh_enabled:
log.debug("[control connection] Skipping schema refresh because meta refresh is disabled")
return False
try:
if self._connection:
return self._refresh_schema(self._connection, keyspace, table, usertype,
schema_agreement_wait=schema_agreement_wait)
return self._refresh_schema(self._connection, keyspace, table, usertype, function,
aggregate, schema_agreement_wait=schema_agreement_wait)
except ReferenceError:
pass # our weak reference to the Cluster is no good
except Exception:
@@ -2035,12 +2079,12 @@ class ControlConnection(object):
self._signal_error()
return False
def _refresh_schema(self, connection, keyspace=None, table=None, usertype=None,
preloaded_results=None, schema_agreement_wait=None):
def _refresh_schema(self, connection, keyspace=None, table=None, usertype=None, function=None,
aggregate=None, preloaded_results=None, schema_agreement_wait=None):
if self._cluster.is_shutdown:
return False
assert table is None or usertype is None
assert sum(1 for arg in (table, usertype, function, aggregate) if arg) <= 1
agreed = self.wait_for_schema_agreement(connection,
preloaded_results=preloaded_results,
@@ -2084,6 +2128,24 @@ class ControlConnection(object):
log.debug("[control connection] Fetched user type info for %s.%s, rebuilding metadata", keyspace, usertype)
types_result = dict_factory(*types_result.results) if types_result.results else {}
self._cluster.metadata.usertype_changed(keyspace, usertype, types_result)
elif function:
# user defined function within this keyspace changed
where_clause = " WHERE keyspace_name = '%s' AND function_name = '%s' AND signature = [%s]" \
% (keyspace, function.name, ','.join("'%s'" % t for t in function.type_signature))
functions_query = QueryMessage(query=self._SELECT_FUNCTIONS + where_clause, consistency_level=cl)
functions_result = connection.wait_for_response(functions_query)
log.debug("[control connection] Fetched user function info for %s.%s, rebuilding metadata", keyspace, function.signature)
functions_result = dict_factory(*functions_result.results) if functions_result.results else {}
self._cluster.metadata.function_changed(keyspace, function, functions_result)
elif aggregate:
# user defined aggregate within this keyspace changed
where_clause = " WHERE keyspace_name = '%s' AND aggregate_name = '%s' AND signature = [%s]" \
% (keyspace, aggregate.name, ','.join("'%s'" % t for t in aggregate.type_signature))
aggregates_query = QueryMessage(query=self._SELECT_AGGREGATES + where_clause, consistency_level=cl)
aggregates_result = connection.wait_for_response(aggregates_query)
log.debug("[control connection] Fetched user aggregate info for %s.%s, rebuilding metadata", keyspace, aggregate.signature)
aggregates_result = dict_factory(*aggregates_result.results) if aggregates_result.results else {}
self._cluster.metadata.aggregate_changed(keyspace, aggregate, aggregates_result)
elif keyspace:
# only the keyspace itself changed (such as replication settings)
where_clause = " WHERE keyspace_name = '%s'" % (keyspace,)
@@ -2099,12 +2161,16 @@ class ControlConnection(object):
QueryMessage(query=self._SELECT_COLUMN_FAMILIES, consistency_level=cl),
QueryMessage(query=self._SELECT_COLUMNS, consistency_level=cl),
QueryMessage(query=self._SELECT_USERTYPES, consistency_level=cl),
QueryMessage(query=self._SELECT_FUNCTIONS, consistency_level=cl),
QueryMessage(query=self._SELECT_AGGREGATES, consistency_level=cl),
QueryMessage(query=self._SELECT_TRIGGERS, consistency_level=cl)
]
responses = connection.wait_for_responses(*queries, timeout=self._timeout, fail_on_error=False)
(ks_success, ks_result), (cf_success, cf_result), \
(col_success, col_result), (types_success, types_result), \
(functions_success, functions_result), \
(aggregates_success, aggregates_result), \
(trigger_success, triggers_result) = responses
if ks_success:
@@ -2146,8 +2212,29 @@ class ControlConnection(object):
else:
raise types_result
# functions were introduced in Cassandra 3.0
if functions_success:
functions_result = dict_factory(*functions_result.results) if functions_result.results else {}
else:
if isinstance(functions_result, InvalidRequest):
log.debug("[control connection] user functions table not found")
functions_result = {}
else:
raise functions_result
# aggregates were introduced in Cassandra 3.0
if aggregates_success:
aggregates_result = dict_factory(*aggregates_result.results) if aggregates_result.results else {}
else:
if isinstance(aggregates_result, InvalidRequest):
log.debug("[control connection] user aggregates table not found")
aggregates_result = {}
else:
raise aggregates_result
log.debug("[control connection] Fetched schema, rebuilding metadata")
self._cluster.metadata.rebuild_schema(ks_result, types_result, cf_result, col_result, triggers_result)
self._cluster.metadata.rebuild_schema(ks_result, types_result, functions_result,
aggregates_result, cf_result, col_result, triggers_result)
return True
def refresh_node_list_and_token_map(self, force_token_rebuild=False):
@@ -2292,12 +2379,13 @@ class ControlConnection(object):
def _handle_schema_change(self, event):
if self._schema_event_refresh_window < 0:
return
keyspace = event.get('keyspace')
table = event.get('table')
usertype = event.get('type')
function = event.get('function')
aggregate = event.get('aggregate')
delay = random() * self._schema_event_refresh_window
self._cluster.scheduler.schedule_unique(delay, self.refresh_schema, keyspace, table, usertype)
self._cluster.scheduler.schedule_unique(delay, self.refresh_schema, keyspace, table, usertype, function, aggregate)
def wait_for_schema_agreement(self, connection=None, preloaded_results=None, wait_time=None):
@@ -2526,19 +2614,20 @@ class _Scheduler(object):
exc_info=exc)
def refresh_schema_and_set_result(keyspace, table, usertype, control_conn, response_future):
def refresh_schema_and_set_result(keyspace, table, usertype, function, aggregate, control_conn, response_future):
try:
if control_conn._meta_refresh_enabled:
log.debug("Refreshing schema in response to schema change. Keyspace: %s; Table: %s, Type: %s",
keyspace, table, usertype)
control_conn._refresh_schema(response_future._connection, keyspace, table, usertype)
log.debug("Refreshing schema in response to schema change. "
"Keyspace: %s; Table: %s, Type: %s, Function: %s, Aggregate: %s",
keyspace, table, usertype, function, aggregate)
control_conn._refresh_schema(response_future._connection, keyspace, table, usertype, function, aggregate)
else:
log.debug("Skipping schema refresh in response to schema change because meta refresh is disabled; "
"Keyspace: %s; Table: %s, Type: %s", keyspace, table, usertype)
"Keyspace: %s; Table: %s, Type: %s, Function: %s", keyspace, table, usertype, function, aggregate)
except Exception:
log.exception("Exception refreshing schema in response to schema change:")
response_future.session.submit(
control_conn.refresh_schema, keyspace, table, usertype)
control_conn.refresh_schema, keyspace, table, usertype, function, aggregate)
finally:
response_future._set_final_result(None)
@@ -2579,6 +2668,7 @@ class ResponseFuture(object):
_start_time = None
_metrics = None
_paging_state = None
_custom_payload = None
def __init__(self, session, message, query, default_timeout=None, metrics=None, prepared_statement=None):
self.session = session
@@ -2666,6 +2756,23 @@ class ResponseFuture(object):
"""
return self._paging_state is not None
@property
def custom_payload(self):
"""
The custom payload returned from the server, if any. This will only be
set by Cassandra servers implementing a custom QueryHandler, and only
for protocol_version 4+.
Ensure the future is complete before trying to access this property
(call :meth:`.result()`, or after callback is invoked).
Otherwise it may throw if the response has not been received.
:return: :ref:`custom_payload`.
"""
if not self._event.is_set():
raise Exception("custom_payload cannot be retrieved before ResponseFuture is finalized")
return self._custom_payload
def start_fetching_next_page(self):
"""
If there are more pages left in the query result, this asynchronously
@@ -2704,6 +2811,8 @@ class ResponseFuture(object):
self.query.trace_id = trace_id
self._query_trace = QueryTrace(trace_id, self.session)
self._custom_payload = getattr(response, 'custom_payload', None)
if isinstance(response, ResultMessage):
if response.kind == RESULT_KIND_SET_KEYSPACE:
session = getattr(self, 'session', None)
@@ -2725,6 +2834,8 @@ class ResponseFuture(object):
response.results['keyspace'],
response.results.get('table'),
response.results.get('type'),
response.results.get('function'),
response.results.get('aggregate'),
self.session.cluster.control_connection,
self)
else:

View File

@@ -28,7 +28,8 @@ import types
from uuid import UUID
import six
from cassandra.util import OrderedDict, OrderedMap, sortedset, Time
from cassandra.util import (OrderedDict, OrderedMap, OrderedMapSerializedKey,
sortedset, Time)
if six.PY3:
long = int
@@ -79,6 +80,7 @@ class Encoder(object):
dict: self.cql_encode_map_collection,
OrderedDict: self.cql_encode_map_collection,
OrderedMap: self.cql_encode_map_collection,
OrderedMapSerializedKey: self.cql_encode_map_collection,
list: self.cql_encode_list_collection,
tuple: self.cql_encode_list_collection,
set: self.cql_encode_set_collection,

View File

@@ -19,8 +19,9 @@ from itertools import islice, cycle
import json
import logging
import re
from threading import RLock
import six
from six.moves import zip
from threading import RLock
murmur3 = None
try:
@@ -28,7 +29,9 @@ try:
except ImportError as e:
pass
from cassandra import SignatureDescriptor
import cassandra.cqltypes as types
from cassandra.encoder import Encoder
from cassandra.marshal import varint_unpack
from cassandra.util import OrderedDict
@@ -82,7 +85,8 @@ class Metadata(object):
"""
return "\n".join(ks.export_as_string() for ks in self.keyspaces.values())
def rebuild_schema(self, ks_results, type_results, cf_results, col_results, triggers_result):
def rebuild_schema(self, ks_results, type_results, function_results,
aggregate_results, cf_results, col_results, triggers_result):
"""
Rebuild the view of the current schema from a fresh set of rows from
the system schema tables.
@@ -92,6 +96,8 @@ class Metadata(object):
cf_def_rows = defaultdict(list)
col_def_rows = defaultdict(lambda: defaultdict(list))
usertype_rows = defaultdict(list)
fn_rows = defaultdict(list)
agg_rows = defaultdict(list)
trigger_rows = defaultdict(lambda: defaultdict(list))
for row in cf_results:
@@ -105,6 +111,12 @@ class Metadata(object):
for row in type_results:
usertype_rows[row["keyspace_name"]].append(row)
for row in function_results:
fn_rows[row["keyspace_name"]].append(row)
for row in aggregate_results:
agg_rows[row["keyspace_name"]].append(row)
for row in triggers_result:
ksname = row["keyspace_name"]
cfname = row["columnfamily_name"]
@@ -123,6 +135,14 @@ class Metadata(object):
usertype = self._build_usertype(keyspace_meta.name, usertype_row)
keyspace_meta.user_types[usertype.name] = usertype
for fn_row in fn_rows.get(keyspace_meta.name, []):
fn = self._build_function(keyspace_meta.name, fn_row)
keyspace_meta.functions[fn.signature] = fn
for agg_row in agg_rows.get(keyspace_meta.name, []):
agg = self._build_aggregate(keyspace_meta.name, agg_row)
keyspace_meta.aggregates[agg.signature] = agg
current_keyspaces.add(keyspace_meta.name)
old_keyspace_meta = self.keyspaces.get(keyspace_meta.name, None)
self.keyspaces[keyspace_meta.name] = keyspace_meta
@@ -132,8 +152,8 @@ class Metadata(object):
self._keyspace_added(keyspace_meta.name)
# remove not-just-added keyspaces
removed_keyspaces = [ksname for ksname in self.keyspaces.keys()
if ksname not in current_keyspaces]
removed_keyspaces = [name for name in self.keyspaces.keys()
if name not in current_keyspaces]
self.keyspaces = dict((name, meta) for name, meta in self.keyspaces.items()
if name in current_keyspaces)
for ksname in removed_keyspaces:
@@ -153,6 +173,8 @@ class Metadata(object):
keyspace_meta.tables = old_keyspace_meta.tables
keyspace_meta.user_types = old_keyspace_meta.user_types
keyspace_meta.indexes = old_keyspace_meta.indexes
keyspace_meta.functions = old_keyspace_meta.functions
keyspace_meta.aggregates = old_keyspace_meta.aggregates
if (keyspace_meta.replication_strategy != old_keyspace_meta.replication_strategy):
self._keyspace_updated(keyspace)
else:
@@ -166,6 +188,22 @@ class Metadata(object):
# the type was deleted
self.keyspaces[keyspace].user_types.pop(name, None)
def function_changed(self, keyspace, function, function_results):
if function_results:
new_function = self._build_function(keyspace, function_results[0])
self.keyspaces[keyspace].functions[function.signature] = new_function
else:
# the function was deleted
self.keyspaces[keyspace].functions.pop(function.signature, None)
def aggregate_changed(self, keyspace, aggregate, aggregate_results):
if aggregate_results:
new_aggregate = self._build_aggregate(keyspace, aggregate_results[0])
self.keyspaces[keyspace].aggregates[aggregate.signature] = new_aggregate
else:
# the aggregate was deleted
self.keyspaces[keyspace].aggregates.pop(aggregate.signature, None)
def table_changed(self, keyspace, table, cf_results, col_results, triggers_result):
try:
keyspace_meta = self.keyspaces[keyspace]
@@ -207,6 +245,23 @@ class Metadata(object):
return UserType(usertype_row['keyspace_name'], usertype_row['type_name'],
usertype_row['field_names'], type_classes)
def _build_function(self, keyspace, function_row):
return_type = types.lookup_casstype(function_row['return_type'])
return Function(function_row['keyspace_name'], function_row['function_name'],
function_row['signature'], function_row['argument_names'],
return_type, function_row['language'], function_row['body'],
function_row['is_deterministic'], function_row['called_on_null_input'])
def _build_aggregate(self, keyspace, aggregate_row):
state_type = types.lookup_casstype(aggregate_row['state_type'])
initial_condition = aggregate_row['initcond']
if initial_condition is not None:
initial_condition = state_type.deserialize(initial_condition, 3)
return_type = types.lookup_casstype(aggregate_row['return_type'])
return Aggregate(aggregate_row['keyspace_name'], aggregate_row['aggregate_name'],
aggregate_row['signature'], aggregate_row['state_func'], state_type,
aggregate_row['final_func'], initial_condition, return_type)
def _build_table_metadata(self, keyspace_metadata, row, col_rows, trigger_rows):
cfname = row["columnfamily_name"]
cf_col_rows = col_rows.get(cfname, [])
@@ -736,11 +791,24 @@ class KeyspaceMetadata(object):
user_types = None
"""
A map from user-defined type names to instances of :class:`~cassandra.metadata..UserType`.
A map from user-defined type names to instances of :class:`~cassandra.metadata.UserType`.
.. versionadded:: 2.1.0
"""
functions = None
"""
A map from user-defined function signatures to instances of :class:`~cassandra.metadata.Function`.
.. versionadded:: 3.0.0
"""
aggregates = None
"""
A map from user-defined aggregate signatures to instances of :class:`~cassandra.metadata.Aggregate`.
.. versionadded:: 3.0.0
"""
def __init__(self, name, durable_writes, strategy_class, strategy_options):
self.name = name
self.durable_writes = durable_writes
@@ -748,13 +816,19 @@ class KeyspaceMetadata(object):
self.tables = {}
self.indexes = {}
self.user_types = {}
self.functions = {}
self.aggregates = {}
def export_as_string(self):
"""
Returns a CQL query string that can be used to recreate the entire keyspace,
including user-defined types and tables.
"""
return "\n\n".join([self.as_cql_query()] + self.user_type_strings() + [t.export_as_string() for t in self.tables.values()])
return "\n\n".join([self.as_cql_query()]
+ self.user_type_strings()
+ [f.as_cql_query(True) for f in self.functions.values()]
+ [a.as_cql_query(True) for a in self.aggregates.values()]
+ [t.export_as_string() for t in self.tables.values()])
def as_cql_query(self):
"""
@@ -857,6 +931,191 @@ class UserType(object):
return ret
class Aggregate(object):
"""
A user defined aggregate function, as created by ``CREATE AGGREGATE`` statements.
Aggregate functions were introduced in Cassandra 3.0
.. versionadded:: 3.0.0
"""
keyspace = None
"""
The string name of the keyspace in which this aggregate is defined
"""
name = None
"""
The name of this aggregate
"""
type_signature = None
"""
An ordered list of the types for each argument to the aggregate
"""
final_func = None
"""
Name of a final function
"""
initial_condition = None
"""
Initial condition of the aggregate
"""
return_type = None
"""
Return type of the aggregate
"""
state_func = None
"""
Name of a state function
"""
state_type = None
"""
Flag indicating whether this function is deterministic
(required for functional indexes)
"""
def __init__(self, keyspace, name, type_signature, state_func,
state_type, final_func, initial_condition, return_type):
self.keyspace = keyspace
self.name = name
self.type_signature = type_signature
self.state_func = state_func
self.state_type = state_type
self.final_func = final_func
self.initial_condition = initial_condition
self.return_type = return_type
def as_cql_query(self, formatted=False):
"""
Returns a CQL query that can be used to recreate this aggregate.
If `formatted` is set to :const:`True`, extra whitespace will
be added to make the query more readable.
"""
sep = '\n' if formatted else ' '
keyspace = protect_name(self.keyspace)
name = protect_name(self.name)
type_list = ', '.join(self.type_signature)
state_func = protect_name(self.state_func)
state_type = self.state_type.cql_parameterized_type()
ret = "CREATE AGGREGATE %(keyspace)s.%(name)s(%(type_list)s)%(sep)s" \
"SFUNC %(state_func)s%(sep)s" \
"STYPE %(state_type)s" % locals()
ret += ''.join((sep, 'FINALFUNC ', protect_name(self.final_func))) if self.final_func else ''
ret += ''.join((sep, 'INITCOND ', Encoder().cql_encode_all_types(self.initial_condition)))\
if self.initial_condition is not None else ''
return ret
@property
def signature(self):
return SignatureDescriptor.format_signature(self.name, self.type_signature)
class Function(object):
"""
A user defined function, as created by ``CREATE FUNCTION`` statements.
User-defined functions were introduced in Cassandra 3.0
.. versionadded:: 3.0.0
"""
keyspace = None
"""
The string name of the keyspace in which this function is defined
"""
name = None
"""
The name of this function
"""
type_signature = None
"""
An ordered list of the types for each argument to the function
"""
arguemnt_names = None
"""
An ordered list of the names of each argument to the function
"""
return_type = None
"""
Return type of the function
"""
language = None
"""
Language of the function body
"""
body = None
"""
Function body string
"""
is_deterministic = None
"""
Flag indicating whether this function is deterministic
(required for functional indexes)
"""
called_on_null_input = None
"""
Flag indicating whether this function should be called for rows with null values
(convenience function to avoid handling nulls explicitly if the result will just be null)
"""
def __init__(self, keyspace, name, type_signature, argument_names,
return_type, language, body, is_deterministic, called_on_null_input):
self.keyspace = keyspace
self.name = name
self.type_signature = type_signature
self.argument_names = argument_names
self.return_type = return_type
self.language = language
self.body = body
self.is_deterministic = is_deterministic
self.called_on_null_input = called_on_null_input
def as_cql_query(self, formatted=False):
"""
Returns a CQL query that can be used to recreate this function.
If `formatted` is set to :const:`True`, extra whitespace will
be added to make the query more readable.
"""
sep = '\n' if formatted else ' '
keyspace = protect_name(self.keyspace)
name = protect_name(self.name)
arg_list = ', '.join(["%s %s" % (protect_name(n), t)
for n, t in zip(self.argument_names, self.type_signature)])
determ = '' if self.is_deterministic else 'NON DETERMINISTIC '
typ = self.return_type.cql_parameterized_type()
lang = self.language
body = protect_value(self.body)
on_null = "CALLED" if self.called_on_null_input else "RETURNS NULL"
return "CREATE %(determ)sFUNCTION %(keyspace)s.%(name)s(%(arg_list)s)%(sep)s" \
"%(on_null)s ON NULL INPUT%(sep)s" \
"RETURNS %(typ)s%(sep)s" \
"LANGUAGE %(lang)s%(sep)s" \
"AS %(body)s;" % locals()
@property
def signature(self):
return SignatureDescriptor.format_signature(self.name, self.type_signature)
class TableMetadata(object):
"""
A representation of the schema for a single table.

View File

@@ -22,8 +22,10 @@ from six.moves import range
import io
from cassandra import (Unavailable, WriteTimeout, ReadTimeout,
WriteFailure, ReadFailure, FunctionFailure,
AlreadyExists, InvalidRequest, Unauthorized,
UnsupportedOperation)
UnsupportedOperation, UserFunctionDescriptor,
UserAggregateDescriptor)
from cassandra.marshal import (int32_pack, int32_unpack, uint16_pack, uint16_unpack,
int8_pack, int8_unpack, uint64_pack, header_pack,
v3_header_pack)
@@ -54,6 +56,7 @@ HEADER_DIRECTION_MASK = 0x80
COMPRESSED_FLAG = 0x01
TRACING_FLAG = 0x02
CUSTOM_PAYLOAD_FLAG = 0x04
_message_types_by_name = {}
_message_types_by_opcode = {}
@@ -70,13 +73,19 @@ class _RegisterMessageType(type):
class _MessageType(object):
tracing = False
custom_payload = None
def to_binary(self, stream_id, protocol_version, compression=None):
flags = 0
body = io.BytesIO()
if self.custom_payload:
if protocol_version < 4:
raise UnsupportedOperation("Custom key/value payloads can only be used with protocol version 4 or higher")
flags |= CUSTOM_PAYLOAD_FLAG
write_bytesmap(body, self.custom_payload)
self.send_body(body, protocol_version)
body = body.getvalue()
flags = 0
if compression and len(body) > 0:
body = compression(body)
flags |= COMPRESSED_FLAG
@@ -89,6 +98,14 @@ class _MessageType(object):
return msg.getvalue()
def update_custom_payload(self, other):
if other:
if not self.custom_payload:
self.custom_payload = {}
self.custom_payload.update(other)
if len(self.custom_payload) > 65535:
raise ValueError("Custom payload map exceeds max count allowed by protocol (65535)")
def __repr__(self):
return '<%s(%s)>' % (self.__class__.__name__, ', '.join('%s=%r' % i for i in _get_params(self)))
@@ -116,6 +133,12 @@ def decode_response(protocol_version, user_type_map, stream_id, flags, opcode, b
else:
trace_id = None
if flags & CUSTOM_PAYLOAD_FLAG:
custom_payload = read_bytesmap(body)
flags ^= CUSTOM_PAYLOAD_FLAG
else:
custom_payload = None
if flags:
log.warning("Unknown protocol flags set: %02x. May cause problems.", flags)
@@ -123,6 +146,7 @@ def decode_response(protocol_version, user_type_map, stream_id, flags, opcode, b
msg = msg_class.recv_body(body, protocol_version, user_type_map)
msg.stream_id = stream_id
msg.trace_id = trace_id
msg.custom_payload = custom_payload
return msg
@@ -263,6 +287,58 @@ class ReadTimeoutErrorMessage(RequestExecutionException):
return ReadTimeout(self.summary_msg(), **self.info)
class ReadFailureMessage(RequestExecutionException):
summary = "Replica(s) failed to execute read"
error_code = 0x1300
@staticmethod
def recv_error_info(f):
return {
'consistency': read_consistency_level(f),
'received_responses': read_int(f),
'required_responses': read_int(f),
'failures': read_int(f),
'data_retrieved': bool(read_byte(f)),
}
def to_exception(self):
return ReadFailure(self.summary_msg(), **self.info)
class FunctionFailureMessage(RequestExecutionException):
summary = "User Defined Function failure"
error_code = 0x1400
@staticmethod
def recv_error_info(f):
return {
'keyspace': read_string(f),
'function': read_string(f),
'arg_types': [read_string(f) for _ in range(read_short(f))],
}
def to_exception(self):
return FunctionFailure(self.summary_msg(), **self.info)
class WriteFailureMessage(RequestExecutionException):
summary = "Replica(s) failed to execute write"
error_code = 0x1500
@staticmethod
def recv_error_info(f):
return {
'consistency': read_consistency_level(f),
'received_responses': read_int(f),
'required_responses': read_int(f),
'failures': read_int(f),
'write_type': WriteType.name_to_value[read_string(f)],
}
def to_exception(self):
return WriteFailure(self.summary_msg(), **self.info)
class SyntaxException(RequestValidationException):
summary = 'Syntax error in CQL query'
error_code = 0x2000
@@ -562,7 +638,7 @@ class ResultMessage(_MessageType):
ksname = read_string(f)
results = ksname
elif kind == RESULT_KIND_PREPARED:
results = cls.recv_results_prepared(f, user_type_map)
results = cls.recv_results_prepared(f, protocol_version, user_type_map)
elif kind == RESULT_KIND_SCHEMA_CHANGE:
results = cls.recv_results_schema_change(f, protocol_version)
return cls(kind, results, paging_state)
@@ -581,16 +657,17 @@ class ResultMessage(_MessageType):
return (paging_state, (colnames, parsed_rows))
@classmethod
def recv_results_prepared(cls, f, user_type_map):
def recv_results_prepared(cls, f, protocol_version, user_type_map):
query_id = read_binary_string(f)
_, column_metadata = cls.recv_results_metadata(f, user_type_map)
return (query_id, column_metadata)
column_metadata, pk_indexes = cls.recv_prepared_metadata(f, protocol_version, user_type_map)
return (query_id, column_metadata, pk_indexes)
@classmethod
def recv_results_metadata(cls, f, user_type_map):
flags = read_int(f)
glob_tblspec = bool(flags & cls._FLAGS_GLOBAL_TABLES_SPEC)
colcount = read_int(f)
if flags & cls._HAS_MORE_PAGES_FLAG:
paging_state = read_binary_longstring(f)
else:
@@ -611,6 +688,32 @@ class ResultMessage(_MessageType):
column_metadata.append((colksname, colcfname, colname, coltype))
return paging_state, column_metadata
@classmethod
def recv_prepared_metadata(cls, f, protocol_version, user_type_map):
flags = read_int(f)
glob_tblspec = bool(flags & cls._FLAGS_GLOBAL_TABLES_SPEC)
colcount = read_int(f)
pk_indexes = None
if protocol_version >= 4:
num_pk_indexes = read_int(f)
pk_indexes = [read_short(f) for _ in range(num_pk_indexes)]
if glob_tblspec:
ksname = read_string(f)
cfname = read_string(f)
column_metadata = []
for _ in range(colcount):
if glob_tblspec:
colksname = ksname
colcfname = cfname
else:
colksname = read_string(f)
colcfname = read_string(f)
colname = read_string(f)
coltype = cls.read_type(f, user_type_map)
column_metadata.append((colksname, colcfname, colname, coltype))
return column_metadata, pk_indexes
@classmethod
def recv_results_schema_change(cls, f, protocol_version):
return EventMessage.recv_schema_change(f, protocol_version)
@@ -823,15 +926,20 @@ class EventMessage(_MessageType):
if protocol_version >= 3:
target = read_string(f)
keyspace = read_string(f)
event = {'change_type': change_type, 'keyspace': keyspace}
if target != "KEYSPACE":
table_or_type = read_string(f)
return {'change_type': change_type, 'keyspace': keyspace, target.lower(): table_or_type}
else:
return {'change_type': change_type, 'keyspace': keyspace}
target_name = read_string(f)
if target == 'FUNCTION':
event['function'] = UserFunctionDescriptor(target_name, [read_string(f) for _ in range(read_short(f))])
elif target == 'AGGREGATE':
event['aggregate'] = UserAggregateDescriptor(target_name, [read_string(f) for _ in range(read_short(f))])
else:
event[target.lower()] = target_name
else:
keyspace = read_string(f)
table = read_string(f)
return {'change_type': change_type, 'keyspace': keyspace, 'table': table}
event = {'change_type': change_type, 'keyspace': keyspace, 'table': table}
return event
def write_header(f, version, flags, stream_id, opcode, length):
@@ -891,6 +999,11 @@ def read_binary_string(f):
return contents
def write_binary_string(f, s):
write_short(f, len(s))
f.write(s)
def write_string(f, s):
if isinstance(s, six.text_type):
s = s.encode('utf8')
@@ -942,6 +1055,22 @@ def write_stringmap(f, strmap):
write_string(f, v)
def read_bytesmap(f):
numpairs = read_short(f)
bytesmap = {}
for _ in range(numpairs):
k = read_string(f)
bytesmap[k] = read_binary_string(f)
return bytesmap
def write_bytesmap(f, bytesmap):
write_short(f, len(bytesmap))
for k, v in bytesmap.items():
write_string(f, k)
write_binary_string(f, v)
def read_stringmultimap(f):
numkeys = read_short(f)
strmmap = {}

View File

@@ -203,11 +203,21 @@ class Statement(object):
.. versionadded:: 2.1.3
"""
custom_payload = None
"""
:ref:`custom_payload` to be passed to the server.
These are only allowed when using protocol version 4 or higher.
.. versionadded:: 3.0.0
"""
_serial_consistency_level = None
_routing_key = None
def __init__(self, retry_policy=None, consistency_level=None, routing_key=None,
serial_consistency_level=None, fetch_size=FETCH_SIZE_UNSET, keyspace=None):
serial_consistency_level=None, fetch_size=FETCH_SIZE_UNSET, keyspace=None,
custom_payload=None):
self.retry_policy = retry_policy
if consistency_level is not None:
self.consistency_level = consistency_level
@@ -218,6 +228,8 @@ class Statement(object):
self.fetch_size = fetch_size
if keyspace is not None:
self.keyspace = keyspace
if custom_payload is not None:
self.custom_payload = custom_payload
def _get_routing_key(self):
return self._routing_key
@@ -297,8 +309,7 @@ class Statement(object):
class SimpleStatement(Statement):
"""
A simple, un-prepared query. All attributes of :class:`Statement` apply
to this class as well.
A simple, un-prepared query.
"""
def __init__(self, query_string, *args, **kwargs):
@@ -306,6 +317,8 @@ class SimpleStatement(Statement):
`query_string` should be a literal CQL statement with the exception
of parameter placeholders that will be filled through the
`parameters` argument of :meth:`.Session.execute()`.
All arguments to :class:`Statement` apply to this class as well
"""
Statement.__init__(self, *args, **kwargs)
self._query_string = query_string
@@ -345,44 +358,44 @@ class PreparedStatement(object):
fetch_size = FETCH_SIZE_UNSET
def __init__(self, column_metadata, query_id, routing_key_indexes, query, keyspace,
protocol_version, consistency_level=None, serial_consistency_level=None,
fetch_size=FETCH_SIZE_UNSET):
custom_payload = None
def __init__(self, column_metadata, query_id, routing_key_indexes, query,
keyspace, protocol_version):
self.column_metadata = column_metadata
self.query_id = query_id
self.routing_key_indexes = routing_key_indexes
self.query_string = query
self.keyspace = keyspace
self.protocol_version = protocol_version
self.consistency_level = consistency_level
self.serial_consistency_level = serial_consistency_level
if fetch_size is not FETCH_SIZE_UNSET:
self.fetch_size = fetch_size
@classmethod
def from_message(cls, query_id, column_metadata, cluster_metadata, query, prepared_keyspace, protocol_version):
def from_message(cls, query_id, column_metadata, pk_indexes, cluster_metadata, query, prepared_keyspace, protocol_version):
if not column_metadata:
return PreparedStatement(column_metadata, query_id, None, query, prepared_keyspace, protocol_version)
partition_key_columns = None
routing_key_indexes = None
if pk_indexes:
routing_key_indexes = pk_indexes
else:
partition_key_columns = None
routing_key_indexes = None
ks_name, table_name, _, _ = column_metadata[0]
ks_meta = cluster_metadata.keyspaces.get(ks_name)
if ks_meta:
table_meta = ks_meta.tables.get(table_name)
if table_meta:
partition_key_columns = table_meta.partition_key
ks_name, table_name, _, _ = column_metadata[0]
ks_meta = cluster_metadata.keyspaces.get(ks_name)
if ks_meta:
table_meta = ks_meta.tables.get(table_name)
if table_meta:
partition_key_columns = table_meta.partition_key
# make a map of {column_name: index} for each column in the statement
statement_indexes = dict((c[2], i) for i, c in enumerate(column_metadata))
# make a map of {column_name: index} for each column in the statement
statement_indexes = dict((c[2], i) for i, c in enumerate(column_metadata))
# a list of which indexes in the statement correspond to partition key items
try:
routing_key_indexes = [statement_indexes[c.name]
for c in partition_key_columns]
except KeyError: # we're missing a partition key component in the prepared
pass # statement; just leave routing_key_indexes as None
# a list of which indexes in the statement correspond to partition key items
try:
routing_key_indexes = [statement_indexes[c.name]
for c in partition_key_columns]
except KeyError: # we're missing a partition key component in the prepared
pass # statement; just leave routing_key_indexes as None
return PreparedStatement(column_metadata, query_id, routing_key_indexes,
query, prepared_keyspace, protocol_version)
@@ -406,8 +419,6 @@ class BoundStatement(Statement):
"""
A prepared statement that has been bound to a particular set of values.
These may be created directly or through :meth:`.PreparedStatement.bind()`.
All attributes of :class:`Statement` apply to this class as well.
"""
prepared_statement = None
@@ -423,13 +434,15 @@ class BoundStatement(Statement):
def __init__(self, prepared_statement, *args, **kwargs):
"""
`prepared_statement` should be an instance of :class:`PreparedStatement`.
All other ``*args`` and ``**kwargs`` will be passed to :class:`.Statement`.
All arguments to :class:`Statement` apply to this class as well
"""
self.prepared_statement = prepared_statement
self.consistency_level = prepared_statement.consistency_level
self.serial_consistency_level = prepared_statement.serial_consistency_level
self.fetch_size = prepared_statement.fetch_size
self.custom_payload = prepared_statement.custom_payload
self.values = []
meta = prepared_statement.column_metadata
@@ -610,7 +623,8 @@ class BatchStatement(Statement):
_session = None
def __init__(self, batch_type=BatchType.LOGGED, retry_policy=None,
consistency_level=None, serial_consistency_level=None, session=None):
consistency_level=None, serial_consistency_level=None,
session=None, custom_payload=None):
"""
`batch_type` specifies The :class:`.BatchType` for the batch operation.
Defaults to :attr:`.BatchType.LOGGED`.
@@ -621,6 +635,11 @@ class BatchStatement(Statement):
`consistency_level` should be a :class:`~.ConsistencyLevel` value
to be used for all operations in the batch.
`custom_payload` is a :ref:`custom_payload` passed to the server.
Note: as Statement objects are added to the batch, this map is
updated with any values found in their custom payloads. These are
only allowed when using protocol version 4 or higher.
Example usage:
.. code-block:: python
@@ -646,12 +665,15 @@ class BatchStatement(Statement):
.. versionchanged:: 2.1.0
Added `serial_consistency_level` as a parameter
.. versionchanged:: 3.0.0
Added `custom_payload` as a parameter
"""
self.batch_type = batch_type
self._statements_and_parameters = []
self._session = session
Statement.__init__(self, retry_policy=retry_policy, consistency_level=consistency_level,
serial_consistency_level=serial_consistency_level)
serial_consistency_level=serial_consistency_level, custom_payload=custom_payload)
def add(self, statement, parameters=None):
"""
@@ -669,7 +691,7 @@ class BatchStatement(Statement):
elif isinstance(statement, PreparedStatement):
query_id = statement.query_id
bound_statement = statement.bind(() if parameters is None else parameters)
self._maybe_set_routing_attributes(bound_statement)
self._update_state(bound_statement)
self._statements_and_parameters.append(
(True, query_id, bound_statement.values))
elif isinstance(statement, BoundStatement):
@@ -677,7 +699,7 @@ class BatchStatement(Statement):
raise ValueError(
"Parameters cannot be passed with a BoundStatement "
"to BatchStatement.add()")
self._maybe_set_routing_attributes(statement)
self._update_state(statement)
self._statements_and_parameters.append(
(True, statement.prepared_statement.query_id, statement.values))
else:
@@ -686,7 +708,7 @@ class BatchStatement(Statement):
if parameters:
encoder = Encoder() if self._session is None else self._session.encoder
query_string = bind_params(query_string, parameters, encoder)
self._maybe_set_routing_attributes(statement)
self._update_state(statement)
self._statements_and_parameters.append((False, query_string, ()))
return self
@@ -705,6 +727,16 @@ class BatchStatement(Statement):
self.routing_key = statement.routing_key
self.keyspace = statement.keyspace
def _update_custom_payload(self, statement):
if statement.custom_payload:
if self.custom_payload is None:
self.custom_payload = {}
self.custom_payload.update(statement.custom_payload)
def _update_state(self, statement):
self._maybe_set_routing_attributes(statement)
self._update_custom_payload(statement)
def __str__(self):
consistency = ConsistencyLevel.value_to_name.get(self.consistency_level, 'Not Set')
return (u'<BatchStatement type=%s, statements=%d, consistency=%s>' %
@@ -765,6 +797,13 @@ class QueryTrace(object):
A :class:`datetime.timedelta` measure of the duration of the query.
"""
client = None
"""
The IP address of the client that issued this request
This is only available when using Cassandra 3.0+
"""
coordinator = None
"""
The IP address of the host that acted as coordinator for this request.
@@ -833,6 +872,8 @@ class QueryTrace(object):
self.started_at = session_row.started_at
self.coordinator = session_row.coordinator
self.parameters = session_row.parameters
# since C* 3.0
self.client = getattr(session_row, 'client', None)
log.debug("Attempting to fetch trace events for trace ID: %s", self.trace_id)
time_spent = time.time() - start
@@ -845,7 +886,7 @@ class QueryTrace(object):
def _execute(self, query, parameters, time_spent, max_wait):
# in case the user switched the row factory, set it to namedtuple for this query
future = self._session._create_response_future(query, parameters, trace=False)
future = self._session._create_response_future(query, parameters, trace=False, custom_payload=None)
future.row_factory = named_tuple_factory
future.send_request()

View File

@@ -14,6 +14,14 @@
.. autoclass:: ConsistencyLevel
:members:
.. autoclass:: UserFunctionDescriptor
:members:
:inherited-members:
.. autoclass:: UserAggregateDescriptor
:members:
:inherited-members:
.. autoexception:: Unavailable()
:members:
@@ -26,6 +34,15 @@
.. autoexception:: WriteTimeout()
:members:
.. autoexception:: ReadFailure()
:members:
.. autoexception:: WriteFailure()
:members:
.. autoexception:: FunctionFailure()
:members:
.. autoexception:: AlreadyExists()
:members:

View File

@@ -102,6 +102,8 @@
.. automethod:: get_query_trace()
.. autoattribute:: custom_payload()
.. autoattribute:: has_more_pages
.. automethod:: start_fetching_next_page()

View File

@@ -13,6 +13,15 @@ Schemas
.. autoclass:: KeyspaceMetadata ()
:members:
.. autoclass:: UserType ()
:members:
.. autoclass:: Function ()
:members:
.. autoclass:: Aggregate ()
:members:
.. autoclass:: TableMetadata ()
:members:

View File

@@ -0,0 +1,11 @@
.. _custom_payload:
Custom Payload
==============
Native protocol version 4+ allows for a custom payload to be sent between clients
and custom query handlers. The payload is specified as a string:binary_type dict
holding custom key/value pairs.
By default these are ignored by the server. They can be useful for servers implementing
a custom QueryHandler.

View File

@@ -14,6 +14,7 @@ Core Driver
cassandra/metrics
cassandra/query
cassandra/pool
cassandra/protocol
cassandra/encoder
cassandra/decoder
cassandra/concurrent

View File

@@ -119,7 +119,9 @@ else:
log.info('Using Cassandra version: %s', CASSANDRA_VERSION)
CCM_KWARGS['version'] = CASSANDRA_VERSION
if CASSANDRA_VERSION >= '2.1':
if CASSANDRA_VERSION >= '3.0':
default_protocol_version = 4
elif CASSANDRA_VERSION >= '2.1':
default_protocol_version = 3
elif CASSANDRA_VERSION >= '2.0':
default_protocol_version = 2

View File

@@ -19,13 +19,18 @@ except ImportError:
import difflib
from mock import Mock
import six, logging, sys, traceback
import logging
import six
import sys
import traceback
from cassandra import AlreadyExists, OperationTimedOut
from cassandra import AlreadyExists, OperationTimedOut, SignatureDescriptor
from cassandra.cluster import Cluster
from cassandra.cqltypes import DoubleType, Int32Type, ListType, UTF8Type, MapType
from cassandra.encoder import Encoder
from cassandra.metadata import (Metadata, KeyspaceMetadata, TableMetadata, IndexMetadata,
Token, MD5Token, TokenMap, murmur3)
Token, MD5Token, TokenMap, murmur3, Function, Aggregate)
from cassandra.policies import SimpleConvictionPolicy
from cassandra.pool import Host
@@ -1002,3 +1007,311 @@ class IndexMapTests(unittest.TestCase):
self.assertIsInstance(ks_meta.indexes[idx], IndexMetadata)
self.assertIsInstance(table_meta.indexes[idx], IndexMetadata)
self.drop_basic_table()
class FunctionTest(unittest.TestCase):
"""
Base functionality for Function and Aggregate metadata test classes
"""
@property
def function_name(self):
return self._testMethodName.lower()
@classmethod
def setup_class(cls):
if PROTOCOL_VERSION < 4:
raise unittest.SkipTest("Function metadata requires native protocol version 4+")
cls.cluster = Cluster(protocol_version=PROTOCOL_VERSION)
cls.keyspace_name = cls.__name__.lower()
cls.session = cls.cluster.connect()
cls.session.execute("CREATE KEYSPACE IF NOT EXISTS %s WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}" % cls.keyspace_name)
cls.session.set_keyspace(cls.keyspace_name)
cls.keyspace_function_meta = cls.cluster.metadata.keyspaces[cls.keyspace_name].functions
cls.keyspace_aggregate_meta = cls.cluster.metadata.keyspaces[cls.keyspace_name].aggregates
@classmethod
def teardown_class(cls):
cls.session.execute("DROP KEYSPACE IF EXISTS %s" % cls.keyspace_name)
cls.cluster.shutdown()
class Verified(object):
def __init__(self, test_case, meta_class, element_meta, **function_kwargs):
self.test_case = test_case
self.function_kwargs = dict(function_kwargs)
self.meta_class = meta_class
self.element_meta = element_meta
def __enter__(self):
tc = self.test_case
expected_meta = self.meta_class(**self.function_kwargs)
tc.assertNotIn(expected_meta.signature, self.element_meta)
tc.session.execute(expected_meta.as_cql_query())
tc.assertIn(expected_meta.signature, self.element_meta)
generated_meta = self.element_meta[expected_meta.signature]
self.test_case.assertEqual(generated_meta.as_cql_query(), expected_meta.as_cql_query())
return self
def __exit__(self, exc_type, exc_val, exc_tb):
tc = self.test_case
tc.session.execute("DROP %s %s.%s" % (self.meta_class.__name__, tc.keyspace_name, self.signature))
tc.assertNotIn(self.signature, self.element_meta)
@property
def signature(self):
return SignatureDescriptor.format_signature(self.function_kwargs['name'],
self.function_kwargs['type_signature'])
class VerifiedFunction(Verified):
def __init__(self, test_case, **kwargs):
super(FunctionTest.VerifiedFunction, self).__init__(test_case, Function, test_case.keyspace_function_meta, **kwargs)
class VerifiedAggregate(Verified):
def __init__(self, test_case, **kwargs):
super(FunctionTest.VerifiedAggregate, self).__init__(test_case, Aggregate, test_case.keyspace_aggregate_meta, **kwargs)
class FunctionMetadata(FunctionTest):
def make_function_kwargs(self, deterministic=True, called_on_null=True):
return {'keyspace': self.keyspace_name,
'name': self.function_name,
'type_signature': ['double', 'int'],
'argument_names': ['d', 'i'],
'return_type': DoubleType,
'language': 'java',
'body': 'return new Double(0.0);',
'is_deterministic': deterministic,
'called_on_null_input': called_on_null}
def test_functions_after_udt(self):
self.assertNotIn(self.function_name, self.keyspace_function_meta)
udt_name = 'udtx'
self.session.execute("CREATE TYPE %s (x int)" % udt_name)
# Ideally we would make a function that takes a udt type, but
# this presently fails because C* c059a56 requires udt to be frozen to create, but does not store meta indicating frozen
# https://issues.apache.org/jira/browse/CASSANDRA-9186
# Maybe update this after release
#kwargs = self.make_function_kwargs()
#kwargs['type_signature'][0] = "frozen<%s>" % udt_name
#expected_meta = Function(**kwargs)
#with self.VerifiedFunction(self, **kwargs):
with self.VerifiedFunction(self, **self.make_function_kwargs()):
# udts must come before functions in keyspace dump
keyspace_cql = self.cluster.metadata.keyspaces[self.keyspace_name].export_as_string()
type_idx = keyspace_cql.rfind("CREATE TYPE")
func_idx = keyspace_cql.find("CREATE FUNCTION")
self.assertNotIn(-1, (type_idx, func_idx), "TYPE or FUNCTION not found in keyspace_cql: " + keyspace_cql)
self.assertGreater(func_idx, type_idx)
def test_function_same_name_diff_types(self):
kwargs = self.make_function_kwargs()
with self.VerifiedFunction(self, **kwargs):
# another function: same name, different type sig.
self.assertGreater(len(kwargs['type_signature']), 1)
self.assertGreater(len(kwargs['argument_names']), 1)
kwargs['type_signature'] = kwargs['type_signature'][:1]
kwargs['argument_names'] = kwargs['argument_names'][:1]
with self.VerifiedFunction(self, **kwargs):
functions = [f for f in self.keyspace_function_meta.values() if f.name == self.function_name]
self.assertEqual(len(functions), 2)
self.assertNotEqual(functions[0].type_signature, functions[1].type_signature)
def test_functions_follow_keyspace_alter(self):
with self.VerifiedFunction(self, **self.make_function_kwargs()):
original_keyspace_meta = self.cluster.metadata.keyspaces[self.keyspace_name]
self.session.execute('ALTER KEYSPACE %s WITH durable_writes = false' % self.keyspace_name)
try:
new_keyspace_meta = self.cluster.metadata.keyspaces[self.keyspace_name]
self.assertNotEqual(original_keyspace_meta, new_keyspace_meta)
self.assertIs(original_keyspace_meta.functions, new_keyspace_meta.functions)
finally:
self.session.execute('ALTER KEYSPACE %s WITH durable_writes = true' % self.keyspace_name)
def test_function_cql_determinism(self):
kwargs = self.make_function_kwargs()
kwargs['is_deterministic'] = True
with self.VerifiedFunction(self, **kwargs) as vf:
fn_meta = self.keyspace_function_meta[vf.signature]
self.assertRegexpMatches(fn_meta.as_cql_query(), "CREATE FUNCTION.*")
kwargs['is_deterministic'] = False
with self.VerifiedFunction(self, **kwargs) as vf:
fn_meta = self.keyspace_function_meta[vf.signature]
self.assertRegexpMatches(fn_meta.as_cql_query(), "CREATE NON DETERMINISTIC FUNCTION.*")
def test_function_cql_called_on_null(self):
kwargs = self.make_function_kwargs()
kwargs['called_on_null_input'] = True
with self.VerifiedFunction(self, **kwargs) as vf:
fn_meta = self.keyspace_function_meta[vf.signature]
self.assertRegexpMatches(fn_meta.as_cql_query(), "CREATE FUNCTION.*\) CALLED ON NULL INPUT RETURNS .*")
kwargs['called_on_null_input'] = False
with self.VerifiedFunction(self, **kwargs) as vf:
fn_meta = self.keyspace_function_meta[vf.signature]
self.assertRegexpMatches(fn_meta.as_cql_query(), "CREATE FUNCTION.*\) RETURNS NULL ON NULL INPUT RETURNS .*")
class AggregateMetadata(FunctionTest):
@classmethod
def setup_class(cls):
super(AggregateMetadata, cls).setup_class()
cls.session.execute("""CREATE OR REPLACE FUNCTION sum_int(s int, i int)
RETURNS NULL ON NULL INPUT
RETURNS int
LANGUAGE javascript AS 's + i';""")
cls.session.execute("""CREATE OR REPLACE FUNCTION sum_int_two(s int, i int, j int)
RETURNS NULL ON NULL INPUT
RETURNS int
LANGUAGE javascript AS 's + i + j';""")
cls.session.execute("""CREATE OR REPLACE FUNCTION "List_As_String"(l list<text>)
RETURNS NULL ON NULL INPUT
RETURNS int
LANGUAGE javascript AS ''''' + l';""")
cls.session.execute("""CREATE OR REPLACE FUNCTION extend_list(s list<text>, i int)
CALLED ON NULL INPUT
RETURNS list<text>
LANGUAGE java AS 'if (i != null) s.add(i.toString()); return s;';""")
cls.session.execute("""CREATE OR REPLACE FUNCTION update_map(s map<int, int>, i int)
RETURNS NULL ON NULL INPUT
RETURNS map<int, int>
LANGUAGE java AS 's.put(new Integer(i), new Integer(i)); return s;';""")
cls.session.execute("""CREATE TABLE IF NOT EXISTS t
(k int PRIMARY KEY, v int)""")
for x in range(4):
cls.session.execute("INSERT INTO t (k,v) VALUES (%s, %s)", (x, x))
cls.session.execute("INSERT INTO t (k) VALUES (%s)", (4,))
def make_aggregate_kwargs(self, state_func, state_type, final_func=None, init_cond=None):
return {'keyspace': self.keyspace_name,
'name': self.function_name + '_aggregate',
'type_signature': ['int'],
'state_func': state_func,
'state_type': state_type,
'final_func': final_func,
'initial_condition': init_cond,
'return_type': "does not matter for creation"}
def test_return_type_meta(self):
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('sum_int', Int32Type, init_cond=1)) as va:
self.assertIs(self.keyspace_aggregate_meta[va.signature].return_type, Int32Type)
def test_init_cond(self):
# This is required until the java driver bundled with C* is updated to support v4
c = Cluster(protocol_version=3)
s = c.connect(self.keyspace_name)
expected_values = range(4)
# int32
for init_cond in (-1, 0, 1):
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('sum_int', Int32Type, init_cond=init_cond)) as va:
sum_res = s.execute("SELECT %s(v) AS sum FROM t" % va.function_kwargs['name'])[0].sum
self.assertEqual(sum_res, init_cond + sum(expected_values))
# list<text>
for init_cond in ([], ['1', '2']):
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('extend_list', ListType.apply_parameters([UTF8Type]), init_cond=init_cond)) as va:
list_res = s.execute("SELECT %s(v) AS list_res FROM t" % va.function_kwargs['name'])[0].list_res
self.assertListEqual(list_res[:len(init_cond)], init_cond)
self.assertEqual(set(i for i in list_res[len(init_cond):]),
set(str(i) for i in expected_values))
# map<int,int>
expected_map_values = dict((i, i) for i in expected_values)
expected_key_set = set(expected_values)
for init_cond in ({}, {1: 2, 3: 4}, {5: 5}):
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('update_map', MapType.apply_parameters([Int32Type, Int32Type]), init_cond=init_cond)) as va:
map_res = s.execute("SELECT %s(v) AS map_res FROM t" % va.function_kwargs['name'])[0].map_res
self.assertDictContainsSubset(expected_map_values, map_res)
init_not_updated = dict((k, init_cond[k]) for k in set(init_cond) - expected_key_set)
self.assertDictContainsSubset(init_not_updated, map_res)
c.shutdown()
def test_aggregates_after_functions(self):
# functions must come before functions in keyspace dump
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('extend_list', ListType.apply_parameters([UTF8Type]))):
keyspace_cql = self.cluster.metadata.keyspaces[self.keyspace_name].export_as_string()
func_idx = keyspace_cql.find("CREATE FUNCTION")
aggregate_idx = keyspace_cql.rfind("CREATE AGGREGATE")
self.assertNotIn(-1, (aggregate_idx, func_idx), "AGGREGATE or FUNCTION not found in keyspace_cql: " + keyspace_cql)
self.assertGreater(aggregate_idx, func_idx)
def test_same_name_diff_types(self):
kwargs = self.make_aggregate_kwargs('sum_int', Int32Type, init_cond=0)
with self.VerifiedAggregate(self, **kwargs):
kwargs['state_func'] = 'sum_int_two'
kwargs['type_signature'] = ['int', 'int']
with self.VerifiedAggregate(self, **kwargs):
aggregates = [a for a in self.keyspace_aggregate_meta.values() if a.name == kwargs['name']]
self.assertEqual(len(aggregates), 2)
self.assertNotEqual(aggregates[0].type_signature, aggregates[1].type_signature)
def test_aggregates_follow_keyspace_alter(self):
with self.VerifiedAggregate(self, **self.make_aggregate_kwargs('sum_int', Int32Type, init_cond=0)):
original_keyspace_meta = self.cluster.metadata.keyspaces[self.keyspace_name]
self.session.execute('ALTER KEYSPACE %s WITH durable_writes = false' % self.keyspace_name)
try:
new_keyspace_meta = self.cluster.metadata.keyspaces[self.keyspace_name]
self.assertNotEqual(original_keyspace_meta, new_keyspace_meta)
self.assertIs(original_keyspace_meta.aggregates, new_keyspace_meta.aggregates)
finally:
self.session.execute('ALTER KEYSPACE %s WITH durable_writes = true' % self.keyspace_name)
def test_cql_optional_params(self):
kwargs = self.make_aggregate_kwargs('extend_list', ListType.apply_parameters([UTF8Type]))
# no initial condition, final func
self.assertIsNone(kwargs['initial_condition'])
self.assertIsNone(kwargs['final_func'])
with self.VerifiedAggregate(self, **kwargs) as va:
meta = self.keyspace_aggregate_meta[va.signature]
self.assertIsNone(meta.initial_condition)
self.assertIsNone(meta.final_func)
cql = meta.as_cql_query()
self.assertEqual(cql.find('INITCOND'), -1)
self.assertEqual(cql.find('FINALFUNC'), -1)
# initial condition, no final func
kwargs['initial_condition'] = ['init', 'cond']
with self.VerifiedAggregate(self, **kwargs) as va:
meta = self.keyspace_aggregate_meta[va.signature]
self.assertListEqual(meta.initial_condition, kwargs['initial_condition'])
self.assertIsNone(meta.final_func)
cql = meta.as_cql_query()
search_string = "INITCOND %s" % Encoder().cql_encode_all_types(kwargs['initial_condition'])
self.assertGreater(cql.find(search_string), 0, '"%s" search string not found in cql:\n%s' % (search_string, cql))
self.assertEqual(cql.find('FINALFUNC'), -1)
# no initial condition, final func
kwargs['initial_condition'] = None
kwargs['final_func'] = 'List_As_String'
with self.VerifiedAggregate(self, **kwargs) as va:
meta = self.keyspace_aggregate_meta[va.signature]
self.assertIsNone(meta.initial_condition)
self.assertEqual(meta.final_func, kwargs['final_func'])
cql = meta.as_cql_query()
self.assertEqual(cql.find('INITCOND'), -1)
search_string = 'FINALFUNC "%s"' % kwargs['final_func']
self.assertGreater(cql.find(search_string), 0, '"%s" search string not found in cql:\n%s' % (search_string, cql))
# both
kwargs['initial_condition'] = ['init', 'cond']
kwargs['final_func'] = 'List_As_String'
with self.VerifiedAggregate(self, **kwargs) as va:
meta = self.keyspace_aggregate_meta[va.signature]
self.assertListEqual(meta.initial_condition, kwargs['initial_condition'])
self.assertEqual(meta.final_func, kwargs['final_func'])
cql = meta.as_cql_query()
init_cond_idx = cql.find("INITCOND %s" % Encoder().cql_encode_all_types(kwargs['initial_condition']))
final_func_idx = cql.find('FINALFUNC "%s"' % kwargs['final_func'])
self.assertNotIn(-1, (init_cond_idx, final_func_idx))
self.assertGreater(init_cond_idx, final_func_idx)

View File

@@ -213,6 +213,31 @@ class PreparedStatementTests(unittest.TestCase):
cluster.shutdown()
def test_no_meta(self):
cluster = Cluster(protocol_version=PROTOCOL_VERSION)
session = cluster.connect()
prepared = session.prepare(
"""
INSERT INTO test3rf.test (k, v) VALUES (0, 0)
""")
self.assertIsInstance(prepared, PreparedStatement)
bound = prepared.bind(None)
session.execute(bound)
prepared = session.prepare(
"""
SELECT * FROM test3rf.test WHERE k=0
""")
self.assertIsInstance(prepared, PreparedStatement)
bound = prepared.bind(None)
results = session.execute(bound)
self.assertEqual(results[0].v, 0)
cluster.shutdown()
def test_none_values_dicts(self):
"""
Ensure binding None is handled correctly with dict bindings

View File

@@ -12,7 +12,6 @@
# See the License for the specific language governing permissions and
# limitations under the License.
import os
from cassandra.concurrent import execute_concurrent
@@ -27,7 +26,7 @@ from cassandra.query import (PreparedStatement, BoundStatement, SimpleStatement,
from cassandra.cluster import Cluster
from cassandra.policies import HostDistance
from tests.integration import use_singledc, PROTOCOL_VERSION
from tests.integration import use_singledc, PROTOCOL_VERSION, get_server_versions
def setup_module():
@@ -104,6 +103,47 @@ class QueryTests(unittest.TestCase):
cluster.shutdown()
def test_client_ip_in_trace(self):
"""
Test to validate that client trace contains client ip information.
creates a simple query and ensures that the client trace information is present. This will
only be the case if the c* version is 3.0 or greater
@since 3.0
@jira_ticket PYTHON-235
@expected_result client address should be present in C* >= 3, otherwise should be none.
@test_category tracing
+ """
#The current version on the trunk doesn't have the version set to 3.0 yet.
#For now we will use the protocol version. Once they update the version on C* trunk
#we can use the C*. See below
#self._cass_version, self._cql_version = get_server_versions()
#if self._cass_version < (3, 0):
# raise unittest.SkipTest("Client IP was not present in trace until C* 3.0")
if PROTOCOL_VERSION < 4:
raise unittest.SkipTest(
"Protocol 4+ is required for client ip tracing, currently testing against %r"
% (PROTOCOL_VERSION,))
cluster = Cluster(protocol_version=PROTOCOL_VERSION)
session = cluster.connect()
query = "SELECT * FROM system.local"
statement = SimpleStatement(query)
response_future = session.execute_async(statement, trace=True)
response_future.result(10.0)
current_host = response_future._current_host.address
# Fetch the client_ip from the trace.
trace = response_future.get_query_trace(2.0)
client_ip = trace.client
# Ensure that ip is set for c* >3
self.assertIsNotNone(client_ip,"Client IP was not set in trace with C* > 3.0")
self.assertEqual(client_ip,current_host,"Client IP from trace did not match the expected value")
cluster.shutdown()
class PreparedStatementTests(unittest.TestCase):

View File

@@ -410,12 +410,12 @@ class ControlConnectionTest(unittest.TestCase):
}
self.cluster.scheduler.reset_mock()
self.control_connection._handle_schema_change(event)
self.cluster.scheduler.schedule_unique.assert_called_once_with(0.0, self.control_connection.refresh_schema, 'ks1', 'table1', None)
self.cluster.scheduler.schedule_unique.assert_called_once_with(0.0, self.control_connection.refresh_schema, 'ks1', 'table1', None, None, None)
self.cluster.scheduler.reset_mock()
event['table'] = None
self.control_connection._handle_schema_change(event)
self.cluster.scheduler.schedule_unique.assert_called_once_with(0.0, self.control_connection.refresh_schema, 'ks1', None, None)
self.cluster.scheduler.schedule_unique.assert_called_once_with(0.0, self.control_connection.refresh_schema, 'ks1', None, None, None, None)
def test_refresh_disabled(self):
cluster = MockCluster()
@@ -463,4 +463,4 @@ class ControlConnectionTest(unittest.TestCase):
cc_no_topo_refresh._handle_schema_change(schema_event)
cluster.scheduler.schedule_unique.assert_has_calls([call(ANY, cc_no_topo_refresh.refresh_node_list_and_token_map),
call(0.0, cc_no_topo_refresh.refresh_schema,
schema_event['keyspace'], schema_event['table'], None)])
schema_event['keyspace'], schema_event['table'], None, None, None)])

View File

@@ -128,8 +128,8 @@ class BoundStatementTestCase(unittest.TestCase):
routing_key_indexes=[],
query=None,
keyspace=keyspace,
protocol_version=2,
fetch_size=1234)
protocol_version=2)
prepared_statement.fetch_size = 1234
bound_statement = BoundStatement(prepared_statement=prepared_statement)
self.assertEqual(1234, bound_statement.fetch_size)
@@ -147,10 +147,9 @@ class BoundStatementTestCase(unittest.TestCase):
routing_key_indexes=[0, 1],
query=None,
keyspace=keyspace,
protocol_version=2,
fetch_size=1234)
protocol_version=2)
self.assertRaises(ValueError, prepared_statement.bind, (1,))
bound = prepared_statement.bind((1,2))
bound = prepared_statement.bind((1, 2))
self.assertEqual(bound.keyspace, keyspace)

View File

@@ -105,7 +105,7 @@ class ResponseFutureTests(unittest.TestCase):
kind=RESULT_KIND_SCHEMA_CHANGE,
results={'keyspace': "keyspace1", "table": "table1"})
rf._set_result(result)
session.submit.assert_called_once_with(ANY, 'keyspace1', 'table1', None, ANY, rf)
session.submit.assert_called_once_with(ANY, 'keyspace1', 'table1', None, None, None, ANY, rf)
def test_other_result_message_kind(self):
session = self.make_session()