sc_partner_chains_consensus_aura/
lib.rs

1// Copyright (C) Parity Technologies (UK) Ltd.
2// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0
3
4// This program is free software: you can redistribute it and/or modify
5// it under the terms of the GNU General Public License as published by
6// the Free Software Foundation, either version 3 of the License, or
7// (at your option) any later version.
8
9// This program is distributed in the hope that it will be useful,
10// but WITHOUT ANY WARRANTY; without even the implied warranty of
11// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
12// GNU General Public License for more details.
13
14// You should have received a copy of the GNU General Public License
15// along with this program. If not, see <https://www.gnu.org/licenses/>.
16
17// Additional modifications by Input Output Global, Inc.
18// Copyright (C) 2024, Input Output Global, Inc.
19
20pub mod import_queue;
21
22use futures::prelude::*;
23use parity_scale_codec::Codec;
24use sc_client_api::{BlockOf, backend::AuxStore};
25use sc_consensus::block_import::BlockImport;
26use sc_consensus::{BlockImportParams, ForkChoiceStrategy, StateAction};
27use sc_consensus_aura::{
28	BuildAuraWorkerParams, CompatibilityMode, StartAuraParams, find_pre_digest,
29};
30use sc_consensus_slots::{
31	BackoffAuthoringBlocksStrategy, InherentDataProviderExt, SimpleSlotWorkerToSlotWorker,
32	SlotInfo, SlotProportion, StorageChanges,
33};
34use sc_telemetry::TelemetryHandle;
35use sp_api::{Core, ProvideRuntimeApi};
36use sp_application_crypto::AppPublic;
37use sp_blockchain::HeaderBackend;
38use sp_consensus::{
39	BlockOrigin, Environment, Error as ConsensusError, Proposer, SelectChain, SyncOracle,
40};
41use sp_consensus_aura::AuraApi;
42use sp_consensus_slots::Slot;
43use sp_core::crypto::Pair;
44use sp_inherents::CreateInherentDataProviders;
45use sp_keystore::KeystorePtr;
46use sp_partner_chains_consensus_aura::InherentDigest;
47use sp_runtime::traits::{Block as BlockT, Header, Member, NumberFor};
48use std::{fmt::Debug, marker::PhantomData, pin::Pin, sync::Arc};
49
50type AuthorityId<P> = <P as Pair>::Public;
51
52const LOG_TARGET: &str = "aura";
53
54/// Start the aura worker. The returned future should be run in a futures executor.
55pub fn start_aura<P, B, C, SC, I, PF, SO, L, CIDP, BS, Error, ID>(
56	StartAuraParams {
57		slot_duration,
58		client,
59		select_chain,
60		block_import,
61		proposer_factory,
62		sync_oracle,
63		justification_sync_link,
64		create_inherent_data_providers,
65		force_authoring,
66		backoff_authoring_blocks,
67		keystore,
68		block_proposal_slot_portion,
69		max_block_proposal_slot_portion,
70		telemetry,
71		compatibility_mode,
72	}: StartAuraParams<C, SC, I, PF, SO, L, CIDP, BS, NumberFor<B>>,
73) -> Result<impl Future<Output = ()>, ConsensusError>
74where
75	P: Pair,
76	P::Public: AppPublic + Member,
77	P::Signature: TryFrom<Vec<u8>> + Member + Codec,
78	B: BlockT,
79	C: ProvideRuntimeApi<B> + BlockOf + AuxStore + HeaderBackend<B> + Send + Sync,
80	C::Api: AuraApi<B, AuthorityId<P>>,
81	SC: SelectChain<B>,
82	I: BlockImport<B> + Send + Sync + 'static,
83	PF: Environment<B, Error = Error> + Send + Sync + 'static,
84	PF::Proposer: Proposer<B, Error = Error>,
85	SO: SyncOracle + Send + Sync + Clone,
86	L: sc_consensus::JustificationSyncLink<B>,
87	CIDP: CreateInherentDataProviders<B, ()> + Send + 'static,
88	CIDP::InherentDataProviders: InherentDataProviderExt + Send,
89	BS: BackoffAuthoringBlocksStrategy<NumberFor<B>> + Send + Sync + 'static,
90	Error: std::error::Error + Send + From<ConsensusError> + 'static,
91	ID: InherentDigest + Send + Sync + 'static,
92{
93	let worker = build_aura_worker::<P, _, _, _, _, _, _, _, _, ID>(BuildAuraWorkerParams {
94		client,
95		block_import,
96		proposer_factory,
97		keystore,
98		sync_oracle: sync_oracle.clone(),
99		justification_sync_link,
100		force_authoring,
101		backoff_authoring_blocks,
102		telemetry,
103		block_proposal_slot_portion,
104		max_block_proposal_slot_portion,
105		compatibility_mode,
106	});
107
108	Ok(sc_consensus_slots::start_slot_worker(
109		slot_duration,
110		select_chain,
111		SimpleSlotWorkerToSlotWorker(worker),
112		sync_oracle,
113		create_inherent_data_providers,
114	))
115}
116
117/// Build the aura worker.
118///
119/// The caller is responsible for running this worker, otherwise it will do nothing.
120pub fn build_aura_worker<P, B, C, PF, I, SO, L, BS, Error, ID>(
121	BuildAuraWorkerParams {
122		client,
123		block_import,
124		proposer_factory,
125		sync_oracle,
126		justification_sync_link,
127		backoff_authoring_blocks,
128		keystore,
129		block_proposal_slot_portion,
130		max_block_proposal_slot_portion,
131		telemetry,
132		force_authoring,
133		compatibility_mode,
134	}: BuildAuraWorkerParams<C, I, PF, SO, L, BS, NumberFor<B>>,
135) -> impl sc_consensus_slots::SimpleSlotWorker<
136	B,
137	Proposer = PF::Proposer,
138	BlockImport = I,
139	SyncOracle = SO,
140	JustificationSyncLink = L,
141	Claim = P::Public,
142	AuxData = Vec<AuthorityId<P>>,
143>
144where
145	B: BlockT,
146	C: ProvideRuntimeApi<B> + BlockOf + AuxStore + HeaderBackend<B> + Send + Sync,
147	C::Api: AuraApi<B, AuthorityId<P>>,
148	PF: Environment<B, Error = Error> + Send + Sync + 'static,
149	PF::Proposer: Proposer<B, Error = Error>,
150	P: Pair,
151	P::Public: AppPublic + Member,
152	P::Signature: TryFrom<Vec<u8>> + Member + Codec,
153	I: BlockImport<B> + Send + Sync + 'static,
154	Error: std::error::Error + Send + From<ConsensusError> + 'static,
155	SO: SyncOracle + Send + Sync + Clone,
156	L: sc_consensus::JustificationSyncLink<B>,
157	BS: BackoffAuthoringBlocksStrategy<NumberFor<B>> + Send + Sync + 'static,
158	ID: InherentDigest + Send + Sync + 'static,
159{
160	AuraWorker {
161		client,
162		block_import,
163		env: proposer_factory,
164		keystore,
165		sync_oracle,
166		justification_sync_link,
167		force_authoring,
168		backoff_authoring_blocks,
169		telemetry,
170		block_proposal_slot_portion,
171		max_block_proposal_slot_portion,
172		compatibility_mode,
173		_phantom: PhantomData::<(fn() -> P, ID)>,
174	}
175}
176
177struct AuraWorker<C, E, I, P, SO, L, BS, N, ID> {
178	client: Arc<C>,
179	block_import: I,
180	env: E,
181	keystore: KeystorePtr,
182	sync_oracle: SO,
183	justification_sync_link: L,
184	force_authoring: bool,
185	backoff_authoring_blocks: Option<BS>,
186	block_proposal_slot_portion: SlotProportion,
187	max_block_proposal_slot_portion: Option<SlotProportion>,
188	telemetry: Option<TelemetryHandle>,
189	compatibility_mode: CompatibilityMode<N>,
190	_phantom: PhantomData<(fn() -> P, ID)>,
191}
192
193#[async_trait::async_trait]
194impl<B, C, E, I, P, Error, SO, L, BS, ID> sc_consensus_slots::SimpleSlotWorker<B>
195	for AuraWorker<C, E, I, P, SO, L, BS, NumberFor<B>, ID>
196where
197	B: BlockT,
198	C: ProvideRuntimeApi<B> + BlockOf + HeaderBackend<B> + Sync,
199	C::Api: AuraApi<B, AuthorityId<P>>,
200	E: Environment<B, Error = Error> + Send + Sync,
201	E::Proposer: Proposer<B, Error = Error>,
202	I: BlockImport<B> + Send + Sync + 'static,
203	P: Pair,
204	P::Public: AppPublic + Member,
205	P::Signature: TryFrom<Vec<u8>> + Member + Codec,
206	SO: SyncOracle + Send + Clone + Sync,
207	L: sc_consensus::JustificationSyncLink<B>,
208	BS: BackoffAuthoringBlocksStrategy<NumberFor<B>> + Send + Sync + 'static,
209	Error: std::error::Error + Send + From<ConsensusError> + 'static,
210	ID: InherentDigest + Send + Sync + 'static,
211{
212	type BlockImport = I;
213	type SyncOracle = SO;
214	type JustificationSyncLink = L;
215	type CreateProposer =
216		Pin<Box<dyn Future<Output = Result<E::Proposer, ConsensusError>> + Send + 'static>>;
217	type Proposer = E::Proposer;
218	type Claim = P::Public;
219	type AuxData = Vec<AuthorityId<P>>;
220
221	fn logging_target(&self) -> &'static str {
222		"aura"
223	}
224
225	fn block_import(&mut self) -> &mut Self::BlockImport {
226		&mut self.block_import
227	}
228
229	fn aux_data(&self, header: &B::Header, _slot: Slot) -> Result<Self::AuxData, ConsensusError> {
230		authorities(
231			self.client.as_ref(),
232			header.hash(),
233			*header.number() + 1u32.into(),
234			&self.compatibility_mode,
235		)
236	}
237
238	fn authorities_len(&self, authorities: &Self::AuxData) -> Option<usize> {
239		Some(authorities.len())
240	}
241
242	async fn claim_slot(
243		&mut self,
244		_header: &B::Header,
245		slot: Slot,
246		authorities: &Self::AuxData,
247	) -> Option<Self::Claim> {
248		sc_consensus_aura::standalone::claim_slot::<P>(slot, authorities, &self.keystore).await
249	}
250
251	fn pre_digest_data(&self, slot: Slot, _claim: &Self::Claim) -> Vec<sp_runtime::DigestItem> {
252		vec![sc_consensus_aura::standalone::pre_digest::<P>(slot)]
253	}
254
255	async fn block_import_params(
256		&self,
257		header: B::Header,
258		header_hash: &B::Hash,
259		body: Vec<B::Extrinsic>,
260		storage_changes: StorageChanges<B>,
261		public: Self::Claim,
262		_authorities: Self::AuxData,
263	) -> Result<BlockImportParams<B>, ConsensusError> {
264		let signature_digest_item =
265			sc_consensus_aura::standalone::seal::<_, P>(header_hash, &public, &self.keystore)?;
266
267		let mut import_block = BlockImportParams::new(BlockOrigin::Own, header);
268		import_block.post_digests.push(signature_digest_item);
269		import_block.body = Some(body);
270		import_block.state_action =
271			StateAction::ApplyChanges(sc_consensus::StorageChanges::Changes(storage_changes));
272		import_block.fork_choice = Some(ForkChoiceStrategy::LongestChain);
273
274		Ok(import_block)
275	}
276
277	fn force_authoring(&self) -> bool {
278		self.force_authoring
279	}
280
281	fn should_backoff(&self, slot: Slot, chain_head: &B::Header) -> bool {
282		if let Some(ref strategy) = self.backoff_authoring_blocks {
283			if let Ok(chain_head_slot) = find_pre_digest::<B, P::Signature>(chain_head) {
284				return strategy.should_backoff(
285					*chain_head.number(),
286					chain_head_slot,
287					self.client.info().finalized_number,
288					slot,
289					self.logging_target(),
290				);
291			}
292		}
293		false
294	}
295
296	fn sync_oracle(&mut self) -> &mut Self::SyncOracle {
297		&mut self.sync_oracle
298	}
299
300	fn justification_sync_link(&mut self) -> &mut Self::JustificationSyncLink {
301		&mut self.justification_sync_link
302	}
303
304	fn proposer(&mut self, block: &B::Header) -> Self::CreateProposer {
305		self.env
306			.init(block)
307			.map_err(|e| ConsensusError::ClientImport(format!("{:?}", e)))
308			.boxed()
309	}
310
311	fn telemetry(&self) -> Option<TelemetryHandle> {
312		self.telemetry.clone()
313	}
314
315	fn proposing_remaining_duration(&self, slot_info: &SlotInfo<B>) -> std::time::Duration {
316		let parent_slot = find_pre_digest::<B, P::Signature>(&slot_info.chain_head).ok();
317
318		sc_consensus_slots::proposing_remaining_duration(
319			parent_slot,
320			slot_info,
321			&self.block_proposal_slot_portion,
322			self.max_block_proposal_slot_portion.as_ref(),
323			sc_consensus_slots::SlotLenienceType::Exponential,
324			self.logging_target(),
325		)
326	}
327}
328
329fn authorities<A, B, C>(
330	client: &C,
331	parent_hash: B::Hash,
332	context_block_number: NumberFor<B>,
333	compatibility_mode: &CompatibilityMode<NumberFor<B>>,
334) -> Result<Vec<A>, ConsensusError>
335where
336	A: Codec + Debug,
337	B: BlockT,
338	C: ProvideRuntimeApi<B>,
339	C::Api: AuraApi<B, A>,
340{
341	let runtime_api = client.runtime_api();
342
343	match compatibility_mode {
344		CompatibilityMode::None => {},
345		// Use `initialize_block` until we hit the block that should disable the mode.
346		CompatibilityMode::UseInitializeBlock { until } => {
347			if *until > context_block_number {
348				runtime_api
349					.initialize_block(
350						parent_hash,
351						&B::Header::new(
352							context_block_number,
353							Default::default(),
354							Default::default(),
355							parent_hash,
356							Default::default(),
357						),
358					)
359					.map_err(|_| ConsensusError::InvalidAuthoritiesSet)?;
360			}
361		},
362	}
363
364	runtime_api
365		.authorities(parent_hash)
366		.ok()
367		.ok_or(ConsensusError::InvalidAuthoritiesSet)
368}
369
370#[cfg(test)]
371mod tests {
372	use super::*;
373	use parking_lot::Mutex;
374	use sc_block_builder::BlockBuilderBuilder;
375	use sc_client_api::BlockchainEvents;
376	use sc_consensus::BoxJustificationImport;
377	use sc_consensus_aura::{CheckForEquivocation, standalone::slot_duration};
378	use sc_consensus_slots::{BackoffAuthoringOnFinalizedHeadLagging, SimpleSlotWorker};
379	use sc_keystore::LocalKeystore;
380	use sc_network_test::{Block as TestBlock, *};
381	use sp_application_crypto::{AppCrypto, key_types::AURA};
382	use sp_consensus::{DisableProofRecording, NoNetwork as DummyOracle, Proposal};
383	use sp_consensus_aura::SlotDuration;
384	use sp_consensus_aura::inherents::InherentDataProvider;
385	use sp_consensus_aura::sr25519::AuthorityPair;
386	use sp_inherents::InherentData;
387	use sp_keyring::sr25519::Keyring;
388	use sp_keystore::Keystore;
389	use sp_partner_chains_consensus_aura::CurrentSlotProvider;
390	use sp_runtime::{
391		Digest,
392		traits::{Block as BlockT, Header as _},
393	};
394	use sp_timestamp::Timestamp;
395	use std::{
396		task::Poll,
397		time::{Duration, Instant},
398	};
399	use substrate_test_runtime_client::{
400		TestClient,
401		runtime::{H256, Header},
402	};
403
404	const SLOT_DURATION_MS: u64 = 1000;
405
406	type Error = sp_blockchain::Error;
407
408	struct DummyFactory(Arc<TestClient>);
409	struct DummyProposer(Arc<TestClient>);
410
411	impl Environment<TestBlock> for DummyFactory {
412		type Proposer = DummyProposer;
413		type CreateProposer = futures::future::Ready<Result<DummyProposer, Error>>;
414		type Error = Error;
415
416		fn init(&mut self, _: &<TestBlock as BlockT>::Header) -> Self::CreateProposer {
417			futures::future::ready(Ok(DummyProposer(self.0.clone())))
418		}
419	}
420
421	impl Proposer<TestBlock> for DummyProposer {
422		type Error = Error;
423		type Proposal = future::Ready<Result<Proposal<TestBlock, ()>, Error>>;
424		type ProofRecording = DisableProofRecording;
425		type Proof = ();
426
427		fn propose(
428			self,
429			_: InherentData,
430			digests: Digest,
431			_: Duration,
432			_: Option<usize>,
433		) -> Self::Proposal {
434			let r = BlockBuilderBuilder::new(&*self.0)
435				.on_parent_block(self.0.chain_info().best_hash)
436				.fetch_parent_block_number(&*self.0)
437				.unwrap()
438				.with_inherent_digests(digests)
439				.build()
440				.unwrap()
441				.build();
442
443			future::ready(r.map(|b| Proposal {
444				block: b.block,
445				proof: (),
446				storage_changes: b.storage_changes,
447			}))
448		}
449	}
450
451	type AuraVerifier =
452		import_queue::AuraVerifier<PeersFullClient, AuthorityPair, TestCIDP, u64, ()>;
453	type AuraPeer = Peer<(), PeersClient>;
454
455	#[derive(Default)]
456	pub struct AuraTestNet {
457		peers: Vec<AuraPeer>,
458	}
459
460	pub struct TestCIDP;
461
462	#[async_trait::async_trait]
463	impl CreateInherentDataProviders<Block, (Slot, ())> for TestCIDP {
464		type InherentDataProviders = ();
465
466		async fn create_inherent_data_providers(
467			&self,
468			_parent: <Block as BlockT>::Hash,
469			_extra_args: (Slot, ()),
470		) -> Result<Self::InherentDataProviders, Box<dyn std::error::Error + Send + Sync>> {
471			Ok(())
472		}
473	}
474
475	impl CurrentSlotProvider for TestCIDP {
476		fn slot(&self) -> Slot {
477			Slot::from_timestamp(Timestamp::current(), SlotDuration::from_millis(SLOT_DURATION_MS))
478		}
479	}
480
481	impl TestNetFactory for AuraTestNet {
482		type Verifier = AuraVerifier;
483		type PeerData = ();
484		type BlockImport = PeersClient;
485
486		fn make_verifier(&self, client: PeersClient, _peer_data: &()) -> Self::Verifier {
487			let client = client.as_client();
488			let slot_duration = slot_duration(&*client).expect("slot duration available");
489
490			assert_eq!(slot_duration.as_millis() as u64, SLOT_DURATION_MS);
491			AuraVerifier::new(
492				client,
493				TestCIDP,
494				CheckForEquivocation::Yes,
495				None,
496				CompatibilityMode::None,
497			)
498		}
499
500		fn make_block_import(
501			&self,
502			client: PeersClient,
503		) -> (
504			BlockImportAdapter<Self::BlockImport>,
505			Option<BoxJustificationImport<Block>>,
506			Self::PeerData,
507		) {
508			(client.as_block_import(), None, ())
509		}
510
511		fn peer(&mut self, i: usize) -> &mut AuraPeer {
512			&mut self.peers[i]
513		}
514
515		fn peers(&self) -> &Vec<AuraPeer> {
516			&self.peers
517		}
518
519		fn peers_mut(&mut self) -> &mut Vec<AuraPeer> {
520			&mut self.peers
521		}
522
523		fn mut_peers<F: FnOnce(&mut Vec<AuraPeer>)>(&mut self, closure: F) {
524			closure(&mut self.peers);
525		}
526	}
527
528	#[tokio::test]
529	async fn authoring_blocks() {
530		sp_tracing::try_init_simple();
531		let net = AuraTestNet::new(3);
532
533		let peers = &[(0, Keyring::Alice), (1, Keyring::Bob), (2, Keyring::Charlie)];
534
535		let net = Arc::new(Mutex::new(net));
536		let mut import_notifications = Vec::new();
537		let mut aura_futures = Vec::new();
538
539		let mut keystore_paths = Vec::new();
540		for (peer_id, key) in peers {
541			let mut net = net.lock();
542			let peer = net.peer(*peer_id);
543			let client = peer.client().as_client();
544			let select_chain = peer.select_chain().expect("full client has a select chain");
545			let keystore_path = tempfile::tempdir().expect("Creates keystore path");
546			let keystore = Arc::new(
547				LocalKeystore::open(keystore_path.path(), None).expect("Creates keystore."),
548			);
549
550			keystore
551				.sr25519_generate_new(AURA, Some(&key.to_seed()))
552				.expect("Creates authority key");
553			keystore_paths.push(keystore_path);
554
555			let environ = DummyFactory(client.clone());
556			import_notifications.push(
557				client
558					.import_notification_stream()
559					.take_while(|n| {
560						future::ready(!(n.origin != BlockOrigin::Own && n.header.number() < &5))
561					})
562					.for_each(move |_| future::ready(())),
563			);
564
565			let slot_duration = slot_duration(&*client).expect("slot duration available");
566
567			aura_futures.push(
568				start_aura::<AuthorityPair, _, _, _, _, _, _, _, _, _, _, ()>(StartAuraParams {
569					slot_duration,
570					block_import: client.clone(),
571					select_chain,
572					client,
573					proposer_factory: environ,
574					sync_oracle: DummyOracle,
575					justification_sync_link: (),
576					create_inherent_data_providers: |_, _| async {
577						let slot = InherentDataProvider::from_timestamp_and_slot_duration(
578							Timestamp::current(),
579							SlotDuration::from_millis(SLOT_DURATION_MS),
580						);
581
582						Ok((slot,))
583					},
584					force_authoring: false,
585					backoff_authoring_blocks: Some(
586						BackoffAuthoringOnFinalizedHeadLagging::default(),
587					),
588					keystore,
589					block_proposal_slot_portion: SlotProportion::new(0.5),
590					max_block_proposal_slot_portion: None,
591					telemetry: None,
592					compatibility_mode: CompatibilityMode::None,
593				})
594				.expect("Starts aura"),
595			);
596		}
597
598		future::select(
599			future::poll_fn(move |cx| {
600				net.lock().poll(cx);
601				Poll::<()>::Pending
602			}),
603			future::select(future::join_all(aura_futures), future::join_all(import_notifications)),
604		)
605		.await;
606	}
607
608	#[tokio::test]
609	async fn current_node_authority_should_claim_slot() {
610		let net = AuraTestNet::new(4);
611
612		let mut authorities = vec![
613			Keyring::Alice.public().into(),
614			Keyring::Bob.public().into(),
615			Keyring::Charlie.public().into(),
616		];
617
618		let keystore_path = tempfile::tempdir().expect("Creates keystore path");
619		let keystore = LocalKeystore::open(keystore_path.path(), None).expect("Creates keystore.");
620		let public = keystore
621			.sr25519_generate_new(AuthorityPair::ID, None)
622			.expect("Key should be created");
623		authorities.push(public.into());
624
625		let net = Arc::new(Mutex::new(net));
626
627		let mut net = net.lock();
628		let peer = net.peer(3);
629		let client = peer.client().as_client();
630		let environ = DummyFactory(client.clone());
631
632		let mut worker = AuraWorker {
633			client: client.clone(),
634			block_import: client,
635			env: environ,
636			keystore: keystore.into(),
637			sync_oracle: DummyOracle,
638			justification_sync_link: (),
639			force_authoring: false,
640			backoff_authoring_blocks: Some(BackoffAuthoringOnFinalizedHeadLagging::default()),
641			telemetry: None,
642			block_proposal_slot_portion: SlotProportion::new(0.5),
643			max_block_proposal_slot_portion: None,
644			compatibility_mode: Default::default(),
645			_phantom: PhantomData::<(fn() -> AuthorityPair, ())>,
646		};
647
648		let head = Header::new(
649			1,
650			H256::from_low_u64_be(0),
651			H256::from_low_u64_be(0),
652			Default::default(),
653			Default::default(),
654		);
655		assert!(worker.claim_slot(&head, 0.into(), &authorities).await.is_none());
656		assert!(worker.claim_slot(&head, 1.into(), &authorities).await.is_none());
657		assert!(worker.claim_slot(&head, 2.into(), &authorities).await.is_none());
658		assert!(worker.claim_slot(&head, 3.into(), &authorities).await.is_some());
659		assert!(worker.claim_slot(&head, 4.into(), &authorities).await.is_none());
660		assert!(worker.claim_slot(&head, 5.into(), &authorities).await.is_none());
661		assert!(worker.claim_slot(&head, 6.into(), &authorities).await.is_none());
662		assert!(worker.claim_slot(&head, 7.into(), &authorities).await.is_some());
663	}
664
665	#[tokio::test]
666	async fn on_slot_returns_correct_block() {
667		let net = AuraTestNet::new(4);
668
669		let keystore_path = tempfile::tempdir().expect("Creates keystore path");
670		let keystore = LocalKeystore::open(keystore_path.path(), None).expect("Creates keystore.");
671		keystore
672			.sr25519_generate_new(AuthorityPair::ID, Some(&Keyring::Alice.to_seed()))
673			.expect("Key should be created");
674
675		let net = Arc::new(Mutex::new(net));
676
677		let mut net = net.lock();
678		let peer = net.peer(3);
679		let client = peer.client().as_client();
680		let environ = DummyFactory(client.clone());
681
682		let mut worker = AuraWorker {
683			client: client.clone(),
684			block_import: client.clone(),
685			env: environ,
686			keystore: keystore.into(),
687			sync_oracle: DummyOracle,
688			justification_sync_link: (),
689			force_authoring: false,
690			backoff_authoring_blocks: Option::<()>::None,
691			telemetry: None,
692			block_proposal_slot_portion: SlotProportion::new(0.5),
693			max_block_proposal_slot_portion: None,
694			compatibility_mode: Default::default(),
695			_phantom: PhantomData::<(fn() -> AuthorityPair, ())>,
696		};
697
698		let head = client.expect_header(client.info().genesis_hash).unwrap();
699
700		let res = worker
701			.on_slot(SlotInfo {
702				slot: 0.into(),
703				ends_at: Instant::now() + Duration::from_secs(100),
704				create_inherent_data: Box::new(()),
705				duration: Duration::from_millis(1000),
706				chain_head: head,
707				block_size_limit: None,
708			})
709			.await
710			.unwrap();
711
712		// The returned block should be imported and we should be able to get its header by now.
713		assert!(client.header(res.block.hash()).unwrap().is_some());
714	}
715}