mirror of
https://gitlab.com/pulsechaincom/prysm-pulse.git
synced 2024-12-22 03:30:35 +00:00
5a66807989
* First take at updating everything to v5 * Patch gRPC gateway to use prysm v5 Fix patch * Update go ssz --------- Co-authored-by: Preston Van Loon <pvanloon@offchainlabs.com>
459 lines
14 KiB
Go
459 lines
14 KiB
Go
package util
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"testing"
|
|
|
|
"github.com/pkg/errors"
|
|
"github.com/prysmaticlabs/go-bitfield"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
|
|
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
|
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stateutil"
|
|
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
|
"github.com/prysmaticlabs/prysm/v5/config/params"
|
|
consensusblocks "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
|
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
|
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
|
|
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
|
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
|
)
|
|
|
|
// DeterministicGenesisStateAltair returns a genesis state in hard fork 1 format made using the deterministic deposits.
|
|
func DeterministicGenesisStateAltair(t testing.TB, numValidators uint64) (state.BeaconState, []bls.SecretKey) {
|
|
deposits, privKeys, err := DeterministicDepositsAndKeys(numValidators)
|
|
if err != nil {
|
|
t.Fatal(errors.Wrapf(err, "failed to get %d deposits", numValidators))
|
|
}
|
|
eth1Data, err := DeterministicEth1Data(len(deposits))
|
|
if err != nil {
|
|
t.Fatal(errors.Wrapf(err, "failed to get eth1data for %d deposits", numValidators))
|
|
}
|
|
beaconState, err := GenesisBeaconState(context.Background(), deposits, uint64(0), eth1Data)
|
|
if err != nil {
|
|
t.Fatal(errors.Wrapf(err, "failed to get genesis beacon state of %d validators", numValidators))
|
|
}
|
|
resetCache()
|
|
return beaconState, privKeys
|
|
}
|
|
|
|
// GenesisBeaconState returns the genesis beacon state.
|
|
func GenesisBeaconState(ctx context.Context, deposits []*ethpb.Deposit, genesisTime uint64, eth1Data *ethpb.Eth1Data) (state.BeaconState, error) {
|
|
st, err := emptyGenesisState()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
// Process initial deposits.
|
|
st, err = helpers.UpdateGenesisEth1Data(st, deposits, eth1Data)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
st, err = processPreGenesisDeposits(ctx, st, deposits)
|
|
if err != nil {
|
|
return nil, errors.Wrap(err, "could not process validator deposits")
|
|
}
|
|
|
|
return buildGenesisBeaconState(genesisTime, st, st.Eth1Data())
|
|
}
|
|
|
|
// processPreGenesisDeposits processes a deposit for the beacon state Altair before chain start.
|
|
func processPreGenesisDeposits(
|
|
ctx context.Context,
|
|
beaconState state.BeaconState,
|
|
deposits []*ethpb.Deposit,
|
|
) (state.BeaconState, error) {
|
|
var err error
|
|
beaconState, err = altair.ProcessDeposits(ctx, beaconState, deposits)
|
|
if err != nil {
|
|
return nil, errors.Wrap(err, "could not process deposit")
|
|
}
|
|
beaconState, err = blocks.ActivateValidatorWithEffectiveBalance(beaconState, deposits)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
return beaconState, nil
|
|
}
|
|
|
|
func buildGenesisBeaconState(genesisTime uint64, preState state.BeaconState, eth1Data *ethpb.Eth1Data) (state.BeaconState, error) {
|
|
if eth1Data == nil {
|
|
return nil, errors.New("no eth1data provided for genesis state")
|
|
}
|
|
|
|
randaoMixes := make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector)
|
|
for i := 0; i < len(randaoMixes); i++ {
|
|
h := make([]byte, 32)
|
|
copy(h, eth1Data.BlockHash)
|
|
randaoMixes[i] = h
|
|
}
|
|
|
|
zeroHash := params.BeaconConfig().ZeroHash[:]
|
|
|
|
activeIndexRoots := make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector)
|
|
for i := 0; i < len(activeIndexRoots); i++ {
|
|
activeIndexRoots[i] = zeroHash
|
|
}
|
|
|
|
blockRoots := make([][]byte, params.BeaconConfig().SlotsPerHistoricalRoot)
|
|
for i := 0; i < len(blockRoots); i++ {
|
|
blockRoots[i] = zeroHash
|
|
}
|
|
|
|
stateRoots := make([][]byte, params.BeaconConfig().SlotsPerHistoricalRoot)
|
|
for i := 0; i < len(stateRoots); i++ {
|
|
stateRoots[i] = zeroHash
|
|
}
|
|
|
|
slashings := make([]uint64, params.BeaconConfig().EpochsPerSlashingsVector)
|
|
|
|
genesisValidatorsRoot, err := stateutil.ValidatorRegistryRoot(preState.Validators())
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "could not hash tree root genesis validators %v", err)
|
|
}
|
|
|
|
prevEpochParticipation, err := preState.PreviousEpochParticipation()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
currEpochParticipation, err := preState.CurrentEpochParticipation()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
scores, err := preState.InactivityScores()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
st := ðpb.BeaconStateAltair{
|
|
// Misc fields.
|
|
Slot: 0,
|
|
GenesisTime: genesisTime,
|
|
GenesisValidatorsRoot: genesisValidatorsRoot[:],
|
|
|
|
Fork: ðpb.Fork{
|
|
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
|
|
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
|
|
Epoch: 0,
|
|
},
|
|
|
|
// Validator registry fields.
|
|
Validators: preState.Validators(),
|
|
Balances: preState.Balances(),
|
|
PreviousEpochParticipation: prevEpochParticipation,
|
|
CurrentEpochParticipation: currEpochParticipation,
|
|
InactivityScores: scores,
|
|
|
|
// Randomness and committees.
|
|
RandaoMixes: randaoMixes,
|
|
|
|
// Finality.
|
|
PreviousJustifiedCheckpoint: ðpb.Checkpoint{
|
|
Epoch: 0,
|
|
Root: params.BeaconConfig().ZeroHash[:],
|
|
},
|
|
CurrentJustifiedCheckpoint: ðpb.Checkpoint{
|
|
Epoch: 0,
|
|
Root: params.BeaconConfig().ZeroHash[:],
|
|
},
|
|
JustificationBits: []byte{0},
|
|
FinalizedCheckpoint: ðpb.Checkpoint{
|
|
Epoch: 0,
|
|
Root: params.BeaconConfig().ZeroHash[:],
|
|
},
|
|
|
|
HistoricalRoots: [][]byte{},
|
|
BlockRoots: blockRoots,
|
|
StateRoots: stateRoots,
|
|
Slashings: slashings,
|
|
|
|
// Eth1 data.
|
|
Eth1Data: eth1Data,
|
|
Eth1DataVotes: []*ethpb.Eth1Data{},
|
|
Eth1DepositIndex: preState.Eth1DepositIndex(),
|
|
}
|
|
|
|
var scBits [fieldparams.SyncAggregateSyncCommitteeBytesLength]byte
|
|
bodyRoot, err := (ðpb.BeaconBlockBodyAltair{
|
|
RandaoReveal: make([]byte, 96),
|
|
Eth1Data: ðpb.Eth1Data{
|
|
DepositRoot: make([]byte, fieldparams.RootLength),
|
|
BlockHash: make([]byte, 32),
|
|
},
|
|
Graffiti: make([]byte, 32),
|
|
SyncAggregate: ðpb.SyncAggregate{
|
|
SyncCommitteeBits: make([]byte, len(scBits[:])),
|
|
SyncCommitteeSignature: make([]byte, 96),
|
|
},
|
|
}).HashTreeRoot()
|
|
if err != nil {
|
|
return nil, errors.Wrap(err, "could not hash tree root empty block body")
|
|
}
|
|
|
|
st.LatestBlockHeader = ðpb.BeaconBlockHeader{
|
|
ParentRoot: zeroHash,
|
|
StateRoot: zeroHash,
|
|
BodyRoot: bodyRoot[:],
|
|
}
|
|
|
|
var pubKeys [][]byte
|
|
for i := uint64(0); i < params.BeaconConfig().SyncCommitteeSize; i++ {
|
|
pubKeys = append(pubKeys, bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength))
|
|
}
|
|
st.CurrentSyncCommittee = ðpb.SyncCommittee{
|
|
Pubkeys: pubKeys,
|
|
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
|
|
}
|
|
st.NextSyncCommittee = ðpb.SyncCommittee{
|
|
Pubkeys: bytesutil.SafeCopy2dBytes(pubKeys),
|
|
AggregatePubkey: bytesutil.PadTo([]byte{}, params.BeaconConfig().BLSPubkeyLength),
|
|
}
|
|
|
|
return state_native.InitializeFromProtoAltair(st)
|
|
}
|
|
|
|
func emptyGenesisState() (state.BeaconState, error) {
|
|
st := ðpb.BeaconStateAltair{
|
|
// Misc fields.
|
|
Slot: 0,
|
|
Fork: ðpb.Fork{
|
|
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
|
|
CurrentVersion: params.BeaconConfig().AltairForkVersion,
|
|
Epoch: 0,
|
|
},
|
|
// Validator registry fields.
|
|
Validators: []*ethpb.Validator{},
|
|
Balances: []uint64{},
|
|
InactivityScores: []uint64{},
|
|
|
|
JustificationBits: []byte{0},
|
|
HistoricalRoots: [][]byte{},
|
|
CurrentEpochParticipation: []byte{},
|
|
PreviousEpochParticipation: []byte{},
|
|
|
|
// Eth1 data.
|
|
Eth1Data: ðpb.Eth1Data{},
|
|
Eth1DataVotes: []*ethpb.Eth1Data{},
|
|
Eth1DepositIndex: 0,
|
|
}
|
|
return state_native.InitializeFromProtoAltair(st)
|
|
}
|
|
|
|
// NewBeaconBlockAltair creates a beacon block with minimum marshalable fields.
|
|
func NewBeaconBlockAltair() *ethpb.SignedBeaconBlockAltair {
|
|
var scBits [fieldparams.SyncAggregateSyncCommitteeBytesLength]byte
|
|
return ðpb.SignedBeaconBlockAltair{
|
|
Block: ðpb.BeaconBlockAltair{
|
|
ParentRoot: make([]byte, fieldparams.RootLength),
|
|
StateRoot: make([]byte, fieldparams.RootLength),
|
|
Body: ðpb.BeaconBlockBodyAltair{
|
|
RandaoReveal: make([]byte, 96),
|
|
Eth1Data: ðpb.Eth1Data{
|
|
DepositRoot: make([]byte, fieldparams.RootLength),
|
|
BlockHash: make([]byte, 32),
|
|
},
|
|
Graffiti: make([]byte, 32),
|
|
Attestations: []*ethpb.Attestation{},
|
|
AttesterSlashings: []*ethpb.AttesterSlashing{},
|
|
Deposits: []*ethpb.Deposit{},
|
|
ProposerSlashings: []*ethpb.ProposerSlashing{},
|
|
VoluntaryExits: []*ethpb.SignedVoluntaryExit{},
|
|
SyncAggregate: ðpb.SyncAggregate{
|
|
SyncCommitteeBits: scBits[:],
|
|
SyncCommitteeSignature: make([]byte, 96),
|
|
},
|
|
},
|
|
},
|
|
Signature: make([]byte, 96),
|
|
}
|
|
}
|
|
|
|
// BlockSignatureAltair calculates the post-state root of the block and returns the signature.
|
|
func BlockSignatureAltair(
|
|
bState state.BeaconState,
|
|
block *ethpb.BeaconBlockAltair,
|
|
privKeys []bls.SecretKey,
|
|
) (bls.Signature, error) {
|
|
var err error
|
|
wsb, err := consensusblocks.NewSignedBeaconBlock(ðpb.SignedBeaconBlockAltair{Block: block})
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
s, err := transition.CalculateStateRoot(context.Background(), bState, wsb)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
block.StateRoot = s[:]
|
|
domain, err := signing.Domain(bState.Fork(), time.CurrentEpoch(bState), params.BeaconConfig().DomainBeaconProposer, bState.GenesisValidatorsRoot())
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
blockRoot, err := signing.ComputeSigningRoot(block, domain)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
// Temporarily increasing the beacon state slot here since BeaconProposerIndex is a
|
|
// function deterministic on beacon state slot.
|
|
currentSlot := bState.Slot()
|
|
if err := bState.SetSlot(block.Slot); err != nil {
|
|
return nil, err
|
|
}
|
|
proposerIdx, err := helpers.BeaconProposerIndex(context.Background(), bState)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
if err := bState.SetSlot(currentSlot); err != nil {
|
|
return nil, err
|
|
}
|
|
return privKeys[proposerIdx].Sign(blockRoot[:]), nil
|
|
}
|
|
|
|
// GenerateFullBlockAltair generates a fully valid Altair block with the requested parameters.
|
|
// Use BlockGenConfig to declare the conditions you would like the block generated under.
|
|
func GenerateFullBlockAltair(
|
|
bState state.BeaconState,
|
|
privs []bls.SecretKey,
|
|
conf *BlockGenConfig,
|
|
slot primitives.Slot,
|
|
) (*ethpb.SignedBeaconBlockAltair, error) {
|
|
ctx := context.Background()
|
|
currentSlot := bState.Slot()
|
|
if currentSlot > slot {
|
|
return nil, fmt.Errorf("current slot in state is larger than given slot. %d > %d", currentSlot, slot)
|
|
}
|
|
bState = bState.Copy()
|
|
|
|
if conf == nil {
|
|
conf = &BlockGenConfig{}
|
|
}
|
|
|
|
var err error
|
|
var pSlashings []*ethpb.ProposerSlashing
|
|
numToGen := conf.NumProposerSlashings
|
|
if numToGen > 0 {
|
|
pSlashings, err = generateProposerSlashings(bState, privs, numToGen)
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "failed generating %d proposer slashings:", numToGen)
|
|
}
|
|
}
|
|
|
|
numToGen = conf.NumAttesterSlashings
|
|
var aSlashings []*ethpb.AttesterSlashing
|
|
if numToGen > 0 {
|
|
aSlashings, err = generateAttesterSlashings(bState, privs, numToGen)
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "failed generating %d attester slashings:", numToGen)
|
|
}
|
|
}
|
|
|
|
numToGen = conf.NumAttestations
|
|
var atts []*ethpb.Attestation
|
|
if numToGen > 0 {
|
|
atts, err = GenerateAttestations(bState, privs, numToGen, slot, false)
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "failed generating %d attestations:", numToGen)
|
|
}
|
|
}
|
|
|
|
numToGen = conf.NumDeposits
|
|
var newDeposits []*ethpb.Deposit
|
|
eth1Data := bState.Eth1Data()
|
|
if numToGen > 0 {
|
|
newDeposits, eth1Data, err = generateDepositsAndEth1Data(bState, numToGen)
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "failed generating %d deposits:", numToGen)
|
|
}
|
|
}
|
|
|
|
numToGen = conf.NumVoluntaryExits
|
|
var exits []*ethpb.SignedVoluntaryExit
|
|
if numToGen > 0 {
|
|
exits, err = generateVoluntaryExits(bState, privs, numToGen)
|
|
if err != nil {
|
|
return nil, errors.Wrapf(err, "failed generating %d attester slashings:", numToGen)
|
|
}
|
|
}
|
|
|
|
newHeader := bState.LatestBlockHeader()
|
|
prevStateRoot, err := bState.HashTreeRoot(ctx)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
newHeader.StateRoot = prevStateRoot[:]
|
|
parentRoot, err := newHeader.HashTreeRoot()
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
var newSyncAggregate *ethpb.SyncAggregate
|
|
if conf.FullSyncAggregate {
|
|
newSyncAggregate, err = generateSyncAggregate(bState, privs, parentRoot)
|
|
if err != nil {
|
|
return nil, errors.Wrap(err, "failed generating syncAggregate")
|
|
}
|
|
} else {
|
|
var syncCommitteeBits []byte
|
|
currSize := new(ethpb.SyncAggregate).SyncCommitteeBits.Len()
|
|
switch currSize {
|
|
case 512:
|
|
syncCommitteeBits = bitfield.NewBitvector512()
|
|
case 32:
|
|
syncCommitteeBits = bitfield.NewBitvector32()
|
|
default:
|
|
return nil, errors.New("invalid bit vector size")
|
|
}
|
|
newSyncAggregate = ðpb.SyncAggregate{
|
|
SyncCommitteeBits: syncCommitteeBits,
|
|
SyncCommitteeSignature: append([]byte{0xC0}, make([]byte, 95)...),
|
|
}
|
|
}
|
|
|
|
if slot == currentSlot {
|
|
slot = currentSlot + 1
|
|
}
|
|
|
|
stCopy := bState.Copy()
|
|
stCopy, err = transition.ProcessSlots(context.Background(), stCopy, slot)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
reveal, err := RandaoReveal(stCopy, time.CurrentEpoch(stCopy), privs)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
idx, err := helpers.BeaconProposerIndex(ctx, stCopy)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
block := ðpb.BeaconBlockAltair{
|
|
Slot: slot,
|
|
ParentRoot: parentRoot[:],
|
|
ProposerIndex: idx,
|
|
Body: ðpb.BeaconBlockBodyAltair{
|
|
Eth1Data: eth1Data,
|
|
RandaoReveal: reveal,
|
|
ProposerSlashings: pSlashings,
|
|
AttesterSlashings: aSlashings,
|
|
Attestations: atts,
|
|
VoluntaryExits: exits,
|
|
Deposits: newDeposits,
|
|
Graffiti: make([]byte, fieldparams.RootLength),
|
|
SyncAggregate: newSyncAggregate,
|
|
},
|
|
}
|
|
|
|
signature, err := BlockSignature(bState, block, privs)
|
|
if err != nil {
|
|
return nil, err
|
|
}
|
|
|
|
return ðpb.SignedBeaconBlockAltair{Block: block, Signature: signature.Marshal()}, nil
|
|
}
|