add Electra attester slashing gossip support (#6582)

This commit is contained in:
tersec 2024-09-26 06:11:58 +00:00 committed by GitHub
parent daf7f899c2
commit 28b20936e9
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
11 changed files with 97 additions and 103 deletions

View File

@ -12,7 +12,6 @@ import
chronicles, chronos, metrics,
taskpools,
../spec/[helpers, forks],
../spec/datatypes/[altair, phase0, deneb],
../consensus_object_pools/[
blob_quarantine, block_clearance, block_quarantine, blockchain_dag,
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"
declareCounter beacon_attester_slashings_dropped,
"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,
"Number of valid proposer slashings processed by this node"
declareCounter beacon_proposer_slashings_dropped,
@ -338,7 +333,7 @@ proc setupDoppelgangerDetection*(self: var Eth2Processor, slot: Slot) =
epoch = slot.epoch,
broadcast_epoch = self.doppelgangerDetection.broadcastStartEpoch
proc clearDoppelgangerProtection*(self: var Eth2Processor) =
func clearDoppelgangerProtection*(self: var Eth2Processor) =
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
proc checkForPotentialDoppelganger(
@ -504,7 +499,8 @@ proc processBlsToExecutionChange*(
proc processAttesterSlashing*(
self: var Eth2Processor, src: MsgSource,
attesterSlashing: phase0.AttesterSlashing): ValidationRes =
attesterSlashing: phase0.AttesterSlashing | electra.AttesterSlashing):
ValidationRes =
logScope:
attesterSlashing = shortLog(attesterSlashing)
@ -621,7 +617,8 @@ proc processSyncCommitteeMessage*(
proc processSignedContributionAndProof*(
self: ref Eth2Processor, src: MsgSource,
contributionAndProof: SignedContributionAndProof,
checkSignature: bool = true): Future[Result[void, ValidationError]] {.async: (raises: [CancelledError]).} =
checkSignature: bool = true):
Future[Result[void, ValidationError]] {.async: (raises: [CancelledError]).} =
let
wallTime = self.getCurrentBeaconTime()
wallSlot = wallTime.slotOrZero()

View File

@ -1228,7 +1228,8 @@ proc validateBlsToExecutionChange*(
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/phase0/p2p-interface.md#attester_slashing
proc validateAttesterSlashing*(
pool: ValidatorChangePool, attester_slashing: phase0.AttesterSlashing):
pool: ValidatorChangePool,
attester_slashing: phase0.AttesterSlashing | electra.AttesterSlashing):
Result[void, ValidationError] =
# [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.
@ -1246,10 +1247,14 @@ proc validateAttesterSlashing*(
return pool.checkedReject(attester_slashing_validity.error)
# Send notification about new attester slashing via callback
if not(isNil(pool.onPhase0AttesterSlashingReceived)):
pool.onPhase0AttesterSlashingReceived(attester_slashing)
debugComment "apparently there's no gopssip validation in place for electra attslashings"
when attester_slashing is phase0.AttesterSlashing:
if not(isNil(pool.onPhase0AttesterSlashingReceived)):
pool.onPhase0AttesterSlashingReceived(attester_slashing)
elif attester_slashing is electra.AttesterSlashing:
if not(isNil(pool.onElectraAttesterSlashingReceived)):
pool.onElectraAttesterSlashingReceived(attester_slashing)
else:
static: doAssert false
ok()

View File

@ -27,7 +27,6 @@ import
eth/[keys, async_utils],
eth/net/nat, eth/p2p/discoveryv5/[enr, node, random2],
".."/[version, conf, beacon_clock, conf_light_client],
../spec/datatypes/[phase0, altair, bellatrix],
../spec/[eth2_ssz_serialization, network, helpers, forks],
../validators/keystore_management,
"."/[eth2_discovery, eth2_protocol_dsl, eth2_agents,
@ -340,7 +339,7 @@ func shortProtocolId(protocolId: string): string =
protocolId.high
protocolId[start..ends]
proc updateAgent*(peer: Peer) =
func updateAgent*(peer: Peer) =
let
agent = toLowerAscii(peer.network.switch.peerStore[AgentBook][peer.peerId])
# proto = peer.network.switch.peerStore[ProtoVersionBook][peer.peerId]
@ -360,7 +359,7 @@ proc updateAgent*(peer: Peer) =
else:
peer.remoteAgent = Eth2Agent.Unknown
proc getRemoteAgent*(peer: Peer): Eth2Agent =
func getRemoteAgent*(peer: Peer): Eth2Agent =
if peer.remoteAgent == Eth2Agent.Unknown:
peer.updateAgent()
peer.remoteAgent
@ -386,7 +385,7 @@ proc openStream(node: Eth2Node,
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
peer.protocolStates[proto.index]
@ -398,7 +397,7 @@ template state*(peer: Peer, Protocol: type): untyped =
type S = Protocol.State
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
node.protocolStates[proto.index]
@ -408,8 +407,7 @@ template protocolState*(node: Eth2Node, Protocol: type): untyped =
type S = Protocol.NetworkState
S(getNetworkState(node, Protocol.protocolInfo))
proc initProtocolState*[T](state: T, x: Peer|Eth2Node)
{.gcsafe, raises: [].} =
func initProtocolState*[T](state: T, x: Peer|Eth2Node) {.raises: [].} =
discard
template networkState*(connection: Peer, Protocol: type): untyped =
@ -615,7 +613,7 @@ proc getRequestProtoName(fn: NimNode): NimNode =
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
pos += bytes.len
@ -852,9 +850,6 @@ func chunkMaxSize[T](): uint32 =
static: doAssert MAX_CHUNK_SIZE < high(uint32).uint64
MAX_CHUNK_SIZE.uint32
from ../spec/datatypes/capella import SignedBeaconBlock
from ../spec/datatypes/deneb import SignedBeaconBlock
template gossipMaxSize(T: untyped): uint32 =
const maxSize = static:
when isFixedSize(T):
@ -869,7 +864,8 @@ template gossipMaxSize(T: untyped): uint32 =
elif T is phase0.Attestation or T is phase0.AttesterSlashing or
T is phase0.SignedAggregateAndProof or T is phase0.SignedBeaconBlock 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
else:
{.fatal: "unknown type " & name(T).}
@ -1042,10 +1038,10 @@ proc makeEth2Request(peer: Peer, protocolId: string, requestBytes: seq[byte],
debug "Unexpected error while closing stream",
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))
proc init*[MsgType](T: type SingleChunkResponse[MsgType],
func init*[MsgType](T: type SingleChunkResponse[MsgType],
peer: Peer, conn: Connection): T =
T(UntypedResponse(peer: peer, stream: conn))
@ -1087,7 +1083,7 @@ proc performProtocolHandshakes(peer: Peer, incoming: bool) {.async: (raises: [Ca
if protocol.onPeerConnected != nil:
await protocol.onPeerConnected(peer, incoming)
proc initProtocol(name: string,
func initProtocol(name: string,
peerInit: PeerStateInitializer,
networkInit: NetworkStateInitializer,
index: int): ProtocolInfoObj =
@ -1098,7 +1094,7 @@ proc initProtocol(name: string,
peerStateInitializer: peerInit,
networkStateInitializer: networkInit)
proc setEventHandlers(p: ProtocolInfo,
func setEventHandlers(p: ProtocolInfo,
onPeerConnected: OnPeerConnectedHandler,
onPeerDisconnected: OnPeerDisconnectedHandler) =
p.onPeerConnected = onPeerConnected
@ -1298,7 +1294,7 @@ proc handleIncomingStream(network: Eth2Node,
debug "Unexpected error while closing incoming connection", exc = exc.msg
releasePeer(peer)
proc toPeerAddr*(r: enr.TypedRecord,
func toPeerAddr*(r: enr.TypedRecord,
proto: IpTransportProtocol): Result[PeerAddr, cstring] =
if not r.secp256k1.isSome:
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.
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 peerAddr = ? nodeRecord.toPeerAddr(tcpProtocol)
ok(peerAddr)
@ -1950,7 +1946,7 @@ proc init(T: type Peer, network: Eth2Node, peerId: PeerId): Peer =
res.protocolStates[proto.index] = proto.peerStateInitializer(res)
res
proc registerMsg(protocol: ProtocolInfo,
func registerMsg(protocol: ProtocolInfo,
name: string,
mounter: MounterProc,
libp2pCodecName: string) =
@ -2426,7 +2422,7 @@ proc newValidationResultFuture(v: ValidationResult): Future[ValidationResult]
res.complete(v)
res
proc addValidator*[MsgType](node: Eth2Node,
func addValidator*[MsgType](node: Eth2Node,
topic: string,
msgValidator: proc(msg: MsgType):
ValidationResult {.gcsafe, raises: [].} ) =
@ -2494,7 +2490,7 @@ proc addAsyncValidator*[MsgType](node: Eth2Node,
proc unsubscribe*(node: Eth2Node, topic: string) =
node.pubsub.unsubscribeAll(topic)
proc gossipEncode(msg: auto): seq[byte] =
func gossipEncode(msg: auto): seq[byte] =
let uncompressed = SSZ.encode(msg)
# This function only for messages we create. A message this large amounts to
# an internal logic error.
@ -2627,7 +2623,8 @@ proc broadcastVoluntaryExit*(
node.broadcast(topic, exit)
proc broadcastAttesterSlashing*(
node: Eth2Node, slashing: phase0.AttesterSlashing):
node: Eth2Node,
slashing: phase0.AttesterSlashing | electra.AttesterSlashing):
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
let topic = getAttesterSlashingsTopic(
node.forkDigestAtEpoch(node.getWallEpoch))

View File

@ -1858,13 +1858,23 @@ proc installMessageValidators(node: BeaconNode) =
# attester_slashing
# https://github.com/ethereum/consensus-specs/blob/v1.4.0-beta.5/specs/phase0/p2p-interface.md#attester_slashing
node.network.addValidator(
getAttesterSlashingsTopic(digest), proc (
attesterSlashing: phase0.AttesterSlashing
): ValidationResult =
toValidationResult(
node.processor[].processAttesterSlashing(
MsgSource.gossip, attesterSlashing)))
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/electra/p2p-interface.md#modifications-in-electra
when consensusFork >= ConsensusFork.Electra:
node.network.addValidator(
getAttesterSlashingsTopic(digest), proc (
attesterSlashing: electra.AttesterSlashing
): ValidationResult =
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
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/phase0/p2p-interface.md#proposer_slashing

View File

@ -11,12 +11,10 @@ import
stew/assign2,
json_serialization/std/sets,
chronicles,
./datatypes/[phase0, altair, bellatrix],
"."/[eth2_merkleization, forks, signatures, validator]
from std/algorithm import fill, sort
from std/sequtils import anyIt, mapIt, toSeq
from ./datatypes/capella import BeaconState, ExecutionPayloadHeader, Withdrawal
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
proc is_valid_indexed_attestation*(
state: ForkyBeaconState,
indexed_attestation: SomeIndexedAttestation | electra.IndexedAttestation |
electra.TrustedIndexedAttestation,
# phase0.SomeIndexedAttestation | electra.SomeIndexedAttestation:
# https://github.com/nim-lang/Nim/issues/18095
indexed_attestation:
phase0.IndexedAttestation | phase0.TrustedIndexedAttestation |
electra.IndexedAttestation | electra.TrustedIndexedAttestation,
flags: UpdateFlags): Result[void, cstring] =
## Check if ``indexed_attestation`` is not empty, has sorted and unique
## indices and has a valid aggregate signature.

View File

@ -611,6 +611,8 @@ type
signature*: TrustedSig
committee_bits*: AttestationCommitteeBits # [New in Electra:EIP7549]
SomeIndexedAttestation* = IndexedAttestation | TrustedIndexedAttestation
SomeAttesterSlashing* = AttesterSlashing | TrustedAttesterSlashing
SomeSignedBeaconBlock* =
SignedBeaconBlock |
SigVerifiedSignedBeaconBlock |
@ -645,6 +647,19 @@ type
func initHashedBeaconState*(s: BeaconState): HashedBeaconState =
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 =
(
slot: shortLog(v.slot),

View File

@ -415,7 +415,6 @@ func partialBeaconBlock*(
): auto =
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
consensusFork.BeaconBlock(
slot: state.data.slot,
@ -426,7 +425,7 @@ func partialBeaconBlock*(
eth1_data: eth1_data,
graffiti: graffiti,
proposer_slashings: validator_changes.proposer_slashings,
#attester_slashings: validator_changes.attester_slashings,
attester_slashings: validator_changes.electra_attester_slashings,
attestations:
List[electra.Attestation, Limit MAX_ATTESTATIONS_ELECTRA](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),

View File

@ -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
proc check_attester_slashing*(
state: ForkyBeaconState,
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
electra.TrustedAttesterSlashing,
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
# https://github.com/nim-lang/Nim/issues/18095
attester_slashing:
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
let
attestation_1 = attester_slashing.attestation_1
@ -243,8 +246,11 @@ proc check_attester_slashing*(
proc check_attester_slashing*(
state: var ForkedHashedBeaconState,
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
electra.TrustedAttesterSlashing,
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
# https://github.com/nim-lang/Nim/issues/18095
attester_slashing:
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
flags: UpdateFlags): Result[seq[ValidatorIndex], cstring] =
withState(state):
check_attester_slashing(forkyState.data, attester_slashing, flags)
@ -253,8 +259,11 @@ proc check_attester_slashing*(
proc process_attester_slashing*(
cfg: RuntimeConfig,
state: var ForkyBeaconState,
attester_slashing: SomeAttesterSlashing | electra.AttesterSlashing |
electra.TrustedAttesterSlashing,
# phase0.SomeAttesterSlashing | electra.SomeAttesterSlashing:
# https://github.com/nim-lang/Nim/issues/18095
attester_slashing:
phase0.AttesterSlashing | phase0.TrustedAttesterSlashing |
electra.AttesterSlashing | electra.TrustedAttesterSlashing,
flags: UpdateFlags,
exit_queue_info: ExitQueueInfo, cache: var StateCache
): Result[(Gwei, ExitQueueInfo), cstring] =

View File

@ -335,10 +335,6 @@ proc weigh_justification_and_finalization(
res.current_justified_checkpoint = Checkpoint(
epoch: previous_epoch, root: get_block_root(state, previous_epoch))
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:
fatal "Low attestation participation in previous epoch",
total_active_balance,
@ -352,10 +348,6 @@ proc weigh_justification_and_finalization(
epoch: current_epoch, root: get_block_root(state, current_epoch))
uint8(res.justification_bits).setBit 0
trace "Justified with current epoch",
current_epoch = current_epoch,
checkpoint = shortLog(res.current_justified_checkpoint)
# Process finalizations
let bitfield = uint8(res.justification_bits)
@ -365,40 +357,24 @@ proc weigh_justification_and_finalization(
old_previous_justified_checkpoint.epoch + 3 == current_epoch:
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
## source
if (bitfield and 0b110) == 0b110 and
old_previous_justified_checkpoint.epoch + 2 == current_epoch:
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
## source
if (bitfield and 0b111) == 0b111 and
old_current_justified_checkpoint.epoch + 2 == current_epoch:
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
## source
if (bitfield and 0b11) == 0b11 and
old_current_justified_checkpoint.epoch + 1 == current_epoch:
res.finalized_checkpoint = old_current_justified_checkpoint
trace "Finalized with rule 12",
current_epoch = current_epoch,
checkpoint = shortLog(res.finalized_checkpoint)
res
# 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,
cache: var StateCache, info: var phase0.EpochInfo): Result[void, cstring] =
let epoch = get_current_epoch(state)
trace "process_epoch", epoch
info.init(state)
info.process_attestations(state, cache)
@ -1407,8 +1381,6 @@ proc process_epoch*(
flags: UpdateFlags, cache: var StateCache, info: var altair.EpochInfo):
Result[void, cstring] =
let epoch = get_current_epoch(state)
trace "process_epoch", epoch
info.init(state)
# 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):
Result[void, cstring] =
let epoch = get_current_epoch(state)
trace "process_epoch", epoch
info.init(state)
# 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):
Result[void, cstring] =
let epoch = get_current_epoch(state)
trace "process_epoch", epoch
info.init(state)
# https://github.com/ethereum/consensus-specs/blob/v1.5.0-alpha.6/specs/altair/beacon-chain.md#justification-and-finalization

View File

@ -32,7 +32,6 @@ macro copyFields*(
dst: untyped, src: untyped, fieldNames: static[seq[string]]): untyped =
result = newStmtList()
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 [
# These fields are the ones which vary between the blinded and
# unblinded objects, and can't simply be copied.

View File

@ -10,13 +10,9 @@
import
std/tables,
metrics, chronicles,
../spec/datatypes/phase0,
../spec/[beaconstate, forks, helpers],
../beacon_clock
# TODO when forks re-exports capella, drop this
from ../spec/datatypes/capella import shortLog
logScope: topics = "val_mon"
# 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
## on the network and therefore be double-counted.
attestations: int64
attestation_min_delay: Option[TimeDiff]
attestation_min_delay: Opt[TimeDiff]
attestation_aggregate_inclusions: int64
attestation_block_inclusions: int64
attestation_min_block_inclusion_distance: Option[uint64]
attestation_min_block_inclusion_distance: Opt[uint64]
aggregates: int64
aggregate_min_delay: Option[TimeDiff]
aggregate_min_delay: Opt[TimeDiff]
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_contribution_inclusions: int64
sync_contributions: int64
sync_contribution_min_delay: Option[TimeDiff]
sync_contribution_min_delay: Opt[TimeDiff]
exits: int64
proposer_slashings: int64
@ -222,11 +218,11 @@ template toGaugeValue(v: bool): int64 =
template toGaugeValue(v: TimeDiff): float =
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():
current = some(val)
current = Opt.some(val)
proc addMonitor*(
func addMonitor*(
self: var ValidatorMonitor, pubkey: ValidatorPubKey,
index: Opt[ValidatorIndex]) =
if pubkey in self.monitors:
@ -259,7 +255,7 @@ proc addAutoMonitor*(
info "Started monitoring validator",
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)
template summaryIdx(epoch: Epoch): int = (epoch.uint64 mod 2).int
@ -751,8 +747,6 @@ proc registerAttestationInBlock*(
update_if_lt(
epochSummary.attestation_min_block_inclusion_distance, inclusion_lag)
from ../spec/datatypes/deneb import shortLog
proc registerBeaconBlock*(
self: var ValidatorMonitor,
src: MsgSource,
@ -885,7 +879,7 @@ proc registerProposerSlashing*(
proc registerAttesterSlashing*(
self: var ValidatorMonitor, src: MsgSource,
slashing: phase0.AttesterSlashing) =
slashing: phase0.AttesterSlashing | electra.AttesterSlashing) =
let data = slashing.attestation_1.data
for idx in slashing.attestation_2.attesting_indices: