Skip to content

Commit bb501dd

Browse files
committed
Moved test_getting_replicas to unit tests
1 parent 32037f3 commit bb501dd

2 files changed

Lines changed: 39 additions & 31 deletions

File tree

tests/integration/standard/test_metadata.py

Lines changed: 2 additions & 30 deletions
Original file line numberDiff line numberDiff line change
@@ -27,11 +27,8 @@
2727

2828
from cassandra.cluster import Cluster
2929
from cassandra.encoder import Encoder
30-
from cassandra.metadata import (Metadata, KeyspaceMetadata, IndexMetadata,
31-
Token, MD5Token, TokenMap, murmur3, Function, Aggregate, protect_name, protect_names,
32-
get_schema_parser, RegisteredTableExtension, _RegisteredExtensionType)
33-
from cassandra.policies import SimpleConvictionPolicy
34-
from cassandra.pool import Host
30+
from cassandra.metadata import (IndexMetadata, Token, murmur3, Function, Aggregate, protect_name, protect_names,
31+
RegisteredTableExtension, _RegisteredExtensionType, get_schema_parser,)
3532

3633
from tests.integration import (get_cluster, use_singledc, PROTOCOL_VERSION, get_server_versions, execute_until_pass,
3734
BasicSegregatedKeyspaceUnitTestCase, BasicSharedKeyspaceUnitTestCase,
@@ -1465,31 +1462,6 @@ def test_token(self):
14651462
self.assertEqual(expected_node_count, len(tmap.ring))
14661463
cluster.shutdown()
14671464

1468-
def test_getting_replicas(self):
1469-
tokens = [MD5Token(i) for i in range(0, (2 ** 127 - 1), 2 ** 125)]
1470-
hosts = [Host("ip%d" % i, SimpleConvictionPolicy) for i in range(len(tokens))]
1471-
token_to_primary_replica = dict(zip(tokens, hosts))
1472-
keyspace = KeyspaceMetadata("ks", True, "SimpleStrategy", {"replication_factor": "1"})
1473-
metadata = Mock(spec=Metadata, keyspaces={'ks': keyspace})
1474-
token_map = TokenMap(MD5Token, token_to_primary_replica, tokens, metadata)
1475-
1476-
# tokens match node tokens exactly
1477-
for i, token in enumerate(tokens):
1478-
expected_host = hosts[(i + 1) % len(hosts)]
1479-
replicas = token_map.get_replicas("ks", token)
1480-
self.assertEqual(set(replicas), set([expected_host]))
1481-
1482-
# shift the tokens back by one
1483-
for token, expected_host in zip(tokens, hosts):
1484-
replicas = token_map.get_replicas("ks", MD5Token(token.value - 1))
1485-
self.assertEqual(set(replicas), set([expected_host]))
1486-
1487-
# shift the tokens forward by one
1488-
for i, token in enumerate(tokens):
1489-
replicas = token_map.get_replicas("ks", MD5Token(token.value + 1))
1490-
expected_host = hosts[(i + 1) % len(hosts)]
1491-
self.assertEqual(set(replicas), set([expected_host]))
1492-
14931465

14941466
class KeyspaceAlterMetadata(unittest.TestCase):
14951467
"""

tests/unit/test_metadata.py

Lines changed: 37 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -33,7 +33,7 @@
3333
UserType, KeyspaceMetadata, get_schema_parser,
3434
_UnknownStrategy, ColumnMetadata, TableMetadata,
3535
IndexMetadata, Function, Aggregate,
36-
Metadata)
36+
Metadata, TokenMap)
3737
from cassandra.policies import SimpleConvictionPolicy
3838
from cassandra.pool import Host
3939

@@ -302,6 +302,42 @@ def test_is_valid_name(self):
302302
self.assertEqual(is_valid_name(keyword), False)
303303

304304

305+
class GetReplicasTest(unittest.TestCase):
306+
def _get_replicas(self, token_klass):
307+
tokens = [token_klass(i) for i in range(0, (2 ** 127 - 1), 2 ** 125)]
308+
hosts = [Host("ip%d" % i, SimpleConvictionPolicy) for i in range(len(tokens))]
309+
token_to_primary_replica = dict(zip(tokens, hosts))
310+
keyspace = KeyspaceMetadata("ks", True, "SimpleStrategy", {"replication_factor": "1"})
311+
metadata = Mock(spec=Metadata, keyspaces={'ks': keyspace})
312+
token_map = TokenMap(token_klass, token_to_primary_replica, tokens, metadata)
313+
314+
# tokens match node tokens exactly
315+
for i, token in enumerate(tokens):
316+
expected_host = hosts[(i + 1) % len(hosts)]
317+
replicas = token_map.get_replicas("ks", token)
318+
self.assertEqual(set(replicas), {expected_host})
319+
320+
# shift the tokens back by one
321+
for token, expected_host in zip(tokens, hosts):
322+
replicas = token_map.get_replicas("ks", token_klass(token.value - 1))
323+
self.assertEqual(set(replicas), {expected_host})
324+
325+
# shift the tokens forward by one
326+
for i, token in enumerate(tokens):
327+
replicas = token_map.get_replicas("ks", token_klass(token.value + 1))
328+
expected_host = hosts[(i + 1) % len(hosts)]
329+
self.assertEqual(set(replicas), {expected_host})
330+
331+
def test_murmur3_tokens(self):
332+
self._get_replicas(Murmur3Token)
333+
334+
def test_md5_tokens(self):
335+
self._get_replicas(MD5Token)
336+
337+
def test_bytes_tokens(self):
338+
self._get_replicas(BytesToken)
339+
340+
305341
class Murmur3TokensTest(unittest.TestCase):
306342

307343
def test_murmur3_init(self):

0 commit comments

Comments
 (0)