Merge remote-tracking branch 'origin/3.1'
Conflicts: CHANGELOG.rst tests/integration/cqlengine/columns/test_validation.py
This commit is contained in:
@@ -1,13 +1,3 @@
|
||||
3.2.0
|
||||
=====
|
||||
|
||||
Features
|
||||
--------
|
||||
|
||||
Bug Fixes
|
||||
---------
|
||||
* Fix Python3 bug with Timers and heaps (github #466)
|
||||
|
||||
3.1.0
|
||||
=====
|
||||
|
||||
@@ -16,12 +6,14 @@ Features
|
||||
* Pass name of server auth class to AuthProvider (PYTHON-454)
|
||||
* Surface schema agreed flag for DDL statements (PYTHON-458)
|
||||
* Automatically convert float and int to Decimal on serialization (PYTHON-468)
|
||||
* Expose prior state information via cqlengine LWTException (github #343)
|
||||
|
||||
Bug Fixes
|
||||
---------
|
||||
* Bus error (alignment issues) when running cython on some ARM platforms (PYTHON-450)
|
||||
* Overflow when decoding large collections (cython) (PYTHON-459)
|
||||
* Crash when updating a UDT column with a None value (github #467)
|
||||
* Timer heap comparison issue with Python 3 (github #466)
|
||||
|
||||
3.0.0
|
||||
=====
|
||||
|
@@ -23,7 +23,7 @@ class NullHandler(logging.Handler):
|
||||
logging.getLogger('cassandra').addHandler(NullHandler())
|
||||
|
||||
|
||||
__version_info__ = (3, 1, '0a1', 'post0')
|
||||
__version_info__ = (3, 1, '0a2', 'post0')
|
||||
__version__ = '.'.join(map(str, __version_info__))
|
||||
|
||||
|
||||
|
@@ -41,7 +41,7 @@ except ImportError:
|
||||
from cassandra.util import WeakSet # NOQA
|
||||
|
||||
from functools import partial, wraps
|
||||
from itertools import groupby
|
||||
from itertools import groupby, count
|
||||
|
||||
from cassandra import (ConsistencyLevel, AuthenticationFailed,
|
||||
OperationTimedOut, UnsupportedOperation,
|
||||
@@ -70,8 +70,8 @@ from cassandra.pool import (Host, _ReconnectionHandler, _HostReconnectionHandler
|
||||
HostConnectionPool, HostConnection,
|
||||
NoConnectionsAvailable)
|
||||
from cassandra.query import (SimpleStatement, PreparedStatement, BoundStatement,
|
||||
BatchStatement, bind_params, QueryTrace, Statement,
|
||||
named_tuple_factory, dict_factory, FETCH_SIZE_UNSET)
|
||||
BatchStatement, bind_params, QueryTrace,
|
||||
named_tuple_factory, dict_factory, tuple_factory, FETCH_SIZE_UNSET)
|
||||
|
||||
|
||||
def _is_eventlet_monkey_patched():
|
||||
@@ -2542,6 +2542,7 @@ class _Scheduler(object):
|
||||
def __init__(self, executor):
|
||||
self._queue = Queue.PriorityQueue()
|
||||
self._scheduled_tasks = set()
|
||||
self._count = count()
|
||||
self._executor = executor
|
||||
|
||||
t = Thread(target=self.run, name="Task Scheduler")
|
||||
@@ -2559,7 +2560,7 @@ class _Scheduler(object):
|
||||
# this can happen on interpreter shutdown
|
||||
pass
|
||||
self.is_shutdown = True
|
||||
self._queue.put_nowait((0, None))
|
||||
self._queue.put_nowait((0, 0, None))
|
||||
|
||||
def schedule(self, delay, fn, *args, **kwargs):
|
||||
self._insert_task(delay, (fn, args, tuple(kwargs.items())))
|
||||
@@ -2575,7 +2576,7 @@ class _Scheduler(object):
|
||||
if not self.is_shutdown:
|
||||
run_at = time.time() + delay
|
||||
self._scheduled_tasks.add(task)
|
||||
self._queue.put_nowait((run_at, task))
|
||||
self._queue.put_nowait((run_at, next(self._count), task))
|
||||
else:
|
||||
log.debug("Ignoring scheduled task after shutdown: %r", task)
|
||||
|
||||
@@ -2586,7 +2587,7 @@ class _Scheduler(object):
|
||||
|
||||
try:
|
||||
while True:
|
||||
run_at, task = self._queue.get(block=True, timeout=None)
|
||||
run_at, i, task = self._queue.get(block=True, timeout=None)
|
||||
if self.is_shutdown:
|
||||
log.debug("Not executing scheduled task due to Scheduler shutdown")
|
||||
return
|
||||
@@ -2597,7 +2598,7 @@ class _Scheduler(object):
|
||||
future = self._executor.submit(fn, *args, **kwargs)
|
||||
future.add_done_callback(self._log_if_failed)
|
||||
else:
|
||||
self._queue.put_nowait((run_at, task))
|
||||
self._queue.put_nowait((run_at, i, task))
|
||||
break
|
||||
except Queue.Empty:
|
||||
pass
|
||||
@@ -3408,3 +3409,24 @@ class ResultSet(object):
|
||||
See :meth:`.ResponseFuture.get_all_query_traces` for details.
|
||||
"""
|
||||
return self.response_future.get_all_query_traces(max_wait_sec_per)
|
||||
|
||||
@property
|
||||
def was_applied(self):
|
||||
"""
|
||||
For LWT results, returns whether the transaction was applied.
|
||||
|
||||
Result is indeterminate if called on a result that was not an LWT request.
|
||||
|
||||
Only valid when one of tne of the internal row factories is in use.
|
||||
"""
|
||||
if self.response_future.row_factory not in (named_tuple_factory, dict_factory, tuple_factory):
|
||||
raise RuntimeError("Cannot determine LWT result with row factory %s" % (self.response_future.row_factsory,))
|
||||
if len(self.current_rows) != 1:
|
||||
raise RuntimeError("LWT result should have exactly one row. This has %d." % (len(self.current_rows)))
|
||||
|
||||
row = self.current_rows[0]
|
||||
if isinstance(row, tuple):
|
||||
return row[0]
|
||||
else:
|
||||
return row['[applied]']
|
||||
|
||||
|
@@ -40,7 +40,17 @@ class IfNotExistsWithCounterColumn(CQLEngineException):
|
||||
|
||||
|
||||
class LWTException(CQLEngineException):
|
||||
pass
|
||||
"""Lightweight transaction exception.
|
||||
|
||||
This exception will be raised when a write using an `IF` clause could not be
|
||||
applied due to existing data violating the condition. The existing data is
|
||||
available through the ``existing`` attribute.
|
||||
|
||||
:param existing: The current state of the data which prevented the write.
|
||||
"""
|
||||
def __init__(self, existing):
|
||||
super(LWTException, self).__init__(self)
|
||||
self.existing = existing
|
||||
|
||||
|
||||
class DoesNotExist(QueryException):
|
||||
@@ -53,12 +63,14 @@ class MultipleObjectsReturned(QueryException):
|
||||
|
||||
def check_applied(result):
|
||||
"""
|
||||
check if result contains some column '[applied]' with false value,
|
||||
if that value is false, it means our light-weight transaction didn't
|
||||
applied to database.
|
||||
Raises LWTException if it looks like a failed LWT request.
|
||||
"""
|
||||
if result and '[applied]' in result[0] and not result[0]['[applied]']:
|
||||
raise LWTException('')
|
||||
try:
|
||||
applied = result.was_applied
|
||||
except Exception:
|
||||
applied = True # result was not LWT form
|
||||
if not applied:
|
||||
raise LWTException(result[0])
|
||||
|
||||
|
||||
class AbstractQueryableColumn(UnicodeMixin):
|
||||
|
@@ -35,10 +35,15 @@ from cassandra.util import is_little_endian
|
||||
|
||||
import_datetime()
|
||||
|
||||
DEF DAY_IN_SECONDS = 86400
|
||||
|
||||
DATETIME_EPOC = datetime.datetime(1970, 1, 1)
|
||||
|
||||
|
||||
cdef datetime_from_timestamp(double timestamp):
|
||||
cdef int seconds = <int> timestamp
|
||||
cdef int microseconds = (<int64_t> (timestamp * 1000000)) % 1000000
|
||||
return DATETIME_EPOC + timedelta_new(0, seconds, microseconds)
|
||||
cdef int days = <int> (timestamp / DAY_IN_SECONDS)
|
||||
cdef int64_t days_in_seconds = (<int64_t> days) * DAY_IN_SECONDS
|
||||
cdef int seconds = <int> (timestamp - days_in_seconds)
|
||||
cdef int microseconds = <int> ((timestamp - days_in_seconds - seconds) * 1000000)
|
||||
|
||||
return DATETIME_EPOC + timedelta_new(days, seconds, microseconds)
|
||||
|
@@ -48,7 +48,6 @@ Model
|
||||
See the `list of supported table properties for more information
|
||||
<http://www.datastax.com/documentation/cql/3.1/cql/cql_reference/tabProp.html>`_.
|
||||
|
||||
|
||||
.. attribute:: __options__
|
||||
|
||||
For example:
|
||||
@@ -89,7 +88,7 @@ Model
|
||||
object is determined by its primary key(s). And please note using this flag
|
||||
would incur performance cost.
|
||||
|
||||
if the insertion didn't applied, a LWTException exception would be raised.
|
||||
If the insertion isn't applied, a :class:`~cassandra.cqlengine.query.LWTException` is raised.
|
||||
|
||||
.. code-block:: python
|
||||
|
||||
@@ -97,7 +96,7 @@ Model
|
||||
TestIfNotExistsModel.if_not_exists().create(id=id, count=9, text='111111111111')
|
||||
except LWTException as e:
|
||||
# handle failure case
|
||||
print e.existing # existing object
|
||||
print e.existing # dict containing LWT result fields
|
||||
|
||||
This method is supported on Cassandra 2.0 or later.
|
||||
|
||||
@@ -111,7 +110,7 @@ Model
|
||||
Simply specify the column(s) and the expected value(s). As with if_not_exists,
|
||||
this incurs a performance cost.
|
||||
|
||||
If the insertion isn't applied, a LWTException is raised
|
||||
If the insertion isn't applied, a :class:`~cassandra.cqlengine.query.LWTException` is raised.
|
||||
|
||||
.. code-block:: python
|
||||
|
||||
@@ -119,7 +118,8 @@ Model
|
||||
try:
|
||||
t.iff(count=5).update('other text')
|
||||
except LWTException as e:
|
||||
# handle failure
|
||||
# handle failure case
|
||||
print e.existing # existing object
|
||||
|
||||
.. automethod:: get
|
||||
|
||||
|
@@ -42,3 +42,4 @@ The methods here are used to filter, order, and constrain results.
|
||||
|
||||
.. autoclass:: MultipleObjectsReturned
|
||||
|
||||
.. autoclass:: LWTException
|
||||
|
58
setup.py
58
setup.py
@@ -171,7 +171,6 @@ Some optional C extensions are not supported in PyPy. Only murmur3 will be built
|
||||
=================================================================================
|
||||
"""
|
||||
|
||||
|
||||
is_windows = os.name == 'nt'
|
||||
|
||||
is_pypy = "PyPy" in sys.version
|
||||
@@ -323,6 +322,54 @@ On OSX, via homebrew:
|
||||
sys.stderr.write("Failed to cythonize one or more modules. These will not be compiled as extensions (optional).\n")
|
||||
|
||||
|
||||
def pre_build_check():
|
||||
"""
|
||||
Try to verify build tools
|
||||
"""
|
||||
if os.environ.get('CASS_DRIVER_NO_PRE_BUILD_CHECK'):
|
||||
return True
|
||||
|
||||
try:
|
||||
from distutils.ccompiler import new_compiler
|
||||
from distutils.sysconfig import customize_compiler
|
||||
from distutils.dist import Distribution
|
||||
|
||||
# base build_ext just to emulate compiler option setup
|
||||
be = build_ext(Distribution())
|
||||
be.initialize_options()
|
||||
be.finalize_options()
|
||||
|
||||
# First, make sure we have a Python include directory
|
||||
have_python_include = any(os.path.isfile(os.path.join(p, 'Python.h')) for p in be.include_dirs)
|
||||
if not have_python_include:
|
||||
sys.stderr.write("Did not find 'Python.h' in %s.\n" % (be.include_dirs,))
|
||||
return False
|
||||
|
||||
compiler = new_compiler(compiler=be.compiler)
|
||||
customize_compiler(compiler)
|
||||
|
||||
executables = []
|
||||
if compiler.compiler_type in ('unix', 'cygwin'):
|
||||
executables = [compiler.executables[exe][0] for exe in ('compiler_so', 'linker_so')]
|
||||
elif compiler.compiler_type == 'nt':
|
||||
executables = [getattr(compiler, exe) for exe in ('cc', 'linker')]
|
||||
|
||||
if executables:
|
||||
from distutils.spawn import find_executable
|
||||
for exe in executables:
|
||||
if not find_executable(exe):
|
||||
sys.stderr.write("Failed to find %s for compiler type %s.\n" % (exe, compiler.compiler_type))
|
||||
return False
|
||||
|
||||
except Exception as exc:
|
||||
sys.stderr.write('%s\n' % str(exc))
|
||||
sys.stderr.write("Failed pre-build check. Attempting anyway.\n")
|
||||
|
||||
# if we are unable to positively id the compiler type, or one of these assumptions fails,
|
||||
# just proceed as we would have without the check
|
||||
return True
|
||||
|
||||
|
||||
def run_setup(extensions):
|
||||
|
||||
kw = {'cmdclass': {'doc': DocCommand}}
|
||||
@@ -336,7 +383,14 @@ def run_setup(extensions):
|
||||
kw['ext_modules'] = [Extension('DUMMY', [])] # dummy extension makes sure build_ext is called for install
|
||||
|
||||
if try_cython:
|
||||
kw['setup_requires'] = ['Cython>=0.20']
|
||||
# precheck compiler before adding to setup_requires
|
||||
# we don't actually negate try_cython because:
|
||||
# 1.) build_ext eats errors at compile time, letting the install complete while producing useful feedback
|
||||
# 2.) there could be a case where the python environment has cython installed but the system doesn't have build tools
|
||||
if pre_build_check():
|
||||
kw['setup_requires'] = ['Cython>=0.20']
|
||||
else:
|
||||
sys.stderr.write("Bypassing Cython setup requirement\n")
|
||||
|
||||
dependencies = ['six >=1.6']
|
||||
|
||||
|
@@ -17,8 +17,12 @@ try:
|
||||
except ImportError:
|
||||
import unittest # noqa
|
||||
|
||||
import logging
|
||||
import os
|
||||
import socket
|
||||
import os, six, time, sys, logging, traceback
|
||||
import sys
|
||||
import time
|
||||
import traceback
|
||||
from threading import Event
|
||||
from subprocess import call
|
||||
from itertools import groupby
|
||||
@@ -147,9 +151,9 @@ lessthancass30 = unittest.skipUnless(CASSANDRA_VERSION < '3.0', 'Cassandra versi
|
||||
def wait_for_node_socket(node, timeout):
|
||||
binary_itf = node.network_interfaces['binary']
|
||||
if not common.check_socket_listening(binary_itf, timeout=timeout):
|
||||
print("Unable to connect to binary socket for node"+str(node))
|
||||
log.warn("Unable to connect to binary socket for node " + node.name)
|
||||
else:
|
||||
print("Node is up and listening "+str(node))
|
||||
log.debug("Node %s is up and listening " % (node.name,))
|
||||
|
||||
|
||||
def check_socket_listening(itf, timeout=60):
|
||||
@@ -270,13 +274,13 @@ def use_cluster(cluster_name, nodes, ipformat=None, start=True, workloads=[]):
|
||||
node.set_workloads(workloads)
|
||||
log.debug("Starting CCM cluster: {0}".format(cluster_name))
|
||||
CCM_CLUSTER.start(wait_for_binary_proto=True, wait_other_notice=True, jvm_args=jvm_args)
|
||||
#Added to wait for slow nodes to start up
|
||||
# Added to wait for slow nodes to start up
|
||||
for node in CCM_CLUSTER.nodes.values():
|
||||
wait_for_node_socket(node, 120)
|
||||
setup_keyspace(ipformat=ipformat)
|
||||
except Exception:
|
||||
log.exception("Failed to start CCM cluster; removing cluster.")
|
||||
|
||||
|
||||
if os.name == "nt":
|
||||
if CCM_CLUSTER:
|
||||
for node in CCM_CLUSTER.nodes.itervalues():
|
||||
@@ -591,5 +595,3 @@ class BasicExistingSegregatedKeyspaceUnitTestCase(BasicKeyspaceUnitTestCase):
|
||||
|
||||
def tearDown(self):
|
||||
self.cluster.shutdown()
|
||||
|
||||
|
||||
|
@@ -70,35 +70,41 @@ class TestDatetime(BaseCassEngTestCase):
|
||||
return None
|
||||
|
||||
now = datetime(1982, 1, 1, tzinfo=TZ())
|
||||
dt = self.DatetimeTest.objects.create(test_id=0, created_at=now)
|
||||
dt2 = self.DatetimeTest.objects(test_id=0).first()
|
||||
dt = self.DatetimeTest.objects.create(test_id=1, created_at=now)
|
||||
dt2 = self.DatetimeTest.objects(test_id=1).first()
|
||||
assert dt2.created_at.timetuple()[:6] == (now + timedelta(hours=1)).timetuple()[:6]
|
||||
|
||||
def test_datetime_date_support(self):
|
||||
today = date.today()
|
||||
self.DatetimeTest.objects.create(test_id=0, created_at=today)
|
||||
dt2 = self.DatetimeTest.objects(test_id=0).first()
|
||||
self.DatetimeTest.objects.create(test_id=2, created_at=today)
|
||||
dt2 = self.DatetimeTest.objects(test_id=2).first()
|
||||
assert dt2.created_at.isoformat() == datetime(today.year, today.month, today.day).isoformat()
|
||||
|
||||
def test_datetime_none(self):
|
||||
dt = self.DatetimeTest.objects.create(test_id=1, created_at=None)
|
||||
dt2 = self.DatetimeTest.objects(test_id=1).first()
|
||||
dt = self.DatetimeTest.objects.create(test_id=3, created_at=None)
|
||||
dt2 = self.DatetimeTest.objects(test_id=3).first()
|
||||
assert dt2.created_at is None
|
||||
|
||||
dts = self.DatetimeTest.objects.filter(test_id=1).values_list('created_at')
|
||||
dts = self.DatetimeTest.objects.filter(test_id=3).values_list('created_at')
|
||||
assert dts[0][0] is None
|
||||
|
||||
def test_datetime_invalid(self):
|
||||
dt_value= 'INVALID'
|
||||
with self.assertRaises(TypeError):
|
||||
self.DatetimeTest.objects.create(test_id=2, created_at=dt_value)
|
||||
self.DatetimeTest.objects.create(test_id=4, created_at=dt_value)
|
||||
|
||||
def test_datetime_timestamp(self):
|
||||
dt_value = 1454520554
|
||||
self.DatetimeTest.objects.create(test_id=2, created_at=dt_value)
|
||||
dt2 = self.DatetimeTest.objects(test_id=2).first()
|
||||
self.DatetimeTest.objects.create(test_id=5, created_at=dt_value)
|
||||
dt2 = self.DatetimeTest.objects(test_id=5).first()
|
||||
assert dt2.created_at == datetime.utcfromtimestamp(dt_value)
|
||||
|
||||
def test_datetime_large(self):
|
||||
dt_value = datetime(2038, 12, 31, 10, 10, 10, 123000)
|
||||
self.DatetimeTest.objects.create(test_id=6, created_at=dt_value)
|
||||
dt2 = self.DatetimeTest.objects(test_id=6).first()
|
||||
assert dt2.created_at == dt_value
|
||||
|
||||
|
||||
class TestBoolDefault(BaseCassEngTestCase):
|
||||
class BoolDefaultValueTest(Model):
|
||||
|
@@ -81,9 +81,17 @@ class IfNotExistsInsertTests(BaseIfNotExistsTest):
|
||||
id = uuid4()
|
||||
|
||||
TestIfNotExistsModel.create(id=id, count=8, text='123456789')
|
||||
with self.assertRaises(LWTException):
|
||||
|
||||
with self.assertRaises(LWTException) as assertion:
|
||||
TestIfNotExistsModel.if_not_exists().create(id=id, count=9, text='111111111111')
|
||||
|
||||
self.assertEqual(assertion.exception.existing, {
|
||||
'count': 8,
|
||||
'id': id,
|
||||
'text': '123456789',
|
||||
'[applied]': False,
|
||||
})
|
||||
|
||||
q = TestIfNotExistsModel.objects(id=id)
|
||||
self.assertEqual(len(q), 1)
|
||||
|
||||
@@ -117,9 +125,16 @@ class IfNotExistsInsertTests(BaseIfNotExistsTest):
|
||||
|
||||
b = BatchQuery()
|
||||
TestIfNotExistsModel.batch(b).if_not_exists().create(id=id, count=9, text='111111111111')
|
||||
with self.assertRaises(LWTException):
|
||||
with self.assertRaises(LWTException) as assertion:
|
||||
b.execute()
|
||||
|
||||
self.assertEqual(assertion.exception.existing, {
|
||||
'count': 8,
|
||||
'id': id,
|
||||
'text': '123456789',
|
||||
'[applied]': False,
|
||||
})
|
||||
|
||||
q = TestIfNotExistsModel.objects(id=id)
|
||||
self.assertEqual(len(q), 1)
|
||||
|
||||
|
@@ -29,8 +29,9 @@ from cassandra.cqlengine.statements import TransactionClause
|
||||
from tests.integration.cqlengine.base import BaseCassEngTestCase
|
||||
from tests.integration import CASSANDRA_VERSION
|
||||
|
||||
|
||||
class TestTransactionModel(Model):
|
||||
id = columns.UUID(primary_key=True, default=lambda:uuid4())
|
||||
id = columns.UUID(primary_key=True, default=uuid4)
|
||||
count = columns.Integer()
|
||||
text = columns.Text(required=False)
|
||||
|
||||
@@ -71,7 +72,14 @@ class TestTransaction(BaseCassEngTestCase):
|
||||
t = TestTransactionModel.create(text='blah blah')
|
||||
t.text = 'new blah'
|
||||
t = t.iff(text='something wrong')
|
||||
self.assertRaises(LWTException, t.save)
|
||||
|
||||
with self.assertRaises(LWTException) as assertion:
|
||||
t.save()
|
||||
|
||||
self.assertEqual(assertion.exception.existing, {
|
||||
'text': 'blah blah',
|
||||
'[applied]': False,
|
||||
})
|
||||
|
||||
def test_blind_update(self):
|
||||
t = TestTransactionModel.create(text='blah blah')
|
||||
@@ -89,7 +97,13 @@ class TestTransaction(BaseCassEngTestCase):
|
||||
t.text = 'something else'
|
||||
uid = t.id
|
||||
qs = TestTransactionModel.objects(id=uid).iff(text='Not dis!')
|
||||
self.assertRaises(LWTException, qs.update, text='this will never work')
|
||||
with self.assertRaises(LWTException) as assertion:
|
||||
qs.update(text='this will never work')
|
||||
|
||||
self.assertEqual(assertion.exception.existing, {
|
||||
'text': 'blah blah',
|
||||
'[applied]': False,
|
||||
})
|
||||
|
||||
def test_transaction_clause(self):
|
||||
tc = TransactionClause('some_value', 23)
|
||||
@@ -109,7 +123,14 @@ class TestTransaction(BaseCassEngTestCase):
|
||||
|
||||
b = BatchQuery()
|
||||
updated.batch(b).iff(count=6).update(text='and another thing')
|
||||
self.assertRaises(LWTException, b.execute)
|
||||
with self.assertRaises(LWTException) as assertion:
|
||||
b.execute()
|
||||
|
||||
self.assertEqual(assertion.exception.existing, {
|
||||
'id': id,
|
||||
'count': 5,
|
||||
'[applied]': False,
|
||||
})
|
||||
|
||||
updated = TestTransactionModel.objects(id=id).first()
|
||||
self.assertEqual(updated.text, 'something else')
|
||||
|
@@ -113,3 +113,44 @@ class SchemaTests(unittest.TestCase):
|
||||
execute_until_pass(session, "INSERT INTO test.cf (key, value) VALUES ({0}, {0})".format(j))
|
||||
|
||||
execute_until_pass(session, "DROP KEYSPACE test")
|
||||
|
||||
def test_for_schema_disagreement_attribute(self):
|
||||
"""
|
||||
Tests to ensure that schema disagreement is properly surfaced on the response future.
|
||||
|
||||
Creates and destroys keyspaces/tables with various schema agreement timeouts set.
|
||||
First part runs cql create/drop cmds with schema agreement set in such away were it will be impossible for agreement to occur during timeout.
|
||||
It then validates that the correct value is set on the result.
|
||||
Second part ensures that when schema agreement occurs, that the result set reflects that appropriately
|
||||
|
||||
@since 3.1.0
|
||||
@jira_ticket PYTHON-458
|
||||
@expected_result is_schema_agreed is set appropriately on response thefuture
|
||||
|
||||
@test_category schema
|
||||
"""
|
||||
# This should yield a schema disagreement
|
||||
cluster = Cluster(protocol_version=PROTOCOL_VERSION, max_schema_agreement_wait=0.001)
|
||||
session = cluster.connect()
|
||||
|
||||
rs = session.execute("CREATE KEYSPACE test_schema_disagreement WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}")
|
||||
self.check_and_wait_for_agreement(session, rs, False)
|
||||
rs = session.execute("CREATE TABLE test_schema_disagreement.cf (key int PRIMARY KEY, value int)")
|
||||
self.check_and_wait_for_agreement(session, rs, False)
|
||||
rs = session.execute("DROP KEYSPACE test_schema_disagreement")
|
||||
self.check_and_wait_for_agreement(session, rs, False)
|
||||
|
||||
# These should have schema agreement
|
||||
cluster = Cluster(protocol_version=PROTOCOL_VERSION, max_schema_agreement_wait=100)
|
||||
session = cluster.connect()
|
||||
rs = session.execute("CREATE KEYSPACE test_schema_disagreement WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}")
|
||||
self.check_and_wait_for_agreement(session, rs, True)
|
||||
rs = session.execute("CREATE TABLE test_schema_disagreement.cf (key int PRIMARY KEY, value int)")
|
||||
self.check_and_wait_for_agreement(session, rs, True)
|
||||
rs = session.execute("DROP KEYSPACE test_schema_disagreement")
|
||||
self.check_and_wait_for_agreement(session, rs, True)
|
||||
|
||||
def check_and_wait_for_agreement(self, session, rs, exepected):
|
||||
self.assertEqual(rs.response_future.is_schema_agreed, exepected)
|
||||
if not rs.response_future.is_schema_agreed:
|
||||
session.cluster.control_connection.wait_for_schema_agreement(wait_time=1000)
|
||||
|
28
tests/unit/cython/test_types.py
Normal file
28
tests/unit/cython/test_types.py
Normal file
@@ -0,0 +1,28 @@
|
||||
# Copyright 2013-2016 DataStax, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
from tests.unit.cython.utils import cyimport, cythontest
|
||||
types_testhelper = cyimport('tests.unit.cython.types_testhelper')
|
||||
|
||||
try:
|
||||
import unittest2 as unittest
|
||||
except ImportError:
|
||||
import unittest # noqa
|
||||
|
||||
|
||||
class TypesTest(unittest.TestCase):
|
||||
|
||||
@cythontest
|
||||
def test_datetype(self):
|
||||
types_testhelper.test_datetype(self.assertEqual)
|
29
tests/unit/cython/test_utils.py
Normal file
29
tests/unit/cython/test_utils.py
Normal file
@@ -0,0 +1,29 @@
|
||||
# Copyright 2013-2016 DataStax, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
from tests.unit.cython.utils import cyimport, cythontest
|
||||
utils_testhelper = cyimport('tests.unit.cython.utils_testhelper')
|
||||
|
||||
try:
|
||||
import unittest2 as unittest
|
||||
except ImportError:
|
||||
import unittest # noqa
|
||||
|
||||
|
||||
class UtilsTest(unittest.TestCase):
|
||||
"""Test Cython Utils functions"""
|
||||
|
||||
@cythontest
|
||||
def test_datetime_from_timestamp(self):
|
||||
utils_testhelper.test_datetime_from_timestamp(self.assertEqual)
|
72
tests/unit/cython/types_testhelper.pyx
Normal file
72
tests/unit/cython/types_testhelper.pyx
Normal file
@@ -0,0 +1,72 @@
|
||||
# Copyright 2013-2016 DataStax, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
|
||||
import time
|
||||
import datetime
|
||||
|
||||
include '../../../cassandra/ioutils.pyx'
|
||||
|
||||
import io
|
||||
|
||||
from cassandra.cqltypes import DateType
|
||||
from cassandra.protocol import write_value
|
||||
from cassandra.deserializers import find_deserializer
|
||||
from cassandra.bytesio cimport BytesIOReader
|
||||
from cassandra.buffer cimport Buffer
|
||||
from cassandra.deserializers cimport from_binary, Deserializer
|
||||
|
||||
|
||||
def test_datetype(assert_equal):
|
||||
|
||||
cdef Deserializer des = find_deserializer(DateType)
|
||||
|
||||
def deserialize(timestamp):
|
||||
"""Serialize a datetime and deserialize it using the cython deserializer"""
|
||||
|
||||
cdef BytesIOReader reader
|
||||
cdef Buffer buf
|
||||
|
||||
dt = datetime.datetime.utcfromtimestamp(timestamp)
|
||||
|
||||
bytes = io.BytesIO()
|
||||
write_value(bytes, DateType.serialize(dt, 0))
|
||||
bytes.seek(0)
|
||||
reader = BytesIOReader(bytes.read())
|
||||
get_buf(reader, &buf)
|
||||
deserialized_dt = from_binary(des, &buf, 0)
|
||||
|
||||
return deserialized_dt
|
||||
|
||||
# deserialize
|
||||
# epoc
|
||||
expected = 0
|
||||
assert_equal(deserialize(expected), datetime.datetime.utcfromtimestamp(expected))
|
||||
|
||||
# beyond 32b
|
||||
expected = 2 ** 33
|
||||
assert_equal(deserialize(expected), datetime.datetime(2242, 3, 16, 12, 56, 32))
|
||||
|
||||
# less than epoc (PYTHON-119)
|
||||
expected = -770172256
|
||||
assert_equal(deserialize(expected), datetime.datetime(1945, 8, 5, 23, 15, 44))
|
||||
|
||||
# work around rounding difference among Python versions (PYTHON-230)
|
||||
# This wont pass with the cython extension until we fix the microseconds alignment with CPython
|
||||
#expected = 1424817268.274
|
||||
#assert_equal(deserialize(expected), datetime.datetime(2015, 2, 24, 22, 34, 28, 274000))
|
||||
|
||||
# Large date overflow (PYTHON-452)
|
||||
expected = 2177403010.123
|
||||
assert_equal(deserialize(expected), datetime.datetime(2038, 12, 31, 10, 10, 10, 123000))
|
23
tests/unit/cython/utils_testhelper.pyx
Normal file
23
tests/unit/cython/utils_testhelper.pyx
Normal file
@@ -0,0 +1,23 @@
|
||||
# Copyright 2013-2016 DataStax, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
import datetime
|
||||
|
||||
from cassandra.cython_utils cimport datetime_from_timestamp
|
||||
|
||||
|
||||
def test_datetime_from_timestamp(assert_equal):
|
||||
assert_equal(datetime_from_timestamp(1454781157.123456), datetime.datetime(2016, 2, 6, 17, 52, 37, 123456))
|
||||
# PYTHON-452
|
||||
assert_equal(datetime_from_timestamp(2177403010.123456), datetime.datetime(2038, 12, 31, 10, 10, 10, 123456))
|
38
tests/unit/test_cluster_task_scheduler.py
Normal file
38
tests/unit/test_cluster_task_scheduler.py
Normal file
@@ -0,0 +1,38 @@
|
||||
# Copyright 2013-2016 DataStax, Inc.
|
||||
#
|
||||
# Licensed under the Apache License, Version 2.0 (the "License");
|
||||
# you may not use this file except in compliance with the License.
|
||||
# You may obtain a copy of the License at
|
||||
#
|
||||
# http://www.apache.org/licenses/LICENSE-2.0
|
||||
#
|
||||
# Unless required by applicable law or agreed to in writing, software
|
||||
# distributed under the License is distributed on an "AS IS" BASIS,
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
|
||||
try:
|
||||
import unittest2 as unittest
|
||||
except ImportError:
|
||||
import unittest # noqa
|
||||
|
||||
from mock import patch
|
||||
|
||||
from cassandra.cluster import _Scheduler
|
||||
|
||||
|
||||
class SchedulerTest(unittest.TestCase):
|
||||
# TODO: this suite could be expanded; for now just adding a test covering a ticket
|
||||
|
||||
@patch('time.time', return_value=3) # always queue at same time
|
||||
@patch('cassandra.cluster._Scheduler.run') # don't actually run the thread
|
||||
def test_event_delay_timing(self, *args):
|
||||
"""
|
||||
Schedule something with a time collision to make sure the heap comparison works
|
||||
|
||||
PYTHON-473
|
||||
"""
|
||||
sched = _Scheduler(None)
|
||||
sched.schedule(0, lambda: None)
|
||||
sched.schedule(0, lambda: None) # pre-473: "TypeError: unorderable types: function() < function()"t
|
@@ -24,7 +24,7 @@ from threading import Lock
|
||||
|
||||
from cassandra.cluster import Cluster, Session
|
||||
from cassandra.connection import (Connection, HEADER_DIRECTION_TO_CLIENT, ProtocolError,
|
||||
locally_supported_compressions, ConnectionHeartbeat, _Frame)
|
||||
locally_supported_compressions, ConnectionHeartbeat, _Frame, Timer, TimerManager)
|
||||
from cassandra.marshal import uint8_pack, uint32_pack, int32_pack
|
||||
from cassandra.protocol import (write_stringmultimap, write_int, write_string,
|
||||
SupportedMessage, ProtocolHandler)
|
||||
@@ -413,3 +413,19 @@ class ConnectionHeartbeatTest(unittest.TestCase):
|
||||
self.assertIsInstance(exc, Exception)
|
||||
self.assertEqual(exc.args, Exception('Connection heartbeat failure').args)
|
||||
holder.return_connection.assert_has_calls([call(connection)] * get_holders.call_count)
|
||||
|
||||
|
||||
class TimerTest(unittest.TestCase):
|
||||
|
||||
def test_timer_collision(self):
|
||||
# simple test demonstrating #466
|
||||
# same timeout, comparison will defer to the Timer object itself
|
||||
t1 = Timer(0, lambda: None)
|
||||
t2 = Timer(0, lambda: None)
|
||||
t2.end = t1.end
|
||||
|
||||
tm = TimerManager()
|
||||
tm.add_timer(t1)
|
||||
tm.add_timer(t2)
|
||||
# Prior to #466: "TypeError: unorderable types: Timer() < Timer()"
|
||||
tm.service_timeouts()
|
||||
|
@@ -11,6 +11,7 @@
|
||||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
# See the License for the specific language governing permissions and
|
||||
# limitations under the License.
|
||||
from cassandra.query import named_tuple_factory, dict_factory, tuple_factory
|
||||
|
||||
try:
|
||||
import unittest2 as unittest
|
||||
@@ -161,3 +162,29 @@ class ResultSetTests(unittest.TestCase):
|
||||
def test_bool(self):
|
||||
self.assertFalse(ResultSet(Mock(has_more_pages=False), []))
|
||||
self.assertTrue(ResultSet(Mock(has_more_pages=False), [1]))
|
||||
|
||||
def test_was_applied(self):
|
||||
# unknown row factory raises
|
||||
with self.assertRaises(RuntimeError):
|
||||
ResultSet(Mock(), []).was_applied
|
||||
|
||||
response_future = Mock(row_factory=named_tuple_factory)
|
||||
|
||||
# no row
|
||||
with self.assertRaises(RuntimeError):
|
||||
ResultSet(response_future, []).was_applied
|
||||
|
||||
# too many rows
|
||||
with self.assertRaises(RuntimeError):
|
||||
ResultSet(response_future, [tuple(), tuple()]).was_applied
|
||||
|
||||
# various internal row factories
|
||||
for row_factory in (named_tuple_factory, tuple_factory):
|
||||
for applied in (True, False):
|
||||
rs = ResultSet(Mock(row_factory=row_factory), [(applied,)])
|
||||
self.assertEqual(rs.was_applied, applied)
|
||||
|
||||
row_factory = dict_factory
|
||||
for applied in (True, False):
|
||||
rs = ResultSet(Mock(row_factory=row_factory), [{'[applied]': applied}])
|
||||
self.assertEqual(rs.was_applied, applied)
|
||||
|
@@ -36,6 +36,8 @@ class TimeUtilTest(unittest.TestCase):
|
||||
|
||||
self.assertEqual(util.datetime_from_timestamp(0.123456), datetime.datetime(1970, 1, 1, 0, 0, 0, 123456))
|
||||
|
||||
self.assertEqual(util.datetime_from_timestamp(2177403010.123456), datetime.datetime(2038, 12, 31, 10, 10, 10, 123456))
|
||||
|
||||
def test_times_from_uuid1(self):
|
||||
node = uuid.getnode()
|
||||
now = time.time()
|
||||
|
@@ -204,6 +204,10 @@ class TypeTests(unittest.TestCase):
|
||||
expected = 1424817268.274
|
||||
self.assertEqual(DateType.deserialize(int64_pack(int(1000 * expected)), 0), datetime.datetime(2015, 2, 24, 22, 34, 28, 274000))
|
||||
|
||||
# Large date overflow (PYTHON-452)
|
||||
expected = 2177403010.123
|
||||
self.assertEqual(DateType.deserialize(int64_pack(int(1000 * expected)), 0), datetime.datetime(2038, 12, 31, 10, 10, 10, 123000))
|
||||
|
||||
def test_write_read_string(self):
|
||||
with tempfile.TemporaryFile() as f:
|
||||
value = u'test'
|
||||
|
Reference in New Issue
Block a user