#![cfg_attr(not(feature = "std"), no_std)]
#![feature(array_windows, let_chains, variant_count)]
#[cfg(feature = "runtime-benchmarks")]
mod benchmarking;
#[cfg(test)]
mod tests;
pub mod block_tree;
mod bundle_storage_fund;
pub mod domain_registry;
pub mod runtime_registry;
mod staking;
mod staking_epoch;
pub mod weights;
extern crate alloc;
use crate::block_tree::{verify_execution_receipt, Error as BlockTreeError};
use crate::bundle_storage_fund::{charge_bundle_storage_fee, storage_fund_account};
use crate::domain_registry::{DomainConfig, Error as DomainRegistryError};
use crate::runtime_registry::into_complete_raw_genesis;
#[cfg(feature = "runtime-benchmarks")]
pub use crate::staking::do_register_operator;
use crate::staking::{do_reward_operators, OperatorStatus};
use crate::staking_epoch::EpochTransitionResult;
use crate::weights::WeightInfo;
#[cfg(not(feature = "std"))]
use alloc::boxed::Box;
use alloc::collections::btree_map::BTreeMap;
#[cfg(not(feature = "std"))]
use alloc::vec::Vec;
use codec::{Decode, Encode};
use frame_support::ensure;
use frame_support::pallet_prelude::StorageVersion;
use frame_support::traits::fungible::{Inspect, InspectHold};
use frame_support::traits::tokens::{Fortitude, Preservation};
use frame_support::traits::{Get, Randomness as RandomnessT};
use frame_support::weights::Weight;
use frame_system::offchain::SubmitTransaction;
use frame_system::pallet_prelude::*;
pub use pallet::*;
use scale_info::TypeInfo;
use sp_consensus_subspace::consensus::is_proof_of_time_valid;
use sp_consensus_subspace::WrappedPotOutput;
use sp_core::H256;
use sp_domains::bundle_producer_election::BundleProducerElectionParams;
use sp_domains::{
DomainBundleLimit, DomainId, DomainInstanceData, ExecutionReceipt, OpaqueBundle, OperatorId,
OperatorPublicKey, OperatorRewardSource, OperatorSignature, ProofOfElection, RuntimeId,
SealedSingletonReceipt, DOMAIN_EXTRINSICS_SHUFFLING_SEED_SUBJECT, EMPTY_EXTRINSIC_ROOT,
};
use sp_domains_fraud_proof::fraud_proof::{
DomainRuntimeCodeAt, FraudProof, FraudProofVariant, InvalidBlockFeesProof,
InvalidDomainBlockHashProof, InvalidTransfersProof,
};
use sp_domains_fraud_proof::storage_proof::{self, BasicStorageProof, DomainRuntimeCodeProof};
use sp_domains_fraud_proof::verification::{
verify_invalid_block_fees_fraud_proof, verify_invalid_bundles_fraud_proof,
verify_invalid_domain_block_hash_fraud_proof,
verify_invalid_domain_extrinsics_root_fraud_proof, verify_invalid_state_transition_fraud_proof,
verify_invalid_transfers_fraud_proof, verify_valid_bundle_fraud_proof,
};
use sp_runtime::traits::{BlockNumberProvider, CheckedSub, Hash, Header, One, Zero};
use sp_runtime::transaction_validity::TransactionPriority;
use sp_runtime::{RuntimeAppPublic, SaturatedConversion, Saturating};
use sp_subspace_mmr::{ConsensusChainMmrLeafProof, MmrProofVerifier};
pub use staking::OperatorConfig;
use subspace_core_primitives::pot::PotOutput;
use subspace_core_primitives::{BlockHash, SlotNumber, U256};
pub const MAX_NOMINATORS_TO_SLASH: u32 = 10;
pub(crate) type BalanceOf<T> = <T as Config>::Balance;
pub(crate) type FungibleHoldId<T> =
<<T as Config>::Currency as InspectHold<<T as frame_system::Config>::AccountId>>::Reason;
pub(crate) type NominatorId<T> = <T as frame_system::Config>::AccountId;
pub trait HoldIdentifier<T: Config> {
fn staking_staked() -> FungibleHoldId<T>;
fn domain_instantiation_id() -> FungibleHoldId<T>;
fn storage_fund_withdrawal() -> FungibleHoldId<T>;
}
pub trait BlockSlot<T: frame_system::Config> {
fn future_slot(block_number: BlockNumberFor<T>) -> Option<sp_consensus_slots::Slot>;
fn slot_produced_after(to_check: sp_consensus_slots::Slot) -> Option<BlockNumberFor<T>>;
}
pub type ExecutionReceiptOf<T> = ExecutionReceipt<
BlockNumberFor<T>,
<T as frame_system::Config>::Hash,
DomainBlockNumberFor<T>,
<T as Config>::DomainHash,
BalanceOf<T>,
>;
pub type OpaqueBundleOf<T> = OpaqueBundle<
BlockNumberFor<T>,
<T as frame_system::Config>::Hash,
<T as Config>::DomainHeader,
BalanceOf<T>,
>;
pub type SingletonReceiptOf<T> = SealedSingletonReceipt<
BlockNumberFor<T>,
<T as frame_system::Config>::Hash,
<T as Config>::DomainHeader,
BalanceOf<T>,
>;
pub type FraudProofFor<T> = FraudProof<
BlockNumberFor<T>,
<T as frame_system::Config>::Hash,
<T as Config>::DomainHeader,
<T as Config>::MmrHash,
>;
#[derive(TypeInfo, Debug, Encode, Decode, Clone, PartialEq, Eq)]
pub(crate) struct ElectionVerificationParams<Balance> {
operators: BTreeMap<OperatorId, Balance>,
total_domain_stake: Balance,
}
pub type DomainBlockNumberFor<T> = <<T as Config>::DomainHeader as Header>::Number;
pub type DomainHashingFor<T> = <<T as Config>::DomainHeader as Header>::Hashing;
pub type ReceiptHashFor<T> = <<T as Config>::DomainHeader as Header>::Hash;
pub type BlockTreeNodeFor<T> = crate::block_tree::BlockTreeNode<
BlockNumberFor<T>,
<T as frame_system::Config>::Hash,
DomainBlockNumberFor<T>,
<T as Config>::DomainHash,
BalanceOf<T>,
>;
const STORAGE_VERSION: StorageVersion = StorageVersion::new(1);
const MAX_BUNDLE_PER_BLOCK: u32 = 100;
pub(crate) type StateRootOf<T> = <<T as frame_system::Config>::Hashing as Hash>::Output;
#[frame_support::pallet]
mod pallet {
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::block_tree::AcceptedReceiptType;
use crate::block_tree::{
execution_receipt_type, process_execution_receipt, prune_receipt, Error as BlockTreeError,
ReceiptType,
};
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::bundle_storage_fund::refund_storage_fee;
use crate::bundle_storage_fund::Error as BundleStorageFundError;
use crate::domain_registry::{
do_instantiate_domain, do_update_domain_allow_list, DomainConfigParams, DomainObject,
Error as DomainRegistryError,
};
use crate::runtime_registry::{
do_register_runtime, do_schedule_runtime_upgrade, do_upgrade_runtimes,
register_runtime_at_genesis, DomainRuntimeUpgradeEntry, Error as RuntimeRegistryError,
ScheduledRuntimeUpgrade,
};
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::staking::do_reward_operators;
use crate::staking::{
do_deregister_operator, do_mark_operators_as_slashed, do_nominate_operator,
do_register_operator, do_unlock_funds, do_unlock_nominator, do_withdraw_stake, Deposit,
DomainEpoch, Error as StakingError, Operator, OperatorConfig, SharePrice, StakingSummary,
WithdrawStake, Withdrawal,
};
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::staking_epoch::do_slash_operator;
use crate::staking_epoch::{do_finalize_domain_current_epoch, Error as StakingEpochError};
use crate::weights::WeightInfo;
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::DomainHashingFor;
#[cfg(not(feature = "runtime-benchmarks"))]
use crate::MAX_NOMINATORS_TO_SLASH;
use crate::{
BalanceOf, BlockSlot, BlockTreeNodeFor, DomainBlockNumberFor, ElectionVerificationParams,
ExecutionReceiptOf, FraudProofFor, HoldIdentifier, NominatorId, OpaqueBundleOf,
ReceiptHashFor, SingletonReceiptOf, StateRootOf, MAX_BUNDLE_PER_BLOCK, STORAGE_VERSION,
};
#[cfg(not(feature = "std"))]
use alloc::string::String;
#[cfg(not(feature = "std"))]
use alloc::vec;
#[cfg(not(feature = "std"))]
use alloc::vec::Vec;
use codec::FullCodec;
use domain_runtime_primitives::EVMChainId;
use frame_support::pallet_prelude::*;
use frame_support::traits::fungible::{Inspect, InspectHold, Mutate, MutateHold};
use frame_support::traits::tokens::Preservation;
use frame_support::traits::Randomness as RandomnessT;
use frame_support::weights::Weight;
use frame_support::{Identity, PalletError};
use frame_system::pallet_prelude::*;
use sp_core::H256;
use sp_domains::bundle_producer_election::ProofOfElectionError;
use sp_domains::{
BundleDigest, DomainBundleSubmitted, DomainId, DomainSudoCall, DomainsTransfersTracker,
EpochIndex, GenesisDomain, OnChainRewards, OnDomainInstantiated, OperatorAllowList,
OperatorId, OperatorPublicKey, OperatorRewardSource, OperatorSignature, RuntimeId,
RuntimeObject, RuntimeType,
};
use sp_domains_fraud_proof::fraud_proof_runtime_interface::domain_runtime_call;
use sp_domains_fraud_proof::storage_proof::{self, FraudProofStorageKeyProvider};
use sp_domains_fraud_proof::{InvalidTransactionCode, StatelessDomainRuntimeCall};
use sp_runtime::traits::{
AtLeast32BitUnsigned, BlockNumberProvider, CheckEqual, CheckedAdd, Header as HeaderT,
MaybeDisplay, One, SimpleBitOps, Zero,
};
use sp_runtime::Saturating;
use sp_std::boxed::Box;
use sp_std::collections::btree_map::BTreeMap;
use sp_std::collections::btree_set::BTreeSet;
use sp_std::fmt::Debug;
use sp_subspace_mmr::MmrProofVerifier;
use subspace_core_primitives::U256;
use subspace_runtime_primitives::StorageFee;
#[pallet::config]
pub trait Config: frame_system::Config<Hash: Into<H256>> {
type RuntimeEvent: From<Event<Self>> + IsType<<Self as frame_system::Config>::RuntimeEvent>;
type DomainHash: Parameter
+ Member
+ MaybeSerializeDeserialize
+ Debug
+ MaybeDisplay
+ SimpleBitOps
+ Ord
+ Default
+ Copy
+ CheckEqual
+ sp_std::hash::Hash
+ AsRef<[u8]>
+ AsMut<[u8]>
+ MaxEncodedLen
+ Into<H256>
+ From<H256>;
type Balance: Parameter
+ Member
+ MaybeSerializeDeserialize
+ AtLeast32BitUnsigned
+ FullCodec
+ Debug
+ MaybeDisplay
+ Default
+ Copy
+ MaxEncodedLen
+ From<u64>;
type DomainHeader: HeaderT<Hash = Self::DomainHash>;
#[pallet::constant]
type ConfirmationDepthK: Get<BlockNumberFor<Self>>;
#[pallet::constant]
type DomainRuntimeUpgradeDelay: Get<BlockNumberFor<Self>>;
type Currency: Inspect<Self::AccountId, Balance = Self::Balance>
+ Mutate<Self::AccountId>
+ InspectHold<Self::AccountId>
+ MutateHold<Self::AccountId>;
type Share: Parameter
+ Member
+ MaybeSerializeDeserialize
+ Debug
+ AtLeast32BitUnsigned
+ FullCodec
+ Copy
+ Default
+ TypeInfo
+ MaxEncodedLen
+ IsType<BalanceOf<Self>>;
type HoldIdentifier: HoldIdentifier<Self>;
#[pallet::constant]
type BlockTreePruningDepth: Get<DomainBlockNumberFor<Self>>;
#[pallet::constant]
type ConsensusSlotProbability: Get<(u64, u64)>;
#[pallet::constant]
type MaxDomainBlockSize: Get<u32>;
#[pallet::constant]
type MaxDomainBlockWeight: Get<Weight>;
#[pallet::constant]
type MaxDomainNameLength: Get<u32>;
#[pallet::constant]
type DomainInstantiationDeposit: Get<BalanceOf<Self>>;
type WeightInfo: WeightInfo;
#[pallet::constant]
type InitialDomainTxRange: Get<u64>;
#[pallet::constant]
type DomainTxRangeAdjustmentInterval: Get<u64>;
#[pallet::constant]
type MinOperatorStake: Get<BalanceOf<Self>>;
#[pallet::constant]
type MinNominatorStake: Get<BalanceOf<Self>>;
#[pallet::constant]
type StakeWithdrawalLockingPeriod: Get<DomainBlockNumberFor<Self>>;
#[pallet::constant]
type StakeEpochDuration: Get<DomainBlockNumberFor<Self>>;
#[pallet::constant]
type TreasuryAccount: Get<Self::AccountId>;
#[pallet::constant]
type MaxPendingStakingOperation: Get<u32>;
type Randomness: RandomnessT<Self::Hash, BlockNumberFor<Self>>;
#[pallet::constant]
type PalletId: Get<frame_support::PalletId>;
type StorageFee: StorageFee<BalanceOf<Self>>;
type BlockSlot: BlockSlot<Self>;
type DomainsTransfersTracker: DomainsTransfersTracker<BalanceOf<Self>>;
type MaxInitialDomainAccounts: Get<u32>;
type MinInitialDomainAccountBalance: Get<BalanceOf<Self>>;
#[pallet::constant]
type BundleLongevity: Get<u32>;
type DomainBundleSubmitted: DomainBundleSubmitted;
type OnDomainInstantiated: OnDomainInstantiated;
type MmrHash: Parameter + Member + Default + Clone;
type MmrProofVerifier: MmrProofVerifier<
Self::MmrHash,
BlockNumberFor<Self>,
StateRootOf<Self>,
>;
type FraudProofStorageKeyProvider: FraudProofStorageKeyProvider<BlockNumberFor<Self>>;
type OnChainRewards: OnChainRewards<BalanceOf<Self>>;
#[pallet::constant]
type WithdrawalLimit: Get<u32>;
}
#[pallet::pallet]
#[pallet::without_storage_info]
#[pallet::storage_version(STORAGE_VERSION)]
pub struct Pallet<T>(_);
#[pallet::storage]
pub type SuccessfulBundles<T> = StorageMap<_, Identity, DomainId, Vec<H256>, ValueQuery>;
#[pallet::storage]
pub(super) type NextRuntimeId<T> = StorageValue<_, RuntimeId, ValueQuery>;
pub struct StartingEVMChainId;
impl Get<EVMChainId> for StartingEVMChainId {
fn get() -> EVMChainId {
490000
}
}
#[pallet::storage]
pub(super) type NextEVMChainId<T> = StorageValue<_, EVMChainId, ValueQuery, StartingEVMChainId>;
#[pallet::storage]
pub type RuntimeRegistry<T: Config> =
StorageMap<_, Identity, RuntimeId, RuntimeObject<BlockNumberFor<T>, T::Hash>, OptionQuery>;
#[pallet::storage]
pub(super) type ScheduledRuntimeUpgrades<T: Config> = StorageDoubleMap<
_,
Identity,
BlockNumberFor<T>,
Identity,
RuntimeId,
ScheduledRuntimeUpgrade<T::Hash>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type NextOperatorId<T> = StorageValue<_, OperatorId, ValueQuery>;
#[pallet::storage]
pub(super) type OperatorIdOwner<T: Config> =
StorageMap<_, Identity, OperatorId, T::AccountId, OptionQuery>;
#[pallet::storage]
#[pallet::getter(fn operator_signing_key)]
pub(super) type OperatorSigningKey<T: Config> =
StorageMap<_, Identity, OperatorPublicKey, OperatorId, OptionQuery>;
#[pallet::storage]
#[pallet::getter(fn domain_staking_summary)]
pub(super) type DomainStakingSummary<T: Config> =
StorageMap<_, Identity, DomainId, StakingSummary<OperatorId, BalanceOf<T>>, OptionQuery>;
#[pallet::storage]
pub(super) type Operators<T: Config> = StorageMap<
_,
Identity,
OperatorId,
Operator<BalanceOf<T>, T::Share, DomainBlockNumberFor<T>>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type OperatorHighestSlot<T: Config> =
StorageMap<_, Identity, OperatorId, u64, ValueQuery>;
#[pallet::storage]
pub(super) type OperatorBundleSlot<T: Config> =
StorageMap<_, Identity, OperatorId, BTreeSet<u64>, ValueQuery>;
#[pallet::storage]
pub type OperatorEpochSharePrice<T: Config> =
StorageDoubleMap<_, Identity, OperatorId, Identity, DomainEpoch, SharePrice, OptionQuery>;
#[pallet::storage]
pub(super) type Deposits<T: Config> = StorageDoubleMap<
_,
Identity,
OperatorId,
Identity,
NominatorId<T>,
Deposit<T::Share, BalanceOf<T>>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type Withdrawals<T: Config> = StorageDoubleMap<
_,
Identity,
OperatorId,
Identity,
NominatorId<T>,
Withdrawal<BalanceOf<T>, T::Share, DomainBlockNumberFor<T>>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type DepositOnHold<T: Config> =
StorageMap<_, Identity, (OperatorId, NominatorId<T>), BalanceOf<T>, ValueQuery>;
#[pallet::storage]
pub(super) type NominatorCount<T: Config> =
StorageMap<_, Identity, OperatorId, u32, ValueQuery>;
#[pallet::storage]
pub(super) type PendingSlashes<T: Config> =
StorageMap<_, Identity, DomainId, BTreeSet<OperatorId>, OptionQuery>;
#[pallet::storage]
pub(super) type PendingStakingOperationCount<T: Config> =
StorageMap<_, Identity, DomainId, u32, ValueQuery>;
#[pallet::storage]
#[pallet::getter(fn next_domain_id)]
pub(super) type NextDomainId<T> = StorageValue<_, DomainId, ValueQuery>;
#[pallet::storage]
pub(super) type DomainRegistry<T: Config> = StorageMap<
_,
Identity,
DomainId,
DomainObject<BlockNumberFor<T>, ReceiptHashFor<T>, T::AccountId, BalanceOf<T>>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type BlockTree<T: Config> = StorageDoubleMap<
_,
Identity,
DomainId,
Identity,
DomainBlockNumberFor<T>,
ReceiptHashFor<T>,
OptionQuery,
>;
#[pallet::storage]
pub(super) type BlockTreeNodes<T: Config> =
StorageMap<_, Identity, ReceiptHashFor<T>, BlockTreeNodeFor<T>, OptionQuery>;
#[pallet::storage]
pub(super) type HeadReceiptNumber<T: Config> =
StorageMap<_, Identity, DomainId, DomainBlockNumberFor<T>, ValueQuery>;
#[pallet::storage]
pub(super) type NewAddedHeadReceipt<T: Config> =
StorageMap<_, Identity, DomainId, T::DomainHash, OptionQuery>;
#[pallet::storage]
#[pallet::getter(fn consensus_block_info)]
pub type ConsensusBlockHash<T: Config> =
StorageDoubleMap<_, Identity, DomainId, Identity, BlockNumberFor<T>, T::Hash, OptionQuery>;
#[pallet::storage]
pub type ExecutionInbox<T: Config> = StorageNMap<
_,
(
NMapKey<Identity, DomainId>,
NMapKey<Identity, DomainBlockNumberFor<T>>,
NMapKey<Identity, BlockNumberFor<T>>,
),
Vec<BundleDigest<T::DomainHash>>,
ValueQuery,
>;
#[pallet::storage]
pub(super) type InboxedBundleAuthor<T: Config> =
StorageMap<_, Identity, T::DomainHash, OperatorId, OptionQuery>;
#[pallet::storage]
pub(super) type HeadDomainNumber<T: Config> =
StorageMap<_, Identity, DomainId, DomainBlockNumberFor<T>, ValueQuery>;
#[pallet::storage]
pub(super) type LastEpochStakingDistribution<T: Config> =
StorageMap<_, Identity, DomainId, ElectionVerificationParams<BalanceOf<T>>, OptionQuery>;
#[pallet::storage]
#[pallet::getter(fn latest_confirmed_domain_execution_receipt)]
pub type LatestConfirmedDomainExecutionReceipt<T: Config> =
StorageMap<_, Identity, DomainId, ExecutionReceiptOf<T>, OptionQuery>;
#[pallet::storage]
#[pallet::getter(fn latest_submitted_er)]
pub(super) type LatestSubmittedER<T: Config> =
StorageMap<_, Identity, (DomainId, OperatorId), DomainBlockNumberFor<T>, ValueQuery>;
#[pallet::storage]
pub(super) type PermissionedActionAllowedBy<T: Config> =
StorageValue<_, sp_domains::PermissionedActionAllowedBy<T::AccountId>, OptionQuery>;
#[pallet::storage]
pub(super) type AccumulatedTreasuryFunds<T> = StorageValue<_, BalanceOf<T>, ValueQuery>;
#[pallet::storage]
pub(super) type DomainRuntimeUpgradeRecords<T: Config> = StorageMap<
_,
Identity,
RuntimeId,
BTreeMap<BlockNumberFor<T>, DomainRuntimeUpgradeEntry<T::Hash>>,
ValueQuery,
>;
#[pallet::storage]
pub type DomainRuntimeUpgrades<T> = StorageValue<_, Vec<RuntimeId>, ValueQuery>;
#[pallet::storage]
pub type DomainSudoCalls<T: Config> =
StorageMap<_, Identity, DomainId, DomainSudoCall, ValueQuery>;
#[pallet::storage]
pub type FrozenDomains<T> = StorageValue<_, BTreeSet<DomainId>, ValueQuery>;
#[derive(TypeInfo, Encode, Decode, PalletError, Debug, PartialEq)]
pub enum BundleError {
InvalidOperatorId,
BadBundleSignature,
BadVrfSignature,
InvalidDomainId,
BadOperator,
ThresholdUnsatisfied,
Receipt(BlockTreeError),
BundleTooLarge,
InvalidExtrinsicRoot,
InvalidProofOfTime,
SlotInTheFuture,
SlotInThePast,
BundleTooHeavy,
SlotSmallerThanPreviousBlockBundle,
EquivocatedBundle,
DomainFrozen,
UnableToPayBundleStorageFee,
UnexpectedReceiptGap,
ExpectingReceiptGap,
FailedToGetMissedUpgradeCount,
}
#[derive(TypeInfo, Encode, Decode, PalletError, Debug, PartialEq)]
pub enum FraudProofError {
BadReceiptNotFound,
ChallengingGenesisReceipt,
DescendantsOfFraudulentERNotPruned,
InvalidBlockFeesFraudProof,
InvalidTransfersFraudProof,
InvalidDomainBlockHashFraudProof,
InvalidExtrinsicRootFraudProof,
InvalidStateTransitionFraudProof,
ParentReceiptNotFound,
InvalidBundleFraudProof,
BadValidBundleFraudProof,
MissingOperator,
UnexpectedFraudProof,
BadReceiptAlreadyReported,
BadMmrProof,
UnexpectedMmrProof,
MissingMmrProof,
RuntimeNotFound,
DomainRuntimeCodeProofNotFound,
UnexpectedDomainRuntimeCodeProof,
StorageProof(storage_proof::VerificationError),
}
impl From<storage_proof::VerificationError> for FraudProofError {
fn from(err: storage_proof::VerificationError) -> Self {
FraudProofError::StorageProof(err)
}
}
impl<T> From<FraudProofError> for Error<T> {
fn from(err: FraudProofError) -> Self {
Error::FraudProof(err)
}
}
impl<T> From<RuntimeRegistryError> for Error<T> {
fn from(err: RuntimeRegistryError) -> Self {
Error::RuntimeRegistry(err)
}
}
impl<T> From<StakingError> for Error<T> {
fn from(err: StakingError) -> Self {
Error::Staking(err)
}
}
impl<T> From<StakingEpochError> for Error<T> {
fn from(err: StakingEpochError) -> Self {
Error::StakingEpoch(err)
}
}
impl<T> From<DomainRegistryError> for Error<T> {
fn from(err: DomainRegistryError) -> Self {
Error::DomainRegistry(err)
}
}
impl<T> From<BlockTreeError> for Error<T> {
fn from(err: BlockTreeError) -> Self {
Error::BlockTree(err)
}
}
impl From<ProofOfElectionError> for BundleError {
fn from(err: ProofOfElectionError) -> Self {
match err {
ProofOfElectionError::BadVrfProof => Self::BadVrfSignature,
ProofOfElectionError::ThresholdUnsatisfied => Self::ThresholdUnsatisfied,
}
}
}
impl<T> From<BundleStorageFundError> for Error<T> {
fn from(err: BundleStorageFundError) -> Self {
Error::BundleStorageFund(err)
}
}
#[pallet::error]
pub enum Error<T> {
FraudProof(FraudProofError),
RuntimeRegistry(RuntimeRegistryError),
Staking(StakingError),
StakingEpoch(StakingEpochError),
DomainRegistry(DomainRegistryError),
BlockTree(BlockTreeError),
BundleStorageFund(BundleStorageFundError),
PermissionedActionNotAllowed,
DomainSudoCallExists,
InvalidDomainSudoCall,
DomainNotFrozen,
}
#[derive(Clone, Debug, PartialEq, Encode, Decode, TypeInfo)]
pub enum SlashedReason<DomainBlock, ReceiptHash> {
InvalidBundle(DomainBlock),
BadExecutionReceipt(ReceiptHash),
}
#[pallet::event]
#[pallet::generate_deposit(pub (super) fn deposit_event)]
pub enum Event<T: Config> {
BundleStored {
domain_id: DomainId,
bundle_hash: H256,
bundle_author: OperatorId,
},
DomainRuntimeCreated {
runtime_id: RuntimeId,
runtime_type: RuntimeType,
},
DomainRuntimeUpgradeScheduled {
runtime_id: RuntimeId,
scheduled_at: BlockNumberFor<T>,
},
DomainRuntimeUpgraded {
runtime_id: RuntimeId,
},
OperatorRegistered {
operator_id: OperatorId,
domain_id: DomainId,
},
NominatedStakedUnlocked {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
unlocked_amount: BalanceOf<T>,
},
StorageFeeUnlocked {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
storage_fee: BalanceOf<T>,
},
OperatorNominated {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
amount: BalanceOf<T>,
},
DomainInstantiated {
domain_id: DomainId,
},
OperatorSwitchedDomain {
old_domain_id: DomainId,
new_domain_id: DomainId,
},
OperatorDeregistered {
operator_id: OperatorId,
},
NominatorUnlocked {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
},
WithdrewStake {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
},
PreferredOperator {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
},
OperatorRewarded {
source: OperatorRewardSource<BlockNumberFor<T>>,
operator_id: OperatorId,
reward: BalanceOf<T>,
},
OperatorTaxCollected {
operator_id: OperatorId,
tax: BalanceOf<T>,
},
DomainEpochCompleted {
domain_id: DomainId,
completed_epoch_index: EpochIndex,
},
ForceDomainEpochTransition {
domain_id: DomainId,
completed_epoch_index: EpochIndex,
},
FraudProofProcessed {
domain_id: DomainId,
new_head_receipt_number: Option<DomainBlockNumberFor<T>>,
},
DomainOperatorAllowListUpdated {
domain_id: DomainId,
},
OperatorSlashed {
operator_id: OperatorId,
reason: SlashedReason<DomainBlockNumberFor<T>, ReceiptHashFor<T>>,
},
StorageFeeDeposited {
operator_id: OperatorId,
nominator_id: NominatorId<T>,
amount: BalanceOf<T>,
},
DomainFrozen {
domain_id: DomainId,
},
DomainUnfrozen {
domain_id: DomainId,
},
PrunedExecutionReceipt {
domain_id: DomainId,
new_head_receipt_number: Option<DomainBlockNumberFor<T>>,
},
}
#[derive(Debug, Default, Decode, Encode, TypeInfo, PartialEq, Eq)]
pub struct TxRangeState {
pub tx_range: U256,
pub interval_blocks: u64,
pub interval_bundles: u64,
}
impl TxRangeState {
pub fn on_bundle(&mut self) {
self.interval_bundles += 1;
}
}
#[pallet::storage]
pub(super) type DomainTxRangeState<T: Config> =
StorageMap<_, Identity, DomainId, TxRangeState, OptionQuery>;
#[pallet::call]
impl<T: Config> Pallet<T> {
#[pallet::call_index(0)]
#[pallet::weight(Pallet::<T>::max_submit_bundle_weight())]
pub fn submit_bundle(
origin: OriginFor<T>,
opaque_bundle: OpaqueBundleOf<T>,
) -> DispatchResultWithPostInfo {
ensure_none(origin)?;
log::trace!(target: "runtime::domains", "Processing bundle: {opaque_bundle:?}");
let domain_id = opaque_bundle.domain_id();
let bundle_hash = opaque_bundle.hash();
let bundle_header_hash = opaque_bundle.sealed_header.pre_hash();
let extrinsics_root = opaque_bundle.extrinsics_root();
let operator_id = opaque_bundle.operator_id();
let bundle_size = opaque_bundle.size();
let slot_number = opaque_bundle.slot_number();
let receipt = opaque_bundle.into_receipt();
#[cfg_attr(feature = "runtime-benchmarks", allow(unused_variables))]
let receipt_block_number = receipt.domain_block_number;
#[cfg(not(feature = "runtime-benchmarks"))]
let mut actual_weight = T::WeightInfo::submit_bundle();
#[cfg(feature = "runtime-benchmarks")]
let actual_weight = T::WeightInfo::submit_bundle();
match execution_receipt_type::<T>(domain_id, &receipt) {
ReceiptType::Rejected(rejected_receipt_type) => {
return Err(Error::<T>::BlockTree(rejected_receipt_type.into()).into());
}
ReceiptType::Accepted(accepted_receipt_type) => {
#[cfg(not(feature = "runtime-benchmarks"))]
if accepted_receipt_type == AcceptedReceiptType::NewHead {
if let Some(block_tree_node) =
prune_receipt::<T>(domain_id, receipt_block_number)
.map_err(Error::<T>::from)?
{
actual_weight =
actual_weight.saturating_add(T::WeightInfo::handle_bad_receipt(
block_tree_node.operator_ids.len() as u32,
));
let bad_receipt_hash = block_tree_node
.execution_receipt
.hash::<DomainHashingFor<T>>();
do_mark_operators_as_slashed::<T>(
block_tree_node.operator_ids.into_iter(),
SlashedReason::BadExecutionReceipt(bad_receipt_hash),
)
.map_err(Error::<T>::from)?;
}
}
#[cfg_attr(feature = "runtime-benchmarks", allow(unused_variables))]
let maybe_confirmed_domain_block_info = process_execution_receipt::<T>(
domain_id,
operator_id,
receipt,
accepted_receipt_type,
)
.map_err(Error::<T>::from)?;
#[cfg(not(feature = "runtime-benchmarks"))]
if let Some(confirmed_block_info) = maybe_confirmed_domain_block_info {
actual_weight =
actual_weight.saturating_add(T::WeightInfo::confirm_domain_block(
confirmed_block_info.operator_ids.len() as u32,
confirmed_block_info.invalid_bundle_authors.len() as u32,
));
refund_storage_fee::<T>(
confirmed_block_info.total_storage_fee,
confirmed_block_info.paid_bundle_storage_fees,
)
.map_err(Error::<T>::from)?;
do_reward_operators::<T>(
domain_id,
OperatorRewardSource::Bundle {
at_block_number: confirmed_block_info.consensus_block_number,
},
confirmed_block_info.operator_ids.into_iter(),
confirmed_block_info.rewards,
)
.map_err(Error::<T>::from)?;
do_mark_operators_as_slashed::<T>(
confirmed_block_info.invalid_bundle_authors.into_iter(),
SlashedReason::InvalidBundle(confirmed_block_info.domain_block_number),
)
.map_err(Error::<T>::from)?;
}
}
}
if SuccessfulBundles::<T>::get(domain_id).is_empty() {
let missed_upgrade =
Self::missed_domain_runtime_upgrade(domain_id).map_err(Error::<T>::from)?;
let next_number = HeadDomainNumber::<T>::get(domain_id)
.checked_add(&One::one())
.ok_or::<Error<T>>(BlockTreeError::MaxHeadDomainNumber.into())?
.checked_add(&missed_upgrade.into())
.ok_or::<Error<T>>(BlockTreeError::MaxHeadDomainNumber.into())?;
#[cfg(not(feature = "runtime-benchmarks"))]
if next_number % T::StakeEpochDuration::get() == Zero::zero() {
let epoch_transition_res = do_finalize_domain_current_epoch::<T>(domain_id)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::DomainEpochCompleted {
domain_id,
completed_epoch_index: epoch_transition_res.completed_epoch_index,
});
actual_weight = actual_weight
.saturating_add(Self::actual_epoch_transition_weight(epoch_transition_res));
}
HeadDomainNumber::<T>::set(domain_id, next_number);
}
let head_domain_number = HeadDomainNumber::<T>::get(domain_id);
let consensus_block_number = frame_system::Pallet::<T>::current_block_number();
ExecutionInbox::<T>::append(
(domain_id, head_domain_number, consensus_block_number),
BundleDigest {
header_hash: bundle_header_hash,
extrinsics_root,
size: bundle_size,
},
);
InboxedBundleAuthor::<T>::insert(bundle_header_hash, operator_id);
SuccessfulBundles::<T>::append(domain_id, bundle_hash);
OperatorBundleSlot::<T>::mutate(operator_id, |slot_set| slot_set.insert(slot_number));
#[cfg(not(feature = "runtime-benchmarks"))]
{
let slashed_nominator_count =
do_slash_operator::<T>(domain_id, MAX_NOMINATORS_TO_SLASH)
.map_err(Error::<T>::from)?;
actual_weight = actual_weight
.saturating_add(T::WeightInfo::slash_operator(slashed_nominator_count));
}
Self::deposit_event(Event::BundleStored {
domain_id,
bundle_hash,
bundle_author: operator_id,
});
Ok(Some(actual_weight.min(Self::max_submit_bundle_weight())).into())
}
#[pallet::call_index(15)]
#[pallet::weight((
T::WeightInfo::submit_fraud_proof().saturating_add(
T::WeightInfo::handle_bad_receipt(MAX_BUNDLE_PER_BLOCK)
),
DispatchClass::Operational
))]
pub fn submit_fraud_proof(
origin: OriginFor<T>,
fraud_proof: Box<FraudProofFor<T>>,
) -> DispatchResultWithPostInfo {
ensure_none(origin)?;
log::trace!(target: "runtime::domains", "Processing fraud proof: {fraud_proof:?}");
#[cfg(not(feature = "runtime-benchmarks"))]
let mut actual_weight = T::WeightInfo::submit_fraud_proof();
#[cfg(feature = "runtime-benchmarks")]
let actual_weight = T::WeightInfo::submit_fraud_proof();
let domain_id = fraud_proof.domain_id();
let bad_receipt_hash = fraud_proof.targeted_bad_receipt_hash();
let head_receipt_number = HeadReceiptNumber::<T>::get(domain_id);
let bad_receipt_number = BlockTreeNodes::<T>::get(bad_receipt_hash)
.ok_or::<Error<T>>(FraudProofError::BadReceiptNotFound.into())?
.execution_receipt
.domain_block_number;
ensure!(
head_receipt_number >= bad_receipt_number,
Error::<T>::from(FraudProofError::BadReceiptNotFound),
);
#[cfg(not(feature = "runtime-benchmarks"))]
{
let block_tree_node = prune_receipt::<T>(domain_id, bad_receipt_number)
.map_err(Error::<T>::from)?
.ok_or::<Error<T>>(FraudProofError::BadReceiptNotFound.into())?;
actual_weight = actual_weight.saturating_add(T::WeightInfo::handle_bad_receipt(
(block_tree_node.operator_ids.len() as u32).min(MAX_BUNDLE_PER_BLOCK),
));
do_mark_operators_as_slashed::<T>(
block_tree_node.operator_ids.into_iter(),
SlashedReason::BadExecutionReceipt(bad_receipt_hash),
)
.map_err(Error::<T>::from)?;
}
let new_head_receipt_number = bad_receipt_number.saturating_sub(One::one());
HeadReceiptNumber::<T>::insert(domain_id, new_head_receipt_number);
Self::deposit_event(Event::FraudProofProcessed {
domain_id,
new_head_receipt_number: Some(new_head_receipt_number),
});
Ok(Some(actual_weight).into())
}
#[pallet::call_index(2)]
#[pallet::weight(T::WeightInfo::register_domain_runtime())]
pub fn register_domain_runtime(
origin: OriginFor<T>,
runtime_name: String,
runtime_type: RuntimeType,
raw_genesis_storage: Vec<u8>,
) -> DispatchResult {
ensure_root(origin)?;
let block_number = frame_system::Pallet::<T>::current_block_number();
let runtime_id = do_register_runtime::<T>(
runtime_name,
runtime_type.clone(),
raw_genesis_storage,
block_number,
)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::DomainRuntimeCreated {
runtime_id,
runtime_type,
});
Ok(())
}
#[pallet::call_index(3)]
#[pallet::weight(T::WeightInfo::upgrade_domain_runtime())]
pub fn upgrade_domain_runtime(
origin: OriginFor<T>,
runtime_id: RuntimeId,
raw_genesis_storage: Vec<u8>,
) -> DispatchResult {
ensure_root(origin)?;
let block_number = frame_system::Pallet::<T>::current_block_number();
let scheduled_at =
do_schedule_runtime_upgrade::<T>(runtime_id, raw_genesis_storage, block_number)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::DomainRuntimeUpgradeScheduled {
runtime_id,
scheduled_at,
});
Ok(())
}
#[pallet::call_index(4)]
#[pallet::weight(T::WeightInfo::register_operator())]
pub fn register_operator(
origin: OriginFor<T>,
domain_id: DomainId,
amount: BalanceOf<T>,
config: OperatorConfig<BalanceOf<T>>,
signing_key_proof_of_ownership: OperatorSignature,
) -> DispatchResult {
let owner = ensure_signed(origin)?;
let (operator_id, current_epoch_index) = do_register_operator::<T>(
owner,
domain_id,
amount,
config,
Some(signing_key_proof_of_ownership),
)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::OperatorRegistered {
operator_id,
domain_id,
});
if current_epoch_index.is_zero() {
do_finalize_domain_current_epoch::<T>(domain_id).map_err(Error::<T>::from)?;
}
Ok(())
}
#[pallet::call_index(5)]
#[pallet::weight(T::WeightInfo::nominate_operator())]
pub fn nominate_operator(
origin: OriginFor<T>,
operator_id: OperatorId,
amount: BalanceOf<T>,
) -> DispatchResult {
let nominator_id = ensure_signed(origin)?;
do_nominate_operator::<T>(operator_id, nominator_id.clone(), amount)
.map_err(Error::<T>::from)?;
Ok(())
}
#[pallet::call_index(6)]
#[pallet::weight(T::WeightInfo::instantiate_domain())]
pub fn instantiate_domain(
origin: OriginFor<T>,
domain_config_params: DomainConfigParams<T::AccountId, BalanceOf<T>>,
) -> DispatchResult {
let who = ensure_signed(origin)?;
ensure!(
PermissionedActionAllowedBy::<T>::get()
.map(|allowed_by| allowed_by.is_allowed(&who))
.unwrap_or_default(),
Error::<T>::PermissionedActionNotAllowed
);
let created_at = frame_system::Pallet::<T>::current_block_number();
let domain_id = do_instantiate_domain::<T>(domain_config_params, who, created_at)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::DomainInstantiated { domain_id });
Ok(())
}
#[pallet::call_index(8)]
#[pallet::weight(T::WeightInfo::deregister_operator())]
pub fn deregister_operator(
origin: OriginFor<T>,
operator_id: OperatorId,
) -> DispatchResult {
let who = ensure_signed(origin)?;
do_deregister_operator::<T>(who, operator_id).map_err(Error::<T>::from)?;
Self::deposit_event(Event::OperatorDeregistered { operator_id });
Ok(())
}
#[pallet::call_index(9)]
#[pallet::weight(T::WeightInfo::withdraw_stake())]
pub fn withdraw_stake(
origin: OriginFor<T>,
operator_id: OperatorId,
to_withdraw: WithdrawStake<BalanceOf<T>, T::Share>,
) -> DispatchResult {
let who = ensure_signed(origin)?;
do_withdraw_stake::<T>(operator_id, who.clone(), to_withdraw)
.map_err(Error::<T>::from)?;
Self::deposit_event(Event::WithdrewStake {
operator_id,
nominator_id: who,
});
Ok(())
}
#[pallet::call_index(10)]
#[pallet::weight(T::WeightInfo::unlock_funds())]
pub fn unlock_funds(origin: OriginFor<T>, operator_id: OperatorId) -> DispatchResult {
let nominator_id = ensure_signed(origin)?;
do_unlock_funds::<T>(operator_id, nominator_id.clone())
.map_err(crate::pallet::Error::<T>::from)?;
Ok(())
}
#[pallet::call_index(11)]
#[pallet::weight(T::WeightInfo::unlock_nominator())]
pub fn unlock_nominator(origin: OriginFor<T>, operator_id: OperatorId) -> DispatchResult {
let nominator = ensure_signed(origin)?;
do_unlock_nominator::<T>(operator_id, nominator.clone())
.map_err(crate::pallet::Error::<T>::from)?;
Self::deposit_event(Event::NominatorUnlocked {
operator_id,
nominator_id: nominator,
});
Ok(())
}
#[pallet::call_index(12)]
#[pallet::weight(T::WeightInfo::update_domain_operator_allow_list())]
pub fn update_domain_operator_allow_list(
origin: OriginFor<T>,
domain_id: DomainId,
operator_allow_list: OperatorAllowList<T::AccountId>,
) -> DispatchResult {
let who = ensure_signed(origin)?;
do_update_domain_allow_list::<T>(who, domain_id, operator_allow_list)
.map_err(Error::<T>::from)?;
Self::deposit_event(crate::pallet::Event::DomainOperatorAllowListUpdated { domain_id });
Ok(())
}
#[pallet::call_index(13)]
#[pallet::weight(Pallet::<T>::max_staking_epoch_transition())]
pub fn force_staking_epoch_transition(
origin: OriginFor<T>,
domain_id: DomainId,
) -> DispatchResultWithPostInfo {
ensure_root(origin)?;
let epoch_transition_res =
do_finalize_domain_current_epoch::<T>(domain_id).map_err(Error::<T>::from)?;
Self::deposit_event(Event::ForceDomainEpochTransition {
domain_id,
completed_epoch_index: epoch_transition_res.completed_epoch_index,
});
let actual_weight = Self::actual_epoch_transition_weight(epoch_transition_res)
.min(Self::max_staking_epoch_transition());
Ok(Some(actual_weight).into())
}
#[pallet::call_index(14)]
#[pallet::weight(<T as frame_system::Config>::DbWeight::get().reads_writes(0, 1))]
pub fn set_permissioned_action_allowed_by(
origin: OriginFor<T>,
permissioned_action_allowed_by: sp_domains::PermissionedActionAllowedBy<T::AccountId>,
) -> DispatchResult {
ensure_root(origin)?;
PermissionedActionAllowedBy::<T>::put(permissioned_action_allowed_by);
Ok(())
}
#[pallet::call_index(16)]
#[pallet::weight(<T as frame_system::Config>::DbWeight::get().reads_writes(3, 1))]
pub fn send_domain_sudo_call(
origin: OriginFor<T>,
domain_id: DomainId,
call: Vec<u8>,
) -> DispatchResult {
ensure_root(origin)?;
ensure!(
DomainSudoCalls::<T>::get(domain_id).maybe_call.is_none(),
Error::<T>::DomainSudoCallExists
);
let domain_runtime = Self::domain_runtime_code(domain_id).ok_or(
Error::<T>::DomainRegistry(DomainRegistryError::DomainNotFound),
)?;
ensure!(
domain_runtime_call(
domain_runtime,
StatelessDomainRuntimeCall::IsValidDomainSudoCall(call.clone()),
)
.unwrap_or(false),
Error::<T>::InvalidDomainSudoCall
);
DomainSudoCalls::<T>::set(
domain_id,
DomainSudoCall {
maybe_call: Some(call),
},
);
Ok(())
}
#[pallet::call_index(17)]
#[pallet::weight(<T as frame_system::Config>::DbWeight::get().reads_writes(0, 1))]
pub fn freeze_domain(origin: OriginFor<T>, domain_id: DomainId) -> DispatchResult {
ensure_root(origin)?;
FrozenDomains::<T>::mutate(|frozen_domains| frozen_domains.insert(domain_id));
Self::deposit_event(Event::DomainFrozen { domain_id });
Ok(())
}
#[pallet::call_index(18)]
#[pallet::weight(<T as frame_system::Config>::DbWeight::get().reads_writes(0, 1))]
pub fn unfreeze_domain(origin: OriginFor<T>, domain_id: DomainId) -> DispatchResult {
ensure_root(origin)?;
FrozenDomains::<T>::mutate(|frozen_domains| frozen_domains.remove(&domain_id));
Self::deposit_event(Event::DomainUnfrozen { domain_id });
Ok(())
}
#[pallet::call_index(19)]
#[pallet::weight(Pallet::<T>::max_prune_domain_execution_receipt())]
pub fn prune_domain_execution_receipt(
origin: OriginFor<T>,
domain_id: DomainId,
bad_receipt_hash: ReceiptHashFor<T>,
) -> DispatchResultWithPostInfo {
ensure_root(origin)?;
ensure!(
FrozenDomains::<T>::get().contains(&domain_id),
Error::<T>::DomainNotFrozen
);
let head_receipt_number = HeadReceiptNumber::<T>::get(domain_id);
let bad_receipt_number = BlockTreeNodes::<T>::get(bad_receipt_hash)
.ok_or::<Error<T>>(FraudProofError::BadReceiptNotFound.into())?
.execution_receipt
.domain_block_number;
ensure!(
head_receipt_number >= bad_receipt_number,
Error::<T>::from(FraudProofError::BadReceiptNotFound),
);
let mut actual_weight = T::DbWeight::get().reads(3);
let block_tree_node = prune_receipt::<T>(domain_id, bad_receipt_number)
.map_err(Error::<T>::from)?
.ok_or::<Error<T>>(FraudProofError::BadReceiptNotFound.into())?;
actual_weight = actual_weight.saturating_add(T::WeightInfo::handle_bad_receipt(
(block_tree_node.operator_ids.len() as u32).min(MAX_BUNDLE_PER_BLOCK),
));
do_mark_operators_as_slashed::<T>(
block_tree_node.operator_ids.into_iter(),
SlashedReason::BadExecutionReceipt(bad_receipt_hash),
)
.map_err(Error::<T>::from)?;
let new_head_receipt_number = bad_receipt_number.saturating_sub(One::one());
HeadReceiptNumber::<T>::insert(domain_id, new_head_receipt_number);
actual_weight = actual_weight.saturating_add(T::DbWeight::get().reads_writes(0, 1));
Self::deposit_event(Event::PrunedExecutionReceipt {
domain_id,
new_head_receipt_number: Some(new_head_receipt_number),
});
Ok(Some(actual_weight).into())
}
#[pallet::call_index(20)]
#[pallet::weight(T::WeightInfo::transfer_treasury_funds())]
pub fn transfer_treasury_funds(
origin: OriginFor<T>,
account_id: T::AccountId,
balance: BalanceOf<T>,
) -> DispatchResult {
ensure_root(origin)?;
T::Currency::transfer(
&T::TreasuryAccount::get(),
&account_id,
balance,
Preservation::Preserve,
)?;
Ok(())
}
#[pallet::call_index(21)]
#[pallet::weight(Pallet::<T>::max_submit_receipt_weight())]
pub fn submit_receipt(
origin: OriginFor<T>,
singleton_receipt: SingletonReceiptOf<T>,
) -> DispatchResultWithPostInfo {
ensure_none(origin)?;
let domain_id = singleton_receipt.domain_id();
let operator_id = singleton_receipt.operator_id();
let receipt = singleton_receipt.into_receipt();
#[cfg(not(feature = "runtime-benchmarks"))]
let mut actual_weight = T::WeightInfo::submit_receipt();
#[cfg(feature = "runtime-benchmarks")]
let actual_weight = T::WeightInfo::submit_receipt();
match execution_receipt_type::<T>(domain_id, &receipt) {
ReceiptType::Rejected(rejected_receipt_type) => {
return Err(Error::<T>::BlockTree(rejected_receipt_type.into()).into());
}
ReceiptType::Accepted(accepted_receipt_type) => {
#[cfg(not(feature = "runtime-benchmarks"))]
if accepted_receipt_type == AcceptedReceiptType::NewHead {
if let Some(block_tree_node) =
prune_receipt::<T>(domain_id, receipt.domain_block_number)
.map_err(Error::<T>::from)?
{
actual_weight =
actual_weight.saturating_add(T::WeightInfo::handle_bad_receipt(
block_tree_node.operator_ids.len() as u32,
));
let bad_receipt_hash = block_tree_node
.execution_receipt
.hash::<DomainHashingFor<T>>();
do_mark_operators_as_slashed::<T>(
block_tree_node.operator_ids.into_iter(),
SlashedReason::BadExecutionReceipt(bad_receipt_hash),
)
.map_err(Error::<T>::from)?;
}
}
#[cfg_attr(feature = "runtime-benchmarks", allow(unused_variables))]
let maybe_confirmed_domain_block_info = process_execution_receipt::<T>(
domain_id,
operator_id,
receipt,
accepted_receipt_type,
)
.map_err(Error::<T>::from)?;
#[cfg(not(feature = "runtime-benchmarks"))]
if let Some(confirmed_block_info) = maybe_confirmed_domain_block_info {
actual_weight =
actual_weight.saturating_add(T::WeightInfo::confirm_domain_block(
confirmed_block_info.operator_ids.len() as u32,
confirmed_block_info.invalid_bundle_authors.len() as u32,
));
refund_storage_fee::<T>(
confirmed_block_info.total_storage_fee,
confirmed_block_info.paid_bundle_storage_fees,
)
.map_err(Error::<T>::from)?;
do_reward_operators::<T>(
domain_id,
OperatorRewardSource::Bundle {
at_block_number: confirmed_block_info.consensus_block_number,
},
confirmed_block_info.operator_ids.into_iter(),
confirmed_block_info.rewards,
)
.map_err(Error::<T>::from)?;
do_mark_operators_as_slashed::<T>(
confirmed_block_info.invalid_bundle_authors.into_iter(),
SlashedReason::InvalidBundle(confirmed_block_info.domain_block_number),
)
.map_err(Error::<T>::from)?;
}
}
}
#[cfg(not(feature = "runtime-benchmarks"))]
{
let slashed_nominator_count =
do_slash_operator::<T>(domain_id, MAX_NOMINATORS_TO_SLASH)
.map_err(Error::<T>::from)?;
actual_weight = actual_weight
.saturating_add(T::WeightInfo::slash_operator(slashed_nominator_count));
}
Ok(Some(actual_weight.min(Self::max_submit_receipt_weight())).into())
}
}
#[pallet::genesis_config]
pub struct GenesisConfig<T: Config> {
pub permissioned_action_allowed_by:
Option<sp_domains::PermissionedActionAllowedBy<T::AccountId>>,
pub genesis_domains: Vec<GenesisDomain<T::AccountId, BalanceOf<T>>>,
}
impl<T: Config> Default for GenesisConfig<T> {
fn default() -> Self {
GenesisConfig {
permissioned_action_allowed_by: None,
genesis_domains: vec![],
}
}
}
#[pallet::genesis_build]
impl<T: Config> BuildGenesisConfig for GenesisConfig<T> {
fn build(&self) {
if let Some(permissioned_action_allowed_by) =
self.permissioned_action_allowed_by.as_ref().cloned()
{
PermissionedActionAllowedBy::<T>::put(permissioned_action_allowed_by)
}
self.genesis_domains
.clone()
.into_iter()
.for_each(|genesis_domain| {
let runtime_id = register_runtime_at_genesis::<T>(
genesis_domain.runtime_name,
genesis_domain.runtime_type,
genesis_domain.runtime_version,
genesis_domain.raw_genesis_storage,
Zero::zero(),
)
.expect("Genesis runtime registration must always succeed");
let domain_config_params = DomainConfigParams {
domain_name: genesis_domain.domain_name,
runtime_id,
maybe_bundle_limit: None,
bundle_slot_probability: genesis_domain.bundle_slot_probability,
operator_allow_list: genesis_domain.operator_allow_list,
initial_balances: genesis_domain.initial_balances,
};
let domain_owner = genesis_domain.owner_account_id;
let domain_id = do_instantiate_domain::<T>(
domain_config_params,
domain_owner.clone(),
Zero::zero(),
)
.expect("Genesis domain instantiation must always succeed");
let operator_config = OperatorConfig {
signing_key: genesis_domain.signing_key.clone(),
minimum_nominator_stake: genesis_domain.minimum_nominator_stake,
nomination_tax: genesis_domain.nomination_tax,
};
let operator_stake = T::MinOperatorStake::get();
do_register_operator::<T>(
domain_owner,
domain_id,
operator_stake,
operator_config,
None,
)
.expect("Genesis operator registration must succeed");
do_finalize_domain_current_epoch::<T>(domain_id)
.expect("Genesis epoch must succeed");
});
}
}
#[pallet::hooks]
impl<T: Config> Hooks<BlockNumberFor<T>> for Pallet<T> {
fn on_initialize(block_number: BlockNumberFor<T>) -> Weight {
let parent_number = block_number - One::one();
let parent_hash = frame_system::Pallet::<T>::block_hash(parent_number);
for runtime_id in DomainRuntimeUpgrades::<T>::take() {
let reference_count = RuntimeRegistry::<T>::get(runtime_id)
.expect("Runtime object must be present since domain is insantiated; qed")
.instance_count;
if !reference_count.is_zero() {
DomainRuntimeUpgradeRecords::<T>::mutate(runtime_id, |upgrade_record| {
upgrade_record.insert(
parent_number,
DomainRuntimeUpgradeEntry {
at_hash: parent_hash,
reference_count,
},
)
});
}
}
do_upgrade_runtimes::<T>(block_number);
for (domain_id, _) in SuccessfulBundles::<T>::drain() {
ConsensusBlockHash::<T>::insert(domain_id, parent_number, parent_hash);
T::DomainBundleSubmitted::domain_bundle_submitted(domain_id);
DomainSudoCalls::<T>::mutate(domain_id, |sudo_call| {
sudo_call.clear();
});
}
for (operator_id, slot_set) in OperatorBundleSlot::<T>::drain() {
if let Some(highest_slot) = slot_set.last() {
OperatorHighestSlot::<T>::insert(operator_id, highest_slot);
}
}
Weight::zero()
}
fn on_finalize(_: BlockNumberFor<T>) {
let _ = LastEpochStakingDistribution::<T>::clear(u32::MAX, None);
let _ = NewAddedHeadReceipt::<T>::clear(u32::MAX, None);
}
}
#[pallet::validate_unsigned]
impl<T: Config> ValidateUnsigned for Pallet<T> {
type Call = Call<T>;
fn pre_dispatch(call: &Self::Call) -> Result<(), TransactionValidityError> {
match call {
Call::submit_bundle { opaque_bundle } => {
Self::validate_submit_bundle(opaque_bundle, true)
.map_err(|_| InvalidTransaction::Call.into())
}
Call::submit_fraud_proof { fraud_proof } => Self::validate_fraud_proof(fraud_proof)
.map(|_| ())
.map_err(|_| InvalidTransaction::Call.into()),
Call::submit_receipt { singleton_receipt } => {
Self::validate_singleton_receipt(singleton_receipt, true)
.map_err(|_| InvalidTransaction::Call.into())
}
_ => Err(InvalidTransaction::Call.into()),
}
}
fn validate_unsigned(_source: TransactionSource, call: &Self::Call) -> TransactionValidity {
match call {
Call::submit_bundle { opaque_bundle } => {
let domain_id = opaque_bundle.domain_id();
let operator_id = opaque_bundle.operator_id();
let slot_number = opaque_bundle.slot_number();
if let Err(e) = Self::validate_submit_bundle(opaque_bundle, false) {
Self::log_bundle_error(&e, domain_id, operator_id);
if BundleError::UnableToPayBundleStorageFee == e {
return InvalidTransactionCode::BundleStorageFeePayment.into();
} else if let BundleError::Receipt(_) = e {
return InvalidTransactionCode::ExecutionReceipt.into();
} else {
return InvalidTransactionCode::Bundle.into();
}
}
ValidTransaction::with_tag_prefix("SubspaceSubmitBundle")
.priority(1)
.longevity(T::ConfirmationDepthK::get().try_into().unwrap_or_else(|_| {
panic!("Block number always fits in TransactionLongevity; qed")
}))
.and_provides((operator_id, slot_number))
.propagate(true)
.build()
}
Call::submit_fraud_proof { fraud_proof } => {
let (tag, priority) = match Self::validate_fraud_proof(fraud_proof) {
Err(e) => {
log::warn!(
target: "runtime::domains",
"Bad fraud proof {fraud_proof:?}, error: {e:?}",
);
return InvalidTransactionCode::FraudProof.into();
}
Ok(tp) => tp,
};
ValidTransaction::with_tag_prefix("SubspaceSubmitFraudProof")
.priority(priority)
.and_provides(tag)
.longevity(TransactionLongevity::MAX)
.propagate(true)
.build()
}
Call::submit_receipt { singleton_receipt } => {
let domain_id = singleton_receipt.domain_id();
let operator_id = singleton_receipt.operator_id();
let slot_number = singleton_receipt.slot_number();
if let Err(e) = Self::validate_singleton_receipt(singleton_receipt, false) {
Self::log_bundle_error(&e, domain_id, operator_id);
if BundleError::UnableToPayBundleStorageFee == e {
return InvalidTransactionCode::BundleStorageFeePayment.into();
} else if let BundleError::Receipt(_) = e {
return InvalidTransactionCode::ExecutionReceipt.into();
} else {
return InvalidTransactionCode::Bundle.into();
}
}
ValidTransaction::with_tag_prefix("SubspaceSubmitReceipt")
.priority(1)
.longevity(T::ConfirmationDepthK::get().try_into().unwrap_or_else(|_| {
panic!("Block number always fits in TransactionLongevity; qed")
}))
.and_provides((operator_id, slot_number))
.propagate(true)
.build()
}
_ => InvalidTransaction::Call.into(),
}
}
}
}
impl<T: Config> Pallet<T> {
fn log_bundle_error(err: &BundleError, domain_id: DomainId, operator_id: OperatorId) {
match err {
BundleError::Receipt(BlockTreeError::InFutureReceipt)
| BundleError::Receipt(BlockTreeError::StaleReceipt)
| BundleError::Receipt(BlockTreeError::NewBranchReceipt)
| BundleError::Receipt(BlockTreeError::UnavailableConsensusBlockHash)
| BundleError::Receipt(BlockTreeError::BuiltOnUnknownConsensusBlock)
| BundleError::SlotInThePast
| BundleError::SlotInTheFuture
| BundleError::InvalidProofOfTime
| BundleError::SlotSmallerThanPreviousBlockBundle
| BundleError::ExpectingReceiptGap
| BundleError::UnexpectedReceiptGap => {
log::debug!(
target: "runtime::domains",
"Bad bundle/receipt, domain {domain_id:?}, operator {operator_id:?}, error: {err:?}",
);
}
_ => {
log::warn!(
target: "runtime::domains",
"Bad bundle/receipt, domain {domain_id:?}, operator {operator_id:?}, error: {err:?}",
);
}
}
}
pub fn successful_bundles(domain_id: DomainId) -> Vec<H256> {
SuccessfulBundles::<T>::get(domain_id)
}
pub fn domain_runtime_code(domain_id: DomainId) -> Option<Vec<u8>> {
RuntimeRegistry::<T>::get(Self::runtime_id(domain_id)?)
.and_then(|mut runtime_object| runtime_object.raw_genesis.take_runtime_code())
}
pub fn domain_best_number(domain_id: DomainId) -> Result<DomainBlockNumberFor<T>, BundleError> {
let missed_upgrade = Self::missed_domain_runtime_upgrade(domain_id)
.map_err(|_| BundleError::FailedToGetMissedUpgradeCount)?;
Ok(HeadDomainNumber::<T>::get(domain_id) + missed_upgrade.into())
}
pub fn runtime_id(domain_id: DomainId) -> Option<RuntimeId> {
DomainRegistry::<T>::get(domain_id)
.map(|domain_object| domain_object.domain_config.runtime_id)
}
pub fn domain_instance_data(
domain_id: DomainId,
) -> Option<(DomainInstanceData, BlockNumberFor<T>)> {
let domain_obj = DomainRegistry::<T>::get(domain_id)?;
let runtime_object = RuntimeRegistry::<T>::get(domain_obj.domain_config.runtime_id)?;
let runtime_type = runtime_object.runtime_type.clone();
let total_issuance = domain_obj.domain_config.total_issuance()?;
let raw_genesis = into_complete_raw_genesis::<T>(
runtime_object,
domain_id,
domain_obj.domain_runtime_info,
total_issuance,
domain_obj.domain_config.initial_balances,
)
.ok()?;
Some((
DomainInstanceData {
runtime_type,
raw_genesis,
},
domain_obj.created_at,
))
}
pub fn genesis_state_root(domain_id: DomainId) -> Option<H256> {
BlockTree::<T>::get(domain_id, DomainBlockNumberFor::<T>::zero())
.and_then(BlockTreeNodes::<T>::get)
.map(|block| block.execution_receipt.final_state_root.into())
}
pub fn domain_tx_range(domain_id: DomainId) -> U256 {
DomainTxRangeState::<T>::try_get(domain_id)
.map(|state| state.tx_range)
.ok()
.unwrap_or_else(Self::initial_tx_range)
}
pub fn bundle_producer_election_params(
domain_id: DomainId,
) -> Option<BundleProducerElectionParams<BalanceOf<T>>> {
match (
DomainRegistry::<T>::get(domain_id),
DomainStakingSummary::<T>::get(domain_id),
) {
(Some(domain_object), Some(stake_summary)) => Some(BundleProducerElectionParams {
total_domain_stake: stake_summary.current_total_stake,
bundle_slot_probability: domain_object.domain_config.bundle_slot_probability,
}),
_ => None,
}
}
pub fn operator(operator_id: OperatorId) -> Option<(OperatorPublicKey, BalanceOf<T>)> {
Operators::<T>::get(operator_id)
.map(|operator| (operator.signing_key, operator.current_total_stake))
}
fn check_extrinsics_root(opaque_bundle: &OpaqueBundleOf<T>) -> Result<(), BundleError> {
let expected_extrinsics_root = <T::DomainHeader as Header>::Hashing::ordered_trie_root(
opaque_bundle
.extrinsics
.iter()
.map(|xt| xt.encode())
.collect(),
sp_core::storage::StateVersion::V1,
);
ensure!(
expected_extrinsics_root == opaque_bundle.extrinsics_root(),
BundleError::InvalidExtrinsicRoot
);
Ok(())
}
fn check_slot_and_proof_of_time(
slot_number: u64,
proof_of_time: PotOutput,
pre_dispatch: bool,
) -> Result<(), BundleError> {
let current_block_number = frame_system::Pallet::<T>::current_block_number();
if pre_dispatch {
if let Some(future_slot) = T::BlockSlot::future_slot(current_block_number) {
ensure!(slot_number <= *future_slot, BundleError::SlotInTheFuture)
}
}
let produced_after_block_number =
match T::BlockSlot::slot_produced_after(slot_number.into()) {
Some(n) => n,
None => {
if current_block_number > T::BundleLongevity::get().into() {
return Err(BundleError::SlotInThePast);
} else {
Zero::zero()
}
}
};
let produced_after_block_hash = if produced_after_block_number == current_block_number {
frame_system::Pallet::<T>::parent_hash()
} else {
frame_system::Pallet::<T>::block_hash(produced_after_block_number)
};
if let Some(last_eligible_block) =
current_block_number.checked_sub(&T::BundleLongevity::get().into())
{
ensure!(
produced_after_block_number >= last_eligible_block,
BundleError::SlotInThePast
);
}
if !is_proof_of_time_valid(
BlockHash::try_from(produced_after_block_hash.as_ref())
.expect("Must be able to convert to block hash type"),
SlotNumber::from(slot_number),
WrappedPotOutput::from(proof_of_time),
!pre_dispatch,
) {
return Err(BundleError::InvalidProofOfTime);
}
Ok(())
}
fn validate_bundle(
opaque_bundle: &OpaqueBundleOf<T>,
domain_config: &DomainConfig<T::AccountId, BalanceOf<T>>,
) -> Result<(), BundleError> {
ensure!(
opaque_bundle.body_size() <= domain_config.max_bundle_size,
BundleError::BundleTooLarge
);
ensure!(
opaque_bundle
.estimated_weight()
.all_lte(domain_config.max_bundle_weight),
BundleError::BundleTooHeavy
);
Self::check_extrinsics_root(opaque_bundle)?;
Ok(())
}
fn validate_eligibility(
to_sign: &[u8],
signature: &OperatorSignature,
proof_of_election: &ProofOfElection,
domain_config: &DomainConfig<T::AccountId, BalanceOf<T>>,
pre_dispatch: bool,
) -> Result<(), BundleError> {
let domain_id = proof_of_election.domain_id;
let operator_id = proof_of_election.operator_id;
let slot_number = proof_of_election.slot_number;
ensure!(
!FrozenDomains::<T>::get().contains(&domain_id),
BundleError::DomainFrozen
);
let operator = Operators::<T>::get(operator_id).ok_or(BundleError::InvalidOperatorId)?;
let operator_status = operator.status::<T>(operator_id);
ensure!(
*operator_status != OperatorStatus::Slashed
&& *operator_status != OperatorStatus::PendingSlash,
BundleError::BadOperator
);
if !operator.signing_key.verify(&to_sign, signature) {
return Err(BundleError::BadBundleSignature);
}
ensure!(
slot_number
> Self::operator_highest_slot_from_previous_block(operator_id, pre_dispatch),
BundleError::SlotSmallerThanPreviousBlockBundle,
);
ensure!(
!OperatorBundleSlot::<T>::get(operator_id).contains(&slot_number),
BundleError::EquivocatedBundle,
);
let (operator_stake, total_domain_stake) =
Self::fetch_operator_stake_info(domain_id, &operator_id)?;
Self::check_slot_and_proof_of_time(
slot_number,
proof_of_election.proof_of_time,
pre_dispatch,
)?;
sp_domains::bundle_producer_election::check_proof_of_election(
&operator.signing_key,
domain_config.bundle_slot_probability,
proof_of_election,
operator_stake.saturated_into(),
total_domain_stake.saturated_into(),
)?;
Ok(())
}
fn validate_submit_bundle(
opaque_bundle: &OpaqueBundleOf<T>,
pre_dispatch: bool,
) -> Result<(), BundleError> {
let domain_id = opaque_bundle.domain_id();
let operator_id = opaque_bundle.operator_id();
let sealed_header = &opaque_bundle.sealed_header;
ensure!(
Self::receipt_gap(domain_id)? <= One::one(),
BundleError::UnexpectedReceiptGap,
);
let domain_config = DomainRegistry::<T>::get(domain_id)
.ok_or(BundleError::InvalidDomainId)?
.domain_config;
Self::validate_bundle(opaque_bundle, &domain_config)?;
Self::validate_eligibility(
sealed_header.pre_hash().as_ref(),
&sealed_header.signature,
&sealed_header.header.proof_of_election,
&domain_config,
pre_dispatch,
)?;
verify_execution_receipt::<T>(domain_id, &sealed_header.header.receipt)
.map_err(BundleError::Receipt)?;
charge_bundle_storage_fee::<T>(operator_id, opaque_bundle.size())
.map_err(|_| BundleError::UnableToPayBundleStorageFee)?;
Ok(())
}
fn validate_singleton_receipt(
sealed_singleton_receipt: &SingletonReceiptOf<T>,
pre_dispatch: bool,
) -> Result<(), BundleError> {
let domain_id = sealed_singleton_receipt.domain_id();
let operator_id = sealed_singleton_receipt.operator_id();
ensure!(
Self::receipt_gap(domain_id)? > One::one(),
BundleError::ExpectingReceiptGap,
);
let domain_config = DomainRegistry::<T>::get(domain_id)
.ok_or(BundleError::InvalidDomainId)?
.domain_config;
Self::validate_eligibility(
sealed_singleton_receipt.pre_hash().as_ref(),
&sealed_singleton_receipt.signature,
&sealed_singleton_receipt.singleton_receipt.proof_of_election,
&domain_config,
pre_dispatch,
)?;
verify_execution_receipt::<T>(
domain_id,
&sealed_singleton_receipt.singleton_receipt.receipt,
)
.map_err(BundleError::Receipt)?;
charge_bundle_storage_fee::<T>(operator_id, sealed_singleton_receipt.size())
.map_err(|_| BundleError::UnableToPayBundleStorageFee)?;
Ok(())
}
fn validate_fraud_proof(
fraud_proof: &FraudProofFor<T>,
) -> Result<(DomainId, TransactionPriority), FraudProofError> {
let domain_id = fraud_proof.domain_id();
let bad_receipt_hash = fraud_proof.targeted_bad_receipt_hash();
let bad_receipt = BlockTreeNodes::<T>::get(bad_receipt_hash)
.ok_or(FraudProofError::BadReceiptNotFound)?
.execution_receipt;
let bad_receipt_domain_block_number = bad_receipt.domain_block_number;
ensure!(
!bad_receipt_domain_block_number.is_zero(),
FraudProofError::ChallengingGenesisReceipt
);
ensure!(
!Self::is_bad_er_pending_to_prune(domain_id, bad_receipt_domain_block_number),
FraudProofError::BadReceiptAlreadyReported,
);
ensure!(
!fraud_proof.is_unexpected_domain_runtime_code_proof(),
FraudProofError::UnexpectedDomainRuntimeCodeProof,
);
ensure!(
!fraud_proof.is_unexpected_mmr_proof(),
FraudProofError::UnexpectedMmrProof,
);
let maybe_state_root = match &fraud_proof.maybe_mmr_proof {
Some(mmr_proof) => Some(Self::verify_mmr_proof_and_extract_state_root(
mmr_proof.clone(),
bad_receipt.consensus_block_number,
)?),
None => None,
};
match &fraud_proof.proof {
FraudProofVariant::InvalidBlockFees(InvalidBlockFeesProof { storage_proof }) => {
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
verify_invalid_block_fees_fraud_proof::<
T::Block,
DomainBlockNumberFor<T>,
T::DomainHash,
BalanceOf<T>,
DomainHashingFor<T>,
>(bad_receipt, storage_proof, domain_runtime_code)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Block fees proof verification failed: {err:?}"
);
FraudProofError::InvalidBlockFeesFraudProof
})?;
}
FraudProofVariant::InvalidTransfers(InvalidTransfersProof { storage_proof }) => {
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
verify_invalid_transfers_fraud_proof::<
T::Block,
DomainBlockNumberFor<T>,
T::DomainHash,
BalanceOf<T>,
DomainHashingFor<T>,
>(bad_receipt, storage_proof, domain_runtime_code)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Domain transfers proof verification failed: {err:?}"
);
FraudProofError::InvalidTransfersFraudProof
})?;
}
FraudProofVariant::InvalidDomainBlockHash(InvalidDomainBlockHashProof {
digest_storage_proof,
}) => {
let parent_receipt =
BlockTreeNodes::<T>::get(bad_receipt.parent_domain_block_receipt_hash)
.ok_or(FraudProofError::ParentReceiptNotFound)?
.execution_receipt;
verify_invalid_domain_block_hash_fraud_proof::<
T::Block,
BalanceOf<T>,
T::DomainHeader,
>(
bad_receipt,
digest_storage_proof.clone(),
parent_receipt.domain_block_hash,
)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Invalid Domain block hash proof verification failed: {err:?}"
);
FraudProofError::InvalidDomainBlockHashFraudProof
})?;
}
FraudProofVariant::InvalidExtrinsicsRoot(proof) => {
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
let runtime_id =
Self::runtime_id(domain_id).ok_or(FraudProofError::RuntimeNotFound)?;
let state_root = maybe_state_root.ok_or(FraudProofError::MissingMmrProof)?;
verify_invalid_domain_extrinsics_root_fraud_proof::<
T::Block,
BalanceOf<T>,
T::DomainHeader,
T::Hashing,
T::FraudProofStorageKeyProvider,
>(
bad_receipt,
proof,
domain_id,
runtime_id,
state_root,
domain_runtime_code,
)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Invalid Domain extrinsic root proof verification failed: {err:?}"
);
FraudProofError::InvalidExtrinsicRootFraudProof
})?;
}
FraudProofVariant::InvalidStateTransition(proof) => {
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
let bad_receipt_parent =
BlockTreeNodes::<T>::get(bad_receipt.parent_domain_block_receipt_hash)
.ok_or(FraudProofError::ParentReceiptNotFound)?
.execution_receipt;
verify_invalid_state_transition_fraud_proof::<
T::Block,
T::DomainHeader,
BalanceOf<T>,
>(bad_receipt, bad_receipt_parent, proof, domain_runtime_code)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Invalid State transition proof verification failed: {err:?}"
);
FraudProofError::InvalidStateTransitionFraudProof
})?;
}
FraudProofVariant::InvalidBundles(proof) => {
let state_root = maybe_state_root.ok_or(FraudProofError::MissingMmrProof)?;
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
let bad_receipt_parent =
BlockTreeNodes::<T>::get(bad_receipt.parent_domain_block_receipt_hash)
.ok_or(FraudProofError::ParentReceiptNotFound)?
.execution_receipt;
verify_invalid_bundles_fraud_proof::<
T::Block,
T::DomainHeader,
T::MmrHash,
BalanceOf<T>,
T::FraudProofStorageKeyProvider,
T::MmrProofVerifier,
>(
bad_receipt,
bad_receipt_parent,
proof,
domain_id,
state_root,
domain_runtime_code,
)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Invalid Bundle proof verification failed: {err:?}"
);
FraudProofError::InvalidBundleFraudProof
})?;
}
FraudProofVariant::ValidBundle(proof) => {
let state_root = maybe_state_root.ok_or(FraudProofError::MissingMmrProof)?;
let domain_runtime_code = Self::get_domain_runtime_code_for_receipt(
domain_id,
&bad_receipt,
fraud_proof.maybe_domain_runtime_code_proof.clone(),
)?;
verify_valid_bundle_fraud_proof::<
T::Block,
T::DomainHeader,
BalanceOf<T>,
T::FraudProofStorageKeyProvider,
>(
bad_receipt,
proof,
domain_id,
state_root,
domain_runtime_code,
)
.map_err(|err| {
log::error!(
target: "runtime::domains",
"Valid bundle proof verification failed: {err:?}"
);
FraudProofError::BadValidBundleFraudProof
})?
}
#[cfg(any(feature = "std", feature = "runtime-benchmarks"))]
FraudProofVariant::Dummy => {}
}
let block_before_bad_er_confirm = bad_receipt_domain_block_number.saturating_sub(
Self::latest_confirmed_domain_block_number(fraud_proof.domain_id()),
);
let priority =
TransactionPriority::MAX - block_before_bad_er_confirm.saturated_into::<u64>();
let tag = fraud_proof.domain_id();
Ok((tag, priority))
}
fn fetch_operator_stake_info(
domain_id: DomainId,
operator_id: &OperatorId,
) -> Result<(BalanceOf<T>, BalanceOf<T>), BundleError> {
if let Some(pending_election_params) = LastEpochStakingDistribution::<T>::get(domain_id) {
if let Some(operator_stake) = pending_election_params.operators.get(operator_id) {
return Ok((*operator_stake, pending_election_params.total_domain_stake));
}
}
let domain_stake_summary =
DomainStakingSummary::<T>::get(domain_id).ok_or(BundleError::InvalidDomainId)?;
let operator_stake = domain_stake_summary
.current_operators
.get(operator_id)
.ok_or(BundleError::BadOperator)?;
Ok((*operator_stake, domain_stake_summary.current_total_stake))
}
fn initial_tx_range() -> U256 {
U256::MAX / T::InitialDomainTxRange::get()
}
pub fn head_receipt_number(domain_id: DomainId) -> DomainBlockNumberFor<T> {
HeadReceiptNumber::<T>::get(domain_id)
}
pub fn oldest_unconfirmed_receipt_number(
domain_id: DomainId,
) -> Option<DomainBlockNumberFor<T>> {
let oldest_nonconfirmed_er_number =
Self::latest_confirmed_domain_block_number(domain_id).saturating_add(One::one());
let is_er_exist = BlockTree::<T>::get(domain_id, oldest_nonconfirmed_er_number).is_some();
let is_pending_to_prune =
Self::is_bad_er_pending_to_prune(domain_id, oldest_nonconfirmed_er_number);
if is_er_exist && !is_pending_to_prune {
Some(oldest_nonconfirmed_er_number)
} else {
None
}
}
pub fn latest_confirmed_domain_block_number(domain_id: DomainId) -> DomainBlockNumberFor<T> {
LatestConfirmedDomainExecutionReceipt::<T>::get(domain_id)
.map(|er| er.domain_block_number)
.unwrap_or_default()
}
pub fn latest_confirmed_domain_block(
domain_id: DomainId,
) -> Option<(DomainBlockNumberFor<T>, T::DomainHash)> {
LatestConfirmedDomainExecutionReceipt::<T>::get(domain_id)
.map(|er| (er.domain_block_number, er.domain_block_hash))
}
pub fn domain_bundle_limit(
domain_id: DomainId,
) -> Result<Option<DomainBundleLimit>, DomainRegistryError> {
let domain_config = match DomainRegistry::<T>::get(domain_id) {
None => return Ok(None),
Some(domain_obj) => domain_obj.domain_config,
};
Ok(Some(DomainBundleLimit {
max_bundle_size: domain_config.max_bundle_size,
max_bundle_weight: domain_config.max_bundle_weight,
}))
}
pub fn non_empty_er_exists(domain_id: DomainId) -> bool {
if BlockTree::<T>::contains_key(domain_id, DomainBlockNumberFor::<T>::zero()) {
return true;
}
let mut to_check =
Self::latest_confirmed_domain_block_number(domain_id).saturating_add(One::one());
let head_number = HeadDomainNumber::<T>::get(domain_id);
while to_check <= head_number {
if !ExecutionInbox::<T>::iter_prefix_values((domain_id, to_check)).all(|digests| {
digests
.iter()
.all(|digest| digest.extrinsics_root == EMPTY_EXTRINSIC_ROOT.into())
}) {
return true;
}
to_check = to_check.saturating_add(One::one())
}
false
}
pub fn extrinsics_shuffling_seed() -> T::Hash {
let seed = DOMAIN_EXTRINSICS_SHUFFLING_SEED_SUBJECT;
let (randomness, _) = T::Randomness::random(seed);
randomness
}
pub fn execution_receipt(receipt_hash: ReceiptHashFor<T>) -> Option<ExecutionReceiptOf<T>> {
BlockTreeNodes::<T>::get(receipt_hash).map(|db| db.execution_receipt)
}
pub fn receipt_hash(
domain_id: DomainId,
domain_number: DomainBlockNumberFor<T>,
) -> Option<ReceiptHashFor<T>> {
BlockTree::<T>::get(domain_id, domain_number)
}
pub fn confirmed_domain_block_storage_key(domain_id: DomainId) -> Vec<u8> {
LatestConfirmedDomainExecutionReceipt::<T>::hashed_key_for(domain_id)
}
pub fn is_bad_er_pending_to_prune(
domain_id: DomainId,
receipt_number: DomainBlockNumberFor<T>,
) -> bool {
if receipt_number.is_zero() {
return false;
}
let head_receipt_number = HeadReceiptNumber::<T>::get(domain_id);
head_receipt_number < receipt_number
}
pub fn is_operator_pending_to_slash(domain_id: DomainId, operator_id: OperatorId) -> bool {
let latest_submitted_er = LatestSubmittedER::<T>::get((domain_id, operator_id));
if latest_submitted_er.is_zero() {
return false;
}
let head_receipt_number = HeadReceiptNumber::<T>::get(domain_id);
head_receipt_number < latest_submitted_er
}
pub fn max_submit_bundle_weight() -> Weight {
T::WeightInfo::submit_bundle()
.saturating_add(
T::WeightInfo::handle_bad_receipt(MAX_BUNDLE_PER_BLOCK).max(
T::WeightInfo::confirm_domain_block(MAX_BUNDLE_PER_BLOCK, MAX_BUNDLE_PER_BLOCK),
),
)
.saturating_add(Self::max_staking_epoch_transition())
.saturating_add(T::WeightInfo::slash_operator(MAX_NOMINATORS_TO_SLASH))
}
pub fn max_submit_receipt_weight() -> Weight {
T::WeightInfo::submit_bundle()
.saturating_add(
T::WeightInfo::handle_bad_receipt(MAX_BUNDLE_PER_BLOCK).max(
T::WeightInfo::confirm_domain_block(MAX_BUNDLE_PER_BLOCK, MAX_BUNDLE_PER_BLOCK),
),
)
.saturating_add(T::WeightInfo::slash_operator(MAX_NOMINATORS_TO_SLASH))
}
pub fn max_staking_epoch_transition() -> Weight {
T::WeightInfo::operator_reward_tax_and_restake(MAX_BUNDLE_PER_BLOCK).saturating_add(
T::WeightInfo::finalize_domain_epoch_staking(T::MaxPendingStakingOperation::get()),
)
}
pub fn max_prune_domain_execution_receipt() -> Weight {
T::WeightInfo::handle_bad_receipt(MAX_BUNDLE_PER_BLOCK)
.saturating_add(T::DbWeight::get().reads_writes(3, 1))
}
fn actual_epoch_transition_weight(epoch_transition_res: EpochTransitionResult) -> Weight {
let EpochTransitionResult {
rewarded_operator_count,
finalized_operator_count,
completed_epoch_index: _,
} = epoch_transition_res;
T::WeightInfo::operator_reward_tax_and_restake(rewarded_operator_count).saturating_add(
T::WeightInfo::finalize_domain_epoch_staking(finalized_operator_count),
)
}
pub fn reward_domain_operators(
domain_id: DomainId,
source: OperatorRewardSource<BlockNumberFor<T>>,
rewards: BalanceOf<T>,
) {
if let Some(domain_stake_summary) = DomainStakingSummary::<T>::get(domain_id) {
let operators = domain_stake_summary
.current_epoch_rewards
.into_keys()
.collect::<Vec<OperatorId>>();
let _ = do_reward_operators::<T>(domain_id, source, operators.into_iter(), rewards);
}
}
pub fn storage_fund_account_balance(operator_id: OperatorId) -> BalanceOf<T> {
let storage_fund_acc = storage_fund_account::<T>(operator_id);
T::Currency::reducible_balance(&storage_fund_acc, Preservation::Preserve, Fortitude::Polite)
}
pub fn operator_highest_slot_from_previous_block(
operator_id: OperatorId,
pre_dispatch: bool,
) -> u64 {
if pre_dispatch {
OperatorHighestSlot::<T>::get(operator_id)
} else {
*OperatorBundleSlot::<T>::get(operator_id)
.last()
.unwrap_or(&OperatorHighestSlot::<T>::get(operator_id))
}
}
pub fn get_domain_runtime_code_for_receipt(
domain_id: DomainId,
receipt: &ExecutionReceiptOf<T>,
maybe_domain_runtime_code_at: Option<
DomainRuntimeCodeAt<BlockNumberFor<T>, T::Hash, T::MmrHash>,
>,
) -> Result<Vec<u8>, FraudProofError> {
let runtime_id = Self::runtime_id(domain_id).ok_or(FraudProofError::RuntimeNotFound)?;
let current_runtime_obj =
RuntimeRegistry::<T>::get(runtime_id).ok_or(FraudProofError::RuntimeNotFound)?;
let at = {
let parent_receipt = BlockTreeNodes::<T>::get(receipt.parent_domain_block_receipt_hash)
.ok_or(FraudProofError::ParentReceiptNotFound)?
.execution_receipt;
parent_receipt.consensus_block_number
};
let is_domain_runtime_updraded = current_runtime_obj.updated_at >= at;
let mut runtime_obj = match (is_domain_runtime_updraded, maybe_domain_runtime_code_at) {
(true, None) => return Err(FraudProofError::DomainRuntimeCodeProofNotFound),
(true, Some(domain_runtime_code_at)) => {
let DomainRuntimeCodeAt {
mmr_proof,
domain_runtime_code_proof,
} = domain_runtime_code_at;
let state_root = Self::verify_mmr_proof_and_extract_state_root(mmr_proof, at)?;
<DomainRuntimeCodeProof as BasicStorageProof<T::Block>>::verify::<
T::FraudProofStorageKeyProvider,
>(domain_runtime_code_proof, runtime_id, &state_root)?
}
(false, Some(_)) => return Err(FraudProofError::UnexpectedDomainRuntimeCodeProof),
(false, None) => current_runtime_obj,
};
let code = runtime_obj
.raw_genesis
.take_runtime_code()
.ok_or(storage_proof::VerificationError::RuntimeCodeNotFound)?;
Ok(code)
}
pub fn is_domain_runtime_upgraded_since(
domain_id: DomainId,
at: BlockNumberFor<T>,
) -> Option<bool> {
Self::runtime_id(domain_id)
.and_then(RuntimeRegistry::<T>::get)
.map(|runtime_obj| runtime_obj.updated_at >= at)
}
pub fn verify_mmr_proof_and_extract_state_root(
mmr_leaf_proof: ConsensusChainMmrLeafProof<BlockNumberFor<T>, T::Hash, T::MmrHash>,
expected_block_number: BlockNumberFor<T>,
) -> Result<T::Hash, FraudProofError> {
let leaf_data = T::MmrProofVerifier::verify_proof_and_extract_leaf(mmr_leaf_proof)
.ok_or(FraudProofError::BadMmrProof)?;
if expected_block_number != leaf_data.block_number() {
return Err(FraudProofError::UnexpectedMmrProof);
}
Ok(leaf_data.state_root())
}
fn missed_domain_runtime_upgrade(domain_id: DomainId) -> Result<u32, BlockTreeError> {
let runtime_id = Self::runtime_id(domain_id).ok_or(BlockTreeError::RuntimeNotFound)?;
let last_domain_block_number = HeadDomainNumber::<T>::get(domain_id);
let last_block_at =
ExecutionInbox::<T>::iter_key_prefix((domain_id, last_domain_block_number))
.next()
.or(DomainRegistry::<T>::get(domain_id).map(|domain_obj| domain_obj.created_at))
.ok_or(BlockTreeError::LastBlockNotFound)?;
Ok(DomainRuntimeUpgradeRecords::<T>::get(runtime_id)
.into_keys()
.rev()
.take_while(|upgraded_at| *upgraded_at > last_block_at)
.count() as u32)
}
pub fn is_domain_registered(domain_id: DomainId) -> bool {
DomainStakingSummary::<T>::contains_key(domain_id)
}
pub fn domain_sudo_call(domain_id: DomainId) -> Option<Vec<u8>> {
DomainSudoCalls::<T>::get(domain_id).maybe_call
}
pub fn receipt_gap(domain_id: DomainId) -> Result<DomainBlockNumberFor<T>, BundleError> {
let domain_best_number = Self::domain_best_number(domain_id)?;
let head_receipt_number = HeadReceiptNumber::<T>::get(domain_id);
Ok(domain_best_number.saturating_sub(head_receipt_number))
}
}
impl<T: Config> sp_domains::DomainOwner<T::AccountId> for Pallet<T> {
fn is_domain_owner(domain_id: DomainId, acc: T::AccountId) -> bool {
if let Some(domain_obj) = DomainRegistry::<T>::get(domain_id) {
domain_obj.owner_account_id == acc
} else {
false
}
}
}
impl<T> Pallet<T>
where
T: Config + frame_system::offchain::SendTransactionTypes<Call<T>>,
{
pub fn submit_bundle_unsigned(opaque_bundle: OpaqueBundleOf<T>) {
let slot = opaque_bundle.sealed_header.slot_number();
let extrincis_count = opaque_bundle.extrinsics.len();
let call = Call::submit_bundle { opaque_bundle };
match SubmitTransaction::<T, Call<T>>::submit_unsigned_transaction(call.into()) {
Ok(()) => {
log::info!(
target: "runtime::domains",
"Submitted bundle from slot {slot}, extrinsics: {extrincis_count}",
);
}
Err(()) => {
log::error!(target: "runtime::domains", "Error submitting bundle");
}
}
}
pub fn submit_receipt_unsigned(singleton_receipt: SingletonReceiptOf<T>) {
let slot = singleton_receipt.slot_number();
let domain_block_number = singleton_receipt.receipt().domain_block_number;
let call = Call::submit_receipt { singleton_receipt };
match SubmitTransaction::<T, Call<T>>::submit_unsigned_transaction(call.into()) {
Ok(()) => {
log::info!(
target: "runtime::domains",
"Submitted singleton receipt from slot {slot}, domain_block_number: {domain_block_number:?}",
);
}
Err(()) => {
log::error!(target: "runtime::domains", "Error submitting singleton receipt");
}
}
}
pub fn submit_fraud_proof_unsigned(fraud_proof: FraudProofFor<T>) {
let call = Call::submit_fraud_proof {
fraud_proof: Box::new(fraud_proof),
};
match SubmitTransaction::<T, Call<T>>::submit_unsigned_transaction(call.into()) {
Ok(()) => {
log::info!(target: "runtime::domains", "Submitted fraud proof");
}
Err(()) => {
log::error!(target: "runtime::domains", "Error submitting fraud proof");
}
}
}
}
pub fn calculate_tx_range(
cur_tx_range: U256,
actual_bundle_count: u64,
expected_bundle_count: u64,
) -> U256 {
if actual_bundle_count == 0 || expected_bundle_count == 0 {
return cur_tx_range;
}
let Some(new_tx_range) = U256::from(actual_bundle_count)
.saturating_mul(&cur_tx_range)
.checked_div(&U256::from(expected_bundle_count))
else {
return cur_tx_range;
};
let upper_bound = cur_tx_range.saturating_mul(&U256::from(4_u64));
let Some(lower_bound) = cur_tx_range.checked_div(&U256::from(4_u64)) else {
return cur_tx_range;
};
new_tx_range.clamp(lower_bound, upper_bound)
}