Skip to content

Commit ad272f2

Browse files
teor2345dconnolly
authored andcommitted
Make sure handshake version negotiation always has a timeout
As part of this change, refactor handshake version negotiation into its own function.
1 parent 43e792b commit ad272f2

File tree

1 file changed

+157
-127
lines changed

1 file changed

+157
-127
lines changed

zebra-network/src/peer/handshake.rs

Lines changed: 157 additions & 127 deletions
Original file line numberDiff line numberDiff line change
@@ -12,7 +12,7 @@ use futures::{
1212
channel::{mpsc, oneshot},
1313
future, FutureExt, SinkExt, StreamExt,
1414
};
15-
use tokio::{net::TcpStream, sync::broadcast, time::timeout};
15+
use tokio::{net::TcpStream, sync::broadcast, task::JoinError, time::timeout};
1616
use tokio_util::codec::Framed;
1717
use tower::Service;
1818
use tracing::{span, Level, Span};
@@ -180,6 +180,138 @@ where
180180
}
181181
}
182182

183+
/// Negotiate the Zcash network protocol version with the remote peer
184+
/// at `addr`, using the connection `peer_conn`.
185+
///
186+
/// We split `Handshake` into its components before calling this function,
187+
/// to avoid infectious `Sync` bounds on the returned future.
188+
pub async fn negotiate_version(
189+
peer_conn: &mut Framed<TcpStream, Codec>,
190+
addr: &SocketAddr,
191+
config: Config,
192+
nonces: Arc<Mutex<HashSet<Nonce>>>,
193+
user_agent: String,
194+
our_services: PeerServices,
195+
relay: bool,
196+
) -> Result<(Version, PeerServices), HandshakeError> {
197+
// Create a random nonce for this connection
198+
let local_nonce = Nonce::default();
199+
nonces
200+
.lock()
201+
.expect("mutex should be unpoisoned")
202+
.insert(local_nonce);
203+
204+
// Don't leak our exact clock skew to our peers. On the other hand,
205+
// we can't deviate too much, or zcashd will get confused.
206+
// Inspection of the zcashd source code reveals that the timestamp
207+
// is only ever used at the end of parsing the version message, in
208+
//
209+
// pfrom->nTimeOffset = timeWarning.AddTimeData(pfrom->addr, nTime, GetTime());
210+
//
211+
// AddTimeData is defined in src/timedata.cpp and is a no-op as long
212+
// as the difference between the specified timestamp and the
213+
// zcashd's local time is less than TIMEDATA_WARNING_THRESHOLD, set
214+
// to 10 * 60 seconds (10 minutes).
215+
//
216+
// nTimeOffset is peer metadata that is never used, except for
217+
// statistics.
218+
//
219+
// To try to stay within the range where zcashd will ignore our clock skew,
220+
// truncate the timestamp to the nearest 5 minutes.
221+
let now = Utc::now().timestamp();
222+
let timestamp = Utc.timestamp(now - now.rem_euclid(5 * 60), 0);
223+
224+
let our_version = Message::Version {
225+
version: constants::CURRENT_VERSION,
226+
services: our_services,
227+
timestamp,
228+
address_recv: (PeerServices::NODE_NETWORK, *addr),
229+
// TODO: detect external address (#1893)
230+
address_from: (our_services, config.listen_addr),
231+
nonce: local_nonce,
232+
user_agent: user_agent.clone(),
233+
// The protocol works fine if we don't reveal our current block height,
234+
// and not sending it means we don't need to be connected to the chain state.
235+
start_height: block::Height(0),
236+
relay,
237+
};
238+
239+
debug!(?our_version, "sending initial version message");
240+
peer_conn.send(our_version).await?;
241+
242+
let remote_msg = peer_conn
243+
.next()
244+
.await
245+
.ok_or(HandshakeError::ConnectionClosed)??;
246+
247+
// Check that we got a Version and destructure its fields into the local scope.
248+
debug!(?remote_msg, "got message from remote peer");
249+
let (remote_nonce, remote_services, remote_version) = if let Message::Version {
250+
nonce,
251+
services,
252+
version,
253+
..
254+
} = remote_msg
255+
{
256+
(nonce, services, version)
257+
} else {
258+
Err(HandshakeError::UnexpectedMessage(Box::new(remote_msg)))?
259+
};
260+
261+
// Check for nonce reuse, indicating self-connection.
262+
let nonce_reuse = {
263+
let mut locked_nonces = nonces.lock().expect("mutex should be unpoisoned");
264+
let nonce_reuse = locked_nonces.contains(&remote_nonce);
265+
// Regardless of whether we observed nonce reuse, clean up the nonce set.
266+
locked_nonces.remove(&local_nonce);
267+
nonce_reuse
268+
};
269+
if nonce_reuse {
270+
Err(HandshakeError::NonceReuse)?;
271+
}
272+
273+
peer_conn.send(Message::Verack).await?;
274+
275+
let remote_msg = peer_conn
276+
.next()
277+
.await
278+
.ok_or(HandshakeError::ConnectionClosed)??;
279+
if let Message::Verack = remote_msg {
280+
debug!("got verack from remote peer");
281+
} else {
282+
Err(HandshakeError::UnexpectedMessage(Box::new(remote_msg)))?;
283+
}
284+
285+
// XXX in zcashd remote peer can only send one version message and
286+
// we would disconnect here if it received a second one. Is it even possible
287+
// for that to happen to us here?
288+
289+
// TODO: Reject incoming connections from nodes that don't know about the current epoch.
290+
// zcashd does this:
291+
// const Consensus::Params& consensusParams = chainparams.GetConsensus();
292+
// auto currentEpoch = CurrentEpoch(GetHeight(), consensusParams);
293+
// if (pfrom->nVersion < consensusParams.vUpgrades[currentEpoch].nProtocolVersion)
294+
//
295+
// For approximately 1.5 days before a network upgrade, zcashd also:
296+
// - avoids old peers, and
297+
// - prefers updated peers.
298+
// We haven't decided if we need this behaviour in Zebra yet (see #706).
299+
//
300+
// At the network upgrade, we also need to disconnect from old peers (see #1334).
301+
//
302+
// TODO: replace min_for_upgrade(network, MIN_NETWORK_UPGRADE) with
303+
// current_min(network, height) where network is the
304+
// configured network, and height is the best tip's block
305+
// height.
306+
307+
if remote_version < Version::min_for_upgrade(config.network, constants::MIN_NETWORK_UPGRADE) {
308+
// Disconnect if peer is using an obsolete version.
309+
Err(HandshakeError::ObsoleteVersion(remote_version))?;
310+
}
311+
312+
Ok((remote_version, remote_services))
313+
}
314+
183315
impl<S> Service<(TcpStream, SocketAddr)> for Handshake<S>
184316
where
185317
S: Service<Request, Response = Response, Error = BoxError> + Clone + Send + 'static,
@@ -197,148 +329,51 @@ where
197329
fn call(&mut self, req: (TcpStream, SocketAddr)) -> Self::Future {
198330
let (tcp_stream, addr) = req;
199331

200-
let connector_span = span!(Level::INFO, "connector", addr = ?addr);
332+
let connector_span = span!(Level::INFO, "connector", ?addr);
201333
// set the peer connection span's parent to the global span, as it
202334
// should exist independently of its creation source (inbound
203335
// connection, crawler, initial peer, ...)
204-
let connection_span = span!(parent: &self.parent_span, Level::INFO, "peer", addr = ?addr);
336+
let connection_span = span!(parent: &self.parent_span, Level::INFO, "peer", ?addr);
205337

206338
// Clone these upfront, so they can be moved into the future.
207339
let nonces = self.nonces.clone();
208340
let inbound_service = self.inbound_service.clone();
209341
let timestamp_collector = self.timestamp_collector.clone();
210342
let inv_collector = self.inv_collector.clone();
211-
let network = self.config.network;
212-
let our_addr = self.config.listen_addr;
343+
let config = self.config.clone();
213344
let user_agent = self.user_agent.clone();
214345
let our_services = self.our_services;
215346
let relay = self.relay;
216347

217348
let fut = async move {
218-
debug!("connecting to remote peer");
349+
debug!(?addr, "negotiating protocol version with remote peer");
219350

220351
// CORRECTNESS
221352
//
222353
// As a defence-in-depth against hangs, every send or next on stream
223354
// should be wrapped in a timeout.
224-
let mut stream = Framed::new(
355+
let mut peer_conn = Framed::new(
225356
tcp_stream,
226357
Codec::builder()
227-
.for_network(network)
358+
.for_network(config.network)
228359
.with_metrics_label(addr.ip().to_string())
229360
.finish(),
230361
);
231362

232-
let local_nonce = Nonce::default();
233-
nonces
234-
.lock()
235-
.expect("mutex should be unpoisoned")
236-
.insert(local_nonce);
237-
238-
// Don't leak our exact clock skew to our peers. On the other hand,
239-
// we can't deviate too much, or zcashd will get confused.
240-
// Inspection of the zcashd source code reveals that the timestamp
241-
// is only ever used at the end of parsing the version message, in
242-
//
243-
// pfrom->nTimeOffset = timeWarning.AddTimeData(pfrom->addr, nTime, GetTime());
244-
//
245-
// AddTimeData is defined in src/timedata.cpp and is a no-op as long
246-
// as the difference between the specified timestamp and the
247-
// zcashd's local time is less than TIMEDATA_WARNING_THRESHOLD, set
248-
// to 10 * 60 seconds (10 minutes).
249-
//
250-
// nTimeOffset is peer metadata that is never used, except for
251-
// statistics.
252-
//
253-
// To try to stay within the range where zcashd will ignore our clock skew,
254-
// truncate the timestamp to the nearest 5 minutes.
255-
let now = Utc::now().timestamp();
256-
let timestamp = Utc.timestamp(now - now.rem_euclid(5 * 60), 0);
257-
258-
let version = Message::Version {
259-
version: constants::CURRENT_VERSION,
260-
services: our_services,
261-
timestamp,
262-
address_recv: (PeerServices::NODE_NETWORK, addr),
263-
address_from: (our_services, our_addr),
264-
nonce: local_nonce,
265-
user_agent,
266-
// The protocol works fine if we don't reveal our current block height,
267-
// and not sending it means we don't need to be connected to the chain state.
268-
start_height: block::Height(0),
269-
relay,
270-
};
271-
272-
debug!(?version, "sending initial version message");
273-
timeout(constants::REQUEST_TIMEOUT, stream.send(version)).await??;
274-
275-
let remote_msg = timeout(constants::REQUEST_TIMEOUT, stream.next())
276-
.await?
277-
.ok_or(HandshakeError::ConnectionClosed)??;
278-
279-
// Check that we got a Version and destructure its fields into the local scope.
280-
debug!(?remote_msg, "got message from remote peer");
281-
let (remote_nonce, remote_services, remote_version) = if let Message::Version {
282-
nonce,
283-
services,
284-
version,
285-
..
286-
} = remote_msg
287-
{
288-
(nonce, services, version)
289-
} else {
290-
return Err(HandshakeError::UnexpectedMessage(Box::new(remote_msg)));
291-
};
292-
293-
// Check for nonce reuse, indicating self-connection.
294-
let nonce_reuse = {
295-
let mut locked_nonces = nonces.lock().expect("mutex should be unpoisoned");
296-
let nonce_reuse = locked_nonces.contains(&remote_nonce);
297-
// Regardless of whether we observed nonce reuse, clean up the nonce set.
298-
locked_nonces.remove(&local_nonce);
299-
nonce_reuse
300-
};
301-
if nonce_reuse {
302-
return Err(HandshakeError::NonceReuse);
303-
}
304-
305-
timeout(constants::REQUEST_TIMEOUT, stream.send(Message::Verack)).await??;
306-
307-
let remote_msg = timeout(constants::REQUEST_TIMEOUT, stream.next())
308-
.await?
309-
.ok_or(HandshakeError::ConnectionClosed)??;
310-
if let Message::Verack = remote_msg {
311-
debug!("got verack from remote peer");
312-
} else {
313-
return Err(HandshakeError::UnexpectedMessage(Box::new(remote_msg)));
314-
}
315-
316-
// XXX in zcashd remote peer can only send one version message and
317-
// we would disconnect here if it received a second one. Is it even possible
318-
// for that to happen to us here?
319-
320-
// TODO: Reject incoming connections from nodes that don't know about the current epoch.
321-
// zcashd does this:
322-
// const Consensus::Params& consensusParams = chainparams.GetConsensus();
323-
// auto currentEpoch = CurrentEpoch(GetHeight(), consensusParams);
324-
// if (pfrom->nVersion < consensusParams.vUpgrades[currentEpoch].nProtocolVersion)
325-
//
326-
// For approximately 1.5 days before a network upgrade, zcashd also:
327-
// - avoids old peers, and
328-
// - prefers updated peers.
329-
// We haven't decided if we need this behaviour in Zebra yet (see #706).
330-
//
331-
// At the network upgrade, we also need to disconnect from old peers (see #1334).
332-
//
333-
// TODO: replace min_for_upgrade(network, MIN_NETWORK_UPGRADE) with
334-
// current_min(network, height) where network is the
335-
// configured network, and height is the best tip's block
336-
// height.
337-
338-
if remote_version < Version::min_for_upgrade(network, constants::MIN_NETWORK_UPGRADE) {
339-
// Disconnect if peer is using an obsolete version.
340-
return Err(HandshakeError::ObsoleteVersion(remote_version));
341-
}
363+
// Wrap the entire initial connection setup in a timeout.
364+
let (remote_version, remote_services) = timeout(
365+
constants::HANDSHAKE_TIMEOUT,
366+
negotiate_version(
367+
&mut peer_conn,
368+
&addr,
369+
config,
370+
nonces,
371+
user_agent,
372+
our_services,
373+
relay,
374+
),
375+
)
376+
.await??;
342377

343378
// Set the connection's version to the minimum of the received version or our own.
344379
let negotiated_version = std::cmp::min(remote_version, constants::CURRENT_VERSION);
@@ -348,7 +383,7 @@ where
348383
// XXX The tokio documentation says not to do this while any frames are still being processed.
349384
// Since we don't know that here, another way might be to release the tcp
350385
// stream from the unversioned Framed wrapper and construct a new one with a versioned codec.
351-
let bare_codec = stream.codec_mut();
386+
let bare_codec = peer_conn.codec_mut();
352387
bare_codec.reconfigure_version(negotiated_version);
353388

354389
debug!("constructing client, spawning server");
@@ -365,7 +400,7 @@ where
365400
error_slot: slot.clone(),
366401
};
367402

368-
let (peer_tx, peer_rx) = stream.split();
403+
let (peer_tx, peer_rx) = peer_conn.split();
369404

370405
// Instrument the peer's rx and tx streams.
371406

@@ -389,6 +424,7 @@ where
389424
// Every message and error must update the peer address state via
390425
// the inbound_ts_collector.
391426
let inbound_ts_collector = timestamp_collector.clone();
427+
let inv_collector = inv_collector.clone();
392428
let peer_rx = peer_rx
393429
.then(move |msg| {
394430
// Add a metric for inbound messages and errors.
@@ -554,13 +590,7 @@ where
554590

555591
// Spawn a new task to drive this handshake.
556592
tokio::spawn(fut.instrument(connector_span))
557-
// This is required to get error types to line up.
558-
// Probably there's a nicer way to express this using combinators.
559-
.map(|x| match x {
560-
Ok(Ok(client)) => Ok(client),
561-
Ok(Err(handshake_err)) => Err(handshake_err.into()),
562-
Err(join_err) => Err(join_err.into()),
563-
})
593+
.map(|x: Result<Result<Client, HandshakeError>, JoinError>| Ok(x??))
564594
.boxed()
565595
}
566596
}

0 commit comments

Comments
 (0)