void NonReplicatedStrategy::tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const { output->clear(); for (TokenHostMap::const_iterator i = primary.begin(); i != primary.end(); ++i) { CopyOnWriteHostVec token_replicas(new HostVec(1, i->second)); output->insert(std::make_pair(i->first, token_replicas)); } }
void SimpleStrategy::tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const { size_t target_replicas = std::min<size_t>(replication_factor_, primary.size()); output->clear(); for (TokenHostMap::const_iterator i = primary.begin(); i != primary.end(); ++i) { CopyOnWriteHostVec token_replicas(new HostVec()); TokenHostMap::const_iterator j = i; do { token_replicas->push_back(j->second); ++j; if (j == primary.end()) { j = primary.begin(); } } while (token_replicas->size() < target_replicas); output->insert(std::make_pair(i->first, token_replicas)); } }
static DCRackMap racks_in_dcs(const TokenHostMap& token_hosts) { DCRackMap racks; for (TokenHostMap::const_iterator i = token_hosts.begin(); i != token_hosts.end(); ++i) { const std::string& dc = i->second->dc(); const std::string& rack = i->second->rack(); if (!dc.empty() && !rack.empty()) { racks[dc].insert(rack); } } return racks; }
ReplicaSet get_expected_replicas(size_t rf, const std::string& value, const std::string& local_dc = "") { ReplicaSet replicas; TokenHostMap::iterator i = tokens.upper_bound(cass::create_murmur3_hash_from_string(value)); while (replicas.size() < rf) { if (local_dc.empty() || local_dc == i->second.dc) { replicas.insert(i->second.ip); } ++i; if (i == tokens.end()) { i = tokens.begin(); } } return replicas; }
void NetworkTopologyStrategy::tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const { DCRackMap racks = racks_in_dcs(primary); output->clear(); for (TokenHostMap::const_iterator i = primary.begin(); i != primary.end(); ++i) { DCReplicaCountMap replica_counts; std::map<std::string, std::set<std::string> > racks_observed; std::map<std::string, std::list<SharedRefPtr<Host> > > skipped_endpoints; CopyOnWriteHostVec replicas(new HostVec()); TokenHostMap::const_iterator j = i; for (size_t count = 0; count < primary.size() && replica_counts != replication_factors_; ++count) { const SharedRefPtr<Host>& host = j->second; const std::string& dc = host->dc(); ++j; if (j == primary.end()) { j = primary.begin(); } DCReplicaCountMap::const_iterator rf_it = replication_factors_.find(dc); if (dc.empty() || rf_it == replication_factors_.end()) { continue; } const size_t rf = rf_it->second; size_t& replica_count_this_dc = replica_counts[dc] ; if (replica_count_this_dc >= rf) { continue; } const size_t rack_count_this_dc = racks[dc].size(); std::set<std::string>& racks_observed_this_dc = racks_observed[dc]; const std::string& rack = host->rack(); if (rack.empty() || racks_observed_this_dc.size() == rack_count_this_dc) { ++replica_count_this_dc; replicas->push_back(host); } else { if (racks_observed_this_dc.count(rack) > 0) { skipped_endpoints[dc].push_back(host); } else { ++replica_count_this_dc; replicas->push_back(host); racks_observed_this_dc.insert(rack); if (racks_observed_this_dc.size() == rack_count_this_dc) { std::list<SharedRefPtr<Host> >& skipped_endpoints_this_dc = skipped_endpoints[dc]; while (!skipped_endpoints_this_dc.empty() && replica_count_this_dc < rf) { ++replica_count_this_dc; replicas->push_back(skipped_endpoints_this_dc.front()); skipped_endpoints_this_dc.pop_front(); } } } } } output->insert(std::make_pair(i->first, replicas)); } }