mirror of
https://github.com/hl-archive-node/nanoreth.git
synced 2025-12-06 10:59:55 +00:00
feat(net): draft for sending status updates through NetworkHandle (#436)
* feat(net): draft for sending status updates through `NetworkHandle` * feat(net): draft for sending status updates through `NetworkHandle` * fix(net): remove unused import * feat(net): implement getters for status update sender half * docs(net): document methods * chore: cargo fmt * feat(net): move status updating logic to NetworkManager and NetworkHandle * feat(net): move status updating logic to NetworkManager and NetworkHandle * fix(net): fix headers stage testing * fix: derive default * fix: remove StatusUpdate struct Co-authored-by: Matthias Seitz <matthias.seitz@outlook.de>
This commit is contained in:
1
Cargo.lock
generated
1
Cargo.lock
generated
@ -3635,6 +3635,7 @@ dependencies = [
|
||||
"reth-eth-wire",
|
||||
"reth-executor",
|
||||
"reth-interfaces",
|
||||
"reth-network",
|
||||
"reth-primitives",
|
||||
"reth-provider",
|
||||
"reth-rlp",
|
||||
|
||||
@ -63,6 +63,7 @@ impl Command {
|
||||
.build(consensus.clone(), fetch_client.clone()),
|
||||
consensus: consensus.clone(),
|
||||
client: fetch_client.clone(),
|
||||
network_handle: network.clone(),
|
||||
commit_threshold: 100,
|
||||
},
|
||||
false,
|
||||
|
||||
@ -20,12 +20,13 @@ pub struct HeadersRequest {
|
||||
#[async_trait]
|
||||
#[auto_impl::auto_impl(&, Arc, Box)]
|
||||
pub trait HeadersClient: Send + Sync + Debug {
|
||||
/// Update the node's Status message.
|
||||
///
|
||||
/// The updated Status message will be used during any new eth/65 handshakes.
|
||||
fn update_status(&self, height: u64, hash: H256, td: U256);
|
||||
|
||||
/// Sends the header request to the p2p network and returns the header response received from a
|
||||
/// peer.
|
||||
async fn get_headers(&self, request: HeadersRequest) -> PeerRequestResult<BlockHeaders>;
|
||||
}
|
||||
|
||||
/// The status updater for updating the status of the p2p node
|
||||
pub trait StatusUpdater: Send + Sync {
|
||||
/// Updates the status of the p2p node
|
||||
fn update_status(&self, height: u64, hash: H256, total_difficulty: U256);
|
||||
}
|
||||
|
||||
@ -5,7 +5,7 @@ use crate::{
|
||||
downloader::{DownloadStream, Downloader},
|
||||
error::{PeerRequestResult, RequestError},
|
||||
headers::{
|
||||
client::{HeadersClient, HeadersRequest},
|
||||
client::{HeadersClient, HeadersRequest, StatusUpdater},
|
||||
downloader::HeaderDownloader,
|
||||
error::DownloadError,
|
||||
},
|
||||
@ -14,7 +14,7 @@ use crate::{
|
||||
use futures::{Future, FutureExt, Stream};
|
||||
use reth_eth_wire::BlockHeaders;
|
||||
use reth_primitives::{
|
||||
BlockLocked, BlockNumber, Header, HeadersDirection, PeerId, SealedHeader, H256, U256,
|
||||
BlockLocked, BlockNumber, Header, HeadersDirection, PeerId, SealedHeader, H256,
|
||||
};
|
||||
use reth_rpc_types::engine::ForkchoiceState;
|
||||
use std::{
|
||||
@ -170,8 +170,6 @@ impl TestHeadersClient {
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl HeadersClient for TestHeadersClient {
|
||||
fn update_status(&self, _height: u64, _hash: H256, _td: U256) {}
|
||||
|
||||
async fn get_headers(&self, request: HeadersRequest) -> PeerRequestResult<BlockHeaders> {
|
||||
if let Some(err) = &mut *self.error.lock().await {
|
||||
return Err(err.clone())
|
||||
@ -228,6 +226,14 @@ impl TestConsensus {
|
||||
}
|
||||
}
|
||||
|
||||
/// Nil status updater for testing
|
||||
#[derive(Debug, Clone, Default)]
|
||||
pub struct TestStatusUpdater;
|
||||
|
||||
impl StatusUpdater for TestStatusUpdater {
|
||||
fn update_status(&self, _height: u64, _hash: H256, _total_difficulty: reth_primitives::U256) {}
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl Consensus for TestConsensus {
|
||||
fn fork_choice_state(&self) -> watch::Receiver<ForkchoiceState> {
|
||||
|
||||
@ -1,13 +1,13 @@
|
||||
//! A client implementation that can interact with the network and download data.
|
||||
|
||||
use crate::fetch::{DownloadRequest, StatusUpdate};
|
||||
use crate::fetch::DownloadRequest;
|
||||
use reth_eth_wire::{BlockBody, BlockHeaders};
|
||||
use reth_interfaces::p2p::{
|
||||
bodies::client::BodiesClient,
|
||||
error::PeerRequestResult,
|
||||
headers::client::{HeadersClient, HeadersRequest},
|
||||
};
|
||||
use reth_primitives::{WithPeerId, H256, U256};
|
||||
use reth_primitives::{WithPeerId, H256};
|
||||
use tokio::sync::{mpsc::UnboundedSender, oneshot};
|
||||
|
||||
/// Front-end API for fetching data from the network.
|
||||
@ -15,16 +15,10 @@ use tokio::sync::{mpsc::UnboundedSender, oneshot};
|
||||
pub struct FetchClient {
|
||||
/// Sender half of the request channel.
|
||||
pub(crate) request_tx: UnboundedSender<DownloadRequest>,
|
||||
/// Sender for sending Status updates
|
||||
pub(crate) status_tx: UnboundedSender<StatusUpdate>,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl HeadersClient for FetchClient {
|
||||
fn update_status(&self, height: u64, hash: H256, total_difficulty: U256) {
|
||||
let _ = self.status_tx.send(StatusUpdate { height, hash, total_difficulty });
|
||||
}
|
||||
|
||||
/// Sends a `GetBlockHeaders` request to an available peer.
|
||||
async fn get_headers(&self, request: HeadersRequest) -> PeerRequestResult<BlockHeaders> {
|
||||
let (response, rx) = oneshot::channel();
|
||||
|
||||
@ -7,7 +7,7 @@ use reth_interfaces::p2p::{
|
||||
error::{PeerRequestResult, RequestError, RequestResult},
|
||||
headers::client::HeadersRequest,
|
||||
};
|
||||
use reth_primitives::{Header, PeerId, H256, U256};
|
||||
use reth_primitives::{Header, PeerId, H256};
|
||||
use std::{
|
||||
collections::{HashMap, VecDeque},
|
||||
task::{Context, Poll},
|
||||
@ -38,10 +38,6 @@ pub struct StateFetcher {
|
||||
download_requests_rx: UnboundedReceiverStream<DownloadRequest>,
|
||||
/// Sender for download requests, used to detach a [`FetchClient`]
|
||||
download_requests_tx: UnboundedSender<DownloadRequest>,
|
||||
/// Receiver for new incoming [`StatusUpdate`] requests.
|
||||
status_rx: UnboundedReceiverStream<StatusUpdate>,
|
||||
/// Sender for updating the status, used to detach a [`FetchClient`]
|
||||
status_tx: UnboundedSender<StatusUpdate>,
|
||||
}
|
||||
|
||||
// === impl StateSyncer ===
|
||||
@ -120,10 +116,6 @@ impl StateFetcher {
|
||||
PollAction::NoPeersAvailable => true,
|
||||
};
|
||||
|
||||
if let Poll::Ready(Some(status)) = self.status_rx.poll_next_unpin(cx) {
|
||||
return Poll::Ready(FetchAction::StatusUpdate(status))
|
||||
}
|
||||
|
||||
loop {
|
||||
// poll incoming requests
|
||||
match self.download_requests_rx.poll_next_unpin(cx) {
|
||||
@ -229,17 +221,13 @@ impl StateFetcher {
|
||||
|
||||
/// Returns a new [`FetchClient`] that can send requests to this type.
|
||||
pub(crate) fn client(&self) -> FetchClient {
|
||||
FetchClient {
|
||||
request_tx: self.download_requests_tx.clone(),
|
||||
status_tx: self.status_tx.clone(),
|
||||
}
|
||||
FetchClient { request_tx: self.download_requests_tx.clone() }
|
||||
}
|
||||
}
|
||||
|
||||
impl Default for StateFetcher {
|
||||
fn default() -> Self {
|
||||
let (download_requests_tx, download_requests_rx) = mpsc::unbounded_channel();
|
||||
let (status_tx, status_rx) = mpsc::unbounded_channel();
|
||||
Self {
|
||||
inflight_headers_requests: Default::default(),
|
||||
inflight_bodies_requests: Default::default(),
|
||||
@ -247,8 +235,6 @@ impl Default for StateFetcher {
|
||||
queued_requests: Default::default(),
|
||||
download_requests_rx: UnboundedReceiverStream::new(download_requests_rx),
|
||||
download_requests_tx,
|
||||
status_rx: UnboundedReceiverStream::new(status_rx),
|
||||
status_tx,
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -314,14 +300,6 @@ struct Request<Req, Resp> {
|
||||
response: oneshot::Sender<Resp>,
|
||||
}
|
||||
|
||||
/// A message to update the status.
|
||||
#[derive(Debug, Clone)]
|
||||
pub(crate) struct StatusUpdate {
|
||||
pub(crate) height: u64,
|
||||
pub(crate) hash: H256,
|
||||
pub(crate) total_difficulty: U256,
|
||||
}
|
||||
|
||||
/// Requests that can be sent to the Syncer from a [`FetchClient`]
|
||||
pub(crate) enum DownloadRequest {
|
||||
/// Download the requested headers and send response through channel
|
||||
@ -357,8 +335,6 @@ pub(crate) enum FetchAction {
|
||||
/// The request to send
|
||||
request: BlockRequest,
|
||||
},
|
||||
/// Propagate a received status update for the node
|
||||
StatusUpdate(StatusUpdate),
|
||||
}
|
||||
|
||||
/// Outcome of a processed response.
|
||||
|
||||
@ -456,6 +456,9 @@ where
|
||||
NetworkHandleMessage::FetchClient(tx) => {
|
||||
let _ = tx.send(self.fetch_client());
|
||||
}
|
||||
NetworkHandleMessage::StatusUpdate { height, hash, total_difficulty } => {
|
||||
self.swarm.sessions_mut().on_status_update(height, hash, total_difficulty);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
@ -612,14 +615,6 @@ where
|
||||
.peers_mut()
|
||||
.apply_reputation_change(&peer_id, ReputationChangeKind::FailedToConnect);
|
||||
}
|
||||
SwarmEvent::StatusUpdate(status) => {
|
||||
trace!(
|
||||
target : "net",
|
||||
?status,
|
||||
"Status Update received"
|
||||
);
|
||||
this.swarm.sessions_mut().on_status_update(status.clone())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@ -7,7 +7,8 @@ use crate::{
|
||||
};
|
||||
use parking_lot::Mutex;
|
||||
use reth_eth_wire::{NewBlock, NewPooledTransactionHashes, SharedTransactions};
|
||||
use reth_primitives::{PeerId, TransactionSigned, TxHash, H256};
|
||||
use reth_interfaces::p2p::headers::client::StatusUpdater;
|
||||
use reth_primitives::{PeerId, TransactionSigned, TxHash, H256, U256};
|
||||
use std::{
|
||||
net::SocketAddr,
|
||||
sync::{
|
||||
@ -21,7 +22,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||
/// A _shareable_ network frontend. Used to interact with the network.
|
||||
///
|
||||
/// See also [`NetworkManager`](crate::NetworkManager).
|
||||
#[derive(Clone)]
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct NetworkHandle {
|
||||
/// The Arc'ed delegate that contains the state.
|
||||
inner: Arc<NetworkInner>,
|
||||
@ -102,6 +103,11 @@ impl NetworkHandle {
|
||||
let _ = self.inner.to_manager_tx.send(msg);
|
||||
}
|
||||
|
||||
/// Update the status of the node.
|
||||
pub fn update_status(&self, height: u64, hash: H256, total_difficulty: U256) {
|
||||
self.send_message(NetworkHandleMessage::StatusUpdate { height, hash, total_difficulty });
|
||||
}
|
||||
|
||||
/// Announce a block over devp2p
|
||||
///
|
||||
/// Caution: in PoS this is a noop, since new block propagation will happen over devp2p
|
||||
@ -148,6 +154,14 @@ impl NetworkHandle {
|
||||
}
|
||||
}
|
||||
|
||||
impl StatusUpdater for NetworkHandle {
|
||||
/// Update the status of the node.
|
||||
fn update_status(&self, height: u64, hash: H256, total_difficulty: U256) {
|
||||
self.send_message(NetworkHandleMessage::StatusUpdate { height, hash, total_difficulty });
|
||||
}
|
||||
}
|
||||
|
||||
#[derive(Debug)]
|
||||
struct NetworkInner {
|
||||
/// Number of active peer sessions the node's currently handling.
|
||||
num_active_peers: Arc<AtomicUsize>,
|
||||
@ -189,4 +203,6 @@ pub(crate) enum NetworkHandleMessage {
|
||||
ReputationChange(PeerId, ReputationChangeKind),
|
||||
/// Returns the client that can be used to interact with the network.
|
||||
FetchClient(oneshot::Sender<FetchClient>),
|
||||
/// Apply a status update.
|
||||
StatusUpdate { height: u64, hash: H256, total_difficulty: U256 },
|
||||
}
|
||||
|
||||
@ -22,7 +22,7 @@ use tokio_stream::wrappers::UnboundedReceiverStream;
|
||||
use tracing::trace;
|
||||
|
||||
/// A communication channel to the [`PeersManager`] to apply manual changes to the peer set.
|
||||
#[derive(Clone)]
|
||||
#[derive(Clone, Debug)]
|
||||
pub struct PeersHandle {
|
||||
/// Sender half of command channel back to the [`PeersManager`]
|
||||
manager_tx: mpsc::UnboundedSender<PeerCommand>,
|
||||
|
||||
@ -1,7 +1,6 @@
|
||||
//! Support for handling peer sessions.
|
||||
pub use crate::message::PeerRequestSender;
|
||||
use crate::{
|
||||
fetch::StatusUpdate,
|
||||
message::PeerMessage,
|
||||
session::{
|
||||
active::ActiveSession,
|
||||
@ -20,7 +19,7 @@ use reth_eth_wire::{
|
||||
error::EthStreamError,
|
||||
DisconnectReason, HelloMessage, Status, UnauthedEthStream, UnauthedP2PStream,
|
||||
};
|
||||
use reth_primitives::{ForkFilter, Hardfork, PeerId};
|
||||
use reth_primitives::{ForkFilter, Hardfork, PeerId, H256, U256};
|
||||
use secp256k1::SecretKey;
|
||||
use std::{
|
||||
collections::HashMap,
|
||||
@ -150,10 +149,10 @@ impl SessionManager {
|
||||
}
|
||||
|
||||
/// Invoked on a received status update
|
||||
pub(crate) fn on_status_update(&mut self, status: StatusUpdate) {
|
||||
self.status.blockhash = status.hash;
|
||||
self.status.total_difficulty = status.total_difficulty;
|
||||
self.fork_filter.set_head(status.height);
|
||||
pub(crate) fn on_status_update(&mut self, height: u64, hash: H256, total_difficulty: U256) {
|
||||
self.status.blockhash = hash;
|
||||
self.status.total_difficulty = total_difficulty;
|
||||
self.fork_filter.set_head(height);
|
||||
}
|
||||
|
||||
/// An incoming TCP connection was received. This starts the authentication process to turn this
|
||||
|
||||
@ -3,7 +3,7 @@
|
||||
use crate::{
|
||||
cache::LruCache,
|
||||
discovery::{Discovery, DiscoveryEvent},
|
||||
fetch::{BlockResponseOutcome, FetchAction, StateFetcher, StatusUpdate},
|
||||
fetch::{BlockResponseOutcome, FetchAction, StateFetcher},
|
||||
message::{
|
||||
BlockRequest, NewBlockMessage, PeerRequest, PeerRequestSender, PeerResponse,
|
||||
PeerResponseResult,
|
||||
@ -353,10 +353,6 @@ where
|
||||
FetchAction::BlockRequest { peer_id, request } => {
|
||||
self.handle_block_request(peer_id, request)
|
||||
}
|
||||
FetchAction::StatusUpdate(status) => {
|
||||
// we want to return this directly
|
||||
return Poll::Ready(StateAction::StatusUpdate(status))
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@ -427,8 +423,6 @@ pub(crate) struct ActivePeer {
|
||||
|
||||
/// Message variants triggered by the [`State`]
|
||||
pub(crate) enum StateAction {
|
||||
/// Received a node status update.
|
||||
StatusUpdate(StatusUpdate),
|
||||
/// Dispatch a `NewBlock` message to the peer
|
||||
NewBlock {
|
||||
/// Target of the message
|
||||
|
||||
@ -1,5 +1,4 @@
|
||||
use crate::{
|
||||
fetch::StatusUpdate,
|
||||
listener::{ConnectionListener, ListenerEvent},
|
||||
message::{PeerMessage, PeerRequestSender},
|
||||
peers::InboundConnectionError,
|
||||
@ -221,7 +220,6 @@ where
|
||||
let msg = PeerMessage::NewBlockHashes(hashes);
|
||||
self.sessions.send_message(&peer_id, msg);
|
||||
}
|
||||
StateAction::StatusUpdate(status) => return Some(SwarmEvent::StatusUpdate(status)),
|
||||
}
|
||||
None
|
||||
}
|
||||
@ -279,8 +277,6 @@ where
|
||||
/// All events created or delegated by the [`Swarm`] that represents changes to the state of the
|
||||
/// network.
|
||||
pub(crate) enum SwarmEvent {
|
||||
/// Received a node status update.
|
||||
StatusUpdate(StatusUpdate),
|
||||
/// Events related to the actual network protocol.
|
||||
ValidMessage {
|
||||
/// The peer that sent the message
|
||||
|
||||
@ -14,7 +14,8 @@ reth-interfaces = { path = "../interfaces" }
|
||||
reth-executor = { path = "../executor" }
|
||||
reth-rlp = { path = "../common/rlp" }
|
||||
reth-db = { path = "../storage/db" }
|
||||
reth-provider = { path = "../storage/provider"}
|
||||
reth-provider = { path = "../storage/provider" }
|
||||
reth-network = { path = "../net/network" }
|
||||
|
||||
#async
|
||||
tokio = { version = "1.21.2", features = ["sync"] }
|
||||
|
||||
@ -13,7 +13,7 @@ use reth_db::{
|
||||
use reth_interfaces::{
|
||||
consensus::{Consensus, ForkchoiceState},
|
||||
p2p::headers::{
|
||||
client::HeadersClient,
|
||||
client::{HeadersClient, StatusUpdater},
|
||||
downloader::{ensure_parent, HeaderDownloader},
|
||||
error::DownloadError,
|
||||
},
|
||||
@ -40,20 +40,22 @@ const HEADERS: StageId = StageId("Headers");
|
||||
/// [`HeaderTD`][reth_interfaces::db::tables::HeaderTD] table). The stage does not return the
|
||||
/// control flow to the pipeline in order to preserve the context of the chain tip.
|
||||
#[derive(Debug)]
|
||||
pub struct HeaderStage<D: HeaderDownloader, C: Consensus, H: HeadersClient> {
|
||||
pub struct HeaderStage<D: HeaderDownloader, C: Consensus, H: HeadersClient, S: StatusUpdater> {
|
||||
/// Strategy for downloading the headers
|
||||
pub downloader: D,
|
||||
/// Consensus client implementation
|
||||
pub consensus: Arc<C>,
|
||||
/// Downloader client implementation
|
||||
pub client: Arc<H>,
|
||||
/// Network handle for updating status
|
||||
pub network_handle: S,
|
||||
/// The number of block headers to commit at once
|
||||
pub commit_threshold: usize,
|
||||
}
|
||||
|
||||
#[async_trait::async_trait]
|
||||
impl<DB: Database, D: HeaderDownloader, C: Consensus, H: HeadersClient> Stage<DB>
|
||||
for HeaderStage<D, C, H>
|
||||
impl<DB: Database, D: HeaderDownloader, C: Consensus, H: HeadersClient, S: StatusUpdater> Stage<DB>
|
||||
for HeaderStage<D, C, H, S>
|
||||
{
|
||||
/// Return the id of the stage
|
||||
fn id(&self) -> StageId {
|
||||
@ -146,7 +148,9 @@ impl<DB: Database, D: HeaderDownloader, C: Consensus, H: HeadersClient> Stage<DB
|
||||
}
|
||||
}
|
||||
|
||||
impl<D: HeaderDownloader, C: Consensus, H: HeadersClient> HeaderStage<D, C, H> {
|
||||
impl<D: HeaderDownloader, C: Consensus, H: HeadersClient, S: StatusUpdater>
|
||||
HeaderStage<D, C, H, S>
|
||||
{
|
||||
async fn update_head<DB: Database>(
|
||||
&self,
|
||||
db: &StageDB<'_, DB>,
|
||||
@ -156,7 +160,8 @@ impl<D: HeaderDownloader, C: Consensus, H: HeadersClient> HeaderStage<D, C, H> {
|
||||
let td: U256 = *db
|
||||
.get::<tables::HeaderTD>(block_key)?
|
||||
.ok_or(DatabaseIntegrityError::TotalDifficulty { number: height })?;
|
||||
self.client.update_status(height, block_key.hash(), td);
|
||||
// self.client.update_status(height, block_key.hash(), td);
|
||||
self.network_handle.update_status(height, block_key.hash(), td);
|
||||
Ok(())
|
||||
}
|
||||
|
||||
@ -366,7 +371,7 @@ mod tests {
|
||||
p2p::headers::downloader::HeaderDownloader,
|
||||
test_utils::{
|
||||
generators::{random_header, random_header_range},
|
||||
TestConsensus, TestHeaderDownloader, TestHeadersClient,
|
||||
TestConsensus, TestHeaderDownloader, TestHeadersClient, TestStatusUpdater,
|
||||
},
|
||||
};
|
||||
use reth_primitives::{BlockNumber, SealedHeader, U256};
|
||||
@ -376,6 +381,7 @@ mod tests {
|
||||
pub(crate) consensus: Arc<TestConsensus>,
|
||||
pub(crate) client: Arc<TestHeadersClient>,
|
||||
downloader: Arc<D>,
|
||||
network_handle: TestStatusUpdater,
|
||||
db: TestStageDB,
|
||||
}
|
||||
|
||||
@ -387,13 +393,14 @@ mod tests {
|
||||
client: client.clone(),
|
||||
consensus: consensus.clone(),
|
||||
downloader: Arc::new(TestHeaderDownloader::new(client, consensus, 1000)),
|
||||
network_handle: TestStatusUpdater::default(),
|
||||
db: TestStageDB::default(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
impl<D: HeaderDownloader + 'static> StageTestRunner for HeadersTestRunner<D> {
|
||||
type S = HeaderStage<Arc<D>, TestConsensus, TestHeadersClient>;
|
||||
type S = HeaderStage<Arc<D>, TestConsensus, TestHeadersClient, TestStatusUpdater>;
|
||||
|
||||
fn db(&self) -> &TestStageDB {
|
||||
&self.db
|
||||
@ -404,6 +411,7 @@ mod tests {
|
||||
consensus: self.consensus.clone(),
|
||||
client: self.client.clone(),
|
||||
downloader: self.downloader.clone(),
|
||||
network_handle: self.network_handle.clone(),
|
||||
commit_threshold: 100,
|
||||
}
|
||||
}
|
||||
@ -504,7 +512,13 @@ mod tests {
|
||||
let downloader = Arc::new(
|
||||
LinearDownloadBuilder::default().build(consensus.clone(), client.clone()),
|
||||
);
|
||||
Self { client, consensus, downloader, db: TestStageDB::default() }
|
||||
Self {
|
||||
client,
|
||||
consensus,
|
||||
downloader,
|
||||
network_handle: TestStatusUpdater::default(),
|
||||
db: TestStageDB::default(),
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
Reference in New Issue
Block a user