storage_service: Implement leave_ring

Needed by unbootstrap.
This commit is contained in:
Asias He
2015-10-21 09:18:00 +08:00
parent 2f86feb581
commit d1eaccd234
2 changed files with 15 additions and 14 deletions

View File

@@ -1912,4 +1912,18 @@ void storage_service::send_replication_notification(inet_address remote) {
#endif
}
// Runs inside seastar::async context
void storage_service::leave_ring() {
db::system_keyspace::set_bootstrap_state(db::system_keyspace::bootstrap_state::NEEDS_BOOTSTRAP).get();
_token_metadata.remove_endpoint(get_broadcast_address());
get_local_pending_range_calculator_service().update().get();
auto& gossiper = gms::get_local_gossiper();
auto expire_time = gossiper.compute_expire_time().time_since_epoch().count();
gossiper.add_local_application_state(gms::application_state::STATUS, value_factory.left(get_local_tokens(), expire_time));
auto delay = std::max(std::chrono::milliseconds(RING_DELAY), gms::gossiper::INTERVAL);
logger.info("Announcing that I have left the ring for {}ms", delay.count());
sleep(delay).get();
}
} // namespace service

View File

@@ -1887,23 +1887,10 @@ public:
future<> decommission();
#if 0
private void leaveRing()
{
SystemKeyspace.setBootstrapState(SystemKeyspace.BootstrapState.NEEDS_BOOTSTRAP);
_token_metadata.removeEndpoint(FBUtilities.getBroadcastAddress());
PendingRangeCalculatorService.instance.update();
Gossiper.instance.addLocalApplicationState(ApplicationState.STATUS, valueFactory.left(getLocalTokens(),Gossiper.computeExpireTime()));
int delay = Math.max(RING_DELAY, Gossiper.intervalInMillis * 2);
logger.info("Announcing that I have left the ring for {}ms", delay);
Uninterruptibles.sleepUninterruptibly(delay, TimeUnit.MILLISECONDS);
}
#endif
private:
void leave_ring();
future<> unbootstrap();
#if 0
private Future<StreamState> streamHints()
{
// StreamPlan will not fail if there are zero files to transfer, so flush anyway (need to get any in-memory hints, as well)