mirror of
https://github.com/status-im/nimbus-eth2.git
synced 2025-01-12 07:14:20 +00:00
d07113767d
Since the sync committee duties are no longer updated on every slot and previously the sync committee aggregators selection proofs were generated during the duties update, this now resulted in the client using stale selection proofs (they must be generated at each slot). The fix consists of moving the selection proof generation logic in a different function which is properly executed on each slot. Other changes: * The logtrace tool has been enhanced with a framework for adding new simpler log aggregation and analysis algorithms. The default CI testnet simulation will now ensure that the blocks in the network have reasonable sync committee participation.
895 lines
31 KiB
Nim
895 lines
31 KiB
Nim
# beacon_chain
|
|
# Copyright (c) 2021-2022 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.
|
|
|
|
import
|
|
std/[tables, os, sets, sequtils, strutils, uri],
|
|
stew/[base10, results, byteutils],
|
|
bearssl/rand, chronos, presto, presto/client as presto_client,
|
|
chronicles, confutils, json_serialization/std/[options, net],
|
|
metrics, metrics/chronos_httpserver,
|
|
".."/spec/datatypes/[phase0, altair],
|
|
".."/spec/[eth2_merkleization, helpers, signatures, validator],
|
|
".."/spec/eth2_apis/[eth2_rest_serialization, rest_beacon_client,
|
|
dynamic_fee_recipients],
|
|
".."/validators/[keystore_management, validator_pool, slashing_protection],
|
|
".."/[conf, beacon_clock, version, nimbus_binary_common]
|
|
|
|
from std/times import Time, toUnix, fromUnix, getTime
|
|
|
|
export
|
|
os, sets, sequtils, chronos, presto, chronicles, confutils,
|
|
nimbus_binary_common, version, conf, options, tables, results, base10,
|
|
byteutils, presto_client, eth2_rest_serialization, rest_beacon_client,
|
|
phase0, altair, helpers, signatures, validator, eth2_merkleization,
|
|
beacon_clock, keystore_management, slashing_protection, validator_pool,
|
|
dynamic_fee_recipients, Time, toUnix, fromUnix, getTime
|
|
|
|
const
|
|
SYNC_TOLERANCE* = 4'u64
|
|
SLOT_LOOKAHEAD* = 1.seconds
|
|
HISTORICAL_DUTIES_EPOCHS* = 2'u64
|
|
TIME_DELAY_FROM_SLOT* = 79.milliseconds
|
|
SUBSCRIPTION_BUFFER_SLOTS* = 2'u64
|
|
VALIDATOR_DEFAULT_GAS_LIMIT* = 30_000_000'u64 # Stand-in, reasonable default
|
|
EPOCHS_BETWEEN_VALIDATOR_REGISTRATION* = 1
|
|
|
|
DelayBuckets* = [-Inf, -4.0, -2.0, -1.0, -0.5, -0.1, -0.05,
|
|
0.05, 0.1, 0.5, 1.0, 2.0, 4.0, 8.0, Inf]
|
|
|
|
type
|
|
ServiceState* {.pure.} = enum
|
|
Initialized, Running, Error, Closing, Closed
|
|
|
|
BlockServiceEventRef* = ref object of RootObj
|
|
slot*: Slot
|
|
proposers*: seq[ValidatorPubKey]
|
|
|
|
RegistrationKind* {.pure.} = enum
|
|
Cached, IncorrectTime, MissingIndex, MissingFee, ErrorSignature, NoSignature
|
|
|
|
PendingValidatorRegistration* = object
|
|
registration*: SignedValidatorRegistrationV1
|
|
future*: Future[SignatureResult]
|
|
|
|
ClientServiceRef* = ref object of RootObj
|
|
name*: string
|
|
state*: ServiceState
|
|
lifeFut*: Future[void]
|
|
client*: ValidatorClientRef
|
|
|
|
DutiesServiceRef* = ref object of ClientServiceRef
|
|
|
|
FallbackServiceRef* = ref object of ClientServiceRef
|
|
onlineEvent*: AsyncEvent
|
|
|
|
ForkServiceRef* = ref object of ClientServiceRef
|
|
|
|
AttestationServiceRef* = ref object of ClientServiceRef
|
|
|
|
BlockServiceRef* = ref object of ClientServiceRef
|
|
|
|
SyncCommitteeServiceRef* = ref object of ClientServiceRef
|
|
|
|
DoppelgangerServiceRef* = ref object of ClientServiceRef
|
|
enabled*: bool
|
|
|
|
DutyAndProof* = object
|
|
epoch*: Epoch
|
|
dependentRoot*: Eth2Digest
|
|
data*: RestAttesterDuty
|
|
slotSig*: Option[ValidatorSig]
|
|
|
|
SyncCommitteeDuty* = object
|
|
pubkey*: ValidatorPubKey
|
|
validator_index*: ValidatorIndex
|
|
validator_sync_committee_index*: IndexInSyncCommittee
|
|
|
|
SyncCommitteeSubscriptionInfo* = object
|
|
validator_index*: ValidatorIndex
|
|
validator_sync_committee_indices*: seq[IndexInSyncCommittee]
|
|
|
|
ProposerTask* = object
|
|
duty*: RestProposerDuty
|
|
future*: Future[void]
|
|
|
|
ProposedData* = object
|
|
epoch*: Epoch
|
|
dependentRoot*: Eth2Digest
|
|
duties*: seq[ProposerTask]
|
|
|
|
BeaconNodeRole* {.pure.} = enum
|
|
Duties,
|
|
AttestationData, AttestationPublish,
|
|
AggregatedData, AggregatedPublish,
|
|
BlockProposalData, BlockProposalPublish,
|
|
SyncCommitteeData, SyncCommitteePublish
|
|
|
|
BeaconNodeServer* = object
|
|
client*: RestClientRef
|
|
endpoint*: string
|
|
config*: Option[RestSpecVC]
|
|
ident*: Option[string]
|
|
genesis*: Option[RestGenesis]
|
|
syncInfo*: Option[RestSyncInfo]
|
|
status*: RestBeaconNodeStatus
|
|
roles*: set[BeaconNodeRole]
|
|
logIdent*: string
|
|
index*: int
|
|
|
|
EpochDuties* = object
|
|
duties*: Table[Epoch, DutyAndProof]
|
|
|
|
EpochSyncDuties* = object
|
|
duties*: Table[Epoch, SyncCommitteeDuty]
|
|
|
|
RestBeaconNodeStatus* {.pure.} = enum
|
|
Uninitalized, Offline, Incompatible, NotSynced, Online
|
|
|
|
BeaconNodeServerRef* = ref BeaconNodeServer
|
|
|
|
AttesterMap* = Table[ValidatorPubKey, EpochDuties]
|
|
SyncCommitteeDutiesMap* = Table[ValidatorPubKey, EpochSyncDuties]
|
|
ProposerMap* = Table[Epoch, ProposedData]
|
|
|
|
DoppelgangerStatus* {.pure.} = enum
|
|
None, Checking, Passed
|
|
|
|
DoppelgangerAttempt* {.pure.} = enum
|
|
None, Failure, SuccessTrue, SuccessFalse
|
|
|
|
DoppelgangerState* = object
|
|
startEpoch*: Epoch
|
|
epochsCount*: uint64
|
|
lastAttempt*: DoppelgangerAttempt
|
|
status*: DoppelgangerStatus
|
|
|
|
DoppelgangerDetection* = object
|
|
startSlot*: Slot
|
|
validators*: Table[ValidatorIndex, DoppelgangerState]
|
|
|
|
ValidatorClient* = object
|
|
config*: ValidatorClientConf
|
|
metricsServer*: Option[MetricsHttpServerRef]
|
|
graffitiBytes*: GraffitiBytes
|
|
beaconNodes*: seq[BeaconNodeServerRef]
|
|
fallbackService*: FallbackServiceRef
|
|
forkService*: ForkServiceRef
|
|
dutiesService*: DutiesServiceRef
|
|
attestationService*: AttestationServiceRef
|
|
blockService*: BlockServiceRef
|
|
syncCommitteeService*: SyncCommitteeServiceRef
|
|
doppelgangerService*: DoppelgangerServiceRef
|
|
runSlotLoopFut*: Future[void]
|
|
sigintHandleFut*: Future[void]
|
|
sigtermHandleFut*: Future[void]
|
|
keymanagerHost*: ref KeymanagerHost
|
|
keymanagerServer*: RestServerRef
|
|
beaconClock*: BeaconClock
|
|
doppelgangerDetection*: DoppelgangerDetection
|
|
attachedValidators*: ref ValidatorPool
|
|
forks*: seq[Fork]
|
|
forksAvailable*: AsyncEvent
|
|
nodesAvailable*: AsyncEvent
|
|
indicesAvailable*: AsyncEvent
|
|
gracefulExit*: AsyncEvent
|
|
attesters*: AttesterMap
|
|
proposers*: ProposerMap
|
|
syncCommitteeDuties*: SyncCommitteeDutiesMap
|
|
beaconGenesis*: RestGenesis
|
|
proposerTasks*: Table[Slot, seq[ProposerTask]]
|
|
dynamicFeeRecipientsStore*: ref DynamicFeeRecipientsStore
|
|
validatorsRegCache*: Table[ValidatorPubKey, SignedValidatorRegistrationV1]
|
|
rng*: ref HmacDrbgContext
|
|
|
|
ValidatorClientRef* = ref ValidatorClient
|
|
|
|
ValidatorClientError* = object of CatchableError
|
|
ValidatorApiError* = object of ValidatorClientError
|
|
|
|
const
|
|
DefaultDutyAndProof* = DutyAndProof(epoch: Epoch(0xFFFF_FFFF_FFFF_FFFF'u64))
|
|
SlotDuration* = int64(SECONDS_PER_SLOT).seconds
|
|
OneThirdDuration* = int64(SECONDS_PER_SLOT).seconds div INTERVALS_PER_SLOT
|
|
AllBeaconNodeRoles* = {
|
|
BeaconNodeRole.Duties,
|
|
BeaconNodeRole.AttestationData,
|
|
BeaconNodeRole.AttestationPublish,
|
|
BeaconNodeRole.AggregatedData,
|
|
BeaconNodeRole.AggregatedPublish,
|
|
BeaconNodeRole.BlockProposalData,
|
|
BeaconNodeRole.BlockProposalPublish,
|
|
BeaconNodeRole.SyncCommitteeData,
|
|
BeaconNodeRole.SyncCommitteePublish,
|
|
}
|
|
|
|
proc `$`*(roles: set[BeaconNodeRole]): string =
|
|
if card(roles) > 0:
|
|
if roles != AllBeaconNodeRoles:
|
|
var res: seq[string]
|
|
if BeaconNodeRole.Duties in roles:
|
|
res.add("duties")
|
|
if BeaconNodeRole.AttestationData in roles:
|
|
res.add("attestation-data")
|
|
if BeaconNodeRole.AttestationPublish in roles:
|
|
res.add("attestation-publish")
|
|
if BeaconNodeRole.AggregatedData in roles:
|
|
res.add("aggregated-data")
|
|
if BeaconNodeRole.AggregatedPublish in roles:
|
|
res.add("aggregated-publish")
|
|
if BeaconNodeRole.BlockProposalData in roles:
|
|
res.add("block-data")
|
|
if BeaconNodeRole.BlockProposalPublish in roles:
|
|
res.add("block-publish")
|
|
if BeaconNodeRole.SyncCommitteeData in roles:
|
|
res.add("sync-data")
|
|
if BeaconNodeRole.SyncCommitteePublish in roles:
|
|
res.add("sync-publish")
|
|
res.join(",")
|
|
else:
|
|
"{all}"
|
|
else:
|
|
"{}"
|
|
|
|
proc shortLog*(roles: set[BeaconNodeRole]): string =
|
|
var r = "AGBSD"
|
|
if BeaconNodeRole.AttestationData in roles:
|
|
if BeaconNodeRole.AttestationPublish in roles: r[0] = 'A' else: r[0] = 'a'
|
|
else:
|
|
if BeaconNodeRole.AttestationPublish in roles: r[0] = '+' else: r[0] = '-'
|
|
if BeaconNodeRole.AggregatedData in roles:
|
|
if BeaconNodeRole.AggregatedPublish in roles: r[1] = 'G' else: r[1] = 'g'
|
|
else:
|
|
if BeaconNodeRole.AggregatedPublish in roles: r[1] = '+' else: r[1] = '-'
|
|
if BeaconNodeRole.BlockProposalData in roles:
|
|
if BeaconNodeRole.BlockProposalPublish in roles: r[2] = 'B' else: r[2] = 'b'
|
|
else:
|
|
if BeaconNodeRole.BlockProposalPublish in roles: r[2] = '+' else: r[2] = '-'
|
|
if BeaconNodeRole.SyncCommitteeData in roles:
|
|
if BeaconNodeRole.SyncCommitteePublish in roles:
|
|
r[3] = 'S' else: r[3] = 's'
|
|
else:
|
|
if BeaconNodeRole.SyncCommitteePublish in roles:
|
|
r[3] = '+' else: r[3] = '-'
|
|
if BeaconNodeRole.Duties in roles: r[4] = 'D' else: r[4] = '-'
|
|
r
|
|
|
|
proc `$`*(bn: BeaconNodeServerRef): string =
|
|
if bn.ident.isSome():
|
|
bn.logIdent & "[" & bn.ident.get() & "]"
|
|
else:
|
|
bn.logIdent
|
|
|
|
proc validatorLog*(key: ValidatorPubKey,
|
|
index: ValidatorIndex): string =
|
|
var res = shortLog(key)
|
|
res.add('@')
|
|
res.add(Base10.toString(uint64(index)))
|
|
res
|
|
|
|
chronicles.expandIt(BeaconNodeServerRef):
|
|
node = $it
|
|
node_index = it.index
|
|
node_roles = shortLog(it.roles)
|
|
|
|
chronicles.expandIt(RestAttesterDuty):
|
|
pubkey = shortLog(it.pubkey)
|
|
slot = it.slot
|
|
validator_index = it.validator_index
|
|
committee_index = it.committee_index
|
|
committee_length = it.committee_length
|
|
committees_at_slot = it.committees_at_slot
|
|
validator_committee_index = it.validator_committee_index
|
|
|
|
chronicles.expandIt(SyncCommitteeDuty):
|
|
pubkey = shortLog(it.pubkey)
|
|
validator_index = it.validator_index
|
|
validator_sync_committee_index = it.validator_sync_committee_index
|
|
|
|
proc stop*(csr: ClientServiceRef) {.async.} =
|
|
debug "Stopping service", service = csr.name
|
|
if csr.state == ServiceState.Running:
|
|
csr.state = ServiceState.Closing
|
|
if not(csr.lifeFut.finished()):
|
|
await csr.lifeFut.cancelAndWait()
|
|
csr.state = ServiceState.Closed
|
|
debug "Service stopped", service = csr.name
|
|
|
|
proc isDefault*(dap: DutyAndProof): bool =
|
|
dap.epoch == Epoch(0xFFFF_FFFF_FFFF_FFFF'u64)
|
|
|
|
proc isDefault*(prd: ProposedData): bool =
|
|
prd.epoch == Epoch(0xFFFF_FFFF_FFFF_FFFF'u64)
|
|
|
|
proc parseRoles*(data: string): Result[set[BeaconNodeRole], cstring] =
|
|
var res: set[BeaconNodeRole]
|
|
if len(data) == 0:
|
|
return ok(AllBeaconNodeRoles)
|
|
let parts = data.split("roles=")
|
|
if (len(parts) != 2) or (len(parts[0]) != 0):
|
|
return err("Invalid beacon node roles string")
|
|
let sroles = parts[1].split(",")
|
|
for srole in sroles:
|
|
case toLower(strip(srole))
|
|
of "":
|
|
discard
|
|
of "all":
|
|
res.incl(AllBeaconNodeRoles)
|
|
of "attestation":
|
|
res.incl({BeaconNodeRole.AttestationData,
|
|
BeaconNodeRole.AttestationPublish})
|
|
of "block":
|
|
res.incl({BeaconNodeRole.BlockProposalData,
|
|
BeaconNodeRole.BlockProposalPublish})
|
|
of "aggregated":
|
|
res.incl({BeaconNodeRole.AggregatedData,
|
|
BeaconNodeRole.AggregatedPublish})
|
|
of "sync":
|
|
res.incl({BeaconNodeRole.SyncCommitteeData,
|
|
BeaconNodeRole.SyncCommitteePublish})
|
|
of "attestation-data":
|
|
res.incl(BeaconNodeRole.AttestationData)
|
|
of "attestation-publish":
|
|
res.incl(BeaconNodeRole.AttestationPublish)
|
|
of "aggregated-data":
|
|
res.incl(BeaconNodeRole.AggregatedData)
|
|
of "aggregated-publish":
|
|
res.incl(BeaconNodeRole.AggregatedPublish)
|
|
of "block-data":
|
|
res.incl(BeaconNodeRole.BlockProposalData)
|
|
of "block-publish":
|
|
res.incl(BeaconNodeRole.BlockProposalPublish)
|
|
of "sync-data":
|
|
res.incl(BeaconNodeRole.SyncCommitteeData)
|
|
of "sync-publish":
|
|
res.incl(BeaconNodeRole.SyncCommitteePublish)
|
|
of "duties":
|
|
res.incl(BeaconNodeRole.Duties)
|
|
else:
|
|
return err("Invalid beacon node role string found")
|
|
ok(res)
|
|
|
|
proc init*(t: typedesc[BeaconNodeServerRef], remote: Uri,
|
|
index: int): Result[BeaconNodeServerRef, string] =
|
|
doAssert(index >= 0)
|
|
let
|
|
flags = {RestClientFlag.CommaSeparatedArray}
|
|
client =
|
|
block:
|
|
let res = RestClientRef.new($remote, flags = flags)
|
|
if res.isErr(): return err($res.error())
|
|
res.get()
|
|
roles =
|
|
block:
|
|
let res = parseRoles(remote.anchor)
|
|
if res.isErr(): return err($res.error())
|
|
res.get()
|
|
|
|
let server = BeaconNodeServerRef(
|
|
client: client, endpoint: $remote, index: index, roles: roles,
|
|
logIdent: client.address.hostname & ":" &
|
|
Base10.toString(client.address.port)
|
|
)
|
|
ok(server)
|
|
|
|
proc getMissingRoles*(n: openArray[BeaconNodeServerRef]): set[BeaconNodeRole] =
|
|
var res: set[BeaconNodeRole] = AllBeaconNodeRoles
|
|
for node in n.items():
|
|
res.excl(node.roles)
|
|
res
|
|
|
|
proc init*(t: typedesc[DutyAndProof], epoch: Epoch, dependentRoot: Eth2Digest,
|
|
duty: RestAttesterDuty,
|
|
slotSig: Option[ValidatorSig]): DutyAndProof =
|
|
DutyAndProof(epoch: epoch, dependentRoot: dependentRoot, data: duty,
|
|
slotSig: slotSig)
|
|
|
|
proc init*(t: typedesc[ProposedData], epoch: Epoch, dependentRoot: Eth2Digest,
|
|
data: openArray[ProposerTask]): ProposedData =
|
|
ProposedData(epoch: epoch, dependentRoot: dependentRoot, duties: @data)
|
|
|
|
proc getCurrentSlot*(vc: ValidatorClientRef): Option[Slot] =
|
|
let
|
|
wallTime = vc.beaconClock.now()
|
|
wallSlot = wallTime.toSlot()
|
|
|
|
if not(wallSlot.afterGenesis):
|
|
let checkGenesisTime = vc.beaconClock.fromNow(start_beacon_time(Slot(0)))
|
|
warn "Jump in time detected, something wrong with wallclock",
|
|
wall_time = wallTime, genesisIn = checkGenesisTime.offset
|
|
none[Slot]()
|
|
else:
|
|
some(wallSlot.slot)
|
|
|
|
proc getAttesterDutiesForSlot*(vc: ValidatorClientRef,
|
|
slot: Slot): seq[DutyAndProof] =
|
|
## Returns all `DutyAndProof` for the given `slot`.
|
|
var res: seq[DutyAndProof]
|
|
let epoch = slot.epoch()
|
|
for key, item in vc.attesters:
|
|
let duty = item.duties.getOrDefault(epoch, DefaultDutyAndProof)
|
|
if not(duty.isDefault()):
|
|
if duty.data.slot == slot:
|
|
res.add(duty)
|
|
res
|
|
|
|
proc getSyncCommitteeDutiesForSlot*(vc: ValidatorClientRef,
|
|
slot: Slot): seq[SyncCommitteeDuty] =
|
|
## Returns all `SyncCommitteeDuty` for the given `slot`.
|
|
var res: seq[SyncCommitteeDuty]
|
|
let epoch = slot.epoch()
|
|
for key, item in mpairs(vc.syncCommitteeDuties):
|
|
item.duties.withValue(epoch, duty):
|
|
res.add(duty[])
|
|
res
|
|
|
|
proc getDurationToNextAttestation*(vc: ValidatorClientRef,
|
|
slot: Slot): string =
|
|
var minSlot = FAR_FUTURE_SLOT
|
|
let currentEpoch = slot.epoch()
|
|
for epoch in [currentEpoch, currentEpoch + 1'u64]:
|
|
for key, item in vc.attesters:
|
|
let duty = item.duties.getOrDefault(epoch, DefaultDutyAndProof)
|
|
if not(duty.isDefault()):
|
|
let dutySlotTime = duty.data.slot
|
|
if (duty.data.slot < minSlot) and (duty.data.slot >= slot):
|
|
minSlot = duty.data.slot
|
|
if minSlot != FAR_FUTURE_SLOT:
|
|
break
|
|
|
|
if minSlot == FAR_FUTURE_SLOT:
|
|
"<unknown>"
|
|
else:
|
|
$(minSlot.attestation_deadline() - slot.start_beacon_time())
|
|
|
|
proc getDurationToNextBlock*(vc: ValidatorClientRef, slot: Slot): string =
|
|
var minSlot = FAR_FUTURE_SLOT
|
|
let currentEpoch = slot.epoch()
|
|
for epoch in [currentEpoch, currentEpoch + 1'u64]:
|
|
let data = vc.proposers.getOrDefault(epoch)
|
|
if not(data.isDefault()):
|
|
for item in data.duties:
|
|
if item.duty.pubkey in vc.attachedValidators[]:
|
|
if (item.duty.slot < minSlot) and (item.duty.slot >= slot):
|
|
minSlot = item.duty.slot
|
|
if minSlot != FAR_FUTURE_SLOT:
|
|
break
|
|
if minSlot == FAR_FUTURE_SLOT:
|
|
"<unknown>"
|
|
else:
|
|
$(minSlot.block_deadline() - slot.start_beacon_time())
|
|
|
|
iterator attesterDutiesForEpoch*(vc: ValidatorClientRef,
|
|
epoch: Epoch): DutyAndProof =
|
|
for key, item in vc.attesters:
|
|
let epochDuties = item.duties.getOrDefault(epoch)
|
|
if not(isDefault(epochDuties)):
|
|
yield epochDuties
|
|
|
|
proc syncMembersSubscriptionInfoForEpoch*(
|
|
vc: ValidatorClientRef,
|
|
epoch: Epoch): seq[SyncCommitteeSubscriptionInfo] =
|
|
var res: seq[SyncCommitteeSubscriptionInfo]
|
|
for key, item in mpairs(vc.syncCommitteeDuties):
|
|
var cur: SyncCommitteeSubscriptionInfo
|
|
var initialized = false
|
|
|
|
item.duties.withValue(epoch, epochDuties):
|
|
if not initialized:
|
|
cur.validator_index = epochDuties.validator_index
|
|
initialized = true
|
|
cur.validator_sync_committee_indices.add(
|
|
epochDuties.validator_sync_committee_index)
|
|
|
|
if initialized:
|
|
res.add cur
|
|
|
|
res
|
|
|
|
proc getDelay*(vc: ValidatorClientRef, deadline: BeaconTime): TimeDiff =
|
|
vc.beaconClock.now() - deadline
|
|
|
|
proc getValidator*(vc: ValidatorClientRef,
|
|
key: ValidatorPubKey): Opt[AttachedValidator] =
|
|
let validator = vc.attachedValidators[].getValidator(key)
|
|
if isNil(validator):
|
|
info "Validator not in pool anymore", validator = shortLog(validator)
|
|
Opt.none(AttachedValidator)
|
|
else:
|
|
if validator.index.isNone():
|
|
info "Validator index is missing", validator = shortLog(validator)
|
|
Opt.none(AttachedValidator)
|
|
else:
|
|
Opt.some(validator)
|
|
|
|
proc forkAtEpoch*(vc: ValidatorClientRef, epoch: Epoch): Fork =
|
|
# If schedule is present, it MUST not be empty.
|
|
doAssert(len(vc.forks) > 0)
|
|
var res: Fork
|
|
for item in vc.forks:
|
|
if item.epoch <= epoch:
|
|
res = item
|
|
else:
|
|
break
|
|
res
|
|
|
|
proc getSubcommitteeIndex*(index: IndexInSyncCommittee): SyncSubcommitteeIndex =
|
|
SyncSubcommitteeIndex(uint16(index) div SYNC_SUBCOMMITTEE_SIZE)
|
|
|
|
proc currentSlot*(vc: ValidatorClientRef): Slot =
|
|
vc.beaconClock.now().slotOrZero()
|
|
|
|
proc addDoppelganger*(vc: ValidatorClientRef,
|
|
validators: openArray[AttachedValidator]) =
|
|
if vc.config.doppelgangerDetection:
|
|
let startEpoch = vc.currentSlot().epoch()
|
|
var
|
|
check: seq[string]
|
|
skip: seq[string]
|
|
exist: seq[string]
|
|
|
|
for validator in validators:
|
|
let
|
|
vindex = validator.index.get()
|
|
state =
|
|
if (startEpoch == GENESIS_EPOCH) and
|
|
(validator.startSlot == GENESIS_SLOT):
|
|
DoppelgangerState(startEpoch: startEpoch, epochsCount: 0'u64,
|
|
lastAttempt: DoppelgangerAttempt.None,
|
|
status: DoppelgangerStatus.Passed)
|
|
else:
|
|
if validator.activationEpoch.isSome() and
|
|
(validator.activationEpoch.get() >= startEpoch):
|
|
DoppelgangerState(startEpoch: startEpoch, epochsCount: 0'u64,
|
|
lastAttempt: DoppelgangerAttempt.None,
|
|
status: DoppelgangerStatus.Passed)
|
|
else:
|
|
DoppelgangerState(startEpoch: startEpoch, epochsCount: 0'u64,
|
|
lastAttempt: DoppelgangerAttempt.None,
|
|
status: DoppelgangerStatus.Checking)
|
|
res = vc.doppelgangerDetection.validators.hasKeyOrPut(vindex, state)
|
|
if res:
|
|
exist.add(validatorLog(validator.pubkey, vindex))
|
|
else:
|
|
if state.status == DoppelgangerStatus.Checking:
|
|
check.add(validatorLog(validator.pubkey, vindex))
|
|
else:
|
|
skip.add(validatorLog(validator.pubkey, vindex))
|
|
info "Validator's doppelganger protection activated",
|
|
validators_count = len(validators),
|
|
pending_check_count = len(check),
|
|
skipped_count = len(skip),
|
|
exist_count = len(exist)
|
|
debug "Validator's doppelganger protection dump",
|
|
checking_validators = check,
|
|
skip_validators = skip,
|
|
existing_validators = exist
|
|
|
|
proc addDoppelganger*(vc: ValidatorClientRef, validator: AttachedValidator) =
|
|
logScope:
|
|
validator = shortLog(validator)
|
|
|
|
if vc.config.doppelgangerDetection:
|
|
let
|
|
vindex = validator.index.get()
|
|
startEpoch = vc.currentSlot().epoch()
|
|
state =
|
|
if (startEpoch == GENESIS_EPOCH) and
|
|
(validator.startSlot == GENESIS_SLOT):
|
|
DoppelgangerState(startEpoch: startEpoch, epochsCount: 0'u64,
|
|
lastAttempt: DoppelgangerAttempt.None,
|
|
status: DoppelgangerStatus.Passed)
|
|
else:
|
|
DoppelgangerState(startEpoch: startEpoch, epochsCount: 0'u64,
|
|
lastAttempt: DoppelgangerAttempt.None,
|
|
status: DoppelgangerStatus.Checking)
|
|
res = vc.doppelgangerDetection.validators.hasKeyOrPut(vindex, state)
|
|
|
|
if res:
|
|
warn "Validator is already in doppelganger table",
|
|
validator_index = vindex, start_epoch = startEpoch,
|
|
start_slot = validator.startSlot
|
|
else:
|
|
if state.status == DoppelgangerStatus.Checking:
|
|
info "Doppelganger protection activated", validator_index = vindex,
|
|
start_epoch = startEpoch, start_slot = validator.startSlot
|
|
else:
|
|
info "Doppelganger protection skipped", validator_index = vindex,
|
|
start_epoch = startEpoch, start_slot = validator.startSlot
|
|
|
|
proc removeDoppelganger*(vc: ValidatorClientRef, index: ValidatorIndex) =
|
|
if vc.config.doppelgangerDetection:
|
|
var state: DoppelgangerState
|
|
# We do not care about race condition, when validator is not yet added to
|
|
# the doppelganger's table, but it should be removed.
|
|
discard vc.doppelgangerDetection.validators.pop(index, state)
|
|
|
|
proc addValidator*(vc: ValidatorClientRef, keystore: KeystoreData) =
|
|
let
|
|
slot = vc.currentSlot()
|
|
feeRecipient = vc.config.validatorsDir.getSuggestedFeeRecipient(
|
|
keystore.pubkey, vc.config.defaultFeeRecipient).valueOr(
|
|
vc.config.defaultFeeRecipient)
|
|
case keystore.kind
|
|
of KeystoreKind.Local:
|
|
vc.attachedValidators[].addLocalValidator(keystore, Opt.none ValidatorIndex,
|
|
feeRecipient, slot,
|
|
Opt.none(Epoch))
|
|
of KeystoreKind.Remote:
|
|
let
|
|
httpFlags =
|
|
block:
|
|
var res: set[HttpClientFlag]
|
|
if RemoteKeystoreFlag.IgnoreSSLVerification in keystore.flags:
|
|
res.incl({HttpClientFlag.NoVerifyHost,
|
|
HttpClientFlag.NoVerifyServerName})
|
|
res
|
|
prestoFlags = {RestClientFlag.CommaSeparatedArray}
|
|
clients =
|
|
block:
|
|
var res: seq[(RestClientRef, RemoteSignerInfo)]
|
|
for remote in keystore.remotes:
|
|
let client = RestClientRef.new($remote.url, prestoFlags,
|
|
httpFlags)
|
|
if client.isErr():
|
|
warn "Unable to resolve distributed signer address",
|
|
remote_url = $remote.url, validator = $remote.pubkey
|
|
else:
|
|
res.add((client.get(), remote))
|
|
res
|
|
if len(clients) > 0:
|
|
vc.attachedValidators[].addRemoteValidator(keystore, clients,
|
|
Opt.none(ValidatorIndex),
|
|
feeRecipient, slot,
|
|
Opt.none(Epoch))
|
|
else:
|
|
warn "Unable to initialize remote validator",
|
|
validator = $keystore.pubkey
|
|
|
|
proc removeValidator*(vc: ValidatorClientRef,
|
|
pubkey: ValidatorPubKey) {.async.} =
|
|
let validator = vc.attachedValidators[].getValidator(pubkey)
|
|
if not(isNil(validator)):
|
|
if vc.config.doppelgangerDetection:
|
|
if validator.index.isSome():
|
|
vc.removeDoppelganger(validator.index.get())
|
|
case validator.kind
|
|
of ValidatorKind.Local:
|
|
discard
|
|
of ValidatorKind.Remote:
|
|
# We must close all the REST clients running for the remote validator.
|
|
let pending =
|
|
block:
|
|
var res: seq[Future[void]]
|
|
for item in validator.clients:
|
|
res.add(item[0].closeWait())
|
|
res
|
|
await allFutures(pending)
|
|
# Remove validator from ValidatorPool.
|
|
vc.attachedValidators[].removeValidator(pubkey)
|
|
|
|
proc doppelgangerCheck*(vc: ValidatorClientRef,
|
|
validator: AttachedValidator): bool =
|
|
if vc.config.doppelgangerDetection:
|
|
if validator.index.isNone():
|
|
false
|
|
else:
|
|
let
|
|
vindex = validator.index.get()
|
|
default = DoppelgangerState(status: DoppelgangerStatus.None)
|
|
state = vc.doppelgangerDetection.validators.getOrDefault(vindex,
|
|
default)
|
|
state.status == DoppelgangerStatus.Passed
|
|
else:
|
|
true
|
|
|
|
proc doppelgangerCheck*(vc: ValidatorClientRef,
|
|
key: ValidatorPubKey): bool =
|
|
let validator = vc.getValidator(key).valueOr: return false
|
|
vc.doppelgangerCheck(validator)
|
|
|
|
proc doppelgangerFilter*(
|
|
vc: ValidatorClientRef,
|
|
duties: openArray[DutyAndProof]
|
|
): tuple[filtered: seq[DutyAndProof], skipped: seq[string]] =
|
|
var
|
|
pending: seq[string]
|
|
ready: seq[DutyAndProof]
|
|
for duty in duties:
|
|
let
|
|
vindex = duty.data.validator_index
|
|
vkey = duty.data.pubkey
|
|
if vc.doppelgangerCheck(vkey):
|
|
ready.add(duty)
|
|
else:
|
|
pending.add(validatorLog(vkey, vindex))
|
|
(ready, pending)
|
|
|
|
proc getFeeRecipient*(vc: ValidatorClientRef, pubkey: ValidatorPubKey,
|
|
validatorIdx: ValidatorIndex,
|
|
epoch: Epoch): Opt[Eth1Address] =
|
|
let dynamicRecipient = vc.dynamicFeeRecipientsStore[].getDynamicFeeRecipient(
|
|
validatorIdx, epoch)
|
|
if dynamicRecipient.isSome():
|
|
Opt.some(dynamicRecipient.get())
|
|
else:
|
|
let staticRecipient = getSuggestedFeeRecipient(
|
|
vc.config.validatorsDir, pubkey, vc.config.defaultFeeRecipient)
|
|
if staticRecipient.isOk():
|
|
Opt.some(staticRecipient.get())
|
|
else:
|
|
Opt.none(Eth1Address)
|
|
|
|
proc prepareProposersList*(vc: ValidatorClientRef,
|
|
epoch: Epoch): seq[PrepareBeaconProposer] =
|
|
var res: seq[PrepareBeaconProposer]
|
|
for validator in vc.attachedValidators[].items():
|
|
if validator.index.isSome():
|
|
let
|
|
index = validator.index.get()
|
|
feeRecipient = vc.getFeeRecipient(validator.pubkey, index, epoch)
|
|
if feeRecipient.isSome():
|
|
res.add(PrepareBeaconProposer(validator_index: index,
|
|
fee_recipient: feeRecipient.get()))
|
|
res
|
|
|
|
proc isDefault*(reg: SignedValidatorRegistrationV1): bool =
|
|
(reg.message.timestamp == 0'u64) or (reg.message.gas_limit == 0'u64)
|
|
|
|
proc isExpired*(vc: ValidatorClientRef,
|
|
reg: SignedValidatorRegistrationV1, slot: Slot): bool =
|
|
let
|
|
regTime = fromUnix(int64(reg.message.timestamp))
|
|
regSlot =
|
|
block:
|
|
let res = vc.beaconClock.toSlot(regTime)
|
|
if not(res.afterGenesis):
|
|
# This case should not be happend, but it could in case of time jumps
|
|
# (time could be modified by admin or ntpd).
|
|
return false
|
|
uint64(res.slot)
|
|
|
|
if regSlot > slot:
|
|
# This case should not be happened, but if it happens (time could be
|
|
# modified by admin or ntpd).
|
|
false
|
|
else:
|
|
if (slot - regSlot) div SLOTS_PER_EPOCH >=
|
|
EPOCHS_BETWEEN_VALIDATOR_REGISTRATION:
|
|
false
|
|
else:
|
|
true
|
|
|
|
proc getValidatorRegistraion(
|
|
vc: ValidatorClientRef,
|
|
validator: AttachedValidator,
|
|
timestamp: Time,
|
|
fork: Fork
|
|
): Result[PendingValidatorRegistration, RegistrationKind] =
|
|
if validator.index.isNone():
|
|
debug "Validator registration missing validator index",
|
|
validator = shortLog(validator)
|
|
return err(RegistrationKind.MissingIndex)
|
|
|
|
let
|
|
vindex = validator.index.get()
|
|
cached = vc.validatorsRegCache.getOrDefault(validator.pubkey)
|
|
currentSlot =
|
|
block:
|
|
let res = vc.beaconClock.toSlot(timestamp)
|
|
if not(res.afterGenesis):
|
|
return err(RegistrationKind.IncorrectTime)
|
|
res.slot
|
|
|
|
if cached.isDefault() or vc.isExpired(cached, currentSlot):
|
|
let feeRecipient = vc.getFeeRecipient(validator.pubkey, vindex,
|
|
currentSlot.epoch())
|
|
if feeRecipient.isNone():
|
|
debug "Could not get fee recipient for registration data",
|
|
validator = shortLog(validator)
|
|
return err(RegistrationKind.MissingFee)
|
|
|
|
var registration =
|
|
SignedValidatorRegistrationV1(
|
|
message: ValidatorRegistrationV1(
|
|
fee_recipient:
|
|
ExecutionAddress(data: distinctBase(feeRecipient.get())),
|
|
gas_limit: VALIDATOR_DEFAULT_GAS_LIMIT,
|
|
timestamp: uint64(timestamp.toUnix()),
|
|
pubkey: validator.pubkey
|
|
)
|
|
)
|
|
|
|
let sigfut = validator.getBuilderSignature(fork, registration.message)
|
|
if sigfut.finished():
|
|
# This is short-path if we able to create signature locally.
|
|
if not(sigfut.done()):
|
|
let exc = sigfut.readError()
|
|
debug "Got unexpected exception while signing validator registration",
|
|
validator = shortLog(validator), error_name = $exc.name,
|
|
error_msg = $exc.msg
|
|
return err(RegistrationKind.ErrorSignature)
|
|
let sigres = sigfut.read()
|
|
if sigres.isErr():
|
|
debug "Failed to get signature for validator registration",
|
|
validator = shortLog(validator), error = sigres.error()
|
|
return err(RegistrationKind.NoSignature)
|
|
registration.signature = sigres.get()
|
|
# Updating cache table with new signed registration data
|
|
vc.validatorsRegCache[registration.message.pubkey] = registration
|
|
ok(PendingValidatorRegistration(registration: registration, future: nil))
|
|
else:
|
|
# Remote signature service involved, cache will be updated later.
|
|
ok(PendingValidatorRegistration(registration: registration,
|
|
future: sigfut))
|
|
else:
|
|
# Returning cached result.
|
|
err(RegistrationKind.Cached)
|
|
|
|
proc prepareRegistrationList*(
|
|
vc: ValidatorClientRef,
|
|
timestamp: Time,
|
|
fork: Fork
|
|
): Future[seq[SignedValidatorRegistrationV1]] {.async.} =
|
|
|
|
var
|
|
messages: seq[SignedValidatorRegistrationV1]
|
|
futures: seq[Future[SignatureResult]]
|
|
registrations: seq[SignedValidatorRegistrationV1]
|
|
total = vc.attachedValidators[].count()
|
|
succeed = 0
|
|
bad = 0
|
|
errors = 0
|
|
indexMissing = 0
|
|
feeMissing = 0
|
|
cached = 0
|
|
timed = 0
|
|
|
|
for validator in vc.attachedValidators[].items():
|
|
let res = vc.getValidatorRegistraion(validator, timestamp, fork)
|
|
if res.isOk():
|
|
let preg = res.get()
|
|
if preg.future.isNil():
|
|
registrations.add(preg.registration)
|
|
else:
|
|
messages.add(preg.registration)
|
|
futures.add(preg.future)
|
|
else:
|
|
case res.error()
|
|
of RegistrationKind.Cached: inc(cached)
|
|
of RegistrationKind.IncorrectTime: inc(timed)
|
|
of RegistrationKind.NoSignature: inc(bad)
|
|
of RegistrationKind.ErrorSignature: inc(errors)
|
|
of RegistrationKind.MissingIndex: inc(indexMissing)
|
|
of RegistrationKind.MissingFee: inc(feeMissing)
|
|
|
|
succeed = len(registrations)
|
|
|
|
if len(futures) > 0:
|
|
await allFutures(futures)
|
|
|
|
for index, future in futures.pairs():
|
|
if future.done():
|
|
let sres = future.read()
|
|
if sres.isOk():
|
|
var reg = messages[index]
|
|
reg.signature = sres.get()
|
|
registrations.add(reg)
|
|
# Updating cache table
|
|
vc.validatorsRegCache[reg.message.pubkey] = reg
|
|
inc(succeed)
|
|
else:
|
|
inc(bad)
|
|
else:
|
|
inc(errors)
|
|
|
|
debug "Validator registrations prepared", total = total, succeed = succeed,
|
|
cached = cached, bad = bad, errors = errors,
|
|
index_missing = indexMissing, fee_missing = feeMissing,
|
|
incorrect_time = timed
|
|
|
|
return registrations
|