Start chain from bellatrix state (#11746)

* WIP trying to start from bellatrix state

* env var to control log path with unique paths

due to flaky test re-run behavior, logs from a failed test run are
overwritten by subsequent retries. This makes it difficult to retrieve
logs after the first failed run. It also takes some squinting through
output to find the location of the log file in the first place. This
flag enables logs to be placed in an arbitrary path. Note that bazel
sandboxing generally will force this path to be in the /tmp tree.

* WIP - grabbing changes from rm-pre-genesis branch

* combine bellatrix state w/ rm-pre-genesis branch

* WIP

* use encoding/detect for genesis state bytes

* WIP more fixes towards start from bellatrix

* remove debug wrapping

* WIP

* multiple bugfixes

* fix fork ordering bug and bellatrix genesis blocks

* send deposits, spam tx to advance, fix miner alloc

* WIP

* WIP mess

* WIP

* Print process ID information for purposes of attaching a debugger

* bugs: genesis body_root and deposit index mismatch

* fix voting period start, skip altair check

* add changes

* make it better

* rm startup FCU, rm logs

* cleanup import grouping&ordering

* restore FCU log, get rid of tmp var

* rm newline

* restore newline

* restore wrapped error

* rm newline

* removing boot node version override

this doesn't seem to matter?

* add issue number to todo comment

* rm commented code

* rm vmdebug geth flag

* unexport values only used with genesis test pkg

and add comments where missing from exported values.

* adding comments to special cases for testnets

* migrate comments from PR to actual code :)

* rm unused test param

* mark e2e spawns exempt from gosec warning

* Fix DeepSource errors in `proposer_bellatrix.go` (#11739)

* Fix DeepSource errors in

* Omit receiver name

* Address PR comments

* Remove unused variable

* Fix more DeepSource errors

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Remove `Test_IsExecutionEnabledCapella` (#11752)

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* Add REST implementation for Validator's `ProposeBeaconBlock` (#11731)

* WIP

* WIP

* WIP

* Add tests

* WIP

* Add more tests

* Address DeepSource errors

* Remove unused param

* Add more tests

* Address PR comments

* Address PR comments

* Fix formatting

* Remove unused parameter

* Fix TestLittleEndianBytesToBigInt

Co-authored-by: Radosław Kapka <rkapka@wp.pl>

* fix validator client (#11755)

* fix validator client

(cherry picked from commit deb138959a2ffcb89cd2e3eb8304477526f4a168)

* Use signed changes in middleware block

Co-authored-by: Potuz <potuz@prysmaticlabs.com>

* API `finalized` metadata field - update protos (#11749)

* API `finalized` metadata field - update protos

* change nums

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>

* log breaks unit tests that don't do full arg setup

easiest to just remove it for now

* restore prior behavior of phase0 block for altair

* update unit tests to account for special case

* loosen condition for fork version to match config

we don't know which fork version genesis will start from, so we
shouldn't force it to be a phase0 genesis.

* skip until we can mod configs at runtime

* NewGenesisBlockForState computes state root itself

* rm noisy log

* this log would be noisy in mainnet

* fix format specifier, []byte -> string

* core.Genesis UnmarshalJson has a value receiver :)

* no longer needs to be exported

Co-authored-by: Kasey Kirkham <kasey@users.noreply.github.com>
Co-authored-by: prestonvanloon <preston@prysmaticlabs.com>
Co-authored-by: nisdas <nishdas93@gmail.com>
Co-authored-by: Patrice Vignola <vignola.patrice@gmail.com>
Co-authored-by: Radosław Kapka <rkapka@wp.pl>
Co-authored-by: terencechain <terence@prysmaticlabs.com>
Co-authored-by: Potuz <potuz@prysmaticlabs.com>
Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
This commit is contained in:
kasey 2022-12-13 17:13:49 -06:00 committed by GitHub
parent a01d08b857
commit 2e49fdb3d2
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
40 changed files with 1013 additions and 179 deletions

View File

@ -87,6 +87,7 @@ func logBlockSyncStatus(block interfaces.BeaconBlock, blockRoot [32]byte, justif
"version": version.String(block.Version()),
"sinceSlotStartTime": prysmTime.Now().Sub(startTime),
"chainServiceProcessedTime": prysmTime.Now().Sub(receivedTime),
"deposits": len(block.Body().Deposits()),
}).Debug("Synced new block")
} else {
log.WithFields(logrus.Fields{

View File

@ -19,12 +19,7 @@ go_library(
"withdrawals.go",
],
importpath = "github.com/prysmaticlabs/prysm/v3/beacon-chain/core/blocks",
visibility = [
"//beacon-chain:__subpackages__",
"//testing/spectest:__subpackages__",
"//testing/util:__pkg__",
"//validator:__subpackages__",
],
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",

View File

@ -3,9 +3,16 @@
package blocks
import (
"context"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/v3/config/fieldparams"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
@ -29,3 +36,64 @@ func NewGenesisBlock(stateRoot []byte) *ethpb.SignedBeaconBlock {
}
return block
}
var ErrUnrecognizedState = errors.New("uknonwn underlying type for state.BeaconState value")
func NewGenesisBlockForState(ctx context.Context, st state.BeaconState) (interfaces.SignedBeaconBlock, error) {
root, err := st.HashTreeRoot(ctx)
if err != nil {
return nil, err
}
ps := st.ToProto()
switch ps.(type) {
case *ethpb.BeaconState, *ethpb.BeaconStateAltair:
return blocks.NewSignedBeaconBlock(&ethpb.SignedBeaconBlock{
Block: &ethpb.BeaconBlock{
ParentRoot: params.BeaconConfig().ZeroHash[:],
StateRoot: root[:],
Body: &ethpb.BeaconBlockBody{
RandaoReveal: make([]byte, fieldparams.BLSSignatureLength),
Eth1Data: &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
},
Graffiti: make([]byte, 32),
},
},
Signature: params.BeaconConfig().EmptySignature[:],
})
case *ethpb.BeaconStateBellatrix:
return blocks.NewSignedBeaconBlock(&ethpb.SignedBeaconBlockBellatrix{
Block: &ethpb.BeaconBlockBellatrix{
ParentRoot: params.BeaconConfig().ZeroHash[:],
StateRoot: root[:],
Body: &ethpb.BeaconBlockBodyBellatrix{
RandaoReveal: make([]byte, 96),
Eth1Data: &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
},
Graffiti: make([]byte, 32),
SyncAggregate: &ethpb.SyncAggregate{
SyncCommitteeBits: make([]byte, fieldparams.SyncCommitteeLength/8),
SyncCommitteeSignature: make([]byte, fieldparams.BLSSignatureLength),
},
ExecutionPayload: &enginev1.ExecutionPayload{
ParentHash: make([]byte, 32),
FeeRecipient: make([]byte, 20),
StateRoot: make([]byte, 32),
ReceiptsRoot: make([]byte, 32),
LogsBloom: make([]byte, 256),
PrevRandao: make([]byte, 32),
BaseFeePerGas: make([]byte, 32),
BlockHash: make([]byte, 32),
Transactions: make([][]byte, 0),
},
},
},
Signature: params.BeaconConfig().EmptySignature[:],
})
default:
return nil, ErrUnrecognizedState
}
}

View File

@ -6,6 +6,7 @@ go_library(
"log.go",
"skip_slot_cache.go",
"state.go",
"state-bellatrix.go",
"trailing_slot_state_cache.go",
"transition.go",
"transition_no_verify_sig.go",
@ -36,6 +37,7 @@ go_library(
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//beacon-chain/state/stateutil:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
@ -45,6 +47,7 @@ go_library(
"//encoding/bytesutil:go_default_library",
"//math:go_default_library",
"//monitoring/tracing:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"@com_github_pkg_errors//:go_default_library",

View File

@ -0,0 +1,279 @@
package transition
import (
"context"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/altair"
b "github.com/prysmaticlabs/prysm/v3/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state"
state_native "github.com/prysmaticlabs/prysm/v3/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state/stateutil"
fieldparams "github.com/prysmaticlabs/prysm/v3/config/fieldparams"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
// GenesisBeaconStateBellatrix gets called when MinGenesisActiveValidatorCount count of
// full deposits were made to the deposit contract and the ChainStart log gets emitted.
//
// Spec pseudocode definition:
//
// def initialize_beacon_state_from_eth1(eth1_block_hash: Bytes32,
// eth1_timestamp: uint64,
// deposits: Sequence[Deposit]) -> BeaconState:
// fork = Fork(
// previous_version=GENESIS_FORK_VERSION,
// current_version=GENESIS_FORK_VERSION,
// epoch=GENESIS_EPOCH,
// )
// state = BeaconState(
// genesis_time=eth1_timestamp + GENESIS_DELAY,
// fork=fork,
// eth1_data=Eth1Data(block_hash=eth1_block_hash, deposit_count=uint64(len(deposits))),
// latest_block_header=BeaconBlockHeader(body_root=hash_tree_root(BeaconBlockBody())),
// randao_mixes=[eth1_block_hash] * EPOCHS_PER_HISTORICAL_VECTOR, # Seed RANDAO with Eth1 entropy
// )
//
// # Process deposits
// leaves = list(map(lambda deposit: deposit.data, deposits))
// for index, deposit in enumerate(deposits):
// deposit_data_list = List[DepositData, 2**DEPOSIT_CONTRACT_TREE_DEPTH](*leaves[:index + 1])
// state.eth1_data.deposit_root = hash_tree_root(deposit_data_list)
// process_deposit(state, deposit)
//
// # Process activations
// for index, validator in enumerate(state.validators):
// balance = state.balances[index]
// validator.effective_balance = min(balance - balance % EFFECTIVE_BALANCE_INCREMENT, MAX_EFFECTIVE_BALANCE)
// if validator.effective_balance == MAX_EFFECTIVE_BALANCE:
// validator.activation_eligibility_epoch = GENESIS_EPOCH
// validator.activation_epoch = GENESIS_EPOCH
//
// # Set genesis validators root for domain separation and chain versioning
// state.genesis_validators_root = hash_tree_root(state.validators)
//
// return state
//
// This method differs from the spec so as to process deposits beforehand instead of the end of the function.
func GenesisBeaconStateBellatrix(ctx context.Context, deposits []*ethpb.Deposit, genesisTime uint64, eth1Data *ethpb.Eth1Data, ep *enginev1.ExecutionPayload) (state.BeaconState, error) {
st, err := EmptyGenesisStateBellatrix()
if err != nil {
return nil, err
}
// Process initial deposits.
st, err = helpers.UpdateGenesisEth1Data(st, deposits, eth1Data)
if err != nil {
return nil, err
}
st, err = b.ProcessPreGenesisDeposits(ctx, st, deposits)
if err != nil {
return nil, errors.Wrap(err, "could not process validator deposits")
}
// After deposits have been processed, overwrite eth1data to what is passed in. This allows us to "pre-mine" validators
// without the deposit root and count mismatching the real deposit contract.
if err := st.SetEth1Data(eth1Data); err != nil {
return nil, err
}
if err := st.SetEth1DepositIndex(eth1Data.DepositCount); err != nil {
return nil, err
}
return OptimizedGenesisBeaconStateBellatrix(genesisTime, st, st.Eth1Data(), ep)
}
// OptimizedGenesisBeaconState is used to create a state that has already processed deposits. This is to efficiently
// create a mainnet state at chainstart.
func OptimizedGenesisBeaconStateBellatrix(genesisTime uint64, preState state.BeaconState, eth1Data *ethpb.Eth1Data, ep *enginev1.ExecutionPayload) (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)
}
scores, err := preState.InactivityScores()
if err != nil {
return nil, err
}
scoresMissing := len(preState.Validators()) - len(scores)
if scoresMissing > 0 {
for i := 0; i < scoresMissing; i++ {
scores = append(scores, 0)
}
}
wep, err := blocks.WrappedExecutionPayload(ep)
if err != nil {
return nil, err
}
eph, err := blocks.PayloadToHeader(wep)
if err != nil {
return nil, err
}
st := &ethpb.BeaconStateBellatrix{
// Misc fields.
Slot: 0,
GenesisTime: genesisTime,
GenesisValidatorsRoot: genesisValidatorsRoot[:],
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().AltairForkVersion,
CurrentVersion: params.BeaconConfig().BellatrixForkVersion,
Epoch: 0,
},
// Validator registry fields.
Validators: preState.Validators(),
Balances: preState.Balances(),
// Randomness and committees.
RandaoMixes: randaoMixes,
// Finality.
PreviousJustifiedCheckpoint: &ethpb.Checkpoint{
Epoch: 0,
Root: params.BeaconConfig().ZeroHash[:],
},
CurrentJustifiedCheckpoint: &ethpb.Checkpoint{
Epoch: 0,
Root: params.BeaconConfig().ZeroHash[:],
},
JustificationBits: []byte{0},
FinalizedCheckpoint: &ethpb.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(),
LatestExecutionPayloadHeader: eph,
InactivityScores: scores,
}
bodyRoot, err := (&ethpb.BeaconBlockBodyBellatrix{
RandaoReveal: make([]byte, 96),
Eth1Data: &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
},
Graffiti: make([]byte, 32),
SyncAggregate: &ethpb.SyncAggregate{
SyncCommitteeBits: make([]byte, fieldparams.SyncCommitteeLength/8),
SyncCommitteeSignature: make([]byte, fieldparams.BLSSignatureLength),
},
ExecutionPayload: &enginev1.ExecutionPayload{
ParentHash: make([]byte, 32),
FeeRecipient: make([]byte, 20),
StateRoot: make([]byte, 32),
ReceiptsRoot: make([]byte, 32),
LogsBloom: make([]byte, 256),
PrevRandao: make([]byte, 32),
BaseFeePerGas: make([]byte, 32),
BlockHash: make([]byte, 32),
Transactions: make([][]byte, 0),
},
}).HashTreeRoot()
if err != nil {
return nil, errors.Wrap(err, "could not hash tree root empty block body")
}
st.LatestBlockHeader = &ethpb.BeaconBlockHeader{
ParentRoot: zeroHash,
StateRoot: zeroHash,
BodyRoot: bodyRoot[:],
}
ist, err := state_native.InitializeFromProtoBellatrix(st)
if err != nil {
return nil, err
}
sc, err := altair.NextSyncCommittee(context.Background(), ist)
if err != nil {
return nil, err
}
if err := ist.SetNextSyncCommittee(sc); err != nil {
return nil, err
}
if err := ist.SetCurrentSyncCommittee(sc); err != nil {
return nil, err
}
return ist, nil
}
// EmptyGenesisState returns an empty beacon state object.
func EmptyGenesisStateBellatrix() (state.BeaconState, error) {
st := &ethpb.BeaconStateBellatrix{
// Misc fields.
Slot: 0,
Fork: &ethpb.Fork{
PreviousVersion: params.BeaconConfig().AltairForkVersion,
CurrentVersion: params.BeaconConfig().BellatrixForkVersion,
Epoch: 0,
},
// Validator registry fields.
Validators: []*ethpb.Validator{},
Balances: []uint64{},
JustificationBits: []byte{0},
HistoricalRoots: [][]byte{},
// Eth1 data.
Eth1Data: &ethpb.Eth1Data{},
Eth1DataVotes: []*ethpb.Eth1Data{},
Eth1DepositIndex: 0,
LatestExecutionPayloadHeader: &enginev1.ExecutionPayloadHeader{
ParentHash: make([]byte, 32),
FeeRecipient: make([]byte, 20),
StateRoot: make([]byte, 32),
ReceiptsRoot: make([]byte, 32),
LogsBloom: make([]byte, 256),
PrevRandao: make([]byte, 32),
BaseFeePerGas: make([]byte, 32),
BlockHash: make([]byte, 32),
TransactionsRoot: make([]byte, 32),
},
}
return state_native.InitializeFromProtoBellatrix(st)
}

View File

@ -1,34 +1,25 @@
package kv
import (
"bytes"
"context"
"fmt"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/blocks"
dbIface "github.com/prysmaticlabs/prysm/v3/beacon-chain/db/iface"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state"
state_native "github.com/prysmaticlabs/prysm/v3/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v3/config/params"
consensusblocks "github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v3/encoding/ssz/detect"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
// SaveGenesisData bootstraps the beaconDB with a given genesis state.
func (s *Store) SaveGenesisData(ctx context.Context, genesisState state.BeaconState) error {
stateRoot, err := genesisState.HashTreeRoot(ctx)
if err != nil {
return err
}
genesisBlk := blocks.NewGenesisBlock(stateRoot[:])
genesisBlkRoot, err := genesisBlk.Block.HashTreeRoot()
wsb, err := blocks.NewGenesisBlockForState(ctx, genesisState)
if err != nil {
return errors.Wrap(err, "could not get genesis block root")
}
wsb, err := consensusblocks.NewSignedBeaconBlock(genesisBlk)
genesisBlkRoot, err := wsb.Block().HashTreeRoot()
if err != nil {
return errors.Wrap(err, "could not wrap genesis block")
return errors.Wrap(err, "could not get genesis block root")
}
if err := s.SaveBlock(ctx, wsb); err != nil {
return errors.Wrap(err, "could not save genesis block")
@ -54,11 +45,11 @@ func (s *Store) SaveGenesisData(ctx context.Context, genesisState state.BeaconSt
// LoadGenesis loads a genesis state from a ssz-serialized byte slice, if no genesis exists already.
func (s *Store) LoadGenesis(ctx context.Context, sb []byte) error {
st := &ethpb.BeaconState{}
if err := st.UnmarshalSSZ(sb); err != nil {
vu, err := detect.FromState(sb)
if err != nil {
return err
}
gs, err := state_native.InitializeFromProtoUnsafePhase0(st)
gs, err := vu.UnmarshalBeaconState(sb)
if err != nil {
return err
}
@ -83,10 +74,6 @@ func (s *Store) LoadGenesis(ctx context.Context, sb []byte) error {
return dbIface.ErrExistingGenesisState
}
if !bytes.Equal(gs.Fork().CurrentVersion, params.BeaconConfig().GenesisForkVersion) {
return fmt.Errorf("loaded genesis fork version (%#x) does not match config genesis "+
"fork version (%#x)", gs.Fork().CurrentVersion, params.BeaconConfig().GenesisForkVersion)
}
return s.SaveGenesisData(ctx, gs)
}

View File

@ -106,7 +106,7 @@ func TestLoadGenesisFromFile_mismatchedForkVersion(t *testing.T) {
// Loading a genesis with the wrong fork version as beacon config should throw an error.
db := setupDB(t)
assert.ErrorContains(t, "does not match config genesis fork version", db.LoadGenesis(context.Background(), sb))
assert.ErrorContains(t, "not found in any known fork choice schedule", db.LoadGenesis(context.Background(), sb))
}
func TestEnsureEmbeddedGenesis(t *testing.T) {

View File

@ -88,13 +88,13 @@ func (s *Service) pollConnectionStatus(ctx context.Context) {
// Forces to retry an execution client connection.
func (s *Service) retryExecutionClientConnection(ctx context.Context, err error) {
s.runError = err
s.runError = errors.Wrap(err, "retryExecutionClientConnection")
s.updateConnectedETH1(false)
// Back off for a while before redialing.
time.Sleep(backOffPeriod)
currClient := s.rpcClient
if err := s.setupExecutionClientConnections(ctx, s.cfg.currHttpEndpoint); err != nil {
s.runError = err
s.runError = errors.Wrap(err, "setupExecutionClientConnections")
return
}
// Close previous client, if connection was successful.

View File

@ -319,10 +319,15 @@ func (s *Service) followedBlockHeight(ctx context.Context) (uint64, error) {
latestBlockTime := uint64(0)
if s.latestEth1Data.BlockTime > followTime {
latestBlockTime = s.latestEth1Data.BlockTime - followTime
// This should only come into play in testnets - when the chain hasn't advanced past the follow distance,
// we don't want to consider any block before the genesis block.
if s.latestEth1Data.BlockHeight < params.BeaconConfig().Eth1FollowDistance {
latestBlockTime = s.latestEth1Data.BlockTime
}
}
blk, err := s.BlockByTimestamp(ctx, latestBlockTime)
if err != nil {
return 0, err
return 0, errors.Wrapf(err, "BlockByTimestamp=%d", latestBlockTime)
}
return blk.Number.Uint64(), nil
}
@ -467,11 +472,12 @@ func (s *Service) handleETH1FollowDistance() {
}
if !s.chainStartData.Chainstarted {
if err := s.processChainStartFromBlockNum(ctx, big.NewInt(int64(s.latestEth1Data.LastRequestedBlock))); err != nil {
s.runError = err
s.runError = errors.Wrap(err, "processChainStartFromBlockNum")
log.Error(err)
return
}
}
// If the last requested block has not changed,
// we do not request batched logs as this means there are no new
// logs for the powchain service to process. Also it is a potential
@ -481,7 +487,7 @@ func (s *Service) handleETH1FollowDistance() {
return
}
if err := s.requestBatchedHeadersAndLogs(ctx); err != nil {
s.runError = err
s.runError = errors.Wrap(err, "requestBatchedHeadersAndLogs")
log.Error(err)
return
}
@ -511,6 +517,7 @@ func (s *Service) initPOWService() {
ctx := s.ctx
header, err := s.HeaderByNumber(ctx, nil)
if err != nil {
err = errors.Wrap(err, "HeaderByNumber")
s.retryExecutionClientConnection(ctx, err)
errorLogger(err, "Unable to retrieve latest execution client header")
continue
@ -523,6 +530,7 @@ func (s *Service) initPOWService() {
s.latestEth1DataLock.Unlock()
if err := s.processPastLogs(ctx); err != nil {
err = errors.Wrap(err, "processPastLogs")
s.retryExecutionClientConnection(ctx, err)
errorLogger(
err,
@ -532,6 +540,7 @@ func (s *Service) initPOWService() {
}
// Cache eth1 headers from our voting period.
if err := s.cacheHeadersForEth1DataVote(ctx); err != nil {
err = errors.Wrap(err, "cacheHeadersForEth1DataVote")
s.retryExecutionClientConnection(ctx, err)
if errors.Is(err, errBlockTimeTooLate) {
log.WithError(err).Debug("Unable to cache headers for execution client votes")
@ -550,6 +559,7 @@ func (s *Service) initPOWService() {
if genHash != [32]byte{} {
genHeader, err := s.HeaderByHash(ctx, genHash)
if err != nil {
err = errors.Wrapf(err, "HeaderByHash, hash=%#x", genHash)
s.retryExecutionClientConnection(ctx, err)
errorLogger(err, "Unable to retrieve proof-of-stake genesis block data")
continue
@ -558,6 +568,7 @@ func (s *Service) initPOWService() {
}
s.chainStartData.GenesisBlock = genBlock
if err := s.savePowchainData(ctx); err != nil {
err = errors.Wrap(err, "savePowchainData")
s.retryExecutionClientConnection(ctx, err)
errorLogger(err, "Unable to save execution client data")
continue
@ -641,11 +652,11 @@ func (s *Service) cacheHeadersForEth1DataVote(ctx context.Context) error {
// Find the end block to request from.
end, err := s.followedBlockHeight(ctx)
if err != nil {
return err
return errors.Wrap(err, "followedBlockHeight")
}
start, err := s.determineEarliestVotingBlock(ctx, end)
if err != nil {
return err
return errors.Wrapf(err, "determineEarliestVotingBlock=%d", end)
}
return s.cacheBlockHeaders(start, end)
}
@ -677,7 +688,7 @@ func (s *Service) cacheBlockHeaders(start, end uint64) error {
}
continue
}
return err
return errors.Wrapf(err, "cacheBlockHeaders, start=%d, end=%d", startReq, endReq)
}
}
return nil
@ -696,6 +707,11 @@ func (s *Service) determineEarliestVotingBlock(ctx context.Context, followBlock
}
return earliestBlk, nil
}
// This should only come into play in testnets - when the chain hasn't advanced past the follow distance,
// we don't want to consider any block before the genesis block.
if s.latestEth1Data.BlockHeight < params.BeaconConfig().Eth1FollowDistance {
return 0, nil
}
votingTime := slots.VotingPeriodStartTime(genesisTime, currSlot)
followBackDist := 2 * params.BeaconConfig().SecondsPerETH1Block * params.BeaconConfig().Eth1FollowDistance
if followBackDist > votingTime {

View File

@ -4,13 +4,14 @@ go_library(
name = "go_default_library",
testonly = True,
srcs = [
"genesis.go",
"mock_engine_client.go",
"mock_execution_chain.go",
"mock_faulty_powchain.go",
],
importpath = "github.com/prysmaticlabs/prysm/v3/beacon-chain/execution/testing",
visibility = [
"//beacon-chain:__subpackages__",
"//visibility:public",
],
deps = [
"//async/event:go_default_library",
@ -27,7 +28,9 @@ go_library(
"@com_github_ethereum_go_ethereum//accounts/abi/bind/backends:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_ethereum_go_ethereum//core:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_ethereum_go_ethereum//params:go_default_library",
"@com_github_ethereum_go_ethereum//rpc:go_default_library",
"@com_github_holiman_uint256//:go_default_library",
"@com_github_pkg_errors//:go_default_library",

File diff suppressed because one or more lines are too long

View File

@ -45,9 +45,18 @@ func (vs *Server) eth1DataMajorityVote(ctx context.Context, beaconState state.Be
}
eth1DataNotification = false
eth1FollowDistance := params.BeaconConfig().Eth1FollowDistance
earliestValidTime := votingPeriodStartTime - 2*params.BeaconConfig().SecondsPerETH1Block*eth1FollowDistance
latestValidTime := votingPeriodStartTime - params.BeaconConfig().SecondsPerETH1Block*eth1FollowDistance
genesisTime, _ := vs.Eth1InfoFetcher.GenesisExecutionChainInfo()
followDistanceSeconds := params.BeaconConfig().Eth1FollowDistance * params.BeaconConfig().SecondsPerETH1Block
latestValidTime := votingPeriodStartTime - followDistanceSeconds
earliestValidTime := votingPeriodStartTime - 2*followDistanceSeconds
// Special case for starting from a pre-mined genesis: the eth1 vote should be genesis until the chain has advanced
// by ETH1_FOLLOW_DISTANCE. The head state should maintain the same ETH1Data until this condition has passed, so
// trust the existing head for the right eth1 vote until we can get a meaningful value from the deposit contract.
if latestValidTime < genesisTime+followDistanceSeconds {
log.WithField("genesisTime", genesisTime).WithField("latestValidTime", latestValidTime).Warn("voting period before genesis + follow distance, using eth1data from head")
return vs.HeadFetcher.HeadETH1Data(), nil
}
lastBlockByLatestValidTime, err := vs.Eth1BlockFetcher.BlockByTimestamp(ctx, latestValidTime)
if err != nil {

View File

@ -1103,8 +1103,46 @@ func TestProposer_ValidateDepositTrie(t *testing.T) {
}
}
func TestProposer_Eth1Data_MajorityVote_SpansGenesis(t *testing.T) {
ctx := context.Background()
// Voting period will span genesis, causing the special case for pre-mined genesis to kick in.
// In other words some part of the valid time range is before genesis, so querying the block cache would fail
// without the special case added to allow this for testnets.
slot := types.Slot(0)
earliestValidTime, latestValidTime := majorityVoteBoundaryTime(slot)
p := mockExecution.New().
InsertBlock(50, earliestValidTime, []byte("earliest")).
InsertBlock(100, latestValidTime, []byte("latest"))
headBlockHash := []byte("headb")
depositCache, err := depositcache.New()
require.NoError(t, err)
ps := &Server{
ChainStartFetcher: p,
Eth1InfoFetcher: p,
Eth1BlockFetcher: p,
BlockFetcher: p,
DepositFetcher: depositCache,
HeadFetcher: &mock.ChainService{ETH1Data: &ethpb.Eth1Data{BlockHash: headBlockHash, DepositCount: 0}},
}
beaconState, err := state_native.InitializeFromProtoPhase0(&ethpb.BeaconState{
Slot: slot,
Eth1DataVotes: []*ethpb.Eth1Data{
{BlockHash: []byte("earliest"), DepositCount: 1},
},
})
require.NoError(t, err)
majorityVoteEth1Data, err := ps.eth1DataMajorityVote(ctx, beaconState)
require.NoError(t, err)
assert.DeepEqual(t, headBlockHash, majorityVoteEth1Data.BlockHash)
}
func TestProposer_Eth1Data_MajorityVote(t *testing.T) {
slot := types.Slot(64)
followDistanceSecs := params.BeaconConfig().Eth1FollowDistance * params.BeaconConfig().SecondsPerETH1Block
followSlots := followDistanceSecs / params.BeaconConfig().SecondsPerSlot
slot := types.Slot(64 + followSlots)
earliestValidTime, latestValidTime := majorityVoteBoundaryTime(slot)
dc := ethpb.DepositContainer{

View File

@ -23,6 +23,8 @@ import (
var placeholderFields = []string{"UPDATE_TIMEOUT", "EIP4844_FORK_EPOCH", "EIP4844_FORK_VERSION"}
func TestLoadConfigFile(t *testing.T) {
// TODO(11750)
t.Skip("needs https://github.com/prysmaticlabs/prysm/issues/11750")
// See https://media.githubusercontent.com/media/ethereum/consensus-spec-tests/master/tests/minimal/config/phase0.yaml
assertVals := func(name string, fields []string, expected, actual *params.BeaconChainConfig) {
// Misc params.

View File

@ -20,6 +20,8 @@ func testnetConfigFilePath(t *testing.T, network string) string {
}
func TestE2EConfigParity(t *testing.T) {
// TODO(11750)
t.Skip("needs https://github.com/prysmaticlabs/prysm/issues/11750")
params.SetupTestConfigCleanup(t)
testDir := bazel.TestTmpDir()
yamlDir := filepath.Join(testDir, "config.yaml")

View File

@ -1,8 +1,8 @@
package params
const (
altairE2EForkEpoch = 6
bellatrixE2EForkEpoch = 8
altairE2EForkEpoch = 0
bellatrixE2EForkEpoch = 0
)
// E2ETestConfig retrieves the configurations made specifically for E2E testing.
@ -10,6 +10,8 @@ const (
// WARNING: This config is only for testing, it is not meant for use outside of E2E.
func E2ETestConfig() *BeaconChainConfig {
e2eConfig := MinimalSpecConfig()
e2eConfig.DepositContractAddress = "0x4242424242424242424242424242424242424242"
e2eConfig.Eth1FollowDistance = 8
// Misc.
e2eConfig.MinGenesisActiveValidatorCount = 256
@ -21,7 +23,6 @@ func E2ETestConfig() *BeaconChainConfig {
e2eConfig.SlotsPerEpoch = 6
e2eConfig.SqrRootSlotsPerEpoch = 2
e2eConfig.SecondsPerETH1Block = 2
e2eConfig.Eth1FollowDistance = 8
e2eConfig.EpochsPerEth1VotingPeriod = 2
e2eConfig.ShardCommitteePeriod = 4
e2eConfig.MaxSeedLookahead = 1
@ -35,7 +36,7 @@ func E2ETestConfig() *BeaconChainConfig {
e2eConfig.BellatrixForkEpoch = bellatrixE2EForkEpoch
// Terminal Total Difficulty.
e2eConfig.TerminalTotalDifficulty = "616"
e2eConfig.TerminalTotalDifficulty = "0"
// Prysm constants.
e2eConfig.ConfigName = EndToEndName
@ -50,6 +51,8 @@ func E2ETestConfig() *BeaconChainConfig {
func E2EMainnetTestConfig() *BeaconChainConfig {
e2eConfig := MainnetConfig().Copy()
e2eConfig.DepositContractAddress = "0x4242424242424242424242424242424242424242"
e2eConfig.Eth1FollowDistance = 8
// Misc.
e2eConfig.MinGenesisActiveValidatorCount = 256
@ -60,7 +63,6 @@ func E2EMainnetTestConfig() *BeaconChainConfig {
e2eConfig.SecondsPerSlot = 6
e2eConfig.SqrRootSlotsPerEpoch = 5
e2eConfig.SecondsPerETH1Block = 2
e2eConfig.Eth1FollowDistance = 8
e2eConfig.ShardCommitteePeriod = 4
// PoW parameters.
@ -72,7 +74,7 @@ func E2EMainnetTestConfig() *BeaconChainConfig {
e2eConfig.BellatrixForkEpoch = bellatrixE2EForkEpoch
// Terminal Total Difficulty.
e2eConfig.TerminalTotalDifficulty = "616"
e2eConfig.TerminalTotalDifficulty = "0"
// Prysm constants.
e2eConfig.ConfigName = EndToEndMainnetName

View File

@ -2,6 +2,7 @@
package forks
import (
"bytes"
"math"
"sort"
"time"
@ -165,7 +166,19 @@ func SortedForkVersions(forkSchedule map[[4]byte]types.Epoch) [][4]byte {
i++
}
sort.Slice(sortedVersions, func(a, b int) bool {
return forkSchedule[sortedVersions[a]] < forkSchedule[sortedVersions[b]]
// va == "version" a, ie the [4]byte version id
va, vb := sortedVersions[a], sortedVersions[b]
// ea == "epoch" a, ie the types.Epoch corresponding to va
ea, eb := forkSchedule[va], forkSchedule[vb]
// Try to sort by epochs first, which works fine when epochs are all distinct.
// in the case of testnets starting from a given fork, all epochs leading to the fork will be zero.
if ea != eb {
return ea < eb
}
// If the epochs are equal, break the tie with a lexicographic comparison of the fork version bytes.
// eg 2 versions both with a fork epoch of 0, 0x00000000 would come before 0x01000000.
// sort.Slice takes a 'less' func, ie `return a < b`, and when va < vb, bytes.Compare will return -1
return bytes.Compare(va[:], vb[:]) < 0
})
return sortedVersions
}

View File

@ -1,6 +1,7 @@
package forks
import (
"bytes"
"sort"
"strings"
@ -28,7 +29,12 @@ func (o OrderedSchedule) Len() int { return len(o) }
func (o OrderedSchedule) Swap(i, j int) { o[i], o[j] = o[j], o[i] }
// Less implements the Less method of sort.Interface
func (o OrderedSchedule) Less(i, j int) bool { return o[i].Epoch < o[j].Epoch }
func (o OrderedSchedule) Less(i, j int) bool {
if o[i].Epoch == o[j].Epoch {
return bytes.Compare(o[i].Version[:], o[j].Version[:]) < 0
}
return o[i].Epoch < o[j].Epoch
}
// VersionForEpoch finds the Version with the highest epoch <= the given epoch
func (o OrderedSchedule) VersionForEpoch(epoch types.Epoch) ([fieldparams.VersionLength]byte, error) {

View File

@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"generate_genesis_state.go",
"generate_genesis_state_bellatrix.go",
"generate_keys.go",
],
importpath = "github.com/prysmaticlabs/prysm/v3/runtime/interop",
@ -18,6 +19,7 @@ go_library(
"//crypto/bls:go_default_library",
"//crypto/hash:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time:go_default_library",
"@com_github_pkg_errors//:go_default_library",
@ -27,17 +29,20 @@ go_library(
go_test(
name = "go_default_test",
srcs = [
"generate_genesis_state_bellatrix_test.go",
"generate_genesis_state_test.go",
"generate_keys_test.go",
],
data = [
"keygen_test_vector.yaml",
],
embed = [":go_default_library"],
deps = [
":go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/params:go_default_library",
"//container/trie:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/assert:go_default_library",
"//testing/require:go_default_library",

View File

@ -0,0 +1,62 @@
// Package interop contains deterministic utilities for generating
// genesis states and keys.
package interop
import (
"context"
"github.com/pkg/errors"
coreState "github.com/prysmaticlabs/prysm/v3/beacon-chain/core/transition"
statenative "github.com/prysmaticlabs/prysm/v3/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/container/trie"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/time"
)
// GenerateGenesisStateBellatrix deterministically given a genesis time and number of validators.
// If a genesis time of 0 is supplied it is set to the current time.
func GenerateGenesisStateBellatrix(ctx context.Context, genesisTime, numValidators uint64, ep *enginev1.ExecutionPayload, ed *ethpb.Eth1Data) (*ethpb.BeaconStateBellatrix, []*ethpb.Deposit, error) {
privKeys, pubKeys, err := DeterministicallyGenerateKeys(0 /*startIndex*/, numValidators)
if err != nil {
return nil, nil, errors.Wrapf(err, "could not deterministically generate keys for %d validators", numValidators)
}
depositDataItems, depositDataRoots, err := DepositDataFromKeys(privKeys, pubKeys)
if err != nil {
return nil, nil, errors.Wrap(err, "could not generate deposit data from keys")
}
return GenerateGenesisStateBellatrixFromDepositData(ctx, genesisTime, depositDataItems, depositDataRoots, ep, ed)
}
// GenerateGenesisStateBellatrixFromDepositData creates a genesis state given a list of
// deposit data items and their corresponding roots.
func GenerateGenesisStateBellatrixFromDepositData(
ctx context.Context, genesisTime uint64, depositData []*ethpb.Deposit_Data, depositDataRoots [][]byte, ep *enginev1.ExecutionPayload, e1d *ethpb.Eth1Data,
) (*ethpb.BeaconStateBellatrix, []*ethpb.Deposit, error) {
t, err := trie.GenerateTrieFromItems(depositDataRoots, params.BeaconConfig().DepositContractTreeDepth)
if err != nil {
return nil, nil, errors.Wrap(err, "could not generate Merkle trie for deposit proofs")
}
deposits, err := GenerateDepositsFromData(depositData, t)
if err != nil {
return nil, nil, errors.Wrap(err, "could not generate deposits from the deposit data provided")
}
if genesisTime == 0 {
genesisTime = uint64(time.Now().Unix())
}
beaconState, err := coreState.GenesisBeaconStateBellatrix(ctx, deposits, genesisTime, e1d, ep)
if err != nil {
return nil, nil, errors.Wrap(err, "could not generate genesis state")
}
bsi := beaconState.ToProtoUnsafe()
pbb, ok := bsi.(*ethpb.BeaconStateBellatrix)
if !ok {
return nil, nil, errors.New("unexpected BeaconState version")
}
pbState, err := statenative.ProtobufBeaconStateBellatrix(pbb)
if err != nil {
return nil, nil, err
}
return pbState, deposits, nil
}

View File

@ -0,0 +1,50 @@
package interop
import (
"context"
"testing"
state_native "github.com/prysmaticlabs/prysm/v3/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/container/trie"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/testing/require"
)
func TestGenerateGenesisStateBellatrix(t *testing.T) {
ep := &enginev1.ExecutionPayload{
ParentHash: make([]byte, 32),
FeeRecipient: make([]byte, 20),
StateRoot: make([]byte, 32),
ReceiptsRoot: make([]byte, 32),
LogsBloom: make([]byte, 256),
PrevRandao: make([]byte, 32),
BlockNumber: 0,
GasLimit: 0,
GasUsed: 0,
Timestamp: 0,
ExtraData: make([]byte, 32),
BaseFeePerGas: make([]byte, 32),
BlockHash: make([]byte, 32),
Transactions: make([][]byte, 0),
}
e1d := &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
DepositCount: 0,
BlockHash: make([]byte, 32),
}
g, _, err := GenerateGenesisStateBellatrix(context.Background(), 0, params.BeaconConfig().MinGenesisActiveValidatorCount, ep, e1d)
require.NoError(t, err)
tr, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
require.NoError(t, err)
dr, err := tr.HashTreeRoot()
require.NoError(t, err)
g.Eth1Data.DepositRoot = dr[:]
g.Eth1Data.BlockHash = make([]byte, 32)
st, err := state_native.InitializeFromProtoUnsafeBellatrix(g)
require.NoError(t, err)
_, err = st.MarshalSSZ()
require.NoError(t, err)
}

View File

@ -20,15 +20,23 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v3/testing/endtoend/components",
visibility = ["//testing/endtoend:__subpackages__"],
deps = [
"//beacon-chain/core/blocks:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//cmd:go_default_library",
"//cmd/beacon-chain/flags:go_default_library",
"//cmd/beacon-chain/sync/genesis:go_default_library",
"//cmd/validator/flags:go_default_library",
"//config/features:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//config/validator/service:go_default_library",
"//container/trie:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",
"//io/file:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/interop:go_default_library",
"//testing/endtoend/helpers:go_default_library",
"//testing/endtoend/params:go_default_library",

View File

@ -14,10 +14,21 @@ import (
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state"
state_native "github.com/prysmaticlabs/prysm/v3/beacon-chain/state/state-native"
cmdshared "github.com/prysmaticlabs/prysm/v3/cmd"
"github.com/prysmaticlabs/prysm/v3/cmd/beacon-chain/flags"
"github.com/prysmaticlabs/prysm/v3/cmd/beacon-chain/sync/genesis"
"github.com/prysmaticlabs/prysm/v3/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v3/config/fieldparams"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/container/trie"
"github.com/prysmaticlabs/prysm/v3/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v3/io/file"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/runtime/interop"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/helpers"
e2e "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params"
e2etypes "github.com/prysmaticlabs/prysm/v3/testing/endtoend/types"
@ -166,6 +177,108 @@ func NewBeaconNode(config *e2etypes.E2EConfig, index int, enr string) *BeaconNod
}
}
func (node *BeaconNode) generateGenesis(ctx context.Context) (state.BeaconState, error) {
if e2e.TestParams.Eth1GenesisBlock == nil {
return nil, errors.New("Cannot construct bellatrix block, e2e.TestParams.Eth1GenesisBlock == nil")
}
gb := e2e.TestParams.Eth1GenesisBlock
// so the DepositRoot in the BeaconState should be set to the HTR of an empty deposit trie.
t, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
if err != nil {
return nil, err
}
dr, err := t.HashTreeRoot()
if err != nil {
return nil, err
}
e1d := &ethpb.Eth1Data{
DepositRoot: dr[:],
DepositCount: 0,
BlockHash: gb.Hash().Bytes(),
}
payload := &enginev1.ExecutionPayload{
ParentHash: gb.ParentHash().Bytes(),
FeeRecipient: gb.Coinbase().Bytes(),
StateRoot: gb.Root().Bytes(),
ReceiptsRoot: gb.ReceiptHash().Bytes(),
LogsBloom: gb.Bloom().Bytes(),
PrevRandao: params.BeaconConfig().ZeroHash[:],
BlockNumber: gb.NumberU64(),
GasLimit: gb.GasLimit(),
GasUsed: gb.GasUsed(),
Timestamp: gb.Time(),
ExtraData: gb.Extra()[:32],
BaseFeePerGas: bytesutil.PadTo(bytesutil.ReverseByteOrder(gb.BaseFee().Bytes()), fieldparams.RootLength),
BlockHash: gb.Hash().Bytes(),
Transactions: make([][]byte, 0),
}
genesis, _, err := interop.GenerateGenesisStateBellatrix(ctx, e2e.TestParams.CLGenesisTime, params.BeaconConfig().MinGenesisActiveValidatorCount, payload, e1d)
if err != nil {
return nil, err
}
lbhr, err := genesis.LatestBlockHeader.HashTreeRoot()
if err != nil {
return nil, err
}
si, err := state_native.InitializeFromProtoUnsafeBellatrix(genesis)
if err != nil {
return nil, err
}
genb, err := blocks.NewGenesisBlockForState(ctx, si)
if err != nil {
return nil, err
}
gbr, err := genb.Block().HashTreeRoot()
if err != nil {
return nil, err
}
log.WithField("el_block_time", gb.Time()).
WithField("cl_genesis_time", genesis.GenesisTime).
WithField("state_root", fmt.Sprintf("%#x", genb.Block().StateRoot())).
WithField("latest_block_header_root", fmt.Sprintf("%#x", lbhr)).
WithField("latest_block_header_state_root", fmt.Sprintf("%#x", genesis.LatestBlockHeader.StateRoot)).
WithField("latest_block_header_parent_root", fmt.Sprintf("%#x", genesis.LatestBlockHeader.ParentRoot)).
WithField("latest_block_header_body_root", fmt.Sprintf("%#x", genesis.LatestBlockHeader.BodyRoot)).
WithField("derived_block_root", fmt.Sprintf("%#x", gbr)).
WithField("el_block_root", fmt.Sprintf("%#x", genesis.Eth1Data.BlockHash)).
Info("genesis eth1 data")
return si, nil
}
func (node *BeaconNode) saveGenesis(ctx context.Context) (string, error) {
// The deposit contract starts with an empty trie, we use the BeaconState to "pre-mine" the validator registry,
g, err := node.generateGenesis(ctx)
if err != nil {
return "", err
}
root, err := g.HashTreeRoot(ctx)
if err != nil {
return "", err
}
lbhr, err := g.LatestBlockHeader().HashTreeRoot()
if err != nil {
return "", err
}
log.WithField("fork_version", g.Fork().CurrentVersion).
WithField("latest_block_header.root", fmt.Sprintf("%#x", lbhr)).
WithField("state_root", fmt.Sprintf("%#x", root)).
Infof("BeaconState info")
genesisBytes, err := g.MarshalSSZ()
if err != nil {
return "", err
}
genesisDir := path.Join(e2e.TestParams.TestPath, fmt.Sprintf("genesis/%d", node.index))
if err := file.MkdirAll(genesisDir); err != nil {
return "", err
}
genesisPath := path.Join(genesisDir, "genesis.ssz")
return genesisPath, file.WriteFile(genesisPath, genesisBytes)
}
// Start starts a fresh beacon node, connecting to all passed in beacon nodes.
func (node *BeaconNode) Start(ctx context.Context) error {
binaryPath, found := bazel.FindBinary("cmd/beacon-chain", "beacon-chain")
@ -191,10 +304,16 @@ func (node *BeaconNode) Start(ctx context.Context) error {
jwtPath = path.Join(e2e.TestParams.TestPath, "eth1data/miner/")
}
jwtPath = path.Join(jwtPath, "geth/jwtsecret")
genesisPath, err := node.saveGenesis(ctx)
if err != nil {
return err
}
args := []string{
fmt.Sprintf("--%s=%s", genesis.StatePath.Name, genesisPath),
fmt.Sprintf("--%s=%s/eth2-beacon-node-%d", cmdshared.DataDirFlag.Name, e2e.TestParams.TestPath, index),
fmt.Sprintf("--%s=%s", cmdshared.LogFileName.Name, stdOutFile.Name()),
fmt.Sprintf("--%s=%s", flags.DepositContractFlag.Name, e2e.TestParams.ContractAddress.Hex()),
fmt.Sprintf("--%s=%s", flags.DepositContractFlag.Name, params.BeaconConfig().DepositContractAddress),
fmt.Sprintf("--%s=%d", flags.RPCPort.Name, e2e.TestParams.Ports.PrysmBeaconNodeRPCPort+index),
fmt.Sprintf("--%s=http://127.0.0.1:%d", flags.ExecutionEngineEndpoint.Name, e2e.TestParams.Ports.Eth1ProxyPort+index),
fmt.Sprintf("--%s=%s", flags.ExecutionJWTSecretFlag.Name, jwtPath),

View File

@ -15,9 +15,9 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v3/testing/endtoend/components/eth1",
visibility = ["//testing/endtoend:__subpackages__"],
deps = [
"//beacon-chain/execution/testing:go_default_library",
"//config/params:go_default_library",
"//contracts/deposit:go_default_library",
"//contracts/deposit/mock:go_default_library",
"//crypto/rand:go_default_library",
"//encoding/bytesutil:go_default_library",
"//io/file:go_default_library",

View File

@ -10,6 +10,7 @@ import (
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/accounts/keystore"
"github.com/ethereum/go-ethereum/common"
gethtypes "github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/pkg/errors"
@ -20,6 +21,7 @@ import (
e2e "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/types"
"github.com/prysmaticlabs/prysm/v3/testing/util"
log "github.com/sirupsen/logrus"
)
var gweiPerEth = big.NewInt(int64(params.BeaconConfig().GweiPerEth))
@ -143,6 +145,11 @@ type SentDeposit struct {
// (using 2 transactions for partial deposits) and then uses WaitForBlocks (which spams the miner node with transactions
// to and from its own address) to advance the chain until it has moved forward ETH1_FOLLOW_DISTANCE blocks.
func (d *Depositor) SendAndMine(ctx context.Context, offset, nvals int, batch types.DepositBatch, partial bool) error {
balance, err := d.Client.BalanceAt(ctx, d.Key.Address, nil)
if err != nil {
return err
}
log.WithField("balance", balance.String()).WithField("account", d.Key.Address.Hex()).Info("SendAndMine balance check")
// This is the "Send" part of the function. Compute deposits for `nvals` validators,
// with half of those deposits being split over 2 transactions if the `partial` flag is true,
// and throwing away any validators before `offset`.
@ -211,7 +218,8 @@ func (d *Depositor) txops(ctx context.Context) (*bind.TransactOpts, error) {
// DepositContract is a special-purpose client for calling the deposit contract.
func (d *Depositor) contractDepositor() (*contracts.DepositContract, error) {
if d.cd == nil {
contract, err := contracts.NewDepositContract(e2e.TestParams.ContractAddress, d.Client)
addr := common.HexToAddress(params.BeaconConfig().DepositContractAddress)
contract, err := contracts.NewDepositContract(addr, d.Client)
if err != nil {
return nil, err
}

View File

@ -20,7 +20,6 @@ const KeystorePassword = "password"
const minerPasswordFile = "password.txt"
const minerFile = "UTC--2021-12-22T19-14-08.590377700Z--878705ba3f8bc32fcf7f4caa1a35e72af65cf766"
const timeGapPerTX = 100 * time.Millisecond
const timeGapPerMiningTX = 250 * time.Millisecond
var _ e2etypes.ComponentRunner = (*NodeSet)(nil)

View File

@ -2,23 +2,24 @@ package eth1
import (
"context"
"encoding/json"
"fmt"
"math/big"
"os"
"os/exec"
"path"
"strings"
"syscall"
"time"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
"github.com/ethereum/go-ethereum/rpc"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/execution/testing"
"github.com/prysmaticlabs/prysm/v3/config/params"
contracts "github.com/prysmaticlabs/prysm/v3/contracts/deposit/mock"
io "github.com/prysmaticlabs/prysm/v3/io/file"
contracts "github.com/prysmaticlabs/prysm/v3/contracts/deposit"
"github.com/prysmaticlabs/prysm/v3/io/file"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/helpers"
e2e "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params"
e2etypes "github.com/prysmaticlabs/prysm/v3/testing/endtoend/types"
@ -86,21 +87,24 @@ func (m *Miner) initAttempt(ctx context.Context, attempt int) (*os.File, error)
return nil, errors.New("go-ethereum binary not found")
}
staticGenesis, err := e2e.TestParams.Paths.Eth1Runfile("genesis.json")
gethJsonPath := path.Join(path.Dir(binaryPath), "genesis.json")
gen := testing.GethTestnetGenesis(e2e.TestParams.Eth1GenesisTime, params.BeaconConfig())
log.Infof("eth1 miner genesis timestamp=%d", e2e.TestParams.Eth1GenesisTime)
b, err := json.Marshal(gen)
if err != nil {
return nil, err
}
genesisPath := path.Join(path.Dir(binaryPath), "genesis.json")
if err := io.CopyFile(staticGenesis, genesisPath); err != nil {
return nil, errors.Wrapf(err, "error copying %s to %s", staticGenesis, genesisPath)
if err := file.WriteFile(gethJsonPath, b); err != nil {
return nil, err
}
initCmd := exec.CommandContext(
ctx,
binaryPath,
"init",
fmt.Sprintf("--datadir=%s", m.DataDir()),
genesisPath) // #nosec G204 -- Safe
// write the same thing to the logs dir for inspection
gethJsonLogPath := e2e.TestParams.Logfile("genesis.json")
if err := file.WriteFile(gethJsonLogPath, b); err != nil {
return nil, err
}
initCmd := exec.CommandContext(ctx, binaryPath, "init", fmt.Sprintf("--datadir=%s", m.DataDir()), gethJsonPath) // #nosec G204 -- Safe
// redirect stderr to a log file
initFile, err := helpers.DeleteAndCreatePath(e2e.TestParams.Logfile("eth1-init_miner.log"))
@ -151,10 +155,10 @@ func (m *Miner) initAttempt(ctx context.Context, attempt int) (*os.File, error)
if err != nil {
return nil, err
}
if err = io.CopyFile(keystorePath, m.DataDir("keystore", minerFile)); err != nil {
if err = file.CopyFile(keystorePath, m.DataDir("keystore", minerFile)); err != nil {
return nil, errors.Wrapf(err, "error copying %s to %s", keystorePath, m.DataDir("keystore", minerFile))
}
err = io.WriteFile(pwFile, []byte(KeystorePassword))
err = file.WriteFile(pwFile, []byte(KeystorePassword))
if err != nil {
return nil, err
}
@ -170,14 +174,6 @@ func (m *Miner) initAttempt(ctx context.Context, attempt int) (*os.File, error)
if err = runCmd.Start(); err != nil {
return nil, fmt.Errorf("failed to start eth1 chain: %w", err)
}
// check logs for common issues that prevent the EL miner from starting up.
if err = helpers.WaitForTextInFile(minerLog, "Commit new sealing work"); err != nil {
kerr := runCmd.Process.Kill()
if kerr != nil {
log.WithError(kerr).Error("error sending kill to failed miner command process")
}
return nil, fmt.Errorf("mining log not found, this means the eth1 chain had issues starting: %w", err)
}
if err = helpers.WaitForTextInFile(minerLog, "Started P2P networking"); err != nil {
kerr := runCmd.Process.Kill()
if kerr != nil {
@ -220,54 +216,33 @@ func (m *Miner) Start(ctx context.Context) error {
return fmt.Errorf("failed to connect to ipc: %w", err)
}
web3 := ethclient.NewClient(client)
keystorePath, err := e2e.TestParams.Paths.MinerKeyPath()
block, err := web3.BlockByNumber(ctx, nil)
if err != nil {
return err
}
// this is the key for the miner account. miner account balance is pre-mined in genesis.json.
key, err := helpers.KeyFromPath(keystorePath, KeystorePassword)
log.Infof("genesis block timestamp=%d", block.Time())
eth1BlockHash := block.Hash()
e2e.TestParams.Eth1GenesisBlock = block
log.Infof("miner says genesis block root=%#x", eth1BlockHash)
cAddr := common.HexToAddress(params.BeaconConfig().DepositContractAddress)
code, err := web3.CodeAt(ctx, cAddr, nil)
if err != nil {
return err
}
// Waiting for the blocks to advance by eth1follow to prevent issues reading the chain.
// Note that WaitForBlocks spams transfer transactions (to and from the miner's address) in order to advance.
if err = WaitForBlocks(web3, key, params.BeaconConfig().Eth1FollowDistance); err != nil {
return fmt.Errorf("unable to advance chain: %w", err)
}
// Time to deploy the contract using the miner's key.
txOpts, err := bind.NewKeyedTransactorWithChainID(key.PrivateKey, big.NewInt(NetworkId))
log.Infof("contract code size = %d", len(code))
depositContractCaller, err := contracts.NewDepositContractCaller(cAddr, web3)
if err != nil {
return err
}
nonce, err := web3.PendingNonceAt(ctx, key.Address)
dCount, err := depositContractCaller.GetDepositCount(&bind.CallOpts{})
if err != nil {
log.Error("failed to call get_deposit_count method of deposit contract")
return err
}
txOpts.Nonce = big.NewInt(0).SetUint64(nonce)
txOpts.Context = ctx
contractAddr, tx, _, err := contracts.DeployDepositContract(txOpts, web3)
if err != nil {
return fmt.Errorf("failed to deploy deposit contract: %w", err)
}
e2e.TestParams.ContractAddress = contractAddr
// Wait for contract to mine.
for pending := true; pending; _, pending, err = web3.TransactionByHash(ctx, tx.Hash()) {
if err != nil {
return err
}
time.Sleep(timeGapPerTX)
}
// Advancing the blocks another eth1follow distance to prevent issues reading the chain.
if err = WaitForBlocks(web3, key, params.BeaconConfig().Eth1FollowDistance); err != nil {
return fmt.Errorf("unable to advance chain: %w", err)
}
log.Infof("deposit contract count=%d", dCount)
// Mark node as ready.
close(m.started)
return m.cmd.Wait()
}

View File

@ -2,6 +2,7 @@ package eth1
import (
"context"
"encoding/json"
"fmt"
"os"
"os/exec"
@ -10,13 +11,15 @@ import (
"strings"
"syscall"
log "github.com/sirupsen/logrus"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/execution/testing"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/io/file"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/helpers"
e2e "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params"
e2etypes "github.com/prysmaticlabs/prysm/v3/testing/endtoend/types"
log "github.com/sirupsen/logrus"
)
// Node represents an ETH1 node.
@ -53,12 +56,26 @@ func (node *Node) Start(ctx context.Context) error {
}
}
initCmd := exec.CommandContext(
ctx,
binaryPath,
"init",
fmt.Sprintf("--datadir=%s", eth1Path),
binaryPath[:strings.LastIndex(binaryPath, "/")]+"/genesis.json") // #nosec G204 -- Safe
if err := file.MkdirAll(eth1Path); err != nil {
return err
}
gethJsonPath := path.Join(eth1Path, "genesis.json")
gen := testing.GethTestnetGenesis(e2e.TestParams.Eth1GenesisTime, params.BeaconConfig())
b, err := json.Marshal(gen)
if err != nil {
return err
}
if err := file.WriteFile(gethJsonPath, b); err != nil {
return err
}
copyPath := path.Join(e2e.TestParams.LogPath, "eth1-genesis.json")
if err := file.WriteFile(copyPath, b); err != nil {
return err
}
initCmd := exec.CommandContext(ctx, binaryPath, "init", fmt.Sprintf("--datadir=%s", eth1Path), gethJsonPath) // #nosec G204 -- Safe
initFile, err := helpers.DeleteAndCreateFile(e2e.TestParams.LogPath, "eth1-init_"+strconv.Itoa(node.index)+".log")
if err != nil {
return err

View File

@ -70,7 +70,7 @@ func (t *TransactionGenerator) Start(ctx context.Context) error {
}
f := filler.NewFiller(rnd)
// Broadcast Transactions every 3 blocks
txPeriod := time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
txPeriod := 3 * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
ticker := time.NewTicker(txPeriod)
gasPrice := big.NewInt(1e11)
for {
@ -78,7 +78,7 @@ func (t *TransactionGenerator) Start(ctx context.Context) error {
case <-ctx.Done():
return nil
case <-ticker.C:
err := SendTransaction(client, mineKey.PrivateKey, f, gasPrice, mineKey.Address.String(), 200, false)
err := SendTransaction(client, mineKey.PrivateKey, f, gasPrice, mineKey.Address.String(), 100, false)
if err != nil {
return err
}

View File

@ -262,7 +262,7 @@ func (node *LighthouseBeaconNode) createTestnetDir(index int) (string, error) {
configPath := filepath.Join(testNetDir, "config.yaml")
rawYaml := params.E2EMainnetConfigYaml()
// Add in deposit contract in yaml
depContractStr := fmt.Sprintf("\nDEPOSIT_CONTRACT_ADDRESS: %#x", e2e.TestParams.ContractAddress)
depContractStr := fmt.Sprintf("\nDEPOSIT_CONTRACT_ADDRESS: %s", params.BeaconConfig().DepositContractAddress)
rawYaml = append(rawYaml, []byte(depContractStr)...)
if err := file.MkdirAll(testNetDir); err != nil {

View File

@ -261,7 +261,7 @@ func (w *Web3RemoteSigner) createTestnetDir() (string, error) {
configPath := filepath.Join(testNetDir, "config.yaml")
rawYaml := params.E2ETestConfigYaml()
// Add in deposit contract in yaml
depContractStr := fmt.Sprintf("\nDEPOSIT_CONTRACT_ADDRESS: %#x", e2e.TestParams.ContractAddress)
depContractStr := fmt.Sprintf("\nDEPOSIT_CONTRACT_ADDRESS: %s", params.BeaconConfig().DepositContractAddress)
rawYaml = append(rawYaml, []byte(depContractStr)...)
if err := file.MkdirAll(testNetDir); err != nil {

View File

@ -84,8 +84,6 @@ func (r *testRunner) runBase(runEvents []runEvent) {
if err := helpers.ComponentsStarted(r.comHandler.ctx, []e2etypes.ComponentRunner{miner}); err != nil {
return errors.Wrap(err, "eth1Miner component never started - cannot send deposits")
}
// refactored send and mine goes here
minGenesisActiveCount := int(params.BeaconConfig().MinGenesisActiveValidatorCount)
keyPath, err := e2e.TestParams.Paths.MinerKeyPath()
if err != nil {
return errors.Wrap(err, "error getting miner key file from bazel static files")
@ -99,9 +97,6 @@ func (r *testRunner) runBase(runEvents []runEvent) {
return errors.Wrap(err, "failed to initialize a client to connect to the miner EL node")
}
r.depositor = &eth1.Depositor{Key: key, Client: client, NetworkId: big.NewInt(eth1.NetworkId)}
if err := r.depositor.SendAndMine(r.comHandler.ctx, 0, minGenesisActiveCount, e2etypes.GenesisDepositBatch, true); err != nil {
return errors.Wrap(err, "failed to send and mine deposits")
}
if err := r.depositor.Start(r.comHandler.ctx); err != nil {
return errors.Wrap(err, "depositor.Start failed")
}

View File

@ -40,6 +40,7 @@ go_library(
"//proto/eth/v2:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/interop:go_default_library",
"//runtime/version:go_default_library",
"//testing/endtoend/components:go_default_library",
"//testing/endtoend/helpers:go_default_library",
"//testing/endtoend/params:go_default_library",

View File

@ -6,7 +6,9 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v3/consensus-types/blocks"
ptypes "github.com/prysmaticlabs/prysm/v3/consensus-types/primitives"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v3/runtime/version"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/helpers"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/policies"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/types"
@ -15,11 +17,19 @@ import (
)
var streamDeadline = 1 * time.Minute
var startingFork = version.Bellatrix
// AltairForkTransition ensures that the Altair hard fork has occurred successfully.
var AltairForkTransition = types.Evaluator{
Name: "altair_fork_transition_%d",
Policy: policies.OnEpoch(helpers.AltairE2EForkEpoch),
Policy: func(e ptypes.Epoch) bool {
altair := policies.OnEpoch(helpers.AltairE2EForkEpoch)
// TODO (11750): modify policies to take an end to end config
if startingFork == version.Phase0 {
return altair(e)
}
return false
},
Evaluation: altairForkOccurs,
}

View File

@ -34,7 +34,7 @@ var churnLimit = 4
var depositValCount = e2e.DepositCount
// Deposits should be processed in twice the length of the epochs per eth1 voting period.
var depositsInBlockStart = types.Epoch(math.Floor(float64(params.E2ETestConfig().EpochsPerEth1VotingPeriod) * 2))
var depositsInBlockStart = params.E2ETestConfig().EpochsPerEth1VotingPeriod * 2
// deposits included + finalization + MaxSeedLookahead for activation.
var depositActivationStartEpoch = depositsInBlockStart + 2 + params.E2ETestConfig().MaxSeedLookahead
@ -120,15 +120,7 @@ func processesDepositsInBlocks(ec e2etypes.EvaluationContext, conns ...*grpc.Cli
return errors.Wrap(err, "failed to convert api response type to SignedBeaconBlock interface")
}
b := sb.Block()
slot := b.Slot()
eth1Data := b.Body().Eth1Data()
deposits := b.Body().Deposits()
fmt.Printf(
"Slot: %d with %d deposits, Eth1 block %#x with %d deposits\n",
slot,
len(deposits),
eth1Data.BlockHash, eth1Data.DepositCount,
)
for _, d := range deposits {
k := bytesutil.ToBytes48(d.Data.PublicKey)
v := observed[k]
@ -155,7 +147,12 @@ func verifyGraffitiInBlocks(_ e2etypes.EvaluationContext, conns ...*grpc.ClientC
if err != nil {
return errors.Wrap(err, "failed to get chain head")
}
req := &ethpb.ListBlocksRequest{QueryFilter: &ethpb.ListBlocksRequest_Epoch{Epoch: chainHead.HeadEpoch.Sub(1)}}
begin := chainHead.HeadEpoch
// Prevent underflow when this runs at epoch 0.
if begin > 0 {
begin = begin.Sub(1)
}
req := &ethpb.ListBlocksRequest{QueryFilter: &ethpb.ListBlocksRequest_Epoch{Epoch: begin}}
blks, err := client.ListBeaconBlocks(context.Background(), req)
if err != nil {
return errors.Wrap(err, "failed to get blocks from beacon-chain")
@ -393,12 +390,18 @@ func validatorsVoteWithTheMajority(_ e2etypes.EvaluationContext, conns ...*grpc.
return errors.Wrap(err, "failed to get chain head")
}
req := &ethpb.ListBlocksRequest{QueryFilter: &ethpb.ListBlocksRequest_Epoch{Epoch: chainHead.HeadEpoch.Sub(1)}}
begin := chainHead.HeadEpoch
// Prevent underflow when this runs at epoch 0.
if begin > 0 {
begin = begin.Sub(1)
}
req := &ethpb.ListBlocksRequest{QueryFilter: &ethpb.ListBlocksRequest_Epoch{Epoch: begin}}
blks, err := client.ListBeaconBlocks(context.Background(), req)
if err != nil {
return errors.Wrap(err, "failed to get blocks from beacon-chain")
}
slotsPerVotingPeriod := params.E2ETestConfig().SlotsPerEpoch.Mul(uint64(params.E2ETestConfig().EpochsPerEth1VotingPeriod))
for _, blk := range blks.BlockContainers {
var slot types.Slot
var vote []byte
@ -422,7 +425,6 @@ func validatorsVoteWithTheMajority(_ e2etypes.EvaluationContext, conns ...*grpc.
default:
return errors.New("block neither phase0,altair or bellatrix")
}
slotsPerVotingPeriod := params.E2ETestConfig().SlotsPerEpoch.Mul(uint64(params.E2ETestConfig().EpochsPerEth1VotingPeriod))
// We treat epoch 1 differently from other epoch for two reasons:
// - this evaluator is not executed for epoch 0 so we have to calculate the first slot differently

View File

@ -8,10 +8,10 @@ go_library(
"params.go",
],
importpath = "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params",
visibility = ["//testing/endtoend:__subpackages__"],
visibility = ["//visibility:public"],
deps = [
"//io/file:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@io_bazel_rules_go//go/tools/bazel:go_default_library",
],
)

View File

@ -14,8 +14,9 @@ import (
"testing"
"time"
"github.com/ethereum/go-ethereum/core/types"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/ethereum/go-ethereum/common"
"github.com/prysmaticlabs/prysm/v3/io/file"
)
@ -26,9 +27,12 @@ type params struct {
TestShardIndex int
BeaconNodeCount int
LighthouseBeaconNodeCount int
ContractAddress common.Address
Ports *ports
Paths *paths
Eth1GenesisBlock *types.Block
StartTime time.Time
CLGenesisTime uint64
Eth1GenesisTime uint64
}
type ports struct {
@ -149,6 +153,8 @@ const (
ValidatorMetricsPort = ValidatorGatewayPort + portSpan
JaegerTracingPort = 9150
StartupBufferSecs = 5
)
func logDir() string {
@ -198,12 +204,15 @@ func Init(t *testing.T, beaconNodeCount int) error {
return err
}
genTime := uint64(time.Now().Unix()) + StartupBufferSecs
TestParams = &params{
TestPath: filepath.Join(testPath, fmt.Sprintf("shard-%d", testShardIndex)),
LogPath: logPath,
TestShardIndex: testShardIndex,
BeaconNodeCount: beaconNodeCount,
Ports: testPorts,
CLGenesisTime: genTime,
Eth1GenesisTime: genTime,
}
return nil
}
@ -247,6 +256,7 @@ func InitMultiClient(t *testing.T, beaconNodeCount int, lighthouseNodeCount int)
return err
}
genTime := uint64(time.Now().Unix()) + StartupBufferSecs
TestParams = &params{
TestPath: filepath.Join(testPath, fmt.Sprintf("shard-%d", testShardIndex)),
LogPath: logPath,
@ -254,6 +264,8 @@ func InitMultiClient(t *testing.T, beaconNodeCount int, lighthouseNodeCount int)
BeaconNodeCount: beaconNodeCount,
LighthouseBeaconNodeCount: lighthouseNodeCount,
Ports: testPorts,
CLGenesisTime: genTime,
Eth1GenesisTime: genTime,
}
return nil
}

View File

@ -2,7 +2,6 @@ filegroup(
name = "eth1data",
srcs = [
"UTC--2021-12-22T19-14-08.590377700Z--878705ba3f8bc32fcf7f4caa1a35e72af65cf766",
"genesis.json",
],
visibility = ["//testing/endtoend:__subpackages__"],
)

View File

@ -1,32 +0,0 @@
{
"config": {
"chainId": 1337,
"homesteadBlock": 0,
"eip150Block": 0,
"eip155Block": 0,
"eip158Block": 0,
"byzantiumBlock": 0,
"constantinopleBlock": 0,
"petersburgBlock": 0,
"istanbulBlock": 0,
"berlinBlock": 0,
"londonBlock": 0,
"mergeForkBlock": 308,
"terminalTotalDifficulty": 616,
"clique": {
"period": 2,
"epoch": 30000
}
},
"alloc": {
"0x878705ba3f8bc32fcf7f4caa1a35e72af65cf766": {"balance": "100000000000000000000000000000"}
},
"coinbase" : "0x0000000000000000000000000000000000000000",
"difficulty": "1",
"extradata": "0x0000000000000000000000000000000000000000000000000000000000000000878705ba3f8bc32fcf7f4caa1a35e72af65cf7660000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
"gasLimit" : "0xffffff",
"nonce" : "0x0000000000000042",
"mixhash" : "0x0000000000000000000000000000000000000000000000000000000000000000",
"parentHash" : "0x0000000000000000000000000000000000000000000000000000000000000000",
"timestamp" : "0x00"
}

View File

@ -134,6 +134,12 @@ func buildGenesisBeaconStateBellatrix(genesisTime uint64, preState state.BeaconS
if err != nil {
return nil, err
}
scoresMissing := len(preState.Validators()) - len(scores)
if scoresMissing > 0 {
for i := 0; i < scoresMissing; i++ {
scores = append(scores, 0)
}
}
st := &ethpb.BeaconStateBellatrix{
// Misc fields.
Slot: 0,
@ -240,5 +246,16 @@ func buildGenesisBeaconStateBellatrix(genesisTime uint64, preState state.BeaconS
TransactionsRoot: make([]byte, 32),
}
return state_native.InitializeFromProtoBellatrix(st)
bs, err := state_native.InitializeFromProtoBellatrix(st)
if err != nil {
return nil, err
}
is, err := bs.InactivityScores()
if err != nil {
return nil, err
}
if bs.NumValidators() != len(is) {
return nil, errors.New("inactivity score mismatch with num vals")
}
return bs, nil
}