* Remove NetworkId from processor-messages

Because intent binds to the sender/receiver, it's not needed for intent.

The processor knows what the network is.

The coordinator knows which to use because it's sending this message to the
processor for that network.

Also removes the unused zeroize.

* ProcessorMessage::Completed use Session instead of key

* Move SubstrateSignId to Session

* Finish replacing key with session
This commit is contained in:
Luke Parker
2023-11-26 12:14:23 -05:00
committed by GitHub
parent b79cf8abde
commit 571195bfda
31 changed files with 304 additions and 455 deletions

View File

@@ -34,18 +34,12 @@ impl<D: Db> MainDb<D> {
getter.get(Self::handled_message_key(network, id)).is_some()
}
fn in_tributary_key(set: ValidatorSet) -> Vec<u8> {
Self::main_key(b"in_tributary", set.encode())
}
fn active_tributaries_key() -> Vec<u8> {
Self::main_key(b"active_tributaries", [])
}
fn retired_tributary_key(set: ValidatorSet) -> Vec<u8> {
Self::main_key(b"retired_tributary", set.encode())
}
pub fn in_tributary<G: Get>(getter: &G, set: ValidatorSet) -> bool {
getter.get(Self::in_tributary_key(set)).is_some()
}
pub fn active_tributaries<G: Get>(getter: &G) -> (Vec<u8>, Vec<TributarySpec>) {
let bytes = getter.get(Self::active_tributaries_key()).unwrap_or(vec![]);
let mut bytes_ref: &[u8] = bytes.as_ref();
@@ -58,8 +52,6 @@ impl<D: Db> MainDb<D> {
(bytes, tributaries)
}
pub fn add_participating_in_tributary(txn: &mut D::Transaction<'_>, spec: &TributarySpec) {
txn.put(Self::in_tributary_key(spec.set()), []);
let key = Self::active_tributaries_key();
let (mut existing_bytes, existing) = Self::active_tributaries(txn);
for tributary in &existing {

View File

@@ -57,7 +57,7 @@ pub mod processors;
use processors::Processors;
mod substrate;
use substrate::{CosignTransactions, SubstrateDb};
use substrate::CosignTransactions;
mod cosign_evaluator;
use cosign_evaluator::CosignEvaluator;
@@ -116,7 +116,7 @@ async fn add_tributary<D: Db, Pro: Processors, P: P2p>(
.send(
set.network,
processor_messages::key_gen::CoordinatorMessage::GenerateKey {
id: processor_messages::key_gen::KeyGenId { set, attempt: 0 },
id: processor_messages::key_gen::KeyGenId { session: set.session, attempt: 0 },
params: frost::ThresholdParams::new(spec.t(), spec.n(), our_i.start).unwrap(),
shares: u16::from(our_i.end) - u16::from(our_i.start),
},
@@ -195,66 +195,50 @@ async fn handle_processor_message<D: Db, P: P2p>(
// We'll only receive these if we fired GenerateKey, which we'll only do if if we're
// in-set, making the Tributary relevant
ProcessorMessage::KeyGen(inner_msg) => match inner_msg {
key_gen::ProcessorMessage::Commitments { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::InvalidCommitments { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::Shares { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::InvalidShare { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::GeneratedKeyPair { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::Blame { id, .. } => Some(id.set.session),
key_gen::ProcessorMessage::Commitments { id, .. } => Some(id.session),
key_gen::ProcessorMessage::InvalidCommitments { id, .. } => Some(id.session),
key_gen::ProcessorMessage::Shares { id, .. } => Some(id.session),
key_gen::ProcessorMessage::InvalidShare { id, .. } => Some(id.session),
key_gen::ProcessorMessage::GeneratedKeyPair { id, .. } => Some(id.session),
key_gen::ProcessorMessage::Blame { id, .. } => Some(id.session),
},
// TODO: Review replacing key with Session in messages?
ProcessorMessage::Sign(inner_msg) => match inner_msg {
// We'll only receive InvalidParticipant/Preprocess/Share if we're actively signing
sign::ProcessorMessage::InvalidParticipant { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
sign::ProcessorMessage::Preprocess { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
sign::ProcessorMessage::Share { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
sign::ProcessorMessage::InvalidParticipant { id, .. } => Some(id.session),
sign::ProcessorMessage::Preprocess { id, .. } => Some(id.session),
sign::ProcessorMessage::Share { id, .. } => Some(id.session),
// 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(SubstrateDb::<D>::session_for_key(&txn, key).unwrap())
}
sign::ProcessorMessage::Completed { session, .. } => Some(*session),
},
ProcessorMessage::Coordinator(inner_msg) => match inner_msg {
// This is a special case as it's relevant to *all* Tributaries for this network
// This is a special case as it's relevant to *all* Tributaries for this network we're
// signing in
// It doesn't return a Tributary to become `relevant_tributary` though
coordinator::ProcessorMessage::SubstrateBlockAck { network, block, plans } => {
assert_eq!(
*network, msg.network,
"processor claimed to be a different network than it was for SubstrateBlockAck",
);
coordinator::ProcessorMessage::SubstrateBlockAck { block, plans } => {
// Get the sessions for these keys
let keys = plans.iter().map(|plan| plan.key.clone()).collect::<HashSet<_>>();
let mut sessions = vec![];
for key in keys {
let session = SubstrateDb::<D>::session_for_key(&txn, &key).unwrap();
// Only keep them if we're in the Tributary AND they haven't been retied
let set = ValidatorSet { network: *network, session };
if MainDb::<D>::in_tributary(&txn, set) && (!MainDb::<D>::is_tributary_retired(&txn, set))
{
sessions.push((session, key));
}
}
let sessions = plans
.iter()
.map(|plan| plan.session)
.filter(|session| {
!MainDb::<D>::is_tributary_retired(&txn, ValidatorSet { network, session: *session })
})
.collect::<HashSet<_>>();
// Ensure we have the Tributaries
for (session, _) in &sessions {
for session in &sessions {
if !tributaries.contains_key(session) {
return false;
}
}
for (session, key) in sessions {
for session in sessions {
let tributary = &tributaries[&session];
let plans = plans
.iter()
.filter_map(|plan| Some(plan.id).filter(|_| plan.key == key))
.filter_map(|plan| Some(plan.id).filter(|_| plan.session == session))
.collect::<Vec<_>>();
PlanIds::set(&mut txn, &tributary.spec.genesis(), *block, &plans);
@@ -286,18 +270,10 @@ async fn handle_processor_message<D: Db, P: P2p>(
None
}
// We'll only fire these if we are the Substrate signer, making the Tributary relevant
coordinator::ProcessorMessage::InvalidParticipant { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::CosignPreprocess { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::SubstrateShare { id, .. } => {
Some(SubstrateDb::<D>::session_for_key(&txn, &id.key).unwrap())
}
coordinator::ProcessorMessage::InvalidParticipant { id, .. } => Some(id.session),
coordinator::ProcessorMessage::CosignPreprocess { id, .. } => Some(id.session),
coordinator::ProcessorMessage::BatchPreprocess { id, .. } => Some(id.session),
coordinator::ProcessorMessage::SubstrateShare { id, .. } => Some(id.session),
coordinator::ProcessorMessage::CosignedBlock { block_number, block, signature } => {
let cosigned_block = CosignedBlock {
network,
@@ -462,11 +438,6 @@ async fn handle_processor_message<D: Db, P: P2p>(
}]
}
key_gen::ProcessorMessage::InvalidShare { id, accuser, faulty, blame } => {
assert_eq!(
id.set.network, msg.network,
"processor claimed to be a different network than it was for in InvalidShare",
);
// Check if the MuSig signature had any errors as if so, we need to provide
// RemoveParticipant
// As for the safety of calling error_generating_key_pair, the processor is presumed
@@ -490,11 +461,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
txs
}
key_gen::ProcessorMessage::GeneratedKeyPair { id, substrate_key, network_key } => {
assert_eq!(
id.set.network, msg.network,
"processor claimed to be a different network than it was for in GeneratedKeyPair",
);
// TODO2: Also check the other KeyGenId fields
// TODO2: Check the KeyGenId fields
// Tell the Tributary the key pair, get back the share for the MuSig signature
let share = crate::tributary::generated_key_pair::<D>(
@@ -514,11 +481,9 @@ async fn handle_processor_message<D: Db, P: P2p>(
}
}
}
key_gen::ProcessorMessage::Blame { id, participant } => {
assert_eq!(
id.set.network, msg.network,
"processor claimed to be a different network than it was for in Blame",
);
// This is a response to the ordered VerifyBlame, which is why this satisfies the provided
// transaction's needs to be perfectly ordered
key_gen::ProcessorMessage::Blame { id: _, participant } => {
vec![Transaction::RemoveParticipant(participant)]
}
},
@@ -556,7 +521,7 @@ async fn handle_processor_message<D: Db, P: P2p>(
signed: Transaction::empty_signed(),
})]
}
sign::ProcessorMessage::Completed { key: _, id, tx } => {
sign::ProcessorMessage::Completed { session: _, id, tx } => {
let r = Zeroizing::new(<Ristretto as Ciphersuite>::F::random(&mut OsRng));
#[allow(non_snake_case)]
let R = <Ristretto as Ciphersuite>::generator() * r.deref();

View File

@@ -1,12 +1,12 @@
use scale::{Encode, Decode};
pub use serai_db::*;
use scale::Encode;
use serai_client::{
primitives::NetworkId,
validator_sets::primitives::{Session, ValidatorSet, KeyPair},
validator_sets::primitives::{Session, ValidatorSet},
};
pub use serai_db::*;
create_db! {
SubstrateDb {
CosignTriggered: () -> (),
@@ -86,24 +86,6 @@ impl<D: Db> SubstrateDb<D> {
txn.put(Self::event_key(&id, index), []);
}
fn session_key(key: &[u8]) -> Vec<u8> {
Self::substrate_key(b"session", key)
}
pub fn session_for_key<G: Get>(getter: &G, key: &[u8]) -> Option<Session> {
getter.get(Self::session_key(key)).map(|bytes| Session::decode(&mut bytes.as_ref()).unwrap())
}
pub fn save_session_for_keys(txn: &mut D::Transaction<'_>, key_pair: &KeyPair, session: Session) {
let session = session.encode();
let key_0 = Self::session_key(&key_pair.0);
let existing = txn.get(&key_0);
// This may trigger if 100% of a DKG are malicious, and they create a key equivalent to a prior
// key. Since it requires 100% maliciousness, not just 67% maliciousness, this will only assert
// in a modified-to-be-malicious stack, making it safe
assert!(existing.is_none() || (existing.as_ref() == Some(&session)));
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())
}

View File

@@ -30,7 +30,7 @@ use tokio::{sync::mpsc, time::sleep};
use crate::{
Db,
processors::Processors,
tributary::{TributarySpec, SeraiBlockNumber, KeyPairDb},
tributary::{TributarySpec, SeraiBlockNumber},
};
mod db;
@@ -116,19 +116,13 @@ async fn handle_new_set<D: Db>(
Ok(())
}
async fn handle_key_gen<D: Db, Pro: Processors>(
db: &mut D,
async fn handle_key_gen<Pro: Processors>(
processors: &Pro,
serai: &Serai,
block: &Block,
set: ValidatorSet,
key_pair: KeyPair,
) -> Result<(), SeraiError> {
// This has to be saved *before* we send ConfirmKeyPair
let mut txn = db.txn();
SubstrateDb::<D>::save_session_for_keys(&mut txn, &key_pair, set.session);
txn.commit();
processors
.send(
set.network,
@@ -144,7 +138,7 @@ async fn handle_key_gen<D: Db, Pro: Processors>(
// block which has a time greater than or equal to the Serai time
.unwrap_or(BlockHash([0; 32])),
},
set,
session: set.session,
key_pair,
},
)
@@ -232,7 +226,6 @@ async fn handle_batch_and_burns<D: Db, Pro: Processors>(
serai_time: block.time().unwrap() / 1000,
network_latest_finalized_block,
},
network,
block: block.number(),
burns: burns.remove(&network).unwrap(),
batches: batches.remove(&network).unwrap(),
@@ -291,13 +284,7 @@ async fn handle_block<D: Db, Pro: Processors>(
if !SubstrateDb::<D>::handled_event(&db.0, hash, event_id) {
log::info!("found fresh key gen event {:?}", key_gen);
if let ValidatorSetsEvent::KeyGen { set, key_pair } = key_gen {
// Immediately ensure this key pair is accessible to the tributary, before we fire any
// events off of it
let mut txn = db.0.txn();
KeyPairDb::set(&mut txn, set, &key_pair);
txn.commit();
handle_key_gen(&mut db.0, processors, serai, &block, set, key_pair).await?;
handle_key_gen(processors, serai, &block, set, key_pair).await?;
} else {
panic!("KeyGen event wasn't KeyGen: {key_gen:?}");
}

View File

@@ -132,7 +132,7 @@ async fn dkg_test() {
assert_eq!(
msgs.pop_front().unwrap(),
CoordinatorMessage::KeyGen(key_gen::CoordinatorMessage::Commitments {
id: KeyGenId { set: spec.set(), attempt: 0 },
id: KeyGenId { session: spec.set().session, attempt: 0 },
commitments: expected_commitments
})
);
@@ -150,7 +150,7 @@ async fn dkg_test() {
assert_eq!(
msgs.pop_front().unwrap(),
CoordinatorMessage::KeyGen(key_gen::CoordinatorMessage::Commitments {
id: KeyGenId { set: spec.set(), attempt: 0 },
id: KeyGenId { session: spec.set().session, attempt: 0 },
commitments: expected_commitments
})
);
@@ -214,7 +214,7 @@ async fn dkg_test() {
// Each scanner should emit a distinct shares message
let shares_for = |i: usize| {
CoordinatorMessage::KeyGen(key_gen::CoordinatorMessage::Shares {
id: KeyGenId { set: spec.set(), attempt: 0 },
id: KeyGenId { session: spec.set().session, attempt: 0 },
shares: vec![txs
.iter()
.enumerate()
@@ -267,7 +267,7 @@ async fn dkg_test() {
assert_eq!(
msgs.pop_front().unwrap(),
CoordinatorMessage::KeyGen(key_gen::CoordinatorMessage::Commitments {
id: KeyGenId { set: spec.set(), attempt: 0 },
id: KeyGenId { session: spec.set().session, attempt: 0 },
commitments: expected_commitments
})
);

View File

@@ -5,7 +5,7 @@ use zeroize::Zeroizing;
use ciphersuite::{Ciphersuite, Ristretto, group::GroupEncoding};
use frost::Participant;
use serai_client::validator_sets::primitives::{ValidatorSet, KeyPair};
use serai_client::validator_sets::primitives::KeyPair;
use processor_messages::coordinator::SubstrateSignableId;
@@ -50,7 +50,6 @@ create_db!(
PlanIds: (genesis: &[u8], block: u64) -> Vec<[u8; 32]>,
ConfirmationNonces: (genesis: [u8; 32], attempt: u32) -> HashMap<Participant, Vec<u8>>,
CurrentlyCompletingKeyPair: (genesis: [u8; 32]) -> KeyPair,
KeyPairDb: (set: ValidatorSet) -> KeyPair,
AttemptDb: (genesis: [u8; 32], topic: &Topic) -> u32,
DataReceived: (genesis: [u8; 32], data_spec: &DataSpecification) -> u16,
DataDb: (genesis: [u8; 32], data_spec: &DataSpecification, signer_bytes: &[u8; 32]) -> Vec<u8>,

View File

@@ -30,7 +30,7 @@ use crate::{
nonce_decider::NonceDecider,
dkg_confirmer::DkgConfirmer,
scanner::{RecognizedIdType, RIDTrait},
FatallySlashed, DkgShare, PlanIds, ConfirmationNonces, KeyPairDb, AttemptDb, DataDb,
FatallySlashed, DkgShare, PlanIds, ConfirmationNonces, AttemptDb, DataDb,
},
};
@@ -260,7 +260,7 @@ pub(crate) async fn handle_application_tx<
.send(
spec.set().network,
key_gen::CoordinatorMessage::Commitments {
id: KeyGenId { set: spec.set(), attempt },
id: KeyGenId { session: spec.set().session, attempt },
commitments,
},
)
@@ -397,7 +397,7 @@ pub(crate) async fn handle_application_tx<
.send(
spec.set().network,
key_gen::CoordinatorMessage::Shares {
id: KeyGenId { set: spec.set(), attempt },
id: KeyGenId { session: spec.set().session, attempt },
shares: expanded_shares,
},
)
@@ -443,7 +443,7 @@ pub(crate) async fn handle_application_tx<
.send(
spec.set().network,
key_gen::CoordinatorMessage::VerifyBlame {
id: KeyGenId { set: spec.set(), attempt },
id: KeyGenId { session: spec.set().session, attempt },
accuser,
accused: faulty,
share,
@@ -504,15 +504,6 @@ pub(crate) async fn handle_application_tx<
SubstrateSignableId::CosigningSubstrateBlock(hash),
);
let key = loop {
let Some(key_pair) = KeyPairDb::get(txn, spec.set()) else {
// This can happen based on a timing condition
log::warn!("CosignSubstrateBlock yet keys weren't set yet");
tokio::time::sleep(core::time::Duration::from_secs(1)).await;
continue;
};
break key_pair.0.into();
};
let block_number = SeraiBlockNumber::get(txn, hash)
.expect("CosignSubstrateBlock yet didn't save Serai block number");
processors
@@ -520,7 +511,7 @@ pub(crate) async fn handle_application_tx<
spec.set().network,
coordinator::CoordinatorMessage::CosignSubstrateBlock {
id: SubstrateSignId {
key,
session: spec.set().session,
id: SubstrateSignableId::CosigningSubstrateBlock(hash),
attempt: 0,
},
@@ -579,12 +570,15 @@ pub(crate) async fn handle_application_tx<
Accumulation::Ready(DataSet::Participating(mut preprocesses)) => {
unflatten(spec, &mut preprocesses);
NonceDecider::selected_for_signing_substrate(txn, genesis, data.plan);
let key = KeyPairDb::get(txn, spec.set()).unwrap().0 .0;
processors
.send(
spec.set().network,
coordinator::CoordinatorMessage::SubstratePreprocesses {
id: SubstrateSignId { key, id: data.plan, attempt: data.attempt },
id: SubstrateSignId {
session: spec.set().session,
id: data.plan,
attempt: data.attempt,
},
preprocesses: preprocesses
.into_iter()
.map(|(k, v)| (k, v.try_into().unwrap()))
@@ -613,12 +607,15 @@ pub(crate) async fn handle_application_tx<
) {
Accumulation::Ready(DataSet::Participating(mut shares)) => {
unflatten(spec, &mut shares);
let key = KeyPairDb::get(txn, spec.set()).unwrap().0 .0;
processors
.send(
spec.set().network,
coordinator::CoordinatorMessage::SubstrateShares {
id: SubstrateSignId { key, id: data.plan, attempt: data.attempt },
id: SubstrateSignId {
session: spec.set().session,
id: data.plan,
attempt: data.attempt,
},
shares: shares
.into_iter()
.map(|(validator, share)| (validator, share.try_into().unwrap()))
@@ -636,7 +633,6 @@ pub(crate) async fn handle_application_tx<
let Ok(_) = check_sign_data_len::<D>(txn, spec, data.signed.signer, data.data.len()) else {
return;
};
let key_pair = KeyPairDb::get(txn, spec.set());
match handle(
txn,
&DataSpecification {
@@ -654,14 +650,7 @@ pub(crate) async fn handle_application_tx<
.send(
spec.set().network,
sign::CoordinatorMessage::Preprocesses {
id: SignId {
key: key_pair
.expect("completed SignPreprocess despite not setting the key pair")
.1
.into(),
id: data.plan,
attempt: data.attempt,
},
id: SignId { session: spec.set().session, id: data.plan, attempt: data.attempt },
preprocesses,
},
)
@@ -675,7 +664,6 @@ pub(crate) async fn handle_application_tx<
let Ok(_) = check_sign_data_len::<D>(txn, spec, data.signed.signer, data.data.len()) else {
return;
};
let key_pair = KeyPairDb::get(txn, spec.set());
match handle(
txn,
&DataSpecification {
@@ -692,14 +680,7 @@ pub(crate) async fn handle_application_tx<
.send(
spec.set().network,
sign::CoordinatorMessage::Shares {
id: SignId {
key: key_pair
.expect("completed SignShares despite not setting the key pair")
.1
.into(),
id: data.plan,
attempt: data.attempt,
},
id: SignId { session: spec.set().session, id: data.plan, attempt: data.attempt },
shares,
},
)
@@ -727,13 +708,15 @@ pub(crate) async fn handle_application_tx<
};
// TODO: Confirm this signer hasn't prior published a completion
let Some(key_pair) = KeyPairDb::get(txn, spec.set()) else {
panic!("SignCompleted for recognized plan ID despite not having a key pair for this set")
};
processors
.send(
spec.set().network,
sign::CoordinatorMessage::Completed { key: key_pair.1.to_vec(), id: plan, tx: tx_hash },
sign::CoordinatorMessage::Completed {
session: spec.set().session,
id: plan,
tx: tx_hash,
},
)
.await;
}