mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-08 21:08:10 -05:00
Remove EnableEIP4881 flag (#13826)
* Remove EnableEIP4881 flag * Gaz * Fix missing error handler * Remove old tree and fix tests * Gaz * Fix build import * Replace depositcache * Add pendingDeposit tests * Nishant's fix * Fix unsafe uint64 to int * Fix other unsafe uint64 to int * Remove: RemovePendingDeposit * Deprecate and remove DisableEIP4881 flag * Check: index not greater than deposit count * Move index check
This commit is contained in:
@@ -137,7 +137,7 @@ go_test(
|
||||
"//async/event:go_default_library",
|
||||
"//beacon-chain/blockchain/testing:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/blocks:go_default_library",
|
||||
"//beacon-chain/core/feed/state:go_default_library",
|
||||
"//beacon-chain/core/helpers:go_default_library",
|
||||
|
||||
@@ -9,7 +9,7 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
||||
@@ -80,7 +80,7 @@ func setupBeaconChain(t *testing.T, beaconDB db.Database) *Service {
|
||||
attService, err := attestations.NewService(ctx, &attestations.Config{Pool: attestations.NewPool()})
|
||||
require.NoError(t, err)
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
fc := doublylinkedtree.New()
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/async/event"
|
||||
mock "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
|
||||
@@ -79,7 +79,7 @@ type testServiceRequirements struct {
|
||||
attPool attestations.Pool
|
||||
attSrv *attestations.Service
|
||||
blsPool *blstoexec.Pool
|
||||
dc *depositcache.DepositCache
|
||||
dc *depositsnapshot.Cache
|
||||
}
|
||||
|
||||
func minimalTestService(t *testing.T, opts ...Option) (*Service, *testServiceRequirements) {
|
||||
@@ -94,7 +94,7 @@ func minimalTestService(t *testing.T, opts ...Option) (*Service, *testServiceReq
|
||||
attSrv, err := attestations.NewService(ctx, &attestations.Config{Pool: attPool})
|
||||
require.NoError(t, err)
|
||||
blsPool := blstoexec.NewPool()
|
||||
dc, err := depositcache.New()
|
||||
dc, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
req := &testServiceRequirements{
|
||||
ctx: ctx,
|
||||
|
||||
50
beacon-chain/cache/depositcache/BUILD.bazel
vendored
50
beacon-chain/cache/depositcache/BUILD.bazel
vendored
@@ -1,50 +0,0 @@
|
||||
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
|
||||
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
srcs = [
|
||||
"deposits_cache.go",
|
||||
"log.go",
|
||||
"pending_deposits.go",
|
||||
],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache",
|
||||
visibility = [
|
||||
"//beacon-chain:__subpackages__",
|
||||
"//testing/spectest:__subpackages__",
|
||||
],
|
||||
deps = [
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//container/trie:go_default_library",
|
||||
"//crypto/hash:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common:go_default_library",
|
||||
"@com_github_pkg_errors//:go_default_library",
|
||||
"@com_github_prometheus_client_golang//prometheus:go_default_library",
|
||||
"@com_github_prometheus_client_golang//prometheus/promauto:go_default_library",
|
||||
"@com_github_sirupsen_logrus//:go_default_library",
|
||||
"@io_opencensus_go//trace:go_default_library",
|
||||
],
|
||||
)
|
||||
|
||||
go_test(
|
||||
name = "go_default_test",
|
||||
srcs = [
|
||||
"deposits_cache_test.go",
|
||||
"pending_deposits_test.go",
|
||||
],
|
||||
embed = [":go_default_library"],
|
||||
deps = [
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//container/trie:go_default_library",
|
||||
"//encoding/bytesutil:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//testing/assert:go_default_library",
|
||||
"//testing/require:go_default_library",
|
||||
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
|
||||
"@org_golang_google_protobuf//proto:go_default_library",
|
||||
],
|
||||
)
|
||||
327
beacon-chain/cache/depositcache/deposits_cache.go
vendored
327
beacon-chain/cache/depositcache/deposits_cache.go
vendored
@@ -1,327 +0,0 @@
|
||||
// Package depositcache is the source of validator deposits maintained
|
||||
// in-memory by the beacon node – deposits processed from the
|
||||
// eth1 powchain are then stored in this cache to be accessed by
|
||||
// any other service during a beacon node's runtime.
|
||||
package depositcache
|
||||
|
||||
import (
|
||||
"context"
|
||||
"encoding/hex"
|
||||
"math/big"
|
||||
"sort"
|
||||
"sync"
|
||||
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v5/container/trie"
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/sirupsen/logrus"
|
||||
"go.opencensus.io/trace"
|
||||
)
|
||||
|
||||
var (
|
||||
historicalDepositsCount = promauto.NewCounter(prometheus.CounterOpts{
|
||||
Name: "beacondb_all_deposits",
|
||||
Help: "The number of total deposits in the beaconDB in-memory database",
|
||||
})
|
||||
)
|
||||
|
||||
// FinalizedDeposits stores the trie of deposits that have been included
|
||||
// in the beacon state up to the latest finalized checkpoint.
|
||||
type FinalizedDeposits struct {
|
||||
deposits *trie.SparseMerkleTrie
|
||||
merkleTrieIndex int64
|
||||
}
|
||||
|
||||
// DepositCache stores all in-memory deposit objects. This
|
||||
// stores all the deposit related data that is required by the beacon-node.
|
||||
type DepositCache struct {
|
||||
// Beacon chain deposits in memory.
|
||||
pendingDeposits []*ethpb.DepositContainer
|
||||
deposits []*ethpb.DepositContainer
|
||||
finalizedDeposits FinalizedDeposits
|
||||
depositsByKey map[[fieldparams.BLSPubkeyLength]byte][]*ethpb.DepositContainer
|
||||
depositsLock sync.RWMutex
|
||||
}
|
||||
|
||||
// New instantiates a new deposit cache
|
||||
func New() (*DepositCache, error) {
|
||||
finalizedDepositsTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// finalizedDeposits.merkleTrieIndex is initialized to -1 because it represents the index of the last trie item.
|
||||
// Inserting the first item into the trie will set the value of the index to 0.
|
||||
return &DepositCache{
|
||||
pendingDeposits: []*ethpb.DepositContainer{},
|
||||
deposits: []*ethpb.DepositContainer{},
|
||||
depositsByKey: map[[fieldparams.BLSPubkeyLength]byte][]*ethpb.DepositContainer{},
|
||||
finalizedDeposits: FinalizedDeposits{deposits: finalizedDepositsTrie, merkleTrieIndex: -1},
|
||||
}, nil
|
||||
}
|
||||
|
||||
// InsertDeposit into the database. If deposit or block number are nil
|
||||
// then this method does nothing.
|
||||
func (dc *DepositCache) InsertDeposit(ctx context.Context, d *ethpb.Deposit, blockNum uint64, index int64, depositRoot [32]byte) error {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.InsertDeposit")
|
||||
defer span.End()
|
||||
if d == nil {
|
||||
log.WithFields(logrus.Fields{
|
||||
"block": blockNum,
|
||||
"deposit": d,
|
||||
"index": index,
|
||||
"depositRoot": hex.EncodeToString(depositRoot[:]),
|
||||
}).Warn("Ignoring nil deposit insertion")
|
||||
return errors.New("nil deposit inserted into the cache")
|
||||
}
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
if int(index) != len(dc.deposits) {
|
||||
return errors.Errorf("wanted deposit with index %d to be inserted but received %d", len(dc.deposits), index)
|
||||
}
|
||||
// Keep the slice sorted on insertion in order to avoid costly sorting on retrieval.
|
||||
heightIdx := sort.Search(len(dc.deposits), func(i int) bool { return dc.deposits[i].Index >= index })
|
||||
depCtr := ðpb.DepositContainer{Deposit: d, Eth1BlockHeight: blockNum, DepositRoot: depositRoot[:], Index: index}
|
||||
newDeposits := append(
|
||||
[]*ethpb.DepositContainer{depCtr},
|
||||
dc.deposits[heightIdx:]...)
|
||||
dc.deposits = append(dc.deposits[:heightIdx], newDeposits...)
|
||||
// Append the deposit to our map, in the event no deposits
|
||||
// exist for the pubkey , it is simply added to the map.
|
||||
pubkey := bytesutil.ToBytes48(d.Data.PublicKey)
|
||||
dc.depositsByKey[pubkey] = append(dc.depositsByKey[pubkey], depCtr)
|
||||
historicalDepositsCount.Inc()
|
||||
return nil
|
||||
}
|
||||
|
||||
// InsertDepositContainers inserts a set of deposit containers into our deposit cache.
|
||||
func (dc *DepositCache) InsertDepositContainers(ctx context.Context, ctrs []*ethpb.DepositContainer) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.InsertDepositContainers")
|
||||
defer span.End()
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
sort.SliceStable(ctrs, func(i int, j int) bool { return ctrs[i].Index < ctrs[j].Index })
|
||||
dc.deposits = ctrs
|
||||
for _, c := range ctrs {
|
||||
// Use a new value, as the reference
|
||||
// of c changes in the next iteration.
|
||||
newPtr := c
|
||||
pKey := bytesutil.ToBytes48(newPtr.Deposit.Data.PublicKey)
|
||||
dc.depositsByKey[pKey] = append(dc.depositsByKey[pKey], newPtr)
|
||||
}
|
||||
historicalDepositsCount.Add(float64(len(ctrs)))
|
||||
}
|
||||
|
||||
// InsertFinalizedDeposits inserts deposits up to eth1DepositIndex (inclusive) into the finalized deposits cache.
|
||||
func (dc *DepositCache) InsertFinalizedDeposits(ctx context.Context,
|
||||
eth1DepositIndex int64, _ common.Hash, _ uint64) error {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.InsertFinalizedDeposits")
|
||||
defer span.End()
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
depositTrie := dc.finalizedDeposits.Deposits()
|
||||
insertIndex := int(dc.finalizedDeposits.merkleTrieIndex + 1)
|
||||
|
||||
// Don't insert into finalized trie if there is no deposit to
|
||||
// insert.
|
||||
if len(dc.deposits) == 0 {
|
||||
return nil
|
||||
}
|
||||
// In the event we have less deposits than we need to
|
||||
// finalize we finalize till the index on which we do have it.
|
||||
if len(dc.deposits) <= int(eth1DepositIndex) {
|
||||
eth1DepositIndex = int64(len(dc.deposits)) - 1
|
||||
}
|
||||
// If we finalize to some lower deposit index, we
|
||||
// ignore it.
|
||||
if int(eth1DepositIndex) < insertIndex {
|
||||
return nil
|
||||
}
|
||||
for _, d := range dc.deposits {
|
||||
if d.Index <= dc.finalizedDeposits.merkleTrieIndex {
|
||||
continue
|
||||
}
|
||||
if d.Index > eth1DepositIndex {
|
||||
break
|
||||
}
|
||||
depHash, err := d.Deposit.Data.HashTreeRoot()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not hash deposit data")
|
||||
}
|
||||
if err = depositTrie.Insert(depHash[:], insertIndex); err != nil {
|
||||
return errors.Wrap(err, "could not insert deposit hash")
|
||||
}
|
||||
insertIndex++
|
||||
}
|
||||
tree, ok := depositTrie.(*trie.SparseMerkleTrie)
|
||||
if !ok {
|
||||
return errors.New("not a sparse merkle tree")
|
||||
}
|
||||
dc.finalizedDeposits = FinalizedDeposits{
|
||||
deposits: tree,
|
||||
merkleTrieIndex: eth1DepositIndex,
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// AllDepositContainers returns all historical deposit containers.
|
||||
func (dc *DepositCache) AllDepositContainers(ctx context.Context) []*ethpb.DepositContainer {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.AllDepositContainers")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
// Make a shallow copy of the deposits and return that. This way, the
|
||||
// caller can safely iterate over the returned list of deposits without
|
||||
// the possibility of new deposits showing up. If we were to return the
|
||||
// list without a copy, when a new deposit is added to the cache, it
|
||||
// would also be present in the returned value. This could result in a
|
||||
// race condition if the list is being iterated over.
|
||||
//
|
||||
// It's not necessary to make a deep copy of this list because the
|
||||
// deposits in the cache should never be modified. It is still possible
|
||||
// for the caller to modify one of the underlying deposits and modify
|
||||
// the cache, but that's not a race condition. Also, a deep copy would
|
||||
// take too long and use too much memory.
|
||||
deposits := make([]*ethpb.DepositContainer, len(dc.deposits))
|
||||
copy(deposits, dc.deposits)
|
||||
return deposits
|
||||
}
|
||||
|
||||
// AllDeposits returns a list of historical deposits until the given block number
|
||||
// (inclusive). If no block is specified then this method returns all historical deposits.
|
||||
func (dc *DepositCache) AllDeposits(ctx context.Context, untilBlk *big.Int) []*ethpb.Deposit {
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
return dc.allDeposits(untilBlk)
|
||||
}
|
||||
|
||||
func (dc *DepositCache) allDeposits(untilBlk *big.Int) []*ethpb.Deposit {
|
||||
var deposits []*ethpb.Deposit
|
||||
for _, ctnr := range dc.deposits {
|
||||
if untilBlk == nil || untilBlk.Uint64() >= ctnr.Eth1BlockHeight {
|
||||
deposits = append(deposits, ctnr.Deposit)
|
||||
}
|
||||
}
|
||||
return deposits
|
||||
}
|
||||
|
||||
// DepositsNumberAndRootAtHeight returns number of deposits made up to blockheight and the
|
||||
// root that corresponds to the latest deposit at that blockheight.
|
||||
func (dc *DepositCache) DepositsNumberAndRootAtHeight(ctx context.Context, blockHeight *big.Int) (uint64, [32]byte) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.DepositsNumberAndRootAtHeight")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
heightIdx := sort.Search(len(dc.deposits), func(i int) bool { return dc.deposits[i].Eth1BlockHeight > blockHeight.Uint64() })
|
||||
// send the deposit root of the empty trie, if eth1follow distance is greater than the time of the earliest
|
||||
// deposit.
|
||||
if heightIdx == 0 {
|
||||
return 0, [32]byte{}
|
||||
}
|
||||
return uint64(heightIdx), bytesutil.ToBytes32(dc.deposits[heightIdx-1].DepositRoot)
|
||||
}
|
||||
|
||||
// DepositByPubkey looks through historical deposits and finds one which contains
|
||||
// a certain public key within its deposit data.
|
||||
func (dc *DepositCache) DepositByPubkey(ctx context.Context, pubKey []byte) (*ethpb.Deposit, *big.Int) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.DepositByPubkey")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
var deposit *ethpb.Deposit
|
||||
var blockNum *big.Int
|
||||
deps, ok := dc.depositsByKey[bytesutil.ToBytes48(pubKey)]
|
||||
if !ok || len(deps) == 0 {
|
||||
return deposit, blockNum
|
||||
}
|
||||
// We always return the first deposit if a particular
|
||||
// validator key has multiple deposits assigned to
|
||||
// it.
|
||||
deposit = deps[0].Deposit
|
||||
blockNum = big.NewInt(int64(deps[0].Eth1BlockHeight))
|
||||
return deposit, blockNum
|
||||
}
|
||||
|
||||
// FinalizedDeposits returns the finalized deposits trie.
|
||||
func (dc *DepositCache) FinalizedDeposits(ctx context.Context) (cache.FinalizedDeposits, error) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.FinalizedDeposits")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
return &FinalizedDeposits{
|
||||
deposits: dc.finalizedDeposits.deposits.Copy(),
|
||||
merkleTrieIndex: dc.finalizedDeposits.merkleTrieIndex,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// NonFinalizedDeposits returns the list of non-finalized deposits until the given block number (inclusive).
|
||||
// If no block is specified then this method returns all non-finalized deposits.
|
||||
func (dc *DepositCache) NonFinalizedDeposits(ctx context.Context, lastFinalizedIndex int64, untilBlk *big.Int) []*ethpb.Deposit {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.NonFinalizedDeposits")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
if dc.finalizedDeposits.Deposits() == nil {
|
||||
return dc.allDeposits(untilBlk)
|
||||
}
|
||||
|
||||
var deposits []*ethpb.Deposit
|
||||
for _, d := range dc.deposits {
|
||||
if (d.Index > lastFinalizedIndex) && (untilBlk == nil || untilBlk.Uint64() >= d.Eth1BlockHeight) {
|
||||
deposits = append(deposits, d.Deposit)
|
||||
}
|
||||
}
|
||||
|
||||
return deposits
|
||||
}
|
||||
|
||||
// PruneProofs removes proofs from all deposits whose index is equal or less than untilDepositIndex.
|
||||
func (dc *DepositCache) PruneProofs(ctx context.Context, untilDepositIndex int64) error {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.PruneProofs")
|
||||
defer span.End()
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
if untilDepositIndex >= int64(len(dc.deposits)) {
|
||||
untilDepositIndex = int64(len(dc.deposits) - 1)
|
||||
}
|
||||
|
||||
for i := untilDepositIndex; i >= 0; i-- {
|
||||
// Finding a nil proof means that all proofs up to this deposit have been already pruned.
|
||||
if dc.deposits[i].Deposit.Proof == nil {
|
||||
break
|
||||
}
|
||||
dc.deposits[i].Deposit.Proof = nil
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
// Deposits returns the cached internal deposit tree.
|
||||
func (fd *FinalizedDeposits) Deposits() cache.MerkleTree {
|
||||
if fd.deposits != nil {
|
||||
return fd.deposits
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// MerkleTrieIndex represents the last finalized index in
|
||||
// the finalized deposit container.
|
||||
func (fd *FinalizedDeposits) MerkleTrieIndex() int64 {
|
||||
return fd.merkleTrieIndex
|
||||
}
|
||||
1074
beacon-chain/cache/depositcache/deposits_cache_test.go
vendored
1074
beacon-chain/cache/depositcache/deposits_cache_test.go
vendored
File diff suppressed because it is too large
Load Diff
5
beacon-chain/cache/depositcache/log.go
vendored
5
beacon-chain/cache/depositcache/log.go
vendored
@@ -1,5 +0,0 @@
|
||||
package depositcache
|
||||
|
||||
import "github.com/sirupsen/logrus"
|
||||
|
||||
var log = logrus.WithField("prefix", "depositcache")
|
||||
151
beacon-chain/cache/depositcache/pending_deposits.go
vendored
151
beacon-chain/cache/depositcache/pending_deposits.go
vendored
@@ -1,151 +0,0 @@
|
||||
package depositcache
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/big"
|
||||
"sort"
|
||||
|
||||
"github.com/prometheus/client_golang/prometheus"
|
||||
"github.com/prometheus/client_golang/prometheus/promauto"
|
||||
"github.com/prysmaticlabs/prysm/v5/crypto/hash"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/sirupsen/logrus"
|
||||
"go.opencensus.io/trace"
|
||||
)
|
||||
|
||||
var (
|
||||
pendingDepositsCount = promauto.NewGauge(prometheus.GaugeOpts{
|
||||
Name: "beacondb_pending_deposits",
|
||||
Help: "The number of pending deposits in the beaconDB in-memory database",
|
||||
})
|
||||
)
|
||||
|
||||
// PendingDepositsFetcher specifically outlines a struct that can retrieve deposits
|
||||
// which have not yet been included in the chain.
|
||||
type PendingDepositsFetcher interface {
|
||||
PendingContainers(ctx context.Context, untilBlk *big.Int) []*ethpb.DepositContainer
|
||||
}
|
||||
|
||||
// InsertPendingDeposit into the database. If deposit or block number are nil
|
||||
// then this method does nothing.
|
||||
func (dc *DepositCache) InsertPendingDeposit(ctx context.Context, d *ethpb.Deposit, blockNum uint64, index int64, depositRoot [32]byte) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.InsertPendingDeposit")
|
||||
defer span.End()
|
||||
if d == nil {
|
||||
log.WithFields(logrus.Fields{
|
||||
"block": blockNum,
|
||||
"deposit": d,
|
||||
}).Debug("Ignoring nil deposit insertion")
|
||||
return
|
||||
}
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
dc.pendingDeposits = append(dc.pendingDeposits,
|
||||
ðpb.DepositContainer{Deposit: d, Eth1BlockHeight: blockNum, Index: index, DepositRoot: depositRoot[:]})
|
||||
pendingDepositsCount.Inc()
|
||||
span.AddAttributes(trace.Int64Attribute("count", int64(len(dc.pendingDeposits))))
|
||||
}
|
||||
|
||||
// PendingDeposits returns a list of deposits until the given block number
|
||||
// (inclusive). If no block is specified then this method returns all pending
|
||||
// deposits.
|
||||
func (dc *DepositCache) PendingDeposits(ctx context.Context, untilBlk *big.Int) []*ethpb.Deposit {
|
||||
ctx, span := trace.StartSpan(ctx, "DepositsCache.PendingDeposits")
|
||||
defer span.End()
|
||||
|
||||
depositCntrs := dc.PendingContainers(ctx, untilBlk)
|
||||
|
||||
deposits := make([]*ethpb.Deposit, 0, len(depositCntrs))
|
||||
for _, dep := range depositCntrs {
|
||||
deposits = append(deposits, dep.Deposit)
|
||||
}
|
||||
|
||||
return deposits
|
||||
}
|
||||
|
||||
// PendingContainers returns a list of deposit containers until the given block number
|
||||
// (inclusive).
|
||||
func (dc *DepositCache) PendingContainers(ctx context.Context, untilBlk *big.Int) []*ethpb.DepositContainer {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.PendingDeposits")
|
||||
defer span.End()
|
||||
dc.depositsLock.RLock()
|
||||
defer dc.depositsLock.RUnlock()
|
||||
|
||||
depositCntrs := make([]*ethpb.DepositContainer, 0, len(dc.pendingDeposits))
|
||||
for _, ctnr := range dc.pendingDeposits {
|
||||
if untilBlk == nil || untilBlk.Uint64() >= ctnr.Eth1BlockHeight {
|
||||
depositCntrs = append(depositCntrs, ctnr)
|
||||
}
|
||||
}
|
||||
// Sort the deposits by Merkle index.
|
||||
sort.SliceStable(depositCntrs, func(i, j int) bool {
|
||||
return depositCntrs[i].Index < depositCntrs[j].Index
|
||||
})
|
||||
|
||||
span.AddAttributes(trace.Int64Attribute("count", int64(len(depositCntrs))))
|
||||
|
||||
return depositCntrs
|
||||
}
|
||||
|
||||
// RemovePendingDeposit from the database. The deposit is indexed by the
|
||||
// Index. This method does nothing if deposit ptr is nil.
|
||||
func (dc *DepositCache) RemovePendingDeposit(ctx context.Context, d *ethpb.Deposit) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.RemovePendingDeposit")
|
||||
defer span.End()
|
||||
|
||||
if d == nil {
|
||||
log.Debug("Ignoring nil deposit removal")
|
||||
return
|
||||
}
|
||||
|
||||
depRoot, err := hash.Proto(d)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not remove deposit")
|
||||
return
|
||||
}
|
||||
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
idx := -1
|
||||
for i, ctnr := range dc.pendingDeposits {
|
||||
h, err := hash.Proto(ctnr.Deposit)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("Could not hash deposit")
|
||||
continue
|
||||
}
|
||||
if h == depRoot {
|
||||
idx = i
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
if idx >= 0 {
|
||||
dc.pendingDeposits = append(dc.pendingDeposits[:idx], dc.pendingDeposits[idx+1:]...)
|
||||
pendingDepositsCount.Dec()
|
||||
}
|
||||
}
|
||||
|
||||
// PrunePendingDeposits removes any deposit which is older than the given deposit merkle tree index.
|
||||
func (dc *DepositCache) PrunePendingDeposits(ctx context.Context, merkleTreeIndex int64) {
|
||||
_, span := trace.StartSpan(ctx, "DepositsCache.PrunePendingDeposits")
|
||||
defer span.End()
|
||||
|
||||
if merkleTreeIndex == 0 {
|
||||
log.Debug("Ignoring 0 deposit removal")
|
||||
return
|
||||
}
|
||||
|
||||
dc.depositsLock.Lock()
|
||||
defer dc.depositsLock.Unlock()
|
||||
|
||||
cleanDeposits := make([]*ethpb.DepositContainer, 0, len(dc.pendingDeposits))
|
||||
for _, dp := range dc.pendingDeposits {
|
||||
if dp.Index >= merkleTreeIndex {
|
||||
cleanDeposits = append(cleanDeposits, dp)
|
||||
}
|
||||
}
|
||||
|
||||
dc.pendingDeposits = cleanDeposits
|
||||
pendingDepositsCount.Set(float64(len(dc.pendingDeposits)))
|
||||
}
|
||||
@@ -34,6 +34,7 @@ go_test(
|
||||
name = "go_default_test",
|
||||
srcs = [
|
||||
"deposit_cache_test.go",
|
||||
"deposit_fetcher_test.go",
|
||||
"deposit_tree_snapshot_test.go",
|
||||
"merkle_tree_test.go",
|
||||
"spec_test.go",
|
||||
|
||||
@@ -262,6 +262,12 @@ func toFinalizedDepositsContainer(deposits *DepositTree, index int64) finalizedD
|
||||
}
|
||||
}
|
||||
|
||||
// PendingDepositsFetcher specifically outlines a struct that can retrieve deposits
|
||||
// which have not yet been included in the chain.
|
||||
type PendingDepositsFetcher interface {
|
||||
PendingContainers(ctx context.Context, untilBlk *big.Int) []*ethpb.DepositContainer
|
||||
}
|
||||
|
||||
// PendingDeposits returns a list of deposits until the given block number
|
||||
// (inclusive). If no block is specified then this method returns all pending
|
||||
// deposits.
|
||||
|
||||
@@ -1,82 +1,32 @@
|
||||
package depositcache
|
||||
package depositsnapshot
|
||||
|
||||
import (
|
||||
"context"
|
||||
"math/big"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/assert"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
var _ PendingDepositsFetcher = (*DepositCache)(nil)
|
||||
var _ PendingDepositsFetcher = (*Cache)(nil)
|
||||
|
||||
func TestInsertPendingDeposit_OK(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc := Cache{}
|
||||
dc.InsertPendingDeposit(context.Background(), ðpb.Deposit{}, 111, 100, [32]byte{})
|
||||
|
||||
assert.Equal(t, 1, len(dc.pendingDeposits), "deposit not inserted")
|
||||
}
|
||||
|
||||
func TestInsertPendingDeposit_ignoresNilDeposit(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc := Cache{}
|
||||
dc.InsertPendingDeposit(context.Background(), nil /*deposit*/, 0 /*blockNum*/, 0, [32]byte{})
|
||||
|
||||
assert.Equal(t, 0, len(dc.pendingDeposits))
|
||||
}
|
||||
|
||||
func TestRemovePendingDeposit_OK(t *testing.T) {
|
||||
db := DepositCache{}
|
||||
proof1 := makeDepositProof()
|
||||
proof1[0] = bytesutil.PadTo([]byte{'A'}, 32)
|
||||
proof2 := makeDepositProof()
|
||||
proof2[0] = bytesutil.PadTo([]byte{'A'}, 32)
|
||||
data := ðpb.Deposit_Data{
|
||||
PublicKey: make([]byte, 48),
|
||||
WithdrawalCredentials: make([]byte, 32),
|
||||
Amount: 0,
|
||||
Signature: make([]byte, 96),
|
||||
}
|
||||
depToRemove := ðpb.Deposit{Proof: proof1, Data: data}
|
||||
otherDep := ðpb.Deposit{Proof: proof2, Data: data}
|
||||
db.pendingDeposits = []*ethpb.DepositContainer{
|
||||
{Deposit: depToRemove, Index: 1},
|
||||
{Deposit: otherDep, Index: 5},
|
||||
}
|
||||
db.RemovePendingDeposit(context.Background(), depToRemove)
|
||||
|
||||
if len(db.pendingDeposits) != 1 || !proto.Equal(db.pendingDeposits[0].Deposit, otherDep) {
|
||||
t.Error("Failed to remove deposit")
|
||||
}
|
||||
}
|
||||
|
||||
func TestRemovePendingDeposit_IgnoresNilDeposit(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc.pendingDeposits = []*ethpb.DepositContainer{{Deposit: ðpb.Deposit{}}}
|
||||
dc.RemovePendingDeposit(context.Background(), nil /*deposit*/)
|
||||
assert.Equal(t, 1, len(dc.pendingDeposits), "deposit unexpectedly removed")
|
||||
}
|
||||
|
||||
func TestPendingDeposit_RoundTrip(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
proof := makeDepositProof()
|
||||
proof[0] = bytesutil.PadTo([]byte{'A'}, 32)
|
||||
data := ðpb.Deposit_Data{
|
||||
PublicKey: make([]byte, 48),
|
||||
WithdrawalCredentials: make([]byte, 32),
|
||||
Amount: 0,
|
||||
Signature: make([]byte, 96),
|
||||
}
|
||||
dep := ðpb.Deposit{Proof: proof, Data: data}
|
||||
dc.InsertPendingDeposit(context.Background(), dep, 111, 100, [32]byte{})
|
||||
dc.RemovePendingDeposit(context.Background(), dep)
|
||||
assert.Equal(t, 0, len(dc.pendingDeposits), "Failed to insert & delete a pending deposit")
|
||||
}
|
||||
|
||||
func TestPendingDeposits_OK(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc := Cache{}
|
||||
|
||||
dc.pendingDeposits = []*ethpb.DepositContainer{
|
||||
{Eth1BlockHeight: 2, Deposit: ðpb.Deposit{Proof: [][]byte{[]byte("A")}}},
|
||||
@@ -96,7 +46,7 @@ func TestPendingDeposits_OK(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestPrunePendingDeposits_ZeroMerkleIndex(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc := Cache{}
|
||||
|
||||
dc.pendingDeposits = []*ethpb.DepositContainer{
|
||||
{Eth1BlockHeight: 2, Index: 2},
|
||||
@@ -120,7 +70,7 @@ func TestPrunePendingDeposits_ZeroMerkleIndex(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestPrunePendingDeposits_OK(t *testing.T) {
|
||||
dc := DepositCache{}
|
||||
dc := Cache{}
|
||||
|
||||
dc.pendingDeposits = []*ethpb.DepositContainer{
|
||||
{Eth1BlockHeight: 2, Index: 2},
|
||||
@@ -99,11 +99,23 @@ func (d *DepositTree) getProof(index uint64) ([32]byte, [][32]byte, error) {
|
||||
if d.depositCount <= 0 {
|
||||
return [32]byte{}, nil, ErrInvalidDepositCount
|
||||
}
|
||||
finalizedDeposits, _ := d.tree.GetFinalized([][32]byte{})
|
||||
if finalizedDeposits != 0 {
|
||||
finalizedDeposits = finalizedDeposits - 1
|
||||
if index >= d.depositCount {
|
||||
return [32]byte{}, nil, ErrInvalidIndex
|
||||
}
|
||||
if index <= finalizedDeposits {
|
||||
finalizedDeposits, _ := d.tree.GetFinalized([][32]byte{})
|
||||
finalizedIdx := -1
|
||||
if finalizedDeposits != 0 {
|
||||
fd, err := math.Int(finalizedDeposits)
|
||||
if err != nil {
|
||||
return [32]byte{}, nil, err
|
||||
}
|
||||
finalizedIdx = fd - 1
|
||||
}
|
||||
i, err := math.Int(index)
|
||||
if err != nil {
|
||||
return [32]byte{}, nil, err
|
||||
}
|
||||
if finalizedDeposits > 0 && i <= finalizedIdx {
|
||||
return [32]byte{}, nil, ErrInvalidIndex
|
||||
}
|
||||
leaf, proof := generateProof(d.tree, index, DepositContractDepth)
|
||||
|
||||
@@ -36,7 +36,6 @@ go_library(
|
||||
"//beacon-chain/state:go_default_library",
|
||||
"//beacon-chain/state/state-native:go_default_library",
|
||||
"//beacon-chain/state/stategen:go_default_library",
|
||||
"//config/features:go_default_library",
|
||||
"//config/fieldparams:go_default_library",
|
||||
"//config/params:go_default_library",
|
||||
"//consensus-types/blocks:go_default_library",
|
||||
@@ -94,7 +93,7 @@ go_test(
|
||||
embed = [":go_default_library"],
|
||||
deps = [
|
||||
"//async/event:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/feed:go_default_library",
|
||||
"//beacon-chain/core/feed/state:go_default_library",
|
||||
"//beacon-chain/core/helpers:go_default_library",
|
||||
|
||||
@@ -20,9 +20,7 @@ import (
|
||||
coreState "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/types"
|
||||
statenative "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/features"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v5/container/trie"
|
||||
contracts "github.com/prysmaticlabs/prysm/v5/contracts/deposit"
|
||||
"github.com/prysmaticlabs/prysm/v5/crypto/hash"
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
@@ -226,16 +224,14 @@ func (s *Service) ProcessDepositLog(ctx context.Context, depositLog *gethtypes.L
|
||||
"merkleTreeIndex": index,
|
||||
}).Info("Invalid deposit registered in deposit contract")
|
||||
}
|
||||
if features.Get().EnableEIP4881 {
|
||||
// We finalize the trie here so that old deposits are not kept around, as they make
|
||||
// deposit tree htr computation expensive.
|
||||
dTrie, ok := s.depositTrie.(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return errors.Errorf("wrong trie type initialized: %T", dTrie)
|
||||
}
|
||||
if err := dTrie.Finalize(index, depositLog.BlockHash, depositLog.BlockNumber); err != nil {
|
||||
log.WithError(err).Error("Could not finalize trie")
|
||||
}
|
||||
// We finalize the trie here so that old deposits are not kept around, as they make
|
||||
// deposit tree htr computation expensive.
|
||||
dTrie, ok := s.depositTrie.(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return errors.Errorf("wrong trie type initialized: %T", dTrie)
|
||||
}
|
||||
if err := dTrie.Finalize(index, depositLog.BlockHash, depositLog.BlockNumber); err != nil {
|
||||
log.WithError(err).Error("Could not finalize trie")
|
||||
}
|
||||
|
||||
return nil
|
||||
@@ -579,25 +575,17 @@ func (s *Service) savePowchainData(ctx context.Context) error {
|
||||
BeaconState: pbState, // I promise not to mutate it!
|
||||
DepositContainers: s.cfg.depositCache.AllDepositContainers(ctx),
|
||||
}
|
||||
if features.Get().EnableEIP4881 {
|
||||
fd, err := s.cfg.depositCache.FinalizedDeposits(ctx)
|
||||
if err != nil {
|
||||
return errors.Errorf("could not get finalized deposit tree: %v", err)
|
||||
}
|
||||
tree, ok := fd.Deposits().(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return errors.New("deposit tree was not EIP4881 DepositTree")
|
||||
}
|
||||
eth1Data.DepositSnapshot, err = tree.ToProto()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
tree, ok := s.depositTrie.(*trie.SparseMerkleTrie)
|
||||
if !ok {
|
||||
return errors.New("deposit tree was not SparseMerkleTrie")
|
||||
}
|
||||
eth1Data.Trie = tree.ToProto()
|
||||
fd, err := s.cfg.depositCache.FinalizedDeposits(ctx)
|
||||
if err != nil {
|
||||
return errors.Errorf("could not get finalized deposit tree: %v", err)
|
||||
}
|
||||
tree, ok := fd.Deposits().(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return errors.New("deposit tree was not EIP4881 DepositTree")
|
||||
}
|
||||
eth1Data.DepositSnapshot, err = tree.ToProto()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.cfg.beaconDB.SaveExecutionChainData(ctx, eth1Data)
|
||||
}
|
||||
|
||||
@@ -9,7 +9,7 @@ import (
|
||||
|
||||
"github.com/ethereum/go-ethereum"
|
||||
"github.com/ethereum/go-ethereum/common"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
|
||||
@@ -31,7 +31,7 @@ func TestProcessDepositLog_OK(t *testing.T) {
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
|
||||
beaconDB := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
@@ -100,7 +100,7 @@ func TestProcessDepositLog_InsertsPendingDeposit(t *testing.T) {
|
||||
testAcc, err := mock.Setup()
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
beaconDB := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -216,7 +216,7 @@ func TestProcessETH2GenesisLog_8DuplicatePubkeys(t *testing.T) {
|
||||
testAcc, err := mock.Setup()
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
beaconDB := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -291,7 +291,7 @@ func TestProcessETH2GenesisLog(t *testing.T) {
|
||||
testAcc, err := mock.Setup()
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
beaconDB := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
@@ -384,7 +384,7 @@ func TestProcessETH2GenesisLog_CorrectNumOfDeposits(t *testing.T) {
|
||||
testAcc, err := mock.Setup()
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
kvStore := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -481,7 +481,7 @@ func TestProcessETH2GenesisLog_LargePeriodOfNoLogs(t *testing.T) {
|
||||
testAcc, err := mock.Setup()
|
||||
require.NoError(t, err, "Unable to set up simulated backend")
|
||||
kvStore := testDB.SetupDB(t)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -593,7 +593,7 @@ func TestCheckForChainstart_NoValidator(t *testing.T) {
|
||||
}
|
||||
|
||||
func newPowchainService(t *testing.T, eth1Backend *mock.TestAccount, beaconDB db.Database) *Service {
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
server, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
|
||||
@@ -29,7 +29,6 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
|
||||
native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/features"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v5/container/trie"
|
||||
contracts "github.com/prysmaticlabs/prysm/v5/contracts/deposit"
|
||||
@@ -164,14 +163,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
|
||||
_ = cancel // govet fix for lost cancel. Cancel is handled in service.Stop()
|
||||
var depositTrie cache.MerkleTree
|
||||
var err error
|
||||
if features.Get().EnableEIP4881 {
|
||||
depositTrie = depositsnapshot.NewDepositTree()
|
||||
} else {
|
||||
depositTrie, err = trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not set up deposit trie")
|
||||
}
|
||||
}
|
||||
depositTrie = depositsnapshot.NewDepositTree()
|
||||
genState, err := transition.EmptyGenesisState()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not set up genesis state")
|
||||
@@ -740,20 +732,12 @@ func (s *Service) initializeEth1Data(ctx context.Context, eth1DataInDB *ethpb.ET
|
||||
return nil
|
||||
}
|
||||
var err error
|
||||
if features.Get().EnableEIP4881 {
|
||||
if eth1DataInDB.DepositSnapshot != nil {
|
||||
s.depositTrie, err = depositsnapshot.DepositTreeFromSnapshotProto(eth1DataInDB.DepositSnapshot)
|
||||
} else {
|
||||
if err := s.migrateOldDepositTree(eth1DataInDB); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
if eth1DataInDB.DepositSnapshot != nil {
|
||||
s.depositTrie, err = depositsnapshot.DepositTreeFromSnapshotProto(eth1DataInDB.DepositSnapshot)
|
||||
} else {
|
||||
if eth1DataInDB.Trie == nil && eth1DataInDB.DepositSnapshot != nil {
|
||||
return errors.Errorf("trying to use old deposit trie after migration to the new trie. "+
|
||||
"Remove the --%s flag to resume normal operations.", features.DisableEIP4881.Name)
|
||||
if err = s.migrateOldDepositTree(eth1DataInDB); err != nil {
|
||||
return err
|
||||
}
|
||||
s.depositTrie, err = trie.CreateTrieFromProto(eth1DataInDB.Trie)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
@@ -766,21 +750,19 @@ func (s *Service) initializeEth1Data(ctx context.Context, eth1DataInDB *ethpb.ET
|
||||
}
|
||||
}
|
||||
s.latestEth1Data = eth1DataInDB.CurrentEth1Data
|
||||
if features.Get().EnableEIP4881 {
|
||||
ctrs := eth1DataInDB.DepositContainers
|
||||
// Look at previously finalized index, as we are building off a finalized
|
||||
// snapshot rather than the full trie.
|
||||
lastFinalizedIndex := int64(s.depositTrie.NumOfItems() - 1)
|
||||
// Correctly initialize missing deposits into active trie.
|
||||
for _, c := range ctrs {
|
||||
if c.Index > lastFinalizedIndex {
|
||||
depRoot, err := c.Deposit.Data.HashTreeRoot()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.depositTrie.Insert(depRoot[:], int(c.Index)); err != nil {
|
||||
return err
|
||||
}
|
||||
ctrs := eth1DataInDB.DepositContainers
|
||||
// Look at previously finalized index, as we are building off a finalized
|
||||
// snapshot rather than the full trie.
|
||||
lastFinalizedIndex := int64(s.depositTrie.NumOfItems() - 1)
|
||||
// Correctly initialize missing deposits into active trie.
|
||||
for _, c := range ctrs {
|
||||
if c.Index > lastFinalizedIndex {
|
||||
depRoot, err := c.Deposit.Data.HashTreeRoot()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
if err := s.depositTrie.Insert(depRoot[:], int(c.Index)); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
@@ -847,21 +829,13 @@ func (s *Service) validPowchainData(ctx context.Context) (*ethpb.ETH1ChainData,
|
||||
BeaconState: pbState,
|
||||
DepositContainers: s.cfg.depositCache.AllDepositContainers(ctx),
|
||||
}
|
||||
if features.Get().EnableEIP4881 {
|
||||
trie, ok := s.depositTrie.(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return nil, errors.New("deposit trie was not EIP4881 DepositTree")
|
||||
}
|
||||
eth1Data.DepositSnapshot, err = trie.ToProto()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
} else {
|
||||
trie, ok := s.depositTrie.(*trie.SparseMerkleTrie)
|
||||
if !ok {
|
||||
return nil, errors.New("deposit trie was not SparseMerkleTrie")
|
||||
}
|
||||
eth1Data.Trie = trie.ToProto()
|
||||
trie, ok := s.depositTrie.(*depositsnapshot.DepositTree)
|
||||
if !ok {
|
||||
return nil, errors.New("deposit trie was not EIP4881 DepositTree")
|
||||
}
|
||||
eth1Data.DepositSnapshot, err = trie.ToProto()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if err := s.cfg.beaconDB.SaveExecutionChainData(ctx, eth1Data); err != nil {
|
||||
return nil, err
|
||||
|
||||
@@ -15,7 +15,7 @@ import (
|
||||
"github.com/ethereum/go-ethereum/rpc"
|
||||
"github.com/pkg/errors"
|
||||
"github.com/prysmaticlabs/prysm/v5/async/event"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
dbutil "github.com/prysmaticlabs/prysm/v5/beacon-chain/db/testing"
|
||||
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/types"
|
||||
@@ -348,7 +348,7 @@ func TestInitDepositCache_OK(t *testing.T) {
|
||||
cfg: &config{beaconDB: beaconDB},
|
||||
}
|
||||
var err error
|
||||
s.cfg.depositCache, err = depositcache.New()
|
||||
s.cfg.depositCache, err = depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, s.initDepositCaches(context.Background(), ctrs))
|
||||
|
||||
@@ -409,7 +409,7 @@ func TestInitDepositCacheWithFinalization_OK(t *testing.T) {
|
||||
cfg: &config{beaconDB: beaconDB},
|
||||
}
|
||||
var err error
|
||||
s.cfg.depositCache, err = depositcache.New()
|
||||
s.cfg.depositCache, err = depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, s.initDepositCaches(context.Background(), ctrs))
|
||||
|
||||
@@ -553,7 +553,7 @@ func Test_batchRequestHeaders_UnderflowChecks(t *testing.T) {
|
||||
|
||||
func TestService_EnsureConsistentPowchainData(t *testing.T) {
|
||||
beaconDB := dbutil.SetupDB(t)
|
||||
cache, err := depositcache.New()
|
||||
cache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
srv, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -583,7 +583,7 @@ func TestService_EnsureConsistentPowchainData(t *testing.T) {
|
||||
|
||||
func TestService_InitializeCorrectly(t *testing.T) {
|
||||
beaconDB := dbutil.SetupDB(t)
|
||||
cache, err := depositcache.New()
|
||||
cache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
srv, endpoint, err := mockExecution.SetupRPCServer()
|
||||
@@ -614,7 +614,7 @@ func TestService_InitializeCorrectly(t *testing.T) {
|
||||
|
||||
func TestService_EnsureValidPowchainData(t *testing.T) {
|
||||
beaconDB := dbutil.SetupDB(t)
|
||||
cache, err := depositcache.New()
|
||||
cache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
srv, endpoint, err := mockExecution.SetupRPCServer()
|
||||
require.NoError(t, err)
|
||||
@@ -809,7 +809,7 @@ func (s *slowRPCClient) CallContext(_ context.Context, _ interface{}, _ string,
|
||||
|
||||
func TestService_migrateOldDepositTree(t *testing.T) {
|
||||
beaconDB := dbutil.SetupDB(t)
|
||||
cache, err := depositcache.New()
|
||||
cache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
srv, endpoint, err := mockExecution.SetupRPCServer()
|
||||
|
||||
@@ -21,7 +21,6 @@ go_library(
|
||||
"//beacon-chain/blockchain:go_default_library",
|
||||
"//beacon-chain/builder:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/db:go_default_library",
|
||||
"//beacon-chain/db/filesystem:go_default_library",
|
||||
|
||||
@@ -25,7 +25,6 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/builder"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
|
||||
@@ -567,11 +566,7 @@ func (b *BeaconNode) startDB(cliCtx *cli.Context, depositAddress string) error {
|
||||
|
||||
b.db = d
|
||||
|
||||
if features.Get().EnableEIP4881 {
|
||||
depositCache, err = depositsnapshot.New()
|
||||
} else {
|
||||
depositCache, err = depositcache.New()
|
||||
}
|
||||
depositCache, err = depositsnapshot.New()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not create deposit cache")
|
||||
}
|
||||
|
||||
@@ -13,7 +13,7 @@ go_library(
|
||||
"//beacon-chain/blockchain:go_default_library",
|
||||
"//beacon-chain/builder:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/feed/block:go_default_library",
|
||||
"//beacon-chain/core/feed/operation:go_default_library",
|
||||
"//beacon-chain/core/feed/state:go_default_library",
|
||||
|
||||
@@ -37,7 +37,7 @@ go_library(
|
||||
"//beacon-chain/blockchain:go_default_library",
|
||||
"//beacon-chain/builder:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/blocks:go_default_library",
|
||||
"//beacon-chain/core/feed:go_default_library",
|
||||
"//beacon-chain/core/feed/block:go_default_library",
|
||||
@@ -112,7 +112,7 @@ common_deps = [
|
||||
"//beacon-chain/builder:go_default_library",
|
||||
"//beacon-chain/builder/testing:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/altair:go_default_library",
|
||||
"//beacon-chain/core/blocks:go_default_library",
|
||||
"//beacon-chain/core/execution:go_default_library",
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/execution"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
@@ -334,7 +334,7 @@ func TestGetAltairDuties_UnknownPubkey(t *testing.T) {
|
||||
chain := &mockChain.ChainService{
|
||||
State: bs, Root: genesisRoot[:], Genesis: time.Now().Add(time.Duration(-1*int64(slot-1)) * time.Second),
|
||||
}
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
vs := &Server{
|
||||
|
||||
@@ -14,7 +14,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/builder"
|
||||
builderTest "github.com/prysmaticlabs/prysm/v5/beacon-chain/builder/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
b "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
|
||||
@@ -1000,7 +1000,7 @@ func TestProposer_PendingDeposits_OutsideEth1FollowWindow(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
@@ -1139,7 +1139,7 @@ func TestProposer_PendingDeposits_FollowsCorrectEth1Block(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
@@ -1244,7 +1244,7 @@ func TestProposer_PendingDeposits_CantReturnBelowStateEth1DepositIndex(t *testin
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, dp := range append(readyDeposits, recentDeposits...) {
|
||||
@@ -1344,7 +1344,7 @@ func TestProposer_PendingDeposits_CantReturnMoreThanMax(t *testing.T) {
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, dp := range append(readyDeposits, recentDeposits...) {
|
||||
@@ -1442,7 +1442,7 @@ func TestProposer_PendingDeposits_CantReturnMoreThanDepositCount(t *testing.T) {
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, dp := range append(readyDeposits, recentDeposits...) {
|
||||
@@ -1553,7 +1553,7 @@ func TestProposer_DepositTrie_UtilizesCachedFinalizedDeposits(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
@@ -1669,7 +1669,7 @@ func TestProposer_DepositTrie_RebuildTrie(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
@@ -1810,7 +1810,7 @@ func TestProposer_Eth1Data_MajorityVote_SpansGenesis(t *testing.T) {
|
||||
InsertBlock(100, latestValidTime, []byte("latest"))
|
||||
|
||||
headBlockHash := []byte("headb")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
ps := &Server{
|
||||
ChainStartFetcher: p,
|
||||
@@ -1851,7 +1851,7 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
@@ -2351,7 +2351,7 @@ func TestProposer_Eth1Data_MajorityVote(t *testing.T) {
|
||||
BlockHash: []byte("eth1data"),
|
||||
}
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
beaconState, err := state_native.InitializeFromProtoPhase0(ðpb.BeaconState{
|
||||
@@ -2547,7 +2547,7 @@ func TestProposer_Deposits_ReturnsEmptyList_IfLatestEth1DataEqGenesisEth1Block(t
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, dp := range append(readyDeposits, recentDeposits...) {
|
||||
|
||||
@@ -10,7 +10,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/builder"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
|
||||
opfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
@@ -69,7 +69,7 @@ type Server struct {
|
||||
BlobReceiver blockchain.BlobReceiver
|
||||
MockEth1Votes bool
|
||||
Eth1BlockFetcher execution.POWBlockFetcher
|
||||
PendingDepositsFetcher depositcache.PendingDepositsFetcher
|
||||
PendingDepositsFetcher depositsnapshot.PendingDepositsFetcher
|
||||
OperationNotifier opfeed.Notifier
|
||||
StateGen stategen.StateManager
|
||||
ReplayerBuilder stategen.ReplayerBuilder
|
||||
|
||||
@@ -5,7 +5,7 @@ import (
|
||||
"testing"
|
||||
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
|
||||
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
|
||||
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
||||
@@ -65,7 +65,7 @@ func TestWaitForActivation_ValidatorOriginallyExists(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/async/event"
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
|
||||
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
|
||||
@@ -70,7 +70,7 @@ func TestWaitForActivation_ContextClosed(t *testing.T) {
|
||||
require.NoError(t, err, "Could not get signing root")
|
||||
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
vs := &Server{
|
||||
|
||||
@@ -7,7 +7,7 @@ import (
|
||||
"time"
|
||||
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
|
||||
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
||||
@@ -40,7 +40,7 @@ func TestValidatorStatus_Active(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
|
||||
@@ -8,7 +8,7 @@ import (
|
||||
|
||||
"github.com/d4l3k/messagediff"
|
||||
mockChain "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
|
||||
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
|
||||
@@ -38,7 +38,7 @@ func TestValidatorStatus_DepositedEth1(t *testing.T) {
|
||||
pubKey1 := deposit.Data.PublicKey
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -80,7 +80,7 @@ func TestValidatorStatus_Deposited(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -125,7 +125,7 @@ func TestValidatorStatus_PartiallyDeposited(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -178,7 +178,7 @@ func TestValidatorStatus_Pending_MultipleDeposits(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -255,7 +255,7 @@ func TestValidatorStatus_Pending(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -318,7 +318,7 @@ func TestValidatorStatus_Exiting(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -377,7 +377,7 @@ func TestValidatorStatus_Slashing(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -435,7 +435,7 @@ func TestValidatorStatus_Exited(t *testing.T) {
|
||||
}
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -464,7 +464,7 @@ func TestValidatorStatus_Exited(t *testing.T) {
|
||||
|
||||
func TestValidatorStatus_UnknownStatus(t *testing.T) {
|
||||
pubKey := pubKey(1)
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
stateObj, err := state_native.InitializeFromProtoUnsafePhase0(ðpb.BeaconState{
|
||||
@@ -520,7 +520,7 @@ func TestActivationStatus_OK(t *testing.T) {
|
||||
dep := deposits[0]
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
@@ -659,7 +659,7 @@ func TestValidatorStatus_CorrectActivationQueue(t *testing.T) {
|
||||
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
for i := 0; i < 6; i++ {
|
||||
@@ -751,7 +751,7 @@ func TestMultipleValidatorStatus_Pubkeys(t *testing.T) {
|
||||
require.NoError(t, err, "Could not get signing root")
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
dep := deposits[0]
|
||||
@@ -916,7 +916,7 @@ func TestValidatorStatus_Invalid(t *testing.T) {
|
||||
deposit.Data.Signature = deposit.Data.Signature[1:]
|
||||
depositTrie, err := trie.NewTrie(params.BeaconConfig().DepositContractTreeDepth)
|
||||
require.NoError(t, err, "Could not setup deposit trie")
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
root, err := depositTrie.HashTreeRoot()
|
||||
|
||||
@@ -20,7 +20,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/builder"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
|
||||
opfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
@@ -127,7 +127,7 @@ type Config struct {
|
||||
PeerManager p2p.PeerManager
|
||||
MetadataProvider p2p.MetadataProvider
|
||||
DepositFetcher cache.DepositFetcher
|
||||
PendingDepositFetcher depositcache.PendingDepositsFetcher
|
||||
PendingDepositFetcher depositsnapshot.PendingDepositsFetcher
|
||||
StateNotifier statefeed.Notifier
|
||||
BlockNotifier blockfeed.Notifier
|
||||
OperationNotifier opfeed.Notifier
|
||||
|
||||
@@ -68,7 +68,6 @@ type Flags struct {
|
||||
DisableStakinContractCheck bool // Disables check for deposit contract when proposing blocks
|
||||
|
||||
EnableVerboseSigVerification bool // EnableVerboseSigVerification specifies whether to verify individual signature if batch verification fails
|
||||
EnableEIP4881 bool // EnableEIP4881 specifies whether to use the deposit tree from EIP4881
|
||||
|
||||
PrepareAllPayloads bool // PrepareAllPayloads informs the engine to prepare a block on every slot.
|
||||
// BlobSaveFsync requires blob saving to block on fsync to ensure blobs are durably persisted before passing DA.
|
||||
@@ -242,11 +241,6 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
|
||||
logEnabled(disableResourceManager)
|
||||
cfg.DisableResourceManager = true
|
||||
}
|
||||
cfg.EnableEIP4881 = true
|
||||
if ctx.IsSet(DisableEIP4881.Name) {
|
||||
logEnabled(DisableEIP4881)
|
||||
cfg.EnableEIP4881 = false
|
||||
}
|
||||
if ctx.IsSet(EnableLightClient.Name) {
|
||||
logEnabled(EnableLightClient)
|
||||
cfg.EnableLightClient = true
|
||||
|
||||
@@ -42,6 +42,11 @@ var (
|
||||
Usage: deprecatedUsage,
|
||||
Hidden: true,
|
||||
}
|
||||
deprecatedDisableEIP4881 = &cli.BoolFlag{
|
||||
Name: "disable-eip-4881",
|
||||
Usage: deprecatedUsage,
|
||||
Hidden: true,
|
||||
}
|
||||
deprecatedVerboseSigVerification = &cli.BoolFlag{
|
||||
Name: "enable-verbose-sig-verification",
|
||||
Usage: deprecatedUsage,
|
||||
@@ -58,6 +63,7 @@ var deprecatedFlags = []cli.Flag{
|
||||
deprecatedDisableOptionalEngineMethods,
|
||||
deprecatedDisableAggregateParallel,
|
||||
deprecatedEnableEIP4881,
|
||||
deprecatedDisableEIP4881,
|
||||
deprecatedVerboseSigVerification,
|
||||
}
|
||||
|
||||
|
||||
@@ -143,10 +143,6 @@ var (
|
||||
Name: "prepare-all-payloads",
|
||||
Usage: "Informs the engine to prepare all local payloads. Useful for relayers and builders.",
|
||||
}
|
||||
DisableEIP4881 = &cli.BoolFlag{
|
||||
Name: "disable-eip-4881",
|
||||
Usage: "Disables the deposit tree specified in EIP-4881.",
|
||||
}
|
||||
EnableLightClient = &cli.BoolFlag{
|
||||
Name: "enable-lightclient",
|
||||
Usage: "Enables the light client support in the beacon node",
|
||||
@@ -222,7 +218,6 @@ var BeaconChainFlags = append(deprecatedBeaconFlags, append(deprecatedFlags, []c
|
||||
aggregateFirstInterval,
|
||||
aggregateSecondInterval,
|
||||
aggregateThirdInterval,
|
||||
DisableEIP4881,
|
||||
disableResourceManager,
|
||||
DisableRegistrationCache,
|
||||
EnableLightClient,
|
||||
|
||||
@@ -16,7 +16,7 @@ go_library(
|
||||
"//beacon-chain/blockchain/kzg:go_default_library",
|
||||
"//beacon-chain/blockchain/testing:go_default_library",
|
||||
"//beacon-chain/cache:go_default_library",
|
||||
"//beacon-chain/cache/depositcache:go_default_library",
|
||||
"//beacon-chain/cache/depositsnapshot:go_default_library",
|
||||
"//beacon-chain/core/time:go_default_library",
|
||||
"//beacon-chain/core/transition:go_default_library",
|
||||
"//beacon-chain/db/filesystem:go_default_library",
|
||||
|
||||
@@ -12,7 +12,7 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/kzg"
|
||||
mock "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositcache"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache/depositsnapshot"
|
||||
coreTime "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/db/filesystem"
|
||||
testDB "github.com/prysmaticlabs/prysm/v5/beacon-chain/db/testing"
|
||||
@@ -56,7 +56,7 @@ func startChainService(t testing.TB,
|
||||
})
|
||||
require.NoError(t, err)
|
||||
|
||||
depositCache, err := depositcache.New()
|
||||
depositCache, err := depositsnapshot.New()
|
||||
require.NoError(t, err)
|
||||
|
||||
fc := doublylinkedtree.New()
|
||||
|
||||
Reference in New Issue
Block a user