Propagate RPC through network service.

- Basic network message handler threading
- Correct references
This commit is contained in:
Age Manning 2019-03-17 21:49:56 +11:00
parent 7370306366
commit 9803ab30f2
No known key found for this signature in database
GPG Key ID: 05EED64B79E06A93
11 changed files with 68 additions and 60 deletions

View File

@ -34,7 +34,7 @@ impl<T: ClientTypes> Client<T> {
pub fn new( pub fn new(
config: ClientConfig, config: ClientConfig,
log: slog::Logger, log: slog::Logger,
executor: TaskExecutor, executor: &TaskExecutor,
) -> error::Result<Self> { ) -> error::Result<Self> {
let (exit_signal, exit) = exit_future::signal(); let (exit_signal, exit) = exit_future::signal();

View File

@ -1,4 +1,4 @@
use crate::rpc::{RPCMethod, RPCRequest, RPCResponse, Rpc, RpcEvent}; use crate::rpc::{Rpc, RpcEvent};
use futures::prelude::*; use futures::prelude::*;
use libp2p::{ use libp2p::{
core::swarm::{NetworkBehaviourAction, NetworkBehaviourEventProcess}, core::swarm::{NetworkBehaviourAction, NetworkBehaviourEventProcess},
@ -42,22 +42,7 @@ impl<TSubstream: AsyncRead + AsyncWrite> NetworkBehaviourEventProcess<RpcEvent>
for Behaviour<TSubstream> for Behaviour<TSubstream>
{ {
fn inject_event(&mut self, event: RpcEvent) { fn inject_event(&mut self, event: RpcEvent) {
match event { self.events.push(BehaviourEvent::RPC(event));
RpcEvent::Request {
id,
method_id,
body,
} => self.events.push(BehaviourEvent::RPCRequest {
id,
method: RPCMethod::from(method_id),
body,
}),
RpcEvent::Response {
id,
method_id,
result,
} => self.events.push(BehaviourEvent::RPCResponse { id, result }),
}
} }
} }
@ -95,15 +80,7 @@ impl<TSubstream: AsyncRead + AsyncWrite> Behaviour<TSubstream> {
/// The types of events than can be obtained from polling the behaviour. /// The types of events than can be obtained from polling the behaviour.
pub enum BehaviourEvent { pub enum BehaviourEvent {
RPCRequest { RPC(RpcEvent),
id: u64,
method: RPCMethod,
body: RPCRequest,
},
RPCResponse {
id: u64,
result: RPCResponse,
},
// TODO: This is a stub at the moment // TODO: This is a stub at the moment
Message(String), Message(String),
} }

View File

@ -5,7 +5,7 @@
pub mod behaviour; pub mod behaviour;
pub mod error; pub mod error;
mod network_config; mod network_config;
mod rpc; pub mod rpc;
mod service; mod service;
pub use libp2p::{ pub use libp2p::{
@ -13,6 +13,8 @@ pub use libp2p::{
PeerId, PeerId,
}; };
pub use network_config::NetworkConfig; pub use network_config::NetworkConfig;
pub use rpc::HelloMessage;
pub use rpc::RpcEvent;
pub use service::Libp2pEvent; pub use service::Libp2pEvent;
pub use service::Service; pub use service::Service;
pub use types::multiaddr; pub use types::multiaddr;

View File

@ -19,17 +19,17 @@ impl From<u16> for RPCMethod {
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum RPCRequest { pub enum RPCRequest {
Hello(HelloBody), Hello(HelloMessage),
} }
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum RPCResponse { pub enum RPCResponse {
Hello(HelloBody), Hello(HelloMessage),
} }
// request/response structs for RPC methods // request/response structs for RPC methods
#[derive(Encode, Decode, Clone, Debug)] #[derive(Encode, Decode, Clone, Debug)]
pub struct HelloBody { pub struct HelloMessage {
pub network_id: u8, pub network_id: u8,
pub latest_finalized_root: Hash256, pub latest_finalized_root: Hash256,
pub latest_finalized_epoch: Epoch, pub latest_finalized_epoch: Epoch,

View File

@ -11,7 +11,7 @@ use libp2p::core::swarm::{
ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters, ConnectedPoint, NetworkBehaviour, NetworkBehaviourAction, PollParameters,
}; };
use libp2p::{Multiaddr, PeerId}; use libp2p::{Multiaddr, PeerId};
pub use methods::{RPCMethod, RPCRequest, RPCResponse}; pub use methods::{HelloMessage, RPCMethod, RPCRequest, RPCResponse};
pub use protocol::{RPCProtocol, RpcEvent}; pub use protocol::{RPCProtocol, RpcEvent};
use std::marker::PhantomData; use std::marker::PhantomData;
use tokio::io::{AsyncRead, AsyncWrite}; use tokio::io::{AsyncRead, AsyncWrite};

View File

@ -1,4 +1,4 @@
use super::methods::{HelloBody, RPCMethod, RPCRequest, RPCResponse}; use super::methods::{HelloMessage, RPCMethod, RPCRequest, RPCResponse};
use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, UpgradeInfo}; use libp2p::core::{upgrade, InboundUpgrade, OutboundUpgrade, UpgradeInfo};
use ssz::{ssz_encode, Decodable, Encodable, SszStream}; use ssz::{ssz_encode, Decodable, Encodable, SszStream};
use std::io; use std::io;
@ -78,7 +78,7 @@ fn decode(packet: Vec<u8>) -> Result<RpcEvent, DecodeError> {
if request { if request {
let body = match RPCMethod::from(method_id) { let body = match RPCMethod::from(method_id) {
RPCMethod::Hello => { RPCMethod::Hello => {
let (hello_body, _index) = HelloBody::ssz_decode(&packet, index)?; let (hello_body, _index) = HelloMessage::ssz_decode(&packet, index)?;
RPCRequest::Hello(hello_body) RPCRequest::Hello(hello_body)
} }
RPCMethod::Unknown => return Err(DecodeError::UnknownRPCMethod), RPCMethod::Unknown => return Err(DecodeError::UnknownRPCMethod),
@ -94,7 +94,7 @@ fn decode(packet: Vec<u8>) -> Result<RpcEvent, DecodeError> {
else { else {
let result = match RPCMethod::from(method_id) { let result = match RPCMethod::from(method_id) {
RPCMethod::Hello => { RPCMethod::Hello => {
let (body, _index) = HelloBody::ssz_decode(&packet, index)?; let (body, _index) = HelloMessage::ssz_decode(&packet, index)?;
RPCResponse::Hello(body) RPCResponse::Hello(body)
} }
RPCMethod::Unknown => return Err(DecodeError::UnknownRPCMethod), RPCMethod::Unknown => return Err(DecodeError::UnknownRPCMethod),

View File

@ -1,6 +1,7 @@
use crate::behaviour::{Behaviour, BehaviourEvent}; use crate::behaviour::{Behaviour, BehaviourEvent};
use crate::error; use crate::error;
use crate::multiaddr::Protocol; use crate::multiaddr::Protocol;
use crate::rpc::RpcEvent;
use crate::NetworkConfig; use crate::NetworkConfig;
use futures::prelude::*; use futures::prelude::*;
use futures::Stream; use futures::Stream;
@ -104,8 +105,9 @@ impl Stream for Service {
debug!(self.log, "Message received: {}", m); debug!(self.log, "Message received: {}", m);
return Ok(Async::Ready(Some(Libp2pEvent::Message(m)))); return Ok(Async::Ready(Some(Libp2pEvent::Message(m))));
} }
// TODO: Fill with all behaviour events Ok(Async::Ready(Some(BehaviourEvent::RPC(event)))) => {
_ => break, return Ok(Async::Ready(Some(Libp2pEvent::RPC(event))));
}
Ok(Async::Ready(None)) => unreachable!("Swarm stream shouldn't end"), Ok(Async::Ready(None)) => unreachable!("Swarm stream shouldn't end"),
Ok(Async::NotReady) => break, Ok(Async::NotReady) => break,
_ => break, _ => break,
@ -152,5 +154,7 @@ fn build_transport(
/// Events that can be obtained from polling the Libp2p Service. /// Events that can be obtained from polling the Libp2p Service.
pub enum Libp2pEvent { pub enum Libp2pEvent {
// We have received an RPC event on the swarm
RPC(RpcEvent),
Message(String), Message(String),
} }

View File

@ -1,7 +1,10 @@
use crate::error; use crate::error;
use crate::messages::NodeMessage; use crate::messages::NodeMessage;
use crossbeam_channel::{unbounded as channel, Sender}; use crossbeam_channel::{unbounded as channel, Sender, TryRecvError};
use libp2p::PeerId; use futures::future;
use futures::prelude::*;
use libp2p::rpc;
use libp2p::{PeerId, RpcEvent};
use slog::debug; use slog::debug;
use sync::SimpleSync; use sync::SimpleSync;
use types::Hash256; use types::Hash256;
@ -11,9 +14,11 @@ pub struct MessageHandler {
sync: SimpleSync, sync: SimpleSync,
//TODO: Implement beacon chain //TODO: Implement beacon chain
//chain: BeaconChain //chain: BeaconChain
log: slog::Logger,
} }
/// Types of messages the handler can receive. /// Types of messages the handler can receive.
#[derive(Debug, Clone)]
pub enum HandlerMessage { pub enum HandlerMessage {
/// Peer has connected. /// Peer has connected.
PeerConnected(PeerId), PeerConnected(PeerId),
@ -21,11 +26,16 @@ pub enum HandlerMessage {
PeerDisconnected(PeerId), PeerDisconnected(PeerId),
/// A Node message has been received. /// A Node message has been received.
Message(PeerId, NodeMessage), Message(PeerId, NodeMessage),
/// An RPC response/request has been received.
RPC(RpcEvent),
} }
impl MessageHandler { impl MessageHandler {
/// Initializes and runs the MessageHandler. /// Initializes and runs the MessageHandler.
pub fn new(log: slog::Logger) -> error::Result<Sender<HandlerMessage>> { pub fn new(
executor: &tokio::runtime::TaskExecutor,
log: slog::Logger,
) -> error::Result<Sender<HandlerMessage>> {
debug!(log, "Service starting"); debug!(log, "Service starting");
let (handler_send, handler_recv) = channel(); let (handler_send, handler_recv) = channel();
@ -33,12 +43,29 @@ impl MessageHandler {
// Initialise sync and begin processing in thread // Initialise sync and begin processing in thread
//TODO: Load genesis from BeaconChain //TODO: Load genesis from BeaconChain
let temp_genesis = Hash256::zero(); let temp_genesis = Hash256::zero();
// generate the Message handler
let sync = SimpleSync::new(temp_genesis); let sync = SimpleSync::new(temp_genesis);
//TODO: Initialise beacon chain
let mut handler = MessageHandler {
sync,
log: log.clone(),
};
let handler = MessageHandler { sync }; // spawn handler task
// TODO: Handle manual termination of thread
// spawn handler thread executor.spawn(future::poll_fn(move || -> Result<_, _> {
loop {
handler.handle_message(handler_recv.recv().map_err(|_| {
debug!(log, "Handler channel closed. Handler terminating");
})?);
}
}));
Ok(handler_send) Ok(handler_send)
} }
fn handle_message(&mut self, message: HandlerMessage) {
debug!(self.log, "Message received {:?}", message);
}
} }

View File

@ -1,27 +1,16 @@
use libp2p::PeerId; use libp2p::PeerId;
use libp2p::{HelloMessage, RpcEvent};
use types::{Hash256, Slot}; use types::{Hash256, Slot};
/// Messages between nodes across the network. /// Messages between nodes across the network.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum NodeMessage { pub enum NodeMessage {
Status(Status), RPC(RpcEvent),
BlockRequest, BlockRequest,
// TODO: only for testing - remove // TODO: only for testing - remove
Message(String), Message(String),
} }
#[derive(Debug, Clone)]
pub struct Status {
/// Current node version.
version: u8,
/// Genesis Hash.
genesis_hash: Hash256,
/// Best known slot number.
best_slot: Slot,
/// Best known slot hash.
best_slot_hash: Hash256,
}
/// Types of messages that the network service can receive. /// Types of messages that the network service can receive.
#[derive(Debug, Clone)] #[derive(Debug, Clone)]
pub enum NetworkMessage { pub enum NetworkMessage {

View File

@ -28,12 +28,12 @@ pub struct Service {
impl Service { impl Service {
pub fn new( pub fn new(
config: NetworkConfig, config: NetworkConfig,
executor: TaskExecutor, executor: &TaskExecutor,
log: slog::Logger, log: slog::Logger,
) -> error::Result<(Arc<Self>, Sender<NetworkMessage>)> { ) -> error::Result<(Arc<Self>, Sender<NetworkMessage>)> {
// launch message handler thread // launch message handler thread
let message_handler_log = log.new(o!("Service" => "MessageHandler")); let message_handler_log = log.new(o!("Service" => "MessageHandler"));
let message_handler_send = MessageHandler::new(message_handler_log)?; let message_handler_send = MessageHandler::new(executor, message_handler_log)?;
// launch libp2p service // launch libp2p service
let libp2p_log = log.new(o!("Service" => "Libp2p")); let libp2p_log = log.new(o!("Service" => "Libp2p"));
@ -61,7 +61,7 @@ impl Service {
fn spawn_service( fn spawn_service(
libp2p_service: LibP2PService, libp2p_service: LibP2PService,
message_handler_send: crossbeam_channel::Sender<HandlerMessage>, message_handler_send: crossbeam_channel::Sender<HandlerMessage>,
executor: TaskExecutor, executor: &TaskExecutor,
log: slog::Logger, log: slog::Logger,
) -> error::Result<( ) -> error::Result<(
crossbeam_channel::Sender<NetworkMessage>, crossbeam_channel::Sender<NetworkMessage>,
@ -99,6 +99,15 @@ fn network_service(
// poll the swarm // poll the swarm
loop { loop {
match libp2p_service.poll() { match libp2p_service.poll() {
Ok(Async::Ready(Some(Libp2pEvent::RPC(rpc_event)))) => {
debug!(
libp2p_service.log,
"RPC Event: Rpc message received: {:?}", rpc_event
);
message_handler_send
.send(HandlerMessage::RPC(rpc_event))
.map_err(|_| "failed to send rpc to handler");
}
Ok(Async::Ready(Some(Libp2pEvent::Message(m)))) => debug!( Ok(Async::Ready(Some(Libp2pEvent::Message(m)))) => debug!(
libp2p_service.log, libp2p_service.log,
"Network Service: Message received: {}", m "Network Service: Message received: {}", m

View File

@ -32,7 +32,7 @@ pub fn run_beacon_node(config: ClientConfig, log: slog::Logger) -> error::Result
let executor = runtime.executor(); let executor = runtime.executor();
// currently testing - using TestingNode type // currently testing - using TestingNode type
let client: Client<TestingClientType> = Client::new(config, log.clone(), executor.clone())?; let client: Client<TestingClientType> = Client::new(config, log.clone(), &executor)?;
notifier::run(&client, executor, exit); notifier::run(&client, executor, exit);
runtime.block_on(ctrlc); runtime.block_on(ctrlc);