Skip to content
This repository has been archived by the owner on Nov 15, 2023. It is now read-only.

Add a DirectedGossip struct #6803

Merged
13 commits merged into from
Aug 17, 2020
2 changes: 1 addition & 1 deletion client/network/Cargo.toml
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ targets = ["x86_64-unknown-linux-gnu"]
prost-build = "0.6.1"

[dependencies]
async-std = { version = "1.6.2", features = ["unstable"] }
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do I understand correctly that this uses the unstable feature to have futures-timer? If so, should we not do that across the entire crate for all usage of futures-timer?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

No, it is for Convar. It is unfortunate that we have to depend on the unstable feature, but I couldn't find any crate other than async-std that provides an asynchronous Condvar.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If this is something we want to avoid, one can use a channel as a condvar.

use futures::channel::mpsc::channel;

fn main() {
    let (tx, rx) = channel(0);
    
    let producer = async move {
        // Produce something and put it to place X.
        
        // Signal the consumer that that something can be optained.
        tx.send(()).await;
    }
    
    let consumer = async move {
        // Wait for the signal. Maybe include a timeout like you do today with 10 secs.
        let _ = rx.next().await;
        
        // Access place X.
    }
}

Problem is that this can't be used in the Drop implementations and thus cleanup would depend on the 10 sec timeout.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Problem is that this can't be used in the Drop implementations [...]

It does not always need to. When the Sender is dropped, the receiver will notice and the task can terminate. Instead of let _ = rx.next().await; one would write if rx.next().await.is_some() { ... }. In the Drop impl of QueueGuard one can use Sender::try_send.

Copy link
Contributor Author

@tomaka tomaka Aug 13, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I find it a bit weird to use a channel, which involves an additional Arc, Mutex and Vec, just to wake up a task, rather than a Waker.

But I have now also tried using a Waker, and the implementation is considerably more tricky and difficult to read because of potential race conditions and having to introduce manual polling within an async function and having to implement your own Waker.

Before going on, I'd like to understand what is wrong with the Condvar solution, as a Condvar is exactly the tool that is designed for this specific job.

bitflags = "1.2.0"
bs58 = "0.3.1"
bytes = "0.5.0"
Expand Down Expand Up @@ -66,7 +67,6 @@ default-features = false
features = ["identify", "kad", "mdns", "mplex", "noise", "ping", "tcp-async-std", "websocket", "yamux"]

[dev-dependencies]
async-std = "1.6.2"
assert_matches = "1.3"
env_logger = "0.7.0"
libp2p = { version = "0.22.0", default-features = false, features = ["secio"] }
Expand Down
312 changes: 312 additions & 0 deletions client/network/src/gossip.rs
Original file line number Diff line number Diff line change
@@ -0,0 +1,312 @@
// This file is part of Substrate.

// Copyright (C) 2017-2020 Parity Technologies (UK) Ltd.
// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0

// This program is free software: you can redistribute it and/or modify
// it under the terms of the GNU General Public License as published by
// the Free Software Foundation, either version 3 of the License, or
// (at your option) any later version.

// This program is distributed in the hope that it will be useful,
// but WITHOUT ANY WARRANTY; without even the implied warranty of
// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
// GNU General Public License for more details.

// You should have received a copy of the GNU General Public License
// along with this program. If not, see <https://www.gnu.org/licenses/>.

//! Helper for sending rate-limited gossip messages.
tomaka marked this conversation as resolved.
Show resolved Hide resolved
//!
//! # Context
//!
//! The [`NetworkService`] struct provides a way to send notifications to a certain peer through
//! the [`NetworkService::notification_sender`] method. This method is quite low level and isn't
//! expected to be used directly.
//!
//! The [`DirectedGossip`] struct provided by this module is built on top of
//! [`NetworkService::notification_sender`] and provides a cleaner way to send notifications.
//!
//! # Behaviour
//!
//! An instance of [`DirectedGossip`] is specific to a certain combination of `PeerId` and
//! protocol name. It maintains a buffer of messages waiting to be sent out. The user of this API
//! is able to manipulate that queue, adding or removing obsolete messages.
//!
//! Creating a [`DirectedGossip`] also returns a opaque `Future` whose responsibility it to
//! drain that queue and actually send the messages. If the substream with the given combination
//! of peer and protocol is closed, the queue is silently discarded. It is the role of the user
//! to track which peers we are connected to.
//!
//! In normal situations, messages sent through a [`DirectedGossip`] will arrive in the same
//! order as they have been sent.
//! It is possible, in the situation of disconnects and reconnects, that messages arrive in a
//! different order. See also https://github.com/paritytech/substrate/issues/6756.
//! However, if multiple instances of [`DirectedGossip`] exist for the same peer and protocol, or
//! if some other code uses the [`NetworkService`] to send notifications to this combination or
//! peer and protocol, then the notifications will be interleaved in an unpredictable way.
//!

use crate::{ExHashT, NetworkService, service::{NotificationSender, NotificationSenderError}};

use async_std::sync::{Condvar, Mutex, MutexGuard};
use futures::prelude::*;
use libp2p::PeerId;
use sp_runtime::{traits::Block as BlockT, ConsensusEngineId};
use std::{
collections::VecDeque,
fmt,
sync::{atomic, Arc},
time::Duration,
};

#[cfg(test)]
mod tests;

/// Notifications sender for a specific combination of network service, peer, and protocol.
pub struct DirectedGossip<M> {
/// Shared between the front and the back task.
shared: Arc<Shared<M>>,
}

impl<M> DirectedGossip<M> {
/// Returns a new [`DirectedGossip`] containing a queue of message for this specific
/// combination of peer and protocol.
///
/// In addition to the [`DirectedGossip`], also returns a `Future` whose role is to drive
/// the messages sending forward.
pub fn new<B, H, F>(
service: Arc<NetworkService<B, H>>,
peer_id: PeerId,
protocol: ConsensusEngineId,
queue_size_limit: usize,
messages_encode: F
) -> (Self, impl Future<Output = ()> + Send + 'static)
where
M: Send + 'static,
B: BlockT + 'static,
H: ExHashT,
F: Fn(M) -> Vec<u8> + Send + 'static,
{
DirectedGossipPrototype::new(service, peer_id)
.build(protocol, queue_size_limit, messages_encode)
}

/// Locks the queue of messages towards this peer.
///
/// The returned `Future` is expected to be ready quite quickly.
pub async fn lock_queue<'a>(&'a self) -> QueueLock<'a, M> {
QueueLock {
messages_queue: self.shared.messages_queue.lock().await,
condvar: &self.shared.condvar,
queue_size_limit: self.shared.queue_size_limit,
}
}

/// Pushes a message to the queue, or discards it if the queue is full.
///
/// The returned `Future` is expected to be ready quite quickly.
pub async fn queue_or_discard(&self, message: M)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Why offer two ways to queue_or_discard? Is writing self.lock_queue().await.push_or_discard(message); as a user not fine as well?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This API could return a Result<(), M> or similar as well

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's a convenient shortcut.

where
M: Send + 'static
{
self.lock_queue().await.push_or_discard(message);
}
}

impl<M> fmt::Debug for DirectedGossip<M> {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_struct("DirectedGossip").finish()
}
}

impl<M> Drop for DirectedGossip<M> {
fn drop(&mut self) {
// The "clean" way to notify the `Condvar` here is normally to first lock the `Mutex`,
// then notify the `Condvar` while the `Mutex` is locked. Unfortunately, the `Mutex`
// being asynchronous, it can't reasonably be locked from within a destructor.
// For this reason, this destructor is a "best effort" destructor.
tomaka marked this conversation as resolved.
Show resolved Hide resolved
// See also the corresponding code in the background task.
self.shared.stop_task.store(true, atomic::Ordering::Release);
self.shared.condvar.notify_all();
mxinden marked this conversation as resolved.
Show resolved Hide resolved
}
}

/// Utility. Generic over the type of the messages. Holds a [`NetworkService`] and a [`PeerId`].
/// Provides a [`DirectedGossipPrototype::build`] function that builds a [`DirectedGossip`].
#[derive(Clone)]
pub struct DirectedGossipPrototype {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is pub so I assume this will be used from somewhere external? Could you explain this a little more please? Currently it is not in use and the DirectedGossip and task construction could happen directly in DirectedGossip::new.

Copy link
Contributor Author

@tomaka tomaka Aug 4, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I admit that this is type is specifically targeted for the Polkadot use case.

In Polkadot's code, the so-called network bridge communicates via messages with other subsystems. For example, when we connect, a PeerConnected message is sent.

The network bridge doesn't know what is the type of the networking messages that the various subsystems would manipulate (the M generic in this PR), and the various subsystems don't know what the B and H generics would be.

This DirectedGossipPrototype would easily plug into that scheme. We would send a DirectedGossipPrototype to the subsystems, without the need for the subsystem to know what B and H would be.

Copy link
Contributor

@twittner twittner Aug 4, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should Polkadot-specific code not be put in Polkadot?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The network bridge doesn't know what is the type of the networking messages that the various subsystems would manipulate (the M generic in this PR), and the various subsystems don't know what the B and H generics would be.

I am actually changing this right now, so we may not need this type. Will push a PR shortly

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

paritytech/polkadot#1535 - now the network bridge is aware of the specific information flowing over the network and we may be able to avoid these prototypes. However, different subsystems will still want Senders for different types. However, the sender for a specific variant should also be encoding as |x| ValidationProtocolV1::SomeConcreteVariant(x).encode() which I think is covered by the encode-fn part of the API here

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should Polkadot-specific code not be put in Polkadot?

To answer specifically this: the way I see it, this code isn't really Polkadot-specific.
This code provides a convenient-to-use high-level API on top of a low-level primitive, and people are free to use either the convenient API if it suits them, or the low-level primitive if it doesn't suit them.

I think that, in general, we want to be as "universal" as possible when designing low-level code, but as long as the lower-level primitives are exposed, higher-level code can be a bit more targeted towards certain use-cases.

service: Arc<dyn AbstractNotificationSender + Send + Sync + 'static>,
peer_id: PeerId,
}

impl DirectedGossipPrototype {
/// Builds a new [`DirectedGossipPrototype`] containing the given components.
pub fn new<B, H>(
service: Arc<NetworkService<B, H>>,
peer_id: PeerId,
) -> Self
where
B: BlockT + 'static,
H: ExHashT,
{
DirectedGossipPrototype {
service,
peer_id,
}
}

/// Turns this [`DirectedGossipPrototype`] into a [`DirectedGossip`] and a future.
///
/// See [`DirectGossip::new`] for details.
pub fn build<M, F>(
self,
protocol: ConsensusEngineId,
queue_size_limit: usize,
messages_encode: F
) -> (DirectedGossip<M>, impl Future<Output = ()> + Send + 'static)
where
M: Send + 'static,
F: Fn(M) -> Vec<u8> + Send + 'static,
{
let shared = Arc::new(Shared {
stop_task: atomic::AtomicBool::new(false),
condvar: Condvar::new(),
queue_size_limit,
messages_queue: Mutex::new(VecDeque::with_capacity(queue_size_limit)),
});

let task = spawn_task(
self.service,
self.peer_id,
protocol,
shared.clone(),
messages_encode
);

(DirectedGossip { shared }, task)
}
}

impl fmt::Debug for DirectedGossipPrototype {
fn fmt(&self, f: &mut fmt::Formatter) -> fmt::Result {
f.debug_struct("DirectedGossipPrototype")
.field("peer_id", &self.peer_id)
.finish()
}
}

/// Locked queue of messages to the given peer.
///
/// As long as this struct exists, the background task is asleep and the owner of the [`QueueLock`]
/// is in total control of the buffer.
tomaka marked this conversation as resolved.
Show resolved Hide resolved
#[must_use]
pub struct QueueLock<'a, M> {
tomaka marked this conversation as resolved.
Show resolved Hide resolved
messages_queue: MutexGuard<'a, VecDeque<M>>,
condvar: &'a Condvar,
/// Same as [`Shared::queue_size_limit`].
queue_size_limit: usize,
}

impl<'a, M: Send + 'static> QueueLock<'a, M> {
tomaka marked this conversation as resolved.
Show resolved Hide resolved
/// Pushes a message to the queue, or discards it if the queue is full.
pub fn push_or_discard(&mut self, message: M) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Same here. It would be good to get back the message if the queue is full

Copy link
Contributor Author

@tomaka tomaka Aug 4, 2020

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But what would you do with the message that is returned? Put it in another queue?
The only sane things you can do when the queue is full is either discard the message or force-close the connection.

Additionally, what if the connection with the remote is closed? Are we supposed to return back the message as well? If so, then it's very problematic because we can't detect this.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The only sane things you can do when the queue is full is either discard the message or force-close the connection.

Isn't retain another option? We could decide to do that based on the type of the message we are trying to send.

And is there no sane thing you can do? Why can't you wait for space to appear in the queue or something like that?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The way I see retain is that you'd call it all the time, even if there is space in the buffer. As far as I can tell, when a message is obsolete, there is no point in leaving it in the queue anyway.

Why can't you wait for space to appear in the queue or something like that?

The entire reason for this API to exist is to remove the need for any waiting. See also this paragraph.

Ultimately there has to be a code somewhere that holds some sort of HashMap<PeerId, DirectedGossip>. If it needs to send a message to one of the peers and its buffer is full, then it shouldn't wait for this peer and instead continue its processing.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It would be nice to have a way to wait, although I agree that the API for that should be used sparingly so you don't degrade to the performance of the slowest peer. There are cases where we don't want to drop messages, for instance when responding to a validator's request.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There are cases where we don't want to drop messages

I believe that everything that would fall in this category should be covered by request-response protocols.

Rather than adding a wait, I could restore the push_unbounded method that I've removed after a review.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@rphmeier @tomaka is this conversation resolved?

if self.messages_queue.len() < self.queue_size_limit {
self.messages_queue.push_back(message);
}
}

/// Calls `filter` for each message in the queue, and removes the ones for which `false` is
/// returned.
///
/// > **Note**: The parameter of `filter` is a `&M` and not a `&mut M` (which would be
/// > better) because the underlying implementation relies on `VecDeque::retain`.
pub fn retain(&mut self, filter: impl FnMut(&M) -> bool) {
self.messages_queue.retain(filter);
}
}

impl<'a, M> Drop for QueueLock<'a, M> {
fn drop(&mut self) {
// We notify the `Condvar` in the destructor in order to be able to push multiple
// messages and wake up the background task only once afterwards.
self.condvar.notify_one();
}
}

#[derive(Debug)]
struct Shared<M> {
/// Read by the background task after locking `locked`. If true, the task stops.
stop_task: atomic::AtomicBool,
/// Queue of messages waiting to be sent out.
messages_queue: Mutex<VecDeque<M>>,
/// Must be notified every time the content of `locked` changes.
condvar: Condvar,
/// Maximum number of elements in `messages_queue`.
queue_size_limit: usize,
}

async fn spawn_task<M, F: Fn(M) -> Vec<u8>>(
service: Arc<dyn AbstractNotificationSender + Send + Sync + 'static>,
peer_id: PeerId,
protocol: ConsensusEngineId,
shared: Arc<Shared<M>>,
messages_encode: F,
) {
loop {
let next_message = 'next_msg: loop {
let mut lock = shared.messages_queue.lock().await;
tomaka marked this conversation as resolved.
Show resolved Hide resolved

loop {
if shared.stop_task.load(atomic::Ordering::Acquire) {
return;
}

if let Some(msg) = lock.pop_front() {
break 'next_msg msg;
}

// It is possible that the destructor of `DirectedGossip` sets `stop_task` to
// true and notifies the `Condvar` after the background task loads `stop_task`
// and before it calls `Condvar::wait`.
// See also the corresponding comment in `DirectedGossip::drop`.
// For this reason, we use `wait_timeout`. In the worst case scenario,
// `stop_task` will always be checked again after the timeout is reached.
lock = shared.condvar.wait_timeout(lock, Duration::from_secs(10)).await.0;
}
};

// Starting from below, we try to send the message. If an error happens when sending,
// the only sane option we have is to silently discard the message.
let sender = match service.notification_sender(peer_id.clone(), protocol) {
Ok(s) => s,
Err(_) => continue,
};

let ready = match sender.ready().await {
Ok(r) => r,
Err(_) => continue,
};

let _ = ready.send(messages_encode(next_message));
}
}

/// Abstraction around `NetworkService` that permits removing the `B` and `H` parameters.
trait AbstractNotificationSender {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I suggest to remove this trait. The two type parameters B and H are only necessary in the private spawn_task method and adding those is far less code than this trait plus impl. And your calls to NetworkService::notification_sender are not dynamically dispatched.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It's necessary for DirectedGossipPrototype to work. See your other comment.

fn notification_sender(
&self,
target: PeerId,
engine_id: ConsensusEngineId,
) -> Result<NotificationSender, NotificationSenderError>;
}

impl<B: BlockT, H: ExHashT> AbstractNotificationSender for NetworkService<B, H> {
fn notification_sender(
&self,
target: PeerId,
engine_id: ConsensusEngineId,
) -> Result<NotificationSender, NotificationSenderError> {
NetworkService::notification_sender(self, target, engine_id)
}
}
Loading