mirror of
https://github.com/serai-dex/serai.git
synced 2024-12-23 12:09:37 +00:00
Verify Batch signatures
Starts further fleshing out the Serai client tests with common utils.
This commit is contained in:
parent
30b8636641
commit
ae4e98c052
14 changed files with 291 additions and 201 deletions
2
Cargo.lock
generated
2
Cargo.lock
generated
|
@ -3861,9 +3861,11 @@ dependencies = [
|
|||
"parity-scale-codec",
|
||||
"scale-info",
|
||||
"serai-primitives",
|
||||
"sp-application-crypto",
|
||||
"sp-runtime",
|
||||
"thiserror",
|
||||
"tokens-pallet",
|
||||
"validator-sets-pallet",
|
||||
]
|
||||
|
||||
[[package]]
|
||||
|
|
|
@ -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 = [
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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"),
|
||||
],
|
||||
)
|
||||
},
|
||||
|
|
|
@ -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, ®istry).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,
|
||||
|
|
|
@ -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 }]);
|
||||
|
||||
|
|
|
@ -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;
|
||||
|
||||
|
|
26
substrate/serai/client/tests/common/in_instructions.rs
Normal file
26
substrate/serai/client/tests/common/in_instructions.rs
Normal 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
|
||||
}
|
82
substrate/serai/client/tests/common/mod.rs
Normal file
82
substrate/serai/client/tests/common/mod.rs
Normal 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;
|
||||
}
|
||||
)*
|
||||
}
|
||||
}
|
51
substrate/serai/client/tests/common/tx.rs
Normal file
51
substrate/serai/client/tests/common/tx.rs
Normal 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();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
46
substrate/serai/client/tests/common/validator_sets.rs
Normal file
46
substrate/serai/client/tests/common/validator_sets.rs
Normal 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
|
||||
}
|
|
@ -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;
|
||||
}
|
||||
)*
|
||||
}
|
||||
}
|
|
@ -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 }]
|
||||
|
|
|
@ -2,7 +2,6 @@
|
|||
|
||||
#[frame_support::pallet]
|
||||
pub mod pallet {
|
||||
use scale::{Encode, Decode};
|
||||
use scale_info::TypeInfo;
|
||||
|
||||
use frame_system::pallet_prelude::*;
|
||||
|
|
Loading…
Reference in a new issue