2021-01-11 21:27:01 +00:00
|
|
|
# beacon_chain
|
2023-01-12 17:11:38 +00:00
|
|
|
# Copyright (c) 2018-2023 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.
|
|
|
|
|
2023-01-20 14:14:37 +00:00
|
|
|
{.push raises: [].}
|
2021-01-11 21:27:01 +00:00
|
|
|
|
2020-08-20 16:30:47 +00:00
|
|
|
import
|
2021-03-12 09:46:26 +00:00
|
|
|
std/tables,
|
2022-06-21 08:29:16 +00:00
|
|
|
stew/results,
|
2021-12-06 09:49:01 +00:00
|
|
|
chronicles, chronos, metrics, taskpools,
|
2021-08-12 13:08:20 +00:00
|
|
|
../spec/[helpers, forks],
|
2023-01-04 12:34:15 +00:00
|
|
|
../spec/datatypes/[altair, phase0, eip4844],
|
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,
|
2022-05-23 12:02:54 +00:00
|
|
|
light_client_pool, 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,
|
2022-12-09 16:05:55 +00:00
|
|
|
"."/[gossip_validation, block_processor, batch_validation],
|
|
|
|
../nimbus_binary_common
|
2021-10-19 14:09:26 +00:00
|
|
|
|
2021-11-05 15:39:47 +00:00
|
|
|
export
|
2022-06-21 08:29:16 +00:00
|
|
|
results, taskpools, block_clearance, blockchain_dag, exit_pool, attestation_pool,
|
2022-05-23 12:02:54 +00:00
|
|
|
light_client_pool, sync_committee_msg_pool, validator_pool, beacon_clock,
|
|
|
|
gossip_validation, block_processor, batch_validation, block_quarantine
|
2020-08-20 16:30:47 +00:00
|
|
|
|
2022-06-07 17:01:11 +00:00
|
|
|
logScope: topics = "gossip_eth2"
|
|
|
|
|
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"]
|
2023-01-23 13:58:40 +00:00
|
|
|
declareCounter bls_to_execution_change_received,
|
|
|
|
"Number of valid BLS to execution changes processed by this node"
|
|
|
|
declareCounter bls_to_execution_change_dropped,
|
|
|
|
"Number of invalid BLS to execution changes 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"]
|
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.
|
|
|
|
|
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
|
2022-05-23 12:02:54 +00:00
|
|
|
lightClientPool: ref LightClientPool
|
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
|
|
|
|
# ----------------------------------------------------------------
|
2022-07-06 16:11:44 +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
|
|
|
|
# ----------------------------------------------------------------
|
2023-01-19 22:00:40 +00:00
|
|
|
validatorChangePool: ref ValidatorChangePool
|
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]
|
|
|
|
|
2022-05-31 10:45:37 +00:00
|
|
|
func toValidationResult*(res: ValidationRes): ValidationResult =
|
|
|
|
if res.isOk(): ValidationResult.Accept else: res.error()[0]
|
|
|
|
|
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,
|
2023-01-19 22:00:40 +00:00
|
|
|
validatorChangePool: ref ValidatorChangePool,
|
2021-03-11 10:10:57 +00:00
|
|
|
validatorPool: ref ValidatorPool,
|
2021-10-19 14:09:26 +00:00
|
|
|
syncCommitteeMsgPool: ref SyncCommitteeMsgPool,
|
2022-05-23 12:02:54 +00:00
|
|
|
lightClientPool: ref LightClientPool,
|
2021-12-06 09:49:01 +00:00
|
|
|
quarantine: ref Quarantine,
|
2022-06-21 08:29:16 +00:00
|
|
|
rng: ref HmacDrbgContext,
|
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
|
|
|
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,
|
2023-01-19 22:00:40 +00:00
|
|
|
validatorChangePool: validatorChangePool,
|
2021-03-11 10:10:57 +00:00
|
|
|
validatorPool: validatorPool,
|
2021-08-28 22:27:51 +00:00
|
|
|
syncCommitteeMsgPool: syncCommitteeMsgPool,
|
2022-05-23 12:02:54 +00:00
|
|
|
lightClientPool: lightClientPool,
|
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(),
|
2023-02-20 08:26:22 +00:00
|
|
|
genesis_validators_root = dag.genesis_validators_root, 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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processSignedBeaconBlock*(
|
2021-12-20 19:20:31 +00:00
|
|
|
self: var Eth2Processor, src: MsgSource,
|
2023-02-28 11:36:17 +00:00
|
|
|
signedBlock: ForkySignedBeaconBlock,
|
2023-02-06 07:22:08 +00:00
|
|
|
maybeFinalized: bool = false): 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 =
|
2023-02-28 11:36:17 +00:00
|
|
|
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),
|
2023-02-28 11:36:17 +00:00
|
|
|
BlobSidecars @[],
|
2023-02-06 07:22:08 +00:00
|
|
|
maybeFinalized = maybeFinalized,
|
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.
|
2022-11-24 07:48:10 +00:00
|
|
|
|
2022-12-09 16:05:55 +00:00
|
|
|
# Round up to ensure that we cover the entire epoch - used by rest api also
|
|
|
|
self.doppelgangerDetection.broadcastStartEpoch =
|
|
|
|
(slot + SLOTS_PER_EPOCH - 1).epoch
|
|
|
|
|
|
|
|
if self.doppelgangerDetectionEnabled:
|
2022-11-20 13:55:43 +00:00
|
|
|
notice "Setting up doppelganger detection",
|
|
|
|
epoch = slot.epoch,
|
2023-02-20 11:28:56 +00:00
|
|
|
broadcast_epoch = self.doppelgangerDetection.broadcastStartEpoch
|
2022-01-03 21:18:49 +00:00
|
|
|
|
2022-12-09 16:05:55 +00:00
|
|
|
proc clearDoppelgangerProtection*(self: var Eth2Processor) =
|
|
|
|
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
|
|
|
|
|
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
|
|
|
|
|
2022-11-20 13:55:43 +00:00
|
|
|
for validatorIndex in attesterIndices:
|
|
|
|
let
|
2023-02-20 11:28:56 +00:00
|
|
|
pubkey = self.dag.validatorKey(validatorIndex).get().toPubKey()
|
|
|
|
|
|
|
|
if self.validatorPool[].triggersDoppelganger(
|
|
|
|
pubkey, attestation.data.slot.epoch):
|
|
|
|
warn "Doppelganger attestation",
|
|
|
|
validator = shortLog(pubkey),
|
|
|
|
validator_index = validatorIndex,
|
|
|
|
attestation = shortLog(attestation)
|
|
|
|
quitDoppelganger()
|
2020-10-27 17:21:35 +00:00
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processAttestation*(
|
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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processSignedAggregateAndProof*(
|
2021-12-20 19:20:31 +00:00
|
|
|
self: ref Eth2Processor, src: MsgSource,
|
2022-07-06 16:11:44 +00:00
|
|
|
signedAggregateAndProof: SignedAggregateAndProof,
|
|
|
|
checkSignature = true, checkCover = true): 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"
|
2022-07-06 16:11:44 +00:00
|
|
|
return errIgnore("Aggregate 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(
|
2022-07-06 16:11:44 +00:00
|
|
|
self.batchCrypto, signedAggregateAndProof, wallTime,
|
|
|
|
checkSignature = checkSignature, checkCover = checkCover)
|
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
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
proc processBlsToExecutionChange*(
|
2023-02-17 13:35:12 +00:00
|
|
|
self: ref Eth2Processor, src: MsgSource,
|
|
|
|
blsToExecutionChange: SignedBLSToExecutionChange):
|
|
|
|
Future[ValidationRes] {.async.} =
|
2023-01-19 22:00:40 +00:00
|
|
|
logScope:
|
|
|
|
blsToExecutionChange = shortLog(blsToExecutionChange)
|
|
|
|
|
|
|
|
debug "BLS to execution change received"
|
|
|
|
|
2023-02-17 13:35:12 +00:00
|
|
|
let v = await self.validatorChangePool[].validateBlsToExecutionChange(
|
|
|
|
self.batchCrypto, blsToExecutionChange,
|
|
|
|
self.getCurrentBeaconTime().slotOrZero.epoch)
|
2023-01-19 22:00:40 +00:00
|
|
|
|
|
|
|
if v.isOk():
|
|
|
|
trace "BLS to execution change validated"
|
2023-02-03 15:28:28 +00:00
|
|
|
# Prioritize API-provided messages
|
|
|
|
self.validatorChangePool[].addMessage(
|
|
|
|
blsToExecutionChange, src == MsgSource.api)
|
2023-01-23 13:58:40 +00:00
|
|
|
else:
|
|
|
|
debug "Dropping BLS to execution change", validationError = v.error
|
|
|
|
beacon_attester_slashings_dropped.inc(1, [$v.error[0]])
|
2023-01-19 22:00:40 +00:00
|
|
|
|
2023-02-17 13:35:12 +00:00
|
|
|
return v
|
2023-01-19 22:00:40 +00:00
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processAttesterSlashing*(
|
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"
|
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
let v = self.validatorChangePool[].validateAttesterSlashing(attesterSlashing)
|
2021-11-05 15:39:47 +00:00
|
|
|
|
|
|
|
if v.isOk():
|
|
|
|
trace "Attester slashing validated"
|
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
self.validatorChangePool[].addMessage(attesterSlashing)
|
2021-11-05 15:39:47 +00:00
|
|
|
|
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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processProposerSlashing*(
|
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"
|
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
let v = self.validatorChangePool[].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
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
self.validatorChangePool[].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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processSignedVoluntaryExit*(
|
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"
|
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
let v = self.validatorChangePool[].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
|
|
|
|
2023-01-19 22:00:40 +00:00
|
|
|
self.validatorChangePool[].addMessage(signedVoluntaryExit)
|
2021-11-05 15:39:47 +00:00
|
|
|
|
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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processSyncCommitteeMessage*(
|
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
|
|
|
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processSignedContributionAndProof*(
|
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"
|
2023-01-12 14:08:08 +00:00
|
|
|
self.syncCommitteeMsgPool[].addContribution(
|
2021-12-20 19:20:31 +00:00
|
|
|
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()
|
|
|
|
|
2023-01-12 14:08:08 +00:00
|
|
|
ok()
|
2021-11-05 15:39:47 +00:00
|
|
|
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
|
|
|
|
2022-11-08 02:37:28 +00:00
|
|
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.0/specs/altair/light-client/sync-protocol.md#process_light_client_finality_update
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processLightClientFinalityUpdate*(
|
2022-03-14 13:05:38 +00:00
|
|
|
self: var Eth2Processor, src: MsgSource,
|
2023-01-12 17:11:38 +00:00
|
|
|
finality_update: ForkedLightClientFinalityUpdate
|
2022-05-23 12:02:54 +00:00
|
|
|
): Result[void, ValidationError] =
|
|
|
|
let
|
|
|
|
wallTime = self.getCurrentBeaconTime()
|
|
|
|
v = validateLightClientFinalityUpdate(
|
|
|
|
self.lightClientPool[], self.dag, finality_update, wallTime)
|
|
|
|
v
|
|
|
|
|
2022-11-08 02:37:28 +00:00
|
|
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0-alpha.0/specs/altair/light-client/sync-protocol.md#process_light_client_optimistic_update
|
2022-07-06 16:11:44 +00:00
|
|
|
proc processLightClientOptimisticUpdate*(
|
2022-05-23 12:02:54 +00:00
|
|
|
self: var Eth2Processor, src: MsgSource,
|
2023-01-12 17:11:38 +00:00
|
|
|
optimistic_update: ForkedLightClientOptimisticUpdate
|
2022-03-14 13:05:38 +00:00
|
|
|
): Result[void, ValidationError] =
|
2022-05-23 12:02:54 +00:00
|
|
|
let
|
|
|
|
wallTime = self.getCurrentBeaconTime()
|
|
|
|
v = validateLightClientOptimisticUpdate(
|
|
|
|
self.lightClientPool[], self.dag, optimistic_update, wallTime)
|
2022-03-14 13:05:38 +00:00
|
|
|
v
|