// create a ring or 10 nodes
createInitialRing(ss, partitioner, endPointTokens, keyTokens, hosts, 10);
// nodes 6, 8 and 9 leave
ss.onChange(hosts.get(6), StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_LEAVING + StorageService.Delimiter + partitioner.getTokenFactory().toString(endPointTokens.get(6))));
ss.onChange(hosts.get(8), StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_LEAVING + StorageService.Delimiter + partitioner.getTokenFactory().toString(endPointTokens.get(8))));
ss.onChange(hosts.get(9), StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_LEAVING + StorageService.Delimiter + partitioner.getTokenFactory().toString(endPointTokens.get(9))));
// boot two new nodes with keyTokens.get(5) and keyTokens.get(7)
InetAddress boot1 = InetAddress.getByName("127.0.1.1");
ss.onChange(boot1, StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_BOOTSTRAPPING + StorageService.Delimiter + partitioner.getTokenFactory().toString(keyTokens.get(5))));
InetAddress boot2 = InetAddress.getByName("127.0.1.2");
ss.onChange(boot2, StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_BOOTSTRAPPING + StorageService.Delimiter + partitioner.getTokenFactory().toString(keyTokens.get(7))));
Collection<InetAddress> endPoints = null;
// tokens 5, 15 and 25 should go three nodes
for (int i=0; i<3; ++i)
{
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(i), testStrategy.getNaturalEndpoints(keyTokens.get(i)));
assertTrue(endPoints.size() == 3);
assertTrue(endPoints.contains(hosts.get(i+1)));
assertTrue(endPoints.contains(hosts.get(i+2)));
assertTrue(endPoints.contains(hosts.get(i+3)));
}
// token 35 should go to nodes 4, 5, 6, 7 and boot1
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(3), testStrategy.getNaturalEndpoints(keyTokens.get(3)));
assertTrue(endPoints.size() == 5);
assertTrue(endPoints.contains(hosts.get(4)));
assertTrue(endPoints.contains(hosts.get(5)));
assertTrue(endPoints.contains(hosts.get(6)));
assertTrue(endPoints.contains(hosts.get(7)));
assertTrue(endPoints.contains(boot1));
// token 45 should go to nodes 5, 6, 7, 0, boot1 and boot2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(4), testStrategy.getNaturalEndpoints(keyTokens.get(4)));
assertTrue(endPoints.size() == 6);
assertTrue(endPoints.contains(hosts.get(5)));
assertTrue(endPoints.contains(hosts.get(6)));
assertTrue(endPoints.contains(hosts.get(7)));
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(boot1));
assertTrue(endPoints.contains(boot2));
// token 55 should go to nodes 6, 7, 8, 0, 1, boot1 and boot2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(5), testStrategy.getNaturalEndpoints(keyTokens.get(5)));
assertTrue(endPoints.size() == 7);
assertTrue(endPoints.contains(hosts.get(6)));
assertTrue(endPoints.contains(hosts.get(7)));
assertTrue(endPoints.contains(hosts.get(8)));
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(hosts.get(1)));
assertTrue(endPoints.contains(boot1));
assertTrue(endPoints.contains(boot2));
// token 65 should go to nodes 7, 8, 9, 0, 1 and boot2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(6), testStrategy.getNaturalEndpoints(keyTokens.get(6)));
assertTrue(endPoints.size() == 6);
assertTrue(endPoints.contains(hosts.get(7)));
assertTrue(endPoints.contains(hosts.get(8)));
assertTrue(endPoints.contains(hosts.get(9)));
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(hosts.get(1)));
assertTrue(endPoints.contains(boot2));
// token 75 should to go nodes 8, 9, 0, 1, 2 and boot2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(7), testStrategy.getNaturalEndpoints(keyTokens.get(7)));
assertTrue(endPoints.size() == 6);
assertTrue(endPoints.contains(hosts.get(8)));
assertTrue(endPoints.contains(hosts.get(9)));
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(hosts.get(1)));
assertTrue(endPoints.contains(hosts.get(2)));
assertTrue(endPoints.contains(boot2));
// token 85 should go to nodes 9, 0, 1 and 2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(8), testStrategy.getNaturalEndpoints(keyTokens.get(8)));
assertTrue(endPoints.size() == 4);
assertTrue(endPoints.contains(hosts.get(9)));
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(hosts.get(1)));
assertTrue(endPoints.contains(hosts.get(2)));
// token 95 should go to nodes 0, 1 and 2
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(9), testStrategy.getNaturalEndpoints(keyTokens.get(9)));
assertTrue(endPoints.size() == 3);
assertTrue(endPoints.contains(hosts.get(0)));
assertTrue(endPoints.contains(hosts.get(1)));
assertTrue(endPoints.contains(hosts.get(2)));
// Now finish node 6 and node 9 leaving, as well as boot1 (after this node 8 is still
// leaving and boot2 in progress
ss.onChange(hosts.get(6), StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_LEFT + StorageService.Delimiter + StorageService.LEFT_NORMALLY + StorageService.Delimiter + partitioner.getTokenFactory().toString(endPointTokens.get(6))));
ss.onChange(hosts.get(9), StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_LEFT + StorageService.Delimiter + StorageService.LEFT_NORMALLY + StorageService.Delimiter + partitioner.getTokenFactory().toString(endPointTokens.get(9))));
ss.onChange(boot1, StorageService.MOVE_STATE, new ApplicationState(StorageService.STATE_NORMAL + StorageService.Delimiter + partitioner.getTokenFactory().toString(keyTokens.get(5))));
// tokens 5, 15 and 25 should go three nodes
for (int i=0; i<3; ++i)
{
endPoints = testStrategy.getWriteEndpoints(keyTokens.get(i), testStrategy.getNaturalEndpoints(keyTokens.get(i)));