2025-01-11 02:10:15 -05:00
|
|
|
use core::{future::Future, time::Duration};
|
2025-01-11 01:55:36 -05:00
|
|
|
use std::sync::Arc;
|
|
|
|
|
|
|
|
|
|
use serai_db::{DbTxn, Db};
|
|
|
|
|
|
|
|
|
|
use serai_client::validator_sets::primitives::ValidatorSet;
|
|
|
|
|
|
2025-01-11 02:10:15 -05:00
|
|
|
use ::tributary::{ProvidedError, Tributary};
|
|
|
|
|
|
|
|
|
|
use serai_task::{TaskHandle, ContinuallyRan};
|
2025-01-11 01:55:36 -05:00
|
|
|
|
|
|
|
|
use message_queue::{Service, Metadata, client::MessageQueue};
|
|
|
|
|
|
2025-01-11 02:10:15 -05:00
|
|
|
use serai_cosign::Cosigning;
|
2025-01-11 01:55:36 -05:00
|
|
|
use serai_coordinator_substrate::NewSetInformation;
|
|
|
|
|
use serai_coordinator_p2p::P2p;
|
|
|
|
|
|
Coordinator Cleanup (#481)
* Move logic for evaluating if a cosign should occur to its own file
Cleans it up and makes it more robust.
* Have expected_next_batch return an error instead of retrying
While convenient to offer an error-free implementation, it potentially caused
very long lived lock acquisitions in handle_processor_message.
* Unify and clean DkgConfirmer and DkgRemoval
Does so via adding a new file for the common code, SigningProtocol.
Modifies from_cache to return the preprocess with the machine, as there's no
reason not to. Also removes an unused Result around the type.
Clarifies the security around deterministic nonces, removing them for
saved-to-disk cached preprocesses. The cached preprocesses are encrypted as the
DB is not a proper secret store.
Moves arguments always present in the protocol from function arguments into the
struct itself.
Removes the horribly ugly code in DkgRemoval, fixing multiple issues present
with it which would cause it to fail on use.
* Set SeraiBlockNumber in cosign.rs as it's used by the cosigning protocol
* Remove unnecessary Clone from lambdas in coordinator
* Remove the EventDb from Tributary scanner
We used per-Transaction DB TXNs so on error, we don't have to rescan the entire
block yet only the rest of it. We prevented scanning multiple transactions by
tracking which we already had.
This is over-engineered and not worth it.
* Implement borsh for HasEvents, removing the manual encoding
* Merge DkgConfirmer and DkgRemoval into signing_protocol.rs
Fixes a bug in DkgConfirmer which would cause it to improperly handle indexes
if any validator had multiple key shares.
* Strictly type DataSpecification's Label
* Correct threshold_i_map_to_keys_and_musig_i_map
It didn't include the participant's own index and accordingly was offset.
* Create TributaryBlockHandler
This struct contains all variables prior passed to handle_block and stops them
from being passed around again and again.
This also ensures fatal_slash is only called while handling a block, as needed
as it expects to operate under perfect consensus.
* Inline accumulate, store confirmation nonces with shares
Inlining accumulate makes sense due to the amount of data accumulate needed to
be passed.
Storing confirmation nonces with shares ensures that both are available or
neither. Prior, one could be yet the other may not have been (requiring an
assert in runtime to ensure we didn't bungle it somehow).
* Create helper functions for handling DkgRemoval/SubstrateSign/Sign Tributary TXs
* Move Label into SignData
All of our transactions which use SignData end up with the same common usage
pattern for Label, justifying this.
Removes 3 transactions, explicitly de-duplicating their handlers.
* Remove CurrentlyCompletingKeyPair for the non-contextual DkgKeyPair
* Remove the manual read/write for TributarySpec for borsh
This struct doesn't have any optimizations booned by the manual impl. Using
borsh reduces our scope.
* Use temporary variables to further minimize LoC in tributary handler
* Remove usage of tuples for non-trivial Tributary transactions
* Remove serde from dkg
serde could be used to deserialize intenrally inconsistent objects which could
lead to panics or faults.
The BorshDeserialize derives have been replaced with a manual implementation
which won't produce inconsistent objects.
* Abstract Future generics using new trait definitions in coordinator
* Move published_signed_transaction to tributary/mod.rs to reduce the size of main.rs
* Split coordinator/src/tributary/mod.rs into spec.rs and transaction.rs
2023-12-10 20:21:44 -05:00
|
|
|
mod transaction;
|
2025-01-02 09:11:04 -05:00
|
|
|
pub use transaction::Transaction;
|
Slash malevolent validators (#294)
* add slash tx
* ignore unsigned tx replays
* verify that provided evidence is valid
* fix clippy + fmt
* move application tx handling to another module
* partially handle the tendermint txs
* fix pr comments
* support unsigned app txs
* add slash target to the votes
* enforce provided, unsigned, signed tx ordering within a block
* bug fixes
* add unit test for tendermint txs
* bug fixes
* update tests for tendermint txs
* add tx ordering test
* tidy up tx ordering test
* cargo +nightly fmt
* Misc fixes from rebasing
* Finish resolving clippy
* Remove sha3 from tendermint-machine
* Resolve a DoS in SlashEvidence's read
Also moves Evidence from Vec<Message> to (Message, Option<Message>). That
should meet all requirements while being a bit safer.
* Make lazy_static a dev-depend for tributary
* Various small tweaks
One use of sort was inefficient, sorting unsigned || signed when unsigned was
already properly sorted. Given how the unsigned TXs were given a nonce of 0, an
unstable sort may swap places with an unsigned TX and a signed TX with a nonce
of 0 (leading to a faulty block).
The extra protection added here sorts signed, then concats.
* Fix Tributary tests I broke, start review on tendermint/tx.rs
* Finish reviewing everything outside tests and empty_signature
* Remove empty_signature
empty_signature led to corrupted local state histories. Unfortunately, the API
is only sane with a signature.
We now use the actual signature, which risks creating a signature over a
malicious message if we have ever have an invariant producing malicious
messages. Prior, we only signed the message after the local machine confirmed
it was okay per the local view of consensus.
This is tolerated/preferred over a corrupt state history since production of
such messages is already an invariant. TODOs are added to make handling of this
theoretical invariant further robust.
* Remove async_sequential for tokio::test
There was no competition for resources forcing them to be run sequentially.
* Modify block order test to be statistically significant without multiple runs
* Clean tests
---------
Co-authored-by: Luke Parker <lukeparker5132@gmail.com>
2023-08-21 07:28:23 +03:00
|
|
|
|
2025-01-02 09:11:04 -05:00
|
|
|
mod db;
|
Coordinator Cleanup (#481)
* Move logic for evaluating if a cosign should occur to its own file
Cleans it up and makes it more robust.
* Have expected_next_batch return an error instead of retrying
While convenient to offer an error-free implementation, it potentially caused
very long lived lock acquisitions in handle_processor_message.
* Unify and clean DkgConfirmer and DkgRemoval
Does so via adding a new file for the common code, SigningProtocol.
Modifies from_cache to return the preprocess with the machine, as there's no
reason not to. Also removes an unused Result around the type.
Clarifies the security around deterministic nonces, removing them for
saved-to-disk cached preprocesses. The cached preprocesses are encrypted as the
DB is not a proper secret store.
Moves arguments always present in the protocol from function arguments into the
struct itself.
Removes the horribly ugly code in DkgRemoval, fixing multiple issues present
with it which would cause it to fail on use.
* Set SeraiBlockNumber in cosign.rs as it's used by the cosigning protocol
* Remove unnecessary Clone from lambdas in coordinator
* Remove the EventDb from Tributary scanner
We used per-Transaction DB TXNs so on error, we don't have to rescan the entire
block yet only the rest of it. We prevented scanning multiple transactions by
tracking which we already had.
This is over-engineered and not worth it.
* Implement borsh for HasEvents, removing the manual encoding
* Merge DkgConfirmer and DkgRemoval into signing_protocol.rs
Fixes a bug in DkgConfirmer which would cause it to improperly handle indexes
if any validator had multiple key shares.
* Strictly type DataSpecification's Label
* Correct threshold_i_map_to_keys_and_musig_i_map
It didn't include the participant's own index and accordingly was offset.
* Create TributaryBlockHandler
This struct contains all variables prior passed to handle_block and stops them
from being passed around again and again.
This also ensures fatal_slash is only called while handling a block, as needed
as it expects to operate under perfect consensus.
* Inline accumulate, store confirmation nonces with shares
Inlining accumulate makes sense due to the amount of data accumulate needed to
be passed.
Storing confirmation nonces with shares ensures that both are available or
neither. Prior, one could be yet the other may not have been (requiring an
assert in runtime to ensure we didn't bungle it somehow).
* Create helper functions for handling DkgRemoval/SubstrateSign/Sign Tributary TXs
* Move Label into SignData
All of our transactions which use SignData end up with the same common usage
pattern for Label, justifying this.
Removes 3 transactions, explicitly de-duplicating their handlers.
* Remove CurrentlyCompletingKeyPair for the non-contextual DkgKeyPair
* Remove the manual read/write for TributarySpec for borsh
This struct doesn't have any optimizations booned by the manual impl. Using
borsh reduces our scope.
* Use temporary variables to further minimize LoC in tributary handler
* Remove usage of tuples for non-trivial Tributary transactions
* Remove serde from dkg
serde could be used to deserialize intenrally inconsistent objects which could
lead to panics or faults.
The BorshDeserialize derives have been replaced with a manual implementation
which won't produce inconsistent objects.
* Abstract Future generics using new trait definitions in coordinator
* Move published_signed_transaction to tributary/mod.rs to reduce the size of main.rs
* Split coordinator/src/tributary/mod.rs into spec.rs and transaction.rs
2023-12-10 20:21:44 -05:00
|
|
|
|
2025-01-02 09:11:04 -05:00
|
|
|
mod scan;
|
2025-01-10 02:22:58 -05:00
|
|
|
pub(crate) use scan::ScanTributaryTask;
|
2025-01-11 01:55:36 -05:00
|
|
|
|
|
|
|
|
pub(crate) struct ScanTributaryMessagesTask<TD: Db> {
|
|
|
|
|
pub(crate) tributary_db: TD,
|
|
|
|
|
pub(crate) set: ValidatorSet,
|
|
|
|
|
pub(crate) message_queue: Arc<MessageQueue>,
|
|
|
|
|
}
|
2025-01-11 02:10:15 -05:00
|
|
|
|
2025-01-11 01:55:36 -05:00
|
|
|
impl<TD: Db> ContinuallyRan for ScanTributaryMessagesTask<TD> {
|
|
|
|
|
fn run_iteration(&mut self) -> impl Send + Future<Output = Result<bool, String>> {
|
|
|
|
|
async move {
|
|
|
|
|
let mut made_progress = false;
|
|
|
|
|
loop {
|
|
|
|
|
let mut txn = self.tributary_db.txn();
|
|
|
|
|
let Some(msg) = db::TributaryDb::try_recv_message(&mut txn, self.set) else { break };
|
|
|
|
|
let metadata = Metadata {
|
|
|
|
|
from: Service::Coordinator,
|
|
|
|
|
to: Service::Processor(self.set.network),
|
|
|
|
|
intent: msg.intent(),
|
|
|
|
|
};
|
|
|
|
|
let msg = borsh::to_vec(&msg).unwrap();
|
|
|
|
|
// TODO: Make this fallible
|
|
|
|
|
self.message_queue.queue(metadata, msg).await;
|
|
|
|
|
txn.commit();
|
|
|
|
|
made_progress = true;
|
|
|
|
|
}
|
|
|
|
|
Ok(made_progress)
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|
2025-01-11 02:10:15 -05:00
|
|
|
|
|
|
|
|
async fn provide_transaction<TD: Db, P: P2p>(
|
|
|
|
|
set: ValidatorSet,
|
|
|
|
|
tributary: &Tributary<TD, Transaction, P>,
|
|
|
|
|
tx: Transaction,
|
|
|
|
|
) {
|
|
|
|
|
match tributary.provide_transaction(tx.clone()).await {
|
|
|
|
|
// The Tributary uses its own DB, so we may provide this multiple times if we reboot before
|
|
|
|
|
// committing the txn which provoked this
|
|
|
|
|
Ok(()) | Err(ProvidedError::AlreadyProvided) => {}
|
|
|
|
|
Err(ProvidedError::NotProvided) => {
|
|
|
|
|
panic!("providing a Transaction which wasn't a Provided transaction: {tx:?}");
|
|
|
|
|
}
|
|
|
|
|
Err(ProvidedError::InvalidProvided(e)) => {
|
|
|
|
|
panic!("providing an invalid Provided transaction, tx: {tx:?}, error: {e:?}")
|
|
|
|
|
}
|
|
|
|
|
Err(ProvidedError::LocalMismatchesOnChain) => loop {
|
|
|
|
|
// The Tributary's scan task won't advance if we don't have the Provided transactions
|
|
|
|
|
// present on-chain, and this enters an infinite loop to block the calling task from
|
|
|
|
|
// advancing
|
|
|
|
|
log::error!(
|
|
|
|
|
"Tributary {:?} was supposed to provide {:?} but peers disagree, halting Tributary",
|
|
|
|
|
set,
|
|
|
|
|
tx,
|
|
|
|
|
);
|
|
|
|
|
// Print this every five minutes as this does need to be handled
|
|
|
|
|
tokio::time::sleep(Duration::from_secs(5 * 60)).await;
|
|
|
|
|
},
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
/// Run a Tributary.
|
|
|
|
|
///
|
|
|
|
|
/// The Tributary handle existing causes the Tributary's consensus engine to be run. We distinctly
|
|
|
|
|
/// have `ScanTributaryTask` to scan the produced blocks. This function provides Provided
|
|
|
|
|
/// transactions onto the Tributary and invokes ScanTributaryTask whenver a new Tributary block is
|
|
|
|
|
/// produced (instead of only on the standard interval).
|
|
|
|
|
pub(crate) async fn run<CD: Db, TD: Db, P: P2p>(
|
|
|
|
|
mut db: CD,
|
|
|
|
|
set: NewSetInformation,
|
|
|
|
|
tributary: Tributary<TD, Transaction, P>,
|
|
|
|
|
scan_tributary_task: TaskHandle,
|
|
|
|
|
) {
|
|
|
|
|
loop {
|
|
|
|
|
// Break once this Tributary is retired
|
|
|
|
|
if crate::RetiredTributary::get(&db, set.set.network).map(|session| session.0) >=
|
|
|
|
|
Some(set.set.session.0)
|
|
|
|
|
{
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Check if we produced any cosigns we were supposed to
|
|
|
|
|
let mut pending_notable_cosign = false;
|
|
|
|
|
loop {
|
|
|
|
|
let mut txn = db.txn();
|
|
|
|
|
|
|
|
|
|
// Fetch the next cosign this tributary should handle
|
|
|
|
|
let Some(cosign) = crate::PendingCosigns::try_recv(&mut txn, set.set) else { break };
|
|
|
|
|
pending_notable_cosign = cosign.notable;
|
|
|
|
|
|
|
|
|
|
// If we (Serai) haven't cosigned this block, break as this is still pending
|
|
|
|
|
let Ok(latest) = Cosigning::<CD>::latest_cosigned_block_number(&txn) else { break };
|
|
|
|
|
if latest < cosign.block_number {
|
|
|
|
|
break;
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Because we've cosigned it, provide the TX for that
|
|
|
|
|
provide_transaction(
|
|
|
|
|
set.set,
|
|
|
|
|
&tributary,
|
|
|
|
|
Transaction::Cosigned { substrate_block_hash: cosign.block_hash },
|
|
|
|
|
)
|
|
|
|
|
.await;
|
|
|
|
|
// Clear pending_notable_cosign since this cosign isn't pending
|
|
|
|
|
pending_notable_cosign = false;
|
|
|
|
|
|
|
|
|
|
// Commit the txn to clear this from PendingCosigns
|
|
|
|
|
txn.commit();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// If we don't have any notable cosigns pending, provide the next set of cosign intents
|
|
|
|
|
if pending_notable_cosign {
|
|
|
|
|
let mut txn = db.txn();
|
|
|
|
|
// intended_cosigns will only yield up to and including the next notable cosign
|
|
|
|
|
for cosign in Cosigning::<CD>::intended_cosigns(&mut txn, set.set) {
|
|
|
|
|
// Flag this cosign as pending
|
|
|
|
|
crate::PendingCosigns::send(&mut txn, set.set, &cosign);
|
|
|
|
|
// Provide the transaction to queue it for work
|
|
|
|
|
provide_transaction(
|
|
|
|
|
set.set,
|
|
|
|
|
&tributary,
|
|
|
|
|
Transaction::Cosign { substrate_block_hash: cosign.block_hash },
|
|
|
|
|
)
|
|
|
|
|
.await;
|
|
|
|
|
}
|
|
|
|
|
txn.commit();
|
|
|
|
|
}
|
|
|
|
|
|
|
|
|
|
// Have the tributary scanner run as soon as there's a new block
|
|
|
|
|
// This is wrapped in a timeout so we don't go too long without running the above code
|
|
|
|
|
match tokio::time::timeout(
|
|
|
|
|
Duration::from_millis(::tributary::tendermint::TARGET_BLOCK_TIME.into()),
|
|
|
|
|
tributary.next_block_notification().await,
|
|
|
|
|
)
|
|
|
|
|
.await
|
|
|
|
|
{
|
|
|
|
|
// Future resolved within the timeout, notification
|
|
|
|
|
Ok(Ok(())) => scan_tributary_task.run_now(),
|
|
|
|
|
// Future resolved within the timeout, notification failed due to sender being dropped
|
|
|
|
|
// unreachable since this owns the tributary object and doesn't drop it
|
|
|
|
|
Ok(Err(_)) => panic!("tributary was dropped causing notification to error"),
|
|
|
|
|
// Future didn't resolve within the timeout
|
|
|
|
|
Err(_) => {}
|
|
|
|
|
}
|
|
|
|
|
}
|
|
|
|
|
}
|