use IntSet rather than HashSet[ValidatorIndex] (#2267)

* use IntSet rather than HashSet[ValidatorIndex]

* add bounds check before uint64 -> int conversion

* use intsets in block transitions

* remove superfluous Nim issue explanation/reference
This commit is contained in:
tersec 2021-01-26 12:52:00 +01:00 committed by GitHub
parent 64e4d74f87
commit 1bdbf099cc
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
17 changed files with 108 additions and 83 deletions

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2019-2020 Status Research & Development GmbH # Copyright (c) 2019-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT). # * MIT license (license terms in the root directory or at http://opensource.org/licenses/MIT).
# * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0). # * Apache v2 license (license terms in the root directory or at http://www.apache.org/licenses/LICENSE-2.0).
@ -8,7 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[options, sequtils, sets], std/[intsets, options, sequtils],
chronos, chronicles, chronos, chronicles,
./spec/[ ./spec/[
beaconstate, datatypes, crypto, digest, helpers, network, signatures], beaconstate, datatypes, crypto, digest, helpers, network, signatures],
@ -163,7 +163,7 @@ proc validateAttestation*(
pool: var AttestationPool, pool: var AttestationPool,
attestation: Attestation, wallTime: BeaconTime, attestation: Attestation, wallTime: BeaconTime,
topicCommitteeIndex: uint64, checksExpensive: bool): topicCommitteeIndex: uint64, checksExpensive: bool):
Result[HashSet[ValidatorIndex], (ValidationResult, cstring)] = Result[IntSet, (ValidationResult, cstring)] =
# [REJECT] The attestation's epoch matches its target -- i.e. # [REJECT] The attestation's epoch matches its target -- i.e.
# attestation.data.target.epoch == # attestation.data.target.epoch ==
# compute_epoch_at_slot(attestation.data.slot) # compute_epoch_at_slot(attestation.data.slot)
@ -252,7 +252,7 @@ proc validateAttestation*(
# validator index. # validator index.
# Slightly modified to allow only newer attestations than were previously # Slightly modified to allow only newer attestations than were previously
# seen (no point in propagating older votes) # seen (no point in propagating older votes)
if (pool.nextAttestationEpoch.lenu64.ValidatorIndex > validator_index) and if (pool.nextAttestationEpoch.len > validator_index) and
pool.nextAttestationEpoch[validator_index].subnet > pool.nextAttestationEpoch[validator_index].subnet >
attestation.data.target.epoch: attestation.data.target.epoch:
return err((ValidationResult.Ignore, cstring( return err((ValidationResult.Ignore, cstring(
@ -287,8 +287,8 @@ proc validateAttestation*(
"validateAttestation: attestation's target block not an ancestor of LMD vote block"))) "validateAttestation: attestation's target block not an ancestor of LMD vote block")))
# Only valid attestations go in the list # Only valid attestations go in the list
if not (pool.nextAttestationEpoch.lenu64.ValidatorIndex > validator_index): if not (pool.nextAttestationEpoch.len > validator_index):
pool.nextAttestationEpoch.setLen(validator_index.int + 1) pool.nextAttestationEpoch.setLen(validator_index + 1)
pool.nextAttestationEpoch[validator_index].subnet = pool.nextAttestationEpoch[validator_index].subnet =
attestation.data.target.epoch + 1 attestation.data.target.epoch + 1
@ -298,7 +298,7 @@ proc validateAttestation*(
proc validateAggregate*( proc validateAggregate*(
pool: var AttestationPool, pool: var AttestationPool,
signedAggregateAndProof: SignedAggregateAndProof, wallTime: BeaconTime): signedAggregateAndProof: SignedAggregateAndProof, wallTime: BeaconTime):
Result[HashSet[ValidatorIndex], (ValidationResult, cstring)] = Result[IntSet, (ValidationResult, cstring)] =
let let
aggregate_and_proof = signedAggregateAndProof.message aggregate_and_proof = signedAggregateAndProof.message
aggregate = aggregate_and_proof.aggregate aggregate = aggregate_and_proof.aggregate

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH # Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -9,7 +9,7 @@
import import
# Standard libraries # Standard libraries
std/[deques, sequtils, sets, tables, options], std/[deques, intsets, options, sequtils, tables],
# Status libraries # Status libraries
chronicles, stew/[byteutils], json_serialization/std/sets as jsonSets, chronicles, stew/[byteutils], json_serialization/std/sets as jsonSets,
# Internal # Internal
@ -19,7 +19,7 @@ import
./beacon_node_types, ./beacon_node_types,
./fork_choice/fork_choice ./fork_choice/fork_choice
export beacon_node_types, sets export beacon_node_types, intsets
logScope: topics = "attpool" logScope: topics = "attpool"
@ -89,7 +89,7 @@ proc init*(T: type AttestationPool, chainDag: ChainDAGRef, quarantine: Quarantin
) )
proc addForkChoiceVotes( proc addForkChoiceVotes(
pool: var AttestationPool, slot: Slot, participants: HashSet[ValidatorIndex], pool: var AttestationPool, slot: Slot, participants: IntSet,
block_root: Eth2Digest, wallSlot: Slot) = block_root: Eth2Digest, wallSlot: Slot) =
# Add attestation votes to fork choice # Add attestation votes to fork choice
if (let v = pool.forkChoice.on_attestation( if (let v = pool.forkChoice.on_attestation(
@ -150,7 +150,7 @@ func addToAggregates(pool: var AttestationPool, attestation: Attestation) =
proc addAttestation*(pool: var AttestationPool, proc addAttestation*(pool: var AttestationPool,
attestation: Attestation, attestation: Attestation,
participants: HashSet[ValidatorIndex], participants: IntSet,
wallSlot: Slot) = wallSlot: Slot) =
## Add an attestation to the pool, assuming it's been validated already. ## Add an attestation to the pool, assuming it's been validated already.
## Attestations may be either agggregated or not - we're pursuing an eager ## Attestations may be either agggregated or not - we're pursuing an eager

View File

@ -1,7 +1,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[deques, sets, streams, tables], std/[deques, intsets, streams, tables],
stew/endians2, stew/endians2,
spec/[datatypes, digest, crypto], spec/[datatypes, digest, crypto],
block_pools/block_pools_types, block_pools/block_pools_types,
@ -106,13 +106,13 @@ type
voluntary_exits*: Deque[SignedVoluntaryExit] ## \ voluntary_exits*: Deque[SignedVoluntaryExit] ## \
## Not a function of chain DAG branch; just used as a FIFO queue for blocks ## Not a function of chain DAG branch; just used as a FIFO queue for blocks
prior_seen_attester_slashed_indices*: HashSet[uint64] ##\ prior_seen_attester_slashed_indices*: IntSet ##\
## Records attester-slashed indices seen. ## Records attester-slashed indices seen.
prior_seen_proposer_slashed_indices*: HashSet[uint64] ##\ prior_seen_proposer_slashed_indices*: IntSet ##\
## Records proposer-slashed indices seen. ## Records proposer-slashed indices seen.
prior_seen_voluntary_exit_indices*: HashSet[uint64] ##\ prior_seen_voluntary_exit_indices*: IntSet ##\
## Records voluntary exit indices seen. ## Records voluntary exit indices seen.
chainDag*: ChainDAGRef chainDag*: ChainDAGRef

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH # Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -8,7 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[algorithm, sequtils, sets], std/[algorithm, intsets, sequtils],
chronicles, chronicles,
../spec/[ ../spec/[
crypto, datatypes, digest, helpers, presets, signatures, crypto, datatypes, digest, helpers, presets, signatures,
@ -82,9 +82,11 @@ iterator get_attesting_indices*(epochRef: EpochRef,
func get_attesting_indices*(epochRef: EpochRef, func get_attesting_indices*(epochRef: EpochRef,
data: AttestationData, data: AttestationData,
bits: CommitteeValidatorsBits): bits: CommitteeValidatorsBits):
HashSet[ValidatorIndex] = IntSet =
for idx in get_attesting_indices(epochRef, data, bits): for idx in get_attesting_indices(epochRef, data, bits):
result.incl(idx) # Because it must have been in get_beacon_committee(...), it's a valid
# validator index, so the conversion is as safe as it is anywhere.
result.incl(idx.int)
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_indexed_attestation # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_indexed_attestation
func get_indexed_attestation*(epochRef: EpochRef, attestation: Attestation): IndexedAttestation = func get_indexed_attestation*(epochRef: EpochRef, attestation: Attestation): IndexedAttestation =
@ -144,7 +146,7 @@ proc is_valid_indexed_attestation*(
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#is_valid_indexed_attestation
proc is_valid_indexed_attestation*( proc is_valid_indexed_attestation*(
fork: Fork, genesis_validators_root: Eth2Digest, fork: Fork, genesis_validators_root: Eth2Digest,
epochRef: EpochRef, attesting_indices: HashSet[ValidatorIndex], epochRef: EpochRef, attesting_indices: IntSet,
attestation: SomeAttestation, flags: UpdateFlags): Result[void, cstring] = attestation: SomeAttestation, flags: UpdateFlags): Result[void, cstring] =
# This is a variation on `is_valid_indexed_attestation` that works directly # This is a variation on `is_valid_indexed_attestation` that works directly
# with an attestation instead of first constructing an `IndexedAttestation` # with an attestation instead of first constructing an `IndexedAttestation`

View File

@ -60,7 +60,7 @@ type
AttestationEntry* = object AttestationEntry* = object
v*: Attestation v*: Attestation
attesting_indices*: HashSet[ValidatorIndex] attesting_indices*: IntSet
AggregateEntry* = AttestationEntry AggregateEntry* = AttestationEntry
@ -306,7 +306,7 @@ proc blockValidator*(
proc checkForPotentialSelfSlashing( proc checkForPotentialSelfSlashing(
self: var Eth2Processor, attestationData: AttestationData, self: var Eth2Processor, attestationData: AttestationData,
attesterIndices: HashSet[ValidatorIndex], wallSlot: Slot) = attesterIndices: IntSet, wallSlot: Slot) =
# Attestations remain valid for 32 slots, so avoid confusing with one's own # Attestations remain valid for 32 slots, so avoid confusing with one's own
# reflections, for a ATTESTATION_PROPAGATION_SLOT_RANGE div SLOTS_PER_EPOCH # reflections, for a ATTESTATION_PROPAGATION_SLOT_RANGE div SLOTS_PER_EPOCH
# period after the attestation slot. For mainnet this can be one additional # period after the attestation slot. For mainnet this can be one additional

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2020 Status Research & Development GmbH # Copyright (c) 2020-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -9,7 +9,7 @@
import import
# Standard libraries # Standard libraries
std/[deques, options, sequtils, sets, tables], std/[deques, intsets, options, sequtils, tables],
# Status libraries # Status libraries
chronicles, json_serialization/std/sets as jsonSets, chronicles, json_serialization/std/sets as jsonSets,
# Internal # Internal
@ -17,7 +17,7 @@ import
./block_pools/[chain_dag, clearance, quarantine], ./block_pools/[chain_dag, clearance, quarantine],
./beacon_node_types ./beacon_node_types
export beacon_node_types, sets export beacon_node_types, intsets
logScope: topics = "exitpool" logScope: topics = "exitpool"
@ -58,10 +58,10 @@ iterator getValidatorIndices(attester_slashing: AttesterSlashing): uint64 =
attestation_2_indices = attestation_2_indices =
attester_slashing.attestation_2.attesting_indices.asSeq attester_slashing.attestation_2.attesting_indices.asSeq
attester_slashed_indices = attester_slashed_indices =
toHashSet(attestation_1_indices) * toHashSet(attestation_2_indices) toIntSet(attestation_1_indices) * toIntSet(attestation_2_indices)
for validator_index in attester_slashed_indices: for validator_index in attester_slashed_indices:
yield validator_index yield validator_index.uint64
iterator getValidatorIndices(proposer_slashing: ProposerSlashing): uint64 = iterator getValidatorIndices(proposer_slashing: ProposerSlashing): uint64 =
yield proposer_slashing.signed_header_1.message.proposer_index yield proposer_slashing.signed_header_1.message.proposer_index
@ -155,7 +155,7 @@ proc validateAttesterSlashing*(
attestation_2_indices = attestation_2_indices =
attester_slashing.attestation_2.attesting_indices.asSeq attester_slashing.attestation_2.attesting_indices.asSeq
attester_slashed_indices = attester_slashed_indices =
toHashSet(attestation_1_indices) * toHashSet(attestation_2_indices) toIntSet(attestation_1_indices) * toIntSet(attestation_2_indices)
if not disjoint( if not disjoint(
attester_slashed_indices, pool.prior_seen_attester_slashed_indices): attester_slashed_indices, pool.prior_seen_attester_slashed_indices):
@ -180,10 +180,15 @@ proc validateAttesterSlashing*(
proc validateProposerSlashing*( proc validateProposerSlashing*(
pool: var ExitPool, proposer_slashing: ProposerSlashing): pool: var ExitPool, proposer_slashing: ProposerSlashing):
Result[bool, (ValidationResult, cstring)] = Result[bool, (ValidationResult, cstring)] =
# Not from spec; the rest of NBC wouldn't have correctly processed it either.
if proposer_slashing.signed_header_1.message.proposer_index > high(int).uint64:
return err((ValidationResult.Ignore, cstring(
"validateProposerSlashing: proposer-slashed index too high")))
# [IGNORE] The proposer slashing is the first valid proposer slashing # [IGNORE] The proposer slashing is the first valid proposer slashing
# received for the proposer with index # received for the proposer with index
# proposer_slashing.signed_header_1.message.proposer_index. # proposer_slashing.signed_header_1.message.proposer_index.
if proposer_slashing.signed_header_1.message.proposer_index in if proposer_slashing.signed_header_1.message.proposer_index.int in
pool.prior_seen_proposer_slashed_indices: pool.prior_seen_proposer_slashed_indices:
return err((ValidationResult.Ignore, cstring( return err((ValidationResult.Ignore, cstring(
"validateProposerSlashing: proposer-slashed index already proposer-slashed"))) "validateProposerSlashing: proposer-slashed index already proposer-slashed")))
@ -196,7 +201,7 @@ proc validateProposerSlashing*(
return err((ValidationResult.Reject, proposer_slashing_validity.error)) return err((ValidationResult.Reject, proposer_slashing_validity.error))
pool.prior_seen_proposer_slashed_indices.incl( pool.prior_seen_proposer_slashed_indices.incl(
proposer_slashing.signed_header_1.message.proposer_index) proposer_slashing.signed_header_1.message.proposer_index.int)
pool.proposer_slashings.addExitMessage( pool.proposer_slashings.addExitMessage(
proposer_slashing, PROPOSER_SLASHINGS_BOUND) proposer_slashing, PROPOSER_SLASHINGS_BOUND)
@ -212,7 +217,11 @@ proc validateVoluntaryExit*(
pool.chainDag.headState.data.data.validators.lenu64: pool.chainDag.headState.data.data.validators.lenu64:
return err((ValidationResult.Ignore, cstring( return err((ValidationResult.Ignore, cstring(
"validateVoluntaryExit: validator index too high"))) "validateVoluntaryExit: validator index too high")))
if signed_voluntary_exit.message.validator_index in
# Since pool.chainDag.headState.data.data.validators is a seq, this means
# signed_voluntary_exit.message.validator_index.int is already valid, but
# check explicitly if one changes that data structure.
if signed_voluntary_exit.message.validator_index.int in
pool.prior_seen_voluntary_exit_indices: pool.prior_seen_voluntary_exit_indices:
return err((ValidationResult.Ignore, cstring( return err((ValidationResult.Ignore, cstring(
"validateVoluntaryExit: validator index already voluntarily exited"))) "validateVoluntaryExit: validator index already voluntarily exited")))
@ -226,7 +235,7 @@ proc validateVoluntaryExit*(
return err((ValidationResult.Reject, voluntary_exit_validity.error)) return err((ValidationResult.Reject, voluntary_exit_validity.error))
pool.prior_seen_voluntary_exit_indices.incl( pool.prior_seen_voluntary_exit_indices.incl(
signed_voluntary_exit.message.validator_index) signed_voluntary_exit.message.validator_index.int)
pool.voluntary_exits.addExitMessage( pool.voluntary_exits.addExitMessage(
signed_voluntary_exit, VOLUNTARY_EXITS_BOUND) signed_voluntary_exit, VOLUNTARY_EXITS_BOUND)

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH # Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -9,7 +9,7 @@
import import
# Standard library # Standard library
std/[sequtils, sets, tables], std/[intsets, sequtils, tables],
# Status libraries # Status libraries
stew/results, chronicles, stew/results, chronicles,
# Internal # Internal
@ -18,7 +18,7 @@ import
./fork_choice_types, ./proto_array, ./fork_choice_types, ./proto_array,
../block_pools/[spec_cache, chain_dag] ../block_pools/[spec_cache, chain_dag]
export sets, results, fork_choice_types export intsets, results, fork_choice_types
export proto_array.len export proto_array.len
# https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/fork-choice.md # https://github.com/ethereum/eth2.0-specs/blob/v0.12.1/specs/phase0/fork-choice.md
@ -138,7 +138,7 @@ func process_attestation*(
## Add an attestation to the fork choice context ## Add an attestation to the fork choice context
self.votes.extend(validator_index.int + 1) self.votes.extend(validator_index.int + 1)
template vote: untyped = self.votes[validator_index.int] template vote: untyped = self.votes[validator_index]
# alias # alias
if target_epoch > vote.next_epoch or vote == default(VoteTracker): if target_epoch > vote.next_epoch or vote == default(VoteTracker):
@ -156,7 +156,7 @@ proc process_attestation_queue(self: var ForkChoice) =
if it.slot < self.checkpoints.time: if it.slot < self.checkpoints.time:
for validator_index in it.attesting_indices: for validator_index in it.attesting_indices:
self.backend.process_attestation( self.backend.process_attestation(
validator_index, it.block_root, it.slot.epoch()) validator_index.ValidatorIndex, it.block_root, it.slot.epoch())
false false
else: else:
true true
@ -174,7 +174,7 @@ proc on_attestation*(
dag: ChainDAGRef, dag: ChainDAGRef,
attestation_slot: Slot, attestation_slot: Slot,
beacon_block_root: Eth2Digest, beacon_block_root: Eth2Digest,
attesting_indices: HashSet[ValidatorIndex], attesting_indices: IntSet,
wallSlot: Slot wallSlot: Slot
): FcResult[void] = ): FcResult[void] =
? self.update_time(dag, wallSlot) ? self.update_time(dag, wallSlot)
@ -186,7 +186,8 @@ proc on_attestation*(
for validator_index in attesting_indices: for validator_index in attesting_indices:
# attestation_slot and target epoch must match, per attestation rules # attestation_slot and target epoch must match, per attestation rules
self.backend.process_attestation( self.backend.process_attestation(
validator_index, beacon_block_root, attestation_slot.epoch) validator_index.ValidatorIndex, beacon_block_root,
attestation_slot.epoch)
else: else:
self.queuedAttestations.add(QueuedAttestation( self.queuedAttestations.add(QueuedAttestation(
slot: attestation_slot, slot: attestation_slot,

View File

@ -160,7 +160,7 @@ type
QueuedAttestation* = object QueuedAttestation* = object
slot*: Slot slot*: Slot
attesting_indices*: seq[ValidatorIndex] attesting_indices*: seq[int]
block_root*: Eth2Digest block_root*: Eth2Digest
target_epoch*: Epoch target_epoch*: Epoch

View File

@ -407,7 +407,7 @@ proc updateSubscriptionSchedule(node: BeaconNode, epoch: Epoch) {.async.} =
doAssert epoch >= 1 doAssert epoch >= 1
let let
attachedValidators = node.getAttachedValidators() attachedValidators = node.getAttachedValidators()
validatorIndices = toHashSet(toSeq(attachedValidators.keys())) validatorIndices = toIntSet(toSeq(attachedValidators.keys()))
var cache = StateCache() var cache = StateCache()
@ -446,7 +446,7 @@ proc updateSubscriptionSchedule(node: BeaconNode, epoch: Epoch) {.async.} =
validatorIndices, validatorIndices,
is_aggregator( is_aggregator(
committeeLen, committeeLen,
await attachedValidators[it].getSlotSig( await attachedValidators[it.ValidatorIndex].getSlotSig(
node.chainDag.headState.data.data.fork, node.chainDag.headState.data.data.fork,
node.chainDag.headState.data.data.genesis_validators_root, slot))) node.chainDag.headState.data.data.genesis_validators_root, slot)))
@ -594,8 +594,7 @@ proc cycleAttestationSubnets(node: BeaconNode, wallSlot: Slot) {.async.} =
proc getInitialAttestationSubnets(node: BeaconNode): Table[uint8, Slot] = proc getInitialAttestationSubnets(node: BeaconNode): Table[uint8, Slot] =
let let
wallEpoch = node.beaconClock.now().slotOrZero().epoch wallEpoch = node.beaconClock.now().slotOrZero().epoch
validatorIndices = validatorIndices = toIntSet(toSeq(node.getAttachedValidators().keys()))
toHashSet(toSeq(node.getAttachedValidators().keys()))
var cache = StateCache() var cache = StateCache()

View File

@ -5,7 +5,7 @@
# at your option. This file may not be copied, modified, or distributed except according to those terms. # at your option. This file may not be copied, modified, or distributed except according to those terms.
import import
std/[deques, sequtils], std/[deques, sequtils, sets],
chronos, chronos,
stew/shims/macros, stew/shims/macros,
stew/byteutils, stew/byteutils,

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH # Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -8,7 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[tables, algorithm, math, sequtils, options], std/[algorithm, intsets, math, options, sequtils, tables],
stew/assign2, stew/assign2,
json_serialization/std/sets, json_serialization/std/sets,
chronicles, chronicles,
@ -495,10 +495,9 @@ iterator get_attesting_indices*(bits: CommitteeValidatorsBits,
trace "get_attesting_indices: inconsistent aggregation and committee length" trace "get_attesting_indices: inconsistent aggregation and committee length"
func get_attesting_indices*(bits: CommitteeValidatorsBits, func get_attesting_indices*(bits: CommitteeValidatorsBits,
committee: openArray[ValidatorIndex]): committee: openArray[ValidatorIndex]): IntSet =
HashSet[ValidatorIndex] =
for idx in get_attesting_indices(bits, committee): for idx in get_attesting_indices(bits, committee):
result.incl idx result.incl idx.int
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_attesting_indices # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_attesting_indices
iterator get_attesting_indices*(state: BeaconState, iterator get_attesting_indices*(state: BeaconState,
@ -518,11 +517,10 @@ iterator get_attesting_indices*(state: BeaconState,
func get_attesting_indices*(state: BeaconState, func get_attesting_indices*(state: BeaconState,
data: AttestationData, data: AttestationData,
bits: CommitteeValidatorsBits, bits: CommitteeValidatorsBits,
cache: var StateCache): cache: var StateCache): IntSet =
HashSet[ValidatorIndex] =
# Return the set of attesting indices corresponding to ``data`` and ``bits``. # Return the set of attesting indices corresponding to ``data`` and ``bits``.
for index in get_attesting_indices(state, data, bits, cache): for index in get_attesting_indices(state, data, bits, cache):
result.incl index result.incl index.int
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_indexed_attestation # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#get_indexed_attestation
func get_indexed_attestation(state: BeaconState, attestation: Attestation, func get_indexed_attestation(state: BeaconState, attestation: Attestation,

View File

@ -25,7 +25,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[macros, hashes, json, strutils, tables, typetraits], std/[macros, hashes, intsets, json, strutils, tables, typetraits],
stew/[assign2, byteutils], chronicles, stew/[assign2, byteutils], chronicles,
json_serialization/types as jsonTypes, json_serialization/types as jsonTypes,
../version, ../ssz/types as sszTypes, ./crypto, ./digest, ./presets ../version, ../ssz/types as sszTypes, ./crypto, ./digest, ./presets
@ -746,6 +746,18 @@ proc readValue*(reader: var JsonReader, value: var ForkDigest)
except ValueError: except ValueError:
raiseUnexpectedValue(reader, "Hex string of 4 bytes expected") raiseUnexpectedValue(reader, "Hex string of 4 bytes expected")
# In general, ValidatorIndex is assumed to be convertible to/from an int. This
# should be valid for a long time, because
# https://github.com/ethereum/annotated-spec/blob/master/phase0/beacon-chain.md#configuration
# notes that "The maximum supported validator count is 2**22 (=4,194,304), or
# ~134 million ETH staking. Assuming 32 slots per epoch and 64 committees per
# slot, this gets us to a max 2048 validators in a committee."
#
# That's only active validators, so in principle, it can grow larger, but it
# should be orders of magnitude more validators than expected in the next in
# the next couple of years, than int32 indexing supports.
static: doAssert high(int) >= high(int32)
# `ValidatorIndex` seq handling. # `ValidatorIndex` seq handling.
func `[]`*[T](a: var seq[T], b: ValidatorIndex): var T = func `[]`*[T](a: var seq[T], b: ValidatorIndex): var T =
a[b.int] a[b.int]
@ -762,6 +774,15 @@ proc `<`*(x, y: ValidatorIndex) : bool {.borrow, noSideEffect.}
proc hash*(x: ValidatorIndex): Hash {.borrow, noSideEffect.} proc hash*(x: ValidatorIndex): Hash {.borrow, noSideEffect.}
func `$`*(x: ValidatorIndex): auto = $(distinctBase(x)) func `$`*(x: ValidatorIndex): auto = $(distinctBase(x))
# TODO Nim 1.4, but not Nim 1.2, defines a function by this name, which works
# only on openArray[int]. They do the same thing, so either's fine, when both
# overloads match. The Nim 1.4 stdlib doesn't int-convert but it's a no-op in
# its case regardless.
func toIntSet*[T](x: openArray[T]): IntSet =
result = initIntSet()
for item in items(x):
result.incl(item.int)
proc `==`*(x, y: CommitteeIndex) : bool {.borrow, noSideEffect.} proc `==`*(x, y: CommitteeIndex) : bool {.borrow, noSideEffect.}
proc `<`*(x, y: CommitteeIndex) : bool {.borrow, noSideEffect.} proc `<`*(x, y: CommitteeIndex) : bool {.borrow, noSideEffect.}
proc hash*(x: CommitteeIndex): Hash {.borrow, noSideEffect.} proc hash*(x: CommitteeIndex): Hash {.borrow, noSideEffect.}

View File

@ -8,7 +8,7 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[strformat, sets], std/[intsets, strformat],
./datatypes, ./helpers, ./validator ./datatypes, ./helpers, ./validator
const const
@ -88,9 +88,9 @@ func getAttestationTopic*(forkDigest: ForkDigest, subnetIndex: uint64):
iterator get_committee_assignments*( iterator get_committee_assignments*(
state: BeaconState, epoch: Epoch, state: BeaconState, epoch: Epoch,
validator_indices: HashSet[ValidatorIndex], validator_indices: IntSet,
cache: var StateCache): cache: var StateCache):
tuple[validatorIndices: HashSet[ValidatorIndex], tuple[validatorIndices: IntSet,
committeeIndex: CommitteeIndex, committeeIndex: CommitteeIndex,
subnetIndex: uint8, slot: Slot] = subnetIndex: uint8, slot: Slot] =
let let
@ -102,7 +102,7 @@ iterator get_committee_assignments*(
let let
idx = index.CommitteeIndex idx = index.CommitteeIndex
includedIndices = includedIndices =
toHashSet(get_beacon_committee(state, slot, idx, cache)) * toIntSet(get_beacon_committee(state, slot, idx, cache)) *
validator_indices validator_indices
if includedIndices.len > 0: if includedIndices.len > 0:
yield ( yield (

View File

@ -1,5 +1,5 @@
# beacon_chain # beacon_chain
# Copyright (c) 2018-2020 Status Research & Development GmbH # Copyright (c) 2018-2021 Status Research & Development GmbH
# Licensed and distributed under either of # Licensed and distributed under either of
# * MIT license (license terms in the root directory or at https://opensource.org/licenses/MIT). # * 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). # * Apache v2 license (license terms in the root directory or at https://www.apache.org/licenses/LICENSE-2.0).
@ -20,17 +20,13 @@
{.push raises: [Defect].} {.push raises: [Defect].}
import import
std/[algorithm, collections/sets, options, sequtils, sets], std/[algorithm, intsets, options, sequtils],
chronicles, chronicles,
../extras, ../ssz/merkleization, metrics, ../extras, ../ssz/merkleization, metrics,
./beaconstate, ./crypto, ./datatypes, ./digest, ./helpers, ./validator, ./beaconstate, ./crypto, ./datatypes, ./digest, ./helpers, ./validator,
./signatures, ./presets, ./signatures, ./presets,
../../nbench/bench_lab ../../nbench/bench_lab
# Generics visibility issue with toSeq(items(intersection(HashSet, HashSet)))
# https://github.com/nim-lang/Nim/issues/11225
export sets
# https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#block-header # https://github.com/ethereum/eth2.0-specs/blob/v1.0.0/specs/phase0/beacon-chain.md#block-header
func process_block_header*( func process_block_header*(
state: var BeaconState, blck: SomeBeaconBlock, flags: UpdateFlags, state: var BeaconState, blck: SomeBeaconBlock, flags: UpdateFlags,
@ -215,8 +211,8 @@ proc check_attester_slashing*(
var slashed_indices: seq[ValidatorIndex] var slashed_indices: seq[ValidatorIndex]
for index in sorted(toSeq(intersection( for index in sorted(toSeq(intersection(
toHashSet(attestation_1.attesting_indices.asSeq), toIntSet(attestation_1.attesting_indices.asSeq),
toHashSet(attestation_2.attesting_indices.asSeq)).items), system.cmp): toIntSet(attestation_2.attesting_indices.asSeq)).items), system.cmp):
if is_slashable_validator( if is_slashable_validator(
state.validators.asSeq()[index], get_current_epoch(state)): state.validators.asSeq()[index], get_current_epoch(state)):
slashed_indices.add index.ValidatorIndex slashed_indices.add index.ValidatorIndex

View File

@ -25,8 +25,7 @@ import
../beacon_chain/[ ../beacon_chain/[
attestation_pool, beacon_node_types, beacon_chain_db, attestation_pool, beacon_node_types, beacon_chain_db,
validator_pool, eth1_monitor, extras], validator_pool, eth1_monitor, extras],
../beacon_chain/block_pools/[ ../beacon_chain/block_pools/[chain_dag, quarantine, clearance],
spec_cache, chain_dag, quarantine, clearance],
../beacon_chain/ssz/[merkleization, ssz_serialization], ../beacon_chain/ssz/[merkleization, ssz_serialization],
./simutils ./simutils
@ -118,7 +117,7 @@ cli do(slots = SLOTS_PER_EPOCH * 6,
data: data, data: data,
aggregation_bits: aggregation_bits, aggregation_bits: aggregation_bits,
signature: sig signature: sig
), [validatorIdx].toHashSet(), data.slot) ), [validatorIdx.int].toIntSet(), data.slot)
proc proposeBlock(slot: Slot) = proc proposeBlock(slot: Slot) =
if rand(r, 1.0) > blockRatio: if rand(r, 1.0) > blockRatio:

View File

@ -10,7 +10,7 @@
import import
# Standard library # Standard library
sets, intsets,
# Status # Status
chronicles, chronicles,
# Specs # Specs

View File

@ -77,7 +77,7 @@ suiteReport "Attestation pool processing" & preset():
state.data.data, state.blck.root, beacon_committee[0], cache) state.data.data, state.blck.root, beacon_committee[0], cache)
pool[].addAttestation( pool[].addAttestation(
attestation, [beacon_committee[0]].toHashSet(), attestation.data.slot) attestation, [beacon_committee[0]].toIntSet(), attestation.data.slot)
check: check:
process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache) process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache)
@ -107,9 +107,9 @@ suiteReport "Attestation pool processing" & preset():
# test reverse order # test reverse order
pool[].addAttestation( pool[].addAttestation(
attestation1, [bc1[0]].toHashSet, attestation1.data.slot) attestation1, [bc1[0]].toIntSet, attestation1.data.slot)
pool[].addAttestation( pool[].addAttestation(
attestation0, [bc0[0]].toHashSet, attestation1.data.slot) attestation0, [bc0[0]].toIntSet, attestation1.data.slot)
discard process_slots( discard process_slots(
state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache) state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache)
@ -131,9 +131,9 @@ suiteReport "Attestation pool processing" & preset():
state.data.data, state.blck.root, bc0[1], cache) state.data.data, state.blck.root, bc0[1], cache)
pool[].addAttestation( pool[].addAttestation(
attestation0, [bc0[0]].toHashSet, attestation0.data.slot) attestation0, [bc0[0]].toIntSet, attestation0.data.slot)
pool[].addAttestation( pool[].addAttestation(
attestation1, [bc0[1]].toHashSet, attestation1.data.slot) attestation1, [bc0[1]].toIntSet, attestation1.data.slot)
check: check:
process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache) process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache)
@ -158,9 +158,9 @@ suiteReport "Attestation pool processing" & preset():
attestation0.combine(attestation1) attestation0.combine(attestation1)
pool[].addAttestation( pool[].addAttestation(
attestation0, [bc0[0]].toHashSet, attestation0.data.slot) attestation0, [bc0[0]].toIntSet, attestation0.data.slot)
pool[].addAttestation( pool[].addAttestation(
attestation1, [bc0[1]].toHashSet, attestation1.data.slot) attestation1, [bc0[1]].toIntSet, attestation1.data.slot)
check: check:
process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache) process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache)
@ -184,9 +184,9 @@ suiteReport "Attestation pool processing" & preset():
attestation0.combine(attestation1) attestation0.combine(attestation1)
pool[].addAttestation( pool[].addAttestation(
attestation1, [bc0[1]].toHashSet, attestation1.data.slot) attestation1, [bc0[1]].toIntSet, attestation1.data.slot)
pool[].addAttestation( pool[].addAttestation(
attestation0, [bc0[0]].toHashSet, attestation0.data.slot) attestation0, [bc0[0]].toIntSet, attestation0.data.slot)
check: check:
process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache) process_slots(state.data, MIN_ATTESTATION_INCLUSION_DELAY.Slot + 1, cache)
@ -254,7 +254,7 @@ suiteReport "Attestation pool processing" & preset():
attestation0 = makeAttestation(state.data.data, b10.root, bc1[0], cache) attestation0 = makeAttestation(state.data.data, b10.root, bc1[0], cache)
pool[].addAttestation( pool[].addAttestation(
attestation0, [bc1[0]].toHashSet, attestation0.data.slot) attestation0, [bc1[0]].toIntSet, attestation0.data.slot)
let head2 = pool[].selectHead(b10Add[].slot) let head2 = pool[].selectHead(b10Add[].slot)
@ -266,7 +266,7 @@ suiteReport "Attestation pool processing" & preset():
attestation1 = makeAttestation(state.data.data, b11.root, bc1[1], cache) attestation1 = makeAttestation(state.data.data, b11.root, bc1[1], cache)
attestation2 = makeAttestation(state.data.data, b11.root, bc1[2], cache) attestation2 = makeAttestation(state.data.data, b11.root, bc1[2], cache)
pool[].addAttestation( pool[].addAttestation(
attestation1, [bc1[1]].toHashSet, attestation1.data.slot) attestation1, [bc1[1]].toIntSet, attestation1.data.slot)
let head3 = pool[].selectHead(b10Add[].slot) let head3 = pool[].selectHead(b10Add[].slot)
let bigger = if b11.root.data < b10.root.data: b10Add else: b11Add let bigger = if b11.root.data < b10.root.data: b10Add else: b11Add
@ -276,7 +276,7 @@ suiteReport "Attestation pool processing" & preset():
head3 == bigger[] head3 == bigger[]
pool[].addAttestation( pool[].addAttestation(
attestation2, [bc1[2]].toHashSet, attestation2.data.slot) attestation2, [bc1[2]].toIntSet, attestation2.data.slot)
let head4 = pool[].selectHead(b11Add[].slot) let head4 = pool[].selectHead(b11Add[].slot)