Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
7 changes: 7 additions & 0 deletions CHANGELOG.rst
Original file line number Diff line number Diff line change
@@ -1,5 +1,12 @@
3.15.0
======
Bug Fixes
---------
* Tokenmap.get_replicas returns the wrong value if token coincides with the end of the range (PYTHON-978)

3.14.0
======
April 17, 2018

Features
--------
Expand Down
9 changes: 4 additions & 5 deletions cassandra/metadata.py
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@
# limitations under the License.

from binascii import unhexlify
from bisect import bisect_right
from bisect import bisect_left
from collections import defaultdict, Mapping
from functools import total_ordering
from hashlib import md5
Expand Down Expand Up @@ -1486,10 +1486,9 @@ def get_replicas(self, keyspace, token):
tokens_to_hosts = self.tokens_to_hosts_by_ks.get(keyspace, None)

if tokens_to_hosts:
# token range ownership is exclusive on the LHS (the start token), so
# we use bisect_right, which, in the case of a tie/exact match,
# picks an insertion point to the right of the existing match
point = bisect_right(self.ring, token)
# The values in self.ring correspond to the end of the
# token range up to and including the value listed.
point = bisect_left(self.ring, token)
if point == len(self.ring):
return tokens_to_hosts[self.ring[0]]
else:
Expand Down
4 changes: 2 additions & 2 deletions tests/integration/standard/test_metadata.py
Original file line number Diff line number Diff line change
Expand Up @@ -1172,8 +1172,8 @@ def test_token_map(self):

for i, token in enumerate(ring):
self.assertEqual(set(get_replicas('test3rf', token)), set(owners))
self.assertEqual(set(get_replicas('test2rf', token)), set([owners[(i + 1) % 3], owners[(i + 2) % 3]]))
self.assertEqual(set(get_replicas('test1rf', token)), set([owners[(i + 1) % 3]]))
self.assertEqual(set(get_replicas('test2rf', token)), set([owners[i], owners[(i + 1) % 3]]))
self.assertEqual(set(get_replicas('test1rf', token)), set([owners[i]]))
cluster.shutdown()


Expand Down
3 changes: 1 addition & 2 deletions tests/unit/test_metadata.py
Original file line number Diff line number Diff line change
Expand Up @@ -312,8 +312,7 @@ def _get_replicas(self, token_klass):
token_map = TokenMap(token_klass, token_to_primary_replica, tokens, metadata)

# tokens match node tokens exactly
for i, token in enumerate(tokens):
expected_host = hosts[(i + 1) % len(hosts)]
for token, expected_host in zip(tokens, hosts):
replicas = token_map.get_replicas("ks", token)
self.assertEqual(set(replicas), {expected_host})

Expand Down