diff --git a/beacon_chain/beacon_node.nim b/beacon_chain/beacon_node.nim index 4a014d849..f1551f51c 100644 --- a/beacon_chain/beacon_node.nim +++ b/beacon_chain/beacon_node.nim @@ -25,7 +25,7 @@ import spec/[datatypes, digest, crypto, beaconstate, helpers, network, presets], spec/state_transition, conf, time, beacon_chain_db, validator_pool, extras, - attestation_pool, eth2_network, eth2_discovery, + attestation_pool, exit_pool, eth2_network, eth2_discovery, beacon_node_common, beacon_node_types, block_pools/[spec_cache, chain_dag, quarantine, clearance, block_pools_types], nimbus_binary_common, network_metadata, @@ -249,6 +249,7 @@ proc init*(T: type BeaconNode, topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest) network = createEth2Node(rng, conf, enrForkId) attestationPool = newClone(AttestationPool.init(chainDag, quarantine)) + exitPool = newClone(ExitPool.init(chainDag, quarantine)) var res = BeaconNode( nickname: nickname, graffitiBytes: if conf.graffiti.isSome: conf.graffiti.get.GraffitiBytes @@ -261,6 +262,7 @@ proc init*(T: type BeaconNode, chainDag: chainDag, quarantine: quarantine, attestationPool: attestationPool, + exitPool: exitPool, mainchainMonitor: mainchainMonitor, beaconClock: BeaconClock.init(chainDag.headState.data.data), rpcServer: rpcServer, @@ -272,7 +274,7 @@ proc init*(T: type BeaconNode, proc getWallTime(): BeaconTime = res.beaconClock.now() res.processor = Eth2Processor.new( - conf, chainDag, attestationPool, quarantine, getWallTime) + conf, chainDag, attestationPool, exitPool, quarantine, getWallTime) res.requestManager = RequestManager.init( network, res.processor.blocksQueue) @@ -397,6 +399,7 @@ proc addMessageHandlers(node: BeaconNode): Future[void] = allFutures( # As a side-effect, this gets the attestation subnets too. node.network.subscribe(node.topicBeaconBlocks), + node.network.subscribe(getAttesterSlashingsTopic(node.forkDigest)), node.getAttestationHandlers() ) @@ -773,6 +776,21 @@ proc installMessageValidators(node: BeaconNode) = proc (signedBlock: SignedBeaconBlock): bool = node.processor[].blockValidator(signedBlock)) + node.network.addValidator( + getAttesterSlashingsTopic(node.forkDigest), + proc (attesterSlashing: AttesterSlashing): bool = + node.processor[].attesterSlashingValidator(attesterSlashing)) + + node.network.addValidator( + getProposerSlashingsTopic(node.forkDigest), + proc (proposerSlashing: ProposerSlashing): bool = + node.processor[].proposerSlashingValidator(proposerSlashing)) + + node.network.addValidator( + getVoluntaryExitsTopic(node.forkDigest), + proc (voluntaryExit: VoluntaryExit): bool = + node.processor[].voluntaryExitValidator(voluntaryExit)) + proc removeMessageHandlers(node: BeaconNode) = var unsubscriptions: seq[Future[void]] diff --git a/beacon_chain/beacon_node_common.nim b/beacon_chain/beacon_node_common.nim index cc8b47a01..53caa92cc 100644 --- a/beacon_chain/beacon_node_common.nim +++ b/beacon_chain/beacon_node_common.nim @@ -43,6 +43,7 @@ type chainDag*: ChainDAGRef quarantine*: QuarantineRef attestationPool*: ref AttestationPool + exitPool*: ref ExitPool mainchainMonitor*: MainchainMonitor beaconClock*: BeaconClock rpcServer*: RpcServer diff --git a/beacon_chain/beacon_node_types.nim b/beacon_chain/beacon_node_types.nim index e82fc563e..e7ce86eb6 100644 --- a/beacon_chain/beacon_node_types.nim +++ b/beacon_chain/beacon_node_types.nim @@ -1,7 +1,7 @@ {.push raises: [Defect].} import - deques, tables, streams, + std/[deques, tables, streams], stew/endians2, spec/[datatypes, crypto], block_pools/block_pools_types, @@ -64,6 +64,22 @@ type forkChoice*: ForkChoice + ExitPool* = object + ## The exit pool tracks attester slashings, proposer slashings, and + ## voluntary exits that could be added to a proposed block. + + attester_slashings*: Deque[AttesterSlashing] ## \ + ## Not a function of chain DAG branch; just used as a FIFO queue for blocks + + proposer_slashings*: Deque[ProposerSlashing] ## \ + ## Not a function of chain DAG branch; just used as a FIFO queue for blocks + + voluntary_exits*: Deque[VoluntaryExit] ## \ + ## Not a function of chain DAG branch; just used as a FIFO queue for blocks + + chainDag*: ChainDAGRef + quarantine*: QuarantineRef + # ############################################# # # Validator Pool diff --git a/beacon_chain/eth2_processor.nim b/beacon_chain/eth2_processor.nim index b98885a1e..fb1340b1f 100644 --- a/beacon_chain/eth2_processor.nim +++ b/beacon_chain/eth2_processor.nim @@ -4,7 +4,7 @@ import chronicles, chronicles/chronos_tools, chronos, metrics, ./spec/[crypto, datatypes, digest], ./block_pools/[clearance, chain_dag], - ./attestation_aggregation, + ./attestation_aggregation, ./exit_pool, ./beacon_node_types, ./attestation_pool, ./time, ./conf, ./sszdump @@ -15,6 +15,12 @@ declareCounter beacon_aggregates_received, "Number of beacon chain aggregate attestations received by this peer" declareCounter beacon_blocks_received, "Number of beacon chain blocks received by this peer" +declareCounter beacon_attester_slashings_received, + "Number of beacon chain attester slashings received by this peer" +declareCounter beacon_proposer_slashings_received, + "Number of beacon chain proposer slashings received by this peer" +declareCounter beacon_voluntary_exits_received, + "Number of beacon chain voluntary exits received by this peer" const delayBuckets = [2.0, 4.0, 6.0, 8.0, 10.0, 12.0, 14.0, Inf] @@ -54,6 +60,7 @@ type getWallTime*: GetWallTimeFn chainDag*: ChainDAGRef attestationPool*: ref AttestationPool + exitPool: ref ExitPool quarantine*: QuarantineRef blocksQueue*: AsyncQueue[BlockEntry] @@ -367,6 +374,42 @@ proc aggregateValidator*( true +proc attesterSlashingValidator*( + self: var Eth2Processor, attesterSlashing: AttesterSlashing): bool = + logScope: + attesterSlashing = shortLog(attesterSlashing) + + let v = self.exitPool[].validateAttesterSlashing(attesterSlashing) + if v.isErr: + debug "Dropping attester slashing", err = v.error + return false + + beacon_attester_slashings_received.inc() + +proc proposerSlashingValidator*( + self: var Eth2Processor, proposerSlashing: ProposerSlashing): bool = + logScope: + proposerSlashing = shortLog(proposerSlashing) + + let v = self.exitPool[].validateProposerSlashing(proposerSlashing) + if v.isErr: + debug "Dropping proposer slashing", err = v.error + return false + + beacon_proposer_slashings_received.inc() + +proc voluntaryExitValidator*( + self: var Eth2Processor, voluntaryExit: VoluntaryExit): bool = + logScope: + voluntaryExit = shortLog(voluntaryExit) + + let v = self.exitPool[].validateVoluntaryExit(voluntaryExit) + if v.isErr: + debug "Dropping voluntary exit", err = v.error + return false + + beacon_voluntary_exits_received.inc() + proc runQueueProcessingLoop*(self: ref Eth2Processor) {.async.} = # Blocks in eth2 arrive on a schedule for every slot: # @@ -403,6 +446,7 @@ proc new*(T: type Eth2Processor, config: BeaconNodeConf, chainDag: ChainDAGRef, attestationPool: ref AttestationPool, + exitPool: ref ExitPool, quarantine: QuarantineRef, getWallTime: GetWallTimeFn): ref Eth2Processor = (ref Eth2Processor)( @@ -410,6 +454,7 @@ proc new*(T: type Eth2Processor, getWallTime: getWallTime, chainDag: chainDag, attestationPool: attestationPool, + exitPool: exitPool, quarantine: quarantine, blocksQueue: newAsyncQueue[BlockEntry](1), aggregatesQueue: newAsyncQueue[AggregateEntry](MAX_ATTESTATIONS.int), diff --git a/beacon_chain/exit_pool.nim b/beacon_chain/exit_pool.nim new file mode 100644 index 000000000..4b2d48bb1 --- /dev/null +++ b/beacon_chain/exit_pool.nim @@ -0,0 +1,168 @@ +# beacon_chain +# Copyright (c) 2020 Status Research & Development GmbH +# Licensed and distributed under either of +# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). +# * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0). +# at your option. This file may not be copied, modified, or distributed except according to those terms. + +{.push raises: [Defect].} + +import + # Standard libraries + std/[deques, options, sequtils, sets], + # Status libraries + chronicles, json_serialization/std/sets as jsonSets, + # Internal + ./spec/[datatypes, crypto, state_transition_block], + ./block_pools/[chain_dag, clearance, quarantine, spec_cache], + ./beacon_node_types + +export beacon_node_types, sets + +logScope: topics = "slashpool" + +proc init*( + T: type ExitPool, chainDag: ChainDAGRef, quarantine: QuarantineRef): T = + ## Initialize an ExitPool from the chainDag `headState` + T( + attester_slashings: + initDeque[AttesterSlashing](initialSize = MAX_ATTESTER_SLASHINGS.int), + proposer_slashings: + initDeque[ProposerSlashing](initialSize = MAX_PROPOSER_SLASHINGS.int), + voluntary_exits: + initDeque[VoluntaryExit](initialSize = MAX_VOLUNTARY_EXITS.int), + chainDag: chainDag, + quarantine: quarantine + ) + +func addExitMessage(subpool: var auto, exitMessage, bound: auto) = + # Prefer newer to older exit message + while subpool.lenu64 >= bound: + discard subpool.popFirst() + + subpool.addLast(exitMessage) + doAssert subpool.lenu64 <= bound + +proc getAttesterSlashingsForBlock*(pool: var ExitPool): + seq[AttesterSlashing] = + ## Retrieve attester slashings that may be added to a new block at the slot + ## of the given state + logScope: pcs = "retrieve_attester_slashing" + + for i in 0 ..< MAX_ATTESTER_SLASHINGS: + if pool.attester_slashings.len == 0: + break + result.add pool.attester_slashings.popFirst() + + doAssert result.lenu64 <= MAX_ATTESTER_SLASHINGS + +proc getProposerSlashingsForBlock*(pool: var ExitPool): + seq[ProposerSlashing] = + ## Retrieve proposer slashings that may be added to a new block at the slot + ## of the given state + logScope: pcs = "retrieve_proposer_slashing" + + for i in 0 ..< MAX_PROPOSER_SLASHINGS: + if pool.proposer_slashings.len == 0: + break + result.add pool.proposer_slashings.popFirst() + + doAssert result.lenu64 <= MAX_PROPOSER_SLASHINGS + +proc getVoluntaryExitsForBlock*(pool: var ExitPool): + seq[VoluntaryExit] = + ## Retrieve voluntary exits that may be added to a new block at the slot + ## of the given state + logScope: pcs = "retrieve_voluntary_exit" + + for i in 0 ..< MAX_VOLUNTARY_EXITS: + if pool.voluntary_exits.len == 0: + break + result.add pool.voluntary_exits.popFirst() + + doAssert result.lenu64 <= MAX_VOLUNTARY_EXITS + +# 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] = + # [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))). + let + attestation_1 = attester_slashing.attestation_1 + attestation_2 = attester_slashing.attestation_2 + # TODO sequtils2 should be able to make this more reasonable, from asSeq on + # down + attesting_indices_1 = + toHashSet(mapIt(attestation_1.attesting_indices.asSeq, it.ValidatorIndex)) + attesting_indices_2 = + toHashSet(mapIt(attestation_1.attesting_indices.asSeq, it.ValidatorIndex)) + attester_slashed_indices = attesting_indices_1 * attesting_indices_2 + # TODO this arguably ties in with slashing protection in general + + # [REJECT] All of the conditions within process_attester_slashing pass + # validation. + # This is similar to process_attester_slashing, but both cut-down (it doesn't + # have the loop over attesting indices) and using EpochRef caches, so there's + # no real overlap in code terms with process_proposer_slashing(). + block: + 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") + + 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") + + let + epochRef_1 = pool.chainDag.getEpochRef( + tgtBlck_1, attestation_1.data.target.epoch) + epochRef_2 = pool.chainDag.getEpochRef( + tgtBlck_2, attestation_2.data.target.epoch) + fork = pool.chainDag.headState.data.data.fork + genesis_validators_root = + pool.chainDag.headState.data.data.genesis_validators_root + + if not is_slashable_attestation_data( + attestation_1.data, attestation_2.data): + return err("Attestation data not slashable") + ? is_valid_indexed_attestation( + fork, genesis_validators_root, epochRef_1, attestation_1, {}) + ? is_valid_indexed_attestation( + fork, genesis_validators_root, epochRef_2, attestation_2, {}) + + pool.attester_slashings.addExitMessage( + attesterSlashing, MAX_ATTESTER_SLASHINGS) + + ok(true) + +# 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] = + # [IGNORE] The proposer slashing is the first valid proposer slashing + # received for the proposer with index + # proposer_slashing.signed_header_1.message.proposer_index. + + # [REJECT] All of the conditions within process_proposer_slashing pass validation. + + # TODO not called yet, so vacuousness is fine + + ok(true) + +# 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] = + # [IGNORE] The voluntary exit is the first valid voluntary exit received for + # the validator with index signed_voluntary_exit.message.validator_index. + + # [REJECT] All of the conditions within process_voluntary_exit pass + # validation. + + # TODO not called yet, so vacuousness is fine + + ok(true) diff --git a/beacon_chain/spec/datatypes.nim b/beacon_chain/spec/datatypes.nim index c4d418100..beec8754b 100644 --- a/beacon_chain/spec/datatypes.nim +++ b/beacon_chain/spec/datatypes.nim @@ -656,6 +656,20 @@ func shortLog*(v: SomeSignedBeaconBlock): auto = signature: shortLog(v.signature) ) +func shortLog*(v: BeaconBlockHeader): auto = + ( + slot: shortLog(v.slot), + proposer_index: v.proposer_index, + parent_root: shortLog(v.parent_root), + state_root: shortLog(v.state_root) + ) + +func shortLog*(v: SignedBeaconBlockHeader): auto = + ( + message: shortLog(v.message), + signature: shortLog(v.signature) + ) + func shortLog*(v: DepositData): auto = ( pubkey: shortLog(v.pubkey), @@ -694,6 +708,31 @@ func shortLog*(v: SomeAttestation): auto = signature: shortLog(v.signature) ) +func shortLog*(v: SomeIndexedAttestation): auto = + ( + attestating_indices: v.attesting_indices, + data: shortLog(v.data), + signature: shortLog(v.signature) + ) + +func shortLog*(v: AttesterSlashing): auto = + ( + attestation_1: shortLog(v.attestation_1), + attestation_2: shortLog(v.attestation_2), + ) + +func shortLog*(v: ProposerSlashing): auto = + ( + signed_header_1: shortLog(v.signed_header_1), + signed_header_2: shortLog(v.signed_header_2) + ) + +func shortLog*(v: VoluntaryExit): auto = + ( + epoch: shortLog(v.epoch), + validator_index: v.validator_index + ) + chronicles.formatIt Slot: it.shortLog chronicles.formatIt Epoch: it.shortLog chronicles.formatIt BeaconBlock: it.shortLog diff --git a/beacon_chain/spec/state_transition_block.nim b/beacon_chain/spec/state_transition_block.nim index cee63e98c..46a91144e 100644 --- a/beacon_chain/spec/state_transition_block.nim +++ b/beacon_chain/spec/state_transition_block.nim @@ -173,12 +173,12 @@ proc process_proposer_slashing*( signed_header.message, proposer.pubkey, signed_header.signature): return err("process_proposer_slashing: invalid signature") - slashValidator(state, header_1.proposer_index.ValidatorIndex, stateCache) + slash_validator(state, header_1.proposer_index.ValidatorIndex, stateCache) ok() # https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/beacon-chain.md#is_slashable_attestation_data -func is_slashable_attestation_data( +func is_slashable_attestation_data*( data_1: AttestationData, data_2: AttestationData): bool = ## Check if ``data_1`` and ``data_2`` are slashable according to Casper FFG ## rules.