Verify Batch signatures

Starts further fleshing out the Serai client tests with common utils.
This commit is contained in:
Luke Parker 2023-03-31 06:34:09 -04:00
parent 30b8636641
commit ae4e98c052
No known key found for this signature in database
14 changed files with 291 additions and 201 deletions

2
Cargo.lock generated
View file

@ -3861,9 +3861,11 @@ dependencies = [
"parity-scale-codec",
"scale-info",
"serai-primitives",
"sp-application-crypto",
"sp-runtime",
"thiserror",
"tokens-pallet",
"validator-sets-pallet",
]
[[package]]

View file

@ -17,6 +17,7 @@ thiserror = { version = "1", optional = true }
scale = { package = "parity-scale-codec", version = "3", default-features = false, features = ["derive", "max-encoded-len"] }
scale-info = { version = "2", default-features = false, features = ["derive"] }
sp-application-crypto = { git = "https://github.com/serai-dex/substrate", default-features = false }
sp-runtime = { git = "https://github.com/serai-dex/substrate", default-features = false }
frame-system = { git = "https://github.com/serai-dex/substrate", default-features = false }
@ -26,6 +27,7 @@ serai-primitives = { path = "../../serai/primitives", default-features = false }
in-instructions-primitives = { path = "../primitives", default-features = false }
tokens-pallet = { path = "../../tokens/pallet", default-features = false }
validator-sets-pallet = { path = "../../validator-sets/pallet", default-features = false }
[features]
std = [

View file

@ -22,15 +22,23 @@ pub enum PalletError {
#[frame_support::pallet]
pub mod pallet {
use sp_application_crypto::RuntimePublic;
use frame_support::pallet_prelude::*;
use frame_system::pallet_prelude::*;
use tokens_pallet::{Config as TokensConfig, Pallet as Tokens};
use validator_sets_pallet::{
primitives::{Session, ValidatorSet},
Config as ValidatorSetsConfig, Pallet as ValidatorSets,
};
use super::*;
#[pallet::config]
pub trait Config: frame_system::Config<BlockNumber = u64> + TokensConfig {
pub trait Config:
frame_system::Config<BlockNumber = u64> + ValidatorSetsConfig + TokensConfig
{
type RuntimeEvent: From<Event<Self>> + IsType<<Self as frame_system::Config>::RuntimeEvent>;
}
@ -101,9 +109,31 @@ pub mod pallet {
let network = batch.batch.network;
// TODO: Get the key for this network or Err(UnrecognizedNetwork)
// TODO: Get the latest session
let session = Session(0);
// TODO: Verify the signature or Err(InvalidSignature)
let mut set = ValidatorSet { session, network };
// TODO: If this session just set their keys, it'll invalidate anything in the mempool
// Should there be a transitory period/future-set cut off?
let key = if let Some(keys) = ValidatorSets::<T>::keys(set) {
keys.0
} else {
// If this set hasn't set their keys yet, use the previous set's
if set.session.0 == 0 {
Err(InvalidTransaction::BadProof)?;
}
set.session.0 -= 1;
if let Some(keys) = ValidatorSets::<T>::keys(set) {
keys.0
} else {
Err(InvalidTransaction::BadProof)?
}
};
if !key.verify(&batch.batch.encode(), &batch.signature) {
Err(InvalidTransaction::BadProof)?;
}
// Verify the batch is sequential
// Batches has the last ID set. The next ID should be it + 1

View file

@ -88,12 +88,6 @@ pub fn development_config() -> Result<ChainSpec, &'static str> {
account_from_name("Dave"),
account_from_name("Eve"),
account_from_name("Ferdie"),
account_from_name("Alice//stash"),
account_from_name("Bob//stash"),
account_from_name("Charlie//stash"),
account_from_name("Dave//stash"),
account_from_name("Eve//stash"),
account_from_name("Ferdie//stash"),
],
)
},
@ -132,12 +126,6 @@ pub fn testnet_config() -> Result<ChainSpec, &'static str> {
account_from_name("Dave"),
account_from_name("Eve"),
account_from_name("Ferdie"),
account_from_name("Alice//stash"),
account_from_name("Bob//stash"),
account_from_name("Charlie//stash"),
account_from_name("Dave//stash"),
account_from_name("Eve//stash"),
account_from_name("Ferdie//stash"),
],
)
},

View file

@ -2,14 +2,14 @@ use ::scale::Encode;
use scale_info::{MetaType, TypeInfo, Registry, PortableRegistry};
use subxt::ext::scale_value::{Composite, ValueDef, Value, scale};
pub(crate) fn scale_value<V: Encode + TypeInfo + 'static>(value: V) -> Value {
pub(crate) fn scale_value<V: 'static + Encode + TypeInfo>(value: V) -> Value {
let mut registry = Registry::new();
let id = registry.register_type(&MetaType::new::<V>()).id;
let registry = PortableRegistry::from(registry);
scale::decode_as_type(&mut value.encode().as_ref(), id, &registry).unwrap().remove_context()
}
pub(crate) fn scale_composite<V: Encode + TypeInfo + 'static>(value: V) -> Composite<()> {
pub(crate) fn scale_composite<V: 'static + Encode + TypeInfo>(value: V) -> Composite<()> {
match scale_value(value).value {
ValueDef::Composite(composite) => composite,
ValueDef::Variant(variant) => variant.values,

View file

@ -1,10 +1,15 @@
use rand_core::{RngCore, OsRng};
use sp_core::sr25519::Signature;
use scale::Encode;
use sp_core::Pair;
use serai_client::{
primitives::{BITCOIN_NET_ID, BITCOIN, BlockHash, SeraiAddress, Amount, Balance},
primitives::{
BITCOIN_NET_ID, BITCOIN, BlockHash, SeraiAddress, Amount, Balance, insecure_pair_from_name,
},
tokens::TokensEvent,
validator_sets::primitives::{Session, ValidatorSet},
in_instructions::{
primitives::{InInstruction, InInstructionWithBalance, Batch, SignedBatch},
InInstructionsEvent,
@ -12,8 +17,8 @@ use serai_client::{
Serai,
};
mod runner;
use runner::{URL, provide_batch};
mod common;
use common::{serai, validator_sets::vote_in_key, in_instructions::provide_batch};
serai_test!(
async fn publish_batch() {
@ -39,10 +44,15 @@ serai_test!(
balance,
}],
};
let signed = SignedBatch { batch, signature: Signature::from_raw([0; 64]) };
let batch_pair = insecure_pair_from_name("Bitcoin");
let key_pair = (batch_pair.public(), vec![].try_into().unwrap());
vote_in_key(ValidatorSet { session: Session(0), network: BITCOIN_NET_ID }, key_pair).await;
let signature = batch_pair.sign(&batch.encode());
let signed = SignedBatch { batch, signature };
let block = provide_batch(signed).await;
let serai = Serai::new(URL).await.unwrap();
let serai = serai().await;
let batches = serai.get_batch_events(block).await.unwrap();
assert_eq!(batches, vec![InInstructionsEvent::Batch { network, id, block: block_hash }]);

View file

@ -1,13 +1,16 @@
use rand_core::{RngCore, OsRng};
use sp_core::{sr25519::Signature, Pair};
use subxt::{config::extrinsic_params::BaseExtrinsicParamsBuilder};
use scale::Encode;
use sp_core::Pair;
use serai_client::{
subxt::config::extrinsic_params::BaseExtrinsicParamsBuilder,
primitives::{
BITCOIN_NET_ID, BITCOIN, BlockHash, SeraiAddress, Amount, Balance, Data, ExternalAddress,
insecure_pair_from_name,
},
validator_sets::primitives::{Session, ValidatorSet},
in_instructions::{
InInstructionsEvent,
primitives::{InInstruction, InInstructionWithBalance, Batch, SignedBatch},
@ -16,8 +19,8 @@ use serai_client::{
PairSigner, Serai,
};
mod runner;
use runner::{URL, publish_tx, provide_batch};
mod common;
use common::{serai, tx::publish_tx, validator_sets::vote_in_key, in_instructions::provide_batch};
serai_test!(
async fn burn() {
@ -27,7 +30,7 @@ serai_test!(
let mut block_hash = BlockHash([0; 32]);
OsRng.fill_bytes(&mut block_hash.0);
let pair = insecure_pair_from_name("Alice");
let pair = insecure_pair_from_name("Dave");
let public = pair.public();
let address = SeraiAddress::from(public);
@ -44,10 +47,15 @@ serai_test!(
balance,
}],
};
let signed = SignedBatch { batch, signature: Signature::from_raw([0; 64]) };
let batch_pair = insecure_pair_from_name("Bitcoin");
let key_pair = (batch_pair.public(), vec![].try_into().unwrap());
vote_in_key(ValidatorSet { session: Session(0), network }, key_pair).await;
let signature = batch_pair.sign(&batch.encode());
let signed = SignedBatch { batch, signature };
let block = provide_batch(signed).await;
let serai = Serai::new(URL).await.unwrap();
let serai = serai().await;
let batches = serai.get_batch_events(block).await.unwrap();
assert_eq!(batches, vec![InInstructionsEvent::Batch { network, id, block: block_hash }]);
@ -68,12 +76,15 @@ serai_test!(
let data = Data::new(rand_bytes).unwrap();
let out = OutInstruction { address: external_address, data: Some(data) };
let burn = Serai::burn(balance, out.clone());
let signer = PairSigner::new(pair);
let block = publish_tx(
&serai,
&serai.sign(&signer, &burn, 0, BaseExtrinsicParamsBuilder::new()).unwrap(),
&serai
.sign(
&PairSigner::new(pair),
&Serai::burn(balance, out.clone()),
0,
BaseExtrinsicParamsBuilder::new(),
)
.unwrap(),
)
.await;

View file

@ -0,0 +1,26 @@
use serai_client::in_instructions::{primitives::SignedBatch, InInstructionsEvent};
use crate::common::{serai, tx::publish_tx};
#[allow(dead_code)]
pub async fn provide_batch(batch: SignedBatch) -> [u8; 32] {
let serai = serai().await;
let execution = serai.execute_batch(batch.clone()).unwrap();
let block = publish_tx(&execution).await;
let batches = serai.get_batch_events(block).await.unwrap();
// TODO: impl From<Batch> for BatchEvent?
assert_eq!(
batches,
vec![InInstructionsEvent::Batch {
network: batch.batch.network,
id: batch.batch.id,
block: batch.batch.block,
}],
);
// TODO: Check the tokens events
block
}

View file

@ -0,0 +1,82 @@
use lazy_static::lazy_static;
use tokio::sync::Mutex;
use serai_client::Serai;
pub mod tx;
pub mod validator_sets;
pub mod in_instructions;
pub const URL: &str = "ws://127.0.0.1:9944";
pub async fn serai() -> Serai {
Serai::new(URL).await.unwrap()
}
lazy_static! {
pub static ref SEQUENTIAL: Mutex<()> = Mutex::new(());
}
#[macro_export]
macro_rules! serai_test {
($(async fn $name: ident() $body: block)*) => {
$(
#[tokio::test]
async fn $name() {
use std::process::Command;
let guard = common::SEQUENTIAL.lock().await;
let is_running = || {
!Command::new("pidof").arg("serai-node").output().unwrap().stdout.is_empty()
};
// Spawn a fresh Serai node
let mut command = {
use core::time::Duration;
use std::path::Path;
// Make sure a node isn't already running
assert!(!is_running());
let node = {
let this_crate = Path::new(env!("CARGO_MANIFEST_DIR"));
let top_level = this_crate.join("../../../");
top_level.join("target/debug/serai-node")
};
let command = Command::new(node).arg("--dev").spawn().unwrap();
while Serai::new(common::URL).await.is_err() {
tokio::time::sleep(Duration::from_secs(1)).await;
}
let serai = serai().await;
while serai.get_latest_block_hash().await.is_err() {
tokio::time::sleep(Duration::from_secs(1)).await;
}
// TODO: https://github.com/serai-dex/serai/247
if std::env::var("GITHUB_CI") == Ok("true".to_string()) {
tokio::time::sleep(Duration::from_secs(60)).await;
}
// Sanity check the pidof command is well-formed
assert!(is_running());
command
};
let local = tokio::task::LocalSet::new();
local.run_until(async move {
if let Err(err) = tokio::task::spawn_local(async move { $body }).await {
drop(guard);
let _ = command.kill();
Err(err).unwrap()
} else {
command.kill().unwrap();
}
assert!(!is_running());
}).await;
}
)*
}
}

View file

@ -0,0 +1,51 @@
use core::time::Duration;
use tokio::time::sleep;
use serai_client::subxt::{config::Header, utils::Encoded};
use crate::common::serai;
#[allow(dead_code)]
pub async fn publish_tx(tx: &Encoded) -> [u8; 32] {
let serai = serai().await;
let mut latest = serai
.get_block(serai.get_latest_block_hash().await.unwrap())
.await
.unwrap()
.unwrap()
.header
.number();
serai.publish(tx).await.unwrap();
// Get the block it was included in
// TODO: Add an RPC method for this/check the guarantee on the subscription
let mut ticks = 0;
loop {
latest += 1;
let block = {
let mut block;
while {
block = serai.get_block_by_number(latest).await.unwrap();
block.is_none()
} {
sleep(Duration::from_secs(1)).await;
ticks += 1;
if ticks > 60 {
panic!("60 seconds without inclusion in a finalized block");
}
}
block.unwrap()
};
for extrinsic in block.extrinsics {
if extrinsic.0 == tx.0[2 ..] {
return block.header.hash().into();
}
}
}
}

View file

@ -0,0 +1,46 @@
use sp_core::Pair;
use serai_client::{
subxt::config::extrinsic_params::BaseExtrinsicParamsBuilder,
primitives::insecure_pair_from_name,
validator_sets::{
primitives::{ValidatorSet, KeyPair},
ValidatorSetsEvent,
},
PairSigner, Serai,
};
use crate::common::{serai, tx::publish_tx};
#[allow(dead_code)]
pub async fn vote_in_key(set: ValidatorSet, key_pair: KeyPair) -> [u8; 32] {
let pair = insecure_pair_from_name("Alice");
let public = pair.public();
let serai = serai().await;
// Vote in a key pair
let block = publish_tx(
&serai
.sign(
&PairSigner::new(pair),
&Serai::vote(set.network, key_pair.clone()),
0,
BaseExtrinsicParamsBuilder::new(),
)
.unwrap(),
)
.await;
assert_eq!(
serai.get_vote_events(block).await.unwrap(),
vec![ValidatorSetsEvent::Vote { voter: public, set, key_pair: key_pair.clone(), votes: 1 }]
);
assert_eq!(
serai.get_key_gen_events(block).await.unwrap(),
vec![ValidatorSetsEvent::KeyGen { set, key_pair: key_pair.clone() }]
);
assert_eq!(serai.get_keys(set).await.unwrap(), Some(key_pair));
block
}

View file

@ -1,144 +0,0 @@
use core::time::Duration;
use lazy_static::lazy_static;
use tokio::{sync::Mutex, time::sleep};
use serai_client::{
subxt::{config::Header, utils::Encoded},
in_instructions::{primitives::SignedBatch, InInstructionsEvent},
Serai,
};
pub const URL: &str = "ws://127.0.0.1:9944";
lazy_static! {
pub static ref SEQUENTIAL: Mutex<()> = Mutex::new(());
}
#[allow(dead_code)]
pub async fn publish_tx(serai: &Serai, tx: &Encoded) -> [u8; 32] {
let mut latest = serai
.get_block(serai.get_latest_block_hash().await.unwrap())
.await
.unwrap()
.unwrap()
.header
.number();
serai.publish(tx).await.unwrap();
// Get the block it was included in
// TODO: Add an RPC method for this/check the guarantee on the subscription
let mut ticks = 0;
loop {
latest += 1;
let block = {
let mut block;
while {
block = serai.get_block_by_number(latest).await.unwrap();
block.is_none()
} {
sleep(Duration::from_secs(1)).await;
ticks += 1;
if ticks > 60 {
panic!("60 seconds without inclusion in a finalized block");
}
}
block.unwrap()
};
for extrinsic in block.extrinsics {
if extrinsic.0 == tx.0[2 ..] {
return block.header.hash().into();
}
}
}
}
#[allow(dead_code)]
pub async fn provide_batch(batch: SignedBatch) -> [u8; 32] {
let serai = Serai::new(URL).await.unwrap();
let execution = serai.execute_batch(batch.clone()).unwrap();
let block = publish_tx(&serai, &execution).await;
let batches = serai.get_batch_events(block).await.unwrap();
// TODO: impl From<Batch> for BatchEvent?
assert_eq!(
batches,
vec![InInstructionsEvent::Batch {
network: batch.batch.network,
id: batch.batch.id,
block: batch.batch.block,
}],
);
// TODO: Check the tokens events
block
}
#[macro_export]
macro_rules! serai_test {
($(async fn $name: ident() $body: block)*) => {
$(
#[tokio::test]
async fn $name() {
use std::process::Command;
let guard = runner::SEQUENTIAL.lock().await;
let is_running = || {
!Command::new("pidof").arg("serai-node").output().unwrap().stdout.is_empty()
};
// Spawn a fresh Serai node
let mut command = {
use core::time::Duration;
use std::path::Path;
// Make sure a node isn't already running
assert!(!is_running());
let node = {
let this_crate = Path::new(env!("CARGO_MANIFEST_DIR"));
let top_level = this_crate.join("../../../");
top_level.join("target/debug/serai-node")
};
let command = Command::new(node).arg("--dev").spawn().unwrap();
while Serai::new(URL).await.is_err() {
tokio::time::sleep(Duration::from_secs(1)).await;
}
while Serai::new(URL).await.unwrap().get_latest_block_hash().await.is_err() {
tokio::time::sleep(Duration::from_secs(1)).await;
}
// TODO: https://github.com/serai-dex/serai/247
if std::env::var("GITHUB_CI") == Ok("true".to_string()) {
tokio::time::sleep(Duration::from_secs(60)).await;
}
// Sanity check the pidof command is well-formed
assert!(is_running());
command
};
let local = tokio::task::LocalSet::new();
local.run_until(async move {
if let Err(err) = tokio::task::spawn_local(async move { $body }).await {
drop(guard);
let _ = command.kill();
Err(err).unwrap()
} else {
command.kill().unwrap();
}
assert!(!is_running());
}).await;
}
)*
}
}

View file

@ -1,7 +1,6 @@
use rand_core::{RngCore, OsRng};
use sp_core::{sr25519::Public, Pair};
use subxt::{config::extrinsic_params::BaseExtrinsicParamsBuilder};
use serai_client::{
primitives::{BITCOIN_NET_ID, BITCOIN_NET, insecure_pair_from_name},
@ -9,17 +8,19 @@ use serai_client::{
primitives::{Session, ValidatorSet},
ValidatorSetsEvent,
},
PairSigner, Serai,
Serai,
};
mod runner;
use runner::{URL, publish_tx};
mod common;
use common::{serai, validator_sets::vote_in_key};
serai_test!(
async fn vote_keys() {
let network = BITCOIN_NET_ID;
let set = ValidatorSet { session: Session(0), network };
let public = insecure_pair_from_name("Alice").public();
// Neither of these keys are validated
// The external key is infeasible to validate on-chain, the Ristretto key is feasible
// TODO: Should the Ristretto key be validated?
@ -29,10 +30,7 @@ serai_test!(
OsRng.fill_bytes(&mut external_key);
let key_pair = (Public(ristretto_key), external_key.try_into().unwrap());
let pair = insecure_pair_from_name("Alice");
let public = pair.public();
let serai = Serai::new(URL).await.unwrap();
let serai = serai().await;
// Make sure the genesis is as expected
let set_data = serai.get_validator_set(set).await.unwrap().unwrap();
@ -40,20 +38,9 @@ serai_test!(
let participants_ref: &[_] = set_data.participants.as_ref();
assert_eq!(participants_ref, [(public, set_data.bond)].as_ref());
// Vote in a key pair
let block = publish_tx(
&serai,
&serai
.sign(
&PairSigner::new(pair),
&Serai::vote(network, key_pair.clone()),
0,
BaseExtrinsicParamsBuilder::new(),
)
.unwrap(),
)
.await;
let block = vote_in_key(set, key_pair.clone()).await;
// While the vote_in_key function should handle this, it's beneficial to independently test it
assert_eq!(
serai.get_vote_events(block).await.unwrap(),
vec![ValidatorSetsEvent::Vote { voter: public, set, key_pair: key_pair.clone(), votes: 1 }]

View file

@ -2,7 +2,6 @@
#[frame_support::pallet]
pub mod pallet {
use scale::{Encode, Decode};
use scale_info::TypeInfo;
use frame_system::pallet_prelude::*;