explicitly refer to phase0.{Attestation,TrustedAttestation} rather than sans module name (#6214)

This commit is contained in:
tersec 2024-04-17 22:44:29 +02:00 committed by GitHub
parent 4c031a7887
commit 603c83522e
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
30 changed files with 104 additions and 95 deletions

View File

@ -44,7 +44,7 @@ type
EventBus* = object
headQueue*: AsyncEventQueue[HeadChangeInfoObject]
blocksQueue*: AsyncEventQueue[EventBeaconBlockObject]
attestQueue*: AsyncEventQueue[Attestation]
attestQueue*: AsyncEventQueue[phase0.Attestation]
exitQueue*: AsyncEventQueue[SignedVoluntaryExit]
blsToExecQueue*: AsyncEventQueue[SignedBLSToExecutionChange]
propSlashQueue*: AsyncEventQueue[ProposerSlashing]

View File

@ -30,7 +30,7 @@ const
## that potentially could be added to a newly created block
type
OnAttestationCallback = proc(data: Attestation) {.gcsafe, raises: [].}
OnAttestationCallback = proc(data: phase0.Attestation) {.gcsafe, raises: [].}
Validation = object
## Validations collect a set of signatures for a distict attestation - in
@ -222,8 +222,9 @@ func oneIndex(bits: CommitteeValidatorsBits): Opt[int] =
return Opt.none(int)
res
func toAttestation(entry: AttestationEntry, validation: Validation): Attestation =
Attestation(
func toAttestation(entry: AttestationEntry, validation: Validation):
phase0.Attestation =
phase0.Attestation(
aggregation_bits: validation.aggregation_bits,
data: entry.data,
signature: validation.aggregate_signature.finish().toValidatorSig()
@ -293,7 +294,7 @@ func covers(entry: AttestationEntry, bits: CommitteeValidatorsBits): bool =
false
proc addAttestation(entry: var AttestationEntry,
attestation: Attestation,
attestation: phase0.Attestation,
signature: CookedSig): bool =
logScope:
attestation = shortLog(attestation)
@ -335,7 +336,7 @@ proc addAttestation(entry: var AttestationEntry,
true
proc addAttestation*(pool: var AttestationPool,
attestation: Attestation,
attestation: phase0.Attestation,
attesting_indices: openArray[ValidatorIndex],
signature: CookedSig,
wallTime: BeaconTime) =
@ -419,8 +420,9 @@ proc addForkChoice*(pool: var AttestationPool,
error "Couldn't add block to fork choice, bug?",
blck = shortLog(blck), err = state.error
iterator attestations*(pool: AttestationPool, slot: Opt[Slot],
committee_index: Opt[CommitteeIndex]): Attestation =
iterator attestations*(
pool: AttestationPool, slot: Opt[Slot],
committee_index: Opt[CommitteeIndex]): phase0.Attestation =
let candidateIndices =
if slot.isSome():
let candidateIdx = pool.candidateIdx(slot.get())
@ -434,7 +436,7 @@ iterator attestations*(pool: AttestationPool, slot: Opt[Slot],
for candidateIndex in candidateIndices:
for _, entry in pool.candidates[candidateIndex]:
if committee_index.isNone() or entry.data.index == committee_index.get():
var singleAttestation = Attestation(
var singleAttestation = phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(entry.committee_len),
data: entry.data)
@ -555,7 +557,7 @@ proc check_attestation_compatible*(
proc getAttestationsForBlock*(pool: var AttestationPool,
state: ForkyHashedBeaconState,
cache: var StateCache): seq[Attestation] =
cache: var StateCache): seq[phase0.Attestation] =
## Retrieve attestations that may be added to a new block at the slot of the
## given state
## https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#attestations
@ -641,7 +643,7 @@ proc getAttestationsForBlock*(pool: var AttestationPool,
state.data.previous_epoch_attestations.maxLen -
state.data.previous_epoch_attestations.len()
var res: seq[Attestation]
var res: seq[phase0.Attestation]
let totalCandidates = candidates.len()
while candidates.len > 0 and res.lenu64() < MAX_ATTESTATIONS:
let entryCacheKey = block:
@ -698,7 +700,7 @@ proc getAttestationsForBlock*(pool: var AttestationPool,
proc getAttestationsForBlock*(pool: var AttestationPool,
state: ForkedHashedBeaconState,
cache: var StateCache): seq[Attestation] =
cache: var StateCache): seq[phase0.Attestation] =
withState(state):
pool.getAttestationsForBlock(forkyState, cache)
@ -717,13 +719,13 @@ func bestValidation(aggregates: openArray[Validation]): (int, int) =
bestIndex = i
(bestIndex, best)
func getAggregatedAttestation*(pool: var AttestationPool,
slot: Slot,
attestation_data_root: Eth2Digest): Opt[Attestation] =
func getAggregatedAttestation*(
pool: var AttestationPool, slot: Slot, attestation_data_root: Eth2Digest):
Opt[phase0.Attestation] =
let
candidateIdx = pool.candidateIdx(slot)
if candidateIdx.isNone:
return Opt.none(Attestation)
return Opt.none(phase0.Attestation)
pool.candidates[candidateIdx.get].withValue(attestation_data_root, entry):
entry[].updateAggregates()
@ -733,19 +735,19 @@ func getAggregatedAttestation*(pool: var AttestationPool,
# Found the right hash, no need to look further
return Opt.some(entry[].toAttestation(entry[].aggregates[bestIndex]))
Opt.none(Attestation)
Opt.none(phase0.Attestation)
func getAggregatedAttestation*(pool: var AttestationPool,
slot: Slot,
index: CommitteeIndex): Opt[Attestation] =
func getAggregatedAttestation*(
pool: var AttestationPool, slot: Slot, index: CommitteeIndex):
Opt[phase0.Attestation] =
## Select the attestation that has the most votes going for it in the given
## slot/index
## https://github.com/ethereum/consensus-specs/blob/v1.4.0/specs/phase0/validator.md#construct-aggregate
let candidateIdx = pool.candidateIdx(slot)
if candidateIdx.isNone:
return Opt.none(Attestation)
return Opt.none(phase0.Attestation)
var res: Opt[Attestation]
var res: Opt[phase0.Attestation]
for _, entry in pool.candidates[candidateIdx.get].mpairs():
doAssert entry.data.slot == slot
if index != entry.data.index:

View File

@ -98,7 +98,7 @@ iterator get_attesting_indices*(shufflingRef: ShufflingRef,
yield validator_index
iterator get_attesting_indices*(
dag: ChainDAGRef, attestation: TrustedAttestation): ValidatorIndex =
dag: ChainDAGRef, attestation: phase0.TrustedAttestation): ValidatorIndex =
block: # `return` is not allowed in an inline iterator
let
slot =

View File

@ -342,7 +342,7 @@ proc clearDoppelgangerProtection*(self: var Eth2Processor) =
self.doppelgangerDetection.broadcastStartEpoch = FAR_FUTURE_EPOCH
proc checkForPotentialDoppelganger(
self: var Eth2Processor, attestation: Attestation,
self: var Eth2Processor, attestation: phase0.Attestation,
attesterIndices: openArray[ValidatorIndex]) =
# Only check for attestations after node launch. There might be one slot of
# overlap in quick intra-slot restarts so trade off a few true negatives in
@ -364,7 +364,7 @@ proc checkForPotentialDoppelganger(
proc processAttestation*(
self: ref Eth2Processor, src: MsgSource,
attestation: Attestation, subnet_id: SubnetId,
attestation: phase0.Attestation, subnet_id: SubnetId,
checkSignature: bool = true): Future[ValidationRes] {.async: (raises: [CancelledError]).} =
var wallTime = self.getCurrentBeaconTime()
let (afterGenesis, wallSlot) = wallTime.toSlot()

View File

@ -158,7 +158,8 @@ func check_beacon_and_target_block(
ok(target)
func check_aggregation_count(
attestation: Attestation, singular: bool): Result[void, ValidationError] =
attestation: phase0.Attestation, singular: bool):
Result[void, ValidationError] =
let ones = attestation.aggregation_bits.countOnes()
if singular and ones != 1:
return errReject("Attestation must have a single attestation bit set")
@ -650,7 +651,7 @@ proc validateBeaconBlock*(
proc validateAttestation*(
pool: ref AttestationPool,
batchCrypto: ref BatchCrypto,
attestation: Attestation,
attestation: phase0.Attestation,
wallTime: BeaconTime,
subnet_id: SubnetId, checkSignature: bool):
Future[Result[

View File

@ -837,7 +837,7 @@ template gossipMaxSize(T: untyped): uint32 =
# Attestation, AttesterSlashing, and SignedAggregateAndProof, which all
# have lists bounded at MAX_VALIDATORS_PER_COMMITTEE (2048) items, thus
# having max sizes significantly smaller than GOSSIP_MAX_SIZE.
elif T is Attestation or T is AttesterSlashing or
elif T is phase0.Attestation or T is AttesterSlashing or
T is SignedAggregateAndProof or T is phase0.SignedBeaconBlock or
T is altair.SignedBeaconBlock or T is SomeForkyLightClientObject:
GOSSIP_MAX_SIZE
@ -2565,7 +2565,7 @@ proc getWallEpoch(node: Eth2Node): Epoch =
node.getBeaconTime().slotOrZero.epoch
proc broadcastAttestation*(
node: Eth2Node, subnet_id: SubnetId, attestation: Attestation):
node: Eth2Node, subnet_id: SubnetId, attestation: phase0.Attestation):
Future[SendResult] {.async: (raises: [CancelledError], raw: true).} =
# Regardless of the contents of the attestation,
# https://github.com/ethereum/consensus-specs/blob/v1.3.0/specs/altair/p2p-interface.md#transitioning-the-gossip

View File

@ -277,7 +277,7 @@ proc initFullNode(
getBeaconTime: GetBeaconTimeFn) {.async.} =
template config(): auto = node.config
proc onAttestationReceived(data: Attestation) =
proc onAttestationReceived(data: phase0.Attestation) =
node.eventBus.attestQueue.emit(data)
proc onSyncContribution(data: SignedContributionAndProof) =
node.eventBus.contribQueue.emit(data)
@ -1753,7 +1753,7 @@ proc installMessageValidators(node: BeaconNode) =
let subnet_id = it
node.network.addAsyncValidator(
getAttestationTopic(digest, subnet_id), proc (
attestation: Attestation
attestation: phase0.Attestation
): Future[ValidationResult] {.async: (raises: [CancelledError]).} =
return toValidationResult(
await node.processor.processAttestation(

View File

@ -1192,7 +1192,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
Opt.some(rslot.get())
else:
Opt.none(Slot)
var res: seq[Attestation]
var res: seq[phase0.Attestation]
for item in node.attestationPool[].attestations(vslot, vindex):
res.add(item)
RestApiResponse.jsonResponse(res)
@ -1204,7 +1204,7 @@ proc installBeaconApiHandlers*(router: var RestRouter, node: BeaconNode) =
block:
if contentBody.isNone():
return RestApiResponse.jsonError(Http400, EmptyRequestBodyError)
let dres = decodeBody(seq[Attestation], contentBody.get())
let dres = decodeBody(seq[phase0.Attestation], contentBody.get())
if dres.isErr():
return RestApiResponse.jsonError(Http400,
InvalidAttestationObjectError,

View File

@ -45,7 +45,6 @@ createJsonFlavor RestJson
RestJson.useDefaultSerializationFor(
AggregateAndProof,
Attestation,
AttestationData,
AttesterSlashing,
BLSToExecutionChange,
@ -180,7 +179,6 @@ RestJson.useDefaultSerializationFor(
SyncCommittee,
SyncCommitteeContribution,
SyncCommitteeMessage,
TrustedAttestation,
Validator,
ValidatorRegistrationV1,
VoluntaryExit,
@ -258,10 +256,12 @@ RestJson.useDefaultSerializationFor(
electra_mev.ExecutionPayloadAndBlobsBundle,
electra_mev.SignedBlindedBeaconBlock,
electra_mev.SignedBuilderBid,
phase0.Attestation,
phase0.BeaconBlock,
phase0.BeaconBlockBody,
phase0.BeaconState,
phase0.SignedBeaconBlock,
phase0.TrustedAttestation
)
# TODO
@ -349,7 +349,7 @@ type
ForkedMaybeBlindedBeaconBlock
EncodeArrays* =
seq[Attestation] |
seq[phase0.Attestation] |
seq[PrepareBeaconProposer] |
seq[RemoteKeystoreInfo] |
seq[RestCommitteeSubscription] |
@ -1705,7 +1705,7 @@ proc readValue*(reader: var JsonReader[RestJson],
Opt[List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]]
attester_slashings:
Opt[List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]]
attestations: Opt[List[Attestation, Limit MAX_ATTESTATIONS]]
attestations: Opt[List[phase0.Attestation, Limit MAX_ATTESTATIONS]]
deposits: Opt[List[Deposit, Limit MAX_DEPOSITS]]
voluntary_exits: Opt[List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]]
sync_aggregate: Opt[SyncAggregate]
@ -1749,7 +1749,7 @@ proc readValue*(reader: var JsonReader[RestJson],
reader.raiseUnexpectedField("Multiple `attestations` fields found",
"RestPublishedBeaconBlockBody")
attestations = Opt.some(
reader.readValue(List[Attestation, Limit MAX_ATTESTATIONS]))
reader.readValue(List[phase0.Attestation, Limit MAX_ATTESTATIONS]))
of "deposits":
if deposits.isSome():
reader.raiseUnexpectedField("Multiple `deposits` fields found",

View File

@ -321,7 +321,8 @@ proc getPoolAttestations*(
meth: MethodGet.}
## https://ethereum.github.io/beacon-APIs/#/Beacon/getPoolAttestations
proc submitPoolAttestations*(body: seq[Attestation]): RestPlainResponse {.
proc submitPoolAttestations*(body: seq[phase0.Attestation]):
RestPlainResponse {.
rest, endpoint: "/eth/v1/beacon/pool/attestations",
meth: MethodPost.}
## https://ethereum.github.io/beacon-APIs/#/Beacon/submitPoolAttestations

View File

@ -535,9 +535,9 @@ type
# Types based on the OAPI yaml file - used in responses to requests
GetBeaconHeadResponse* = DataEnclosedObject[Slot]
GetAggregatedAttestationResponse* = DataEnclosedObject[Attestation]
GetAggregatedAttestationResponse* = DataEnclosedObject[phase0.Attestation]
GetAttesterDutiesResponse* = DataRootEnclosedObject[seq[RestAttesterDuty]]
GetBlockAttestationsResponse* = DataEnclosedObject[seq[Attestation]]
GetBlockAttestationsResponse* = DataEnclosedObject[seq[phase0.Attestation]]
GetBlockHeaderResponse* = DataOptimisticAndFinalizedObject[RestBlockHeaderInfo]
GetBlockHeadersResponse* = DataEnclosedObject[seq[RestBlockHeaderInfo]]
GetBlockRootResponse* = DataOptimisticObject[RestRoot]
@ -553,7 +553,7 @@ type
GetPeerCountResponse* = DataMetaEnclosedObject[RestPeerCount]
GetPeerResponse* = DataMetaEnclosedObject[RestNodePeer]
GetPeersResponse* = DataMetaEnclosedObject[seq[RestNodePeer]]
GetPoolAttestationsResponse* = DataEnclosedObject[seq[Attestation]]
GetPoolAttestationsResponse* = DataEnclosedObject[seq[phase0.Attestation]]
GetPoolAttesterSlashingsResponse* = DataEnclosedObject[seq[AttesterSlashing]]
GetPoolProposerSlashingsResponse* = DataEnclosedObject[seq[ProposerSlashing]]
GetPoolVoluntaryExitsResponse* = DataEnclosedObject[seq[SignedVoluntaryExit]]

View File

@ -34,7 +34,7 @@ type
graffiti*: GraffitiBytes
proposer_slashings*: List[ProposerSlashing, Limit MAX_PROPOSER_SLASHINGS]
attester_slashings*: List[AttesterSlashing, Limit MAX_ATTESTER_SLASHINGS]
attestations*: List[Attestation, Limit MAX_ATTESTATIONS]
attestations*: List[phase0.Attestation, Limit MAX_ATTESTATIONS]
deposits*: List[Deposit, Limit MAX_DEPOSITS]
voluntary_exits*: List[SignedVoluntaryExit, Limit MAX_VOLUNTARY_EXITS]
sync_aggregate*: SyncAggregate

View File

@ -353,7 +353,7 @@ func partialBeaconBlock*(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -372,7 +372,8 @@ func partialBeaconBlock*(
graffiti: graffiti,
proposer_slashings: validator_changes.proposer_slashings,
attester_slashings: validator_changes.attester_slashings,
attestations: List[Attestation, Limit MAX_ATTESTATIONS](attestations),
attestations:
List[phase0.Attestation, Limit MAX_ATTESTATIONS](attestations),
deposits: List[Deposit, Limit MAX_DEPOSITS](deposits),
voluntary_exits: validator_changes.voluntary_exits))
@ -404,7 +405,7 @@ proc makeBeaconBlockWithRewards*(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -462,7 +463,9 @@ proc makeBeaconBlockWithRewards*(
hash_tree_root(graffiti),
hash_tree_root(validator_changes.proposer_slashings),
hash_tree_root(validator_changes.attester_slashings),
hash_tree_root(List[Attestation, Limit MAX_ATTESTATIONS](attestations)),
hash_tree_root(
List[phase0.Attestation, Limit MAX_ATTESTATIONS](
attestations)),
hash_tree_root(List[Deposit, Limit MAX_DEPOSITS](deposits)),
hash_tree_root(validator_changes.voluntary_exits),
hash_tree_root(sync_aggregate),
@ -508,7 +511,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,
@ -530,7 +533,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,
@ -548,7 +551,7 @@ proc makeBeaconBlock*(
cfg: RuntimeConfig, state: var ForkedHashedBeaconState,
proposer_index: ValidatorIndex, randao_reveal: ValidatorSig,
eth1_data: Eth1Data, graffiti: GraffitiBytes,
attestations: seq[Attestation], deposits: seq[Deposit],
attestations: seq[phase0.Attestation], deposits: seq[Deposit],
validator_changes: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
executionPayload: ForkyExecutionPayloadForSigning,

View File

@ -1489,7 +1489,7 @@ proc produceAttestationData*(
proc submitPoolAttestations*(
vc: ValidatorClientRef,
data: seq[Attestation],
data: seq[phase0.Attestation],
strategy: ApiStrategyKind
): Future[bool] {.async.} =
const
@ -1635,7 +1635,7 @@ proc getAggregatedAttestation*(
slot: Slot,
root: Eth2Digest,
strategy: ApiStrategyKind
): Future[Attestation] {.async.} =
): Future[phase0.Attestation] {.async.} =
const
RequestName = "getAggregatedAttestation"

View File

@ -161,7 +161,7 @@ proc getSyncCommitteeMessageDataScore*(
vc.rootsSeen, vc.beaconClock.now().slotOrZero(), cdata)
proc processVotes(bits: var CommitteeBitsArray,
attestation: Attestation): int =
attestation: phase0.Attestation): int =
doAssert(len(attestation.aggregation_bits) <= len(bits))
var res = 0
for index in 0 ..< len(attestation.aggregation_bits):
@ -171,7 +171,7 @@ proc processVotes(bits: var CommitteeBitsArray,
bits[index] = true
res
proc getUniqueVotes*(attestations: openArray[Attestation]): int =
proc getUniqueVotes*(attestations: openArray[phase0.Attestation]): int =
var
res = 0
attested: Table[Slot, CommitteeTable]

View File

@ -191,7 +191,7 @@ proc routeSignedBeaconBlock*(
ok(blockRef)
proc routeAttestation*(
router: ref MessageRouter, attestation: Attestation,
router: ref MessageRouter, attestation: phase0.Attestation,
subnet_id: SubnetId, checkSignature: bool):
Future[SendResult] {.async: (raises: [CancelledError]).} =
## Process and broadcast attestation - processing will register the it with
@ -223,7 +223,7 @@ proc routeAttestation*(
return ok()
proc routeAttestation*(
router: ref MessageRouter, attestation: Attestation):
router: ref MessageRouter, attestation: phase0.Attestation):
Future[SendResult] {.async: (raises: [CancelledError]).} =
# Compute subnet, then route attestation
let

View File

@ -30,8 +30,9 @@ type
data*: AttestationData
proc toAttestation*(
registered: RegisteredAttestation, signature: ValidatorSig): Attestation =
Attestation.init(
registered: RegisteredAttestation, signature: ValidatorSig):
phase0.Attestation =
phase0.Attestation.init(
[registered.index_in_committee], registered.committee_len,
registered.data, signature).expect("valid data")

View File

@ -660,7 +660,7 @@ proc registerAttestation*(
self: var ValidatorMonitor,
src: MsgSource,
seen_timestamp: BeaconTime,
attestation: Attestation,
attestation: phase0.Attestation,
idx: ValidatorIndex) =
let
slot = attestation.data.slot

View File

@ -233,7 +233,7 @@ proc doSSZ(conf: NcliConf) =
case kind
of "attester_slashing": printit(AttesterSlashing)
of "attestation": printit(Attestation)
of "attestation": printit(phase0.Attestation)
of "phase0_signed_block": printit(phase0.SignedBeaconBlock)
of "altair_signed_block": printit(altair.SignedBeaconBlock)
of "bellatrix_signed_block": printit(bellatrix.SignedBeaconBlock)

View File

@ -21,7 +21,7 @@ import
type
AttestationInput = object
state: phase0.BeaconState
attestation: Attestation
attestation: phase0.Attestation
AttesterSlashingInput = object
state: phase0.BeaconState
attesterSlashing: AttesterSlashing

View File

@ -63,7 +63,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -106,7 +106,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -149,7 +149,7 @@ proc makeSimulationBlock(
randao_reveal: ValidatorSig,
eth1_data: Eth1Data,
graffiti: GraffitiBytes,
attestations: seq[Attestation],
attestations: seq[phase0.Attestation],
deposits: seq[Deposit],
exits: BeaconBlockValidatorChanges,
sync_aggregate: SyncAggregate,
@ -259,7 +259,7 @@ cli do(slots = SLOTS_PER_EPOCH * 7,
get_attestation_signature(
fork, genesis_validators_root, data,
MockPrivKeys[validator_index])
attestation = Attestation.init(
attestation = phase0.Attestation.init(
[uint64 index_in_committee], committee.len, data,
sig.toValidatorSig()).expect("valid data")

View File

@ -164,7 +164,7 @@ cli do(validatorsDir: string, secretsDir: string,
blockRoot = withState(state[]): forkyState.latest_block_root
cache: StateCache
info: ForkedEpochInfo
aggregates: seq[Attestation]
aggregates: seq[phase0.Attestation]
syncAggregate = SyncAggregate.init()
let
@ -341,7 +341,7 @@ cli do(validatorsDir: string, secretsDir: string,
forkyState.data, slot, committee_index, cache)
var
attestation = Attestation(
attestation = phase0.Attestation(
data: makeAttestationData(
forkyState.data, slot, committee_index, blockRoot),
aggregation_bits: CommitteeValidatorsBits.init(committee.len))

View File

@ -70,7 +70,7 @@ proc runTest[T, U](
suite baseDescription & "Attestation " & preset():
proc applyAttestation(
preState: var altair.BeaconState, attestation: Attestation):
preState: var altair.BeaconState, attestation: phase0.Attestation):
Result[void, cstring] =
var cache: StateCache
let
@ -85,7 +85,7 @@ suite baseDescription & "Attestation " & preset():
ok()
for path in walkTests(OpAttestationsDir):
runTest[Attestation, typeof applyAttestation](
runTest[phase0.Attestation, typeof applyAttestation](
OpAttestationsDir, suiteName, "Attestation", "attestation",
applyAttestation, path)

View File

@ -68,7 +68,7 @@ proc runTest[T, U](
suite baseDescription & "Attestation " & preset():
proc applyAttestation(
preState: var phase0.BeaconState, attestation: Attestation):
preState: var phase0.BeaconState, attestation: phase0.Attestation):
Result[void, cstring] =
var cache: StateCache
doAssert (? process_attestation(
@ -76,7 +76,7 @@ suite baseDescription & "Attestation " & preset():
ok()
for path in walkTests(OpAttestationsDir):
runTest[Attestation, typeof applyAttestation](
runTest[phase0.Attestation, typeof applyAttestation](
OpAttestationsDir, suiteName, "Attestation", "attestation",
applyAttestation, path)

View File

@ -55,7 +55,7 @@ type
of opOnTick:
tick: int
of opOnAttestation:
att: Attestation
att: phase0.Attestation
of opOnBlock:
blck: ForkedSignedBeaconBlock
blobData: Opt[BlobData]
@ -115,7 +115,7 @@ proc loadOps(
let filename = step["attestation"].getStr()
let att = parseTest(
path/filename & ".ssz_snappy",
SSZ, Attestation
SSZ, phase0.Attestation
)
result.add Operation(kind: opOnAttestation,
att: att)

View File

@ -28,7 +28,7 @@ import
from std/sequtils import toSeq
from ./testbcutil import addHeadBlock
func combine(tgt: var Attestation, src: Attestation) =
func combine(tgt: var phase0.Attestation, src: phase0.Attestation) =
## Combine the signature and participation bitfield, with the assumption that
## the same data is being signed - if the signatures overlap, they are not
## combined.
@ -47,7 +47,7 @@ func combine(tgt: var Attestation, src: Attestation) =
agg.aggregate(src.signature.load.get())
tgt.signature = agg.finish().toValidatorSig()
func loadSig(a: Attestation): CookedSig =
func loadSig(a: phase0.Attestation): CookedSig =
a.signature.load.get()
proc pruneAtFinalization(dag: ChainDAGRef, attPool: AttestationPool) =
@ -666,7 +666,7 @@ suite "Attestation pool processing" & preset():
# -------------------------------------------------------------
# Pass an epoch
var attestations: seq[Attestation]
var attestations: seq[phase0.Attestation]
for epoch in 0 ..< 5:
let start_slot = start_slot(Epoch epoch)
@ -703,7 +703,7 @@ suite "Attestation pool processing" & preset():
for v in 0 ..< committee.len * 2 div 3 + 1:
aggregation_bits[v] = true
attestations.add Attestation(
attestations.add phase0.Attestation(
aggregation_bits: aggregation_bits,
data: makeAttestationData(state[], getStateField(state[], slot),
committee_index, blockRef.get().root)

View File

@ -61,7 +61,8 @@ suite "Message signatures":
test "Aggregate and proof signatures":
let aggregate_and_proof = AggregateAndProof(
aggregate: Attestation(aggregation_bits: CommitteeValidatorsBits.init(8)))
aggregate: phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(8)))
check:
# Matching public/private keys and genesis validator roots

View File

@ -56,7 +56,7 @@ template bellatrix_steps() =
check: b.message.body.attester_slashings.add(default(AttesterSlashing))
do_check
check: b.message.body.attestations.add(
Attestation(aggregation_bits: CommitteeValidatorsBits.init(1)))
phase0.Attestation(aggregation_bits: CommitteeValidatorsBits.init(1)))
do_check
check: b.message.body.deposits.add(default(Deposit))
do_check

View File

@ -465,8 +465,8 @@ proc init(t: typedesc[ProduceAttestationDataResponse],
target: Checkpoint(epoch: Epoch(ad.target))
))
proc init(t: typedesc[Attestation], bits: string,
slot: Slot = GENESIS_SLOT, index: uint64 = 0'u64): Attestation =
proc init(t: typedesc[phase0.Attestation], bits: string,
slot: Slot = GENESIS_SLOT, index: uint64 = 0'u64): phase0.Attestation =
let
jdata = "{\"data\":\"" & bits & "\"}"
bits =
@ -767,8 +767,8 @@ suite "Validator Client test suite":
test "getUniqueVotes() test vectors":
for vector in AttestationBitsVectors:
let
a1 = Attestation.init(vector[0][0][0], vector[0][0][1], vector[0][0][2])
a2 = Attestation.init(vector[0][1][0], vector[0][1][1], vector[0][1][2])
a1 = phase0.Attestation.init(vector[0][0][0], vector[0][0][1], vector[0][0][2])
a2 = phase0.Attestation.init(vector[0][1][0], vector[0][1][1], vector[0][1][2])
check getUniqueVotes([a1, a2]) == vector[1]
asyncTest "firstSuccessParallel() API timeout test":

View File

@ -157,7 +157,7 @@ proc addTestBlock*(
state: var ForkedHashedBeaconState,
cache: var StateCache,
eth1_data: Eth1Data = Eth1Data(),
attestations: seq[Attestation] = newSeq[Attestation](),
attestations: seq[phase0.Attestation] = newSeq[phase0.Attestation](),
deposits: seq[Deposit] = newSeq[Deposit](),
sync_aggregate: SyncAggregate = SyncAggregate.init(),
graffiti: GraffitiBytes = default(GraffitiBytes),
@ -242,7 +242,7 @@ proc makeTestBlock*(
state: ForkedHashedBeaconState,
cache: var StateCache,
eth1_data = Eth1Data(),
attestations = newSeq[Attestation](),
attestations = newSeq[phase0.Attestation](),
deposits = newSeq[Deposit](),
sync_aggregate = SyncAggregate.init(),
graffiti = default(GraffitiBytes),
@ -323,7 +323,7 @@ func makeAttestation(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest,
committee: seq[ValidatorIndex], slot: Slot, committee_index: CommitteeIndex,
validator_index: ValidatorIndex, cache: var StateCache,
flags: UpdateFlags = {}): Attestation =
flags: UpdateFlags = {}): phase0.Attestation =
let
index_in_committee = committee.find(validator_index)
data = makeAttestationData(state, slot, committee_index, beacon_block_root)
@ -341,7 +341,7 @@ func makeAttestation(
getStateField(state, genesis_validators_root),
data, committee, aggregation_bits)
Attestation(
phase0.Attestation(
data: data,
aggregation_bits: aggregation_bits,
signature: sig
@ -364,7 +364,7 @@ func find_beacon_committee(
func makeAttestation*(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest,
validator_index: ValidatorIndex, cache: var StateCache): Attestation =
validator_index: ValidatorIndex, cache: var StateCache): phase0.Attestation =
let (committee, slot, index) =
find_beacon_committee(state, validator_index, cache)
makeAttestation(state, beacon_block_root, committee, slot, index,
@ -373,7 +373,7 @@ func makeAttestation*(
func makeFullAttestations*(
state: ForkedHashedBeaconState, beacon_block_root: Eth2Digest, slot: Slot,
cache: var StateCache,
flags: UpdateFlags = {}): seq[Attestation] =
flags: UpdateFlags = {}): seq[phase0.Attestation] =
# Create attestations in which the full committee participates for each shard
# that should be attested to during a particular slot
let committees_per_slot = get_committee_count_per_slot(
@ -384,7 +384,7 @@ func makeFullAttestations*(
data = makeAttestationData(state, slot, committee_index, beacon_block_root)
doAssert committee.len() >= 1
var attestation = Attestation(
var attestation = phase0.Attestation(
aggregation_bits: CommitteeValidatorsBits.init(committee.len),
data: data)
for i in 0..<committee.len: