RPC to ourself do not pass through serialization + HTTPS

This commit is contained in:
Alex 2020-04-23 14:40:59 +00:00
parent 37f880bc09
commit 2fe82be3bc
8 changed files with 88 additions and 50 deletions

2
TODO
View file

@ -10,8 +10,6 @@ Improvements
Membership: keep IP addresses of failed nodes and try to reping them regularly Membership: keep IP addresses of failed nodes and try to reping them regularly
RPC client/server: do not go through the serialization+HTTP+TLS+deserialization when doing a request to ourself.
Attaining S3 compatibility Attaining S3 compatibility
-------------------------- --------------------------

View file

@ -172,7 +172,7 @@ impl AdminRpcHandler {
if self if self
.rpc_client .rpc_client
.call( .call(
node, *node,
AdminRPC::LaunchRepair(opt_to_send.clone()), AdminRPC::LaunchRepair(opt_to_send.clone()),
ADMIN_RPC_TIMEOUT, ADMIN_RPC_TIMEOUT,
) )

View file

@ -96,19 +96,27 @@ impl BlockManager {
} }
fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) { fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) {
let self2 = self.clone();
rpc_server.add_handler::<Message, _, _>(path, move |msg, _addr| { rpc_server.add_handler::<Message, _, _>(path, move |msg, _addr| {
let self2 = self.clone(); let self2 = self2.clone();
async move { async move { self2.handle(&msg).await }
match msg {
Message::PutBlock(m) => self2.write_block(&m.hash, &m.data).await,
Message::GetBlock(h) => self2.read_block(&h).await,
Message::NeedBlockQuery(h) => {
self2.need_block(&h).await.map(Message::NeedBlockReply)
}
_ => Err(Error::BadRequest(format!("Unexpected RPC message"))),
}
}
}); });
let self2 = self.clone();
self.rpc_client
.set_local_handler(self.system.id, move |msg| {
let self2 = self2.clone();
async move { self2.handle(&msg).await }
});
}
async fn handle(self: Arc<Self>, msg: &Message) -> Result<Message, Error> {
match msg {
Message::PutBlock(m) => self.write_block(&m.hash, &m.data).await,
Message::GetBlock(h) => self.read_block(h).await,
Message::NeedBlockQuery(h) => self.need_block(h).await.map(Message::NeedBlockReply),
_ => Err(Error::BadRequest(format!("Unexpected RPC message"))),
}
} }
pub async fn spawn_background_worker(self: Arc<Self>) { pub async fn spawn_background_worker(self: Arc<Self>) {
@ -299,7 +307,7 @@ impl BlockManager {
let msg = Arc::new(Message::NeedBlockQuery(*hash)); let msg = Arc::new(Message::NeedBlockQuery(*hash));
let who_needs_fut = who.iter().map(|to| { let who_needs_fut = who.iter().map(|to| {
self.rpc_client self.rpc_client
.call(to, msg.clone(), NEED_BLOCK_QUERY_TIMEOUT) .call_arc(*to, msg.clone(), NEED_BLOCK_QUERY_TIMEOUT)
}); });
let who_needs = join_all(who_needs_fut).await; let who_needs = join_all(who_needs_fut).await;

View file

@ -297,7 +297,7 @@ impl System {
let (update_ring, ring) = watch::channel(Arc::new(ring)); let (update_ring, ring) = watch::channel(Arc::new(ring));
let rpc_http_client = Arc::new( let rpc_http_client = Arc::new(
RpcHttpClient::new(config.max_concurrent_requests, &config.rpc_tls) RpcHttpClient::new(config.max_concurrent_rpc_requests, &config.rpc_tls)
.expect("Could not create RPC client"), .expect("Could not create RPC client"),
); );
@ -633,7 +633,7 @@ impl System {
async move { async move {
let resp = self let resp = self
.rpc_client .rpc_client
.call(&peer, Message::PullStatus, PING_TIMEOUT) .call(peer, Message::PullStatus, PING_TIMEOUT)
.await; .await;
if let Ok(Message::AdvertiseNodesUp(nodes)) = resp { if let Ok(Message::AdvertiseNodesUp(nodes)) = resp {
let _: Result<_, _> = self.handle_advertise_nodes_up(&nodes).await; let _: Result<_, _> = self.handle_advertise_nodes_up(&nodes).await;
@ -644,7 +644,7 @@ impl System {
pub async fn pull_config(self: Arc<Self>, peer: UUID) { pub async fn pull_config(self: Arc<Self>, peer: UUID) {
let resp = self let resp = self
.rpc_client .rpc_client
.call(&peer, Message::PullConfig, PING_TIMEOUT) .call(peer, Message::PullConfig, PING_TIMEOUT)
.await; .await;
if let Ok(Message::AdvertiseConfig(config)) = resp { if let Ok(Message::AdvertiseConfig(config)) = resp {
let _: Result<_, _> = self.handle_advertise_config(&config).await; let _: Result<_, _> = self.handle_advertise_config(&config).await;

View file

@ -1,10 +1,13 @@
use std::borrow::Borrow; use std::borrow::Borrow;
use std::marker::PhantomData; use std::marker::PhantomData;
use std::net::SocketAddr; use std::net::SocketAddr;
use std::pin::Pin;
use std::sync::Arc; use std::sync::Arc;
use std::time::Duration; use std::time::Duration;
use arc_swap::ArcSwapOption;
use bytes::IntoBuf; use bytes::IntoBuf;
use futures::future::Future;
use futures::stream::futures_unordered::FuturesUnordered; use futures::stream::futures_unordered::FuturesUnordered;
use futures::stream::StreamExt; use futures::stream::StreamExt;
use futures_util::future::FutureExt; use futures_util::future::FutureExt;
@ -47,10 +50,15 @@ impl RequestStrategy {
} }
} }
pub type LocalHandlerFn<M> =
Box<dyn Fn(Arc<M>) -> Pin<Box<dyn Future<Output = Result<M, Error>> + Send>> + Send + Sync>;
pub struct RpcClient<M: RpcMessage> { pub struct RpcClient<M: RpcMessage> {
status: watch::Receiver<Arc<Status>>, status: watch::Receiver<Arc<Status>>,
background: Arc<BackgroundRunner>, background: Arc<BackgroundRunner>,
local_handler: ArcSwapOption<(UUID, LocalHandlerFn<M>)>,
pub rpc_addr_client: RpcAddrClient<M>, pub rpc_addr_client: RpcAddrClient<M>,
} }
@ -64,19 +72,38 @@ impl<M: RpcMessage + 'static> RpcClient<M> {
rpc_addr_client: rac, rpc_addr_client: rac,
background, background,
status, status,
local_handler: ArcSwapOption::new(None),
}) })
} }
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))));
}
pub fn by_addr(&self) -> &RpcAddrClient<M> { pub fn by_addr(&self) -> &RpcAddrClient<M> {
&self.rpc_addr_client &self.rpc_addr_client
} }
pub async fn call<MB: Borrow<M>, N: Borrow<UUID>>( pub async fn call(&self, to: UUID, msg: M, timeout: Duration) -> Result<M, Error> {
&self, self.call_arc(to, Arc::new(msg), timeout).await
to: N, }
msg: MB,
timeout: Duration, pub async fn call_arc(&self, to: UUID, msg: Arc<M>, timeout: Duration) -> Result<M, Error> {
) -> 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;
}
}
let addr = { let addr = {
let status = self.status.borrow().clone(); let status = self.status.borrow().clone();
match status.nodes.get(to.borrow()) { match status.nodes.get(to.borrow()) {
@ -96,7 +123,7 @@ impl<M: RpcMessage + 'static> RpcClient<M> {
let msg = Arc::new(msg); let msg = Arc::new(msg);
let mut resp_stream = to let mut resp_stream = to
.iter() .iter()
.map(|to| self.call(to, msg.clone(), timeout)) .map(|to| self.call_arc(*to, msg.clone(), timeout))
.collect::<FuturesUnordered<_>>(); .collect::<FuturesUnordered<_>>();
let mut results = vec![]; let mut results = vec![];
@ -121,7 +148,7 @@ impl<M: RpcMessage + 'static> RpcClient<M> {
.map(|to| { .map(|to| {
let self2 = self.clone(); let self2 = self.clone();
let msg = msg.clone(); let msg = msg.clone();
async move { self2.call(to, msg, timeout).await } async move { self2.call_arc(to, msg, timeout).await }
}) })
.collect::<FuturesUnordered<_>>(); .collect::<FuturesUnordered<_>>();
@ -155,7 +182,7 @@ impl<M: RpcMessage + 'static> RpcClient<M> {
resp_stream.collect::<Vec<_>>().await; resp_stream.collect::<Vec<_>>().await;
Ok(()) Ok(())
}); });
self.clone().background.spawn(wait_finished_fut.map(|x| { self.background.spawn(wait_finished_fut.map(|x| {
x.unwrap_or_else(|e| Err(Error::Message(format!("Await failed: {}", e)))) x.unwrap_or_else(|e| Err(Error::Message(format!("Await failed: {}", e))))
})); }));
} }

View file

@ -35,8 +35,8 @@ pub struct Config {
pub bootstrap_peers: Vec<SocketAddr>, pub bootstrap_peers: Vec<SocketAddr>,
#[serde(default = "default_max_concurrent_requests")] #[serde(default = "default_max_concurrent_rpc_requests")]
pub max_concurrent_requests: usize, pub max_concurrent_rpc_requests: usize,
#[serde(default = "default_block_size")] #[serde(default = "default_block_size")]
pub block_size: usize, pub block_size: usize,
@ -53,7 +53,7 @@ pub struct Config {
pub rpc_tls: Option<TlsConfig>, pub rpc_tls: Option<TlsConfig>,
} }
fn default_max_concurrent_requests() -> usize { fn default_max_concurrent_rpc_requests() -> usize {
12 12
} }
fn default_block_size() -> usize { fn default_block_size() -> usize {
@ -262,7 +262,7 @@ pub async fn run_server(config_file: PathBuf) -> Result<(), Error> {
info!("Initializing background runner..."); info!("Initializing background runner...");
let (send_cancel, watch_cancel) = watch::channel(false); let (send_cancel, watch_cancel) = watch::channel(false);
let background = BackgroundRunner::new(8, watch_cancel.clone()); let background = BackgroundRunner::new(16, watch_cancel.clone());
let garage = Garage::new(config, id, db, background.clone(), &mut rpc_server).await; let garage = Garage::new(config, id, db, background.clone(), &mut rpc_server).await;

View file

@ -204,7 +204,7 @@ where
let call_futures = call_list.drain().map(|(node, entries)| async move { let call_futures = call_list.drain().map(|(node, entries)| async move {
let rpc = TableRPC::<F>::Update(entries); let rpc = TableRPC::<F>::Update(entries);
let resp = self.rpc_client.call(&node, rpc, TABLE_RPC_TIMEOUT).await?; let resp = self.rpc_client.call(node, rpc, TABLE_RPC_TIMEOUT).await?;
Ok::<_, Error>((node, resp)) Ok::<_, Error>((node, resp))
}); });
let mut resps = call_futures.collect::<FuturesUnordered<_>>(); let mut resps = call_futures.collect::<FuturesUnordered<_>>();
@ -358,20 +358,28 @@ where
// =============== HANDLERS FOR RPC OPERATIONS (SERVER SIDE) ============== // =============== HANDLERS FOR RPC OPERATIONS (SERVER SIDE) ==============
fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) { fn register_handler(self: Arc<Self>, rpc_server: &mut RpcServer, path: String) {
let self2 = self.clone();
rpc_server.add_handler::<TableRPC<F>, _, _>(path, move |msg, _addr| { rpc_server.add_handler::<TableRPC<F>, _, _>(path, move |msg, _addr| {
let self2 = self.clone(); let self2 = self2.clone();
async move { self2.handle(msg).await } async move { self2.handle(&msg).await }
}) });
let self2 = self.clone();
self.rpc_client
.set_local_handler(self.system.id, move |msg| {
let self2 = self2.clone();
async move { self2.handle(&msg).await }
});
} }
async fn handle(self: &Arc<Self>, msg: TableRPC<F>) -> Result<TableRPC<F>, Error> { async fn handle(self: &Arc<Self>, msg: &TableRPC<F>) -> Result<TableRPC<F>, Error> {
match msg { match msg {
TableRPC::ReadEntry(key, sort_key) => { TableRPC::ReadEntry(key, sort_key) => {
let value = self.handle_read_entry(&key, &sort_key)?; let value = self.handle_read_entry(key, sort_key)?;
Ok(TableRPC::ReadEntryResponse(value)) Ok(TableRPC::ReadEntryResponse(value))
} }
TableRPC::ReadRange(key, begin_sort_key, filter, limit) => { TableRPC::ReadRange(key, begin_sort_key, filter, limit) => {
let values = self.handle_read_range(&key, &begin_sort_key, &filter, limit)?; let values = self.handle_read_range(key, begin_sort_key, filter, *limit)?;
Ok(TableRPC::Update(values)) Ok(TableRPC::Update(values))
} }
TableRPC::Update(pairs) => { TableRPC::Update(pairs) => {
@ -381,7 +389,7 @@ where
TableRPC::SyncRPC(rpc) => { TableRPC::SyncRPC(rpc) => {
let syncer = self.syncer.load_full().unwrap(); let syncer = self.syncer.load_full().unwrap();
let response = syncer let response = syncer
.handle_rpc(&rpc, self.system.background.stop_signal.clone()) .handle_rpc(rpc, self.system.background.stop_signal.clone())
.await?; .await?;
Ok(TableRPC::SyncRPC(response)) Ok(TableRPC::SyncRPC(response))
} }
@ -433,14 +441,11 @@ where
Ok(ret) Ok(ret)
} }
pub async fn handle_update( pub async fn handle_update(self: &Arc<Self>, entries: &[Arc<ByteBuf>]) -> Result<(), Error> {
self: &Arc<Self>,
mut entries: Vec<Arc<ByteBuf>>,
) -> Result<(), Error> {
let syncer = self.syncer.load_full().unwrap(); let syncer = self.syncer.load_full().unwrap();
let mut epidemic_propagate = vec![]; let mut epidemic_propagate = vec![];
for update_bytes in entries.drain(..) { for update_bytes in entries.iter() {
let update = rmp_serde::decode::from_read_ref::<_, F::E>(update_bytes.as_slice())?; let update = rmp_serde::decode::from_read_ref::<_, F::E>(update_bytes.as_slice())?;
let tree_key = self.tree_key(update.partition_key(), update.sort_key()); let tree_key = self.tree_key(update.partition_key(), update.sort_key());

View file

@ -457,8 +457,8 @@ where
.table .table
.rpc_client .rpc_client
.call( .call(
&who, who,
&TableRPC::<F>::SyncRPC(SyncRPC::GetRootChecksumRange( TableRPC::<F>::SyncRPC(SyncRPC::GetRootChecksumRange(
partition.begin.clone(), partition.begin.clone(),
partition.end.clone(), partition.end.clone(),
)), )),
@ -496,8 +496,8 @@ where
.table .table
.rpc_client .rpc_client
.call( .call(
&who, who,
&TableRPC::<F>::SyncRPC(SyncRPC::Checksums(step, retain)), TableRPC::<F>::SyncRPC(SyncRPC::Checksums(step, retain)),
TABLE_SYNC_RPC_TIMEOUT, TABLE_SYNC_RPC_TIMEOUT,
) )
.await?; .await?;
@ -523,7 +523,7 @@ where
} }
} }
if retain && diff_items.len() > 0 { if retain && diff_items.len() > 0 {
self.table.handle_update(diff_items).await?; self.table.handle_update(&diff_items[..]).await?;
} }
if items_to_send.len() > 0 { if items_to_send.len() > 0 {
self.send_items(who, items_to_send).await?; self.send_items(who, items_to_send).await?;
@ -555,7 +555,7 @@ where
let rpc_resp = self let rpc_resp = self
.table .table
.rpc_client .rpc_client
.call(&who, &TableRPC::<F>::Update(values), TABLE_SYNC_RPC_TIMEOUT) .call(who, TableRPC::<F>::Update(values), TABLE_SYNC_RPC_TIMEOUT)
.await?; .await?;
if let TableRPC::<F>::Ok = rpc_resp { if let TableRPC::<F>::Ok = rpc_resp {
Ok(()) Ok(())