Adds flags to enable/disable backfilling and enable full historical beacon node ()

* Correct naming of hash func in Eth2
* Customizable mode of operation for Caplin
This commit is contained in:
Giulio rebuffo 2023-11-22 13:24:35 +01:00 committed by GitHub
parent 30c05cc7a9
commit 57bcbaa21f
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
29 changed files with 93 additions and 78 deletions

View File

@ -29,6 +29,11 @@ import (
"github.com/ledgerwatch/erigon/cl/utils"
)
type CaplinConfig struct {
Backfilling bool
Archive bool
}
type NetworkType int
const (

View File

@ -116,7 +116,7 @@ func (u *BitList) HashSSZ() ([32]byte, error) {
}
}
lengthRoot := merkle_tree.Uint64Root(uint64(u.l))
return utils.Keccak256(baseRoot[:], lengthRoot[:]), nil
return utils.Sha256(baseRoot[:], lengthRoot[:]), nil
}
func (arr *BitList) getBaseHash(xs []byte, depth uint8) error {

View File

@ -153,7 +153,7 @@ func (h *hashList) HashSSZ() ([32]byte, error) {
}
}
lengthRoot := merkle_tree.Uint64Root(uint64(h.l))
return utils.Keccak256(baseRoot[:], lengthRoot[:]), nil
return utils.Sha256(baseRoot[:], lengthRoot[:]), nil
}
func (h *hashList) Range(fn func(int, libcommon.Hash, int) bool) {

View File

@ -182,7 +182,7 @@ func (arr *byteBasedUint64Slice) HashListSSZ() ([32]byte, error) {
}
}
lengthRoot := merkle_tree.Uint64Root(uint64(arr.l))
return utils.Keccak256(baseRoot[:], lengthRoot[:]), nil
return utils.Sha256(baseRoot[:], lengthRoot[:]), nil
}
// HashVectorSSZ computes the SSZ hash of the slice as a vector. It returns the hash and any error encountered.

View File

@ -164,7 +164,7 @@ func (v *ValidatorSet) HashSSZ() ([32]byte, error) {
lengthRoot := merkle_tree.Uint64Root(uint64(v.l))
if v.l == 0 {
return utils.Keccak256(merkle_tree.ZeroHashes[depth][:], lengthRoot[:]), nil
return utils.Sha256(merkle_tree.ZeroHashes[depth][:], lengthRoot[:]), nil
}
emptyHashBytes := make([]byte, length.Hash)
@ -211,7 +211,7 @@ func (v *ValidatorSet) HashSSZ() ([32]byte, error) {
elements = elements[:outputLen]
}
return utils.Keccak256(elements[:length.Hash], lengthRoot[:]), nil
return utils.Sha256(elements[:length.Hash], lengthRoot[:]), nil
}
func computeFlatRootsToBuffer(depth uint8, layerBuffer, output []byte) error {

View File

@ -163,7 +163,7 @@ func forkList(schedule map[libcommon.Bytes4]uint64) (f []fork) {
func ComputeForkDigestForVersion(currentVersion [4]byte, genesisValidatorsRoot [32]byte) (digest [4]byte, err error) {
var currentVersion32 libcommon.Hash
copy(currentVersion32[:], currentVersion[:])
dataRoot := utils.Keccak256(currentVersion32[:], genesisValidatorsRoot[:])
dataRoot := utils.Sha256(currentVersion32[:], genesisValidatorsRoot[:])
// copy first four bytes to output
copy(digest[:], dataRoot[:4])
return
@ -227,7 +227,7 @@ func ComputeDomain(
) ([]byte, error) {
var currentVersion32 libcommon.Hash
copy(currentVersion32[:], currentVersion[:])
forkDataRoot := utils.Keccak256(currentVersion32[:], genesisValidatorsRoot[:])
forkDataRoot := utils.Sha256(currentVersion32[:], genesisValidatorsRoot[:])
return append(domainType, forkDataRoot[:28]...), nil
}
@ -239,7 +239,7 @@ func ComputeSigningRoot(
if err != nil {
return [32]byte{}, err
}
return utils.Keccak256(objRoot[:], domain), nil
return utils.Sha256(objRoot[:], domain), nil
}
func Domain(fork *cltypes.Fork, epoch uint64, domainType [4]byte, genesisRoot libcommon.Hash) ([]byte, error) {

View File

@ -86,7 +86,7 @@ func (m *merkleHasher) transactionsListRoot(transactions [][]byte) ([32]byte, er
}
lengthRoot := Uint64Root(transactionLength)
leaves[i] = utils.Keccak256(transactionsBaseRoot[:], lengthRoot[:])
leaves[i] = utils.Sha256(transactionsBaseRoot[:], lengthRoot[:])
}
transactionsBaseRoot, err := MerkleizeVector(leaves, 1048576)
if err != nil {
@ -95,5 +95,5 @@ func (m *merkleHasher) transactionsListRoot(transactions [][]byte) ([32]byte, er
countRoot := Uint64Root(txCount)
return utils.Keccak256(transactionsBaseRoot[:], countRoot[:]), nil
return utils.Sha256(transactionsBaseRoot[:], countRoot[:]), nil
}

View File

@ -71,7 +71,7 @@ func BitlistRootWithLimit(bits []byte, limit uint64) ([32]byte, error) {
}
lengthRoot := Uint64Root(size)
return utils.Keccak256(base[:], lengthRoot[:]), nil
return utils.Sha256(base[:], lengthRoot[:]), nil
}
func packBits(bytes []byte) [][32]byte {
@ -123,5 +123,5 @@ func ListObjectSSZRoot[T ssz.HashableSSZ](list []T, limit uint64) ([32]byte, err
return [32]byte{}, err
}
lenLeaf := Uint64Root(uint64(len(list)))
return utils.Keccak256(vectorLeaf[:], lenLeaf[:]), nil
return utils.Sha256(vectorLeaf[:], lenLeaf[:]), nil
}

View File

@ -13,8 +13,6 @@ type DatabaseConfiguration struct{ PruneDepth uint64 }
var DefaultDatabaseConfiguration = DatabaseConfiguration{PruneDepth: math.MaxUint64}
// should be 1_000_000
func WriteConfigurationIfNotExist(ctx context.Context, tx kv.RwTx, cfg DatabaseConfiguration) error {
var b bytes.Buffer
if err := cbor.Encoder(&b).Encode(cfg); err != nil {

View File

@ -12,19 +12,19 @@ var attestationIndiciesCache *lru.Cache[common.Hash, []uint64]
const attestationIndiciesCacheSize = 1024
func LoadAttestatingIndicies(attestation *solid.AttestationData, aggregationBits []byte) ([]uint64, bool) {
bitsHash := utils.Keccak256(aggregationBits)
bitsHash := utils.Sha256(aggregationBits)
hash, err := attestation.HashSSZ()
if err != nil {
return nil, false
}
return attestationIndiciesCache.Get(utils.Keccak256(hash[:], bitsHash[:]))
return attestationIndiciesCache.Get(utils.Sha256(hash[:], bitsHash[:]))
}
func StoreAttestation(attestation *solid.AttestationData, aggregationBits []byte, indicies []uint64) {
bitsHash := utils.Keccak256(aggregationBits)
bitsHash := utils.Sha256(aggregationBits)
hash, err := attestation.HashSSZ()
if err != nil {
return
}
attestationIndiciesCache.Add(utils.Keccak256(hash[:], bitsHash[:]), indicies)
attestationIndiciesCache.Add(utils.Sha256(hash[:], bitsHash[:]), indicies)
}

View File

@ -185,7 +185,7 @@ func (b *CachingBeaconState) GetBeaconCommitee(slot, committeeIndex uint64) ([]u
func (b *CachingBeaconState) ComputeNextSyncCommittee() (*solid.SyncCommittee, error) {
beaconConfig := b.BeaconConfig()
optimizedHashFunc := utils.OptimizedKeccak256NotThreadSafe()
optimizedHashFunc := utils.OptimizedSha256NotThreadSafe()
epoch := Epoch(b) + 1
//math.MaxUint8
activeValidatorIndicies := b.GetActiveValidatorsIndices(epoch)
@ -215,7 +215,7 @@ func (b *CachingBeaconState) ComputeNextSyncCommittee() (*solid.SyncCommittee, e
buf := make([]byte, 8)
binary.LittleEndian.PutUint64(buf, i/32)
input := append(seed[:], buf...)
randomByte := uint64(utils.Keccak256(input)[i%32])
randomByte := uint64(utils.Sha256(input)[i%32])
// retrieve validator.
validator, err := b.ValidatorForValidatorIndex(int(candidateIndex))
if err != nil {

View File

@ -19,7 +19,7 @@ func ComputeProposerIndex(b *raw.BeaconState, indices []uint64, seed [32]byte) (
input := make([]byte, 40)
preInputs := ComputeShuffledIndexPreInputs(b.BeaconConfig(), seed)
for {
shuffled, err := ComputeShuffledIndex(b.BeaconConfig(), i%total, total, seed, preInputs, utils.Keccak256)
shuffled, err := ComputeShuffledIndex(b.BeaconConfig(), i%total, total, seed, preInputs, utils.Sha256)
if err != nil {
return 0, err
}
@ -29,7 +29,7 @@ func ComputeProposerIndex(b *raw.BeaconState, indices []uint64, seed [32]byte) (
}
copy(input, seed[:])
binary.LittleEndian.PutUint64(input[32:], i/32)
randomByte := uint64(utils.Keccak256(input)[i%32])
randomByte := uint64(utils.Sha256(input)[i%32])
validator, err := b.ValidatorForValidatorIndex(int(candidateIndex))
if err != nil {
return 0, err

View File

@ -2,9 +2,10 @@ package shuffling_test
import (
_ "embed"
"github.com/ledgerwatch/erigon-lib/common/eth2shuffle"
"testing"
"github.com/ledgerwatch/erigon-lib/common/eth2shuffle"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state/raw"
"github.com/ledgerwatch/erigon/cl/phase1/core/state/shuffling"
@ -15,7 +16,7 @@ import (
)
func BenchmarkLambdaShuffledIndex(b *testing.B) {
keccakOptimized := utils.OptimizedKeccak256()
keccakOptimized := utils.OptimizedSha256NotThreadSafe()
eth2ShuffleHash := func(data []byte) []byte {
hashed := keccakOptimized(data)
return hashed[:]
@ -30,7 +31,7 @@ func BenchmarkLambdaShuffledIndex(b *testing.B) {
// Faster by ~40%, the effects of it will be felt mostly on computation of the proposer index.
func BenchmarkErigonShuffledIndex(b *testing.B) {
s := state.New(&clparams.MainnetBeaconConfig)
keccakOptimized := utils.OptimizedKeccak256NotThreadSafe()
keccakOptimized := utils.OptimizedSha256NotThreadSafe()
seed := [32]byte{2, 35, 6}
preInputs := shuffling.ComputeShuffledIndexPreInputs(s.BeaconConfig(), seed)

View File

@ -3,6 +3,7 @@ package shuffling
import (
"encoding/binary"
"fmt"
"github.com/ledgerwatch/erigon-lib/common/eth2shuffle"
"github.com/ledgerwatch/erigon-lib/common"
@ -49,7 +50,7 @@ func ComputeShuffledIndex(conf *clparams.BeaconChainConfig, ind, ind_count uint6
func ComputeShuffledIndexPreInputs(conf *clparams.BeaconChainConfig, seed [32]byte) [][32]byte {
ret := make([][32]byte, conf.ShuffleRoundCount)
for i := range ret {
ret[i] = utils.Keccak256(append(seed[:], byte(i)))
ret[i] = utils.Sha256(append(seed[:], byte(i)))
}
return ret
}
@ -59,13 +60,13 @@ func GetSeed(beaconConfig *clparams.BeaconChainConfig, mix common.Hash, epoch ui
binary.LittleEndian.PutUint64(epochByteArray, epoch)
input := append(domain[:], epochByteArray...)
input = append(input, mix[:]...)
return utils.Keccak256(input)
return utils.Sha256(input)
}
func ComputeShuffledIndicies(beaconConfig *clparams.BeaconChainConfig, mix common.Hash, indicies []uint64, slot uint64) []uint64 {
shuffledIndicies := make([]uint64, len(indicies))
copy(shuffledIndicies, indicies)
hashFunc := utils.OptimizedKeccak256NotThreadSafe()
hashFunc := utils.OptimizedSha256NotThreadSafe()
epoch := slot / beaconConfig.SlotsPerEpoch
seed := GetSeed(beaconConfig, mix, epoch, beaconConfig.DomainBeaconAttester)
eth2ShuffleHashFunc := func(data []byte) []byte {

View File

@ -188,7 +188,7 @@ func (f *ForkChoiceStore) OnBlsToExecutionChange(signedChange *cltypes.SignedBLS
// Perform full validation if requested.
if !test {
// Check the validator's withdrawal credentials against the provided message.
hashedFrom := utils.Keccak256(change.From[:])
hashedFrom := utils.Sha256(change.From[:])
if !bytes.Equal(hashedFrom[1:], wc[1:]) {
return fmt.Errorf("invalid withdrawal credentials")
}

View File

@ -583,10 +583,15 @@ func ConsensusClStages(ctx context.Context,
return err
}
// TODO(Giulio2002): schedule snapshots retirement if needed.
// err = cfg.beaconDB.PurgeRange(ctx, tx, 1, cfg.forkChoice.HighestSeen()-cfg.dbConfig.PruneDepth)
// if err != nil {
// return err
// }
if !cfg.backfilling {
if err := cfg.beaconDB.PurgeRange(ctx, tx, 1, cfg.forkChoice.HighestSeen()-100_000); err != nil {
return err
}
if err := beacon_indicies.PruneBlockRoots(ctx, tx, 0, cfg.forkChoice.HighestSeen()-100_000); err != nil {
return err
}
}
return tx.Commit()
},
},

View File

@ -42,7 +42,7 @@ func (s *Sentinel) msgId(pmsg *pubsubpb.Message) string {
combinedData = append(combinedData, topicLenBytes...)
combinedData = append(combinedData, topic...)
combinedData = append(combinedData, pmsg.Data...)
h := utils.Keccak256(combinedData)
h := utils.Sha256(combinedData)
return string(h[:20])
}
totalLength := len(s.cfg.NetworkConfig.MessageDomainValidSnappy) +
@ -55,6 +55,6 @@ func (s *Sentinel) msgId(pmsg *pubsubpb.Message) string {
combinedData = append(combinedData, topicLenBytes...)
combinedData = append(combinedData, topic...)
combinedData = append(combinedData, decodedData...)
h := utils.Keccak256(combinedData)
h := utils.Sha256(combinedData)
return string(h[:20])
}

View File

@ -33,7 +33,7 @@ func TestMsgID(t *testing.T) {
combinedObj := append(n.MessageDomainInvalidSnappy[:], topicLenBytes...)
combinedObj = append(combinedObj, tpc...)
combinedObj = append(combinedObj, pMsg.Data...)
hashedData := utils.Keccak256(combinedObj)
hashedData := utils.Sha256(combinedObj)
msgID := string(hashedData[:20])
require.Equal(t, msgID, s.msgId(pMsg), "Got incorrect msg id")
@ -44,7 +44,7 @@ func TestMsgID(t *testing.T) {
combinedObj = append(n.MessageDomainValidSnappy[:], topicLenBytes...)
combinedObj = append(combinedObj, tpc...)
combinedObj = append(combinedObj, validObj[:]...)
hashedData = utils.Keccak256(combinedObj)
hashedData = utils.Sha256(combinedObj)
msgID = string(hashedData[:20])
require.Equal(t, msgID, s.msgId(nMsg), "Got incorrect msg id")
}

View File

@ -1,10 +1,11 @@
package consensus_tests
import (
"github.com/ledgerwatch/erigon/spectest"
"io/fs"
"testing"
"github.com/ledgerwatch/erigon/spectest"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/cl/phase1/core/state"
"github.com/ledgerwatch/erigon/cl/phase1/core/state/shuffling"
@ -29,7 +30,7 @@ func (b *ShufflingCore) Run(t *testing.T, root fs.FS, c spectest.TestCase) (err
}
s := state.New(&clparams.MainnetBeaconConfig)
keccakOptimized := utils.OptimizedKeccak256NotThreadSafe()
keccakOptimized := utils.OptimizedSha256NotThreadSafe()
preInputs := shuffling.ComputeShuffledIndexPreInputs(s.BeaconConfig(), meta.Seed)
for idx, v := range meta.Mapping {
shuffledIdx, err := shuffling.ComputeShuffledIndex(s.BeaconConfig(), uint64(idx), uint64(meta.Count), meta.Seed, preInputs, keccakOptimized)

View File

@ -165,7 +165,7 @@ func (I *impl) ProcessDeposit(s abstract.BeaconState, deposit *cltypes.Deposit)
if err != nil {
return err
}
signedRoot := utils.Keccak256(depositMessageRoot[:], domain)
signedRoot := utils.Sha256(depositMessageRoot[:], domain)
// Perform BLS verification and if successful noice.
valid, err := bls.Verify(deposit.Data.Signature[:], signedRoot[:], publicKey[:])
// Literally you can input it trash.
@ -320,7 +320,7 @@ func (I *impl) ProcessSyncAggregate(s abstract.BeaconState, sync *cltypes.SyncAg
if err != nil {
return err
}
msg := utils.Keccak256(blockRoot[:], domain)
msg := utils.Sha256(blockRoot[:], domain)
isValid, err := bls.VerifyAggregate(sync.SyncCommiteeSignature[:], msg[:], votedKeys)
if err != nil {
return err
@ -404,7 +404,7 @@ func (I *impl) ProcessBlsToExecutionChange(s abstract.BeaconState, signedChange
}
// Check the validator's withdrawal credentials against the provided message.
hashedFrom := utils.Keccak256(change.From[:])
hashedFrom := utils.Sha256(change.From[:])
if !bytes.Equal(hashedFrom[1:], wc[1:]) {
return fmt.Errorf("invalid withdrawal credentials")
}
@ -810,7 +810,7 @@ func (I *impl) ProcessRandao(s abstract.BeaconState, randao [96]byte, proposerIn
}
randaoMixes := s.GetRandaoMixes(epoch)
randaoHash := utils.Keccak256(randao[:])
randaoHash := utils.Sha256(randao[:])
mix := [32]byte{}
for i := range mix {
mix[i] = randaoMixes[i] ^ randaoHash[i]

View File

@ -37,7 +37,7 @@ func ProcessHistoricalRootsUpdate(s abstract.BeaconState) error {
StateSummaryRoot: stateRootsLeaf,
})
} else {
historicalRoot := utils.Keccak256(blockRootsLeaf[:], stateRootsLeaf[:])
historicalRoot := utils.Sha256(blockRootsLeaf[:], stateRootsLeaf[:])
s.AddHistoricalRoot(historicalRoot)
}

View File

@ -15,7 +15,7 @@ const VERSIONED_HASH_VERSION_KZG byte = byte(1)
func kzgCommitmentToVersionedHash(kzgCommitment *cltypes.KZGCommitment) (libcommon.Hash, error) {
versionedHash := [32]byte{}
kzgCommitmentHash := utils.Keccak256(kzgCommitment[:])
kzgCommitmentHash := utils.Sha256(kzgCommitment[:])
buf := append([]byte{}, VERSIONED_HASH_VERSION_KZG)
buf = append(buf, kzgCommitmentHash[1:]...)
@ -61,7 +61,7 @@ func txPeekBlobVersionedHashes(txBytes []byte) []libcommon.Hash {
func computeSigningRootEpoch(epoch uint64, domain []byte) (libcommon.Hash, error) {
b := make([]byte, 32)
binary.LittleEndian.PutUint64(b, epoch)
return utils.Keccak256(b, domain), nil
return utils.Sha256(b, domain), nil
}
// transitionSlot is called each time there is a new slot to process

View File

@ -27,8 +27,8 @@ var hasherPool = sync.Pool{
},
}
// General purpose Keccak256
func Keccak256(data []byte, extras ...[]byte) [32]byte {
// General purpose Sha256
func Sha256(data []byte, extras ...[]byte) [32]byte {
h, ok := hasherPool.Get().(hash.Hash)
if !ok {
h = sha256.New()
@ -46,24 +46,9 @@ func Keccak256(data []byte, extras ...[]byte) [32]byte {
return b
}
// Optimized Keccak256, avoid pool.put/pool.get, meant for intensive operations.
func OptimizedKeccak256() HashFunc {
h := sha256.New()
return func(data []byte, extras ...[]byte) [32]byte {
h.Reset()
var b [32]byte
h.Write(data)
for _, extra := range extras {
h.Write(extra)
}
h.Sum(b[:0])
return b
}
}
// Optimized Keccak256, avoid pool.put/pool.get, meant for intensive operations.
// Optimized Sha256, avoid pool.put/pool.get, meant for intensive operations.
// this version is not thread safe
func OptimizedKeccak256NotThreadSafe() HashFunc {
func OptimizedSha256NotThreadSafe() HashFunc {
h := sha256.New()
var b [32]byte
return func(data []byte, extras ...[]byte) [32]byte {

View File

@ -13,12 +13,12 @@ func TestKeccak256(t *testing.T) {
[]byte("extra2"),
}
expectedHash := utils.Keccak256(data, extras...)
hashFunc := utils.OptimizedKeccak256()
expectedHash := utils.Sha256(data, extras...)
hashFunc := utils.OptimizedSha256NotThreadSafe()
expectedOptimizedHash := hashFunc(data, extras...)
// Test Keccak256 function
hash := utils.Keccak256(data, extras...)
hash := utils.Sha256(data, extras...)
if hash != expectedHash {
t.Errorf("Keccak256 returned an incorrect hash. Expected: %x, Got: %x", expectedHash, hash)
}
@ -37,12 +37,12 @@ func TestOptimizedKeccak256NotThreadSafe(t *testing.T) {
[]byte("extra2"),
}
expectedHash := utils.Keccak256(data, extras...)
hashFunc := utils.OptimizedKeccak256NotThreadSafe()
expectedHash := utils.Sha256(data, extras...)
hashFunc := utils.OptimizedSha256NotThreadSafe()
expectedOptimizedHash := hashFunc(data, extras...)
// Test OptimizedKeccak256NotThreadSafe function
hash := utils.Keccak256(data, extras...)
hash := utils.Sha256(data, extras...)
if hash != expectedHash {
t.Errorf("Keccak256 returned an incorrect hash. Expected: %x, Got: %x", expectedHash, hash)
}

View File

@ -9,9 +9,9 @@ func IsValidMerkleBranch(leaf libcommon.Hash, branch []libcommon.Hash, depth uin
value := leaf
for i := uint64(0); i < depth; i++ {
if (index / PowerOf2(i) % 2) == 1 {
value = Keccak256(append(branch[i][:], value[:]...))
value = Sha256(append(branch[i][:], value[:]...))
} else {
value = Keccak256(append(value[:], branch[i][:]...))
value = Sha256(append(value[:], branch[i][:]...))
}
}
return value == root

View File

@ -898,6 +898,16 @@ var (
Usage: "sets the port to listen for beacon api requests",
Value: 5555,
}
CaplinBackfillingFlag = cli.BoolFlag{
Name: "caplin.backfilling",
Usage: "sets whether backfilling is enabled for caplin",
Value: true,
}
CaplinArchiveFlag = cli.BoolFlag{
Name: "caplin.archive",
Usage: "enables archival node in caplin",
Value: false,
}
)
var MetricFlags = []cli.Flag{&MetricsEnabledFlag, &MetricsHTTPFlag, &MetricsPortFlag}
@ -1527,6 +1537,11 @@ func setBeaconAPI(ctx *cli.Context, cfg *ethconfig.Config) {
cfg.BeaconRouter.IdleTimeout = time.Duration(ctx.Uint64(BeaconApiIdleTimeoutFlag.Name)) * time.Second
}
func setCaplin(ctx *cli.Context, cfg *ethconfig.Config) {
cfg.CaplinConfig.Backfilling = ctx.Bool(CaplinBackfillingFlag.Name)
cfg.CaplinConfig.Archive = ctx.Bool(CaplinArchiveFlag.Name)
}
func setSilkworm(ctx *cli.Context, cfg *ethconfig.Config) {
cfg.SilkwormLibraryPath = ctx.String(SilkwormLibraryPathFlag.Name)
if ctx.IsSet(SilkwormExecutionFlag.Name) {
@ -1645,6 +1660,7 @@ func SetEthConfig(ctx *cli.Context, nodeConfig *nodecfg.Config, cfg *ethconfig.C
setBorConfig(ctx, cfg)
setSilkworm(ctx, cfg)
setBeaconAPI(ctx, cfg)
setCaplin(ctx, cfg)
cfg.Ethstats = ctx.String(EthStatsURLFlag.Name)
cfg.HistoryV3 = ctx.Bool(HistoryV3Flag.Name)

View File

@ -5,10 +5,11 @@ import (
"encoding/binary"
"flag"
"fmt"
"github.com/ledgerwatch/erigon/cl/utils"
"os"
"time"
"github.com/ledgerwatch/erigon/cl/utils"
"github.com/c2h5oh/datasize"
libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/etl"
@ -323,7 +324,7 @@ func dump_acc_preimages(ctx context.Context, cfg optionsCfg) error {
if _, err := file.Write(k); err != nil {
return err
}
addressHash := utils.Keccak256(k)
addressHash := utils.Sha256(k)
if _, err := file.Write(addressHash[:]); err != nil {
return err
@ -389,7 +390,7 @@ func dump_storage_preimages(ctx context.Context, cfg optionsCfg, logger log.Logg
}
currentAddress = libcommon.BytesToAddress(k)
currentIncarnation = acc.Incarnation
addressHash = utils.Keccak256(currentAddress[:])
addressHash = utils.Sha256(currentAddress[:])
} else {
address := libcommon.BytesToAddress(k[:20])
if address != currentAddress {
@ -398,7 +399,7 @@ func dump_storage_preimages(ctx context.Context, cfg optionsCfg, logger log.Logg
if binary.BigEndian.Uint64(k[20:]) != currentIncarnation {
continue
}
storageHash := utils.Keccak256(k[28:])
storageHash := utils.Sha256(k[28:])
buf.Write(k[28:])
buf.Write(storageHash[:])
}

View File

@ -854,7 +854,7 @@ func New(ctx context.Context, stack *node.Node, config *ethconfig.Config, logger
go func() {
eth1Getter := getters.NewExecutionSnapshotReader(ctx, blockReader, backend.chainDB)
if err := caplin1.RunCaplinPhase1(ctx, client, engine, beaconCfg, genesisCfg, state, nil, dirs, config.BeaconRouter, eth1Getter, backend.downloaderClient, true); err != nil {
if err := caplin1.RunCaplinPhase1(ctx, client, engine, beaconCfg, genesisCfg, state, nil, dirs, config.BeaconRouter, eth1Getter, backend.downloaderClient, config.CaplinConfig.Backfilling); err != nil {
logger.Error("could not start caplin", "err", err)
}
ctxCancel()

View File

@ -35,6 +35,7 @@ import (
"github.com/ledgerwatch/erigon-lib/txpool/txpoolcfg"
"github.com/ledgerwatch/erigon/cl/beacon/beacon_router_configuration"
"github.com/ledgerwatch/erigon/cl/clparams"
"github.com/ledgerwatch/erigon/consensus/ethash/ethashcfg"
"github.com/ledgerwatch/erigon/core/types"
"github.com/ledgerwatch/erigon/eth/ethconfig/estimate"
@ -190,6 +191,7 @@ type Config struct {
Snapshot BlocksFreezing
Downloader *downloadercfg.Cfg
BeaconRouter beacon_router_configuration.RouterConfiguration
CaplinConfig clparams.CaplinConfig
Dirs datadir.Dirs