veilid/veilid-core/src/rpc_processor/mod.rs

994 lines
35 KiB
Rust
Raw Normal View History

2021-11-22 16:28:30 +00:00
mod coders;
2021-12-17 02:57:28 +00:00
mod debug;
mod private_route;
2022-07-04 16:03:21 +00:00
mod rpc_cancel_tunnel;
mod rpc_complete_tunnel;
mod rpc_find_block;
mod rpc_find_node;
mod rpc_get_value;
mod rpc_node_info_update;
mod rpc_return_receipt;
mod rpc_route;
mod rpc_set_value;
mod rpc_signal;
mod rpc_start_tunnel;
mod rpc_status;
mod rpc_supply_block;
mod rpc_validate_dial_info;
mod rpc_value_changed;
mod rpc_watch_value;
2021-12-17 02:57:28 +00:00
pub use debug::*;
pub use private_route::*;
2021-11-22 16:28:30 +00:00
2022-07-04 16:03:21 +00:00
use super::*;
2021-11-22 16:28:30 +00:00
use crate::dht::*;
use crate::xx::*;
use capnp::message::ReaderSegments;
use coders::*;
2022-06-13 00:58:02 +00:00
use futures_util::StreamExt;
2021-11-22 16:28:30 +00:00
use network_manager::*;
use receipt_manager::*;
use routing_table::*;
2022-06-13 00:58:02 +00:00
use stop_token::future::FutureExt;
2021-11-22 16:28:30 +00:00
/////////////////////////////////////////////////////////////////////
type OperationId = u64;
2021-12-14 19:20:05 +00:00
#[derive(Debug, Clone)]
2021-11-22 16:28:30 +00:00
pub enum Destination {
2022-05-25 02:46:30 +00:00
Direct(NodeRef), // Send to node (target noderef)
Relay(NodeRef, DHTKey), // Send to node for relay purposes (relay noderef, target nodeid)
PrivateRoute(PrivateRoute), // Send to private route (privateroute)
}
impl fmt::Display for Destination {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
match self {
Destination::Direct(nr) => {
write!(f, "{:?}", nr)
}
Destination::Relay(nr, key) => {
write!(f, "{:?}@{:?}", key.encode(), nr)
}
Destination::PrivateRoute(pr) => {
write!(f, "{}", pr)
}
}
}
2021-11-22 16:28:30 +00:00
}
#[derive(Debug, Clone)]
struct RPCMessageHeader {
2022-07-04 16:03:21 +00:00
timestamp: u64, // time the message was received, not sent
2022-05-31 23:54:52 +00:00
envelope: Envelope,
2021-11-22 16:28:30 +00:00
body_len: u64,
peer_noderef: NodeRef, // ensures node doesn't get evicted from routing table until we're done with it
}
2022-06-30 02:17:19 +00:00
#[derive(Debug)]
2021-11-22 16:28:30 +00:00
struct RPCMessageData {
2022-07-04 16:03:21 +00:00
contents: Vec<u8>, // rpc messages must be a canonicalized single segment
2021-11-22 16:28:30 +00:00
}
impl ReaderSegments for RPCMessageData {
2021-11-27 17:44:21 +00:00
fn get_segment(&self, idx: u32) -> Option<&[u8]> {
2021-11-22 16:28:30 +00:00
if idx > 0 {
None
} else {
Some(self.contents.as_slice())
}
}
}
#[derive(Debug)]
2022-07-04 16:03:21 +00:00
struct RPCMessageEncoded {
2021-11-22 16:28:30 +00:00
header: RPCMessageHeader,
data: RPCMessageData,
}
2022-07-04 16:03:21 +00:00
struct RPCMessage {
2021-11-22 16:28:30 +00:00
header: RPCMessageHeader,
2022-07-04 16:03:21 +00:00
operation: RPCOperation,
opt_sender_nr: Option<NodeRef>,
2021-11-22 16:28:30 +00:00
}
fn builder_to_vec<'a, T>(builder: capnp::message::Builder<T>) -> Result<Vec<u8>, RPCError>
where
T: capnp::message::Allocator + 'a,
{
let wordvec = builder
.into_reader()
.canonicalize()
2021-12-18 00:18:25 +00:00
.map_err(map_error_capnp_error!())
.map_err(logthru_rpc!())?;
2021-11-22 16:28:30 +00:00
Ok(capnp::Word::words_to_bytes(wordvec.as_slice()).to_vec())
}
2022-05-25 02:46:30 +00:00
fn reader_to_vec<'a, T>(reader: &capnp::message::Reader<T>) -> Result<Vec<u8>, RPCError>
2021-11-22 16:28:30 +00:00
where
T: capnp::message::ReaderSegments + 'a,
{
2021-12-18 00:18:25 +00:00
let wordvec = reader
.canonicalize()
.map_err(map_error_capnp_error!())
.map_err(logthru_rpc!())?;
2021-11-22 16:28:30 +00:00
Ok(capnp::Word::words_to_bytes(wordvec.as_slice()).to_vec())
}
#[derive(Debug)]
struct WaitableReply {
op_id: OperationId,
2022-07-04 16:03:21 +00:00
eventual: EventualValue<RPCMessage>,
2021-11-22 16:28:30 +00:00
timeout: u64,
node_ref: NodeRef,
send_ts: u64,
2022-04-23 01:30:09 +00:00
send_data_kind: SendDataKind,
2021-11-22 16:28:30 +00:00
}
/////////////////////////////////////////////////////////////////////
#[derive(Clone, Debug, Default)]
2022-07-04 16:03:21 +00:00
pub struct Answer<T> {
pub latency: u64, // how long it took to get this answer
pub answer: T, // the answer itself
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
impl<T> Answer<T> {
pub fn new(latency: u64, answer: T) -> Self {
Self { latency, answer }
}
2021-11-22 16:28:30 +00:00
}
2022-07-04 03:20:30 +00:00
struct RenderedOperation {
2022-07-05 02:44:04 +00:00
message: Vec<u8>, // The rendered operation bytes
node_id: DHTKey, // Node id we're sending to
node_ref: Option<NodeRef>, // Node to send envelope to (may not be destination node id in case of relay)
hop_count: usize, // Total safety + private route hop count
2022-07-04 03:20:30 +00:00
}
2021-11-22 16:28:30 +00:00
/////////////////////////////////////////////////////////////////////
pub struct RPCProcessorInner {
network_manager: NetworkManager,
routing_table: RoutingTable,
2022-05-31 23:54:52 +00:00
node_id: DHTKey,
node_id_secret: DHTKeySecret,
2022-07-04 16:03:21 +00:00
send_channel: Option<flume::Sender<RPCMessageEncoded>>,
2021-11-22 16:28:30 +00:00
timeout: u64,
max_route_hop_count: usize,
2022-07-04 16:03:21 +00:00
waiting_rpc_table: BTreeMap<OperationId, EventualValue<RPCMessage>>,
2022-06-13 00:58:02 +00:00
stop_source: Option<StopSource>,
worker_join_handles: Vec<MustJoinHandle<()>>,
2021-11-22 16:28:30 +00:00
}
#[derive(Clone)]
pub struct RPCProcessor {
crypto: Crypto,
config: VeilidConfig,
2022-04-16 15:18:54 +00:00
enable_local_peer_scope: bool,
2021-11-22 16:28:30 +00:00
inner: Arc<Mutex<RPCProcessorInner>>,
}
impl RPCProcessor {
fn new_inner(network_manager: NetworkManager) -> RPCProcessorInner {
RPCProcessorInner {
network_manager: network_manager.clone(),
routing_table: network_manager.routing_table(),
2022-05-31 23:54:52 +00:00
node_id: DHTKey::default(),
node_id_secret: DHTKeySecret::default(),
2021-12-17 02:57:28 +00:00
send_channel: None,
2021-11-22 16:28:30 +00:00
timeout: 10000000,
max_route_hop_count: 7,
waiting_rpc_table: BTreeMap::new(),
2022-06-13 00:58:02 +00:00
stop_source: None,
2021-11-22 16:28:30 +00:00
worker_join_handles: Vec::new(),
}
}
pub fn new(network_manager: NetworkManager) -> Self {
Self {
crypto: network_manager.crypto(),
config: network_manager.config(),
2022-04-16 15:18:54 +00:00
enable_local_peer_scope: network_manager
2021-12-27 16:31:31 +00:00
.config()
.get()
.network
2022-04-16 15:18:54 +00:00
.enable_local_peer_scope,
2021-11-22 16:28:30 +00:00
inner: Arc::new(Mutex::new(Self::new_inner(network_manager))),
}
}
pub fn network_manager(&self) -> NetworkManager {
self.inner.lock().network_manager.clone()
}
pub fn routing_table(&self) -> RoutingTable {
self.inner.lock().routing_table.clone()
}
2022-05-31 23:54:52 +00:00
pub fn node_id(&self) -> DHTKey {
2021-11-22 16:28:30 +00:00
self.inner.lock().node_id
}
2022-05-31 23:54:52 +00:00
pub fn node_id_secret(&self) -> DHTKeySecret {
2021-11-22 16:28:30 +00:00
self.inner.lock().node_id_secret
}
//////////////////////////////////////////////////////////////////////
2022-04-25 00:16:13 +00:00
fn filter_peer_scope(&self, node_info: &NodeInfo) -> bool {
2022-04-17 17:28:39 +00:00
// if local peer scope is enabled, then don't reject any peer info
if self.enable_local_peer_scope {
return true;
}
2021-12-27 16:31:31 +00:00
// reject attempts to include non-public addresses in results
2022-04-25 00:16:13 +00:00
for did in &node_info.dial_info_detail_list {
if !did.dial_info.is_global() {
2022-04-17 17:28:39 +00:00
// non-public address causes rejection
return false;
}
}
2022-04-25 00:16:13 +00:00
if let Some(rpi) = &node_info.relay_peer_info {
2022-05-11 01:49:42 +00:00
for did in &rpi.signed_node_info.node_info.dial_info_detail_list {
2022-04-25 00:16:13 +00:00
if !did.dial_info.is_global() {
2022-04-08 14:17:09 +00:00
// non-public address causes rejection
return false;
}
}
2021-12-27 16:31:31 +00:00
}
true
}
2021-11-22 16:28:30 +00:00
//////////////////////////////////////////////////////////////////////
2022-03-27 01:25:24 +00:00
// Search the DHT for a single node closest to a key and add it to the routing table and return the node reference
2021-11-22 16:28:30 +00:00
pub async fn search_dht_single_key(
&self,
2022-05-31 23:54:52 +00:00
node_id: DHTKey,
2021-11-22 16:28:30 +00:00
_count: u32,
_fanout: u32,
_timeout: Option<u64>,
) -> Result<NodeRef, RPCError> {
let routing_table = self.routing_table();
// xxx find node but stop if we find the exact node we want
// xxx return whatever node is closest after the timeout
2021-12-18 00:18:25 +00:00
Err(rpc_error_unimplemented("search_dht_single_key")).map_err(logthru_rpc!(error))
2021-11-22 16:28:30 +00:00
}
// Search the DHT for the 'count' closest nodes to a key, adding them all to the routing table if they are not there and returning their node references
pub async fn search_dht_multi_key(
&self,
2022-05-31 23:54:52 +00:00
_node_id: DHTKey,
2021-11-22 16:28:30 +00:00
_count: u32,
_fanout: u32,
_timeout: Option<u64>,
) -> Result<Vec<NodeRef>, RPCError> {
// xxx return closest nodes after the timeout
2021-12-18 00:18:25 +00:00
Err(rpc_error_unimplemented("search_dht_multi_key")).map_err(logthru_rpc!(error))
2021-11-22 16:28:30 +00:00
}
// Search the DHT for a specific node corresponding to a key unless we have that node in our routing table already, and return the node reference
2022-03-27 01:25:24 +00:00
// Note: This routine can possible be recursive, hence the SystemPinBoxFuture async form
2022-07-04 16:03:21 +00:00
pub fn resolve_node(&self, node_id: DHTKey) -> SystemPinBoxFuture<Result<NodeRef, RPCError>> {
2022-03-27 01:25:24 +00:00
let this = self.clone();
Box::pin(async move {
let routing_table = this.routing_table();
// First see if we have the node in our routing table already
if let Some(nr) = routing_table.lookup_node_ref(node_id) {
2022-04-16 15:18:54 +00:00
// ensure we have some dial info for the entry already,
2022-03-27 01:25:24 +00:00
// if not, we should do the find_node anyway
2022-04-17 17:28:39 +00:00
if nr.has_any_dial_info() {
2022-03-27 01:25:24 +00:00
return Ok(nr);
}
}
2021-11-22 16:28:30 +00:00
2022-03-27 01:25:24 +00:00
// If nobody knows where this node is, ask the DHT for it
let (count, fanout, timeout) = {
let c = this.config.get();
(
c.network.dht.resolve_node_count,
c.network.dht.resolve_node_fanout,
c.network.dht.resolve_node_timeout_ms.map(ms_to_us),
)
};
let nr = this
.search_dht_single_key(node_id, count, fanout, timeout)
.await?;
2021-11-22 16:28:30 +00:00
2022-03-27 01:25:24 +00:00
if nr.node_id() != node_id {
// found a close node, but not exact within our configured resolve_node timeout
return Err(RPCError::Timeout).map_err(logthru_rpc!());
}
Ok(nr)
})
2021-11-22 16:28:30 +00:00
}
// set up wait for reply
2022-07-04 16:03:21 +00:00
fn add_op_id_waiter(&self, op_id: OperationId) -> EventualValue<RPCMessage> {
2021-11-22 16:28:30 +00:00
let mut inner = self.inner.lock();
let e = EventualValue::new();
inner.waiting_rpc_table.insert(op_id, e.clone());
e
}
// remove wait for reply
fn cancel_op_id_waiter(&self, op_id: OperationId) {
let mut inner = self.inner.lock();
inner.waiting_rpc_table.remove(&op_id);
}
// complete the reply
2022-07-04 16:03:21 +00:00
async fn complete_op_id_waiter(&self, msg: RPCMessage) -> Result<(), RPCError> {
let op_id = msg.operation.op_id();
2021-11-22 16:28:30 +00:00
let eventual = {
let mut inner = self.inner.lock();
2021-11-27 17:44:21 +00:00
inner
.waiting_rpc_table
.remove(&op_id)
.ok_or_else(|| rpc_error_internal("Unmatched operation id"))?
2021-11-22 16:28:30 +00:00
};
2022-07-04 16:03:21 +00:00
eventual.resolve(msg).await;
2021-11-27 17:44:21 +00:00
Ok(())
2021-11-22 16:28:30 +00:00
}
// wait for reply
async fn do_wait_for_reply(
&self,
waitable_reply: &WaitableReply,
2022-07-04 16:03:21 +00:00
) -> Result<(RPCMessage, u64), RPCError> {
2021-11-22 16:28:30 +00:00
let timeout_ms = u32::try_from(waitable_reply.timeout / 1000u64)
.map_err(map_error_internal!("invalid timeout"))?;
// wait for eventualvalue
2022-06-28 03:46:29 +00:00
let start_ts = intf::get_timestamp();
let res = intf::timeout(timeout_ms, waitable_reply.eventual.instance())
2021-11-22 16:28:30 +00:00
.await
.map_err(|_| RPCError::Timeout)?;
let rpcreader = res.take_value().unwrap();
2022-06-28 03:46:29 +00:00
let end_ts = intf::get_timestamp();
Ok((rpcreader, end_ts - start_ts))
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
2021-11-22 16:28:30 +00:00
async fn wait_for_reply(
&self,
waitable_reply: WaitableReply,
2022-07-04 16:03:21 +00:00
) -> Result<(RPCMessage, u64), RPCError> {
2021-11-22 16:28:30 +00:00
let out = self.do_wait_for_reply(&waitable_reply).await;
match &out {
Err(_) => {
self.cancel_op_id_waiter(waitable_reply.op_id);
2022-04-18 22:49:33 +00:00
self.routing_table()
.stats_question_lost(waitable_reply.node_ref.clone());
2021-11-22 16:28:30 +00:00
}
Ok((rpcreader, _)) => {
2022-04-16 15:18:54 +00:00
// Note that we definitely received this node info since we got a reply
waitable_reply.node_ref.set_seen_our_node_info();
2022-03-25 02:07:55 +00:00
2021-11-22 16:28:30 +00:00
// Reply received
2022-06-28 03:46:29 +00:00
let recv_ts = intf::get_timestamp();
2022-04-18 22:49:33 +00:00
self.routing_table().stats_answer_rcvd(
waitable_reply.node_ref,
waitable_reply.send_ts,
recv_ts,
rpcreader.header.body_len,
)
2021-11-22 16:28:30 +00:00
}
};
out
}
2022-07-04 16:03:21 +00:00
// Gets a 'RespondTo::Sender' that contains either our dial info,
// or None if the peer has seen our dial info before or our node info is not yet valid
// because of an unknown network class
pub fn make_respond_to_sender(&self, peer: NodeRef) -> RespondTo {
if peer.has_seen_our_node_info()
|| matches!(
self.network_manager()
.get_network_class()
.unwrap_or(NetworkClass::Invalid),
NetworkClass::Invalid
)
{
RespondTo::Sender(None)
} else {
let our_sni = self.routing_table().get_own_signed_node_info();
RespondTo::Sender(Some(our_sni))
}
}
2022-07-05 02:44:04 +00:00
// Produce a byte buffer that represents the wire encoding of the entire
// unencrypted envelope body for a RPC message. This incorporates
// wrapping a private and/or safety route if they are specified.
2022-07-04 03:20:30 +00:00
#[instrument(level = "debug", skip(self, operation, safety_route_spec), err)]
2022-07-04 16:03:21 +00:00
fn render_operation(
&self,
2022-07-04 21:58:26 +00:00
dest: Destination,
2022-07-04 16:03:21 +00:00
operation: &RPCOperation,
safety_route_spec: Option<&SafetyRouteSpec>,
) -> Result<RenderedOperation, RPCError> {
2022-07-05 02:44:04 +00:00
let out_node_id; // Envelope Node Id
let mut out_node_ref: Option<NodeRef> = None; // Node to send envelope to
let out_hop_count: usize; // Total safety + private route hop count
let out_message; // Envelope data
2022-07-04 03:20:30 +00:00
// Encode message to a builder and make a message reader for it
2022-07-05 02:44:04 +00:00
// Then produce the message as an unencrypted byte buffer
let message_vec = {
let mut msg_builder = ::capnp::message::Builder::new_default();
let mut op_builder = msg_builder.init_root::<veilid_capnp::operation::Builder>();
operation.encode(&mut op_builder)?;
builder_to_vec(msg_builder)?
};
2021-11-22 16:28:30 +00:00
2022-07-05 02:44:04 +00:00
// To where are we sending the request
match dest {
Destination::Direct(node_ref) | Destination::Relay(node_ref, _) => {
// Send to a node without a private route
// --------------------------------------
// Get the actual destination node id accounting for relays
let (node_ref, node_id) = if let Destination::Relay(_, dht_key) = dest {
(node_ref.clone(), dht_key.clone())
} else {
let node_id = node_ref.node_id();
(node_ref.clone(), node_id)
};
// Handle the existence of safety route
match safety_route_spec {
None => {
// If no safety route is being used, and we're not sending to a private
// route, we can use a direct envelope instead of routing
out_message = message_vec;
// Message goes directly to the node
out_node_id = node_id;
out_node_ref = Some(node_ref);
out_hop_count = 1;
}
Some(sr) => {
// No private route was specified for the request
// but we are using a safety route, so we must create an empty private route
let mut pr_builder = ::capnp::message::Builder::new_default();
let private_route = PrivateRoute::new_stub(node_id);
// first
out_node_id = sr
.hops
.first()
.ok_or_else(|| rpc_error_internal("no hop in safety route"))?
.dial_info
.node_id
.key;
out_message = self.wrap_with_route(Some(sr), private_route, message_vec)?;
out_hop_count = 1 + sr.hops.len();
}
};
}
Destination::PrivateRoute(private_route) => {
// Send to private route
// ---------------------
// Reply with 'route' operation
out_node_id = match safety_route_spec {
None => {
// If no safety route, the first node is the first hop of the private route
out_hop_count = private_route.hop_count as usize;
let out_node_id = match &private_route.hops {
Some(rh) => rh.dial_info.node_id.key,
_ => return Err(rpc_error_internal("private route has no hops")),
};
out_message = self.wrap_with_route(None, private_route, message_vec)?;
out_node_id
}
Some(sr) => {
// If safety route is in use, first node is the first hop of the safety route
out_hop_count = 1 + sr.hops.len() + (private_route.hop_count as usize);
let out_node_id = sr
.hops
.first()
.ok_or_else(|| rpc_error_internal("no hop in safety route"))?
.dial_info
.node_id
.key;
out_message = self.wrap_with_route(Some(sr), private_route, message_vec)?;
out_node_id
2021-11-22 16:28:30 +00:00
}
}
}
2022-07-05 02:44:04 +00:00
}
2021-11-22 16:28:30 +00:00
// Verify hop count isn't larger than out maximum routed hop count
2022-07-05 02:44:04 +00:00
if out_hop_count > self.inner.lock().max_route_hop_count {
2022-01-05 17:01:02 +00:00
return Err(rpc_error_internal("hop count too long for route"))
.map_err(logthru_rpc!(warn));
2022-07-04 16:03:21 +00:00
}
2022-07-04 03:20:30 +00:00
Ok(RenderedOperation {
2022-07-05 02:44:04 +00:00
message: out_message,
node_id: out_node_id,
node_ref: out_node_ref,
hop_count: out_hop_count,
2022-07-04 03:20:30 +00:00
})
}
// Issue a question over the network, possibly using an anonymized route
#[instrument(level = "debug", skip(self, question, safety_route_spec), err)]
async fn question(
&self,
dest: Destination,
question: RPCQuestion,
safety_route_spec: Option<&SafetyRouteSpec>,
) -> Result<WaitableReply, RPCError> {
// Wrap question in operation
let operation = RPCOperation::new_question(question);
2022-07-04 21:58:26 +00:00
let op_id = operation.op_id();
// Log rpc send
debug!(target: "rpc_message", dir = "send", kind = "question", op_id, desc = operation.kind().desc(), ?dest);
2022-07-04 03:20:30 +00:00
// Produce rendered operation
let RenderedOperation {
2022-07-05 02:44:04 +00:00
message,
node_id,
node_ref,
hop_count,
2022-07-04 21:58:26 +00:00
} = self.render_operation(dest, &operation, safety_route_spec)?;
2022-07-04 03:20:30 +00:00
2022-07-04 16:03:21 +00:00
// If we need to resolve the first hop, do it
2022-07-05 02:44:04 +00:00
let node_ref = match node_ref {
2021-11-22 16:28:30 +00:00
None => {
// resolve node
2022-07-05 02:44:04 +00:00
self.resolve_node(node_id)
2021-12-17 02:57:28 +00:00
.await
.map_err(logthru_rpc!(error))?
2021-11-22 16:28:30 +00:00
}
Some(nr) => {
// got the node in the routing table already
nr
}
};
2022-07-05 02:44:04 +00:00
// Calculate answer timeout
// Timeout is number of hops times the timeout per hop
let timeout = self.inner.lock().timeout * (hop_count as u64);
2022-07-04 03:20:30 +00:00
// Set up op id eventual
let eventual = self.add_op_id_waiter(op_id);
2022-07-03 19:52:27 +00:00
2022-07-04 03:20:30 +00:00
// Send question
2022-07-05 02:44:04 +00:00
let bytes = message.len() as u64;
2022-06-28 03:46:29 +00:00
let send_ts = intf::get_timestamp();
2022-04-23 01:30:09 +00:00
let send_data_kind = match self
2021-11-22 16:28:30 +00:00
.network_manager()
2022-07-05 02:44:04 +00:00
.send_envelope(node_ref.clone(), Some(node_id), message)
2021-11-22 16:28:30 +00:00
.await
2021-11-27 17:44:21 +00:00
.map_err(RPCError::Internal)
2021-11-22 16:28:30 +00:00
{
2022-04-23 01:30:09 +00:00
Ok(v) => v,
Err(e) => {
// Make sure to clean up op id waiter in case of error
2022-07-04 03:20:30 +00:00
self.cancel_op_id_waiter(op_id);
self.routing_table()
2022-07-04 03:20:30 +00:00
.stats_failed_to_send(node_ref, send_ts, true);
2022-04-23 01:30:09 +00:00
return Err(e);
2021-11-22 16:28:30 +00:00
}
2022-04-23 01:30:09 +00:00
};
2021-11-22 16:28:30 +00:00
// Successfully sent
2022-04-18 22:49:33 +00:00
self.routing_table()
2022-07-04 03:20:30 +00:00
.stats_question_sent(node_ref.clone(), send_ts, bytes, true);
2021-11-22 16:28:30 +00:00
// Pass back waitable reply completion
2022-07-04 03:20:30 +00:00
Ok(WaitableReply {
op_id,
eventual,
timeout,
node_ref,
send_ts,
send_data_kind,
})
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
// Issue a statement over the network, possibly using an anonymized route
#[instrument(level = "debug", skip(self, statement, safety_route_spec), err)]
async fn statement(
2021-11-22 16:28:30 +00:00
&self,
2022-07-04 16:03:21 +00:00
dest: Destination,
statement: RPCStatement,
2021-11-22 16:28:30 +00:00
safety_route_spec: Option<&SafetyRouteSpec>,
) -> Result<(), RPCError> {
2022-07-04 16:03:21 +00:00
// Wrap statement in operation
let operation = RPCOperation::new_statement(statement);
2021-11-22 16:28:30 +00:00
2022-07-04 21:58:26 +00:00
// Log rpc send
debug!(target: "rpc_message", dir = "send", kind = "statement", op_id = operation.op_id(), desc = operation.kind().desc(), ?dest);
2022-07-04 16:03:21 +00:00
// Produce rendered operation
let RenderedOperation {
2022-07-05 02:44:04 +00:00
message,
node_id,
node_ref,
hop_count,
2022-07-04 21:58:26 +00:00
} = self.render_operation(dest, &operation, safety_route_spec)?;
2021-11-22 16:28:30 +00:00
2022-07-04 16:03:21 +00:00
// If we need to resolve the first hop, do it
2022-07-05 02:44:04 +00:00
let node_ref = match node_ref {
2022-07-04 16:03:21 +00:00
None => {
// resolve node
2022-07-05 02:44:04 +00:00
self.resolve_node(node_id)
2022-07-04 16:03:21 +00:00
.await
.map_err(logthru_rpc!(error))?
}
Some(nr) => {
// got the node in the routing table already
nr
}
};
2021-11-22 16:28:30 +00:00
2022-07-05 02:44:04 +00:00
// Calculate answer timeout
// Timeout is number of hops times the timeout per hop
let timeout = self.inner.lock().timeout * (hop_count as u64);
2022-07-04 16:03:21 +00:00
// Send statement
2022-07-05 02:44:04 +00:00
let bytes = message.len() as u64;
2022-07-04 16:03:21 +00:00
let send_ts = intf::get_timestamp();
let send_data_kind = match self
.network_manager()
2022-07-05 02:44:04 +00:00
.send_envelope(node_ref.clone(), Some(node_id), message)
2022-07-04 16:03:21 +00:00
.await
.map_err(RPCError::Internal)
{
Ok(v) => v,
Err(e) => {
self.routing_table()
.stats_failed_to_send(node_ref, send_ts, true);
return Err(e);
2021-11-22 16:28:30 +00:00
}
};
2022-07-04 16:03:21 +00:00
// Successfully sent
self.routing_table()
.stats_question_sent(node_ref.clone(), send_ts, bytes, true);
Ok(())
}
// Convert the 'RespondTo' into a 'Destination' for a response
fn get_respond_to_destination(&self, request: &RPCMessage) -> Destination {
// Get the question 'respond to'
let respond_to = match request.operation.kind() {
RPCOperationKind::Question(q) => q.respond_to(),
_ => {
panic!("not a question");
}
};
// To where should we respond?
match respond_to {
RespondTo::Sender(_) => {
// Reply directly to the request's source
let sender_id = request.header.envelope.get_sender_id();
// This may be a different node's reference than the 'sender' in the case of a relay
let peer_noderef = request.header.peer_noderef.clone();
// If the sender_id is that of the peer, then this is a direct reply
// else it is a relayed reply through the peer
if peer_noderef.node_id() == sender_id {
Destination::Direct(peer_noderef)
} else {
Destination::Relay(peer_noderef, sender_id)
}
}
RespondTo::PrivateRoute(pr) => Destination::PrivateRoute(pr.clone()),
}
}
// Issue a reply over the network, possibly using an anonymized route
// The request must want a response, or this routine fails
#[instrument(level = "debug", skip(self, request, answer, safety_route_spec), err)]
async fn answer(
&self,
request: RPCMessage,
answer: RPCAnswer,
safety_route_spec: Option<&SafetyRouteSpec>,
) -> Result<(), RPCError> {
// Wrap answer in operation
let operation = RPCOperation::new_answer(&request.operation, answer);
// Extract destination from respond_to
let dest = self.get_respond_to_destination(&request);
2022-07-04 21:58:26 +00:00
// Log rpc send
debug!(target: "rpc_message", dir = "send", kind = "answer", op_id = operation.op_id(), desc = operation.kind().desc(), ?dest);
2022-07-04 16:03:21 +00:00
// Produce rendered operation
let RenderedOperation {
2022-07-05 02:44:04 +00:00
message,
node_id,
node_ref,
hop_count,
2022-07-04 21:58:26 +00:00
} = self.render_operation(dest, &operation, safety_route_spec)?;
2022-07-04 16:03:21 +00:00
// If we need to resolve the first hop, do it
2022-07-05 02:44:04 +00:00
let node_ref = match node_ref {
2021-11-22 16:28:30 +00:00
None => {
// resolve node
2022-07-05 02:44:04 +00:00
self.resolve_node(node_id).await?
2021-11-22 16:28:30 +00:00
}
Some(nr) => {
// got the node in the routing table already
nr
}
};
// Send the reply
2022-07-05 02:44:04 +00:00
let bytes = message.len() as u64;
2022-06-28 03:46:29 +00:00
let send_ts = intf::get_timestamp();
2021-11-22 16:28:30 +00:00
self.network_manager()
2022-07-05 02:44:04 +00:00
.send_envelope(node_ref.clone(), Some(node_id), message)
2021-11-22 16:28:30 +00:00
.await
.map_err(RPCError::Internal)
.map_err(|e| {
self.routing_table()
.stats_failed_to_send(node_ref.clone(), send_ts, false);
e
})?;
2021-11-22 16:28:30 +00:00
// Reply successfully sent
self.routing_table().stats_answer_sent(node_ref, bytes);
2021-11-22 16:28:30 +00:00
Ok(())
}
2022-07-03 19:52:27 +00:00
async fn generate_sender_info(peer_noderef: NodeRef) -> SenderInfo {
2022-04-17 23:10:10 +00:00
let socket_address = peer_noderef
.last_connection()
.await
2022-06-05 17:23:18 +00:00
.map(|c| c.remote_address().clone());
SenderInfo { socket_address }
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
//////////////////////////////////////////////////////////////////////
async fn process_rpc_message_version_0(
2021-11-22 16:28:30 +00:00
&self,
2022-07-04 16:03:21 +00:00
encoded_msg: RPCMessageEncoded,
2021-11-22 16:28:30 +00:00
) -> Result<(), RPCError> {
2022-07-04 21:58:26 +00:00
// Decode the operation
2022-07-04 16:03:21 +00:00
let sender_node_id = encoded_msg.header.envelope.get_sender_id();
2022-05-26 00:56:13 +00:00
2022-07-04 16:03:21 +00:00
// Decode the RPC message
2022-07-04 21:58:26 +00:00
let operation = {
let reader = capnp::message::Reader::new(encoded_msg.data, Default::default());
let op_reader = reader
.get_root::<veilid_capnp::operation::Reader>()
.map_err(map_error_capnp_error!())
.map_err(logthru_rpc!())?;
RPCOperation::decode(&op_reader, &sender_node_id)?
};
2022-05-26 00:56:13 +00:00
2022-07-04 16:03:21 +00:00
// Get the sender noderef, incorporating and 'sender node info' we have from a question
let mut opt_sender_nr: Option<NodeRef> = None;
match operation.kind() {
RPCOperationKind::Question(q) => {
match q.respond_to() {
RespondTo::Sender(Some(sender_ni)) => {
// Sender NodeInfo was specified, update our routing table with it
if !self.filter_peer_scope(&sender_ni.node_info) {
return Err(rpc_error_invalid_format(
"respond_to_sender_signed_node_info has invalid peer scope",
));
}
let nr = self
.routing_table()
.register_node_with_signed_node_info(sender_node_id, sender_ni.clone())
.map_err(RPCError::Internal)?;
opt_sender_nr = Some(nr);
2021-12-24 01:34:52 +00:00
}
2022-07-04 16:03:21 +00:00
_ => {}
2021-12-24 01:34:52 +00:00
}
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
_ => {}
2021-11-22 16:28:30 +00:00
};
2022-07-04 16:03:21 +00:00
if opt_sender_nr.is_none() {
// look up sender node, in case it's different than our peer due to relaying
opt_sender_nr = self.routing_table().lookup_node_ref(sender_node_id)
2022-05-11 01:49:42 +00:00
}
2021-11-22 16:28:30 +00:00
2022-07-04 16:03:21 +00:00
// Make the RPC message
let msg = RPCMessage {
header: encoded_msg.header,
operation,
opt_sender_nr,
2022-04-17 23:10:10 +00:00
};
2022-07-04 16:03:21 +00:00
// Process stats
let kind = match msg.operation.kind() {
RPCOperationKind::Question(_) => {
if let Some(sender_nr) = msg.opt_sender_nr.clone() {
self.routing_table().stats_question_rcvd(
sender_nr,
msg.header.timestamp,
msg.header.body_len,
);
}
"question"
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
RPCOperationKind::Statement(_) => {
if let Some(sender_nr) = msg.opt_sender_nr.clone() {
2022-04-18 22:49:33 +00:00
self.routing_table().stats_question_rcvd(
sender_nr,
msg.header.timestamp,
msg.header.body_len,
);
2021-11-22 16:28:30 +00:00
}
2022-07-04 16:03:21 +00:00
"statement"
}
RPCOperationKind::Answer(_) => {
// Answer stats are processed in wait_for_reply
"answer"
}
};
2022-07-04 16:03:21 +00:00
// Log rpc receive
debug!(target: "rpc_message", dir = "recv", kind, op_id = msg.operation.op_id(), desc = msg.operation.kind().desc(), sender_id = ?sender_node_id);
// Process specific message kind
match msg.operation.kind() {
RPCOperationKind::Question(q) => match q.detail() {
RPCQuestionDetail::StatusQ(_) => self.process_status_q(msg).await,
RPCQuestionDetail::FindNodeQ(_) => self.process_find_node_q(msg).await,
RPCQuestionDetail::GetValueQ(_) => self.process_get_value_q(msg).await,
RPCQuestionDetail::SetValueQ(_) => self.process_set_value_q(msg).await,
RPCQuestionDetail::WatchValueQ(_) => self.process_watch_value_q(msg).await,
RPCQuestionDetail::SupplyBlockQ(_) => self.process_supply_block_q(msg).await,
RPCQuestionDetail::FindBlockQ(_) => self.process_find_block_q(msg).await,
RPCQuestionDetail::StartTunnelQ(_) => self.process_start_tunnel_q(msg).await,
RPCQuestionDetail::CompleteTunnelQ(_) => self.process_complete_tunnel_q(msg).await,
RPCQuestionDetail::CancelTunnelQ(_) => self.process_cancel_tunnel_q(msg).await,
},
RPCOperationKind::Statement(s) => match s.detail() {
RPCStatementDetail::ValidateDialInfo(_) => {
self.process_validate_dial_info(msg).await
}
RPCStatementDetail::Route(_) => self.process_route(msg).await,
RPCStatementDetail::NodeInfoUpdate(_) => self.process_node_info_update(msg).await,
RPCStatementDetail::ValueChanged(_) => self.process_value_changed(msg).await,
RPCStatementDetail::Signal(_) => self.process_signal(msg).await,
RPCStatementDetail::ReturnReceipt(_) => self.process_return_receipt(msg).await,
},
RPCOperationKind::Answer(a) => self.complete_op_id_waiter(msg).await,
2021-11-22 16:28:30 +00:00
}
}
2022-07-04 16:03:21 +00:00
async fn process_rpc_message(&self, msg: RPCMessageEncoded) -> Result<(), RPCError> {
2021-11-22 16:28:30 +00:00
if msg.header.envelope.get_version() == 0 {
self.process_rpc_message_version_0(msg).await
} else {
Err(RPCError::Internal(format!(
"unsupported envelope version: {}, newest supported is version 0",
msg.header.envelope.get_version()
)))
}
}
2022-07-04 16:03:21 +00:00
async fn rpc_worker(self, stop_token: StopToken, receiver: flume::Receiver<RPCMessageEncoded>) {
2022-06-13 00:58:02 +00:00
while let Ok(Ok(msg)) = receiver.recv_async().timeout_at(stop_token.clone()).await {
2021-12-17 02:57:28 +00:00
let _ = self
.process_rpc_message(msg)
.await
.map_err(logthru_rpc!("couldn't process rpc message"));
2021-11-22 16:28:30 +00:00
}
}
2022-07-03 19:52:27 +00:00
#[instrument(level = "debug", skip_all, err)]
2021-11-22 16:28:30 +00:00
pub async fn startup(&self) -> Result<(), String> {
2022-03-10 14:51:53 +00:00
trace!("startup rpc processor");
2021-11-22 16:28:30 +00:00
let mut inner = self.inner.lock();
// make local copy of node id for easy access
let c = self.config.get();
inner.node_id = c.network.node_id;
inner.node_id_secret = c.network.node_id_secret;
// set up channel
let mut concurrency = c.network.rpc.concurrency;
let mut queue_size = c.network.rpc.queue_size;
2022-01-27 14:53:01 +00:00
let mut timeout = ms_to_us(c.network.rpc.timeout_ms);
2021-11-22 16:28:30 +00:00
let mut max_route_hop_count = c.network.rpc.max_route_hop_count as usize;
if concurrency == 0 {
2022-06-28 03:46:29 +00:00
concurrency = intf::get_concurrency() / 2;
2021-11-22 16:28:30 +00:00
if concurrency == 0 {
concurrency = 1;
}
}
if queue_size == 0 {
queue_size = 1024;
}
if timeout == 0 {
timeout = 10000000;
}
if max_route_hop_count == 0 {
max_route_hop_count = 7usize;
}
inner.timeout = timeout;
inner.max_route_hop_count = max_route_hop_count;
2022-03-10 15:18:47 +00:00
let channel = flume::bounded(queue_size as usize);
2021-12-17 02:57:28 +00:00
inner.send_channel = Some(channel.0.clone());
2022-06-13 00:58:02 +00:00
inner.stop_source = Some(StopSource::new());
2021-11-22 16:28:30 +00:00
// spin up N workers
trace!("Spinning up {} RPC workers", concurrency);
for _ in 0..concurrency {
let this = self.clone();
let receiver = channel.1.clone();
2022-07-04 16:03:21 +00:00
let jh = intf::spawn(Self::rpc_worker(
this,
inner.stop_source.as_ref().unwrap().token(),
receiver,
));
2022-06-28 03:46:29 +00:00
inner.worker_join_handles.push(jh);
2021-11-22 16:28:30 +00:00
}
Ok(())
}
2022-07-03 19:52:27 +00:00
#[instrument(level = "debug", skip_all)]
2021-11-22 16:28:30 +00:00
pub async fn shutdown(&self) {
2022-06-15 18:05:04 +00:00
debug!("starting rpc processor shutdown");
2022-07-04 16:03:21 +00:00
2022-06-13 00:58:02 +00:00
// Stop the rpc workers
let mut unord = FuturesUnordered::new();
{
let mut inner = self.inner.lock();
// take the join handles out
for h in inner.worker_join_handles.drain(..) {
unord.push(h);
}
2022-06-15 18:05:04 +00:00
// drop the stop
drop(inner.stop_source.take());
2022-06-13 00:58:02 +00:00
}
2022-06-15 18:05:04 +00:00
debug!("stopping {} rpc worker tasks", unord.len());
2022-07-04 16:03:21 +00:00
2022-06-13 00:58:02 +00:00
// Wait for them to complete
while unord.next().await.is_some() {}
2022-07-04 16:03:21 +00:00
2022-06-15 18:05:04 +00:00
debug!("resetting rpc processor state");
2022-07-04 16:03:21 +00:00
2022-06-13 00:58:02 +00:00
// Release the rpc processor
2021-11-22 16:28:30 +00:00
*self.inner.lock() = Self::new_inner(self.network_manager());
2022-06-15 18:05:04 +00:00
debug!("finished rpc processor shutdown");
2021-11-22 16:28:30 +00:00
}
2022-01-05 17:01:02 +00:00
pub fn enqueue_message(
2021-11-22 16:28:30 +00:00
&self,
2022-05-31 23:54:52 +00:00
envelope: Envelope,
2021-11-22 16:28:30 +00:00
body: Vec<u8>,
peer_noderef: NodeRef,
2021-12-17 02:57:28 +00:00
) -> Result<(), String> {
2022-07-04 16:03:21 +00:00
let msg = RPCMessageEncoded {
2021-11-22 16:28:30 +00:00
header: RPCMessageHeader {
2022-06-28 03:46:29 +00:00
timestamp: intf::get_timestamp(),
envelope,
2021-11-22 16:28:30 +00:00
body_len: body.len() as u64,
peer_noderef,
2021-11-22 16:28:30 +00:00
},
data: RPCMessageData { contents: body },
};
let send_channel = {
let inner = self.inner.lock();
2021-12-17 02:57:28 +00:00
inner.send_channel.as_ref().unwrap().clone()
2021-11-22 16:28:30 +00:00
};
2021-12-17 02:57:28 +00:00
send_channel
.try_send(msg)
.map_err(|e| format!("failed to enqueue received RPC message: {:?}", e))?;
2021-11-22 16:28:30 +00:00
Ok(())
}
}