keep REJECT/IGNORE of messages failing validation for libp2p scoring (#1676)

* keep REJECT/IGNORE status of messages failing validation for libp2p scoring

* fix test suite
This commit is contained in:
tersec 2020-09-18 11:53:09 +00:00 committed by GitHub
parent 000a0ecc52
commit e106549efe
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
11 changed files with 167 additions and 96 deletions

View File

@ -58,42 +58,50 @@ proc aggregate_attestations*(
selection_proof: slot_signature))
func check_attestation_block_slot(
pool: AttestationPool, attestationSlot: Slot, attestationBlck: BlockRef): Result[void, cstring] =
pool: AttestationPool, attestationSlot: Slot, attestationBlck: BlockRef):
Result[void, (ValidationResult, cstring)] =
# If we allow voting for very old blocks, the state transaction below will go
# nuts and keep processing empty slots
if not (attestationBlck.slot > pool.chainDag.finalizedHead.slot):
return err("Voting for already-finalized block")
const err_str: cstring = "Voting for already-finalized block"
return err((EVRESULT_IGNORE, err_str))
# we'll also cap it at 4 epochs which is somewhat arbitrary, but puts an
# upper bound on the processing done to validate the attestation
# TODO revisit with less arbitrary approach
if not (attestationSlot >= attestationBlck.slot):
return err("Voting for block that didn't exist at the time")
const err_str: cstring = "Voting for block that didn't exist at the time"
return err((EVRESULT_IGNORE, err_str))
if not ((attestationSlot - attestationBlck.slot) <= uint64(4 * SLOTS_PER_EPOCH)):
return err("Voting for very old block")
const err_str: cstring = "Voting for very old block"
return err((EVRESULT_IGNORE, err_str))
ok()
func check_propagation_slot_range(
data: AttestationData, wallTime: BeaconTime): Result[void, cstring] =
data: AttestationData, wallTime: BeaconTime):
Result[void, (ValidationResult, cstring)] =
let
futureSlot = (wallTime + MAXIMUM_GOSSIP_CLOCK_DISPARITY).toSlot()
if not futureSlot.afterGenesis or data.slot > futureSlot.slot:
return err("Attestation slot in the future")
const err_str: cstring = "Attestation slot in the future"
return err((EVRESULT_IGNORE, err_str))
let
pastSlot = (wallTime - MAXIMUM_GOSSIP_CLOCK_DISPARITY).toSlot()
if pastSlot.afterGenesis and
data.slot + ATTESTATION_PROPAGATION_SLOT_RANGE < pastSlot.slot:
return err("Attestation slot in the past")
const err_str: cstring = "Attestation slot in the past"
return err((EVRESULT_IGNORE, err_str))
ok()
func check_attestation_beacon_block(
pool: var AttestationPool, attestation: Attestation): Result[void, cstring] =
pool: var AttestationPool, attestation: Attestation):
Result[void, (ValidationResult, cstring)] =
# The block being voted for (attestation.data.beacon_block_root) passes
# validation.
# We rely on the chain DAG to have been validated, so check for the existence
@ -101,7 +109,8 @@ func check_attestation_beacon_block(
let attestationBlck = pool.chainDag.getRef(attestation.data.beacon_block_root)
if attestationBlck.isNil:
pool.quarantine.addMissing(attestation.data.beacon_block_root)
return err("Attestation block unknown")
const err_msg: cstring = "Attestation block unknown"
return err((EVRESULT_IGNORE, err_msg))
# Not in spec - check that rewinding to the state is sane
? check_attestation_block_slot(pool, attestation.data.slot, attestationBlck)
@ -109,7 +118,8 @@ func check_attestation_beacon_block(
ok()
func check_aggregation_count(
attestation: Attestation, singular: bool): Result[void, cstring] =
attestation: Attestation, singular: bool):
Result[void, (ValidationResult, cstring)] =
var onesCount = 0
# TODO a cleverer algorithm, along the lines of countOnes() in nim-stew
# But that belongs in nim-stew, since it'd break abstraction layers, to
@ -121,18 +131,20 @@ func check_aggregation_count(
onesCount += 1
if singular: # More than one ok
if onesCount > 1:
return err("Attestation has too many aggregation bits")
const err_str: cstring = "Attestation has too many aggregation bits"
return err((EVRESULT_REJECT, err_str))
else:
break # Found the one we needed
if onesCount < 1:
return err("Attestation has too few aggregation bits")
const err_str: cstring = "Attestation has too few aggregation bits"
return err((EVRESULT_REJECT, err_str))
ok()
func check_attestation_subnet(
epochRef: EpochRef, attestation: Attestation,
topicCommitteeIndex: uint64): Result[void, cstring] =
topicCommitteeIndex: uint64): Result[void, (ValidationResult, cstring)] =
let
expectedSubnet =
compute_subnet_for_attestation(
@ -140,7 +152,9 @@ func check_attestation_subnet(
attestation.data.slot, attestation.data.index.CommitteeIndex)
if expectedSubnet != topicCommitteeIndex:
return err("Attestation's committee index not for the correct subnet")
const err_str: cstring =
"Attestation's committee index not for the correct subnet"
return err((EVRESULT_REJECT, err_str))
ok()
@ -148,8 +162,12 @@ func check_attestation_subnet(
proc validateAttestation*(
pool: var AttestationPool,
attestation: Attestation, wallTime: BeaconTime,
topicCommitteeIndex: uint64): Result[HashSet[ValidatorIndex], cstring] =
? check_attestation_slot_target(attestation.data) # Not in spec - ignore
topicCommitteeIndex: uint64):
Result[HashSet[ValidatorIndex], (ValidationResult, cstring)] =
block:
let v = check_attestation_slot_target(attestation.data) # Not in spec
if v.isErr():
return err((EVRESULT_IGNORE, v.error))
# attestation.data.slot is within the last ATTESTATION_PROPAGATION_SLOT_RANGE
# slots (within a MAXIMUM_GOSSIP_CLOCK_DISPARITY allowance) -- i.e.
@ -180,12 +198,14 @@ proc validateAttestation*(
# Attestations might be aggregated eagerly or lazily; allow for both.
for validation in attestationEntry.validations:
if attestation.aggregation_bits.isSubsetOf(validation.aggregation_bits):
return err("Attestation already exists at slot") # [IGNORE]
const err_str: cstring = "Attestation already exists at slot"
return err((EVRESULT_IGNORE, err_str))
let tgtBlck = pool.chainDag.getRef(attestation.data.target.root)
if tgtBlck.isNil:
pool.quarantine.addMissing(attestation.data.target.root)
return err("Attestation target block unknown")
const err_str: cstring = "Attestation target block unknown"
return err((EVRESULT_IGNORE, err_str))
# The following rule follows implicitly from that we clear out any
# unviable blocks from the chain dag:
@ -214,22 +234,29 @@ proc validateAttestation*(
attesting_indices = get_attesting_indices(
epochRef, attestation.data, attestation.aggregation_bits)
# The signature of attestation is valid.
? is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef, attesting_indices, attestation, {})
# The signature of attestation is valid.
block:
let v = is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef, attesting_indices,
attestation, {})
if v.isErr():
return err((EVRESULT_REJECT, v.error))
ok(attesting_indices)
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#beacon_aggregate_and_proof
proc validateAggregate*(
pool: var AttestationPool,
signedAggregateAndProof: SignedAggregateAndProof,
wallTime: BeaconTime): Result[HashSet[ValidatorIndex], cstring] =
signedAggregateAndProof: SignedAggregateAndProof, wallTime: BeaconTime):
Result[HashSet[ValidatorIndex], (ValidationResult, cstring)] =
let
aggregate_and_proof = signedAggregateAndProof.message
aggregate = aggregate_and_proof.aggregate
? check_attestation_slot_target(aggregate.data) # Not in spec - ignore
block:
let v = check_attestation_slot_target(aggregate.data) # Not in spec
if v.isErr():
return err((EVRESULT_IGNORE, v.error))
# [IGNORE] aggregate.data.slot is within the last
# ATTESTATION_PROPAGATION_SLOT_RANGE slots (with a
@ -277,14 +304,16 @@ proc validateAggregate*(
let tgtBlck = pool.chainDag.getRef(aggregate.data.target.root)
if tgtBlck.isNil:
pool.quarantine.addMissing(aggregate.data.target.root)
return err("Aggregate target block unknown")
const err_str: cstring = "Aggregate target block unknown"
return err((EVRESULT_IGNORE, err_str))
let epochRef = pool.chainDag.getEpochRef(tgtBlck, aggregate.data.target.epoch)
if not is_aggregator(
epochRef, aggregate.data.slot, aggregate.data.index.CommitteeIndex,
aggregate_and_proof.selection_proof):
return err("Incorrect aggregator")
const err_str: cstring = "Incorrect aggregator"
return err((EVRESULT_REJECT, err_str))
# [REJECT] The aggregator's validator index is within the committee -- i.e.
# aggregate_and_proof.aggregator_index in get_beacon_committee(state,
@ -292,14 +321,16 @@ proc validateAggregate*(
if aggregate_and_proof.aggregator_index.ValidatorIndex notin
get_beacon_committee(
epochRef, aggregate.data.slot, aggregate.data.index.CommitteeIndex):
return err("Aggregator's validator index not in committee")
const err_str: cstring = "Aggregator's validator index not in committee"
return err((EVRESULT_REJECT, err_str))
# [REJECT] The aggregate_and_proof.selection_proof is a valid signature of the
# aggregate.data.slot by the validator with index
# aggregate_and_proof.aggregator_index.
# get_slot_signature(state, aggregate.data.slot, privkey)
if aggregate_and_proof.aggregator_index >= epochRef.validator_keys.lenu64:
return err("Invalid aggregator_index")
const err_str: cstring = "Invalid aggregator_index"
return err((EVRESULT_REJECT, err_str))
let
fork = pool.chainDag.headState.data.data.fork
@ -309,21 +340,28 @@ proc validateAggregate*(
fork, genesis_validators_root, aggregate.data.slot,
epochRef.validator_keys[aggregate_and_proof.aggregator_index],
aggregate_and_proof.selection_proof):
return err("Selection_proof signature verification failed")
const err_str: cstring = "Selection_proof signature verification failed"
return err((EVRESULT_REJECT, err_str))
# [REJECT] The aggregator signature, signed_aggregate_and_proof.signature, is valid.
if not verify_aggregate_and_proof_signature(
fork, genesis_validators_root, aggregate_and_proof,
epochRef.validator_keys[aggregate_and_proof.aggregator_index],
signed_aggregate_and_proof.signature):
return err("signed_aggregate_and_proof signature verification failed")
const err_str: cstring =
"signed_aggregate_and_proof signature verification failed"
return err((EVRESULT_REJECT, err_str))
let attesting_indices = get_attesting_indices(
epochRef, aggregate.data, aggregate.aggregation_bits)
# [REJECT] The signature of aggregate is valid.
? is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef, attesting_indices, aggregate, {})
block:
let v = is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef, attesting_indices,
aggregate, {})
if v.isErr():
return err((EVRESULT_REJECT, v.error))
# The following rule follows implicitly from that we clear out any
# unviable blocks from the chain dag:

View File

@ -813,32 +813,32 @@ proc installMessageValidators(node: BeaconNode) =
node.network.addValidator(
getAttestationTopic(node.forkDigest, ci),
# This proc needs to be within closureScope; don't lift out of loop.
proc(attestation: Attestation): bool =
proc(attestation: Attestation): ValidationResult =
node.processor[].attestationValidator(attestation, ci))
node.network.addValidator(
getAggregateAndProofsTopic(node.forkDigest),
proc(signedAggregateAndProof: SignedAggregateAndProof): bool =
proc(signedAggregateAndProof: SignedAggregateAndProof): ValidationResult =
node.processor[].aggregateValidator(signedAggregateAndProof))
node.network.addValidator(
node.topicBeaconBlocks,
proc (signedBlock: SignedBeaconBlock): bool =
proc (signedBlock: SignedBeaconBlock): ValidationResult =
node.processor[].blockValidator(signedBlock))
node.network.addValidator(
getAttesterSlashingsTopic(node.forkDigest),
proc (attesterSlashing: AttesterSlashing): bool =
proc (attesterSlashing: AttesterSlashing): ValidationResult =
node.processor[].attesterSlashingValidator(attesterSlashing))
node.network.addValidator(
getProposerSlashingsTopic(node.forkDigest),
proc (proposerSlashing: ProposerSlashing): bool =
proc (proposerSlashing: ProposerSlashing): ValidationResult =
node.processor[].proposerSlashingValidator(proposerSlashing))
node.network.addValidator(
getVoluntaryExitsTopic(node.forkDigest),
proc (voluntaryExit: VoluntaryExit): bool =
proc (voluntaryExit: VoluntaryExit): ValidationResult =
node.processor[].voluntaryExitValidator(voluntaryExit))
proc stop*(node: BeaconNode) =

View File

@ -38,7 +38,7 @@ func getOrResolve*(dag: ChainDAGRef, quarantine: var QuarantineRef, root: Eth2Di
proc addRawBlock*(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock, onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] {.gcsafe.}
): Result[BlockRef, (ValidationResult, BlockError)] {.gcsafe.}
proc addResolvedBlock(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
@ -124,7 +124,7 @@ proc addRawBlock*(
dag: var ChainDAGRef, quarantine: var QuarantineRef,
signedBlock: SignedBeaconBlock,
onBlockAdded: OnBlockAdded
): Result[BlockRef, BlockError] =
): Result[BlockRef, (ValidationResult, BlockError)] =
## Try adding a block to the chain, verifying first that it passes the state
## transition function.
@ -140,8 +140,9 @@ proc addRawBlock*(
# We should not call the block added callback for blocks that already
# existed in the pool, as that may confuse consumers such as the fork
# choice.
return err Duplicate
# choice. While the validation result won't be accessed, it's IGNORE,
# according to the spec.
return err((EVRESULT_IGNORE, Duplicate))
quarantine.missing.del(blockRoot)
@ -154,7 +155,9 @@ proc addRawBlock*(
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
return err Unviable
# Doesn't correspond to any specific validation condition, and still won't
# be used, but certainly would be IGNORE.
return err((EVRESULT_IGNORE, Unviable))
let parent = dag.blocks.getOrDefault(blck.parent_root)
@ -165,7 +168,7 @@ proc addRawBlock*(
notice "Invalid block slot",
parentBlock = shortLog(parent)
return err Invalid
return err((EVRESULT_REJECT, Invalid))
if (parent.slot < dag.finalizedHead.slot) or
(parent.slot == dag.finalizedHead.slot and
@ -182,7 +185,7 @@ proc addRawBlock*(
finalizedHead = shortLog(dag.finalizedHead),
tail = shortLog(dag.tail)
return err Unviable
return err((EVRESULT_IGNORE, Unviable))
# The block might have been in either of `orphans` or `missing` - we don't
# want any more work done on its behalf
@ -210,7 +213,7 @@ proc addRawBlock*(
cache, dag.updateFlags + {slotProcessed}, restore):
notice "Invalid block"
return err Invalid
return err((EVRESULT_REJECT, Invalid))
# Careful, clearanceState.data has been updated but not blck - we need to
# create the BlockRef first!
@ -238,7 +241,7 @@ proc addRawBlock*(
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err MissingParent
return err((EVRESULT_IGNORE, MissingParent))
# This is an unresolved block - put its parent on the missing list for now...
# TODO if we receive spam blocks, one heurestic to implement might be to wait
@ -257,13 +260,14 @@ proc addRawBlock*(
orphans = quarantine.orphans.len,
missing = quarantine.missing.len
return err MissingParent
return err((EVRESULT_IGNORE, MissingParent))
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#beacon_block
proc isValidBeaconBlock*(
dag: ChainDAGRef, quarantine: var QuarantineRef,
signed_beacon_block: SignedBeaconBlock, current_slot: Slot,
flags: UpdateFlags): Result[void, BlockError] =
flags: UpdateFlags):
Result[void, (ValidationResult, BlockError)] =
logScope:
topics = "clearance valid_blck"
received_block = shortLog(signed_beacon_block.message)
@ -281,14 +285,14 @@ proc isValidBeaconBlock*(
if not (signed_beacon_block.message.slot <= current_slot + 1):
debug "block is from a future slot",
current_slot
return err(Invalid)
return err((EVRESULT_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(Invalid)
return err((EVRESULT_IGNORE, Invalid))
# [IGNORE] The block is the first block with valid signature received for the
# proposer for the slot, signed_beacon_block.message.slot.
@ -328,7 +332,7 @@ proc isValidBeaconBlock*(
debug "block isn't first block with valid signature received for the proposer",
blckRef = slotBlockRef,
existing_block = shortLog(blck.message)
return err(Invalid)
return err((EVRESULT_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
@ -347,7 +351,7 @@ proc isValidBeaconBlock*(
current_slot = shortLog(current_slot)
if not quarantine.add(dag, signed_beacon_block):
debug "Block quarantine full"
return err(MissingParent)
return err((EVRESULT_IGNORE, MissingParent))
# [REJECT] The current finalized_checkpoint is an ancestor of block -- i.e.
# get_ancestor(store, block.parent_root,
@ -360,11 +364,11 @@ proc isValidBeaconBlock*(
if ancestor.isNil:
debug "couldn't find ancestor block"
return err(Invalid)
return err((EVRESULT_IGNORE, Invalid)) # might just not have received block
if not (finalized_checkpoint.root in [ancestor.root, Eth2Digest()]):
debug "block not descendent of finalized block"
return err(Invalid)
return err((EVRESULT_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
@ -377,13 +381,13 @@ proc isValidBeaconBlock*(
if proposer.isNone:
notice "cannot compute proposer for message"
return err(Invalid)
return err((EVRESULT_IGNORE, Invalid)) # basically an internal issue
if proposer.get()[0] !=
ValidatorIndex(signed_beacon_block.message.proposer_index):
debug "block had unexpected proposer",
expected_proposer = proposer.get()[0]
return err(Invalid)
return err((EVRESULT_REJECT, Invalid))
# [REJECT] The proposer signature, signed_beacon_block.signature, is valid
# with respect to the proposer_index pubkey.
@ -397,6 +401,6 @@ proc isValidBeaconBlock*(
debug "block failed signature verification",
signature = shortLog(signed_beacon_block.signature)
return err(Invalid)
return err((EVRESULT_REJECT, Invalid))
ok()

View File

@ -1278,7 +1278,8 @@ proc subscribe*(node: Eth2Node, topic: string) {.async.} =
proc addValidator*[MsgType](node: Eth2Node,
topic: string,
msgValidator: proc(msg: MsgType): bool {.gcsafe.} ) =
msgValidator: proc(msg: MsgType):
ValidationResult {.gcsafe.} ) =
# Validate messages as soon as subscribed
proc execValidator(
topic: string, message: GossipMsg): Future[bool] {.async.} =
@ -1287,7 +1288,7 @@ proc addValidator*[MsgType](node: Eth2Node,
try:
let decompressed = snappy.decode(message.data, GOSSIP_MAX_SIZE)
if decompressed.len > 0:
return msgValidator SSZ.decode(decompressed, MsgType)
return msgValidator(SSZ.decode(decompressed, MsgType)) == EVRESULT_ACCEPT
else:
# TODO penalize peer?
debug "Failed to decompress gossip payload"

View File

@ -90,9 +90,9 @@ proc updateHead*(self: var Eth2Processor, wallSlot: Slot): BlockRef =
proc dumpBlock[T](
self: Eth2Processor, signedBlock: SignedBeaconBlock,
res: Result[T, BlockError]) =
res: Result[T, (ValidationResult, BlockError)]) =
if self.config.dumpEnabled and res.isErr:
case res.error
case res.error[1]
of Invalid:
dump(
self.config.dumpDirInvalid, signedBlock)
@ -140,7 +140,7 @@ proc storeBlock(
# However this block was before the last finalized epoch and so its parent
# was pruned from the ForkChoice.
if blck.isErr:
return err(blck.error)
return err(blck.error[1])
beacon_store_block_duration_seconds.observe((Moment.now() - start).milliseconds.float64 / 1000)
return ok()
@ -232,7 +232,7 @@ proc processBlock(self: var Eth2Processor, entry: BlockEntry) =
proc blockValidator*(
self: var Eth2Processor,
signedBlock: SignedBeaconBlock): bool =
signedBlock: SignedBeaconBlock): ValidationResult =
logScope:
signedBlock = shortLog(signedBlock.message)
blockRoot = shortLog(signedBlock.root)
@ -242,7 +242,7 @@ proc blockValidator*(
(afterGenesis, wallSlot) = wallTime.toSlot()
if not afterGenesis:
return false
return EVRESULT_IGNORE # not an issue with block, so don't penalize
logScope: wallSlot
@ -253,7 +253,7 @@ proc blockValidator*(
# already-seen data, but it is fairly aggressive about forgetting about
# what it has seen already
debug "Dropping already-seen gossip block", delay
return false
return EVRESULT_IGNORE # "[IGNORE] The block is the first block with ..."
# Start of block processing - in reality, we have already gone through SSZ
# decoding at this stage, which may be significant
@ -265,7 +265,7 @@ proc blockValidator*(
self.dumpBlock(signedBlock, blck)
if not blck.isOk:
return false
return blck.error[0]
beacon_blocks_received.inc()
beacon_block_delay.observe(float(milliseconds(delay)) / 1000.0)
@ -279,12 +279,12 @@ proc blockValidator*(
traceAsyncErrors self.blocksQueue.addLast(
BlockEntry(v: SyncBlock(blk: signedBlock)))
true
EVRESULT_ACCEPT
proc attestationValidator*(
self: var Eth2Processor,
attestation: Attestation,
committeeIndex: uint64): bool =
committeeIndex: uint64): ValidationResult =
logScope:
attestation = shortLog(attestation)
committeeIndex
@ -295,7 +295,7 @@ proc attestationValidator*(
if not afterGenesis:
notice "Attestation before genesis"
return false
return EVRESULT_IGNORE
logScope: wallSlot
@ -306,7 +306,7 @@ proc attestationValidator*(
attestation, wallTime, committeeIndex)
if v.isErr():
debug "Dropping attestation", err = v.error()
return false
return v.error[0]
beacon_attestations_received.inc()
beacon_attestation_delay.observe(float(milliseconds(delay)) / 1000.0)
@ -321,11 +321,11 @@ proc attestationValidator*(
traceAsyncErrors self.attestationsQueue.addLast(
AttestationEntry(v: attestation, attesting_indices: v.get()))
true
EVRESULT_ACCEPT
proc aggregateValidator*(
self: var Eth2Processor,
signedAggregateAndProof: SignedAggregateAndProof): bool =
self: var Eth2Processor,
signedAggregateAndProof: SignedAggregateAndProof): ValidationResult =
logScope:
aggregate = shortLog(signedAggregateAndProof.message.aggregate)
signature = shortLog(signedAggregateAndProof.signature)
@ -336,7 +336,7 @@ proc aggregateValidator*(
if not afterGenesis:
notice "Aggregate before genesis"
return false
return EVRESULT_IGNORE
logScope: wallSlot
@ -349,7 +349,7 @@ proc aggregateValidator*(
signedAggregateAndProof, wallTime)
if v.isErr:
debug "Dropping aggregate", err = v.error
return false
return v.error[0]
beacon_aggregates_received.inc()
beacon_aggregate_delay.observe(float(milliseconds(delay)) / 1000.0)
@ -365,44 +365,52 @@ proc aggregateValidator*(
v: signedAggregateAndProof.message.aggregate,
attesting_indices: v.get()))
true
EVRESULT_ACCEPT
proc attesterSlashingValidator*(
self: var Eth2Processor, attesterSlashing: AttesterSlashing): bool =
self: var Eth2Processor, attesterSlashing: AttesterSlashing):
ValidationResult =
logScope:
attesterSlashing = shortLog(attesterSlashing)
let v = self.exitPool[].validateAttesterSlashing(attesterSlashing)
if v.isErr:
debug "Dropping attester slashing", err = v.error
return false
return v.error[0]
beacon_attester_slashings_received.inc()
EVRESULT_ACCEPT
proc proposerSlashingValidator*(
self: var Eth2Processor, proposerSlashing: ProposerSlashing): bool =
self: var Eth2Processor, proposerSlashing: ProposerSlashing):
ValidationResult =
logScope:
proposerSlashing = shortLog(proposerSlashing)
let v = self.exitPool[].validateProposerSlashing(proposerSlashing)
if v.isErr:
debug "Dropping proposer slashing", err = v.error
return false
return v.error[0]
beacon_proposer_slashings_received.inc()
EVRESULT_ACCEPT
proc voluntaryExitValidator*(
self: var Eth2Processor, voluntaryExit: VoluntaryExit): bool =
self: var Eth2Processor, voluntaryExit: VoluntaryExit): ValidationResult =
logScope:
voluntaryExit = shortLog(voluntaryExit)
let v = self.exitPool[].validateVoluntaryExit(voluntaryExit)
if v.isErr:
debug "Dropping voluntary exit", err = v.error
return false
return v.error[0]
beacon_voluntary_exits_received.inc()
EVRESULT_ACCEPT
proc runQueueProcessingLoop*(self: ref Eth2Processor) {.async.} =
# Blocks in eth2 arrive on a schedule for every slot:
#

View File

@ -72,7 +72,7 @@ func getVoluntaryExitsForBlock*(pool: var ExitPool):
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#attester_slashing
proc validateAttesterSlashing*(
pool: var ExitPool, attesterSlashing: AttesterSlashing):
Result[bool, cstring] =
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),
@ -104,12 +104,14 @@ proc validateAttesterSlashing*(
let tgtBlck_1 = pool.chainDag.getRef(attestation_1.data.target.root)
if tgtBlck_1.isNil:
pool.quarantine.addMissing(attestation_1.data.target.root)
return err("Attestation 1 target block unknown")
const err_str: cstring = "Attestation 1 target block unknown"
return err((EVRESULT_IGNORE, err_str))
let tgtBlck_2 = pool.chainDag.getRef(attestation_2.data.target.root)
if tgtBlck_2.isNil:
pool.quarantine.addMissing(attestation_2.data.target.root)
return err("Attestation 2 target block unknown")
const err_str: cstring = "Attestation 2 target block unknown"
return err((EVRESULT_IGNORE, err_str))
let
epochRef_1 = pool.chainDag.getEpochRef(
@ -122,11 +124,18 @@ proc validateAttesterSlashing*(
if not is_slashable_attestation_data(
attestation_1.data, attestation_2.data):
return err("Attestation data not slashable")
? is_valid_indexed_attestation(
const err_str: cstring = "Attestation data not slashable"
return err((EVRESULT_REJECT, err_str))
block:
let v = is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef_1, attestation_1, {})
? is_valid_indexed_attestation(
if v.isErr():
return err((EVRESULT_REJECT, v.error))
block:
let v = is_valid_indexed_attestation(
fork, genesis_validators_root, epochRef_2, attestation_2, {})
if v.isErr():
return err((EVRESULT_REJECT, v.error))
pool.attester_slashings.addExitMessage(
attesterSlashing, MAX_ATTESTER_SLASHINGS)
@ -136,7 +145,7 @@ proc validateAttesterSlashing*(
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#proposer_slashing
proc validateProposerSlashing*(
pool: var ExitPool, proposerSlashing: ProposerSlashing):
Result[bool, cstring] =
Result[bool, (ValidationResult, cstring)] =
# [IGNORE] The proposer slashing is the first valid proposer slashing
# received for the proposer with index
# proposer_slashing.signed_header_1.message.proposer_index.
@ -158,7 +167,8 @@ proc validateProposerSlashing*(
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#voluntary_exit
proc validateVoluntaryExit*(
pool: var ExitPool, voluntaryExit: VoluntaryExit): Result[bool, cstring] =
pool: var ExitPool, voluntaryExit: VoluntaryExit):
Result[bool, (ValidationResult, cstring)] =
# [IGNORE] The voluntary exit is the first valid voluntary exit received for
# the validator with index signed_voluntary_exit.message.validator_index.

View File

@ -449,6 +449,12 @@ type
stabilitySubnet*: uint64
stabilitySubnetExpirationEpoch*: Epoch
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/p2p-interface.md#topics-and-messages
ValidationResult* = enum
EVRESULT_ACCEPT = 0
EVRESULT_REJECT = 1
EVRESULT_IGNORE = 2
func shortValidatorKey*(state: BeaconState, validatorIdx: int): string =
($state.validators[validatorIdx].pubkey)[0..7]

View File

@ -5,6 +5,8 @@
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or https://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
# Standard library
std/[unittest, os],

View File

@ -5,6 +5,8 @@
# * MIT license ([LICENSE-MIT](LICENSE-MIT) or https://opensource.org/licenses/MIT)
# at your option. This file may not be copied, modified, or distributed except according to those terms.
{.used.}
import
# Standard library
std/unittest,

View File

@ -304,7 +304,7 @@ suiteReport "Attestation pool processing" & preset():
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
doAssert: b10Add_clone.error == Duplicate
doAssert: b10Add_clone.error == (EVRESULT_IGNORE, Duplicate)
wrappedTimedTest "Trying to add a duplicate block from an old pruned epoch is tagged as an error":
# Note: very sensitive to stack usage
@ -388,7 +388,7 @@ suiteReport "Attestation pool processing" & preset():
# Callback add to fork choice if valid
pool[].addForkChoice(epochRef, blckRef, signedBlock.message, blckRef.slot)
doAssert: b10Add_clone.error == Duplicate
doAssert: b10Add_clone.error == (EVRESULT_IGNORE, Duplicate)
suiteReport "Attestation validation " & preset():

View File

@ -207,7 +207,7 @@ suiteReport "Block pool processing" & preset():
wrappedTimedTest "Reverse order block add & get" & preset():
let missing = dag.addRawBlock(quarantine, b2, nil)
check: missing.error == MissingParent
check: missing.error == (EVRESULT_IGNORE, MissingParent)
check:
dag.get(b2.root).isNone() # Unresolved, shouldn't show up
@ -253,7 +253,7 @@ suiteReport "Block pool processing" & preset():
b11 = dag.addRawBlock(quarantine, b1, nil)
check:
b11.error == Duplicate
b11.error == (EVRESULT_IGNORE, Duplicate)
not b10[].isNil
wrappedTimedTest "updateHead updates head and headState" & preset():
@ -388,7 +388,7 @@ suiteReport "chain DAG finalization tests" & preset():
# The late block is a block whose parent was finalized long ago and thus
# is no longer a viable head candidate
let status = dag.addRawBlock(quarantine, lateBlock, nil)
check: status.error == Unviable
check: status.error == (EVRESULT_IGNORE, Unviable)
let
dag2 = init(ChainDAGRef, defaultRuntimePreset, db)