2021-01-11 21:27:01 +00:00
# beacon_chain
2022-01-18 13:36:52 +00:00
# Copyright (c) 2018-2022 Status Research & Development GmbH
2021-01-11 21:27:01 +00:00
# Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{. push raises : [ Defect ] . }
2022-03-14 13:05:38 +00:00
# References to `vFuture` refer to the pre-release proposal of the libp2p based
# light client sync protocol. Conflicting release versions are not in use.
# https://github.com/ethereum/consensus-specs/pull/2802
2020-08-20 16:30:47 +00:00
import
2021-03-12 09:46:26 +00:00
std / tables ,
2021-12-06 09:49:01 +00:00
stew / results , bearssl ,
chronicles , chronos , metrics , taskpools ,
2021-08-12 13:08:20 +00:00
.. / spec / [ helpers , forks ] ,
2022-01-18 13:36:52 +00:00
.. / spec / datatypes / [ altair , phase0 ] ,
2021-10-19 14:09:26 +00:00
.. / consensus_object_pools / [
2021-12-06 09:49:01 +00:00
block_clearance , block_quarantine , blockchain_dag , exit_pool , attestation_pool ,
2021-10-19 14:09:26 +00:00
sync_committee_msg_pool ] ,
2021-03-05 13:12:00 +00:00
.. / validators / validator_pool ,
2021-10-19 14:09:26 +00:00
.. / beacon_clock ,
" . " / [ gossip_validation , block_processor , batch_validation ]
2021-11-05 15:39:47 +00:00
export
2021-12-06 09:49:01 +00:00
results , bearssl , taskpools , block_clearance , blockchain_dag , exit_pool , attestation_pool ,
2021-11-05 15:39:47 +00:00
sync_committee_msg_pool , validator_pool , beacon_clock , gossip_validation ,
2021-12-06 09:49:01 +00:00
block_processor , batch_validation , block_quarantine
2020-08-20 16:30:47 +00:00
# Metrics for tracking attestation and beacon block loss
declareCounter beacon_attestations_received ,
2021-10-19 15:20:55 +00:00
" Number of valid unaggregated attestations processed by this node "
declareCounter beacon_attestations_dropped ,
" Number of invalid unaggregated attestations dropped by this node " , labels = [ " reason " ]
2020-08-20 16:30:47 +00:00
declareCounter beacon_aggregates_received ,
2021-10-19 15:20:55 +00:00
" Number of valid aggregated attestations processed by this node "
declareCounter beacon_aggregates_dropped ,
" Number of invalid aggregated attestations dropped by this node " , labels = [ " reason " ]
2020-08-20 16:30:47 +00:00
declareCounter beacon_blocks_received ,
2021-10-19 15:20:55 +00:00
" Number of valid blocks processed by this node "
declareCounter beacon_blocks_dropped ,
" Number of invalid blocks dropped by this node " , labels = [ " reason " ]
2020-09-14 14:26:31 +00:00
declareCounter beacon_attester_slashings_received ,
2021-10-19 15:20:55 +00:00
" Number of valid attester slashings processed by this node "
declareCounter beacon_attester_slashings_dropped ,
" Number of invalid attester slashings dropped by this node " , labels = [ " reason " ]
2020-09-14 14:26:31 +00:00
declareCounter beacon_proposer_slashings_received ,
2021-10-19 15:20:55 +00:00
" Number of valid proposer slashings processed by this node "
declareCounter beacon_proposer_slashings_dropped ,
" Number of invalid proposer slashings dropped by this node " , labels = [ " reason " ]
2020-09-14 14:26:31 +00:00
declareCounter beacon_voluntary_exits_received ,
2021-10-19 15:20:55 +00:00
" Number of valid voluntary exits processed by this node "
declareCounter beacon_voluntary_exits_dropped ,
" Number of invalid voluntary exits dropped by this node " , labels = [ " reason " ]
declareCounter beacon_sync_committee_messages_received ,
" Number of valid sync committee messages processed by this node "
declareCounter beacon_sync_committee_messages_dropped ,
" Number of invalid sync committee messages dropped by this node " , labels = [ " reason " ]
declareCounter beacon_sync_committee_contributions_received ,
" Number of valid sync committee contributions processed by this node "
declareCounter beacon_sync_committee_contributions_dropped ,
" Number of invalid sync committee contributions dropped by this node " , labels = [ " reason " ]
2022-03-14 13:05:38 +00:00
declareCounter beacon_optimistic_light_client_updates_received ,
" Number of valid optimistic light client updates processed by this node "
declareCounter beacon_optimistic_light_client_updates_dropped ,
" Number of invalid optimistic light client updates dropped by this node " , labels = [ " reason " ]
2020-08-20 16:30:47 +00:00
const delayBuckets = [ 2 .0 , 4 .0 , 6 .0 , 8 .0 , 10 .0 , 12 .0 , 14 .0 , Inf ]
declareHistogram beacon_attestation_delay ,
" Time(s) between slot start and attestation reception " , buckets = delayBuckets
declareHistogram beacon_aggregate_delay ,
" Time(s) between slot start and aggregate reception " , buckets = delayBuckets
declareHistogram beacon_block_delay ,
" Time(s) between slot start and beacon block reception " , buckets = delayBuckets
type
2021-06-17 11:51:04 +00:00
DoppelgangerProtection = object
broadcastStartEpoch * : Epoch ##\
## Set anew, each time gossip is re-enabled after syncing completes, so
## might reset multiple times per instance. This allows some safe level
## of gossip interleaving between nodes so long as they don't gossip at
## the same time.
2022-02-07 05:12:36 +00:00
nodeLaunchSlot * : Slot ##\
2021-06-17 11:51:04 +00:00
## Set once, at node launch. This functions as a basic protection against
## false positives from attestations persisting within the gossip network
## across quick restarts.
2020-08-20 16:30:47 +00:00
Eth2Processor * = object
2021-10-19 15:20:55 +00:00
## The Eth2Processor is the entry point for untrusted message processing -
## when we receive messages from various sources, we pass them to the
## processor for validation and routing - the messages are generally
## validated, and if valid, passed on to the various pools, monitors and
## managers to update the state of the application.
##
## Block processing is special in that part of it is done in the
## `BlockProcessor` instead, via a special block processing queue.
##
## Each validating function generally will do a sanity check on the message
## whose purpose is to quickly filter out spam, then will (usually) delegate
## full validation to the proper manager - finally, metrics and monitoring
## are updated.
2022-02-14 19:24:38 +00:00
doppelgangerDetectionEnabled * : bool
2021-03-11 10:10:57 +00:00
# Local sources of truth for validation
# ----------------------------------------------------------------
2021-06-01 11:13:40 +00:00
dag * : ChainDAGRef
2020-08-20 16:30:47 +00:00
attestationPool * : ref AttestationPool
2020-10-27 17:21:35 +00:00
validatorPool : ref ValidatorPool
2021-10-19 14:09:26 +00:00
syncCommitteeMsgPool : ref SyncCommitteeMsgPool
2020-08-20 16:30:47 +00:00
2021-02-01 11:18:16 +00:00
doppelgangerDetection * : DoppelgangerProtection
2020-10-27 17:21:35 +00:00
2021-03-11 10:10:57 +00:00
# Gossip validated -> enqueue for further verification
# ----------------------------------------------------------------
2021-05-28 16:34:00 +00:00
blockProcessor : ref BlockProcessor
2020-08-20 16:30:47 +00:00
2021-12-20 19:20:31 +00:00
# Validator monitoring
validatorMonitor : ref ValidatorMonitor
2021-03-11 10:10:57 +00:00
# Validated with no further verification required
# ----------------------------------------------------------------
exitPool : ref ExitPool
2020-08-20 16:30:47 +00:00
2021-04-02 14:36:43 +00:00
# Almost validated, pending cryptographic signature check
# ----------------------------------------------------------------
2021-04-03 01:50:47 +00:00
batchCrypto * : ref BatchCrypto
2021-04-02 14:36:43 +00:00
2021-03-11 10:10:57 +00:00
# Missing information
# ----------------------------------------------------------------
2021-12-06 09:49:01 +00:00
quarantine * : ref Quarantine
2020-08-20 16:30:47 +00:00
2021-07-07 09:09:47 +00:00
# Application-provided current time provider (to facilitate testing)
2021-08-19 10:45:31 +00:00
getCurrentBeaconTime * : GetBeaconTimeFn
2021-07-07 09:09:47 +00:00
2021-11-05 15:39:47 +00:00
ValidationRes * = Result [ void , ValidationError ]
2021-03-11 10:10:57 +00:00
# Initialization
# ------------------------------------------------------------------------------
2020-08-20 16:30:47 +00:00
2021-03-11 10:10:57 +00:00
proc new * ( T : type Eth2Processor ,
2022-02-14 19:24:38 +00:00
doppelgangerDetectionEnabled : bool ,
2021-05-28 16:34:00 +00:00
blockProcessor : ref BlockProcessor ,
2021-12-20 19:20:31 +00:00
validatorMonitor : ref ValidatorMonitor ,
2021-06-01 11:13:40 +00:00
dag : ChainDAGRef ,
2021-03-11 10:10:57 +00:00
attestationPool : ref AttestationPool ,
exitPool : ref ExitPool ,
validatorPool : ref ValidatorPool ,
2021-10-19 14:09:26 +00:00
syncCommitteeMsgPool : ref SyncCommitteeMsgPool ,
2021-12-06 09:49:01 +00:00
quarantine : ref Quarantine ,
2021-04-02 14:36:43 +00:00
rng : ref BrHmacDrbgContext ,
2021-09-17 00:13:52 +00:00
getBeaconTime : GetBeaconTimeFn ,
2021-12-06 09:49:01 +00:00
taskpool : TaskPoolPtr
2021-09-17 00:13:52 +00:00
) : ref Eth2Processor =
2021-03-11 10:10:57 +00:00
( ref Eth2Processor ) (
2022-02-14 19:24:38 +00:00
doppelgangerDetectionEnabled : doppelgangerDetectionEnabled ,
2021-06-17 11:51:04 +00:00
doppelgangerDetection : DoppelgangerProtection (
2022-01-03 21:18:49 +00:00
nodeLaunchSlot : getBeaconTime ( ) . slotOrZero ,
broadcastStartEpoch : FAR_FUTURE_EPOCH ) ,
2021-05-28 16:34:00 +00:00
blockProcessor : blockProcessor ,
2021-12-20 19:20:31 +00:00
validatorMonitor : validatorMonitor ,
2021-06-01 11:13:40 +00:00
dag : dag ,
2021-03-11 10:10:57 +00:00
attestationPool : attestationPool ,
exitPool : exitPool ,
validatorPool : validatorPool ,
2021-08-28 22:27:51 +00:00
syncCommitteeMsgPool : syncCommitteeMsgPool ,
2021-04-02 14:36:43 +00:00
quarantine : quarantine ,
2021-08-19 10:45:31 +00:00
getCurrentBeaconTime : getBeaconTime ,
2021-04-26 20:39:44 +00:00
batchCrypto : BatchCrypto . new (
rng = rng ,
# Only run eager attestation signature verification if we're not
# processing blocks in order to give priority to block processing
2021-09-17 00:13:52 +00:00
eager = proc ( ) : bool = not blockProcessor [ ] . hasBlocks ( ) ,
taskpool )
2021-03-11 10:10:57 +00:00
)
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
# Each validator logs, validates then passes valid data to its destination
# further down the line - in particular, validation should generally not have
# any side effects until the message is fully validated, or invalid messages
# could be used to push out valid messages.
2021-01-11 21:27:01 +00:00
2020-08-20 16:30:47 +00:00
proc blockValidator * (
2021-12-20 19:20:31 +00:00
self : var Eth2Processor , src : MsgSource ,
signedBlock : ForkySignedBeaconBlock ) : ValidationRes =
2020-08-20 16:30:47 +00:00
let
2021-07-07 09:09:47 +00:00
wallTime = self . getCurrentBeaconTime ( )
2020-08-20 16:30:47 +00:00
( afterGenesis , wallSlot ) = wallTime . toSlot ( )
2021-11-05 15:39:47 +00:00
logScope :
blockRoot = shortLog ( signedBlock . root )
blck = shortLog ( signedBlock . message )
signature = shortLog ( signedBlock . signature )
wallSlot
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
if not afterGenesis :
notice " Block before genesis "
return errIgnore ( " Block before genesis " )
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
# Potential under/overflows are fine; would just create odd metrics and logs
2022-01-11 10:01:54 +00:00
let delay = wallTime - signedBlock . message . slot . start_beacon_time
2020-08-20 16:30:47 +00:00
# Start of block processing - in reality, we have already gone through SSZ
# decoding at this stage, which may be significant
debug " Block received " , delay
2021-11-05 15:39:47 +00:00
let v =
self . dag . validateBeaconBlock ( self . quarantine , signedBlock , wallTime , { } )
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
if v . isOk ( ) :
# Block passed validation - enqueue it for processing. The block processing
# queue is effectively unbounded as we use a freestanding task to enqueue
# the block - this is done so that when blocks arrive concurrently with
# sync, we don't lose the gossip blocks, but also don't block the gossip
# propagation of seemingly good blocks
trace " Block validated "
self . blockProcessor [ ] . addBlock (
2021-12-20 19:20:31 +00:00
src , ForkedSignedBeaconBlock . init ( signedBlock ) ,
2022-01-11 10:01:54 +00:00
validationDur = nanoseconds (
( self . getCurrentBeaconTime ( ) - wallTime ) . nanoseconds ) )
2020-08-27 07:34:12 +00:00
2021-11-05 15:39:47 +00:00
# Validator monitor registration for blocks is done by the processor
beacon_blocks_received . inc ( )
beacon_block_delay . observe ( delay . toFloatSeconds ( ) )
else :
debug " Dropping block " , error = v . error ( )
2021-10-19 15:20:55 +00:00
2021-11-05 15:39:47 +00:00
self . blockProcessor [ ] . dumpInvalidBlock ( signedBlock )
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
beacon_blocks_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2021-10-19 15:20:55 +00:00
2021-11-05 15:39:47 +00:00
v
2020-08-20 16:30:47 +00:00
2022-01-03 21:18:49 +00:00
proc setupDoppelgangerDetection * ( self : var Eth2Processor , slot : Slot ) =
# When another client's already running, this is very likely to detect
# potential duplicate validators, which can trigger slashing.
#
# Every missed attestation costs approximately 3*get_base_reward(), which
# can be up to around 10,000 Wei. Thus, skipping attestations isn't cheap
# and one should gauge the likelihood of this simultaneous launch to tune
# the epoch delay to one's perceived risk.
if self . validatorPool [ ] . count ( ) > 0 :
const duplicateValidatorEpochs = 2
self . doppelgangerDetection . broadcastStartEpoch =
slot . epoch + duplicateValidatorEpochs
2022-02-14 19:24:38 +00:00
if self . doppelgangerDetectionEnabled :
notice " Setting up doppelganger detection " ,
epoch = slot . epoch ,
broadcastStartEpoch = self . doppelgangerDetection . broadcastStartEpoch
2022-01-03 21:18:49 +00:00
2021-01-29 12:38:52 +00:00
proc checkForPotentialDoppelganger (
2021-05-28 12:51:15 +00:00
self : var Eth2Processor , attestation : Attestation ,
2021-06-17 11:51:04 +00:00
attesterIndices : openArray [ ValidatorIndex ] ) =
# Only check for attestations after node launch. There might be one slot of
# overlap in quick intra-slot restarts so trade off a few true negatives in
# the service of avoiding more likely false positives.
limit by-root requests to non-finalized blocks (#3293)
* limit by-root requests to non-finalized blocks
Presently, we keep a mapping from block root to `BlockRef` in memory -
this has simplified reasoning about the dag, but is not sustainable with
the chain growing.
We can distinguish between two cases where by-root access is useful:
* unfinalized blocks - this is where the beacon chain is operating
generally, by validating incoming data as interesting for future fork
choice decisions - bounded by the length of the unfinalized period
* finalized blocks - historical access in the REST API etc - no bounds,
really
In this PR, we limit the by-root block index to the first use case:
finalized chain data can more efficiently be addressed by slot number.
Future work includes:
* limiting the `BlockRef` horizon in general - each instance is 40
bytes+overhead which adds up - this needs further refactoring to deal
with the tail vs state problem
* persisting the finalized slot-to-hash index - this one also keeps
growing unbounded (albeit slowly)
Anyway, this PR easily shaves ~128mb of memory usage at the time of
writing.
* No longer honor `BeaconBlocksByRoot` requests outside of the
non-finalized period - previously, Nimbus would generously return any
block through this libp2p request - per the spec, finalized blocks
should be fetched via `BeaconBlocksByRange` instead.
* return `Opt[BlockRef]` instead of `nil` when blocks can't be found -
this becomes a lot more common now and thus deserves more attention
* `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized
blocks from now - `finalizedBlocks` covers the other `BlockRef`
instances
* in backfill, verify that the last backfilled block leads back to
genesis, or panic
* add backfill timings to log
* fix missing check that `BlockRef` block can be fetched with
`getForkedBlock` reliably
* shortcut doppelganger check when feature is not enabled
* in REST/JSON-RPC, fetch blocks without involving `BlockRef`
* fix dag.blocks ref
2022-01-21 11:33:16 +00:00
if not self . doppelgangerDetectionEnabled :
return
2021-06-17 11:51:04 +00:00
if attestation . data . slot < = self . doppelgangerDetection . nodeLaunchSlot + 1 :
2021-03-01 10:09:05 +00:00
return
2021-06-17 11:51:04 +00:00
if attestation . data . slot . epoch <
2022-02-07 05:12:36 +00:00
self . doppelgangerDetection . broadcastStartEpoch and
self . doppelgangerDetection . nodeLaunchSlot > GENESIS_SLOT :
2020-10-27 17:21:35 +00:00
for validatorIndex in attesterIndices :
limit by-root requests to non-finalized blocks (#3293)
* limit by-root requests to non-finalized blocks
Presently, we keep a mapping from block root to `BlockRef` in memory -
this has simplified reasoning about the dag, but is not sustainable with
the chain growing.
We can distinguish between two cases where by-root access is useful:
* unfinalized blocks - this is where the beacon chain is operating
generally, by validating incoming data as interesting for future fork
choice decisions - bounded by the length of the unfinalized period
* finalized blocks - historical access in the REST API etc - no bounds,
really
In this PR, we limit the by-root block index to the first use case:
finalized chain data can more efficiently be addressed by slot number.
Future work includes:
* limiting the `BlockRef` horizon in general - each instance is 40
bytes+overhead which adds up - this needs further refactoring to deal
with the tail vs state problem
* persisting the finalized slot-to-hash index - this one also keeps
growing unbounded (albeit slowly)
Anyway, this PR easily shaves ~128mb of memory usage at the time of
writing.
* No longer honor `BeaconBlocksByRoot` requests outside of the
non-finalized period - previously, Nimbus would generously return any
block through this libp2p request - per the spec, finalized blocks
should be fetched via `BeaconBlocksByRange` instead.
* return `Opt[BlockRef]` instead of `nil` when blocks can't be found -
this becomes a lot more common now and thus deserves more attention
* `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized
blocks from now - `finalizedBlocks` covers the other `BlockRef`
instances
* in backfill, verify that the last backfilled block leads back to
genesis, or panic
* add backfill timings to log
* fix missing check that `BlockRef` block can be fetched with
`getForkedBlock` reliably
* shortcut doppelganger check when feature is not enabled
* in REST/JSON-RPC, fetch blocks without involving `BlockRef`
* fix dag.blocks ref
2022-01-21 11:33:16 +00:00
let validatorPubkey = self . dag . validatorKey ( validatorIndex ) . get ( ) . toPubKey ( )
if not isNil ( self . validatorPool [ ] . getValidator ( validatorPubkey ) ) :
2021-04-26 19:41:35 +00:00
warn " We believe you are currently running another instance of the same validator. We ' ve disconnected you from the network as this presents a significant slashing risk. Possible next steps are (a) making sure you ' ve disconnected your validator from your old machine before restarting the client; and (b) running the client again with the gossip-slashing-protection option disabled, only if you are absolutely sure this is the only instance of your validator running, and reporting the issue at https://github.com/status-im/nimbus-eth2/issues. " ,
2020-10-27 17:21:35 +00:00
validatorIndex ,
2021-03-01 10:09:05 +00:00
validatorPubkey ,
2021-05-28 12:51:15 +00:00
attestation = shortLog ( attestation )
2021-04-26 19:41:35 +00:00
quit QuitFailure
2020-10-27 17:21:35 +00:00
2020-08-20 16:30:47 +00:00
proc attestationValidator * (
2021-12-20 19:20:31 +00:00
self : ref Eth2Processor , src : MsgSource ,
attestation : Attestation , subnet_id : SubnetId ,
2021-11-05 15:39:47 +00:00
checkSignature : bool = true ) : Future [ ValidationRes ] {. async . } =
2021-12-21 18:56:08 +00:00
var wallTime = self . getCurrentBeaconTime ( )
let ( afterGenesis , wallSlot ) = wallTime . toSlot ( )
2021-11-05 15:39:47 +00:00
2020-08-20 16:30:47 +00:00
logScope :
attestation = shortLog ( attestation )
2021-05-10 07:13:36 +00:00
subnet_id
2021-11-05 15:39:47 +00:00
wallSlot
2020-08-20 16:30:47 +00:00
if not afterGenesis :
notice " Attestation before genesis "
2021-11-05 15:39:47 +00:00
return errIgnore ( " Attestation before genesis " )
2020-08-20 16:30:47 +00:00
2020-09-02 16:16:25 +00:00
# Potential under/overflows are fine; would just create odd metrics and logs
2022-01-11 10:01:54 +00:00
let delay = wallTime - attestation . data . slot . start_beacon_time
2020-10-09 08:58:54 +00:00
debug " Attestation received " , delay
2021-04-02 14:36:43 +00:00
# Now proceed to validation
2021-11-05 15:39:47 +00:00
let v =
await self . attestationPool . validateAttestation (
2021-05-10 07:13:36 +00:00
self . batchCrypto , attestation , wallTime , subnet_id , checkSignature )
2021-11-05 15:39:47 +00:00
return if v . isOk ( ) :
2022-01-03 16:20:15 +00:00
# Due to async validation the wallTime here might have changed
2021-12-21 18:56:08 +00:00
wallTime = self . getCurrentBeaconTime ( )
2021-04-26 20:39:44 +00:00
2021-11-05 15:39:47 +00:00
let ( attester_index , sig ) = v . get ( )
2020-10-27 17:21:35 +00:00
2021-11-05 15:39:47 +00:00
self [ ] . checkForPotentialDoppelganger ( attestation , [ attester_index ] )
2021-10-19 15:20:55 +00:00
2021-11-05 15:39:47 +00:00
trace " Attestation validated "
self . attestationPool [ ] . addAttestation (
2021-12-21 18:56:08 +00:00
attestation , [ attester_index ] , sig , wallTime )
2020-08-20 16:30:47 +00:00
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerAttestation (
src , wallTime , attestation , attester_index )
2021-11-05 15:39:47 +00:00
beacon_attestations_received . inc ( )
beacon_attestation_delay . observe ( delay . toFloatSeconds ( ) )
2021-12-20 19:20:31 +00:00
2021-11-05 15:39:47 +00:00
ok ( )
else :
debug " Dropping attestation " , validationError = v . error
beacon_attestations_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
err ( v . error ( ) )
2020-08-20 16:30:47 +00:00
proc aggregateValidator * (
2021-12-20 19:20:31 +00:00
self : ref Eth2Processor , src : MsgSource ,
2021-11-05 15:39:47 +00:00
signedAggregateAndProof : SignedAggregateAndProof ) : Future [ ValidationRes ] {. async . } =
2022-01-03 16:20:15 +00:00
var wallTime = self . getCurrentBeaconTime ( )
let ( afterGenesis , wallSlot ) = wallTime . toSlot ( )
2021-11-05 15:39:47 +00:00
2020-08-20 16:30:47 +00:00
logScope :
aggregate = shortLog ( signedAggregateAndProof . message . aggregate )
2021-11-05 15:39:47 +00:00
aggregator_index = signedAggregateAndProof . message . aggregator_index
selection_proof = shortLog ( signedAggregateAndProof . message . selection_proof )
2020-08-20 16:30:47 +00:00
signature = shortLog ( signedAggregateAndProof . signature )
2021-11-05 15:39:47 +00:00
wallSlot
2020-08-20 16:30:47 +00:00
if not afterGenesis :
notice " Aggregate before genesis "
2021-11-05 15:39:47 +00:00
return errIgnore ( " Aggreagte before genesis " )
2020-08-20 16:30:47 +00:00
2020-09-02 16:16:25 +00:00
# Potential under/overflows are fine; would just create odd logs
2020-08-20 16:30:47 +00:00
let delay =
2022-01-11 10:01:54 +00:00
wallTime - signedAggregateAndProof . message . aggregate . data . slot . start_beacon_time
2020-08-20 16:30:47 +00:00
debug " Aggregate received " , delay
2021-11-05 15:39:47 +00:00
let v =
await self . attestationPool . validateAggregate (
self . batchCrypto , signedAggregateAndProof , wallTime )
2020-08-20 16:30:47 +00:00
2021-11-05 15:39:47 +00:00
return if v . isOk ( ) :
2022-01-03 16:20:15 +00:00
# Due to async validation the wallTime here might have changed
wallTime = self . getCurrentBeaconTime ( )
2021-06-17 14:38:25 +00:00
2021-11-05 15:39:47 +00:00
let ( attesting_indices , sig ) = v . get ( )
2021-04-26 20:39:44 +00:00
2021-11-05 15:39:47 +00:00
self [ ] . checkForPotentialDoppelganger (
signedAggregateAndProof . message . aggregate , attesting_indices )
2020-10-27 17:21:35 +00:00
2021-11-05 15:39:47 +00:00
trace " Aggregate validated "
2020-12-14 20:58:32 +00:00
2021-11-05 15:39:47 +00:00
self . attestationPool [ ] . addAttestation (
2021-12-21 18:56:08 +00:00
signedAggregateAndProof . message . aggregate , attesting_indices , sig ,
wallTime )
2020-08-20 16:30:47 +00:00
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerAggregate (
2022-02-04 07:33:20 +00:00
src , wallTime , signedAggregateAndProof . message , attesting_indices )
2021-12-20 19:20:31 +00:00
2021-11-05 15:39:47 +00:00
beacon_aggregates_received . inc ( )
beacon_aggregate_delay . observe ( delay . toFloatSeconds ( ) )
ok ( )
else :
debug " Dropping aggregate " , error = v . error
beacon_aggregates_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2021-10-19 15:20:55 +00:00
2021-11-05 15:39:47 +00:00
err ( v . error ( ) )
2021-04-02 14:36:43 +00:00
2020-09-14 14:26:31 +00:00
proc attesterSlashingValidator * (
2021-12-20 19:20:31 +00:00
self : var Eth2Processor , src : MsgSource ,
attesterSlashing : AttesterSlashing ) : ValidationRes =
2020-09-14 14:26:31 +00:00
logScope :
attesterSlashing = shortLog ( attesterSlashing )
2021-11-05 15:39:47 +00:00
debug " Attester slashing received "
2020-09-14 14:26:31 +00:00
let v = self . exitPool [ ] . validateAttesterSlashing ( attesterSlashing )
2021-11-05 15:39:47 +00:00
if v . isOk ( ) :
trace " Attester slashing validated "
self . exitPool [ ] . addMessage ( attesterSlashing )
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerAttesterSlashing ( src , attesterSlashing )
2021-11-05 15:39:47 +00:00
beacon_attester_slashings_received . inc ( )
else :
2021-06-10 08:39:53 +00:00
debug " Dropping attester slashing " , validationError = v . error
2021-10-19 15:20:55 +00:00
beacon_attester_slashings_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2020-09-14 14:26:31 +00:00
2021-11-05 15:39:47 +00:00
v
2020-09-18 11:53:09 +00:00
2020-09-14 14:26:31 +00:00
proc proposerSlashingValidator * (
2021-12-20 19:20:31 +00:00
self : var Eth2Processor , src : MsgSource ,
proposerSlashing : ProposerSlashing ) : Result [ void , ValidationError ] =
2020-09-14 14:26:31 +00:00
logScope :
proposerSlashing = shortLog ( proposerSlashing )
2021-11-05 15:39:47 +00:00
debug " Proposer slashing received "
2020-09-14 14:26:31 +00:00
let v = self . exitPool [ ] . validateProposerSlashing ( proposerSlashing )
2021-11-05 15:39:47 +00:00
if v . isOk ( ) :
trace " Proposer slashing validated "
2020-09-14 14:26:31 +00:00
2021-11-05 15:39:47 +00:00
self . exitPool [ ] . addMessage ( proposerSlashing )
2020-09-14 14:26:31 +00:00
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerProposerSlashing ( src , proposerSlashing )
beacon_proposer_slashings_received . inc ( )
else :
debug " Dropping proposer slashing " , validationError = v . error
2021-11-05 15:39:47 +00:00
beacon_proposer_slashings_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2021-12-20 19:20:31 +00:00
2021-11-05 15:39:47 +00:00
v
2020-09-18 11:53:09 +00:00
2020-09-14 14:26:31 +00:00
proc voluntaryExitValidator * (
2021-12-20 19:20:31 +00:00
self : var Eth2Processor , src : MsgSource ,
signedVoluntaryExit : SignedVoluntaryExit ) : Result [ void , ValidationError ] =
2020-09-14 14:26:31 +00:00
logScope :
2020-09-24 17:05:49 +00:00
signedVoluntaryExit = shortLog ( signedVoluntaryExit )
2020-09-14 14:26:31 +00:00
2021-11-05 15:39:47 +00:00
debug " Voluntary exit received "
2020-09-24 17:05:49 +00:00
let v = self . exitPool [ ] . validateVoluntaryExit ( signedVoluntaryExit )
2021-11-05 15:39:47 +00:00
if v . isOk ( ) :
trace " Voluntary exit validated "
2020-09-14 14:26:31 +00:00
2021-11-05 15:39:47 +00:00
self . exitPool [ ] . addMessage ( signedVoluntaryExit )
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerVoluntaryExit (
src , signedVoluntaryExit . message )
2021-11-05 15:39:47 +00:00
beacon_voluntary_exits_received . inc ( )
else :
debug " Dropping voluntary exit " , error = v . error
beacon_voluntary_exits_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2020-09-14 14:26:31 +00:00
2021-11-05 15:39:47 +00:00
v
2021-08-28 22:27:51 +00:00
2021-11-25 12:20:36 +00:00
proc syncCommitteeMessageValidator * (
2021-12-20 19:20:31 +00:00
self : ref Eth2Processor , src : MsgSource ,
2021-08-28 22:27:51 +00:00
syncCommitteeMsg : SyncCommitteeMessage ,
2021-11-05 15:39:47 +00:00
subcommitteeIdx : SyncSubcommitteeIndex ,
2021-12-09 12:56:54 +00:00
checkSignature : bool = true ) : Future [ Result [ void , ValidationError ] ] {. async . } =
2021-11-05 15:39:47 +00:00
let
wallTime = self . getCurrentBeaconTime ( )
wallSlot = wallTime . slotOrZero ( )
2021-08-28 22:27:51 +00:00
logScope :
syncCommitteeMsg = shortLog ( syncCommitteeMsg )
2021-11-05 15:39:47 +00:00
subcommitteeIdx
wallSlot
2021-08-28 22:27:51 +00:00
# Potential under/overflows are fine; would just create odd metrics and logs
2022-01-11 10:01:54 +00:00
let delay = wallTime - syncCommitteeMsg . slot . start_beacon_time
2021-08-28 22:27:51 +00:00
debug " Sync committee message received " , delay
# Now proceed to validation
2021-12-09 12:56:54 +00:00
let v = await validateSyncCommitteeMessage (
2021-12-11 15:39:24 +00:00
self . dag , self . batchCrypto , self . syncCommitteeMsgPool ,
2021-12-09 12:56:54 +00:00
syncCommitteeMsg , subcommitteeIdx , wallTime , checkSignature )
2021-11-05 15:39:47 +00:00
return if v . isOk ( ) :
trace " Sync committee message validated "
let ( positions , cookedSig ) = v . get ( )
2021-11-25 12:20:36 +00:00
self . syncCommitteeMsgPool [ ] . addSyncCommitteeMessage (
2021-11-05 15:39:47 +00:00
syncCommitteeMsg . slot ,
syncCommitteeMsg . beacon_block_root ,
syncCommitteeMsg . validator_index ,
cookedSig ,
subcommitteeIdx ,
positions )
2021-12-20 19:20:31 +00:00
self . validatorMonitor [ ] . registerSyncCommitteeMessage (
src , wallTime , syncCommitteeMsg )
2021-11-05 15:39:47 +00:00
beacon_sync_committee_messages_received . inc ( )
ok ( )
else :
debug " Dropping sync committee message " , error = v . error
2021-10-19 15:20:55 +00:00
beacon_sync_committee_messages_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2021-11-05 15:39:47 +00:00
err ( v . error ( ) )
2021-08-28 22:27:51 +00:00
2021-11-25 12:20:36 +00:00
proc contributionValidator * (
2021-12-20 19:20:31 +00:00
self : ref Eth2Processor , src : MsgSource ,
2021-08-28 22:27:51 +00:00
contributionAndProof : SignedContributionAndProof ,
2021-12-09 12:56:54 +00:00
checkSignature : bool = true ) : Future [ Result [ void , ValidationError ] ] {. async . } =
2021-11-05 15:39:47 +00:00
let
wallTime = self . getCurrentBeaconTime ( )
wallSlot = wallTime . slotOrZero ( )
2021-08-28 22:27:51 +00:00
logScope :
2021-11-05 15:39:47 +00:00
contribution = shortLog ( contributionAndProof . message . contribution )
2021-08-28 22:27:51 +00:00
signature = shortLog ( contributionAndProof . signature )
aggregator_index = contributionAndProof . message . aggregator_index
2021-11-05 15:39:47 +00:00
selection_proof = contributionAndProof . message . selection_proof
wallSlot
2021-08-28 22:27:51 +00:00
# Potential under/overflows are fine; would just create odd metrics and logs
2022-01-11 10:01:54 +00:00
let delay = wallTime - contributionAndProof . message . contribution . slot . start_beacon_time
2021-08-28 22:27:51 +00:00
debug " Contribution received " , delay
# Now proceed to validation
2021-12-09 12:56:54 +00:00
let v = await validateContribution (
self . dag , self . batchCrypto , self . syncCommitteeMsgPool ,
contributionAndProof , wallTime , checkSignature )
2021-08-28 22:27:51 +00:00
2021-11-05 15:39:47 +00:00
return if v . isOk ( ) :
trace " Contribution validated "
2021-12-20 19:20:31 +00:00
self . syncCommitteeMsgPool [ ] . addContribution (
contributionAndProof , v . get ( ) [ 0 ] )
self . validatorMonitor [ ] . registerSyncContribution (
2022-02-04 07:33:20 +00:00
src , wallTime , contributionAndProof . message , v . get ( ) [ 1 ] )
2021-12-20 19:20:31 +00:00
2021-11-05 15:39:47 +00:00
beacon_sync_committee_contributions_received . inc ( )
ok ( )
else :
debug " Dropping contribution " , error = v . error
beacon_sync_committee_contributions_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
2021-10-19 15:20:55 +00:00
2021-11-05 15:39:47 +00:00
err ( v . error ( ) )
2022-03-14 13:05:38 +00:00
# https://github.com/ethereum/consensus-specs/blob/vFuture/specs/altair/sync-protocol.md#optimistic_light_client_update
proc optimisticLightClientUpdateValidator * (
self : var Eth2Processor , src : MsgSource ,
optimistic_update : OptimisticLightClientUpdate
) : Result [ void , ValidationError ] =
logScope :
optimistic_update
debug " Optimistic light client update received "
let v = self . dag . validateOptimisticLightClientUpdate ( optimistic_update )
if v . isOk ( ) :
trace " Optimistic light client update validated "
beacon_optimistic_light_client_updates_received . inc ( )
else :
debug " Dropping optimistic light client update " , error = v . error
beacon_optimistic_light_client_updates_dropped . inc ( 1 , [ $ v . error [ 0 ] ] )
v