| @@ -5,6 +5,8 @@ | |||||||
| Bug Fixes | Bug Fixes | ||||||
| --------- | --------- | ||||||
| * Include table indexes in ``KeyspaceMetadata.export_as_string()`` | * Include table indexes in ``KeyspaceMetadata.export_as_string()`` | ||||||
|  | * Fix token-aware routing for tokens that fall before the first node token in | ||||||
|  |   the ring and tokens that exactly match a node's token | ||||||
|  |  | ||||||
| 1.0.0 Final | 1.0.0 Final | ||||||
| =========== | =========== | ||||||
|   | |||||||
| @@ -1,4 +1,4 @@ | |||||||
| from bisect import bisect_left | from bisect import bisect_right | ||||||
| from collections import defaultdict | from collections import defaultdict | ||||||
| try: | try: | ||||||
|     from collections import OrderedDict |     from collections import OrderedDict | ||||||
| @@ -890,7 +890,10 @@ class TokenMap(object): | |||||||
|             if tokens_to_hosts is None: |             if tokens_to_hosts is None: | ||||||
|                 return [] |                 return [] | ||||||
|  |  | ||||||
|         point = bisect_left(self.ring, token) |         # 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) | ||||||
|         if point == len(self.ring): |         if point == len(self.ring): | ||||||
|             return tokens_to_hosts[self.ring[0]] |             return tokens_to_hosts[self.ring[0]] | ||||||
|         else: |         else: | ||||||
|   | |||||||
| @@ -365,8 +365,8 @@ class TestCodeCoverage(unittest.TestCase): | |||||||
|  |  | ||||||
|         for i, token in enumerate(ring): |         for i, token in enumerate(ring): | ||||||
|             self.assertEqual(set(get_replicas('test3rf', token)), set(owners)) |             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('test2rf', token)), set([owners[(i + 1) % 3], owners[(i + 2) % 3]])) | ||||||
|             self.assertEqual(set(get_replicas('test1rf', token)), set([owners[i]])) |             self.assertEqual(set(get_replicas('test1rf', token)), set([owners[(i + 1) % 3]])) | ||||||
|  |  | ||||||
|  |  | ||||||
| class TokenMetadataTest(unittest.TestCase): | class TokenMetadataTest(unittest.TestCase): | ||||||
| @@ -385,7 +385,7 @@ class TokenMetadataTest(unittest.TestCase): | |||||||
|         cluster.shutdown() |         cluster.shutdown() | ||||||
|  |  | ||||||
|     def test_getting_replicas(self): |     def test_getting_replicas(self): | ||||||
|         tokens = [MD5Token(str(i)) for i in range(1, (2 ** 127 - 1), 2 ** 125)] |         tokens = [MD5Token(str(i)) for i in range(0, (2 ** 127 - 1), 2 ** 125)] | ||||||
|         hosts = [Host("ip%d" % i, SimpleConvictionPolicy) for i in range(len(tokens))] |         hosts = [Host("ip%d" % i, SimpleConvictionPolicy) for i in range(len(tokens))] | ||||||
|         token_to_primary_replica = dict(zip(tokens, hosts)) |         token_to_primary_replica = dict(zip(tokens, hosts)) | ||||||
|         keyspace = KeyspaceMetadata("ks", True, "SimpleStrategy", {"replication_factor": "1"}) |         keyspace = KeyspaceMetadata("ks", True, "SimpleStrategy", {"replication_factor": "1"}) | ||||||
| @@ -393,7 +393,8 @@ class TokenMetadataTest(unittest.TestCase): | |||||||
|         token_map = TokenMap(MD5Token, token_to_primary_replica, tokens, metadata) |         token_map = TokenMap(MD5Token, token_to_primary_replica, tokens, metadata) | ||||||
|  |  | ||||||
|         # tokens match node tokens exactly |         # tokens match node tokens exactly | ||||||
|         for token, expected_host in zip(tokens, hosts): |         for i, token in enumerate(tokens): | ||||||
|  |             expected_host = hosts[(i + 1) % len(hosts)] | ||||||
|             replicas = token_map.get_replicas("ks", token) |             replicas = token_map.get_replicas("ks", token) | ||||||
|             self.assertEqual(set(replicas), set([expected_host])) |             self.assertEqual(set(replicas), set([expected_host])) | ||||||
|  |  | ||||||
|   | |||||||
		Reference in New Issue
	
	Block a user
	 Tyler Hobbs
					Tyler Hobbs