Compare commits

..

3 Commits

Author SHA1 Message Date
terence tsao
bfa4f7ac3a gloas: add process execution payload 2026-01-11 17:19:01 +08:00
Jun Song
21366e11ca feat: generalize proof generation for beacon state fields (#15443)
**What type of PR is this?**

Feature

**What does this PR do? Why is it needed?**

**Which issues(s) does this PR fix?**

This PR is the part of EPF6 project: [Merkle Proofs of
Everything](https://github.com/eth-protocol-fellows/cohort-six/blob/master/projects/project-ideas.md#prysm-merkle-proofs-of-everything).

**Other notes for review**

You can see some rationale and future TODOs in [my HackMD
post](https://hackmd.io/@junsong/SJGze5cNxg). The main next task is
following: (excerpted from the post)

> More generally speaking, we should find a way to prove for an item
with generalized index more than 128.

**Acknowledgements**

- [x] I have read
[CONTRIBUTING.md](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md).
- [x] I have included a uniquely named [changelog fragment
file](https://github.com/prysmaticlabs/prysm/blob/develop/CONTRIBUTING.md#maintaining-changelogmd).
- [x] I have added a description to this PR with sufficient context for
reviewers to understand this PR.

---------

Co-authored-by: Radosław Kapka <rkapka@wp.pl>
2026-01-10 13:15:25 +00:00
Potuz
7ed4d496dd Add feature flag to verify signatures before proposing (#15920)
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.

---------

Co-authored-by: Claude Opus 4.5 <noreply@anthropic.com>
2026-01-09 18:35:40 +00:00
62 changed files with 2515 additions and 367 deletions

View File

@@ -12,7 +12,6 @@ go_library(
"log.go",
"registry_updates.go",
"transition.go",
"transition_no_verify_sig.go",
"upgrade.go",
"validator.go",
"withdrawals.go",
@@ -62,7 +61,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

@@ -0,0 +1,43 @@
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = ["payload.go"],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/electra:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/params:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/ssz:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = ["payload_test.go"],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/ssz:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"//time/slots:go_default_library",
],
)

View File

@@ -0,0 +1,341 @@
package gloas
import (
"bytes"
"context"
"fmt"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/electra"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/encoding/ssz"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/time/slots"
"github.com/pkg/errors"
)
// ProcessExecutionPayload processes the signed execution payload envelope for the Gloas fork.
// Spec v1.7.0-alpha.0 (pseudocode):
// def process_execution_payload(
//
// state: BeaconState,
// signed_envelope: SignedExecutionPayloadEnvelope,
// execution_engine: ExecutionEngine,
// verify: bool = True,
//
// ) -> None:
//
// envelope = signed_envelope.message
// payload = envelope.payload
//
// if verify:
// assert verify_execution_payload_envelope_signature(state, signed_envelope)
//
// previous_state_root = hash_tree_root(state)
// if state.latest_block_header.state_root == Root():
// state.latest_block_header.state_root = previous_state_root
//
// assert envelope.beacon_block_root == hash_tree_root(state.latest_block_header)
// assert envelope.slot == state.slot
//
// committed_bid = state.latest_execution_payload_bid
// assert envelope.builder_index == committed_bid.builder_index
// assert committed_bid.blob_kzg_commitments_root == hash_tree_root(envelope.blob_kzg_commitments)
// assert committed_bid.prev_randao == payload.prev_randao
//
// assert hash_tree_root(payload.withdrawals) == hash_tree_root(state.payload_expected_withdrawals)
//
// assert committed_bid.gas_limit == payload.gas_limit
// assert committed_bid.block_hash == payload.block_hash
// assert payload.parent_hash == state.latest_block_hash
// assert payload.timestamp == compute_time_at_slot(state, state.slot)
// assert (
// len(envelope.blob_kzg_commitments)
// <= get_blob_parameters(get_current_epoch(state)).max_blobs_per_block
// )
// versioned_hashes = [
// kzg_commitment_to_versioned_hash(commitment) for commitment in envelope.blob_kzg_commitments
// ]
// requests = envelope.execution_requests
// assert execution_engine.verify_and_notify_new_payload(
// NewPayloadRequest(
// execution_payload=payload,
// versioned_hashes=versioned_hashes,
// parent_beacon_block_root=state.latest_block_header.parent_root,
// execution_requests=requests,
// )
// )
//
// for op in requests.deposits: process_deposit_request(state, op)
// for op in requests.withdrawals: process_withdrawal_request(state, op)
// for op in requests.consolidations: process_consolidation_request(state, op)
//
// payment = state.builder_pending_payments[SLOTS_PER_EPOCH + state.slot % SLOTS_PER_EPOCH]
// amount = payment.withdrawal.amount
// if amount > 0:
// state.builder_pending_withdrawals.append(payment.withdrawal)
// state.builder_pending_payments[SLOTS_PER_EPOCH + state.slot % SLOTS_PER_EPOCH] = (
// BuilderPendingPayment()
// )
//
// state.execution_payload_availability[state.slot % SLOTS_PER_HISTORICAL_ROOT] = 0b1
// state.latest_block_hash = payload.block_hash
//
// if verify:
// assert envelope.state_root == hash_tree_root(state)
func ProcessExecutionPayload(
ctx context.Context,
st state.BeaconState,
signedEnvelope interfaces.ROSignedExecutionPayloadEnvelope,
) error {
// Verify the signature on the signed execution payload envelope
if err := verifyExecutionPayloadEnvelopeSignature(st, signedEnvelope); err != nil {
return errors.Wrap(err, "signature verification failed")
}
envelope, err := signedEnvelope.Envelope()
if err != nil {
return errors.Wrap(err, "could not get envelope from signed envelope")
}
payload, err := envelope.Execution()
if err != nil {
return errors.Wrap(err, "could not get execution payload from envelope")
}
latestHeader := st.LatestBlockHeader()
if len(latestHeader.StateRoot) == 0 || bytes.Equal(latestHeader.StateRoot, make([]byte, 32)) {
previousStateRoot, err := st.HashTreeRoot(ctx)
if err != nil {
return errors.Wrap(err, "could not compute state root")
}
latestHeader.StateRoot = previousStateRoot[:]
if err := st.SetLatestBlockHeader(latestHeader); err != nil {
return errors.Wrap(err, "could not set latest block header")
}
}
blockHeaderRoot, err := latestHeader.HashTreeRoot()
if err != nil {
return errors.Wrap(err, "could not compute block header root")
}
beaconBlockRoot := envelope.BeaconBlockRoot()
if !bytes.Equal(beaconBlockRoot[:], blockHeaderRoot[:]) {
return errors.Errorf("envelope beacon block root does not match state latest block header root: envelope=%#x, header=%#x", beaconBlockRoot, blockHeaderRoot)
}
if envelope.Slot() != st.Slot() {
return errors.Errorf("envelope slot does not match state slot: envelope=%d, state=%d", envelope.Slot(), st.Slot())
}
latestBid, err := st.LatestExecutionPayloadBid()
if err != nil {
return errors.Wrap(err, "could not get latest execution payload bid")
}
if latestBid == nil {
return errors.New("latest execution payload bid is nil")
}
if envelope.BuilderIndex() != latestBid.BuilderIndex() {
return errors.Errorf("envelope builder index does not match committed bid builder index: envelope=%d, bid=%d", envelope.BuilderIndex(), latestBid.BuilderIndex())
}
envelopeBlobCommitments := envelope.BlobKzgCommitments()
envelopeBlobRoot, err := ssz.KzgCommitmentsRoot(envelopeBlobCommitments)
if err != nil {
return errors.Wrap(err, "could not compute envelope blob KZG commitments root")
}
committedBlobRoot := latestBid.BlobKzgCommitmentsRoot()
if !bytes.Equal(committedBlobRoot[:], envelopeBlobRoot[:]) {
return errors.Errorf("committed bid blob KZG commitments root does not match envelope: bid=%#x, envelope=%#x", committedBlobRoot, envelopeBlobRoot)
}
withdrawals, err := payload.Withdrawals()
if err != nil {
return errors.Wrap(err, "could not get withdrawals from payload")
}
cfg := params.BeaconConfig()
withdrawalsRoot, err := ssz.WithdrawalSliceRoot(withdrawals, cfg.MaxWithdrawalsPerPayload)
if err != nil {
return errors.Wrap(err, "could not compute withdrawals root")
}
expectedWithdrawals, err := st.PayloadExpectedWithdrawals()
if err != nil {
return errors.Wrap(err, "could not get expected withdrawals")
}
expectedRoot, err := ssz.WithdrawalSliceRoot(expectedWithdrawals, cfg.MaxWithdrawalsPerPayload)
if err != nil {
return errors.Wrap(err, "could not compute expected withdrawals root")
}
if !bytes.Equal(withdrawalsRoot[:], expectedRoot[:]) {
return errors.Errorf("payload withdrawals root does not match expected withdrawals: payload=%#x, expected=%#x", withdrawalsRoot, expectedRoot)
}
if latestBid.GasLimit() != payload.GasLimit() {
return errors.Errorf("committed bid gas limit does not match payload gas limit: bid=%d, payload=%d", latestBid.GasLimit(), payload.GasLimit())
}
latestBidPrevRandao := latestBid.PrevRandao()
if !bytes.Equal(payload.PrevRandao(), latestBidPrevRandao[:]) {
return errors.Errorf("payload prev randao does not match committed bid prev randao: payload=%#x, bid=%#x", payload.PrevRandao(), latestBidPrevRandao)
}
bidBlockHash := latestBid.BlockHash()
payloadBlockHash := payload.BlockHash()
if !bytes.Equal(bidBlockHash[:], payloadBlockHash) {
return errors.Errorf("committed bid block hash does not match payload block hash: bid=%#x, payload=%#x", bidBlockHash, payloadBlockHash)
}
latestBlockHash, err := st.LatestBlockHash()
if err != nil {
return errors.Wrap(err, "could not get latest block hash")
}
if !bytes.Equal(payload.ParentHash(), latestBlockHash[:]) {
return errors.Errorf("payload parent hash does not match state latest block hash: payload=%#x, state=%#x", payload.ParentHash(), latestBlockHash)
}
t, err := slots.StartTime(st.GenesisTime(), st.Slot())
if err != nil {
return errors.Wrap(err, "could not compute timestamp")
}
if payload.Timestamp() != uint64(t.Unix()) {
return errors.Errorf("payload timestamp does not match expected timestamp: payload=%d, expected=%d", payload.Timestamp(), uint64(t.Unix()))
}
maxBlobsPerBlock := cfg.MaxBlobsPerBlock(envelope.Slot())
if len(envelopeBlobCommitments) > maxBlobsPerBlock {
return errors.Errorf("too many blob KZG commitments: got=%d, max=%d", len(envelopeBlobCommitments), maxBlobsPerBlock)
}
if err := processExecutionRequests(ctx, st, envelope.ExecutionRequests()); err != nil {
return errors.Wrap(err, "could not process execution requests")
}
if err := queueBuilderPayment(ctx, st); err != nil {
return errors.Wrap(err, "could not queue builder payment")
}
if err := st.SetExecutionPayloadAvailability(st.Slot(), true); err != nil {
return errors.Wrap(err, "could not set execution payload availability")
}
if err := st.SetLatestBlockHash([32]byte(payload.BlockHash())); err != nil {
return errors.Wrap(err, "could not set latest block hash")
}
r, err := st.HashTreeRoot(ctx)
if err != nil {
return errors.Wrap(err, "could not get hash tree root")
}
if r != envelope.StateRoot() {
return fmt.Errorf("state root mismatch: expected %#x, got %#x", envelope.StateRoot(), r)
}
return nil
}
// processExecutionRequests processes deposits, withdrawals, and consolidations from execution requests.
// Spec v1.7.0-alpha.0 (pseudocode):
// for op in requests.deposits: process_deposit_request(state, op)
// for op in requests.withdrawals: process_withdrawal_request(state, op)
// for op in requests.consolidations: process_consolidation_request(state, op)
func processExecutionRequests(ctx context.Context, st state.BeaconState, requests *enginev1.ExecutionRequests) error {
var err error
st, err = electra.ProcessDepositRequests(ctx, st, requests.Deposits)
if err != nil {
return errors.Wrap(err, "could not process deposit requests")
}
st, err = electra.ProcessWithdrawalRequests(ctx, st, requests.Withdrawals)
if err != nil {
return errors.Wrap(err, "could not process withdrawal requests")
}
err = electra.ProcessConsolidationRequests(ctx, st, requests.Consolidations)
if err != nil {
return errors.Wrap(err, "could not process consolidation requests")
}
return nil
}
// queueBuilderPayment implements the builder payment queuing logic for Gloas.
// Spec v1.7.0-alpha.0 (pseudocode):
// payment = state.builder_pending_payments[SLOTS_PER_EPOCH + state.slot % SLOTS_PER_EPOCH]
// amount = payment.withdrawal.amount
// if amount > 0:
//
// state.builder_pending_withdrawals.append(payment.withdrawal)
//
// state.builder_pending_payments[SLOTS_PER_EPOCH + state.slot % SLOTS_PER_EPOCH] = BuilderPendingPayment()
func queueBuilderPayment(ctx context.Context, st state.BeaconState) error {
slot := st.Slot()
payment, err := st.BuilderPendingPayment(slot)
if err != nil {
return errors.Wrap(err, "could not get builder pending payment")
}
if payment.Withdrawal != nil && payment.Withdrawal.Amount > 0 {
if err := st.AppendBuilderPendingWithdrawal(payment.Withdrawal); err != nil {
return errors.Wrap(err, "could not append builder pending withdrawal")
}
}
reset := &ethpb.BuilderPendingPayment{
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
},
}
if err := st.SetBuilderPendingPayment(slot, reset); err != nil {
return errors.Wrap(err, "could not set builder pending payment")
}
return nil
}
// verifyExecutionPayloadEnvelopeSignature verifies the BLS signature on a signed execution payload envelope.
// Spec v1.7.0-alpha.0 (pseudocode):
// if verify: assert verify_execution_payload_envelope_signature(state, signed_envelope)
func verifyExecutionPayloadEnvelopeSignature(st state.BeaconState, signedEnvelope interfaces.ROSignedExecutionPayloadEnvelope) error {
envelope, err := signedEnvelope.Envelope()
if err != nil {
return fmt.Errorf("failed to get envelope: %w", err)
}
builderIdx := envelope.BuilderIndex()
builderPubkey := st.PubkeyAtIndex(primitives.ValidatorIndex(builderIdx))
publicKey, err := bls.PublicKeyFromBytes(builderPubkey[:])
if err != nil {
return fmt.Errorf("invalid builder public key: %w", err)
}
signatureBytes := signedEnvelope.Signature()
signature, err := bls.SignatureFromBytes(signatureBytes[:])
if err != nil {
return fmt.Errorf("invalid signature format: %w", err)
}
currentEpoch := slots.ToEpoch(envelope.Slot())
domain, err := signing.Domain(
st.Fork(),
currentEpoch,
params.BeaconConfig().DomainBeaconBuilder,
st.GenesisValidatorsRoot(),
)
if err != nil {
return fmt.Errorf("failed to compute signing domain: %w", err)
}
signingRoot, err := signedEnvelope.SigningRoot(domain)
if err != nil {
return fmt.Errorf("failed to compute signing root: %w", err)
}
if !signature.Verify(publicKey, signingRoot[:]) {
return fmt.Errorf("signature verification failed: %w", signing.ErrSigFailedToVerify)
}
return nil
}

View File

@@ -0,0 +1,269 @@
package gloas
import (
"bytes"
"context"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
"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/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/crypto/bls"
"github.com/OffchainLabs/prysm/v7/encoding/ssz"
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/time/slots"
)
type payloadFixture struct {
state state.BeaconState
signed interfaces.ROSignedExecutionPayloadEnvelope
signedProto *ethpb.SignedExecutionPayloadEnvelope
envelope *ethpb.ExecutionPayloadEnvelope
payload *enginev1.ExecutionPayloadDeneb
slot primitives.Slot
}
func buildPayloadFixture(t *testing.T, mutate func(payload *enginev1.ExecutionPayloadDeneb, bid *ethpb.ExecutionPayloadBid, envelope *ethpb.ExecutionPayloadEnvelope)) payloadFixture {
t.Helper()
cfg := params.BeaconConfig()
slot := primitives.Slot(5)
builderIdx := primitives.BuilderIndex(0)
sk, err := bls.RandKey()
require.NoError(t, err)
pk := sk.PublicKey().Marshal()
randao := bytes.Repeat([]byte{0xAA}, 32)
parentHash := bytes.Repeat([]byte{0xBB}, 32)
blockHash := bytes.Repeat([]byte{0xCC}, 32)
withdrawals := []*enginev1.Withdrawal{
{Index: 0, ValidatorIndex: 1, Address: bytes.Repeat([]byte{0x01}, 20), Amount: 0},
}
blobCommitments := [][]byte{}
blobRoot, err := ssz.KzgCommitmentsRoot(blobCommitments)
require.NoError(t, err)
payload := &enginev1.ExecutionPayloadDeneb{
ParentHash: parentHash,
FeeRecipient: bytes.Repeat([]byte{0x01}, 20),
StateRoot: bytes.Repeat([]byte{0x02}, 32),
ReceiptsRoot: bytes.Repeat([]byte{0x03}, 32),
LogsBloom: bytes.Repeat([]byte{0x04}, 256),
PrevRandao: randao,
BlockNumber: 1,
GasLimit: 1,
GasUsed: 0,
Timestamp: 100,
ExtraData: []byte{},
BaseFeePerGas: bytes.Repeat([]byte{0x05}, 32),
BlockHash: blockHash,
Transactions: [][]byte{},
Withdrawals: withdrawals,
BlobGasUsed: 0,
ExcessBlobGas: 0,
}
bid := &ethpb.ExecutionPayloadBid{
ParentBlockHash: parentHash,
ParentBlockRoot: bytes.Repeat([]byte{0xDD}, 32),
BlockHash: blockHash,
PrevRandao: randao,
GasLimit: 1,
BuilderIndex: builderIdx,
Slot: slot,
Value: 0,
ExecutionPayment: 0,
BlobKzgCommitmentsRoot: blobRoot[:],
FeeRecipient: bytes.Repeat([]byte{0xEE}, 20),
}
header := &ethpb.BeaconBlockHeader{
Slot: slot,
ParentRoot: bytes.Repeat([]byte{0x11}, 32),
StateRoot: bytes.Repeat([]byte{0x22}, 32),
BodyRoot: bytes.Repeat([]byte{0x33}, 32),
}
headerRoot, err := header.HashTreeRoot()
require.NoError(t, err)
envelope := &ethpb.ExecutionPayloadEnvelope{
Slot: slot,
BuilderIndex: builderIdx,
BeaconBlockRoot: headerRoot[:],
Payload: payload,
BlobKzgCommitments: blobCommitments,
ExecutionRequests: &enginev1.ExecutionRequests{},
}
if mutate != nil {
mutate(payload, bid, envelope)
}
genesisRoot := bytes.Repeat([]byte{0xAB}, 32)
blockRoots := make([][]byte, cfg.SlotsPerHistoricalRoot)
stateRoots := make([][]byte, cfg.SlotsPerHistoricalRoot)
for i := range blockRoots {
blockRoots[i] = bytes.Repeat([]byte{0x44}, 32)
stateRoots[i] = bytes.Repeat([]byte{0x55}, 32)
}
randaoMixes := make([][]byte, cfg.EpochsPerHistoricalVector)
for i := range randaoMixes {
randaoMixes[i] = randao
}
withdrawalCreds := make([]byte, 32)
withdrawalCreds[0] = cfg.ETH1AddressWithdrawalPrefixByte
eth1Data := &ethpb.Eth1Data{
DepositRoot: bytes.Repeat([]byte{0x66}, 32),
DepositCount: 0,
BlockHash: bytes.Repeat([]byte{0x77}, 32),
}
vals := []*ethpb.Validator{
{
PublicKey: pk,
WithdrawalCredentials: withdrawalCreds,
EffectiveBalance: cfg.MinActivationBalance + 1_000,
},
}
balances := []uint64{cfg.MinActivationBalance + 1_000}
payments := make([]*ethpb.BuilderPendingPayment, cfg.SlotsPerEpoch*2)
for i := range payments {
payments[i] = &ethpb.BuilderPendingPayment{
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: make([]byte, 20),
},
}
}
executionPayloadAvailability := make([]byte, cfg.SlotsPerHistoricalRoot/8)
genesisTime := uint64(0)
slotSeconds := cfg.SecondsPerSlot * uint64(slot)
if payload.Timestamp > slotSeconds {
genesisTime = payload.Timestamp - slotSeconds
}
stProto := &ethpb.BeaconStateGloas{
Slot: slot,
GenesisTime: genesisTime,
GenesisValidatorsRoot: genesisRoot,
Fork: &ethpb.Fork{
CurrentVersion: bytes.Repeat([]byte{0x01}, 4),
PreviousVersion: bytes.Repeat([]byte{0x01}, 4),
Epoch: 0,
},
LatestBlockHeader: header,
BlockRoots: blockRoots,
StateRoots: stateRoots,
RandaoMixes: randaoMixes,
Eth1Data: eth1Data,
Validators: vals,
Balances: balances,
LatestBlockHash: payload.ParentHash,
LatestExecutionPayloadBid: bid,
BuilderPendingPayments: payments,
ExecutionPayloadAvailability: executionPayloadAvailability,
BuilderPendingWithdrawals: []*ethpb.BuilderPendingWithdrawal{},
PayloadExpectedWithdrawals: payload.Withdrawals,
}
st, err := state_native.InitializeFromProtoGloas(stProto)
require.NoError(t, err)
expected := st.Copy()
ctx := context.Background()
require.NoError(t, processExecutionRequests(ctx, expected, envelope.ExecutionRequests))
require.NoError(t, queueBuilderPayment(ctx, expected))
require.NoError(t, expected.SetExecutionPayloadAvailability(slot, true))
var blockHashArr [32]byte
copy(blockHashArr[:], payload.BlockHash)
require.NoError(t, expected.SetLatestBlockHash(blockHashArr))
expectedRoot, err := expected.HashTreeRoot(ctx)
require.NoError(t, err)
envelope.StateRoot = expectedRoot[:]
epoch := slots.ToEpoch(slot)
domain, err := signing.Domain(st.Fork(), epoch, cfg.DomainBeaconBuilder, st.GenesisValidatorsRoot())
require.NoError(t, err)
signingRoot, err := signing.ComputeSigningRoot(envelope, domain)
require.NoError(t, err)
signature := sk.Sign(signingRoot[:]).Marshal()
signedProto := &ethpb.SignedExecutionPayloadEnvelope{
Message: envelope,
Signature: signature,
}
signed, err := blocks.WrappedROSignedExecutionPayloadEnvelope(signedProto)
require.NoError(t, err)
return payloadFixture{
state: st,
signed: signed,
signedProto: signedProto,
envelope: envelope,
payload: payload,
slot: slot,
}
}
func TestProcessExecutionPayload_Success(t *testing.T) {
fixture := buildPayloadFixture(t, nil)
require.NoError(t, ProcessExecutionPayload(t.Context(), fixture.state, fixture.signed))
latestHash, err := fixture.state.LatestBlockHash()
require.NoError(t, err)
var expectedHash [32]byte
copy(expectedHash[:], fixture.payload.BlockHash)
require.Equal(t, expectedHash, latestHash)
payment, err := fixture.state.BuilderPendingPayment(fixture.slot)
require.NoError(t, err)
require.NotNil(t, payment)
require.Equal(t, primitives.Gwei(0), payment.Withdrawal.Amount)
}
func TestProcessExecutionPayload_PrevRandaoMismatch(t *testing.T) {
fixture := buildPayloadFixture(t, func(_ *enginev1.ExecutionPayloadDeneb, bid *ethpb.ExecutionPayloadBid, _ *ethpb.ExecutionPayloadEnvelope) {
bid.PrevRandao = bytes.Repeat([]byte{0xFF}, 32)
})
err := ProcessExecutionPayload(t.Context(), fixture.state, fixture.signed)
require.ErrorContains(t, "prev randao", err)
}
func TestQueueBuilderPayment_ZeroAmountClearsSlot(t *testing.T) {
fixture := buildPayloadFixture(t, nil)
require.NoError(t, queueBuilderPayment(t.Context(), fixture.state))
payment, err := fixture.state.BuilderPendingPayment(fixture.slot)
require.NoError(t, err)
require.NotNil(t, payment)
require.Equal(t, primitives.Gwei(0), payment.Withdrawal.Amount)
}
func TestVerifyExecutionPayloadEnvelopeSignature_InvalidSig(t *testing.T) {
fixture := buildPayloadFixture(t, nil)
signedProto := &ethpb.SignedExecutionPayloadEnvelope{
Message: fixture.signedProto.Message,
Signature: bytes.Repeat([]byte{0xFF}, 96),
}
badSigned, err := blocks.WrappedROSignedExecutionPayloadEnvelope(signedProto)
require.NoError(t, err)
err = verifyExecutionPayloadEnvelopeSignature(fixture.state, badSigned)
require.ErrorContains(t, "invalid signature format", 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, 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, 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

@@ -205,7 +205,7 @@ func TestGetSpec(t *testing.T) {
require.NoError(t, json.Unmarshal(writer.Body.Bytes(), &resp))
data, ok := resp.Data.(map[string]any)
require.Equal(t, true, ok)
assert.Equal(t, 175, len(data))
assert.Equal(t, 176, len(data))
for k, v := range data {
t.Run(k, func(t *testing.T) {
switch k {
@@ -419,6 +419,8 @@ func TestGetSpec(t *testing.T) {
assert.Equal(t, "0x0a000000", v)
case "DOMAIN_APPLICATION_BUILDER":
assert.Equal(t, "0x00000001", v)
case "DOMAIN_BEACON_BUILDER":
assert.Equal(t, "0x1b000000", v)
case "DOMAIN_BLOB_SIDECAR":
assert.Equal(t, "0x00000000", v)
case "TRANSITION_TOTAL_DIFFICULTY":

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,72 @@ 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
}
type computeStateRootAttemptsKeyType string
const computeStateRootAttemptsKey = computeStateRootAttemptsKeyType("compute-state-root-attempts")
const maxComputeStateRootAttempts = 3
// 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")
}
// prevent deep recursion by limiting max attempts.
if v, ok := ctx.Value(computeStateRootAttemptsKey).(int); !ok {
ctx = context.WithValue(ctx, computeStateRootAttemptsKey, int(1))
} else if v >= maxComputeStateRootAttempts {
return nil, fmt.Errorf("attempted max compute state root attempts %d", maxComputeStateRootAttempts)
} else {
ctx = context.WithValue(ctx, computeStateRootAttemptsKey, v+1)
}
// 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

@@ -1313,6 +1313,59 @@ func TestProposer_ComputeStateRoot_OK(t *testing.T) {
require.NoError(t, err)
}
func TestHandleStateRootError_MaxAttemptsReached(t *testing.T) {
// Test that handleStateRootError returns an error when max attempts is reached
// instead of recursing infinitely.
ctx := t.Context()
vs := &Server{}
// Create a minimal block for testing
blk := util.NewBeaconBlock()
wsb, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
// Pre-seed the context with max attempts already reached
ctx = context.WithValue(ctx, computeStateRootAttemptsKey, maxComputeStateRootAttempts)
// Call handleStateRootError with a retryable error
_, err = vs.handleStateRootError(ctx, wsb, transition.ErrAttestationsSignatureInvalid)
// Should return an error about max attempts instead of recursing
require.ErrorContains(t, "attempted max compute state root attempts", err)
}
func TestHandleStateRootError_IncrementsAttempts(t *testing.T) {
// Test that handleStateRootError properly increments the attempts counter
// and eventually fails after max attempts.
db := dbutil.SetupDB(t)
ctx := t.Context()
beaconState, parentRoot, _ := util.DeterministicGenesisStateWithGenesisBlock(t, ctx, db, 100)
stateGen := stategen.New(db, doublylinkedtree.New())
vs := &Server{
StateGen: stateGen,
}
// Create a block that will trigger retries
blk := util.NewBeaconBlock()
blk.Block.ParentRoot = parentRoot[:]
blk.Block.Slot = 1
wsb, err := blocks.NewSignedBeaconBlock(blk)
require.NoError(t, err)
// Add a state for the parent root so StateByRoot succeeds
require.NoError(t, stateGen.SaveState(ctx, parentRoot, beaconState))
// Call handleStateRootError with a retryable error - it will recurse
// but eventually hit the max attempts limit since CalculateStateRoot
// will keep failing (no valid attestations, randao, etc.)
_, err = vs.handleStateRootError(ctx, wsb, transition.ErrAttestationsSignatureInvalid)
// Should eventually fail - either with max attempts or another error
require.NotNil(t, err)
}
func TestProposer_PendingDeposits_Eth1DataVoteOK(t *testing.T) {
ctx := t.Context()

View File

@@ -5,12 +5,14 @@ go_library(
srcs = [
"error.go",
"interfaces.go",
"interfaces_gloas.go",
"prometheus.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/beacon-chain/state",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/state/state-native/custom-types:go_default_library",
"//beacon-chain/state/state-native/types:go_default_library",
"//config/fieldparams:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",

View File

@@ -10,6 +10,7 @@ import (
"github.com/OffchainLabs/go-bitfield"
customtypes "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/custom-types"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/types"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
@@ -43,6 +44,8 @@ type Prover interface {
FinalizedRootProof(ctx context.Context) ([][]byte, error)
CurrentSyncCommitteeProof(ctx context.Context) ([][]byte, error)
NextSyncCommitteeProof(ctx context.Context) ([][]byte, error)
ProofByFieldIndex(ctx context.Context, f types.FieldIndex) ([][]byte, error)
}
// ReadOnlyBeaconState defines a struct which only has read access to beacon state methods.
@@ -63,6 +66,7 @@ type ReadOnlyBeaconState interface {
ReadOnlyDeposits
ReadOnlyConsolidations
ReadOnlyProposerLookahead
ReadOnlyGloasFields
ToProtoUnsafe() any
ToProto() any
GenesisTime() time.Time
@@ -99,6 +103,7 @@ type WriteOnlyBeaconState interface {
WriteOnlyDeposits
WriteOnlyProposerLookahead
SetGenesisTime(val time.Time) error
WriteOnlyGloasFields
SetGenesisValidatorsRoot(val []byte) error
SetSlot(val primitives.Slot) error
SetFork(val *ethpb.Fork) error

View File

@@ -0,0 +1,26 @@
package state
import (
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"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"
)
type WriteOnlyGloasFields interface {
SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid) error
SetBuilderPendingPayment(index primitives.Slot, payment *ethpb.BuilderPendingPayment) error
SetLatestBlockHash(hash [32]byte) error
SetExecutionPayloadAvailability(index primitives.Slot, available bool) error
SetBuilderPendingPayments(payments []*ethpb.BuilderPendingPayment) error
AppendBuilderPendingWithdrawal(withdrawal *ethpb.BuilderPendingWithdrawal) error
}
type ReadOnlyGloasFields interface {
LatestBlockHash() ([32]byte, error)
BuilderPendingPayment(slot primitives.Slot) (*ethpb.BuilderPendingPayment, error)
BuilderPendingPayments() ([]*ethpb.BuilderPendingPayment, error)
BuilderPendingWithdrawals() ([]*ethpb.BuilderPendingWithdrawal, error)
LatestExecutionPayloadBid() (interfaces.ROExecutionPayloadBid, error)
PayloadExpectedWithdrawals() ([]*enginev1.Withdrawal, error)
}

View File

@@ -14,6 +14,7 @@ go_library(
"getters_deposits.go",
"getters_eth1.go",
"getters_exit.go",
"getters_gloas.go",
"getters_misc.go",
"getters_participation.go",
"getters_payload_header.go",
@@ -36,6 +37,7 @@ go_library(
"setters_deposit_requests.go",
"setters_deposits.go",
"setters_eth1.go",
"setters_gloas.go",
"setters_misc.go",
"setters_participation.go",
"setters_payload_header.go",
@@ -97,6 +99,7 @@ go_test(
"getters_deposit_requests_test.go",
"getters_deposits_test.go",
"getters_exit_test.go",
"getters_gloas_test.go",
"getters_participation_test.go",
"getters_setters_lookahead_test.go",
"getters_test.go",
@@ -114,6 +117,7 @@ go_test(
"setters_deposit_requests_test.go",
"setters_deposits_test.go",
"setters_eth1_test.go",
"setters_gloas_test.go",
"setters_misc_test.go",
"setters_participation_test.go",
"setters_payload_header_test.go",

View File

@@ -0,0 +1,81 @@
package state_native
import (
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"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"
)
// LatestBlockHash returns the hash of the latest execution block.
func (b *BeaconState) LatestBlockHash() ([32]byte, error) {
b.lock.RLock()
defer b.lock.RUnlock()
if b.latestBlockHash == nil {
return [32]byte{}, nil
}
return [32]byte(b.latestBlockHash), nil
}
// BuilderPendingPayment returns a specific builder pending payment for the given slot.
func (b *BeaconState) BuilderPendingPayment(slot primitives.Slot) (*ethpb.BuilderPendingPayment, error) {
b.lock.RLock()
defer b.lock.RUnlock()
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
paymentIndex := slotsPerEpoch + (slot % slotsPerEpoch)
return ethpb.CopyBuilderPendingPayment(b.builderPendingPayments[paymentIndex]), nil
}
// BuilderPendingPayments returns the builder pending payments.
func (b *BeaconState) BuilderPendingPayments() ([]*ethpb.BuilderPendingPayment, error) {
b.lock.RLock()
defer b.lock.RUnlock()
if b.builderPendingPayments == nil {
return make([]*ethpb.BuilderPendingPayment, 0), nil
}
return ethpb.CopyBuilderPendingPaymentSlice(b.builderPendingPayments), nil
}
// BuilderPendingWithdrawals returns the builder pending withdrawals.
func (b *BeaconState) BuilderPendingWithdrawals() ([]*ethpb.BuilderPendingWithdrawal, error) {
b.lock.RLock()
defer b.lock.RUnlock()
if b.builderPendingWithdrawals == nil {
return make([]*ethpb.BuilderPendingWithdrawal, 0), nil
}
return ethpb.CopyBuilderPendingWithdrawalSlice(b.builderPendingWithdrawals), nil
}
// LatestExecutionPayloadBid returns the cached latest execution payload bid for Gloas.
func (b *BeaconState) LatestExecutionPayloadBid() (interfaces.ROExecutionPayloadBid, error) {
b.lock.RLock()
defer b.lock.RUnlock()
if b.latestExecutionPayloadBid == nil {
return nil, nil
}
return blocks.WrappedROExecutionPayloadBid(b.latestExecutionPayloadBid.Copy())
}
// PayloadExpectedWithdrawals returns the expected withdrawals for the current payload.
func (b *BeaconState) PayloadExpectedWithdrawals() ([]*enginev1.Withdrawal, error) {
b.lock.RLock()
defer b.lock.RUnlock()
if b.payloadExpectedWithdrawals == nil {
return make([]*enginev1.Withdrawal, 0), nil
}
return b.payloadExpectedWithdrawalsVal(), nil
}

View File

@@ -0,0 +1,67 @@
package state_native_test
import (
"testing"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/assert"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
func TestLatestBlockHash(t *testing.T) {
t.Run("nil hash returns zero", func(t *testing.T) {
s, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{})
require.NoError(t, err)
hash, err := s.LatestBlockHash()
require.NoError(t, err)
assert.Equal(t, [32]byte{}, hash)
})
t.Run("returns value", func(t *testing.T) {
var want [32]byte
copy(want[:], []byte("latest-block-hash"))
s, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{
LatestBlockHash: want[:],
})
require.NoError(t, err)
hash, err := s.LatestBlockHash()
require.NoError(t, err)
assert.Equal(t, want, hash)
})
}
func TestBuilderPendingPayment(t *testing.T) {
t.Run("returns payment for slot", func(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
slot := primitives.Slot(7)
paymentIndex := slotsPerEpoch + (slot % slotsPerEpoch)
payments := make([]*ethpb.BuilderPendingPayment, slotsPerEpoch*2)
payment := &ethpb.BuilderPendingPayment{
Weight: 11,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20},
Amount: 99,
BuilderIndex: 3,
},
}
payments[paymentIndex] = payment
s, err := state_native.InitializeFromProtoGloas(&ethpb.BeaconStateGloas{
BuilderPendingPayments: payments,
})
require.NoError(t, err)
got, err := s.BuilderPendingPayment(slot)
require.NoError(t, err)
require.DeepEqual(t, payment, got)
got.Withdrawal.FeeRecipient[0] = 9
require.Equal(t, byte(1), payment.Withdrawal.FeeRecipient[0])
})
}

View File

@@ -5,6 +5,7 @@ import (
"encoding/binary"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/types"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/container/trie"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
"github.com/OffchainLabs/prysm/v7/runtime/version"
@@ -39,42 +40,12 @@ func (b *BeaconState) NextSyncCommitteeGeneralizedIndex() (uint64, error) {
// CurrentSyncCommitteeProof from the state's Merkle trie representation.
func (b *BeaconState) CurrentSyncCommitteeProof(ctx context.Context) ([][]byte, error) {
b.lock.Lock()
defer b.lock.Unlock()
if b.version == version.Phase0 {
return nil, errNotSupported("CurrentSyncCommitteeProof", b.version)
}
// In case the Merkle layers of the trie are not populated, we need
// to perform some initialization.
if err := b.initializeMerkleLayers(ctx); err != nil {
return nil, err
}
// Our beacon state uses a "dirty" fields pattern which requires us to
// recompute branches of the Merkle layers that are marked as dirty.
if err := b.recomputeDirtyFields(ctx); err != nil {
return nil, err
}
return trie.ProofFromMerkleLayers(b.merkleLayers, types.CurrentSyncCommittee.RealPosition()), nil
return b.ProofByFieldIndex(ctx, types.CurrentSyncCommittee)
}
// NextSyncCommitteeProof from the state's Merkle trie representation.
func (b *BeaconState) NextSyncCommitteeProof(ctx context.Context) ([][]byte, error) {
b.lock.Lock()
defer b.lock.Unlock()
if b.version == version.Phase0 {
return nil, errNotSupported("NextSyncCommitteeProof", b.version)
}
if err := b.initializeMerkleLayers(ctx); err != nil {
return nil, err
}
if err := b.recomputeDirtyFields(ctx); err != nil {
return nil, err
}
return trie.ProofFromMerkleLayers(b.merkleLayers, types.NextSyncCommittee.RealPosition()), nil
return b.ProofByFieldIndex(ctx, types.NextSyncCommittee)
}
// FinalizedRootProof crafts a Merkle proof for the finalized root
@@ -83,8 +54,37 @@ func (b *BeaconState) FinalizedRootProof(ctx context.Context) ([][]byte, error)
b.lock.Lock()
defer b.lock.Unlock()
if b.version == version.Phase0 {
return nil, errNotSupported("FinalizedRootProof", b.version)
branchProof, err := b.proofByFieldIndex(ctx, types.FinalizedCheckpoint)
if err != nil {
return nil, err
}
// The epoch field of a finalized checkpoint is the neighbor
// index of the finalized root field in its Merkle tree representation
// of the checkpoint. This neighbor is the first element added to the proof.
epochBuf := make([]byte, 8)
binary.LittleEndian.PutUint64(epochBuf, uint64(b.finalizedCheckpointVal().Epoch))
epochRoot := bytesutil.ToBytes32(epochBuf)
proof := make([][]byte, 0)
proof = append(proof, epochRoot[:])
proof = append(proof, branchProof...)
return proof, nil
}
// ProofByFieldIndex constructs proofs for given field index with lock acquisition.
func (b *BeaconState) ProofByFieldIndex(ctx context.Context, f types.FieldIndex) ([][]byte, error) {
b.lock.Lock()
defer b.lock.Unlock()
return b.proofByFieldIndex(ctx, f)
}
// proofByFieldIndex constructs proofs for given field index.
// Important: it is assumed that beacon state mutex is locked when calling this method.
func (b *BeaconState) proofByFieldIndex(ctx context.Context, f types.FieldIndex) ([][]byte, error) {
err := b.validateFieldIndex(f)
if err != nil {
return nil, err
}
if err := b.initializeMerkleLayers(ctx); err != nil {
@@ -93,16 +93,40 @@ func (b *BeaconState) FinalizedRootProof(ctx context.Context) ([][]byte, error)
if err := b.recomputeDirtyFields(ctx); err != nil {
return nil, err
}
cpt := b.finalizedCheckpointVal()
// The epoch field of a finalized checkpoint is the neighbor
// index of the finalized root field in its Merkle tree representation
// of the checkpoint. This neighbor is the first element added to the proof.
epochBuf := make([]byte, 8)
binary.LittleEndian.PutUint64(epochBuf, uint64(cpt.Epoch))
epochRoot := bytesutil.ToBytes32(epochBuf)
proof := make([][]byte, 0)
proof = append(proof, epochRoot[:])
branch := trie.ProofFromMerkleLayers(b.merkleLayers, types.FinalizedCheckpoint.RealPosition())
proof = append(proof, branch...)
return proof, nil
return trie.ProofFromMerkleLayers(b.merkleLayers, f.RealPosition()), nil
}
func (b *BeaconState) validateFieldIndex(f types.FieldIndex) error {
switch b.version {
case version.Phase0:
if f.RealPosition() > params.BeaconConfig().BeaconStateFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Altair:
if f.RealPosition() > params.BeaconConfig().BeaconStateAltairFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Bellatrix:
if f.RealPosition() > params.BeaconConfig().BeaconStateBellatrixFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Capella:
if f.RealPosition() > params.BeaconConfig().BeaconStateCapellaFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Deneb:
if f.RealPosition() > params.BeaconConfig().BeaconStateDenebFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Electra:
if f.RealPosition() > params.BeaconConfig().BeaconStateElectraFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
case version.Fulu:
if f.RealPosition() > params.BeaconConfig().BeaconStateFuluFieldCount-1 {
return errNotSupported(f.String(), b.version)
}
}
return nil
}

View File

@@ -21,10 +21,6 @@ func TestBeaconStateMerkleProofs_phase0_notsupported(t *testing.T) {
_, err := st.NextSyncCommitteeProof(ctx)
require.ErrorContains(t, "not supported", err)
})
t.Run("finalized root", func(t *testing.T) {
_, err := st.FinalizedRootProof(ctx)
require.ErrorContains(t, "not supported", err)
})
}
func TestBeaconStateMerkleProofs_altair(t *testing.T) {
ctx := t.Context()

View File

@@ -0,0 +1,101 @@
package state_native
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/types"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
func (b *BeaconState) SetExecutionPayloadBid(h interfaces.ROExecutionPayloadBid) error {
b.lock.Lock()
defer b.lock.Unlock()
parentBlockHash := h.ParentBlockHash()
parentBlockRoot := h.ParentBlockRoot()
blockHash := h.BlockHash()
randao := h.PrevRandao()
blobKzgCommitmentsRoot := h.BlobKzgCommitmentsRoot()
feeRecipient := h.FeeRecipient()
b.latestExecutionPayloadBid = &ethpb.ExecutionPayloadBid{
ParentBlockHash: parentBlockHash[:],
ParentBlockRoot: parentBlockRoot[:],
BlockHash: blockHash[:],
PrevRandao: randao[:],
GasLimit: h.GasLimit(),
BuilderIndex: h.BuilderIndex(),
Slot: h.Slot(),
Value: h.Value(),
ExecutionPayment: h.ExecutionPayment(),
BlobKzgCommitmentsRoot: blobKzgCommitmentsRoot[:],
FeeRecipient: feeRecipient[:],
}
b.markFieldAsDirty(types.LatestExecutionPayloadBid)
return nil
}
// SetBuilderPendingPayment sets a builder pending payment for the specified slot.
func (b *BeaconState) SetBuilderPendingPayment(slot primitives.Slot, payment *ethpb.BuilderPendingPayment) error {
b.lock.Lock()
defer b.lock.Unlock()
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
paymentIndex := slotsPerEpoch + (slot % slotsPerEpoch)
b.builderPendingPayments[paymentIndex] = ethpb.CopyBuilderPendingPayment(payment)
b.markFieldAsDirty(types.BuilderPendingPayments)
return nil
}
// SetLatestBlockHash sets the latest execution block hash.
func (b *BeaconState) SetLatestBlockHash(hash [32]byte) error {
b.lock.Lock()
defer b.lock.Unlock()
b.latestBlockHash = hash[:]
b.markFieldAsDirty(types.LatestBlockHash)
return nil
}
// SetExecutionPayloadAvailability sets the execution payload availability bit for a specific slot.
func (b *BeaconState) SetExecutionPayloadAvailability(index primitives.Slot, available bool) error {
b.lock.Lock()
defer b.lock.Unlock()
bitIndex := index % params.BeaconConfig().SlotsPerHistoricalRoot
byteIndex := bitIndex / 8
bitPosition := bitIndex % 8
// Set or clear the bit
if available {
b.executionPayloadAvailability[byteIndex] |= 1 << bitPosition
} else {
b.executionPayloadAvailability[byteIndex] &^= 1 << bitPosition
}
b.markFieldAsDirty(types.ExecutionPayloadAvailability)
return nil
}
// SetBuilderPendingPayments sets the entire builder pending payments array.
func (b *BeaconState) SetBuilderPendingPayments(payments []*ethpb.BuilderPendingPayment) error {
b.lock.Lock()
defer b.lock.Unlock()
b.builderPendingPayments = ethpb.CopyBuilderPendingPaymentSlice(payments)
b.markFieldAsDirty(types.BuilderPendingPayments)
return nil
}
// AppendBuilderPendingWithdrawal appends a builder pending withdrawal to the list.
func (b *BeaconState) AppendBuilderPendingWithdrawal(withdrawal *ethpb.BuilderPendingWithdrawal) error {
b.lock.Lock()
defer b.lock.Unlock()
b.builderPendingWithdrawals = append(b.builderPendingWithdrawals, ethpb.CopyBuilderPendingWithdrawal(withdrawal))
b.markFieldAsDirty(types.BuilderPendingWithdrawals)
return nil
}

View File

@@ -0,0 +1,121 @@
package state_native
import (
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native/types"
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
func TestSetBuilderPendingPayment(t *testing.T) {
slotsPerEpoch := params.BeaconConfig().SlotsPerEpoch
slot := primitives.Slot(7)
paymentIndex := slotsPerEpoch + (slot % slotsPerEpoch)
payment := &ethpb.BuilderPendingPayment{
Weight: 11,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: []byte{1, 2, 3, 4, 5},
Amount: 99,
BuilderIndex: 3,
},
}
state := &BeaconState{
builderPendingPayments: make([]*ethpb.BuilderPendingPayment, slotsPerEpoch*2),
dirtyFields: make(map[types.FieldIndex]bool),
}
require.NoError(t, state.SetBuilderPendingPayment(slot, payment))
require.Equal(t, true, state.dirtyFields[types.BuilderPendingPayments])
require.DeepEqual(t, payment, state.builderPendingPayments[paymentIndex])
state.builderPendingPayments[paymentIndex].Withdrawal.FeeRecipient[0] = 9
require.Equal(t, byte(1), payment.Withdrawal.FeeRecipient[0])
}
func TestSetLatestBlockHash(t *testing.T) {
var hash [32]byte
copy(hash[:], []byte("latest-block-hash"))
state := &BeaconState{
dirtyFields: make(map[types.FieldIndex]bool),
}
require.NoError(t, state.SetLatestBlockHash(hash))
require.Equal(t, true, state.dirtyFields[types.LatestBlockHash])
require.DeepEqual(t, hash[:], state.latestBlockHash)
}
func TestSetExecutionPayloadAvailability(t *testing.T) {
state := &BeaconState{
executionPayloadAvailability: make([]byte, params.BeaconConfig().SlotsPerHistoricalRoot/8),
dirtyFields: make(map[types.FieldIndex]bool),
}
slot := primitives.Slot(10)
bitIndex := slot % params.BeaconConfig().SlotsPerHistoricalRoot
byteIndex := bitIndex / 8
bitPosition := bitIndex % 8
require.NoError(t, state.SetExecutionPayloadAvailability(slot, true))
require.Equal(t, true, state.dirtyFields[types.ExecutionPayloadAvailability])
require.Equal(t, byte(1<<bitPosition), state.executionPayloadAvailability[byteIndex]&(1<<bitPosition))
require.NoError(t, state.SetExecutionPayloadAvailability(slot, false))
require.Equal(t, byte(0), state.executionPayloadAvailability[byteIndex]&(1<<bitPosition))
}
func TestSetBuilderPendingPayments(t *testing.T) {
payments := []*ethpb.BuilderPendingPayment{
{
Weight: 1,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: []byte{1, 1, 1},
Amount: 5,
BuilderIndex: 2,
},
},
{
Weight: 2,
Withdrawal: &ethpb.BuilderPendingWithdrawal{
FeeRecipient: []byte{2, 2, 2},
Amount: 6,
BuilderIndex: 4,
},
},
}
state := &BeaconState{
dirtyFields: make(map[types.FieldIndex]bool),
}
require.NoError(t, state.SetBuilderPendingPayments(payments))
require.Equal(t, true, state.dirtyFields[types.BuilderPendingPayments])
require.DeepEqual(t, payments, state.builderPendingPayments)
state.builderPendingPayments[0].Weight = 99
require.Equal(t, primitives.Gwei(1), payments[0].Weight)
}
func TestAppendBuilderPendingWithdrawal(t *testing.T) {
withdrawal := &ethpb.BuilderPendingWithdrawal{
FeeRecipient: []byte{9, 8, 7},
Amount: 77,
BuilderIndex: 12,
}
state := &BeaconState{
dirtyFields: make(map[types.FieldIndex]bool),
}
require.NoError(t, state.AppendBuilderPendingWithdrawal(withdrawal))
require.Equal(t, true, state.dirtyFields[types.BuilderPendingWithdrawals])
require.Equal(t, 1, len(state.builderPendingWithdrawals))
require.DeepEqual(t, withdrawal, state.builderPendingWithdrawals[0])
state.builderPendingWithdrawals[0].FeeRecipient[0] = 1
require.Equal(t, byte(9), withdrawal.FeeRecipient[0])
}

View File

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

View File

@@ -1,2 +0,0 @@
### Added
- Add feature flag to use hashtree instead of gohashtre.

View File

@@ -0,0 +1,3 @@
### Added
- Add `ProofByFieldIndex` to generalize merkle proof generation for `BeaconState`.

View File

@@ -0,0 +1,3 @@
### Added
- Add process execution payload for gloas

View File

@@ -70,8 +70,8 @@ type Flags struct {
DisableStakinContractCheck bool // Disables check for deposit contract when proposing blocks
IgnoreUnviableAttestations bool // Ignore attestations whose target state is not viable (avoids lagging-node DoS).
EnableHashtree bool // Enables usage of the hashtree library for hashing
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.
@@ -238,15 +238,15 @@ func ConfigureBeaconChain(ctx *cli.Context) error {
logEnabled(enableFullSSZDataLogging)
cfg.EnableFullSSZDataLogging = true
}
if ctx.IsSet(enableHashtree.Name) {
logEnabled(enableHashtree)
cfg.EnableHashtree = true
}
cfg.EnableVerboseSigVerification = true
if ctx.IsSet(disableVerboseSigVerification.Name) {
logEnabled(disableVerboseSigVerification)
cfg.EnableVerboseSigVerification = false
}
cfg.EnableProposerPreprocessing = ctx.Bool(enableProposerPreprocessing.Name)
if cfg.EnableProposerPreprocessing {
logEnabled(enableProposerPreprocessing)
}
if ctx.IsSet(prepareAllPayloads.Name) {
logEnabled(prepareAllPayloads)
cfg.PrepareAllPayloads = true

View File

@@ -133,14 +133,15 @@ var (
Name: "enable-beacon-rest-api",
Usage: "(Experimental): Enables of the beacon REST API when querying a beacon node.",
}
enableHashtree = &cli.BoolFlag{
Name: "enable-hashtree",
Usage: "(Experimental): Enables the hashtree hashing library.",
}
disableVerboseSigVerification = &cli.BoolFlag{
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.",
@@ -270,6 +271,7 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
ignoreUnviableAttestations,
enableFullSSZDataLogging,
disableVerboseSigVerification,
enableProposerPreprocessing,
prepareAllPayloads,
aggregateFirstInterval,
aggregateSecondInterval,
@@ -284,7 +286,6 @@ var BeaconChainFlags = combinedFlags([]cli.Flag{
forceHeadFlag,
blacklistRoots,
lowValcountSweep,
enableHashtree,
}, deprecatedBeaconFlags, deprecatedFlags, upcomingDeprecation)
func combinedFlags(flags ...[]cli.Flag) []cli.Flag {

View File

@@ -139,6 +139,7 @@ type BeaconChainConfig struct {
DomainApplicationMask [4]byte `yaml:"DOMAIN_APPLICATION_MASK" spec:"true"` // DomainApplicationMask defines the BLS signature domain for application mask.
DomainApplicationBuilder [4]byte `yaml:"DOMAIN_APPLICATION_BUILDER" spec:"true"` // DomainApplicationBuilder defines the BLS signature domain for application builder.
DomainBLSToExecutionChange [4]byte `yaml:"DOMAIN_BLS_TO_EXECUTION_CHANGE" spec:"true"` // DomainBLSToExecutionChange defines the BLS signature domain to change withdrawal addresses to ETH1 prefix
DomainBeaconBuilder [4]byte `yaml:"DOMAIN_BEACON_BUILDER" spec:"true"` // DomainBeaconBuilder defines the BLS signature domain for beacon block builder.
// Prysm constants.
GenesisValidatorsRoot [32]byte // GenesisValidatorsRoot is the root hash of the genesis validators.

View File

@@ -182,6 +182,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
DomainApplicationMask: bytesutil.Uint32ToBytes4(0x00000001),
DomainApplicationBuilder: bytesutil.Uint32ToBytes4(0x00000001),
DomainBLSToExecutionChange: bytesutil.Uint32ToBytes4(0x0A000000),
DomainBeaconBuilder: bytesutil.Uint32ToBytes4(0x1B000000),
// Prysm constants.
GenesisValidatorsRoot: [32]byte{75, 54, 61, 185, 78, 40, 97, 32, 215, 110, 185, 5, 52, 15, 221, 78, 84, 191, 233, 240, 107, 243, 63, 246, 207, 90, 210, 127, 81, 27, 254, 149},

View File

@@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"execution.go",
"execution_payload_envelope.go",
"factory.go",
"get_payload.go",
"getters.go",
@@ -14,11 +15,13 @@ go_library(
"roblock.go",
"rodatacolumn.go",
"setters.go",
"signed_execution_bid.go",
"types.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/consensus-types/blocks",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/state/stateutil:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
@@ -34,8 +37,10 @@ go_library(
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/validator-client:go_default_library",
"//runtime/version:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_prysmaticlabs_gohashtree//:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
],
)
@@ -43,6 +48,7 @@ go_library(
go_test(
name = "go_default_test",
srcs = [
"execution_payload_envelope_test.go",
"execution_test.go",
"factory_test.go",
"getters_test.go",
@@ -56,13 +62,13 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/signing:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/trie:go_default_library",
"//crypto/hash/htr:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
@@ -72,5 +78,6 @@ go_test(
"//testing/require:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_prysmaticlabs_gohashtree//:go_default_library",
],
)

View File

@@ -0,0 +1,156 @@
package blocks
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
field_params "github.com/OffchainLabs/prysm/v7/config/fieldparams"
consensus_types "github.com/OffchainLabs/prysm/v7/consensus-types"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
"github.com/OffchainLabs/prysm/v7/encoding/ssz"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/ethereum/go-ethereum/common"
"google.golang.org/protobuf/proto"
)
type signedExecutionPayloadEnvelope struct {
s *ethpb.SignedExecutionPayloadEnvelope
}
type executionPayloadEnvelope struct {
p *ethpb.ExecutionPayloadEnvelope
}
// WrappedROSignedExecutionPayloadEnvelope wraps a signed execution payload envelope proto in a read-only interface.
func WrappedROSignedExecutionPayloadEnvelope(s *ethpb.SignedExecutionPayloadEnvelope) (interfaces.ROSignedExecutionPayloadEnvelope, error) {
w := signedExecutionPayloadEnvelope{s: s}
if w.IsNil() {
return nil, consensus_types.ErrNilObjectWrapped
}
return w, nil
}
// WrappedROExecutionPayloadEnvelope wraps an execution payload envelope proto in a read-only interface.
func WrappedROExecutionPayloadEnvelope(p *ethpb.ExecutionPayloadEnvelope) (interfaces.ROExecutionPayloadEnvelope, error) {
w := &executionPayloadEnvelope{p: p}
if w.IsNil() {
return nil, consensus_types.ErrNilObjectWrapped
}
return w, nil
}
// Envelope returns the execution payload envelope as a read-only interface.
func (s signedExecutionPayloadEnvelope) Envelope() (interfaces.ROExecutionPayloadEnvelope, error) {
return WrappedROExecutionPayloadEnvelope(s.s.Message)
}
// Signature returns the BLS signature as a 96-byte array.
func (s signedExecutionPayloadEnvelope) Signature() [field_params.BLSSignatureLength]byte {
return [field_params.BLSSignatureLength]byte(s.s.Signature)
}
// IsNil reports whether the signed envelope or its contents are invalid.
func (s signedExecutionPayloadEnvelope) IsNil() bool {
if s.s == nil {
return true
}
if len(s.s.Signature) != field_params.BLSSignatureLength {
return true
}
w := executionPayloadEnvelope{p: s.s.Message}
return w.IsNil()
}
// SigningRoot computes the signing root for the signed envelope with the provided domain.
func (s signedExecutionPayloadEnvelope) SigningRoot(domain []byte) (root [32]byte, err error) {
return signing.ComputeSigningRoot(s.s.Message, domain)
}
// Proto returns the underlying protobuf message.
func (s signedExecutionPayloadEnvelope) Proto() proto.Message {
return s.s
}
// IsNil reports whether the envelope or its required fields are invalid.
func (p *executionPayloadEnvelope) IsNil() bool {
if p.p == nil {
return true
}
if p.p.Payload == nil {
return true
}
if len(p.p.BeaconBlockRoot) != field_params.RootLength {
return true
}
if p.p.BlobKzgCommitments == nil {
return true
}
return false
}
// IsBlinded reports whether the envelope contains a blinded payload.
func (p *executionPayloadEnvelope) IsBlinded() bool {
return !p.IsNil() && p.p.Payload == nil
}
// Execution returns the execution payload as a read-only interface.
func (p *executionPayloadEnvelope) Execution() (interfaces.ExecutionData, error) {
if p.IsBlinded() {
return nil, consensus_types.ErrNilObjectWrapped
}
return WrappedExecutionPayloadDeneb(p.p.Payload)
}
// ExecutionRequests returns the execution requests attached to the envelope.
func (p *executionPayloadEnvelope) ExecutionRequests() *enginev1.ExecutionRequests {
return p.p.ExecutionRequests
}
// BuilderIndex returns the proposer/builder index for the envelope.
func (p *executionPayloadEnvelope) BuilderIndex() primitives.BuilderIndex {
return p.p.BuilderIndex
}
// BeaconBlockRoot returns the beacon block root referenced by the envelope.
func (p *executionPayloadEnvelope) BeaconBlockRoot() [field_params.RootLength]byte {
return [field_params.RootLength]byte(p.p.BeaconBlockRoot)
}
// BlobKzgCommitments returns a copy of the envelope's KZG commitments.
func (p *executionPayloadEnvelope) BlobKzgCommitments() [][]byte {
commitments := make([][]byte, len(p.p.BlobKzgCommitments))
for i, commit := range p.p.BlobKzgCommitments {
commitments[i] = make([]byte, len(commit))
copy(commitments[i], commit)
}
return commitments
}
// VersionedHashes returns versioned hashes derived from the KZG commitments.
func (p *executionPayloadEnvelope) VersionedHashes() []common.Hash {
commitments := p.p.BlobKzgCommitments
versionedHashes := make([]common.Hash, len(commitments))
for i, commitment := range commitments {
versionedHashes[i] = primitives.ConvertKzgCommitmentToVersionedHash(commitment)
}
return versionedHashes
}
// BlobKzgCommitmentsRoot returns the SSZ root of the envelope's KZG commitments.
func (p *executionPayloadEnvelope) BlobKzgCommitmentsRoot() ([field_params.RootLength]byte, error) {
if p.IsNil() || p.p.BlobKzgCommitments == nil {
return [field_params.RootLength]byte{}, consensus_types.ErrNilObjectWrapped
}
return ssz.KzgCommitmentsRoot(p.p.BlobKzgCommitments)
}
// Slot returns the slot of the envelope.
func (p *executionPayloadEnvelope) Slot() primitives.Slot {
return p.p.Slot
}
// StateRoot returns the state root carried by the envelope.
func (p *executionPayloadEnvelope) StateRoot() [field_params.RootLength]byte {
return [field_params.RootLength]byte(p.p.StateRoot)
}

View File

@@ -0,0 +1,115 @@
package blocks_test
import (
"bytes"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
consensus_types "github.com/OffchainLabs/prysm/v7/consensus-types"
"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/assert"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
func validExecutionPayloadEnvelope() *ethpb.ExecutionPayloadEnvelope {
payload := &enginev1.ExecutionPayloadDeneb{
ParentHash: bytes.Repeat([]byte{0x01}, 32),
FeeRecipient: bytes.Repeat([]byte{0x02}, 20),
StateRoot: bytes.Repeat([]byte{0x03}, 32),
ReceiptsRoot: bytes.Repeat([]byte{0x04}, 32),
LogsBloom: bytes.Repeat([]byte{0x05}, 256),
PrevRandao: bytes.Repeat([]byte{0x06}, 32),
BlockNumber: 1,
GasLimit: 2,
GasUsed: 3,
Timestamp: 4,
BaseFeePerGas: bytes.Repeat([]byte{0x07}, 32),
BlockHash: bytes.Repeat([]byte{0x08}, 32),
Transactions: [][]byte{},
Withdrawals: []*enginev1.Withdrawal{},
BlobGasUsed: 0,
ExcessBlobGas: 0,
}
return &ethpb.ExecutionPayloadEnvelope{
Payload: payload,
ExecutionRequests: &enginev1.ExecutionRequests{},
BuilderIndex: 10,
BeaconBlockRoot: bytes.Repeat([]byte{0xAA}, 32),
Slot: 9,
BlobKzgCommitments: [][]byte{bytes.Repeat([]byte{0x0C}, 48)},
StateRoot: bytes.Repeat([]byte{0xBB}, 32),
}
}
func TestWrappedROExecutionPayloadEnvelope(t *testing.T) {
t.Run("returns error on invalid beacon root length", func(t *testing.T) {
invalid := validExecutionPayloadEnvelope()
invalid.BeaconBlockRoot = []byte{0x01}
_, err := blocks.WrappedROExecutionPayloadEnvelope(invalid)
require.Equal(t, consensus_types.ErrNilObjectWrapped, err)
})
t.Run("wraps and exposes fields", func(t *testing.T) {
env := validExecutionPayloadEnvelope()
wrapped, err := blocks.WrappedROExecutionPayloadEnvelope(env)
require.NoError(t, err)
require.Equal(t, primitives.BuilderIndex(10), wrapped.BuilderIndex())
require.Equal(t, primitives.Slot(9), wrapped.Slot())
assert.DeepEqual(t, [32]byte(bytes.Repeat([]byte{0xAA}, 32)), wrapped.BeaconBlockRoot())
assert.DeepEqual(t, [32]byte(bytes.Repeat([]byte{0xBB}, 32)), wrapped.StateRoot())
commitments := wrapped.BlobKzgCommitments()
assert.DeepEqual(t, env.BlobKzgCommitments, commitments)
versioned := wrapped.VersionedHashes()
require.Equal(t, 1, len(versioned))
reqs := wrapped.ExecutionRequests()
require.NotNil(t, reqs)
exec, err := wrapped.Execution()
require.NoError(t, err)
assert.DeepEqual(t, env.Payload.ParentHash, exec.ParentHash())
})
}
func TestWrappedROSignedExecutionPayloadEnvelope(t *testing.T) {
t.Run("returns error for invalid signature length", func(t *testing.T) {
signed := &ethpb.SignedExecutionPayloadEnvelope{
Message: validExecutionPayloadEnvelope(),
Signature: bytes.Repeat([]byte{0xAA}, 95),
}
_, err := blocks.WrappedROSignedExecutionPayloadEnvelope(signed)
require.Equal(t, consensus_types.ErrNilObjectWrapped, err)
})
t.Run("wraps and provides envelope/signing data", func(t *testing.T) {
sig := bytes.Repeat([]byte{0xAB}, 96)
signed := &ethpb.SignedExecutionPayloadEnvelope{
Message: validExecutionPayloadEnvelope(),
Signature: sig,
}
wrapped, err := blocks.WrappedROSignedExecutionPayloadEnvelope(signed)
require.NoError(t, err)
gotSig := wrapped.Signature()
assert.DeepEqual(t, [96]byte(sig), gotSig)
env, err := wrapped.Envelope()
require.NoError(t, err)
assert.DeepEqual(t, [32]byte(bytes.Repeat([]byte{0xAA}, 32)), env.BeaconBlockRoot())
domain := bytes.Repeat([]byte{0xCC}, 32)
wantRoot, err := signing.ComputeSigningRoot(signed.Message, domain)
require.NoError(t, err)
gotRoot, err := wrapped.SigningRoot(domain)
require.NoError(t, err)
require.Equal(t, wantRoot, gotRoot)
})
}

View File

@@ -5,10 +5,10 @@ import (
"github.com/OffchainLabs/prysm/v7/config/params"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/container/trie"
"github.com/OffchainLabs/prysm/v7/crypto/hash/htr"
"github.com/OffchainLabs/prysm/v7/encoding/ssz"
"github.com/OffchainLabs/prysm/v7/runtime/version"
"github.com/pkg/errors"
"github.com/prysmaticlabs/gohashtree"
)
const (
@@ -45,7 +45,7 @@ func VerifyKZGInclusionProof(blob ROBlob) error {
return errInvalidBodyRoot
}
chunks := makeChunk(blob.KzgCommitment)
htr.HashChunks(chunks, chunks)
gohashtree.HashChunks(chunks, chunks)
verified := trie.VerifyMerkleProof(root, chunks[0][:], blob.Index+KZGOffset, blob.CommitmentInclusionProof)
if !verified {
return errInvalidInclusionProof
@@ -182,7 +182,7 @@ func LeavesFromCommitments(commitments [][]byte) [][]byte {
leaves := make([][]byte, len(commitments))
for i, kzg := range commitments {
chunk := makeChunk(kzg)
htr.HashChunks(chunk, chunk)
gohashtree.HashChunks(chunk, chunk)
leaves[i] = chunk[0][:]
}
return leaves

View File

@@ -8,10 +8,10 @@ import (
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/container/trie"
"github.com/OffchainLabs/prysm/v7/crypto/hash/htr"
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/prysmaticlabs/gohashtree"
)
func Test_MerkleProofKZGCommitment_Altair(t *testing.T) {
@@ -108,7 +108,7 @@ func Test_MerkleProofKZGCommitment(t *testing.T) {
require.Equal(t, true, trie.VerifyMerkleProof(root[:], commitmentsRoot[:], kzgPosition, topProof[:len(topProof)-1]))
chunk := makeChunk(kzgs[index])
htr.HashChunks(chunk, chunk)
gohashtree.HashChunks(chunk, chunk)
require.Equal(t, true, trie.VerifyMerkleProof(root[:], chunk[0][:], uint64(index+KZGOffset), proof))
}

View File

@@ -0,0 +1,140 @@
package blocks
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/signing"
consensus_types "github.com/OffchainLabs/prysm/v7/consensus-types"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
// signedExecutionPayloadBid wraps the protobuf signed execution payload bid
// and implements the ROSignedExecutionPayloadBid interface.
type signedExecutionPayloadBid struct {
bid *ethpb.SignedExecutionPayloadBid
}
// executionPayloadBidGloas wraps the protobuf execution payload bid for Gloas fork
// and implements the ROExecutionPayloadBidGloas interface.
type executionPayloadBidGloas struct {
payload *ethpb.ExecutionPayloadBid
}
// IsNil checks if the signed execution payload bid is nil or invalid.
func (s signedExecutionPayloadBid) IsNil() bool {
if s.bid == nil {
return true
}
if _, err := WrappedROExecutionPayloadBid(s.bid.Message); err != nil {
return true
}
return len(s.bid.Signature) != 96
}
// IsNil checks if the execution payload bid is nil or has invalid fields.
func (h executionPayloadBidGloas) IsNil() bool {
if h.payload == nil {
return true
}
if len(h.payload.ParentBlockHash) != 32 ||
len(h.payload.ParentBlockRoot) != 32 ||
len(h.payload.BlockHash) != 32 ||
len(h.payload.BlobKzgCommitmentsRoot) != 32 {
return true
}
return false
}
// WrappedROSignedExecutionPayloadBid creates a new read-only signed execution payload bid
// wrapper from the given protobuf message.
func WrappedROSignedExecutionPayloadBid(pb *ethpb.SignedExecutionPayloadBid) (interfaces.ROSignedExecutionPayloadBid, error) {
wrapper := signedExecutionPayloadBid{bid: pb}
if wrapper.IsNil() {
return nil, consensus_types.ErrNilObjectWrapped
}
return wrapper, nil
}
// WrappedROExecutionPayloadBid creates a new read-only execution payload bid
// wrapper for the Gloas fork from the given protobuf message.
func WrappedROExecutionPayloadBid(pb *ethpb.ExecutionPayloadBid) (interfaces.ROExecutionPayloadBid, error) {
wrapper := executionPayloadBidGloas{payload: pb}
if wrapper.IsNil() {
return nil, consensus_types.ErrNilObjectWrapped
}
return wrapper, nil
}
// Bid returns the execution payload bid as a read-only interface.
func (s signedExecutionPayloadBid) Bid() (interfaces.ROExecutionPayloadBid, error) {
return WrappedROExecutionPayloadBid(s.bid.Message)
}
// SigningRoot computes the signing root for the execution payload bid with the given domain.
func (s signedExecutionPayloadBid) SigningRoot(domain []byte) ([32]byte, error) {
return signing.ComputeSigningRoot(s.bid.Message, domain)
}
// Signature returns the BLS signature as a 96-byte array.
func (s signedExecutionPayloadBid) Signature() [96]byte {
return [96]byte(s.bid.Signature)
}
// ParentBlockHash returns the hash of the parent execution block.
func (h executionPayloadBidGloas) ParentBlockHash() [32]byte {
return [32]byte(h.payload.ParentBlockHash)
}
// ParentBlockRoot returns the beacon block root of the parent block.
func (h executionPayloadBidGloas) ParentBlockRoot() [32]byte {
return [32]byte(h.payload.ParentBlockRoot)
}
// PrevRandao returns the previous randao value for the execution block.
func (h executionPayloadBidGloas) PrevRandao() [32]byte {
return [32]byte(h.payload.PrevRandao)
}
// BlockHash returns the hash of the execution block.
func (h executionPayloadBidGloas) BlockHash() [32]byte {
return [32]byte(h.payload.BlockHash)
}
// GasLimit returns the gas limit for the execution block.
func (h executionPayloadBidGloas) GasLimit() uint64 {
return h.payload.GasLimit
}
// BuilderIndex returns the builder index of the builder who created this bid.
func (h executionPayloadBidGloas) BuilderIndex() primitives.BuilderIndex {
return h.payload.BuilderIndex
}
// Slot returns the beacon chain slot for which this bid was created.
func (h executionPayloadBidGloas) Slot() primitives.Slot {
return h.payload.Slot
}
// Value returns the payment value offered by the builder in Gwei.
func (h executionPayloadBidGloas) Value() primitives.Gwei {
return primitives.Gwei(h.payload.Value)
}
// ExecutionPayment returns the execution payment offered by the builder.
func (h executionPayloadBidGloas) ExecutionPayment() primitives.Gwei {
return primitives.Gwei(h.payload.ExecutionPayment)
}
// BlobKzgCommitmentsRoot returns the root of the KZG commitments for blobs.
func (h executionPayloadBidGloas) BlobKzgCommitmentsRoot() [32]byte {
return [32]byte(h.payload.BlobKzgCommitmentsRoot)
}
// FeeRecipient returns the execution address that will receive the builder payment.
func (h executionPayloadBidGloas) FeeRecipient() [20]byte {
return [20]byte(h.payload.FeeRecipient)
}

View File

@@ -5,7 +5,9 @@ go_library(
srcs = [
"beacon_block.go",
"error.go",
"execution_payload_envelope.go",
"light_client.go",
"signed_execution_payload_bid.go",
"utils.go",
"validator.go",
],
@@ -18,6 +20,7 @@ go_library(
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/validator-client:go_default_library",
"//runtime/version:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",

View File

@@ -0,0 +1,31 @@
package interfaces
import (
field_params "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
"github.com/ethereum/go-ethereum/common"
"google.golang.org/protobuf/proto"
)
type ROSignedExecutionPayloadEnvelope interface {
Envelope() (ROExecutionPayloadEnvelope, error)
Signature() [field_params.BLSSignatureLength]byte
SigningRoot([]byte) ([32]byte, error)
IsNil() bool
Proto() proto.Message
}
type ROExecutionPayloadEnvelope interface {
Execution() (ExecutionData, error)
ExecutionRequests() *enginev1.ExecutionRequests
BuilderIndex() primitives.BuilderIndex
BeaconBlockRoot() [field_params.RootLength]byte
BlobKzgCommitments() [][]byte
BlobKzgCommitmentsRoot() ([field_params.RootLength]byte, error)
VersionedHashes() []common.Hash
Slot() primitives.Slot
StateRoot() [field_params.RootLength]byte
IsBlinded() bool
IsNil() bool
}

View File

@@ -0,0 +1,28 @@
package interfaces
import (
field_params "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/consensus-types/primitives"
)
type ROSignedExecutionPayloadBid interface {
Bid() (ROExecutionPayloadBid, error)
Signature() [field_params.BLSSignatureLength]byte
SigningRoot([]byte) ([32]byte, error)
IsNil() bool
}
type ROExecutionPayloadBid interface {
ParentBlockHash() [32]byte
ParentBlockRoot() [32]byte
PrevRandao() [32]byte
BlockHash() [32]byte
GasLimit() uint64
BuilderIndex() primitives.BuilderIndex
Slot() primitives.Slot
Value() primitives.Gwei
ExecutionPayment() primitives.Gwei
BlobKzgCommitmentsRoot() [32]byte
FeeRecipient() [20]byte
IsNil() bool
}

View File

@@ -5,12 +5,7 @@ go_library(
srcs = ["hashtree.go"],
importpath = "github.com/OffchainLabs/prysm/v7/crypto/hash/htr",
visibility = ["//visibility:public"],
deps = [
"//config/features:go_default_library",
"@com_github_offchainlabs_hashtree//:go_default_library",
"@com_github_prysmaticlabs_gohashtree//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
deps = ["@com_github_prysmaticlabs_gohashtree//:go_default_library"],
)
go_test(
@@ -18,8 +13,5 @@ go_test(
size = "small",
srcs = ["hashtree_test.go"],
embed = [":go_default_library"],
deps = [
"//config/features:go_default_library",
"//testing/require:go_default_library",
],
deps = ["//testing/require:go_default_library"],
)

View File

@@ -4,38 +4,14 @@ import (
"runtime"
"sync"
"github.com/OffchainLabs/hashtree"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/prysmaticlabs/gohashtree"
log "github.com/sirupsen/logrus"
)
const minSliceSizeToParallelize = 5000
// Hash hashes chunks pairwise into digests using the configured hashing library.
// It performs input validation (odd chunks, digest length).
func Hash(digests, chunks [][32]byte) error {
if features.Get().EnableHashtree {
return hashtree.Hash(digests, chunks)
}
return gohashtree.Hash(digests, chunks)
}
// HashChunks hashes chunks pairwise into digests without error checking.
// The caller must ensure inputs are valid (even chunks, sufficient digest space).
func HashChunks(digests, chunks [][32]byte) {
if features.Get().EnableHashtree {
if err := hashtree.Hash(digests, chunks); err != nil {
log.WithError(err).Error("Could not hash chunks")
}
} else {
gohashtree.HashChunks(digests, chunks)
}
}
func hashParallel(inputList [][32]byte, outputList [][32]byte, wg *sync.WaitGroup) {
defer wg.Done()
err := Hash(outputList, inputList)
err := gohashtree.Hash(outputList, inputList)
if err != nil {
panic(err) // lint:nopanic -- This should never panic.
}
@@ -49,7 +25,7 @@ func hashParallel(inputList [][32]byte, outputList [][32]byte, wg *sync.WaitGrou
func VectorizedSha256(inputList [][32]byte) [][32]byte {
outputList := make([][32]byte, len(inputList)/2)
if len(inputList) < minSliceSizeToParallelize {
err := Hash(outputList, inputList)
err := gohashtree.Hash(outputList, inputList)
if err != nil {
panic(err) // lint:nopanic -- This should never panic.
}
@@ -62,7 +38,7 @@ func VectorizedSha256(inputList [][32]byte) [][32]byte {
for j := range n {
go hashParallel(inputList[j*2*groupSize:(j+1)*2*groupSize], outputList[j*groupSize:], &wg)
}
err := Hash(outputList[n*groupSize:], inputList[n*2*groupSize:])
err := gohashtree.Hash(outputList[n*groupSize:], inputList[n*2*groupSize:])
if err != nil {
panic(err) // lint:nopanic -- This should never panic.
}

View File

@@ -4,7 +4,6 @@ import (
"sync"
"testing"
"github.com/OffchainLabs/prysm/v7/config/features"
"github.com/OffchainLabs/prysm/v7/testing/require"
)
@@ -24,25 +23,3 @@ func Test_VectorizedSha256(t *testing.T) {
require.Equal(t, r, hash2[i])
}
}
func Test_VectorizedSha256_hashtree_enabled(t *testing.T) {
resetCfg := features.InitWithReset(&features.Flags{
EnableHashtree: true,
})
defer resetCfg()
largeSlice := make([][32]byte, 32*minSliceSizeToParallelize)
secondLargeSlice := make([][32]byte, 32*minSliceSizeToParallelize)
hash1 := make([][32]byte, 16*minSliceSizeToParallelize)
wg := sync.WaitGroup{}
wg.Go(func() {
tempHash := VectorizedSha256(largeSlice)
copy(hash1, tempHash)
})
wg.Wait()
hash2 := VectorizedSha256(secondLargeSlice)
require.Equal(t, len(hash1), len(hash2))
for i, r := range hash1 {
require.Equal(t, r, hash2[i])
}
}

View File

@@ -2449,15 +2449,6 @@ def prysm_deps():
sum = "h1:8feyoE3OzPrcshW5/MJ4sGESc5cqmGkGCWlco4l0bqY=",
version = "v1.4.11",
)
go_repository(
name = "com_github_offchainlabs_hashtree",
build_file_generation = "off",
importpath = "github.com/OffchainLabs/hashtree",
patch_args = ["-p1"],
patches = ["//third_party:com_github_offchainlabs_hashtree.patch"],
sum = "h1:R6DAjgAUwwfgji3jEI4WUxtZ3eJ+FbRHjW21UPMBJyo=",
version = "v0.2.2",
)
go_repository(
name = "com_github_oklog_oklog",
importpath = "github.com/oklog/oklog",

View File

@@ -21,6 +21,7 @@ go_library(
"@com_github_minio_sha256_simd//:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_prysmaticlabs_gohashtree//:go_default_library",
],
)

View File

@@ -5,6 +5,7 @@ import (
"encoding/binary"
fieldparams "github.com/OffchainLabs/prysm/v7/config/fieldparams"
"github.com/OffchainLabs/prysm/v7/crypto/hash/htr"
"github.com/OffchainLabs/prysm/v7/encoding/bytesutil"
enginev1 "github.com/OffchainLabs/prysm/v7/proto/engine/v1"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
@@ -141,3 +142,24 @@ func withdrawalRoot(w *enginev1.Withdrawal) ([32]byte, error) {
}
return w.HashTreeRoot()
}
// KzgCommitmentsRoot computes the HTR for a list of KZG commitments
func KzgCommitmentsRoot(commitments [][]byte) ([32]byte, error) {
roots := make([][32]byte, len(commitments))
for i, commitment := range commitments {
chunks, err := PackByChunk([][]byte{commitment})
if err != nil {
return [32]byte{}, err
}
roots[i] = htr.VectorizedSha256(chunks)[0]
}
commitmentsRoot, err := BitwiseMerkleize(roots, uint64(len(roots)), fieldparams.MaxBlobCommitmentsPerBlock)
if err != nil {
return [32]byte{}, errors.Wrap(err, "could not compute merkleization")
}
length := make([]byte, 32)
binary.LittleEndian.PutUint64(length[:8], uint64(len(roots)))
return MixInLength(commitmentsRoot, length), nil
}

View File

@@ -6,6 +6,7 @@ import (
"github.com/OffchainLabs/prysm/v7/container/trie"
"github.com/OffchainLabs/prysm/v7/crypto/hash/htr"
"github.com/pkg/errors"
"github.com/prysmaticlabs/gohashtree"
)
var errInvalidNilSlice = errors.New("invalid empty slice")
@@ -181,10 +182,10 @@ func MerkleizeListSSZ[T Hashable](elements []T, limit uint64) ([32]byte, error)
chunks := make([][32]byte, 2)
chunks[0] = body
binary.LittleEndian.PutUint64(chunks[1][:], uint64(len(elements)))
if err = htr.Hash(chunks, chunks); err != nil {
if err := gohashtree.Hash(chunks, chunks); err != nil {
return [32]byte{}, err
}
return chunks[0], nil
return chunks[0], err
}
// MerkleizeByteSliceSSZ hashes a byteslice by chunkifying it and returning the

1
go.mod
View File

@@ -6,7 +6,6 @@ require (
github.com/MariusVanDerWijden/FuzzyVM v0.0.0-20240516070431-7828990cad7d
github.com/MariusVanDerWijden/tx-fuzz v1.4.0
github.com/OffchainLabs/go-bitfield v0.0.0-20251031151322-f427d04d8506
github.com/OffchainLabs/hashtree v0.2.3
github.com/aristanetworks/goarista v0.0.0-20200805130819-fd197cf57d96
github.com/bazelbuild/rules_go v0.23.2
github.com/btcsuite/btcd/btcec/v2 v2.3.4

2
go.sum
View File

@@ -59,8 +59,6 @@ github.com/Microsoft/go-winio v0.6.2 h1:F2VQgta7ecxGYO8k3ZZz3RS8fVIXVxONVUPlNERo
github.com/Microsoft/go-winio v0.6.2/go.mod h1:yd8OoFMLzJbo9gZq8j5qaps8bJ9aShtEA8Ipt1oGCvU=
github.com/OffchainLabs/go-bitfield v0.0.0-20251031151322-f427d04d8506 h1:d/SJkN8/9Ca+1YmuDiUJxAiV4w/a9S8NcsG7GMQSrVI=
github.com/OffchainLabs/go-bitfield v0.0.0-20251031151322-f427d04d8506/go.mod h1:6TZI4FU6zT8x6ZfWa1J8YQ2NgW0wLV/W3fHRca8ISBo=
github.com/OffchainLabs/hashtree v0.2.3 h1:nM8dBAQZzHLzzM14FaAHXnHTAXZIst69v5xWuS48y/c=
github.com/OffchainLabs/hashtree v0.2.3/go.mod h1:b07+cRZs+eAR8TR57CB9TQlt5Gnl/06Xs76xt/1wq0M=
github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo=
github.com/Shopify/sarama v1.26.1/go.mod h1:NbSGBSSndYaIhRcBtY9V0U7AyH+x71bG668AuWys/yU=
github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI=

View File

@@ -189,3 +189,54 @@ func copyBeaconBlockBodyGloas(body *BeaconBlockBodyGloas) *BeaconBlockBodyGloas
return copied
}
// CopyBuilderPendingPaymentSlice creates a deep copy of a builder pending payment slice.
func CopyBuilderPendingPaymentSlice(original []*BuilderPendingPayment) []*BuilderPendingPayment {
if original == nil {
return nil
}
copied := make([]*BuilderPendingPayment, len(original))
for i, payment := range original {
copied[i] = CopyBuilderPendingPayment(payment)
}
return copied
}
// CopyBuilderPendingPayment creates a deep copy of a builder pending payment.
func CopyBuilderPendingPayment(original *BuilderPendingPayment) *BuilderPendingPayment {
if original == nil {
return nil
}
return &BuilderPendingPayment{
Weight: original.Weight,
Withdrawal: CopyBuilderPendingWithdrawal(original.Withdrawal),
}
}
// CopyBuilderPendingWithdrawalSlice creates a deep copy of a builder pending withdrawal slice.
func CopyBuilderPendingWithdrawalSlice(original []*BuilderPendingWithdrawal) []*BuilderPendingWithdrawal {
if original == nil {
return nil
}
copied := make([]*BuilderPendingWithdrawal, len(original))
for i, withdrawal := range original {
copied[i] = CopyBuilderPendingWithdrawal(withdrawal)
}
return copied
}
// CopyBuilderPendingWithdrawal creates a deep copy of a builder pending withdrawal.
func CopyBuilderPendingWithdrawal(original *BuilderPendingWithdrawal) *BuilderPendingWithdrawal {
if original == nil {
return nil
}
return &BuilderPendingWithdrawal{
FeeRecipient: bytesutil.SafeCopyBytes(original.FeeRecipient),
Amount: original.Amount,
BuilderIndex: original.BuilderIndex,
}
}

View File

@@ -1246,3 +1246,89 @@ func genPayloadAttestations(num int) []*v1alpha1.PayloadAttestation {
}
return pas
}
func TestCopyBuilderPendingWithdrawal(t *testing.T) {
t.Run("nil returns nil", func(t *testing.T) {
if got := v1alpha1.CopyBuilderPendingWithdrawal(nil); got != nil {
t.Fatalf("CopyBuilderPendingWithdrawal(nil) = %v, want nil", got)
}
})
t.Run("deep copy", func(t *testing.T) {
original := &v1alpha1.BuilderPendingWithdrawal{
FeeRecipient: []byte{0x01, 0x02},
Amount: primitives.Gwei(10),
BuilderIndex: primitives.BuilderIndex(3),
}
copied := v1alpha1.CopyBuilderPendingWithdrawal(original)
if copied == original {
t.Fatalf("expected new pointer for withdrawal copy")
}
if !reflect.DeepEqual(copied, original) {
t.Fatalf("CopyBuilderPendingWithdrawal() = %v, want %v", copied, original)
}
original.FeeRecipient[0] = 0xFF
original.Amount = primitives.Gwei(20)
original.BuilderIndex = primitives.BuilderIndex(9)
if copied.FeeRecipient[0] == original.FeeRecipient[0] {
t.Fatalf("fee recipient was not deep copied")
}
if copied.Amount != primitives.Gwei(10) {
t.Fatalf("amount mutated on copy: %d", copied.Amount)
}
if copied.BuilderIndex != primitives.BuilderIndex(3) {
t.Fatalf("builder index mutated on copy: %d", copied.BuilderIndex)
}
})
}
func TestCopyBuilderPendingPayment(t *testing.T) {
t.Run("nil returns nil", func(t *testing.T) {
if got := v1alpha1.CopyBuilderPendingPayment(nil); got != nil {
t.Fatalf("CopyBuilderPendingPayment(nil) = %v, want nil", got)
}
})
t.Run("deep copy", func(t *testing.T) {
original := &v1alpha1.BuilderPendingPayment{
Weight: primitives.Gwei(5),
Withdrawal: &v1alpha1.BuilderPendingWithdrawal{
FeeRecipient: []byte{0x0A, 0x0B},
Amount: primitives.Gwei(50),
BuilderIndex: primitives.BuilderIndex(7),
},
}
copied := v1alpha1.CopyBuilderPendingPayment(original)
if copied == original {
t.Fatalf("expected new pointer for payment copy")
}
if copied.Withdrawal == original.Withdrawal {
t.Fatalf("expected nested withdrawal to be deep copied")
}
if !reflect.DeepEqual(copied, original) {
t.Fatalf("CopyBuilderPendingPayment() = %v, want %v", copied, original)
}
original.Weight = primitives.Gwei(10)
original.Withdrawal.FeeRecipient[0] = 0xFF
original.Withdrawal.Amount = primitives.Gwei(75)
original.Withdrawal.BuilderIndex = primitives.BuilderIndex(9)
if copied.Weight != primitives.Gwei(5) {
t.Fatalf("weight mutated on copy: %d", copied.Weight)
}
if copied.Withdrawal.FeeRecipient[0] == original.Withdrawal.FeeRecipient[0] {
t.Fatalf("withdrawal fee recipient was not deep copied")
}
if copied.Withdrawal.Amount != primitives.Gwei(50) {
t.Fatalf("withdrawal amount mutated on copy: %d", copied.Withdrawal.Amount)
}
if copied.Withdrawal.BuilderIndex != primitives.BuilderIndex(7) {
t.Fatalf("withdrawal builder index mutated on copy: %d", copied.Withdrawal.BuilderIndex)
}
})
}

View File

@@ -6272,8 +6272,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

@@ -200,6 +200,7 @@ go_test(
"fulu__sanity__blocks_test.go",
"fulu__sanity__slots_test.go",
"fulu__ssz_static__ssz_static_test.go",
"gloas__operations__execution_payload_test.go",
"gloas__ssz_static__ssz_static_test.go",
"phase0__epoch_processing__effective_balance_updates_test.go",
"phase0__epoch_processing__epoch_processing_test.go",
@@ -278,6 +279,7 @@ go_test(
"//testing/spectest/shared/fulu/rewards:go_default_library",
"//testing/spectest/shared/fulu/sanity:go_default_library",
"//testing/spectest/shared/fulu/ssz_static:go_default_library",
"//testing/spectest/shared/gloas/operations:go_default_library",
"//testing/spectest/shared/gloas/ssz_static:go_default_library",
"//testing/spectest/shared/phase0/epoch_processing:go_default_library",
"//testing/spectest/shared/phase0/finality:go_default_library",

View File

@@ -0,0 +1,11 @@
package mainnet
import (
"testing"
"github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations"
)
func TestMainnet_Gloas_Operations_ExecutionPayloadEnvelope(t *testing.T) {
operations.RunExecutionPayloadTest(t, "mainnet")
}

View File

@@ -0,0 +1,29 @@
load("@prysm//tools/go:def.bzl", "go_library")
go_library(
name = "go_default_library",
testonly = True,
srcs = [
"execution_payload.go",
"helpers.go",
],
importpath = "github.com/OffchainLabs/prysm/v7/testing/spectest/shared/gloas/operations",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/gloas:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"//testing/spectest/utils:go_default_library",
"//testing/util:go_default_library",
"@com_github_golang_snappy//:go_default_library",
"@com_github_google_go_cmp//cmp:go_default_library",
"@io_bazel_rules_go//go/tools/bazel:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
"@org_golang_google_protobuf//testing/protocmp:go_default_library",
],
)

View File

@@ -0,0 +1,119 @@
package operations
import (
"context"
"os"
"path"
"strings"
"testing"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/gloas"
"github.com/OffchainLabs/prysm/v7/beacon-chain/core/helpers"
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
"github.com/OffchainLabs/prysm/v7/consensus-types/blocks"
"github.com/OffchainLabs/prysm/v7/consensus-types/interfaces"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
"github.com/OffchainLabs/prysm/v7/testing/require"
"github.com/OffchainLabs/prysm/v7/testing/spectest/utils"
"github.com/OffchainLabs/prysm/v7/testing/util"
"github.com/bazelbuild/rules_go/go/tools/bazel"
"github.com/golang/snappy"
"github.com/google/go-cmp/cmp"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/testing/protocmp"
)
type ExecutionConfig struct {
Valid bool `json:"execution_valid"`
}
func sszToSignedExecutionPayloadEnvelope(b []byte) (interfaces.ROSignedExecutionPayloadEnvelope, error) {
envelope := &ethpb.SignedExecutionPayloadEnvelope{}
if err := envelope.UnmarshalSSZ(b); err != nil {
return nil, err
}
return blocks.WrappedROSignedExecutionPayloadEnvelope(envelope)
}
func RunExecutionPayloadTest(t *testing.T, config string) {
require.NoError(t, utils.SetConfig(t, config))
testFolders, testsFolderPath := utils.TestFolders(t, config, "gloas", "operations/execution_payload/pyspec_tests")
if len(testFolders) == 0 {
t.Fatalf("No test folders found for %s/%s/%s", config, "gloas", "operations/execution_payload/pyspec_tests")
}
for _, folder := range testFolders {
t.Run(folder.Name(), func(t *testing.T) {
helpers.ClearCache()
// Check if signed_envelope.ssz_snappy exists, skip if not
_, err := bazel.Runfile(path.Join(testsFolderPath, folder.Name(), "signed_envelope.ssz_snappy"))
if err != nil && strings.Contains(err.Error(), "could not locate file") {
t.Skipf("Skipping test %s: signed_envelope.ssz_snappy not found", folder.Name())
return
}
// Read the signed execution payload envelope
envelopeFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "signed_envelope.ssz_snappy")
require.NoError(t, err)
envelopeSSZ, err := snappy.Decode(nil /* dst */, envelopeFile)
require.NoError(t, err, "Failed to decompress envelope")
signedEnvelope, err := sszToSignedExecutionPayloadEnvelope(envelopeSSZ)
require.NoError(t, err, "Failed to unmarshal signed envelope")
preBeaconStateFile, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "pre.ssz_snappy")
require.NoError(t, err)
preBeaconStateSSZ, err := snappy.Decode(nil /* dst */, preBeaconStateFile)
require.NoError(t, err, "Failed to decompress")
preBeaconState, err := sszToState(preBeaconStateSSZ)
require.NoError(t, err)
postSSZFilepath, err := bazel.Runfile(path.Join(testsFolderPath, folder.Name(), "post.ssz_snappy"))
postSSZExists := true
if err != nil && strings.Contains(err.Error(), "could not locate file") {
postSSZExists = false
} else {
require.NoError(t, err)
}
file, err := util.BazelFileBytes(testsFolderPath, folder.Name(), "execution.yaml")
require.NoError(t, err)
config := &ExecutionConfig{}
require.NoError(t, utils.UnmarshalYaml(file, config), "Failed to Unmarshal")
if !config.Valid {
t.Skip("Skipping invalid execution engine test as it's never supported")
}
err = gloas.ProcessExecutionPayload(context.Background(), preBeaconState, signedEnvelope)
if postSSZExists {
require.NoError(t, err)
comparePostState(t, postSSZFilepath, preBeaconState)
} else if config.Valid {
// Note: This doesn't test anything worthwhile. It essentially tests
// that *any* error has occurred, not any specific error.
if err == nil {
t.Fatal("Did not fail when expected")
}
t.Logf("Expected failure; failure reason = %v", err)
return
}
})
}
}
func comparePostState(t *testing.T, postSSZFilepath string, want state.BeaconState) {
postBeaconStateFile, err := os.ReadFile(postSSZFilepath) // #nosec G304
require.NoError(t, err)
postBeaconStateSSZ, err := snappy.Decode(nil /* dst */, postBeaconStateFile)
require.NoError(t, err, "Failed to decompress")
postBeaconState, err := sszToState(postBeaconStateSSZ)
require.NoError(t, err)
postBeaconStatePb, ok := postBeaconState.ToProtoUnsafe().(proto.Message)
require.Equal(t, true, ok, "post beacon state did not return a proto.Message")
pbState, ok := want.ToProtoUnsafe().(proto.Message)
require.Equal(t, true, ok, "beacon state did not return a proto.Message")
if !proto.Equal(postBeaconStatePb, pbState) {
diff := cmp.Diff(pbState, postBeaconStatePb, protocmp.Transform())
t.Fatalf("Post state does not match expected state, diff: %s", diff)
}
}

View File

@@ -0,0 +1,15 @@
package operations
import (
"github.com/OffchainLabs/prysm/v7/beacon-chain/state"
state_native "github.com/OffchainLabs/prysm/v7/beacon-chain/state/state-native"
ethpb "github.com/OffchainLabs/prysm/v7/proto/prysm/v1alpha1"
)
func sszToState(b []byte) (state.BeaconState, error) {
base := &ethpb.BeaconStateGloas{}
if err := base.UnmarshalSSZ(b); err != nil {
return nil, err
}
return state_native.InitializeFromProtoGloas(base)
}

View File

@@ -1,90 +0,0 @@
diff -urN a/BUILD.bazel b/BUILD.bazel
--- a/BUILD.bazel 1969-12-31 18:00:00.000000000 -0600
+++ b/BUILD.bazel 2025-01-05 12:00:00.000000000 -0600
@@ -0,0 +1,86 @@
+load("@io_bazel_rules_go//go:def.bzl", "go_library")
+
+go_library(
+ name = "go_default_library",
+ srcs = [
+ "bindings.go",
+ "sha256_1_generic.go",
+ ] + select({
+ "@io_bazel_rules_go//go/platform:linux_amd64": [
+ "bindings_amd64.go",
+ "wrapper_linux_amd64.s",
+ ":hashtree_amd64_syso",
+ ],
+ "@io_bazel_rules_go//go/platform:windows_amd64": [
+ "bindings_amd64.go",
+ "wrapper_windows_amd64.s",
+ ":hashtree_amd64_syso",
+ ],
+ "@io_bazel_rules_go//go/platform:linux_arm64": [
+ "bindings_arm64.go",
+ "wrapper_arm64.s",
+ ":hashtree_arm64_syso",
+ ],
+ "@io_bazel_rules_go//go/platform:darwin_arm64": [
+ "bindings_arm64.go",
+ "wrapper_arm64.s",
+ ":hashtree_arm64_syso",
+ ],
+ "@io_bazel_rules_go//go/platform:darwin_amd64": [
+ "bindings_darwin_amd64.go",
+ ],
+ "//conditions:default": [],
+ }),
+ importpath = "github.com/OffchainLabs/hashtree",
+ visibility = ["//visibility:public"],
+ deps = ["@com_github_klauspost_cpuid_v2//:go_default_library"],
+)
+
+genrule(
+ name = "hashtree_arm64_syso",
+ srcs = [":hashtree_arm64"],
+ outs = ["hashtree_arm64.syso"],
+ cmd = "cp $(location :hashtree_arm64) $@",
+)
+
+genrule(
+ name = "hashtree_amd64_syso",
+ srcs = [":hashtree_amd64"],
+ outs = ["hashtree_amd64.syso"],
+ cmd = "cp $(location :hashtree_amd64) $@",
+)
+
+cc_library(
+ name = "hashtree_arm64",
+ srcs = [
+ "src/hashtree.c",
+ "src/sha256_generic.c",
+ "src/sha256_armv8_crypto.S",
+ "src/sha256_armv8_neon_x1.S",
+ "src/sha256_armv8_neon_x4.S",
+ ],
+ hdrs = ["src/hashtree.h"],
+ copts = ["-O3", "-Wall"],
+ linkstatic = True,
+ strip_include_prefix = "src",
+ visibility = ["//visibility:public"],
+)
+
+cc_library(
+ name = "hashtree_amd64",
+ srcs = [
+ "src/hashtree.c",
+ "src/sha256_generic.c",
+ "src/sha256_avx_x1.S",
+ "src/sha256_avx_x4.S",
+ "src/sha256_avx_x8.S",
+ "src/sha256_avx_x16.S",
+ "src/sha256_shani.S",
+ "src/sha256_sse_x1.S",
+ ],
+ hdrs = ["src/hashtree.h"],
+ copts = ["-O3", "-Wall", "-fno-integrated-as"],
+ linkstatic = True,
+ strip_include_prefix = "src",
+ visibility = ["//visibility:public"],
+)

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")
}