Skip to content

Commit a109a7f

Browse files
committed
Fix token-aware routing for tokens before first node
Fixes apache#82
1 parent 59a88ea commit a109a7f

File tree

3 files changed

+12
-6
lines changed

3 files changed

+12
-6
lines changed

CHANGELOG.rst

Lines changed: 2 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -5,6 +5,8 @@
55
Bug Fixes
66
---------
77
* Include table indexes in ``KeyspaceMetadata.export_as_string()``
8+
* Fix token-aware routing for tokens that fall before the first node token in
9+
the ring and tokens that exactly match a node's token
810

911
1.0.0 Final
1012
===========

cassandra/metadata.py

Lines changed: 5 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,4 @@
1-
from bisect import bisect_left
1+
from bisect import bisect_right
22
from collections import defaultdict
33
try:
44
from collections import OrderedDict
@@ -890,7 +890,10 @@ def get_replicas(self, keyspace, token):
890890
if tokens_to_hosts is None:
891891
return []
892892

893-
point = bisect_left(self.ring, token)
893+
# token range ownership is exclusive on the LHS (the start token), so
894+
# we use bisect_right, which, in the case of a tie/exact match,
895+
# picks an insertion point to the right of the existing match
896+
point = bisect_right(self.ring, token)
894897
if point == len(self.ring):
895898
return tokens_to_hosts[self.ring[0]]
896899
else:

tests/integration/standard/test_metadata.py

Lines changed: 5 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -365,8 +365,8 @@ def test_token_map(self):
365365

366366
for i, token in enumerate(ring):
367367
self.assertEqual(set(get_replicas('test3rf', token)), set(owners))
368-
self.assertEqual(set(get_replicas('test2rf', token)), set([owners[i], owners[(i + 1) % 3]]))
369-
self.assertEqual(set(get_replicas('test1rf', token)), set([owners[i]]))
368+
self.assertEqual(set(get_replicas('test2rf', token)), set([owners[(i + 1) % 3], owners[(i + 2) % 3]]))
369+
self.assertEqual(set(get_replicas('test1rf', token)), set([owners[(i + 1) % 3]]))
370370

371371

372372
class TokenMetadataTest(unittest.TestCase):
@@ -385,15 +385,16 @@ def test_token(self):
385385
cluster.shutdown()
386386

387387
def test_getting_replicas(self):
388-
tokens = [MD5Token(str(i)) for i in range(1, (2 ** 127 - 1), 2 ** 125)]
388+
tokens = [MD5Token(str(i)) for i in range(0, (2 ** 127 - 1), 2 ** 125)]
389389
hosts = [Host("ip%d" % i, SimpleConvictionPolicy) for i in range(len(tokens))]
390390
token_to_primary_replica = dict(zip(tokens, hosts))
391391
keyspace = KeyspaceMetadata("ks", True, "SimpleStrategy", {"replication_factor": "1"})
392392
metadata = Mock(spec=Metadata, keyspaces={'ks': keyspace})
393393
token_map = TokenMap(MD5Token, token_to_primary_replica, tokens, metadata)
394394

395395
# tokens match node tokens exactly
396-
for token, expected_host in zip(tokens, hosts):
396+
for i, token in enumerate(tokens):
397+
expected_host = hosts[(i + 1) % len(hosts)]
397398
replicas = token_map.get_replicas("ks", token)
398399
self.assertEqual(set(replicas), set([expected_host]))
399400

0 commit comments

Comments
 (0)