diff --git a/client/beefy/src/aux_schema.rs b/client/beefy/src/aux_schema.rs index 9d6a4292f32d4..67deee19b58b0 100644 --- a/client/beefy/src/aux_schema.rs +++ b/client/beefy/src/aux_schema.rs @@ -77,7 +77,6 @@ pub(crate) mod tests { use super::*; use crate::tests::BeefyTestNet; use sc_network_test::TestNetFactory; - use tokio::runtime::Runtime; // also used in tests.rs pub fn verify_persisted_version>(backend: &BE) -> bool { @@ -85,10 +84,9 @@ pub(crate) mod tests { version == CURRENT_VERSION } - #[test] - fn should_load_persistent_sanity_checks() { - let runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + #[tokio::test] + async fn should_load_persistent_sanity_checks() { + let mut net = BeefyTestNet::new(1); let backend = net.peer(0).client().as_backend(); // version not available in db -> None diff --git a/client/beefy/src/tests.rs b/client/beefy/src/tests.rs index f6ab0dd1020f1..d2befc979b737 100644 --- a/client/beefy/src/tests.rs +++ b/client/beefy/src/tests.rs @@ -47,7 +47,7 @@ use sc_consensus::{ use sc_network::{config::RequestResponseConfig, ProtocolName}; use sc_network_test::{ Block, BlockImportAdapter, FullPeerConfig, PassThroughVerifier, Peer, PeersClient, - PeersFullClient, TestNetFactory, WithRuntime, + PeersFullClient, TestNetFactory, }; use sc_utils::notification::NotificationReceiver; use serde::{Deserialize, Serialize}; @@ -64,10 +64,7 @@ use sp_runtime::{ }; use std::{collections::HashMap, marker::PhantomData, sync::Arc, task::Poll}; use substrate_test_runtime_client::{runtime::Header, ClientExt}; -use tokio::{ - runtime::{Handle, Runtime}, - time::Duration, -}; +use tokio::time::Duration; const GENESIS_HASH: H256 = H256::zero(); fn beefy_gossip_proto_name() -> ProtocolName { @@ -106,23 +103,14 @@ pub(crate) struct PeerData { Mutex>>, } +#[derive(Default)] pub(crate) struct BeefyTestNet { - rt_handle: Handle, peers: Vec, } -impl WithRuntime for BeefyTestNet { - fn with_runtime(rt_handle: Handle) -> Self { - BeefyTestNet { rt_handle, peers: Vec::new() } - } - fn rt_handle(&self) -> &Handle { - &self.rt_handle - } -} - impl BeefyTestNet { - pub(crate) fn new(rt_handle: Handle, n_authority: usize) -> Self { - let mut net = BeefyTestNet::with_runtime(rt_handle); + pub(crate) fn new(n_authority: usize) -> Self { + let mut net = BeefyTestNet { peers: Vec::with_capacity(n_authority) }; for i in 0..n_authority { let (rx, cfg) = on_demand_justifications_protocol_config(GENESIS_HASH, None); @@ -151,13 +139,12 @@ impl BeefyTestNet { }); } - pub(crate) fn generate_blocks_and_sync( + pub(crate) async fn generate_blocks_and_sync( &mut self, count: usize, session_length: u64, validator_set: &BeefyValidatorSet, include_mmr_digest: bool, - runtime: &mut Runtime, ) { self.peer(0).generate_blocks(count, BlockOrigin::File, |builder| { let mut block = builder.build().unwrap().block; @@ -175,7 +162,7 @@ impl BeefyTestNet { block }); - runtime.block_on(self.wait_until_sync()); + self.run_until_sync().await; } } @@ -404,17 +391,16 @@ where tasks.for_each(|_| async move {}) } -fn block_until(future: impl Future + Unpin, net: &Arc>, runtime: &mut Runtime) { +async fn run_until(future: impl Future + Unpin, net: &Arc>) { let drive_to_completion = futures::future::poll_fn(|cx| { net.lock().poll(cx); Poll::<()>::Pending }); - runtime.block_on(future::select(future, drive_to_completion)); + let _ = future::select(future, drive_to_completion).await; } -fn run_for(duration: Duration, net: &Arc>, runtime: &mut Runtime) { - let sleep = runtime.spawn(async move { tokio::time::sleep(duration).await }); - block_until(sleep, net, runtime); +async fn run_for(duration: Duration, net: &Arc>) { + run_until(Box::pin(tokio::time::sleep(duration)), net).await; } pub(crate) fn get_beefy_streams( @@ -435,10 +421,9 @@ pub(crate) fn get_beefy_streams( (best_block_streams, versioned_finality_proof_streams) } -fn wait_for_best_beefy_blocks( +async fn wait_for_best_beefy_blocks( streams: Vec>, net: &Arc>, - runtime: &mut Runtime, expected_beefy_blocks: &[u64], ) { let mut wait_for = Vec::new(); @@ -458,13 +443,12 @@ fn wait_for_best_beefy_blocks( }))); }); let wait_for = futures::future::join_all(wait_for); - block_until(wait_for, net, runtime); + run_until(wait_for, net).await; } -fn wait_for_beefy_signed_commitments( +async fn wait_for_beefy_signed_commitments( streams: Vec>>, net: &Arc>, - runtime: &mut Runtime, expected_commitment_block_nums: &[u64], ) { let mut wait_for = Vec::new(); @@ -484,34 +468,33 @@ fn wait_for_beefy_signed_commitments( }))); }); let wait_for = futures::future::join_all(wait_for); - block_until(wait_for, net, runtime); + run_until(wait_for, net).await; } -fn streams_empty_after_timeout( +async fn streams_empty_after_timeout( streams: Vec>, net: &Arc>, - runtime: &mut Runtime, timeout: Option, ) where T: std::fmt::Debug, T: std::cmp::PartialEq, { if let Some(timeout) = timeout { - run_for(timeout, net, runtime); + run_for(timeout, net).await; } - streams.into_iter().for_each(|mut stream| { - runtime.block_on(future::poll_fn(move |cx| { + for mut stream in streams.into_iter() { + future::poll_fn(move |cx| { assert_eq!(stream.poll_next_unpin(cx), Poll::Pending); Poll::Ready(()) - })); - }); + }) + .await; + } } -fn finalize_block_and_wait_for_beefy( +async fn finalize_block_and_wait_for_beefy( net: &Arc>, // peer index and key peers: impl Iterator + Clone, - runtime: &mut Runtime, finalize_targets: &[u64], expected_beefy: &[u64], ) { @@ -528,33 +511,32 @@ fn finalize_block_and_wait_for_beefy( if expected_beefy.is_empty() { // run for quarter second then verify no new best beefy block available let timeout = Some(Duration::from_millis(250)); - streams_empty_after_timeout(best_blocks, &net, runtime, timeout); - streams_empty_after_timeout(versioned_finality_proof, &net, runtime, None); + streams_empty_after_timeout(best_blocks, &net, timeout).await; + streams_empty_after_timeout(versioned_finality_proof, &net, None).await; } else { // run until expected beefy blocks are received - wait_for_best_beefy_blocks(best_blocks, &net, runtime, expected_beefy); - wait_for_beefy_signed_commitments(versioned_finality_proof, &net, runtime, expected_beefy); + wait_for_best_beefy_blocks(best_blocks, &net, expected_beefy).await; + wait_for_beefy_signed_commitments(versioned_finality_proof, &net, expected_beefy).await; } } -#[test] -fn beefy_finalizing_blocks() { +#[tokio::test] +async fn beefy_finalizing_blocks() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let peers = [BeefyKeyring::Alice, BeefyKeyring::Bob]; let validator_set = ValidatorSet::new(make_beefy_ids(&peers), 0).unwrap(); let session_len = 10; let min_block_delta = 4; - let mut net = BeefyTestNet::new(runtime.handle().clone(), 2); + let mut net = BeefyTestNet::new(2); let api = Arc::new(two_validators::TestApi {}); let beefy_peers = peers.iter().enumerate().map(|(id, key)| (id, key, api.clone())).collect(); - runtime.spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); // push 42 blocks including `AuthorityChange` digests every 10 blocks. - net.generate_blocks_and_sync(42, session_len, &validator_set, true, &mut runtime); + net.generate_blocks_and_sync(42, session_len, &validator_set, true).await; let net = Arc::new(Mutex::new(net)); @@ -562,51 +544,44 @@ fn beefy_finalizing_blocks() { let peers = peers.into_iter().enumerate(); // finalize block #5 -> BEEFY should finalize #1 (mandatory) and #5 from diff-power-of-two rule. - finalize_block_and_wait_for_beefy(&net, peers.clone(), &mut runtime, &[1, 5], &[1, 5]); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[1, 5], &[1, 5]).await; // GRANDPA finalize #10 -> BEEFY finalize #10 (mandatory) - finalize_block_and_wait_for_beefy(&net, peers.clone(), &mut runtime, &[10], &[10]); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[10], &[10]).await; // GRANDPA finalize #18 -> BEEFY finalize #14, then #18 (diff-power-of-two rule) - finalize_block_and_wait_for_beefy(&net, peers.clone(), &mut runtime, &[18], &[14, 18]); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[18], &[14, 18]).await; // GRANDPA finalize #20 -> BEEFY finalize #20 (mandatory) - finalize_block_and_wait_for_beefy(&net, peers.clone(), &mut runtime, &[20], &[20]); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[20], &[20]).await; // GRANDPA finalize #21 -> BEEFY finalize nothing (yet) because min delta is 4 - finalize_block_and_wait_for_beefy(&net, peers, &mut runtime, &[21], &[]); + finalize_block_and_wait_for_beefy(&net, peers, &[21], &[]).await; } -#[test] -fn lagging_validators() { +#[tokio::test] +async fn lagging_validators() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let peers = [BeefyKeyring::Alice, BeefyKeyring::Bob]; let validator_set = ValidatorSet::new(make_beefy_ids(&peers), 0).unwrap(); let session_len = 30; let min_block_delta = 1; - let mut net = BeefyTestNet::new(runtime.handle().clone(), 2); + let mut net = BeefyTestNet::new(2); let api = Arc::new(two_validators::TestApi {}); let beefy_peers = peers.iter().enumerate().map(|(id, key)| (id, key, api.clone())).collect(); - runtime.spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); // push 62 blocks including `AuthorityChange` digests every 30 blocks. - net.generate_blocks_and_sync(62, session_len, &validator_set, true, &mut runtime); + net.generate_blocks_and_sync(62, session_len, &validator_set, true).await; let net = Arc::new(Mutex::new(net)); let peers = peers.into_iter().enumerate(); // finalize block #15 -> BEEFY should finalize #1 (mandatory) and #9, #13, #14, #15 from // diff-power-of-two rule. - finalize_block_and_wait_for_beefy( - &net, - peers.clone(), - &mut runtime, - &[1, 15], - &[1, 9, 13, 14, 15], - ); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[1, 15], &[1, 9, 13, 14, 15]).await; // Alice finalizes #25, Bob lags behind let finalize = net @@ -620,18 +595,18 @@ fn lagging_validators() { net.lock().peer(0).client().as_client().finalize_block(finalize, None).unwrap(); // verify nothing gets finalized by BEEFY let timeout = Some(Duration::from_millis(250)); - streams_empty_after_timeout(best_blocks, &net, &mut runtime, timeout); - streams_empty_after_timeout(versioned_finality_proof, &net, &mut runtime, None); + streams_empty_after_timeout(best_blocks, &net, timeout).await; + streams_empty_after_timeout(versioned_finality_proof, &net, None).await; // Bob catches up and also finalizes #25 let (best_blocks, versioned_finality_proof) = get_beefy_streams(&mut net.lock(), peers.clone()); net.lock().peer(1).client().as_client().finalize_block(finalize, None).unwrap(); // expected beefy finalizes block #17 from diff-power-of-two - wait_for_best_beefy_blocks(best_blocks, &net, &mut runtime, &[23, 24, 25]); - wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &mut runtime, &[23, 24, 25]); + wait_for_best_beefy_blocks(best_blocks, &net, &[23, 24, 25]).await; + wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &[23, 24, 25]).await; // Both finalize #30 (mandatory session) and #32 -> BEEFY finalize #30 (mandatory), #31, #32 - finalize_block_and_wait_for_beefy(&net, peers.clone(), &mut runtime, &[30, 32], &[30, 31, 32]); + finalize_block_and_wait_for_beefy(&net, peers.clone(), &[30, 32], &[30, 31, 32]).await; // Verify that session-boundary votes get buffered by client and only processed once // session-boundary block is GRANDPA-finalized (this guarantees authenticity for the new session @@ -649,28 +624,27 @@ fn lagging_validators() { net.lock().peer(0).client().as_client().finalize_block(finalize, None).unwrap(); // verify nothing gets finalized by BEEFY let timeout = Some(Duration::from_millis(250)); - streams_empty_after_timeout(best_blocks, &net, &mut runtime, timeout); - streams_empty_after_timeout(versioned_finality_proof, &net, &mut runtime, None); + streams_empty_after_timeout(best_blocks, &net, timeout).await; + streams_empty_after_timeout(versioned_finality_proof, &net, None).await; // Bob catches up and also finalizes #60 (and should have buffered Alice's vote on #60) let (best_blocks, versioned_finality_proof) = get_beefy_streams(&mut net.lock(), peers); net.lock().peer(1).client().as_client().finalize_block(finalize, None).unwrap(); // verify beefy skips intermediary votes, and successfully finalizes mandatory block #60 - wait_for_best_beefy_blocks(best_blocks, &net, &mut runtime, &[60]); - wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &mut runtime, &[60]); + wait_for_best_beefy_blocks(best_blocks, &net, &[60]).await; + wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &[60]).await; } -#[test] -fn correct_beefy_payload() { +#[tokio::test] +async fn correct_beefy_payload() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let peers = [BeefyKeyring::Alice, BeefyKeyring::Bob, BeefyKeyring::Charlie, BeefyKeyring::Dave]; let validator_set = ValidatorSet::new(make_beefy_ids(&peers), 0).unwrap(); let session_len = 20; let min_block_delta = 2; - let mut net = BeefyTestNet::new(runtime.handle().clone(), 4); + let mut net = BeefyTestNet::new(4); // Alice, Bob, Charlie will vote on good payloads let good_api = Arc::new(four_validators::TestApi {}); @@ -679,20 +653,20 @@ fn correct_beefy_payload() { .enumerate() .map(|(id, key)| (id, key, good_api.clone())) .collect(); - runtime.spawn(initialize_beefy(&mut net, good_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, good_peers, min_block_delta)); // Dave will vote on bad mmr roots let bad_api = Arc::new(bad_four_validators::TestApi {}); let bad_peers = vec![(3, &BeefyKeyring::Dave, bad_api)]; - runtime.spawn(initialize_beefy(&mut net, bad_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, bad_peers, min_block_delta)); // push 12 blocks - net.generate_blocks_and_sync(12, session_len, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(12, session_len, &validator_set, false).await; let net = Arc::new(Mutex::new(net)); let peers = peers.into_iter().enumerate(); // with 3 good voters and 1 bad one, consensus should happen and best blocks produced. - finalize_block_and_wait_for_beefy(&net, peers, &mut runtime, &[1, 10], &[1, 9]); + finalize_block_and_wait_for_beefy(&net, peers, &[1, 10], &[1, 9]).await; let (best_blocks, versioned_finality_proof) = get_beefy_streams(&mut net.lock(), [(0, BeefyKeyring::Alice)].into_iter()); @@ -711,8 +685,8 @@ fn correct_beefy_payload() { // verify consensus is _not_ reached let timeout = Some(Duration::from_millis(250)); - streams_empty_after_timeout(best_blocks, &net, &mut runtime, timeout); - streams_empty_after_timeout(versioned_finality_proof, &net, &mut runtime, None); + streams_empty_after_timeout(best_blocks, &net, timeout).await; + streams_empty_after_timeout(versioned_finality_proof, &net, None).await; // 3rd good validator catches up and votes as well let (best_blocks, versioned_finality_proof) = @@ -720,21 +694,19 @@ fn correct_beefy_payload() { net.lock().peer(2).client().as_client().finalize_block(hashof11, None).unwrap(); // verify consensus is reached - wait_for_best_beefy_blocks(best_blocks, &net, &mut runtime, &[11]); - wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &mut runtime, &[11]); + wait_for_best_beefy_blocks(best_blocks, &net, &[11]).await; + wait_for_beefy_signed_commitments(versioned_finality_proof, &net, &[11]).await; } -#[test] -fn beefy_importing_blocks() { +#[tokio::test] +async fn beefy_importing_blocks() { use futures::{future::poll_fn, task::Poll}; use sc_block_builder::BlockBuilderProvider; use sc_client_api::BlockBackend; sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - - let mut net = BeefyTestNet::new(runtime.handle().clone(), 2); + let mut net = BeefyTestNet::new(2); let client = net.peer(0).client().clone(); let (mut block_import, _, peer_data) = net.make_block_import(client.clone()); @@ -759,16 +731,15 @@ fn beefy_importing_blocks() { // Import without justifications. let mut justif_recv = justif_stream.subscribe(); assert_eq!( - runtime - .block_on(block_import.import_block(params(block.clone(), None), HashMap::new())) + block_import + .import_block(params(block.clone(), None), HashMap::new()) + .await .unwrap(), ImportResult::Imported(ImportedAux { is_new_best: true, ..Default::default() }), ); assert_eq!( - runtime - .block_on(block_import.import_block(params(block, None), HashMap::new())) - .unwrap(), - ImportResult::AlreadyInChain + block_import.import_block(params(block, None), HashMap::new()).await.unwrap(), + ImportResult::AlreadyInChain, ); // Verify no BEEFY justifications present: { @@ -781,10 +752,11 @@ fn beefy_importing_blocks() { None ); // and none sent to BEEFY worker. - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { assert_eq!(justif_recv.poll_next_unpin(cx), Poll::Pending); Poll::Ready(()) - })); + }) + .await; } // Import with valid justification. @@ -802,9 +774,7 @@ fn beefy_importing_blocks() { let hashof2 = block.header.hash(); let mut justif_recv = justif_stream.subscribe(); assert_eq!( - runtime - .block_on(block_import.import_block(params(block, justif), HashMap::new())) - .unwrap(), + block_import.import_block(params(block, justif), HashMap::new()).await.unwrap(), ImportResult::Imported(ImportedAux { bad_justification: false, is_new_best: true, @@ -823,13 +793,14 @@ fn beefy_importing_blocks() { ); // but sent to BEEFY worker // (worker will append it to backend when all previous mandatory justifs are there as well). - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { match justif_recv.poll_next_unpin(cx) { Poll::Ready(Some(_justification)) => (), v => panic!("unexpected value: {:?}", v), } Poll::Ready(()) - })); + }) + .await; } // Import with invalid justification (incorrect validator set). @@ -847,9 +818,7 @@ fn beefy_importing_blocks() { let hashof3 = block.header.hash(); let mut justif_recv = justif_stream.subscribe(); assert_eq!( - runtime - .block_on(block_import.import_block(params(block, justif), HashMap::new())) - .unwrap(), + block_import.import_block(params(block, justif), HashMap::new()).await.unwrap(), ImportResult::Imported(ImportedAux { // Still `false` because we don't want to fail import on bad BEEFY justifications. bad_justification: false, @@ -868,33 +837,33 @@ fn beefy_importing_blocks() { None ); // and none sent to BEEFY worker. - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { assert_eq!(justif_recv.poll_next_unpin(cx), Poll::Pending); Poll::Ready(()) - })); + }) + .await; } } -#[test] -fn voter_initialization() { +#[tokio::test] +async fn voter_initialization() { sp_tracing::try_init_simple(); // Regression test for voter initialization where finality notifications were dropped // after waiting for BEEFY pallet availability. - let mut runtime = Runtime::new().unwrap(); let peers = [BeefyKeyring::Alice, BeefyKeyring::Bob]; let validator_set = ValidatorSet::new(make_beefy_ids(&peers), 0).unwrap(); let session_len = 5; // Should vote on all mandatory blocks no matter the `min_block_delta`. let min_block_delta = 10; - let mut net = BeefyTestNet::new(runtime.handle().clone(), 2); + let mut net = BeefyTestNet::new(2); let api = Arc::new(two_validators::TestApi {}); let beefy_peers = peers.iter().enumerate().map(|(id, key)| (id, key, api.clone())).collect(); - runtime.spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, beefy_peers, min_block_delta)); // push 26 blocks - net.generate_blocks_and_sync(26, session_len, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(26, session_len, &validator_set, false).await; let net = Arc::new(Mutex::new(net)); // Finalize multiple blocks at once to get a burst of finality notifications right from start. @@ -903,31 +872,30 @@ fn voter_initialization() { finalize_block_and_wait_for_beefy( &net, peers.into_iter().enumerate(), - &mut runtime, &[1, 6, 10, 17, 24, 26], &[1, 5, 10, 15, 20, 25], - ); + ) + .await; } -#[test] -fn on_demand_beefy_justification_sync() { +#[tokio::test] +async fn on_demand_beefy_justification_sync() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let all_peers = [BeefyKeyring::Alice, BeefyKeyring::Bob, BeefyKeyring::Charlie, BeefyKeyring::Dave]; let validator_set = ValidatorSet::new(make_beefy_ids(&all_peers), 0).unwrap(); let session_len = 5; let min_block_delta = 5; - let mut net = BeefyTestNet::new(runtime.handle().clone(), 4); + let mut net = BeefyTestNet::new(4); // Alice, Bob, Charlie start first and make progress through voting. let api = Arc::new(four_validators::TestApi {}); let fast_peers = [BeefyKeyring::Alice, BeefyKeyring::Bob, BeefyKeyring::Charlie]; let voting_peers = fast_peers.iter().enumerate().map(|(id, key)| (id, key, api.clone())).collect(); - runtime.spawn(initialize_beefy(&mut net, voting_peers, min_block_delta)); + tokio::spawn(initialize_beefy(&mut net, voting_peers, min_block_delta)); // Dave will start late and have to catch up using on-demand justification requests (since // in this test there is no block import queue to automatically import justifications). @@ -937,7 +905,7 @@ fn on_demand_beefy_justification_sync() { let dave_index = 3; // push 30 blocks - net.generate_blocks_and_sync(30, session_len, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(30, session_len, &validator_set, false).await; let fast_peers = fast_peers.into_iter().enumerate(); let net = Arc::new(Mutex::new(net)); @@ -946,15 +914,15 @@ fn on_demand_beefy_justification_sync() { finalize_block_and_wait_for_beefy( &net, fast_peers.clone(), - &mut runtime, &[1, 6, 10, 17, 24], &[1, 5, 10, 15, 20], - ); + ) + .await; // Spawn Dave, he's now way behind voting and can only catch up through on-demand justif sync. - runtime.spawn(dave_task); + tokio::spawn(dave_task); // give Dave a chance to spawn and init. - run_for(Duration::from_millis(400), &net, &mut runtime); + run_for(Duration::from_millis(400), &net).await; let (dave_best_blocks, _) = get_beefy_streams(&mut net.lock(), [(dave_index, BeefyKeyring::Dave)].into_iter()); @@ -962,41 +930,40 @@ fn on_demand_beefy_justification_sync() { let hashof1 = client.expect_block_hash_from_id(&BlockId::number(1)).unwrap(); client.finalize_block(hashof1, None).unwrap(); // Give Dave task some cpu cycles to process the finality notification, - run_for(Duration::from_millis(100), &net, &mut runtime); + run_for(Duration::from_millis(100), &net).await; // freshly spun up Dave now needs to listen for gossip to figure out the state of his peers. // Have the other peers do some gossip so Dave finds out about their progress. - finalize_block_and_wait_for_beefy(&net, fast_peers, &mut runtime, &[25], &[25]); + finalize_block_and_wait_for_beefy(&net, fast_peers, &[25], &[25]).await; // Now verify Dave successfully finalized #1 (through on-demand justification request). - wait_for_best_beefy_blocks(dave_best_blocks, &net, &mut runtime, &[1]); + wait_for_best_beefy_blocks(dave_best_blocks, &net, &[1]).await; // Give Dave all tasks some cpu cycles to burn through their events queues, - run_for(Duration::from_millis(100), &net, &mut runtime); + run_for(Duration::from_millis(100), &net).await; // then verify Dave catches up through on-demand justification requests. finalize_block_and_wait_for_beefy( &net, [(dave_index, BeefyKeyring::Dave)].into_iter(), - &mut runtime, &[6, 10, 17, 24, 26], &[5, 10, 15, 20, 25], - ); + ) + .await; let all_peers = all_peers.into_iter().enumerate(); // Now that Dave has caught up, sanity check voting works for all of them. - finalize_block_and_wait_for_beefy(&net, all_peers, &mut runtime, &[30], &[30]); + finalize_block_and_wait_for_beefy(&net, all_peers, &[30], &[30]).await; } -#[test] -fn should_initialize_voter_at_genesis() { +#[tokio::test] +async fn should_initialize_voter_at_genesis() { let keys = &[BeefyKeyring::Alice]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let mut runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let backend = net.peer(0).client().as_backend(); // push 15 blocks with `AuthorityChange` digests every 10 blocks - net.generate_blocks_and_sync(15, 10, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(15, 10, &validator_set, false).await; let mut finality = net.peer(0).client().as_client().finality_notification_stream().fuse(); @@ -1033,16 +1000,15 @@ fn should_initialize_voter_at_genesis() { assert_eq!(state, persisted_state); } -#[test] -fn should_initialize_voter_when_last_final_is_session_boundary() { +#[tokio::test] +async fn should_initialize_voter_when_last_final_is_session_boundary() { let keys = &[BeefyKeyring::Alice]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let mut runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let backend = net.peer(0).client().as_backend(); // push 15 blocks with `AuthorityChange` digests every 10 blocks - net.generate_blocks_and_sync(15, 10, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(15, 10, &validator_set, false).await; let mut finality = net.peer(0).client().as_client().finality_notification_stream().fuse(); @@ -1094,16 +1060,15 @@ fn should_initialize_voter_when_last_final_is_session_boundary() { assert_eq!(state, persisted_state); } -#[test] -fn should_initialize_voter_at_latest_finalized() { +#[tokio::test] +async fn should_initialize_voter_at_latest_finalized() { let keys = &[BeefyKeyring::Alice]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let mut runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let backend = net.peer(0).client().as_backend(); // push 15 blocks with `AuthorityChange` digests every 10 blocks - net.generate_blocks_and_sync(15, 10, &validator_set, false, &mut runtime); + net.generate_blocks_and_sync(15, 10, &validator_set, false).await; let mut finality = net.peer(0).client().as_client().finality_notification_stream().fuse(); diff --git a/client/beefy/src/worker.rs b/client/beefy/src/worker.rs index bba3563a8f70e..df465b2dbf52c 100644 --- a/client/beefy/src/worker.rs +++ b/client/beefy/src/worker.rs @@ -983,7 +983,6 @@ pub(crate) mod tests { runtime::{Block, Digest, DigestItem, Header, H256}, Backend, }; - use tokio::runtime::Runtime; impl PersistedState { pub fn voting_oracle(&self) -> &VoterOracle { @@ -1295,12 +1294,11 @@ pub(crate) mod tests { assert_eq!(extracted, Some(validator_set)); } - #[test] - fn keystore_vs_validator_set() { + #[tokio::test] + async fn keystore_vs_validator_set() { let keys = &[Keyring::Alice]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let mut worker = create_beefy_worker(&net.peer(0), &keys[0], 1, validator_set.clone()); // keystore doesn't contain other keys than validators' @@ -1319,12 +1317,11 @@ pub(crate) mod tests { assert_eq!(worker.verify_validator_set(&1, &validator_set), expected_err); } - #[test] - fn should_finalize_correctly() { + #[tokio::test] + async fn should_finalize_correctly() { let keys = [Keyring::Alice]; let validator_set = ValidatorSet::new(make_beefy_ids(&keys), 0).unwrap(); - let runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let backend = net.peer(0).client().as_backend(); let mut worker = create_beefy_worker(&net.peer(0), &keys[0], 1, validator_set.clone()); // remove default session, will manually add custom one. @@ -1347,11 +1344,12 @@ pub(crate) mod tests { // no 'best beefy block' or finality proofs assert_eq!(worker.best_beefy_block(), 0); - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { assert_eq!(best_block_stream.poll_next_unpin(cx), Poll::Pending); assert_eq!(finality_proof.poll_next_unpin(cx), Poll::Pending); Poll::Ready(()) - })); + }) + .await; // unknown hash for block #1 let (mut best_block_streams, mut finality_proofs) = @@ -1368,7 +1366,7 @@ pub(crate) mod tests { worker.finalize(justif.clone()).unwrap(); // verify block finalized assert_eq!(worker.best_beefy_block(), 1); - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { // unknown hash -> nothing streamed assert_eq!(best_block_stream.poll_next_unpin(cx), Poll::Pending); // commitment streamed @@ -1378,7 +1376,8 @@ pub(crate) mod tests { v => panic!("unexpected value: {:?}", v), } Poll::Ready(()) - })); + }) + .await; // generate 2 blocks, try again expect success let (mut best_block_streams, _) = get_beefy_streams(&mut net, keys); @@ -1400,7 +1399,7 @@ pub(crate) mod tests { assert_eq!(worker.active_rounds().unwrap().session_start(), 2); // verify block finalized assert_eq!(worker.best_beefy_block(), 2); - runtime.block_on(poll_fn(move |cx| { + poll_fn(move |cx| { match best_block_stream.poll_next_unpin(cx) { // expect Some(hash-of-block-2) Poll::Ready(Some(hash)) => { @@ -1410,19 +1409,19 @@ pub(crate) mod tests { v => panic!("unexpected value: {:?}", v), } Poll::Ready(()) - })); + }) + .await; // check BEEFY justifications are also appended to backend let justifs = backend.blockchain().justifications(hashof2).unwrap().unwrap(); assert!(justifs.get(BEEFY_ENGINE_ID).is_some()) } - #[test] - fn should_init_session() { + #[tokio::test] + async fn should_init_session() { let keys = &[Keyring::Alice, Keyring::Bob]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let mut worker = create_beefy_worker(&net.peer(0), &keys[0], 1, validator_set.clone()); let worker_rounds = worker.active_rounds().unwrap(); @@ -1449,12 +1448,11 @@ pub(crate) mod tests { assert_eq!(rounds.validator_set_id(), new_validator_set.id()); } - #[test] - fn should_triage_votes_and_process_later() { + #[tokio::test] + async fn should_triage_votes_and_process_later() { let keys = &[Keyring::Alice, Keyring::Bob]; let validator_set = ValidatorSet::new(make_beefy_ids(keys), 0).unwrap(); - let runtime = Runtime::new().unwrap(); - let mut net = BeefyTestNet::new(runtime.handle().clone(), 1); + let mut net = BeefyTestNet::new(1); let mut worker = create_beefy_worker(&net.peer(0), &keys[0], 1, validator_set.clone()); // remove default session, will manually add custom one. worker.persisted_state.voting_oracle.sessions.clear(); diff --git a/client/consensus/aura/src/lib.rs b/client/consensus/aura/src/lib.rs index a8ed80d7c0432..212376dcd009c 100644 --- a/client/consensus/aura/src/lib.rs +++ b/client/consensus/aura/src/lib.rs @@ -660,7 +660,6 @@ mod tests { runtime::{Header, H256}, TestClient, }; - use tokio::runtime::{Handle, Runtime}; const SLOT_DURATION_MS: u64 = 1000; @@ -718,20 +717,11 @@ mod tests { >; type AuraPeer = Peer<(), PeersClient>; + #[derive(Default)] pub struct AuraTestNet { - rt_handle: Handle, peers: Vec, } - impl WithRuntime for AuraTestNet { - fn with_runtime(rt_handle: Handle) -> Self { - AuraTestNet { rt_handle, peers: Vec::new() } - } - fn rt_handle(&self) -> &Handle { - &self.rt_handle - } - } - impl TestNetFactory for AuraTestNet { type Verifier = AuraVerifier; type PeerData = (); @@ -780,11 +770,10 @@ mod tests { } } - #[test] - fn authoring_blocks() { + #[tokio::test] + async fn authoring_blocks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let net = AuraTestNet::new(runtime.handle().clone(), 3); + let net = AuraTestNet::new(3); let peers = &[(0, Keyring::Alice), (1, Keyring::Bob), (2, Keyring::Charlie)]; @@ -850,13 +839,14 @@ mod tests { ); } - runtime.block_on(future::select( + future::select( future::poll_fn(move |cx| { net.lock().poll(cx); Poll::<()>::Pending }), future::select(future::join_all(aura_futures), future::join_all(import_notifications)), - )); + ) + .await; } #[test] @@ -875,10 +865,9 @@ mod tests { ); } - #[test] - fn current_node_authority_should_claim_slot() { - let runtime = Runtime::new().unwrap(); - let net = AuraTestNet::new(runtime.handle().clone(), 4); + #[tokio::test] + async fn current_node_authority_should_claim_slot() { + let net = AuraTestNet::new(4); let mut authorities = vec![ Keyring::Alice.public().into(), @@ -922,20 +911,19 @@ mod tests { Default::default(), Default::default(), ); - assert!(runtime.block_on(worker.claim_slot(&head, 0.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 1.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 2.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 3.into(), &authorities)).is_some()); - assert!(runtime.block_on(worker.claim_slot(&head, 4.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 5.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 6.into(), &authorities)).is_none()); - assert!(runtime.block_on(worker.claim_slot(&head, 7.into(), &authorities)).is_some()); + assert!(worker.claim_slot(&head, 0.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 1.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 2.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 3.into(), &authorities).await.is_some()); + assert!(worker.claim_slot(&head, 4.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 5.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 6.into(), &authorities).await.is_none()); + assert!(worker.claim_slot(&head, 7.into(), &authorities).await.is_some()); } - #[test] - fn on_slot_returns_correct_block() { - let runtime = Runtime::new().unwrap(); - let net = AuraTestNet::new(runtime.handle().clone(), 4); + #[tokio::test] + async fn on_slot_returns_correct_block() { + let net = AuraTestNet::new(4); let keystore_path = tempfile::tempdir().expect("Creates keystore path"); let keystore = LocalKeystore::open(keystore_path.path(), None).expect("Creates keystore."); @@ -971,15 +959,16 @@ mod tests { let head = client.header(&BlockId::Number(0)).unwrap().unwrap(); - let res = runtime - .block_on(worker.on_slot(SlotInfo { + let res = worker + .on_slot(SlotInfo { slot: 0.into(), ends_at: Instant::now() + Duration::from_secs(100), create_inherent_data: Box::new(()), duration: Duration::from_millis(1000), chain_head: head, block_size_limit: None, - })) + }) + .await .unwrap(); // The returned block should be imported and we should be able to get its header by now. diff --git a/client/consensus/babe/src/tests.rs b/client/consensus/babe/src/tests.rs index d7691235a550d..6861379ebbeb5 100644 --- a/client/consensus/babe/src/tests.rs +++ b/client/consensus/babe/src/tests.rs @@ -49,7 +49,6 @@ use sp_runtime::{ }; use sp_timestamp::Timestamp; use std::{cell::RefCell, task::Poll, time::Duration}; -use tokio::runtime::{Handle, Runtime}; type Item = DigestItem; @@ -227,20 +226,11 @@ where type BabePeer = Peer, BabeBlockImport>; +#[derive(Default)] pub struct BabeTestNet { - rt_handle: Handle, peers: Vec, } -impl WithRuntime for BabeTestNet { - fn with_runtime(rt_handle: Handle) -> Self { - BabeTestNet { rt_handle, peers: Vec::new() } - } - fn rt_handle(&self) -> &Handle { - &self.rt_handle - } -} - type TestHeader = ::Header; type TestSelectChain = @@ -366,12 +356,11 @@ impl TestNetFactory for BabeTestNet { } } -#[test] +#[tokio::test] #[should_panic] -fn rejects_empty_block() { +async fn rejects_empty_block() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 3); + let mut net = BabeTestNet::new(3); let block_builder = |builder: BlockBuilder<_, _, _>| builder.build().unwrap().block; net.mut_peers(|peer| { peer[0].generate_blocks(1, BlockOrigin::NetworkInitialSync, block_builder); @@ -385,14 +374,13 @@ fn create_keystore(authority: Sr25519Keyring) -> SyncCryptoStorePtr { keystore } -fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static) { +async fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static) { sp_tracing::try_init_simple(); let mutator = Arc::new(mutator) as Mutator; MUTATOR.with(|m| *m.borrow_mut() = mutator.clone()); - let runtime = Runtime::new().unwrap(); - let net = BabeTestNet::new(runtime.handle().clone(), 3); + let net = BabeTestNet::new(3); let peers = [Sr25519Keyring::Alice, Sr25519Keyring::Bob, Sr25519Keyring::Charlie]; @@ -469,7 +457,7 @@ fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static .expect("Starts babe"), ); } - runtime.block_on(future::select( + future::select( futures::future::poll_fn(move |cx| { let mut net = net.lock(); net.poll(cx); @@ -482,17 +470,18 @@ fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static Poll::<()>::Pending }), future::select(future::join_all(import_notifications), future::join_all(babe_futures)), - )); + ) + .await; } -#[test] -fn authoring_blocks() { - run_one_test(|_, _| ()) +#[tokio::test] +async fn authoring_blocks() { + run_one_test(|_, _| ()).await; } -#[test] +#[tokio::test] #[should_panic] -fn rejects_missing_inherent_digest() { +async fn rejects_missing_inherent_digest() { run_one_test(|header: &mut TestHeader, stage| { let v = std::mem::take(&mut header.digest_mut().logs); header.digest_mut().logs = v @@ -500,11 +489,12 @@ fn rejects_missing_inherent_digest() { .filter(|v| stage == Stage::PostSeal || v.as_babe_pre_digest().is_none()) .collect() }) + .await; } -#[test] +#[tokio::test] #[should_panic] -fn rejects_missing_seals() { +async fn rejects_missing_seals() { run_one_test(|header: &mut TestHeader, stage| { let v = std::mem::take(&mut header.digest_mut().logs); header.digest_mut().logs = v @@ -512,18 +502,20 @@ fn rejects_missing_seals() { .filter(|v| stage == Stage::PreSeal || v.as_babe_seal().is_none()) .collect() }) + .await; } -#[test] +#[tokio::test] #[should_panic] -fn rejects_missing_consensus_digests() { +async fn rejects_missing_consensus_digests() { run_one_test(|header: &mut TestHeader, stage| { let v = std::mem::take(&mut header.digest_mut().logs); header.digest_mut().logs = v .into_iter() .filter(|v| stage == Stage::PostSeal || v.as_next_epoch_descriptor().is_none()) .collect() - }); + }) + .await; } #[test] @@ -601,14 +593,13 @@ fn can_author_block() { } // Propose and import a new BABE block on top of the given parent. -fn propose_and_import_block( +async fn propose_and_import_block( parent: &TestHeader, slot: Option, proposer_factory: &mut DummyFactory, block_import: &mut BoxBlockImport, - runtime: &Runtime, ) -> Hash { - let mut proposer = runtime.block_on(proposer_factory.init(parent)).unwrap(); + let mut proposer = proposer_factory.init(parent).await.unwrap(); let slot = slot.unwrap_or_else(|| { let parent_pre_digest = find_pre_digest::(parent).unwrap(); @@ -624,7 +615,7 @@ fn propose_and_import_block( let parent_hash = parent.hash(); - let mut block = runtime.block_on(proposer.propose_with(pre_digest)).unwrap().block; + let mut block = proposer.propose_with(pre_digest).await.unwrap().block; let epoch_descriptor = proposer_factory .epoch_changes @@ -660,8 +651,7 @@ fn propose_and_import_block( import .insert_intermediate(INTERMEDIATE_KEY, BabeIntermediate:: { epoch_descriptor }); import.fork_choice = Some(ForkChoiceStrategy::LongestChain); - let import_result = - runtime.block_on(block_import.import_block(import, Default::default())).unwrap(); + let import_result = block_import.import_block(import, Default::default()).await.unwrap(); match import_result { ImportResult::Imported(_) => {}, @@ -674,20 +664,19 @@ fn propose_and_import_block( // Propose and import n valid BABE blocks that are built on top of the given parent. // The proposer takes care of producing epoch change digests according to the epoch // duration (which is set to 6 slots in the test runtime). -fn propose_and_import_blocks( +async fn propose_and_import_blocks( client: &PeersFullClient, proposer_factory: &mut DummyFactory, block_import: &mut BoxBlockImport, parent_id: BlockId, n: usize, - runtime: &Runtime, ) -> Vec { let mut hashes = Vec::with_capacity(n); let mut parent_header = client.header(&parent_id).unwrap().unwrap(); for _ in 0..n { let block_hash = - propose_and_import_block(&parent_header, None, proposer_factory, block_import, runtime); + propose_and_import_block(&parent_header, None, proposer_factory, block_import).await; hashes.push(block_hash); parent_header = client.header(&BlockId::Hash(block_hash)).unwrap().unwrap(); } @@ -695,10 +684,9 @@ fn propose_and_import_blocks( hashes } -#[test] -fn importing_block_one_sets_genesis_epoch() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +#[tokio::test] +async fn importing_block_one_sets_genesis_epoch() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -720,8 +708,8 @@ fn importing_block_one_sets_genesis_epoch() { Some(999.into()), &mut proposer_factory, &mut block_import, - &runtime, - ); + ) + .await; let genesis_epoch = Epoch::genesis(&data.link.config, 999.into()); @@ -736,10 +724,9 @@ fn importing_block_one_sets_genesis_epoch() { assert_eq!(epoch_for_second_block, genesis_epoch); } -#[test] -fn revert_prunes_epoch_changes_and_removes_weights() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +#[tokio::test] +async fn revert_prunes_epoch_changes_and_removes_weights() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -756,17 +743,6 @@ fn revert_prunes_epoch_changes_and_removes_weights() { mutator: Arc::new(|_, _| ()), }; - let mut propose_and_import_blocks_wrap = |parent_id, n| { - propose_and_import_blocks( - &client, - &mut proposer_factory, - &mut block_import, - parent_id, - n, - &runtime, - ) - }; - // Test scenario. // Information for epoch 19 is produced on three different forks at block #13. // One branch starts before the revert point (epoch data should be maintained). @@ -779,10 +755,38 @@ fn revert_prunes_epoch_changes_and_removes_weights() { // \ revert *---- G(#13) ---- H(#19) ---#20 < fork #3 // \ to #10 // *-----E(#7)---#11 < fork #1 - let canon = propose_and_import_blocks_wrap(BlockId::Number(0), 21); - let fork1 = propose_and_import_blocks_wrap(BlockId::Hash(canon[0]), 10); - let fork2 = propose_and_import_blocks_wrap(BlockId::Hash(canon[7]), 10); - let fork3 = propose_and_import_blocks_wrap(BlockId::Hash(canon[11]), 8); + let canon = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(0), + 21, + ) + .await; + let fork1 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[0]), + 10, + ) + .await; + let fork2 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[7]), + 10, + ) + .await; + let fork3 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[11]), + 8, + ) + .await; // We should be tracking a total of 9 epochs in the fork tree assert_eq!(epoch_changes.shared_data().tree().iter().count(), 8); @@ -824,10 +828,9 @@ fn revert_prunes_epoch_changes_and_removes_weights() { assert!(weight_data_check(&fork3, false)); } -#[test] -fn revert_not_allowed_for_finalized() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +#[tokio::test] +async fn revert_not_allowed_for_finalized() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -843,18 +846,14 @@ fn revert_not_allowed_for_finalized() { mutator: Arc::new(|_, _| ()), }; - let mut propose_and_import_blocks_wrap = |parent_id, n| { - propose_and_import_blocks( - &client, - &mut proposer_factory, - &mut block_import, - parent_id, - n, - &runtime, - ) - }; - - let canon = propose_and_import_blocks_wrap(BlockId::Number(0), 3); + let canon = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(0), + 3, + ) + .await; // Finalize best block client.finalize_block(canon[2], None, false).unwrap(); @@ -870,10 +869,9 @@ fn revert_not_allowed_for_finalized() { assert!(weight_data_check(&canon, true)); } -#[test] -fn importing_epoch_change_block_prunes_tree() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +#[tokio::test] +async fn importing_epoch_change_block_prunes_tree() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -889,17 +887,6 @@ fn importing_epoch_change_block_prunes_tree() { mutator: Arc::new(|_, _| ()), }; - let mut propose_and_import_blocks_wrap = |parent_id, n| { - propose_and_import_blocks( - &client, - &mut proposer_factory, - &mut block_import, - parent_id, - n, - &runtime, - ) - }; - // This is the block tree that we're going to use in this test. Each node // represents an epoch change block, the epoch duration is 6 slots. // @@ -912,12 +899,40 @@ fn importing_epoch_change_block_prunes_tree() { // Create and import the canon chain and keep track of fork blocks (A, C, D) // from the diagram above. - let canon = propose_and_import_blocks_wrap(BlockId::Number(0), 30); + let canon = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(0), + 30, + ) + .await; // Create the forks - let fork_1 = propose_and_import_blocks_wrap(BlockId::Hash(canon[0]), 10); - let fork_2 = propose_and_import_blocks_wrap(BlockId::Hash(canon[12]), 15); - let fork_3 = propose_and_import_blocks_wrap(BlockId::Hash(canon[18]), 10); + let fork_1 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[0]), + 10, + ) + .await; + let fork_2 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[12]), + 15, + ) + .await; + let fork_3 = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(canon[18]), + 10, + ) + .await; // We should be tracking a total of 9 epochs in the fork tree assert_eq!(epoch_changes.shared_data().tree().iter().count(), 9); @@ -928,7 +943,14 @@ fn importing_epoch_change_block_prunes_tree() { // We finalize block #13 from the canon chain, so on the next epoch // change the tree should be pruned, to not contain F (#7). client.finalize_block(canon[12], None, false).unwrap(); - propose_and_import_blocks_wrap(BlockId::Hash(client.chain_info().best_hash), 7); + propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(client.chain_info().best_hash), + 7, + ) + .await; let nodes: Vec<_> = epoch_changes.shared_data().tree().iter().map(|(h, _, _)| *h).collect(); @@ -941,7 +963,14 @@ fn importing_epoch_change_block_prunes_tree() { // finalizing block #25 from the canon chain should prune out the second fork client.finalize_block(canon[24], None, false).unwrap(); - propose_and_import_blocks_wrap(BlockId::Hash(client.chain_info().best_hash), 8); + propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Hash(client.chain_info().best_hash), + 8, + ) + .await; let nodes: Vec<_> = epoch_changes.shared_data().tree().iter().map(|(h, _, _)| *h).collect(); @@ -954,11 +983,10 @@ fn importing_epoch_change_block_prunes_tree() { assert!(nodes.iter().any(|h| *h == canon[24])); } -#[test] +#[tokio::test] #[should_panic] -fn verify_slots_are_strictly_increasing() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +async fn verify_slots_are_strictly_increasing() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -981,20 +1009,14 @@ fn verify_slots_are_strictly_increasing() { Some(999.into()), &mut proposer_factory, &mut block_import, - &runtime, - ); + ) + .await; let b1 = client.header(&BlockId::Hash(b1)).unwrap().unwrap(); // we should fail to import this block since the slot number didn't increase. // we will panic due to the `PanickingBlockImport` defined above. - propose_and_import_block( - &b1, - Some(999.into()), - &mut proposer_factory, - &mut block_import, - &runtime, - ); + propose_and_import_block(&b1, Some(999.into()), &mut proposer_factory, &mut block_import).await; } #[test] @@ -1027,10 +1049,9 @@ fn babe_transcript_generation_match() { debug_assert!(test(orig_transcript) == test(transcript_from_data(new_transcript))); } -#[test] -fn obsolete_blocks_aux_data_cleanup() { - let runtime = Runtime::new().unwrap(); - let mut net = BabeTestNet::new(runtime.handle().clone(), 1); +#[tokio::test] +async fn obsolete_blocks_aux_data_cleanup() { + let mut net = BabeTestNet::new(1); let peer = net.peer(0); let data = peer.data.as_ref().expect("babe link set up during initialization"); @@ -1052,17 +1073,6 @@ fn obsolete_blocks_aux_data_cleanup() { let mut block_import = data.block_import.lock().take().expect("import set up during init"); - let mut propose_and_import_blocks_wrap = |parent_id, n| { - propose_and_import_blocks( - &client, - &mut proposer_factory, - &mut block_import, - parent_id, - n, - &runtime, - ) - }; - let aux_data_check = |hashes: &[Hash], expected: bool| { hashes.iter().all(|hash| { aux_schema::load_block_weight(&*peer.client().as_backend(), hash) @@ -1077,9 +1087,30 @@ fn obsolete_blocks_aux_data_cleanup() { // G --- A1 --- A2 --- A3 --- A4 ( < fork1 ) // \-----C4 --- C5 ( < fork3 ) - let fork1_hashes = propose_and_import_blocks_wrap(BlockId::Number(0), 4); - let fork2_hashes = propose_and_import_blocks_wrap(BlockId::Number(0), 2); - let fork3_hashes = propose_and_import_blocks_wrap(BlockId::Number(3), 2); + let fork1_hashes = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(0), + 4, + ) + .await; + let fork2_hashes = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(0), + 2, + ) + .await; + let fork3_hashes = propose_and_import_blocks( + &client, + &mut proposer_factory, + &mut block_import, + BlockId::Number(3), + 2, + ) + .await; // Check that aux data is present for all but the genesis block. assert!(aux_data_check(&[client.chain_info().genesis_hash], false)); diff --git a/client/finality-grandpa/src/tests.rs b/client/finality-grandpa/src/tests.rs index 6b577fd712930..97c0c9fe7ce49 100644 --- a/client/finality-grandpa/src/tests.rs +++ b/client/finality-grandpa/src/tests.rs @@ -30,7 +30,7 @@ use sc_consensus::{ use sc_network::config::Role; use sc_network_test::{ Block, BlockImportAdapter, FullPeerConfig, Hash, PassThroughVerifier, Peer, PeersClient, - PeersFullClient, TestClient, TestNetFactory, WithRuntime, + PeersFullClient, TestClient, TestNetFactory, }; use sp_api::{ApiRef, ProvideRuntimeApi}; use sp_blockchain::Result; @@ -52,7 +52,7 @@ use std::{ pin::Pin, }; use substrate_test_runtime_client::runtime::BlockNumber; -use tokio::runtime::{Handle, Runtime}; +use tokio::runtime::Handle; use authorities::AuthoritySet; use communication::grandpa_protocol_name; @@ -71,26 +71,16 @@ type GrandpaBlockImport = crate::GrandpaBlockImport< LongestChain, >; +#[derive(Default)] struct GrandpaTestNet { peers: Vec, test_config: TestApi, - rt_handle: Handle, -} - -impl WithRuntime for GrandpaTestNet { - fn with_runtime(rt_handle: Handle) -> Self { - GrandpaTestNet { peers: Vec::new(), test_config: TestApi::default(), rt_handle } - } - fn rt_handle(&self) -> &Handle { - &self.rt_handle - } } impl GrandpaTestNet { - fn new(test_config: TestApi, n_authority: usize, n_full: usize, rt_handle: Handle) -> Self { - let mut net = GrandpaTestNet::with_runtime(rt_handle); - net.peers = Vec::with_capacity(n_authority + n_full); - net.test_config = test_config; + fn new(test_config: TestApi, n_authority: usize, n_full: usize) -> Self { + let mut net = + GrandpaTestNet { peers: Vec::with_capacity(n_authority + n_full), test_config }; for _ in 0..n_authority { net.add_authority_peer(); @@ -228,16 +218,12 @@ fn create_keystore(authority: Ed25519Keyring) -> SyncCryptoStorePtr { keystore } -fn block_until_complete( - future: impl Future + Unpin, - net: &Arc>, - runtime: &mut Runtime, -) { +async fn run_until_complete(future: impl Future + Unpin, net: &Arc>) { let drive_to_completion = futures::future::poll_fn(|cx| { net.lock().poll(cx); Poll::<()>::Pending }); - runtime.block_on(future::select(future, drive_to_completion)); + future::select(future, drive_to_completion).await; } // Spawns grandpa voters. Returns a future to spawn on the runtime. @@ -289,8 +275,7 @@ fn initialize_grandpa( // run the voters to completion. provide a closure to be invoked after // the voters are spawned but before blocking on them. -fn run_to_completion_with( - runtime: &mut Runtime, +async fn run_to_completion_with( blocks: u64, net: Arc>, peers: &[Ed25519Keyring], @@ -303,7 +288,7 @@ where let highest_finalized = Arc::new(RwLock::new(0)); - if let Some(f) = (with)(runtime.handle().clone()) { + if let Some(f) = (with)(Handle::current()) { wait_for.push(f); }; @@ -329,18 +314,17 @@ where // wait for all finalized on each. let wait_for = ::futures::future::join_all(wait_for); - block_until_complete(wait_for, &net, runtime); + run_until_complete(wait_for, &net).await; let highest_finalized = *highest_finalized.read(); highest_finalized } -fn run_to_completion( - runtime: &mut Runtime, +async fn run_to_completion( blocks: u64, net: Arc>, peers: &[Ed25519Keyring], ) -> u64 { - run_to_completion_with(runtime, blocks, net, peers, |_| None) + run_to_completion_with(blocks, net, peers, |_| None).await } fn add_scheduled_change(block: &mut Block, change: ScheduledChange) { @@ -361,17 +345,16 @@ fn add_forced_change( )); } -#[test] -fn finalize_3_voters_no_observers() { +#[tokio::test] +async fn finalize_3_voters_no_observers() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let peers = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let voters = make_ids(peers); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 0, runtime.handle().clone()); - runtime.spawn(initialize_grandpa(&mut net, peers)); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 0); + tokio::spawn(initialize_grandpa(&mut net, peers)); net.peer(0).push_blocks(20, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let hashof20 = net.peer(0).client().info().best_hash; for i in 0..3 { @@ -380,7 +363,7 @@ fn finalize_3_voters_no_observers() { } let net = Arc::new(Mutex::new(net)); - run_to_completion(&mut runtime, 20, net.clone(), peers); + run_to_completion(20, net.clone(), peers).await; // normally there's no justification for finalized blocks assert!( @@ -389,17 +372,15 @@ fn finalize_3_voters_no_observers() { ); } -#[test] -fn finalize_3_voters_1_full_observer() { - let mut runtime = Runtime::new().unwrap(); - +#[tokio::test] +async fn finalize_3_voters_1_full_observer() { let peers = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let voters = make_ids(peers); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 1, runtime.handle().clone()); - runtime.spawn(initialize_grandpa(&mut net, peers)); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 1); + tokio::spawn(initialize_grandpa(&mut net, peers)); - runtime.spawn({ + tokio::spawn({ let peer_id = 3; let net_service = net.peers[peer_id].network_service().clone(); let link = net.peers[peer_id].data.lock().take().expect("link initialized at startup; qed"); @@ -444,7 +425,7 @@ fn finalize_3_voters_1_full_observer() { // wait for all finalized on each. let wait_for = futures::future::join_all(finality_notifications).map(|_| ()); - block_until_complete(wait_for, &net, &mut runtime); + run_until_complete(wait_for, &net).await; // all peers should have stored the justification for the best finalized block #20 for peer_id in 0..4 { @@ -456,8 +437,8 @@ fn finalize_3_voters_1_full_observer() { } } -#[test] -fn transition_3_voters_twice_1_full_observer() { +#[tokio::test] +async fn transition_3_voters_twice_1_full_observer() { sp_tracing::try_init_simple(); let peers_a = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; @@ -478,8 +459,7 @@ fn transition_3_voters_twice_1_full_observer() { let genesis_voters = make_ids(peers_a); let api = TestApi::new(genesis_voters); - let mut runtime = Runtime::new().unwrap(); - let net = Arc::new(Mutex::new(GrandpaTestNet::new(api, 8, 1, runtime.handle().clone()))); + let net = Arc::new(Mutex::new(GrandpaTestNet::new(api, 8, 1))); let mut voters = Vec::new(); for (peer_id, local_key) in all_peers.clone().into_iter().enumerate() { @@ -516,7 +496,7 @@ fn transition_3_voters_twice_1_full_observer() { } net.lock().peer(0).push_blocks(1, false); - runtime.block_on(net.lock().wait_until_sync()); + net.lock().run_until_sync().await; for (i, peer) in net.lock().peers().iter().enumerate() { let full_client = peer.client().as_client(); @@ -577,13 +557,13 @@ fn transition_3_voters_twice_1_full_observer() { future::ready(()) }); - runtime.spawn(block_production); + tokio::spawn(block_production); } let mut finality_notifications = Vec::new(); for voter in voters { - runtime.spawn(voter); + tokio::spawn(voter); } for (peer_id, _) in all_peers.into_iter().enumerate() { @@ -607,24 +587,23 @@ fn transition_3_voters_twice_1_full_observer() { // wait for all finalized on each. let wait_for = ::futures::future::join_all(finality_notifications); - block_until_complete(wait_for, &net, &mut runtime); + run_until_complete(wait_for, &net).await; } -#[test] -fn justification_is_generated_periodically() { - let mut runtime = Runtime::new().unwrap(); +#[tokio::test] +async fn justification_is_generated_periodically() { let peers = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let voters = make_ids(peers); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 0, runtime.handle().clone()); - runtime.spawn(initialize_grandpa(&mut net, peers)); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 0); + tokio::spawn(initialize_grandpa(&mut net, peers)); net.peer(0).push_blocks(32, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let hashof32 = net.peer(0).client().info().best_hash; let net = Arc::new(Mutex::new(net)); - run_to_completion(&mut runtime, 32, net.clone(), peers); + run_to_completion(32, net.clone(), peers).await; // when block#32 (justification_period) is finalized, justification // is required => generated @@ -633,16 +612,15 @@ fn justification_is_generated_periodically() { } } -#[test] -fn sync_justifications_on_change_blocks() { - let mut runtime = Runtime::new().unwrap(); +#[tokio::test] +async fn sync_justifications_on_change_blocks() { let peers_a = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let peers_b = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob]; let voters = make_ids(peers_b); // 4 peers, 3 of them are authorities and participate in grandpa let api = TestApi::new(voters); - let mut net = GrandpaTestNet::new(api, 3, 1, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api, 3, 1); let voters = initialize_grandpa(&mut net, peers_a); // add 20 blocks @@ -660,15 +638,15 @@ fn sync_justifications_on_change_blocks() { // add more blocks on top of it (until we have 25) net.peer(0).push_blocks(4, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; for i in 0..4 { assert_eq!(net.peer(i).client().info().best_number, 25, "Peer #{} failed to sync", i); } let net = Arc::new(Mutex::new(net)); - runtime.spawn(voters); - run_to_completion(&mut runtime, 25, net.clone(), peers_a); + tokio::spawn(voters); + run_to_completion(25, net.clone(), peers_a).await; // the first 3 peers are grandpa voters and therefore have already finalized // block 21 and stored a justification @@ -677,20 +655,20 @@ fn sync_justifications_on_change_blocks() { } // the last peer should get the justification by syncing from other peers - futures::executor::block_on(futures::future::poll_fn(move |cx| { + futures::future::poll_fn(move |cx| { if net.lock().peer(3).client().justifications(hashof21).unwrap().is_none() { net.lock().poll(cx); Poll::Pending } else { Poll::Ready(()) } - })) + }) + .await; } -#[test] -fn finalizes_multiple_pending_changes_in_order() { +#[tokio::test] +async fn finalizes_multiple_pending_changes_in_order() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let peers_a = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let peers_b = &[Ed25519Keyring::Dave, Ed25519Keyring::Eve, Ed25519Keyring::Ferdie]; @@ -710,8 +688,8 @@ fn finalizes_multiple_pending_changes_in_order() { // but all of them will be part of the voter set eventually so they should be // all added to the network as authorities let api = TestApi::new(genesis_voters); - let mut net = GrandpaTestNet::new(api, 6, 0, runtime.handle().clone()); - runtime.spawn(initialize_grandpa(&mut net, all_peers)); + let mut net = GrandpaTestNet::new(api, 6, 0); + tokio::spawn(initialize_grandpa(&mut net, all_peers)); // add 20 blocks net.peer(0).push_blocks(20, false); @@ -742,7 +720,7 @@ fn finalizes_multiple_pending_changes_in_order() { // add more blocks on top of it (until we have 30) net.peer(0).push_blocks(4, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; // all peers imported both change blocks for i in 0..6 { @@ -750,13 +728,12 @@ fn finalizes_multiple_pending_changes_in_order() { } let net = Arc::new(Mutex::new(net)); - run_to_completion(&mut runtime, 30, net.clone(), all_peers); + run_to_completion(30, net.clone(), all_peers).await; } -#[test] -fn force_change_to_new_set() { +#[tokio::test] +async fn force_change_to_new_set() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); // two of these guys are offline. let genesis_authorities = &[ Ed25519Keyring::Alice, @@ -769,7 +746,7 @@ fn force_change_to_new_set() { let api = TestApi::new(make_ids(genesis_authorities)); let voters = make_ids(peers_a); - let mut net = GrandpaTestNet::new(api, 3, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api, 3, 0); let voters_future = initialize_grandpa(&mut net, peers_a); let net = Arc::new(Mutex::new(net)); @@ -793,7 +770,7 @@ fn force_change_to_new_set() { }); net.lock().peer(0).push_blocks(25, false); - runtime.block_on(net.lock().wait_until_sync()); + net.lock().run_until_sync().await; for (i, peer) in net.lock().peers().iter().enumerate() { assert_eq!(peer.client().info().best_number, 26, "Peer #{} failed to sync", i); @@ -809,18 +786,17 @@ fn force_change_to_new_set() { // it will only finalize if the forced transition happens. // we add_blocks after the voters are spawned because otherwise // the link-halves have the wrong AuthoritySet - runtime.spawn(voters_future); - run_to_completion(&mut runtime, 25, net, peers_a); + tokio::spawn(voters_future); + run_to_completion(25, net, peers_a).await; } -#[test] -fn allows_reimporting_change_blocks() { +#[tokio::test] +async fn allows_reimporting_change_blocks() { let peers_a = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let peers_b = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob]; let voters = make_ids(peers_a); let api = TestApi::new(voters); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(api.clone(), 3, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api.clone(), 3, 0); let client = net.peer(0).client().clone(); let (mut block_import, ..) = net.make_block_import(client.clone()); @@ -845,7 +821,7 @@ fn allows_reimporting_change_blocks() { }; assert_eq!( - runtime.block_on(block_import.import_block(block(), HashMap::new())).unwrap(), + block_import.import_block(block(), HashMap::new()).await.unwrap(), ImportResult::Imported(ImportedAux { needs_justification: true, clear_justification_requests: false, @@ -856,19 +832,18 @@ fn allows_reimporting_change_blocks() { ); assert_eq!( - runtime.block_on(block_import.import_block(block(), HashMap::new())).unwrap(), + block_import.import_block(block(), HashMap::new()).await.unwrap(), ImportResult::AlreadyInChain ); } -#[test] -fn test_bad_justification() { +#[tokio::test] +async fn test_bad_justification() { let peers_a = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let peers_b = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob]; let voters = make_ids(peers_a); let api = TestApi::new(voters); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(api.clone(), 3, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api.clone(), 3, 0); let client = net.peer(0).client().clone(); let (mut block_import, ..) = net.make_block_import(client.clone()); @@ -895,7 +870,7 @@ fn test_bad_justification() { }; assert_eq!( - runtime.block_on(block_import.import_block(block(), HashMap::new())).unwrap(), + block_import.import_block(block(), HashMap::new()).await.unwrap(), ImportResult::Imported(ImportedAux { needs_justification: true, clear_justification_requests: false, @@ -906,18 +881,17 @@ fn test_bad_justification() { ); assert_eq!( - runtime.block_on(block_import.import_block(block(), HashMap::new())).unwrap(), + block_import.import_block(block(), HashMap::new()).await.unwrap(), ImportResult::AlreadyInChain ); } -#[test] -fn voter_persists_its_votes() { +#[tokio::test] +async fn voter_persists_its_votes() { use futures::future; use std::sync::atomic::{AtomicUsize, Ordering}; sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); // we have two authorities but we'll only be running the voter for alice // we are going to be listening for the prevotes it casts @@ -925,7 +899,7 @@ fn voter_persists_its_votes() { let voters = make_ids(peers); // alice has a chain with 20 blocks - let mut net = GrandpaTestNet::new(TestApi::new(voters.clone()), 2, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(TestApi::new(voters.clone()), 2, 0); // create the communication layer for bob, but don't start any // voter. instead we'll listen for the prevote that alice casts @@ -1042,10 +1016,10 @@ fn voter_persists_its_votes() { }) } - runtime.spawn(alice_voter1); + tokio::spawn(alice_voter1); net.peer(0).push_blocks(20, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert_eq!(net.peer(0).client().info().best_number, 20, "Peer #{} failed to sync", 0); @@ -1062,7 +1036,7 @@ fn voter_persists_its_votes() { HasVoted::No, ); - runtime.spawn(bob_network); + tokio::spawn(bob_network); let round_tx = Arc::new(Mutex::new(round_tx)); let exit_tx = Arc::new(Mutex::new(Some(exit_tx))); @@ -1070,15 +1044,13 @@ fn voter_persists_its_votes() { let net = net.clone(); let state = Arc::new(AtomicUsize::new(0)); - let runtime_handle = runtime.handle().clone(); - runtime.spawn(round_rx.for_each(move |signed| { + tokio::spawn(round_rx.for_each(move |signed| { let net2 = net.clone(); let net = net.clone(); let abort = abort.clone(); let round_tx = round_tx.clone(); let state = state.clone(); let exit_tx = exit_tx.clone(); - let runtime_handle = runtime_handle.clone(); async move { if state.compare_exchange(0, 1, Ordering::SeqCst, Ordering::SeqCst).unwrap() == 0 { @@ -1115,7 +1087,7 @@ fn voter_persists_its_votes() { // we restart alice's voter abort.abort(); - runtime_handle.spawn(alice_voter2(peers, net.clone())); + tokio::spawn(alice_voter2(peers, net.clone())); // and we push our own prevote for block 30 let prevote = @@ -1164,17 +1136,16 @@ fn voter_persists_its_votes() { })); } - block_until_complete(exit_rx.into_future(), &net, &mut runtime); + run_until_complete(exit_rx.into_future(), &net).await; } -#[test] -fn finalize_3_voters_1_light_observer() { +#[tokio::test] +async fn finalize_3_voters_1_light_observer() { sp_tracing::try_init_simple(); - let mut runtime = Runtime::new().unwrap(); let authorities = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; let voters = make_ids(authorities); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 1, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 3, 1); let voters = initialize_grandpa(&mut net, authorities); let observer = observer::run_grandpa_observer( Config { @@ -1192,7 +1163,7 @@ fn finalize_3_voters_1_light_observer() { ) .unwrap(); net.peer(0).push_blocks(20, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; for i in 0..4 { assert_eq!(net.peer(i).client().info().best_number, 20, "Peer #{} failed to sync", i); @@ -1200,20 +1171,19 @@ fn finalize_3_voters_1_light_observer() { let net = Arc::new(Mutex::new(net)); - runtime.spawn(voters); - runtime.spawn(observer); - run_to_completion(&mut runtime, 20, net.clone(), authorities); + tokio::spawn(voters); + tokio::spawn(observer); + run_to_completion(20, net.clone(), authorities).await; } -#[test] -fn voter_catches_up_to_latest_round_when_behind() { +#[tokio::test] +async fn voter_catches_up_to_latest_round_when_behind() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); let peers = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob]; let voters = make_ids(peers); - let net = GrandpaTestNet::new(TestApi::new(voters), 2, 0, runtime.handle().clone()); + let net = GrandpaTestNet::new(TestApi::new(voters), 2, 0); let net = Arc::new(Mutex::new(net)); let mut finality_notifications = Vec::new(); @@ -1265,11 +1235,11 @@ fn voter_catches_up_to_latest_round_when_behind() { let voter = voter(Some(keystore), peer_id, link, net.clone()); - runtime.spawn(voter); + tokio::spawn(voter); } net.lock().peer(0).push_blocks(50, false); - runtime.block_on(net.lock().wait_until_sync()); + net.lock().run_until_sync().await; // wait for them to finalize block 50. since they'll vote on 3/4 of the // unfinalized chain it will take at least 4 rounds to do it. @@ -1279,7 +1249,6 @@ fn voter_catches_up_to_latest_round_when_behind() { // able to catch up to the latest round let test = { let net = net.clone(); - let runtime = runtime.handle().clone(); wait_for_finality.then(move |_| { net.lock().add_authority_peer(); @@ -1290,7 +1259,7 @@ fn voter_catches_up_to_latest_round_when_behind() { link.take().expect("link initialized at startup; qed") }; let set_state = link.persistent_data.set_state.clone(); - runtime.spawn(voter(None, 2, link, net.clone())); + tokio::spawn(voter(None, 2, link, net.clone())); let start_time = std::time::Instant::now(); let timeout = Duration::from_secs(5 * 60); @@ -1315,7 +1284,7 @@ fn voter_catches_up_to_latest_round_when_behind() { net.lock().poll(cx); Poll::<()>::Pending }); - runtime.block_on(future::select(test, drive_to_completion)); + future::select(test, drive_to_completion).await; } type TestEnvironment = Environment< @@ -1370,15 +1339,14 @@ where } } -#[test] -fn grandpa_environment_respects_voting_rules() { +#[tokio::test] +async fn grandpa_environment_respects_voting_rules() { use finality_grandpa::voter::Environment; let peers = &[Ed25519Keyring::Alice]; let voters = make_ids(peers); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0); let peer = net.peer(0); let network_service = peer.network_service().clone(); let link = peer.data.lock().take().unwrap(); @@ -1408,8 +1376,9 @@ fn grandpa_environment_respects_voting_rules() { // the unrestricted environment should just return the best block assert_eq!( - runtime - .block_on(unrestricted_env.best_chain_containing(peer.client().info().finalized_hash)) + unrestricted_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1419,8 +1388,9 @@ fn grandpa_environment_respects_voting_rules() { // both the other environments should return block 16, which is 3/4 of the // way in the unfinalized chain assert_eq!( - runtime - .block_on(three_quarters_env.best_chain_containing(peer.client().info().finalized_hash)) + three_quarters_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1428,8 +1398,9 @@ fn grandpa_environment_respects_voting_rules() { ); assert_eq!( - runtime - .block_on(default_env.best_chain_containing(peer.client().info().finalized_hash)) + default_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1446,8 +1417,9 @@ fn grandpa_environment_respects_voting_rules() { // the 3/4 environment should propose block 21 for voting assert_eq!( - runtime - .block_on(three_quarters_env.best_chain_containing(peer.client().info().finalized_hash)) + three_quarters_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1457,8 +1429,9 @@ fn grandpa_environment_respects_voting_rules() { // while the default environment will always still make sure we don't vote // on the best block (2 behind) assert_eq!( - runtime - .block_on(default_env.best_chain_containing(peer.client().info().finalized_hash)) + default_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1477,8 +1450,9 @@ fn grandpa_environment_respects_voting_rules() { // best block, there's a hard rule that we can't cast any votes lower than // the given base (#21). assert_eq!( - runtime - .block_on(default_env.best_chain_containing(peer.client().info().finalized_hash)) + default_env + .best_chain_containing(peer.client().info().finalized_hash) + .await .unwrap() .unwrap() .1, @@ -1486,15 +1460,14 @@ fn grandpa_environment_respects_voting_rules() { ); } -#[test] -fn grandpa_environment_never_overwrites_round_voter_state() { +#[tokio::test] +async fn grandpa_environment_never_overwrites_round_voter_state() { use finality_grandpa::voter::Environment; let peers = &[Ed25519Keyring::Alice]; let voters = make_ids(peers); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0); let peer = net.peer(0); let network_service = peer.network_service().clone(); let link = peer.data.lock().take().unwrap(); @@ -1549,16 +1522,15 @@ fn grandpa_environment_never_overwrites_round_voter_state() { assert_matches!(get_current_round(2).unwrap(), HasVoted::Yes(_, _)); } -#[test] -fn justification_with_equivocation() { +#[tokio::test] +async fn justification_with_equivocation() { use sp_application_crypto::Pair; // we have 100 authorities let pairs = (0..100).map(|n| AuthorityPair::from_seed(&[n; 32])).collect::>(); let voters = pairs.iter().map(AuthorityPair::public).map(|id| (id, 1)).collect::>(); let api = TestApi::new(voters.clone()); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(api.clone(), 1, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api.clone(), 1, 0); // we create a basic chain with 3 blocks (no forks) net.peer(0).push_blocks(3, false); @@ -1617,16 +1589,15 @@ fn justification_with_equivocation() { assert!(justification.verify(set_id, &voters).is_ok()); } -#[test] -fn imports_justification_for_regular_blocks_on_import() { +#[tokio::test] +async fn imports_justification_for_regular_blocks_on_import() { // NOTE: this is a regression test since initially we would only import // justifications for authority change blocks, and would discard any // existing justification otherwise. let peers = &[Ed25519Keyring::Alice]; let voters = make_ids(peers); let api = TestApi::new(voters); - let runtime = Runtime::new().unwrap(); - let mut net = GrandpaTestNet::new(api.clone(), 1, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(api.clone(), 1, 0); let client = net.peer(0).client().clone(); let (mut block_import, ..) = net.make_block_import(client.clone()); @@ -1675,7 +1646,7 @@ fn imports_justification_for_regular_blocks_on_import() { import.fork_choice = Some(ForkChoiceStrategy::LongestChain); assert_eq!( - runtime.block_on(block_import.import_block(import, HashMap::new())).unwrap(), + block_import.import_block(import, HashMap::new()).await.unwrap(), ImportResult::Imported(ImportedAux { needs_justification: false, clear_justification_requests: false, @@ -1689,17 +1660,15 @@ fn imports_justification_for_regular_blocks_on_import() { assert!(client.justifications(block_hash).unwrap().is_some()); } -#[test] -fn grandpa_environment_doesnt_send_equivocation_reports_for_itself() { +#[tokio::test] +async fn grandpa_environment_doesnt_send_equivocation_reports_for_itself() { use finality_grandpa::voter::Environment; let alice = Ed25519Keyring::Alice; let voters = make_ids(&[alice]); - let runtime = Runtime::new().unwrap(); - let environment = { - let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0, runtime.handle().clone()); + let mut net = GrandpaTestNet::new(TestApi::new(voters), 1, 0); let peer = net.peer(0); let network_service = peer.network_service().clone(); let link = peer.data.lock().take().unwrap(); @@ -1737,10 +1706,9 @@ fn grandpa_environment_doesnt_send_equivocation_reports_for_itself() { assert!(environment.report_equivocation(equivocation_proof).is_ok()); } -#[test] -fn revert_prunes_authority_changes() { +#[tokio::test] +async fn revert_prunes_authority_changes() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); let peers = &[Ed25519Keyring::Alice, Ed25519Keyring::Bob, Ed25519Keyring::Charlie]; @@ -1757,8 +1725,8 @@ fn revert_prunes_authority_changes() { let api = TestApi::new(make_ids(peers)); - let mut net = GrandpaTestNet::new(api, 3, 0, runtime.handle().clone()); - runtime.spawn(initialize_grandpa(&mut net, peers)); + let mut net = GrandpaTestNet::new(api, 3, 0); + tokio::spawn(initialize_grandpa(&mut net, peers)); let peer = net.peer(0); let client = peer.client().as_client(); diff --git a/client/network-gossip/src/bridge.rs b/client/network-gossip/src/bridge.rs index 462677f53c5fd..5d7fd96c13e57 100644 --- a/client/network-gossip/src/bridge.rs +++ b/client/network-gossip/src/bridge.rs @@ -489,7 +489,7 @@ mod tests { })) } - #[tokio::test(flavor = "multi_thread")] + #[tokio::test(flavor = "multi_thread", worker_threads = 2)] async fn keeps_multiple_subscribers_per_topic_updated_with_both_old_and_new_messages() { let topic = H256::default(); let protocol = ProtocolName::from("/my_protocol"); diff --git a/client/network/src/protocol/notifications/upgrade/notifications.rs b/client/network/src/protocol/notifications/upgrade/notifications.rs index 5d61e10727b66..5a4ec832dbea1 100644 --- a/client/network/src/protocol/notifications/upgrade/notifications.rs +++ b/client/network/src/protocol/notifications/upgrade/notifications.rs @@ -483,20 +483,15 @@ mod tests { use super::{NotificationsIn, NotificationsInOpen, NotificationsOut, NotificationsOutOpen}; use futures::{channel::oneshot, prelude::*}; use libp2p::core::upgrade; - use tokio::{ - net::{TcpListener, TcpStream}, - runtime::Runtime, - }; + use tokio::net::{TcpListener, TcpStream}; use tokio_util::compat::TokioAsyncReadCompatExt; - #[test] - fn basic_works() { + #[tokio::test] + async fn basic_works() { const PROTO_NAME: &str = "/test/proto/1"; let (listener_addr_tx, listener_addr_rx) = oneshot::channel(); - let runtime = Runtime::new().unwrap(); - - let client = runtime.spawn(async move { + let client = tokio::spawn(async move { let socket = TcpStream::connect(listener_addr_rx.await.unwrap()).await.unwrap(); let NotificationsOutOpen { handshake, mut substream, .. } = upgrade::apply_outbound( socket.compat(), @@ -510,38 +505,34 @@ mod tests { substream.send(b"test message".to_vec()).await.unwrap(); }); - runtime.block_on(async move { - let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); - listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); - let (socket, _) = listener.accept().await.unwrap(); - let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( - socket.compat(), - NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), - ) - .await - .unwrap(); + let (socket, _) = listener.accept().await.unwrap(); + let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( + socket.compat(), + NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), + ) + .await + .unwrap(); - assert_eq!(handshake, b"initial message"); - substream.send_handshake(&b"hello world"[..]); + assert_eq!(handshake, b"initial message"); + substream.send_handshake(&b"hello world"[..]); - let msg = substream.next().await.unwrap().unwrap(); - assert_eq!(msg.as_ref(), b"test message"); - }); + let msg = substream.next().await.unwrap().unwrap(); + assert_eq!(msg.as_ref(), b"test message"); - runtime.block_on(client).unwrap(); + client.await.unwrap(); } - #[test] - fn empty_handshake() { + #[tokio::test] + async fn empty_handshake() { // Check that everything still works when the handshake messages are empty. const PROTO_NAME: &str = "/test/proto/1"; let (listener_addr_tx, listener_addr_rx) = oneshot::channel(); - let runtime = Runtime::new().unwrap(); - - let client = runtime.spawn(async move { + let client = tokio::spawn(async move { let socket = TcpStream::connect(listener_addr_rx.await.unwrap()).await.unwrap(); let NotificationsOutOpen { handshake, mut substream, .. } = upgrade::apply_outbound( socket.compat(), @@ -555,36 +546,32 @@ mod tests { substream.send(Default::default()).await.unwrap(); }); - runtime.block_on(async move { - let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); - listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); - let (socket, _) = listener.accept().await.unwrap(); - let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( - socket.compat(), - NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), - ) - .await - .unwrap(); + let (socket, _) = listener.accept().await.unwrap(); + let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( + socket.compat(), + NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), + ) + .await + .unwrap(); - assert!(handshake.is_empty()); - substream.send_handshake(vec![]); + assert!(handshake.is_empty()); + substream.send_handshake(vec![]); - let msg = substream.next().await.unwrap().unwrap(); - assert!(msg.as_ref().is_empty()); - }); + let msg = substream.next().await.unwrap().unwrap(); + assert!(msg.as_ref().is_empty()); - runtime.block_on(client).unwrap(); + client.await.unwrap(); } - #[test] - fn refused() { + #[tokio::test] + async fn refused() { const PROTO_NAME: &str = "/test/proto/1"; let (listener_addr_tx, listener_addr_rx) = oneshot::channel(); - let runtime = Runtime::new().unwrap(); - - let client = runtime.spawn(async move { + let client = tokio::spawn(async move { let socket = TcpStream::connect(listener_addr_rx.await.unwrap()).await.unwrap(); let outcome = upgrade::apply_outbound( socket.compat(), @@ -599,35 +586,31 @@ mod tests { assert!(outcome.is_err()); }); - runtime.block_on(async move { - let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); - listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); - let (socket, _) = listener.accept().await.unwrap(); - let NotificationsInOpen { handshake, substream, .. } = upgrade::apply_inbound( - socket.compat(), - NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), - ) - .await - .unwrap(); + let (socket, _) = listener.accept().await.unwrap(); + let NotificationsInOpen { handshake, substream, .. } = upgrade::apply_inbound( + socket.compat(), + NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), + ) + .await + .unwrap(); - assert_eq!(handshake, b"hello"); + assert_eq!(handshake, b"hello"); - // We successfully upgrade to the protocol, but then close the substream. - drop(substream); - }); + // We successfully upgrade to the protocol, but then close the substream. + drop(substream); - runtime.block_on(client).unwrap(); + client.await.unwrap(); } - #[test] - fn large_initial_message_refused() { + #[tokio::test] + async fn large_initial_message_refused() { const PROTO_NAME: &str = "/test/proto/1"; let (listener_addr_tx, listener_addr_rx) = oneshot::channel(); - let runtime = Runtime::new().unwrap(); - - let client = runtime.spawn(async move { + let client = tokio::spawn(async move { let socket = TcpStream::connect(listener_addr_rx.await.unwrap()).await.unwrap(); let ret = upgrade::apply_outbound( socket.compat(), @@ -644,30 +627,26 @@ mod tests { assert!(ret.is_err()); }); - runtime.block_on(async move { - let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); - listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); - let (socket, _) = listener.accept().await.unwrap(); - let ret = upgrade::apply_inbound( - socket.compat(), - NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), - ) - .await; - assert!(ret.is_err()); - }); + let (socket, _) = listener.accept().await.unwrap(); + let ret = upgrade::apply_inbound( + socket.compat(), + NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), + ) + .await; + assert!(ret.is_err()); - runtime.block_on(client).unwrap(); + client.await.unwrap(); } - #[test] - fn large_handshake_refused() { + #[tokio::test] + async fn large_handshake_refused() { const PROTO_NAME: &str = "/test/proto/1"; let (listener_addr_tx, listener_addr_rx) = oneshot::channel(); - let runtime = Runtime::new().unwrap(); - - let client = runtime.spawn(async move { + let client = tokio::spawn(async move { let socket = TcpStream::connect(listener_addr_rx.await.unwrap()).await.unwrap(); let ret = upgrade::apply_outbound( socket.compat(), @@ -678,24 +657,22 @@ mod tests { assert!(ret.is_err()); }); - runtime.block_on(async move { - let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); - listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); + let listener = TcpListener::bind("127.0.0.1:0").await.unwrap(); + listener_addr_tx.send(listener.local_addr().unwrap()).unwrap(); - let (socket, _) = listener.accept().await.unwrap(); - let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( - socket.compat(), - NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), - ) - .await - .unwrap(); - assert_eq!(handshake, b"initial message"); + let (socket, _) = listener.accept().await.unwrap(); + let NotificationsInOpen { handshake, mut substream, .. } = upgrade::apply_inbound( + socket.compat(), + NotificationsIn::new(PROTO_NAME, Vec::new(), 1024 * 1024), + ) + .await + .unwrap(); + assert_eq!(handshake, b"initial message"); - // We check that a handshake that is too large gets refused. - substream.send_handshake((0..32768).map(|_| 0).collect::>()); - let _ = substream.next().await; - }); + // We check that a handshake that is too large gets refused. + substream.send_handshake((0..32768).map(|_| 0).collect::>()); + let _ = substream.next().await; - runtime.block_on(client).unwrap(); + client.await.unwrap(); } } diff --git a/client/network/src/service/tests/chain_sync.rs b/client/network/src/service/tests/chain_sync.rs index 0f47b64c352f2..ce4430dd87db9 100644 --- a/client/network/src/service/tests/chain_sync.rs +++ b/client/network/src/service/tests/chain_sync.rs @@ -44,7 +44,6 @@ use std::{ time::Duration, }; use substrate_test_runtime_client::{TestClientBuilder, TestClientBuilderExt as _}; -use tokio::runtime::Handle; fn set_default_expecations_no_peers( chain_sync: &mut MockChainSync, @@ -72,7 +71,7 @@ async fn normal_network_poll_no_peers() { let chain_sync_service = Box::new(MockChainSyncInterface::::new()); - let mut network = TestNetworkBuilder::new(Handle::current()) + let mut network = TestNetworkBuilder::new() .with_chain_sync((chain_sync, chain_sync_service)) .build(); @@ -104,7 +103,7 @@ async fn request_justification() { let mut chain_sync = MockChainSync::::new(); set_default_expecations_no_peers(&mut chain_sync); - let mut network = TestNetworkBuilder::new(Handle::current()) + let mut network = TestNetworkBuilder::new() .with_chain_sync((Box::new(chain_sync), chain_sync_service)) .build(); @@ -135,7 +134,7 @@ async fn clear_justification_requests() { Box::new(MockChainSync::::new()); set_default_expecations_no_peers(&mut chain_sync); - let mut network = TestNetworkBuilder::new(Handle::current()) + let mut network = TestNetworkBuilder::new() .with_chain_sync((chain_sync, chain_sync_service)) .build(); @@ -174,7 +173,7 @@ async fn set_sync_fork_request() { .once() .returning(|_, _, _| ()); - let mut network = TestNetworkBuilder::new(Handle::current()) + let mut network = TestNetworkBuilder::new() .with_chain_sync((chain_sync, Box::new(chain_sync_service))) .build(); @@ -218,7 +217,7 @@ async fn on_block_finalized() { .returning(|_, _| ()); set_default_expecations_no_peers(&mut chain_sync); - let mut network = TestNetworkBuilder::new(Handle::current()) + let mut network = TestNetworkBuilder::new() .with_client(client) .with_chain_sync((chain_sync, chain_sync_service)) .build(); @@ -316,7 +315,7 @@ async fn invalid_justification_imported() { let justification_info = Arc::new(RwLock::new(None)); let listen_addr = config::build_multiaddr![Memory(rand::random::())]; - let (service1, mut event_stream1) = TestNetworkBuilder::new(Handle::current()) + let (service1, mut event_stream1) = TestNetworkBuilder::new() .with_import_queue(Box::new(DummyImportQueue( justification_info.clone(), DummyImportQueueHandle {}, @@ -325,7 +324,7 @@ async fn invalid_justification_imported() { .build() .start_network(); - let (service2, mut event_stream2) = TestNetworkBuilder::new(Handle::current()) + let (service2, mut event_stream2) = TestNetworkBuilder::new() .with_set_config(SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, @@ -393,7 +392,7 @@ async fn disconnect_peer_using_chain_sync_handle() { ) .unwrap(); - let (node1, mut event_stream1) = TestNetworkBuilder::new(Handle::current()) + let (node1, mut event_stream1) = TestNetworkBuilder::new() .with_listen_addresses(vec![listen_addr.clone()]) .with_chain_sync((Box::new(chain_sync), Box::new(chain_sync_service))) .with_chain_sync_network((chain_sync_network_provider, chain_sync_network_handle)) @@ -401,7 +400,7 @@ async fn disconnect_peer_using_chain_sync_handle() { .build() .start_network(); - let (node2, mut event_stream2) = TestNetworkBuilder::new(Handle::current()) + let (node2, mut event_stream2) = TestNetworkBuilder::new() .with_set_config(SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, diff --git a/client/network/src/service/tests/mod.rs b/client/network/src/service/tests/mod.rs index fa1486a791213..9c97a7f73837d 100644 --- a/client/network/src/service/tests/mod.rs +++ b/client/network/src/service/tests/mod.rs @@ -44,7 +44,6 @@ use substrate_test_runtime_client::{ runtime::{Block as TestBlock, Hash as TestHash}, TestClient, TestClientBuilder, TestClientBuilderExt as _, }; -use tokio::runtime::Handle; #[cfg(test)] mod chain_sync; @@ -59,12 +58,11 @@ const PROTOCOL_NAME: &str = "/foo"; struct TestNetwork { network: TestNetworkWorker, - rt_handle: Handle, } impl TestNetwork { - pub fn new(network: TestNetworkWorker, rt_handle: Handle) -> Self { - Self { network, rt_handle } + pub fn new(network: TestNetworkWorker) -> Self { + Self { network } } pub fn service(&self) -> &Arc { @@ -82,7 +80,7 @@ impl TestNetwork { let service = worker.service().clone(); let event_stream = service.event_stream("test"); - self.rt_handle.spawn(async move { + tokio::spawn(async move { futures::pin_mut!(worker); let _ = worker.await; }); @@ -100,11 +98,10 @@ struct TestNetworkBuilder { chain_sync: Option<(Box>, Box>)>, chain_sync_network: Option<(NetworkServiceProvider, NetworkServiceHandle)>, config: Option, - rt_handle: Handle, } impl TestNetworkBuilder { - pub fn new(rt_handle: Handle) -> Self { + pub fn new() -> Self { Self { import_queue: None, link: None, @@ -114,7 +111,6 @@ impl TestNetworkBuilder { chain_sync: None, chain_sync_network: None, config: None, - rt_handle, } } @@ -229,21 +225,21 @@ impl TestNetworkBuilder { let block_request_protocol_config = { let (handler, protocol_config) = BlockRequestHandler::new(&protocol_id, None, client.clone(), 50); - self.rt_handle.spawn(handler.run().boxed()); + tokio::spawn(handler.run().boxed()); protocol_config }; let state_request_protocol_config = { let (handler, protocol_config) = StateRequestHandler::new(&protocol_id, None, client.clone(), 50); - self.rt_handle.spawn(handler.run().boxed()); + tokio::spawn(handler.run().boxed()); protocol_config }; let light_client_request_protocol_config = { let (handler, protocol_config) = LightClientRequestHandler::new(&protocol_id, None, client.clone()); - self.rt_handle.spawn(handler.run().boxed()); + tokio::spawn(handler.run().boxed()); protocol_config }; @@ -310,11 +306,6 @@ impl TestNetworkBuilder { .link .unwrap_or(Box::new(sc_network_sync::service::mock::MockChainSyncInterface::new())); - let handle = self.rt_handle.clone(); - let executor = move |f| { - handle.spawn(f); - }; - let worker = NetworkWorker::< substrate_test_runtime_client::runtime::Block, substrate_test_runtime_client::runtime::Hash, @@ -322,7 +313,9 @@ impl TestNetworkBuilder { >::new(config::Params { block_announce_config, role: config::Role::Full, - executor: Box::new(executor), + executor: Box::new(|f| { + tokio::spawn(f); + }), network_config, chain: client.clone(), protocol_id, @@ -340,10 +333,10 @@ impl TestNetworkBuilder { .unwrap(); let service = worker.service().clone(); - self.rt_handle.spawn(async move { + tokio::spawn(async move { let _ = chain_sync_network_provider.run(service).await; }); - self.rt_handle.spawn(async move { + tokio::spawn(async move { loop { futures::future::poll_fn(|cx| { import_queue.poll_actions(cx, &mut *link); @@ -354,6 +347,6 @@ impl TestNetworkBuilder { } }); - TestNetwork::new(worker, self.rt_handle) + TestNetwork::new(worker) } } diff --git a/client/network/src/service/tests/service.rs b/client/network/src/service/tests/service.rs index aa74e595fff7e..bdf30096356f2 100644 --- a/client/network/src/service/tests/service.rs +++ b/client/network/src/service/tests/service.rs @@ -26,7 +26,6 @@ use sc_network_common::{ service::{NetworkNotification, NetworkPeers, NetworkStateInfo}, }; use std::{sync::Arc, time::Duration}; -use tokio::runtime::Handle; type TestNetworkService = NetworkService< substrate_test_runtime_client::runtime::Block, @@ -38,9 +37,7 @@ const PROTOCOL_NAME: &str = "/foo"; /// Builds two nodes and their associated events stream. /// The nodes are connected together and have the `PROTOCOL_NAME` protocol registered. -fn build_nodes_one_proto( - rt_handle: &Handle, -) -> ( +fn build_nodes_one_proto() -> ( Arc, impl Stream, Arc, @@ -48,12 +45,12 @@ fn build_nodes_one_proto( ) { let listen_addr = config::build_multiaddr![Memory(rand::random::())]; - let (node1, events_stream1) = TestNetworkBuilder::new(rt_handle.clone()) + let (node1, events_stream1) = TestNetworkBuilder::new() .with_listen_addresses(vec![listen_addr.clone()]) .build() .start_network(); - let (node2, events_stream2) = TestNetworkBuilder::new(rt_handle.clone()) + let (node2, events_stream2) = TestNetworkBuilder::new() .with_set_config(SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, @@ -67,15 +64,12 @@ fn build_nodes_one_proto( (node1, events_stream1, node2, events_stream2) } -#[test] -fn notifications_state_consistent() { +#[tokio::test] +async fn notifications_state_consistent() { // Runs two nodes and ensures that events are propagated out of the API in a consistent // correct order, which means no notification received on a closed substream. - let runtime = tokio::runtime::Runtime::new().unwrap(); - - let (node1, mut events_stream1, node2, mut events_stream2) = - build_nodes_one_proto(runtime.handle()); + let (node1, mut events_stream1, node2, mut events_stream2) = build_nodes_one_proto(); // Write some initial notifications that shouldn't get through. for _ in 0..(rand::random::() % 5) { @@ -93,140 +87,130 @@ fn notifications_state_consistent() { ); } - runtime.block_on(async move { - // True if we have an active substream from node1 to node2. - let mut node1_to_node2_open = false; - // True if we have an active substream from node2 to node1. - let mut node2_to_node1_open = false; - // We stop the test after a certain number of iterations. - let mut iterations = 0; - // Safe guard because we don't want the test to pass if no substream has been open. - let mut something_happened = false; + // True if we have an active substream from node1 to node2. + let mut node1_to_node2_open = false; + // True if we have an active substream from node2 to node1. + let mut node2_to_node1_open = false; + // We stop the test after a certain number of iterations. + let mut iterations = 0; + // Safe guard because we don't want the test to pass if no substream has been open. + let mut something_happened = false; + + loop { + iterations += 1; + if iterations >= 1_000 { + assert!(something_happened); + break + } - loop { - iterations += 1; - if iterations >= 1_000 { - assert!(something_happened); - break - } + // Start by sending a notification from node1 to node2 and vice-versa. Part of the + // test consists in ensuring that notifications get ignored if the stream isn't open. + if rand::random::() % 5 >= 3 { + node1.write_notification( + node2.local_peer_id(), + PROTOCOL_NAME.into(), + b"hello world".to_vec(), + ); + } + if rand::random::() % 5 >= 3 { + node2.write_notification( + node1.local_peer_id(), + PROTOCOL_NAME.into(), + b"hello world".to_vec(), + ); + } - // Start by sending a notification from node1 to node2 and vice-versa. Part of the - // test consists in ensuring that notifications get ignored if the stream isn't open. - if rand::random::() % 5 >= 3 { - node1.write_notification( - node2.local_peer_id(), - PROTOCOL_NAME.into(), - b"hello world".to_vec(), - ); - } - if rand::random::() % 5 >= 3 { - node2.write_notification( - node1.local_peer_id(), - PROTOCOL_NAME.into(), - b"hello world".to_vec(), - ); - } + // Also randomly disconnect the two nodes from time to time. + if rand::random::() % 20 == 0 { + node1.disconnect_peer(node2.local_peer_id(), PROTOCOL_NAME.into()); + } + if rand::random::() % 20 == 0 { + node2.disconnect_peer(node1.local_peer_id(), PROTOCOL_NAME.into()); + } - // Also randomly disconnect the two nodes from time to time. - if rand::random::() % 20 == 0 { - node1.disconnect_peer(node2.local_peer_id(), PROTOCOL_NAME.into()); + // Grab next event from either `events_stream1` or `events_stream2`. + let next_event = { + let next1 = events_stream1.next(); + let next2 = events_stream2.next(); + // We also await on a small timer, otherwise it is possible for the test to wait + // forever while nothing at all happens on the network. + let continue_test = futures_timer::Delay::new(Duration::from_millis(20)); + match future::select(future::select(next1, next2), continue_test).await { + future::Either::Left((future::Either::Left((Some(ev), _)), _)) => + future::Either::Left(ev), + future::Either::Left((future::Either::Right((Some(ev), _)), _)) => + future::Either::Right(ev), + future::Either::Right(_) => continue, + _ => break, } - if rand::random::() % 20 == 0 { - node2.disconnect_peer(node1.local_peer_id(), PROTOCOL_NAME.into()); - } - - // Grab next event from either `events_stream1` or `events_stream2`. - let next_event = { - let next1 = events_stream1.next(); - let next2 = events_stream2.next(); - // We also await on a small timer, otherwise it is possible for the test to wait - // forever while nothing at all happens on the network. - let continue_test = futures_timer::Delay::new(Duration::from_millis(20)); - match future::select(future::select(next1, next2), continue_test).await { - future::Either::Left((future::Either::Left((Some(ev), _)), _)) => - future::Either::Left(ev), - future::Either::Left((future::Either::Right((Some(ev), _)), _)) => - future::Either::Right(ev), - future::Either::Right(_) => continue, - _ => break, - } - }; - - match next_event { - future::Either::Left(Event::NotificationStreamOpened { - remote, protocol, .. - }) => - if protocol == PROTOCOL_NAME.into() { - something_happened = true; - assert!(!node1_to_node2_open); - node1_to_node2_open = true; - assert_eq!(remote, node2.local_peer_id()); - }, - future::Either::Right(Event::NotificationStreamOpened { - remote, protocol, .. - }) => - if protocol == PROTOCOL_NAME.into() { - something_happened = true; - assert!(!node2_to_node1_open); - node2_to_node1_open = true; - assert_eq!(remote, node1.local_peer_id()); - }, - future::Either::Left(Event::NotificationStreamClosed { - remote, protocol, .. - }) => - if protocol == PROTOCOL_NAME.into() { - assert!(node1_to_node2_open); - node1_to_node2_open = false; - assert_eq!(remote, node2.local_peer_id()); - }, - future::Either::Right(Event::NotificationStreamClosed { - remote, protocol, .. - }) => - if protocol == PROTOCOL_NAME.into() { - assert!(node2_to_node1_open); - node2_to_node1_open = false; - assert_eq!(remote, node1.local_peer_id()); - }, - future::Either::Left(Event::NotificationsReceived { remote, .. }) => { + }; + + match next_event { + future::Either::Left(Event::NotificationStreamOpened { remote, protocol, .. }) => + if protocol == PROTOCOL_NAME.into() { + something_happened = true; + assert!(!node1_to_node2_open); + node1_to_node2_open = true; + assert_eq!(remote, node2.local_peer_id()); + }, + future::Either::Right(Event::NotificationStreamOpened { remote, protocol, .. }) => + if protocol == PROTOCOL_NAME.into() { + something_happened = true; + assert!(!node2_to_node1_open); + node2_to_node1_open = true; + assert_eq!(remote, node1.local_peer_id()); + }, + future::Either::Left(Event::NotificationStreamClosed { remote, protocol, .. }) => + if protocol == PROTOCOL_NAME.into() { assert!(node1_to_node2_open); + node1_to_node2_open = false; assert_eq!(remote, node2.local_peer_id()); - if rand::random::() % 5 >= 4 { - node1.write_notification( - node2.local_peer_id(), - PROTOCOL_NAME.into(), - b"hello world".to_vec(), - ); - } }, - future::Either::Right(Event::NotificationsReceived { remote, .. }) => { + future::Either::Right(Event::NotificationStreamClosed { remote, protocol, .. }) => + if protocol == PROTOCOL_NAME.into() { assert!(node2_to_node1_open); + node2_to_node1_open = false; assert_eq!(remote, node1.local_peer_id()); - if rand::random::() % 5 >= 4 { - node2.write_notification( - node1.local_peer_id(), - PROTOCOL_NAME.into(), - b"hello world".to_vec(), - ); - } }, + future::Either::Left(Event::NotificationsReceived { remote, .. }) => { + assert!(node1_to_node2_open); + assert_eq!(remote, node2.local_peer_id()); + if rand::random::() % 5 >= 4 { + node1.write_notification( + node2.local_peer_id(), + PROTOCOL_NAME.into(), + b"hello world".to_vec(), + ); + } + }, + future::Either::Right(Event::NotificationsReceived { remote, .. }) => { + assert!(node2_to_node1_open); + assert_eq!(remote, node1.local_peer_id()); + if rand::random::() % 5 >= 4 { + node2.write_notification( + node1.local_peer_id(), + PROTOCOL_NAME.into(), + b"hello world".to_vec(), + ); + } + }, - // Add new events here. - future::Either::Left(Event::SyncConnected { .. }) => {}, - future::Either::Right(Event::SyncConnected { .. }) => {}, - future::Either::Left(Event::SyncDisconnected { .. }) => {}, - future::Either::Right(Event::SyncDisconnected { .. }) => {}, - future::Either::Left(Event::Dht(_)) => {}, - future::Either::Right(Event::Dht(_)) => {}, - }; - } - }); + // Add new events here. + future::Either::Left(Event::SyncConnected { .. }) => {}, + future::Either::Right(Event::SyncConnected { .. }) => {}, + future::Either::Left(Event::SyncDisconnected { .. }) => {}, + future::Either::Right(Event::SyncDisconnected { .. }) => {}, + future::Either::Left(Event::Dht(_)) => {}, + future::Either::Right(Event::Dht(_)) => {}, + }; + } } #[tokio::test] async fn lots_of_incoming_peers_works() { let listen_addr = config::build_multiaddr![Memory(rand::random::())]; - let (main_node, _) = TestNetworkBuilder::new(Handle::current()) + let (main_node, _) = TestNetworkBuilder::new() .with_listen_addresses(vec![listen_addr.clone()]) .with_set_config(SetConfig { in_peers: u32::MAX, ..Default::default() }) .build() @@ -239,7 +223,7 @@ async fn lots_of_incoming_peers_works() { let mut background_tasks_to_wait = Vec::new(); for _ in 0..32 { - let (_dialing_node, event_stream) = TestNetworkBuilder::new(Handle::current()) + let (_dialing_node, event_stream) = TestNetworkBuilder::new() .with_set_config(SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr.clone(), @@ -286,20 +270,17 @@ async fn lots_of_incoming_peers_works() { future::join_all(background_tasks_to_wait).await; } -#[test] -fn notifications_back_pressure() { +#[tokio::test] +async fn notifications_back_pressure() { // Node 1 floods node 2 with notifications. Random sleeps are done on node 2 to simulate the // node being busy. We make sure that all notifications are received. const TOTAL_NOTIFS: usize = 10_000; - let runtime = tokio::runtime::Runtime::new().unwrap(); - - let (node1, mut events_stream1, node2, mut events_stream2) = - build_nodes_one_proto(runtime.handle()); + let (node1, mut events_stream1, node2, mut events_stream2) = build_nodes_one_proto(); let node2_id = node2.local_peer_id(); - let receiver = runtime.spawn(async move { + let receiver = tokio::spawn(async move { let mut received_notifications = 0; while received_notifications < TOTAL_NOTIFS { @@ -320,40 +301,36 @@ fn notifications_back_pressure() { } }); - runtime.block_on(async move { - // Wait for the `NotificationStreamOpened`. - loop { - match events_stream1.next().await.unwrap() { - Event::NotificationStreamOpened { .. } => break, - _ => {}, - }; - } + // Wait for the `NotificationStreamOpened`. + loop { + match events_stream1.next().await.unwrap() { + Event::NotificationStreamOpened { .. } => break, + _ => {}, + }; + } - // Sending! - for num in 0..TOTAL_NOTIFS { - let notif = node1.notification_sender(node2_id, PROTOCOL_NAME.into()).unwrap(); - notif - .ready() - .await - .unwrap() - .send(format!("hello #{}", num).into_bytes()) - .unwrap(); - } + // Sending! + for num in 0..TOTAL_NOTIFS { + let notif = node1.notification_sender(node2_id, PROTOCOL_NAME.into()).unwrap(); + notif + .ready() + .await + .unwrap() + .send(format!("hello #{}", num).into_bytes()) + .unwrap(); + } - receiver.await.unwrap(); - }); + receiver.await.unwrap(); } -#[test] -fn fallback_name_working() { +#[tokio::test] +async fn fallback_name_working() { // Node 1 supports the protocols "new" and "old". Node 2 only supports "old". Checks whether // they can connect. const NEW_PROTOCOL_NAME: &str = "/new-shiny-protocol-that-isnt-PROTOCOL_NAME"; - let runtime = tokio::runtime::Runtime::new().unwrap(); - let listen_addr = config::build_multiaddr![Memory(rand::random::())]; - let (node1, mut events_stream1) = TestNetworkBuilder::new(runtime.handle().clone()) + let (node1, mut events_stream1) = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { extra_sets: vec![NonDefaultSetConfig { notifications_protocol: NEW_PROTOCOL_NAME.into(), @@ -369,7 +346,7 @@ fn fallback_name_working() { .build() .start_network(); - let (_, mut events_stream2) = TestNetworkBuilder::new(runtime.handle().clone()) + let (_, mut events_stream2) = TestNetworkBuilder::new() .with_set_config(SetConfig { reserved_nodes: vec![MultiaddrWithPeerId { multiaddr: listen_addr, @@ -380,7 +357,7 @@ fn fallback_name_working() { .build() .start_network(); - let receiver = runtime.spawn(async move { + let receiver = tokio::spawn(async move { // Wait for the `NotificationStreamOpened`. loop { match events_stream2.next().await.unwrap() { @@ -394,30 +371,27 @@ fn fallback_name_working() { } }); - runtime.block_on(async move { - // Wait for the `NotificationStreamOpened`. - loop { - match events_stream1.next().await.unwrap() { - Event::NotificationStreamOpened { protocol, negotiated_fallback, .. } - if protocol == NEW_PROTOCOL_NAME.into() => - { - assert_eq!(negotiated_fallback, Some(PROTOCOL_NAME.into())); - break - }, - _ => {}, - }; - } + // Wait for the `NotificationStreamOpened`. + loop { + match events_stream1.next().await.unwrap() { + Event::NotificationStreamOpened { protocol, negotiated_fallback, .. } + if protocol == NEW_PROTOCOL_NAME.into() => + { + assert_eq!(negotiated_fallback, Some(PROTOCOL_NAME.into())); + break + }, + _ => {}, + }; + } - receiver.await.unwrap(); - }); + receiver.await.unwrap(); } // Disconnect peer by calling `Protocol::disconnect_peer()` with the supplied block announcement // protocol name and verify that `SyncDisconnected` event is emitted #[tokio::test] async fn disconnect_sync_peer_using_block_announcement_protocol_name() { - let (node1, mut events_stream1, node2, mut events_stream2) = - build_nodes_one_proto(&Handle::current()); + let (node1, mut events_stream1, node2, mut events_stream2) = build_nodes_one_proto(); async fn wait_for_events(stream: &mut (impl Stream + std::marker::Unpin)) { let mut notif_received = false; @@ -454,7 +428,7 @@ async fn disconnect_sync_peer_using_block_announcement_protocol_name() { async fn ensure_listen_addresses_consistent_with_transport_memory() { let listen_addr = config::build_multiaddr![Ip4([127, 0, 0, 1]), Tcp(0_u16)]; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], transport: TransportConfig::MemoryOnly, @@ -474,7 +448,7 @@ async fn ensure_listen_addresses_consistent_with_transport_memory() { async fn ensure_listen_addresses_consistent_with_transport_not_memory() { let listen_addr = config::build_multiaddr![Memory(rand::random::())]; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], ..config::NetworkConfiguration::new( @@ -497,7 +471,7 @@ async fn ensure_boot_node_addresses_consistent_with_transport_memory() { peer_id: PeerId::random(), }; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], transport: TransportConfig::MemoryOnly, @@ -522,7 +496,7 @@ async fn ensure_boot_node_addresses_consistent_with_transport_not_memory() { peer_id: PeerId::random(), }; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], boot_nodes: vec![boot_node], @@ -546,7 +520,7 @@ async fn ensure_reserved_node_addresses_consistent_with_transport_memory() { peer_id: PeerId::random(), }; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], transport: TransportConfig::MemoryOnly, @@ -574,7 +548,7 @@ async fn ensure_reserved_node_addresses_consistent_with_transport_not_memory() { peer_id: PeerId::random(), }; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], default_peers_set: SetConfig { @@ -598,7 +572,7 @@ async fn ensure_public_addresses_consistent_with_transport_memory() { let listen_addr = config::build_multiaddr![Memory(rand::random::())]; let public_address = config::build_multiaddr![Ip4([127, 0, 0, 1]), Tcp(0_u16)]; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], transport: TransportConfig::MemoryOnly, @@ -620,7 +594,7 @@ async fn ensure_public_addresses_consistent_with_transport_not_memory() { let listen_addr = config::build_multiaddr![Ip4([127, 0, 0, 1]), Tcp(0_u16)]; let public_address = config::build_multiaddr![Memory(rand::random::())]; - let _ = TestNetworkBuilder::new(Handle::current()) + let _ = TestNetworkBuilder::new() .with_config(config::NetworkConfiguration { listen_addresses: vec![listen_addr.clone()], public_addresses: vec![public_address], diff --git a/client/network/test/src/lib.rs b/client/network/test/src/lib.rs index 173ca81653b1a..c3304df629e4e 100644 --- a/client/network/test/src/lib.rs +++ b/client/network/test/src/lib.rs @@ -708,16 +708,8 @@ pub struct FullPeerConfig { pub storage_chain: bool, } -/// Trait for text fixtures with tokio runtime. -pub trait WithRuntime { - /// Construct with runtime handle. - fn with_runtime(rt_handle: tokio::runtime::Handle) -> Self; - /// Get runtime handle. - fn rt_handle(&self) -> &tokio::runtime::Handle; -} - #[async_trait::async_trait] -pub trait TestNetFactory: WithRuntime + Sized +pub trait TestNetFactory: Default + Sized where >::Transaction: Send, { @@ -747,9 +739,9 @@ where ); /// Create new test network with this many peers. - fn new(rt_handle: tokio::runtime::Handle, n: usize) -> Self { + fn new(n: usize) -> Self { trace!(target: "test_network", "Creating test network"); - let mut net = Self::with_runtime(rt_handle); + let mut net = Self::default(); for i in 0..n { trace!(target: "test_network", "Adding peer {}", i); @@ -905,14 +897,11 @@ where ) .unwrap(); - let handle = self.rt_handle().clone(); - let executor = move |f| { - handle.spawn(f); - }; - let network = NetworkWorker::new(sc_network::config::Params { role: if config.is_authority { Role::Authority } else { Role::Full }, - executor: Box::new(executor), + executor: Box::new(|f| { + tokio::spawn(f); + }), network_config, chain: client.clone(), protocol_id, @@ -934,10 +923,10 @@ where trace!(target: "test_network", "Peer identifier: {}", network.service().local_peer_id()); let service = network.service().clone(); - self.rt_handle().spawn(async move { + tokio::spawn(async move { chain_sync_network_provider.run(service).await; }); - self.rt_handle().spawn(async move { + tokio::spawn(async move { import_queue.run(Box::new(chain_sync_service)).await; }); @@ -968,7 +957,7 @@ where /// Used to spawn background tasks, e.g. the block request protocol handler. fn spawn_task(&self, f: BoxFuture<'static, ()>) { - self.rt_handle().spawn(f); + tokio::spawn(f); } /// Polls the testnet until all nodes are in sync. @@ -1027,11 +1016,11 @@ where Poll::Pending } - /// Wait until we are sync'ed. + /// Run the network until we are sync'ed. /// /// Calls `poll_until_sync` repeatedly. /// (If we've not synced within 10 mins then panic rather than hang.) - async fn wait_until_sync(&mut self) { + async fn run_until_sync(&mut self) { timeout( Duration::from_secs(10 * 60), futures::future::poll_fn::<(), _>(|cx| self.poll_until_sync(cx)), @@ -1040,17 +1029,17 @@ where .expect("sync didn't happen within 10 mins"); } - /// Wait until there are no pending packets. + /// Run the network until there are no pending packets. /// /// Calls `poll_until_idle` repeatedly with the runtime passed as parameter. - async fn wait_until_idle(&mut self) { + async fn run_until_idle(&mut self) { futures::future::poll_fn::<(), _>(|cx| self.poll_until_idle(cx)).await; } - /// Wait until all peers are connected to each other. + /// Run the network until all peers are connected to each other. /// /// Calls `poll_until_connected` repeatedly with the runtime passed as parameter. - async fn wait_until_connected(&mut self) { + async fn run_until_connected(&mut self) { futures::future::poll_fn::<(), _>(|cx| self.poll_until_connected(cx)).await; } @@ -1082,20 +1071,11 @@ where } } +#[derive(Default)] pub struct TestNet { - rt_handle: tokio::runtime::Handle, peers: Vec>, } -impl WithRuntime for TestNet { - fn with_runtime(rt_handle: tokio::runtime::Handle) -> Self { - TestNet { rt_handle, peers: Vec::new() } - } - fn rt_handle(&self) -> &tokio::runtime::Handle { - &self.rt_handle - } -} - impl TestNetFactory for TestNet { type Verifier = PassThroughVerifier; type PeerData = (); @@ -1150,16 +1130,9 @@ impl JustificationImport for ForceFinalized { .map_err(|_| ConsensusError::InvalidJustification) } } -pub struct JustificationTestNet(TestNet); -impl WithRuntime for JustificationTestNet { - fn with_runtime(rt_handle: tokio::runtime::Handle) -> Self { - JustificationTestNet(TestNet::with_runtime(rt_handle)) - } - fn rt_handle(&self) -> &tokio::runtime::Handle { - &self.0.rt_handle() - } -} +#[derive(Default)] +pub struct JustificationTestNet(TestNet); impl TestNetFactory for JustificationTestNet { type Verifier = PassThroughVerifier; diff --git a/client/network/test/src/sync.rs b/client/network/test/src/sync.rs index efe0e0577c11e..9a9b5782dc2a7 100644 --- a/client/network/test/src/sync.rs +++ b/client/network/test/src/sync.rs @@ -21,12 +21,10 @@ use futures::Future; use sp_consensus::{block_validation::Validation, BlockOrigin}; use sp_runtime::Justifications; use substrate_test_runtime::Header; -use tokio::runtime::Runtime; -fn test_ancestor_search_when_common_is(n: usize) { +async fn test_ancestor_search_when_common_is(n: usize) { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(n, false); net.peer(1).push_blocks(n, false); @@ -36,18 +34,17 @@ fn test_ancestor_search_when_common_is(n: usize) { net.peer(1).push_blocks(100, false); net.peer(2).push_blocks(100, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn sync_peers_works() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_peers_works() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); for peer in 0..3 { if net.peer(peer).num_peers() != 2 { @@ -55,14 +52,14 @@ fn sync_peers_works() { } } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn sync_cycle_from_offline_to_syncing_to_offline() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_cycle_from_offline_to_syncing_to_offline() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); for peer in 0..3 { // Offline, and not major syncing. assert!(net.peer(peer).is_offline()); @@ -73,7 +70,7 @@ fn sync_cycle_from_offline_to_syncing_to_offline() { net.peer(2).push_blocks(100, false); // Block until all nodes are online and nodes 0 and 1 and major syncing. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); for peer in 0..3 { // Online @@ -88,10 +85,11 @@ fn sync_cycle_from_offline_to_syncing_to_offline() { } } Poll::Ready(()) - })); + }) + .await; // Block until all nodes are done syncing. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); for peer in 0..3 { if net.peer(peer).is_major_syncing() { @@ -99,26 +97,27 @@ fn sync_cycle_from_offline_to_syncing_to_offline() { } } Poll::Ready(()) - })); + }) + .await; // Now drop nodes 1 and 2, and check that node 0 is offline. net.peers.remove(2); net.peers.remove(1); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if !net.peer(0).is_offline() { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; } -#[test] -fn syncing_node_not_major_syncing_when_disconnected() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncing_node_not_major_syncing_when_disconnected() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); // Generate blocks. net.peer(2).push_blocks(100, false); @@ -127,136 +126,132 @@ fn syncing_node_not_major_syncing_when_disconnected() { assert!(!net.peer(1).is_major_syncing()); // Check that we switch to major syncing. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if !net.peer(1).is_major_syncing() { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; // Destroy two nodes, and check that we switch to non-major syncing. net.peers.remove(2); net.peers.remove(0); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).is_major_syncing() { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; } -#[test] -fn sync_from_two_peers_works() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_from_two_peers_works() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(1).push_blocks(100, false); net.peer(2).push_blocks(100, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); assert!(!net.peer(0).is_major_syncing()); } -#[test] -fn sync_from_two_peers_with_ancestry_search_works() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_from_two_peers_with_ancestry_search_works() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(10, true); net.peer(1).push_blocks(100, false); net.peer(2).push_blocks(100, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn ancestry_search_works_when_backoff_is_one() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn ancestry_search_works_when_backoff_is_one() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(1, false); net.peer(1).push_blocks(2, false); net.peer(2).push_blocks(2, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn ancestry_search_works_when_ancestor_is_genesis() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn ancestry_search_works_when_ancestor_is_genesis() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(13, true); net.peer(1).push_blocks(100, false); net.peer(2).push_blocks(100, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn ancestry_search_works_when_common_is_one() { - test_ancestor_search_when_common_is(1); +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn ancestry_search_works_when_common_is_one() { + test_ancestor_search_when_common_is(1).await; } -#[test] -fn ancestry_search_works_when_common_is_two() { - test_ancestor_search_when_common_is(2); +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn ancestry_search_works_when_common_is_two() { + test_ancestor_search_when_common_is(2).await; } -#[test] -fn ancestry_search_works_when_common_is_hundred() { - test_ancestor_search_when_common_is(100); +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn ancestry_search_works_when_common_is_hundred() { + test_ancestor_search_when_common_is(100).await; } -#[test] -fn sync_long_chain_works() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_long_chain_works() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(1).push_blocks(500, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn sync_no_common_longer_chain_fails() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_no_common_longer_chain_fails() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(20, true); net.peer(1).push_blocks(20, false); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).is_major_syncing() { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; let peer1 = &net.peers()[1]; assert!(!net.peers()[0].blockchain_canon_equals(peer1)); } -#[test] -fn sync_justifications() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_justifications() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = JustificationTestNet::new(runtime.handle().clone(), 3); + let mut net = JustificationTestNet::new(3); net.peer(0).push_blocks(20, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let backend = net.peer(0).client().as_backend(); let hashof10 = backend.blockchain().expect_block_hash_from_id(&BlockId::Number(10)).unwrap(); @@ -282,7 +277,7 @@ fn sync_justifications() { net.peer(1).request_justification(&hashof15, 15); net.peer(1).request_justification(&hashof20, 20); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); for hash in [hashof10, hashof15, hashof20] { @@ -299,14 +294,14 @@ fn sync_justifications() { } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn sync_justifications_across_forks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_justifications_across_forks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = JustificationTestNet::new(runtime.handle().clone(), 3); + let mut net = JustificationTestNet::new(3); // we push 5 blocks net.peer(0).push_blocks(5, false); // and then two forks 5 and 6 blocks long @@ -315,7 +310,7 @@ fn sync_justifications_across_forks() { // peer 1 will only see the longer fork. but we'll request justifications // for both and finalize the small fork instead. - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let just = (*b"FRNK", Vec::new()); net.peer(0).client().finalize_block(f1_best, Some(just), true).unwrap(); @@ -323,7 +318,7 @@ fn sync_justifications_across_forks() { net.peer(1).request_justification(&f1_best, 10); net.peer(1).request_justification(&f2_best, 11); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).client().justifications(f1_best).unwrap() == @@ -335,14 +330,14 @@ fn sync_justifications_across_forks() { } else { Poll::Pending } - })); + }) + .await; } -#[test] -fn sync_after_fork_works() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_after_fork_works() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); net.peer(0).push_blocks(30, false); net.peer(1).push_blocks(30, false); net.peer(2).push_blocks(30, false); @@ -355,25 +350,24 @@ fn sync_after_fork_works() { net.peer(2).push_blocks(1, false); // peer 1 has the best chain - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let peer1 = &net.peers()[1]; assert!(net.peers()[0].blockchain_canon_equals(peer1)); (net.peers()[1].blockchain_canon_equals(peer1)); (net.peers()[2].blockchain_canon_equals(peer1)); } -#[test] -fn syncs_all_forks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_all_forks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 4); + let mut net = TestNet::new(4); net.peer(0).push_blocks(2, false); net.peer(1).push_blocks(2, false); let b1 = net.peer(0).push_blocks(2, true); let b2 = net.peer(1).push_blocks(4, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; // Check that all peers have all of the branches. assert!(net.peer(0).has_block(b1)); assert!(net.peer(0).has_block(b2)); @@ -381,16 +375,15 @@ fn syncs_all_forks() { assert!(net.peer(1).has_block(b2)); } -#[test] -fn own_blocks_are_announced() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn own_blocks_are_announced() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); - runtime.block_on(net.wait_until_sync()); // connect'em + let mut net = TestNet::new(3); + net.run_until_sync().await; // connect'em net.peer(0) .generate_blocks(1, BlockOrigin::Own, |builder| builder.build().unwrap().block); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert_eq!(net.peer(0).client.info().best_number, 1); assert_eq!(net.peer(1).client.info().best_number, 1); @@ -399,11 +392,10 @@ fn own_blocks_are_announced() { (net.peers()[2].blockchain_canon_equals(peer0)); } -#[test] -fn can_sync_small_non_best_forks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn can_sync_small_non_best_forks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_blocks(30, false); net.peer(1).push_blocks(30, false); @@ -421,14 +413,15 @@ fn can_sync_small_non_best_forks() { assert!(net.peer(1).client().header(&BlockId::Hash(small_hash)).unwrap().is_none()); // poll until the two nodes connect, otherwise announcing the block will not work - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).num_peers() == 0 { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; // synchronization: 0 synced to longer chain and 1 didn't sync to small chain. @@ -441,7 +434,7 @@ fn can_sync_small_non_best_forks() { // after announcing, peer 1 downloads the block. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); assert!(net.peer(0).client().header(&BlockId::Hash(small_hash)).unwrap().is_some()); @@ -449,29 +442,30 @@ fn can_sync_small_non_best_forks() { return Poll::Pending } Poll::Ready(()) - })); - runtime.block_on(net.wait_until_sync()); + }) + .await; + net.run_until_sync().await; let another_fork = net.peer(0).push_blocks_at(BlockId::Number(35), 2, true); net.peer(0).announce_block(another_fork, None); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client().header(&BlockId::Hash(another_fork)).unwrap().is_none() { return Poll::Pending } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn can_sync_forks_ahead_of_the_best_chain() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn can_sync_forks_ahead_of_the_best_chain() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_blocks(1, false); net.peer(1).push_blocks(1, false); - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // Peer 0 is on 2-block fork which is announced with is_best=false let fork_hash = net.peer(0).generate_blocks_with_fork_choice( 2, @@ -486,21 +480,21 @@ fn can_sync_forks_ahead_of_the_best_chain() { assert_eq!(net.peer(1).client().info().best_number, 2); // after announcing, peer 1 downloads the block. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client().header(&BlockId::Hash(fork_hash)).unwrap().is_none() { return Poll::Pending } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn can_sync_explicit_forks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn can_sync_explicit_forks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_blocks(30, false); net.peer(1).push_blocks(30, false); @@ -519,14 +513,15 @@ fn can_sync_explicit_forks() { assert!(net.peer(1).client().header(&BlockId::Hash(small_hash)).unwrap().is_none()); // poll until the two nodes connect, otherwise announcing the block will not work - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).num_peers() == 0 || net.peer(1).num_peers() == 0 { Poll::Pending } else { Poll::Ready(()) } - })); + }) + .await; // synchronization: 0 synced to longer chain and 1 didn't sync to small chain. @@ -540,7 +535,7 @@ fn can_sync_explicit_forks() { net.peer(1).set_sync_fork_request(vec![first_peer_id], small_hash, small_number); // peer 1 downloads the block. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); assert!(net.peer(0).client().header(&BlockId::Hash(small_hash)).unwrap().is_some()); @@ -548,14 +543,14 @@ fn can_sync_explicit_forks() { return Poll::Pending } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn syncs_header_only_forks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_header_only_forks() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); net.add_full_peer_with_config(Default::default()); net.add_full_peer_with_config(FullPeerConfig { blocks_pruning: Some(3), ..Default::default() }); net.peer(0).push_blocks(2, false); @@ -567,19 +562,18 @@ fn syncs_header_only_forks() { // Peer 1 will sync the small fork even though common block state is missing while !net.peer(1).has_block(small_hash) { - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; } - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert_eq!(net.peer(0).client().info().best_hash, net.peer(1).client().info().best_hash); assert_ne!(small_hash, net.peer(0).client().info().best_hash); } -#[test] -fn does_not_sync_announced_old_best_block() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn does_not_sync_announced_old_best_block() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); let old_hash = net.peer(0).push_blocks(1, false); let old_hash_with_parent = net.peer(0).push_blocks(1, false); @@ -587,92 +581,93 @@ fn does_not_sync_announced_old_best_block() { net.peer(1).push_blocks(20, true); net.peer(0).announce_block(old_hash, None); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { // poll once to import announcement net.poll(cx); Poll::Ready(()) - })); + }) + .await; assert!(!net.peer(1).is_major_syncing()); net.peer(0).announce_block(old_hash_with_parent, None); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { // poll once to import announcement net.poll(cx); Poll::Ready(()) - })); + }) + .await; assert!(!net.peer(1).is_major_syncing()); } -#[test] -fn full_sync_requires_block_body() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn full_sync_requires_block_body() { // Check that we don't sync headers-only in full mode. sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_headers(1); // Wait for nodes to connect - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(0).num_peers() == 0 || net.peer(1).num_peers() == 0 { Poll::Pending } else { Poll::Ready(()) } - })); - runtime.block_on(net.wait_until_idle()); + }) + .await; + net.run_until_idle().await; assert_eq!(net.peer(1).client.info().best_number, 0); } -#[test] -fn imports_stale_once() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn imports_stale_once() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - fn import_with_announce(runtime: &Runtime, net: &mut TestNet, hash: H256) { + async fn import_with_announce(net: &mut TestNet, hash: H256) { // Announce twice net.peer(0).announce_block(hash, None); net.peer(0).announce_block(hash, None); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client().header(&BlockId::Hash(hash)).unwrap().is_some() { Poll::Ready(()) } else { Poll::Pending } - })); + }) + .await; } // given the network with 2 full nodes - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); // let them connect to each other - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; // check that NEW block is imported from announce message let new_hash = net.peer(0).push_blocks(1, false); - import_with_announce(&runtime, &mut net, new_hash); + import_with_announce(&mut net, new_hash).await; assert_eq!(net.peer(1).num_downloaded_blocks(), 1); // check that KNOWN STALE block is imported from announce message let known_stale_hash = net.peer(0).push_blocks_at(BlockId::Number(0), 1, true); - import_with_announce(&runtime, &mut net, known_stale_hash); + import_with_announce(&mut net, known_stale_hash).await; assert_eq!(net.peer(1).num_downloaded_blocks(), 2); } -#[test] -fn can_sync_to_peers_with_wrong_common_block() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn can_sync_to_peers_with_wrong_common_block() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_blocks(2, true); net.peer(1).push_blocks(2, true); let fork_hash = net.peer(0).push_blocks_at(BlockId::Number(0), 2, false); net.peer(1).push_blocks_at(BlockId::Number(0), 2, false); // wait for connection - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // both peers re-org to the same fork without notifying each other let just = Some((*b"FRNK", Vec::new())); @@ -680,7 +675,7 @@ fn can_sync_to_peers_with_wrong_common_block() { net.peer(1).client().finalize_block(fork_hash, just, true).unwrap(); let final_hash = net.peer(0).push_blocks(1, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(net.peer(1).has_block(final_hash)); } @@ -722,11 +717,10 @@ impl BlockAnnounceValidator for FailingBlockAnnounceValidator { } } -#[test] -fn sync_blocks_when_block_announce_validator_says_it_is_new_best() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_blocks_when_block_announce_validator_says_it_is_new_best() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); net.add_full_peer_with_config(Default::default()); net.add_full_peer_with_config(Default::default()); net.add_full_peer_with_config(FullPeerConfig { @@ -734,7 +728,7 @@ fn sync_blocks_when_block_announce_validator_says_it_is_new_best() { ..Default::default() }); - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // Add blocks but don't set them as best let block_hash = net.peer(0).generate_blocks_with_fork_choice( @@ -745,7 +739,7 @@ fn sync_blocks_when_block_announce_validator_says_it_is_new_best() { ); while !net.peer(2).has_block(block_hash) { - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; } } @@ -767,18 +761,17 @@ impl BlockAnnounceValidator for DeferredBlockAnnounceValidator { } } -#[test] -fn wait_until_deferred_block_announce_validation_is_ready() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn wait_until_deferred_block_announce_validation_is_ready() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); net.add_full_peer_with_config(Default::default()); net.add_full_peer_with_config(FullPeerConfig { block_announce_validator: Some(Box::new(NewBestBlockAnnounceValidator)), ..Default::default() }); - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // Add blocks but don't set them as best let block_hash = net.peer(0).generate_blocks_with_fork_choice( @@ -789,17 +782,16 @@ fn wait_until_deferred_block_announce_validation_is_ready() { ); while !net.peer(1).has_block(block_hash) { - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; } } /// When we don't inform the sync protocol about the best block, a node will not sync from us as the /// handshake is not does not contain our best block. -#[test] -fn sync_to_tip_requires_that_sync_protocol_is_informed_about_best_block() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_to_tip_requires_that_sync_protocol_is_informed_about_best_block() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 1); + let mut net = TestNet::new(1); // Produce some blocks let block_hash = @@ -807,18 +799,16 @@ fn sync_to_tip_requires_that_sync_protocol_is_informed_about_best_block() { .push_blocks_at_without_informing_sync(BlockId::Number(0), 3, true, true); // Add a node and wait until they are connected - runtime.block_on(async { - net.add_full_peer_with_config(Default::default()); - net.wait_until_connected().await; - net.wait_until_idle().await; - }); + net.add_full_peer_with_config(Default::default()); + net.run_until_connected().await; + net.run_until_idle().await; // The peer should not have synced the block. assert!(!net.peer(1).has_block(block_hash)); // Make sync protocol aware of the best block net.peer(0).network_service().new_best_block_imported(block_hash, 3); - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; // Connect another node that should now sync to the tip net.add_full_peer_with_config(FullPeerConfig { @@ -826,14 +816,15 @@ fn sync_to_tip_requires_that_sync_protocol_is_informed_about_best_block() { ..Default::default() }); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(2).has_block(block_hash) { Poll::Ready(()) } else { Poll::Pending } - })); + }) + .await; // However peer 1 should still not have the block. assert!(!net.peer(1).has_block(block_hash)); @@ -841,12 +832,11 @@ fn sync_to_tip_requires_that_sync_protocol_is_informed_about_best_block() { /// Ensures that if we as a syncing node sync to the tip while we are connected to another peer /// that is currently also doing a major sync. -#[test] -fn sync_to_tip_when_we_sync_together_with_multiple_peers() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn sync_to_tip_when_we_sync_together_with_multiple_peers() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 3); + let mut net = TestNet::new(3); let block_hash = net.peer(0) @@ -855,10 +845,8 @@ fn sync_to_tip_when_we_sync_together_with_multiple_peers() { net.peer(1) .push_blocks_at_without_informing_sync(BlockId::Number(0), 5_000, false, false); - runtime.block_on(async { - net.wait_until_connected().await; - net.wait_until_idle().await; - }); + net.run_until_connected().await; + net.run_until_idle().await; assert!(!net.peer(2).has_block(block_hash)); @@ -866,14 +854,14 @@ fn sync_to_tip_when_we_sync_together_with_multiple_peers() { net.peer(0).network_service().announce_block(block_hash, None); while !net.peer(2).has_block(block_hash) && !net.peer(1).has_block(block_hash) { - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; } } /// Ensures that when we receive a block announcement with some data attached, that we propagate /// this data when reannouncing the block. -#[test] -fn block_announce_data_is_propagated() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn block_announce_data_is_propagated() { struct TestBlockAnnounceValidator; impl BlockAnnounceValidator for TestBlockAnnounceValidator { @@ -897,8 +885,7 @@ fn block_announce_data_is_propagated() { } sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 1); + let mut net = TestNet::new(1); net.add_full_peer_with_config(FullPeerConfig { block_announce_validator: Some(Box::new(TestBlockAnnounceValidator)), @@ -912,7 +899,7 @@ fn block_announce_data_is_propagated() { }); // Wait until peer 1 is connected to both nodes. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).num_peers() == 2 && net.peer(0).num_peers() == 1 && @@ -922,18 +909,19 @@ fn block_announce_data_is_propagated() { } else { Poll::Pending } - })); + }) + .await; let block_hash = net.peer(0).push_blocks_at_without_announcing(BlockId::Number(0), 1, true); net.peer(0).announce_block(block_hash, Some(vec![137])); while !net.peer(1).has_block(block_hash) || !net.peer(2).has_block(block_hash) { - runtime.block_on(net.wait_until_idle()); + net.run_until_idle().await; } } -#[test] -fn continue_to_sync_after_some_block_announcement_verifications_failed() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn continue_to_sync_after_some_block_announcement_verifications_failed() { struct TestBlockAnnounceValidator; impl BlockAnnounceValidator for TestBlockAnnounceValidator { @@ -958,22 +946,19 @@ fn continue_to_sync_after_some_block_announcement_verifications_failed() { } sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 1); + let mut net = TestNet::new(1); net.add_full_peer_with_config(FullPeerConfig { block_announce_validator: Some(Box::new(TestBlockAnnounceValidator)), ..Default::default() }); - runtime.block_on(async { - net.wait_until_connected().await; - net.wait_until_idle().await; - }); + net.run_until_connected().await; + net.run_until_idle().await; let block_hash = net.peer(0).push_blocks(500, true); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(net.peer(1).has_block(block_hash)); } @@ -981,13 +966,12 @@ fn continue_to_sync_after_some_block_announcement_verifications_failed() { /// this peer if the request was successful. In the case of a justification request for example, /// we ask our peers multiple times until we got the requested justification. This test ensures that /// asking for the same justification multiple times doesn't ban a peer. -#[test] -fn multiple_requests_are_accepted_as_long_as_they_are_not_fulfilled() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn multiple_requests_are_accepted_as_long_as_they_are_not_fulfilled() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = JustificationTestNet::new(runtime.handle().clone(), 2); + let mut net = JustificationTestNet::new(2); net.peer(0).push_blocks(10, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; let hashof10 = net.peer(1).client().header(&BlockId::Number(10)).unwrap().unwrap().hash(); @@ -1002,9 +986,9 @@ fn multiple_requests_are_accepted_as_long_as_they_are_not_fulfilled() { for _ in 0..5 { // We need to sleep 10 seconds as this is the time we wait between sending a new // justification request. - std::thread::sleep(std::time::Duration::from_secs(10)); + tokio::time::sleep(tokio::time::Duration::from_secs(10)).await; net.peer(0).push_blocks(1, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert_eq!(1, net.peer(0).num_peers()); } @@ -1021,7 +1005,7 @@ fn multiple_requests_are_accepted_as_long_as_they_are_not_fulfilled() { .finalize_block(hashof10, Some((*b"FRNK", Vec::new())), true) .unwrap(); - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client().justifications(hashof10).unwrap() != @@ -1031,47 +1015,47 @@ fn multiple_requests_are_accepted_as_long_as_they_are_not_fulfilled() { } Poll::Ready(()) - })); + }) + .await; } -#[test] -fn syncs_all_forks_from_single_peer() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_all_forks_from_single_peer() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); net.peer(0).push_blocks(10, false); net.peer(1).push_blocks(10, false); // poll until the two nodes connect, otherwise announcing the block will not work - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // Peer 0 produces new blocks and announces. let branch1 = net.peer(0).push_blocks_at(BlockId::Number(10), 2, true); // Wait till peer 1 starts downloading - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).network().best_seen_block() != Some(12) { return Poll::Pending } Poll::Ready(()) - })); + }) + .await; // Peer 0 produces and announces another fork let branch2 = net.peer(0).push_blocks_at(BlockId::Number(10), 2, false); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; // Peer 1 should have both branches, assert!(net.peer(1).client().header(&BlockId::Hash(branch1)).unwrap().is_some()); assert!(net.peer(1).client().header(&BlockId::Hash(branch2)).unwrap().is_some()); } -#[test] -fn syncs_after_missing_announcement() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_after_missing_announcement() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); net.add_full_peer_with_config(Default::default()); // Set peer 1 to ignore announcement net.add_full_peer_with_config(FullPeerConfig { @@ -1081,23 +1065,22 @@ fn syncs_after_missing_announcement() { net.peer(0).push_blocks(10, false); net.peer(1).push_blocks(10, false); - runtime.block_on(net.wait_until_connected()); + net.run_until_connected().await; // Peer 0 produces a new block and announces. Peer 1 ignores announcement. net.peer(0).push_blocks_at(BlockId::Number(10), 1, false); // Peer 0 produces another block and announces. let final_block = net.peer(0).push_blocks_at(BlockId::Number(11), 1, false); net.peer(1).push_blocks_at(BlockId::Number(10), 1, true); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(net.peer(1).client().header(&BlockId::Hash(final_block)).unwrap().is_some()); } -#[test] -fn syncs_state() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_state() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); for skip_proofs in &[false, true] { - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); let mut genesis_storage: sp_core::storage::Storage = Default::default(); genesis_storage.top.insert(b"additional_key".to_vec(), vec![1]); let mut child_data: std::collections::BTreeMap, Vec> = Default::default(); @@ -1138,7 +1121,7 @@ fn syncs_state() { net.add_full_peer_with_config(config_two); net.peer(0).push_blocks(64, false); // Wait for peer 1 to sync header chain. - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(!net.peer(1).client().has_state_at(&BlockId::Number(64))); let just = (*b"FRNK", Vec::new()); @@ -1151,33 +1134,34 @@ fn syncs_state() { .unwrap(); net.peer(1).client().finalize_block(hashof60, Some(just), true).unwrap(); // Wait for state sync. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client.info().finalized_state.is_some() { Poll::Ready(()) } else { Poll::Pending } - })); + }) + .await; assert!(!net.peer(1).client().has_state_at(&BlockId::Number(64))); // Wait for the rest of the states to be imported. - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(1).client().has_state_at(&BlockId::Number(64)) { Poll::Ready(()) } else { Poll::Pending } - })); + }) + .await; } } -#[test] -fn syncs_indexed_blocks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_indexed_blocks() { use sp_runtime::traits::Hash; sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); let mut n: u64 = 0; net.add_full_peer_with_config(FullPeerConfig { storage_chain: true, ..Default::default() }); net.add_full_peer_with_config(FullPeerConfig { @@ -1216,7 +1200,7 @@ fn syncs_indexed_blocks() { .unwrap() .is_none()); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(net .peer(1) .client() @@ -1226,11 +1210,10 @@ fn syncs_indexed_blocks() { .is_some()); } -#[test] -fn warp_sync() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn warp_sync() { sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 0); + let mut net = TestNet::new(0); // Create 3 synced peers and 1 peer trying to warp sync. net.add_full_peer_with_config(Default::default()); net.add_full_peer_with_config(Default::default()); @@ -1244,30 +1227,30 @@ fn warp_sync() { net.peer(1).push_blocks(64, false); net.peer(2).push_blocks(64, false); // Wait for peer 1 to sync state. - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert!(!net.peer(3).client().has_state_at(&BlockId::Number(1))); assert!(net.peer(3).client().has_state_at(&BlockId::Number(64))); // Wait for peer 1 download block history - runtime.block_on(futures::future::poll_fn::<(), _>(|cx| { + futures::future::poll_fn::<(), _>(|cx| { net.poll(cx); if net.peer(3).has_body(gap_end) && net.peer(3).has_body(target) { Poll::Ready(()) } else { Poll::Pending } - })); + }) + .await; } -#[test] -fn syncs_huge_blocks() { +#[tokio::test(flavor = "multi_thread", worker_threads = 2)] +async fn syncs_huge_blocks() { use sp_core::storage::well_known_keys::HEAP_PAGES; use sp_runtime::codec::Encode; use substrate_test_runtime_client::BlockBuilderExt; sp_tracing::try_init_simple(); - let runtime = Runtime::new().unwrap(); - let mut net = TestNet::new(runtime.handle().clone(), 2); + let mut net = TestNet::new(2); // Increase heap space for bigger blocks. net.peer(0).generate_blocks(1, BlockOrigin::Own, |mut builder| { @@ -1284,7 +1267,7 @@ fn syncs_huge_blocks() { builder.build().unwrap().block }); - runtime.block_on(net.wait_until_sync()); + net.run_until_sync().await; assert_eq!(net.peer(0).client.info().best_number, 33); assert_eq!(net.peer(1).client.info().best_number, 33); }