2020-04-17 13:36:16 +00:00
|
|
|
use std::borrow::Borrow;
|
2020-04-18 17:21:34 +00:00
|
|
|
use std::marker::PhantomData;
|
2020-04-07 14:26:22 +00:00
|
|
|
use std::net::SocketAddr;
|
2020-04-23 14:40:59 +00:00
|
|
|
use std::pin::Pin;
|
2020-04-23 16:05:43 +00:00
|
|
|
use std::sync::atomic::Ordering;
|
2020-04-07 14:26:22 +00:00
|
|
|
use std::sync::Arc;
|
|
|
|
use std::time::Duration;
|
|
|
|
|
2020-04-23 14:40:59 +00:00
|
|
|
use arc_swap::ArcSwapOption;
|
2020-04-07 14:26:22 +00:00
|
|
|
use bytes::IntoBuf;
|
2020-04-23 14:40:59 +00:00
|
|
|
use futures::future::Future;
|
2020-04-07 14:26:22 +00:00
|
|
|
use futures::stream::futures_unordered::FuturesUnordered;
|
|
|
|
use futures::stream::StreamExt;
|
2020-04-10 20:01:48 +00:00
|
|
|
use futures_util::future::FutureExt;
|
2020-04-12 13:51:19 +00:00
|
|
|
use hyper::client::{Client, HttpConnector};
|
2020-04-19 15:15:48 +00:00
|
|
|
use hyper::{Body, Method, Request};
|
2020-04-22 16:51:52 +00:00
|
|
|
use tokio::sync::{watch, Semaphore};
|
2020-04-07 14:26:22 +00:00
|
|
|
|
2020-04-24 10:10:01 +00:00
|
|
|
use garage_util::background::BackgroundRunner;
|
|
|
|
use garage_util::config::TlsConfig;
|
|
|
|
use garage_util::data::*;
|
|
|
|
use garage_util::error::{Error, RPCError};
|
2020-04-23 17:05:46 +00:00
|
|
|
|
2020-04-24 10:10:01 +00:00
|
|
|
use crate::membership::Status;
|
|
|
|
use crate::rpc_server::RpcMessage;
|
|
|
|
use crate::tls_util;
|
2020-04-07 14:26:22 +00:00
|
|
|
|
2020-04-23 13:37:10 +00:00
|
|
|
const DEFAULT_TIMEOUT: Duration = Duration::from_secs(10);
|
|
|
|
|
|
|
|
#[derive(Copy, Clone)]
|
|
|
|
pub struct RequestStrategy {
|
|
|
|
pub rs_timeout: Duration,
|
|
|
|
pub rs_quorum: usize,
|
|
|
|
pub rs_interrupt_after_quorum: bool,
|
|
|
|
}
|
|
|
|
|
|
|
|
impl RequestStrategy {
|
|
|
|
pub fn with_quorum(quorum: usize) -> Self {
|
|
|
|
RequestStrategy {
|
|
|
|
rs_timeout: DEFAULT_TIMEOUT,
|
|
|
|
rs_quorum: quorum,
|
|
|
|
rs_interrupt_after_quorum: false,
|
|
|
|
}
|
|
|
|
}
|
|
|
|
pub fn with_timeout(mut self, timeout: Duration) -> Self {
|
|
|
|
self.rs_timeout = timeout;
|
|
|
|
self
|
|
|
|
}
|
|
|
|
pub fn interrupt_after_quorum(mut self, interrupt: bool) -> Self {
|
|
|
|
self.rs_interrupt_after_quorum = interrupt;
|
|
|
|
self
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-23 14:40:59 +00:00
|
|
|
pub type LocalHandlerFn<M> =
|
|
|
|
Box<dyn Fn(Arc<M>) -> Pin<Box<dyn Future<Output = Result<M, Error>> + Send>> + Send + Sync>;
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
pub struct RpcClient<M: RpcMessage> {
|
|
|
|
status: watch::Receiver<Arc<Status>>,
|
|
|
|
background: Arc<BackgroundRunner>,
|
|
|
|
|
2020-04-23 14:40:59 +00:00
|
|
|
local_handler: ArcSwapOption<(UUID, LocalHandlerFn<M>)>,
|
|
|
|
|
2020-12-12 16:58:19 +00:00
|
|
|
rpc_addr_client: RpcAddrClient<M>,
|
2020-04-07 22:39:07 +00:00
|
|
|
}
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
impl<M: RpcMessage + 'static> RpcClient<M> {
|
|
|
|
pub fn new(
|
|
|
|
rac: RpcAddrClient<M>,
|
|
|
|
background: Arc<BackgroundRunner>,
|
|
|
|
status: watch::Receiver<Arc<Status>>,
|
|
|
|
) -> Arc<Self> {
|
|
|
|
Arc::new(Self {
|
|
|
|
rpc_addr_client: rac,
|
|
|
|
background,
|
|
|
|
status,
|
2020-04-23 14:40:59 +00:00
|
|
|
local_handler: ArcSwapOption::new(None),
|
2020-04-18 17:21:34 +00:00
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2020-04-23 14:40:59 +00:00
|
|
|
pub fn set_local_handler<F, Fut>(&self, my_id: UUID, handler: F)
|
|
|
|
where
|
|
|
|
F: Fn(Arc<M>) -> Fut + Send + Sync + 'static,
|
|
|
|
Fut: Future<Output = Result<M, Error>> + Send + 'static,
|
|
|
|
{
|
|
|
|
let handler_arc = Arc::new(handler);
|
|
|
|
let handler: LocalHandlerFn<M> = Box::new(move |msg| {
|
|
|
|
let handler_arc2 = handler_arc.clone();
|
|
|
|
Box::pin(async move { handler_arc2(msg).await })
|
|
|
|
});
|
|
|
|
self.local_handler.swap(Some(Arc::new((my_id, handler))));
|
|
|
|
}
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
pub fn by_addr(&self) -> &RpcAddrClient<M> {
|
|
|
|
&self.rpc_addr_client
|
|
|
|
}
|
|
|
|
|
2020-04-23 14:40:59 +00:00
|
|
|
pub async fn call(&self, to: UUID, msg: M, timeout: Duration) -> Result<M, Error> {
|
|
|
|
self.call_arc(to, Arc::new(msg), timeout).await
|
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn call_arc(&self, to: UUID, msg: Arc<M>, timeout: Duration) -> Result<M, Error> {
|
|
|
|
if let Some(lh) = self.local_handler.load_full() {
|
|
|
|
let (my_id, local_handler) = lh.as_ref();
|
|
|
|
if to.borrow() == my_id {
|
|
|
|
return local_handler(msg).await;
|
|
|
|
}
|
|
|
|
}
|
2020-04-23 16:05:43 +00:00
|
|
|
let status = self.status.borrow().clone();
|
|
|
|
let node_status = match status.nodes.get(&to) {
|
|
|
|
Some(node_status) => {
|
|
|
|
if node_status.is_up() {
|
|
|
|
node_status
|
|
|
|
} else {
|
|
|
|
return Err(Error::from(RPCError::NodeDown(to)));
|
2020-04-07 22:39:07 +00:00
|
|
|
}
|
|
|
|
}
|
2020-04-23 16:05:43 +00:00
|
|
|
None => {
|
|
|
|
return Err(Error::Message(format!(
|
|
|
|
"Peer ID not found: {:?}",
|
|
|
|
to.borrow()
|
|
|
|
)))
|
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
};
|
2020-04-23 16:05:43 +00:00
|
|
|
match self
|
|
|
|
.rpc_addr_client
|
|
|
|
.call(&node_status.addr, msg, timeout)
|
|
|
|
.await
|
|
|
|
{
|
|
|
|
Err(rpc_error) => {
|
|
|
|
node_status.num_failures.fetch_add(1, Ordering::SeqCst);
|
|
|
|
// TODO: Save failure info somewhere
|
|
|
|
Err(Error::from(rpc_error))
|
|
|
|
}
|
|
|
|
Ok(x) => x,
|
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn call_many(&self, to: &[UUID], msg: M, timeout: Duration) -> Vec<Result<M, Error>> {
|
|
|
|
let msg = Arc::new(msg);
|
|
|
|
let mut resp_stream = to
|
|
|
|
.iter()
|
2020-04-23 14:40:59 +00:00
|
|
|
.map(|to| self.call_arc(*to, msg.clone(), timeout))
|
2020-04-18 17:21:34 +00:00
|
|
|
.collect::<FuturesUnordered<_>>();
|
|
|
|
|
|
|
|
let mut results = vec![];
|
|
|
|
while let Some(resp) = resp_stream.next().await {
|
|
|
|
results.push(resp);
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
results
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
2020-04-07 22:39:07 +00:00
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
pub async fn try_call_many(
|
|
|
|
self: &Arc<Self>,
|
|
|
|
to: &[UUID],
|
|
|
|
msg: M,
|
2020-04-23 13:37:10 +00:00
|
|
|
strategy: RequestStrategy,
|
2020-04-18 17:21:34 +00:00
|
|
|
) -> Result<Vec<M>, Error> {
|
2020-04-23 13:37:10 +00:00
|
|
|
let timeout = strategy.rs_timeout;
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
let msg = Arc::new(msg);
|
|
|
|
let mut resp_stream = to
|
|
|
|
.to_vec()
|
|
|
|
.into_iter()
|
|
|
|
.map(|to| {
|
|
|
|
let self2 = self.clone();
|
|
|
|
let msg = msg.clone();
|
2020-04-23 14:40:59 +00:00
|
|
|
async move { self2.call_arc(to, msg, timeout).await }
|
2020-04-18 17:21:34 +00:00
|
|
|
})
|
|
|
|
.collect::<FuturesUnordered<_>>();
|
|
|
|
|
|
|
|
let mut results = vec![];
|
|
|
|
let mut errors = vec![];
|
|
|
|
|
|
|
|
while let Some(resp) = resp_stream.next().await {
|
|
|
|
match resp {
|
|
|
|
Ok(msg) => {
|
|
|
|
results.push(msg);
|
2020-04-23 13:37:10 +00:00
|
|
|
if results.len() >= strategy.rs_quorum {
|
2020-04-18 17:21:34 +00:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
Err(e) => {
|
|
|
|
errors.push(e);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-04-23 13:37:10 +00:00
|
|
|
if results.len() >= strategy.rs_quorum {
|
|
|
|
// Continue requests in background.
|
2020-04-22 20:42:23 +00:00
|
|
|
// Continue the remaining requests immediately using tokio::spawn
|
|
|
|
// but enqueue a task in the background runner
|
|
|
|
// to ensure that the process won't exit until the requests are done
|
|
|
|
// (if we had just enqueued the resp_stream.collect directly in the background runner,
|
|
|
|
// the requests might have been put on hold in the background runner's queue,
|
|
|
|
// in which case they might timeout or otherwise fail)
|
2020-04-23 13:37:10 +00:00
|
|
|
if !strategy.rs_interrupt_after_quorum {
|
|
|
|
let wait_finished_fut = tokio::spawn(async move {
|
|
|
|
resp_stream.collect::<Vec<_>>().await;
|
|
|
|
Ok(())
|
|
|
|
});
|
2020-04-23 14:40:59 +00:00
|
|
|
self.background.spawn(wait_finished_fut.map(|x| {
|
2020-04-23 13:37:10 +00:00
|
|
|
x.unwrap_or_else(|e| Err(Error::Message(format!("Await failed: {}", e))))
|
|
|
|
}));
|
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
|
|
|
|
Ok(results)
|
|
|
|
} else {
|
2020-04-23 16:23:06 +00:00
|
|
|
let errors = errors.iter().map(|e| format!("{}", e)).collect::<Vec<_>>();
|
|
|
|
Err(Error::from(RPCError::TooManyErrors(errors)))
|
2020-04-07 22:39:07 +00:00
|
|
|
}
|
|
|
|
}
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
pub struct RpcAddrClient<M: RpcMessage> {
|
|
|
|
phantom: PhantomData<M>,
|
|
|
|
|
2020-12-12 16:58:19 +00:00
|
|
|
http_client: Arc<RpcHttpClient>,
|
|
|
|
path: String,
|
2020-04-18 17:21:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
impl<M: RpcMessage> RpcAddrClient<M> {
|
|
|
|
pub fn new(http_client: Arc<RpcHttpClient>, path: String) -> Self {
|
|
|
|
Self {
|
|
|
|
phantom: PhantomData::default(),
|
|
|
|
http_client: http_client,
|
|
|
|
path,
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
2020-04-18 17:21:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
pub async fn call<MB>(
|
|
|
|
&self,
|
|
|
|
to_addr: &SocketAddr,
|
|
|
|
msg: MB,
|
|
|
|
timeout: Duration,
|
2020-04-23 16:05:43 +00:00
|
|
|
) -> Result<Result<M, Error>, RPCError>
|
2020-04-18 17:21:34 +00:00
|
|
|
where
|
|
|
|
MB: Borrow<M>,
|
|
|
|
{
|
|
|
|
self.http_client
|
|
|
|
.call(&self.path, to_addr, msg, timeout)
|
|
|
|
.await
|
|
|
|
}
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
|
|
|
|
2020-04-22 16:51:52 +00:00
|
|
|
pub struct RpcHttpClient {
|
|
|
|
request_limiter: Semaphore,
|
|
|
|
method: ClientMethod,
|
|
|
|
}
|
|
|
|
|
|
|
|
enum ClientMethod {
|
2020-04-12 13:51:19 +00:00
|
|
|
HTTP(Client<HttpConnector, hyper::Body>),
|
2020-04-12 17:00:30 +00:00
|
|
|
HTTPS(Client<tls_util::HttpsConnectorFixedDnsname<HttpConnector>, hyper::Body>),
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
impl RpcHttpClient {
|
2020-04-22 16:51:52 +00:00
|
|
|
pub fn new(
|
|
|
|
max_concurrent_requests: usize,
|
|
|
|
tls_config: &Option<TlsConfig>,
|
|
|
|
) -> Result<Self, Error> {
|
|
|
|
let method = if let Some(cf) = tls_config {
|
2020-04-12 13:51:19 +00:00
|
|
|
let ca_certs = tls_util::load_certs(&cf.ca_cert)?;
|
|
|
|
let node_certs = tls_util::load_certs(&cf.node_cert)?;
|
|
|
|
let node_key = tls_util::load_private_key(&cf.node_key)?;
|
|
|
|
|
|
|
|
let mut config = rustls::ClientConfig::new();
|
|
|
|
|
|
|
|
for crt in ca_certs.iter() {
|
|
|
|
config.root_store.add(crt)?;
|
|
|
|
}
|
|
|
|
|
2020-04-12 17:00:30 +00:00
|
|
|
config.set_single_client_cert([&node_certs[..], &ca_certs[..]].concat(), node_key)?;
|
2020-04-12 13:51:19 +00:00
|
|
|
|
|
|
|
let connector =
|
2020-04-12 17:00:30 +00:00
|
|
|
tls_util::HttpsConnectorFixedDnsname::<HttpConnector>::new(config, "garage");
|
2020-04-12 13:51:19 +00:00
|
|
|
|
2020-04-22 16:51:52 +00:00
|
|
|
ClientMethod::HTTPS(Client::builder().build(connector))
|
2020-04-12 13:51:19 +00:00
|
|
|
} else {
|
2020-04-22 16:51:52 +00:00
|
|
|
ClientMethod::HTTP(Client::new())
|
|
|
|
};
|
|
|
|
Ok(RpcHttpClient {
|
|
|
|
method,
|
|
|
|
request_limiter: Semaphore::new(max_concurrent_requests),
|
|
|
|
})
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
|
|
|
|
2020-04-18 17:21:34 +00:00
|
|
|
async fn call<M, MB>(
|
2020-04-10 20:01:48 +00:00
|
|
|
&self,
|
2020-04-18 17:21:34 +00:00
|
|
|
path: &str,
|
2020-04-10 20:01:48 +00:00
|
|
|
to_addr: &SocketAddr,
|
2020-04-18 17:21:34 +00:00
|
|
|
msg: MB,
|
2020-04-10 20:01:48 +00:00
|
|
|
timeout: Duration,
|
2020-04-23 16:05:43 +00:00
|
|
|
) -> Result<Result<M, Error>, RPCError>
|
2020-04-18 17:21:34 +00:00
|
|
|
where
|
|
|
|
MB: Borrow<M>,
|
|
|
|
M: RpcMessage,
|
|
|
|
{
|
2020-04-22 16:51:52 +00:00
|
|
|
let uri = match self.method {
|
|
|
|
ClientMethod::HTTP(_) => format!("http://{}/{}", to_addr, path),
|
|
|
|
ClientMethod::HTTPS(_) => format!("https://{}/{}", to_addr, path),
|
2020-04-12 13:51:19 +00:00
|
|
|
};
|
|
|
|
|
2020-04-07 14:26:22 +00:00
|
|
|
let req = Request::builder()
|
|
|
|
.method(Method::POST)
|
|
|
|
.uri(uri)
|
2020-04-16 21:13:15 +00:00
|
|
|
.body(Body::from(rmp_to_vec_all_named(msg.borrow())?))?;
|
2020-04-07 14:26:22 +00:00
|
|
|
|
2020-04-22 16:51:52 +00:00
|
|
|
let resp_fut = match &self.method {
|
|
|
|
ClientMethod::HTTP(client) => client.request(req).fuse(),
|
|
|
|
ClientMethod::HTTPS(client) => client.request(req).fuse(),
|
2020-04-12 13:51:19 +00:00
|
|
|
};
|
2020-04-22 16:51:52 +00:00
|
|
|
|
|
|
|
let slot = self.request_limiter.acquire().await;
|
2020-04-12 13:51:19 +00:00
|
|
|
let resp = tokio::time::timeout(timeout, resp_fut)
|
2020-04-22 16:51:52 +00:00
|
|
|
.await
|
|
|
|
.map_err(|e| {
|
|
|
|
debug!(
|
|
|
|
"RPC timeout to {}: {}",
|
|
|
|
to_addr,
|
|
|
|
debug_serialize(msg.borrow())
|
|
|
|
);
|
|
|
|
e
|
|
|
|
})?
|
2020-04-12 13:51:19 +00:00
|
|
|
.map_err(|e| {
|
2020-04-21 12:54:55 +00:00
|
|
|
warn!(
|
2020-04-16 12:50:49 +00:00
|
|
|
"RPC HTTP client error when connecting to {}: {}",
|
|
|
|
to_addr, e
|
|
|
|
);
|
2020-04-12 13:51:19 +00:00
|
|
|
e
|
|
|
|
})?;
|
2020-04-07 14:26:22 +00:00
|
|
|
|
2020-04-19 15:15:48 +00:00
|
|
|
let status = resp.status();
|
|
|
|
let body = hyper::body::to_bytes(resp.into_body()).await?;
|
2020-05-01 19:18:54 +00:00
|
|
|
drop(slot);
|
|
|
|
|
2020-04-23 16:05:43 +00:00
|
|
|
match rmp_serde::decode::from_read::<_, Result<M, String>>(body.into_buf())? {
|
|
|
|
Err(e) => Ok(Err(Error::RemoteError(e, status))),
|
|
|
|
Ok(x) => Ok(Ok(x)),
|
2020-04-07 14:26:22 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|