Add feature flag to verify signatures before proposing

Adds a feature flag --enable-proposer-preprocessing to verify individual signatures in the block right before proposing to fallback to empty fields in case of failure.
This commit is contained in:
Potuz
2025-12-31 08:02:51 -03:00
parent d4ea8fafd6
commit 1063e27d7a
16 changed files with 475 additions and 134 deletions

View File

@@ -11,7 +11,6 @@ go_library(
"error.go",
"registry_updates.go",
"transition.go",
"transition_no_verify_sig.go",
"upgrade.go",
"validator.go",
"withdrawals.go",
@@ -61,7 +60,6 @@ go_test(
"error_test.go",
"export_test.go",
"registry_updates_test.go",
"transition_no_verify_sig_test.go",
"transition_test.go",
"upgrade_test.go",
"validator_test.go",

View File

@@ -6,6 +6,11 @@ type execReqErr struct {
error
}
// NewExecReqError creates a new execReqErr.
func NewExecReqError(msg string) error {
return execReqErr{errors.New(msg)}
}
// IsExecutionRequestError returns true if the error has `execReqErr`.
func IsExecutionRequestError(e error) bool {
if e == nil {

View File

@@ -1,60 +0,0 @@
package electra_test
import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/electra"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
)
func TestProcessOperationsWithNilRequests(t *testing.T) {
tests := []struct {
name string
modifyBlk func(blockElectra *ethpb.SignedBeaconBlockElectra)
errMsg string
}{
{
name: "Nil deposit request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Deposits = []*enginev1.DepositRequest{nil}
},
errMsg: "nil deposit request",
},
{
name: "Nil withdrawal request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Withdrawals = []*enginev1.WithdrawalRequest{nil}
},
errMsg: "nil withdrawal request",
},
{
name: "Nil consolidation request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Consolidations = []*enginev1.ConsolidationRequest{nil}
},
errMsg: "nil consolidation request",
},
}
for _, tc := range tests {
t.Run(tc.name, func(t *testing.T) {
st, ks := util.DeterministicGenesisStateElectra(t, 128)
blk, err := util.GenerateFullBlockElectra(st, ks, util.DefaultBlockGenConfig(), 1)
require.NoError(t, err)
tc.modifyBlk(blk)
b, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
require.NoError(t, st.SetSlot(1))
_, err = electra.ProcessOperations(t.Context(), st, b.Block())
require.ErrorContains(t, tc.errMsg, err)
})
}
}

View File

@@ -3,6 +3,8 @@ load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"electra.go",
"errors.go",
"log.go",
"skip_slot_cache.go",
"state.go",
@@ -62,6 +64,8 @@ go_test(
"altair_transition_no_verify_sig_test.go",
"bellatrix_transition_no_verify_sig_test.go",
"benchmarks_test.go",
"electra_test.go",
"exports_test.go",
"skip_slot_cache_test.go",
"state_fuzz_test.go",
"state_test.go",

View File

@@ -1,9 +1,10 @@
package electra
package transition
import (
"context"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/electra"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
v "github.com/OffchainLabs/prysm/v7/beacon-chain/core/validators"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
@@ -47,7 +48,7 @@ var (
// # [New in Electra:EIP7251]
// for_ops(body.execution_payload.consolidation_requests, process_consolidation_request)
func ProcessOperations(ctx context.Context, st state.BeaconState, block interfaces.ReadOnlyBeaconBlock) (state.BeaconState, error) {
func electraOperations(ctx context.Context, st state.BeaconState, block interfaces.ReadOnlyBeaconBlock) (state.BeaconState, error) {
var err error
// 6110 validations are in VerifyOperationLengths
@@ -63,59 +64,60 @@ func ProcessOperations(ctx context.Context, st state.BeaconState, block interfac
return nil, errors.Wrap(err, "could not update total active balance cache")
}
}
st, err = ProcessProposerSlashings(ctx, st, bb.ProposerSlashings(), exitInfo)
st, err = blocks.ProcessProposerSlashings(ctx, st, bb.ProposerSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process altair proposer slashing")
return nil, errors.Wrap(ErrProcessProposerSlashingsFailed, err.Error())
}
st, err = ProcessAttesterSlashings(ctx, st, bb.AttesterSlashings(), exitInfo)
st, err = blocks.ProcessAttesterSlashings(ctx, st, bb.AttesterSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process altair attester slashing")
return nil, errors.Wrap(ErrProcessAttesterSlashingsFailed, err.Error())
}
st, err = ProcessAttestationsNoVerifySignature(ctx, st, block)
st, err = electra.ProcessAttestationsNoVerifySignature(ctx, st, block)
if err != nil {
return nil, errors.Wrap(err, "could not process altair attestation")
return nil, errors.Wrap(ErrProcessAttestationsFailed, err.Error())
}
if _, err := ProcessDeposits(ctx, st, bb.Deposits()); err != nil { // new in electra
return nil, errors.Wrap(err, "could not process altair deposit")
if _, err := electra.ProcessDeposits(ctx, st, bb.Deposits()); err != nil {
return nil, errors.Wrap(ErrProcessDepositsFailed, err.Error())
}
st, err = ProcessVoluntaryExits(ctx, st, bb.VoluntaryExits(), exitInfo)
st, err = blocks.ProcessVoluntaryExits(ctx, st, bb.VoluntaryExits(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process voluntary exits")
return nil, errors.Wrap(ErrProcessVoluntaryExitsFailed, err.Error())
}
st, err = ProcessBLSToExecutionChanges(st, block)
st, err = blocks.ProcessBLSToExecutionChanges(st, block)
if err != nil {
return nil, errors.Wrap(err, "could not process bls-to-execution changes")
return nil, errors.Wrap(ErrProcessBLSChangesFailed, err.Error())
}
// new in electra
requests, err := bb.ExecutionRequests()
if err != nil {
return nil, errors.Wrap(err, "could not get execution requests")
return nil, electra.NewExecReqError(errors.Wrap(err, "could not get execution requests").Error())
}
for _, d := range requests.Deposits {
if d == nil {
return nil, errors.New("nil deposit request")
return nil, electra.NewExecReqError("nil deposit request")
}
}
st, err = ProcessDepositRequests(ctx, st, requests.Deposits)
st, err = electra.ProcessDepositRequests(ctx, st, requests.Deposits)
if err != nil {
return nil, execReqErr{errors.Wrap(err, "could not process deposit requests")}
return nil, electra.NewExecReqError(errors.Wrap(err, "could not process deposit requests").Error())
}
for _, w := range requests.Withdrawals {
if w == nil {
return nil, errors.New("nil withdrawal request")
return nil, electra.NewExecReqError("nil withdrawal request")
}
}
st, err = ProcessWithdrawalRequests(ctx, st, requests.Withdrawals)
st, err = electra.ProcessWithdrawalRequests(ctx, st, requests.Withdrawals)
if err != nil {
return nil, execReqErr{errors.Wrap(err, "could not process withdrawal requests")}
return nil, electra.NewExecReqError(errors.Wrap(err, "could not process withdrawal requests").Error())
}
for _, c := range requests.Consolidations {
if c == nil {
return nil, errors.New("nil consolidation request")
return nil, electra.NewExecReqError("nil consolidation request")
}
}
if err := ProcessConsolidationRequests(ctx, st, requests.Consolidations); err != nil {
return nil, execReqErr{errors.Wrap(err, "could not process consolidation requests")}
if err := electra.ProcessConsolidationRequests(ctx, st, requests.Consolidations); err != nil {
return nil, electra.NewExecReqError(errors.Wrap(err, "could not process consolidation requests").Error())
}
return st, nil
}

View File

@@ -0,0 +1,216 @@
package transition_test
import (
"context"
"errors"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/transition"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/util"
)
func TestProcessOperationsWithNilRequests(t *testing.T) {
tests := []struct {
name string
modifyBlk func(blockElectra *ethpb.SignedBeaconBlockElectra)
errMsg string
}{
{
name: "Nil deposit request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Deposits = []*enginev1.DepositRequest{nil}
},
errMsg: "nil deposit request",
},
{
name: "Nil withdrawal request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Withdrawals = []*enginev1.WithdrawalRequest{nil}
},
errMsg: "nil withdrawal request",
},
{
name: "Nil consolidation request",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
blk.Block.Body.ExecutionRequests.Consolidations = []*enginev1.ConsolidationRequest{nil}
},
errMsg: "nil consolidation request",
},
}
for _, tc := range tests {
t.Run(tc.name, func(t *testing.T) {
st, ks := util.DeterministicGenesisStateElectra(t, 128)
blk, err := util.GenerateFullBlockElectra(st, ks, util.DefaultBlockGenConfig(), 1)
require.NoError(t, err)
tc.modifyBlk(blk)
b, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
require.NoError(t, st.SetSlot(1))
_, err = transition.ElectraOperations(t.Context(), st, b.Block())
require.ErrorContains(t, tc.errMsg, err)
})
}
}
func TestElectraOperations_ProcessingErrors(t *testing.T) {
tests := []struct {
name string
modifyBlk func(blk *ethpb.SignedBeaconBlockElectra)
errCheck func(t *testing.T, err error)
}{
{
name: "ErrProcessProposerSlashingsFailed",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
// Create invalid proposer slashing with out-of-bounds proposer index
blk.Block.Body.ProposerSlashings = []*ethpb.ProposerSlashing{
{
Header_1: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: 1,
ProposerIndex: 999999, // Invalid index (out of bounds)
ParentRoot: make([]byte, 32),
StateRoot: make([]byte, 32),
BodyRoot: make([]byte, 32),
},
Signature: make([]byte, 96),
},
Header_2: &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: 1,
ProposerIndex: 999999,
ParentRoot: make([]byte, 32),
StateRoot: make([]byte, 32),
BodyRoot: make([]byte, 32),
},
Signature: make([]byte, 96),
},
},
}
},
errCheck: func(t *testing.T, err error) {
require.ErrorContains(t, "process proposer slashings failed", err)
require.Equal(t, true, errors.Is(err, transition.ErrProcessProposerSlashingsFailed))
},
},
{
name: "ErrProcessAttestationsFailed",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
// Create attestation with invalid committee index
blk.Block.Body.Attestations = []*ethpb.AttestationElectra{
{
AggregationBits: []byte{0b00000001},
Data: &ethpb.AttestationData{
Slot: 1,
CommitteeIndex: 999999, // Invalid committee index
BeaconBlockRoot: make([]byte, 32),
Source: &ethpb.Checkpoint{
Epoch: 0,
Root: make([]byte, 32),
},
Target: &ethpb.Checkpoint{
Epoch: 0,
Root: make([]byte, 32),
},
},
CommitteeBits: []byte{0b00000001},
Signature: make([]byte, 96),
},
}
},
errCheck: func(t *testing.T, err error) {
require.ErrorContains(t, "process attestations failed", err)
require.Equal(t, true, errors.Is(err, transition.ErrProcessAttestationsFailed))
},
},
{
name: "ErrProcessDepositsFailed",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
// Create deposit with invalid proof length
blk.Block.Body.Deposits = []*ethpb.Deposit{
{
Proof: [][]byte{}, // Invalid: empty proof
Data: &ethpb.Deposit_Data{
PublicKey: make([]byte, 48),
WithdrawalCredentials: make([]byte, 32),
Amount: 32000000000, // 32 ETH in Gwei
Signature: make([]byte, 96),
},
},
}
},
errCheck: func(t *testing.T, err error) {
require.ErrorContains(t, "process deposits failed", err)
require.Equal(t, true, errors.Is(err, transition.ErrProcessDepositsFailed))
},
},
{
name: "ErrProcessVoluntaryExitsFailed",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
// Create voluntary exit with invalid validator index
blk.Block.Body.VoluntaryExits = []*ethpb.SignedVoluntaryExit{
{
Exit: &ethpb.VoluntaryExit{
Epoch: 0,
ValidatorIndex: 999999, // Invalid index (out of bounds)
},
Signature: make([]byte, 96),
},
}
},
errCheck: func(t *testing.T, err error) {
require.ErrorContains(t, "process voluntary exits failed", err)
require.Equal(t, true, errors.Is(err, transition.ErrProcessVoluntaryExitsFailed))
},
},
{
name: "ErrProcessBLSChangesFailed",
modifyBlk: func(blk *ethpb.SignedBeaconBlockElectra) {
// Create BLS to execution change with invalid validator index
blk.Block.Body.BlsToExecutionChanges = []*ethpb.SignedBLSToExecutionChange{
{
Message: &ethpb.BLSToExecutionChange{
ValidatorIndex: 999999, // Invalid index (out of bounds)
FromBlsPubkey: make([]byte, 48),
ToExecutionAddress: make([]byte, 20),
},
Signature: make([]byte, 96),
},
}
},
errCheck: func(t *testing.T, err error) {
require.ErrorContains(t, "process BLS to execution changes failed", err)
require.Equal(t, true, errors.Is(err, transition.ErrProcessBLSChangesFailed))
},
},
}
for _, tc := range tests {
t.Run(tc.name, func(t *testing.T) {
ctx := context.Background()
st, ks := util.DeterministicGenesisStateElectra(t, 128)
blk, err := util.GenerateFullBlockElectra(st, ks, util.DefaultBlockGenConfig(), 1)
require.NoError(t, err)
tc.modifyBlk(blk)
b, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
require.NoError(t, st.SetSlot(primitives.Slot(1)))
_, err = transition.ElectraOperations(ctx, st, b.Block())
require.NotNil(t, err, "Expected an error but got nil")
tc.errCheck(t, err)
})
}
}

View File

@@ -0,0 +1,19 @@
package transition
import "errors"
var (
ErrAttestationsSignatureInvalid = errors.New("attestations signature invalid")
ErrRandaoSignatureInvalid = errors.New("randao signature invalid")
ErrBLSToExecutionChangesSignatureInvalid = errors.New("BLS to execution changes signature invalid")
ErrProcessWithdrawalsFailed = errors.New("process withdrawals failed")
ErrProcessRandaoFailed = errors.New("process randao failed")
ErrProcessEth1DataFailed = errors.New("process eth1 data failed")
ErrProcessProposerSlashingsFailed = errors.New("process proposer slashings failed")
ErrProcessAttesterSlashingsFailed = errors.New("process attester slashings failed")
ErrProcessAttestationsFailed = errors.New("process attestations failed")
ErrProcessDepositsFailed = errors.New("process deposits failed")
ErrProcessVoluntaryExitsFailed = errors.New("process voluntary exits failed")
ErrProcessBLSChangesFailed = errors.New("process BLS to execution changes failed")
ErrProcessSyncAggregateFailed = errors.New("process sync aggregate failed")
)

View File

@@ -0,0 +1,3 @@
package transition
var ElectraOperations = electraOperations

View File

@@ -7,12 +7,11 @@ import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/altair"
b "github.com/OffchainLabs/prysm/v7/beacon-chain/core/blocks"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/electra"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/transition/interop"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/validators"
v "github.com/OffchainLabs/prysm/v7/beacon-chain/core/validators"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
@@ -70,10 +69,11 @@ func ExecuteStateTransitionNoVerifyAnySig(
}
// Execute per block transition.
set, st, err := ProcessBlockNoVerifyAnySig(ctx, st, signed)
sigSlice, st, err := ProcessBlockNoVerifyAnySig(ctx, st, signed)
if err != nil {
return nil, nil, errors.Wrap(err, "could not process block")
}
set := sigSlice.Batch()
// State root validation.
postStateRoot, err := st.HashTreeRoot(ctx)
@@ -113,7 +113,7 @@ func ExecuteStateTransitionNoVerifyAnySig(
// assert block.state_root == hash_tree_root(state)
func CalculateStateRoot(
ctx context.Context,
state state.BeaconState,
rollback state.BeaconState,
signed interfaces.ReadOnlySignedBeaconBlock,
) ([32]byte, error) {
ctx, span := trace.StartSpan(ctx, "core.state.CalculateStateRoot")
@@ -122,7 +122,7 @@ func CalculateStateRoot(
tracing.AnnotateError(span, ctx.Err())
return [32]byte{}, ctx.Err()
}
if state == nil || state.IsNil() {
if rollback == nil || rollback.IsNil() {
return [32]byte{}, errors.New("nil state")
}
if signed == nil || signed.IsNil() || signed.Block().IsNil() {
@@ -130,7 +130,7 @@ func CalculateStateRoot(
}
// Copy state to avoid mutating the state reference.
state = state.Copy()
state := rollback.Copy()
// Execute per slots transition.
var err error
@@ -141,12 +141,101 @@ func CalculateStateRoot(
}
// Execute per block transition.
state, err = ProcessBlockForStateRoot(ctx, state, signed)
if features.Get().EnableProposerPreprocessing {
state, err = processBlockForProposing(ctx, rollback, state, signed)
if err != nil {
return [32]byte{}, errors.Wrap(err, "could not process block for proposing")
}
} else {
state, err = ProcessBlockForStateRoot(ctx, state, signed)
if err != nil {
return [32]byte{}, errors.Wrap(err, "could not process block")
}
}
return state.HashTreeRoot(ctx)
}
// processBlockVerifySigs processes the block and verifies the signatures within it. Block signatures are not verified as this block is not yet signed.
func processBlockForProposing(ctx context.Context, rollback state.BeaconState, st state.BeaconState, signed interfaces.ReadOnlySignedBeaconBlock) (state.BeaconState, error) {
var err error
var set BlockSignatureBatches
set, st, err = ProcessBlockNoVerifyAnySig(ctx, st, signed)
if err != nil {
return [32]byte{}, errors.Wrap(err, "could not process block")
return nil, err
}
// We first try to verify all sigantures batched optimistically. We ignore block proposer signature.
sigSet := set.Batch()
valid, err := sigSet.Verify()
if err != nil || valid {
return st, err
}
// Some signature failed to verify.
// Verify Attestations signatures
attSigs := set.AttestationSignatures
if attSigs == nil {
return nil, ErrAttestationsSignatureInvalid
}
valid, err = attSigs.Verify()
if err != nil {
return nil, err
}
if !valid {
return nil, ErrAttestationsSignatureInvalid
}
return state.HashTreeRoot(ctx)
// Verify Randao signature
randaoSigs := set.RandaoSignatures
if randaoSigs == nil {
return nil, ErrRandaoSignatureInvalid
}
valid, err = randaoSigs.Verify()
if err != nil {
return nil, err
}
if !valid {
return nil, ErrRandaoSignatureInvalid
}
if signed.Block().Version() < version.Capella {
//This should not happen as we must have failed one of the above signatures.
return st, nil
}
// Verify BLS to execution changes signatures
blsChangeSigs := set.BLSChangeSignatures
if blsChangeSigs == nil {
return nil, ErrBLSToExecutionChangesSignatureInvalid
}
valid, err = blsChangeSigs.Verify()
if err != nil {
return nil, err
}
if !valid {
return nil, ErrBLSToExecutionChangesSignatureInvalid
}
// We should not reach this point as one of the above signatures must have failed.
return st, nil
}
// BlockSignatureBatches holds the signature batches for different parts of a beacon block.
type BlockSignatureBatches struct {
RandaoSignatures *bls.SignatureBatch
AttestationSignatures *bls.SignatureBatch
BLSChangeSignatures *bls.SignatureBatch
}
// Batch returns the batch of signature batches in the BlockSignatureBatches.
func (b BlockSignatureBatches) Batch() *bls.SignatureBatch {
sigs := bls.NewSet()
if b.RandaoSignatures != nil {
sigs.Join(b.RandaoSignatures)
}
if b.AttestationSignatures != nil {
sigs.Join(b.AttestationSignatures)
}
if b.BLSChangeSignatures != nil {
sigs.Join(b.BLSChangeSignatures)
}
return sigs
}
// ProcessBlockNoVerifyAnySig creates a new, modified beacon state by applying block operation
@@ -165,48 +254,48 @@ func ProcessBlockNoVerifyAnySig(
ctx context.Context,
st state.BeaconState,
signed interfaces.ReadOnlySignedBeaconBlock,
) (*bls.SignatureBatch, state.BeaconState, error) {
) (BlockSignatureBatches, state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "core.state.ProcessBlockNoVerifyAnySig")
defer span.End()
set := BlockSignatureBatches{}
if err := blocks.BeaconBlockIsNil(signed); err != nil {
return nil, nil, err
return set, nil, err
}
if st.Version() != signed.Block().Version() {
return nil, nil, fmt.Errorf("state and block are different version. %d != %d", st.Version(), signed.Block().Version())
return set, nil, fmt.Errorf("state and block are different version. %d != %d", st.Version(), signed.Block().Version())
}
blk := signed.Block()
st, err := ProcessBlockForStateRoot(ctx, st, signed)
if err != nil {
return nil, nil, err
return set, nil, err
}
randaoReveal := signed.Block().Body().RandaoReveal()
rSet, err := b.RandaoSignatureBatch(ctx, st, randaoReveal[:])
if err != nil {
tracing.AnnotateError(span, err)
return nil, nil, errors.Wrap(err, "could not retrieve randao signature set")
return set, nil, errors.Wrap(err, "could not retrieve randao signature set")
}
set.RandaoSignatures = rSet
aSet, err := b.AttestationSignatureBatch(ctx, st, signed.Block().Body().Attestations())
if err != nil {
return nil, nil, errors.Wrap(err, "could not retrieve attestation signature set")
return set, nil, errors.Wrap(err, "could not retrieve attestation signature set")
}
set.AttestationSignatures = aSet
// Merge beacon block, randao and attestations signatures into a set.
set := bls.NewSet()
set.Join(rSet).Join(aSet)
if blk.Version() >= version.Capella {
changes, err := signed.Block().Body().BLSToExecutionChanges()
if err != nil {
return nil, nil, errors.Wrap(err, "could not get BLSToExecutionChanges")
return set, nil, errors.Wrap(err, "could not get BLSToExecutionChanges")
}
cSet, err := b.BLSChangesSignatureBatch(st, changes)
if err != nil {
return nil, nil, errors.Wrap(err, "could not get BLSToExecutionChanges signatures")
return set, nil, errors.Wrap(err, "could not get BLSToExecutionChanges signatures")
}
set.Join(cSet)
set.BLSChangeSignatures = cSet
}
return set, st, nil
}
@@ -268,7 +357,7 @@ func ProcessOperationsNoVerifyAttsSigs(
return nil, err
}
} else {
state, err = electra.ProcessOperations(ctx, state, beaconBlock)
state, err = electraOperations(ctx, state, beaconBlock)
if err != nil {
return nil, err
}
@@ -326,7 +415,7 @@ func ProcessBlockForStateRoot(
if state.Version() >= version.Capella {
state, err = b.ProcessWithdrawals(state, executionData)
if err != nil {
return nil, errors.Wrap(err, "could not process withdrawals")
return nil, errors.Wrap(ErrProcessWithdrawalsFailed, err.Error())
}
}
if err = b.ProcessPayload(state, blk.Body()); err != nil {
@@ -338,13 +427,13 @@ func ProcessBlockForStateRoot(
state, err = b.ProcessRandaoNoVerify(state, randaoReveal[:])
if err != nil {
tracing.AnnotateError(span, err)
return nil, errors.Wrap(err, "could not verify and process randao")
return nil, errors.Wrap(ErrProcessRandaoFailed, err.Error())
}
state, err = b.ProcessEth1DataInBlock(ctx, state, signed.Block().Body().Eth1Data())
if err != nil {
tracing.AnnotateError(span, err)
return nil, errors.Wrap(err, "could not process eth1 data")
return nil, errors.Wrap(ErrProcessEth1DataFailed, err.Error())
}
state, err = ProcessOperationsNoVerifyAttsSigs(ctx, state, signed.Block())
@@ -363,7 +452,7 @@ func ProcessBlockForStateRoot(
}
state, _, err = altair.ProcessSyncAggregate(ctx, state, sa)
if err != nil {
return nil, errors.Wrap(err, "process_sync_aggregate failed")
return nil, errors.Wrap(ErrProcessSyncAggregateFailed, err.Error())
}
return state, nil
@@ -379,31 +468,35 @@ func altairOperations(ctx context.Context, st state.BeaconState, beaconBlock int
exitInfo := &validators.ExitInfo{}
if hasSlashings || hasExits {
// ExitInformation is expensive to compute, only do it if we need it.
exitInfo = v.ExitInformation(st)
exitInfo = validators.ExitInformation(st)
if err := helpers.UpdateTotalActiveBalanceCache(st, exitInfo.TotalActiveBalance); err != nil {
return nil, errors.Wrap(err, "could not update total active balance cache")
}
}
st, err = b.ProcessProposerSlashings(ctx, st, beaconBlock.Body().ProposerSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process altair proposer slashing")
return nil, errors.Wrap(ErrProcessProposerSlashingsFailed, err.Error())
}
st, err = b.ProcessAttesterSlashings(ctx, st, beaconBlock.Body().AttesterSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process altair attester slashing")
return nil, errors.Wrap(ErrProcessAttesterSlashingsFailed, err.Error())
}
st, err = altair.ProcessAttestationsNoVerifySignature(ctx, st, beaconBlock)
if err != nil {
return nil, errors.Wrap(err, "could not process altair attestation")
return nil, errors.Wrap(ErrProcessAttestationsFailed, err.Error())
}
if _, err := altair.ProcessDeposits(ctx, st, beaconBlock.Body().Deposits()); err != nil {
return nil, errors.Wrap(err, "could not process altair deposit")
return nil, errors.Wrap(ErrProcessDepositsFailed, err.Error())
}
st, err = b.ProcessVoluntaryExits(ctx, st, beaconBlock.Body().VoluntaryExits(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process voluntary exits")
return nil, errors.Wrap(ErrProcessVoluntaryExitsFailed, err.Error())
}
return b.ProcessBLSToExecutionChanges(st, beaconBlock)
st, err = b.ProcessBLSToExecutionChanges(st, beaconBlock)
if err != nil {
return nil, errors.Wrap(ErrProcessBLSChangesFailed, err.Error())
}
return st, nil
}
// This calls phase 0 block operations.
@@ -411,32 +504,32 @@ func phase0Operations(ctx context.Context, st state.BeaconState, beaconBlock int
var err error
hasSlashings := len(beaconBlock.Body().ProposerSlashings()) > 0 || len(beaconBlock.Body().AttesterSlashings()) > 0
hasExits := len(beaconBlock.Body().VoluntaryExits()) > 0
var exitInfo *v.ExitInfo
var exitInfo *validators.ExitInfo
if hasSlashings || hasExits {
// ExitInformation is expensive to compute, only do it if we need it.
exitInfo = v.ExitInformation(st)
exitInfo = validators.ExitInformation(st)
if err := helpers.UpdateTotalActiveBalanceCache(st, exitInfo.TotalActiveBalance); err != nil {
return nil, errors.Wrap(err, "could not update total active balance cache")
}
}
st, err = b.ProcessProposerSlashings(ctx, st, beaconBlock.Body().ProposerSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process block proposer slashings")
return nil, errors.Wrap(ErrProcessProposerSlashingsFailed, err.Error())
}
st, err = b.ProcessAttesterSlashings(ctx, st, beaconBlock.Body().AttesterSlashings(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process block attester slashings")
return nil, errors.Wrap(ErrProcessAttesterSlashingsFailed, err.Error())
}
st, err = b.ProcessAttestationsNoVerifySignature(ctx, st, beaconBlock)
if err != nil {
return nil, errors.Wrap(err, "could not process block attestations")
return nil, errors.Wrap(ErrProcessAttestationsFailed, err.Error())
}
if _, err := altair.ProcessDeposits(ctx, st, beaconBlock.Body().Deposits()); err != nil {
return nil, errors.Wrap(err, "could not process deposits")
return nil, errors.Wrap(ErrProcessDepositsFailed, err.Error())
}
st, err = b.ProcessVoluntaryExits(ctx, st, beaconBlock.Body().VoluntaryExits(), exitInfo)
if err != nil {
return nil, errors.Wrap(err, "could not process voluntary exits")
return nil, errors.Wrap(ErrProcessVoluntaryExitsFailed, err.Error())
}
return st, nil
}

View File

@@ -132,7 +132,8 @@ func TestProcessBlockNoVerify_PassesProcessingConditions(t *testing.T) {
set, _, err := transition.ProcessBlockNoVerifyAnySig(t.Context(), beaconState, wsb)
require.NoError(t, err)
// Test Signature set verifies.
verified, err := set.Verify()
sigSet := set.Batch()
verified, err := sigSet.Verify()
require.NoError(t, err)
assert.Equal(t, true, verified, "Could not verify signature set.")
}
@@ -145,7 +146,8 @@ func TestProcessBlockNoVerifyAnySigAltair_OK(t *testing.T) {
require.NoError(t, err)
set, _, err := transition.ProcessBlockNoVerifyAnySig(t.Context(), beaconState, wsb)
require.NoError(t, err)
verified, err := set.Verify()
sigSet := set.Batch()
verified, err := sigSet.Verify()
require.NoError(t, err)
require.Equal(t, true, verified, "Could not verify signature set")
}
@@ -154,8 +156,9 @@ func TestProcessBlockNoVerify_SigSetContainsDescriptions(t *testing.T) {
beaconState, block, _, _, _ := createFullBlockWithOperations(t)
wsb, err := blocks.NewSignedBeaconBlock(block)
require.NoError(t, err)
set, _, err := transition.ProcessBlockNoVerifyAnySig(t.Context(), beaconState, wsb)
signatures, _, err := transition.ProcessBlockNoVerifyAnySig(t.Context(), beaconState, wsb)
require.NoError(t, err)
set := signatures.Batch()
assert.Equal(t, len(set.Signatures), len(set.Descriptions), "Signatures and descriptions do not match up")
assert.Equal(t, "randao signature", set.Descriptions[0])
assert.Equal(t, "attestation signature", set.Descriptions[1])

View File

@@ -602,7 +602,7 @@ func (vs *Server) GetFeeRecipientByPubKey(ctx context.Context, request *ethpb.Fe
// computeStateRoot computes the state root after a block has been processed through a state transition and
// returns it to the validator client.
func (vs *Server) computeStateRoot(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock) ([]byte, error) {
func (vs *Server) computeStateRoot(ctx context.Context, block interfaces.SignedBeaconBlock) ([]byte, error) {
beaconState, err := vs.StateGen.StateByRoot(ctx, block.Block().ParentRoot())
if err != nil {
return nil, errors.Wrap(err, "could not retrieve beacon state")
@@ -613,13 +613,59 @@ func (vs *Server) computeStateRoot(ctx context.Context, block interfaces.ReadOnl
block,
)
if err != nil {
return nil, errors.Wrapf(err, "could not calculate state root at slot %d", beaconState.Slot())
return vs.handleStateRootError(ctx, block, err)
}
log.WithField("beaconStateRoot", fmt.Sprintf("%#x", root)).Debugf("Computed state root")
return root[:], nil
}
// handleStateRootError retries block construction in some error cases.
func (vs *Server) handleStateRootError(ctx context.Context, block interfaces.SignedBeaconBlock, err error) ([]byte, error) {
if ctx.Err() != nil {
return nil, status.Errorf(codes.Canceled, "context error: %v", ctx.Err())
}
switch {
case errors.Is(err, transition.ErrAttestationsSignatureInvalid),
errors.Is(err, transition.ErrProcessAttestationsFailed):
log.WithError(err).Warn("Retrying block construction without attestations")
if err := block.SetAttestations([]ethpb.Att{}); err != nil {
return nil, errors.Wrap(err, "could not set attestations")
}
case errors.Is(err, transition.ErrProcessBLSChangesFailed), errors.Is(err, transition.ErrBLSToExecutionChangesSignatureInvalid):
log.WithError(err).Warn("Retrying block construction without BLS to execution changes")
if err := block.SetBLSToExecutionChanges([]*ethpb.SignedBLSToExecutionChange{}); err != nil {
return nil, errors.Wrap(err, "could not set BLS to execution changes")
}
case errors.Is(err, transition.ErrProcessProposerSlashingsFailed):
log.WithError(err).Warn("Retrying block construction without proposer slashings")
block.SetProposerSlashings([]*ethpb.ProposerSlashing{})
case errors.Is(err, transition.ErrProcessAttesterSlashingsFailed):
log.WithError(err).Warn("Retrying block construction without attester slashings")
if err := block.SetAttesterSlashings([]ethpb.AttSlashing{}); err != nil {
return nil, errors.Wrap(err, "could not set attester slashings")
}
case errors.Is(err, transition.ErrProcessVoluntaryExitsFailed):
log.WithError(err).Warn("Retrying block construction without voluntary exits")
block.SetVoluntaryExits([]*ethpb.SignedVoluntaryExit{})
case errors.Is(err, transition.ErrProcessSyncAggregateFailed):
log.WithError(err).Warn("Retrying block construction without sync aggregate")
emptySig := [96]byte{0xC0}
emptyAggregate := &ethpb.SyncAggregate{
SyncCommitteeBits: make([]byte, params.BeaconConfig().SyncCommitteeSize/8),
SyncCommitteeSignature: emptySig[:],
}
if err := block.SetSyncAggregate(emptyAggregate); err != nil {
log.WithError(err).Error("Could not set sync aggregate")
}
default:
return nil, errors.Wrap(err, "could not compute state root")
}
// recursive call to compute state root again
return vs.computeStateRoot(ctx, block)
}
// Deprecated: The gRPC API will remain the default and fully supported through v8 (expected in 2026) but will be eventually removed in favor of REST API.
//
// SubmitValidatorRegistrations submits validator registrations.

View File

@@ -0,0 +1,3 @@
### Added
- Add feature flag `--enable-proposer-preprocessing` to process the block and verify signatures before proposing.

View File

@@ -74,6 +74,7 @@ type Flags struct {
IgnoreUnviableAttestations bool // Ignore attestations whose target state is not viable (avoids lagging-node DoS).
EnableVerboseSigVerification bool // EnableVerboseSigVerification specifies whether to verify individual signature if batch verification fails
EnableProposerPreprocessing bool // EnableProposerPreprocessing enables proposer pre-processing of blocks before proposing.
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.
@@ -244,6 +245,7 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
logEnabled(disableVerboseSigVerification)
cfg.EnableVerboseSigVerification = false
}
cfg.EnableProposerPreprocessing = ctx.Bool(enableProposerPreprocessing.Name)
if ctx.IsSet(prepareAllPayloads.Name) {
logEnabled(prepareAllPayloads)
cfg.PrepareAllPayloads = true

View File

@@ -137,6 +137,11 @@ var (
Name: "disable-verbose-sig-verification",
Usage: "Disables identifying invalid signatures if batch verification fails when processing block.",
}
enableProposerPreprocessing = &cli.BoolFlag{
Name: "enable-proposer-preprocessing",
Usage: "Enables proposer pre-processing of blocks before proposing.",
Value: false,
}
prepareAllPayloads = &cli.BoolFlag{
Name: "prepare-all-payloads",
Usage: "Informs the engine to prepare all local payloads. Useful for relayers and builders.",
@@ -259,6 +264,7 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
ignoreUnviableAttestations,
enableFullSSZDataLogging,
disableVerboseSigVerification,
enableProposerPreprocessing,
prepareAllPayloads,
aggregateFirstInterval,
aggregateSecondInterval,

View File

@@ -6409,8 +6409,8 @@
- name: process_operations#electra
sources:
- file: beacon-chain/core/electra/transition_no_verify_sig.go
search: func ProcessOperations(
- file: beacon-chain/core/transition/electra.go
search: func electraOperations(
spec: |
<spec fn="process_operations" fork="electra" hash="643f18b4">
def process_operations(state: BeaconState, body: BeaconBlockBody) -> None:

View File

@@ -424,7 +424,8 @@ func debugStateTransition(
return st, errors.Wrap(err, "could not process block")
}
var valid bool
valid, err = set.VerifyVerbosely()
sigSet := set.Batch()
valid, err = sigSet.VerifyVerbosely()
if err != nil {
return st, errors.Wrap(err, "could not batch verify signature")
}