Add Batch messages from processor, verify Batchs published on-chain

Renames Update to SignedBatch.

Checks Batch equality via a hash of the InInstructions. That prevents needing
to keep the Batch in node state or TX introspect.
This commit is contained in:
Luke Parker 2023-09-29 03:51:01 -04:00
parent 0be567ff69
commit 0eff3d9453
No known key found for this signature in database
18 changed files with 281 additions and 80 deletions

2
Cargo.lock generated
View file

@ -8137,6 +8137,7 @@ name = "serai-client"
version = "0.1.0"
dependencies = [
"bitcoin",
"blake2",
"ciphersuite",
"frost-schnorrkel",
"futures",
@ -8262,6 +8263,7 @@ dependencies = [
"serai-validator-sets-pallet",
"sp-application-crypto",
"sp-core",
"sp-io",
"sp-runtime",
"thiserror",
]

View file

@ -1,7 +1,15 @@
use core::marker::PhantomData;
use blake2::{
digest::{consts::U32, Digest},
Blake2b,
};
use scale::{Encode, Decode};
use serai_client::{primitives::NetworkId, in_instructions::primitives::SignedBatch};
use serai_client::{
primitives::NetworkId,
in_instructions::primitives::{Batch, SignedBatch},
};
pub use serai_db::*;
@ -87,6 +95,19 @@ impl<D: Db> MainDb<D> {
getter.get(Self::first_preprocess_key(network, id))
}
fn expected_batch_key(network: NetworkId, id: u32) -> Vec<u8> {
Self::main_key(b"expected_batch", (network, id).encode())
}
pub fn save_expected_batch(txn: &mut D::Transaction<'_>, batch: &Batch) {
txn.put(
Self::expected_batch_key(batch.network, batch.id),
Blake2b::<U32>::digest(batch.instructions.encode()),
);
}
pub fn expected_batch<G: Get>(getter: &G, network: NetworkId, id: u32) -> Option<[u8; 32]> {
getter.get(Self::expected_batch_key(network, id)).map(|batch| batch.try_into().unwrap())
}
fn batch_key(network: NetworkId, id: u32) -> Vec<u8> {
Self::main_key(b"batch", (network, id).encode())
}
@ -98,4 +119,16 @@ impl<D: Db> MainDb<D> {
.get(Self::batch_key(network, id))
.map(|batch| SignedBatch::decode(&mut batch.as_ref()).unwrap())
}
fn last_verified_batch_key(network: NetworkId) -> Vec<u8> {
Self::main_key(b"last_verified_batch", network.encode())
}
pub fn save_last_verified_batch(txn: &mut D::Transaction<'_>, network: NetworkId, id: u32) {
txn.put(Self::last_verified_batch_key(network), id.to_le_bytes());
}
pub fn last_verified_batch<G: Get>(getter: &G, network: NetworkId) -> Option<u32> {
getter
.get(Self::last_verified_batch_key(network))
.map(|id| u32::from_le_bytes(id.try_into().unwrap()))
}
}

View file

@ -47,6 +47,7 @@ pub mod processors;
use processors::Processors;
mod substrate;
use substrate::SubstrateDb;
#[cfg(test)]
pub mod tests;
@ -118,7 +119,7 @@ pub async fn scan_substrate<D: Db, Pro: Processors>(
) {
log::info!("scanning substrate");
let mut db = substrate::SubstrateDb::new(db);
let mut db = SubstrateDb::new(db);
let mut next_substrate_block = db.next_block();
let new_substrate_block_notifier = {
@ -565,16 +566,16 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
ProcessorMessage::Sign(inner_msg) => match inner_msg {
// We'll only receive Preprocess and Share if we're actively signing
sign::ProcessorMessage::Preprocess { id, .. } => {
Some(substrate::SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
sign::ProcessorMessage::Share { id, .. } => {
Some(substrate::SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
// While the Processor's Scanner will always emit Completed, that's routed through the
// Signer and only becomes a ProcessorMessage::Completed if the Signer is present and
// confirms it
sign::ProcessorMessage::Completed { key, .. } => {
Some(substrate::SubstrateDb::<D>::session_for_key(&txn, key).unwrap())
Some(SubstrateDb::<D>::session_for_key(&txn, key).unwrap())
}
},
ProcessorMessage::Coordinator(inner_msg) => match inner_msg {
@ -606,19 +607,69 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
}
// We'll only fire these if we are the Substrate signer, making the Tributary relevant
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => {
Some(substrate::SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::BatchShare { id, .. } => {
Some(substrate::SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
},
// These don't return a relevant Tributary as there's no Tributary with action expected
ProcessorMessage::Substrate(inner_msg) => match inner_msg {
processor_messages::substrate::ProcessorMessage::Batch { batch } => {
assert_eq!(
batch.network, msg.network,
"processor sent us a batch for a different network than it was for",
);
let this_batch_id = batch.id;
MainDb::<D>::save_expected_batch(&mut txn, batch);
// Re-define batch
// We can't drop it, yet it shouldn't be accidentally used in the following block
#[allow(clippy::let_unit_value)]
let batch = ();
#[allow(clippy::let_unit_value)]
let _ = batch;
// Verify all `Batch`s which we've already indexed from Substrate
// This won't be complete, as it only runs when a `Batch` message is received, which
// will be before we get a `SignedBatch`. It is, however, incremental. We can use a
// complete version to finish the last section when we need a complete version.
let last = MainDb::<D>::last_verified_batch(&txn, msg.network);
// This variable exists so Rust can verify Send/Sync properties
let mut faulty = None;
for id in last.map(|last| last + 1).unwrap_or(0) ..= this_batch_id {
if let Some(on_chain) = SubstrateDb::<D>::batch_instructions_hash(&txn, network, id) {
let off_chain = MainDb::<D>::expected_batch(&txn, network, id).unwrap();
if on_chain != off_chain {
faulty = Some((id, off_chain, on_chain));
break;
}
MainDb::<D>::save_last_verified_batch(&mut txn, msg.network, id);
}
}
if let Some((id, off_chain, on_chain)) = faulty {
// Halt operations on this network and spin, as this is a critical fault
loop {
log::error!(
"{}! network: {:?} id: {} off-chain: {} on-chain: {}",
"on-chain batch doesn't match off-chain",
network,
id,
hex::encode(off_chain),
hex::encode(on_chain),
);
sleep(Duration::from_secs(60)).await;
}
}
None
}
// If this is a new Batch, immediately publish it (if we can)
// This doesn't return a relevant Tributary as there's no Tributary with action expected
processor_messages::substrate::ProcessorMessage::Update { batch } => {
processor_messages::substrate::ProcessorMessage::SignedBatch { batch } => {
assert_eq!(
batch.batch.network, msg.network,
"processor sent us a batch for a different network than it was for",
"processor sent us a signed batch for a different network than it was for",
);
// TODO: Check this key's key pair's substrate key is authorized to publish batches
@ -849,7 +900,8 @@ async fn handle_processor_messages<D: Db, Pro: Processors, P: P2p>(
}
},
ProcessorMessage::Substrate(inner_msg) => match inner_msg {
processor_messages::substrate::ProcessorMessage::Update { .. } => unreachable!(),
processor_messages::substrate::ProcessorMessage::Batch { .. } => unreachable!(),
processor_messages::substrate::ProcessorMessage::SignedBatch { .. } => unreachable!(),
},
};

View file

@ -2,7 +2,10 @@ use scale::{Encode, Decode};
pub use serai_db::*;
use serai_client::validator_sets::primitives::{Session, KeyPair};
use serai_client::{
primitives::NetworkId,
validator_sets::primitives::{Session, KeyPair},
};
#[derive(Debug)]
pub struct SubstrateDb<D: Db>(pub D);
@ -55,4 +58,23 @@ impl<D: Db> SubstrateDb<D> {
txn.put(key_0, session.clone());
txn.put(Self::session_key(&key_pair.1), session);
}
fn batch_instructions_key(network: NetworkId, id: u32) -> Vec<u8> {
Self::substrate_key(b"batch", (network, id).encode())
}
pub fn batch_instructions_hash<G: Get>(
getter: &G,
network: NetworkId,
id: u32,
) -> Option<[u8; 32]> {
getter.get(Self::batch_instructions_key(network, id)).map(|bytes| bytes.try_into().unwrap())
}
pub fn save_batch_instructions_hash(
txn: &mut D::Transaction<'_>,
network: NetworkId,
id: u32,
hash: [u8; 32],
) {
txn.put(Self::batch_instructions_key(network, id), hash);
}
}

View file

@ -123,7 +123,8 @@ async fn handle_key_gen<D: Db, Pro: Processors>(
Ok(())
}
async fn handle_batch_and_burns<Pro: Processors>(
async fn handle_batch_and_burns<D: Db, Pro: Processors>(
db: &mut D,
processors: &Pro,
serai: &Serai,
block: &Block,
@ -149,9 +150,15 @@ async fn handle_batch_and_burns<Pro: Processors>(
let mut burns = HashMap::new();
for batch in serai.get_batch_events(hash).await? {
if let InInstructionsEvent::Batch { network, id, block: network_block } = batch {
if let InInstructionsEvent::Batch { network, id, block: network_block, instructions_hash } =
batch
{
network_had_event(&mut burns, &mut batches, network);
let mut txn = db.txn();
SubstrateDb::<D>::save_batch_instructions_hash(&mut txn, network, id, instructions_hash);
txn.commit();
// Make sure this is the only Batch event for this network in this Block
assert!(batch_block.insert(network, network_block).is_none());
@ -277,7 +284,7 @@ async fn handle_block<D: Db, CNT: Clone + Fn(&mut D, TributarySpec), Pro: Proces
// This does break the uniqueness of (hash, event_id) -> one event, yet
// (network, (hash, event_id)) remains valid as a unique ID for an event
if !SubstrateDb::<D>::handled_event(&db.0, hash, event_id) {
handle_batch_and_burns(processors, serai, &block).await?;
handle_batch_and_burns(&mut db.0, processors, serai, &block).await?;
}
let mut txn = db.0.txn();
SubstrateDb::<D>::handle_event(&mut txn, hash, event_id);

View file

@ -7,23 +7,28 @@ node.
This document primarily details its flow with regards to the Serai node and
processor.
## New Set Event
### New Set Event
On `validator_sets::pallet::Event::NewSet`, the coordinator spawns a tributary
for the new set. It additionally sends the processor
`key_gen::CoordinatorMessage::GenerateKey`.
## Key Generation Event
### Key Generation Event
On `validator_sets::pallet::Event::KeyGen`, the coordinator sends
`substrate::CoordinatorMessage::ConfirmKeyPair` to the processor.
# Update
### Batch
On `key_gen::ProcessorMessage::Update`, the coordinator publishes an unsigned
transaction containing the signed batch to the Serai blockchain.
On `substrate::ProcessorMessage::Batch`, the coordinator notes what the on-chain
`Batch` should be, for verification once published.
# Sign Completed
### SignedBatch
On `substrate::ProcessorMessage::SignedBatch`, the coordinator publishes an
unsigned transaction containing the signed batch to the Serai blockchain.
### Sign Completed
On `sign::ProcessorMessage::Completed`, the coordinator makes a tributary
transaction containing the transaction hash the signing process was supposedly

View file

@ -6,7 +6,7 @@ transactions with payments.
This document primarily discusses its flow with regards to the coordinator.
## Generate Key
### Generate Key
On `key_gen::CoordinatorMessage::GenerateKey`, the processor begins a pair of
instances of the distributed key generation protocol specified in the FROST
@ -20,26 +20,26 @@ Serai's overall key generation protocol.
The commitments for both protocols are sent to the coordinator in a single
`key_gen::ProcessorMessage::Commitments`.
## Key Gen Commitments
### Key Gen Commitments
On `key_gen::CoordinatorMessage::Commitments`, the processor continues the
specified key generation instance. The secret shares for each fellow
participant are sent to the coordinator in a
`key_gen::ProcessorMessage::Shares`.
### Key Gen Shares
#### Key Gen Shares
On `key_gen::CoordinatorMessage::Shares`, the processor completes the specified
key generation instance. The generated key pair is sent to the coordinator in a
`key_gen::ProcessorMessage::GeneratedKeyPair`.
## Confirm Key Pair
### Confirm Key Pair
On `substrate::CoordinatorMessage::ConfirmKeyPair`, the processor starts using
the newly confirmed key, scanning blocks on the external network for
transfers to it.
## Connected Network Block
### External Network Block
When the external network has a new block, which is considered finalized
(either due to being literally finalized or due to having a sufficient amount
@ -48,8 +48,14 @@ of confirmations), it's scanned.
Outputs to the key of Serai's multisig are saved to the database. Outputs which
newly transfer into Serai are used to build `Batch`s for the block. The
processor then begins a threshold signature protocol with its key pair's
Ristretto key to sign the `Batch`s. The protocol's preprocess is sent to the
coordinator in a `coordinator::ProcessorMessage::BatchPreprocess`.
Ristretto key to sign the `Batch`s.
The `Batch`s are each sent to the coordinator in a
`substrate::ProcessorMessage::Batch`, enabling the coordinator to know what
`Batch`s *should* be published to Serai. After each
`substrate::ProcessorMessage::Batch`, the preprocess for the first instance of
its signing protocol is sent to the coordinator in a
`coordinator::ProcessorMessage::BatchPreprocess`.
As a design comment, we *may* be able to sign now possible, already scheduled,
branch/leaf transactions at this point. Doing so would be giving a mutable
@ -57,26 +63,26 @@ borrow over the scheduler to both the external network and the Serai network,
and would accordingly be unsafe. We may want to look at splitting the scheduler
in two, in order to reduce latency (TODO).
## Batch Preprocesses
### Batch Preprocesses
On `coordinator::CoordinatorMessage::BatchPreprocesses`, the processor
continues the specified batch signing protocol, sending
`coordinator::ProcessorMessage::BatchShare` to the coordinator.
## Batch Shares
### Batch Shares
On `coordinator::CoordinatorMessage::BatchShares`, the processor
completes the specified batch signing protocol. If successful, the processor
stops signing for this batch and sends `substrate::ProcessorMessage::Update` to
the coordinator.
stops signing for this batch and sends
`substrate::ProcessorMessage::SignedBatch` to the coordinator.
## Batch Re-attempt
### Batch Re-attempt
On `coordinator::CoordinatorMessage::BatchReattempt`, the processor will create
a new instance of the batch signing protocol. The new protocol's preprocess is
sent to the coordinator in a `coordinator::ProcessorMessage::BatchPreprocess`.
## Substrate Block
### Substrate Block
On `substrate::CoordinatorMessage::SubstrateBlock`, the processor:
@ -89,13 +95,13 @@ On `substrate::CoordinatorMessage::SubstrateBlock`, the processor:
4) Sends `sign::ProcessorMessage::Preprocess` for each plan now being signed
for.
## Sign Preprocesses
### Sign Preprocesses
On `sign::CoordinatorMessage::Preprocesses`, the processor continues the
specified transaction signing protocol, sending `sign::ProcessorMessage::Share`
to the coordinator.
## Sign Shares
### Sign Shares
On `sign::CoordinatorMessage::Shares`, the processor completes the specified
transaction signing protocol. If successful, the processor stops signing for
@ -104,7 +110,7 @@ this transaction and publishes the signed transaction. Then,
broadcasted to all validators so everyone can observe the attempt completed,
producing a signed and published transaction.
## Sign Re-attempt
### Sign Re-attempt
On `sign::CoordinatorMessage::Reattempt`, the processor will create a new
a new instance of the transaction signing protocol if it hasn't already
@ -112,7 +118,7 @@ completed/observed completion of an instance of the signing protocol. The new
protocol's preprocess is sent to the coordinator in a
`sign::ProcessorMessage::Preprocess`.
## Sign Completed
### Sign Completed
On `sign::CoordinatorMessage::Completed`, the processor verifies the included
transaction hash actually refers to an accepted transaction which completes the

View file

@ -8,7 +8,7 @@ use serde::{Serialize, Deserialize};
use dkg::{Participant, ThresholdParams};
use serai_primitives::{BlockHash, NetworkId};
use in_instructions_primitives::SignedBatch;
use in_instructions_primitives::{Batch, SignedBatch};
use tokens_primitives::OutInstructionWithBalance;
use validator_sets_primitives::{ValidatorSet, KeyPair};
@ -178,7 +178,8 @@ pub mod substrate {
#[derive(Clone, PartialEq, Eq, Debug, Zeroize, Encode, Decode, Serialize, Deserialize)]
pub enum ProcessorMessage {
Update { batch: SignedBatch },
Batch { batch: Batch },
SignedBatch { batch: SignedBatch },
}
}
@ -364,8 +365,9 @@ impl ProcessorMessage {
ProcessorMessage::Substrate(msg) => {
let (sub, id) = match msg {
// Unique since network and ID binding
substrate::ProcessorMessage::Update { batch, .. } => {
(0, (batch.batch.network, batch.batch.id).encode())
substrate::ProcessorMessage::Batch { batch } => (0, (batch.network, batch.id).encode()),
substrate::ProcessorMessage::SignedBatch { batch, .. } => {
(1, (batch.batch.network, batch.batch.id).encode())
}
};

View file

@ -530,6 +530,12 @@ async fn run<N: Network, D: Db, Co: Coordinator>(mut raw_db: D, network: N, mut
for batch in batches {
info!("created batch {} ({} instructions)", batch.id, batch.instructions.len());
coordinator.send(
messages::ProcessorMessage::Substrate(
messages::substrate::ProcessorMessage::Batch { batch: batch.clone() }
)
).await;
if let Some(substrate_signer) = tributary_mutable.substrate_signer.as_mut() {
substrate_signer.sign(txn, batch).await;
}
@ -588,9 +594,9 @@ async fn run<N: Network, D: Db, Co: Coordinator>(mut raw_db: D, network: N, mut
}
SubstrateSignerEvent::SignedBatch(batch) => {
coordinator
.send(ProcessorMessage::Substrate(messages::substrate::ProcessorMessage::Update {
batch,
}))
.send(ProcessorMessage::Substrate(
messages::substrate::ProcessorMessage::SignedBatch { batch },
))
.await;
}
}

View file

@ -694,6 +694,7 @@ impl<N: Network, D: Db> Scanner<N, D> {
return;
}
// Since we're creating a Batch, mark it as needing ack
scanner.need_ack.push_back(block_being_scanned);
true
} else {
@ -706,7 +707,7 @@ impl<N: Network, D: Db> Scanner<N, D> {
scanner.eventualities.remove(retired.to_bytes().as_ref());
}
// Update ram_scanned/need_ack
// Update ram_scanned
scanner.ram_scanned = Some(block_being_scanned);
drop(scanner_lock);

View file

@ -36,6 +36,8 @@ lazy_static = "1"
rand_core = "0.6"
blake2 = "0.10"
ciphersuite = { path = "../../crypto/ciphersuite", features = ["ristretto"] }
frost = { package = "modular-frost", path = "../../crypto/frost", features = ["tests"] }
schnorrkel = { path = "../../crypto/schnorrkel", package = "frost-schnorrkel" }

View file

@ -1,5 +1,12 @@
use rand_core::{RngCore, OsRng};
use blake2::{
digest::{consts::U32, Digest},
Blake2b,
};
use scale::Encode;
use serai_client::{
primitives::{Amount, NetworkId, Coin, Balance, BlockHash, SeraiAddress},
in_instructions::{
@ -38,12 +45,20 @@ serai_test!(
}],
};
let block = provide_batch(batch).await;
let block = provide_batch(batch.clone()).await;
let serai = serai().await;
assert_eq!(serai.get_latest_block_for_network(block, network).await.unwrap(), Some(block_hash));
let batches = serai.get_batch_events(block).await.unwrap();
assert_eq!(batches, vec![InInstructionsEvent::Batch { network, id, block: block_hash }]);
assert_eq!(
batches,
vec![InInstructionsEvent::Batch {
network,
id,
block: block_hash,
instructions_hash: Blake2b::<U32>::digest(batch.instructions.encode()).into(),
}]
);
assert_eq!(
serai.get_mint_events(block).await.unwrap(),

View file

@ -1,5 +1,12 @@
use rand_core::{RngCore, OsRng};
use blake2::{
digest::{consts::U32, Digest},
Blake2b,
};
use scale::Encode;
use sp_core::Pair;
use serai_client::{
@ -45,11 +52,19 @@ serai_test!(
}],
};
let block = provide_batch(batch).await;
let block = provide_batch(batch.clone()).await;
let serai = serai().await;
let batches = serai.get_batch_events(block).await.unwrap();
assert_eq!(batches, vec![InInstructionsEvent::Batch { network, id, block: block_hash }]);
assert_eq!(
batches,
vec![InInstructionsEvent::Batch {
network,
id,
block: block_hash,
instructions_hash: Blake2b::<U32>::digest(batch.instructions.encode()).into(),
}]
);
assert_eq!(
serai.get_mint_events(block).await.unwrap(),

View file

@ -1,3 +1,10 @@
use blake2::{
digest::{consts::U32, Digest},
Blake2b,
};
use scale::Encode;
use sp_core::Pair;
use serai_client::{
@ -38,7 +45,12 @@ pub async fn provide_batch(batch: Batch) -> [u8; 32] {
// TODO: impl From<Batch> for BatchEvent?
assert_eq!(
batches,
vec![InInstructionsEvent::Batch { network: batch.network, id: batch.id, block: batch.block }],
vec![InInstructionsEvent::Batch {
network: batch.network,
id: batch.id,
block: batch.block,
instructions_hash: Blake2b::<U32>::digest(batch.instructions.encode()).into(),
}],
);
// TODO: Check the tokens events

View file

@ -17,9 +17,10 @@ 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-core = { git = "https://github.com/serai-dex/substrate", default-features = false }
sp-io = { git = "https://github.com/serai-dex/substrate", default-features = false }
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 }
sp-core = { git = "https://github.com/serai-dex/substrate", default-features = false }
frame-system = { git = "https://github.com/serai-dex/substrate", default-features = false }
frame-support = { git = "https://github.com/serai-dex/substrate", default-features = false }
@ -37,6 +38,9 @@ std = [
"scale/std",
"scale-info/std",
"sp-core/std",
"sp-io/std",
"sp-application-crypto/std",
"sp-runtime/std",
"frame-system/std",

View file

@ -4,6 +4,7 @@
use scale::Encode;
use sp_io::hashing::blake2_256;
use sp_runtime::RuntimeDebug;
use serai_primitives::{BlockHash, NetworkId};
@ -45,7 +46,7 @@ pub mod pallet {
#[pallet::event]
#[pallet::generate_deposit(fn deposit_event)]
pub enum Event<T: Config> {
Batch { network: NetworkId, id: u32, block: BlockHash },
Batch { network: NetworkId, id: u32, block: BlockHash, instructions_hash: [u8; 32] },
InstructionFailure { network: NetworkId, id: u32, index: u32 },
}
@ -123,6 +124,7 @@ pub mod pallet {
network: batch.network,
id: batch.id,
block: batch.block,
instructions_hash: blake2_256(&batch.instructions.encode()),
});
for (i, instruction) in batch.instructions.into_iter().enumerate() {
// TODO: Check this balance's coin belongs to this network

View file

@ -10,7 +10,7 @@ use messages::{sign::SignId, SubstrateContext};
use serai_client::{
primitives::{BlockHash, crypto::RuntimePublic, PublicKey, SeraiAddress, NetworkId},
in_instructions::primitives::{
InInstruction, InInstructionWithBalance, SignedBatch, batch_message,
InInstruction, InInstructionWithBalance, Batch, SignedBatch, batch_message,
},
};
@ -19,6 +19,7 @@ use crate::{*, tests::*};
pub(crate) async fn recv_batch_preprocesses(
coordinators: &mut [Coordinator],
substrate_key: &[u8; 32],
batch: &Batch,
attempt: u32,
) -> (SignId, HashMap<Participant, Vec<u8>>) {
let mut id = None;
@ -27,8 +28,18 @@ pub(crate) async fn recv_batch_preprocesses(
for (i, coordinator) in coordinators.iter_mut().enumerate() {
let i = Participant::new(u16::try_from(i).unwrap() + 1).unwrap();
let msg = coordinator.recv_message().await;
match msg {
if attempt == 0 {
match coordinator.recv_message().await {
messages::ProcessorMessage::Substrate(messages::substrate::ProcessorMessage::Batch {
batch: sent_batch,
}) => {
assert_eq!(&sent_batch, batch);
}
_ => panic!("processor didn't send batch"),
}
}
match coordinator.recv_message().await {
messages::ProcessorMessage::Coordinator(
messages::coordinator::ProcessorMessage::BatchPreprocess {
id: this_id,
@ -122,9 +133,9 @@ pub(crate) async fn sign_batch(
if preprocesses.contains_key(&i) {
match coordinator.recv_message().await {
messages::ProcessorMessage::Substrate(messages::substrate::ProcessorMessage::Update {
batch: this_batch,
}) => {
messages::ProcessorMessage::Substrate(
messages::substrate::ProcessorMessage::SignedBatch { batch: this_batch },
) => {
if batch.is_none() {
assert!(PublicKey::from_raw(key)
.verify(&batch_message(&this_batch.batch), &this_batch.signature));
@ -231,9 +242,23 @@ fn batch_test() {
// The scanner works on a 5s interval, so this leaves a few s for any processing/latency
tokio::time::sleep(Duration::from_secs(10)).await;
let expected_batch = Batch {
network,
id: i,
block: BlockHash(block_with_tx.unwrap()),
instructions: if let Some(instruction) = instruction {
vec![InInstructionWithBalance { instruction, balance: balance_sent }]
} else {
// This shouldn't have an instruction as we didn't add any data into the TX we sent
// Empty batches remain valuable as they let us achieve consensus on the block and spend
// contained outputs
vec![]
},
};
// Make sure the proceessors picked it up by checking they're trying to sign a batch for it
let (mut id, mut preprocesses) =
recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, 0).await;
recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, &expected_batch, 0).await;
// Trigger a random amount of re-attempts
for attempt in 1 ..= u32::try_from(OsRng.next_u64() % 4).unwrap() {
// TODO: Double check how the processor handles this ID field
@ -247,27 +272,15 @@ fn batch_test() {
.await;
}
(id, preprocesses) =
recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, attempt).await;
recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, &expected_batch, attempt)
.await;
}
// Continue with signing the batch
let batch = sign_batch(&mut coordinators, key_pair.0 .0, id, preprocesses).await;
// Check it
assert_eq!(batch.batch.network, network);
assert_eq!(batch.batch.id, i);
assert_eq!(batch.batch.block, BlockHash(block_with_tx.unwrap()));
if let Some(instruction) = instruction {
assert_eq!(
batch.batch.instructions,
vec![InInstructionWithBalance { instruction, balance: balance_sent }]
);
} else {
// This shouldn't have an instruction as we didn't add any data into the TX we sent
// Empty batches remain valuable as they let us achieve consensus on the block and spend
// contained outputs
assert!(batch.batch.instructions.is_empty());
}
assert_eq!(batch.batch, expected_batch);
// Fire a SubstrateBlock
let serai_time =

View file

@ -9,6 +9,7 @@ use messages::{sign::SignId, SubstrateContext};
use serai_client::{
primitives::{BlockHash, NetworkId},
in_instructions::primitives::Batch,
tokens::primitives::{OutInstruction, OutInstructionWithBalance},
};
@ -186,17 +187,18 @@ fn send_test() {
// The scanner works on a 5s interval, so this leaves a few s for any processing/latency
tokio::time::sleep(Duration::from_secs(10)).await;
let expected_batch =
Batch { network, id: 0, block: BlockHash(block_with_tx.unwrap()), instructions: vec![] };
// Make sure the proceessors picked it up by checking they're trying to sign a batch for it
let (id, preprocesses) = recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, 0).await;
let (id, preprocesses) =
recv_batch_preprocesses(&mut coordinators, &key_pair.0 .0, &expected_batch, 0).await;
// Continue with signing the batch
let batch = sign_batch(&mut coordinators, key_pair.0 .0, id, preprocesses).await;
// Check it
assert_eq!(batch.batch.network, network);
assert_eq!(batch.batch.id, 0);
assert_eq!(batch.batch.block, BlockHash(block_with_tx.unwrap()));
assert!(batch.batch.instructions.is_empty());
assert_eq!(batch.batch, expected_batch);
// Fire a SubstrateBlock with a burn
let substrate_block_num = (OsRng.next_u64() % 4_000_000_000u64) + 1;