add Electra attester slashing gossip support (#6582)
This commit is contained in:
parent
daf7f899c2
commit
28b20936e9
|
@ -12,7 +12,6 @@ import
|
||||||
chronicles, chronos, metrics,
|
chronicles, chronos, metrics,
|
||||||
taskpools,
|
taskpools,
|
||||||
../spec/[helpers, forks],
|
../spec/[helpers, forks],
|
||||||
../spec/datatypes/[altair, phase0, deneb],
|
|
||||||
../consensus_object_pools/[
|
../consensus_object_pools/[
|
||||||
blob_quarantine, block_clearance, block_quarantine, blockchain_dag,
|
blob_quarantine, block_clearance, block_quarantine, blockchain_dag,
|
||||||
attestation_pool, light_client_pool, sync_committee_msg_pool,
|
attestation_pool, light_client_pool, sync_committee_msg_pool,
|
||||||
|
@ -51,10 +50,6 @@ declareCounter beacon_attester_slashings_received,
|
||||||
"Number of valid attester slashings processed by this node"
|
"Number of valid attester slashings processed by this node"
|
||||||
declareCounter beacon_attester_slashings_dropped,
|
declareCounter beacon_attester_slashings_dropped,
|
||||||
"Number of invalid attester slashings dropped by this node", labels = ["reason"]
|
"Number of invalid attester slashings dropped by this node", labels = ["reason"]
|
||||||
declareCounter bls_to_execution_change_received,
|
|
||||||
"Number of valid BLS to execution changes processed by this node"
|
|
||||||
declareCounter bls_to_execution_change_dropped,
|
|
||||||
"Number of invalid BLS to execution changes dropped by this node", labels = ["reason"]
|
|
||||||
declareCounter beacon_proposer_slashings_received,
|
declareCounter beacon_proposer_slashings_received,
|
||||||
"Number of valid proposer slashings processed by this node"
|
"Number of valid proposer slashings processed by this node"
|
||||||
declareCounter beacon_proposer_slashings_dropped,
|
declareCounter beacon_proposer_slashings_dropped,
|
||||||
|
@ -338,7 +333,7 @@ proc setupDoppelgangerDetection*(self: var Eth2Processor, slot: Slot) =
|
||||||
epoch = slot.epoch,
|
epoch = slot.epoch,
|
||||||
broadcast_epoch = self.doppelgangerDetection.broadcastStartEpoch
|
broadcast_epoch = self.doppelgangerDetection.broadcastStartEpoch
|
||||||
|
|
||||||
proc clearDoppelgangerProtection*(self: var Eth2Processor) =
|
func clearDoppelgangerProtection*(self: var Eth2Processor) =
|
||||||
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
|
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
|
||||||
|
|
||||||
proc checkForPotentialDoppelganger(
|
proc checkForPotentialDoppelganger(
|
||||||
|
@ -504,7 +499,8 @@ proc processBlsToExecutionChange*(
|
||||||
|
|
||||||
proc processAttesterSlashing*(
|
proc processAttesterSlashing*(
|
||||||
self: var Eth2Processor, src: MsgSource,
|
self: var Eth2Processor, src: MsgSource,
|
||||||
attesterSlashing: phase0.AttesterSlashing): ValidationRes =
|
attesterSlashing: phase0.AttesterSlashing | electra.AttesterSlashing):
|
||||||
|
ValidationRes =
|
||||||
logScope:
|
logScope:
|
||||||
attesterSlashing = shortLog(attesterSlashing)
|
attesterSlashing = shortLog(attesterSlashing)
|
||||||
|
|
||||||
|
@ -621,7 +617,8 @@ proc processSyncCommitteeMessage*(
|
||||||
proc processSignedContributionAndProof*(
|
proc processSignedContributionAndProof*(
|
||||||
self: ref Eth2Processor, src: MsgSource,
|
self: ref Eth2Processor, src: MsgSource,
|
||||||
contributionAndProof: SignedContributionAndProof,
|
contributionAndProof: SignedContributionAndProof,
|
||||||
checkSignature: bool = true): Future[Result[void, ValidationError]] {.async: (raises: [CancelledError]).} =
|
checkSignature: bool = true):
|
||||||
|
Future[Result[void, ValidationError]] {.async: (raises: [CancelledError]).} =
|
||||||
let
|
let
|
||||||
wallTime = self.getCurrentBeaconTime()
|
wallTime = self.getCurrentBeaconTime()
|
||||||
wallSlot = wallTime.slotOrZero()
|
wallSlot = wallTime.slotOrZero()
|
||||||
|
|
|
@ -1228,7 +1228,8 @@ proc validateBlsToExecutionChange*(
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attester_slashing
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attester_slashing
|
||||||
proc validateAttesterSlashing*(
|
proc validateAttesterSlashing*(
|
||||||
pool: ValidatorChangePool, attester_slashing: phase0.AttesterSlashing):
|
pool: ValidatorChangePool,
|
||||||
|
attester_slashing: phase0.AttesterSlashing | electra.AttesterSlashing):
|
||||||
Result[void, ValidationError] =
|
Result[void, ValidationError] =
|
||||||
# [IGNORE] At least one index in the intersection of the attesting indices of
|
# [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.
|
# each attestation has not yet been seen in any prior attester_slashing (i.e.
|
||||||
|
@ -1246,10 +1247,14 @@ proc validateAttesterSlashing*(
|
||||||
return pool.checkedReject(attester_slashing_validity.error)
|
return pool.checkedReject(attester_slashing_validity.error)
|
||||||
|
|
||||||
# Send notification about new attester slashing via callback
|
# Send notification about new attester slashing via callback
|
||||||
if not(isNil(pool.onPhase0AttesterSlashingReceived)):
|
when attester_slashing is phase0.AttesterSlashing:
|
||||||
pool.onPhase0AttesterSlashingReceived(attester_slashing)
|
if not(isNil(pool.onPhase0AttesterSlashingReceived)):
|
||||||
|
pool.onPhase0AttesterSlashingReceived(attester_slashing)
|
||||||
debugComment "apparently there's no gopssip validation in place for electra attslashings"
|
elif attester_slashing is electra.AttesterSlashing:
|
||||||
|
if not(isNil(pool.onElectraAttesterSlashingReceived)):
|
||||||
|
pool.onElectraAttesterSlashingReceived(attester_slashing)
|
||||||
|
else:
|
||||||
|
static: doAssert false
|
||||||
|
|
||||||
ok()
|
ok()
|
||||||
|
|
||||||
|
|
|
@ -27,7 +27,6 @@ import
|
||||||
eth/[keys, async_utils],
|
eth/[keys, async_utils],
|
||||||
eth/net/nat, eth/p2p/discoveryv5/[enr, node, random2],
|
eth/net/nat, eth/p2p/discoveryv5/[enr, node, random2],
|
||||||
".."/[version, conf, beacon_clock, conf_light_client],
|
".."/[version, conf, beacon_clock, conf_light_client],
|
||||||
../spec/datatypes/[phase0, altair, bellatrix],
|
|
||||||
../spec/[eth2_ssz_serialization, network, helpers, forks],
|
../spec/[eth2_ssz_serialization, network, helpers, forks],
|
||||||
../validators/keystore_management,
|
../validators/keystore_management,
|
||||||
"."/[eth2_discovery, eth2_protocol_dsl, eth2_agents,
|
"."/[eth2_discovery, eth2_protocol_dsl, eth2_agents,
|
||||||
|
@ -340,7 +339,7 @@ func shortProtocolId(protocolId: string): string =
|
||||||
protocolId.high
|
protocolId.high
|
||||||
protocolId[start..ends]
|
protocolId[start..ends]
|
||||||
|
|
||||||
proc updateAgent*(peer: Peer) =
|
func updateAgent*(peer: Peer) =
|
||||||
let
|
let
|
||||||
agent = toLowerAscii(peer.network.switch.peerStore[AgentBook][peer.peerId])
|
agent = toLowerAscii(peer.network.switch.peerStore[AgentBook][peer.peerId])
|
||||||
# proto = peer.network.switch.peerStore[ProtoVersionBook][peer.peerId]
|
# proto = peer.network.switch.peerStore[ProtoVersionBook][peer.peerId]
|
||||||
|
@ -360,7 +359,7 @@ proc updateAgent*(peer: Peer) =
|
||||||
else:
|
else:
|
||||||
peer.remoteAgent = Eth2Agent.Unknown
|
peer.remoteAgent = Eth2Agent.Unknown
|
||||||
|
|
||||||
proc getRemoteAgent*(peer: Peer): Eth2Agent =
|
func getRemoteAgent*(peer: Peer): Eth2Agent =
|
||||||
if peer.remoteAgent == Eth2Agent.Unknown:
|
if peer.remoteAgent == Eth2Agent.Unknown:
|
||||||
peer.updateAgent()
|
peer.updateAgent()
|
||||||
peer.remoteAgent
|
peer.remoteAgent
|
||||||
|
@ -386,7 +385,7 @@ proc openStream(node: Eth2Node,
|
||||||
|
|
||||||
proc init(T: type Peer, network: Eth2Node, peerId: PeerId): Peer {.gcsafe.}
|
proc init(T: type Peer, network: Eth2Node, peerId: PeerId): Peer {.gcsafe.}
|
||||||
|
|
||||||
proc getState*(peer: Peer, proto: ProtocolInfo): RootRef =
|
func getState*(peer: Peer, proto: ProtocolInfo): RootRef =
|
||||||
doAssert peer.protocolStates[proto.index] != nil, $proto.index
|
doAssert peer.protocolStates[proto.index] != nil, $proto.index
|
||||||
peer.protocolStates[proto.index]
|
peer.protocolStates[proto.index]
|
||||||
|
|
||||||
|
@ -398,7 +397,7 @@ template state*(peer: Peer, Protocol: type): untyped =
|
||||||
type S = Protocol.State
|
type S = Protocol.State
|
||||||
S(getState(peer, Protocol.protocolInfo))
|
S(getState(peer, Protocol.protocolInfo))
|
||||||
|
|
||||||
proc getNetworkState*(node: Eth2Node, proto: ProtocolInfo): RootRef =
|
func getNetworkState*(node: Eth2Node, proto: ProtocolInfo): RootRef =
|
||||||
doAssert node.protocolStates[proto.index] != nil, $proto.index
|
doAssert node.protocolStates[proto.index] != nil, $proto.index
|
||||||
node.protocolStates[proto.index]
|
node.protocolStates[proto.index]
|
||||||
|
|
||||||
|
@ -408,8 +407,7 @@ template protocolState*(node: Eth2Node, Protocol: type): untyped =
|
||||||
type S = Protocol.NetworkState
|
type S = Protocol.NetworkState
|
||||||
S(getNetworkState(node, Protocol.protocolInfo))
|
S(getNetworkState(node, Protocol.protocolInfo))
|
||||||
|
|
||||||
proc initProtocolState*[T](state: T, x: Peer|Eth2Node)
|
func initProtocolState*[T](state: T, x: Peer|Eth2Node) {.raises: [].} =
|
||||||
{.gcsafe, raises: [].} =
|
|
||||||
discard
|
discard
|
||||||
|
|
||||||
template networkState*(connection: Peer, Protocol: type): untyped =
|
template networkState*(connection: Peer, Protocol: type): untyped =
|
||||||
|
@ -615,7 +613,7 @@ proc getRequestProtoName(fn: NimNode): NimNode =
|
||||||
|
|
||||||
return newLit("")
|
return newLit("")
|
||||||
|
|
||||||
proc add(s: var seq[byte], pos: var int, bytes: openArray[byte]) =
|
func add(s: var seq[byte], pos: var int, bytes: openArray[byte]) =
|
||||||
s[pos..<pos+bytes.len] = bytes
|
s[pos..<pos+bytes.len] = bytes
|
||||||
pos += bytes.len
|
pos += bytes.len
|
||||||
|
|
||||||
|
@ -852,9 +850,6 @@ func chunkMaxSize[T](): uint32 =
|
||||||
static: doAssert MAX_CHUNK_SIZE < high(uint32).uint64
|
static: doAssert MAX_CHUNK_SIZE < high(uint32).uint64
|
||||||
MAX_CHUNK_SIZE.uint32
|
MAX_CHUNK_SIZE.uint32
|
||||||
|
|
||||||
from ../spec/datatypes/capella import SignedBeaconBlock
|
|
||||||
from ../spec/datatypes/deneb import SignedBeaconBlock
|
|
||||||
|
|
||||||
template gossipMaxSize(T: untyped): uint32 =
|
template gossipMaxSize(T: untyped): uint32 =
|
||||||
const maxSize = static:
|
const maxSize = static:
|
||||||
when isFixedSize(T):
|
when isFixedSize(T):
|
||||||
|
@ -869,7 +864,8 @@ template gossipMaxSize(T: untyped): uint32 =
|
||||||
elif T is phase0.Attestation or T is phase0.AttesterSlashing or
|
elif T is phase0.Attestation or T is phase0.AttesterSlashing or
|
||||||
T is phase0.SignedAggregateAndProof or T is phase0.SignedBeaconBlock or
|
T is phase0.SignedAggregateAndProof or T is phase0.SignedBeaconBlock or
|
||||||
T is electra.SignedAggregateAndProof or T is electra.Attestation or
|
T is electra.SignedAggregateAndProof or T is electra.Attestation or
|
||||||
T is altair.SignedBeaconBlock or T is SomeForkyLightClientObject:
|
T is electra.AttesterSlashing or T is altair.SignedBeaconBlock or
|
||||||
|
T is SomeForkyLightClientObject:
|
||||||
GOSSIP_MAX_SIZE
|
GOSSIP_MAX_SIZE
|
||||||
else:
|
else:
|
||||||
{.fatal: "unknown type " & name(T).}
|
{.fatal: "unknown type " & name(T).}
|
||||||
|
@ -1042,10 +1038,10 @@ proc makeEth2Request(peer: Peer, protocolId: string, requestBytes: seq[byte],
|
||||||
debug "Unexpected error while closing stream",
|
debug "Unexpected error while closing stream",
|
||||||
peer, protocolId, exc = exc.msg
|
peer, protocolId, exc = exc.msg
|
||||||
|
|
||||||
proc init*(T: type MultipleChunksResponse, peer: Peer, conn: Connection): T =
|
func init*(T: type MultipleChunksResponse, peer: Peer, conn: Connection): T =
|
||||||
T(UntypedResponse(peer: peer, stream: conn))
|
T(UntypedResponse(peer: peer, stream: conn))
|
||||||
|
|
||||||
proc init*[MsgType](T: type SingleChunkResponse[MsgType],
|
func init*[MsgType](T: type SingleChunkResponse[MsgType],
|
||||||
peer: Peer, conn: Connection): T =
|
peer: Peer, conn: Connection): T =
|
||||||
T(UntypedResponse(peer: peer, stream: conn))
|
T(UntypedResponse(peer: peer, stream: conn))
|
||||||
|
|
||||||
|
@ -1087,7 +1083,7 @@ proc performProtocolHandshakes(peer: Peer, incoming: bool) {.async: (raises: [Ca
|
||||||
if protocol.onPeerConnected != nil:
|
if protocol.onPeerConnected != nil:
|
||||||
await protocol.onPeerConnected(peer, incoming)
|
await protocol.onPeerConnected(peer, incoming)
|
||||||
|
|
||||||
proc initProtocol(name: string,
|
func initProtocol(name: string,
|
||||||
peerInit: PeerStateInitializer,
|
peerInit: PeerStateInitializer,
|
||||||
networkInit: NetworkStateInitializer,
|
networkInit: NetworkStateInitializer,
|
||||||
index: int): ProtocolInfoObj =
|
index: int): ProtocolInfoObj =
|
||||||
|
@ -1098,7 +1094,7 @@ proc initProtocol(name: string,
|
||||||
peerStateInitializer: peerInit,
|
peerStateInitializer: peerInit,
|
||||||
networkStateInitializer: networkInit)
|
networkStateInitializer: networkInit)
|
||||||
|
|
||||||
proc setEventHandlers(p: ProtocolInfo,
|
func setEventHandlers(p: ProtocolInfo,
|
||||||
onPeerConnected: OnPeerConnectedHandler,
|
onPeerConnected: OnPeerConnectedHandler,
|
||||||
onPeerDisconnected: OnPeerDisconnectedHandler) =
|
onPeerDisconnected: OnPeerDisconnectedHandler) =
|
||||||
p.onPeerConnected = onPeerConnected
|
p.onPeerConnected = onPeerConnected
|
||||||
|
@ -1298,7 +1294,7 @@ proc handleIncomingStream(network: Eth2Node,
|
||||||
debug "Unexpected error while closing incoming connection", exc = exc.msg
|
debug "Unexpected error while closing incoming connection", exc = exc.msg
|
||||||
releasePeer(peer)
|
releasePeer(peer)
|
||||||
|
|
||||||
proc toPeerAddr*(r: enr.TypedRecord,
|
func toPeerAddr*(r: enr.TypedRecord,
|
||||||
proto: IpTransportProtocol): Result[PeerAddr, cstring] =
|
proto: IpTransportProtocol): Result[PeerAddr, cstring] =
|
||||||
if not r.secp256k1.isSome:
|
if not r.secp256k1.isSome:
|
||||||
return err("enr: no secp256k1 key in record")
|
return err("enr: no secp256k1 key in record")
|
||||||
|
@ -1416,7 +1412,7 @@ proc connectWorker(node: Eth2Node, index: int) {.async: (raises: [CancelledError
|
||||||
# excluding peer here after processing.
|
# excluding peer here after processing.
|
||||||
node.connTable.excl(remotePeerAddr.peerId)
|
node.connTable.excl(remotePeerAddr.peerId)
|
||||||
|
|
||||||
proc toPeerAddr(node: Node): Result[PeerAddr, cstring] =
|
func toPeerAddr(node: Node): Result[PeerAddr, cstring] =
|
||||||
let nodeRecord = TypedRecord.fromRecord(node.record)
|
let nodeRecord = TypedRecord.fromRecord(node.record)
|
||||||
let peerAddr = ? nodeRecord.toPeerAddr(tcpProtocol)
|
let peerAddr = ? nodeRecord.toPeerAddr(tcpProtocol)
|
||||||
ok(peerAddr)
|
ok(peerAddr)
|
||||||
|
@ -1950,7 +1946,7 @@ proc init(T: type Peer, network: Eth2Node, peerId: PeerId): Peer =
|
||||||
res.protocolStates[proto.index] = proto.peerStateInitializer(res)
|
res.protocolStates[proto.index] = proto.peerStateInitializer(res)
|
||||||
res
|
res
|
||||||
|
|
||||||
proc registerMsg(protocol: ProtocolInfo,
|
func registerMsg(protocol: ProtocolInfo,
|
||||||
name: string,
|
name: string,
|
||||||
mounter: MounterProc,
|
mounter: MounterProc,
|
||||||
libp2pCodecName: string) =
|
libp2pCodecName: string) =
|
||||||
|
@ -2426,7 +2422,7 @@ proc newValidationResultFuture(v: ValidationResult): Future[ValidationResult]
|
||||||
res.complete(v)
|
res.complete(v)
|
||||||
res
|
res
|
||||||
|
|
||||||
proc addValidator*[MsgType](node: Eth2Node,
|
func addValidator*[MsgType](node: Eth2Node,
|
||||||
topic: string,
|
topic: string,
|
||||||
msgValidator: proc(msg: MsgType):
|
msgValidator: proc(msg: MsgType):
|
||||||
ValidationResult {.gcsafe, raises: [].} ) =
|
ValidationResult {.gcsafe, raises: [].} ) =
|
||||||
|
@ -2494,7 +2490,7 @@ proc addAsyncValidator*[MsgType](node: Eth2Node,
|
||||||
proc unsubscribe*(node: Eth2Node, topic: string) =
|
proc unsubscribe*(node: Eth2Node, topic: string) =
|
||||||
node.pubsub.unsubscribeAll(topic)
|
node.pubsub.unsubscribeAll(topic)
|
||||||
|
|
||||||
proc gossipEncode(msg: auto): seq[byte] =
|
func gossipEncode(msg: auto): seq[byte] =
|
||||||
let uncompressed = SSZ.encode(msg)
|
let uncompressed = SSZ.encode(msg)
|
||||||
# This function only for messages we create. A message this large amounts to
|
# This function only for messages we create. A message this large amounts to
|
||||||
# an internal logic error.
|
# an internal logic error.
|
||||||
|
@ -2627,7 +2623,8 @@ proc broadcastVoluntaryExit*(
|
||||||
node.broadcast(topic, exit)
|
node.broadcast(topic, exit)
|
||||||
|
|
||||||
proc broadcastAttesterSlashing*(
|
proc broadcastAttesterSlashing*(
|
||||||
node: Eth2Node, slashing: phase0.AttesterSlashing):
|
node: Eth2Node,
|
||||||
|
slashing: phase0.AttesterSlashing | electra.AttesterSlashing):
|
||||||
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
|
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
|
||||||
let topic = getAttesterSlashingsTopic(
|
let topic = getAttesterSlashingsTopic(
|
||||||
node.forkDigestAtEpoch(node.getWallEpoch))
|
node.forkDigestAtEpoch(node.getWallEpoch))
|
||||||
|
|
|
@ -1858,13 +1858,23 @@ proc installMessageValidators(node: BeaconNode) =
|
||||||
|
|
||||||
# attester_slashing
|
# attester_slashing
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing
|
||||||
node.network.addValidator(
|
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/electra/p2p-interface.md#modifications-in-electra
|
||||||
getAttesterSlashingsTopic(digest), proc (
|
when consensusFork >= ConsensusFork.Electra:
|
||||||
attesterSlashing: phase0.AttesterSlashing
|
node.network.addValidator(
|
||||||
): ValidationResult =
|
getAttesterSlashingsTopic(digest), proc (
|
||||||
toValidationResult(
|
attesterSlashing: electra.AttesterSlashing
|
||||||
node.processor[].processAttesterSlashing(
|
): ValidationResult =
|
||||||
MsgSource.gossip, attesterSlashing)))
|
toValidationResult(
|
||||||
|
node.processor[].processAttesterSlashing(
|
||||||
|
MsgSource.gossip, attesterSlashing)))
|
||||||
|
else:
|
||||||
|
node.network.addValidator(
|
||||||
|
getAttesterSlashingsTopic(digest), proc (
|
||||||
|
attesterSlashing: phase0.AttesterSlashing
|
||||||
|
): ValidationResult =
|
||||||
|
toValidationResult(
|
||||||
|
node.processor[].processAttesterSlashing(
|
||||||
|
MsgSource.gossip, attesterSlashing)))
|
||||||
|
|
||||||
# proposer_slashing
|
# proposer_slashing
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/phase0/p2p-interface.md#proposer_slashing
|
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/phase0/p2p-interface.md#proposer_slashing
|
||||||
|
|
|
@ -11,12 +11,10 @@ import
|
||||||
stew/assign2,
|
stew/assign2,
|
||||||
json_serialization/std/sets,
|
json_serialization/std/sets,
|
||||||
chronicles,
|
chronicles,
|
||||||
./datatypes/[phase0, altair, bellatrix],
|
|
||||||
"."/[eth2_merkleization, forks, signatures, validator]
|
"."/[eth2_merkleization, forks, signatures, validator]
|
||||||
|
|
||||||
from std/algorithm import fill, sort
|
from std/algorithm import fill, sort
|
||||||
from std/sequtils import anyIt, mapIt, toSeq
|
from std/sequtils import anyIt, mapIt, toSeq
|
||||||
from ./datatypes/capella import BeaconState, ExecutionPayloadHeader, Withdrawal
|
|
||||||
|
|
||||||
export extras, forks, validator, chronicles
|
export extras, forks, validator, chronicles
|
||||||
|
|
||||||
|
@ -540,8 +538,11 @@ func is_eligible_for_activation*(
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
|
||||||
proc is_valid_indexed_attestation*(
|
proc is_valid_indexed_attestation*(
|
||||||
state: ForkyBeaconState,
|
state: ForkyBeaconState,
|
||||||
indexed_attestation: SomeIndexedAttestation | electra.IndexedAttestation |
|
# phase0.SomeIndexedAttestation | electra.SomeIndexedAttestation:
|
||||||
electra.TrustedIndexedAttestation,
|
# https://github.com/nim-lang/Nim/issues/18095
|
||||||
|
indexed_attestation:
|
||||||
|
phase0.IndexedAttestation | phase0.TrustedIndexedAttestation |
|
||||||
|
electra.IndexedAttestation | electra.TrustedIndexedAttestation,
|
||||||
flags: UpdateFlags): Result[void, cstring] =
|
flags: UpdateFlags): Result[void, cstring] =
|
||||||
## Check if ``indexed_attestation`` is not empty, has sorted and unique
|
## Check if ``indexed_attestation`` is not empty, has sorted and unique
|
||||||
## indices and has a valid aggregate signature.
|
## indices and has a valid aggregate signature.
|
||||||
|
|
|
@ -611,6 +611,8 @@ type
|
||||||
signature*: TrustedSig
|
signature*: TrustedSig
|
||||||
committee_bits*: AttestationCommitteeBits # [New in Electra:EIP7549]
|
committee_bits*: AttestationCommitteeBits # [New in Electra:EIP7549]
|
||||||
|
|
||||||
|
SomeIndexedAttestation* = IndexedAttestation | TrustedIndexedAttestation
|
||||||
|
SomeAttesterSlashing* = AttesterSlashing | TrustedAttesterSlashing
|
||||||
SomeSignedBeaconBlock* =
|
SomeSignedBeaconBlock* =
|
||||||
SignedBeaconBlock |
|
SignedBeaconBlock |
|
||||||
SigVerifiedSignedBeaconBlock |
|
SigVerifiedSignedBeaconBlock |
|
||||||
|
@ -645,6 +647,19 @@ type
|
||||||
func initHashedBeaconState*(s: BeaconState): HashedBeaconState =
|
func initHashedBeaconState*(s: BeaconState): HashedBeaconState =
|
||||||
HashedBeaconState(data: s)
|
HashedBeaconState(data: s)
|
||||||
|
|
||||||
|
func shortLog*(v: SomeIndexedAttestation): auto =
|
||||||
|
(
|
||||||
|
attestating_indices: v.attesting_indices,
|
||||||
|
data: shortLog(v.data),
|
||||||
|
signature: shortLog(v.signature)
|
||||||
|
)
|
||||||
|
|
||||||
|
func shortLog*(v: SomeAttesterSlashing): auto =
|
||||||
|
(
|
||||||
|
attestation_1: shortLog(v.attestation_1),
|
||||||
|
attestation_2: shortLog(v.attestation_2),
|
||||||
|
)
|
||||||
|
|
||||||
func shortLog*(v: SomeBeaconBlock): auto =
|
func shortLog*(v: SomeBeaconBlock): auto =
|
||||||
(
|
(
|
||||||
slot: shortLog(v.slot),
|
slot: shortLog(v.slot),
|
||||||
|
|
|
@ -415,7 +415,6 @@ func partialBeaconBlock*(
|
||||||
): auto =
|
): auto =
|
||||||
const consensusFork = typeof(state).kind
|
const consensusFork = typeof(state).kind
|
||||||
|
|
||||||
debugComment "re-enable attester slashing packing in electra"
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/validator.md#preparing-for-a-beaconblock
|
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/validator.md#preparing-for-a-beaconblock
|
||||||
consensusFork.BeaconBlock(
|
consensusFork.BeaconBlock(
|
||||||
slot: state.data.slot,
|
slot: state.data.slot,
|
||||||
|
@ -426,7 +425,7 @@ func partialBeaconBlock*(
|
||||||
eth1_data: eth1_data,
|
eth1_data: eth1_data,
|
||||||
graffiti: graffiti,
|
graffiti: graffiti,
|
||||||
proposer_slashings: validator_changes.proposer_slashings,
|
proposer_slashings: validator_changes.proposer_slashings,
|
||||||
#attester_slashings: validator_changes.attester_slashings,
|
attester_slashings: validator_changes.electra_attester_slashings,
|
||||||
attestations:
|
attestations:
|
||||||
List[electra.Attestation, Limit MAX_ATTESTATIONS_ELECTRA](attestations),
|
List[electra.Attestation, Limit MAX_ATTESTATIONS_ELECTRA](attestations),
|
||||||
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
|
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
|
||||||
|
|
|
@ -208,8 +208,11 @@ func is_slashable_attestation_data(
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attester-slashings
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#attester-slashings
|
||||||
proc check_attester_slashing*(
|
proc check_attester_slashing*(
|
||||||
state: ForkyBeaconState,
|
state: ForkyBeaconState,
|
||||||
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
|
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
|
||||||
electra.TrustedAttesterSlashing,
|
# https://github.com/nim-lang/Nim/issues/18095
|
||||||
|
attester_slashing:
|
||||||
|
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
|
||||||
|
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
|
||||||
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
|
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
|
||||||
let
|
let
|
||||||
attestation_1 = attester_slashing.attestation_1
|
attestation_1 = attester_slashing.attestation_1
|
||||||
|
@ -243,8 +246,11 @@ proc check_attester_slashing*(
|
||||||
|
|
||||||
proc check_attester_slashing*(
|
proc check_attester_slashing*(
|
||||||
state: var ForkedHashedBeaconState,
|
state: var ForkedHashedBeaconState,
|
||||||
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
|
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
|
||||||
electra.TrustedAttesterSlashing,
|
# https://github.com/nim-lang/Nim/issues/18095
|
||||||
|
attester_slashing:
|
||||||
|
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
|
||||||
|
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
|
||||||
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
|
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
|
||||||
withState(state):
|
withState(state):
|
||||||
check_attester_slashing(forkyState.data, attester_slashing, flags)
|
check_attester_slashing(forkyState.data, attester_slashing, flags)
|
||||||
|
@ -253,8 +259,11 @@ proc check_attester_slashing*(
|
||||||
proc process_attester_slashing*(
|
proc process_attester_slashing*(
|
||||||
cfg: RuntimeConfig,
|
cfg: RuntimeConfig,
|
||||||
state: var ForkyBeaconState,
|
state: var ForkyBeaconState,
|
||||||
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
|
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
|
||||||
electra.TrustedAttesterSlashing,
|
# https://github.com/nim-lang/Nim/issues/18095
|
||||||
|
attester_slashing:
|
||||||
|
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
|
||||||
|
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
|
||||||
flags: UpdateFlags,
|
flags: UpdateFlags,
|
||||||
exit_queue_info: ExitQueueInfo, cache: var StateCache
|
exit_queue_info: ExitQueueInfo, cache: var StateCache
|
||||||
): Result[(Gwei, ExitQueueInfo), cstring] =
|
): Result[(Gwei, ExitQueueInfo), cstring] =
|
||||||
|
|
|
@ -335,10 +335,6 @@ proc weigh_justification_and_finalization(
|
||||||
res.current_justified_checkpoint = Checkpoint(
|
res.current_justified_checkpoint = Checkpoint(
|
||||||
epoch: previous_epoch, root: get_block_root(state, previous_epoch))
|
epoch: previous_epoch, root: get_block_root(state, previous_epoch))
|
||||||
uint8(res.justification_bits).setBit 1
|
uint8(res.justification_bits).setBit 1
|
||||||
|
|
||||||
trace "Justified with previous epoch",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.current_justified_checkpoint)
|
|
||||||
elif strictVerification in flags:
|
elif strictVerification in flags:
|
||||||
fatal "Low attestation participation in previous epoch",
|
fatal "Low attestation participation in previous epoch",
|
||||||
total_active_balance,
|
total_active_balance,
|
||||||
|
@ -352,10 +348,6 @@ proc weigh_justification_and_finalization(
|
||||||
epoch: current_epoch, root: get_block_root(state, current_epoch))
|
epoch: current_epoch, root: get_block_root(state, current_epoch))
|
||||||
uint8(res.justification_bits).setBit 0
|
uint8(res.justification_bits).setBit 0
|
||||||
|
|
||||||
trace "Justified with current epoch",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.current_justified_checkpoint)
|
|
||||||
|
|
||||||
# Process finalizations
|
# Process finalizations
|
||||||
let bitfield = uint8(res.justification_bits)
|
let bitfield = uint8(res.justification_bits)
|
||||||
|
|
||||||
|
@ -365,40 +357,24 @@ proc weigh_justification_and_finalization(
|
||||||
old_previous_justified_checkpoint.epoch + 3 == current_epoch:
|
old_previous_justified_checkpoint.epoch + 3 == current_epoch:
|
||||||
res.finalized_checkpoint = old_previous_justified_checkpoint
|
res.finalized_checkpoint = old_previous_justified_checkpoint
|
||||||
|
|
||||||
trace "Finalized with rule 234",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.finalized_checkpoint)
|
|
||||||
|
|
||||||
## The 2nd/3rd most recent epochs are justified, the 2nd using the 3rd as
|
## The 2nd/3rd most recent epochs are justified, the 2nd using the 3rd as
|
||||||
## source
|
## source
|
||||||
if (bitfield and 0b110) == 0b110 and
|
if (bitfield and 0b110) == 0b110 and
|
||||||
old_previous_justified_checkpoint.epoch + 2 == current_epoch:
|
old_previous_justified_checkpoint.epoch + 2 == current_epoch:
|
||||||
res.finalized_checkpoint = old_previous_justified_checkpoint
|
res.finalized_checkpoint = old_previous_justified_checkpoint
|
||||||
|
|
||||||
trace "Finalized with rule 23",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.finalized_checkpoint)
|
|
||||||
|
|
||||||
## The 1st/2nd/3rd most recent epochs are justified, the 1st using the 3rd as
|
## The 1st/2nd/3rd most recent epochs are justified, the 1st using the 3rd as
|
||||||
## source
|
## source
|
||||||
if (bitfield and 0b111) == 0b111 and
|
if (bitfield and 0b111) == 0b111 and
|
||||||
old_current_justified_checkpoint.epoch + 2 == current_epoch:
|
old_current_justified_checkpoint.epoch + 2 == current_epoch:
|
||||||
res.finalized_checkpoint = old_current_justified_checkpoint
|
res.finalized_checkpoint = old_current_justified_checkpoint
|
||||||
|
|
||||||
trace "Finalized with rule 123",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.finalized_checkpoint)
|
|
||||||
|
|
||||||
## The 1st/2nd most recent epochs are justified, the 1st using the 2nd as
|
## The 1st/2nd most recent epochs are justified, the 1st using the 2nd as
|
||||||
## source
|
## source
|
||||||
if (bitfield and 0b11) == 0b11 and
|
if (bitfield and 0b11) == 0b11 and
|
||||||
old_current_justified_checkpoint.epoch + 1 == current_epoch:
|
old_current_justified_checkpoint.epoch + 1 == current_epoch:
|
||||||
res.finalized_checkpoint = old_current_justified_checkpoint
|
res.finalized_checkpoint = old_current_justified_checkpoint
|
||||||
|
|
||||||
trace "Finalized with rule 12",
|
|
||||||
current_epoch = current_epoch,
|
|
||||||
checkpoint = shortLog(res.finalized_checkpoint)
|
|
||||||
|
|
||||||
res
|
res
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#justification-and-finalization
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.6/specs/phase0/beacon-chain.md#justification-and-finalization
|
||||||
|
@ -1347,8 +1323,6 @@ proc process_epoch*(
|
||||||
cfg: RuntimeConfig, state: var phase0.BeaconState, flags: UpdateFlags,
|
cfg: RuntimeConfig, state: var phase0.BeaconState, flags: UpdateFlags,
|
||||||
cache: var StateCache, info: var phase0.EpochInfo): Result[void, cstring] =
|
cache: var StateCache, info: var phase0.EpochInfo): Result[void, cstring] =
|
||||||
let epoch = get_current_epoch(state)
|
let epoch = get_current_epoch(state)
|
||||||
trace "process_epoch", epoch
|
|
||||||
|
|
||||||
info.init(state)
|
info.init(state)
|
||||||
info.process_attestations(state, cache)
|
info.process_attestations(state, cache)
|
||||||
|
|
||||||
|
@ -1407,8 +1381,6 @@ proc process_epoch*(
|
||||||
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
||||||
Result[void, cstring] =
|
Result[void, cstring] =
|
||||||
let epoch = get_current_epoch(state)
|
let epoch = get_current_epoch(state)
|
||||||
trace "process_epoch", epoch
|
|
||||||
|
|
||||||
info.init(state)
|
info.init(state)
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#justification-and-finalization
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#justification-and-finalization
|
||||||
|
@ -1453,8 +1425,6 @@ proc process_epoch*(
|
||||||
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
||||||
Result[void, cstring] =
|
Result[void, cstring] =
|
||||||
let epoch = get_current_epoch(state)
|
let epoch = get_current_epoch(state)
|
||||||
trace "process_epoch", epoch
|
|
||||||
|
|
||||||
info.init(state)
|
info.init(state)
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#justification-and-finalization
|
# https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/altair/beacon-chain.md#justification-and-finalization
|
||||||
|
@ -1498,8 +1468,6 @@ proc process_epoch*(
|
||||||
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
|
||||||
Result[void, cstring] =
|
Result[void, cstring] =
|
||||||
let epoch = get_current_epoch(state)
|
let epoch = get_current_epoch(state)
|
||||||
trace "process_epoch", epoch
|
|
||||||
|
|
||||||
info.init(state)
|
info.init(state)
|
||||||
|
|
||||||
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/altair/beacon-chain.md#justification-and-finalization
|
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/altair/beacon-chain.md#justification-and-finalization
|
||||||
|
|
|
@ -32,7 +32,6 @@ macro copyFields*(
|
||||||
dst: untyped, src: untyped, fieldNames: static[seq[string]]): untyped =
|
dst: untyped, src: untyped, fieldNames: static[seq[string]]): untyped =
|
||||||
result = newStmtList()
|
result = newStmtList()
|
||||||
for name in fieldNames:
|
for name in fieldNames:
|
||||||
debugComment "deposit_receipts_root and exits_root are not currently filled in anywhere properly, so blinded electra proposals will fail"
|
|
||||||
if name notin [
|
if name notin [
|
||||||
# These fields are the ones which vary between the blinded and
|
# These fields are the ones which vary between the blinded and
|
||||||
# unblinded objects, and can't simply be copied.
|
# unblinded objects, and can't simply be copied.
|
||||||
|
|
|
@ -10,13 +10,9 @@
|
||||||
import
|
import
|
||||||
std/tables,
|
std/tables,
|
||||||
metrics, chronicles,
|
metrics, chronicles,
|
||||||
../spec/datatypes/phase0,
|
|
||||||
../spec/[beaconstate, forks, helpers],
|
../spec/[beaconstate, forks, helpers],
|
||||||
../beacon_clock
|
../beacon_clock
|
||||||
|
|
||||||
# TODO when forks re-exports capella, drop this
|
|
||||||
from ../spec/datatypes/capella import shortLog
|
|
||||||
|
|
||||||
logScope: topics = "val_mon"
|
logScope: topics = "val_mon"
|
||||||
|
|
||||||
# Validator monitoring based on the same feature in Lighthouse - using the same
|
# Validator monitoring based on the same feature in Lighthouse - using the same
|
||||||
|
@ -170,22 +166,22 @@ type
|
||||||
## nature of gossip processing: in particular, old messages may reappear
|
## nature of gossip processing: in particular, old messages may reappear
|
||||||
## on the network and therefore be double-counted.
|
## on the network and therefore be double-counted.
|
||||||
attestations: int64
|
attestations: int64
|
||||||
attestation_min_delay: Option[TimeDiff]
|
attestation_min_delay: Opt[TimeDiff]
|
||||||
attestation_aggregate_inclusions: int64
|
attestation_aggregate_inclusions: int64
|
||||||
attestation_block_inclusions: int64
|
attestation_block_inclusions: int64
|
||||||
attestation_min_block_inclusion_distance: Option[uint64]
|
attestation_min_block_inclusion_distance: Opt[uint64]
|
||||||
|
|
||||||
aggregates: int64
|
aggregates: int64
|
||||||
aggregate_min_delay: Option[TimeDiff]
|
aggregate_min_delay: Opt[TimeDiff]
|
||||||
|
|
||||||
sync_committee_messages: int64
|
sync_committee_messages: int64
|
||||||
sync_committee_message_min_delay: Option[TimeDiff]
|
sync_committee_message_min_delay: Opt[TimeDiff]
|
||||||
|
|
||||||
sync_signature_block_inclusions: int64
|
sync_signature_block_inclusions: int64
|
||||||
sync_signature_contribution_inclusions: int64
|
sync_signature_contribution_inclusions: int64
|
||||||
|
|
||||||
sync_contributions: int64
|
sync_contributions: int64
|
||||||
sync_contribution_min_delay: Option[TimeDiff]
|
sync_contribution_min_delay: Opt[TimeDiff]
|
||||||
|
|
||||||
exits: int64
|
exits: int64
|
||||||
proposer_slashings: int64
|
proposer_slashings: int64
|
||||||
|
@ -222,11 +218,11 @@ template toGaugeValue(v: bool): int64 =
|
||||||
template toGaugeValue(v: TimeDiff): float =
|
template toGaugeValue(v: TimeDiff): float =
|
||||||
toFloatSeconds(v)
|
toFloatSeconds(v)
|
||||||
|
|
||||||
proc update_if_lt[T](current: var Option[T], val: T) =
|
func update_if_lt[T](current: var Opt[T], val: T) =
|
||||||
if current.isNone() or val < current.get():
|
if current.isNone() or val < current.get():
|
||||||
current = some(val)
|
current = Opt.some(val)
|
||||||
|
|
||||||
proc addMonitor*(
|
func addMonitor*(
|
||||||
self: var ValidatorMonitor, pubkey: ValidatorPubKey,
|
self: var ValidatorMonitor, pubkey: ValidatorPubKey,
|
||||||
index: Opt[ValidatorIndex]) =
|
index: Opt[ValidatorIndex]) =
|
||||||
if pubkey in self.monitors:
|
if pubkey in self.monitors:
|
||||||
|
@ -259,7 +255,7 @@ proc addAutoMonitor*(
|
||||||
info "Started monitoring validator",
|
info "Started monitoring validator",
|
||||||
validator = shortLog(pubkey), pubkey, index
|
validator = shortLog(pubkey), pubkey, index
|
||||||
|
|
||||||
proc init*(T: type ValidatorMonitor, autoRegister = false, totals = false): T =
|
func init*(T: type ValidatorMonitor, autoRegister = false, totals = false): T =
|
||||||
T(autoRegister: autoRegister, totals: totals)
|
T(autoRegister: autoRegister, totals: totals)
|
||||||
|
|
||||||
template summaryIdx(epoch: Epoch): int = (epoch.uint64 mod 2).int
|
template summaryIdx(epoch: Epoch): int = (epoch.uint64 mod 2).int
|
||||||
|
@ -751,8 +747,6 @@ proc registerAttestationInBlock*(
|
||||||
update_if_lt(
|
update_if_lt(
|
||||||
epochSummary.attestation_min_block_inclusion_distance, inclusion_lag)
|
epochSummary.attestation_min_block_inclusion_distance, inclusion_lag)
|
||||||
|
|
||||||
from ../spec/datatypes/deneb import shortLog
|
|
||||||
|
|
||||||
proc registerBeaconBlock*(
|
proc registerBeaconBlock*(
|
||||||
self: var ValidatorMonitor,
|
self: var ValidatorMonitor,
|
||||||
src: MsgSource,
|
src: MsgSource,
|
||||||
|
@ -885,7 +879,7 @@ proc registerProposerSlashing*(
|
||||||
|
|
||||||
proc registerAttesterSlashing*(
|
proc registerAttesterSlashing*(
|
||||||
self: var ValidatorMonitor, src: MsgSource,
|
self: var ValidatorMonitor, src: MsgSource,
|
||||||
slashing: phase0.AttesterSlashing) =
|
slashing: phase0.AttesterSlashing | electra.AttesterSlashing) =
|
||||||
let data = slashing.attestation_1.data
|
let data = slashing.attestation_1.data
|
||||||
|
|
||||||
for idx in slashing.attestation_2.attesting_indices:
|
for idx in slashing.attestation_2.attesting_indices:
|
||||||
|
|
Loading…
Reference in New Issue