Re-worked TMTC modules
All checks were successful
Rust/sat-rs/pipeline/pr-main This commit looks good
All checks were successful
Rust/sat-rs/pipeline/pr-main This commit looks good
This commit is contained in:
@ -10,12 +10,13 @@ use std::net::SocketAddr;
|
||||
use std::vec::Vec;
|
||||
|
||||
use crate::encoding::parse_buffer_for_cobs_encoded_packets;
|
||||
use crate::tmtc::ReceivesTc;
|
||||
use crate::tmtc::TmPacketSource;
|
||||
use crate::tmtc::PacketSenderRaw;
|
||||
use crate::tmtc::PacketSource;
|
||||
|
||||
use crate::hal::std::tcp_server::{
|
||||
ConnectionResult, ServerConfig, TcpTcParser, TcpTmSender, TcpTmtcError, TcpTmtcGenericServer,
|
||||
};
|
||||
use crate::ComponentId;
|
||||
|
||||
use super::tcp_server::HandledConnectionHandler;
|
||||
use super::tcp_server::HandledConnectionInfo;
|
||||
@ -28,14 +29,16 @@ impl<TmError, TcError: 'static> TcpTcParser<TmError, TcError> for CobsTcParser {
|
||||
fn handle_tc_parsing(
|
||||
&mut self,
|
||||
tc_buffer: &mut [u8],
|
||||
tc_receiver: &mut (impl ReceivesTc<Error = TcError> + ?Sized),
|
||||
sender_id: ComponentId,
|
||||
tc_sender: &(impl PacketSenderRaw<Error = TcError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
current_write_idx: usize,
|
||||
next_write_idx: &mut usize,
|
||||
) -> Result<(), TcpTmtcError<TmError, TcError>> {
|
||||
conn_result.num_received_tcs += parse_buffer_for_cobs_encoded_packets(
|
||||
&mut tc_buffer[..current_write_idx],
|
||||
tc_receiver.upcast_mut(),
|
||||
sender_id,
|
||||
tc_sender,
|
||||
next_write_idx,
|
||||
)
|
||||
.map_err(|e| TcpTmtcError::TcError(e))?;
|
||||
@ -62,7 +65,7 @@ impl<TmError, TcError> TcpTmSender<TmError, TcError> for CobsTmSender {
|
||||
fn handle_tm_sending(
|
||||
&mut self,
|
||||
tm_buffer: &mut [u8],
|
||||
tm_source: &mut (impl TmPacketSource<Error = TmError> + ?Sized),
|
||||
tm_source: &mut (impl PacketSource<Error = TmError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
stream: &mut TcpStream,
|
||||
) -> Result<bool, TcpTmtcError<TmError, TcError>> {
|
||||
@ -101,7 +104,7 @@ impl<TmError, TcError> TcpTmSender<TmError, TcError> for CobsTmSender {
|
||||
/// Telemetry will be encoded with the COBS protocol using [cobs::encode] in addition to being
|
||||
/// wrapped with the sentinel value 0 as the packet delimiter as well before being sent back to
|
||||
/// the client. Please note that the server will send as much data as it can retrieve from the
|
||||
/// [TmPacketSource] in its current implementation.
|
||||
/// [PacketSource] in its current implementation.
|
||||
///
|
||||
/// Using a framing protocol like COBS imposes minimal restrictions on the type of TMTC data
|
||||
/// exchanged while also allowing packets with flexible size and a reliable way to reconstruct full
|
||||
@ -115,26 +118,26 @@ impl<TmError, TcError> TcpTmSender<TmError, TcError> for CobsTmSender {
|
||||
/// The [TCP integration tests](https://egit.irs.uni-stuttgart.de/rust/sat-rs/src/branch/main/satrs/tests/tcp_servers.rs)
|
||||
/// test also serves as the example application for this module.
|
||||
pub struct TcpTmtcInCobsServer<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcSender: PacketSenderRaw<Error = SendError>,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError,
|
||||
TcError: 'static,
|
||||
SendError: 'static,
|
||||
> {
|
||||
pub generic_server: TcpTmtcGenericServer<
|
||||
TmSource,
|
||||
TcReceiver,
|
||||
TcSender,
|
||||
CobsTmSender,
|
||||
CobsTcParser,
|
||||
HandledConnection,
|
||||
TmError,
|
||||
TcError,
|
||||
SendError,
|
||||
>,
|
||||
}
|
||||
|
||||
impl<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcReceiver: PacketSenderRaw<Error = TcError>,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError: 'static,
|
||||
TcError: 'static,
|
||||
@ -178,8 +181,8 @@ impl<
|
||||
/// useful if using the port number 0 for OS auto-assignment.
|
||||
pub fn local_addr(&self) -> std::io::Result<SocketAddr>;
|
||||
|
||||
/// Delegation to the [TcpTmtcGenericServer::handle_next_connection] call.
|
||||
pub fn handle_next_connection(
|
||||
/// Delegation to the [TcpTmtcGenericServer::handle_all_connections] call.
|
||||
pub fn handle_all_connections(
|
||||
&mut self,
|
||||
poll_duration: Option<Duration>,
|
||||
) -> Result<ConnectionResult, TcpTmtcError<TmError, TcError>>;
|
||||
@ -196,22 +199,29 @@ mod tests {
|
||||
use std::{
|
||||
io::{Read, Write},
|
||||
net::{IpAddr, Ipv4Addr, SocketAddr, TcpStream},
|
||||
panic, thread,
|
||||
panic,
|
||||
sync::mpsc,
|
||||
thread,
|
||||
time::Instant,
|
||||
};
|
||||
|
||||
use crate::{
|
||||
encoding::tests::{INVERTED_PACKET, SIMPLE_PACKET},
|
||||
hal::std::tcp_server::{
|
||||
tests::{ConnectionFinishedHandler, SyncTcCacher, SyncTmSource},
|
||||
tests::{ConnectionFinishedHandler, SyncTmSource},
|
||||
ConnectionResult, ServerConfig,
|
||||
},
|
||||
queue::GenericSendError,
|
||||
tmtc::PacketAsVec,
|
||||
ComponentId,
|
||||
};
|
||||
use alloc::sync::Arc;
|
||||
use cobs::encode;
|
||||
|
||||
use super::TcpTmtcInCobsServer;
|
||||
|
||||
const TCP_SERVER_ID: ComponentId = 0x05;
|
||||
|
||||
fn encode_simple_packet(encoded_buf: &mut [u8], current_idx: &mut usize) {
|
||||
encode_packet(&SIMPLE_PACKET, encoded_buf, current_idx)
|
||||
}
|
||||
@ -230,14 +240,20 @@ mod tests {
|
||||
|
||||
fn generic_tmtc_server(
|
||||
addr: &SocketAddr,
|
||||
tc_receiver: SyncTcCacher,
|
||||
tc_sender: mpsc::Sender<PacketAsVec>,
|
||||
tm_source: SyncTmSource,
|
||||
stop_signal: Option<Arc<AtomicBool>>,
|
||||
) -> TcpTmtcInCobsServer<SyncTmSource, SyncTcCacher, ConnectionFinishedHandler, (), ()> {
|
||||
) -> TcpTmtcInCobsServer<
|
||||
SyncTmSource,
|
||||
mpsc::Sender<PacketAsVec>,
|
||||
ConnectionFinishedHandler,
|
||||
(),
|
||||
GenericSendError,
|
||||
> {
|
||||
TcpTmtcInCobsServer::new(
|
||||
ServerConfig::new(*addr, Duration::from_millis(2), 1024, 1024),
|
||||
ServerConfig::new(TCP_SERVER_ID, *addr, Duration::from_millis(2), 1024, 1024),
|
||||
tm_source,
|
||||
tc_receiver,
|
||||
tc_sender,
|
||||
ConnectionFinishedHandler::default(),
|
||||
stop_signal,
|
||||
)
|
||||
@ -247,10 +263,10 @@ mod tests {
|
||||
#[test]
|
||||
fn test_server_basic_no_tm() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, tc_receiver) = mpsc::channel();
|
||||
let tm_source = SyncTmSource::default();
|
||||
let mut tcp_server =
|
||||
generic_tmtc_server(&auto_port_addr, tc_receiver.clone(), tm_source, None);
|
||||
generic_tmtc_server(&auto_port_addr, tc_sender.clone(), tm_source, None);
|
||||
let dest_addr = tcp_server
|
||||
.local_addr()
|
||||
.expect("retrieving dest addr failed");
|
||||
@ -258,7 +274,7 @@ mod tests {
|
||||
let set_if_done = conn_handled.clone();
|
||||
// Call the connection handler in separate thread, does block.
|
||||
thread::spawn(move || {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(100)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(100)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
@ -293,28 +309,20 @@ mod tests {
|
||||
panic!("connection was not handled properly");
|
||||
}
|
||||
// Check that the packet was received and decoded successfully.
|
||||
let mut tc_queue = tc_receiver
|
||||
.tc_queue
|
||||
.lock()
|
||||
.expect("locking tc queue failed");
|
||||
assert_eq!(tc_queue.len(), 1);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), &SIMPLE_PACKET);
|
||||
drop(tc_queue);
|
||||
let packet_with_sender = tc_receiver.recv().expect("receiving TC failed");
|
||||
assert_eq!(packet_with_sender.packet, &SIMPLE_PACKET);
|
||||
matches!(tc_receiver.try_recv(), Err(mpsc::TryRecvError::Empty));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_server_basic_multi_tm_multi_tc() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, tc_receiver) = mpsc::channel();
|
||||
let mut tm_source = SyncTmSource::default();
|
||||
tm_source.add_tm(&INVERTED_PACKET);
|
||||
tm_source.add_tm(&SIMPLE_PACKET);
|
||||
let mut tcp_server = generic_tmtc_server(
|
||||
&auto_port_addr,
|
||||
tc_receiver.clone(),
|
||||
tm_source.clone(),
|
||||
None,
|
||||
);
|
||||
let mut tcp_server =
|
||||
generic_tmtc_server(&auto_port_addr, tc_sender.clone(), tm_source.clone(), None);
|
||||
let dest_addr = tcp_server
|
||||
.local_addr()
|
||||
.expect("retrieving dest addr failed");
|
||||
@ -322,7 +330,7 @@ mod tests {
|
||||
let set_if_done = conn_handled.clone();
|
||||
// Call the connection handler in separate thread, does block.
|
||||
thread::spawn(move || {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(100)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(100)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
@ -409,27 +417,26 @@ mod tests {
|
||||
panic!("connection was not handled properly");
|
||||
}
|
||||
// Check that the packet was received and decoded successfully.
|
||||
let mut tc_queue = tc_receiver
|
||||
.tc_queue
|
||||
.lock()
|
||||
.expect("locking tc queue failed");
|
||||
assert_eq!(tc_queue.len(), 2);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), &SIMPLE_PACKET);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), &INVERTED_PACKET);
|
||||
drop(tc_queue);
|
||||
let packet_with_sender = tc_receiver.recv().expect("receiving TC failed");
|
||||
let packet = &packet_with_sender.packet;
|
||||
assert_eq!(packet, &SIMPLE_PACKET);
|
||||
let packet_with_sender = tc_receiver.recv().expect("receiving TC failed");
|
||||
let packet = &packet_with_sender.packet;
|
||||
assert_eq!(packet, &INVERTED_PACKET);
|
||||
matches!(tc_receiver.try_recv(), Err(mpsc::TryRecvError::Empty));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_server_accept_timeout() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, _tc_receiver) = mpsc::channel();
|
||||
let tm_source = SyncTmSource::default();
|
||||
let mut tcp_server =
|
||||
generic_tmtc_server(&auto_port_addr, tc_receiver.clone(), tm_source, None);
|
||||
generic_tmtc_server(&auto_port_addr, tc_sender.clone(), tm_source, None);
|
||||
let start = Instant::now();
|
||||
// Call the connection handler in separate thread, does block.
|
||||
let thread_jh = thread::spawn(move || loop {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(20)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(20)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
@ -447,12 +454,12 @@ mod tests {
|
||||
#[test]
|
||||
fn test_server_stop_signal() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, _tc_receiver) = mpsc::channel();
|
||||
let tm_source = SyncTmSource::default();
|
||||
let stop_signal = Arc::new(AtomicBool::new(false));
|
||||
let mut tcp_server = generic_tmtc_server(
|
||||
&auto_port_addr,
|
||||
tc_receiver.clone(),
|
||||
tc_sender.clone(),
|
||||
tm_source,
|
||||
Some(stop_signal.clone()),
|
||||
);
|
||||
@ -463,7 +470,7 @@ mod tests {
|
||||
let start = Instant::now();
|
||||
// Call the connection handler in separate thread, does block.
|
||||
let thread_jh = thread::spawn(move || loop {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(20)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(20)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
|
@ -13,14 +13,13 @@ use std::net::SocketAddr;
|
||||
// use std::net::{SocketAddr, TcpStream};
|
||||
use std::thread;
|
||||
|
||||
use crate::tmtc::{ReceivesTc, TmPacketSource};
|
||||
use crate::tmtc::{PacketSenderRaw, PacketSource};
|
||||
use crate::ComponentId;
|
||||
use thiserror::Error;
|
||||
|
||||
// Re-export the TMTC in COBS server.
|
||||
pub use crate::hal::std::tcp_cobs_server::{CobsTcParser, CobsTmSender, TcpTmtcInCobsServer};
|
||||
pub use crate::hal::std::tcp_spacepackets_server::{
|
||||
SpacepacketsTcParser, SpacepacketsTmSender, TcpSpacepacketsServer,
|
||||
};
|
||||
pub use crate::hal::std::tcp_spacepackets_server::{SpacepacketsTmSender, TcpSpacepacketsServer};
|
||||
|
||||
/// Configuration struct for the generic TCP TMTC server
|
||||
///
|
||||
@ -30,7 +29,7 @@ pub use crate::hal::std::tcp_spacepackets_server::{
|
||||
/// * `inner_loop_delay` - If a client connects for a longer period, but no TC is received or
|
||||
/// no TM needs to be sent, the TCP server will delay for the specified amount of time
|
||||
/// to reduce CPU load.
|
||||
/// * `tm_buffer_size` - Size of the TM buffer used to read TM from the [TmPacketSource] and
|
||||
/// * `tm_buffer_size` - Size of the TM buffer used to read TM from the [PacketSource] and
|
||||
/// encoding of that data. This buffer should at large enough to hold the maximum expected
|
||||
/// TM size read from the packet source.
|
||||
/// * `tc_buffer_size` - Size of the TC buffer used to read encoded telecommands sent from
|
||||
@ -46,6 +45,7 @@ pub use crate::hal::std::tcp_spacepackets_server::{
|
||||
/// default.
|
||||
#[derive(Debug, Copy, Clone)]
|
||||
pub struct ServerConfig {
|
||||
pub id: ComponentId,
|
||||
pub addr: SocketAddr,
|
||||
pub inner_loop_delay: Duration,
|
||||
pub tm_buffer_size: usize,
|
||||
@ -56,12 +56,14 @@ pub struct ServerConfig {
|
||||
|
||||
impl ServerConfig {
|
||||
pub fn new(
|
||||
id: ComponentId,
|
||||
addr: SocketAddr,
|
||||
inner_loop_delay: Duration,
|
||||
tm_buffer_size: usize,
|
||||
tc_buffer_size: usize,
|
||||
) -> Self {
|
||||
Self {
|
||||
id,
|
||||
addr,
|
||||
inner_loop_delay,
|
||||
tm_buffer_size,
|
||||
@ -116,28 +118,29 @@ pub trait HandledConnectionHandler {
|
||||
}
|
||||
|
||||
/// Generic parser abstraction for an object which can parse for telecommands given a raw
|
||||
/// bytestream received from a TCP socket and send them to a generic [ReceivesTc] telecommand
|
||||
/// receiver. This allows different encoding schemes for telecommands.
|
||||
pub trait TcpTcParser<TmError, TcError> {
|
||||
/// bytestream received from a TCP socket and send them using a generic [PacketSenderRaw]
|
||||
/// implementation. This allows different encoding schemes for telecommands.
|
||||
pub trait TcpTcParser<TmError, SendError> {
|
||||
fn handle_tc_parsing(
|
||||
&mut self,
|
||||
tc_buffer: &mut [u8],
|
||||
tc_receiver: &mut (impl ReceivesTc<Error = TcError> + ?Sized),
|
||||
sender_id: ComponentId,
|
||||
tc_sender: &(impl PacketSenderRaw<Error = SendError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
current_write_idx: usize,
|
||||
next_write_idx: &mut usize,
|
||||
) -> Result<(), TcpTmtcError<TmError, TcError>>;
|
||||
) -> Result<(), TcpTmtcError<TmError, SendError>>;
|
||||
}
|
||||
|
||||
/// Generic sender abstraction for an object which can pull telemetry from a given TM source
|
||||
/// using a [TmPacketSource] and then send them back to a client using a given [TcpStream].
|
||||
/// using a [PacketSource] and then send them back to a client using a given [TcpStream].
|
||||
/// The concrete implementation can also perform any encoding steps which are necessary before
|
||||
/// sending back the data to a client.
|
||||
pub trait TcpTmSender<TmError, TcError> {
|
||||
fn handle_tm_sending(
|
||||
&mut self,
|
||||
tm_buffer: &mut [u8],
|
||||
tm_source: &mut (impl TmPacketSource<Error = TmError> + ?Sized),
|
||||
tm_source: &mut (impl PacketSource<Error = TmError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
stream: &mut TcpStream,
|
||||
) -> Result<bool, TcpTmtcError<TmError, TcError>>;
|
||||
@ -150,9 +153,9 @@ pub trait TcpTmSender<TmError, TcError> {
|
||||
/// through the following 4 core abstractions:
|
||||
///
|
||||
/// 1. [TcpTcParser] to parse for telecommands from the raw bytestream received from a client.
|
||||
/// 2. Parsed telecommands will be sent to the [ReceivesTc] telecommand receiver.
|
||||
/// 2. Parsed telecommands will be sent using the [PacketSenderRaw] object.
|
||||
/// 3. [TcpTmSender] to send telemetry pulled from a TM source back to the client.
|
||||
/// 4. [TmPacketSource] as a generic TM source used by the [TcpTmSender].
|
||||
/// 4. [PacketSource] as a generic TM source used by the [TcpTmSender].
|
||||
///
|
||||
/// It is possible to specify custom abstractions to build a dedicated TCP TMTC server without
|
||||
/// having to re-implement common logic.
|
||||
@ -160,46 +163,48 @@ pub trait TcpTmSender<TmError, TcError> {
|
||||
/// Currently, this framework offers the following concrete implementations:
|
||||
///
|
||||
/// 1. [TcpTmtcInCobsServer] to exchange TMTC wrapped inside the COBS framing protocol.
|
||||
/// 2. [TcpSpacepacketsServer] to exchange space packets via TCP.
|
||||
pub struct TcpTmtcGenericServer<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
TmSender: TcpTmSender<TmError, TcError>,
|
||||
TcParser: TcpTcParser<TmError, TcError>,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcSender: PacketSenderRaw<Error = TcSendError>,
|
||||
TmSender: TcpTmSender<TmError, TcSendError>,
|
||||
TcParser: TcpTcParser<TmError, TcSendError>,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError,
|
||||
TcError,
|
||||
TcSendError,
|
||||
> {
|
||||
pub id: ComponentId,
|
||||
pub finished_handler: HandledConnection,
|
||||
pub(crate) listener: TcpListener,
|
||||
pub(crate) inner_loop_delay: Duration,
|
||||
pub(crate) tm_source: TmSource,
|
||||
pub(crate) tm_buffer: Vec<u8>,
|
||||
pub(crate) tc_receiver: TcReceiver,
|
||||
pub(crate) tc_sender: TcSender,
|
||||
pub(crate) tc_buffer: Vec<u8>,
|
||||
poll: Poll,
|
||||
events: Events,
|
||||
tc_handler: TcParser,
|
||||
tm_handler: TmSender,
|
||||
pub tc_handler: TcParser,
|
||||
pub tm_handler: TmSender,
|
||||
stop_signal: Option<Arc<AtomicBool>>,
|
||||
}
|
||||
|
||||
impl<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
TmSender: TcpTmSender<TmError, TcError>,
|
||||
TcParser: TcpTcParser<TmError, TcError>,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcSender: PacketSenderRaw<Error = TcSendError>,
|
||||
TmSender: TcpTmSender<TmError, TcSendError>,
|
||||
TcParser: TcpTcParser<TmError, TcSendError>,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError: 'static,
|
||||
TcError: 'static,
|
||||
TcSendError: 'static,
|
||||
>
|
||||
TcpTmtcGenericServer<
|
||||
TmSource,
|
||||
TcReceiver,
|
||||
TcSender,
|
||||
TmSender,
|
||||
TcParser,
|
||||
HandledConnection,
|
||||
TmError,
|
||||
TcError,
|
||||
TcSendError,
|
||||
>
|
||||
{
|
||||
/// Create a new generic TMTC server instance.
|
||||
@ -212,15 +217,15 @@ impl<
|
||||
/// * `tm_sender` - Sends back telemetry to the client using the specified TM source.
|
||||
/// * `tm_source` - Generic TM source used by the server to pull telemetry packets which are
|
||||
/// then sent back to the client.
|
||||
/// * `tc_receiver` - Any received telecommand which was decoded successfully will be forwarded
|
||||
/// to this TC receiver.
|
||||
/// * `tc_sender` - Any received telecommand which was decoded successfully will be forwarded
|
||||
/// using this TC sender.
|
||||
/// * `stop_signal` - Can be used to stop the server even if a connection is ongoing.
|
||||
pub fn new(
|
||||
cfg: ServerConfig,
|
||||
tc_parser: TcParser,
|
||||
tm_sender: TmSender,
|
||||
tm_source: TmSource,
|
||||
tc_receiver: TcReceiver,
|
||||
tc_receiver: TcSender,
|
||||
finished_handler: HandledConnection,
|
||||
stop_signal: Option<Arc<AtomicBool>>,
|
||||
) -> Result<Self, std::io::Error> {
|
||||
@ -248,6 +253,7 @@ impl<
|
||||
.register(&mut mio_listener, Token(0), Interest::READABLE)?;
|
||||
|
||||
Ok(Self {
|
||||
id: cfg.id,
|
||||
tc_handler: tc_parser,
|
||||
tm_handler: tm_sender,
|
||||
poll,
|
||||
@ -256,7 +262,7 @@ impl<
|
||||
inner_loop_delay: cfg.inner_loop_delay,
|
||||
tm_source,
|
||||
tm_buffer: vec![0; cfg.tm_buffer_size],
|
||||
tc_receiver,
|
||||
tc_sender: tc_receiver,
|
||||
tc_buffer: vec![0; cfg.tc_buffer_size],
|
||||
stop_signal,
|
||||
finished_handler,
|
||||
@ -287,10 +293,10 @@ impl<
|
||||
/// The server will delay for a user-specified period if the client connects to the server
|
||||
/// for prolonged periods and there is no traffic for the server. This is the case if the
|
||||
/// client does not send any telecommands and no telemetry needs to be sent back to the client.
|
||||
pub fn handle_next_connection(
|
||||
pub fn handle_all_connections(
|
||||
&mut self,
|
||||
poll_timeout: Option<Duration>,
|
||||
) -> Result<ConnectionResult, TcpTmtcError<TmError, TcError>> {
|
||||
) -> Result<ConnectionResult, TcpTmtcError<TmError, TcSendError>> {
|
||||
let mut handled_connections = 0;
|
||||
// Poll Mio for events.
|
||||
self.poll.poll(&mut self.events, poll_timeout)?;
|
||||
@ -329,7 +335,7 @@ impl<
|
||||
&mut self,
|
||||
mut stream: TcpStream,
|
||||
addr: SocketAddr,
|
||||
) -> Result<(), TcpTmtcError<TmError, TcError>> {
|
||||
) -> Result<(), TcpTmtcError<TmError, TcSendError>> {
|
||||
let mut current_write_idx;
|
||||
let mut next_write_idx = 0;
|
||||
let mut connection_result = HandledConnectionInfo::new(addr);
|
||||
@ -343,7 +349,8 @@ impl<
|
||||
if current_write_idx > 0 {
|
||||
self.tc_handler.handle_tc_parsing(
|
||||
&mut self.tc_buffer,
|
||||
&mut self.tc_receiver,
|
||||
self.id,
|
||||
&self.tc_sender,
|
||||
&mut connection_result,
|
||||
current_write_idx,
|
||||
&mut next_write_idx,
|
||||
@ -357,7 +364,8 @@ impl<
|
||||
if current_write_idx == self.tc_buffer.capacity() {
|
||||
self.tc_handler.handle_tc_parsing(
|
||||
&mut self.tc_buffer,
|
||||
&mut self.tc_receiver,
|
||||
self.id,
|
||||
&self.tc_sender,
|
||||
&mut connection_result,
|
||||
current_write_idx,
|
||||
&mut next_write_idx,
|
||||
@ -371,7 +379,8 @@ impl<
|
||||
std::io::ErrorKind::WouldBlock | std::io::ErrorKind::TimedOut => {
|
||||
self.tc_handler.handle_tc_parsing(
|
||||
&mut self.tc_buffer,
|
||||
&mut self.tc_receiver,
|
||||
self.id,
|
||||
&self.tc_sender,
|
||||
&mut connection_result,
|
||||
current_write_idx,
|
||||
&mut next_write_idx,
|
||||
@ -424,24 +433,10 @@ pub(crate) mod tests {
|
||||
|
||||
use alloc::{collections::VecDeque, sync::Arc, vec::Vec};
|
||||
|
||||
use crate::tmtc::{ReceivesTcCore, TmPacketSourceCore};
|
||||
use crate::tmtc::PacketSource;
|
||||
|
||||
use super::*;
|
||||
|
||||
#[derive(Default, Clone)]
|
||||
pub(crate) struct SyncTcCacher {
|
||||
pub(crate) tc_queue: Arc<Mutex<VecDeque<Vec<u8>>>>,
|
||||
}
|
||||
impl ReceivesTcCore for SyncTcCacher {
|
||||
type Error = ();
|
||||
|
||||
fn pass_tc(&mut self, tc_raw: &[u8]) -> Result<(), Self::Error> {
|
||||
let mut tc_queue = self.tc_queue.lock().expect("tc forwarder failed");
|
||||
tc_queue.push_back(tc_raw.to_vec());
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Default, Clone)]
|
||||
pub(crate) struct SyncTmSource {
|
||||
tm_queue: Arc<Mutex<VecDeque<Vec<u8>>>>,
|
||||
@ -454,7 +449,7 @@ pub(crate) mod tests {
|
||||
}
|
||||
}
|
||||
|
||||
impl TmPacketSourceCore for SyncTmSource {
|
||||
impl PacketSource for SyncTmSource {
|
||||
type Error = ();
|
||||
|
||||
fn retrieve_packet(&mut self, buffer: &mut [u8]) -> Result<usize, Self::Error> {
|
||||
|
@ -5,9 +5,9 @@ use mio::net::{TcpListener, TcpStream};
|
||||
use std::{io::Write, net::SocketAddr};
|
||||
|
||||
use crate::{
|
||||
encoding::parse_buffer_for_ccsds_space_packets,
|
||||
tmtc::{ReceivesTc, TmPacketSource},
|
||||
ValidatorU16Id,
|
||||
encoding::{ccsds::SpacePacketValidator, parse_buffer_for_ccsds_space_packets},
|
||||
tmtc::{PacketSenderRaw, PacketSource},
|
||||
ComponentId,
|
||||
};
|
||||
|
||||
use super::tcp_server::{
|
||||
@ -15,24 +15,12 @@ use super::tcp_server::{
|
||||
TcpTmSender, TcpTmtcError, TcpTmtcGenericServer,
|
||||
};
|
||||
|
||||
/// Concrete [TcpTcParser] implementation for the [TcpSpacepacketsServer].
|
||||
pub struct SpacepacketsTcParser<PacketIdChecker: ValidatorU16Id> {
|
||||
packet_id_lookup: PacketIdChecker,
|
||||
}
|
||||
|
||||
impl<PacketIdChecker: ValidatorU16Id> SpacepacketsTcParser<PacketIdChecker> {
|
||||
pub fn new(packet_id_lookup: PacketIdChecker) -> Self {
|
||||
Self { packet_id_lookup }
|
||||
}
|
||||
}
|
||||
|
||||
impl<PacketIdChecker: ValidatorU16Id, TmError, TcError: 'static> TcpTcParser<TmError, TcError>
|
||||
for SpacepacketsTcParser<PacketIdChecker>
|
||||
{
|
||||
impl<T: SpacePacketValidator, TmError, TcError: 'static> TcpTcParser<TmError, TcError> for T {
|
||||
fn handle_tc_parsing(
|
||||
&mut self,
|
||||
tc_buffer: &mut [u8],
|
||||
tc_receiver: &mut (impl ReceivesTc<Error = TcError> + ?Sized),
|
||||
sender_id: ComponentId,
|
||||
tc_sender: &(impl PacketSenderRaw<Error = TcError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
current_write_idx: usize,
|
||||
next_write_idx: &mut usize,
|
||||
@ -40,8 +28,9 @@ impl<PacketIdChecker: ValidatorU16Id, TmError, TcError: 'static> TcpTcParser<TmE
|
||||
// Reader vec full, need to parse for packets.
|
||||
conn_result.num_received_tcs += parse_buffer_for_ccsds_space_packets(
|
||||
&mut tc_buffer[..current_write_idx],
|
||||
&self.packet_id_lookup,
|
||||
tc_receiver.upcast_mut(),
|
||||
self,
|
||||
sender_id,
|
||||
tc_sender,
|
||||
next_write_idx,
|
||||
)
|
||||
.map_err(|e| TcpTmtcError::TcError(e))?;
|
||||
@ -57,7 +46,7 @@ impl<TmError, TcError> TcpTmSender<TmError, TcError> for SpacepacketsTmSender {
|
||||
fn handle_tm_sending(
|
||||
&mut self,
|
||||
tm_buffer: &mut [u8],
|
||||
tm_source: &mut (impl TmPacketSource<Error = TmError> + ?Sized),
|
||||
tm_source: &mut (impl PacketSource<Error = TmError> + ?Sized),
|
||||
conn_result: &mut HandledConnectionInfo,
|
||||
stream: &mut TcpStream,
|
||||
) -> Result<bool, TcpTmtcError<TmError, TcError>> {
|
||||
@ -85,48 +74,41 @@ impl<TmError, TcError> TcpTmSender<TmError, TcError> for SpacepacketsTmSender {
|
||||
///
|
||||
/// This serves only works if
|
||||
/// [CCSDS 133.0-B-2 space packets](https://public.ccsds.org/Pubs/133x0b2e1.pdf) are the only
|
||||
/// packet type being exchanged. It uses the CCSDS [spacepackets::PacketId] as the packet delimiter
|
||||
/// and start marker when parsing for packets. The user specifies a set of expected
|
||||
/// [spacepackets::PacketId]s as part of the server configuration for that purpose.
|
||||
/// packet type being exchanged. It uses the CCSDS space packet header [spacepackets::SpHeader] and
|
||||
/// a user specified [SpacePacketValidator] to determine the space packets relevant for further
|
||||
/// processing.
|
||||
///
|
||||
/// ## Example
|
||||
///
|
||||
/// The [TCP server integration tests](https://egit.irs.uni-stuttgart.de/rust/sat-rs/src/branch/main/satrs/tests/tcp_servers.rs)
|
||||
/// also serves as the example application for this module.
|
||||
pub struct TcpSpacepacketsServer<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
PacketIdChecker: ValidatorU16Id,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcSender: PacketSenderRaw<Error = SendError>,
|
||||
Validator: SpacePacketValidator,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError,
|
||||
TcError: 'static,
|
||||
SendError: 'static,
|
||||
> {
|
||||
pub generic_server: TcpTmtcGenericServer<
|
||||
TmSource,
|
||||
TcReceiver,
|
||||
TcSender,
|
||||
SpacepacketsTmSender,
|
||||
SpacepacketsTcParser<PacketIdChecker>,
|
||||
Validator,
|
||||
HandledConnection,
|
||||
TmError,
|
||||
TcError,
|
||||
SendError,
|
||||
>,
|
||||
}
|
||||
|
||||
impl<
|
||||
TmSource: TmPacketSource<Error = TmError>,
|
||||
TcReceiver: ReceivesTc<Error = TcError>,
|
||||
PacketIdChecker: ValidatorU16Id,
|
||||
TmSource: PacketSource<Error = TmError>,
|
||||
TcSender: PacketSenderRaw<Error = TcError>,
|
||||
Validator: SpacePacketValidator,
|
||||
HandledConnection: HandledConnectionHandler,
|
||||
TmError: 'static,
|
||||
TcError: 'static,
|
||||
>
|
||||
TcpSpacepacketsServer<
|
||||
TmSource,
|
||||
TcReceiver,
|
||||
PacketIdChecker,
|
||||
HandledConnection,
|
||||
TmError,
|
||||
TcError,
|
||||
>
|
||||
> TcpSpacepacketsServer<TmSource, TcSender, Validator, HandledConnection, TmError, TcError>
|
||||
{
|
||||
///
|
||||
/// ## Parameter
|
||||
@ -134,26 +116,30 @@ impl<
|
||||
/// * `cfg` - Configuration of the server.
|
||||
/// * `tm_source` - Generic TM source used by the server to pull telemetry packets which are
|
||||
/// then sent back to the client.
|
||||
/// * `tc_receiver` - Any received telecommands which were decoded successfully will be
|
||||
/// forwarded to this TC receiver.
|
||||
/// * `packet_id_lookup` - This lookup table contains the relevant packets IDs for packet
|
||||
/// parsing. This mechanism is used to have a start marker for finding CCSDS packets.
|
||||
/// * `tc_sender` - Any received telecommands which were decoded successfully will be
|
||||
/// forwarded using this [PacketSenderRaw].
|
||||
/// * `validator` - Used to determine the space packets relevant for further processing and
|
||||
/// to detect broken space packets.
|
||||
/// * `handled_connection_hook` - Called to notify the user about a succesfully handled
|
||||
/// connection.
|
||||
/// * `stop_signal` - Can be used to shut down the TCP server even for longer running
|
||||
/// connections.
|
||||
pub fn new(
|
||||
cfg: ServerConfig,
|
||||
tm_source: TmSource,
|
||||
tc_receiver: TcReceiver,
|
||||
packet_id_checker: PacketIdChecker,
|
||||
handled_connection: HandledConnection,
|
||||
tc_sender: TcSender,
|
||||
validator: Validator,
|
||||
handled_connection_hook: HandledConnection,
|
||||
stop_signal: Option<Arc<AtomicBool>>,
|
||||
) -> Result<Self, std::io::Error> {
|
||||
Ok(Self {
|
||||
generic_server: TcpTmtcGenericServer::new(
|
||||
cfg,
|
||||
SpacepacketsTcParser::new(packet_id_checker),
|
||||
validator,
|
||||
SpacepacketsTmSender::default(),
|
||||
tm_source,
|
||||
tc_receiver,
|
||||
handled_connection,
|
||||
tc_sender,
|
||||
handled_connection_hook,
|
||||
stop_signal,
|
||||
)?,
|
||||
})
|
||||
@ -167,8 +153,8 @@ impl<
|
||||
/// useful if using the port number 0 for OS auto-assignment.
|
||||
pub fn local_addr(&self) -> std::io::Result<SocketAddr>;
|
||||
|
||||
/// Delegation to the [TcpTmtcGenericServer::handle_next_connection] call.
|
||||
pub fn handle_next_connection(
|
||||
/// Delegation to the [TcpTmtcGenericServer::handle_all_connections] call.
|
||||
pub fn handle_all_connections(
|
||||
&mut self,
|
||||
poll_timeout: Option<Duration>
|
||||
) -> Result<ConnectionResult, TcpTmtcError<TmError, TcError>>;
|
||||
@ -187,6 +173,7 @@ mod tests {
|
||||
use std::{
|
||||
io::{Read, Write},
|
||||
net::{IpAddr, Ipv4Addr, SocketAddr, TcpStream},
|
||||
sync::mpsc,
|
||||
thread,
|
||||
};
|
||||
|
||||
@ -194,40 +181,60 @@ mod tests {
|
||||
use hashbrown::HashSet;
|
||||
use spacepackets::{
|
||||
ecss::{tc::PusTcCreator, WritablePusPacket},
|
||||
PacketId, SpHeader,
|
||||
CcsdsPacket, PacketId, SpHeader,
|
||||
};
|
||||
|
||||
use crate::hal::std::tcp_server::{
|
||||
tests::{ConnectionFinishedHandler, SyncTcCacher, SyncTmSource},
|
||||
ConnectionResult, ServerConfig,
|
||||
use crate::{
|
||||
encoding::ccsds::{SpValidity, SpacePacketValidator},
|
||||
hal::std::tcp_server::{
|
||||
tests::{ConnectionFinishedHandler, SyncTmSource},
|
||||
ConnectionResult, ServerConfig,
|
||||
},
|
||||
queue::GenericSendError,
|
||||
tmtc::PacketAsVec,
|
||||
ComponentId,
|
||||
};
|
||||
|
||||
use super::TcpSpacepacketsServer;
|
||||
|
||||
const TCP_SERVER_ID: ComponentId = 0x05;
|
||||
const TEST_APID_0: u16 = 0x02;
|
||||
const TEST_PACKET_ID_0: PacketId = PacketId::new_for_tc(true, TEST_APID_0);
|
||||
const TEST_APID_1: u16 = 0x10;
|
||||
const TEST_PACKET_ID_1: PacketId = PacketId::new_for_tc(true, TEST_APID_1);
|
||||
|
||||
#[derive(Default)]
|
||||
pub struct SimpleValidator(pub HashSet<PacketId>);
|
||||
|
||||
impl SpacePacketValidator for SimpleValidator {
|
||||
fn validate(&self, sp_header: &SpHeader, _raw_buf: &[u8]) -> SpValidity {
|
||||
if self.0.contains(&sp_header.packet_id()) {
|
||||
return SpValidity::Valid;
|
||||
}
|
||||
// Simple case: Assume that the interface always contains valid space packets.
|
||||
SpValidity::Skip
|
||||
}
|
||||
}
|
||||
|
||||
fn generic_tmtc_server(
|
||||
addr: &SocketAddr,
|
||||
tc_receiver: SyncTcCacher,
|
||||
tc_sender: mpsc::Sender<PacketAsVec>,
|
||||
tm_source: SyncTmSource,
|
||||
packet_id_lookup: HashSet<PacketId>,
|
||||
validator: SimpleValidator,
|
||||
stop_signal: Option<Arc<AtomicBool>>,
|
||||
) -> TcpSpacepacketsServer<
|
||||
SyncTmSource,
|
||||
SyncTcCacher,
|
||||
HashSet<PacketId>,
|
||||
mpsc::Sender<PacketAsVec>,
|
||||
SimpleValidator,
|
||||
ConnectionFinishedHandler,
|
||||
(),
|
||||
(),
|
||||
GenericSendError,
|
||||
> {
|
||||
TcpSpacepacketsServer::new(
|
||||
ServerConfig::new(*addr, Duration::from_millis(2), 1024, 1024),
|
||||
ServerConfig::new(TCP_SERVER_ID, *addr, Duration::from_millis(2), 1024, 1024),
|
||||
tm_source,
|
||||
tc_receiver,
|
||||
packet_id_lookup,
|
||||
tc_sender,
|
||||
validator,
|
||||
ConnectionFinishedHandler::default(),
|
||||
stop_signal,
|
||||
)
|
||||
@ -237,15 +244,15 @@ mod tests {
|
||||
#[test]
|
||||
fn test_basic_tc_only() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, tc_receiver) = mpsc::channel();
|
||||
let tm_source = SyncTmSource::default();
|
||||
let mut packet_id_lookup = HashSet::new();
|
||||
packet_id_lookup.insert(TEST_PACKET_ID_0);
|
||||
let mut validator = SimpleValidator::default();
|
||||
validator.0.insert(TEST_PACKET_ID_0);
|
||||
let mut tcp_server = generic_tmtc_server(
|
||||
&auto_port_addr,
|
||||
tc_receiver.clone(),
|
||||
tc_sender.clone(),
|
||||
tm_source,
|
||||
packet_id_lookup,
|
||||
validator,
|
||||
None,
|
||||
);
|
||||
let dest_addr = tcp_server
|
||||
@ -255,7 +262,7 @@ mod tests {
|
||||
let set_if_done = conn_handled.clone();
|
||||
// Call the connection handler in separate thread, does block.
|
||||
thread::spawn(move || {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(100)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(100)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
@ -289,16 +296,15 @@ mod tests {
|
||||
if !conn_handled.load(Ordering::Relaxed) {
|
||||
panic!("connection was not handled properly");
|
||||
}
|
||||
// Check that TC has arrived.
|
||||
let mut tc_queue = tc_receiver.tc_queue.lock().unwrap();
|
||||
assert_eq!(tc_queue.len(), 1);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), tc_0);
|
||||
let packet = tc_receiver.try_recv().expect("receiving TC failed");
|
||||
assert_eq!(packet.packet, tc_0);
|
||||
matches!(tc_receiver.try_recv(), Err(mpsc::TryRecvError::Empty));
|
||||
}
|
||||
|
||||
#[test]
|
||||
fn test_multi_tc_multi_tm() {
|
||||
let auto_port_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 0);
|
||||
let tc_receiver = SyncTcCacher::default();
|
||||
let (tc_sender, tc_receiver) = mpsc::channel();
|
||||
let mut tm_source = SyncTmSource::default();
|
||||
|
||||
// Add telemetry
|
||||
@ -315,14 +321,14 @@ mod tests {
|
||||
tm_source.add_tm(&tm_1);
|
||||
|
||||
// Set up server
|
||||
let mut packet_id_lookup = HashSet::new();
|
||||
packet_id_lookup.insert(TEST_PACKET_ID_0);
|
||||
packet_id_lookup.insert(TEST_PACKET_ID_1);
|
||||
let mut validator = SimpleValidator::default();
|
||||
validator.0.insert(TEST_PACKET_ID_0);
|
||||
validator.0.insert(TEST_PACKET_ID_1);
|
||||
let mut tcp_server = generic_tmtc_server(
|
||||
&auto_port_addr,
|
||||
tc_receiver.clone(),
|
||||
tc_sender.clone(),
|
||||
tm_source,
|
||||
packet_id_lookup,
|
||||
validator,
|
||||
None,
|
||||
);
|
||||
let dest_addr = tcp_server
|
||||
@ -333,7 +339,7 @@ mod tests {
|
||||
|
||||
// Call the connection handler in separate thread, does block.
|
||||
thread::spawn(move || {
|
||||
let result = tcp_server.handle_next_connection(Some(Duration::from_millis(100)));
|
||||
let result = tcp_server.handle_all_connections(Some(Duration::from_millis(100)));
|
||||
if result.is_err() {
|
||||
panic!("handling connection failed: {:?}", result.unwrap_err());
|
||||
}
|
||||
@ -397,9 +403,10 @@ mod tests {
|
||||
panic!("connection was not handled properly");
|
||||
}
|
||||
// Check that TC has arrived.
|
||||
let mut tc_queue = tc_receiver.tc_queue.lock().unwrap();
|
||||
assert_eq!(tc_queue.len(), 2);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), tc_0);
|
||||
assert_eq!(tc_queue.pop_front().unwrap(), tc_1);
|
||||
let packet_0 = tc_receiver.try_recv().expect("receiving TC failed");
|
||||
assert_eq!(packet_0.packet, tc_0);
|
||||
let packet_1 = tc_receiver.try_recv().expect("receiving TC failed");
|
||||
assert_eq!(packet_1.packet, tc_1);
|
||||
matches!(tc_receiver.try_recv(), Err(mpsc::TryRecvError::Empty));
|
||||
}
|
||||
}
|
||||
|
@ -1,7 +1,8 @@
|
||||
//! Generic UDP TC server.
|
||||
use crate::tmtc::{ReceivesTc, ReceivesTcCore};
|
||||
use std::boxed::Box;
|
||||
use std::io::{Error, ErrorKind};
|
||||
use crate::tmtc::PacketSenderRaw;
|
||||
use crate::ComponentId;
|
||||
use core::fmt::Debug;
|
||||
use std::io::{self, ErrorKind};
|
||||
use std::net::{SocketAddr, ToSocketAddrs, UdpSocket};
|
||||
use std::vec;
|
||||
use std::vec::Vec;
|
||||
@ -11,45 +12,46 @@ use std::vec::Vec;
|
||||
///
|
||||
/// It caches all received telecomands into a vector. The maximum expected telecommand size should
|
||||
/// be declared upfront. This avoids dynamic allocation during run-time. The user can specify a TC
|
||||
/// receiver in form of a special trait object which implements [ReceivesTc]. Please note that the
|
||||
/// receiver should copy out the received data if it the data is required past the
|
||||
/// [ReceivesTcCore::pass_tc] call.
|
||||
/// sender in form of a special trait object which implements [PacketSenderRaw]. For example, this
|
||||
/// can be used to send the telecommands to a centralized TC source component for further
|
||||
/// processing and routing.
|
||||
///
|
||||
/// # Examples
|
||||
///
|
||||
/// ```
|
||||
/// use std::net::{IpAddr, Ipv4Addr, SocketAddr, UdpSocket};
|
||||
/// use std::sync::mpsc;
|
||||
/// use spacepackets::ecss::WritablePusPacket;
|
||||
/// use satrs::hal::std::udp_server::UdpTcServer;
|
||||
/// use satrs::tmtc::{ReceivesTc, ReceivesTcCore};
|
||||
/// use satrs::ComponentId;
|
||||
/// use satrs::tmtc::PacketSenderRaw;
|
||||
/// use spacepackets::SpHeader;
|
||||
/// use spacepackets::ecss::tc::PusTcCreator;
|
||||
///
|
||||
/// #[derive (Default)]
|
||||
/// struct PingReceiver {}
|
||||
/// impl ReceivesTcCore for PingReceiver {
|
||||
/// type Error = ();
|
||||
/// fn pass_tc(&mut self, tc_raw: &[u8]) -> Result<(), Self::Error> {
|
||||
/// assert_eq!(tc_raw.len(), 13);
|
||||
/// Ok(())
|
||||
/// }
|
||||
/// }
|
||||
/// const UDP_SERVER_ID: ComponentId = 0x05;
|
||||
///
|
||||
/// let mut buf = [0; 32];
|
||||
/// let (packet_sender, packet_receiver) = mpsc::channel();
|
||||
/// let dest_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 7777);
|
||||
/// let ping_receiver = PingReceiver::default();
|
||||
/// let mut udp_tc_server = UdpTcServer::new(dest_addr, 2048, Box::new(ping_receiver))
|
||||
/// let mut udp_tc_server = UdpTcServer::new(UDP_SERVER_ID, dest_addr, 2048, packet_sender)
|
||||
/// .expect("Creating UDP TMTC server failed");
|
||||
/// let sph = SpHeader::new_from_apid(0x02);
|
||||
/// let pus_tc = PusTcCreator::new_simple(sph, 17, 1, &[], true);
|
||||
/// let len = pus_tc
|
||||
/// .write_to_bytes(&mut buf)
|
||||
/// .expect("Error writing PUS TC packet");
|
||||
/// assert_eq!(len, 13);
|
||||
/// let client = UdpSocket::bind("127.0.0.1:7778").expect("Connecting to UDP server failed");
|
||||
/// // Can not fail.
|
||||
/// let ping_tc_raw = pus_tc.to_vec().unwrap();
|
||||
///
|
||||
/// // Now create a UDP client and send the ping telecommand to the server.
|
||||
/// let client = UdpSocket::bind("127.0.0.1:0").expect("creating UDP client failed");
|
||||
/// client
|
||||
/// .send_to(&buf[0..len], dest_addr)
|
||||
/// .send_to(&ping_tc_raw, dest_addr)
|
||||
/// .expect("Error sending PUS TC via UDP");
|
||||
/// let recv_result = udp_tc_server.try_recv_tc();
|
||||
/// assert!(recv_result.is_ok());
|
||||
/// // The packet is received by the UDP TC server and sent via the mpsc channel.
|
||||
/// let sent_packet_with_sender = packet_receiver.try_recv().expect("expected telecommand");
|
||||
/// assert_eq!(sent_packet_with_sender.packet, ping_tc_raw);
|
||||
/// assert_eq!(sent_packet_with_sender.sender_id, UDP_SERVER_ID);
|
||||
/// // No more packets received.
|
||||
/// matches!(packet_receiver.try_recv(), Err(mpsc::TryRecvError::Empty));
|
||||
/// ```
|
||||
///
|
||||
/// The [satrs-example crate](https://egit.irs.uni-stuttgart.de/rust/fsrc-launchpad/src/branch/main/satrs-example)
|
||||
@ -57,65 +59,45 @@ use std::vec::Vec;
|
||||
/// [example code](https://egit.irs.uni-stuttgart.de/rust/sat-rs/src/branch/main/satrs-example/src/tmtc.rs#L67)
|
||||
/// on how to use this TC server. It uses the server to receive PUS telecommands on a specific port
|
||||
/// and then forwards them to a generic CCSDS packet receiver.
|
||||
pub struct UdpTcServer<E> {
|
||||
pub struct UdpTcServer<TcSender: PacketSenderRaw<Error = SendError>, SendError> {
|
||||
pub id: ComponentId,
|
||||
pub socket: UdpSocket,
|
||||
recv_buf: Vec<u8>,
|
||||
sender_addr: Option<SocketAddr>,
|
||||
tc_receiver: Box<dyn ReceivesTc<Error = E>>,
|
||||
pub tc_sender: TcSender,
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
pub enum ReceiveResult<E> {
|
||||
#[derive(Debug, thiserror::Error)]
|
||||
pub enum ReceiveResult<SendError: Debug + 'static> {
|
||||
#[error("nothing was received")]
|
||||
NothingReceived,
|
||||
IoError(Error),
|
||||
ReceiverError(E),
|
||||
#[error(transparent)]
|
||||
Io(#[from] io::Error),
|
||||
#[error(transparent)]
|
||||
Send(SendError),
|
||||
}
|
||||
|
||||
impl<E> From<Error> for ReceiveResult<E> {
|
||||
fn from(e: Error) -> Self {
|
||||
ReceiveResult::IoError(e)
|
||||
}
|
||||
}
|
||||
|
||||
impl<E: PartialEq> PartialEq for ReceiveResult<E> {
|
||||
fn eq(&self, other: &Self) -> bool {
|
||||
use ReceiveResult::*;
|
||||
match (self, other) {
|
||||
(IoError(ref e), IoError(ref other_e)) => e.kind() == other_e.kind(),
|
||||
(NothingReceived, NothingReceived) => true,
|
||||
(ReceiverError(e), ReceiverError(other_e)) => e == other_e,
|
||||
_ => false,
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<E: Eq + PartialEq> Eq for ReceiveResult<E> {}
|
||||
|
||||
impl<E: 'static> ReceivesTcCore for UdpTcServer<E> {
|
||||
type Error = E;
|
||||
|
||||
fn pass_tc(&mut self, tc_raw: &[u8]) -> Result<(), Self::Error> {
|
||||
self.tc_receiver.pass_tc(tc_raw)
|
||||
}
|
||||
}
|
||||
|
||||
impl<E: 'static> UdpTcServer<E> {
|
||||
impl<TcSender: PacketSenderRaw<Error = SendError>, SendError: Debug + 'static>
|
||||
UdpTcServer<TcSender, SendError>
|
||||
{
|
||||
pub fn new<A: ToSocketAddrs>(
|
||||
id: ComponentId,
|
||||
addr: A,
|
||||
max_recv_size: usize,
|
||||
tc_receiver: Box<dyn ReceivesTc<Error = E>>,
|
||||
) -> Result<Self, Error> {
|
||||
tc_sender: TcSender,
|
||||
) -> Result<Self, io::Error> {
|
||||
let server = Self {
|
||||
id,
|
||||
socket: UdpSocket::bind(addr)?,
|
||||
recv_buf: vec![0; max_recv_size],
|
||||
sender_addr: None,
|
||||
tc_receiver,
|
||||
tc_sender,
|
||||
};
|
||||
server.socket.set_nonblocking(true)?;
|
||||
Ok(server)
|
||||
}
|
||||
|
||||
pub fn try_recv_tc(&mut self) -> Result<(usize, SocketAddr), ReceiveResult<E>> {
|
||||
pub fn try_recv_tc(&mut self) -> Result<(usize, SocketAddr), ReceiveResult<SendError>> {
|
||||
let res = match self.socket.recv_from(&mut self.recv_buf) {
|
||||
Ok(res) => res,
|
||||
Err(e) => {
|
||||
@ -128,9 +110,9 @@ impl<E: 'static> UdpTcServer<E> {
|
||||
};
|
||||
let (num_bytes, from) = res;
|
||||
self.sender_addr = Some(from);
|
||||
self.tc_receiver
|
||||
.pass_tc(&self.recv_buf[0..num_bytes])
|
||||
.map_err(|e| ReceiveResult::ReceiverError(e))?;
|
||||
self.tc_sender
|
||||
.send_packet(self.id, &self.recv_buf[0..num_bytes])
|
||||
.map_err(ReceiveResult::Send)?;
|
||||
Ok(res)
|
||||
}
|
||||
|
||||
@ -142,29 +124,35 @@ impl<E: 'static> UdpTcServer<E> {
|
||||
#[cfg(test)]
|
||||
mod tests {
|
||||
use crate::hal::std::udp_server::{ReceiveResult, UdpTcServer};
|
||||
use crate::tmtc::ReceivesTcCore;
|
||||
use crate::queue::GenericSendError;
|
||||
use crate::tmtc::PacketSenderRaw;
|
||||
use crate::ComponentId;
|
||||
use core::cell::RefCell;
|
||||
use spacepackets::ecss::tc::PusTcCreator;
|
||||
use spacepackets::ecss::WritablePusPacket;
|
||||
use spacepackets::SpHeader;
|
||||
use std::boxed::Box;
|
||||
use std::collections::VecDeque;
|
||||
use std::net::{IpAddr, Ipv4Addr, SocketAddr, UdpSocket};
|
||||
use std::vec::Vec;
|
||||
|
||||
fn is_send<T: Send>(_: &T) {}
|
||||
|
||||
const UDP_SERVER_ID: ComponentId = 0x05;
|
||||
|
||||
#[derive(Default)]
|
||||
struct PingReceiver {
|
||||
pub sent_cmds: VecDeque<Vec<u8>>,
|
||||
pub sent_cmds: RefCell<VecDeque<Vec<u8>>>,
|
||||
}
|
||||
|
||||
impl ReceivesTcCore for PingReceiver {
|
||||
type Error = ();
|
||||
impl PacketSenderRaw for PingReceiver {
|
||||
type Error = GenericSendError;
|
||||
|
||||
fn pass_tc(&mut self, tc_raw: &[u8]) -> Result<(), Self::Error> {
|
||||
fn send_packet(&self, sender_id: ComponentId, tc_raw: &[u8]) -> Result<(), Self::Error> {
|
||||
assert_eq!(sender_id, UDP_SERVER_ID);
|
||||
let mut sent_data = Vec::new();
|
||||
sent_data.extend_from_slice(tc_raw);
|
||||
self.sent_cmds.push_back(sent_data);
|
||||
let mut queue = self.sent_cmds.borrow_mut();
|
||||
queue.push_back(sent_data);
|
||||
Ok(())
|
||||
}
|
||||
}
|
||||
@ -175,7 +163,7 @@ mod tests {
|
||||
let dest_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 7777);
|
||||
let ping_receiver = PingReceiver::default();
|
||||
is_send(&ping_receiver);
|
||||
let mut udp_tc_server = UdpTcServer::new(dest_addr, 2048, Box::new(ping_receiver))
|
||||
let mut udp_tc_server = UdpTcServer::new(UDP_SERVER_ID, dest_addr, 2048, ping_receiver)
|
||||
.expect("Creating UDP TMTC server failed");
|
||||
is_send(&udp_tc_server);
|
||||
let sph = SpHeader::new_from_apid(0x02);
|
||||
@ -195,9 +183,10 @@ mod tests {
|
||||
udp_tc_server.last_sender().expect("No sender set"),
|
||||
local_addr
|
||||
);
|
||||
let ping_receiver: &mut PingReceiver = udp_tc_server.tc_receiver.downcast_mut().unwrap();
|
||||
assert_eq!(ping_receiver.sent_cmds.len(), 1);
|
||||
let sent_cmd = ping_receiver.sent_cmds.pop_front().unwrap();
|
||||
let ping_receiver = &mut udp_tc_server.tc_sender;
|
||||
let mut queue = ping_receiver.sent_cmds.borrow_mut();
|
||||
assert_eq!(queue.len(), 1);
|
||||
let sent_cmd = queue.pop_front().unwrap();
|
||||
assert_eq!(sent_cmd, buf[0..len]);
|
||||
}
|
||||
|
||||
@ -205,11 +194,11 @@ mod tests {
|
||||
fn test_nothing_received() {
|
||||
let dest_addr = SocketAddr::new(IpAddr::V4(Ipv4Addr::new(127, 0, 0, 1)), 7779);
|
||||
let ping_receiver = PingReceiver::default();
|
||||
let mut udp_tc_server = UdpTcServer::new(dest_addr, 2048, Box::new(ping_receiver))
|
||||
let mut udp_tc_server = UdpTcServer::new(UDP_SERVER_ID, dest_addr, 2048, ping_receiver)
|
||||
.expect("Creating UDP TMTC server failed");
|
||||
let res = udp_tc_server.try_recv_tc();
|
||||
assert!(res.is_err());
|
||||
let err = res.unwrap_err();
|
||||
assert_eq!(err, ReceiveResult::NothingReceived);
|
||||
matches!(err, ReceiveResult::NothingReceived);
|
||||
}
|
||||
}
|
||||
|
Reference in New Issue
Block a user