diff --git a/Cargo.lock b/Cargo.lock index 1e3aa107a9981..254e5fb4149af 100644 --- a/Cargo.lock +++ b/Cargo.lock @@ -8340,10 +8340,13 @@ dependencies = [ "log", "parity-scale-codec", "parking_lot 0.12.1", + "sc-block-builder", "sc-client-api", "sc-consensus", "sc-consensus-epochs", "sc-consensus-slots", + "sc-keystore", + "sc-network-test", "sc-telemetry", "schnorrkel", "sp-api", @@ -8356,9 +8359,12 @@ dependencies = [ "sp-consensus-vrf", "sp-core", "sp-inherents", + "sp-keyring", "sp-keystore", "sp-runtime", + "sp-timestamp", "substrate-prometheus-endpoint", + "substrate-test-runtime-client", "thiserror", ] @@ -10807,6 +10813,7 @@ dependencies = [ "log", "memory-db", "pallet-babe", + "pallet-sassafras", "pallet-timestamp", "parity-scale-codec", "parity-util-mem", @@ -10821,6 +10828,7 @@ dependencies = [ "sp-consensus", "sp-consensus-aura", "sp-consensus-babe", + "sp-consensus-sassafras", "sp-core", "sp-externalities", "sp-finality-grandpa", diff --git a/bin/node-sassafras/node/src/command.rs b/bin/node-sassafras/node/src/command.rs index 74ac7dc809802..fad50283d2440 100644 --- a/bin/node-sassafras/node/src/command.rs +++ b/bin/node-sassafras/node/src/command.rs @@ -10,7 +10,7 @@ use sc_service::PartialComponents; impl SubstrateCli for Cli { fn impl_name() -> String { - "Substrate Node".into() + "Sassafras Node".into() } fn impl_version() -> String { @@ -30,7 +30,7 @@ impl SubstrateCli for Cli { } fn copyright_start_year() -> i32 { - 2017 + 2022 } fn load_spec(&self, id: &str) -> Result, String> { @@ -96,7 +96,8 @@ pub fn run() -> sc_cli::Result<()> { runner.async_run(|config| { let PartialComponents { client, task_manager, backend, .. } = service::new_partial(&config)?; - let aux_revert = Box::new(|client, _, blocks| { + let aux_revert = Box::new(|client, backend, blocks| { + sc_consensus_sassafras::revert(backend, blocks)?; sc_finality_grandpa::revert(client, blocks)?; Ok(()) }); diff --git a/bin/node-sassafras/node/src/service.rs b/bin/node-sassafras/node/src/service.rs index a8cd614882ea7..1f7beb20f3609 100644 --- a/bin/node-sassafras/node/src/service.rs +++ b/bin/node-sassafras/node/src/service.rs @@ -264,7 +264,7 @@ pub fn new_full(mut config: Configuration) -> Result let slot_duration = sassafras_link.genesis_config().slot_duration(); - let sassafras_params = sc_consensus_sassafras::SassafrasParams { + let sassafras_params = sc_consensus_sassafras::SassafrasWorkerParams { client: client.clone(), keystore: keystore_container.sync_keystore(), select_chain, @@ -303,7 +303,6 @@ pub fn new_full(mut config: Configuration) -> Result if role.is_authority() { Some(keystore_container.sync_keystore()) } else { None }; let grandpa_config = sc_finality_grandpa::Config { - // FIXME #1578 make this available through chainspec gossip_duration: Duration::from_millis(333), justification_period: 512, name: Some(name), diff --git a/bin/node-sassafras/runtime/src/lib.rs b/bin/node-sassafras/runtime/src/lib.rs index e78b280db12da..f0ef011b14d39 100644 --- a/bin/node-sassafras/runtime/src/lib.rs +++ b/bin/node-sassafras/runtime/src/lib.rs @@ -395,6 +395,21 @@ impl_runtime_apis! { fn slot_ticket(slot: sp_consensus_sassafras::Slot) -> Option { Sassafras::slot_ticket(slot) } + + fn generate_key_ownership_proof( + _slot: sp_consensus_sassafras::Slot, + _authority_id: sp_consensus_sassafras::AuthorityId, + ) -> Option { + None + } + + fn submit_report_equivocation_unsigned_extrinsic( + equivocation_proof: sp_consensus_sassafras::EquivocationProof<::Header>, + _key_owner_proof: sp_consensus_sassafras::OpaqueKeyOwnershipProof, + ) -> bool { + //let key_owner_proof = key_owner_proof.decode()?; + Sassafras::submit_unsigned_equivocation_report(equivocation_proof) + } } impl sp_session::SessionKeys for Runtime { diff --git a/client/consensus/babe/src/tests.rs b/client/consensus/babe/src/tests.rs index 58f5e7b8eb6d4..909a8604c138c 100644 --- a/client/consensus/babe/src/tests.rs +++ b/client/consensus/babe/src/tests.rs @@ -36,7 +36,6 @@ use sp_consensus_babe::{ inherents::InherentDataProvider, make_transcript, make_transcript_data, AllowedSlots, AuthorityPair, Slot, }; -use sp_consensus_slots::SlotDuration; use sp_core::crypto::Pair; use sp_keystore::{vrf::make_transcript as transcript_from_data, SyncCryptoStore}; use sp_runtime::{ @@ -68,8 +67,6 @@ type Mutator = Arc; type BabeBlockImport = PanickingBlockImport>>; -const SLOT_DURATION_MS: u64 = 1000; - #[derive(Clone)] struct DummyFactory { client: Arc, @@ -318,14 +315,15 @@ impl TestNetFactory for BabeTestNet { let (_, longest_chain) = TestClientBuilder::new().build_with_longest_chain(); + let slot_duration = data.link.config.slot_duration(); TestVerifier { inner: BabeVerifier { client: client.clone(), select_chain: longest_chain, - create_inherent_data_providers: Box::new(|_, _| async { + create_inherent_data_providers: Box::new(move |_, _| async move { let slot = InherentDataProvider::from_timestamp_and_slot_duration( Timestamp::current(), - SlotDuration::from_millis(SLOT_DURATION_MS), + slot_duration, ); Ok((slot,)) }), @@ -425,6 +423,7 @@ fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static .for_each(|_| future::ready(())), ); + let slot_duration = data.link.config.slot_duration(); babe_futures.push( start_babe(BabeParams { block_import: data.block_import.lock().take().expect("import set up during init"), @@ -432,10 +431,10 @@ fn run_one_test(mutator: impl Fn(&mut TestHeader, Stage) + Send + Sync + 'static client, env: environ, sync_oracle: DummyOracle, - create_inherent_data_providers: Box::new(|_, _| async { + create_inherent_data_providers: Box::new(move |_, _| async move { let slot = InherentDataProvider::from_timestamp_and_slot_duration( Timestamp::current(), - SlotDuration::from_millis(SLOT_DURATION_MS), + slot_duration, ); Ok((slot,)) }), @@ -1004,7 +1003,7 @@ fn obsolete_blocks_aux_data_cleanup() { let data = peer.data.as_ref().expect("babe link set up during initialization"); let client = peer.client().as_client(); - // Register the handler (as done by `babe_start`) + // Register the handler (as done by Babe's `block_import` method) let client_clone = client.clone(); let on_finality = move |summary: &FinalityNotification| { aux_storage_cleanup(client_clone.as_ref(), summary) diff --git a/client/consensus/epochs/src/lib.rs b/client/consensus/epochs/src/lib.rs index 2e0186495db5e..994f3789f4515 100644 --- a/client/consensus/epochs/src/lib.rs +++ b/client/consensus/epochs/src/lib.rs @@ -777,11 +777,6 @@ where } } - /// Return the inner fork tree. - pub fn tree(&self) -> &ForkTree> { - &self.inner - } - /// Reset to a specified pair of epochs, as if they were announced at blocks `parent_hash` and /// `hash`. pub fn reset(&mut self, parent_hash: Hash, hash: Hash, number: Number, current: E, next: E) { @@ -832,6 +827,11 @@ where self.epochs.remove(&(h, n)); }); } + + /// Return the inner fork tree (mostly useful for testing) + pub fn tree(&self) -> &ForkTree> { + &self.inner + } } /// Type alias to produce the epoch-changes tree from a block type. diff --git a/client/consensus/sassafras/Cargo.toml b/client/consensus/sassafras/Cargo.toml index 888959090b31a..a6c6bb59984f1 100644 --- a/client/consensus/sassafras/Cargo.toml +++ b/client/consensus/sassafras/Cargo.toml @@ -40,3 +40,11 @@ sp-core = { version = "6.0.0", path = "../../../primitives/core" } sp-inherents = { version = "4.0.0-dev", path = "../../../primitives/inherents" } sp-keystore = { version = "0.12.0", path = "../../../primitives/keystore" } sp-runtime = { version = "6.0.0", path = "../../../primitives/runtime" } + +[dev-dependencies] +sc-block-builder = { version = "0.10.0-dev", path = "../../block-builder" } +sc-keystore = { version = "4.0.0-dev", path = "../../keystore" } +sc-network-test = { version = "0.8.0", path = "../../network/test" } +sp-keyring = { version = "6.0.0", path = "../../../primitives/keyring" } +sp-timestamp = { version = "4.0.0-dev", path = "../../../primitives/timestamp" } +substrate-test-runtime-client = { version = "2.0.0", path = "../../../test-utils/runtime/client" } \ No newline at end of file diff --git a/client/consensus/sassafras/src/authorship.rs b/client/consensus/sassafras/src/authorship.rs index 92e65666918e5..1f8f7b3be3787 100644 --- a/client/consensus/sassafras/src/authorship.rs +++ b/client/consensus/sassafras/src/authorship.rs @@ -40,13 +40,18 @@ pub(crate) fn secondary_authority_index( /// Try to claim an epoch slot. /// If ticket is `None`, then the slot should be claimed using the fallback mechanism. -fn claim_slot( +pub(crate) fn claim_slot( slot: Slot, epoch: &Epoch, ticket: Option, keystore: &SyncCryptoStorePtr, ) -> Option<(PreDigest, AuthorityId)> { let config = &epoch.config; + + if config.authorities.is_empty() { + return None + } + let (authority_idx, ticket_aux) = match ticket { Some(ticket) => { log::debug!(target: "sassafras", "🌳 [TRY PRIMARY]"); @@ -63,7 +68,7 @@ fn claim_slot( let authority_id = config.authorities.get(authority_idx as usize).map(|auth| &auth.0)?; - let transcript_data = make_slot_transcript_data(&config.randomness, slot, epoch.epoch_index); + let transcript_data = make_slot_transcript_data(&config.randomness, slot, epoch.epoch_idx); let signature = SyncCryptoStore::sr25519_vrf_sign( &**keystore, AuthorityId::ID, @@ -77,7 +82,7 @@ fn claim_slot( authority_idx, slot, vrf_output: VRFOutput(signature.output), - vrf_proof: VRFProof(signature.proof.clone()), + vrf_proof: VRFProof(signature.proof), ticket_aux, }; @@ -87,7 +92,7 @@ fn claim_slot( /// Generate the tickets for the given epoch. /// Tickets additional information will be stored within the `Epoch` structure. /// The additional information will be used later during session to claim slots. -pub fn generate_epoch_tickets(epoch: &mut Epoch, keystore: &SyncCryptoStorePtr) -> Vec { +fn generate_epoch_tickets(epoch: &mut Epoch, keystore: &SyncCryptoStorePtr) -> Vec { let config = &epoch.config; let max_attempts = config.threshold_params.attempts_number; let redundancy_factor = config.threshold_params.redundancy_factor; @@ -111,7 +116,7 @@ pub fn generate_epoch_tickets(epoch: &mut Epoch, keystore: &SyncCryptoStorePtr) let make_ticket = |attempt| { let transcript_data = - make_ticket_transcript_data(&config.randomness, attempt, epoch.epoch_index); + make_ticket_transcript_data(&config.randomness, attempt, epoch.epoch_idx); // TODO-SASS-P4: can be a good idea to replace `vrf_sign` with `vrf_sign_after_check`, // But we need to modify the CryptoStore interface first. @@ -144,7 +149,7 @@ pub fn generate_epoch_tickets(epoch: &mut Epoch, keystore: &SyncCryptoStorePtr) tickets } -struct SassafrasSlotWorker { +struct SlotWorker { client: Arc, block_import: I, env: E, @@ -159,7 +164,7 @@ struct SassafrasSlotWorker { #[async_trait::async_trait] impl sc_consensus_slots::SimpleSlotWorker - for SassafrasSlotWorker + for SlotWorker where B: BlockT, C: ProvideRuntimeApi + HeaderBackend + HeaderMetadata, @@ -315,7 +320,7 @@ where } fn should_backoff(&self, _slot: Slot, _chain_head: &B::Header) -> bool { - // TODO-SASS-P2 + // TODO-SASS-P3 false } @@ -336,7 +341,7 @@ where } fn telemetry(&self) -> Option { - // TODO-SASS-P2 + // TODO-SASS-P4 None } @@ -357,7 +362,14 @@ where } } -async fn tickets_worker( +/// Authoring tickets generation worker. +/// +/// Listens on the client's import notification stream for blocks which contains new epoch +/// information, that is blocks that signals the begin of a new epoch. +/// This event here triggers the begin of the generation of tickets for the next epoch. +/// The tickets generated by the worker are saved within the epoch changes tree +/// and are volatile. +async fn start_tickets_worker( client: Arc, keystore: SyncCryptoStorePtr, epoch_changes: SharedEpochChanges, @@ -369,6 +381,7 @@ async fn tickets_worker( SC: SelectChain + 'static, { let mut notifications = client.import_notification_stream(); + while let Some(notification) = notifications.next().await { let epoch_desc = match find_next_epoch_digest::(¬ification.header) { Ok(Some(epoch_desc)) => epoch_desc, @@ -379,7 +392,7 @@ async fn tickets_worker( _ => continue, }; - debug!(target: "sassafras", "🌳 New epoch annouced {:x?}", epoch_desc); + debug!(target: "sassafras", "🌳 New epoch announced {:x?}", epoch_desc); let number = *notification.header.number(); let position = if number == One::one() { @@ -389,17 +402,20 @@ async fn tickets_worker( }; let epoch_identifier = EpochIdentifier { position, hash: notification.hash, number }; - let tickets = epoch_changes - .shared_data() - .epoch_mut(&epoch_identifier) - .map(|epoch| authorship::generate_epoch_tickets(epoch, &keystore)) - .unwrap_or_default(); + let mut epoch = match epoch_changes.shared_data().epoch(&epoch_identifier).cloned() { + Some(epoch) => epoch, + None => { + warn!(target: "🌳 sassafras", "Unexpected missing epoch data for {:?}", epoch_identifier); + continue + }, + }; + let tickets = generate_epoch_tickets(&mut epoch, &keystore); if tickets.is_empty() { continue } - // Get the best block on which we will build and send the tickets. + // Get the best block on which we will publish the tickets. let best_id = match select_chain.best_chain().await { Ok(header) => BlockId::Hash(header.hash()), Err(err) => { @@ -413,13 +429,20 @@ async fn tickets_worker( Ok(false) => Some("Unknown reason".to_string()), _ => None, }; - if let Some(err) = err { - error!(target: "sassafras", "🌳 Unable to submit tickets: {}", err); - // Remove tickets from epoch tree node. - epoch_changes - .shared_data() - .epoch_mut(&epoch_identifier) - .map(|epoch| epoch.tickets_aux.clear()); + + match err { + None => { + // Cache tickets in the epoch changes tree + epoch_changes + .shared_data() + .epoch_mut(&epoch_identifier) + .map(|target_epoch| target_epoch.tickets_aux = epoch.tickets_aux); + // TODO-SASS-P4: currently we don't persist the tickets proofs + // Thus on reboot/crash we are loosing them. + }, + Some(err) => { + error!(target: "sassafras", "🌳 Unable to submit tickets: {}", err); + }, } } } @@ -458,7 +481,7 @@ type SlotNotificationSinks = Arc< >; /// Parameters for Sassafras. -pub struct SassafrasParams { +pub struct SassafrasWorkerParams { /// The client to use pub client: Arc, /// The keystore that manages the keys of the node. @@ -477,15 +500,15 @@ pub struct SassafrasParams { pub justification_sync_link: L, /// Something that can create the inherent data providers. pub create_inherent_data_providers: CIDP, - /// Force authoring of blocks even if we are offline + /// Force authoring of blocks even if we are offline. pub force_authoring: bool, - /// The source of timestamps for relative slots + /// State shared between import queue and authoring worker. pub sassafras_link: SassafrasLink, } /// Start the Sassafras worker. pub fn start_sassafras( - SassafrasParams { + SassafrasWorkerParams { client, keystore, select_chain, @@ -496,14 +519,13 @@ pub fn start_sassafras( create_inherent_data_providers, force_authoring, sassafras_link, - }: SassafrasParams, + }: SassafrasWorkerParams, ) -> Result, sp_consensus::Error> where B: BlockT, C: ProvideRuntimeApi + ProvideUncles + BlockchainEvents - + PreCommitActions + HeaderBackend + HeaderMetadata + Send @@ -527,7 +549,7 @@ where let slot_notification_sinks = Arc::new(Mutex::new(Vec::new())); - let slot_worker = SassafrasSlotWorker { + let slot_worker = SlotWorker { client: client.clone(), block_import, env, @@ -548,7 +570,7 @@ where create_inherent_data_providers, ); - let tickets_worker = tickets_worker( + let tickets_worker = start_tickets_worker( client.clone(), keystore, sassafras_link.epoch_changes.clone(), diff --git a/client/consensus/sassafras/src/aux_schema.rs b/client/consensus/sassafras/src/aux_schema.rs index 07f723341b069..8c891ea0630f3 100644 --- a/client/consensus/sassafras/src/aux_schema.rs +++ b/client/consensus/sassafras/src/aux_schema.rs @@ -17,14 +17,20 @@ // along with this program. If not, see . //! Schema for auxiliary data persistence. +//! +//! TODO-SASS-P2 : RENAME FROM aux_schema.rs => aux_data.rs + +use std::{collections::HashSet, sync::Arc}; use scale_codec::{Decode, Encode}; use sc_client_api::backend::AuxStore; use sc_consensus_epochs::{EpochChangesFor, SharedEpochChanges}; -use sp_blockchain::{Error as ClientError, Result as ClientResult}; + +use sc_client_api::{blockchain::Backend as _, Backend as BackendT}; +use sp_blockchain::{Error as ClientError, HeaderBackend, HeaderMetadata, Result as ClientResult}; use sp_consensus_sassafras::SassafrasBlockWeight; -use sp_runtime::traits::Block as BlockT; +use sp_runtime::traits::{Block as BlockT, NumberFor, SaturatedConversion, Zero}; use crate::Epoch; @@ -99,3 +105,68 @@ pub fn load_block_weight( ) -> ClientResult> { load_decode(backend, block_weight_key(block_hash).as_slice()) } + +/// Reverts protocol aux data from the best block to at most the last finalized block. +/// +/// Epoch-changes and block weights announced after the revert point are removed. +pub fn revert(backend: Arc, blocks: NumberFor) -> ClientResult<()> +where + Block: BlockT, + Backend: BackendT, +{ + let blockchain = backend.blockchain(); + let best_number = blockchain.info().best_number; + let finalized = blockchain.info().finalized_number; + + let revertible = blocks.min(best_number - finalized); + if revertible == Zero::zero() { + return Ok(()) + } + + let revert_up_to_number = best_number - revertible; + let revert_up_to_hash = blockchain.hash(revert_up_to_number)?.ok_or(ClientError::Backend( + format!("Unexpected hash lookup failure for block number: {}", revert_up_to_number), + ))?; + + // Revert epoch changes tree. + + // This config is only used on-genesis. + let epoch_changes = load_epoch_changes::(&*backend)?; + let mut epoch_changes = epoch_changes.shared_data(); + + if revert_up_to_number == Zero::zero() { + // Special case, no epoch changes data were present on genesis. + *epoch_changes = EpochChangesFor::::new(); + } else { + let descendent_query = sc_consensus_epochs::descendent_query(blockchain); + epoch_changes.revert(descendent_query, revert_up_to_hash, revert_up_to_number); + } + + // Remove block weights added after the revert point. + + let mut weight_keys = HashSet::with_capacity(revertible.saturated_into()); + + let leaves = backend.blockchain().leaves()?.into_iter().filter(|&leaf| { + sp_blockchain::tree_route(blockchain, revert_up_to_hash, leaf) + .map(|route| route.retracted().is_empty()) + .unwrap_or_default() + }); + + for mut hash in leaves { + loop { + let meta = blockchain.header_metadata(hash)?; + if meta.number <= revert_up_to_number || !weight_keys.insert(block_weight_key(hash)) { + // We've reached the revert point or an already processed branch, stop here. + break + } + hash = meta.parent; + } + } + + let weight_keys: Vec<_> = weight_keys.iter().map(|val| val.as_slice()).collect(); + + // Write epoch changes and remove weights in one shot. + write_epoch_changes::(&epoch_changes, |values| { + AuxStore::insert_aux(&*backend, values, weight_keys.iter()) + }) +} diff --git a/client/consensus/sassafras/src/block_import.rs b/client/consensus/sassafras/src/block_import.rs index 555eac3f62638..01e804ecf3ea1 100644 --- a/client/consensus/sassafras/src/block_import.rs +++ b/client/consensus/sassafras/src/block_import.rs @@ -19,6 +19,7 @@ //! Types and functions related to block import. use super::*; +use sc_client_api::{AuxDataOperations, FinalityNotification, PreCommitActions}; /// A block-import handler for Sassafras. /// @@ -45,7 +46,26 @@ impl Clone for SassafrasBlockImport SassafrasBlockImport { +fn aux_storage_cleanup( + _client: &C, + _notification: &FinalityNotification, +) -> AuxDataOperations +where + B: BlockT, + C: HeaderMetadata + HeaderBackend, +{ + // TODO-SASS-P3 + Default::default() +} + +impl SassafrasBlockImport +where + C: AuxStore + + HeaderBackend + + HeaderMetadata + + PreCommitActions + + 'static, +{ /// Constructor. pub fn new( inner: I, @@ -53,6 +73,16 @@ impl SassafrasBlockImport { epoch_changes: SharedEpochChanges, genesis_config: SassafrasConfiguration, ) -> Self { + let client_weak = Arc::downgrade(&client); + let on_finality = move |notification: &FinalityNotification| { + if let Some(client) = client_weak.upgrade() { + aux_storage_cleanup(client.as_ref(), notification) + } else { + Default::default() + } + }; + client.register_finality_action(Box::new(on_finality)); + SassafrasBlockImport { inner, client, epoch_changes, genesis_config } } } @@ -82,9 +112,8 @@ where let hash = block.post_hash(); let number = *block.header.number(); - let pre_digest = find_pre_digest::(&block.header).expect( - "valid sassafras headers must contain a predigest; header has been already verified; qed", - ); + let pre_digest = find_pre_digest::(&block.header) + .expect("valid headers contain a pre-digest; header has been already verified; qed"); let slot = pre_digest.slot; let parent_hash = *block.header.parent_hash(); @@ -98,10 +127,9 @@ where ) })?; - let parent_slot = find_pre_digest::(&parent_header).map(|d| d.slot).expect( - "parent is non-genesis; valid Sassafras headers contain a pre-digest; \ - header has already been verified; qed", - ); + let parent_slot = find_pre_digest::(&parent_header) + .map(|d| d.slot) + .expect("parent is non-genesis; valid headers contain a pre-digest; header has been already verified; qed"); // Make sure that slot number is strictly increasing if slot <= parent_slot { @@ -161,30 +189,57 @@ where _ => (), } - let info = self.client.info(); - if let Some(next_epoch_descriptor) = next_epoch_digest { old_epoch_changes = Some((*epoch_changes).clone()); - let viable_epoch = epoch_changes + let mut viable_epoch = epoch_changes .viable_epoch(&epoch_descriptor, |slot| { Epoch::genesis(&self.genesis_config, slot) }) .ok_or_else(|| { ConsensusError::ClientImport(Error::::FetchEpoch(parent_hash).into()) - })?; + })? + .into_cloned(); + + if viable_epoch.as_ref().end_slot() <= slot { + // Some epochs must have been skipped as our current slot fits outside the + // current epoch. We will figure out which is the first skipped epoch and we + // will partially re-use its data for this "recovery" epoch. + let epoch_data = viable_epoch.as_mut(); + let skipped_epochs = + (*slot - *epoch_data.start_slot) / epoch_data.config.epoch_duration; + let original_epoch_idx = epoch_data.epoch_idx; + + // NOTE: notice that we are only updating a local copy of the `Epoch`, this + // makes it so that when we insert the next epoch into `EpochChanges` below + // (after incrementing it), it will use the correct epoch index and start slot. + // We do not update the original epoch that may be reused because there may be + // some other forks where the epoch isn't skipped. + // Not updating the original epoch works because when we search the tree for + // which epoch to use for a given slot, we will search in-depth with the + // predicate `epoch.start_slot <= slot` which will still match correctly without + // requiring to update `start_slot` to the correct value. + epoch_data.epoch_idx += skipped_epochs; + epoch_data.start_slot = Slot::from( + *epoch_data.start_slot + skipped_epochs * epoch_data.config.epoch_duration, + ); + log::warn!( + target: "sassafras", + "🌳 Epoch(s) skipped from {} to {}", + original_epoch_idx, epoch_data.epoch_idx + ); + } - // restrict info logging during initial sync to avoid spam - let log_level = if block.origin == BlockOrigin::NetworkInitialSync { - log::Level::Debug - } else { - log::Level::Info + // Restrict info logging during initial sync to avoid spam + let log_level = match block.origin { + BlockOrigin::NetworkInitialSync => log::Level::Debug, + _ => log::Level::Info, }; log!(target: "sassafras", log_level, "🌳 🍁 New epoch {} launching at block {} (block slot {} >= start slot {}).", - viable_epoch.as_ref().epoch_index, + viable_epoch.as_ref().epoch_idx, hash, slot, viable_epoch.as_ref().start_slot, @@ -246,18 +301,16 @@ where .extend(values.iter().map(|(k, v)| (k.to_vec(), Some(v.to_vec())))) }); - // The fork choice rule is that we pick the heaviest chain (i.e. - // more primary blocks), if there's a tie we go with the longest - // chain. + // The fork choice rule is that we pick the heaviest chain (i.e. more blocks built + // using primary mechanism), if there's a tie we go with the longest chain. block.fork_choice = { - let (last_best, last_best_number) = (info.best_hash, info.best_number); - - let last_best_weight = if &last_best == block.header.parent_hash() { + let info = self.client.info(); + let best_weight = if &info.best_hash == block.header.parent_hash() { // the parent=genesis case is already covered for loading parent weight, // so we don't need to cover again here. parent_weight } else { - aux_schema::load_block_weight(&*self.client, last_best) + aux_schema::load_block_weight(&*self.client, &info.best_hash) .map_err(|e| ConsensusError::ChainLookup(e.to_string()))? .ok_or_else(|| { ConsensusError::ChainLookup( @@ -266,13 +319,9 @@ where })? }; - Some(ForkChoiceStrategy::Custom(if total_weight > last_best_weight { - true - } else if total_weight == last_best_weight { - number > last_best_number - } else { - false - })) + let is_new_best = total_weight > best_weight || + (total_weight == best_weight && number > info.best_number); + Some(ForkChoiceStrategy::Custom(is_new_best)) }; // Release the mutex, but it stays locked epoch_changes.release_mutex() @@ -317,12 +366,10 @@ where let finalized_header = client .header(BlockId::Hash(info.finalized_hash)) .map_err(|e| ConsensusError::ClientImport(e.to_string()))? - .expect( - "best finalized hash was given by client; finalized headers must exist in db; qed", - ); + .expect("finalized headers must exist in db; qed"); find_pre_digest::(&finalized_header) - .expect("finalized header must be valid; valid blocks have a pre-digest; qed") + .expect("valid blocks have a pre-digest; qed") .slot }; @@ -342,14 +389,18 @@ where /// an import-queue. /// /// Also returns a link object used to correctly instantiate the import queue -/// and background worker. +/// and authoring worker. pub fn block_import( genesis_config: SassafrasConfiguration, inner_block_import: I, client: Arc, ) -> ClientResult<(SassafrasBlockImport, SassafrasLink)> where - C: AuxStore + HeaderBackend + HeaderMetadata + 'static, + C: AuxStore + + HeaderBackend + + HeaderMetadata + + PreCommitActions + + 'static, { let epoch_changes = aux_schema::load_epoch_changes::(&*client)?; diff --git a/client/consensus/sassafras/src/lib.rs b/client/consensus/sassafras/src/lib.rs index d2ad9f274c9b0..f5134e38266f7 100644 --- a/client/consensus/sassafras/src/lib.rs +++ b/client/consensus/sassafras/src/lib.rs @@ -20,7 +20,8 @@ //! //! TODO-SASS-P2: documentation -#![deny(warnings)] +// TODO-SASS-P2: remove this +//#![deny(warnings)] #![forbid(unsafe_code, missing_docs)] use std::{ @@ -41,9 +42,7 @@ use prometheus_endpoint::Registry; use scale_codec::{Decode, Encode}; use schnorrkel::SignatureError; -use sc_client_api::{ - backend::AuxStore, BlockchainEvents, PreCommitActions, ProvideUncles, UsageProvider, -}; +use sc_client_api::{backend::AuxStore, BlockchainEvents, ProvideUncles, UsageProvider}; use sc_consensus::{ block_import::{ BlockCheckParams, BlockImport, BlockImportParams, ForkChoiceStrategy, ImportResult, @@ -56,9 +55,7 @@ use sc_consensus_epochs::{ descendent_query, Epoch as EpochT, EpochChangesFor, EpochIdentifier, EpochIdentifierPosition, SharedEpochChanges, ViableEpochDescriptor, }; -use sc_consensus_slots::{ - check_equivocation, CheckedHeader, InherentDataProviderExt, SlotInfo, StorageChanges, -}; +use sc_consensus_slots::{CheckedHeader, InherentDataProviderExt, SlotInfo, StorageChanges}; use sc_telemetry::{telemetry, TelemetryHandle, CONSENSUS_DEBUG, CONSENSUS_TRACE}; use sp_api::{ApiExt, ProvideRuntimeApi}; use sp_application_crypto::AppKey; @@ -91,9 +88,13 @@ pub use sp_consensus_sassafras::{ mod authorship; mod aux_schema; mod block_import; +#[cfg(test)] +mod tests; mod verification; -pub use authorship::{start_sassafras, SassafrasParams, SassafrasWorker}; +// Export core components. +pub use authorship::{start_sassafras, SassafrasWorker, SassafrasWorkerParams}; +pub use aux_schema::revert; pub use block_import::{block_import, SassafrasBlockImport}; pub use verification::SassafrasVerifier; @@ -187,7 +188,7 @@ fn sassafras_err(error: Error) -> Error { #[derive(Encode, Decode, PartialEq, Eq, Clone, Debug)] pub struct Epoch { /// The epoch index. - pub epoch_index: u64, + pub epoch_idx: u64, /// The starting slot of the epoch. pub start_slot: Slot, /// Epoch configuration @@ -209,7 +210,7 @@ impl EpochT for Epoch { threshold_params: descriptor.config.unwrap_or(self.config.threshold_params.clone()), }; Epoch { - epoch_index: self.epoch_index + 1, + epoch_idx: self.epoch_idx + 1, start_slot: self.start_slot + config.epoch_duration, config, tickets_aux: BTreeMap::new(), @@ -221,7 +222,7 @@ impl EpochT for Epoch { } fn end_slot(&self) -> Slot { - self.start_slot + self.config.slot_duration + self.start_slot + self.config.epoch_duration } } @@ -230,7 +231,7 @@ impl Epoch { /// the first block, so that has to be provided. pub fn genesis(config: &SassafrasConfiguration, slot: Slot) -> Epoch { Epoch { - epoch_index: 0, + epoch_idx: 0, start_slot: slot, config: config.clone(), tickets_aux: BTreeMap::new(), diff --git a/client/consensus/sassafras/src/tests.rs b/client/consensus/sassafras/src/tests.rs new file mode 100644 index 0000000000000..5cf0bef6795e8 --- /dev/null +++ b/client/consensus/sassafras/src/tests.rs @@ -0,0 +1,972 @@ +// This file is part of Substrate. + +// Copyright (C) 2022 Parity Technologies (UK) Ltd. +// SPDX-License-Identifier: GPL-3.0-or-later WITH Classpath-exception-2.0 + +// This program is free software: you can redistribute it and/or modify +// it under the terms of the GNU General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. + +// This program is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU General Public License for more details. + +// You should have received a copy of the GNU General Public License +// along with this program. If not, see . + +//! Sassafras client tests + +// TODO-SASS-P2 +// Missing interesting tests: +// - verify block claimed via primary method + +use super::*; + +use futures::executor::block_on; +use std::sync::Arc; + +use sc_block_builder::BlockBuilderProvider; +use sc_client_api::Finalizer; +use sc_consensus::{BlockImport, BoxJustificationImport}; +use sc_network_test::*; +use sp_application_crypto::key_types::SASSAFRAS; +use sp_blockchain::Error as TestError; +use sp_consensus::{DisableProofRecording, NoNetwork as DummyOracle, Proposal}; +use sp_consensus_sassafras::{inherents::InherentDataProvider, vrf::make_slot_transcript_data}; +use sp_keyring::Sr25519Keyring; +use sp_keystore::testing::KeyStore as TestKeyStore; +use sp_runtime::{Digest, DigestItem}; +use sp_timestamp::Timestamp; + +use substrate_test_runtime_client::{runtime::Block as TestBlock, Backend as TestBackend}; + +// Monomorphization of generic structures for the test context. + +type BlockId = crate::BlockId; + +type TestHeader = ::Header; + +type TestClient = substrate_test_runtime_client::client::Client< + TestBackend, + substrate_test_runtime_client::ExecutorDispatch, + TestBlock, + substrate_test_runtime_client::runtime::RuntimeApi, +>; + +type TestSelectChain = + substrate_test_runtime_client::LongestChain; + +type TestTransaction = + sc_client_api::TransactionFor; + +type TestBlockImportParams = BlockImportParams; + +type TestViableEpochDescriptor = sc_consensus_epochs::ViableEpochDescriptor; + +// Monomorphization of Sassafras structures for the test context. + +type SassafrasIntermediate = crate::SassafrasIntermediate; + +type SassafrasBlockImport = crate::SassafrasBlockImport>; + +type SassafrasVerifier = crate::SassafrasVerifier< + TestBlock, + PeersFullClient, + TestSelectChain, + Box< + dyn CreateInherentDataProviders< + TestBlock, + (), + InherentDataProviders = (InherentDataProvider,), + >, + >, +>; + +type SassafrasLink = crate::SassafrasLink; + +// Epoch duration is slots +const EPOCH_DURATION: u64 = 6; +// Slot duration is milliseconds +const SLOT_DURATION: u64 = 1000; + +struct TestProposer { + client: Arc, + link: SassafrasLink, + parent_hash: Hash, + parent_number: u64, + parent_slot: Slot, +} + +impl TestProposer { + fn propose_block(self, digest: Digest) -> TestBlock { + block_on(self.propose(InherentData::default(), digest, Duration::default(), None)) + .expect("Proposing block") + .block + } +} + +impl Proposer for TestProposer { + type Error = TestError; + type Transaction = TestTransaction; + type Proposal = future::Ready, Self::Error>>; + type ProofRecording = DisableProofRecording; + type Proof = (); + + fn propose( + self, + _: InherentData, + inherent_digests: Digest, + _: Duration, + _: Option, + ) -> Self::Proposal { + let block_builder = self + .client + .new_block_at(&BlockId::Hash(self.parent_hash), inherent_digests, false) + .unwrap(); + + let mut block = match block_builder.build().map_err(|e| e.into()) { + Ok(b) => b.block, + Err(e) => return future::ready(Err(e)), + }; + + // Currently the test runtime doesn't invoke each pallet Hooks such as `on_initialize` and + // `on_finalize`. Thus we have to manually figure out if we should add a consensus digest. + + let this_slot = crate::find_pre_digest::(block.header()) + .expect("baked block has valid pre-digest") + .slot; + + let epoch_changes = self.link.epoch_changes.shared_data(); + let epoch = epoch_changes + .epoch_data_for_child_of( + descendent_query(&*self.client), + &self.parent_hash, + self.parent_number, + this_slot, + |slot| Epoch::genesis(&self.link.genesis_config, slot), + ) + .expect("client has data to find epoch") + .expect("can compute epoch for baked block"); + + let first_in_epoch = self.parent_slot < epoch.start_slot; + if first_in_epoch { + // push a `Consensus` digest signalling next change. + // we just reuse the same randomness and authorities as the prior + // epoch. this will break when we add light client support, since + // that will re-check the randomness logic off-chain. + let digest_data = ConsensusLog::NextEpochData(NextEpochDescriptor { + authorities: epoch.config.authorities.clone(), + randomness: epoch.config.randomness, + config: None, + }) + .encode(); + let digest = DigestItem::Consensus(SASSAFRAS_ENGINE_ID, digest_data); + block.header.digest_mut().push(digest) + } + + future::ready(Ok(Proposal { block, proof: (), storage_changes: Default::default() })) + } +} + +struct TestContext { + client: Arc, + backend: Arc, + link: SassafrasLink, + block_import: SassafrasBlockImport, + verifier: SassafrasVerifier, +} + +fn create_test_verifier( + client: Arc, + link: &SassafrasLink, + config: SassafrasConfiguration, +) -> SassafrasVerifier { + let slot_duration = config.slot_duration(); + + let create_inherent_data_providers = Box::new(move |_, _| async move { + let slot = InherentDataProvider::from_timestamp_and_slot_duration( + Timestamp::current(), + slot_duration, + ); + Ok((slot,)) + }); + + let (_, longest_chain) = TestClientBuilder::with_default_backend().build_with_longest_chain(); + + SassafrasVerifier::new( + client.clone(), + longest_chain, + create_inherent_data_providers, + link.epoch_changes.clone(), + None, + config, + ) +} + +fn create_test_block_import( + client: Arc, + config: SassafrasConfiguration, +) -> (SassafrasBlockImport, SassafrasLink) { + crate::block_import(config, client.clone(), client.clone()) + .expect("can initialize block-import") +} + +impl TestContext { + fn new() -> Self { + let (client, backend) = TestClientBuilder::with_default_backend().build_with_backend(); + let client = Arc::new(client); + + // Note: configuration is loaded using the `TestClient` instance as the runtime-api + // provider. In practice this will use the values defined within the test runtime + // defined in the `substrate_test_runtime` crate. + let config = crate::configuration(&*client).expect("config available"); + + let (block_import, link) = create_test_block_import(client.clone(), config.clone()); + + let verifier = create_test_verifier(client.clone(), &link, config.clone()); + + Self { client, backend, link, block_import, verifier } + } + + // This is a bit hacky solution to use `TestContext` as an `Environment` implementation + fn new_with_pre_built_data( + client: Arc, + backend: Arc, + link: SassafrasLink, + block_import: SassafrasBlockImport, + ) -> Self { + let verifier = create_test_verifier(client.clone(), &link, link.genesis_config.clone()); + Self { client, backend, link, block_import, verifier } + } + + fn import_block(&mut self, mut params: TestBlockImportParams) -> Hash { + let post_hash = params.post_hash(); + + if params.post_digests.is_empty() { + // Assume that the seal has not been removed yet. Remove it here... + // NOTE: digest may be empty because of some test intentionally clearing up + // the whole digest logs. + if let Some(seal) = params.header.digest_mut().pop() { + params.post_digests.push(seal); + } + } + + match block_on(self.block_import.import_block(params, Default::default())).unwrap() { + ImportResult::Imported(_) => (), + _ => panic!("expected block to be imported"), + } + + post_hash + } + + fn verify_block(&mut self, params: TestBlockImportParams) -> TestBlockImportParams { + let tmp_params = params.clear_storage_changes_and_mutate(); + let (tmp_params, _) = block_on(self.verifier.verify(tmp_params)).unwrap(); + tmp_params.clear_storage_changes_and_mutate() + } + + fn epoch_data(&self, parent_hash: &Hash, parent_number: u64, slot: Slot) -> Epoch { + self.link + .epoch_changes + .shared_data() + .epoch_data_for_child_of( + descendent_query(&*self.client), + &parent_hash, + parent_number, + slot, + |slot| Epoch::genesis(&self.link.genesis_config, slot), + ) + .unwrap() + .unwrap() + } + + fn epoch_descriptor( + &self, + parent_hash: &Hash, + parent_number: u64, + slot: Slot, + ) -> TestViableEpochDescriptor { + self.link + .epoch_changes + .shared_data() + .epoch_descriptor_for_child_of( + descendent_query(&*self.client), + &parent_hash, + parent_number, + slot, + ) + .unwrap() + .unwrap() + } + + // Propose a block + fn propose_block(&mut self, parent_id: BlockId, slot: Option) -> TestBlockImportParams { + let parent = self.client.header(&parent_id).unwrap().unwrap(); + let parent_hash = parent.hash(); + let parent_number = *parent.number(); + + let authority = Sr25519Keyring::Alice; + let keystore = create_keystore(authority); + + let proposer = block_on(self.init(&parent)).unwrap(); + + let slot = slot.unwrap_or_else(|| { + let parent_pre_digest = find_pre_digest::(&parent).unwrap(); + parent_pre_digest.slot + 1 + }); + + let epoch = self.epoch_data(&parent_hash, parent_number, slot); + let transcript_data = + make_slot_transcript_data(&self.link.genesis_config.randomness, slot, epoch.epoch_idx); + let signature = SyncCryptoStore::sr25519_vrf_sign( + &*keystore, + SASSAFRAS, + &authority.public(), + transcript_data, + ) + .unwrap() + .unwrap(); + + let pre_digest = PreDigest { + slot, + authority_idx: 0, + vrf_output: VRFOutput(signature.output), + vrf_proof: VRFProof(signature.proof), + ticket_aux: None, + }; + let digest = sp_runtime::generic::Digest { + logs: vec![DigestItem::sassafras_pre_digest(pre_digest)], + }; + + let mut block = proposer.propose_block(digest); + + let epoch_descriptor = self.epoch_descriptor(&parent_hash, parent_number, slot); + + // Sign the pre-sealed hash of the block and then add it to a digest item. + let hash = block.header.hash(); + let public_type_pair = authority.public().into(); + let signature = + SyncCryptoStore::sign_with(&*keystore, SASSAFRAS, &public_type_pair, hash.as_ref()) + .unwrap() + .unwrap() + .try_into() + .unwrap(); + let seal = DigestItem::sassafras_seal(signature); + block.header.digest_mut().push(seal); + + let mut params = BlockImportParams::new(BlockOrigin::Own, block.header); + params.fork_choice = Some(ForkChoiceStrategy::LongestChain); + params.body = Some(block.extrinsics); + params.insert_intermediate(INTERMEDIATE_KEY, SassafrasIntermediate { epoch_descriptor }); + + params + } + + // Propose and import a new block on top of the given parent. + // This skips verification. + fn propose_and_import_block(&mut self, parent_id: BlockId, slot: Option) -> Hash { + let params = self.propose_block(parent_id, slot); + self.import_block(params) + } + + // Propose and import n valid 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 by the test runtime). + fn propose_and_import_blocks(&mut self, mut parent_id: BlockId, n: usize) -> Vec { + let mut hashes = Vec::with_capacity(n); + + for _ in 0..n { + let hash = self.propose_and_import_block(parent_id, None); + hashes.push(hash); + parent_id = BlockId::Hash(hash); + } + + hashes + } +} + +fn create_keystore(authority: Sr25519Keyring) -> SyncCryptoStorePtr { + let keystore = Arc::new(TestKeyStore::new()); + SyncCryptoStore::sr25519_generate_new(&*keystore, SASSAFRAS, Some(&authority.to_seed())) + .expect("Creates authority key"); + keystore +} + +#[test] +fn tests_assumptions_sanity_check() { + let env = TestContext::new(); + let config = env.link.genesis_config; + + // Check that genesis configuration read from test runtime has the expected values + assert_eq!( + config.authorities, + vec![ + (Sr25519Keyring::Alice.public().into(), 1), + (Sr25519Keyring::Bob.public().into(), 1), + (Sr25519Keyring::Charlie.public().into(), 1), + ] + ); + assert_eq!(config.epoch_duration, EPOCH_DURATION); + assert_eq!(config.slot_duration, SLOT_DURATION); + assert_eq!(config.randomness, [0; 32]); + // TODO-SASS-P3: check threshold params +} + +#[test] +fn claim_secondary_slots_works() { + let env = TestContext::new(); + let mut config = env.link.genesis_config.clone(); + config.randomness = [2; 32]; + + let authorities = [Sr25519Keyring::Alice, Sr25519Keyring::Bob, Sr25519Keyring::Charlie]; + + let epoch = Epoch { + epoch_idx: 1, + start_slot: 6.into(), + config: config.clone(), + tickets_aux: Default::default(), + }; + + let mut assignments = vec![usize::MAX; config.epoch_duration as usize]; + + for (auth_idx, auth_id) in authorities.iter().enumerate() { + let keystore = create_keystore(*auth_id); + + for slot in 0..config.epoch_duration { + if let Some((claim, auth_id2)) = + authorship::claim_slot(slot.into(), &epoch, None, &keystore) + { + assert_eq!(claim.authority_idx as usize, auth_idx); + assert_eq!(claim.slot, Slot::from(slot)); + assert_eq!(claim.ticket_aux, None); + assert_eq!(auth_id.public(), auth_id2.into()); + + // Check that this slot has not been assigned before + assert_eq!(assignments[slot as usize], usize::MAX); + assignments[slot as usize] = auth_idx; + } + } + } + // Check that every slot has been assigned + assert!(assignments.iter().all(|v| *v != usize::MAX)); + println!("secondary slots assignments: {:?}", assignments); +} + +#[test] +fn claim_primary_slots_works() { + // Here the test is very deterministic. + // If a node has in its epoch `tickets_aux` the information corresponding to the + // ticket that is presented. Then the claim ticket should just return the + // ticket auxiliary information. + let env = TestContext::new(); + let mut config = env.link.genesis_config.clone(); + config.randomness = [2; 32]; + + let mut epoch = Epoch { + epoch_idx: 1, + start_slot: 6.into(), + config: config.clone(), + tickets_aux: Default::default(), + }; + + let keystore = create_keystore(Sr25519Keyring::Alice); + + // Success if we have ticket data and the key in our keystore + + let authority_idx = 0u32; + let ticket: Ticket = [0u8; 32].try_into().unwrap(); + let ticket_proof: VRFProof = [0u8; 64].try_into().unwrap(); + let ticket_aux = TicketAux { attempt: 0, proof: ticket_proof }; + epoch.tickets_aux.insert(ticket, (authority_idx, ticket_aux)); + + let (pre_digest, auth_id) = + authorship::claim_slot(0.into(), &epoch, Some(ticket), &keystore).unwrap(); + + assert_eq!(pre_digest.authority_idx, authority_idx); + assert_eq!(auth_id, Sr25519Keyring::Alice.public().into()); + + // Fail if we don't have aux data for some ticket + + let ticket: Ticket = [1u8; 32].try_into().unwrap(); + let claim = authorship::claim_slot(0.into(), &epoch, Some(ticket), &keystore); + assert!(claim.is_none()); + + // Fail if we don't have the key for the ticket owner in our keystore + // (even though we have associated data, it doesn't matter) + + let authority_idx = 1u32; + let ticket_proof: VRFProof = [0u8; 64].try_into().unwrap(); + let ticket_aux = TicketAux { attempt: 0, proof: ticket_proof }; + epoch.tickets_aux.insert(ticket, (authority_idx, ticket_aux)); + let claim = authorship::claim_slot(0.into(), &epoch, Some(ticket), &keystore); + assert!(claim.is_none()); +} + +#[test] +#[should_panic(expected = "valid headers contain a pre-digest")] +fn import_rejects_block_without_pre_digest() { + let mut env = TestContext::new(); + + let mut import_params = env.propose_block(BlockId::Number(0), Some(999.into())); + // Remove logs from the header + import_params.header.digest_mut().logs.clear(); + + env.import_block(import_params); +} + +#[test] +#[should_panic(expected = "Unexpected epoch change")] +fn import_rejects_block_with_unexpected_epoch_changes() { + let mut env = TestContext::new(); + + env.propose_and_import_block(BlockId::Number(0), None); + + let mut import_params = env.propose_block(BlockId::Number(1), None); + // Insert an epoch change announcement when it is not required. + let digest_data = ConsensusLog::NextEpochData(NextEpochDescriptor { + authorities: env.link.genesis_config.authorities.clone(), + randomness: env.link.genesis_config.randomness, + config: None, + }) + .encode(); + let digest_item = DigestItem::Consensus(SASSAFRAS_ENGINE_ID, digest_data); + let digest = import_params.header.digest_mut(); + digest.logs.insert(digest.logs.len() - 1, digest_item); + + env.import_block(import_params); +} + +#[test] +#[should_panic(expected = "Expected epoch change to happen")] +fn import_rejects_block_with_missing_epoch_changes() { + let mut env = TestContext::new(); + + let blocks = env.propose_and_import_blocks(BlockId::Number(0), EPOCH_DURATION as usize); + + let mut import_params = + env.propose_block(BlockId::Hash(blocks[EPOCH_DURATION as usize - 1]), None); + + let digest = import_params.header.digest_mut(); + // Remove the epoch change announcement. + // (Implementation detail: should be the second to last entry, just before the seal) + digest.logs.remove(digest.logs.len() - 2); + + env.import_block(import_params); +} + +#[test] +fn importing_block_one_sets_genesis_epoch() { + let mut env = TestContext::new(); + + let block_hash = env.propose_and_import_block(BlockId::Number(0), Some(999.into())); + + let epoch_for_second_block = env.epoch_data(&block_hash, 1, 1000.into()); + let genesis_epoch = Epoch::genesis(&env.link.genesis_config, 999.into()); + assert_eq!(epoch_for_second_block, genesis_epoch); +} + +#[test] +fn allows_to_skip_epochs() { + // Test scenario. + // Epoch lenght: 6 slots + // + // Block# : [ 1 2 3 4 5 6 ][ 7 - - - - - ][ - - - - - - ][ 8 ... ] + // Slot# : [ 1 2 3 4 5 6 ][ 7 8 9 10 11 12 ][ 13 14 15 16 17 18 ][ 19 ... ] + // Epoch# : [ 0 ][ 1 ][ skipped ][ 3 ] + // + // As a recovery strategy, a fallback epoch 3 is created by reusing part of the + // configuration created for epoch 2. + let mut env = TestContext::new(); + + let blocks = env.propose_and_import_blocks(BlockId::Number(0), 7); + + // First block after the a skipped epoch (block #8 @ slot #19) + let block = + env.propose_and_import_block(BlockId::Hash(*blocks.last().unwrap()), Some(19.into())); + + let epoch_changes = env.link.epoch_changes.shared_data(); + let epochs: Vec<_> = epoch_changes.tree().iter().collect(); + assert_eq!(epochs.len(), 3); + assert_eq!(*epochs[0].0, blocks[0]); + assert_eq!(*epochs[0].1, 1); + assert_eq!(*epochs[1].0, blocks[6]); + assert_eq!(*epochs[1].1, 7); + assert_eq!(*epochs[2].0, block); + assert_eq!(*epochs[2].1, 8); + + // Fist block in E0 (B1)) announces E0 (this is special) + let data = epoch_changes + .epoch(&EpochIdentifier { + position: EpochIdentifierPosition::Genesis0, + hash: blocks[0], + number: 1, + }) + .unwrap(); + assert_eq!(data.epoch_idx, 0); + assert_eq!(data.start_slot, Slot::from(1)); + + // First block in E0 (B1) also announces E1 + let data = epoch_changes + .epoch(&EpochIdentifier { + position: EpochIdentifierPosition::Genesis1, + hash: blocks[0], + number: 1, + }) + .unwrap(); + assert_eq!(data.epoch_idx, 1); + assert_eq!(data.start_slot, Slot::from(7)); + + // First block in E1 (B7) announces E2 + // NOTE: config is used by E3 without altering epoch node values. + // This will break as soon as our assumptions about how fork-tree traversal works + // are not met anymore (this is a good thing) + let data = epoch_changes + .epoch(&EpochIdentifier { + position: EpochIdentifierPosition::Regular, + hash: blocks[6], + number: 7, + }) + .unwrap(); + assert_eq!(data.epoch_idx, 2); + assert_eq!(data.start_slot, Slot::from(13)); + + // First block in E3 (B8) announced E4. + let data = epoch_changes + .epoch(&EpochIdentifier { + position: EpochIdentifierPosition::Regular, + hash: block, + number: 8, + }) + .unwrap(); + assert_eq!(data.epoch_idx, 4); + assert_eq!(data.start_slot, Slot::from(25)); +} + +#[test] +fn finalization_prunes_epoch_changes_and_removes_weights() { + let mut env = TestContext::new(); + + let canon = env.propose_and_import_blocks(BlockId::Number(0), 21); + + let _fork1 = env.propose_and_import_blocks(BlockId::Hash(canon[0]), 10); + let _fork2 = env.propose_and_import_blocks(BlockId::Hash(canon[7]), 10); + let _fork3 = env.propose_and_import_blocks(BlockId::Hash(canon[11]), 8); + + let epoch_changes = env.link.epoch_changes.clone(); + + // We should be tracking a total of 9 epochs in the fork tree + assert_eq!(epoch_changes.shared_data().tree().iter().count(), 8); + // And only one root + assert_eq!(epoch_changes.shared_data().tree().roots().count(), 1); + + // Pre-finalize scenario. + // + // X(#y): a block (number y) announcing the next epoch data. + // Information for epoch starting at block #19 is produced on three different forks + // at block #13. + // + // Finalize block #14 + // + // *---------------- F(#13) --#18 < fork #2 + // / + // A(#1) ---- B(#7) ----#8----------#12---- C(#13) ---- D(#19) ------#21 < canon + // \ \ + // \ *---- G(#13) ---- H(#19) ---#20 < fork #3 + // \ + // *-----E(#7)---#11 < fork #1 + + // Finalize block #10 so that on next epoch change the tree is pruned + env.client.finalize_block(BlockId::Hash(canon[13]), None, true).unwrap(); + let canon_cont = env.propose_and_import_blocks(BlockId::Hash(*canon.last().unwrap()), 4); + + // Post-finalize scenario. + // + // B(#7)------ C(#13) ---- D(#19) ------Z(#25) + + let epoch_changes = epoch_changes.shared_data(); + let epoch_changes: Vec<_> = epoch_changes.tree().iter().map(|(h, _, _)| *h).collect(); + + // TODO-SASS-P2: this is fixed by a pending PR on substrate + //assert_eq!(epoch_changes, vec![canon[6], canon[12], canon[18], canon_cont[3]]); + + // TODO-SASS-P2 + //todo!("Requires aux_storage_cleanup"); +} + +#[test] +fn revert_prunes_epoch_changes_and_removes_weights() { + let mut env = TestContext::new(); + + let canon = env.propose_and_import_blocks(BlockId::Number(0), 21); + let fork1 = env.propose_and_import_blocks(BlockId::Hash(canon[0]), 10); + let fork2 = env.propose_and_import_blocks(BlockId::Hash(canon[7]), 10); + let fork3 = env.propose_and_import_blocks(BlockId::Hash(canon[11]), 8); + + let epoch_changes = env.link.epoch_changes.clone(); + + // We should be tracking a total of 9 epochs in the fork tree + assert_eq!(epoch_changes.shared_data().tree().iter().count(), 8); + // And only one root + assert_eq!(epoch_changes.shared_data().tree().roots().count(), 1); + + // Pre-revert scenario. + // + // X(#y): a block (number y) announcing the next epoch data. + // Information for epoch starting at block #19 is produced on three different forks + // at block #13. + // One branch starts before the revert point (epoch data should be maintained). + // One branch starts after the revert point (epoch data should be removed). + // + // *----------------- F(#13) --#18 < fork #2 + // / + // A(#1) ---- B(#7) ----#8----+-----#12----- C(#13) ---- D(#19) ------#21 < canon + // \ ^ \ + // \ revert *---- G(#13) ---- H(#19) ---#20 < fork #3 + // \ to #10 + // *-----E(#7)---#11 < fork #1 + + // Revert canon chain to block #10 (best(21) - 11) + crate::revert(env.backend.clone(), 11).unwrap(); + + // Post-revert expected scenario. + // + // + // *----------------- F(#13) --#18 + // / + // A(#1) ---- B(#7) ----#8----#10 + // \ + // *------ E(#7)---#11 + + // Load and check epoch changes. + + let actual_nodes = aux_schema::load_epoch_changes::(&*env.client) + .unwrap() + .shared_data() + .tree() + .iter() + .map(|(h, _, _)| *h) + .collect::>(); + + let expected_nodes = vec![ + canon[0], // A + canon[6], // B + fork2[4], // F + fork1[5], // E + ]; + + assert_eq!(actual_nodes, expected_nodes); + + let weight_data_check = |hashes: &[Hash], expected: bool| { + hashes.iter().all(|hash| { + aux_schema::load_block_weight(&*env.client, hash).unwrap().is_some() == expected + }) + }; + assert!(weight_data_check(&canon[..10], true)); + assert!(weight_data_check(&canon[10..], false)); + assert!(weight_data_check(&fork1, true)); + assert!(weight_data_check(&fork2, true)); + assert!(weight_data_check(&fork3, false)); +} + +#[test] +fn revert_not_allowed_for_finalized() { + let mut env = TestContext::new(); + + let canon = env.propose_and_import_blocks(BlockId::Number(0), 3); + + // Finalize best block + env.client.finalize_block(BlockId::Hash(canon[2]), None, false).unwrap(); + + // Revert canon chain to last finalized block + crate::revert(env.backend.clone(), 100).expect("revert should work for baked test scenario"); + + let weight_data_check = |hashes: &[Hash], expected: bool| { + hashes.iter().all(|hash| { + aux_schema::load_block_weight(&*env.client, hash).unwrap().is_some() == expected + }) + }; + assert!(weight_data_check(&canon, true)); +} + +#[test] +fn verify_block_claimed_via_secondary_method() { + let mut env = TestContext::new(); + + let blocks = env.propose_and_import_blocks(BlockId::Number(0), 7); + + let in_params = env.propose_block(BlockId::Hash(blocks[6]), Some(9.into())); + + let _out_params = env.verify_block(in_params); +} + +//================================================================================================= +// More complex tests involving communication between multiple nodes. +// +// These tests are performed via a specially crafted test network. +//================================================================================================= + +impl Environment for TestContext { + type CreateProposer = future::Ready>; + type Proposer = TestProposer; + type Error = TestError; + + fn init(&mut self, parent_header: &TestHeader) -> Self::CreateProposer { + let parent_slot = crate::find_pre_digest::(parent_header) + .expect("parent header has a pre-digest") + .slot; + + future::ready(Ok(TestProposer { + client: self.client.clone(), + link: self.link.clone(), + parent_hash: parent_header.hash(), + parent_number: *parent_header.number(), + parent_slot, + })) + } +} + +struct PeerData { + link: SassafrasLink, + block_import: SassafrasBlockImport, +} + +type SassafrasPeer = Peer, SassafrasBlockImport>; + +#[derive(Default)] +struct SassafrasTestNet { + peers: Vec, +} + +impl TestNetFactory for SassafrasTestNet { + type BlockImport = SassafrasBlockImport; + type Verifier = SassafrasVerifier; + type PeerData = Option; + + fn make_block_import( + &self, + client: PeersClient, + ) -> ( + BlockImportAdapter, + Option>, + Option, + ) { + let client = client.as_client(); + + let config = crate::configuration(&*client).expect("config available"); + let (block_import, link) = create_test_block_import(client.clone(), config); + + (BlockImportAdapter::new(block_import.clone()), None, Some(PeerData { link, block_import })) + } + + fn make_verifier(&self, client: PeersClient, maybe_link: &Option) -> Self::Verifier { + let client = client.as_client(); + + let data = maybe_link.as_ref().expect("data provided to verifier instantiation"); + + let config = crate::configuration(&*client).expect("config available"); + create_test_verifier(client.clone(), &data.link, config) + } + + fn peer(&mut self, i: usize) -> &mut SassafrasPeer { + &mut self.peers[i] + } + + fn peers(&self) -> &Vec { + &self.peers + } + + fn mut_peers)>(&mut self, closure: F) { + closure(&mut self.peers); + } +} + +// Multiple nodes authoring and validating blocks +#[test] +fn sassafras_network_progress() { + let net = SassafrasTestNet::new(3); + let net = Arc::new(Mutex::new(net)); + + let peers = [Sr25519Keyring::Alice, Sr25519Keyring::Bob, Sr25519Keyring::Charlie]; + + let mut import_notifications = Vec::new(); + let mut sassafras_workers = Vec::new(); + + for (peer_id, auth_id) in peers.iter().enumerate() { + let mut net = net.lock(); + let peer = net.peer(peer_id); + let client = peer.client().as_client(); + let backend = peer.client().as_backend(); + let select_chain = peer.select_chain().expect("Full client has select_chain"); + + let keystore = create_keystore(*auth_id); + + let data = peer.data.as_ref().expect("sassafras link set up during initialization"); + + let env = TestContext::new_with_pre_built_data( + client.clone(), + backend.clone(), + data.link.clone(), + data.block_import.clone(), + ); + + // Run the imported block number is less than five and we don't receive a block produced + // by us and one produced by another peer. + let mut got_own = false; + let mut got_other = false; + let import_futures = client + .import_notification_stream() + .take_while(move |n| { + future::ready( + n.header.number() < &5 || { + if n.origin == BlockOrigin::Own { + got_own = true; + } else { + got_other = true; + } + !(got_own && got_other) + }, + ) + }) + .for_each(|_| future::ready(())); + import_notifications.push(import_futures); + + let slot_duration = data.link.genesis_config.slot_duration(); + let create_inherent_data_providers = Box::new(move |_, _| async move { + let slot = InherentDataProvider::from_timestamp_and_slot_duration( + Timestamp::current(), + slot_duration, + ); + Ok((slot,)) + }); + let sassafras_params = SassafrasWorkerParams { + client: client.clone(), + keystore, + select_chain, + env, + block_import: data.block_import.clone(), + sassafras_link: data.link.clone(), + sync_oracle: DummyOracle, + justification_sync_link: (), + force_authoring: false, + create_inherent_data_providers, + }; + let sassafras_worker = start_sassafras(sassafras_params).unwrap(); + sassafras_workers.push(sassafras_worker); + } + + block_on(future::select( + futures::future::poll_fn(move |cx| { + let mut net = net.lock(); + net.poll(cx); + net.peers().iter().for_each(|peer| { + peer.failed_verifications().iter().next().map(|(h, e)| { + panic!("Verification failed for {:?}: {}", h, e); + }); + }); + Poll::<()>::Pending + }), + future::select(future::join_all(import_notifications), future::join_all(sassafras_workers)), + )); +} diff --git a/client/consensus/sassafras/src/verification.rs b/client/consensus/sassafras/src/verification.rs index 1020df4cd6688..a7f6707565d0b 100644 --- a/client/consensus/sassafras/src/verification.rs +++ b/client/consensus/sassafras/src/verification.rs @@ -90,7 +90,7 @@ pub fn check_header( (Some(ticket), Some(ticket_aux)) => { log::debug!(target: "sassafras", "🌳 checking primary"); let transcript = - make_ticket_transcript(&config.randomness, ticket_aux.attempt, epoch.epoch_index); + make_ticket_transcript(&config.randomness, ticket_aux.attempt, epoch.epoch_idx); schnorrkel::PublicKey::from_bytes(authority_id.as_slice()) .and_then(|p| p.vrf_verify(transcript, &ticket, &ticket_aux.proof)) .map_err(|s| sassafras_err(Error::VRFVerificationFailed(s)))?; @@ -115,7 +115,7 @@ pub fn check_header( // Check slot-vrf proof - let transcript = make_slot_transcript(&config.randomness, pre_digest.slot, epoch.epoch_index); + let transcript = make_slot_transcript(&config.randomness, pre_digest.slot, epoch.epoch_idx); schnorrkel::PublicKey::from_bytes(authority_id.as_slice()) .and_then(|p| p.vrf_verify(transcript, &pre_digest.vrf_output, &pre_digest.vrf_proof)) .map_err(|s| sassafras_err(Error::VRFVerificationFailed(s)))?; @@ -208,16 +208,22 @@ where } // Check if authorship of this header is an equivocation and return a proof if so. - let equivocation_proof = - match check_equivocation(&*self.client, slot_now, slot, header, author) - .map_err(Error::Client)? - { - Some(proof) => proof, - None => return Ok(()), - }; + let equivocation_proof = match sc_consensus_slots::check_equivocation( + &*self.client, + slot_now, + slot, + header, + author, + ) + .map_err(Error::Client)? + { + Some(proof) => proof, + None => return Ok(()), + }; info!( - "Slot author {:?} is equivocating at slot {} with headers {:?} and {:?}", + target: "sassafras", + "🌳 Slot author {:?} is equivocating at slot {} with headers {:?} and {:?}", author, slot, equivocation_proof.first_header.hash(), @@ -225,14 +231,50 @@ where ); // Get the best block on which we will build and send the equivocation report. - let _best_id: BlockId = self + let best_id = self .select_chain .best_chain() .await .map(|h| BlockId::Hash(h.hash())) .map_err(|e| Error::Client(e.into()))?; - // TODO-SASS-P2 + // Generate a key ownership proof. We start by trying to generate the key owernship proof + // at the parent of the equivocating header, this will make sure that proof generation is + // successful since it happens during the on-going session (i.e. session keys are available + // in the state to be able to generate the proof). This might fail if the equivocation + // happens on the first block of the session, in which case its parent would be on the + // previous session. If generation on the parent header fails we try with best block as + // well. + let generate_key_owner_proof = |block_id: &BlockId| { + self.client + .runtime_api() + .generate_key_ownership_proof(block_id, slot, equivocation_proof.offender.clone()) + .map_err(Error::RuntimeApi) + }; + + let parent_id = BlockId::Hash(*header.parent_hash()); + let key_owner_proof = match generate_key_owner_proof(&parent_id)? { + Some(proof) => proof, + None => match generate_key_owner_proof(&best_id)? { + Some(proof) => proof, + None => { + debug!(target: "babe", "Equivocation offender is not part of the authority set."); + return Ok(()) + }, + }, + }; + + // submit equivocation report at best block. + self.client + .runtime_api() + .submit_report_equivocation_unsigned_extrinsic( + &best_id, + equivocation_proof, + key_owner_proof, + ) + .map_err(Error::RuntimeApi)?; + + info!(target: "sassafras", "🌳 Submitted equivocation report for author {:?}", author); Ok(()) } diff --git a/frame/sassafras/src/lib.rs b/frame/sassafras/src/lib.rs index d77e34344f835..cb1e48a796952 100644 --- a/frame/sassafras/src/lib.rs +++ b/frame/sassafras/src/lib.rs @@ -54,9 +54,10 @@ use frame_support::{traits::Get, weights::Weight, BoundedVec, WeakBoundedVec}; use frame_system::offchain::{SendTransactionTypes, SubmitTransaction}; use sp_consensus_sassafras::{ digests::{ConsensusLog, NextEpochDescriptor, PreDigest}, - AuthorityId, Randomness, SassafrasAuthorityWeight, SassafrasEpochConfiguration, Slot, Ticket, - SASSAFRAS_ENGINE_ID, + AuthorityId, EquivocationProof, Randomness, SassafrasAuthorityWeight, + SassafrasEpochConfiguration, Slot, Ticket, SASSAFRAS_ENGINE_ID, }; +use sp_io::hashing; use sp_runtime::{ generic::DigestItem, traits::{One, Saturating}, @@ -71,17 +72,21 @@ mod mock; #[cfg(all(feature = "std", test))] mod tests; +// To manage epoch changes via session pallet instead of the built-in method +// method (`SameAuthoritiesForever`). pub mod session; +// Re-export pallet symbols. pub use pallet::*; /// Tickets related metadata that is commonly used together. #[derive(Debug, Default, PartialEq, Encode, Decode, TypeInfo, MaxEncodedLen, Clone, Copy)] pub struct TicketsMetadata { - /// Number of tickets available for even and odd sessions, respectivelly. - /// I.e. the index is computed as session-index modulo 2. + /// Number of tickets available into the tickets buffers. + /// The array index is computed as epoch index modulo 2. pub tickets_count: [u32; 2], - /// Number of tickets segments + /// Number of outstanding tickets segments requiring to be sorted and stored + /// in one of the epochs tickets buffer pub segments_count: u32, } @@ -144,7 +149,7 @@ pub mod pallet { ValueQuery, >; - /// Next session authorities. + /// Next epoch authorities. #[pallet::storage] pub type NextAuthorities = StorageValue< _, @@ -152,7 +157,7 @@ pub mod pallet { ValueQuery, >; - /// The slot at which the first session started. + /// The slot at which the first epoch started. /// This is `None` until the first block is imported on chain. #[pallet::storage] #[pallet::getter(fn genesis_slot)] @@ -163,12 +168,12 @@ pub mod pallet { #[pallet::getter(fn current_slot)] pub type CurrentSlot = StorageValue<_, Slot, ValueQuery>; - /// Current session randomness. + /// Current epoch randomness. #[pallet::storage] #[pallet::getter(fn randomness)] pub type CurrentRandomness = StorageValue<_, Randomness, ValueQuery>; - /// Next session randomness. + /// Next epoch randomness. #[pallet::storage] pub type NextRandomness = StorageValue<_, Randomness, ValueQuery>; @@ -193,9 +198,9 @@ pub mod pallet { /// Pending epoch configuration change that will be set as `NextEpochConfig` when the next /// epoch is enacted. - /// In other words, a config change submitted during session N will be enacted on session N+2. + /// In other words, a config change submitted during epoch N will be enacted on epoch N+2. /// This is to maintain coherence for already submitted tickets for epoch N+1 that where - /// computed using configuration parameters stored for session N+1. + /// computed using configuration parameters stored for epoch N+1. #[pallet::storage] pub(super) type PendingEpochConfigChange = StorageValue<_, SassafrasEpochConfiguration>; @@ -203,14 +208,14 @@ pub mod pallet { #[pallet::storage] pub type TicketsMeta = StorageValue<_, TicketsMetadata, ValueQuery>; - /// Tickets to be used for current and next session. - /// The key consists of a - /// - `u8` equal to session-index mod 2 + /// Tickets to be used for current and next epoch. + /// The key is a tuple composed by: + /// - `u8` equal to epoch-index mod 2 /// - `u32` equal to the slot-index. #[pallet::storage] pub type Tickets = StorageMap<_, Identity, (u8, u32), Ticket>; - /// Next session tickets temporary accumulator. + /// Next epoch tickets temporary accumulator. /// Special `u32::MAX` key is reserved for partially sorted segment. #[pallet::storage] pub type NextTicketsSegments = @@ -270,9 +275,7 @@ pub mod pallet { Initialized::::put(pre_digest); - // TODO-SASS-P3: incremental partial ordering for Next epoch tickets. - - // Enact session change, if necessary. + // Enact epoch change, if necessary. T::EpochChangeTrigger::trigger::(now); Weight::zero() @@ -287,6 +290,27 @@ pub mod pallet { let pre_digest = Initialized::::take() .expect("Finalization is called after initialization; qed."); Self::deposit_randomness(pre_digest.vrf_output.as_bytes()); + + // If we are in the second half of the epoch, we can start sorting the next epoch + // tickets. + let epoch_duration = T::EpochDuration::get(); + let current_slot_idx = Self::slot_index(pre_digest.slot); + if current_slot_idx >= epoch_duration / 2 { + let mut metadata = TicketsMeta::::get(); + if metadata.segments_count != 0 { + let epoch_idx = EpochIndex::::get() + 1; + let epoch_key = (epoch_idx & 1) as u8; + if metadata.segments_count != 0 { + let slots_left = epoch_duration.checked_sub(current_slot_idx).unwrap_or(1); + Self::sort_tickets( + u32::max(1, metadata.segments_count / slots_left as u32), + epoch_key, + &mut metadata, + ); + TicketsMeta::::set(metadata); + } + } + } } } @@ -294,7 +318,7 @@ pub mod pallet { impl Pallet { /// Submit next epoch tickets. /// - /// TODO-SASS-P3: this is an unsigned extrinsic. Can we remov ethe weight? + /// TODO-SASS-P3: this is an unsigned extrinsic. Can we remove the weight? #[pallet::weight(10_000)] pub fn submit_tickets( origin: OriginFor, @@ -315,13 +339,13 @@ pub mod pallet { /// Plan an epoch config change. /// - /// The epoch config change is recorded and will be enacted on the next call to - /// `enact_session_change`. - /// - /// The config will be activated one epoch after. Multiple calls to this method will - /// replace any existing planned config change that had not been enacted yet. + /// The epoch config change is recorded and will be announced at the begin of the + /// next epoch together with next epoch authorities information. + /// In other words the configuration will be activated one epoch after. + /// Multiple calls to this method will replace any existing planned config change that had + /// not been enacted yet. /// - /// TODO: TODO-SASS-P4: proper weight + /// TODO-SASS-P4: proper weight #[pallet::weight(10_000)] pub fn plan_config_change( origin: OriginFor, @@ -336,6 +360,32 @@ pub mod pallet { PendingEpochConfigChange::::put(config); Ok(()) } + + /// Report authority equivocation. + /// + /// This method will verify the equivocation proof and validate the given key ownership + /// proof against the extracted offender. If both are valid, the offence will be reported. + /// + /// This extrinsic must be called unsigned and it is expected that only block authors will + /// call it (validated in `ValidateUnsigned`), as such if the block author is defined it + /// will be defined as the equivocation reporter. + /// + /// TODO-SASS-P4: proper weight + #[pallet::weight(10_000)] + pub fn report_equivocation_unsigned( + origin: OriginFor, + _equivocation_proof: EquivocationProof, + //key_owner_proof: T::KeyOwnerProof, + ) -> DispatchResult { + ensure_none(origin)?; + + // Self::do_report_equivocation( + // T::HandleEquivocation::block_author(), + // *equivocation_proof, + // key_owner_proof, + // ) + Ok(()) + } } #[pallet::validate_unsigned] @@ -373,7 +423,8 @@ pub mod pallet { // Current slot should be less than half of epoch duration. let epoch_duration = T::EpochDuration::get(); - if Self::current_slot_index() >= epoch_duration / 2 { + let current_slot_idx = Self::current_slot_index(); + if current_slot_idx >= epoch_duration / 2 { log::warn!( target: "sassafras::runtime", "🌳 Timeout to propose tickets, bailing out.", @@ -382,7 +433,6 @@ pub mod pallet { } // Check tickets are below threshold - let next_auth = NextAuthorities::::get(); let epoch_config = EpochConfig::::get(); let threshold = sp_consensus_sassafras::compute_threshold( @@ -391,11 +441,6 @@ pub mod pallet { epoch_config.attempts_number, next_auth.len() as u32, ); - - // TODO-SASS-P2: if we move this in the `submit_tickets` call then we can - // can drop only the invalid tickets. - // In this way we don't penalize validators that submit tickets together - // with faulty validators. if !tickets .iter() .all(|ticket| sp_consensus_sassafras::check_threshold(ticket, threshold)) @@ -403,15 +448,17 @@ pub mod pallet { return InvalidTransaction::Custom(0).into() } + // This should be set such that it is discarded after the first epoch half + // TODO-SASS-P3: double check this. Should we then check again in the extrinsic + // itself? Is this run also just before the extrinsic execution or only on tx queue + // insertion? + let tickets_longevity = epoch_duration / 2 - current_slot_idx; + let tickets_tag = tickets.using_encoded(|bytes| hashing::blake2_256(bytes)); + ValidTransaction::with_tag_prefix("Sassafras") - // We assign the maximum priority for any equivocation report. .priority(TransactionPriority::max_value()) - // TODO-SASS-P2: if possible use a more efficient way to distinquish - // duplicates... - .and_provides(tickets) - // TODO-SASS-P2: this should be set such that it is discarded after the - // first half - .longevity(3_u64) + .longevity(tickets_longevity) + .and_provides(tickets_tag) .propagate(true) .build() } else { @@ -423,17 +470,9 @@ pub mod pallet { // Inherent methods impl Pallet { - // // TODO-SASS-P2: I don't think this is really required - // /// Determine the Sassafras slot duration based on the Timestamp module configuration. - // pub fn slot_duration() -> T::Moment { - // // We double the minimum block-period so each author can always propose within - // // the majority of their slot. - // ::MinimumPeriod::get().saturating_mul(2u32.into()) - // } - /// Determine whether an epoch change should take place at this block. /// Assumes that initialization has already taken place. - pub fn should_end_session(now: T::BlockNumber) -> bool { + pub fn should_end_epoch(now: T::BlockNumber) -> bool { // The epoch has technically ended during the passage of time between this block and the // last, but we have to "end" the epoch now, since there is no earlier possible block we // could have done it. @@ -457,17 +496,15 @@ impl Pallet { slot.checked_sub(Self::current_epoch_start().into()).unwrap_or(u64::MAX) } - /// DANGEROUS: Enact an epoch change. Should be done on every block where `should_end_session` + /// DANGEROUS: Enact an epoch change. Should be done on every block where `should_end_epoch` /// has returned `true`, and the caller is the only caller of this function. /// - /// Typically, this is not handled directly by the user, but by higher-level validator-set - /// manager logic like `pallet-session`. + /// Typically, this is not handled directly, but by a higher-level validator-set + /// manager like `pallet-session`. /// - /// TODO-SASS-P3: /// If we detect one or more skipped epochs the policy is to use the authorities and values - /// from the first skipped epoch. - /// Should the tickets be invalidated? Currently they are... see the `get-ticket` method. - pub(crate) fn enact_session_change( + /// from the first skipped epoch. The tickets are invalidated. + pub(crate) fn enact_epoch_change( authorities: WeakBoundedVec<(AuthorityId, SassafrasAuthorityWeight), T::MaxAuthorities>, next_authorities: WeakBoundedVec< (AuthorityId, SassafrasAuthorityWeight), @@ -489,11 +526,9 @@ impl Pallet { let slot_idx = CurrentSlot::::get().saturating_sub(Self::epoch_start(epoch_idx)); if slot_idx >= T::EpochDuration::get() { - // Detected one or more skipped epochs, kill tickets and recompute the `epoch_index`. + // Detected one or more skipped epochs, kill tickets and recompute epoch index. TicketsMeta::::kill(); - // TODO-SASS-P2: adjust epoch index (TEST ME) - let idx: u64 = slot_idx.into(); - epoch_idx += idx / T::EpochDuration::get(); + epoch_idx += u64::from(slot_idx) / T::EpochDuration::get(); } EpochIndex::::put(epoch_idx); @@ -546,7 +581,7 @@ impl Pallet { s.extend_from_slice(&next_epoch_index.to_le_bytes()); s.extend_from_slice(&accumulator); - let next_randomness = sp_io::hashing::blake2_256(&s); + let next_randomness = hashing::blake2_256(&s); NextRandomness::::put(&next_randomness); next_randomness @@ -575,7 +610,7 @@ impl Pallet { fn deposit_randomness(randomness: &Randomness) { let mut s = RandomnessAccumulator::::get().to_vec(); s.extend_from_slice(randomness); - let accumulator = sp_io::hashing::blake2_256(&s); + let accumulator = hashing::blake2_256(&s); RandomnessAccumulator::::put(accumulator); } @@ -677,7 +712,7 @@ impl Pallet { // Lexicographically sort the tickets who belongs to the next epoch. // The tickets are fetched from at most `max_iter` segments received via the `submit_tickets` // extrinsic. The resulting sorted vector is truncated and if all the segments where sorted - // it is saved to be as the next session tickets. + // it is saved to be as the next epoch tickets. // Else the result is saved to be used by next calls. fn sort_tickets(max_iter: u32, epoch_key: u8, metadata: &mut TicketsMetadata) { let mut segments_count = metadata.segments_count; @@ -747,6 +782,27 @@ impl Pallet { }, } } + + /// Submits an equivocation via an unsigned extrinsic. + /// + /// Unsigned extrinsic is created with a call to `report_equivocation_unsigned`. + pub fn submit_unsigned_equivocation_report( + equivocation_proof: EquivocationProof, + //key_owner_proof: T::KeyOwnerProof, + ) -> bool { + let call = Call::report_equivocation_unsigned { + equivocation_proof, + // key_owner_proof, + }; + + match SubmitTransaction::>::submit_unsigned_transaction(call.into()) { + Ok(()) => true, + Err(e) => { + log::error!(target: "runtime::sassafras", "Error submitting equivocation report: {:?}", e); + false + }, + } + } } /// Trigger an epoch change, if any should take place. @@ -770,11 +826,11 @@ pub struct SameAuthoritiesForever; impl EpochChangeTrigger for SameAuthoritiesForever { fn trigger(now: T::BlockNumber) { - if >::should_end_session(now) { + if >::should_end_epoch(now) { let authorities = >::authorities(); let next_authorities = authorities.clone(); - >::enact_session_change(authorities, next_authorities); + >::enact_epoch_change(authorities, next_authorities); } } } diff --git a/frame/sassafras/src/mock.rs b/frame/sassafras/src/mock.rs index a8c9ca6e856d7..c7bd93c0b2175 100644 --- a/frame/sassafras/src/mock.rs +++ b/frame/sassafras/src/mock.rs @@ -107,10 +107,13 @@ frame_support::construct_runtime!( } ); +/// Build and returns test storage externalities pub fn new_test_ext(authorities_len: usize) -> sp_io::TestExternalities { new_test_ext_with_pairs(authorities_len).1 } +/// Build and returns test storage externalities and authority set pairs used +/// by Sassafras genesis configuration. pub fn new_test_ext_with_pairs( authorities_len: usize, ) -> (Vec, sp_io::TestExternalities) { @@ -120,13 +123,16 @@ pub fn new_test_ext_with_pairs( let authorities = pairs.iter().map(|p| (p.public(), 1)).collect(); - let mut t = frame_system::GenesisConfig::default().build_storage::().unwrap(); + let mut storage = frame_system::GenesisConfig::default().build_storage::().unwrap(); let config = pallet_sassafras::GenesisConfig { authorities, epoch_config: Default::default() }; - >::assimilate_storage(&config, &mut t) - .unwrap(); + >::assimilate_storage( + &config, + &mut storage, + ) + .unwrap(); - (pairs, t.into()) + (pairs, storage.into()) } fn make_ticket_vrf(slot: Slot, attempt: u32, pair: &AuthorityPair) -> (VRFOutput, VRFProof) { @@ -150,6 +156,8 @@ fn make_ticket_vrf(slot: Slot, attempt: u32, pair: &AuthorityPair) -> (VRFOutput (output, proof) } +/// Construct at most `attempts` tickets for the given `slot`. +/// TODO-SASS-P3: filter out invalid tickets according to test threshold. pub fn make_tickets(slot: Slot, attempts: u32, pair: &AuthorityPair) -> Vec<(VRFOutput, VRFProof)> { (0..attempts) .into_iter() @@ -163,7 +171,7 @@ fn make_slot_vrf(slot: Slot, pair: &AuthorityPair) -> (VRFOutput, VRFProof) { let mut epoch = Sassafras::epoch_index(); let mut randomness = Sassafras::randomness(); - // Check if epoch is going to change on initialization + // Check if epoch is going to change on initialization. let epoch_start = Sassafras::current_epoch_start(); if epoch_start != 0_u64 && slot >= epoch_start + EPOCH_DURATION { epoch += slot.saturating_sub(epoch_start).saturating_div(EPOCH_DURATION); @@ -178,6 +186,7 @@ fn make_slot_vrf(slot: Slot, pair: &AuthorityPair) -> (VRFOutput, VRFProof) { (output, proof) } +/// Produce a `PreDigest` instance for the given parameters. pub fn make_pre_digest( authority_idx: AuthorityIndex, slot: Slot, @@ -187,6 +196,8 @@ pub fn make_pre_digest( PreDigest { authority_idx, slot, vrf_output, vrf_proof, ticket_aux: None } } +/// Produce a `PreDigest` instance for the given parameters and wrap the result into a `Digest` +/// instance. pub fn make_wrapped_pre_digest( authority_idx: AuthorityIndex, slot: Slot, @@ -198,6 +209,7 @@ pub fn make_wrapped_pre_digest( Digest { logs: vec![log] } } +/// Progress the pallet state up to the given block `number` and `slot`. pub fn go_to_block(number: u64, slot: Slot, pair: &AuthorityPair) -> Digest { Sassafras::on_finalize(System::block_number()); let parent_hash = System::finalize().hash(); @@ -211,7 +223,8 @@ pub fn go_to_block(number: u64, slot: Slot, pair: &AuthorityPair) -> Digest { digest } -/// Slots will grow accordingly to blocks +/// Progress the pallet state up to the given block `number`. +/// Slots will grow linearly accordingly to blocks. pub fn progress_to_block(number: u64, pair: &AuthorityPair) -> Option { let mut slot = Sassafras::current_slot() + 1; let mut digest = None; diff --git a/frame/sassafras/src/session.rs b/frame/sassafras/src/session.rs index bfe4e1c79b968..e15fd3637b9ae 100644 --- a/frame/sassafras/src/session.rs +++ b/frame/sassafras/src/session.rs @@ -29,7 +29,7 @@ impl ShouldEndSession for Pallet { // possible that Sassafras's own `on_initialize` has not run yet, so let's ensure that we // have initialized the pallet and updated the current slot. Self::on_initialize(now); - Self::should_end_session(now) + Self::should_end_epoch(now) } } @@ -66,7 +66,7 @@ impl OneSessionHandler for Pallet { ), ); - Self::enact_session_change(bounded_authorities, next_bounded_authorities) + Self::enact_epoch_change(bounded_authorities, next_bounded_authorities) } fn on_disabled(i: u32) { diff --git a/frame/sassafras/src/tests.rs b/frame/sassafras/src/tests.rs index bd253c0c72f40..9c5828ac50b62 100644 --- a/frame/sassafras/src/tests.rs +++ b/frame/sassafras/src/tests.rs @@ -25,6 +25,14 @@ use hex_literal::hex; use sp_consensus_sassafras::Slot; use sp_runtime::traits::Get; +#[test] +fn genesis_values_sanity_check() { + new_test_ext(4).execute_with(|| { + assert_eq!(Sassafras::authorities().len(), 4); + assert_eq!(EpochConfig::::get(), Default::default()); + }); +} + #[test] fn slot_ticket_fetch() { let genesis_slot = Slot::from(100); @@ -49,21 +57,22 @@ fn slot_ticket_fetch() { Tickets::::insert((1, i as u32), ticket); }); TicketsMeta::::set(TicketsMetadata { - tickets_count: [max_tickets, max_tickets - 1], + tickets_count: [curr_tickets.len() as u32, next_tickets.len() as u32], segments_count: 0, }); - // Before initializing `GenesisSlot` value (should return first element of current session) - // This is due to special case hardcoded value. + // Before initializing `GenesisSlot` value the pallet always return the first slot + // This is a kind of special case hardcoded policy that should never happen in practice + // (i.e. the first thing the pallet does is to initialize the genesis slot). assert_eq!(Sassafras::slot_ticket(0.into()), Some(curr_tickets[1])); assert_eq!(Sassafras::slot_ticket(genesis_slot + 0), Some(curr_tickets[1])); assert_eq!(Sassafras::slot_ticket(genesis_slot + 1), Some(curr_tickets[1])); assert_eq!(Sassafras::slot_ticket(genesis_slot + 100), Some(curr_tickets[1])); - // Initialize genesis slot value. + // Initialize genesis slot.. GenesisSlot::::set(genesis_slot); - // Before Current session. + // Try fetch a ticket for a slot before current session. assert_eq!(Sassafras::slot_ticket(0.into()), None); // Current session tickets. @@ -90,20 +99,12 @@ fn slot_ticket_fetch() { assert_eq!(Sassafras::slot_ticket(genesis_slot + 18), Some(next_tickets[2])); assert_eq!(Sassafras::slot_ticket(genesis_slot + 19), Some(next_tickets[0])); - // Beyend next session. + // Try fetch tickets for slots beyend next session. assert_eq!(Sassafras::slot_ticket(genesis_slot + 20), None); assert_eq!(Sassafras::slot_ticket(genesis_slot + 42), None); }); } -#[test] -fn genesis_values() { - new_test_ext(4).execute_with(|| { - assert_eq!(Sassafras::authorities().len(), 4); - assert_eq!(EpochConfig::::get(), Default::default()); - }); -} - #[test] fn on_first_block_after_genesis() { let (pairs, mut ext) = new_test_ext_with_pairs(4); @@ -222,7 +223,7 @@ fn on_normal_block() { } #[test] -fn epoch_change_block() { +fn produce_epoch_change_digest() { let (pairs, mut ext) = new_test_ext_with_pairs(4); ext.execute_with(|| { @@ -293,6 +294,133 @@ fn epoch_change_block() { }) } +#[test] +fn produce_epoch_change_digest_with_config() { + let (pairs, mut ext) = new_test_ext_with_pairs(4); + + ext.execute_with(|| { + let start_slot = Slot::from(100); + let start_block = 1; + + let digest = make_wrapped_pre_digest(0, start_slot, &pairs[0]); + System::initialize(&start_block, &Default::default(), &digest); + Sassafras::on_initialize(start_block); + + let config = SassafrasEpochConfiguration { redundancy_factor: 1, attempts_number: 123 }; + Sassafras::plan_config_change(RuntimeOrigin::root(), config.clone()).unwrap(); + + // We want to trigger an epoch change in this test. + let epoch_duration: u64 = ::EpochDuration::get(); + let digest = progress_to_block(start_block + epoch_duration, &pairs[0]).unwrap(); + + Sassafras::on_finalize(start_block + epoch_duration); + + // Header data check. + // Skip pallet status checks that were already performed by other tests. + + let header = System::finalize(); + assert_eq!(header.digest.logs.len(), 2); + assert_eq!(header.digest.logs[0], digest.logs[0]); + // Deposits consensus log on epoch change + let consensus_log = sp_consensus_sassafras::digests::ConsensusLog::NextEpochData( + sp_consensus_sassafras::digests::NextEpochDescriptor { + authorities: NextAuthorities::::get().to_vec(), + randomness: NextRandomness::::get(), + config: Some(config), // We are mostly interested in this + }, + ); + let consensus_digest = DigestItem::Consensus(SASSAFRAS_ENGINE_ID, consensus_log.encode()); + assert_eq!(header.digest.logs[1], consensus_digest) + }) +} + +#[test] +fn segments_incremental_sortition_works() { + let (pairs, mut ext) = new_test_ext_with_pairs(1); + let pair = &pairs[0]; + let segments_num = 14; + + ext.execute_with(|| { + let start_slot = Slot::from(100); + let start_block = 1; + let max_tickets: u32 = ::MaxTickets::get(); + + let digest = make_wrapped_pre_digest(0, start_slot, &pairs[0]); + System::initialize(&start_block, &Default::default(), &digest); + Sassafras::on_initialize(start_block); + + // Submit authoring tickets in three different batches. + // We can ignore the threshold since we are not passing through the unsigned extrinsic + // validation. + let mut tickets: Vec = + make_tickets(start_slot + 1, segments_num * max_tickets, pair) + .into_iter() + .map(|(output, _)| output) + .collect(); + let segment_len = tickets.len() / segments_num as usize; + for i in 0..segments_num as usize { + let segment = + tickets[i * segment_len..(i + 1) * segment_len].to_vec().try_into().unwrap(); + Sassafras::submit_tickets(RuntimeOrigin::none(), segment).unwrap(); + } + + tickets.sort(); + tickets.truncate(max_tickets as usize); + let _expected_tickets = tickets; + + let epoch_duration: u64 = ::EpochDuration::get(); + + // Proceed to half of the epoch (sortition should not have been started yet) + let half_epoch_block = start_block + epoch_duration / 2; + progress_to_block(half_epoch_block, pair); + + // Check that next epoch tickets sortition is not started yet + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, segments_num); + assert_eq!(meta.tickets_count, [0, 0]); + + // Monitor incremental sortition + + progress_to_block(half_epoch_block + 1, pair); + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 12); + assert_eq!(meta.tickets_count, [0, 0]); + + progress_to_block(half_epoch_block + 2, pair); + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 9); + assert_eq!(meta.tickets_count, [0, 0]); + + progress_to_block(half_epoch_block + 3, pair); + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 6); + assert_eq!(meta.tickets_count, [0, 0]); + + progress_to_block(half_epoch_block + 4, pair); + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 3); + assert_eq!(meta.tickets_count, [0, 0]); + + Sassafras::on_finalize(half_epoch_block + 4); + let header = System::finalize(); + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 0); + assert_eq!(meta.tickets_count, [0, 6]); + assert_eq!(header.digest.logs.len(), 1); + + // The next block will be the first produced on the new epoch, + // At this point the tickets were found to be sorted and ready to be used. + let slot = Sassafras::current_slot() + 1; + let digest = make_wrapped_pre_digest(0, slot, pair); + let number = System::block_number() + 1; + System::initialize(&number, &header.hash(), &digest); + Sassafras::on_initialize(number); + Sassafras::on_finalize(half_epoch_block + 5); + let header = System::finalize(); + assert_eq!(header.digest.logs.len(), 2); + }); +} + #[test] fn submit_enact_claim_tickets() { let (pairs, mut ext) = new_test_ext_with_pairs(4); @@ -341,9 +469,10 @@ fn submit_enact_claim_tickets() { // Process up to the last epoch slot (do not enact epoch change) let _digest = progress_to_block(epoch_duration, &pairs[0]).unwrap(); - // TODO-SASS-P2: at this point next tickets should have been sorted - //assert_eq!(NextTicketsSegmentsCount::::get(), 0); - //assert!(Tickets::::iter().next().is_some()); + // At this point next tickets should have been sorted + let meta = TicketsMeta::::get(); + assert_eq!(meta.segments_count, 0); + assert_eq!(meta.tickets_count, [0, 6]); // Check if we can claim next epoch tickets in outside-in fashion. let slot = Sassafras::current_slot(); @@ -379,7 +508,7 @@ fn submit_enact_claim_tickets() { } #[test] -fn block_skips_epochs() { +fn block_allowed_to_skip_epochs() { let (pairs, mut ext) = new_test_ext_with_pairs(4); ext.execute_with(|| { diff --git a/primitives/consensus/babe/src/lib.rs b/primitives/consensus/babe/src/lib.rs index 621ab859b914f..98a14819ba326 100644 --- a/primitives/consensus/babe/src/lib.rs +++ b/primitives/consensus/babe/src/lib.rs @@ -329,6 +329,7 @@ where /// sure that all usages of `OpaqueKeyOwnershipProof` refer to the same type. #[derive(Decode, Encode, PartialEq)] pub struct OpaqueKeyOwnershipProof(Vec); + impl OpaqueKeyOwnershipProof { /// Create a new `OpaqueKeyOwnershipProof` using the given encoded /// representation. diff --git a/primitives/consensus/sassafras/src/lib.rs b/primitives/consensus/sassafras/src/lib.rs index 56903eb7da7c3..c4bd2daca8f97 100644 --- a/primitives/consensus/sassafras/src/lib.rs +++ b/primitives/consensus/sassafras/src/lib.rs @@ -78,12 +78,15 @@ pub type SassafrasAuthorityWeight = u64; /// Primary blocks have a weight of 1 whereas secondary blocks have a weight of 0. pub type SassafrasBlockWeight = u32; +/// An equivocation proof for multiple block authorships on the same slot (i.e. double vote). +pub type EquivocationProof = sp_consensus_slots::EquivocationProof; + /// Configuration data used by the Sassafras consensus engine. #[derive(Clone, Encode, Decode, RuntimeDebug, PartialEq, Eq)] pub struct SassafrasConfiguration { /// The slot duration in milliseconds. pub slot_duration: u64, - /// The duration of epochs in slots. + /// The duration of epoch in slots. pub epoch_duration: u64, /// The authorities for the epoch. pub authorities: Vec<(AuthorityId, SassafrasAuthorityWeight)>, @@ -130,7 +133,6 @@ pub struct TicketAux { /// The parameters should be chosen such that T <= 1. /// If `attempts * validators` is zero then we fallback to T = 0 // TODO-SASS-P3: this formula must be double-checked... -#[inline] pub fn compute_threshold(redundancy: u32, slots: u32, attempts: u32, validators: u32) -> U256 { let den = attempts as u64 * validators as u64; let num = redundancy as u64 * slots as u64; @@ -141,11 +143,31 @@ pub fn compute_threshold(redundancy: u32, slots: u32, attempts: u32, validators: } /// Returns true if the given VRF output is lower than the given threshold, false otherwise. -#[inline] pub fn check_threshold(ticket: &Ticket, threshold: U256) -> bool { U256::from(ticket.as_bytes()) < threshold } +/// An opaque type used to represent the key ownership proof at the runtime API boundary. +/// The inner value is an encoded representation of the actual key ownership proof which will be +/// parameterized when defining the runtime. At the runtime API boundary this type is unknown and +/// as such we keep this opaque representation, implementors of the runtime API will have to make +/// sure that all usages of `OpaqueKeyOwnershipProof` refer to the same type. +#[derive(Decode, Encode, PartialEq)] +pub struct OpaqueKeyOwnershipProof(Vec); + +impl OpaqueKeyOwnershipProof { + /// Create a new `OpaqueKeyOwnershipProof` using the given encoded representation. + pub fn new(inner: Vec) -> OpaqueKeyOwnershipProof { + OpaqueKeyOwnershipProof(inner) + } + + /// Try to decode this `OpaqueKeyOwnershipProof` into the given concrete key + /// ownership proof type. + pub fn decode(self) -> Option { + Decode::decode(&mut &self.0[..]).ok() + } +} + // Runtime API. sp_api::decl_runtime_apis! { /// API necessary for block authorship with Sassafras. @@ -159,5 +181,34 @@ sp_api::decl_runtime_apis! { /// Get expected ticket for the given slot. fn slot_ticket(slot: Slot) -> Option; + + /// Generates a proof of key ownership for the given authority in the current epoch. + /// + /// An example usage of this module is coupled with the session historical module to prove + /// that a given authority key is tied to a given staking identity during a specific + /// session. Proofs of key ownership are necessary for submitting equivocation reports. + /// + /// NOTE: even though the API takes a `slot` as parameter the current implementations + /// ignores this parameter and instead relies on this method being called at the correct + /// block height, i.e. any point at which the epoch for the given slot is live on-chain. + /// Future implementations will instead use indexed data through an offchain worker, not + /// requiring older states to be available. + fn generate_key_ownership_proof( + slot: Slot, + authority_id: AuthorityId, + ) -> Option; + + /// Submits an unsigned extrinsic to report an equivocation. + /// + /// The caller must provide the equivocation proof and a key ownership proof (should be + /// obtained using `generate_key_ownership_proof`). The extrinsic will be unsigned and + /// should only be accepted for local authorship (not to be broadcast to the network). This + /// method returns `None` when creation of the extrinsic fails, e.g. if equivocation + /// reporting is disabled for the given runtime (i.e. this method is hardcoded to return + /// `None`). Only useful in an offchain context. + fn submit_report_equivocation_unsigned_extrinsic( + equivocation_proof: EquivocationProof, + key_owner_proof: OpaqueKeyOwnershipProof, + ) -> bool; } } diff --git a/test-utils/runtime/Cargo.toml b/test-utils/runtime/Cargo.toml index 744cc527e6012..92c8fd4ad6856 100644 --- a/test-utils/runtime/Cargo.toml +++ b/test-utils/runtime/Cargo.toml @@ -18,6 +18,7 @@ beefy-merkle-tree = { version = "4.0.0-dev", default-features = false, path = ". sp-application-crypto = { version = "6.0.0", default-features = false, path = "../../primitives/application-crypto" } sp-consensus-aura = { version = "0.10.0-dev", default-features = false, path = "../../primitives/consensus/aura" } sp-consensus-babe = { version = "0.10.0-dev", default-features = false, path = "../../primitives/consensus/babe" } +sp-consensus-sassafras = { version = "0.1.0", default-features = false, path = "../../primitives/consensus/sassafras" } sp-block-builder = { version = "4.0.0-dev", default-features = false, path = "../../primitives/block-builder" } codec = { package = "parity-scale-codec", version = "3.0.0", default-features = false, features = ["derive"] } scale-info = { version = "2.1.1", default-features = false, features = ["derive"] } @@ -35,6 +36,7 @@ sp-session = { version = "4.0.0-dev", default-features = false, path = "../../pr sp-api = { version = "4.0.0-dev", default-features = false, path = "../../primitives/api" } sp-runtime = { version = "6.0.0", default-features = false, path = "../../primitives/runtime" } pallet-babe = { version = "4.0.0-dev", default-features = false, path = "../../frame/babe" } +pallet-sassafras = { version = "0.1.0", default-features = false, path = "../../frame/sassafras" } frame-system = { version = "4.0.0-dev", default-features = false, path = "../../frame/system" } frame-system-rpc-runtime-api = { version = "4.0.0-dev", default-features = false, path = "../../frame/system/rpc/runtime-api" } pallet-timestamp = { version = "4.0.0-dev", default-features = false, path = "../../frame/timestamp" } @@ -73,6 +75,7 @@ std = [ "sp-application-crypto/std", "sp-consensus-aura/std", "sp-consensus-babe/std", + "sp-consensus-sassafras/std", "sp-block-builder/std", "codec/std", "scale-info/std", @@ -95,6 +98,7 @@ std = [ "sp-externalities/std", "sp-state-machine/std", "pallet-babe/std", + "pallet-sassafras/std", "frame-system-rpc-runtime-api/std", "frame-system/std", "pallet-timestamp/std", diff --git a/test-utils/runtime/src/lib.rs b/test-utils/runtime/src/lib.rs index a64e3f25ef041..975285c261d33 100644 --- a/test-utils/runtime/src/lib.rs +++ b/test-utils/runtime/src/lib.rs @@ -63,10 +63,10 @@ use sp_runtime::{ use sp_version::NativeVersion; use sp_version::RuntimeVersion; -// Ensure Babe and Aura use the same crypto to simplify things a bit. +// Ensure Babe, Sassafras and Aura use the same crypto to simplify things a bit. pub use sp_consensus_babe::{AllowedSlots, AuthorityId, Slot}; - pub type AuraId = sp_consensus_aura::sr25519::AuthorityId; +pub type SassafrasId = sp_consensus_sassafras::AuthorityId; // Include the WASM binary #[cfg(feature = "std")] @@ -163,6 +163,22 @@ pub enum Extrinsic { OffchainIndexSet(Vec, Vec), OffchainIndexClear(Vec), Store(Vec), + Sassafras, +} + +impl From> for Extrinsic { + fn from(call: pallet_sassafras::Call) -> Self { + use pallet_sassafras::Call; + match call { + Call::submit_tickets { tickets: _ } => Extrinsic::Sassafras, + Call::plan_config_change { config: _ } => Extrinsic::Sassafras, + Call::report_equivocation_unsigned { equivocation_proof: _ } => Extrinsic::Sassafras, + _ => panic!( + "Unexpected Sassafras call type: {:?}, unable to converto to Extrinsic", + call + ), + } + } } parity_util_mem::malloc_size_of_is_0!(Extrinsic); // non-opaque extrinsic does not need this @@ -211,6 +227,8 @@ impl BlindCheckable for Extrinsic { Extrinsic::OffchainIndexSet(key, value) => Ok(Extrinsic::OffchainIndexSet(key, value)), Extrinsic::OffchainIndexClear(key) => Ok(Extrinsic::OffchainIndexClear(key)), Extrinsic::Store(data) => Ok(Extrinsic::Store(data)), + // TODO-SASS-P2 + Extrinsic::Sassafras => Ok(Extrinsic::Sassafras), } } } @@ -524,6 +542,9 @@ impl frame_support::traits::PalletInfo for Runtime { if type_id == sp_std::any::TypeId::of::>() { return Some(2) } + if type_id == sp_std::any::TypeId::of::>() { + return Some(3) + } None } @@ -538,6 +559,9 @@ impl frame_support::traits::PalletInfo for Runtime { if type_id == sp_std::any::TypeId::of::>() { return Some("Babe") } + if type_id == sp_std::any::TypeId::of::>() { + return Some("Sassafras") + } None } @@ -552,6 +576,9 @@ impl frame_support::traits::PalletInfo for Runtime { if type_id == sp_std::any::TypeId::of::>() { return Some("pallet_babe") } + if type_id == sp_std::any::TypeId::of::>() { + return Some("pallet_sassafras") + } None } @@ -567,6 +594,9 @@ impl frame_support::traits::PalletInfo for Runtime { if type_id == sp_std::any::TypeId::of::>() { return Some(pallet_babe::Pallet::::crate_version()) } + if type_id == sp_std::any::TypeId::of::>() { + return Some(pallet_sassafras::Pallet::::crate_version()) + } None } @@ -621,6 +651,7 @@ impl pallet_timestamp::Config for Runtime { } parameter_types! { + pub const SlotDuration: u64 = 1000; pub const EpochDuration: u64 = 6; } @@ -649,6 +680,23 @@ impl pallet_babe::Config for Runtime { type MaxAuthorities = ConstU32<10>; } +impl frame_system::offchain::SendTransactionTypes for Runtime +where + Extrinsic: From, +{ + type Extrinsic = Extrinsic; + type OverarchingCall = Extrinsic; +} + +impl pallet_sassafras::Config for Runtime { + type SlotDuration = SlotDuration; + type EpochDuration = EpochDuration; + //type EpochChangeTrigger = pallet_sassafras::ExternalTrigger; + type EpochChangeTrigger = pallet_sassafras::SameAuthoritiesForever; + type MaxAuthorities = ConstU32<10>; + type MaxTickets = ConstU32<10>; +} + /// Adds one to the given input and returns the final result. #[inline(never)] fn benchmark_add_one(i: u64) -> u64 { @@ -895,6 +943,48 @@ cfg_if! { } } + impl sp_consensus_sassafras::SassafrasApi for Runtime { + fn configuration() -> sp_consensus_sassafras::SassafrasConfiguration { + let authorities = system::authorities().into_iter().map(|x| { + let authority: sr25519::Public = x.into(); + (SassafrasId::from(authority), 1) + }).collect(); + sp_consensus_sassafras::SassafrasConfiguration { + slot_duration: SlotDuration::get(), + epoch_duration: EpochDuration::get(), + authorities, + randomness: >::randomness(), + threshold_params: >::config(), + } + } + + fn submit_tickets_unsigned_extrinsic( + tickets: Vec + ) -> bool { + >::submit_tickets_unsigned_extrinsic(tickets) + } + + fn slot_ticket(slot: sp_consensus_sassafras::Slot) -> Option { + >::slot_ticket(slot) + } + + fn generate_key_ownership_proof( + _slot: sp_consensus_sassafras::Slot, + _authority_id: sp_consensus_sassafras::AuthorityId, + ) -> Option { + // TODO-SASS-P2 + None + } + + fn submit_report_equivocation_unsigned_extrinsic( + _equivocation_proof: sp_consensus_sassafras::EquivocationProof<::Header>, + _key_owner_proof: sp_consensus_sassafras::OpaqueKeyOwnershipProof, + ) -> bool { + // TODO-SASS-P2 + false + } + } + impl sp_offchain::OffchainWorkerApi for Runtime { fn offchain_worker(header: &::Header) { let ex = Extrinsic::IncludeData(header.number.encode()); @@ -1169,6 +1259,44 @@ cfg_if! { } } + impl sp_consensus_sassafras::SassafrasApi for Runtime { + fn configuration() -> sp_consensus_sassafras::SassafrasConfiguration { + sp_consensus_sassafras::SassafrasConfiguration { + slot_duration: SlotDuration::get(), + epoch_duration: EpochDuration::get(), + authorities: >::authorities().to_vec(), + randomness: >::randomness(), + threshold_params: >::config(), + } + } + + fn submit_tickets_unsigned_extrinsic( + tickets: Vec + ) -> bool { + >::submit_tickets_unsigned_extrinsic(tickets) + } + + fn slot_ticket(slot: sp_consensus_sassafras::Slot) -> Option { + >::slot_ticket(slot) + } + + fn generate_key_ownership_proof( + slot: sp_consensus_sassafras::Slot, + authority_id: sp_consensus_sassafras::AuthorityId, + ) -> Option { + // TODO-SASS-P2 + None + } + + fn submit_report_equivocation_unsigned_extrinsic( + _equivocation_proof: sp_consensus_sassafras::EquivocationProof<::Header>, + _key_owner_proof: sp_consensus_sassafras::OpaqueKeyOwnershipProof, + ) -> bool { + // TODO-SASS-P2 + false + } + } + impl sp_offchain::OffchainWorkerApi for Runtime { fn offchain_worker(header: &::Header) { let ex = Extrinsic::IncludeData(header.number.encode()); diff --git a/test-utils/runtime/src/system.rs b/test-utils/runtime/src/system.rs index 6e33d5c25fe6f..e832ea66284a6 100644 --- a/test-utils/runtime/src/system.rs +++ b/test-utils/runtime/src/system.rs @@ -276,6 +276,8 @@ fn execute_transaction_backend(utx: &Extrinsic, extrinsic_index: u32) -> ApplyEx Ok(Ok(())) }, Extrinsic::Store(data) => execute_store(data.clone()), + // TODO-SASS-P2 + Extrinsic::Sassafras => Ok(Ok(())), } }