Skip to content

Commit

Permalink
fix(security): Rate-limit and size-limit peer transaction ID messages (
Browse files Browse the repository at this point in the history
…#6625)

* Update MAX_TX_INV_IN_MESSAGE for ZIP-239 WTX IDs

* Combine multiple transaction updates into a single gossip & rate-limit gossips

* Rate-limit block gossips

* Fix mempool_transaction_expiration gossip test timings

* Enforce MAX_TX_INV_IN_MESSAGE in the network layer, rather than each service

* Fix documentation for `Message::Tx`

* Split MAX_INV_IN_RECEIVED_MESSAGE and MAX_TX_INV_IN_SENT_MESSAGE

* Fix log message typo

* Move some docs to/from another PR

---------

Co-authored-by: mergify[bot] <37929162+mergify[bot]@users.noreply.github.com>
  • Loading branch information
teor2345 and mergify[bot] authored May 15, 2023
1 parent b0d9471 commit dc51989
Show file tree
Hide file tree
Showing 12 changed files with 146 additions and 69 deletions.
2 changes: 1 addition & 1 deletion zebra-network/src/lib.rs
Original file line number Diff line number Diff line change
Expand Up @@ -181,7 +181,7 @@ pub use crate::{
peer_set::init,
policies::RetryLimit,
protocol::{
external::{Version, VersionMessage},
external::{Version, VersionMessage, MAX_TX_INV_IN_SENT_MESSAGE},
internal::{InventoryResponse, Request, Response},
},
};
Expand Down
55 changes: 48 additions & 7 deletions zebra-network/src/peer/connection.rs
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@ use crate::{
external::{types::Nonce, InventoryHash, Message},
internal::{InventoryResponse, Request, Response},
},
BoxError,
BoxError, MAX_TX_INV_IN_SENT_MESSAGE,
};

use InventoryResponse::*;
Expand Down Expand Up @@ -992,9 +992,31 @@ where
)
}
(AwaitingRequest, AdvertiseTransactionIds(hashes)) => {
let max_tx_inv_in_message: usize = MAX_TX_INV_IN_SENT_MESSAGE
.try_into()
.expect("constant fits in usize");

// # Security
//
// In most cases, we try to split over-sized requests into multiple network-layer
// messages. But we are unlikely to reach this limit with the default mempool
// config, so a gossip like this could indicate a network amplification attack.
//
// This limit is particularly important here, because advertisements send the same
// message to half our available peers.
//
// If there are thousands of transactions in the mempool, letting peers know the
// exact transactions we have isn't that important, so it's ok to drop arbitrary
// transaction hashes from our response.
if hashes.len() > max_tx_inv_in_message {
debug!(inv_count = ?hashes.len(), ?MAX_TX_INV_IN_SENT_MESSAGE, "unusually large transaction ID gossip");
}

let hashes = hashes.into_iter().take(max_tx_inv_in_message).map(Into::into).collect();

self
.peer_tx
.send(Message::Inv(hashes.iter().map(|h| (*h).into()).collect()))
.send(Message::Inv(hashes))
.await
.map(|()|
Handler::Finished(Ok(Response::Nil))
Expand Down Expand Up @@ -1351,11 +1373,30 @@ where
}
}
Response::TransactionIds(hashes) => {
if let Err(e) = self
.peer_tx
.send(Message::Inv(hashes.into_iter().map(Into::into).collect()))
.await
{
let max_tx_inv_in_message: usize = MAX_TX_INV_IN_SENT_MESSAGE
.try_into()
.expect("constant fits in usize");

// # Security
//
// In most cases, we try to split over-sized responses into multiple network-layer
// messages. But we are unlikely to reach this limit with the default mempool
// config, so a response like this could indicate a network amplification attack.
//
// If there are thousands of transactions in the mempool, letting peers know the
// exact transactions we have isn't that important, so it's ok to drop arbitrary
// transaction hashes from our response.
if hashes.len() > max_tx_inv_in_message {
debug!(inv_count = ?hashes.len(), ?MAX_TX_INV_IN_SENT_MESSAGE, "unusually large transaction ID response");
}

let hashes = hashes
.into_iter()
.take(max_tx_inv_in_message)
.map(Into::into)
.collect();

if let Err(e) = self.peer_tx.send(Message::Inv(hashes)).await {
self.fail_with(e)
}
}
Expand Down
2 changes: 1 addition & 1 deletion zebra-network/src/protocol/external.rs
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,7 @@ mod tests;

pub use addr::{canonical_peer_addr, canonical_socket_addr, AddrInVersion};
pub use codec::Codec;
pub use inv::InventoryHash;
pub use inv::{InventoryHash, MAX_TX_INV_IN_SENT_MESSAGE};
pub use message::{Message, VersionMessage};
pub use types::{Nonce, Version};

Expand Down
22 changes: 18 additions & 4 deletions zebra-network/src/protocol/external/inv.rs
Original file line number Diff line number Diff line change
Expand Up @@ -179,12 +179,26 @@ impl ZcashDeserialize for InventoryHash {
/// The minimum serialized size of an [`InventoryHash`].
pub(crate) const MIN_INV_HASH_SIZE: usize = 36;

/// The maximum number of transaction inventory items in a network message.
/// We also use this limit for block inventory, because it is typically much smaller.
/// The maximum number of inventory items in a network message received from a peer.
///
/// After [ZIP-239](https://zips.z.cash/zip-0239#deployment), this would allow a message filled
/// with `MSG_WTX` entries to be around 3.4 MB, so we also need a separate constant to limit the
/// number of `inv` entries that we send.
///
/// Same as `MAX_INV_SZ` in `zcashd`:
/// <https://github.com/zcash/zcash/blob/adfc7218435faa1c8985a727f997a795dcffa0c7/src/net.h#L50>
pub const MAX_TX_INV_IN_MESSAGE: u64 = 50_000;
pub const MAX_INV_IN_RECEIVED_MESSAGE: u64 = 50_000;

/// The maximum number of transaction inventory items in a network message received from a peer.
///
/// After [ZIP-239](https://zips.z.cash/zip-0239#deployment), this would allow a message filled
/// with `MSG_WTX` entries to be around 3.4 MB, so we also need a separate constant to limit the
/// number of `inv` entries that we send.
///
/// This constant is not critical to compatibility: it just needs to be less than or equal to
/// `zcashd`'s `MAX_INV_SZ`:
/// <https://github.com/zcash/zcash/blob/adfc7218435faa1c8985a727f997a795dcffa0c7/src/net.h#L50>
pub const MAX_TX_INV_IN_SENT_MESSAGE: u64 = 25_000;

impl TrustedPreallocate for InventoryHash {
fn max_allocation() -> u64 {
Expand All @@ -193,6 +207,6 @@ impl TrustedPreallocate for InventoryHash {
// a single message
let message_size_limit = ((MAX_PROTOCOL_MESSAGE_LEN - 1) / MIN_INV_HASH_SIZE) as u64;

min(message_size_limit, MAX_TX_INV_IN_MESSAGE)
min(message_size_limit, MAX_INV_IN_RECEIVED_MESSAGE)
}
}
6 changes: 5 additions & 1 deletion zebra-network/src/protocol/external/message.rs
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,11 @@ pub enum Message {

/// A `tx` message.
///
/// This message is used to advertise unmined transactions for the mempool.
/// This message can be used to:
/// - send unmined transactions in response to `GetData` requests, and
/// - advertise unmined transactions for the mempool.
///
/// Zebra chooses to advertise new transactions using `Inv(hash)` rather than `Tx(transaction)`.
///
/// [Bitcoin reference](https://en.bitcoin.it/wiki/Protocol_documentation#tx)
Tx(UnminedTx),
Expand Down
4 changes: 2 additions & 2 deletions zebra-network/src/protocol/external/tests/preallocate.rs
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,7 @@ use crate::{
meta_addr::MetaAddr,
protocol::external::{
addr::{AddrV1, AddrV2, ADDR_V1_SIZE, ADDR_V2_MIN_SIZE},
inv::{InventoryHash, MAX_TX_INV_IN_MESSAGE},
inv::{InventoryHash, MAX_INV_IN_RECEIVED_MESSAGE},
},
};

Expand Down Expand Up @@ -66,7 +66,7 @@ proptest! {
//
// Special case: Zcash has a slightly smaller limit for transaction invs,
// so we use it for all invs.
prop_assert!(smallest_disallowed_serialized.len() > min(MAX_PROTOCOL_MESSAGE_LEN, usize::try_from(MAX_TX_INV_IN_MESSAGE).expect("fits in usize")));
prop_assert!(smallest_disallowed_serialized.len() > min(MAX_PROTOCOL_MESSAGE_LEN, usize::try_from(MAX_INV_IN_RECEIVED_MESSAGE).expect("fits in usize")));

// Create largest_allowed_vec by removing one element from smallest_disallowed_vec without copying (for efficiency)
smallest_disallowed_vec.pop();
Expand Down
9 changes: 3 additions & 6 deletions zebra-network/src/protocol/internal/request.rs
Original file line number Diff line number Diff line change
Expand Up @@ -148,14 +148,11 @@ pub enum Request {

/// Advertise a set of unmined transactions to all peers.
///
/// This is intended to be used in Zebra with a single transaction at a time
/// (set of size 1), but multiple transactions are permitted because this is
/// how we interpret advertisements from zcashd, which sometimes advertises
/// multiple transactions at once.
/// Both Zebra and zcashd sometimes advertise multiple transactions at once.
///
/// This is implemented by sending an `inv` message containing the unmined
/// transaction ID, allowing the remote peer to choose whether to download
/// it. Remote peers who choose to download the transaction will generate a
/// transaction IDs, allowing the remote peer to choose whether to download
/// them. Remote peers who choose to download the transaction will generate a
/// [`Request::TransactionsById`] against the "inbound" service passed to
/// [`init`](crate::init).
///
Expand Down
1 change: 1 addition & 0 deletions zebrad/src/components/inbound.rs
Original file line number Diff line number Diff line change
Expand Up @@ -468,6 +468,7 @@ impl Service<zn::Request> for Inbound {
block_downloads.download_and_verify(hash);
async { Ok(zn::Response::Nil) }.boxed()
}
// The size of this response is limited by the `Connection` state machine in the network layer
zn::Request::MempoolTransactionIds => {
mempool.clone().oneshot(mempool::Request::TransactionIds).map_ok(|resp| match resp {
mempool::Response::TransactionIds(transaction_ids) if transaction_ids.is_empty() => zn::Response::Nil,
Expand Down
27 changes: 10 additions & 17 deletions zebrad/src/components/inbound/tests/fake_peer_set.rs
Original file line number Diff line number Diff line change
Expand Up @@ -35,7 +35,7 @@ use crate::{
gossip_mempool_transaction_id, unmined_transactions_in_blocks, Config as MempoolConfig,
Mempool, MempoolError, SameEffectsChainRejectionError, UnboxMempoolError,
},
sync::{self, BlockGossipError, SyncStatus},
sync::{self, BlockGossipError, SyncStatus, TIPS_RESPONSE_TIMEOUT},
},
BoxError,
};
Expand Down Expand Up @@ -420,7 +420,8 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
let mut hs = HashSet::new();
hs.insert(tx1_id);

// Transaction and Block IDs are gossipped, in any order
// Transaction and Block IDs are gossipped, in any order, after waiting for the gossip delay
tokio::time::sleep(TIPS_RESPONSE_TIMEOUT).await;
let possible_requests = &mut [
Request::AdvertiseTransactionIds(hs),
Request::AdvertiseBlock(block_two.hash()),
Expand Down Expand Up @@ -488,7 +489,8 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
.await
.unwrap();

// Block is gossiped
// Test the block is gossiped, after waiting for the multi-gossip delay
tokio::time::sleep(TIPS_RESPONSE_TIMEOUT).await;
peer_set
.expect_request(Request::AdvertiseBlock(block_three.hash()))
.await
Expand Down Expand Up @@ -564,7 +566,9 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
MempoolError::StorageEffectsChain(SameEffectsChainRejectionError::Expired)
);

// Test transaction 2 is gossiped
// Test transaction 2 is gossiped, after waiting for the multi-gossip delay
tokio::time::sleep(TIPS_RESPONSE_TIMEOUT).await;

let mut hs = HashSet::new();
hs.insert(tx2_id);
peer_set
Expand All @@ -583,18 +587,6 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
zebra_test::vectors::BLOCK_MAINNET_6_BYTES
.zcash_deserialize_into()
.unwrap(),
zebra_test::vectors::BLOCK_MAINNET_7_BYTES
.zcash_deserialize_into()
.unwrap(),
zebra_test::vectors::BLOCK_MAINNET_8_BYTES
.zcash_deserialize_into()
.unwrap(),
zebra_test::vectors::BLOCK_MAINNET_9_BYTES
.zcash_deserialize_into()
.unwrap(),
zebra_test::vectors::BLOCK_MAINNET_10_BYTES
.zcash_deserialize_into()
.unwrap(),
];
for block in more_blocks {
state_service
Expand All @@ -605,7 +597,8 @@ async fn mempool_transaction_expiration() -> Result<(), crate::BoxError> {
.await
.unwrap();

// Block is gossiped
// Test the block is gossiped, after waiting for the multi-gossip delay
tokio::time::sleep(TIPS_RESPONSE_TIMEOUT).await;
peer_set
.expect_request(Request::AdvertiseBlock(block.hash()))
.await
Expand Down
11 changes: 4 additions & 7 deletions zebrad/src/components/mempool.rs
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,7 @@ use std::{
};

use futures::{future::FutureExt, stream::Stream};
use tokio::sync::watch;
use tokio::sync::broadcast;
use tower::{buffer::Buffer, timeout::Timeout, util::BoxService, Service};

use zebra_chain::{
Expand Down Expand Up @@ -233,7 +233,7 @@ pub struct Mempool {

/// Sender part of a gossip transactions channel.
/// Used to broadcast transaction ids to peers.
transaction_sender: watch::Sender<HashSet<UnminedTxId>>,
transaction_sender: broadcast::Sender<HashSet<UnminedTxId>>,

// Diagnostics
//
Expand Down Expand Up @@ -267,9 +267,9 @@ impl Mempool {
sync_status: SyncStatus,
latest_chain_tip: zs::LatestChainTip,
chain_tip_change: ChainTipChange,
) -> (Self, watch::Receiver<HashSet<UnminedTxId>>) {
) -> (Self, broadcast::Receiver<HashSet<UnminedTxId>>) {
let (transaction_sender, transaction_receiver) =
tokio::sync::watch::channel(HashSet::new());
tokio::sync::broadcast::channel(gossip::MAX_CHANGES_BEFORE_SEND * 2);

let mut service = Mempool {
config: config.clone(),
Expand Down Expand Up @@ -659,9 +659,6 @@ impl Service<Request> for Mempool {
if !send_to_peers_ids.is_empty() {
tracing::trace!(?send_to_peers_ids, "sending new transactions to peers");

// TODO:
// - if the transaction gossip task is slow, we can overwrite unsent IDs here
// - does this happen often enough to be worth a fix?
self.transaction_sender.send(send_to_peers_ids)?;
}
}
Expand Down
Loading

0 comments on commit dc51989

Please sign in to comment.