Merge branch 'consistency_test_merge'

This commit is contained in:
Tyler Hobbs
2013-12-27 17:52:10 -06:00
16 changed files with 433 additions and 13 deletions

View File

@@ -283,14 +283,23 @@ class Cluster(object):
self.auth_provider = auth_provider
if load_balancing_policy is not None:
if isinstance(load_balancing_policy, type):
raise TypeError("load_balancing_policy should not be a class, it should be an instance of that class")
self.load_balancing_policy = load_balancing_policy
else:
self.load_balancing_policy = RoundRobinPolicy()
if reconnection_policy is not None:
if isinstance(reconnection_policy, type):
raise TypeError("reconnection_policy should not be a class, it should be an instance of that class")
self.reconnection_policy = reconnection_policy
if default_retry_policy is not None:
if isinstance(default_retry_policy, type):
raise TypeError("default_retry_policy should not be a class, it should be an instance of that class")
self.default_retry_policy = default_retry_policy
if conviction_policy_factory is not None:

View File

@@ -362,14 +362,15 @@ class SimpleStrategy(ReplicationStrategy):
def make_token_replica_map(self, token_to_host_owner, ring):
replica_map = {}
for i in range(len(ring)):
j, hosts = 0, set()
j, hosts = 0, list()
while len(hosts) < self.replication_factor and j < len(ring):
token = ring[(i + j) % len(ring)]
hosts.add(token_to_host_owner[token])
host = token_to_host_owner[token]
if not host in hosts:
hosts.append(host)
j += 1
replica_map[ring[i]] = hosts
return replica_map
def export_for_schema(self):
@@ -386,7 +387,7 @@ class NetworkTopologyStrategy(ReplicationStrategy):
def make_token_replica_map(self, token_to_host_owner, ring):
# note: this does not account for hosts having different racks
replica_map = defaultdict(set)
replica_map = defaultdict(list)
ring_len = len(ring)
ring_len_range = range(ring_len)
dc_rf_map = dict((dc, int(rf))
@@ -403,7 +404,8 @@ class NetworkTopologyStrategy(ReplicationStrategy):
# we already have all replicas for this DC
continue
replica_map[ring[i]].add(host)
if not host in replica_map[ring[i]]:
replica_map[ring[i]].append(host)
if remaining[dc] == 1:
del remaining[dc]
@@ -411,7 +413,6 @@ class NetworkTopologyStrategy(ReplicationStrategy):
break
else:
remaining[dc] -= 1
return replica_map
def export_for_schema(self):

View File

View File

@@ -0,0 +1,265 @@
import struct
import traceback
import cassandra
from cassandra import ConsistencyLevel
from cassandra.cluster import Cluster
from cassandra.policies import TokenAwarePolicy, RoundRobinPolicy, \
DowngradingConsistencyRetryPolicy
from cassandra.query import SimpleStatement
from tests.integration.long.utils import force_stop, create_schema, \
wait_for_down, wait_for_up, start, CoordinatorStats
try:
import unittest2 as unittest
except ImportError:
import unittest # noqa
ALL_CONSISTENCY_LEVELS = set([
ConsistencyLevel.ANY, ConsistencyLevel.ONE, ConsistencyLevel.TWO,
ConsistencyLevel.QUORUM, ConsistencyLevel.THREE,
ConsistencyLevel.ALL, ConsistencyLevel.LOCAL_QUORUM,
ConsistencyLevel.EACH_QUORUM])
MULTI_DC_CONSISTENCY_LEVELS = set([
ConsistencyLevel.LOCAL_QUORUM, ConsistencyLevel.EACH_QUORUM])
SINGLE_DC_CONSISTENCY_LEVELS = ALL_CONSISTENCY_LEVELS - MULTI_DC_CONSISTENCY_LEVELS
class ConsistencyTests(unittest.TestCase):
def setUp(self):
self.coordinator_stats = CoordinatorStats()
def _cl_failure(self, consistency_level, e):
self.fail('Instead of success, saw %s for CL.%s:\n\n%s' % (
e, ConsistencyLevel.value_to_name[consistency_level],
traceback.format_exc()))
def _cl_expected_failure(self, cl):
self.fail('Test passed at ConsistencyLevel.%s:\n\n%s' % (
ConsistencyLevel.value_to_name[cl], traceback.format_exc()))
def _insert(self, session, keyspace, count, consistency_level=ConsistencyLevel.ONE):
session.execute('USE %s' % keyspace)
for i in range(count):
ss = SimpleStatement('INSERT INTO cf(k, i) VALUES (0, 0)',
consistency_level=consistency_level)
session.execute(ss)
def _query(self, session, keyspace, count, consistency_level=ConsistencyLevel.ONE):
routing_key = struct.pack('>i', 0)
for i in range(count):
ss = SimpleStatement('SELECT * FROM cf WHERE k = 0',
consistency_level=consistency_level,
routing_key=routing_key)
self.coordinator_stats.add_coordinator(session.execute_async(ss))
def _assert_writes_succeed(self, session, keyspace, consistency_levels):
for cl in consistency_levels:
self.coordinator_stats.reset_counts()
try:
self._insert(session, keyspace, 1, cl)
except Exception as e:
self._cl_failure(cl, e)
def _assert_reads_succeed(self, session, keyspace, consistency_levels, expected_reader=3):
for cl in consistency_levels:
self.coordinator_stats.reset_counts()
try:
self._query(session, keyspace, 1, cl)
for i in range(3):
if i == expected_reader:
self.coordinator_stats.assert_query_count_equals(self, i, 1)
else:
self.coordinator_stats.assert_query_count_equals(self, i, 0)
except Exception as e:
self._cl_failure(cl, e)
def _assert_writes_fail(self, session, keyspace, consistency_levels):
for cl in consistency_levels:
self.coordinator_stats.reset_counts()
try:
self._insert(session, keyspace, 1, cl)
self._cl_expected_failure(cl)
except (cassandra.Unavailable, cassandra.WriteTimeout):
pass
def _assert_reads_fail(self, session, keyspace, consistency_levels):
for cl in consistency_levels:
self.coordinator_stats.reset_counts()
try:
self._query(session, keyspace, 1, cl)
self._cl_expected_failure(cl)
except (cassandra.Unavailable, cassandra.ReadTimeout):
pass
def _test_tokenaware_one_node_down(self, keyspace, rf, accepted):
cluster = Cluster(
load_balancing_policy=TokenAwarePolicy(RoundRobinPolicy()))
session = cluster.connect()
wait_for_up(cluster, 1, wait=False)
wait_for_up(cluster, 2)
create_schema(session, keyspace, replication_factor=rf)
self._insert(session, keyspace, count=1)
self._query(session, keyspace, count=1)
self.coordinator_stats.assert_query_count_equals(self, 1, 0)
self.coordinator_stats.assert_query_count_equals(self, 2, 1)
self.coordinator_stats.assert_query_count_equals(self, 3, 0)
try:
force_stop(2)
wait_for_down(cluster, 2)
self._assert_writes_succeed(session, keyspace, accepted)
self._assert_reads_succeed(session, keyspace,
accepted - set([ConsistencyLevel.ANY]))
self._assert_writes_fail(session, keyspace,
SINGLE_DC_CONSISTENCY_LEVELS - accepted)
self._assert_reads_fail(session, keyspace,
SINGLE_DC_CONSISTENCY_LEVELS - accepted)
finally:
start(2)
wait_for_up(cluster, 2)
def test_rfone_tokenaware_one_node_down(self):
self._test_tokenaware_one_node_down(
keyspace='test_rfone_tokenaware',
rf=1,
accepted=set([ConsistencyLevel.ANY]))
def test_rftwo_tokenaware_one_node_down(self):
self._test_tokenaware_one_node_down(
keyspace='test_rftwo_tokenaware',
rf=2,
accepted=set([ConsistencyLevel.ANY, ConsistencyLevel.ONE]))
def test_rfthree_tokenaware_one_node_down(self):
self._test_tokenaware_one_node_down(
keyspace='test_rfthree_tokenaware',
rf=3,
accepted=set([ConsistencyLevel.ANY, ConsistencyLevel.ONE,
ConsistencyLevel.TWO, ConsistencyLevel.QUORUM]))
def test_rfthree_tokenaware_none_down(self):
keyspace = 'test_rfthree_tokenaware_none_down'
cluster = Cluster(
load_balancing_policy=TokenAwarePolicy(RoundRobinPolicy()))
session = cluster.connect()
wait_for_up(cluster, 1, wait=False)
wait_for_up(cluster, 2)
create_schema(session, keyspace, replication_factor=3)
self._insert(session, keyspace, count=1)
self._query(session, keyspace, count=1)
self.coordinator_stats.assert_query_count_equals(self, 1, 0)
self.coordinator_stats.assert_query_count_equals(self, 2, 1)
self.coordinator_stats.assert_query_count_equals(self, 3, 0)
self.coordinator_stats.reset_counts()
self._assert_writes_succeed(session, keyspace, SINGLE_DC_CONSISTENCY_LEVELS)
self._assert_reads_succeed(session, keyspace,
SINGLE_DC_CONSISTENCY_LEVELS - set([ConsistencyLevel.ANY]),
expected_reader=2)
def _test_downgrading_cl(self, keyspace, rf, accepted):
cluster = Cluster(
load_balancing_policy=TokenAwarePolicy(RoundRobinPolicy()),
default_retry_policy=DowngradingConsistencyRetryPolicy())
session = cluster.connect()
create_schema(session, keyspace, replication_factor=rf)
self._insert(session, keyspace, 1)
self._query(session, keyspace, 1)
self.coordinator_stats.assert_query_count_equals(self, 1, 0)
self.coordinator_stats.assert_query_count_equals(self, 2, 1)
self.coordinator_stats.assert_query_count_equals(self, 3, 0)
try:
force_stop(2)
wait_for_down(cluster, 2)
self._assert_writes_succeed(session, keyspace, accepted)
self._assert_reads_succeed(session, keyspace,
accepted - set([ConsistencyLevel.ANY]))
self._assert_writes_fail(session, keyspace,
SINGLE_DC_CONSISTENCY_LEVELS - accepted)
self._assert_reads_fail(session, keyspace,
SINGLE_DC_CONSISTENCY_LEVELS - accepted)
finally:
start(2)
wait_for_up(cluster, 2)
def test_rfone_downgradingcl(self):
self._test_downgrading_cl(
keyspace='test_rfone_downgradingcl',
rf=1,
accepted=set([ConsistencyLevel.ANY]))
def test_rftwo_downgradingcl(self):
self._test_downgrading_cl(
keyspace='test_rftwo_downgradingcl',
rf=2,
accepted=SINGLE_DC_CONSISTENCY_LEVELS)
def test_rfthree_roundrobin_downgradingcl(self):
keyspace = 'test_rfthree_roundrobin_downgradingcl'
cluster = Cluster(
load_balancing_policy=RoundRobinPolicy(),
default_retry_policy=DowngradingConsistencyRetryPolicy())
self.rfthree_downgradingcl(cluster, keyspace, True)
def test_rfthree_tokenaware_downgradingcl(self):
keyspace = 'test_rfthree_tokenaware_downgradingcl'
cluster = Cluster(
load_balancing_policy=TokenAwarePolicy(RoundRobinPolicy()),
default_retry_policy=DowngradingConsistencyRetryPolicy())
self.rfthree_downgradingcl(cluster, keyspace, False)
def rfthree_downgradingcl(self, cluster, keyspace, roundrobin):
session = cluster.connect()
create_schema(session, keyspace, replication_factor=2)
self._insert(session, keyspace, count=12)
self._query(session, keyspace, count=12)
if roundrobin:
self.coordinator_stats.assert_query_count_equals(self, 1, 4)
self.coordinator_stats.assert_query_count_equals(self, 2, 4)
self.coordinator_stats.assert_query_count_equals(self, 3, 4)
else:
self.coordinator_stats.assert_query_count_equals(self, 1, 0)
self.coordinator_stats.assert_query_count_equals(self, 2, 12)
self.coordinator_stats.assert_query_count_equals(self, 3, 0)
try:
self.coordinator_stats.reset_counts()
force_stop(2)
wait_for_down(cluster, 2)
self._assert_writes_succeed(session, keyspace, SINGLE_DC_CONSISTENCY_LEVELS)
# Test reads that expected to complete successfully
for cl in SINGLE_DC_CONSISTENCY_LEVELS - set([ConsistencyLevel.ANY]):
self.coordinator_stats.reset_counts()
self._query(session, keyspace, 12, consistency_level=cl)
if roundrobin:
self.coordinator_stats.assert_query_count_equals(self, 1, 6)
self.coordinator_stats.assert_query_count_equals(self, 2, 0)
self.coordinator_stats.assert_query_count_equals(self, 3, 6)
else:
self.coordinator_stats.assert_query_count_equals(self, 1, 0)
self.coordinator_stats.assert_query_count_equals(self, 2, 0)
self.coordinator_stats.assert_query_count_equals(self, 3, 12)
finally:
start(2)
wait_for_up(cluster, 2)
# TODO: can't be done in this class since we reuse the ccm cluster
# instead we should create these elsewhere
# def test_rfthree_downgradingcl_twodcs(self):
# def test_rfthree_downgradingcl_twodcs_dcaware(self):

View File

@@ -0,0 +1,51 @@
import logging
import cassandra
from cassandra import ConsistencyLevel
from cassandra.cluster import Cluster
from cassandra.query import SimpleStatement
try:
import unittest2 as unittest
except ImportError:
import unittest # noqa
log = logging.getLogger(__name__)
class SchemaTests(unittest.TestCase):
def test_recreates(self):
cluster = Cluster()
session = cluster.connect()
replication_factor = 3
for i in range(2):
for keyspace in range(0, 100):
keyspace = 'ks_%s' % keyspace
results = session.execute('SELECT keyspace_name FROM system.schema_keyspaces')
existing_keyspaces = [row[0] for row in results]
if keyspace in existing_keyspaces:
ddl = 'DROP KEYSPACE %s' % keyspace
log.debug(ddl)
session.execute(ddl)
ddl = "CREATE KEYSPACE %s WITH replication" \
" = {'class': 'SimpleStrategy', 'replication_factor': '%s'}" % (keyspace, replication_factor)
log.debug(ddl)
session.execute(ddl)
ddl = 'CREATE TABLE %s.cf (k int PRIMARY KEY, i int)' % keyspace
log.debug(ddl)
session.execute(ddl)
statement = 'USE %s' % keyspace
log.debug(ddl)
session.execute(statement)
statement = 'INSERT INTO %s(k, i) VALUES (0, 0)' % 'cf'
log.debug(statement)
ss = SimpleStatement(statement,
consistency_level=ConsistencyLevel.QUORUM)
session.execute(ss)

View File

@@ -0,0 +1,94 @@
import logging
import time
from collections import defaultdict
from tests.integration import get_node
log = logging.getLogger(__name__)
class CoordinatorStats():
def __init__(self):
self.coordinator_counts = defaultdict(int)
def add_coordinator(self, future):
coordinator = future._current_host.address
self.coordinator_counts[coordinator] += 1
if future._errors:
log.error('future._errors: %s', future._errors)
future.result()
def reset_counts(self):
self.coordinator_counts = defaultdict(int)
def assert_query_count_equals(self, testcase, node, expected):
ip = '127.0.0.%d' % node
if self.coordinator_counts[ip] != expected:
testcase.fail('Expected %d queries to %s, but got %d. Query counts: %s' % (
expected, ip, self.coordinator_counts[ip], dict(self.coordinator_counts)))
def create_schema(session, keyspace, simple_strategy=True,
replication_factor=1, replication_strategy=None):
results = session.execute(
'SELECT keyspace_name FROM system.schema_keyspaces')
existing_keyspaces = [row[0] for row in results]
if keyspace in existing_keyspaces:
session.execute('DROP KEYSPACE %s' % keyspace, timeout=10)
if simple_strategy:
ddl = "CREATE KEYSPACE %s WITH replication" \
" = {'class': 'SimpleStrategy', 'replication_factor': '%s'}"
session.execute(ddl % (keyspace, replication_factor), timeout=10)
else:
if not replication_strategy:
raise Exception('replication_strategy is not set')
ddl = "CREATE KEYSPACE %s" \
" WITH replication = { 'class' : 'NetworkTopologyStrategy', %s }"
session.execute(ddl % (keyspace, str(replication_strategy)[1:-1]), timeout=10)
ddl = 'CREATE TABLE %s.cf (k int PRIMARY KEY, i int)'
session.execute(ddl % keyspace, timeout=10)
session.execute('USE %s' % keyspace)
def start(node):
get_node(node).start()
def stop(node):
get_node(node).stop()
def force_stop(node):
get_node(node).stop(wait=False, gently=False)
def ring(node):
print 'From node%s:' % node
get_node(node).nodetool('ring')
def wait_for_up(cluster, node, wait=True):
while True:
host = cluster.metadata.get_host('127.0.0.%s' % node)
if host and host.is_up:
# BUG: shouldn't have to, but we do
if wait:
time.sleep(5)
return
def wait_for_down(cluster, node, wait=True):
while True:
host = cluster.metadata.get_host('127.0.0.%s' % node)
if not host or not host.is_up:
# BUG: shouldn't have to, but we do
if wait:
time.sleep(5)
return

View File

View File

@@ -349,9 +349,9 @@ class TestCodeCoverage(unittest.TestCase):
self.assertNotEqual(list(get_replicas('test3rf', ring[0])), [])
for i, token in enumerate(ring):
self.assertEqual(get_replicas('test3rf', token), set(owners))
self.assertEqual(get_replicas('test2rf', token), set([owners[i], owners[(i + 1) % 3]]))
self.assertEqual(get_replicas('test1rf', token), set([owners[i]]))
self.assertEqual(set(get_replicas('test3rf', token)), set(owners))
self.assertEqual(set(get_replicas('test2rf', token)), set([owners[i], owners[(i + 1) % 3]]))
self.assertEqual(set(get_replicas('test1rf', token)), set([owners[i]]))
class TokenMetadataTest(unittest.TestCase):
@@ -379,15 +379,15 @@ class TokenMetadataTest(unittest.TestCase):
# tokens match node tokens exactly
for token, expected_host in zip(tokens, hosts):
replicas = token_map.get_replicas("ks", token)
self.assertEqual(replicas, set([expected_host]))
self.assertEqual(set(replicas), set([expected_host]))
# shift the tokens back by one
for token, expected_host in zip(tokens[1:], hosts[1:]):
replicas = token_map.get_replicas("ks", MD5Token(str(token.value - 1)))
self.assertEqual(replicas, set([expected_host]))
self.assertEqual(set(replicas), set([expected_host]))
# shift the tokens forward by one
for i, token in enumerate(tokens):
replicas = token_map.get_replicas("ks", MD5Token(str(token.value + 1)))
expected_host = hosts[(i + 1) % len(hosts)]
self.assertEqual(replicas, set([expected_host]))
self.assertEqual(set(replicas), set([expected_host]))

View File

@@ -96,7 +96,7 @@ class TestStrategies(unittest.TestCase):
nts = NetworkTopologyStrategy({'dc1': 1, 'dc2': 0})
replica_map = nts.make_token_replica_map(token_to_host_owner, ring)
self.assertEqual(replica_map[MD5Token(0)], set([host]))
self.assertEqual(set(replica_map[MD5Token(0)]), set([host]))
def test_nts_export_for_schema(self):
# TODO: Cover NetworkTopologyStrategy.export_for_schema()