mirror of
https://github.com/status-im/nimbus-eth2.git
synced 2025-02-20 02:08:12 +00:00
[WIP] skeleton of attester slashing pool & validators (#1639)
* skeleton of attester slashing pool & validators * add skeleton for proposer slashings and voluntary exits; rename pool to more inclusive exit pool to stay consistent with all three; ensure is initialized by beacon_node so is safe to merge, even if it doesn't do much yet
This commit is contained in:
parent
aabb47e46c
commit
804b152d1d
@ -25,7 +25,7 @@ import
|
|||||||
spec/[datatypes, digest, crypto, beaconstate, helpers, network, presets],
|
spec/[datatypes, digest, crypto, beaconstate, helpers, network, presets],
|
||||||
spec/state_transition,
|
spec/state_transition,
|
||||||
conf, time, beacon_chain_db, validator_pool, extras,
|
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,
|
beacon_node_common, beacon_node_types,
|
||||||
block_pools/[spec_cache, chain_dag, quarantine, clearance, block_pools_types],
|
block_pools/[spec_cache, chain_dag, quarantine, clearance, block_pools_types],
|
||||||
nimbus_binary_common, network_metadata,
|
nimbus_binary_common, network_metadata,
|
||||||
@ -249,6 +249,7 @@ proc init*(T: type BeaconNode,
|
|||||||
topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest)
|
topicAggregateAndProofs = getAggregateAndProofsTopic(enrForkId.forkDigest)
|
||||||
network = createEth2Node(rng, conf, enrForkId)
|
network = createEth2Node(rng, conf, enrForkId)
|
||||||
attestationPool = newClone(AttestationPool.init(chainDag, quarantine))
|
attestationPool = newClone(AttestationPool.init(chainDag, quarantine))
|
||||||
|
exitPool = newClone(ExitPool.init(chainDag, quarantine))
|
||||||
var res = BeaconNode(
|
var res = BeaconNode(
|
||||||
nickname: nickname,
|
nickname: nickname,
|
||||||
graffitiBytes: if conf.graffiti.isSome: conf.graffiti.get.GraffitiBytes
|
graffitiBytes: if conf.graffiti.isSome: conf.graffiti.get.GraffitiBytes
|
||||||
@ -261,6 +262,7 @@ proc init*(T: type BeaconNode,
|
|||||||
chainDag: chainDag,
|
chainDag: chainDag,
|
||||||
quarantine: quarantine,
|
quarantine: quarantine,
|
||||||
attestationPool: attestationPool,
|
attestationPool: attestationPool,
|
||||||
|
exitPool: exitPool,
|
||||||
mainchainMonitor: mainchainMonitor,
|
mainchainMonitor: mainchainMonitor,
|
||||||
beaconClock: BeaconClock.init(chainDag.headState.data.data),
|
beaconClock: BeaconClock.init(chainDag.headState.data.data),
|
||||||
rpcServer: rpcServer,
|
rpcServer: rpcServer,
|
||||||
@ -272,7 +274,7 @@ proc init*(T: type BeaconNode,
|
|||||||
proc getWallTime(): BeaconTime = res.beaconClock.now()
|
proc getWallTime(): BeaconTime = res.beaconClock.now()
|
||||||
|
|
||||||
res.processor = Eth2Processor.new(
|
res.processor = Eth2Processor.new(
|
||||||
conf, chainDag, attestationPool, quarantine, getWallTime)
|
conf, chainDag, attestationPool, exitPool, quarantine, getWallTime)
|
||||||
|
|
||||||
res.requestManager = RequestManager.init(
|
res.requestManager = RequestManager.init(
|
||||||
network, res.processor.blocksQueue)
|
network, res.processor.blocksQueue)
|
||||||
@ -397,6 +399,7 @@ proc addMessageHandlers(node: BeaconNode): Future[void] =
|
|||||||
allFutures(
|
allFutures(
|
||||||
# As a side-effect, this gets the attestation subnets too.
|
# As a side-effect, this gets the attestation subnets too.
|
||||||
node.network.subscribe(node.topicBeaconBlocks),
|
node.network.subscribe(node.topicBeaconBlocks),
|
||||||
|
node.network.subscribe(getAttesterSlashingsTopic(node.forkDigest)),
|
||||||
|
|
||||||
node.getAttestationHandlers()
|
node.getAttestationHandlers()
|
||||||
)
|
)
|
||||||
@ -773,6 +776,21 @@ proc installMessageValidators(node: BeaconNode) =
|
|||||||
proc (signedBlock: SignedBeaconBlock): bool =
|
proc (signedBlock: SignedBeaconBlock): bool =
|
||||||
node.processor[].blockValidator(signedBlock))
|
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) =
|
proc removeMessageHandlers(node: BeaconNode) =
|
||||||
var unsubscriptions: seq[Future[void]]
|
var unsubscriptions: seq[Future[void]]
|
||||||
|
|
||||||
|
@ -43,6 +43,7 @@ type
|
|||||||
chainDag*: ChainDAGRef
|
chainDag*: ChainDAGRef
|
||||||
quarantine*: QuarantineRef
|
quarantine*: QuarantineRef
|
||||||
attestationPool*: ref AttestationPool
|
attestationPool*: ref AttestationPool
|
||||||
|
exitPool*: ref ExitPool
|
||||||
mainchainMonitor*: MainchainMonitor
|
mainchainMonitor*: MainchainMonitor
|
||||||
beaconClock*: BeaconClock
|
beaconClock*: BeaconClock
|
||||||
rpcServer*: RpcServer
|
rpcServer*: RpcServer
|
||||||
|
@ -1,7 +1,7 @@
|
|||||||
{.push raises: [Defect].}
|
{.push raises: [Defect].}
|
||||||
|
|
||||||
import
|
import
|
||||||
deques, tables, streams,
|
std/[deques, tables, streams],
|
||||||
stew/endians2,
|
stew/endians2,
|
||||||
spec/[datatypes, crypto],
|
spec/[datatypes, crypto],
|
||||||
block_pools/block_pools_types,
|
block_pools/block_pools_types,
|
||||||
@ -64,6 +64,22 @@ type
|
|||||||
|
|
||||||
forkChoice*: ForkChoice
|
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
|
# Validator Pool
|
||||||
|
@ -4,7 +4,7 @@ import
|
|||||||
chronicles, chronicles/chronos_tools, chronos, metrics,
|
chronicles, chronicles/chronos_tools, chronos, metrics,
|
||||||
./spec/[crypto, datatypes, digest],
|
./spec/[crypto, datatypes, digest],
|
||||||
./block_pools/[clearance, chain_dag],
|
./block_pools/[clearance, chain_dag],
|
||||||
./attestation_aggregation,
|
./attestation_aggregation, ./exit_pool,
|
||||||
./beacon_node_types, ./attestation_pool,
|
./beacon_node_types, ./attestation_pool,
|
||||||
./time, ./conf, ./sszdump
|
./time, ./conf, ./sszdump
|
||||||
|
|
||||||
@ -15,6 +15,12 @@ declareCounter beacon_aggregates_received,
|
|||||||
"Number of beacon chain aggregate attestations received by this peer"
|
"Number of beacon chain aggregate attestations received by this peer"
|
||||||
declareCounter beacon_blocks_received,
|
declareCounter beacon_blocks_received,
|
||||||
"Number of beacon chain blocks received by this peer"
|
"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]
|
const delayBuckets = [2.0, 4.0, 6.0, 8.0, 10.0, 12.0, 14.0, Inf]
|
||||||
|
|
||||||
@ -54,6 +60,7 @@ type
|
|||||||
getWallTime*: GetWallTimeFn
|
getWallTime*: GetWallTimeFn
|
||||||
chainDag*: ChainDAGRef
|
chainDag*: ChainDAGRef
|
||||||
attestationPool*: ref AttestationPool
|
attestationPool*: ref AttestationPool
|
||||||
|
exitPool: ref ExitPool
|
||||||
quarantine*: QuarantineRef
|
quarantine*: QuarantineRef
|
||||||
|
|
||||||
blocksQueue*: AsyncQueue[BlockEntry]
|
blocksQueue*: AsyncQueue[BlockEntry]
|
||||||
@ -367,6 +374,42 @@ proc aggregateValidator*(
|
|||||||
|
|
||||||
true
|
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.} =
|
proc runQueueProcessingLoop*(self: ref Eth2Processor) {.async.} =
|
||||||
# Blocks in eth2 arrive on a schedule for every slot:
|
# Blocks in eth2 arrive on a schedule for every slot:
|
||||||
#
|
#
|
||||||
@ -403,6 +446,7 @@ proc new*(T: type Eth2Processor,
|
|||||||
config: BeaconNodeConf,
|
config: BeaconNodeConf,
|
||||||
chainDag: ChainDAGRef,
|
chainDag: ChainDAGRef,
|
||||||
attestationPool: ref AttestationPool,
|
attestationPool: ref AttestationPool,
|
||||||
|
exitPool: ref ExitPool,
|
||||||
quarantine: QuarantineRef,
|
quarantine: QuarantineRef,
|
||||||
getWallTime: GetWallTimeFn): ref Eth2Processor =
|
getWallTime: GetWallTimeFn): ref Eth2Processor =
|
||||||
(ref Eth2Processor)(
|
(ref Eth2Processor)(
|
||||||
@ -410,6 +454,7 @@ proc new*(T: type Eth2Processor,
|
|||||||
getWallTime: getWallTime,
|
getWallTime: getWallTime,
|
||||||
chainDag: chainDag,
|
chainDag: chainDag,
|
||||||
attestationPool: attestationPool,
|
attestationPool: attestationPool,
|
||||||
|
exitPool: exitPool,
|
||||||
quarantine: quarantine,
|
quarantine: quarantine,
|
||||||
blocksQueue: newAsyncQueue[BlockEntry](1),
|
blocksQueue: newAsyncQueue[BlockEntry](1),
|
||||||
aggregatesQueue: newAsyncQueue[AggregateEntry](MAX_ATTESTATIONS.int),
|
aggregatesQueue: newAsyncQueue[AggregateEntry](MAX_ATTESTATIONS.int),
|
||||||
|
168
beacon_chain/exit_pool.nim
Normal file
168
beacon_chain/exit_pool.nim
Normal file
@ -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)
|
@ -656,6 +656,20 @@ func shortLog*(v: SomeSignedBeaconBlock): auto =
|
|||||||
signature: shortLog(v.signature)
|
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 =
|
func shortLog*(v: DepositData): auto =
|
||||||
(
|
(
|
||||||
pubkey: shortLog(v.pubkey),
|
pubkey: shortLog(v.pubkey),
|
||||||
@ -694,6 +708,31 @@ func shortLog*(v: SomeAttestation): auto =
|
|||||||
signature: shortLog(v.signature)
|
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 Slot: it.shortLog
|
||||||
chronicles.formatIt Epoch: it.shortLog
|
chronicles.formatIt Epoch: it.shortLog
|
||||||
chronicles.formatIt BeaconBlock: it.shortLog
|
chronicles.formatIt BeaconBlock: it.shortLog
|
||||||
|
@ -173,12 +173,12 @@ proc process_proposer_slashing*(
|
|||||||
signed_header.message, proposer.pubkey, signed_header.signature):
|
signed_header.message, proposer.pubkey, signed_header.signature):
|
||||||
return err("process_proposer_slashing: invalid 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()
|
ok()
|
||||||
|
|
||||||
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.2/specs/phase0/beacon-chain.md#is_slashable_attestation_data
|
# 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 =
|
data_1: AttestationData, data_2: AttestationData): bool =
|
||||||
## Check if ``data_1`` and ``data_2`` are slashable according to Casper FFG
|
## Check if ``data_1`` and ``data_2`` are slashable according to Casper FFG
|
||||||
## rules.
|
## rules.
|
||||||
|
Loading…
x
Reference in New Issue
Block a user