nimbus-eth2/beacon_chain/validators/validator_duties.nim

1498 lines
59 KiB
Nim
Raw Normal View History

# beacon_chain
# Copyright (c) 2018-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.
{.push raises: [Defect].}
# References to `vFuture` refer to the pre-release proposal of the libp2p based
# light client sync protocol. Conflicting release versions are not in use.
# https://github.com/ethereum/consensus-specs/pull/2802
import
# Standard library
std/[os, osproc, sequtils, streams, tables],
# Nimble packages
stew/[assign2, byteutils, objects],
chronos, metrics,
chronicles, chronicles/timings,
json_serialization/std/[options, sets, net], serialization/errors,
eth/db/kvstore,
2021-05-12 12:31:02 +00:00
eth/keys, eth/p2p/discoveryv5/[protocol, enr],
2021-12-17 12:23:32 +00:00
web3/ethtypes,
# Local modules
../spec/datatypes/[phase0, altair, bellatrix],
disentangle eth2 types from the ssz library (#2785) * reorganize ssz dependencies This PR continues the work in https://github.com/status-im/nimbus-eth2/pull/2646, https://github.com/status-im/nimbus-eth2/pull/2779 as well as past issues with serialization and type, to disentangle SSZ from eth2 and at the same time simplify imports and exports with a structured approach. The principal idea here is that when a library wants to introduce SSZ support, they do so via 3 files: * `ssz_codecs` which imports and reexports `codecs` - this covers the basic byte conversions and ensures no overloads get lost * `xxx_merkleization` imports and exports `merkleization` to specialize and get access to `hash_tree_root` and friends * `xxx_ssz_serialization` imports and exports `ssz_serialization` to specialize ssz for a specific library Those that need to interact with SSZ always import the `xxx_` versions of the modules and never `ssz` itself so as to keep imports simple and safe. This is similar to how the REST / JSON-RPC serializers are structured in that someone wanting to serialize spec types to REST-JSON will import `eth2_rest_serialization` and nothing else. * split up ssz into a core library that is independendent of eth2 types * rename `bytes_reader` to `codec` to highlight that it contains coding and decoding of bytes and native ssz types * remove tricky List init overload that causes compile issues * get rid of top-level ssz import * reenable merkleization tests * move some "standard" json serializers to spec * remove `ValidatorIndex` serialization for now * remove test_ssz_merkleization * add tests for over/underlong byte sequences * fix broken seq[byte] test - seq[byte] is not an SSZ type There are a few things this PR doesn't solve: * like #2646 this PR is weak on how to handle root and other dontSerialize fields that "sometimes" should be computed - the same problem appears in REST / JSON-RPC etc * Fix a build problem on macOS * Another way to fix the macOS builds Co-authored-by: Zahary Karadjov <zahary@gmail.com>
2021-08-18 18:57:58 +00:00
../spec/[
eth2_merkleization, forks, helpers, network, signatures, state_transition,
validator],
../consensus_object_pools/[
spec_cache, blockchain_dag, block_clearance, attestation_pool, exit_pool,
sync_committee_msg_pool],
../eth1/eth1_monitor,
../networking/eth2_network,
disentangle eth2 types from the ssz library (#2785) * reorganize ssz dependencies This PR continues the work in https://github.com/status-im/nimbus-eth2/pull/2646, https://github.com/status-im/nimbus-eth2/pull/2779 as well as past issues with serialization and type, to disentangle SSZ from eth2 and at the same time simplify imports and exports with a structured approach. The principal idea here is that when a library wants to introduce SSZ support, they do so via 3 files: * `ssz_codecs` which imports and reexports `codecs` - this covers the basic byte conversions and ensures no overloads get lost * `xxx_merkleization` imports and exports `merkleization` to specialize and get access to `hash_tree_root` and friends * `xxx_ssz_serialization` imports and exports `ssz_serialization` to specialize ssz for a specific library Those that need to interact with SSZ always import the `xxx_` versions of the modules and never `ssz` itself so as to keep imports simple and safe. This is similar to how the REST / JSON-RPC serializers are structured in that someone wanting to serialize spec types to REST-JSON will import `eth2_rest_serialization` and nothing else. * split up ssz into a core library that is independendent of eth2 types * rename `bytes_reader` to `codec` to highlight that it contains coding and decoding of bytes and native ssz types * remove tricky List init overload that causes compile issues * get rid of top-level ssz import * reenable merkleization tests * move some "standard" json serializers to spec * remove `ValidatorIndex` serialization for now * remove test_ssz_merkleization * add tests for over/underlong byte sequences * fix broken seq[byte] test - seq[byte] is not an SSZ type There are a few things this PR doesn't solve: * like #2646 this PR is weak on how to handle root and other dontSerialize fields that "sometimes" should be computed - the same problem appears in REST / JSON-RPC etc * Fix a build problem on macOS * Another way to fix the macOS builds Co-authored-by: Zahary Karadjov <zahary@gmail.com>
2021-08-18 18:57:58 +00:00
../sszdump, ../sync/sync_manager,
../gossip_processing/[block_processor, consensus_manager],
".."/[conf, beacon_clock, beacon_node, version],
"."/[slashing_protection, validator_pool, keystore_management]
from eth/async_utils import awaitWithTimeout
from web3/engine_api import ForkchoiceUpdatedResponse
from web3/engine_api_types import PayloadExecutionStatus
# Metrics for tracking attestation and beacon block loss
const 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]
declareCounter beacon_attestations_sent,
"Number of beacon chain attestations sent by this peer"
declareHistogram beacon_attestation_sent_delay,
"Time(s) between slot start and attestation sent moment",
buckets = delayBuckets
declareCounter beacon_sync_committee_messages_sent,
"Number of sync committee messages sent by this peer"
declareCounter beacon_sync_committee_contributions_sent,
"Number of sync committee contributions sent by this peer"
declareHistogram beacon_sync_committee_message_sent_delay,
"Time(s) between slot start and sync committee message sent moment",
buckets = delayBuckets
declareCounter beacon_light_client_finality_updates_sent,
"Number of LC finality updates sent by this peer"
declareCounter beacon_light_client_optimistic_updates_sent,
"Number of LC optimistic updates sent by this peer"
declareCounter beacon_blocks_proposed,
"Number of beacon chain blocks sent by this peer"
2020-11-27 23:34:25 +00:00
declareGauge(attached_validator_balance,
"Validator balance at slot end of the first 64 validators, in Gwei",
labels = ["pubkey"])
declarePublicGauge(attached_validator_balance_total,
2020-11-27 23:34:25 +00:00
"Validator balance of all attached validators, in Gwei")
logScope: topics = "beacval"
type
SendResult* = Result[void, cstring]
SendBlockResult* = Result[bool, cstring]
ForkedBlockResult* = Result[ForkedBeaconBlock, string]
proc findValidator(validators: auto, pubkey: ValidatorPubKey):
2020-11-27 23:34:25 +00:00
Option[ValidatorIndex] =
let idx = validators.findIt(it.pubkey == pubkey)
if idx == -1:
# We allow adding a validator even if its key is not in the state registry:
# it might be that the deposit for this validator has not yet been processed
notice "Validator deposit not yet processed, monitoring", pubkey
2020-11-27 23:34:25 +00:00
none(ValidatorIndex)
else:
some(idx.ValidatorIndex)
proc addLocalValidator(node: BeaconNode, validators: auto,
item: KeystoreData) =
2021-11-30 01:20:21 +00:00
let
pubkey = item.pubkey
index = findValidator(validators, pubkey)
2021-11-30 01:20:21 +00:00
node.attachedValidators[].addLocalValidator(item, index)
proc addRemoteValidator(pool: var ValidatorPool, validators: auto,
item: KeystoreData) =
var clients: seq[(RestClientRef, RemoteSignerInfo)]
2021-11-30 01:20:21 +00:00
let httpFlags =
block:
var res: set[HttpClientFlag]
if RemoteKeystoreFlag.IgnoreSSLVerification in item.flags:
res.incl({HttpClientFlag.NoVerifyHost,
HttpClientFlag.NoVerifyServerName})
res
let prestoFlags = {RestClientFlag.CommaSeparatedArray}
for remote in item.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
clients.add((client.get(), remote))
let index = findValidator(validators, item.pubkey)
pool.addRemoteValidator(item, clients, index)
2021-11-30 01:20:21 +00:00
proc addLocalValidators*(node: BeaconNode,
validators: openArray[KeystoreData]) =
withState(node.dag.headState):
2021-11-30 01:20:21 +00:00
for item in validators:
node.addLocalValidator(state.data.validators.asSeq(), item)
proc addRemoteValidators*(node: BeaconNode,
validators: openArray[KeystoreData]) =
withState(node.dag.headState):
2021-11-30 01:20:21 +00:00
for item in validators:
node.attachedValidators[].addRemoteValidator(
state.data.validators.asSeq(), item)
2021-11-30 01:20:21 +00:00
proc addValidators*(node: BeaconNode) =
let (localValidators, remoteValidators) =
block:
var local, remote, distributed: seq[KeystoreData]
for keystore in listLoadableKeystores(node.config):
case keystore.kind
of KeystoreKind.Local:
local.add(keystore)
of KeystoreKind.Remote:
remote.add(keystore)
2021-11-30 01:20:21 +00:00
(local, remote)
node.addLocalValidators(localValidators)
node.addRemoteValidators(remoteValidators)
2020-08-10 13:21:31 +00:00
proc getAttachedValidator*(node: BeaconNode,
pubkey: ValidatorPubKey): AttachedValidator =
node.attachedValidators[].getValidator(pubkey)
2020-08-10 13:21:31 +00:00
proc getAttachedValidator*(node: BeaconNode,
state_validators: auto,
idx: ValidatorIndex): AttachedValidator =
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
if uint64(idx) < state_validators.lenu64:
let validator = node.getAttachedValidator(state_validators[idx].pubkey)
if validator != nil and validator.index != some(idx):
2020-11-27 23:34:25 +00:00
# Update index, in case the validator was activated!
notice "Validator activated", pubkey = validator.pubkey, index = idx
validator.index = some(idx)
2020-11-27 23:34:25 +00:00
validator
2020-08-10 13:21:31 +00:00
else:
warn "Validator index out of bounds",
idx, validators = state_validators.len
2020-08-10 13:21:31 +00:00
nil
proc getAttachedValidator*(node: BeaconNode,
epochRef: EpochRef,
idx: ValidatorIndex): AttachedValidator =
let key = epochRef.validatorKey(idx)
if key.isSome():
let validator = node.getAttachedValidator(key.get().toPubKey())
if validator != nil and validator.index != some(idx):
2020-11-27 23:34:25 +00:00
# Update index, in case the validator was activated!
notice "Validator activated", pubkey = validator.pubkey, index = idx
validator.index = some(idx)
2020-11-27 23:34:25 +00:00
validator
2020-08-10 13:21:31 +00:00
else:
warn "Validator key not found",
idx, epoch = epochRef.epoch
2020-08-10 13:21:31 +00:00
nil
proc isSynced*(node: BeaconNode, head: BlockRef): bool =
## TODO This function is here as a placeholder for some better heurestics to
## determine if we're in sync and should be producing blocks and
## attestations. Generally, the problem is that slot time keeps advancing
## even when there are no blocks being produced, so there's no way to
## distinguish validators geniunely going missing from the node not being
## well connected (during a network split or an internet outage for
## example). It would generally be correct to simply keep running as if
## we were the only legit node left alive, but then we run into issues:
## with enough many empty slots, the validator pool is emptied leading
## to empty committees and lots of empty slot processing that will be
## thrown away as soon as we're synced again.
let
# The slot we should be at, according to the clock
beaconTime = node.beaconClock.now()
wallSlot = beaconTime.toSlot()
# TODO if everyone follows this logic, the network will not recover from a
# halt: nobody will be producing blocks because everone expects someone
# else to do it
if wallSlot.afterGenesis and head.slot + node.config.syncHorizon < wallSlot.slot:
false
else:
true
func isGoodForSending(validationResult: ValidationRes): bool =
# Validator clients such as Vouch can be configured to work with multiple
# beacon nodes simultaneously. In this configuration, the validator client
# will try to broadcast the gossip messages through each of the connected
# beacon nodes which may lead to a situation where some of the nodes see a
# message arriving from the network before it arrives through the REST API.
# This should not be considered an error and the beacon node should still
# broadcast the message as the intented purpose of the Vouch strategy is
# to ensure that the message will reach as many peers as possible.
validationResult.isOk() or validationResult.error[0] == ValidationResult.Ignore
proc sendAttestation*(
node: BeaconNode, attestation: Attestation,
subnet_id: SubnetId, checkSignature: bool): Future[SendResult] {.async.} =
# Validate attestation before sending it via gossip - validation will also
# register the attestation with the attestation pool. Notably, although
# libp2p calls the data handler for any subscription on the subnet
# topic, it does not perform validation.
let res = await node.processor.attestationValidator(
MsgSource.api, attestation, subnet_id, checkSignature)
return
if res.isGoodForSending:
node.network.broadcastAttestation(subnet_id, attestation)
beacon_attestations_sent.inc()
ok()
else:
notice "Produced attestation failed validation",
attestation = shortLog(attestation),
error = res.error()
err(res.error()[1])
proc handleLightClientUpdates(node: BeaconNode, slot: Slot) {.async.} =
static: doAssert lightClientFinalityUpdateSlotOffset ==
lightClientOptimisticUpdateSlotOffset
let sendTime = node.beaconClock.fromNow(
slot.light_client_finality_update_time())
if sendTime.inFuture:
debug "Waiting to send LC updates", slot, delay = shortLog(sendTime.offset)
await sleepAsync(sendTime.offset)
template latest(): auto = node.dag.lightClientCache.latest
let signature_slot = latest.signature_slot
if slot != signature_slot:
return
template sync_aggregate(): auto = latest.sync_aggregate
template sync_committee_bits(): auto = sync_aggregate.sync_committee_bits
let num_active_participants = countOnes(sync_committee_bits).uint64
if num_active_participants < MIN_SYNC_COMMITTEE_PARTICIPANTS:
return
let finalized_slot = latest.finalized_header.slot
if finalized_slot > node.lightClientPool[].latestForwardedFinalitySlot:
template msg(): auto = latest
node.network.broadcastLightClientFinalityUpdate(msg)
node.lightClientPool[].latestForwardedFinalitySlot = finalized_slot
beacon_light_client_finality_updates_sent.inc()
notice "LC finality update sent", message = shortLog(msg)
let attested_slot = latest.attested_header.slot
if attested_slot > node.lightClientPool[].latestForwardedOptimisticSlot:
let msg = latest.toOptimistic
node.network.broadcastLightClientOptimisticUpdate(msg)
node.lightClientPool[].latestForwardedOptimisticSlot = attested_slot
beacon_light_client_optimistic_updates_sent.inc()
notice "LC optimistic update sent", message = shortLog(msg)
proc scheduleSendingLightClientUpdates(node: BeaconNode, slot: Slot) =
if not node.config.serveLightClientData.get:
return
if node.lightClientPool[].broadcastGossipFut != nil:
return
if slot <= node.lightClientPool[].latestBroadcastedSlot:
return
node.lightClientPool[].latestBroadcastedSlot = slot
template fut(): auto = node.lightClientPool[].broadcastGossipFut
fut = node.handleLightClientUpdates(slot)
fut.addCallback do (p: pointer) {.gcsafe.}:
fut = nil
proc sendSyncCommitteeMessage(
node: BeaconNode, msg: SyncCommitteeMessage,
subcommitteeIdx: SyncSubcommitteeIndex,
checkSignature: bool): Future[SendResult] {.async.} =
# Validate sync committee message before sending it via gossip
# validation will also register the message with the sync committee
# message pool. Notably, although libp2p calls the data handler for
# any subscription on the subnet topic, it does not perform validation.
let res = await node.processor.syncCommitteeMessageValidator(
MsgSource.api, msg, subcommitteeIdx, checkSignature)
return
if res.isGoodForSending:
node.network.broadcastSyncCommitteeMessage(msg, subcommitteeIdx)
beacon_sync_committee_messages_sent.inc()
node.scheduleSendingLightClientUpdates(msg.slot)
SendResult.ok()
else:
notice "Sync committee message failed validation",
msg, error = res.error()
SendResult.err(res.error()[1])
proc sendSyncCommitteeMessages*(node: BeaconNode,
msgs: seq[SyncCommitteeMessage]
): Future[seq[SendResult]] {.async.} =
return withState(node.dag.headState):
when stateFork >= BeaconStateFork.Altair:
var statuses = newSeq[Option[SendResult]](len(msgs))
let
curPeriod = sync_committee_period(state.data.slot)
nextPeriod = curPeriod + 1
let (keysCur, keysNxt) =
block:
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
var resCur: Table[uint64, int]
var resNxt: Table[uint64, int]
for index, msg in msgs:
if msg.validator_index < lenu64(state.data.validators):
let msgPeriod = sync_committee_period(msg.slot + 1)
if msgPeriod == curPeriod:
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
resCur[msg.validator_index] = index
elif msgPeriod == nextPeriod:
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
resNxt[msg.validator_index] = index
else:
statuses[index] =
some(SendResult.err("Message's slot out of state's head range"))
else:
statuses[index] = some(SendResult.err("Incorrect validator's index"))
if (len(resCur) == 0) and (len(resNxt) == 0):
return statuses.mapIt(it.get())
(resCur, resNxt)
let (pending, indices) = block:
var resFutures: seq[Future[SendResult]]
var resIndices: seq[int]
template headSyncCommittees(): auto = node.dag.headSyncCommittees
for subcommitteeIdx in SyncSubcommitteeIndex:
for valKey in syncSubcommittee(
headSyncCommittees.current_sync_committee, subcommitteeIdx):
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
let index = keysCur.getOrDefault(uint64(valKey), -1)
if index >= 0:
resIndices.add(index)
resFutures.add(node.sendSyncCommitteeMessage(
msgs[index], subcommitteeIdx, true))
for subcommitteeIdx in SyncSubcommitteeIndex:
for valKey in syncSubcommittee(
headSyncCommittees.next_sync_committee, subcommitteeIdx):
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
let index = keysNxt.getOrDefault(uint64(valKey), -1)
if index >= 0:
resIndices.add(index)
resFutures.add(node.sendSyncCommitteeMessage(
msgs[index], subcommitteeIdx, true))
(resFutures, resIndices)
await allFutures(pending)
for index, future in pending:
if future.done():
let fres = future.read()
if fres.isErr():
statuses[indices[index]] = some(SendResult.err(fres.error()))
else:
statuses[indices[index]] = some(SendResult.ok())
elif future.failed() or future.cancelled():
let exc = future.readError()
debug "Unexpected failure while sending committee message",
message = msgs[indices[index]], error = $exc.msg
statuses[indices[index]] = some(SendResult.err(
"Unexpected failure while sending committee message"))
var res: seq[SendResult]
for item in statuses:
if item.isSome():
res.add(item.get())
else:
res.add(SendResult.err("Message validator not in sync committee"))
res
else:
var res: seq[SendResult]
for _ in msgs:
res.add(SendResult.err("Waiting for altair fork"))
res
proc sendSyncCommitteeContribution*(
node: BeaconNode,
msg: SignedContributionAndProof,
checkSignature: bool): Future[SendResult] {.async.} =
let res = await node.processor.contributionValidator(
MsgSource.api, msg, checkSignature)
return
if res.isGoodForSending:
node.network.broadcastSignedContributionAndProof(msg)
beacon_sync_committee_contributions_sent.inc()
ok()
else:
notice "Sync committee contribution failed validation",
msg, error = res.error()
err(res.error()[1])
proc createAndSendAttestation(node: BeaconNode,
fork: Fork,
genesis_validators_root: Eth2Digest,
validator: AttachedValidator,
attestationData: AttestationData,
committeeLen: int,
indexInCommittee: int,
subnet_id: SubnetId) {.async.} =
try:
2021-11-30 01:20:21 +00:00
var attestation =
block:
let res = await validator.produceAndSignAttestation(
attestationData, committeeLen, indexInCommittee, fork,
genesis_validators_root)
if res.isErr():
error "Unable to sign attestation", validator = shortLog(validator),
error_msg = res.error()
return
res.get()
let res = await node.sendAttestation(
attestation, subnet_id, checkSignature = false)
if not res.isOk(): # Logged in sendAttestation
return
if node.config.dumpEnabled:
New validator client using REST API. (#2651) * Initial commit. * Exporting getConfig(). * Add beacon node checking procedures. * Post rebase fixes. * Use runSlotLoop() from nimbus_beacon_node. Fallback implementation. Fixes for ETH2 REST serialization. * Add beacon_clock.durationToNextSlot(). Move type declarations from beacon_rest_api to json_rest_serialization. Fix seq[ValidatorIndex] serialization. Refactor ValidatorPool and add some utility procedures. Create separate version of validator_client. * Post-rebase fixes. Remove CookedPubKey from validator_pool.nim. * Now we should be able to produce attestations and aggregate and proofs. But its not working yet. * Debugging attestation sending. * Add durationToNextAttestation. Optimize some debug logs. Fix aggregation_bits encoding. Bump chronos/presto. * Its alive. * Fixes for launch_local_testnet script. Bump chronos. * Switch client API to not use `/api` prefix. * Post-rebase adjustments. * Fix endpoint for publishBlock(). * Add CONFIG_NAME. Add more checks to ensure that beacon_node is compatible. * Add beacon committee subscription support to validator_client. * Fix stacktrace should be an array of strings. Fix committee subscriptions should not be `data` keyed. * Log duration to next block proposal. * Fix beacon_node_status import. * Use jsonMsgResponse() instead of jsonError(). * Fix graffityBytes usage. Remove unnecessary `await`. Adjust creation of SignedBlock instance. Remove legacy files. * Rework durationToNextSlot() and durationToNextEpoch() to use `fromNow`. * Fix race condition for block proposal and attestations for same slot. Fix local_testnet script to properly kill tasks on Windows. Bump chronos and nim-http-tools, to allow connections to infura.io (basic auth). * Catch services errors. Improve performance of local_testnet.sh script on Windows. Fix race condition when attestation producing. * Post-rebase fixes. * Bump chronos and presto. * Calculate block publishing delay. Fix pkill in one more place. * Add error handling and timeouts to firstSuccess() template. Add onceToAll() template. Add checkNodes() procedure. Refactor firstSuccess() template. Add error checking to api.nim calls. * Deprecated usage onceToAll() for better stability. Address comment and send attestations asap. * Avoid unnecessary loop when calculating minimal duration.
2021-07-13 11:15:07 +00:00
dump(node.config.dumpDirOutgoing, attestation.data,
validator.pubkey)
let
wallTime = node.beaconClock.now()
delay = wallTime - attestationData.slot.attestation_deadline()
notice "Attestation sent",
attestation = shortLog(attestation), validator = shortLog(validator),
delay, subnet_id
beacon_attestation_sent_delay.observe(delay.toFloatSeconds())
except CatchableError as exc:
# An error could happen here when the signature task fails - we must
# not leak the exception because this is an asyncSpawn task
notice "Error sending attestation", err = exc.msg
proc getBlockProposalEth1Data*(node: BeaconNode,
state: ForkedHashedBeaconState):
BlockProposalEth1Data =
if node.eth1Monitor.isNil:
var pendingDepositsCount =
getStateField(state, eth1_data).deposit_count -
getStateField(state, eth1_deposit_index)
if pendingDepositsCount > 0:
result.hasMissingDeposits = true
else:
result.vote = getStateField(state, eth1_data)
else:
let finalizedEpochRef = node.dag.getFinalizedEpochRef()
result = node.eth1Monitor.getBlockProposalData(
state, finalizedEpochRef.eth1_data,
finalizedEpochRef.eth1_deposit_index)
proc forkchoice_updated(state: bellatrix.BeaconState,
2021-12-17 12:23:32 +00:00
head_block_hash: Eth2Digest,
finalized_block_hash: Eth2Digest,
fee_recipient: ethtypes.Address,
execution_engine: Eth1Monitor):
Future[Option[bellatrix.PayloadID]] {.async.} =
2021-12-17 12:23:32 +00:00
let
timestamp = compute_timestamp_at_slot(state, state.slot)
random = get_randao_mix(state, get_current_epoch(state))
forkchoiceResponse =
awaitWithTimeout(
execution_engine.forkchoiceUpdated(
head_block_hash, finalized_block_hash, timestamp, random.data,
fee_recipient),
FORKCHOICEUPDATED_TIMEOUT):
info "forkchoice_updated: forkchoiceUpdated timed out"
default(ForkchoiceUpdatedResponse)
payloadId = forkchoiceResponse.payloadId
2021-12-17 12:23:32 +00:00
return if payloadId.isSome:
some(bellatrix.PayloadID(payloadId.get))
2021-12-17 12:23:32 +00:00
else:
none(bellatrix.PayloadID)
2021-12-17 12:23:32 +00:00
proc get_execution_payload(
payload_id: Option[bellatrix.PayloadId], execution_engine: Eth1Monitor):
Future[bellatrix.ExecutionPayload] {.async.} =
return if payload_id.isNone():
# Pre-merge, empty payload
default(bellatrix.ExecutionPayload)
else:
asConsensusExecutionPayload(
await execution_engine.getPayload(payload_id.get))
proc getExecutionPayload(node: BeaconNode, proposalState: auto):
Future[ExecutionPayload] {.async.} =
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/bellatrix/validator.md#executionpayload
# Only current hardfork with execution payloads is Bellatrix
static: doAssert high(BeaconStateFork) == BeaconStateFork.Bellatrix
template empty_execution_payload(): auto =
build_empty_execution_payload(proposalState.bellatrixData.data)
if node.eth1Monitor.isNil:
warn "getExecutionPayload: eth1Monitor not initialized; using empty execution payload"
return empty_execution_payload
try:
# Minimize window for Eth1 monitor to shut down connection
await node.consensusManager.eth1Monitor.ensureDataProvider()
# https://github.com/ethereum/execution-apis/blob/2c3dffa1ad301a5b1d46212e1bd65e918265cd6f/src/engine/specification.md#request-2
const GETPAYLOAD_TIMEOUT = 1.seconds
let
feeRecipient =
if node.config.suggestedFeeRecipient.isSome:
node.config.suggestedFeeRecipient.get
else:
default(Eth1Address)
latestHead =
if not node.dag.head.executionBlockRoot.isZero:
node.dag.head.executionBlockRoot
else:
default(Eth2Digest)
latestFinalized = node.dag.finalizedHead.blck.executionBlockRoot
payload_id = (await forkchoice_updated(
proposalState.bellatrixData.data, latestHead, latestFinalized,
feeRecipient, node.consensusManager.eth1Monitor))
payload = awaitWithTimeout(
get_execution_payload(payload_id, node.consensusManager.eth1Monitor),
GETPAYLOAD_TIMEOUT):
info "getExecutionPayload: getPayload timed out; using empty execution payload"
empty_execution_payload
executionPayloadStatus =
awaitWithTimeout(
node.consensusManager.eth1Monitor.newExecutionPayload(payload),
NEWPAYLOAD_TIMEOUT):
info "getExecutionPayload: newPayload timed out"
PayloadExecutionStatus.syncing
if executionPayloadStatus != PayloadExecutionStatus.valid:
info "getExecutionPayload: newExecutionPayload not valid; using empty execution payload",
executionPayloadStatus
return empty_execution_payload
return payload
except CatchableError as err:
error "Error creating non-empty execution payload; using empty execution payload",
msg = err.msg
return empty_execution_payload
proc makeBeaconBlockForHeadAndSlot*(node: BeaconNode,
randao_reveal: ValidatorSig,
validator_index: ValidatorIndex,
graffiti: GraffitiBytes,
head: BlockRef, slot: Slot
): Future[ForkedBlockResult] {.async.} =
# Advance state to the slot that we're proposing for
let
proposalState = assignClone(node.dag.headState)
Prune `BlockRef` on finalization (#3513) Up til now, the block dag has been using `BlockRef`, a structure adapted for a full DAG, to represent all of chain history. This is a correct and simple design, but does not exploit the linearity of the chain once parts of it finalize. By pruning the in-memory `BlockRef` structure at finalization, we save, at the time of writing, a cool ~250mb (or 25%:ish) chunk of memory landing us at a steady state of ~750mb normal memory usage for a validating node. Above all though, we prevent memory usage from growing proportionally with the length of the chain, something that would not be sustainable over time - instead, the steady state memory usage is roughly determined by the validator set size which grows much more slowly. With these changes, the core should remain sustainable memory-wise post-merge all the way to withdrawals (when the validator set is expected to grow). In-memory indices are still used for the "hot" unfinalized portion of the chain - this ensure that consensus performance remains unchanged. What changes is that for historical access, we use a db-based linear slot index which is cache-and-disk-friendly, keeping the cost for accessing historical data at a similar level as before, achieving the savings at no percievable cost to functionality or performance. A nice collateral benefit is the almost-instant startup since we no longer load any large indicies at dag init. The cost of this functionality instead can be found in the complexity of having to deal with two ways of traversing the chain - by `BlockRef` and by slot. * use `BlockId` instead of `BlockRef` where finalized / historical data may be required * simplify clearance pre-advancement * remove dag.finalizedBlocks (~50:ish mb) * remove `getBlockAtSlot` - use `getBlockIdAtSlot` instead * `parent` and `atSlot` for `BlockId` now require a `ChainDAGRef` instance, unlike `BlockRef` traversal * prune `BlockRef` parents on finality (~200:ish mb) * speed up ChainDAG init by not loading finalized history index * mess up light client server error handling - this need revisiting :)
2022-03-17 17:42:56 +00:00
# TODO fails at checkpoint synced head
node.dag.withUpdatedState(
proposalState[],
head.atSlot(slot - 1).toBlockSlotId().expect("not nil")):
# Advance to the given slot without calculating state root - we'll only
# need a state root _with_ the block applied
var info: ForkedEpochInfo
process_slots(
node.dag.cfg, state, slot, cache, info,
{skipLastStateRootCalculation}).expect("advancing 1 slot should not fail")
let
eth1Proposal = node.getBlockProposalEth1Data(state)
if eth1Proposal.hasMissingDeposits:
warn "Eth1 deposits not available. Skipping block proposal", slot
return ForkedBlockResult.err("Eth1 deposits not available")
# Only current hardfork with execution payloads is Bellatrix
static: doAssert high(BeaconStateFork) == BeaconStateFork.Bellatrix
let exits = withState(state):
node.exitPool[].getBeaconBlockExits(state.data)
let res = makeBeaconBlock(
node.dag.cfg,
state,
validator_index,
randao_reveal,
eth1Proposal.vote,
graffiti,
node.attestationPool[].getAttestationsForBlock(state, cache),
eth1Proposal.deposits,
exits,
if slot.epoch < node.dag.cfg.ALTAIR_FORK_EPOCH:
SyncAggregate.init()
else:
node.syncCommitteeMsgPool[].produceSyncAggregate(head.root),
if slot.epoch < node.dag.cfg.BELLATRIX_FORK_EPOCH or
# TODO when Eth1Monitor TTD following comes in, actually detect
# transition block directly
not is_merge_transition_complete(proposalState.bellatrixData.data):
default(bellatrix.ExecutionPayload)
else:
(await getExecutionPayload(node, proposalState)),
noRollback, # Temporary state - no need for rollback
cache)
if res.isErr():
# This is almost certainly a bug, but it's complex enough that there's a
# small risk it might happen even when most proposals succeed - thus we
# log instead of asserting
error "Cannot create block for proposal",
slot, head = shortLog(head), error = res.error()
return err($res.error)
return ok(res.get())
do:
error "Cannot get proposal state - skipping block production, database corrupt?",
head = shortLog(head),
slot
proc proposeBlock(node: BeaconNode,
validator: AttachedValidator,
validator_index: ValidatorIndex,
head: BlockRef,
slot: Slot): Future[BlockRef] {.async.} =
if head.slot >= slot:
# We should normally not have a head newer than the slot we're proposing for
# but this can happen if block proposal is delayed
warn "Skipping proposal, have newer head already",
headSlot = shortLog(head.slot),
headBlockRoot = shortLog(head.root),
slot = shortLog(slot)
return head
let
fork = node.dag.forkAtEpoch(slot.epoch)
genesis_validators_root =
getStateField(node.dag.headState, genesis_validators_root)
2021-11-30 01:20:21 +00:00
randao =
block:
let res = await validator.genRandaoReveal(fork, genesis_validators_root,
slot)
if res.isErr():
error "Unable to generate randao reveal",
validator = shortLog(validator), error_msg = res.error()
return head
res.get()
var newBlock = await makeBeaconBlockForHeadAndSlot(
node, randao, validator_index, node.graffitiBytes, head, slot)
if newBlock.isErr():
return head # already logged elsewhere!
let forkedBlck = newBlock.get()
withBlck(forkedBlck):
let
blockRoot = hash_tree_root(blck)
signing_root = compute_block_signing_root(
fork, genesis_validators_root, slot, blockRoot)
notSlashable = node.attachedValidators
.slashingProtection
.registerBlock(validator_index, validator.pubkey, slot, signing_root)
if notSlashable.isErr:
warn "Slashing protection activated",
validator = validator.pubkey,
slot = slot,
existingProposal = notSlashable.error
return head
let
signature =
block:
let res = await validator.signBlockProposal(
fork, genesis_validators_root, slot, blockRoot, forkedBlck)
if res.isErr():
error "Unable to sign block proposal",
validator = shortLog(validator), error_msg = res.error()
return head
res.get()
signedBlock =
when blck is phase0.BeaconBlock:
phase0.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is altair.BeaconBlock:
altair.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
elif blck is bellatrix.BeaconBlock:
bellatrix.SignedBeaconBlock(
message: blck, signature: signature, root: blockRoot)
else:
static: doAssert "Unknown SignedBeaconBlock type"
# We produced the block using a state transition, meaning the block is valid
# enough that it will not be rejected by gossip - it is unlikely but
# possible that it will be ignored due to extreme timing conditions, for
# example a delay in signing.
# We'll start broadcasting it before integrating fully in the chaindag
# so that it can start propagating through the network ASAP.
node.network.broadcastBeaconBlock(signedBlock)
let
wallTime = node.beaconClock.now()
# storeBlock puts the block in the chaindag, and if accepted, takes care
# of side effects such as event api notification
newBlockRef = node.blockProcessor[].storeBlock(
MsgSource.api, wallTime, signedBlock)
if newBlockRef.isErr:
warn "Unable to add proposed block to block pool",
blockRoot = shortLog(blockRoot), blck = shortLog(blck),
signature = shortLog(signature), validator = shortLog(validator)
return head
notice "Block proposed",
blockRoot = shortLog(blockRoot), blck = shortLog(blck),
signature = shortLog(signature), validator = shortLog(validator)
beacon_blocks_proposed.inc()
return newBlockRef.get()
proc handleAttestations(node: BeaconNode, head: BlockRef, slot: Slot) =
## Perform all attestations that the validators attached to this node should
## perform during the given slot
if slot + SLOTS_PER_EPOCH < head.slot:
# The latest block we know about is a lot newer than the slot we're being
# asked to attest to - this makes it unlikely that it will be included
# at all.
# TODO the oldest attestations allowed are those that are older than the
# finalized epoch.. also, it seems that posting very old attestations
# is risky from a slashing perspective. More work is needed here.
warn "Skipping attestation, head is too recent",
head = shortLog(head),
slot = shortLog(slot)
return
if slot < node.dag.finalizedHead.slot:
# During checkpoint sync, we implicitly finalize the given slot even if the
# state transition does not yet consider it final - this is a sanity check
# mostly to ensure the `atSlot` below works as expected
warn "Skipping attestation - slot already finalized",
head = shortLog(head),
slot = shortLog(slot),
finalized = shortLog(node.dag.finalizedHead)
return
let attestationHead = head.atSlot(slot)
if head != attestationHead.blck:
# In rare cases, such as when we're busy syncing or just slow, we'll be
# attesting to a past state - we must then recreate the world as it looked
# like back then
notice "Attesting to a state in the past, falling behind?",
attestationHead = shortLog(attestationHead),
head = shortLog(head)
trace "Checking attestations",
attestationHead = shortLog(attestationHead),
head = shortLog(head)
# We need to run attestations exactly for the slot that we're attesting to.
# In case blocks went missing, this means advancing past the latest block
# using empty slots as fillers.
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#validator-assignments
2020-08-10 13:21:31 +00:00
let
epochRef = block:
let tmp = node.dag.getEpochRef(attestationHead.blck, slot.epoch, false)
if isErr(tmp):
warn "Cannot construct EpochRef for attestation head, report bug",
attestationHead = shortLog(attestationHead), slot
return
tmp.get()
committees_per_slot = get_committee_count_per_slot(epochRef)
fork = node.dag.forkAtEpoch(slot.epoch)
2020-08-10 13:21:31 +00:00
genesis_validators_root =
getStateField(node.dag.headState, genesis_validators_root)
2020-08-10 13:21:31 +00:00
for committee_index in get_committee_indices(committees_per_slot):
let committee = get_beacon_committee(epochRef, slot, committee_index)
2020-08-10 13:21:31 +00:00
for index_in_committee, validator_index in committee:
let validator = node.getAttachedValidator(epochRef, validator_index)
if validator == nil:
continue
let
data = makeAttestationData(epochRef, attestationHead, committee_index)
# TODO signing_root is recomputed in produceAndSignAttestation/signAttestation just after
signing_root = compute_attestation_signing_root(
fork, genesis_validators_root, data)
registered = node.attachedValidators
.slashingProtection
.registerAttestation(
validator_index,
New validator client using REST API. (#2651) * Initial commit. * Exporting getConfig(). * Add beacon node checking procedures. * Post rebase fixes. * Use runSlotLoop() from nimbus_beacon_node. Fallback implementation. Fixes for ETH2 REST serialization. * Add beacon_clock.durationToNextSlot(). Move type declarations from beacon_rest_api to json_rest_serialization. Fix seq[ValidatorIndex] serialization. Refactor ValidatorPool and add some utility procedures. Create separate version of validator_client. * Post-rebase fixes. Remove CookedPubKey from validator_pool.nim. * Now we should be able to produce attestations and aggregate and proofs. But its not working yet. * Debugging attestation sending. * Add durationToNextAttestation. Optimize some debug logs. Fix aggregation_bits encoding. Bump chronos/presto. * Its alive. * Fixes for launch_local_testnet script. Bump chronos. * Switch client API to not use `/api` prefix. * Post-rebase adjustments. * Fix endpoint for publishBlock(). * Add CONFIG_NAME. Add more checks to ensure that beacon_node is compatible. * Add beacon committee subscription support to validator_client. * Fix stacktrace should be an array of strings. Fix committee subscriptions should not be `data` keyed. * Log duration to next block proposal. * Fix beacon_node_status import. * Use jsonMsgResponse() instead of jsonError(). * Fix graffityBytes usage. Remove unnecessary `await`. Adjust creation of SignedBlock instance. Remove legacy files. * Rework durationToNextSlot() and durationToNextEpoch() to use `fromNow`. * Fix race condition for block proposal and attestations for same slot. Fix local_testnet script to properly kill tasks on Windows. Bump chronos and nim-http-tools, to allow connections to infura.io (basic auth). * Catch services errors. Improve performance of local_testnet.sh script on Windows. Fix race condition when attestation producing. * Post-rebase fixes. * Bump chronos and presto. * Calculate block publishing delay. Fix pkill in one more place. * Add error handling and timeouts to firstSuccess() template. Add onceToAll() template. Add checkNodes() procedure. Refactor firstSuccess() template. Add error checking to api.nim calls. * Deprecated usage onceToAll() for better stability. Address comment and send attestations asap. * Avoid unnecessary loop when calculating minimal duration.
2021-07-13 11:15:07 +00:00
validator.pubkey,
data.source.epoch,
data.target.epoch,
signing_root)
if registered.isOk():
let subnet_id = compute_subnet_for_attestation(
committees_per_slot, data.slot, committee_index)
asyncSpawn createAndSendAttestation(
node, fork, genesis_validators_root, validator, data,
committee.len(), index_in_committee, subnet_id)
else:
warn "Slashing protection activated for attestation",
validator = validator.pubkey,
badVoteDetails = $registered.error()
proc createAndSendSyncCommitteeMessage(node: BeaconNode,
slot: Slot,
validator: AttachedValidator,
subcommitteeIdx: SyncSubcommitteeIndex,
head: BlockRef) {.async.} =
try:
let
fork = node.dag.forkAtEpoch(slot.epoch)
genesis_validators_root = node.dag.genesis_validators_root
2021-11-30 01:20:21 +00:00
msg =
block:
let res = await signSyncCommitteeMessage(validator, fork,
genesis_validators_root,
slot, head.root)
2021-11-30 01:20:21 +00:00
if res.isErr():
error "Unable to sign committee message using remote signer",
validator = shortLog(validator), slot = slot,
block_root = shortLog(head.root)
return
res.get()
let res = await node.sendSyncCommitteeMessage(
msg, subcommitteeIdx, checkSignature = false)
if res.isErr():
# Logged in sendSyncCommitteeMessage
return
if node.config.dumpEnabled:
dump(node.config.dumpDirOutgoing, msg, validator.pubkey)
let
wallTime = node.beaconClock.now()
delay = wallTime - msg.slot.sync_committee_message_deadline()
notice "Sync committee message sent",
message = shortLog(msg),
validator = shortLog(validator),
delay
beacon_sync_committee_message_sent_delay.observe(delay.toFloatSeconds())
except CatchableError as exc:
# An error could happen here when the signature task fails - we must
# not leak the exception because this is an asyncSpawn task
notice "Error sending sync committee message", err = exc.msg
proc handleSyncCommitteeMessages(node: BeaconNode, head: BlockRef, slot: Slot) =
# TODO Use a view type to avoid the copy
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
var syncCommittee = node.dag.syncCommitteeParticipants(slot + 1)
for subcommitteeIdx in SyncSubcommitteeIndex:
for valIdx in syncSubcommittee(syncCommittee, subcommitteeIdx):
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
let validator = node.getAttachedValidator(
getStateField(node.dag.headState, validators), valIdx)
if isNil(validator) or validator.index.isNone():
continue
asyncSpawn createAndSendSyncCommitteeMessage(node, slot, validator,
subcommitteeIdx, head)
proc signAndSendContribution(node: BeaconNode,
validator: AttachedValidator,
contribution: SyncCommitteeContribution,
selectionProof: ValidatorSig) {.async.} =
try:
let msg = (ref SignedContributionAndProof)(
message: ContributionAndProof(
aggregator_index: uint64 validator.index.get,
contribution: contribution,
selection_proof: selectionProof))
2021-11-30 01:20:21 +00:00
let res = await validator.sign(
msg, node.dag.forkAtEpoch(contribution.slot.epoch),
node.dag.genesis_validators_root)
2021-11-30 01:20:21 +00:00
if res.isErr():
error "Unable to sign sync committee contribution usign remote signer",
validator = shortLog(validator), error_msg = res.error()
return
# Failures logged in sendSyncCommitteeContribution
discard await node.sendSyncCommitteeContribution(msg[], false)
notice "Contribution sent", contribution = shortLog(msg[])
except CatchableError as exc:
# An error could happen here when the signature task fails - we must
# not leak the exception because this is an asyncSpawn task
notice "Error sending sync committee contribution", err = exc.msg
proc handleSyncCommitteeContributions(node: BeaconNode,
head: BlockRef, slot: Slot) {.async.} =
# TODO Use a view type to avoid the copy
let
fork = node.dag.forkAtEpoch(slot.epoch)
genesis_validators_root = node.dag.genesis_validators_root
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
syncCommittee = node.dag.syncCommitteeParticipants(slot + 1)
type
AggregatorCandidate = object
validator: AttachedValidator
subcommitteeIdx: SyncSubcommitteeIndex
var candidateAggregators: seq[AggregatorCandidate]
2021-11-30 01:20:21 +00:00
var selectionProofs: seq[Future[SignatureResult]]
var time = timeIt:
for subcommitteeIdx in SyncSubcommitteeIndex:
# TODO Hoist outside of the loop with a view type
# to avoid the repeated offset calculations
Speed up altair block processing 2x (#3115) * Speed up altair block processing >2x Like #3089, this PR drastially speeds up historical REST queries and other long state replays. * cache sync committee validator indices * use ~80mb less memory for validator pubkey mappings * batch-verify sync aggregate signature (fixes #2985) * document sync committee hack with head block vs sync message block * add batch signature verification failure tests Before: ``` ../env.sh nim c -d:release -r ncli_db --db:mainnet_0/db bench --start-slot:-1000 All time are ms Average, StdDev, Min, Max, Samples, Test Validation is turned off meaning that no BLS operations are performed 5830.675, 0.000, 5830.675, 5830.675, 1, Initialize DB 0.481, 1.878, 0.215, 59.167, 981, Load block from database 8422.566, 0.000, 8422.566, 8422.566, 1, Load state from database 6.996, 1.678, 0.042, 14.385, 969, Advance slot, non-epoch 93.217, 8.318, 84.192, 122.209, 32, Advance slot, epoch 20.513, 23.665, 11.510, 201.561, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` After: ``` 7081.422, 0.000, 7081.422, 7081.422, 1, Initialize DB 0.553, 2.122, 0.175, 66.692, 981, Load block from database 5439.446, 0.000, 5439.446, 5439.446, 1, Load state from database 6.829, 1.575, 0.043, 12.156, 969, Advance slot, non-epoch 94.716, 2.749, 88.395, 100.026, 32, Advance slot, epoch 11.636, 23.766, 4.889, 205.250, 981, Apply block, no slot processing 0.000, 0.000, 0.000, 0.000, 0, Database load 0.000, 0.000, 0.000, 0.000, 0, Database store ``` * add comment
2021-11-24 12:43:50 +00:00
for valIdx in syncSubcommittee(syncCommittee, subcommitteeIdx):
let validator = node.getAttachedValidator(
getStateField(node.dag.headState, validators), valIdx)
if validator == nil:
continue
candidateAggregators.add AggregatorCandidate(
validator: validator,
subcommitteeIdx: subcommitteeIdx)
selectionProofs.add validator.getSyncCommitteeSelectionProof(
fork, genesis_validators_root, slot, subcommitteeIdx)
await allFutures(selectionProofs)
debug "Prepared contributions selection proofs",
count = selectionProofs.len, time
var contributionsSent = 0
time = timeIt:
for i, proof in selectionProofs:
2021-11-30 01:20:21 +00:00
if not proof.completed:
continue
2021-11-30 01:20:21 +00:00
let selectionProofRes = proof.read()
if selectionProofRes.isErr():
error "Unable to sign selection proof using remote signer",
validator = shortLog(candidateAggregators[i].validator),
slot, head, subnet_id = candidateAggregators[i].subcommitteeIdx
continue
let selectionProof = selectionProofRes.get()
if not is_sync_committee_aggregator(selectionProof):
continue
var contribution: SyncCommitteeContribution
let contributionWasProduced =
node.syncCommitteeMsgPool[].produceContribution(
slot,
head.root,
candidateAggregators[i].subcommitteeIdx,
contribution)
if contributionWasProduced:
asyncSpawn signAndSendContribution(
node,
candidateAggregators[i].validator,
contribution,
selectionProof)
inc contributionsSent
else:
debug "Failure to produce contribution",
slot, head, subnet_id = candidateAggregators[i].subcommitteeIdx
proc handleProposal(node: BeaconNode, head: BlockRef, slot: Slot):
Future[BlockRef] {.async.} =
## Perform the proposal for the given slot, iff we have a validator attached
## that is supposed to do so, given the shuffling at that slot for the given
## head - to compute the proposer, we need to advance a state to the given
## slot
let proposer = node.dag.getProposer(head, slot)
if proposer.isNone():
return head
let
proposerKey = node.dag.validatorKey(proposer.get).get().toPubKey
validator = node.attachedValidators[].getValidator(proposerKey)
return
if validator == nil:
debug "Expecting block proposal",
headRoot = shortLog(head.root),
slot = shortLog(slot),
proposer_index = proposer.get(),
proposer = shortLog(proposerKey)
head
else:
await proposeBlock(node, validator, proposer.get(), head, slot)
proc makeAggregateAndProof*(
pool: var AttestationPool, epochRef: EpochRef, slot: Slot,
committee_index: CommitteeIndex,
validator_index: ValidatorIndex,
slot_signature: ValidatorSig): Opt[AggregateAndProof] =
doAssert validator_index in get_beacon_committee(epochRef, slot, committee_index)
# TODO for testing purposes, refactor this into the condition check
# and just calculation
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#aggregation-selection
if not is_aggregator(epochRef, slot, committee_index, slot_signature):
return err()
let maybe_slot_attestation = getAggregatedAttestation(pool, slot, committee_index)
if maybe_slot_attestation.isNone:
return err()
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#construct-aggregate
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#aggregateandproof
ok(AggregateAndProof(
aggregator_index: validator_index.uint64,
aggregate: maybe_slot_attestation.get,
selection_proof: slot_signature))
proc sendAggregatedAttestations(
node: BeaconNode, head: BlockRef, slot: Slot) {.async.} =
# Aggregated attestations must be sent by members of the beacon committees for
# the given slot, for which `is_aggregator` returns `true.
let
epochRef = block:
let tmp = node.dag.getEpochRef(head, slot.epoch, false)
if isErr(tmp): # Some unusual race condition perhaps?
warn "Cannot construct EpochRef for head, report bug",
head = shortLog(head), slot
return
tmp.get()
fork = node.dag.forkAtEpoch(slot.epoch)
genesis_validators_root =
getStateField(node.dag.headState, genesis_validators_root)
committees_per_slot = get_committee_count_per_slot(epochRef)
var
2021-11-30 01:20:21 +00:00
slotSigs: seq[Future[SignatureResult]] = @[]
slotSigsData: seq[tuple[committee_index: CommitteeIndex,
validator_index: ValidatorIndex,
v: AttachedValidator]] = @[]
for committee_index in get_committee_indices(committees_per_slot):
let committee = get_beacon_committee(epochRef, slot, committee_index)
for index_in_committee, validator_index in committee:
let validator = node.getAttachedValidator(epochRef, validator_index)
if validator != nil:
# the validator index and private key pair.
slotSigs.add getSlotSig(validator, fork,
genesis_validators_root, slot)
slotSigsData.add (committee_index, validator_index, validator)
await allFutures(slotSigs)
doAssert slotSigsData.len == slotSigs.len
for i in 0..<slotSigs.len:
let
data = slotSigsData[i]
slotSig = slotSigs[i].read().valueOr:
error "Unable to create slot signature using remote signer",
validator = shortLog(data.v),
slot, error = error
continue
aggregateAndProof = makeAggregateAndProof(
node.attestationPool[], epochRef, slot, data.committee_index,
data.validator_index, slotSig).valueOr:
# Don't broadcast when, e.g., this validator isn't aggregator
continue
sig = block:
let res = await signAggregateAndProof(data.v,
aggregateAndProof, fork, genesis_validators_root)
if res.isErr():
error "Unable to sign aggregated attestation using remote signer",
validator = shortLog(data.v), error_msg = res.error()
return
res.get()
signedAP = SignedAggregateAndProof(
message: aggregateAndProof,
signature: sig)
node.network.broadcastAggregateAndProof(signedAP)
# The subnet on which the attestations (should have) arrived
let
subnet_id = compute_subnet_for_attestation(
committees_per_slot, slot, data.committee_index)
notice "Aggregated attestation sent",
aggregate = shortLog(signedAP.message.aggregate),
aggregator_index = signedAP.message.aggregator_index,
signature = shortLog(signedAP.signature),
validator = shortLog(data.v),
subnet_id
node.validatorMonitor[].registerAggregate(
MsgSource.api, node.beaconClock.now(), signedAP.message,
get_attesting_indices(
epochRef, slot,
data.committee_index,
aggregateAndProof.aggregate.aggregation_bits))
2020-12-16 13:03:04 +00:00
proc updateValidatorMetrics*(node: BeaconNode) =
# Technically, this only needs to be done on epoch transitions and if there's
# a reorg that spans an epoch transition, but it's easier to implement this
# way for now.
# We'll limit labelled metrics to the first 64, so that we don't overload
# Prometheus.
var total: Gwei
var i = 0
for _, v in node.attachedValidators[].validators:
let balance =
if v.index.isNone():
0.Gwei
elif v.index.get().uint64 >=
getStateField(node.dag.headState, balances).lenu64:
debug "Cannot get validator balance, index out of bounds",
pubkey = shortLog(v.pubkey), index = v.index.get(),
balances = getStateField(node.dag.headState, balances).len,
stateRoot = getStateRoot(node.dag.headState)
0.Gwei
else:
getStateField(node.dag.headState, balances).asSeq()[v.index.get()]
if i < 64:
attached_validator_balance.set(
balance.toGaugeValue, labelValues = [shortLog(v.pubkey)])
inc i
total += balance
node.attachedValidatorBalanceTotal = total
attached_validator_balance_total.set(total.toGaugeValue)
2020-11-27 23:34:25 +00:00
proc handleValidatorDuties*(node: BeaconNode, lastSlot, slot: Slot) {.async.} =
## Perform validator duties - create blocks, vote and aggregate existing votes
if node.attachedValidators[].count == 0:
# Nothing to do because we have no validator attached
return
# The dag head might be updated by sync while we're working due to the
# await calls, thus we use a local variable to keep the logic straight here
var head = node.dag.head
if not node.isSynced(head):
info "Syncing in progress; skipping validator duties for now",
slot, headSlot = head.slot
2020-12-16 13:03:04 +00:00
# Rewards will be growing though, as we sync..
updateValidatorMetrics(node)
return
var curSlot = lastSlot + 1
# If broadcastStartEpoch is 0, it hasn't had time to initialize yet, which
# means that it'd be okay not to continue, but it won't gossip regardless.
let doppelgangerDetection = node.processor[].doppelgangerDetection
if curSlot.epoch < doppelgangerDetection.broadcastStartEpoch and
doppelgangerDetection.nodeLaunchSlot > GENESIS_SLOT and
node.config.doppelgangerDetection:
let
nextAttestationSlot = node.actionTracker.getNextAttestationSlot(slot - 1)
nextProposalSlot = node.actionTracker.getNextProposalSlot(slot - 1)
if slot in [nextAttestationSlot, nextProposalSlot]:
notice "Doppelganger detection active - skipping validator duties while observing activity on the network",
slot, epoch = slot.epoch,
broadcastStartEpoch = doppelgangerDetection.broadcastStartEpoch
else:
debug "Doppelganger detection active - skipping validator duties while observing activity on the network",
slot, epoch = slot.epoch,
broadcastStartEpoch = doppelgangerDetection.broadcastStartEpoch
return
# Start by checking if there's work we should have done in the past that we
# can still meaningfully do
while curSlot < slot:
notice "Catching up on validator duties",
curSlot = shortLog(curSlot),
lastSlot = shortLog(lastSlot),
slot = shortLog(slot)
# For every slot we're catching up, we'll propose then send
# attestations - head should normally be advancing along the same branch
# in this case
head = await handleProposal(node, head, curSlot)
# For each slot we missed, we need to send out attestations - if we were
# proposing during this time, we'll use the newly proposed head, else just
# keep reusing the same - the attestation that goes out will actually
# rewind the state to what it looked like at the time of that slot
handleAttestations(node, head, curSlot)
curSlot += 1
let
newHead = await handleProposal(node, head, slot)
didSubmitBlock = (newHead != head)
head = newHead
let
# The latest point in time when we'll be sending out attestations
attestationCutoff = node.beaconClock.fromNow(slot.attestation_deadline())
if attestationCutoff.inFuture:
debug "Waiting to send attestations",
head = shortLog(head),
attestationCutoff = shortLog(attestationCutoff.offset)
# Wait either for the block or the attestation cutoff time to arrive
if await node.consensusManager[].expectBlock(slot)
.withTimeout(attestationCutoff.offset):
# The expected block arrived (or expectBlock was called again which
# shouldn't happen as this is the only place we use it) - in our async
# loop however, we might have been doing other processing that caused delays
# here so we'll cap the waiting to the time when we would have sent out
# attestations had the block not arrived.
# An opposite case is that we received (or produced) a block that has
# not yet reached our neighbours. To protect against our attestations
# being dropped (because the others have not yet seen the block), we'll
# impose a minimum delay of 2000ms. The delay is enforced only when we're
# not hitting the "normal" cutoff time for sending out attestations.
# An earlier delay of 250ms has proven to be not enough, increasing the
# risk of losing attestations, and with growing block sizes, 1000ms
# started to be risky as well.
# Regardless, because we "just" received the block, we'll impose the
# delay.
# Take into consideration chains with a different slot time
const afterBlockDelay = nanos(attestationSlotOffset.nanoseconds div 2)
let
afterBlockTime = node.beaconClock.now() + afterBlockDelay
afterBlockCutoff = node.beaconClock.fromNow(
min(afterBlockTime, slot.attestation_deadline() + afterBlockDelay))
if afterBlockCutoff.inFuture:
debug "Got block, waiting to send attestations",
head = shortLog(head),
afterBlockCutoff = shortLog(afterBlockCutoff.offset)
await sleepAsync(afterBlockCutoff.offset)
# Time passed - we might need to select a new head in that case
node.consensusManager[].updateHead(slot)
head = node.dag.head
static: doAssert attestationSlotOffset == syncCommitteeMessageSlotOffset
handleAttestations(node, head, slot)
handleSyncCommitteeMessages(node, head, slot)
2020-12-16 13:03:04 +00:00
updateValidatorMetrics(node) # the important stuff is done, update the vanity numbers
2020-11-27 23:34:25 +00:00
# https://github.com/ethereum/consensus-specs/blob/v1.1.10/specs/phase0/validator.md#broadcast-aggregate
# If the validator is selected to aggregate (`is_aggregator`), then they
# broadcast their best aggregate as a `SignedAggregateAndProof` to the global
# aggregate channel (`beacon_aggregate_and_proof`) `2 / INTERVALS_PER_SLOT`
# of the way through the `slot`-that is,
# `SECONDS_PER_SLOT * 2 / INTERVALS_PER_SLOT` seconds after the start of `slot`.
if slot > 2:
doAssert slot.aggregate_deadline() == slot.sync_contribution_deadline()
let
aggregateCutoff = node.beaconClock.fromNow(slot.aggregate_deadline())
if aggregateCutoff.inFuture:
debug "Waiting to send aggregate attestations",
aggregateCutoff = shortLog(aggregateCutoff.offset)
await sleepAsync(aggregateCutoff.offset)
let sendAggregatedAttestationsFut =
sendAggregatedAttestations(node, head, slot)
let handleSyncCommitteeContributionsFut =
handleSyncCommitteeContributions(node, head, slot)
await handleSyncCommitteeContributionsFut
await sendAggregatedAttestationsFut
proc sendAttestation*(node: BeaconNode,
attestation: Attestation): Future[SendResult] {.async.} =
# REST/JSON-RPC API helper procedure.
let
limit by-root requests to non-finalized blocks (#3293) * limit by-root requests to non-finalized blocks Presently, we keep a mapping from block root to `BlockRef` in memory - this has simplified reasoning about the dag, but is not sustainable with the chain growing. We can distinguish between two cases where by-root access is useful: * unfinalized blocks - this is where the beacon chain is operating generally, by validating incoming data as interesting for future fork choice decisions - bounded by the length of the unfinalized period * finalized blocks - historical access in the REST API etc - no bounds, really In this PR, we limit the by-root block index to the first use case: finalized chain data can more efficiently be addressed by slot number. Future work includes: * limiting the `BlockRef` horizon in general - each instance is 40 bytes+overhead which adds up - this needs further refactoring to deal with the tail vs state problem * persisting the finalized slot-to-hash index - this one also keeps growing unbounded (albeit slowly) Anyway, this PR easily shaves ~128mb of memory usage at the time of writing. * No longer honor `BeaconBlocksByRoot` requests outside of the non-finalized period - previously, Nimbus would generously return any block through this libp2p request - per the spec, finalized blocks should be fetched via `BeaconBlocksByRange` instead. * return `Opt[BlockRef]` instead of `nil` when blocks can't be found - this becomes a lot more common now and thus deserves more attention * `dag.blocks` -> `dag.forkBlocks` - this index only carries unfinalized blocks from now - `finalizedBlocks` covers the other `BlockRef` instances * in backfill, verify that the last backfilled block leads back to genesis, or panic * add backfill timings to log * fix missing check that `BlockRef` block can be fetched with `getForkedBlock` reliably * shortcut doppelganger check when feature is not enabled * in REST/JSON-RPC, fetch blocks without involving `BlockRef` * fix dag.blocks ref
2022-01-21 11:33:16 +00:00
target = node.dag.getBlockRef(attestation.data.target.root).valueOr:
notice "Attempt to send attestation for unknown target",
attestation = shortLog(attestation)
return SendResult.err(
"Attempt to send attestation for unknown block")
epochRef = node.dag.getEpochRef(
target, attestation.data.target.epoch, false).valueOr:
warn "Cannot construct EpochRef for attestation, skipping send - report bug",
target = shortLog(target),
attestation = shortLog(attestation)
return
committee_index =
epochRef.get_committee_index(attestation.data.index).valueOr:
notice "Invalid committee index in attestation",
attestation = shortLog(attestation)
return SendResult.err("Invalid committee index in attestation")
subnet_id = compute_subnet_for_attestation(
get_committee_count_per_slot(epochRef), attestation.data.slot,
committee_index)
res = await node.sendAttestation(attestation, subnet_id,
checkSignature = true)
if not res.isOk():
return res
let
wallTime = node.processor.getCurrentBeaconTime()
delay = wallTime - attestation.data.slot.attestation_deadline()
notice "Attestation sent",
attestation = shortLog(attestation), delay, subnet_id
beacon_attestation_sent_delay.observe(delay.toFloatSeconds())
return SendResult.ok()
proc sendAggregateAndProof*(node: BeaconNode,
proof: SignedAggregateAndProof): Future[SendResult] {.
async.} =
# REST/JSON-RPC API helper procedure.
let res =
await node.processor.aggregateValidator(MsgSource.api, proof)
return
if res.isGoodForSending:
node.network.broadcastAggregateAndProof(proof)
notice "Aggregated attestation sent",
attestation = shortLog(proof.message.aggregate),
aggregator_index = proof.message.aggregator_index,
signature = shortLog(proof.signature)
ok()
else:
notice "Aggregate and proof failed validation",
proof = shortLog(proof.message.aggregate), error = res.error()
err(res.error()[1])
proc sendVoluntaryExit*(node: BeaconNode,
exit: SignedVoluntaryExit): SendResult =
# REST/JSON-RPC API helper procedure.
let res =
node.processor[].voluntaryExitValidator(MsgSource.api, exit)
if res.isGoodForSending:
node.network.broadcastVoluntaryExit(exit)
ok()
else:
notice "Voluntary exit request failed validation",
exit = shortLog(exit.message), error = res.error()
err(res.error()[1])
proc sendAttesterSlashing*(node: BeaconNode,
slashing: AttesterSlashing): SendResult =
# REST/JSON-RPC API helper procedure.
let res =
node.processor[].attesterSlashingValidator(MsgSource.api, slashing)
if res.isGoodForSending:
node.network.broadcastAttesterSlashing(slashing)
ok()
else:
notice "Attester slashing request failed validation",
slashing = shortLog(slashing), error = res.error()
err(res.error()[1])
proc sendProposerSlashing*(node: BeaconNode,
slashing: ProposerSlashing): SendResult =
# REST/JSON-RPC API helper procedure.
let res =
node.processor[].proposerSlashingValidator(MsgSource.api, slashing)
if res.isGoodForSending:
node.network.broadcastProposerSlashing(slashing)
ok()
else:
notice "Proposer slashing request failed validation",
slashing = shortLog(slashing), error = res.error()
err(res.error()[1])
proc sendBeaconBlock*(node: BeaconNode, forked: ForkedSignedBeaconBlock
): Future[SendBlockResult] {.async.} =
# REST/JSON-RPC API helper procedure.
block:
# Start with a quick gossip validation check such that broadcasting the
# block doesn't get the node into trouble
let res = withBlck(forked):
validateBeaconBlock(node.dag, node.quarantine, blck,
node.beaconClock.now(), {})
if not res.isGoodForSending():
return SendBlockResult.err(res.error()[1])
# The block passed basic gossip validation - we can "safely" broadcast it now.
# In fact, per the spec, we should broadcast it even if it later fails to
# apply to our state.
node.network.broadcastBeaconBlock(forked)
let
wallTime = node.beaconClock.now()
accepted = withBlck(forked):
let newBlockRef = node.blockProcessor[].storeBlock(
MsgSource.api, wallTime, blck)
# The boolean we return tells the caller whether the block was integrated
# into the chain
if newBlockRef.isOk():
notice "Block published",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
signature = shortLog(blck.signature)
true
else:
warn "Unable to add proposed block to block pool",
blockRoot = shortLog(blck.root), blck = shortLog(blck.message),
signature = shortLog(blck.signature), err = newBlockRef.error()
false
return SendBlockResult.ok(accepted)
proc registerDuty*(
node: BeaconNode, slot: Slot, subnet_id: SubnetId, vidx: ValidatorIndex,
isAggregator: bool) =
# Only register relevant duties
node.actionTracker.registerDuty(slot, subnet_id, vidx, isAggregator)
proc registerDuties*(node: BeaconNode, wallSlot: Slot) {.async.} =
## Register upcoming duties of attached validators with the duty tracker
if node.attachedValidators[].count() == 0 or
not node.isSynced(node.dag.head):
# Nothing to do because we have no validator attached
return
let
genesis_validators_root =
getStateField(node.dag.headState, genesis_validators_root)
head = node.dag.head
# Getting the slot signature is expensive but cached - in "normal" cases we'll
# be getting the duties one slot at a time
for slot in wallSlot ..< wallSlot + SUBNET_SUBSCRIPTION_LEAD_TIME_SLOTS:
let
epochRef = block:
let tmp = node.dag.getEpochRef(head, slot.epoch, false)
if tmp.isErr(): # Shouldn't happen
warn "Cannot construct EpochRef for duties - report bug",
head = shortLog(head), slot
return
tmp.get()
let
fork = node.dag.forkAtEpoch(slot.epoch)
committees_per_slot = get_committee_count_per_slot(epochRef)
for committee_index in get_committee_indices(committees_per_slot):
let committee = get_beacon_committee(epochRef, slot, committee_index)
for index_in_committee, validator_index in committee:
let validator = node.getAttachedValidator(epochRef, validator_index)
if validator != nil:
let
subnet_id = compute_subnet_for_attestation(
committees_per_slot, slot, committee_index)
2021-11-30 01:20:21 +00:00
let slotSigRes = await getSlotSig(validator, fork,
genesis_validators_root, slot)
if slotSigRes.isErr():
error "Unable to create slot signature using remote signer",
validator = shortLog(validator),
error_msg = slotSigRes.error()
continue
let isAggregator = is_aggregator(committee.lenu64, slotSigRes.get())
node.registerDuty(slot, subnet_id, validator_index, isAggregator)