centralize p2p validation in a single file and address https://github.com/status-im/nimbus-eth2/pull/2377#issuecomment-791313118 (#2383)
This commit is contained in:
parent
5f750f84b4
commit
de1060e7f3
|
@ -9,13 +9,13 @@
|
|||
|
||||
import
|
||||
# Standard libraries
|
||||
std/[deques, options, sequtils, tables],
|
||||
std/[options, tables, sequtils],
|
||||
# Status libraries
|
||||
chronicles, stew/[byteutils], json_serialization/std/sets as jsonSets,
|
||||
# Internal
|
||||
../spec/[beaconstate, datatypes, crypto, digest, helpers],
|
||||
../spec/[beaconstate, datatypes, crypto, digest],
|
||||
../ssz/merkleization,
|
||||
"."/[spec_cache, blockchain_dag, block_clearance, block_quarantine],
|
||||
"."/[spec_cache, blockchain_dag, block_quarantine],
|
||||
../beacon_node_types,
|
||||
../fork_choice/fork_choice
|
||||
|
||||
|
|
|
@ -328,144 +328,3 @@ proc addRawBlock*(
|
|||
if parent != nil:
|
||||
return addRawBlockKnownParent(dag, quarantine, signedBlock, parent, onBlockAdded)
|
||||
return addRawBlockUnresolved(dag, quarantine, signedBlock)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#beacon_block
|
||||
proc isValidBeaconBlock*(
|
||||
dag: ChainDAGRef, quarantine: var QuarantineRef,
|
||||
signed_beacon_block: SignedBeaconBlock, wallTime: BeaconTime,
|
||||
flags: UpdateFlags):
|
||||
Result[void, (ValidationResult, BlockError)] =
|
||||
logScope:
|
||||
topics = "clearance valid_blck"
|
||||
received_block = shortLog(signed_beacon_block.message)
|
||||
blockRoot = shortLog(signed_beacon_block.root)
|
||||
|
||||
# In general, checks are ordered from cheap to expensive. Especially, crypto
|
||||
# verification could be quite a bit more expensive than the rest. This is an
|
||||
# externally easy-to-invoke function by tossing network packets at the node.
|
||||
|
||||
# [IGNORE] The block is not from a future slot (with a
|
||||
# MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. validate that
|
||||
# signed_beacon_block.message.slot <= current_slot (a client MAY queue future
|
||||
# blocks for processing at the appropriate slot).
|
||||
if not (signed_beacon_block.message.slot <=
|
||||
(wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY).slotOrZero):
|
||||
debug "block is from a future slot",
|
||||
wallSlot = wallTime.toSlot()
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block is from a slot greater than the latest finalized slot --
|
||||
# i.e. validate that signed_beacon_block.message.slot >
|
||||
# compute_start_slot_at_epoch(state.finalized_checkpoint.epoch)
|
||||
if not (signed_beacon_block.message.slot > dag.finalizedHead.slot):
|
||||
debug "block is not from a slot greater than the latest finalized slot"
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block is the first block with valid signature received for the
|
||||
# proposer for the slot, signed_beacon_block.message.slot.
|
||||
#
|
||||
# While this condition is similar to the proposer slashing condition at
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#proposer-slashing
|
||||
# it's not identical, and this check does not address slashing:
|
||||
#
|
||||
# (1) The beacon blocks must be conflicting, i.e. different, for the same
|
||||
# slot and proposer. This check also catches identical blocks.
|
||||
#
|
||||
# (2) By this point in the function, it's not been checked whether they're
|
||||
# signed yet. As in general, expensive checks should be deferred, this
|
||||
# would add complexity not directly relevant this function.
|
||||
#
|
||||
# (3) As evidenced by point (1), the similarity in the validation condition
|
||||
# and slashing condition, while not coincidental, aren't similar enough
|
||||
# to combine, as one or the other might drift.
|
||||
#
|
||||
# (4) Furthermore, this function, as much as possible, simply returns a yes
|
||||
# or no answer, without modifying other state for p2p network interface
|
||||
# validation. Complicating this interface, for the sake of sharing only
|
||||
# couple lines of code, wouldn't be worthwhile.
|
||||
#
|
||||
# TODO might check unresolved/orphaned blocks too, and this might not see all
|
||||
# blocks at a given slot (though, in theory, those get checked elsewhere), or
|
||||
# adding metrics that count how often these conditions occur.
|
||||
let
|
||||
slotBlockRef = getBlockBySlot(dag, signed_beacon_block.message.slot)
|
||||
|
||||
if not slotBlockRef.isNil:
|
||||
let blck = dag.get(slotBlockRef).data
|
||||
if blck.message.proposer_index ==
|
||||
signed_beacon_block.message.proposer_index and
|
||||
blck.message.slot == signed_beacon_block.message.slot and
|
||||
blck.signature.toRaw() != signed_beacon_block.signature.toRaw():
|
||||
notice "block isn't first block with valid signature received for the proposer",
|
||||
blckRef = slotBlockRef,
|
||||
existing_block = shortLog(blck.message)
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block's parent (defined by block.parent_root) has been seen
|
||||
# (via both gossip and non-gossip sources) (a client MAY queue blocks for
|
||||
# processing once the parent block is retrieved).
|
||||
#
|
||||
# And implicitly:
|
||||
# [REJECT] The block's parent (defined by block.parent_root) passes validation.
|
||||
let parent_ref = dag.getRef(signed_beacon_block.message.parent_root)
|
||||
if parent_ref.isNil:
|
||||
# Pending dag gets checked via `ChainDAGRef.add(...)` later, and relevant
|
||||
# checks are performed there. In usual paths beacon_node adds blocks via
|
||||
# ChainDAGRef.add(...) directly, with no additional validity checks.
|
||||
debug "parent unknown, putting block in quarantine",
|
||||
current_slot = wallTime.toSlot()
|
||||
if not quarantine.add(dag, signed_beacon_block):
|
||||
debug "Block quarantine full"
|
||||
return err((ValidationResult.Ignore, MissingParent))
|
||||
|
||||
# [REJECT] The current finalized_checkpoint is an ancestor of block -- i.e.
|
||||
# get_ancestor(store, block.parent_root,
|
||||
# compute_start_slot_at_epoch(store.finalized_checkpoint.epoch)) ==
|
||||
# store.finalized_checkpoint.root
|
||||
let
|
||||
finalized_checkpoint = dag.headState.data.data.finalized_checkpoint
|
||||
ancestor = get_ancestor(
|
||||
parent_ref, compute_start_slot_at_epoch(finalized_checkpoint.epoch))
|
||||
|
||||
if ancestor.isNil:
|
||||
debug "couldn't find ancestor block"
|
||||
return err((ValidationResult.Ignore, Invalid)) # might not've received block
|
||||
|
||||
if not (finalized_checkpoint.root in [ancestor.root, Eth2Digest()]):
|
||||
debug "block not descendent of finalized block"
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
# [REJECT] The block is proposed by the expected proposer_index for the
|
||||
# block's slot in the context of the current shuffling (defined by
|
||||
# parent_root/slot). If the proposer_index cannot immediately be verified
|
||||
# against the expected shuffling, the block MAY be queued for later
|
||||
# processing while proposers for the block's branch are calculated -- in such
|
||||
# a case do not REJECT, instead IGNORE this message.
|
||||
let
|
||||
proposer = getProposer(dag, parent_ref, signed_beacon_block.message.slot)
|
||||
|
||||
if proposer.isNone:
|
||||
warn "cannot compute proposer for message"
|
||||
return err((ValidationResult.Ignore, Invalid)) # internal issue
|
||||
|
||||
if proposer.get()[0] !=
|
||||
ValidatorIndex(signed_beacon_block.message.proposer_index):
|
||||
notice "block had unexpected proposer",
|
||||
expected_proposer = proposer.get()[0]
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
# [REJECT] The proposer signature, signed_beacon_block.signature, is valid
|
||||
# with respect to the proposer_index pubkey.
|
||||
if not verify_block_signature(
|
||||
dag.headState.data.data.fork,
|
||||
dag.headState.data.data.genesis_validators_root,
|
||||
signed_beacon_block.message.slot,
|
||||
signed_beacon_block.message,
|
||||
proposer.get()[1],
|
||||
signed_beacon_block.signature):
|
||||
debug "block failed signature verification",
|
||||
signature = shortLog(signed_beacon_block.signature)
|
||||
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
ok()
|
||||
|
|
|
@ -9,12 +9,12 @@
|
|||
|
||||
import
|
||||
# Standard libraries
|
||||
std/[deques, intsets, options, sequtils, tables],
|
||||
std/[deques, intsets, tables],
|
||||
# Status libraries
|
||||
chronicles, json_serialization/std/sets as jsonSets,
|
||||
chronicles,
|
||||
# Internal
|
||||
../spec/[crypto, datatypes, helpers, state_transition_block],
|
||||
"."/[blockchain_dag, block_clearance, block_quarantine],
|
||||
../spec/[crypto, datatypes, helpers],
|
||||
"."/[blockchain_dag, block_quarantine],
|
||||
../beacon_node_types
|
||||
|
||||
export beacon_node_types, intsets
|
||||
|
@ -22,9 +22,9 @@ export beacon_node_types, intsets
|
|||
logScope: topics = "exitpool"
|
||||
|
||||
const
|
||||
ATTESTER_SLASHINGS_BOUND = MAX_ATTESTER_SLASHINGS * 2
|
||||
PROPOSER_SLASHINGS_BOUND = MAX_PROPOSER_SLASHINGS * 2
|
||||
VOLUNTARY_EXITS_BOUND = MAX_VOLUNTARY_EXITS * 2
|
||||
ATTESTER_SLASHINGS_BOUND* = MAX_ATTESTER_SLASHINGS * 2
|
||||
PROPOSER_SLASHINGS_BOUND* = MAX_PROPOSER_SLASHINGS * 2
|
||||
VOLUNTARY_EXITS_BOUND* = MAX_VOLUNTARY_EXITS * 2
|
||||
|
||||
proc init*(
|
||||
T: type ExitPool, chainDag: ChainDAGRef, quarantine: QuarantineRef): T =
|
||||
|
@ -138,105 +138,3 @@ func getVoluntaryExitsForBlock*(pool: var ExitPool):
|
|||
## Retrieve voluntary exits that may be added to a new block
|
||||
getExitMessagesForBlock[SignedVoluntaryExit](
|
||||
pool.voluntary_exits, pool, MAX_VOLUNTARY_EXITS)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#attester_slashing
|
||||
proc validateAttesterSlashing*(
|
||||
pool: var ExitPool, attester_slashing: AttesterSlashing):
|
||||
Result[bool, (ValidationResult, cstring)] =
|
||||
# [IGNORE] At least one index in the intersection of the attesting indices of
|
||||
# each attestation has not yet been seen in any prior attester_slashing (i.e.
|
||||
# attester_slashed_indices = set(attestation_1.attesting_indices).intersection(attestation_2.attesting_indices),
|
||||
# verify if any(attester_slashed_indices.difference(prior_seen_attester_slashed_indices))).
|
||||
# TODO sequtils2 should be able to make this more reasonable, from asSeq on
|
||||
# down, and can sort and just find intersection that way
|
||||
let
|
||||
attestation_1_indices =
|
||||
attester_slashing.attestation_1.attesting_indices.asSeq
|
||||
attestation_2_indices =
|
||||
attester_slashing.attestation_2.attesting_indices.asSeq
|
||||
attester_slashed_indices =
|
||||
toIntSet(attestation_1_indices) * toIntSet(attestation_2_indices)
|
||||
|
||||
if not disjoint(
|
||||
attester_slashed_indices, pool.prior_seen_attester_slashed_indices):
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateAttesterSlashing: attester-slashed index already attester-slashed")))
|
||||
|
||||
# [REJECT] All of the conditions within process_attester_slashing pass
|
||||
# validation.
|
||||
let attester_slashing_validity =
|
||||
check_attester_slashing(
|
||||
pool.chainDag.headState.data.data, attester_slashing, {})
|
||||
if attester_slashing_validity.isErr:
|
||||
return err((ValidationResult.Reject, attester_slashing_validity.error))
|
||||
|
||||
pool.prior_seen_attester_slashed_indices.incl attester_slashed_indices
|
||||
pool.attester_slashings.addExitMessage(
|
||||
attester_slashing, ATTESTER_SLASHINGS_BOUND)
|
||||
|
||||
ok(true)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#proposer_slashing
|
||||
proc validateProposerSlashing*(
|
||||
pool: var ExitPool, proposer_slashing: ProposerSlashing):
|
||||
Result[bool, (ValidationResult, cstring)] =
|
||||
# Not from spec; the rest of NBC wouldn't have correctly processed it either.
|
||||
if proposer_slashing.signed_header_1.message.proposer_index > high(int).uint64:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateProposerSlashing: proposer-slashed index too high")))
|
||||
|
||||
# [IGNORE] The proposer slashing is the first valid proposer slashing
|
||||
# received for the proposer with index
|
||||
# proposer_slashing.signed_header_1.message.proposer_index.
|
||||
if proposer_slashing.signed_header_1.message.proposer_index.int in
|
||||
pool.prior_seen_proposer_slashed_indices:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateProposerSlashing: proposer-slashed index already proposer-slashed")))
|
||||
|
||||
# [REJECT] All of the conditions within process_proposer_slashing pass validation.
|
||||
let proposer_slashing_validity =
|
||||
check_proposer_slashing(
|
||||
pool.chainDag.headState.data.data, proposer_slashing, {})
|
||||
if proposer_slashing_validity.isErr:
|
||||
return err((ValidationResult.Reject, proposer_slashing_validity.error))
|
||||
|
||||
pool.prior_seen_proposer_slashed_indices.incl(
|
||||
proposer_slashing.signed_header_1.message.proposer_index.int)
|
||||
pool.proposer_slashings.addExitMessage(
|
||||
proposer_slashing, PROPOSER_SLASHINGS_BOUND)
|
||||
|
||||
ok(true)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#voluntary_exit
|
||||
proc validateVoluntaryExit*(
|
||||
pool: var ExitPool, signed_voluntary_exit: SignedVoluntaryExit):
|
||||
Result[void, (ValidationResult, cstring)] =
|
||||
# [IGNORE] The voluntary exit is the first valid voluntary exit received for
|
||||
# the validator with index signed_voluntary_exit.message.validator_index.
|
||||
if signed_voluntary_exit.message.validator_index >=
|
||||
pool.chainDag.headState.data.data.validators.lenu64:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateVoluntaryExit: validator index too high")))
|
||||
|
||||
# Since pool.chainDag.headState.data.data.validators is a seq, this means
|
||||
# signed_voluntary_exit.message.validator_index.int is already valid, but
|
||||
# check explicitly if one changes that data structure.
|
||||
if signed_voluntary_exit.message.validator_index.int in
|
||||
pool.prior_seen_voluntary_exit_indices:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateVoluntaryExit: validator index already voluntarily exited")))
|
||||
|
||||
# [REJECT] All of the conditions within process_voluntary_exit pass
|
||||
# validation.
|
||||
let voluntary_exit_validity =
|
||||
check_voluntary_exit(
|
||||
pool.chainDag.headState.data.data, signed_voluntary_exit, {})
|
||||
if voluntary_exit_validity.isErr:
|
||||
return err((ValidationResult.Reject, voluntary_exit_validity.error))
|
||||
|
||||
pool.prior_seen_voluntary_exit_indices.incl(
|
||||
signed_voluntary_exit.message.validator_index.int)
|
||||
pool.voluntary_exits.addExitMessage(
|
||||
signed_voluntary_exit, VOLUNTARY_EXITS_BOUND)
|
||||
|
||||
ok()
|
||||
|
|
|
@ -13,7 +13,7 @@ import
|
|||
chronicles, chronos, metrics,
|
||||
../spec/[crypto, datatypes, digest],
|
||||
../consensus_object_pools/[block_clearance, blockchain_dag, exit_pool, attestation_pool],
|
||||
./attestation_aggregation,
|
||||
./gossip_validation,
|
||||
../validators/validator_pool,
|
||||
../beacon_node_types,
|
||||
../beacon_clock, ../conf, ../ssz/sszdump
|
||||
|
|
|
@ -8,54 +8,22 @@
|
|||
{.push raises: [Defect].}
|
||||
|
||||
import
|
||||
std/[options, sequtils],
|
||||
chronos, chronicles,
|
||||
std/[sequtils, intsets, deques],
|
||||
chronicles,
|
||||
stew/results,
|
||||
../spec/[
|
||||
beaconstate, datatypes, crypto, digest, helpers, network, signatures],
|
||||
beaconstate, state_transition_block,
|
||||
datatypes, crypto, digest, helpers, network, signatures],
|
||||
../consensus_object_pools/[
|
||||
spec_cache, blockchain_dag, block_quarantine, spec_cache,
|
||||
attestation_pool
|
||||
attestation_pool, exit_pool
|
||||
],
|
||||
".."/[beacon_node_types, ssz, beacon_clock]
|
||||
".."/[beacon_node_types, ssz, beacon_clock],
|
||||
../validators/attestation_aggregation,
|
||||
../extras
|
||||
|
||||
logScope:
|
||||
topics = "att_aggr"
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregation-selection
|
||||
func is_aggregator*(committee_len: uint64, slot_signature: ValidatorSig): bool =
|
||||
let
|
||||
modulo = max(1'u64, committee_len div TARGET_AGGREGATORS_PER_COMMITTEE)
|
||||
bytes_to_uint64(eth2digest(
|
||||
slot_signature.toRaw()).data.toOpenArray(0, 7)) mod modulo == 0
|
||||
|
||||
func is_aggregator(epochRef: EpochRef, slot: Slot, index: CommitteeIndex,
|
||||
slot_signature: ValidatorSig): bool =
|
||||
let
|
||||
committee_len = get_beacon_committee_len(epochRef, slot, index)
|
||||
return is_aggregator(committee_len, slot_signature)
|
||||
|
||||
proc aggregate_attestations*(
|
||||
pool: AttestationPool, epochRef: EpochRef, slot: Slot, index: CommitteeIndex,
|
||||
validatorIndex: ValidatorIndex, slot_signature: ValidatorSig): Option[AggregateAndProof] =
|
||||
doAssert validatorIndex in get_beacon_committee(epochRef, slot, index)
|
||||
doAssert index.uint64 < get_committee_count_per_slot(epochRef)
|
||||
|
||||
# TODO for testing purposes, refactor this into the condition check
|
||||
# and just calculation
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregation-selection
|
||||
if not is_aggregator(epochRef, slot, index, slot_signature):
|
||||
return none(AggregateAndProof)
|
||||
|
||||
let maybe_slot_attestation = getAggregatedAttestation(pool, slot, index)
|
||||
if maybe_slot_attestation.isNone:
|
||||
return none(AggregateAndProof)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#construct-aggregate
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregateandproof
|
||||
some(AggregateAndProof(
|
||||
aggregator_index: validatorIndex.uint64,
|
||||
aggregate: maybe_slot_attestation.get,
|
||||
selection_proof: slot_signature))
|
||||
topics = "gossip_checks"
|
||||
|
||||
func check_attestation_block(
|
||||
pool: AttestationPool, attestationSlot: Slot, blck: BlockRef):
|
||||
|
@ -440,3 +408,246 @@ proc validateAggregate*(
|
|||
aggregate.data.target.epoch + 1
|
||||
|
||||
ok(attesting_indices)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#beacon_block
|
||||
proc isValidBeaconBlock*(
|
||||
dag: ChainDAGRef, quarantine: var QuarantineRef,
|
||||
signed_beacon_block: SignedBeaconBlock, wallTime: BeaconTime,
|
||||
flags: UpdateFlags):
|
||||
Result[void, (ValidationResult, BlockError)] =
|
||||
logScope:
|
||||
received_block = shortLog(signed_beacon_block.message)
|
||||
blockRoot = shortLog(signed_beacon_block.root)
|
||||
|
||||
# In general, checks are ordered from cheap to expensive. Especially, crypto
|
||||
# verification could be quite a bit more expensive than the rest. This is an
|
||||
# externally easy-to-invoke function by tossing network packets at the node.
|
||||
|
||||
# [IGNORE] The block is not from a future slot (with a
|
||||
# MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e. validate that
|
||||
# signed_beacon_block.message.slot <= current_slot (a client MAY queue future
|
||||
# blocks for processing at the appropriate slot).
|
||||
if not (signed_beacon_block.message.slot <=
|
||||
(wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY).slotOrZero):
|
||||
debug "block is from a future slot",
|
||||
wallSlot = wallTime.toSlot()
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block is from a slot greater than the latest finalized slot --
|
||||
# i.e. validate that signed_beacon_block.message.slot >
|
||||
# compute_start_slot_at_epoch(state.finalized_checkpoint.epoch)
|
||||
if not (signed_beacon_block.message.slot > dag.finalizedHead.slot):
|
||||
debug "block is not from a slot greater than the latest finalized slot"
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block is the first block with valid signature received for the
|
||||
# proposer for the slot, signed_beacon_block.message.slot.
|
||||
#
|
||||
# While this condition is similar to the proposer slashing condition at
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#proposer-slashing
|
||||
# it's not identical, and this check does not address slashing:
|
||||
#
|
||||
# (1) The beacon blocks must be conflicting, i.e. different, for the same
|
||||
# slot and proposer. This check also catches identical blocks.
|
||||
#
|
||||
# (2) By this point in the function, it's not been checked whether they're
|
||||
# signed yet. As in general, expensive checks should be deferred, this
|
||||
# would add complexity not directly relevant this function.
|
||||
#
|
||||
# (3) As evidenced by point (1), the similarity in the validation condition
|
||||
# and slashing condition, while not coincidental, aren't similar enough
|
||||
# to combine, as one or the other might drift.
|
||||
#
|
||||
# (4) Furthermore, this function, as much as possible, simply returns a yes
|
||||
# or no answer, without modifying other state for p2p network interface
|
||||
# validation. Complicating this interface, for the sake of sharing only
|
||||
# couple lines of code, wouldn't be worthwhile.
|
||||
#
|
||||
# TODO might check unresolved/orphaned blocks too, and this might not see all
|
||||
# blocks at a given slot (though, in theory, those get checked elsewhere), or
|
||||
# adding metrics that count how often these conditions occur.
|
||||
let
|
||||
slotBlockRef = getBlockBySlot(dag, signed_beacon_block.message.slot)
|
||||
|
||||
if not slotBlockRef.isNil:
|
||||
let blck = dag.get(slotBlockRef).data
|
||||
if blck.message.proposer_index ==
|
||||
signed_beacon_block.message.proposer_index and
|
||||
blck.message.slot == signed_beacon_block.message.slot and
|
||||
blck.signature.toRaw() != signed_beacon_block.signature.toRaw():
|
||||
notice "block isn't first block with valid signature received for the proposer",
|
||||
blckRef = slotBlockRef,
|
||||
existing_block = shortLog(blck.message)
|
||||
return err((ValidationResult.Ignore, Invalid))
|
||||
|
||||
# [IGNORE] The block's parent (defined by block.parent_root) has been seen
|
||||
# (via both gossip and non-gossip sources) (a client MAY queue blocks for
|
||||
# processing once the parent block is retrieved).
|
||||
#
|
||||
# And implicitly:
|
||||
# [REJECT] The block's parent (defined by block.parent_root) passes validation.
|
||||
let parent_ref = dag.getRef(signed_beacon_block.message.parent_root)
|
||||
if parent_ref.isNil:
|
||||
# Pending dag gets checked via `ChainDAGRef.add(...)` later, and relevant
|
||||
# checks are performed there. In usual paths beacon_node adds blocks via
|
||||
# ChainDAGRef.add(...) directly, with no additional validity checks.
|
||||
debug "parent unknown, putting block in quarantine",
|
||||
current_slot = wallTime.toSlot()
|
||||
if not quarantine.add(dag, signed_beacon_block):
|
||||
debug "Block quarantine full"
|
||||
return err((ValidationResult.Ignore, MissingParent))
|
||||
|
||||
# [REJECT] The current finalized_checkpoint is an ancestor of block -- i.e.
|
||||
# get_ancestor(store, block.parent_root,
|
||||
# compute_start_slot_at_epoch(store.finalized_checkpoint.epoch)) ==
|
||||
# store.finalized_checkpoint.root
|
||||
let
|
||||
finalized_checkpoint = dag.headState.data.data.finalized_checkpoint
|
||||
ancestor = get_ancestor(
|
||||
parent_ref, compute_start_slot_at_epoch(finalized_checkpoint.epoch))
|
||||
|
||||
if ancestor.isNil:
|
||||
debug "couldn't find ancestor block"
|
||||
return err((ValidationResult.Ignore, Invalid)) # might not've received block
|
||||
|
||||
if not (finalized_checkpoint.root in [ancestor.root, Eth2Digest()]):
|
||||
debug "block not descendent of finalized block"
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
# [REJECT] The block is proposed by the expected proposer_index for the
|
||||
# block's slot in the context of the current shuffling (defined by
|
||||
# parent_root/slot). If the proposer_index cannot immediately be verified
|
||||
# against the expected shuffling, the block MAY be queued for later
|
||||
# processing while proposers for the block's branch are calculated -- in such
|
||||
# a case do not REJECT, instead IGNORE this message.
|
||||
let
|
||||
proposer = getProposer(dag, parent_ref, signed_beacon_block.message.slot)
|
||||
|
||||
if proposer.isNone:
|
||||
warn "cannot compute proposer for message"
|
||||
return err((ValidationResult.Ignore, Invalid)) # internal issue
|
||||
|
||||
if proposer.get()[0] !=
|
||||
ValidatorIndex(signed_beacon_block.message.proposer_index):
|
||||
notice "block had unexpected proposer",
|
||||
expected_proposer = proposer.get()[0]
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
# [REJECT] The proposer signature, signed_beacon_block.signature, is valid
|
||||
# with respect to the proposer_index pubkey.
|
||||
if not verify_block_signature(
|
||||
dag.headState.data.data.fork,
|
||||
dag.headState.data.data.genesis_validators_root,
|
||||
signed_beacon_block.message.slot,
|
||||
signed_beacon_block.message,
|
||||
proposer.get()[1],
|
||||
signed_beacon_block.signature):
|
||||
debug "block failed signature verification",
|
||||
signature = shortLog(signed_beacon_block.signature)
|
||||
|
||||
return err((ValidationResult.Reject, Invalid))
|
||||
|
||||
ok()
|
||||
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#attester_slashing
|
||||
proc validateAttesterSlashing*(
|
||||
pool: var ExitPool, attester_slashing: AttesterSlashing):
|
||||
Result[bool, (ValidationResult, cstring)] =
|
||||
# [IGNORE] At least one index in the intersection of the attesting indices of
|
||||
# each attestation has not yet been seen in any prior attester_slashing (i.e.
|
||||
# attester_slashed_indices = set(attestation_1.attesting_indices).intersection(attestation_2.attesting_indices),
|
||||
# verify if any(attester_slashed_indices.difference(prior_seen_attester_slashed_indices))).
|
||||
# TODO sequtils2 should be able to make this more reasonable, from asSeq on
|
||||
# down, and can sort and just find intersection that way
|
||||
let
|
||||
attestation_1_indices =
|
||||
attester_slashing.attestation_1.attesting_indices.asSeq
|
||||
attestation_2_indices =
|
||||
attester_slashing.attestation_2.attesting_indices.asSeq
|
||||
attester_slashed_indices =
|
||||
toIntSet(attestation_1_indices) * toIntSet(attestation_2_indices)
|
||||
|
||||
if not disjoint(
|
||||
attester_slashed_indices, pool.prior_seen_attester_slashed_indices):
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateAttesterSlashing: attester-slashed index already attester-slashed")))
|
||||
|
||||
# [REJECT] All of the conditions within process_attester_slashing pass
|
||||
# validation.
|
||||
let attester_slashing_validity =
|
||||
check_attester_slashing(
|
||||
pool.chainDag.headState.data.data, attester_slashing, {})
|
||||
if attester_slashing_validity.isErr:
|
||||
return err((ValidationResult.Reject, attester_slashing_validity.error))
|
||||
|
||||
pool.prior_seen_attester_slashed_indices.incl attester_slashed_indices
|
||||
pool.attester_slashings.addExitMessage(
|
||||
attester_slashing, ATTESTER_SLASHINGS_BOUND)
|
||||
|
||||
ok(true)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#proposer_slashing
|
||||
proc validateProposerSlashing*(
|
||||
pool: var ExitPool, proposer_slashing: ProposerSlashing):
|
||||
Result[bool, (ValidationResult, cstring)] =
|
||||
# Not from spec; the rest of NBC wouldn't have correctly processed it either.
|
||||
if proposer_slashing.signed_header_1.message.proposer_index > high(int).uint64:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateProposerSlashing: proposer-slashed index too high")))
|
||||
|
||||
# [IGNORE] The proposer slashing is the first valid proposer slashing
|
||||
# received for the proposer with index
|
||||
# proposer_slashing.signed_header_1.message.proposer_index.
|
||||
if proposer_slashing.signed_header_1.message.proposer_index.int in
|
||||
pool.prior_seen_proposer_slashed_indices:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateProposerSlashing: proposer-slashed index already proposer-slashed")))
|
||||
|
||||
# [REJECT] All of the conditions within process_proposer_slashing pass validation.
|
||||
let proposer_slashing_validity =
|
||||
check_proposer_slashing(
|
||||
pool.chainDag.headState.data.data, proposer_slashing, {})
|
||||
if proposer_slashing_validity.isErr:
|
||||
return err((ValidationResult.Reject, proposer_slashing_validity.error))
|
||||
|
||||
pool.prior_seen_proposer_slashed_indices.incl(
|
||||
proposer_slashing.signed_header_1.message.proposer_index.int)
|
||||
pool.proposer_slashings.addExitMessage(
|
||||
proposer_slashing, PROPOSER_SLASHINGS_BOUND)
|
||||
|
||||
ok(true)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/p2p-interface.md#voluntary_exit
|
||||
proc validateVoluntaryExit*(
|
||||
pool: var ExitPool, signed_voluntary_exit: SignedVoluntaryExit):
|
||||
Result[void, (ValidationResult, cstring)] =
|
||||
# [IGNORE] The voluntary exit is the first valid voluntary exit received for
|
||||
# the validator with index signed_voluntary_exit.message.validator_index.
|
||||
if signed_voluntary_exit.message.validator_index >=
|
||||
pool.chainDag.headState.data.data.validators.lenu64:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateVoluntaryExit: validator index too high")))
|
||||
|
||||
# Since pool.chainDag.headState.data.data.validators is a seq, this means
|
||||
# signed_voluntary_exit.message.validator_index.int is already valid, but
|
||||
# check explicitly if one changes that data structure.
|
||||
if signed_voluntary_exit.message.validator_index.int in
|
||||
pool.prior_seen_voluntary_exit_indices:
|
||||
return err((ValidationResult.Ignore, cstring(
|
||||
"validateVoluntaryExit: validator index already voluntarily exited")))
|
||||
|
||||
# [REJECT] All of the conditions within process_voluntary_exit pass
|
||||
# validation.
|
||||
let voluntary_exit_validity =
|
||||
check_voluntary_exit(
|
||||
pool.chainDag.headState.data.data, signed_voluntary_exit, {})
|
||||
if voluntary_exit_validity.isErr:
|
||||
return err((ValidationResult.Reject, voluntary_exit_validity.error))
|
||||
|
||||
pool.prior_seen_voluntary_exit_indices.incl(
|
||||
signed_voluntary_exit.message.validator_index.int)
|
||||
pool.voluntary_exits.addExitMessage(
|
||||
signed_voluntary_exit, VOLUNTARY_EXITS_BOUND)
|
||||
|
||||
ok()
|
|
@ -24,7 +24,7 @@ import
|
|||
".."/[
|
||||
version, conf,
|
||||
ssz/ssz_serialization, beacon_clock],
|
||||
../spec/[datatypes, digest, helpers, network],
|
||||
../spec/[datatypes, digest, network],
|
||||
../validators/keystore_management,
|
||||
./eth2_discovery, ./peer_pool, ./libp2p_json_serialization
|
||||
|
||||
|
|
|
@ -29,8 +29,8 @@ import
|
|||
nimbus_binary_common, ssz/merkleization, statusbar,
|
||||
beacon_clock, version],
|
||||
./networking/[eth2_discovery, eth2_network, network_metadata],
|
||||
./gossip_processing/[eth2_processor, attestation_aggregation],
|
||||
./validators/[validator_duties, validator_pool, slashing_protection, keystore_management],
|
||||
./gossip_processing/eth2_processor,
|
||||
./validators/[attestation_aggregation, validator_duties, validator_pool, slashing_protection, keystore_management],
|
||||
./sync/[sync_manager, sync_protocol, request_manager],
|
||||
./rpc/[beacon_api, config_api, debug_api, event_api, nimbus_api, node_api,
|
||||
validator_api],
|
||||
|
|
|
@ -23,11 +23,10 @@ import
|
|||
./networking/[eth2_network, eth2_discovery],
|
||||
./rpc/eth2_json_rpc_serialization,
|
||||
./beacon_node_types,
|
||||
./gossip_processing/attestation_aggregation,
|
||||
./nimbus_binary_common,
|
||||
./ssz/merkleization,
|
||||
./spec/eth2_apis/callsigs_types,
|
||||
./validators/[keystore_management, validator_pool, slashing_protection],
|
||||
./validators/[attestation_aggregation, keystore_management, validator_pool, slashing_protection],
|
||||
./eth/db/[kvstore, kvstore_sqlite3]
|
||||
|
||||
logScope: topics = "vc"
|
||||
|
|
|
@ -13,7 +13,8 @@ import
|
|||
../beacon_node_common,
|
||||
../networking/eth2_network,
|
||||
../validators/validator_duties,
|
||||
../consensus_object_pools/[blockchain_dag, exit_pool],
|
||||
../gossip_processing/gossip_validation,
|
||||
../consensus_object_pools/blockchain_dag,
|
||||
../spec/[crypto, digest, datatypes, validator, network],
|
||||
../spec/eth2_apis/callsigs_types,
|
||||
../ssz/merkleization,
|
||||
|
|
|
@ -0,0 +1,49 @@
|
|||
# beacon_chain
|
||||
# Copyright (c) 2019-2021 Status Research & Development GmbH
|
||||
# Licensed and distributed under either of
|
||||
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
|
||||
# * Apache v2 license (license terms in the root directory or at http://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].}
|
||||
|
||||
import
|
||||
std/options,
|
||||
../spec/[datatypes, digest, crypto, helpers],
|
||||
../consensus_object_pools/[spec_cache, attestation_pool]
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregation-selection
|
||||
func is_aggregator*(committee_len: uint64, slot_signature: ValidatorSig): bool =
|
||||
let
|
||||
modulo = max(1'u64, committee_len div TARGET_AGGREGATORS_PER_COMMITTEE)
|
||||
bytes_to_uint64(eth2digest(
|
||||
slot_signature.toRaw()).data.toOpenArray(0, 7)) mod modulo == 0
|
||||
|
||||
func is_aggregator*(epochRef: EpochRef, slot: Slot, index: CommitteeIndex,
|
||||
slot_signature: ValidatorSig): bool =
|
||||
let
|
||||
committee_len = get_beacon_committee_len(epochRef, slot, index)
|
||||
return is_aggregator(committee_len, slot_signature)
|
||||
|
||||
proc aggregate_attestations*(
|
||||
pool: AttestationPool, epochRef: EpochRef, slot: Slot, index: CommitteeIndex,
|
||||
validatorIndex: ValidatorIndex, slot_signature: ValidatorSig): Option[AggregateAndProof] =
|
||||
doAssert validatorIndex in get_beacon_committee(epochRef, slot, index)
|
||||
doAssert index.uint64 < get_committee_count_per_slot(epochRef)
|
||||
|
||||
# TODO for testing purposes, refactor this into the condition check
|
||||
# and just calculation
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregation-selection
|
||||
if not is_aggregator(epochRef, slot, index, slot_signature):
|
||||
return none(AggregateAndProof)
|
||||
|
||||
let maybe_slot_attestation = getAggregatedAttestation(pool, slot, index)
|
||||
if maybe_slot_attestation.isNone:
|
||||
return none(AggregateAndProof)
|
||||
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#construct-aggregate
|
||||
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.1/specs/phase0/validator.md#aggregateandproof
|
||||
some(AggregateAndProof(
|
||||
aggregator_index: validatorIndex.uint64,
|
||||
aggregate: maybe_slot_attestation.get,
|
||||
selection_proof: slot_signature))
|
|
@ -27,10 +27,9 @@ import
|
|||
attestation_pool, exit_pool],
|
||||
../eth1/eth1_monitor,
|
||||
../networking/eth2_network,
|
||||
".."/[beacon_node_common, beacon_node_types, nimbus_binary_common, version],
|
||||
../gossip_processing/attestation_aggregation,
|
||||
".."/[beacon_node_common, beacon_node_types, version],
|
||||
../ssz, ../ssz/sszdump, ../sync/sync_manager,
|
||||
./slashing_protection,
|
||||
./slashing_protection, ./attestation_aggregation,
|
||||
./validator_pool, ./keystore_management
|
||||
|
||||
# Metrics for tracking attestation and beacon block loss
|
||||
|
|
|
@ -18,7 +18,7 @@ import
|
|||
../beacon_chain/spec/[crypto, datatypes, digest, validator, state_transition,
|
||||
helpers, beaconstate, presets, network],
|
||||
../beacon_chain/[beacon_node_types, extras, beacon_clock],
|
||||
../beacon_chain/gossip_processing/attestation_aggregation,
|
||||
../beacon_chain/gossip_processing/gossip_validation,
|
||||
../beacon_chain/fork_choice/[fork_choice_types, fork_choice],
|
||||
../beacon_chain/consensus_object_pools/[block_quarantine, blockchain_dag, block_clearance, attestation_pool],
|
||||
# Test utilities
|
||||
|
|
|
@ -3,7 +3,7 @@
|
|||
import
|
||||
unittest, ./testutil,
|
||||
../beacon_chain/spec/[crypto, datatypes, network],
|
||||
../beacon_chain/gossip_processing/attestation_aggregation
|
||||
../beacon_chain/validators/attestation_aggregation
|
||||
|
||||
suiteReport "Honest validator":
|
||||
var forkDigest: ForkDigest
|
||||
|
|
Loading…
Reference in New Issue