Compare commits

...

14 Commits

Author SHA1 Message Date
nisdas
ae2adcef15 add check and test case 2022-12-05 21:24:32 +08:00
nisdas
76f938f1c4 add check and test case 2022-12-05 21:22:57 +08:00
Kasey Kirkham
c0297ca0c2 send deposits, spam tx to advance, fix miner alloc 2022-12-01 21:46:24 -06:00
Kasey Kirkham
d51fb3a648 fix fork ordering bug and bellatrix genesis blocks 2022-12-01 14:36:11 -06:00
Kasey Kirkham
3ab486e140 multiple bugfixes 2022-11-30 15:12:57 -06:00
Kasey Kirkham
f52dbe3c8f WIP 2022-11-30 12:08:58 -06:00
Kasey Kirkham
166c284ca4 remove debug wrapping 2022-11-29 18:20:47 -06:00
Kasey Kirkham
2654e3f080 WIP more fixes towards start from bellatrix 2022-11-29 18:20:19 -06:00
Kasey Kirkham
ed65688252 use encoding/detect for genesis state bytes 2022-11-29 10:48:46 -06:00
Kasey Kirkham
4bae07fdbb WIP 2022-11-29 10:23:33 -06:00
Kasey Kirkham
42808a0c29 combine bellatrix state w/ rm-pre-genesis branch 2022-11-28 14:24:53 -06:00
Kasey Kirkham
11cbe8fcb9 WIP - grabbing changes from rm-pre-genesis branch 2022-11-28 14:19:44 -06:00
Kasey Kirkham
938fca1933 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.
2022-11-23 16:46:45 -06:00
Kasey Kirkham
604688a00e WIP trying to start from bellatrix state 2022-11-23 16:09:46 -06:00
32 changed files with 976 additions and 117 deletions

View File

@@ -38,6 +38,7 @@ type notifyForkchoiceUpdateArg struct {
func (s *Service) notifyForkchoiceUpdate(ctx context.Context, arg *notifyForkchoiceUpdateArg) (*enginev1.PayloadIDBytes, error) {
ctx, span := trace.StartSpan(ctx, "blockChain.notifyForkchoiceUpdate")
defer span.End()
log.Infof("notifyForkchoiceUpdate, headRoot=%#x, headState.latest_block_header=%#x", arg.headRoot, arg.headState.LatestBlockHeader().BodyRoot)
headBlk := arg.headBlock
if headBlk == nil || headBlk.IsNil() || headBlk.Body().IsNil() {

View File

@@ -309,7 +309,10 @@ func (s *Service) initializeHeadFromDB(ctx context.Context) error {
if err := s.setHead(finalizedRoot, finalizedBlock, finalizedState); err != nil {
return errors.Wrap(err, "could not set head")
}
_, err = s.notifyForkchoiceUpdate(s.ctx, &notifyForkchoiceUpdateArg{headState: finalizedState, headRoot: finalizedRoot, headBlock: finalizedBlock.Block()})
if err != nil {
return errors.Wrap(err, "error calling FCU with finalized state at startup")
}
return nil
}
@@ -444,6 +447,10 @@ func (s *Service) saveGenesisData(ctx context.Context, genesisState state.Beacon
if err := s.setHead(genesisBlkRoot, genesisBlk, genesisState); err != nil {
log.WithError(err).Fatal("Could not set head")
}
_, err = s.notifyForkchoiceUpdate(s.ctx, &notifyForkchoiceUpdateArg{headState: genesisState, headRoot: genesisBlkRoot, headBlock: genesisBlk.Block()})
if err != nil {
log.WithError(err).Fatal("Could not call FCU with genesis")
}
return nil
}

View File

@@ -3,9 +3,14 @@
package blocks
import (
"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 +34,67 @@ func NewGenesisBlock(stateRoot []byte) *ethpb.SignedBeaconBlock {
}
return block
}
var ErrUnrecognizedState = errors.New("uknonwn underlying type for state.BeaconState value")
func NewGenesisBlockForState(root [32]byte, st state.BeaconState) (interfaces.SignedBeaconBlock, error) {
ps := st.ToProto()
switch ps.(type) {
case *ethpb.BeaconState:
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),
},
},
},
Signature: params.BeaconConfig().EmptySignature[:],
})
/*
return blocks.NewSignedBeaconBlock(&ethpb.BeaconBlockBellatrix{
Body: })
*/
default:
return nil, ErrUnrecognizedState
/*
case *ethpb.BeaconStateAltair:
case *ethpb.BeaconStateCapella:
*/
}
}

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,273 @@
package transition
import (
"context"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/core/altair"
enginev1 "github.com/prysmaticlabs/prysm/v3/proto/engine/v1"
fieldparams "github.com/prysmaticlabs/prysm/v3/config/fieldparams"
"github.com/pkg/errors"
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"
"github.com/prysmaticlabs/prysm/v3/config/params"
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) (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")
}
return OptimizedGenesisBeaconStateBellatrix(genesisTime, st, st.Eth1Data())
}
// 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) (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)
}
}
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: &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),
},
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),
},
}).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,17 +1,14 @@
package kv
import (
"bytes"
"context"
"fmt"
"github.com/prysmaticlabs/prysm/v3/encoding/ssz/detect"
"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"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
@@ -19,17 +16,16 @@ import (
func (s *Store) SaveGenesisData(ctx context.Context, genesisState state.BeaconState) error {
stateRoot, err := genesisState.HashTreeRoot(ctx)
if err != nil {
return err
return errors.Wrap(err, "HashTreeRoot")
}
genesisBlk := blocks.NewGenesisBlock(stateRoot[:])
genesisBlkRoot, err := genesisBlk.Block.HashTreeRoot()
wsb, err := blocks.NewGenesisBlockForState(stateRoot, genesisState)
if err != nil {
return errors.Wrap(err, "NewGenesisBlockForState")
}
genesisBlkRoot, err := wsb.Block().HashTreeRoot()
if err != nil {
return errors.Wrap(err, "could not get genesis block root")
}
wsb, err := consensusblocks.NewSignedBeaconBlock(genesisBlk)
if err != nil {
return errors.Wrap(err, "could not wrap genesis block")
}
if err := s.SaveBlock(ctx, wsb); err != nil {
return errors.Wrap(err, "could not save genesis block")
}
@@ -54,11 +50,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 +79,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

@@ -88,6 +88,7 @@ func (s *Service) pollConnectionStatus(ctx context.Context) {
// Forces to retry an execution client connection.
func (s *Service) retryExecutionClientConnection(ctx context.Context, err error) {
log.Debugf("retryExecutionClientConnection, err=%s", err.Error())
s.runError = err
s.updateConnectedETH1(false)
// Back off for a while before redialing.

View File

@@ -702,6 +702,9 @@ func (s *Service) determineEarliestVotingBlock(ctx context.Context, followBlock
return 0, errors.Errorf("invalid genesis time provided. %d > %d", followBackDist, votingTime)
}
earliestValidTime := votingTime - followBackDist
if earliestValidTime > s.latestEth1Data.BlockTime {
return 0, nil
}
hdr, err := s.BlockByTimestamp(ctx, earliestValidTime)
if err != nil {
return 0, err

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",
@@ -26,7 +27,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

@@ -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 = 0
// 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 = 0
// 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

@@ -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,15 +29,17 @@ 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/prysm/v1alpha1:go_default_library",

View File

@@ -0,0 +1,106 @@
// Package interop contains deterministic utilities for generating
// genesis states and keys.
package interop
import (
"context"
"encoding/hex"
"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"
v1 "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) (*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)
}
// 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,
) (*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")
}
root, err := t.HashTreeRoot()
if err != nil {
return nil, nil, errors.Wrap(err, "could not hash tree root of deposit trie")
}
if genesisTime == 0 {
genesisTime = uint64(time.Now().Unix())
}
beaconState, err := coreState.GenesisBeaconStateBellatrix(ctx, deposits, genesisTime, &ethpb.Eth1Data{
DepositRoot: root[:],
DepositCount: uint64(len(deposits)),
BlockHash: mockEth1BlockHash,
})
if err != nil {
return nil, nil, errors.Wrap(err, "could not generate genesis state")
}
rawSt, ok := beaconState.ToProtoUnsafe().(*ethpb.BeaconStateBellatrix)
if !ok {
return nil, nil, errors.New("state is of invalid type")
}
stateRoot, err := hex.DecodeString("5c9ba34f1167c7f2ccb56623667e04f2d3e769148d181f4a51484e14b3ced910")
if err != nil {
return nil, nil, err
}
receiptsRoot, err := hex.DecodeString("56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421")
if err != nil {
return nil, nil, err
}
blkHash, err := hex.DecodeString("837b03b21e5c2bc5484342b5082848748046071c5882624022b46f20ff0b46db")
if err != nil {
return nil, nil, err
}
txRoot, err := hex.DecodeString("7ffe241ea60187fdb0187bfa22de35d1f9bed7ab061d9401fd47e34a54fbede1")
if err != nil {
return nil, nil, err
}
baseFee := make([]byte, 32)
baseFee[1] = 202
baseFee[2] = 154
baseFee[3] = 59
rawSt.LatestExecutionPayloadHeader = &v1.ExecutionPayloadHeader{
ParentHash: make([]byte, 32),
FeeRecipient: make([]byte, 20),
StateRoot: stateRoot,
ReceiptsRoot: receiptsRoot,
LogsBloom: make([]byte, 256),
PrevRandao: make([]byte, 32),
BlockNumber: 0,
GasLimit: 4000000,
GasUsed: 0,
Timestamp: rawSt.GenesisTime,
BaseFeePerGas: baseFee,
BlockHash: blkHash,
TransactionsRoot: txRoot,
}
pbState, err := statenative.ProtobufBeaconStateBellatrix(rawSt)
if err != nil {
return nil, nil, err
}
return pbState, deposits, nil
}

View File

@@ -0,0 +1,27 @@
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"
"github.com/prysmaticlabs/prysm/v3/testing/require"
)
func TestGenerateGenesisStateBellatrix(t *testing.T) {
g, _, err := GenerateGenesisStateBellatrix(context.Background(), 0, params.BeaconConfig().MinGenesisActiveValidatorCount)
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,13 +20,17 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/v3/testing/endtoend/components",
visibility = ["//testing/endtoend:__subpackages__"],
deps = [
"//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",
"//io/file:go_default_library",
"//runtime/interop:go_default_library",

View File

@@ -12,6 +12,14 @@ import (
"strings"
"syscall"
"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/cmd/beacon-chain/sync/genesis"
"github.com/prysmaticlabs/prysm/v3/container/trie"
"github.com/prysmaticlabs/prysm/v3/io/file"
"github.com/prysmaticlabs/prysm/v3/runtime/interop"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/pkg/errors"
cmdshared "github.com/prysmaticlabs/prysm/v3/cmd"
@@ -166,6 +174,49 @@ func NewBeaconNode(config *e2etypes.E2EConfig, index int, enr string) *BeaconNod
}
}
func (node *BeaconNode) generateGenesis(ctx context.Context) (state.BeaconState, error) {
genesis, _, err := interop.GenerateGenesisStateBellatrix(ctx, e2e.TestParams.CLGenesisTime, params.BeaconConfig().MinGenesisActiveValidatorCount)
if err != nil {
return nil, err
}
// 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
}
genesis.Eth1Data.DepositRoot = dr[:]
if e2e.TestParams.Eth1BlockHash != nil {
genesis.Eth1Data.BlockHash = e2e.TestParams.Eth1BlockHash.Bytes()
genesis.LatestExecutionPayloadHeader.BlockHash = e2e.TestParams.Eth1BlockHash.Bytes()
}
log.Infof("genesis eth1 block root=%#x", genesis.Eth1Data.BlockHash)
return state_native.InitializeFromProtoUnsafeBellatrix(genesis)
}
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
}
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 +242,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

@@ -9,6 +9,8 @@ import (
"strings"
"syscall"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/prysmaticlabs/prysm/v3/testing/endtoend/helpers"
e2e "github.com/prysmaticlabs/prysm/v3/testing/endtoend/params"
@@ -54,6 +56,7 @@ func (node *BootNode) Start(ctx context.Context) error {
fmt.Sprintf("--log-file=%s", stdOutFile.Name()),
fmt.Sprintf("--discv5-port=%d", e2e.TestParams.Ports.BootNodePort),
fmt.Sprintf("--metrics-port=%d", e2e.TestParams.Ports.BootNodeMetricsPort),
fmt.Sprintf("--fork-version=%#x", params.BeaconConfig().BellatrixForkVersion),
"--debug",
}

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

@@ -7,6 +7,10 @@ import (
"sync"
"time"
log "github.com/sirupsen/logrus"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
"github.com/ethereum/go-ethereum/accounts/keystore"
gethtypes "github.com/ethereum/go-ethereum/core/types"
@@ -142,6 +146,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`.
@@ -210,7 +219,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

@@ -5,6 +5,8 @@ import (
"math/big"
"time"
log "github.com/sirupsen/logrus"
"github.com/ethereum/go-ethereum/accounts/keystore"
"github.com/ethereum/go-ethereum/core/types"
"github.com/ethereum/go-ethereum/ethclient"
@@ -47,6 +49,7 @@ func WaitForBlocks(web3 *ethclient.Client, key *keystore.Key, blocksToWait uint6
finishBlock := block.NumberU64() + blocksToWait
for block.NumberU64() <= finishBlock {
log.Infof("waiting for block number %d, last saw %d", finishBlock, block.NumberU64())
spamTX := types.NewTransaction(nonce, key.Address, big.NewInt(0), params.SpamTxGasLimit, big.NewInt(1e6), []byte{})
signed, err := types.SignTx(spamTX, types.NewEIP155Signer(chainID), key.PrivateKey)
if err != nil {

View File

@@ -3,13 +3,14 @@ package eth1
import (
"context"
"fmt"
"math/big"
"os"
"os/exec"
"path"
"strings"
"syscall"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/execution/testing"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/ethereum/go-ethereum/accounts/abi/bind"
@@ -17,8 +18,8 @@ import (
"github.com/ethereum/go-ethereum/rpc"
"github.com/pkg/errors"
"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 := testing.TerribleMarshalHack(gen, params.BeaconConfig().DepositContractAddress, testing.DefaultMinerAddress)
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)
// redirect stderr to a log file
initFile, err := helpers.DeleteAndCreatePath(e2e.TestParams.Logfile("eth1-init_miner.log"))
@@ -139,6 +143,7 @@ func (m *Miner) initAttempt(ctx context.Context, attempt int) (*os.File, error)
"--ws.origins=\"*\"",
"--ipcdisable",
"--verbosity=4",
"--vmdebug",
"--mine",
fmt.Sprintf("--unlock=%s", EthAddress),
"--allow-insecure-unlock",
@@ -151,10 +156,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 +175,16 @@ 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")
/*
// 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)
}
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 +227,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.Eth1BlockHash = &eth1BlockHash
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

@@ -10,6 +10,10 @@ import (
"strings"
"syscall"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/execution/testing"
"github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/io/file"
log "github.com/sirupsen/logrus"
"github.com/bazelbuild/rules_go/go/tools/bazel"
@@ -53,12 +57,31 @@ func (node *Node) Start(ctx context.Context) error {
}
}
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 := testing.TerribleMarshalHack(gen, params.BeaconConfig().DepositContractAddress, testing.DefaultMinerAddress)
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),
binaryPath[:strings.LastIndex(binaryPath, "/")]+"/genesis.json") // #nosec G204 -- Safe
gethJsonPath)
initFile, err := helpers.DeleteAndCreateFile(e2e.TestParams.LogPath, "eth1-init_"+strconv.Itoa(node.index)+".log")
if err != nil {
return err
@@ -92,6 +115,7 @@ func (node *Node) Start(ctx context.Context) error {
"--ws.origins=\"*\"",
"--ipcdisable",
"--verbosity=4",
"--vmdebug",
"--syncmode=full",
fmt.Sprintf("--txpool.locals=%s", EthAddress),
}

View File

@@ -261,7 +261,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: %#x", 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: %#x", params.BeaconConfig().DepositContractAddress)
rawYaml = append(rawYaml, []byte(depContractStr)...)
if err := file.MkdirAll(testNetDir); err != nil {

View File

@@ -85,7 +85,7 @@ func (r *testRunner) runBase(runEvents []runEvent) {
return errors.Wrap(err, "eth1Miner component never started - cannot send deposits")
}
// refactored send and mine goes here
minGenesisActiveCount := int(params.BeaconConfig().MinGenesisActiveValidatorCount)
//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 +99,11 @@ 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.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

@@ -155,7 +155,11 @@ 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
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,7 +397,11 @@ 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
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")

View File

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

View File

@@ -12,9 +12,11 @@ import (
"path/filepath"
"strconv"
"testing"
"time"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/ethereum/go-ethereum/common"
cfgparams "github.com/prysmaticlabs/prysm/v3/config/params"
"github.com/prysmaticlabs/prysm/v3/io/file"
)
@@ -25,9 +27,12 @@ type params struct {
TestShardIndex int
BeaconNodeCount int
LighthouseBeaconNodeCount int
ContractAddress common.Address
Ports *ports
Paths *paths
Eth1BlockHash *common.Hash
StartTime time.Time
CLGenesisTime uint64
Eth1GenesisTime uint64
}
type ports struct {
@@ -148,19 +153,33 @@ const (
ValidatorMetricsPort = ValidatorGatewayPort + portSpan
JaegerTracingPort = 9150
StartupBufferSecs = 5
)
func logDir() string {
wTime := func(p string) string {
return path.Join(p, time.Now().Format("20060102/150405"))
}
path, ok := os.LookupEnv("E2E_LOG_PATH")
if ok {
return wTime(path)
}
path, _ = os.LookupEnv("TEST_UNDECLARED_OUTPUTS_DIR")
return wTime(path)
}
// Init initializes the E2E config, properly handling test sharding.
func Init(t *testing.T, beaconNodeCount int) error {
testPath := bazel.TestTmpDir()
logPath, ok := os.LookupEnv("TEST_UNDECLARED_OUTPUTS_DIR")
if !ok {
return errors.New("expected TEST_UNDECLARED_OUTPUTS_DIR to be defined")
d := logDir()
if d == "" {
return errors.New("unable to determine log directory, no value for E2E_LOG_PATH or TEST_UNDECLARED_OUTPUTS_DIR")
}
logPath = path.Join(logPath, t.Name())
logPath := path.Join(d, t.Name())
if err := file.MkdirAll(logPath); err != nil {
return err
}
testPath := bazel.TestTmpDir()
testTotalShardsStr, ok := os.LookupEnv("TEST_TOTAL_SHARDS")
if !ok {
testTotalShardsStr = "1"
@@ -185,12 +204,22 @@ func Init(t *testing.T, beaconNodeCount int) error {
return err
}
cfg := cfgparams.BeaconConfig()
now := time.Now()
clGenTime := uint64(now.Unix()) + StartupBufferSecs
epochSecs := cfg.SecondsPerSlot * uint64(cfg.SlotsPerEpoch)
// TODO: support starting from any fork, make the genesis offset variable
forkOffset := epochSecs * uint64(cfg.CapellaForkEpoch)
TestParams = &params{
TestPath: filepath.Join(testPath, fmt.Sprintf("shard-%d", testShardIndex)),
LogPath: logPath,
TestShardIndex: testShardIndex,
BeaconNodeCount: beaconNodeCount,
Ports: testPorts,
StartTime: now,
CLGenesisTime: clGenTime,
Eth1GenesisTime: clGenTime + forkOffset,
}
return nil
}
@@ -234,6 +263,12 @@ func InitMultiClient(t *testing.T, beaconNodeCount int, lighthouseNodeCount int)
return err
}
cfg := cfgparams.BeaconConfig()
now := time.Now()
clGenTime := uint64(now.Unix()) + StartupBufferSecs
epochSecs := cfg.SecondsPerSlot * uint64(cfg.SlotsPerEpoch)
// TODO: support starting from any fork, make the genesis offset variable
forkOffset := epochSecs * uint64(cfg.CapellaForkEpoch)
TestParams = &params{
TestPath: filepath.Join(testPath, fmt.Sprintf("shard-%d", testShardIndex)),
LogPath: logPath,
@@ -241,6 +276,9 @@ func InitMultiClient(t *testing.T, beaconNodeCount int, lighthouseNodeCount int)
BeaconNodeCount: beaconNodeCount,
LighthouseBeaconNodeCount: lighthouseNodeCount,
Ports: testPorts,
StartTime: now,
CLGenesisTime: clGenTime,
Eth1GenesisTime: clGenTime + forkOffset,
}
return nil
}

View File

@@ -11,22 +11,18 @@
"istanbulBlock": 0,
"berlinBlock": 0,
"londonBlock": 0,
"mergeForkBlock": 308,
"terminalTotalDifficulty": 616,
"clique": {
"period": 2,
"epoch": 30000
}
"mergeNetsplitBlock": 0,
"terminalTotalDifficulty": 0
},
"alloc": {
"0x878705ba3f8bc32fcf7f4caa1a35e72af65cf766": {"balance": "100000000000000000000000000000"}
},
"coinbase" : "0x0000000000000000000000000000000000000000",
"difficulty": "1",
"extradata": "0x0000000000000000000000000000000000000000000000000000000000000000878705ba3f8bc32fcf7f4caa1a35e72af65cf7660000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000000",
"coinbase": "0x0000000000000000000000000000000000000000",
"difficulty": "0x01",
"extraData": "",
"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) {
panic("inactivity score mismatch with num vals")
}
return bs, nil
}

View File

@@ -193,8 +193,12 @@ func createLocalNode(privKey *ecdsa.PrivateKey, ipAddr net.IP, port int) (*enode
external = ipAddr
}
fVersion := params.BeaconConfig().GenesisForkVersion
if *forkVersion != "" {
fVersion, err = hex.DecodeString(*forkVersion)
fvs := *forkVersion
if fvs != "" {
if len(fvs) == 10 {
fvs = fvs[2:]
}
fVersion, err = hex.DecodeString(fvs)
if err != nil {
return nil, errors.Wrap(err, "Could not retrieve fork version")
}