Skip to content

Commit

Permalink
Resumable beacon state reconstruction (#8918)
Browse files Browse the repository at this point in the history
* Most of the PR changed files are extra and slightly more complicated
unit tests.
* Fixed Eth1DataVotes not inheriting genesis
* Fixed Attestations simulation using wrong slot when reconstructing
partecipation
* Fixed Copy() operation on BeaconState on Eth1DataVotes
* Used correct ListSSZ type for Eth1DataVotes and HistoricalSummaries
* Fixed wrong []uint64 deltas on empty slots
  • Loading branch information
Giulio2002 authored Dec 11, 2023
1 parent 7fb8f9d commit 2498787
Show file tree
Hide file tree
Showing 126 changed files with 466 additions and 114 deletions.
128 changes: 110 additions & 18 deletions cl/antiquary/state_antiquary.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/ledgerwatch/erigon/cl/persistence/base_encoding"
"github.com/ledgerwatch/erigon/cl/persistence/beacon_indicies"
state_accessors "github.com/ledgerwatch/erigon/cl/persistence/state"
"github.com/ledgerwatch/erigon/cl/persistence/state/historical_states_reader"
"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"
Expand Down Expand Up @@ -134,6 +135,7 @@ func uint64BalancesList(s *state.CachingBeaconState, out []uint64) []uint64 {

func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
var tx kv.Tx

tx, err := s.mainDB.BeginRo(ctx)
if err != nil {
return err
Expand Down Expand Up @@ -182,25 +184,56 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
stateEvents := etl.NewCollector(kv.StateEvents, s.dirs.Tmp, etl.NewSortableBuffer(etl.BufferOptimalSize), s.logger)
defer stateEvents.Close()

progress, err := state_accessors.GetStateProcessingProgress(tx)
if err != nil {
return err
}
// Go back a little bit
if progress > s.cfg.SlotsPerEpoch*2 {
progress -= s.cfg.SlotsPerEpoch * 2
} else {
progress = 0
}
progress, err = findNearestSlotBackwards(tx, progress) // Maybe the guess was a missed slot.
if err != nil {
return err
}
// buffers
commonBuffer := &bytes.Buffer{}
compressedWriter, err := zstd.NewWriter(commonBuffer, zstd.WithEncoderLevel(zstd.SpeedBetterCompression))
if err != nil {
return err
}
defer compressedWriter.Close()

// TODO(Giulio2002): also store genesis information and resume from state.
if s.currentState == nil {
s.currentState, err = s.genesisState.Copy()
if err != nil {
return err
}
// Collect genesis state if we are at genesis
if err := s.collectGenesisState(ctx, compressedWriter, s.currentState, slashings, proposers, minimalBeaconStates, stateEvents, changedValidators); err != nil {
return err
// progress is 0 when we are at genesis
if progress == 0 {
s.currentState, err = s.genesisState.Copy()
if err != nil {
return err
}
// Collect genesis state if we are at genesis
if err := s.collectGenesisState(ctx, compressedWriter, s.currentState, slashings, proposers, minimalBeaconStates, stateEvents, changedValidators); err != nil {
return err
}
} else {
start := time.Now()
// progress not 0? we need to load the state from the DB
historicalReader := historical_states_reader.NewHistoricalStatesReader(s.cfg, s.snReader, s.validatorsTable, s.fs, s.genesisState)
s.currentState, err = historicalReader.ReadHistoricalState(ctx, tx, progress)
if err != nil {
return fmt.Errorf("failed to read historical state at slot %d: %w", progress, err)
}
end := time.Since(start)
hashRoot, err := s.currentState.HashSSZ()
if err != nil {
return err
}
log.Info("Recovered Beacon State", "slot", s.currentState.Slot(), "elapsed", end, "root", libcommon.Hash(hashRoot).String())
}
}

logLvl := log.LvlInfo
if to-s.currentState.Slot() < 96 {
logLvl = log.LvlDebug
Expand Down Expand Up @@ -256,8 +289,11 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
return s.validatorsTable.AddWithdrawalCredentials(uint64(index), slot, libcommon.BytesToHash(wc))
},
OnEpochBoundary: func(epoch uint64) error {
v := append(s.currentState.CurrentJustifiedCheckpoint(), append(s.currentState.PreviousJustifiedCheckpoint(), s.currentState.FinalizedCheckpoint()...)...)
k := base_encoding.Encode64ToBytes4(s.cfg.RoundSlotToEpoch(slot))
v := make([]byte, solid.CheckpointSize*3)
copy(v, s.currentState.CurrentJustifiedCheckpoint())
copy(v[solid.CheckpointSize:], s.currentState.PreviousJustifiedCheckpoint())
copy(v[solid.CheckpointSize*2:], s.currentState.FinalizedCheckpoint())
if err := checkpoints.Collect(k, v); err != nil {
return err
}
Expand Down Expand Up @@ -302,9 +338,11 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
progressTimer := time.NewTicker(1 * time.Minute)
defer progressTimer.Stop()
prevSlot := slot
first := false
// This tells us that transition and operations do not happen concurrently and access is safe, so we can optimize for GC.
// there is optimized custom cache to recycle big GC overhead.
for ; slot < to; slot++ {
isDumpSlot := slot%clparams.SlotsPerDump == 0
block, err := s.snReader.ReadBlockBySlot(ctx, tx, slot)
if err != nil {
return err
Expand All @@ -329,11 +367,11 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
}
}

if slot%clparams.SlotsPerDump == 0 {
if isDumpSlot && block == nil {
if err := s.antiquateField(ctx, slot, s.currentState.RawBalances(), compressedWriter, "balances"); err != nil {
return err
}
if err := s.antiquateEffectiveBalances(ctx, slot, s.currentState.RawBalances(), compressedWriter); err != nil {
if err := s.antiquateEffectiveBalances(ctx, slot, s.currentState.RawValidatorSet(), compressedWriter); err != nil {
return err
}
if s.currentState.Version() >= clparams.AltairVersion {
Expand All @@ -360,19 +398,38 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
slashingsBytes = slashingsBytes[:0]
slashingsBytes = append(slashingsBytes, s.currentState.RawSlashings()...)

// We sanity check the state every 100k slots.
if err := transition.TransitionState(s.currentState, block, slot%100_000 == 0); err != nil {
fullValidation := slot%100_000 == 0 || first
// We sanity check the state every 100k slots or when we start.
if err := transition.TransitionState(s.currentState, block, fullValidation); err != nil {
return err
}

first = false
// if s.currentState.Slot() == 3868670 {
// s.dumpFullBeaconState()
// }
if err := s.storeMinimalState(commonBuffer, s.currentState, minimalBeaconStates); err != nil {
return err
}
if err := stateEvents.Collect(base_encoding.Encode64ToBytes4(slot), events.CopyBytes()); err != nil {
return err
}
events.Reset()
if slot%clparams.SlotsPerDump == 0 {

if isDumpSlot {
if err := s.antiquateField(ctx, slot, s.currentState.RawBalances(), compressedWriter, "balances"); err != nil {
return err
}
if err := s.antiquateEffectiveBalances(ctx, slot, s.currentState.RawValidatorSet(), compressedWriter); err != nil {
return err
}
if s.currentState.Version() >= clparams.AltairVersion {
if err := s.antiquateField(ctx, slot, s.currentState.RawInactivityScores(), compressedWriter, "inactivity_scores"); err != nil {
return err
}
}
if err := s.antiquateFullSlashings(slashings, slot, s.currentState.RawSlashings(), commonBuffer, compressedWriter); err != nil {
return err
}
continue
}

Expand Down Expand Up @@ -503,9 +560,16 @@ func (s *Antiquary) IncrementBeaconState(ctx context.Context, to uint64) error {
if err != nil {
return err
}

log.Info("Historical antiquated", "slot", s.currentState.Slot(), "latency", time.Since(start))
return rwTx.Commit()
if err := rwTx.Commit(); err != nil {
return err
}
endTime := time.Since(start)
stateRoot, err := s.currentState.HashSSZ()
if err != nil {
return err
}
log.Info("Historical states antiquated", "slot", s.currentState.Slot(), "root", libcommon.Hash(stateRoot), "latency", endTime)
return nil
}

func (s *Antiquary) antiquateField(ctx context.Context, slot uint64, uncompressed []byte, compressor *zstd.Encoder, name string) error {
Expand Down Expand Up @@ -700,6 +764,19 @@ func (s *Antiquary) dumpPayload(k []byte, v []byte, c *etl.Collector, b *bytes.B
return c.Collect(k, common.Copy(b.Bytes()))
}

// func (s *Antiquary) dumpFullBeaconState() {
// b, err := s.currentState.EncodeSSZ(nil)
// if err != nil {
// s.logger.Error("Failed to encode full beacon state", "err", err)
// return
// }
// // just dump it in a.txt like an idiot without afero
// if err := os.WriteFile("b.txt", b, 0644); err != nil {
// s.logger.Error("Failed to write full beacon state", "err", err)
// }

// }

func flattenRandaoMixes(hashes []libcommon.Hash) []byte {
out := make([]byte, len(hashes)*32)
for i, h := range hashes {
Expand All @@ -720,3 +797,18 @@ func (s *Antiquary) antiquateFullSlashings(collector *etl.Collector, slot uint64
}
return collector.Collect(base_encoding.Encode64ToBytes4(slot), common.Copy(buffer.Bytes()))
}

func findNearestSlotBackwards(tx kv.Tx, slot uint64) (uint64, error) {
canonicalRoot, err := beacon_indicies.ReadCanonicalBlockRoot(tx, slot)
if err != nil {
return 0, err
}
for canonicalRoot == (common.Hash{}) && slot > 0 {
slot--
canonicalRoot, err = beacon_indicies.ReadCanonicalBlockRoot(tx, slot)
if err != nil {
return 0, err
}
}
return slot, nil
}
8 changes: 8 additions & 0 deletions cl/antiquary/state_antiquary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,6 +2,7 @@ package antiquary

import (
"context"
"fmt"
"testing"

_ "embed"
Expand Down Expand Up @@ -36,6 +37,13 @@ func TestStateAntiquaryCapella(t *testing.T) {
runTest(t, blocks, preState, postState)
}

func TestStateAntiquaryBellatrix(t *testing.T) {
t.Skip()
blocks, preState, postState := tests.GetBellatrixRandom()
fmt.Println(len(blocks))
runTest(t, blocks, preState, postState)
}

func TestStateAntiquaryPhase0(t *testing.T) {
t.Skip()
blocks, preState, postState := tests.GetPhase0Random()
Expand Down
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
Binary file not shown.
1 change: 1 addition & 0 deletions cl/antiquary/tests/test_data/bellatrix/meta.yaml
Original file line number Diff line number Diff line change
@@ -0,0 +1 @@
{blocks_count: 96}
Binary file not shown.
Binary file not shown.
42 changes: 42 additions & 0 deletions cl/antiquary/tests/tests.go
Original file line number Diff line number Diff line change
Expand Up @@ -2,7 +2,9 @@ package tests

import (
"context"
"embed"
_ "embed"
"strconv"

libcommon "github.com/ledgerwatch/erigon-lib/common"
"github.com/ledgerwatch/erigon-lib/kv"
Expand Down Expand Up @@ -37,6 +39,11 @@ var phase0_pre_state_ssz_snappy []byte
//go:embed test_data/phase0/post.ssz_snappy
var phase0_post_state_ssz_snappy []byte

// bellatrix is long

//go:embed test_data/bellatrix
var bellatrixFS embed.FS

type MockBlockReader struct {
u map[uint64]*cltypes.SignedBeaconBlock
}
Expand Down Expand Up @@ -130,3 +137,38 @@ func GetPhase0Random() ([]*cltypes.SignedBeaconBlock, *state.CachingBeaconState,
}
return []*cltypes.SignedBeaconBlock{block1, block2}, preState, postState
}

func GetBellatrixRandom() ([]*cltypes.SignedBeaconBlock, *state.CachingBeaconState, *state.CachingBeaconState) {
ret := make([]*cltypes.SignedBeaconBlock, 0, 96)
// format for blocks is blocks_{i}.ssz_snappy where i is the index of the block, starting from 0 to 95 included.
for i := 0; i < 96; i++ {
block := cltypes.NewSignedBeaconBlock(&clparams.MainnetBeaconConfig)
// Lets do te
b, err := bellatrixFS.ReadFile("test_data/bellatrix/blocks_" + strconv.FormatInt(int64(i), 10) + ".ssz_snappy")
if err != nil {
panic(err)
}
if err := utils.DecodeSSZSnappy(block, b, int(clparams.BellatrixVersion)); err != nil {
panic(err)
}
ret = append(ret, block)
}
preState := state.New(&clparams.MainnetBeaconConfig)
b, err := bellatrixFS.ReadFile("test_data/bellatrix/pre.ssz_snappy")
if err != nil {
panic(err)
}
if err := utils.DecodeSSZSnappy(preState, b, int(clparams.BellatrixVersion)); err != nil {
panic(err)
}
postState := state.New(&clparams.MainnetBeaconConfig)
b, err = bellatrixFS.ReadFile("test_data/bellatrix/post.ssz_snappy")
if err != nil {
panic(err)
}
if err := utils.DecodeSSZSnappy(postState, b, int(clparams.BellatrixVersion)); err != nil {
panic(err)
}
return ret, preState, postState

}
7 changes: 5 additions & 2 deletions cl/cltypes/eth1_block.go
Original file line number Diff line number Diff line change
Expand Up @@ -94,8 +94,11 @@ func (*Eth1Block) Static() bool {
func (b *Eth1Block) PayloadHeader() (*Eth1Header, error) {
var err error
var transactionsRoot, withdrawalsRoot libcommon.Hash
if transactionsRoot, err = b.Transactions.HashSSZ(); err != nil {
return nil, err
// Corner case: before TTD this is 0, since all fields are 0, a 0 hash check will suffice.
if b.BlockHash != (libcommon.Hash{}) {
if transactionsRoot, err = b.Transactions.HashSSZ(); err != nil {
return nil, err
}
}
if b.version >= clparams.CapellaVersion {
withdrawalsRoot, err = b.Withdrawals.HashSSZ()
Expand Down
8 changes: 4 additions & 4 deletions cl/cltypes/solid/checkpoint.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,7 +13,7 @@ import (
)

// Constants to represent the size and layout of a Checkpoint
const checkpointSize = 32 + 8 // BlockRoot(32 bytes) + Epoch(8 bytes)
const CheckpointSize = 32 + 8 // BlockRoot(32 bytes) + Epoch(8 bytes)

type Checkpoint []byte // Define Checkpoint as a byte slice

Expand All @@ -22,15 +22,15 @@ func NewCheckpointFromParameters(
blockRoot libcommon.Hash, // A hash representing the block root
epoch uint64, // An unsigned 64-bit integer representing the epoch
) Checkpoint {
var c Checkpoint = make([]byte, checkpointSize)
var c Checkpoint = make([]byte, CheckpointSize)
c.SetBlockRoot(blockRoot)
c.SetEpoch(epoch)
return c
}

// NewCheckpoint returns a new Checkpoint with the underlying byte slice initialized to zeros
func NewCheckpoint() Checkpoint {
return make([]byte, checkpointSize)
return make([]byte, CheckpointSize)
}

func (c Checkpoint) MarshalJSON() ([]byte, error) {
Expand Down Expand Up @@ -92,7 +92,7 @@ func (c Checkpoint) BlockRoot() (o libcommon.Hash) {

// EncodingSizeSSZ returns the size of the Checkpoint object when encoded as SSZ.
func (Checkpoint) EncodingSizeSSZ() int {
return checkpointSize
return CheckpointSize
}

// DecodeSSZ decodes the Checkpoint object from SSZ-encoded data.
Expand Down
2 changes: 1 addition & 1 deletion cl/cltypes/solid/uint64slice_byte.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,7 +81,7 @@ func (arr *byteBasedUint64Slice) CopyTo(target *byteBasedUint64Slice) {
func (arr *byteBasedUint64Slice) MarshalJSON() ([]byte, error) {
list := make([]uint64, arr.l)
for i := 0; i < arr.l; i++ {
list[0] = arr.Get(i)
list[i] = arr.Get(i)
}
return json.Marshal(list)
}
Expand Down
2 changes: 2 additions & 0 deletions cl/cltypes/solid/validator_set.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,8 @@ func NewValidatorSetWithLength(c int, l int) *ValidatorSet {
l: l,
buffer: make([]byte, l*validatorSize),
treeCacheBuffer: make([]byte, getTreeCacheSize(l, validatorTreeCacheGroupLayer)*length.Hash),
phase0Data: make([]Phase0Data, l),
attesterBits: make([]byte, l),
}
}

Expand Down
Loading

0 comments on commit 2498787

Please sign in to comment.