Use Head to validate sidecars when possible (#15977)

* Use Head to validate sidecars when possible

🤖 Generated with [Claude Code](https://claude.com/claude-code)
Co-Authored-By: Claude <noreply@anthropic.com>

* LazyHeadStateProvider that avoids service registry

* Remove extra file

---------

Co-authored-by: Claude <noreply@anthropic.com>
Co-authored-by: james-prysm <90280386+james-prysm@users.noreply.github.com>
Co-authored-by: Kasey Kirkham <kasey@users.noreply.github.com>
This commit is contained in:
Potuz
2025-11-06 13:43:38 -05:00
committed by GitHub
parent 091e868a7b
commit 7794a77ae6
17 changed files with 308 additions and 133 deletions

View File

@@ -100,6 +100,7 @@ go_test(
deps = [ deps = [
"//async/event:go_default_library", "//async/event:go_default_library",
"//beacon-chain/blockchain/kzg:go_default_library", "//beacon-chain/blockchain/kzg:go_default_library",
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/cache/depositsnapshot:go_default_library", "//beacon-chain/cache/depositsnapshot:go_default_library",
"//beacon-chain/core/feed:go_default_library", "//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/state:go_default_library", "//beacon-chain/core/feed/state:go_default_library",

View File

@@ -8,6 +8,7 @@ import (
"time" "time"
"github.com/OffchainLabs/prysm/v7/async/event" "github.com/OffchainLabs/prysm/v7/async/event"
chainMock "github.com/OffchainLabs/prysm/v7/beacon-chain/blockchain/testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/cache/depositsnapshot" "github.com/OffchainLabs/prysm/v7/beacon-chain/cache/depositsnapshot"
dbutil "github.com/OffchainLabs/prysm/v7/beacon-chain/db/testing" dbutil "github.com/OffchainLabs/prysm/v7/beacon-chain/db/testing"
mockExecution "github.com/OffchainLabs/prysm/v7/beacon-chain/execution/testing" mockExecution "github.com/OffchainLabs/prysm/v7/beacon-chain/execution/testing"
@@ -99,7 +100,7 @@ func TestStart_OK(t *testing.T) {
c := startup.NewClockSynchronizer() c := startup.NewClockSynchronizer()
require.NoError(t, c.SetClock(startup.NewClock(time.Unix(0, 0), [32]byte{}))) require.NoError(t, c.SetClock(startup.NewClock(time.Unix(0, 0), [32]byte{})))
waiter := verification.NewInitializerWaiter( waiter := verification.NewInitializerWaiter(
c, forkchoice.NewROForkChoice(nil), nil) c, forkchoice.NewROForkChoice(nil), nil, &chainMock.ChainService{})
web3Service, err := NewService(t.Context(), web3Service, err := NewService(t.Context(),
WithHttpEndpoint(endpoint), WithHttpEndpoint(endpoint),

View File

@@ -124,6 +124,7 @@ type BeaconNode struct {
DataColumnStorage *filesystem.DataColumnStorage DataColumnStorage *filesystem.DataColumnStorage
DataColumnStorageOptions []filesystem.DataColumnStorageOption DataColumnStorageOptions []filesystem.DataColumnStorageOption
verifyInitWaiter *verification.InitializerWaiter verifyInitWaiter *verification.InitializerWaiter
lhsp *verification.LazyHeadStateProvider
syncChecker *initialsync.SyncChecker syncChecker *initialsync.SyncChecker
slasherEnabled bool slasherEnabled bool
lcStore *lightclient.Store lcStore *lightclient.Store
@@ -230,8 +231,9 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
return nil, errors.Wrap(err, "could not start modules") return nil, errors.Wrap(err, "could not start modules")
} }
beacon.lhsp = &verification.LazyHeadStateProvider{}
beacon.verifyInitWaiter = verification.NewInitializerWaiter( beacon.verifyInitWaiter = verification.NewInitializerWaiter(
beacon.clockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen) beacon.clockWaiter, forkchoice.NewROForkChoice(beacon.forkChoicer), beacon.stateGen, beacon.lhsp)
beacon.BackfillOpts = append( beacon.BackfillOpts = append(
beacon.BackfillOpts, beacon.BackfillOpts,
@@ -749,6 +751,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
if err != nil { if err != nil {
return errors.Wrap(err, "could not register blockchain service") return errors.Wrap(err, "could not register blockchain service")
} }
b.lhsp.HeadStateProvider = blockchainService
return b.services.RegisterService(blockchainService) return b.services.RegisterService(blockchainService)
} }

View File

@@ -154,7 +154,7 @@ func TestFetchDataColumnSidecars(t *testing.T) {
err = gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{})) err = gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{}))
require.NoError(t, err) require.NoError(t, err)
waiter := verification.NewInitializerWaiter(gs, nil, nil) waiter := verification.NewInitializerWaiter(gs, nil, nil, nil)
initializer, err := waiter.WaitForInitializer(t.Context()) initializer, err := waiter.WaitForInitializer(t.Context())
require.NoError(t, err) require.NoError(t, err)
@@ -787,7 +787,7 @@ func TestVerifyDataColumnSidecarsByPeer(t *testing.T) {
err := gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{})) err := gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{}))
require.NoError(t, err) require.NoError(t, err)
waiter := verification.NewInitializerWaiter(gs, nil, nil) waiter := verification.NewInitializerWaiter(gs, nil, nil, nil)
initializer, err := waiter.WaitForInitializer(t.Context()) initializer, err := waiter.WaitForInitializer(t.Context())
require.NoError(t, err) require.NoError(t, err)
@@ -832,7 +832,7 @@ func TestVerifyDataColumnSidecarsByPeer(t *testing.T) {
err := gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{})) err := gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{}))
require.NoError(t, err) require.NoError(t, err)
waiter := verification.NewInitializerWaiter(gs, nil, nil) waiter := verification.NewInitializerWaiter(gs, nil, nil, nil)
initializer, err := waiter.WaitForInitializer(t.Context()) initializer, err := waiter.WaitForInitializer(t.Context())
require.NoError(t, err) require.NoError(t, err)

View File

@@ -174,7 +174,7 @@ func TestService_InitStartStop(t *testing.T) {
StateNotifier: &mock.MockStateNotifier{}, StateNotifier: &mock.MockStateNotifier{},
InitialSyncComplete: make(chan struct{}), InitialSyncComplete: make(chan struct{}),
}) })
s.verifierWaiter = verification.NewInitializerWaiter(gs, nil, nil) s.verifierWaiter = verification.NewInitializerWaiter(gs, nil, nil, nil)
time.Sleep(500 * time.Millisecond) time.Sleep(500 * time.Millisecond)
assert.NotNil(t, s) assert.NotNil(t, s)
if tt.setGenesis != nil { if tt.setGenesis != nil {
@@ -217,7 +217,7 @@ func TestService_waitForStateInitialization(t *testing.T) {
counter: ratecounter.NewRateCounter(counterSeconds * time.Second), counter: ratecounter.NewRateCounter(counterSeconds * time.Second),
genesisChan: make(chan time.Time), genesisChan: make(chan time.Time),
} }
s.verifierWaiter = verification.NewInitializerWaiter(cs, nil, nil) s.verifierWaiter = verification.NewInitializerWaiter(cs, nil, nil, nil)
return s, cs return s, cs
} }
@@ -786,7 +786,7 @@ func TestFetchOriginColumns(t *testing.T) {
err = gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{})) err = gs.SetClock(startup.NewClock(time.Unix(4113849600, 0), [fieldparams.RootLength]byte{}))
require.NoError(t, err) require.NoError(t, err)
waiter := verification.NewInitializerWaiter(gs, nil, nil) waiter := verification.NewInitializerWaiter(gs, nil, nil, nil)
initializer, err := waiter.WaitForInitializer(t.Context()) initializer, err := waiter.WaitForInitializer(t.Context())
require.NoError(t, err) require.NoError(t, err)

View File

@@ -332,7 +332,7 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
markInitSyncComplete(t, r) markInitSyncComplete(t, r)
clock := startup.NewClockSynchronizer() clock := startup.NewClockSynchronizer()
require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{}))) require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{})))
r.verifierWaiter = verification.NewInitializerWaiter(clock, chain.ForkChoiceStore, r.cfg.stateGen) r.verifierWaiter = verification.NewInitializerWaiter(clock, chain.ForkChoiceStore, r.cfg.stateGen, chain)
p1.Digest, err = r.currentForkDigest() p1.Digest, err = r.currentForkDigest()
require.NoError(t, err) require.NoError(t, err)
@@ -354,7 +354,7 @@ func TestHandshakeHandlers_Roundtrip(t *testing.T) {
markInitSyncComplete(t, r2) markInitSyncComplete(t, r2)
clock = startup.NewClockSynchronizer() clock = startup.NewClockSynchronizer()
require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{}))) require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{})))
r2.verifierWaiter = verification.NewInitializerWaiter(clock, chain2.ForkChoiceStore, r2.cfg.stateGen) r2.verifierWaiter = verification.NewInitializerWaiter(clock, chain2.ForkChoiceStore, r2.cfg.stateGen, chain2)
p2.Digest, err = r.currentForkDigest() p2.Digest, err = r.currentForkDigest()
require.NoError(t, err) require.NoError(t, err)
@@ -948,7 +948,7 @@ func TestStatusRPCRequest_BadPeerHandshake(t *testing.T) {
markInitSyncComplete(t, r) markInitSyncComplete(t, r)
clock := startup.NewClockSynchronizer() clock := startup.NewClockSynchronizer()
require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{}))) require.NoError(t, clock.SetClock(startup.NewClock(time.Now(), [32]byte{})))
r.verifierWaiter = verification.NewInitializerWaiter(clock, chain.ForkChoiceStore, r.cfg.stateGen) r.verifierWaiter = verification.NewInitializerWaiter(clock, chain.ForkChoiceStore, r.cfg.stateGen, chain)
go r.Start() go r.Start()

View File

@@ -433,7 +433,7 @@ func TestService_ValidateBlsToExecutionChange(t *testing.T) {
tt.clock = startup.NewClock(time.Now(), [32]byte{}) tt.clock = startup.NewClock(time.Now(), [32]byte{})
} }
require.NoError(t, cw.SetClock(tt.clock)) require.NoError(t, cw.SetClock(tt.clock))
svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen) svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen, chainService)
go svc.Start() go svc.Start()
marshalledObj, err := tt.args.msg.MarshalSSZ() marshalledObj, err := tt.args.msg.MarshalSSZ()

View File

@@ -411,7 +411,7 @@ func TestService_ValidateSyncCommitteeMessage(t *testing.T) {
svc, tt.args.topic, clock = tt.setupSvc(svc, tt.args.msg, tt.args.topic) svc, tt.args.topic, clock = tt.setupSvc(svc, tt.args.msg, tt.args.topic)
markInitSyncComplete(t, svc) markInitSyncComplete(t, svc)
require.NoError(t, cw.SetClock(clock)) require.NoError(t, cw.SetClock(clock))
svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen) svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen, chainService)
go svc.Start() go svc.Start()
marshalledObj, err := tt.args.msg.MarshalSSZ() marshalledObj, err := tt.args.msg.MarshalSSZ()

View File

@@ -855,7 +855,7 @@ func TestService_ValidateSyncContributionAndProof(t *testing.T) {
var clock *startup.Clock var clock *startup.Clock
svc, clock = tt.setupSvc(svc, tt.args.msg) svc, clock = tt.setupSvc(svc, tt.args.msg)
require.NoError(t, cw.SetClock(clock)) require.NoError(t, cw.SetClock(clock))
svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen) svc.verifierWaiter = verification.NewInitializerWaiter(cw, chainService.ForkChoiceStore, svc.cfg.stateGen, chainService)
markInitSyncComplete(t, svc) markInitSyncComplete(t, svc)
go svc.Start() go svc.Start()
marshalledObj, err := tt.args.msg.MarshalSSZ() marshalledObj, err := tt.args.msg.MarshalSSZ()

View File

@@ -69,6 +69,7 @@ go_test(
"//beacon-chain/forkchoice/types:go_default_library", "//beacon-chain/forkchoice/types:go_default_library",
"//beacon-chain/startup:go_default_library", "//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library", "//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/fieldparams:go_default_library", "//config/fieldparams:go_default_library",
"//config/params:go_default_library", "//config/params:go_default_library",
"//consensus-types/blocks:go_default_library", "//consensus-types/blocks:go_default_library",

View File

@@ -11,6 +11,7 @@ import (
forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types" forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup" "github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state" "github.com/OffchainLabs/prysm/v7/beacon-chain/state"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
"github.com/OffchainLabs/prysm/v7/config/params" "github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks" "github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives" "github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
@@ -626,6 +627,45 @@ func (sbr *mockStateByRooter) StateByRoot(ctx context.Context, root [32]byte) (s
var _ StateByRooter = &mockStateByRooter{} var _ StateByRooter = &mockStateByRooter{}
type mockHeadStateProvider struct {
headRoot []byte
headSlot primitives.Slot
headState state.BeaconState
headStateReadOnly state.ReadOnlyBeaconState
}
func (m *mockHeadStateProvider) HeadRoot(_ context.Context) ([]byte, error) {
if m.headRoot != nil {
return m.headRoot, nil
}
root := make([]byte, 32)
root[0] = 0xff
return root, nil
}
func (m *mockHeadStateProvider) HeadSlot() primitives.Slot {
if m.headSlot == 0 {
return 1000
}
return m.headSlot
}
func (m *mockHeadStateProvider) HeadState(_ context.Context) (state.BeaconState, error) {
if m.headState == nil {
return nil, errors.New("head state not available")
}
return m.headState, nil
}
func (m *mockHeadStateProvider) HeadStateReadOnly(_ context.Context) (state.ReadOnlyBeaconState, error) {
if m.headStateReadOnly == nil {
return nil, errors.New("head state read only not available")
}
return m.headStateReadOnly, nil
}
var _ HeadStateProvider = &mockHeadStateProvider{}
func sbrErrorIfCalled(t *testing.T) sbrfunc { func sbrErrorIfCalled(t *testing.T) sbrfunc {
return func(_ context.Context, _ [32]byte) (state.BeaconState, error) { return func(_ context.Context, _ [32]byte) (state.BeaconState, error) {
t.Error("StateByRoot should not have been called") t.Error("StateByRoot should not have been called")
@@ -643,15 +683,56 @@ func sbrNotFound(t *testing.T, expectedRoot [32]byte) *mockStateByRooter {
} }
func sbrForValOverride(idx primitives.ValidatorIndex, val *ethpb.Validator) *mockStateByRooter { func sbrForValOverride(idx primitives.ValidatorIndex, val *ethpb.Validator) *mockStateByRooter {
return sbrForValOverrideWithT(nil, idx, val)
}
func sbrForValOverrideWithT(t testing.TB, idx primitives.ValidatorIndex, val *ethpb.Validator) *mockStateByRooter {
return &mockStateByRooter{sbr: func(_ context.Context, root [32]byte) (state.BeaconState, error) { return &mockStateByRooter{sbr: func(_ context.Context, root [32]byte) (state.BeaconState, error) {
return &validxStateOverride{vals: map[primitives.ValidatorIndex]*ethpb.Validator{ // Use a real deterministic state so that helpers.BeaconProposerIndexAtSlot works correctly
idx: val, numValidators := uint64(idx + 1)
}}, nil if numValidators < 64 {
numValidators = 64
}
var st state.BeaconState
var err error
if t != nil {
st, _ = util.DeterministicGenesisStateFulu(t, numValidators)
} else {
// Fallback for blob tests that don't need the full state
return &validxStateOverride{
slot: 0,
vals: map[primitives.ValidatorIndex]*ethpb.Validator{
idx: val,
},
}, nil
}
// Override the specific validator if provided
if val != nil {
vals := st.Validators()
if idx < primitives.ValidatorIndex(len(vals)) {
vals[idx] = val
// Ensure the validator is active
if vals[idx].ActivationEpoch > 0 {
vals[idx].ActivationEpoch = 0
}
if vals[idx].ExitEpoch == 0 || vals[idx].ExitEpoch < params.BeaconConfig().FarFutureEpoch {
vals[idx].ExitEpoch = params.BeaconConfig().FarFutureEpoch
}
if vals[idx].EffectiveBalance == 0 {
vals[idx].EffectiveBalance = params.BeaconConfig().MaxEffectiveBalance
}
_ = st.SetValidators(vals)
}
}
return st, err
}} }}
} }
type validxStateOverride struct { type validxStateOverride struct {
state.BeaconState state.BeaconState
slot primitives.Slot
vals map[primitives.ValidatorIndex]*ethpb.Validator vals map[primitives.ValidatorIndex]*ethpb.Validator
} }
@@ -665,6 +746,105 @@ func (v *validxStateOverride) ValidatorAtIndex(idx primitives.ValidatorIndex) (*
return val, nil return val, nil
} }
func (v *validxStateOverride) Slot() primitives.Slot {
return v.slot
}
func (v *validxStateOverride) Version() int {
// Return Fulu version (6) as default for tests
return 6
}
func (v *validxStateOverride) Validators() []*ethpb.Validator {
// Return all validators in the map as a slice
maxIdx := primitives.ValidatorIndex(0)
for idx := range v.vals {
if idx > maxIdx {
maxIdx = idx
}
}
// Ensure we have at least 64 validators for a valid beacon state
numValidators := maxIdx + 1
if numValidators < 64 {
numValidators = 64
}
validators := make([]*ethpb.Validator, numValidators)
for i := range validators {
if val, ok := v.vals[primitives.ValidatorIndex(i)]; ok {
validators[i] = val
} else {
// Default validator for indices we don't care about
validators[i] = &ethpb.Validator{
ActivationEpoch: 0,
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
EffectiveBalance: params.BeaconConfig().MaxEffectiveBalance,
}
}
}
return validators
}
func (v *validxStateOverride) RandaoMixAtIndex(idx uint64) ([]byte, error) {
// Return a zero mix for simplicity in tests
return make([]byte, 32), nil
}
func (v *validxStateOverride) NumValidators() int {
return len(v.Validators())
}
func (v *validxStateOverride) ValidatorAtIndexReadOnly(idx primitives.ValidatorIndex) (state.ReadOnlyValidator, error) {
validators := v.Validators()
if idx >= primitives.ValidatorIndex(len(validators)) {
return nil, fmt.Errorf("validator index %d out of range", idx)
}
return state_native.NewValidator(validators[idx])
}
func (v *validxStateOverride) IsNil() bool {
return false
}
func (v *validxStateOverride) LatestBlockHeader() *ethpb.BeaconBlockHeader {
// Return a minimal block header for tests
return &ethpb.BeaconBlockHeader{
Slot: v.slot,
ProposerIndex: 0,
ParentRoot: make([]byte, 32),
StateRoot: make([]byte, 32),
BodyRoot: make([]byte, 32),
}
}
func (v *validxStateOverride) HashTreeRoot(ctx context.Context) ([32]byte, error) {
// Return a zero hash for tests
return [32]byte{}, nil
}
func (v *validxStateOverride) UpdateStateRootAtIndex(idx uint64, stateRoot [32]byte) error {
// No-op for mock - we don't track state roots
return nil
}
func (v *validxStateOverride) SetLatestBlockHeader(val *ethpb.BeaconBlockHeader) error {
// No-op for mock - we don't track block headers
return nil
}
func (v *validxStateOverride) ReadFromEveryValidator(f func(idx int, val state.ReadOnlyValidator) error) error {
validators := v.Validators()
for i, val := range validators {
rov, err := state_native.NewValidator(val)
if err != nil {
return err
}
if err := f(i, rov); err != nil {
return err
}
}
return nil
}
type mockProposerCache struct { type mockProposerCache struct {
ComputeProposerCB func(ctx context.Context, root [32]byte, slot primitives.Slot, pst state.BeaconState) (primitives.ValidatorIndex, error) ComputeProposerCB func(ctx context.Context, root [32]byte, slot primitives.Slot, pst state.BeaconState) (primitives.ValidatorIndex, error)
ProposerCB func(c *forkchoicetypes.Checkpoint, slot primitives.Slot) (primitives.ValidatorIndex, bool) ProposerCB func(c *forkchoicetypes.Checkpoint, slot primitives.Slot) (primitives.ValidatorIndex, bool)

View File

@@ -1,13 +1,16 @@
package verification package verification
import ( import (
"bytes"
"context" "context"
"crypto/sha256" "crypto/sha256"
"fmt" "fmt"
"strings" "strings"
"time" "time"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/transition"
forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types" forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state" "github.com/OffchainLabs/prysm/v7/beacon-chain/state"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams" fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
@@ -66,6 +69,12 @@ var (
errBadTopic = errors.New("topic is not of the one expected") errBadTopic = errors.New("topic is not of the one expected")
) )
type LazyHeadStateProvider struct {
HeadStateProvider
}
var _ HeadStateProvider = &LazyHeadStateProvider{}
type ( type (
RODataColumnsVerifier struct { RODataColumnsVerifier struct {
*sharedResources *sharedResources
@@ -262,14 +271,14 @@ func (dv *RODataColumnsVerifier) ValidProposerSignature(ctx context.Context) (er
if _, err, _ = dv.sg.Do(signatureData.concat(), func() (any, error) { if _, err, _ = dv.sg.Do(signatureData.concat(), func() (any, error) {
columnVerificationProposerSignatureCache.WithLabelValues("miss").Inc() columnVerificationProposerSignatureCache.WithLabelValues("miss").Inc()
// Retrieve the parent state. // Retrieve a state compatible with the data column for verification.
parentState, err := dv.state(ctx, dataColumn.ParentRoot()) verifyingState, err := dv.getVerifyingState(ctx, dataColumn)
if err != nil { if err != nil {
return nil, columnErrBuilder(errors.Wrap(err, "parent state")) return nil, columnErrBuilder(errors.Wrap(err, "verifying state"))
} }
// Full verification, which will subsequently be cached for anything sharing the signature cache. // Full verification, which will subsequently be cached for anything sharing the signature cache.
if err = dv.sc.VerifySignature(signatureData, parentState); err != nil { if err = dv.sc.VerifySignature(signatureData, verifyingState); err != nil {
return nil, columnErrBuilder(errors.Wrap(err, "verify signature")) return nil, columnErrBuilder(errors.Wrap(err, "verify signature"))
} }
@@ -282,6 +291,61 @@ func (dv *RODataColumnsVerifier) ValidProposerSignature(ctx context.Context) (er
return nil return nil
} }
// getVerifyingState returns a state that is compatible with the column sidecar and can be used to verify signature and proposer index.
// The returned state is guaranteed to be at the same epoch as the data column's epoch, and have the same randao mix and active
// validator indices as the data column's parent state advanced to the data column's slot.
func (dv *RODataColumnsVerifier) getVerifyingState(ctx context.Context, dataColumn blocks.RODataColumn) (state.ReadOnlyBeaconState, error) {
headRoot, err := dv.hsp.HeadRoot(ctx)
if err != nil {
return nil, err
}
parentRoot := dataColumn.ParentRoot()
dataColumnSlot := dataColumn.Slot()
dataColumnEpoch := slots.ToEpoch(dataColumnSlot)
headSlot := dv.hsp.HeadSlot()
headEpoch := slots.ToEpoch(headSlot)
// Use head if it's the parent
if bytes.Equal(parentRoot[:], headRoot) {
// If they are in the same epoch, then we can return the head state directly
if dataColumnEpoch == headEpoch {
return dv.hsp.HeadStateReadOnly(ctx)
}
// Otherwise, we need to process the head state to the data column's slot
headState, err := dv.hsp.HeadState(ctx)
if err != nil {
return nil, err
}
return transition.ProcessSlotsUsingNextSlotCache(ctx, headState, headRoot, dataColumnSlot)
}
// If head and data column are in the same epoch and head is compatible with the parent's target, then use head
if dataColumnEpoch == headEpoch {
headTarget, err := dv.fc.TargetRootForEpoch(bytesutil.ToBytes32(headRoot), dataColumnEpoch)
if err != nil {
return nil, err
}
parentTarget, err := dv.fc.TargetRootForEpoch(parentRoot, dataColumnEpoch)
if err != nil {
return nil, err
}
if bytes.Equal(headTarget[:], parentTarget[:]) {
return dv.hsp.HeadStateReadOnly(ctx)
}
}
// Otherwise retrieve the parent state and advance it to the data column's slot
parentState, err := dv.sr.StateByRoot(ctx, parentRoot)
if err != nil {
return nil, err
}
parentEpoch := slots.ToEpoch(parentState.Slot())
if dataColumnEpoch == parentEpoch {
return parentState, nil
}
return transition.ProcessSlotsUsingNextSlotCache(ctx, parentState, parentRoot[:], dataColumnSlot)
}
func (dv *RODataColumnsVerifier) SidecarParentSeen(parentSeen func([fieldparams.RootLength]byte) bool) (err error) { func (dv *RODataColumnsVerifier) SidecarParentSeen(parentSeen func([fieldparams.RootLength]byte) bool) (err error) {
if ok, err := dv.results.cached(RequireSidecarParentSeen); ok { if ok, err := dv.results.cached(RequireSidecarParentSeen); ok {
return err return err
@@ -482,14 +546,12 @@ func (dv *RODataColumnsVerifier) SidecarProposerExpected(ctx context.Context) (e
// Ensure the expensive index computation is only performed once for // Ensure the expensive index computation is only performed once for
// concurrent requests for the same signature data. // concurrent requests for the same signature data.
idxAny, err, _ := dv.sg.Do(concatRootSlot(parentRoot, dataColumnSlot), func() (any, error) { idxAny, err, _ := dv.sg.Do(concatRootSlot(parentRoot, dataColumnSlot), func() (any, error) {
// Retrieve the parent state. verifyingState, err := dv.getVerifyingState(ctx, dataColumn)
parentState, err := dv.state(ctx, parentRoot)
if err != nil { if err != nil {
return nil, columnErrBuilder(errors.Wrap(err, "parent state")) return nil, columnErrBuilder(errors.Wrap(err, "verifying state"))
} }
// Compute the proposer index. idx, err = helpers.BeaconProposerIndexAtSlot(ctx, verifyingState, dataColumnSlot)
idx, err = dv.pc.ComputeProposer(ctx, parentRoot, dataColumnSlot, parentState)
if err != nil { if err != nil {
return nil, columnErrBuilder(errors.Wrap(err, "compute proposer")) return nil, columnErrBuilder(errors.Wrap(err, "compute proposer"))
} }
@@ -514,25 +576,6 @@ func (dv *RODataColumnsVerifier) SidecarProposerExpected(ctx context.Context) (e
return nil return nil
} }
// state retrieves the state of the corresponding root from the cache if possible, else retrieves it from the state by rooter.
func (dv *RODataColumnsVerifier) state(ctx context.Context, root [fieldparams.RootLength]byte) (state.BeaconState, error) {
// If the parent root is already in the cache, return it.
if st, ok := dv.stateByRoot[root]; ok {
return st, nil
}
// Retrieve the parent state from the state by rooter.
st, err := dv.sr.StateByRoot(ctx, root)
if err != nil {
return nil, errors.Wrap(err, "state by root")
}
// Store the parent state in the cache.
dv.stateByRoot[root] = st
return st, nil
}
func columnToSignatureData(d blocks.RODataColumn) signatureData { func columnToSignatureData(d blocks.RODataColumn) signatureData {
return signatureData{ return signatureData{
Root: d.BlockRoot(), Root: d.BlockRoot(),

View File

@@ -1,7 +1,6 @@
package verification package verification
import ( import (
"context"
"reflect" "reflect"
"sync" "sync"
"testing" "testing"
@@ -11,7 +10,6 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas" "github.com/OffchainLabs/prysm/v7/beacon-chain/core/peerdas"
forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types" forkchoicetypes "github.com/OffchainLabs/prysm/v7/beacon-chain/forkchoice/types"
"github.com/OffchainLabs/prysm/v7/beacon-chain/startup" "github.com/OffchainLabs/prysm/v7/beacon-chain/startup"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams" fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/config/params" "github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks" "github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
@@ -330,7 +328,7 @@ func TestValidProposerSignature(t *testing.T) {
svcbError: nil, svcbError: nil,
vscbShouldError: false, vscbShouldError: false,
vscbError: nil, vscbError: nil,
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator), stateByRooter: sbrForValOverrideWithT(t, firstColumn.ProposerIndex(), validator),
isError: false, isError: false,
}, },
{ {
@@ -348,7 +346,7 @@ func TestValidProposerSignature(t *testing.T) {
svcbError: nil, svcbError: nil,
vscbShouldError: false, vscbShouldError: false,
vscbError: errors.New("signature, not so good!"), vscbError: errors.New("signature, not so good!"),
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator), stateByRooter: sbrForValOverrideWithT(t, firstColumn.ProposerIndex(), validator),
isError: true, isError: true,
}, },
} }
@@ -378,8 +376,12 @@ func TestValidProposerSignature(t *testing.T) {
initializer := Initializer{ initializer := Initializer{
shared: &sharedResources{ shared: &sharedResources{
sc: signatureCache, sc: signatureCache,
sr: tc.stateByRooter, sr: tc.stateByRooter,
hsp: &mockHeadStateProvider{},
fc: &mockForkchoicer{
TargetRootForEpochCB: fcReturnsTargetRoot([fieldparams.RootLength]byte{}),
},
}, },
} }
@@ -796,20 +798,7 @@ func TestDataColumnsSidecarProposerExpected(t *testing.T) {
parentRoot := [fieldparams.RootLength]byte{} parentRoot := [fieldparams.RootLength]byte{}
columns := GenerateTestDataColumns(t, parentRoot, columnSlot, blobCount) columns := GenerateTestDataColumns(t, parentRoot, columnSlot, blobCount)
firstColumn := columns[0] firstColumn := columns[0]
newColumns := GenerateTestDataColumns(t, parentRoot, 2*params.BeaconConfig().SlotsPerEpoch, blobCount)
firstNewColumn := newColumns[0]
validator := &ethpb.Validator{}
commonComputeProposerCB := func(_ context.Context, root [fieldparams.RootLength]byte, slot primitives.Slot, _ state.BeaconState) (primitives.ValidatorIndex, error) {
require.Equal(t, firstColumn.ParentRoot(), root)
require.Equal(t, firstColumn.Slot(), slot)
return firstColumn.ProposerIndex(), nil
}
ctx := t.Context() ctx := t.Context()
testCases := []struct { testCases := []struct {
name string name string
stateByRooter StateByRooter stateByRooter StateByRooter
@@ -841,66 +830,7 @@ func TestDataColumnsSidecarProposerExpected(t *testing.T) {
ProposerCB: pcReturnsNotFound(), ProposerCB: pcReturnsNotFound(),
}, },
columns: columns, columns: columns,
error: "state by root", error: "verifying state",
},
{
name: "Not cached, proposer matches",
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator),
proposerCache: &mockProposerCache{
ProposerCB: pcReturnsNotFound(),
ComputeProposerCB: commonComputeProposerCB,
},
columns: columns,
},
{
name: "Not cached, proposer matches",
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator),
proposerCache: &mockProposerCache{
ProposerCB: pcReturnsNotFound(),
ComputeProposerCB: commonComputeProposerCB,
},
columns: columns,
},
{
name: "Not cached, proposer matches for next epoch",
stateByRooter: sbrForValOverride(firstNewColumn.ProposerIndex(), validator),
proposerCache: &mockProposerCache{
ProposerCB: pcReturnsNotFound(),
ComputeProposerCB: func(_ context.Context, root [32]byte, slot primitives.Slot, _ state.BeaconState) (primitives.ValidatorIndex, error) {
require.Equal(t, firstNewColumn.ParentRoot(), root)
require.Equal(t, firstNewColumn.Slot(), slot)
return firstColumn.ProposerIndex(), nil
},
},
columns: newColumns,
},
{
name: "Not cached, proposer does not match",
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator),
proposerCache: &mockProposerCache{
ProposerCB: pcReturnsNotFound(),
ComputeProposerCB: func(_ context.Context, root [32]byte, slot primitives.Slot, _ state.BeaconState) (primitives.ValidatorIndex, error) {
require.Equal(t, firstColumn.ParentRoot(), root)
require.Equal(t, firstColumn.Slot(), slot)
return firstColumn.ProposerIndex() + 1, nil
},
},
columns: columns,
error: errSidecarUnexpectedProposer.Error(),
},
{
name: "Not cached, ComputeProposer fails",
stateByRooter: sbrForValOverride(firstColumn.ProposerIndex(), validator),
proposerCache: &mockProposerCache{
ProposerCB: pcReturnsNotFound(),
ComputeProposerCB: func(_ context.Context, root [32]byte, slot primitives.Slot, _ state.BeaconState) (primitives.ValidatorIndex, error) {
require.Equal(t, firstColumn.ParentRoot(), root)
require.Equal(t, firstColumn.Slot(), slot)
return 0, errors.New("ComputeProposer failed")
},
},
columns: columns,
error: "compute proposer",
}, },
} }
@@ -908,8 +838,9 @@ func TestDataColumnsSidecarProposerExpected(t *testing.T) {
t.Run(tc.name, func(t *testing.T) { t.Run(tc.name, func(t *testing.T) {
initializer := Initializer{ initializer := Initializer{
shared: &sharedResources{ shared: &sharedResources{
sr: tc.stateByRooter, sr: tc.stateByRooter,
pc: tc.proposerCache, pc: tc.proposerCache,
hsp: &mockHeadStateProvider{},
fc: &mockForkchoicer{ fc: &mockForkchoicer{
TargetRootForEpochCB: fcReturnsTargetRoot([fieldparams.RootLength]byte{}), TargetRootForEpochCB: fcReturnsTargetRoot([fieldparams.RootLength]byte{}),
}, },

View File

@@ -33,6 +33,16 @@ type StateByRooter interface {
StateByRoot(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error) StateByRoot(ctx context.Context, blockRoot [32]byte) (state.BeaconState, error)
} }
// HeadStateProvider describes a type that can provide access to the current head state and related methods.
// This interface matches blockchain.HeadFetcher but is defined here to avoid import cycles
// (blockchain package imports verification package).
type HeadStateProvider interface {
HeadRoot(ctx context.Context) ([]byte, error)
HeadSlot() primitives.Slot
HeadState(ctx context.Context) (state.BeaconState, error)
HeadStateReadOnly(ctx context.Context) (state.ReadOnlyBeaconState, error)
}
// sharedResources provides access to resources that are required by different verification types. // sharedResources provides access to resources that are required by different verification types.
// for example, sidecar verification and block verification share the block signature verification cache. // for example, sidecar verification and block verification share the block signature verification cache.
type sharedResources struct { type sharedResources struct {
@@ -41,6 +51,7 @@ type sharedResources struct {
sc signatureCache sc signatureCache
pc proposerCache pc proposerCache
sr StateByRooter sr StateByRooter
hsp HeadStateProvider
ic *inclusionProofCache ic *inclusionProofCache
sg singleflight.Group sg singleflight.Group
} }
@@ -96,14 +107,15 @@ func WithForkLookup(fl forkLookup) InitializerOption {
} }
// NewInitializerWaiter creates an InitializerWaiter which can be used to obtain an Initializer once async dependencies are ready. // NewInitializerWaiter creates an InitializerWaiter which can be used to obtain an Initializer once async dependencies are ready.
func NewInitializerWaiter(cw startup.ClockWaiter, fc Forkchoicer, sr StateByRooter, opts ...InitializerOption) *InitializerWaiter { func NewInitializerWaiter(cw startup.ClockWaiter, fc Forkchoicer, sr StateByRooter, hsp HeadStateProvider, opts ...InitializerOption) *InitializerWaiter {
pc := newPropCache() pc := newPropCache()
// signature cache is initialized in WaitForInitializer, since we need the genesis validators root, which can be obtained from startup.Clock. // signature cache is initialized in WaitForInitializer, since we need the genesis validators root, which can be obtained from startup.Clock.
shared := &sharedResources{ shared := &sharedResources{
fc: fc, fc: fc,
pc: pc, pc: pc,
sr: sr, sr: sr,
ic: newInclusionProofCache(defaultInclusionProofCacheSize), hsp: hsp,
ic: newInclusionProofCache(defaultInclusionProofCacheSize),
} }
iw := &InitializerWaiter{cw: cw, ini: &Initializer{shared: shared}} iw := &InitializerWaiter{cw: cw, ini: &Initializer{shared: shared}}
for _, o := range opts { for _, o := range opts {

View File

@@ -18,7 +18,7 @@ func TestInitializerWaiter(t *testing.T) {
cs := startup.NewClockSynchronizer() cs := startup.NewClockSynchronizer()
require.NoError(t, cs.SetClock(c)) require.NoError(t, cs.SetClock(c))
w := NewInitializerWaiter(cs, &mockForkchoicer{}, &mockStateByRooter{}) w := NewInitializerWaiter(cs, &mockForkchoicer{}, &mockStateByRooter{}, &mockHeadStateProvider{})
ini, err := w.WaitForInitializer(ctx) ini, err := w.WaitForInitializer(ctx)
require.NoError(t, err) require.NoError(t, err)
csc, ok := ini.shared.sc.(*sigCache) csc, ok := ini.shared.sc.(*sigCache)

View File

@@ -0,0 +1,3 @@
### Changed
- Use head state readonly when possible to validate data column sidecars.

View File

@@ -41,7 +41,7 @@ func NewBuilder(t testing.TB, initialState state.BeaconState, initialBlock inter
getFork := func(targetEpoch primitives.Epoch) (*ethpb.Fork, error) { getFork := func(targetEpoch primitives.Epoch) (*ethpb.Fork, error) {
return initialState.Fork(), nil return initialState.Fork(), nil
} }
bvw := verification.NewInitializerWaiter(cw, fc, sg, verification.WithForkLookup(getFork)) bvw := verification.NewInitializerWaiter(cw, fc, sg, service, verification.WithForkLookup(getFork))
return &Builder{ return &Builder{
service: service, service: service,
execMock: execMock, execMock: execMock,