Skip to content

Commit 9db1d6d

Browse files
mpenickkfaltas
authored andcommitted
CPP-917 Fix infinite loop in token map calculation when using SimpleStrategy
1 parent d80ab5a commit 9db1d6d

File tree

2 files changed

+25
-3
lines changed

2 files changed

+25
-3
lines changed

src/token_map_impl.hpp

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -516,19 +516,20 @@ void ReplicationStrategy<Partitioner>::build_replicas_simple(const TokenHostVec&
516516
if (it == replication_factors_.end()) {
517517
return;
518518
}
519-
size_t num_replicas = std::min<size_t>(it->second.count, tokens.size());
519+
const size_t num_tokens = tokens.size();
520+
const size_t num_replicas = std::min<size_t>(it->second.count, num_tokens);
520521
for (typename TokenHostVec::const_iterator i = tokens.begin(), end = tokens.end(); i != end;
521522
++i) {
522523
CopyOnWriteHostVec replicas(new HostVec());
523524
replicas->reserve(num_replicas);
524525
typename TokenHostVec::const_iterator token_it = i;
525-
do {
526+
for (size_t j = 0; j < num_tokens && replicas->size() < num_replicas; ++j) {
526527
add_replica(replicas, Host::Ptr(Host::Ptr(token_it->second)));
527528
++token_it;
528529
if (token_it == tokens.end()) {
529530
token_it = tokens.begin();
530531
}
531-
} while (replicas->size() < num_replicas);
532+
}
532533
result.push_back(TokenReplicas(i->first, replicas));
533534
}
534535
}

tests/src/unit/tests/test_replication_strategy.cpp

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -185,6 +185,27 @@ TEST(ReplicationStrategyUnitTest, Simple) {
185185
}
186186
}
187187

188+
TEST(ReplicationStrategyUnitTest, SimpleNumHostsLessThanReplicationFactor) {
189+
MockTokenMap<Murmur3Partitioner> token_map;
190+
191+
token_map.init_simple_strategy(3);
192+
193+
MockTokenMap<Murmur3Partitioner>::Token t1 = 0;
194+
195+
// To reproduce the issue the number of tokens needs to be greater than
196+
// (or equal to) the RF because the RF is bounded by the number of tokens.
197+
token_map.add_token(t1, "1.0.0.1");
198+
token_map.add_token(100, "1.0.0.1");
199+
token_map.add_token(200, "1.0.0.1");
200+
token_map.add_token(300, "1.0.0.1");
201+
202+
token_map.build_replicas();
203+
204+
const CopyOnWriteHostVec& hosts = token_map.find_hosts(t1);
205+
ASSERT_TRUE(hosts && hosts->size() == 1);
206+
check_host((*hosts)[0], "1.0.0.1");
207+
}
208+
188209
TEST(ReplicationStrategyUnitTest, NetworkTopology) {
189210
MockTokenMap<Murmur3Partitioner> token_map;
190211

0 commit comments

Comments
 (0)