garage/src/rpc/membership.rs

707 lines
19 KiB
Rust
Raw Normal View History

2021-04-06 03:25:28 +00:00
//! Module containing structs related to membership management
use std::collections::HashMap;
use std::fmt::Write as FmtWrite;
2021-02-23 17:46:25 +00:00
use std::io::{Read, Write};
use std::net::{IpAddr, SocketAddr};
2021-04-09 00:32:42 +00:00
use std::path::{Path, PathBuf};
use std::sync::atomic::{AtomicUsize, Ordering};
use std::sync::Arc;
2020-04-06 17:55:39 +00:00
use std::time::Duration;
2020-04-06 19:02:15 +00:00
use futures::future::join_all;
use futures::select;
use futures_util::future::*;
2020-04-18 17:21:34 +00:00
use serde::{Deserialize, Serialize};
use tokio::sync::watch;
use tokio::sync::Mutex;
2020-04-06 17:55:39 +00:00
2020-04-24 10:10:01 +00:00
use garage_util::background::BackgroundRunner;
use garage_util::data::*;
use garage_util::error::Error;
use garage_util::persister::Persister;
2021-03-15 15:21:41 +00:00
use garage_util::time::*;
2020-04-23 17:05:46 +00:00
use crate::consul::get_consul_nodes;
2021-02-23 17:46:25 +00:00
use crate::ring::*;
2020-04-24 10:10:01 +00:00
use crate::rpc_client::*;
use crate::rpc_server::*;
2020-04-06 17:55:39 +00:00
const PING_INTERVAL: Duration = Duration::from_secs(10);
const DISCOVERY_INTERVAL: Duration = Duration::from_secs(60);
2020-04-06 17:55:39 +00:00
const PING_TIMEOUT: Duration = Duration::from_secs(2);
const MAX_FAILURES_BEFORE_CONSIDERED_DOWN: usize = 5;
2020-04-06 17:55:39 +00:00
2021-03-21 23:00:09 +00:00
/// RPC endpoint used for calls related to membership
pub const MEMBERSHIP_RPC_PATH: &str = "_membership";
2021-03-21 23:00:09 +00:00
/// RPC messages related to membership
2020-04-18 17:21:34 +00:00
#[derive(Debug, Serialize, Deserialize)]
pub enum Message {
2021-03-21 23:00:09 +00:00
/// Response to successfull advertisements
2020-04-18 17:21:34 +00:00
Ok,
2021-03-21 23:00:09 +00:00
/// Message sent to detect other nodes status
2020-04-18 17:21:34 +00:00
Ping(PingMessage),
2021-03-21 23:00:09 +00:00
/// Ask other node for the nodes it knows. Answered with AdvertiseNodesUp
2020-04-18 17:21:34 +00:00
PullStatus,
2021-03-21 23:00:09 +00:00
/// Ask other node its config. Answered with AdvertiseConfig
2020-04-18 17:21:34 +00:00
PullConfig,
2021-03-21 23:00:09 +00:00
/// Advertisement of nodes the host knows up. Sent spontanously or in response to PullStatus
2020-04-18 17:21:34 +00:00
AdvertiseNodesUp(Vec<AdvertisedNode>),
2021-03-21 23:00:09 +00:00
/// Advertisement of nodes config. Sent spontanously or in response to PullConfig
2020-04-18 17:21:34 +00:00
AdvertiseConfig(NetworkConfig),
}
impl RpcMessage for Message {}
2021-03-21 23:00:09 +00:00
/// A ping, containing informations about status and config
#[derive(Debug, Serialize, Deserialize)]
pub struct PingMessage {
2021-05-02 21:13:08 +00:00
id: Uuid,
2020-12-12 16:58:19 +00:00
rpc_port: u16,
2020-12-12 16:58:19 +00:00
status_hash: Hash,
config_version: u64,
2020-04-19 17:08:48 +00:00
2020-12-12 16:58:19 +00:00
state_info: StateInfo,
}
2021-03-21 23:00:09 +00:00
/// A node advertisement
#[derive(Clone, Debug, Serialize, Deserialize)]
pub struct AdvertisedNode {
2021-03-21 23:00:09 +00:00
/// Id of the node this advertisement relates to
2021-05-02 21:13:08 +00:00
pub id: Uuid,
2021-03-21 23:00:09 +00:00
/// IP and port of the node
pub addr: SocketAddr,
2021-03-21 23:00:09 +00:00
/// Is the node considered up
pub is_up: bool,
2021-03-21 23:00:09 +00:00
/// When was the node last seen up, in milliseconds since UNIX epoch
pub last_seen: u64,
2020-04-19 17:08:48 +00:00
pub state_info: StateInfo,
}
2021-03-21 23:00:09 +00:00
/// This node's membership manager
2020-04-06 17:55:39 +00:00
pub struct System {
2021-03-21 23:00:09 +00:00
/// The id of this node
2021-05-02 21:13:08 +00:00
pub id: Uuid,
2020-04-06 17:55:39 +00:00
persist_config: Persister<NetworkConfig>,
persist_status: Persister<Vec<AdvertisedNode>>,
2020-12-12 16:58:19 +00:00
rpc_local_port: u16,
state_info: StateInfo,
2020-04-19 17:08:48 +00:00
2020-12-12 16:58:19 +00:00
rpc_http_client: Arc<RpcHttpClient>,
2020-04-18 17:21:34 +00:00
rpc_client: Arc<RpcClient<Message>>,
2020-04-06 17:55:39 +00:00
replication_factor: usize,
2021-02-21 12:11:10 +00:00
pub(crate) status: watch::Receiver<Arc<Status>>,
2021-04-06 03:25:28 +00:00
/// The ring
pub ring: watch::Receiver<Arc<Ring>>,
update_lock: Mutex<Updaters>,
2021-03-21 23:00:09 +00:00
/// The job runner of this node
pub background: Arc<BackgroundRunner>,
2020-04-06 17:55:39 +00:00
}
struct Updaters {
update_status: watch::Sender<Arc<Status>>,
update_ring: watch::Sender<Arc<Ring>>,
}
2021-03-21 23:00:09 +00:00
/// The status of each nodes, viewed by this node
#[derive(Debug, Clone)]
pub struct Status {
2021-03-21 23:00:09 +00:00
/// Mapping of each node id to its known status
2021-05-02 21:13:08 +00:00
pub nodes: HashMap<Uuid, Arc<StatusEntry>>,
2021-04-06 03:25:28 +00:00
/// Hash of `nodes`, used to detect when nodes have different views of the cluster
pub hash: Hash,
2020-04-07 15:00:48 +00:00
}
2021-03-21 23:00:09 +00:00
/// The status of a single node
#[derive(Debug)]
2020-04-19 17:08:48 +00:00
pub struct StatusEntry {
2021-03-21 23:00:09 +00:00
/// The IP and port used to connect to this node
2020-04-07 15:00:48 +00:00
pub addr: SocketAddr,
2021-03-21 23:00:09 +00:00
/// Last time this node was seen
pub last_seen: u64,
2021-03-21 23:00:09 +00:00
/// Number of consecutive pings sent without reply to this node
pub num_failures: AtomicUsize,
2020-04-19 17:08:48 +00:00
pub state_info: StateInfo,
}
impl StatusEntry {
2021-03-21 23:00:09 +00:00
/// is the node associated to this entry considered up
pub fn is_up(&self) -> bool {
self.num_failures.load(Ordering::SeqCst) < MAX_FAILURES_BEFORE_CONSIDERED_DOWN
}
}
2020-04-19 17:08:48 +00:00
#[derive(Debug, Clone, Serialize, Deserialize)]
pub struct StateInfo {
pub hostname: String,
2020-04-07 15:00:48 +00:00
}
impl Status {
2020-04-06 20:27:51 +00:00
fn handle_ping(&mut self, ip: IpAddr, info: &PingMessage) -> bool {
2020-04-06 20:54:03 +00:00
let addr = SocketAddr::new(ip, info.rpc_port);
let old_status = self.nodes.insert(
info.id,
Arc::new(StatusEntry {
addr,
last_seen: now_msec(),
num_failures: AtomicUsize::from(0),
2020-04-19 17:08:48 +00:00
state_info: info.state_info.clone(),
}),
);
2020-04-06 20:54:03 +00:00
match old_status {
None => {
2020-04-21 12:54:55 +00:00
info!("Newly pingable node: {}", hex::encode(&info.id));
2020-04-06 20:54:03 +00:00
true
}
Some(x) => x.addr != addr,
}
2020-04-06 20:27:51 +00:00
}
fn recalculate_hash(&mut self) {
let mut nodes = self.nodes.iter().collect::<Vec<_>>();
nodes.sort_unstable_by_key(|(id, _status)| *id);
2020-04-06 20:27:51 +00:00
let mut nodes_txt = String::new();
2020-04-21 12:54:55 +00:00
debug!("Current set of pingable nodes: --");
2020-04-06 20:27:51 +00:00
for (id, status) in nodes {
2020-04-21 12:54:55 +00:00
debug!("{} {}", hex::encode(&id), status.addr);
writeln!(&mut nodes_txt, "{} {}", hex::encode(&id), status.addr).unwrap();
2020-04-06 19:02:15 +00:00
}
2020-04-21 12:54:55 +00:00
debug!("END --");
self.hash = blake2sum(nodes_txt.as_bytes());
2020-04-06 19:02:15 +00:00
}
fn to_serializable_membership(&self, system: &System) -> Vec<AdvertisedNode> {
let mut mem = vec![];
for (node, status) in self.nodes.iter() {
let state_info = if *node == system.id {
system.state_info.clone()
} else {
status.state_info.clone()
};
mem.push(AdvertisedNode {
id: *node,
addr: status.addr,
is_up: status.is_up(),
last_seen: status.last_seen,
state_info,
});
}
mem
}
}
2020-04-06 19:02:15 +00:00
2021-05-02 21:13:08 +00:00
fn gen_node_id(metadata_dir: &Path) -> Result<Uuid, Error> {
2021-04-09 00:32:42 +00:00
let mut id_file = metadata_dir.to_path_buf();
2020-04-23 17:05:46 +00:00
id_file.push("node_id");
if id_file.as_path().exists() {
let mut f = std::fs::File::open(id_file.as_path())?;
let mut d = vec![];
f.read_to_end(&mut d)?;
if d.len() != 32 {
2021-04-09 00:32:42 +00:00
return Err(Error::Message("Corrupt node_id file".to_string()));
2020-04-23 17:05:46 +00:00
}
let mut id = [0u8; 32];
id.copy_from_slice(&d[..]);
Ok(id.into())
} else {
let id = gen_uuid();
let mut f = std::fs::File::create(id_file.as_path())?;
f.write_all(id.as_slice())?;
Ok(id)
}
}
2020-04-06 17:55:39 +00:00
impl System {
2021-03-21 23:00:09 +00:00
/// Create this node's membership manager
2020-04-18 17:21:34 +00:00
pub fn new(
2020-12-12 16:58:19 +00:00
metadata_dir: PathBuf,
2020-04-23 17:05:46 +00:00
rpc_http_client: Arc<RpcHttpClient>,
2020-04-18 17:21:34 +00:00
background: Arc<BackgroundRunner>,
rpc_server: &mut RpcServer,
replication_factor: usize,
2020-04-18 17:21:34 +00:00
) -> Arc<Self> {
2020-12-12 16:58:19 +00:00
let id = gen_node_id(&metadata_dir).expect("Unable to read or generate node ID");
2020-04-23 17:05:46 +00:00
info!("Node ID: {}", hex::encode(&id));
let persist_config = Persister::new(&metadata_dir, "network_config");
let persist_status = Persister::new(&metadata_dir, "peer_info");
let net_config = match persist_config.load() {
Ok(x) => x,
Err(e) => {
match Persister::<garage_rpc_021::ring::NetworkConfig>::new(
&metadata_dir,
"network_config",
)
.load()
{
Ok(old_config) => NetworkConfig::migrate_from_021(old_config),
Err(e2) => {
info!(
"No valid previous network configuration stored ({}, {}), starting fresh.",
e, e2
);
NetworkConfig::new()
}
}
}
};
let mut status = Status {
nodes: HashMap::new(),
hash: Hash::default(),
};
status.recalculate_hash();
let (update_status, status) = watch::channel(Arc::new(status));
2020-04-19 17:08:48 +00:00
let state_info = StateInfo {
hostname: gethostname::gethostname()
.into_string()
2021-04-09 00:32:42 +00:00
.unwrap_or_else(|_| "<invalid utf-8>".to_string()),
2020-04-19 17:08:48 +00:00
};
let ring = Ring::new(net_config, replication_factor);
let (update_ring, ring) = watch::channel(Arc::new(ring));
let rpc_path = MEMBERSHIP_RPC_PATH.to_string();
2020-04-18 17:21:34 +00:00
let rpc_client = RpcClient::new(
RpcAddrClient::<Message>::new(rpc_http_client.clone(), rpc_path.clone()),
2020-04-18 17:21:34 +00:00
background.clone(),
status.clone(),
);
2020-04-12 13:51:19 +00:00
2020-04-18 17:21:34 +00:00
let sys = Arc::new(System {
id,
persist_config,
persist_status,
2020-04-23 17:05:46 +00:00
rpc_local_port: rpc_server.bind_addr.port(),
2020-04-19 17:08:48 +00:00
state_info,
2020-04-18 17:21:34 +00:00
rpc_http_client,
2020-04-12 13:51:19 +00:00
rpc_client,
replication_factor,
status,
ring,
update_lock: Mutex::new(Updaters {
update_status,
update_ring,
}),
background,
2020-04-18 17:21:34 +00:00
});
sys.clone().register_handler(rpc_server, rpc_path);
2020-04-18 17:21:34 +00:00
sys
}
fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) {
rpc_server.add_handler::<Message, _, _>(path, move |msg, addr| {
let self2 = self.clone();
async move {
match msg {
Message::Ping(ping) => self2.handle_ping(&addr, &ping).await,
2021-04-09 00:32:42 +00:00
Message::PullStatus => Ok(self2.handle_pull_status()),
Message::PullConfig => Ok(self2.handle_pull_config()),
2020-04-18 17:21:34 +00:00
Message::AdvertiseNodesUp(adv) => self2.handle_advertise_nodes_up(&adv).await,
Message::AdvertiseConfig(adv) => self2.handle_advertise_config(&adv).await,
2021-05-02 21:13:08 +00:00
_ => Err(Error::BadRpc("Unexpected RPC message".to_string())),
2020-04-18 17:21:34 +00:00
}
}
});
}
2021-03-21 23:00:09 +00:00
/// Get an RPC client
2020-04-18 17:21:34 +00:00
pub fn rpc_client<M: RpcMessage + 'static>(self: &Arc<Self>, path: &str) -> Arc<RpcClient<M>> {
RpcClient::new(
RpcAddrClient::new(self.rpc_http_client.clone(), path.to_string()),
self.background.clone(),
self.status.clone(),
)
2020-04-06 17:55:39 +00:00
}
2021-03-21 23:00:09 +00:00
/// Save network configuration to disc
async fn save_network_config(self: Arc<Self>) -> Result<(), Error> {
let ring = self.ring.borrow().clone();
self.persist_config
.save_async(&ring.config)
.await
.expect("Cannot save current cluster configuration");
Ok(())
}
2020-12-12 16:58:19 +00:00
fn make_ping(&self) -> Message {
let status = self.status.borrow().clone();
let ring = self.ring.borrow().clone();
Message::Ping(PingMessage {
id: self.id,
2020-04-23 17:05:46 +00:00
rpc_port: self.rpc_local_port,
status_hash: status.hash,
config_version: ring.config.version,
2020-04-19 17:08:48 +00:00
state_info: self.state_info.clone(),
2020-04-06 19:02:15 +00:00
})
}
2020-12-12 16:58:19 +00:00
async fn broadcast(self: Arc<Self>, msg: Message, timeout: Duration) {
let status = self.status.borrow().clone();
let to = status
.nodes
.keys()
.filter(|x| **x != self.id)
.cloned()
.collect::<Vec<_>>();
2020-04-18 17:21:34 +00:00
self.rpc_client.call_many(&to[..], msg, timeout).await;
2020-04-06 17:55:39 +00:00
}
2021-03-21 23:00:09 +00:00
/// Perform bootstraping, starting the ping loop
pub async fn bootstrap(
self: Arc<Self>,
peers: Vec<SocketAddr>,
consul_host: Option<String>,
consul_service_name: Option<String>,
) {
let self2 = self.clone();
self.background
2021-04-09 00:32:42 +00:00
.spawn_worker("discovery loop".to_string(), |stop_signal| {
2021-04-05 21:04:08 +00:00
self2.discovery_loop(peers, consul_host, consul_service_name, stop_signal)
});
2020-04-06 22:00:43 +00:00
let self2 = self.clone();
self.background
2021-04-09 00:32:42 +00:00
.spawn_worker("ping loop".to_string(), |stop_signal| {
self2.ping_loop(stop_signal)
});
2020-04-06 22:00:43 +00:00
}
2021-05-02 21:13:08 +00:00
async fn ping_nodes(self: Arc<Self>, peers: Vec<(SocketAddr, Option<Uuid>)>) {
let ping_msg = self.make_ping();
let ping_resps = join_all(peers.iter().map(|(addr, id_option)| {
let sys = self.clone();
let ping_msg_ref = &ping_msg;
async move {
(
id_option,
addr,
2020-04-18 17:21:34 +00:00
sys.rpc_client
2021-04-06 03:25:28 +00:00
.by_addr()
2020-04-18 17:21:34 +00:00
.call(&addr, ping_msg_ref, PING_TIMEOUT)
.await,
)
}
}))
.await;
let update_locked = self.update_lock.lock().await;
let mut status: Status = self.status.borrow().as_ref().clone();
let ring = self.ring.borrow().clone();
2020-04-06 22:00:43 +00:00
let mut has_changes = false;
let mut to_advertise = vec![];
for (id_option, addr, ping_resp) in ping_resps {
if let Ok(Ok(Message::Ping(info))) = ping_resp {
let is_new = status.handle_ping(addr.ip(), &info);
2020-04-06 22:00:43 +00:00
if is_new {
has_changes = true;
to_advertise.push(AdvertisedNode {
id: info.id,
addr: *addr,
is_up: true,
last_seen: now_msec(),
2020-04-19 17:08:48 +00:00
state_info: info.state_info.clone(),
2020-04-06 22:00:43 +00:00
});
}
if is_new || status.hash != info.status_hash {
self.background
.spawn_cancellable(self.clone().pull_status(info.id).map(Ok));
2020-04-06 22:00:43 +00:00
}
if is_new || ring.config.version < info.config_version {
self.background
.spawn_cancellable(self.clone().pull_config(info.id).map(Ok));
2020-04-06 22:00:43 +00:00
}
} else if let Some(id) = id_option {
if let Some(st) = status.nodes.get_mut(id) {
// we need to increment failure counter as call was done using by_addr so the
// counter was not auto-incremented
st.num_failures.fetch_add(1, Ordering::SeqCst);
if !st.is_up() {
warn!("Node {:?} seems to be down.", id);
if !ring.config.members.contains_key(id) {
info!("Removing node {:?} from status (not in config and not responding to pings anymore)", id);
status.nodes.remove(&id);
has_changes = true;
}
2020-04-06 22:00:43 +00:00
}
}
2020-04-06 19:02:15 +00:00
}
}
2020-04-06 22:00:43 +00:00
if has_changes {
status.recalculate_hash();
2020-04-06 22:00:43 +00:00
}
2021-04-05 18:26:01 +00:00
self.update_status(&update_locked, status).await;
drop(update_locked);
2020-04-06 17:55:39 +00:00
2021-04-09 00:32:42 +00:00
if !to_advertise.is_empty() {
self.broadcast(Message::AdvertiseNodesUp(to_advertise), PING_TIMEOUT)
.await;
2020-04-06 22:00:43 +00:00
}
2020-04-06 19:02:15 +00:00
}
2020-12-12 16:58:19 +00:00
async fn handle_ping(
self: Arc<Self>,
from: &SocketAddr,
ping: &PingMessage,
) -> Result<Message, Error> {
let update_locked = self.update_lock.lock().await;
let mut status: Status = self.status.borrow().as_ref().clone();
let is_new = status.handle_ping(from.ip(), ping);
2020-04-06 20:54:03 +00:00
if is_new {
status.recalculate_hash();
2020-04-06 20:54:03 +00:00
}
let status_hash = status.hash;
let config_version = self.ring.borrow().config.version;
2021-04-05 18:26:01 +00:00
self.update_status(&update_locked, status).await;
drop(update_locked);
2020-04-06 19:02:15 +00:00
2020-04-06 20:27:51 +00:00
if is_new || status_hash != ping.status_hash {
self.background
.spawn_cancellable(self.clone().pull_status(ping.id).map(Ok));
2020-04-06 20:27:51 +00:00
}
if is_new || config_version < ping.config_version {
self.background
.spawn_cancellable(self.clone().pull_config(ping.id).map(Ok));
2020-04-06 20:27:51 +00:00
}
Ok(self.make_ping())
2020-04-06 19:02:15 +00:00
}
2021-04-09 00:32:42 +00:00
fn handle_pull_status(&self) -> Message {
Message::AdvertiseNodesUp(self.status.borrow().to_serializable_membership(self))
2020-04-06 20:27:51 +00:00
}
2021-04-09 00:32:42 +00:00
fn handle_pull_config(&self) -> Message {
let ring = self.ring.borrow().clone();
2021-04-09 00:32:42 +00:00
Message::AdvertiseConfig(ring.config.clone())
2020-04-06 20:27:51 +00:00
}
2020-12-12 16:58:19 +00:00
async fn handle_advertise_nodes_up(
self: Arc<Self>,
adv: &[AdvertisedNode],
) -> Result<Message, Error> {
2020-04-06 22:00:43 +00:00
let mut to_ping = vec![];
2020-04-06 20:27:51 +00:00
let update_lock = self.update_lock.lock().await;
let mut status: Status = self.status.borrow().as_ref().clone();
2020-04-06 22:00:43 +00:00
let mut has_changed = false;
2020-04-06 20:27:51 +00:00
for node in adv.iter() {
2020-04-06 22:00:43 +00:00
if node.id == self.id {
// learn our own ip address
2020-04-23 17:05:46 +00:00
let self_addr = SocketAddr::new(node.addr.ip(), self.rpc_local_port);
let old_self = status.nodes.insert(
node.id,
Arc::new(StatusEntry {
2020-04-06 22:00:43 +00:00
addr: self_addr,
last_seen: now_msec(),
num_failures: AtomicUsize::from(0),
2020-04-19 17:08:48 +00:00
state_info: self.state_info.clone(),
}),
);
2020-04-06 22:00:43 +00:00
has_changed = match old_self {
None => true,
Some(x) => x.addr != self_addr,
};
} else {
let ping_them = match status.nodes.get(&node.id) {
// Case 1: new node
None => true,
// Case 2: the node might have changed address
Some(our_node) => node.is_up && !our_node.is_up() && our_node.addr != node.addr,
};
if ping_them {
to_ping.push((node.addr, Some(node.id)));
}
2020-04-06 20:27:51 +00:00
}
}
2020-04-06 22:00:43 +00:00
if has_changed {
status.recalculate_hash();
2020-04-06 22:00:43 +00:00
}
2021-04-05 18:26:01 +00:00
self.update_status(&update_lock, status).await;
drop(update_lock);
2020-04-06 22:00:43 +00:00
2021-04-09 00:32:42 +00:00
if !to_ping.is_empty() {
self.background
.spawn_cancellable(self.clone().ping_nodes(to_ping).map(Ok));
2020-04-06 20:27:51 +00:00
}
Ok(Message::Ok)
}
2020-12-12 16:58:19 +00:00
async fn handle_advertise_config(
self: Arc<Self>,
adv: &NetworkConfig,
) -> Result<Message, Error> {
let update_lock = self.update_lock.lock().await;
2021-02-21 12:11:10 +00:00
let ring: Arc<Ring> = self.ring.borrow().clone();
if adv.version > ring.config.version {
let ring = Ring::new(adv.clone(), self.replication_factor);
update_lock.update_ring.send(Arc::new(ring))?;
drop(update_lock);
self.background.spawn_cancellable(
self.clone()
.broadcast(Message::AdvertiseConfig(adv.clone()), PING_TIMEOUT)
.map(Ok),
);
self.background.spawn(self.clone().save_network_config());
2020-04-06 20:27:51 +00:00
}
Ok(Message::Ok)
}
async fn ping_loop(self: Arc<Self>, mut stop_signal: watch::Receiver<bool>) {
while !*stop_signal.borrow() {
2021-03-15 21:36:41 +00:00
let restart_at = tokio::time::sleep(PING_INTERVAL);
let status = self.status.borrow().clone();
let ping_addrs = status
.nodes
.iter()
.filter(|(id, _)| **id != self.id)
.map(|(id, status)| (status.addr, Some(*id)))
.collect::<Vec<_>>();
2020-04-06 20:27:51 +00:00
2020-04-06 22:00:43 +00:00
self.clone().ping_nodes(ping_addrs).await;
2020-04-06 20:27:51 +00:00
select! {
2021-04-09 00:32:42 +00:00
_ = restart_at.fuse() => {},
_ = stop_signal.changed().fuse() => {},
}
}
}
async fn discovery_loop(
self: Arc<Self>,
bootstrap_peers: Vec<SocketAddr>,
2021-04-05 21:04:08 +00:00
consul_host: Option<String>,
consul_service_name: Option<String>,
mut stop_signal: watch::Receiver<bool>,
) {
2021-04-05 21:04:08 +00:00
let consul_config = match (consul_host, consul_service_name) {
(Some(ch), Some(csn)) => Some((ch, csn)),
_ => None,
};
while !*stop_signal.borrow() {
2021-04-09 00:32:42 +00:00
let not_configured = self.ring.borrow().config.members.is_empty();
let no_peers = self.status.borrow().nodes.len() < 3;
let bad_peers = self
.status
.borrow()
.nodes
.iter()
2021-04-05 21:04:08 +00:00
.filter(|(_, v)| v.is_up())
.count() != self.ring.borrow().config.members.len();
if not_configured || no_peers || bad_peers {
info!("Doing a bootstrap/discovery step (not_configured: {}, no_peers: {}, bad_peers: {})", not_configured, no_peers, bad_peers);
2021-04-05 21:04:08 +00:00
let mut ping_list = bootstrap_peers
.iter()
.map(|ip| (*ip, None))
.collect::<Vec<_>>();
2021-04-09 00:32:42 +00:00
if let Ok(peers) = self.persist_status.load_async().await {
ping_list.extend(peers.iter().map(|x| (x.addr, Some(x.id))));
}
2021-04-05 21:04:08 +00:00
if let Some((consul_host, consul_service_name)) = &consul_config {
match get_consul_nodes(consul_host, consul_service_name).await {
Ok(node_list) => {
ping_list.extend(node_list.iter().map(|a| (*a, None)));
}
Err(e) => {
warn!("Could not retrieve node list from Consul: {}", e);
}
}
}
2021-04-05 21:04:08 +00:00
self.clone().ping_nodes(ping_list).await;
}
let restart_at = tokio::time::sleep(DISCOVERY_INTERVAL);
select! {
2021-04-09 00:32:42 +00:00
_ = restart_at.fuse() => {},
_ = stop_signal.changed().fuse() => {},
}
}
}
2021-04-09 00:32:42 +00:00
// for some reason fixing this is causing compilation error, see https://github.com/rust-lang/rust-clippy/issues/7052
#[allow(clippy::manual_async_fn)]
2020-12-12 16:58:19 +00:00
fn pull_status(
self: Arc<Self>,
2021-05-02 21:13:08 +00:00
peer: Uuid,
) -> impl futures::future::Future<Output = ()> + Send + 'static {
2020-04-06 20:54:03 +00:00
async move {
2020-04-18 17:21:34 +00:00
let resp = self
.rpc_client
.call(peer, Message::PullStatus, PING_TIMEOUT)
2020-04-18 17:21:34 +00:00
.await;
2020-04-06 20:54:03 +00:00
if let Ok(Message::AdvertiseNodesUp(nodes)) = resp {
let _: Result<_, _> = self.handle_advertise_nodes_up(&nodes).await;
}
2020-04-06 20:27:51 +00:00
}
}
2021-05-02 21:13:08 +00:00
async fn pull_config(self: Arc<Self>, peer: Uuid) {
2020-04-18 17:21:34 +00:00
let resp = self
.rpc_client
.call(peer, Message::PullConfig, PING_TIMEOUT)
2020-04-18 17:21:34 +00:00
.await;
2020-04-06 20:27:51 +00:00
if let Ok(Message::AdvertiseConfig(config)) = resp {
2020-04-06 20:54:03 +00:00
let _: Result<_, _> = self.handle_advertise_config(&config).await;
2020-04-06 20:27:51 +00:00
}
2020-04-06 19:02:15 +00:00
}
2021-04-05 18:26:01 +00:00
async fn update_status(self: &Arc<Self>, updaters: &Updaters, status: Status) {
if status.hash != self.status.borrow().hash {
2021-04-05 18:33:24 +00:00
let mut list = status.to_serializable_membership(&self);
// Combine with old peer list to make sure no peer is lost
2021-04-09 00:32:42 +00:00
if let Ok(old_list) = self.persist_status.load_async().await {
for pp in old_list {
if !list.iter().any(|np| pp.id == np.id) {
list.push(pp);
2021-04-05 18:33:24 +00:00
}
}
}
2021-04-09 00:32:42 +00:00
if !list.is_empty() {
2021-04-05 18:42:46 +00:00
info!("Persisting new peer list ({} peers)", list.len());
2021-04-05 18:35:26 +00:00
self.persist_status
.save_async(&list)
.await
2021-04-05 18:33:24 +00:00
.expect("Unable to persist peer list");
}
2021-04-05 18:26:01 +00:00
}
updaters
.update_status
2021-04-05 21:21:25 +00:00
.send(Arc::new(status))
.expect("Could not update internal membership status");
}
2020-04-06 17:55:39 +00:00
}