Op pool and gossip for BLS to execution changes (#3726)

This commit is contained in:
Michael Sproul 2022-11-25 07:09:26 +11:00 committed by GitHub
parent 58b54f0a53
commit 788b337951
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 539 additions and 69 deletions

View File

@ -14,7 +14,12 @@ node_test_rig = { path = "../testing/node_test_rig" }
[features]
write_ssz_files = ["beacon_chain/write_ssz_files"] # Writes debugging .ssz files to /tmp during block processing.
withdrawals = ["beacon_chain/withdrawals", "types/withdrawals", "store/withdrawals", "execution_layer/withdrawals"]
withdrawals-processing = ["beacon_chain/withdrawals-processing", "store/withdrawals-processing", "execution_layer/withdrawals-processing"]
withdrawals-processing = [
"beacon_chain/withdrawals-processing",
"store/withdrawals-processing",
"execution_layer/withdrawals-processing",
"http_api/withdrawals-processing",
]
[dependencies]
eth2_config = { path = "../common/eth2_config" }

View File

@ -11,7 +11,12 @@ write_ssz_files = [] # Writes debugging .ssz files to /tmp during block process
participation_metrics = [] # Exposes validator participation metrics to Prometheus.
fork_from_env = [] # Initialise the harness chain spec from the FORK_NAME env variable
withdrawals = ["state_processing/withdrawals", "types/withdrawals", "store/withdrawals", "execution_layer/withdrawals"]
withdrawals-processing = ["state_processing/withdrawals-processing", "store/withdrawals-processing", "execution_layer/withdrawals-processing"]
withdrawals-processing = [
"state_processing/withdrawals-processing",
"store/withdrawals-processing",
"execution_layer/withdrawals-processing",
"operation_pool/withdrawals-processing"
]
[dev-dependencies]
maplit = "1.0.2"

View File

@ -341,6 +341,10 @@ pub struct BeaconChain<T: BeaconChainTypes> {
/// Maintains a record of which validators we've seen attester slashings for.
pub(crate) observed_attester_slashings:
Mutex<ObservedOperations<AttesterSlashing<T::EthSpec>, T::EthSpec>>,
/// Maintains a record of which validators we've seen BLS to execution changes for.
#[cfg(feature = "withdrawals-processing")]
pub(crate) observed_bls_to_execution_changes:
Mutex<ObservedOperations<SignedBlsToExecutionChange, T::EthSpec>>,
/// Provides information from the Ethereum 1 (PoW) chain.
pub eth1_chain: Option<Eth1Chain<T::Eth1Chain, T::EthSpec>>,
/// Interfaces with the execution client.
@ -2181,6 +2185,42 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
}
}
/// Verify a signed BLS to exection change before allowing it to propagate on the gossip network.
pub fn verify_bls_to_execution_change_for_gossip(
&self,
bls_to_execution_change: SignedBlsToExecutionChange,
) -> Result<ObservationOutcome<SignedBlsToExecutionChange, T::EthSpec>, Error> {
#[cfg(feature = "withdrawals-processing")]
{
let wall_clock_state = self.wall_clock_state()?;
Ok(self
.observed_bls_to_execution_changes
.lock()
.verify_and_observe(bls_to_execution_change, &wall_clock_state, &self.spec)?)
}
#[cfg(not(feature = "withdrawals-processing"))]
{
drop(bls_to_execution_change);
Ok(ObservationOutcome::AlreadyKnown)
}
}
/// Import a BLS to execution change to the op pool.
pub fn import_bls_to_execution_change(
&self,
bls_to_execution_change: SigVerifiedOp<SignedBlsToExecutionChange, T::EthSpec>,
) {
if self.eth1_chain.is_some() {
#[cfg(feature = "withdrawals-processing")]
self.op_pool
.insert_bls_to_execution_change(bls_to_execution_change);
#[cfg(not(feature = "withdrawals-processing"))]
drop(bls_to_execution_change);
}
}
/// Attempt to obtain sync committee duties from the head.
pub fn sync_committee_duties_from_head(
&self,
@ -3491,6 +3531,8 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
let eth1_data = eth1_chain.eth1_data_for_block_production(&state, &self.spec)?;
let deposits = eth1_chain.deposits_for_block_inclusion(&state, &eth1_data, &self.spec)?;
#[cfg(feature = "withdrawals")]
let bls_to_execution_changes = self
.op_pool
.get_bls_to_execution_changes(&state, &self.spec);

View File

@ -780,6 +780,8 @@ where
observed_voluntary_exits: <_>::default(),
observed_proposer_slashings: <_>::default(),
observed_attester_slashings: <_>::default(),
#[cfg(feature = "withdrawals-processing")]
observed_bls_to_execution_changes: <_>::default(),
eth1_chain: self.eth1_chain,
execution_layer: self.execution_layer,
genesis_validators_root,

View File

@ -907,8 +907,12 @@ impl<T: BeaconChainTypes> BeaconChain<T> {
.execution_status
.is_optimistic_or_invalid();
self.op_pool
.prune_all(&new_snapshot.beacon_state, self.epoch()?);
self.op_pool.prune_all(
&new_snapshot.beacon_block,
&new_snapshot.beacon_state,
self.epoch()?,
&self.spec,
);
self.observed_block_producers.write().prune(
new_view

View File

@ -17,8 +17,9 @@ use ssz_types::Error as SszTypesError;
use state_processing::{
block_signature_verifier::Error as BlockSignatureVerifierError,
per_block_processing::errors::{
AttestationValidationError, AttesterSlashingValidationError, ExitValidationError,
ProposerSlashingValidationError, SyncCommitteeMessageValidationError,
AttestationValidationError, AttesterSlashingValidationError,
BlsExecutionChangeValidationError, ExitValidationError, ProposerSlashingValidationError,
SyncCommitteeMessageValidationError,
},
signature_sets::Error as SignatureSetError,
state_advance::Error as StateAdvanceError,
@ -70,6 +71,7 @@ pub enum BeaconChainError {
ExitValidationError(ExitValidationError),
ProposerSlashingValidationError(ProposerSlashingValidationError),
AttesterSlashingValidationError(AttesterSlashingValidationError),
BlsExecutionChangeValidationError(BlsExecutionChangeValidationError),
StateSkipTooLarge {
start_slot: Slot,
requested_slot: Slot,
@ -212,6 +214,7 @@ easy_from_to!(SyncCommitteeMessageValidationError, BeaconChainError);
easy_from_to!(ExitValidationError, BeaconChainError);
easy_from_to!(ProposerSlashingValidationError, BeaconChainError);
easy_from_to!(AttesterSlashingValidationError, BeaconChainError);
easy_from_to!(BlsExecutionChangeValidationError, BeaconChainError);
easy_from_to!(SszTypesError, BeaconChainError);
easy_from_to!(OpPoolError, BeaconChainError);
easy_from_to!(NaiveAggregationError, BeaconChainError);

View File

@ -1,5 +1,5 @@
use derivative::Derivative;
use smallvec::SmallVec;
use smallvec::{smallvec, SmallVec};
use ssz::{Decode, Encode};
use state_processing::{SigVerifiedOp, VerifyOperation};
use std::collections::HashSet;
@ -9,6 +9,9 @@ use types::{
SignedVoluntaryExit, Slot,
};
#[cfg(feature = "withdrawals-processing")]
use types::SignedBlsToExecutionChange;
/// Number of validator indices to store on the stack in `observed_validators`.
pub const SMALL_VEC_SIZE: usize = 8;
@ -39,7 +42,7 @@ pub enum ObservationOutcome<T: Encode + Decode, E: EthSpec> {
AlreadyKnown,
}
/// Trait for exits and slashings which can be observed using `ObservedOperations`.
/// Trait for operations which can be observed using `ObservedOperations`.
pub trait ObservableOperation<E: EthSpec>: VerifyOperation<E> + Sized {
/// The set of validator indices involved in this operation.
///
@ -49,13 +52,13 @@ pub trait ObservableOperation<E: EthSpec>: VerifyOperation<E> + Sized {
impl<E: EthSpec> ObservableOperation<E> for SignedVoluntaryExit {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
std::iter::once(self.message.validator_index).collect()
smallvec![self.message.validator_index]
}
}
impl<E: EthSpec> ObservableOperation<E> for ProposerSlashing {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
std::iter::once(self.signed_header_1.message.proposer_index).collect()
smallvec![self.signed_header_1.message.proposer_index]
}
}
@ -80,6 +83,13 @@ impl<E: EthSpec> ObservableOperation<E> for AttesterSlashing<E> {
}
}
#[cfg(feature = "withdrawals-processing")]
impl<E: EthSpec> ObservableOperation<E> for SignedBlsToExecutionChange {
fn observed_validators(&self) -> SmallVec<[u64; SMALL_VEC_SIZE]> {
smallvec![self.message.validator_index]
}
}
impl<T: ObservableOperation<E>, E: EthSpec> ObservedOperations<T, E> {
pub fn verify_and_observe(
&mut self,

View File

@ -5,6 +5,9 @@ authors = ["Paul Hauner <paul@paulhauner.com>"]
edition = "2021"
autotests = false # using a single test binary compiles faster
[features]
withdrawals-processing = []
[dependencies]
warp = { version = "0.3.2", features = ["tls"] }
serde = { version = "1.0.116", features = ["derive"] }

View File

@ -49,9 +49,9 @@ use types::{
Attestation, AttestationData, AttesterSlashing, BeaconStateError, BlindedPayload,
CommitteeCache, ConfigAndPreset, Epoch, EthSpec, ForkName, FullPayload,
ProposerPreparationData, ProposerSlashing, RelativeEpoch, SignedAggregateAndProof,
SignedBeaconBlock, SignedBlindedBeaconBlock, SignedContributionAndProof,
SignedValidatorRegistrationData, SignedVoluntaryExit, Slot, SyncCommitteeMessage,
SyncContributionData,
SignedBeaconBlock, SignedBlindedBeaconBlock, SignedBlsToExecutionChange,
SignedContributionAndProof, SignedValidatorRegistrationData, SignedVoluntaryExit, Slot,
SyncCommitteeMessage, SyncContributionData,
};
use version::{
add_consensus_version_header, execution_optimistic_fork_versioned_response,
@ -1536,6 +1536,59 @@ pub fn serve<T: BeaconChainTypes>(
},
);
// GET beacon/pool/bls_to_execution_changes
let get_beacon_pool_bls_to_execution_changes = beacon_pool_path
.clone()
.and(warp::path("bls_to_execution_changes"))
.and(warp::path::end())
.and_then(|chain: Arc<BeaconChain<T>>| {
blocking_json_task(move || {
let address_changes = chain.op_pool.get_all_bls_to_execution_changes();
Ok(api_types::GenericResponse::from(address_changes))
})
});
// POST beacon/pool/bls_to_execution_changes
let post_beacon_pool_bls_to_execution_changes = beacon_pool_path
.clone()
.and(warp::path("bls_to_execution_changes"))
.and(warp::path::end())
.and(warp::body::json())
.and(network_tx_filter.clone())
.and_then(
|chain: Arc<BeaconChain<T>>,
address_change: SignedBlsToExecutionChange,
network_tx: UnboundedSender<NetworkMessage<T::EthSpec>>| {
blocking_json_task(move || {
let outcome = chain
.verify_bls_to_execution_change_for_gossip(address_change)
.map_err(|e| {
warp_utils::reject::object_invalid(format!(
"gossip verification failed: {:?}",
e
))
})?;
if let ObservationOutcome::New(address_change) = outcome {
#[cfg(feature = "withdrawals-processing")]
{
publish_pubsub_message(
&network_tx,
PubsubMessage::BlsToExecutionChange(Box::new(
address_change.as_inner().clone(),
)),
)?;
}
drop(network_tx);
chain.import_bls_to_execution_change(address_change);
}
Ok(())
})
},
);
// GET beacon/deposit_snapshot
let get_beacon_deposit_snapshot = eth_v1
.and(warp::path("beacon"))
@ -3170,6 +3223,7 @@ pub fn serve<T: BeaconChainTypes>(
.or(get_beacon_pool_attester_slashings.boxed())
.or(get_beacon_pool_proposer_slashings.boxed())
.or(get_beacon_pool_voluntary_exits.boxed())
.or(get_beacon_pool_bls_to_execution_changes.boxed())
.or(get_beacon_deposit_snapshot.boxed())
.or(get_config_fork_schedule.boxed())
.or(get_config_spec.boxed())
@ -3218,6 +3272,7 @@ pub fn serve<T: BeaconChainTypes>(
.or(post_beacon_pool_proposer_slashings.boxed())
.or(post_beacon_pool_voluntary_exits.boxed())
.or(post_beacon_pool_sync_committees.boxed())
.or(post_beacon_pool_bls_to_execution_changes.boxed())
.or(post_validator_duties_attester.boxed())
.or(post_validator_duties_sync.boxed())
.or(post_validator_aggregate_and_proofs.boxed())

View File

@ -36,6 +36,8 @@ pub struct GossipCache {
signed_contribution_and_proof: Option<Duration>,
/// Timeout for sync committee messages.
sync_committee_message: Option<Duration>,
/// Timeout for signed BLS to execution changes.
bls_to_execution_change: Option<Duration>,
}
#[derive(Default)]
@ -59,6 +61,8 @@ pub struct GossipCacheBuilder {
signed_contribution_and_proof: Option<Duration>,
/// Timeout for sync committee messages.
sync_committee_message: Option<Duration>,
/// Timeout for signed BLS to execution changes.
bls_to_execution_change: Option<Duration>,
}
#[allow(dead_code)]
@ -117,6 +121,12 @@ impl GossipCacheBuilder {
self
}
/// Timeout for BLS to execution change messages.
pub fn bls_to_execution_change_timeout(mut self, timeout: Duration) -> Self {
self.bls_to_execution_change = Some(timeout);
self
}
pub fn build(self) -> GossipCache {
let GossipCacheBuilder {
default_timeout,
@ -129,6 +139,7 @@ impl GossipCacheBuilder {
attester_slashing,
signed_contribution_and_proof,
sync_committee_message,
bls_to_execution_change,
} = self;
GossipCache {
expirations: DelayQueue::default(),
@ -142,6 +153,7 @@ impl GossipCacheBuilder {
attester_slashing: attester_slashing.or(default_timeout),
signed_contribution_and_proof: signed_contribution_and_proof.or(default_timeout),
sync_committee_message: sync_committee_message.or(default_timeout),
bls_to_execution_change: bls_to_execution_change.or(default_timeout),
}
}
}
@ -165,6 +177,7 @@ impl GossipCache {
GossipKind::AttesterSlashing => self.attester_slashing,
GossipKind::SignedContributionAndProof => self.signed_contribution_and_proof,
GossipKind::SyncCommitteeMessage(_) => self.sync_committee_message,
GossipKind::BlsToExecutionChange => self.bls_to_execution_change,
};
let expire_timeout = match expire_timeout {
Some(expire_timeout) => expire_timeout,

View File

@ -199,6 +199,7 @@ impl<AppReqId: ReqId, TSpec: EthSpec> Network<AppReqId, TSpec> {
.attester_slashing_timeout(half_epoch * 2)
// .signed_contribution_and_proof_timeout(timeout) // Do not retry
// .sync_committee_message_timeout(timeout) // Do not retry
.bls_to_execution_change_timeout(half_epoch * 2)
.build()
};

View File

@ -253,6 +253,7 @@ pub(crate) fn create_whitelist_filter(
add(ProposerSlashing);
add(AttesterSlashing);
add(SignedContributionAndProof);
add(BlsToExecutionChange);
for id in 0..attestation_subnet_count {
add(Attestation(SubnetId::new(id)));
}

View File

@ -15,7 +15,8 @@ use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ForkContext, ForkName, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedBeaconBlockAltair, SignedBeaconBlockBase,
SignedBeaconBlockCapella, SignedBeaconBlockEip4844, SignedBeaconBlockMerge,
SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncCommitteeMessage, SyncSubnetId,
SignedBlsToExecutionChange, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
SyncCommitteeMessage, SyncSubnetId,
};
/// TODO(pawan): move this to consensus/types? strictly not a consensus type
@ -48,6 +49,8 @@ pub enum PubsubMessage<T: EthSpec> {
SignedContributionAndProof(Box<SignedContributionAndProof<T>>),
/// Gossipsub message providing notification of unaggregated sync committee signatures with its subnet id.
SyncCommitteeMessage(Box<(SyncSubnetId, SyncCommitteeMessage)>),
/// Gossipsub message for BLS to execution change messages.
BlsToExecutionChange(Box<SignedBlsToExecutionChange>),
}
// Implements the `DataTransform` trait of gossipsub to employ snappy compression
@ -133,6 +136,7 @@ impl<T: EthSpec> PubsubMessage<T> {
PubsubMessage::AttesterSlashing(_) => GossipKind::AttesterSlashing,
PubsubMessage::SignedContributionAndProof(_) => GossipKind::SignedContributionAndProof,
PubsubMessage::SyncCommitteeMessage(data) => GossipKind::SyncCommitteeMessage(data.0),
PubsubMessage::BlsToExecutionChange(_) => GossipKind::BlsToExecutionChange,
}
}
@ -258,6 +262,14 @@ impl<T: EthSpec> PubsubMessage<T> {
sync_committee,
))))
}
GossipKind::BlsToExecutionChange => {
let bls_to_execution_change =
SignedBlsToExecutionChange::from_ssz_bytes(data)
.map_err(|e| format!("{:?}", e))?;
Ok(PubsubMessage::BlsToExecutionChange(Box::new(
bls_to_execution_change,
)))
}
}
}
}
@ -280,6 +292,7 @@ impl<T: EthSpec> PubsubMessage<T> {
PubsubMessage::Attestation(data) => data.1.as_ssz_bytes(),
PubsubMessage::SignedContributionAndProof(data) => data.as_ssz_bytes(),
PubsubMessage::SyncCommitteeMessage(data) => data.1.as_ssz_bytes(),
PubsubMessage::BlsToExecutionChange(data) => data.as_ssz_bytes(),
}
}
}
@ -320,6 +333,13 @@ impl<T: EthSpec> std::fmt::Display for PubsubMessage<T> {
PubsubMessage::SyncCommitteeMessage(data) => {
write!(f, "Sync committee message: subnet_id: {}", *data.0)
}
PubsubMessage::BlsToExecutionChange(data) => {
write!(
f,
"Signed BLS to execution change: validator_index: {}, address: {:?}",
data.message.validator_index, data.message.to_execution_address
)
}
}
}
}

View File

@ -19,8 +19,9 @@ pub const PROPOSER_SLASHING_TOPIC: &str = "proposer_slashing";
pub const ATTESTER_SLASHING_TOPIC: &str = "attester_slashing";
pub const SIGNED_CONTRIBUTION_AND_PROOF_TOPIC: &str = "sync_committee_contribution_and_proof";
pub const SYNC_COMMITTEE_PREFIX_TOPIC: &str = "sync_committee_";
pub const BLS_TO_EXECUTION_CHANGE_TOPIC: &str = "bls_to_execution_change";
pub const CORE_TOPICS: [GossipKind; 7] = [
pub const CORE_TOPICS: [GossipKind; 8] = [
GossipKind::BeaconBlock,
GossipKind::BeaconBlocksAndBlobsSidecar,
GossipKind::BeaconAggregateAndProof,
@ -28,6 +29,7 @@ pub const CORE_TOPICS: [GossipKind; 7] = [
GossipKind::ProposerSlashing,
GossipKind::AttesterSlashing,
GossipKind::SignedContributionAndProof,
GossipKind::BlsToExecutionChange,
];
/// A gossipsub topic which encapsulates the type of messages that should be sent and received over
@ -67,6 +69,8 @@ pub enum GossipKind {
/// Topic for publishing unaggregated sync committee signatures on a particular subnet.
#[strum(serialize = "sync_committee")]
SyncCommitteeMessage(SyncSubnetId),
/// Topic for validator messages which change their withdrawal address.
BlsToExecutionChange,
}
impl std::fmt::Display for GossipKind {
@ -141,6 +145,7 @@ impl GossipTopic {
VOLUNTARY_EXIT_TOPIC => GossipKind::VoluntaryExit,
PROPOSER_SLASHING_TOPIC => GossipKind::ProposerSlashing,
ATTESTER_SLASHING_TOPIC => GossipKind::AttesterSlashing,
BLS_TO_EXECUTION_CHANGE_TOPIC => GossipKind::BlsToExecutionChange,
topic => match committee_topic_index(topic) {
Some(subnet) => match subnet {
Subnet::Attestation(s) => GossipKind::Attestation(s),
@ -177,30 +182,8 @@ impl From<GossipTopic> for Topic {
impl From<GossipTopic> for String {
fn from(topic: GossipTopic) -> String {
let encoding = match topic.encoding {
GossipEncoding::SSZSnappy => SSZ_SNAPPY_ENCODING_POSTFIX,
};
let kind = match topic.kind {
GossipKind::BeaconBlock => BEACON_BLOCK_TOPIC.into(),
GossipKind::BeaconBlocksAndBlobsSidecar => BEACON_BLOCK_AND_BLOBS_SIDECAR_TOPIC.into(),
GossipKind::BeaconAggregateAndProof => BEACON_AGGREGATE_AND_PROOF_TOPIC.into(),
GossipKind::VoluntaryExit => VOLUNTARY_EXIT_TOPIC.into(),
GossipKind::ProposerSlashing => PROPOSER_SLASHING_TOPIC.into(),
GossipKind::AttesterSlashing => ATTESTER_SLASHING_TOPIC.into(),
GossipKind::Attestation(index) => format!("{}{}", BEACON_ATTESTATION_PREFIX, *index,),
GossipKind::SignedContributionAndProof => SIGNED_CONTRIBUTION_AND_PROOF_TOPIC.into(),
GossipKind::SyncCommitteeMessage(index) => {
format!("{}{}", SYNC_COMMITTEE_PREFIX_TOPIC, *index)
}
};
format!(
"/{}/{}/{}/{}",
TOPIC_PREFIX,
hex::encode(topic.fork_digest),
kind,
encoding
)
// Use the `Display` implementation below.
topic.to_string()
}
}
@ -222,6 +205,7 @@ impl std::fmt::Display for GossipTopic {
GossipKind::SyncCommitteeMessage(index) => {
format!("{}{}", SYNC_COMMITTEE_PREFIX_TOPIC, *index)
}
GossipKind::BlsToExecutionChange => BLS_TO_EXECUTION_CHANGE_TOPIC.into(),
};
write!(
f,

View File

@ -64,8 +64,8 @@ use task_executor::TaskExecutor;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, Hash256, ProposerSlashing, SignedAggregateAndProof,
SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit, SubnetId,
SyncCommitteeMessage, SyncSubnetId,
SignedBeaconBlock, SignedBlsToExecutionChange, SignedContributionAndProof, SignedVoluntaryExit,
SubnetId, SyncCommitteeMessage, SyncSubnetId,
};
use work_reprocessing_queue::{
spawn_reprocess_scheduler, QueuedAggregate, QueuedRpcBlock, QueuedUnaggregate, ReadyWork,
@ -163,6 +163,12 @@ const MAX_BLOBS_BY_RANGE_QUEUE_LEN: usize = 1_024;
/// will be stored before we start dropping them.
const MAX_BLOCKS_BY_ROOTS_QUEUE_LEN: usize = 1_024;
/// Maximum number of `SignedBlsToExecutionChange` messages to queue before dropping them.
///
/// This value is set high to accommodate the large spike that is expected immediately after Capella
/// is activated.
const MAX_BLS_TO_EXECUTION_CHANGE_QUEUE_LEN: usize = 16_384;
/// The name of the manager tokio task.
const MANAGER_TASK_NAME: &str = "beacon_processor_manager";
@ -206,6 +212,7 @@ pub const BLOCKS_BY_ROOTS_REQUEST: &str = "blocks_by_roots_request";
pub const BLOBS_BY_RANGE_REQUEST: &str = "blobs_by_range_request";
pub const UNKNOWN_BLOCK_ATTESTATION: &str = "unknown_block_attestation";
pub const UNKNOWN_BLOCK_AGGREGATE: &str = "unknown_block_aggregate";
pub const GOSSIP_BLS_TO_EXECUTION_CHANGE: &str = "gossip_bls_to_execution_change";
/// A simple first-in-first-out queue with a maximum length.
struct FifoQueue<T> {
@ -515,6 +522,22 @@ impl<T: BeaconChainTypes> WorkEvent<T> {
}
}
/// Create a new `Work` event for some BLS to execution change.
pub fn gossip_bls_to_execution_change(
message_id: MessageId,
peer_id: PeerId,
bls_to_execution_change: Box<SignedBlsToExecutionChange>,
) -> Self {
Self {
drop_during_sync: false,
work: Work::GossipBlsToExecutionChange {
message_id,
peer_id,
bls_to_execution_change,
},
}
}
/// Create a new `Work` event for some block, where the result from computation (if any) is
/// sent to the other side of `result_tx`.
pub fn rpc_beacon_block(
@ -789,6 +812,11 @@ pub enum Work<T: BeaconChainTypes> {
request_id: PeerRequestId,
request: BlobsByRangeRequest,
},
GossipBlsToExecutionChange {
message_id: MessageId,
peer_id: PeerId,
bls_to_execution_change: Box<SignedBlsToExecutionChange>,
},
}
impl<T: BeaconChainTypes> Work<T> {
@ -815,6 +843,7 @@ impl<T: BeaconChainTypes> Work<T> {
Work::BlobsByRangeRequest { .. } => BLOBS_BY_RANGE_REQUEST,
Work::UnknownBlockAttestation { .. } => UNKNOWN_BLOCK_ATTESTATION,
Work::UnknownBlockAggregate { .. } => UNKNOWN_BLOCK_AGGREGATE,
Work::GossipBlsToExecutionChange { .. } => GOSSIP_BLS_TO_EXECUTION_CHANGE,
}
}
}
@ -960,6 +989,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
let mut bbroots_queue = FifoQueue::new(MAX_BLOCKS_BY_ROOTS_QUEUE_LEN);
let mut blbrange_queue = FifoQueue::new(MAX_BLOBS_BY_RANGE_QUEUE_LEN);
let mut gossip_bls_to_execution_change_queue =
FifoQueue::new(MAX_BLS_TO_EXECUTION_CHANGE_QUEUE_LEN);
// Channels for sending work to the re-process scheduler (`work_reprocessing_tx`) and to
// receive them back once they are ready (`ready_work_rx`).
let (ready_work_tx, ready_work_rx) = mpsc::channel(MAX_SCHEDULED_WORK_QUEUE_LEN);
@ -1194,9 +1226,12 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
self.spawn_worker(item, toolbox);
} else if let Some(item) = gossip_proposer_slashing_queue.pop() {
self.spawn_worker(item, toolbox);
// Check exits last since our validators don't get rewards from them.
// Check exits and address changes late since our validators don't get
// rewards from them.
} else if let Some(item) = gossip_voluntary_exit_queue.pop() {
self.spawn_worker(item, toolbox);
} else if let Some(item) = gossip_bls_to_execution_change_queue.pop() {
self.spawn_worker(item, toolbox);
// Handle backfill sync chain segments.
} else if let Some(item) = backfill_chain_segment.pop() {
self.spawn_worker(item, toolbox);
@ -1313,6 +1348,9 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
Work::UnknownBlockAggregate { .. } => {
unknown_block_aggregate_queue.push(work)
}
Work::GossipBlsToExecutionChange { .. } => {
gossip_bls_to_execution_change_queue.push(work, work_id, &self.log)
}
}
}
}
@ -1365,6 +1403,10 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
&metrics::BEACON_PROCESSOR_ATTESTER_SLASHING_QUEUE_TOTAL,
gossip_attester_slashing_queue.len() as i64,
);
metrics::set_gauge(
&metrics::BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL,
gossip_bls_to_execution_change_queue.len() as i64,
);
if aggregate_queue.is_full() && aggregate_debounce.elapsed() {
error!(
@ -1623,6 +1665,20 @@ impl<T: BeaconChainTypes> BeaconProcessor<T> {
seen_timestamp,
)
}),
/*
* BLS to execution change verification.
*/
Work::GossipBlsToExecutionChange {
message_id,
peer_id,
bls_to_execution_change,
} => task_spawner.spawn_blocking(move || {
worker.process_gossip_bls_to_execution_change(
message_id,
peer_id,
*bls_to_execution_change,
)
}),
/*
* Verification for beacon blocks received during syncing via RPC.
*/

View File

@ -23,8 +23,9 @@ use store::hot_cold_store::HotColdDBError;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, Hash256, IndexedAttestation,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof,
SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage, SyncSubnetId,
ProposerSlashing, SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange,
SignedContributionAndProof, SignedVoluntaryExit, Slot, SubnetId, SyncCommitteeMessage,
SyncSubnetId,
};
use super::{
@ -1192,6 +1193,65 @@ impl<T: BeaconChainTypes> Worker<T> {
metrics::inc_counter(&metrics::BEACON_PROCESSOR_ATTESTER_SLASHING_IMPORTED_TOTAL);
}
pub fn process_gossip_bls_to_execution_change(
self,
message_id: MessageId,
peer_id: PeerId,
bls_to_execution_change: SignedBlsToExecutionChange,
) {
let validator_index = bls_to_execution_change.message.validator_index;
let address = bls_to_execution_change.message.to_execution_address;
let change = match self
.chain
.verify_bls_to_execution_change_for_gossip(bls_to_execution_change)
{
Ok(ObservationOutcome::New(change)) => change,
Ok(ObservationOutcome::AlreadyKnown) => {
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Ignore);
debug!(
self.log,
"Dropping BLS to execution change";
"validator_index" => validator_index,
"peer" => %peer_id
);
return;
}
Err(e) => {
debug!(
self.log,
"Dropping invalid BLS to execution change";
"validator_index" => validator_index,
"peer" => %peer_id,
"error" => ?e
);
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Reject);
// We penalize the peer slightly to prevent overuse of invalids.
self.gossip_penalize_peer(
peer_id,
PeerAction::HighToleranceError,
"invalid_bls_to_execution_change",
);
return;
}
};
metrics::inc_counter(&metrics::BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_VERIFIED_TOTAL);
self.propagate_validation_result(message_id, peer_id, MessageAcceptance::Accept);
self.chain.import_bls_to_execution_change(change);
debug!(
self.log,
"Successfully imported BLS to execution change";
"validator_index" => validator_index,
"address" => ?address,
);
metrics::inc_counter(&metrics::BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_IMPORTED_TOTAL);
}
/// Process the sync committee signature received from the gossip network and:
///
/// - If it passes gossip propagation criteria, tell the network thread to forward it.

View File

@ -143,6 +143,19 @@ lazy_static! {
"beacon_processor_attester_slashing_imported_total",
"Total number of attester slashings imported to the op pool."
);
// Gossip BLS to execution changes.
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_bls_to_execution_change_queue_total",
"Count of address changes from gossip waiting to be verified."
);
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_VERIFIED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_bls_to_execution_change_verified_total",
"Total number of address changes verified for propagation."
);
pub static ref BEACON_PROCESSOR_BLS_TO_EXECUTION_CHANGE_IMPORTED_TOTAL: Result<IntCounter> = try_create_int_counter(
"beacon_processor_bls_to_execution_change_imported_total",
"Total number of address changes imported to the op pool."
);
// Rpc blocks.
pub static ref BEACON_PROCESSOR_RPC_BLOCK_QUEUE_TOTAL: Result<IntGauge> = try_create_int_gauge(
"beacon_processor_rpc_block_queue_total",

View File

@ -291,6 +291,18 @@ impl<T: BeaconChainTypes> Router<T> {
sync_committtee_msg.0,
);
}
PubsubMessage::BlsToExecutionChange(bls_to_execution_change) => {
trace!(
self.log,
"Received BLS to execution change";
"peer_id" => %peer_id
);
self.processor.on_bls_to_execution_change_gossip(
id,
peer_id,
bls_to_execution_change,
);
}
}
}
}

View File

@ -19,8 +19,8 @@ use store::SyncCommitteeMessage;
use tokio::sync::mpsc;
use types::{
Attestation, AttesterSlashing, BlobsSidecar, EthSpec, ProposerSlashing,
SignedAggregateAndProof, SignedBeaconBlock, SignedContributionAndProof, SignedVoluntaryExit,
SubnetId, SyncSubnetId,
SignedAggregateAndProof, SignedBeaconBlock, SignedBlsToExecutionChange,
SignedContributionAndProof, SignedVoluntaryExit, SubnetId, SyncSubnetId,
};
/// Processes validated messages from the network. It relays necessary data to the syncing thread
@ -411,6 +411,19 @@ impl<T: BeaconChainTypes> Processor<T> {
))
}
pub fn on_bls_to_execution_change_gossip(
&mut self,
message_id: MessageId,
peer_id: PeerId,
bls_to_execution_change: Box<SignedBlsToExecutionChange>,
) {
self.send_beacon_processor_work(BeaconWorkEvent::gossip_bls_to_execution_change(
message_id,
peer_id,
bls_to_execution_change,
))
}
fn send_beacon_processor_work(&mut self, work: BeaconWorkEvent<T>) {
self.beacon_processor_send
.try_send(work)

View File

@ -4,6 +4,9 @@ version = "0.2.0"
authors = ["Michael Sproul <michael@sigmaprime.io>"]
edition = "2021"
[features]
withdrawals-processing = []
[dependencies]
derivative = "2.1.1"
itertools = "0.10.0"

View File

@ -30,10 +30,10 @@ use std::collections::{hash_map::Entry, HashMap, HashSet};
use std::marker::PhantomData;
use std::ptr;
use types::{
sync_aggregate::Error as SyncAggregateError, typenum::Unsigned, Attestation, AttestationData,
AttesterSlashing, BeaconState, BeaconStateError, ChainSpec, Epoch, EthSpec, ProposerSlashing,
SignedBlsToExecutionChange, SignedVoluntaryExit, Slot, SyncAggregate,
SyncCommitteeContribution, Validator,
sync_aggregate::Error as SyncAggregateError, typenum::Unsigned, AbstractExecPayload,
Attestation, AttestationData, AttesterSlashing, BeaconState, BeaconStateError, ChainSpec,
Epoch, EthSpec, ProposerSlashing, SignedBeaconBlock, SignedBlsToExecutionChange,
SignedVoluntaryExit, Slot, SyncAggregate, SyncCommitteeContribution, Validator,
};
type SyncContributions<T> = RwLock<HashMap<SyncAggregateId, Vec<SyncCommitteeContribution<T>>>>;
@ -51,6 +51,7 @@ pub struct OperationPool<T: EthSpec + Default> {
/// Map from exiting validator to their exit data.
voluntary_exits: RwLock<HashMap<u64, SigVerifiedOp<SignedVoluntaryExit, T>>>,
/// Map from credential changing validator to their execution change data.
#[cfg(feature = "withdrawals-processing")]
bls_to_execution_changes: RwLock<HashMap<u64, SigVerifiedOp<SignedBlsToExecutionChange, T>>>,
/// Reward cache for accelerating attestation packing.
reward_cache: RwLock<RewardCache>,
@ -432,7 +433,7 @@ impl<T: EthSpec> OperationPool<T> {
pub fn prune_proposer_slashings(&self, head_state: &BeaconState<T>) {
prune_validator_hash_map(
&mut self.proposer_slashings.write(),
|validator| validator.exit_epoch <= head_state.finalized_checkpoint().epoch,
|_, validator| validator.exit_epoch <= head_state.finalized_checkpoint().epoch,
head_state,
);
}
@ -507,28 +508,115 @@ impl<T: EthSpec> OperationPool<T> {
//
// We choose simplicity over the gain of pruning more exits since they are small and
// should not be seen frequently.
|validator| validator.exit_epoch <= head_state.finalized_checkpoint().epoch,
|_, validator| validator.exit_epoch <= head_state.finalized_checkpoint().epoch,
head_state,
);
}
/// Insert a BLS to execution change into the pool.
pub fn insert_bls_to_execution_change(
&self,
verified_change: SigVerifiedOp<SignedBlsToExecutionChange, T>,
) {
#[cfg(feature = "withdrawals-processing")]
{
self.bls_to_execution_changes.write().insert(
verified_change.as_inner().message.validator_index,
verified_change,
);
}
#[cfg(not(feature = "withdrawals-processing"))]
{
drop(verified_change);
}
}
/// Get a list of execution changes for inclusion in a block.
///
/// They're in random `HashMap` order, which isn't exactly fair, but isn't unfair either.
pub fn get_bls_to_execution_changes(
&self,
state: &BeaconState<T>,
spec: &ChainSpec,
) -> Vec<SignedBlsToExecutionChange> {
// FIXME: actually implement this
return vec![];
#[cfg(feature = "withdrawals-processing")]
{
filter_limit_operations(
self.bls_to_execution_changes.read().values(),
|address_change| {
address_change.signature_is_still_valid(&state.fork())
&& state
.get_validator(
address_change.as_inner().message.validator_index as usize,
)
.map_or(false, |validator| {
!validator.has_eth1_withdrawal_credential(spec)
})
},
|address_change| address_change.as_inner().clone(),
T::MaxBlsToExecutionChanges::to_usize(),
)
}
#[cfg(not(feature = "withdrawals-processing"))]
{
drop((state, spec));
vec![]
}
}
/// Prune BLS to execution changes that have been applied to the state more than 1 block ago.
///
/// The block check is necessary to avoid pruning too eagerly and losing the ability to include
/// address changes during re-orgs. This is isn't *perfect* so some address changes could
/// still get stuck if there are gnarly re-orgs and the changes can't be widely republished
/// due to the gossip duplicate rules.
pub fn prune_bls_to_execution_changes<Payload: AbstractExecPayload<T>>(
&self,
head_block: &SignedBeaconBlock<T, Payload>,
head_state: &BeaconState<T>,
spec: &ChainSpec,
) {
#[cfg(feature = "withdrawals-processing")]
{
prune_validator_hash_map(
&mut self.bls_to_execution_changes.write(),
|validator_index, validator| {
validator.has_eth1_withdrawal_credential(spec)
&& head_block
.message()
.body()
.bls_to_execution_changes()
.map_or(true, |recent_changes| {
!recent_changes
.iter()
.any(|c| c.message.validator_index == validator_index)
})
},
head_state,
);
}
#[cfg(not(feature = "withdrawals-processing"))]
{
drop((head_block, head_state, spec));
}
}
/// Prune all types of transactions given the latest head state and head fork.
pub fn prune_all(&self, head_state: &BeaconState<T>, current_epoch: Epoch) {
pub fn prune_all<Payload: AbstractExecPayload<T>>(
&self,
head_block: &SignedBeaconBlock<T, Payload>,
head_state: &BeaconState<T>,
current_epoch: Epoch,
spec: &ChainSpec,
) {
self.prune_attestations(current_epoch);
self.prune_sync_contributions(head_state.slot());
self.prune_proposer_slashings(head_state);
self.prune_attester_slashings(head_state);
self.prune_voluntary_exits(head_state);
self.prune_bls_to_execution_changes(head_block, head_state, spec);
}
/// Total number of voluntary exits in the pool.
@ -594,6 +682,23 @@ impl<T: EthSpec> OperationPool<T> {
.map(|(_, exit)| exit.as_inner().clone())
.collect()
}
/// Returns all known `SignedBlsToExecutionChange` objects.
///
/// This method may return objects that are invalid for block inclusion.
pub fn get_all_bls_to_execution_changes(&self) -> Vec<SignedBlsToExecutionChange> {
#[cfg(feature = "withdrawals-processing")]
{
self.bls_to_execution_changes
.read()
.iter()
.map(|(_, address_change)| address_change.as_inner().clone())
.collect()
}
#[cfg(not(feature = "withdrawals-processing"))]
vec![]
}
}
/// Filter up to a maximum number of operations out of an iterator.
@ -627,7 +732,7 @@ fn prune_validator_hash_map<T, F, E: EthSpec>(
prune_if: F,
head_state: &BeaconState<E>,
) where
F: Fn(&Validator) -> bool,
F: Fn(u64, &Validator) -> bool,
T: VerifyOperation<E>,
{
map.retain(|&validator_index, op| {
@ -635,7 +740,7 @@ fn prune_validator_hash_map<T, F, E: EthSpec>(
&& head_state
.validators()
.get(validator_index as usize)
.map_or(true, |validator| !prune_if(validator))
.map_or(true, |validator| !prune_if(validator_index, validator))
});
}

View File

@ -142,7 +142,8 @@ impl<T: EthSpec> PersistedOperationPool<T> {
attester_slashings,
proposer_slashings,
voluntary_exits,
// FIXME: IMPLEMENT THIS
// FIXME(capella): implement schema migration for address changes in op pool
#[cfg(feature = "withdrawals-processing")]
bls_to_execution_changes: Default::default(),
reward_cache: Default::default(),
_phantom: Default::default(),

View File

@ -158,6 +158,8 @@ where
// Deposits are not included because they can legally have invalid signatures.
self.include_exits(block)?;
self.include_sync_aggregate(block)?;
#[cfg(feature = "withdrawals")]
self.include_bls_to_execution_changes(block)?;
Ok(())
}
@ -339,6 +341,26 @@ where
Ok(())
}
/// Include the signature of the block's BLS to execution changes for verification.
#[cfg(feature = "withdrawals")]
pub fn include_bls_to_execution_changes<Payload: AbstractExecPayload<T>>(
&mut self,
block: &'a SignedBeaconBlock<T, Payload>,
) -> Result<()> {
// FIXME(capella): to improve performance we might want to decompress the withdrawal pubkeys
// in parallel.
if let Ok(bls_to_execution_changes) = block.message().body().bls_to_execution_changes() {
for bls_to_execution_change in bls_to_execution_changes {
self.sets.push(bls_execution_change_signature_set(
self.state,
bls_to_execution_change,
self.spec,
)?);
}
}
Ok(())
}
/// Verify all the signatures that have been included in `self`, returning `true` if and only if
/// all the signatures are valid.
///

View File

@ -15,6 +15,14 @@ use types::{
SignedVoluntaryExit,
};
#[cfg(feature = "withdrawals-processing")]
use {
crate::per_block_processing::{
errors::BlsExecutionChangeValidationError, verify_bls_to_execution_change,
},
types::SignedBlsToExecutionChange,
};
const MAX_FORKS_VERIFIED_AGAINST: usize = 2;
/// Wrapper around an operation type that acts as proof that its signature has been checked.
@ -65,7 +73,7 @@ where
fn new(op: T, state: &BeaconState<E>) -> Self {
let verified_against = VerifiedAgainst {
fork_versions: op
.verification_epochs()
.verification_epochs(state.current_epoch())
.into_iter()
.map(|epoch| state.fork().get_fork_version(epoch))
.collect(),
@ -87,8 +95,13 @@ where
}
pub fn signature_is_still_valid(&self, current_fork: &Fork) -> bool {
// Pass the fork's epoch as the effective current epoch. If the message is a current-epoch
// style message like `SignedBlsToExecutionChange` then `get_fork_version` will return the
// current fork version and we'll check it matches the fork version the message was checked
// against.
let effective_current_epoch = current_fork.epoch;
self.as_inner()
.verification_epochs()
.verification_epochs(effective_current_epoch)
.into_iter()
.zip(self.verified_against.fork_versions.iter())
.all(|(epoch, verified_fork_version)| {
@ -118,7 +131,13 @@ pub trait VerifyOperation<E: EthSpec>: Encode + Decode + Sized {
/// Return the epochs at which parts of this message were verified.
///
/// These need to map 1-to-1 to the `SigVerifiedOp::verified_against` for this type.
fn verification_epochs(&self) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]>;
///
/// If the message contains no inherent epoch it should return the `current_epoch` that is
/// passed in, as that's the epoch at which it was verified.
fn verification_epochs(
&self,
current_epoch: Epoch,
) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]>;
}
impl<E: EthSpec> VerifyOperation<E> for SignedVoluntaryExit {
@ -134,7 +153,7 @@ impl<E: EthSpec> VerifyOperation<E> for SignedVoluntaryExit {
}
#[allow(clippy::integer_arithmetic)]
fn verification_epochs(&self) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
fn verification_epochs(&self, _: Epoch) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
smallvec![self.message.epoch]
}
}
@ -152,7 +171,7 @@ impl<E: EthSpec> VerifyOperation<E> for AttesterSlashing<E> {
}
#[allow(clippy::integer_arithmetic)]
fn verification_epochs(&self) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
fn verification_epochs(&self, _: Epoch) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
smallvec![
self.attestation_1.data.target.epoch,
self.attestation_2.data.target.epoch
@ -173,7 +192,7 @@ impl<E: EthSpec> VerifyOperation<E> for ProposerSlashing {
}
#[allow(clippy::integer_arithmetic)]
fn verification_epochs(&self) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
fn verification_epochs(&self, _: Epoch) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
// Only need a single epoch because the slots of the two headers must be equal.
smallvec![self
.signed_header_1
@ -182,3 +201,25 @@ impl<E: EthSpec> VerifyOperation<E> for ProposerSlashing {
.epoch(E::slots_per_epoch())]
}
}
#[cfg(feature = "withdrawals-processing")]
impl<E: EthSpec> VerifyOperation<E> for SignedBlsToExecutionChange {
type Error = BlsExecutionChangeValidationError;
fn validate(
self,
state: &BeaconState<E>,
spec: &ChainSpec,
) -> Result<SigVerifiedOp<Self, E>, Self::Error> {
verify_bls_to_execution_change(state, &self, VerifySignatures::True, spec)?;
Ok(SigVerifiedOp::new(self, state))
}
#[allow(clippy::integer_arithmetic)]
fn verification_epochs(
&self,
current_epoch: Epoch,
) -> SmallVec<[Epoch; MAX_FORKS_VERIFIED_AGAINST]> {
smallvec![current_epoch]
}
}

View File

@ -7,8 +7,6 @@ use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// A deposit to potentially become a beacon chain validator.
///
/// Spec v0.12.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Debug, PartialEq, Hash, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,

View File

@ -7,8 +7,6 @@ use test_random_derive::TestRandom;
use tree_hash_derive::TreeHash;
/// A deposit to potentially become a beacon chain validator.
///
/// Spec v0.12.1
#[cfg_attr(feature = "arbitrary-fuzz", derive(arbitrary::Arbitrary))]
#[derive(
Debug, PartialEq, Hash, Clone, Serialize, Deserialize, Encode, Decode, TreeHash, TestRandom,

View File

@ -1,4 +1,4 @@
TESTS_TAG := f5c7cf78
TESTS_TAG := v1.3.0-alpha.1
TESTS = general minimal mainnet
TARBALLS = $(patsubst %,%-$(TESTS_TAG).tar.gz,$(TESTS))