Compare commits

...

33 Commits

Author SHA1 Message Date
terence tsao
8c766ac5f9 Initialize payload att message verfier in sync 2024-08-08 08:42:01 -04:00
Potuz
6833d27474 subscribe to topic 2024-08-08 09:19:47 -03:00
Potuz
deb063a304 change idx back to uint64 2024-08-08 08:06:29 -03:00
terence tsao
a46f164068 With verifier 2024-08-08 08:01:57 -03:00
Potuz
44785273c5 Payload Attestation Sync package changes 2024-08-07 09:40:18 -03:00
Potuz
ae49842311 Process Execution Payload Envelope in Chain Service (#14295)
Adds the processing of execution payload envelope
Corrects the protos for attestations and slashings in Electra versions
Adds generators of full blocks for Electra
2024-08-06 13:51:11 -03:00
Md Amaan
ffae017eb7 Indexed paylaod attestation test (#14299)
* test-added

* nil check fix

* randomized inputs

* hardcoded inputs

* suggestions applied

* minor-typo fixed

* deleted
2024-08-06 13:16:54 -03:00
JihoonSong
130508a3ba Add execution_payload and payload_attestation_message topics (#14304)
* Add `execution_payload` and `payload_attestation_message` topics

* Set `SourcePubkey` to 48 bytes long

* Add randomly populated `PayloadAttestationMessage` object

* Add tests for `execution_payload` and `payload_attestation_message` topics
2024-08-06 13:16:54 -03:00
terence
b2ad6e4163 Broadcast signed execution payload header to peer (#14300) 2024-08-06 13:16:54 -03:00
terence
27b1b7ea95 Read only payload attestation message with Verifier (#14222)
* Read only payload attestation message with verifier

* Payload attestation tests (#14242)

* Payload attestation in verification package

* Feedback #1

---------

Co-authored-by: Md Amaan <114795592+Redidacove@users.noreply.github.com>
2024-08-06 13:16:54 -03:00
Potuz
63b72e1e82 Allow nodes with and without payload in forkchoice (#14288)
* Allow nodes with and without payload in forkchoice

    This PR takes care of adding nodes to forkchoice that may or may not
    have a corresponding payload. The rationale is as follows

    - The node structure is kept almost the same as today.
    - A zero payload hash is considered as if the node was empty (except for
      the tree root)
    - When inserting a node we check what the right parent node would be
      depending on whether the parent had a payload or not.
    - For pre-epbs forks all nodes are full, no logic changes except a new
      steps to gather the parent hash that is needed for block insertion.

    This PR had to change some core consensus types and interfaces.
    - It removed the ROBlockEPBS interface and added the corresponding ePBS
      fields to the ReadOnlyBeaconBlockBody
    - It moved the setters and getters to epbs dedicated files.

    It also added a checker for `IsParentFull` on forkchoice that simply
    checks for the parent hash of the parent node.

* review
2024-08-06 13:16:54 -03:00
Potuz
ca858458c1 Use BeaconCommittees helper to get the ptc (#14286) 2024-08-06 13:16:54 -03:00
JihoonSong
4c04218c6a Add payload attestation helper functions (#14258)
* Add `IndexedPayloadAttestation` container

* Add `GetPayloadAttestingIndices` and its unit test

* Add `GetIndexedPayloadAttestation` and its unit test

* Add `is_valid_indexed_payload_attestation` and its unit test

* Create a smaller set of validators for faster unit test

* Pass context to `GetPayloadTimelinessCommittee`

* Iterate `ValidatorsReadOnly` instead of copying all validators
2024-08-06 13:16:54 -03:00
Potuz
3dd1fa3168 Use slot for latest message in forkchoice (#14279) 2024-08-06 13:16:54 -03:00
terence
24fc61ac27 Ensure epbs state getters & setters check versions (#14276)
* Ensure EPBS state getters and setters check versions

* Rename to LatestExecutionPayloadHeaderEPBS

* Add minimal beacon state
2024-08-06 13:16:54 -03:00
JihoonSong
4239f746e0 Add remove_flag and its unit test (#14260)
* Add `remove_flag` and its unit test

* Add a test case trying to remove a flag that is not set
2024-08-06 13:16:54 -03:00
JihoonSong
cc728c034b Modify get_ptc function to follow the Python spec (#14256)
* Modify `get_ptc` function to follow the Python spec

* Assign PTC members from the beginning of beacon committee array
2024-08-06 13:16:54 -03:00
Potuz
a1c69192b8 Remove inclusion list from epbs (#14188) 2024-08-06 13:16:54 -03:00
Potuz
e45f414c1f Enable validator client to submit payload attestation message (#14064) 2024-08-06 13:16:53 -03:00
Potuz
3f17a44c41 Add PTC assignment support for Duty endpoint (#14032) 2024-08-06 13:16:53 -03:00
Potuz
f5754705d5 use Keymanager() in validator client 2024-08-06 13:16:53 -03:00
Potuz
799e9a13c8 Change gwei math to primitives package for ePBS state 2024-08-06 13:16:53 -03:00
terence
de644f2a25 Fix GetPayloadTimelinessCommittee to return correct PTC size (#14012) 2024-08-06 13:16:53 -03:00
Potuz
8d0799eef0 Add ePBS to db (#13971)
* Add ePBS to db
2024-08-06 13:16:53 -03:00
Potuz
c9221c2067 Add EPBS slashing params 2024-08-06 13:16:53 -03:00
Potuz
6ac3c2ea1a Implement get_ptc
This implements a helper to get the ptc committee from a state. It uses
the cached beacon committees if possible

It also implements a helper to compute the largest power of two of a
uint64 and a helper to test for nil payload attestation messages
2024-08-06 13:16:53 -03:00
Potuz
acbd2a1cd2 Add ePBS to state (#13926) 2024-08-06 13:16:53 -03:00
Potuz
a407537d08 Add testing utility methods to return randomly populated ePBS objects 2024-08-06 13:16:53 -03:00
Potuz
390c5691d2 Add ePBS stuff to consensus-types: block 2024-08-06 13:16:53 -03:00
Potuz
d17c681682 Helper for Payload Attestation Signing (#13901) 2024-08-06 13:16:53 -03:00
Potuz
60168f249f ePBS configuration constants 2024-08-06 13:16:53 -03:00
Potuz
a03cf14be6 Add ePBS beacon state proto 2024-08-06 13:16:53 -03:00
Potuz
a1fe2f81a2 Add protos for ePBS except state 2024-08-06 13:16:53 -03:00
187 changed files with 16425 additions and 3550 deletions

View File

@@ -6,6 +6,7 @@ go_library(
"chain_info.go",
"chain_info_forkchoice.go",
"currently_syncing_block.go",
"currently_syncing_execution_payload_envelope.go",
"defragment.go",
"error.go",
"execution_engine.go",
@@ -26,6 +27,7 @@ go_library(
"receive_attestation.go",
"receive_blob.go",
"receive_block.go",
"receive_execution_payload_envelope.go",
"service.go",
"tracked_proposer.go",
"weak_subjectivity_checks.go",
@@ -44,6 +46,7 @@ go_library(
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/altair:go_default_library",
"//beacon-chain/core/blocks:go_default_library",
"//beacon-chain/core/epbs:go_default_library",
"//beacon-chain/core/epoch/precompute:go_default_library",
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
@@ -126,6 +129,7 @@ go_test(
"process_block_test.go",
"receive_attestation_test.go",
"receive_block_test.go",
"receive_execution_payload_envelope_test.go",
"service_norace_test.go",
"service_test.go",
"setup_test.go",
@@ -167,6 +171,7 @@ go_test(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/epbs:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/trie:go_default_library",

View File

@@ -530,6 +530,11 @@ func (s *Service) recoverStateSummary(ctx context.Context, blockRoot [32]byte) (
return nil, errBlockDoesNotExist
}
// PayloadBeingSynced returns whether the payload for the block with the given root is currently being synced
func (s *Service) PayloadBeingSynced(root [32]byte) bool {
return s.payloadBeingSynced.isSyncing(root)
}
// BlockBeingSynced returns whether the block with the given root is currently being synced
func (s *Service) BlockBeingSynced(root [32]byte) bool {
return s.blockBeingSynced.isSyncing(root)

View File

@@ -0,0 +1,27 @@
package blockchain
import "sync"
type currentlySyncingPayload struct {
sync.Mutex
roots map[[32]byte]struct{}
}
func (b *currentlySyncingPayload) set(root [32]byte) {
b.Lock()
defer b.Unlock()
b.roots[root] = struct{}{}
}
func (b *currentlySyncingPayload) unset(root [32]byte) {
b.Lock()
defer b.Unlock()
delete(b.roots, root)
}
func (b *currentlySyncingPayload) isSyncing(root [32]byte) bool {
b.Lock()
defer b.Unlock()
_, ok := b.roots[root]
return ok
}

View File

@@ -2,7 +2,6 @@ package blockchain
import (
"context"
"crypto/sha256"
"fmt"
"github.com/ethereum/go-ethereum/common"
@@ -28,8 +27,6 @@ import (
"go.opencensus.io/trace"
)
const blobCommitmentVersionKZG uint8 = 0x01
var defaultLatestValidHash = bytesutil.PadTo([]byte{0xff}, 32)
// notifyForkchoiceUpdate signals execution engine the fork choice updates. Execution engine should:
@@ -402,13 +399,7 @@ func kzgCommitmentsToVersionedHashes(body interfaces.ReadOnlyBeaconBlockBody) ([
versionedHashes := make([]common.Hash, len(commitments))
for i, commitment := range commitments {
versionedHashes[i] = ConvertKzgCommitmentToVersionedHash(commitment)
versionedHashes[i] = primitives.ConvertKzgCommitmentToVersionedHash(commitment)
}
return versionedHashes, nil
}
func ConvertKzgCommitmentToVersionedHash(commitment []byte) common.Hash {
versionedHash := sha256.Sum256(commitment)
versionedHash[0] = blobCommitmentVersionKZG
return versionedHash
}

View File

@@ -1056,8 +1056,8 @@ func TestService_removeInvalidBlockAndState(t *testing.T) {
require.NoError(t, service.removeInvalidBlockAndState(ctx, [][32]byte{r1, r2}))
require.Equal(t, false, service.hasBlock(ctx, r1))
require.Equal(t, false, service.hasBlock(ctx, r2))
require.Equal(t, false, service.chainHasBlock(ctx, r1))
require.Equal(t, false, service.chainHasBlock(ctx, r2))
require.Equal(t, false, service.cfg.BeaconDB.HasStateSummary(ctx, r1))
require.Equal(t, false, service.cfg.BeaconDB.HasStateSummary(ctx, r2))
has, err := service.cfg.StateGen.HasState(ctx, r1)

View File

@@ -182,6 +182,10 @@ var (
Name: "chain_service_processing_milliseconds",
Help: "Total time to call a chain service in ReceiveBlock()",
})
executionEngineProcessingTime = promauto.NewSummary(prometheus.SummaryOpts{
Name: "execution_engine_processing_milliseconds",
Help: "Total time to process an execution payload envelope in ReceiveExecutionPayloadEnvelope()",
})
dataAvailWaitedTime = promauto.NewSummary(prometheus.SummaryOpts{
Name: "da_waited_time_milliseconds",
Help: "Total time spent waiting for a data availability check in ReceiveBlock()",

View File

@@ -69,6 +69,14 @@ func WithDepositCache(c cache.DepositCache) Option {
}
}
// WithPayloadAttestationCache for payload attestation cache.
func WithPayloadAttestationCache(c *cache.PayloadAttestationCache) Option {
return func(s *Service) error {
s.cfg.PayloadAttestationCache = c
return nil
}
}
// WithPayloadIDCache for payload ID cache.
func WithPayloadIDCache(c *cache.PayloadIDCache) Option {
return func(s *Service) error {

View File

@@ -97,7 +97,7 @@ func (s *Service) OnAttestation(ctx context.Context, a ethpb.Att, disparity time
// We assume trusted attestation in this function has verified signature.
// Update forkchoice store with the new attestation for updating weight.
s.cfg.ForkChoiceStore.ProcessAttestation(ctx, indexedAtt.GetAttestingIndices(), bytesutil.ToBytes32(a.GetData().BeaconBlockRoot), a.GetData().Target.Epoch)
s.cfg.ForkChoiceStore.ProcessAttestation(ctx, indexedAtt.GetAttestingIndices(), bytesutil.ToBytes32(a.GetData().BeaconBlockRoot), a.GetData().Slot)
return nil
}

View File

@@ -375,7 +375,7 @@ func (s *Service) handleBlockAttestations(ctx context.Context, blk interfaces.Re
}
r := bytesutil.ToBytes32(a.GetData().BeaconBlockRoot)
if s.cfg.ForkChoiceStore.HasNode(r) {
s.cfg.ForkChoiceStore.ProcessAttestation(ctx, indices, r, a.GetData().Target.Epoch)
s.cfg.ForkChoiceStore.ProcessAttestation(ctx, indices, r, a.GetData().Slot)
} else if err := s.cfg.AttPool.SaveBlockAttestation(a); err != nil {
return err
}

View File

@@ -176,7 +176,7 @@ func (s *Service) processAttestations(ctx context.Context, disparity time.Durati
}
hasState := s.cfg.BeaconDB.HasStateSummary(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot))
hasBlock := s.hasBlock(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot))
hasBlock := s.chainHasBlock(ctx, bytesutil.ToBytes32(a.GetData().BeaconBlockRoot))
if !(hasState && hasBlock) {
continue
}

View File

@@ -0,0 +1,166 @@
package blockchain
import (
"context"
"fmt"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/epbs"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/das"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/execution"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
"golang.org/x/sync/errgroup"
)
// ReceiveExecutionPayloadEnvelope is a function that defines the operations (minus pubsub)
// that are performed on a received execution payload envelope. The operations consist of:
// 1. Validate the payload, apply state transition.
// 2. Apply fork choice to the processed payload
// 3. Save latest head info
func (s *Service) ReceiveExecutionPayloadEnvelope(ctx context.Context, envelope interfaces.ROExecutionPayloadEnvelope, _ das.AvailabilityStore) error {
receivedTime := time.Now()
root, err := envelope.BeaconBlockRoot()
if err != nil {
return errors.Wrap(err, "could not get beacon block root")
}
s.payloadBeingSynced.set(root)
defer s.payloadBeingSynced.unset(root)
preState, err := s.getPayloadEnvelopePrestate(ctx, envelope)
if err != nil {
return errors.Wrap(err, "could not get prestate")
}
eg, _ := errgroup.WithContext(ctx)
var postState state.BeaconState
eg.Go(func() error {
if err := epbs.ValidatePayloadStateTransition(ctx, preState, envelope); err != nil {
return errors.Wrap(err, "failed to validate consensus state transition function")
}
return nil
})
var isValidPayload bool
eg.Go(func() error {
var err error
isValidPayload, err = s.validateExecutionOnEnvelope(ctx, envelope)
if err != nil {
return errors.Wrap(err, "could not notify the engine of the new payload")
}
return nil
})
if err := eg.Wait(); err != nil {
return err
}
_ = isValidPayload
_ = postState
daStartTime := time.Now()
// TODO: Add DA check
daWaitedTime := time.Since(daStartTime)
dataAvailWaitedTime.Observe(float64(daWaitedTime.Milliseconds()))
// TODO: Add Head update, cache handling, postProcessing
timeWithoutDaWait := time.Since(receivedTime) - daWaitedTime
executionEngineProcessingTime.Observe(float64(timeWithoutDaWait.Milliseconds()))
return nil
}
// notifyNewPayload signals execution engine on a new payload.
// It returns true if the EL has returned VALID for the block
func (s *Service) notifyNewEnvelope(ctx context.Context, envelope interfaces.ROExecutionPayloadEnvelope) (bool, error) {
ctx, span := trace.StartSpan(ctx, "blockChain.notifyNewPayload")
defer span.End()
payload, err := envelope.Execution()
if err != nil {
return false, errors.Wrap(err, "could not get execution payload")
}
var lastValidHash []byte
var versionedHashes []common.Hash
versionedHashes, err = envelope.VersionedHashes()
if err != nil {
return false, errors.Wrap(err, "could not get versioned hashes to feed the engine")
}
root, err := envelope.BeaconBlockRoot()
if err != nil {
return false, errors.Wrap(err, "could not get beacon block root")
}
parentRoot, err := s.ParentRoot(root)
if err != nil {
return false, errors.Wrap(err, "could not get parent block root")
}
pr := common.Hash(parentRoot)
lastValidHash, err = s.cfg.ExecutionEngineCaller.NewPayload(ctx, payload, versionedHashes, &pr)
switch {
case err == nil:
newPayloadValidNodeCount.Inc()
return true, nil
case errors.Is(err, execution.ErrAcceptedSyncingPayloadStatus):
newPayloadOptimisticNodeCount.Inc()
log.WithFields(logrus.Fields{
"payloadBlockHash": fmt.Sprintf("%#x", bytesutil.Trunc(payload.BlockHash())),
}).Info("Called new payload with optimistic block")
return false, nil
case errors.Is(err, execution.ErrInvalidPayloadStatus):
lvh := bytesutil.ToBytes32(lastValidHash)
return false, invalidBlock{
error: ErrInvalidPayload,
lastValidHash: lvh,
}
default:
return false, errors.WithMessage(ErrUndefinedExecutionEngineError, err.Error())
}
}
// validateExecutionOnEnvelope notifies the engine of the incoming execution payload and returns true if the payload is valid
func (s *Service) validateExecutionOnEnvelope(ctx context.Context, e interfaces.ROExecutionPayloadEnvelope) (bool, error) {
isValidPayload, err := s.notifyNewEnvelope(ctx, e)
if err == nil {
return isValidPayload, nil
}
blockRoot, rootErr := e.BeaconBlockRoot()
if rootErr != nil {
return false, errors.Wrap(rootErr, "could not get beacon block root")
}
parentRoot, rootErr := s.ParentRoot(blockRoot)
if rootErr != nil {
return false, errors.Wrap(rootErr, "could not get parent block root")
}
s.cfg.ForkChoiceStore.Lock()
err = s.handleInvalidExecutionError(ctx, err, blockRoot, parentRoot)
s.cfg.ForkChoiceStore.Unlock()
return false, err
}
func (s *Service) getPayloadEnvelopePrestate(ctx context.Context, e interfaces.ROExecutionPayloadEnvelope) (state.BeaconState, error) {
ctx, span := trace.StartSpan(ctx, "blockChain.getPayloadEnvelopePreState")
defer span.End()
// Verify incoming payload has a valid pre state.
root, err := e.BeaconBlockRoot()
if err != nil {
return nil, errors.Wrap(err, "could not get beacon block root")
}
// Verify the referred block is known to forkchoice
if !s.InForkchoice(root) {
return nil, errors.New("Cannot import execution payload envelope for unknown block")
}
if err := s.verifyBlkPreState(ctx, root); err != nil {
return nil, errors.Wrap(err, "could not verify payload prestate")
}
preState, err := s.cfg.StateGen.StateByRoot(ctx, root)
if err != nil {
return nil, errors.Wrap(err, "could not get pre state")
}
if preState == nil || preState.IsNil() {
return nil, errors.Wrap(err, "nil pre state")
}
return preState, nil
}

View File

@@ -0,0 +1,104 @@
package blockchain
import (
"testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/das"
mockExecution "github.com/prysmaticlabs/prysm/v5/beacon-chain/execution/testing"
forkchoicetypes "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/types"
"github.com/prysmaticlabs/prysm/v5/consensus-types/epbs"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
)
func Test_getPayloadEnvelopePrestate(t *testing.T) {
service, tr := minimalTestService(t)
ctx, fcs := tr.ctx, tr.fcs
gs, _ := util.DeterministicGenesisStateEpbs(t, 32)
require.NoError(t, service.saveGenesisData(ctx, gs))
require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot}))
p := &enginev1.ExecutionPayloadEnvelope{
Payload: &enginev1.ExecutionPayloadElectra{},
BeaconBlockRoot: service.originBlockRoot[:],
}
e, err := epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
_, err = service.getPayloadEnvelopePrestate(ctx, e)
require.NoError(t, err)
}
func Test_notifyNewEnvelope(t *testing.T) {
service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache()))
ctx, fcs := tr.ctx, tr.fcs
gs, _ := util.DeterministicGenesisStateEpbs(t, 32)
require.NoError(t, service.saveGenesisData(ctx, gs))
require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot}))
p := &enginev1.ExecutionPayloadEnvelope{
Payload: &enginev1.ExecutionPayloadElectra{},
BeaconBlockRoot: service.originBlockRoot[:],
}
e, err := epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
engine := &mockExecution.EngineClient{}
service.cfg.ExecutionEngineCaller = engine
isValidPayload, err := service.notifyNewEnvelope(ctx, e)
require.NoError(t, err)
require.Equal(t, true, isValidPayload)
}
func Test_validateExecutionOnEnvelope(t *testing.T) {
service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache()))
ctx, fcs := tr.ctx, tr.fcs
gs, _ := util.DeterministicGenesisStateEpbs(t, 32)
require.NoError(t, service.saveGenesisData(ctx, gs))
require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot}))
p := &enginev1.ExecutionPayloadEnvelope{
Payload: &enginev1.ExecutionPayloadElectra{},
BeaconBlockRoot: service.originBlockRoot[:],
}
e, err := epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
engine := &mockExecution.EngineClient{}
service.cfg.ExecutionEngineCaller = engine
isValidPayload, err := service.validateExecutionOnEnvelope(ctx, e)
require.NoError(t, err)
require.Equal(t, true, isValidPayload)
}
func Test_ReceiveExecutionPayloadEnvelope(t *testing.T) {
service, tr := minimalTestService(t, WithPayloadIDCache(cache.NewPayloadIDCache()))
ctx, fcs := tr.ctx, tr.fcs
gs, _ := util.DeterministicGenesisStateEpbs(t, 32)
require.NoError(t, service.saveGenesisData(ctx, gs))
require.NoError(t, fcs.UpdateFinalizedCheckpoint(&forkchoicetypes.Checkpoint{Root: service.originBlockRoot}))
post := gs.Copy()
p := &enginev1.ExecutionPayloadEnvelope{
Payload: &enginev1.ExecutionPayloadElectra{
ParentHash: make([]byte, 32),
BlockHash: make([]byte, 32),
},
BeaconBlockRoot: service.originBlockRoot[:],
BlobKzgCommitments: make([][]byte, 0),
StateRoot: make([]byte, 32),
}
e, err := epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
das := &das.MockAvailabilityStore{}
blockHeader := post.LatestBlockHeader()
prevStateRoot, err := post.HashTreeRoot(ctx)
require.NoError(t, err)
blockHeader.StateRoot = prevStateRoot[:]
require.NoError(t, post.SetLatestBlockHeader(blockHeader))
stRoot, err := post.HashTreeRoot(ctx)
require.NoError(t, err)
p.StateRoot = stRoot[:]
engine := &mockExecution.EngineClient{}
service.cfg.ExecutionEngineCaller = engine
require.NoError(t, service.ReceiveExecutionPayloadEnvelope(ctx, e, das))
}

View File

@@ -65,6 +65,7 @@ type Service struct {
syncComplete chan struct{}
blobNotifiers *blobNotifierMap
blockBeingSynced *currentlySyncingBlock
payloadBeingSynced *currentlySyncingPayload
blobStorage *filesystem.BlobStorage
lastPublishedLightClientEpoch primitives.Epoch
}
@@ -75,6 +76,7 @@ type config struct {
ChainStartFetcher execution.ChainStartFetcher
BeaconDB db.HeadAccessDatabase
DepositCache cache.DepositCache
PayloadAttestationCache *cache.PayloadAttestationCache
PayloadIDCache *cache.PayloadIDCache
TrackedValidatorsCache *cache.TrackedValidatorsCache
AttPool attestations.Pool
@@ -179,6 +181,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
blobNotifiers: bn,
cfg: &config{},
blockBeingSynced: &currentlySyncingBlock{roots: make(map[[32]byte]struct{})},
payloadBeingSynced: &currentlySyncingPayload{roots: make(map[[32]byte]struct{})},
}
for _, opt := range opts {
if err := opt(srv); err != nil {
@@ -544,7 +547,7 @@ func (s *Service) saveGenesisData(ctx context.Context, genesisState state.Beacon
// 2.) Check DB.
// Checking 1.) is ten times faster than checking 2.)
// this function requires a lock in forkchoice
func (s *Service) hasBlock(ctx context.Context, root [32]byte) bool {
func (s *Service) chainHasBlock(ctx context.Context, root [32]byte) bool {
if s.cfg.ForkChoiceStore.HasNode(root) {
return true
}

View File

@@ -382,8 +382,8 @@ func TestHasBlock_ForkChoiceAndDB_DoublyLinkedTree(t *testing.T) {
require.NoError(t, err)
require.NoError(t, s.cfg.ForkChoiceStore.InsertNode(ctx, beaconState, r))
assert.Equal(t, false, s.hasBlock(ctx, [32]byte{}), "Should not have block")
assert.Equal(t, true, s.hasBlock(ctx, r), "Should have block")
assert.Equal(t, false, s.chainHasBlock(ctx, [32]byte{}), "Should not have block")
assert.Equal(t, true, s.chainHasBlock(ctx, r), "Should have block")
}
func TestServiceStop_SaveCachedBlocks(t *testing.T) {

View File

@@ -15,6 +15,7 @@ go_library(
"doc.go",
"error.go",
"interfaces.go",
"payload_attestation.go",
"payload_id.go",
"proposer_indices.go",
"proposer_indices_disabled.go", # keep
@@ -42,6 +43,7 @@ go_library(
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/slice:go_default_library",
"//crypto/bls:go_default_library",
"//crypto/hash:go_default_library",
"//crypto/rand:go_default_library",
"//encoding/bytesutil:go_default_library",
@@ -70,6 +72,7 @@ go_test(
"checkpoint_state_test.go",
"committee_fuzz_test.go",
"committee_test.go",
"payload_attestation_test.go",
"payload_id_test.go",
"private_access_test.go",
"proposer_indices_test.go",
@@ -88,6 +91,7 @@ go_test(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/assert:go_default_library",

View File

@@ -0,0 +1,116 @@
package cache
import (
"errors"
"sync"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
var errNilPayloadAttestationMessage = errors.New("nil Payload Attestation Message")
// PayloadAttestationCache keeps a map of all the PTC votes that were seen,
// already aggregated. The key is the beacon block root.
type PayloadAttestationCache struct {
root [32]byte
attestations [primitives.PAYLOAD_INVALID_STATUS]*eth.PayloadAttestation
sync.Mutex
}
// Seen returns true if a vote for the given Beacon Block Root has already been processed
// for this Payload Timeliness Committee index. This will return true even if
// the Payload status differs.
func (p *PayloadAttestationCache) Seen(root [32]byte, idx uint64) bool {
p.Lock()
defer p.Unlock()
if p.root != root {
return false
}
for _, agg := range p.attestations {
if agg == nil {
continue
}
if agg.AggregationBits.BitAt(idx) {
return true
}
}
return false
}
// messageToPayloadAttestation creates a PayloadAttestation with a single
// aggregated bit from the passed PayloadAttestationMessage
func messageToPayloadAttestation(att *eth.PayloadAttestationMessage, idx uint64) *eth.PayloadAttestation {
bits := primitives.NewPayloadAttestationAggregationBits()
bits.SetBitAt(idx, true)
data := &eth.PayloadAttestationData{
BeaconBlockRoot: bytesutil.SafeCopyBytes(att.Data.BeaconBlockRoot),
Slot: att.Data.Slot,
PayloadStatus: att.Data.PayloadStatus,
}
return &eth.PayloadAttestation{
AggregationBits: bits,
Data: data,
Signature: bytesutil.SafeCopyBytes(att.Signature),
}
}
// aggregateSigFromMessage returns the aggregated signature from a Payload
// Attestation by adding the passed signature in the PayloadAttestationMessage,
// no signature validation is performed.
func aggregateSigFromMessage(aggregated *eth.PayloadAttestation, message *eth.PayloadAttestationMessage) ([]byte, error) {
aggSig, err := bls.SignatureFromBytesNoValidation(aggregated.Signature)
if err != nil {
return nil, err
}
sig, err := bls.SignatureFromBytesNoValidation(message.Signature)
if err != nil {
return nil, err
}
return bls.AggregateSignatures([]bls.Signature{aggSig, sig}).Marshal(), nil
}
// Add adds a PayloadAttestationMessage to the internal cache of aggregated
// PayloadAttestations.
// If the index has already been seen for this attestation status the function does nothing.
// If the root is not the cached root, the function will clear the previous cache
// This function assumes that the message has already been validated. In
// particular that the signature is valid and that the block root corresponds to
// the given slot in the attestation data.
func (p *PayloadAttestationCache) Add(att *eth.PayloadAttestationMessage, idx uint64) error {
if att == nil || att.Data == nil || att.Data.BeaconBlockRoot == nil {
return errNilPayloadAttestationMessage
}
p.Lock()
defer p.Unlock()
root := [32]byte(att.Data.BeaconBlockRoot)
if p.root != root {
p.root = root
p.attestations = [primitives.PAYLOAD_INVALID_STATUS]*eth.PayloadAttestation{}
}
agg := p.attestations[att.Data.PayloadStatus]
if agg == nil {
p.attestations[att.Data.PayloadStatus] = messageToPayloadAttestation(att, idx)
return nil
}
if agg.AggregationBits.BitAt(idx) {
return nil
}
sig, err := aggregateSigFromMessage(agg, att)
if err != nil {
return err
}
agg.Signature = sig
agg.AggregationBits.SetBitAt(idx, true)
return nil
}
// Clear clears the internal map
func (p *PayloadAttestationCache) Clear() {
p.Lock()
defer p.Unlock()
p.root = [32]byte{}
p.attestations = [primitives.PAYLOAD_INVALID_STATUS]*eth.PayloadAttestation{}
}

View File

@@ -0,0 +1,95 @@
package cache
import (
"testing"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestPayloadAttestationCache(t *testing.T) {
p := &PayloadAttestationCache{}
//Test Has seen
root := [32]byte{'r'}
idx := uint64(5)
require.Equal(t, false, p.Seen(root, idx))
// Test Add
msg := &eth.PayloadAttestationMessage{
Signature: bls.NewAggregateSignature().Marshal(),
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: root[:],
Slot: 1,
PayloadStatus: primitives.PAYLOAD_PRESENT,
},
}
// Add new root
require.NoError(t, p.Add(msg, idx))
require.Equal(t, true, p.Seen(root, idx))
require.Equal(t, root, p.root)
att := p.attestations[primitives.PAYLOAD_PRESENT]
indices := att.AggregationBits.BitIndices()
require.DeepEqual(t, []int{int(idx)}, indices)
singleSig := bytesutil.SafeCopyBytes(msg.Signature)
require.DeepEqual(t, singleSig, att.Signature)
// Test Seen
require.Equal(t, true, p.Seen(root, idx))
require.Equal(t, false, p.Seen(root, idx+1))
// Add another attestation on the same data
msg2 := &eth.PayloadAttestationMessage{
Signature: bls.NewAggregateSignature().Marshal(),
Data: att.Data,
}
idx2 := uint64(7)
require.NoError(t, p.Add(msg2, idx2))
att = p.attestations[primitives.PAYLOAD_PRESENT]
indices = att.AggregationBits.BitIndices()
require.DeepEqual(t, []int{int(idx), int(idx2)}, indices)
require.DeepNotEqual(t, att.Signature, msg.Signature)
// Try again the same index
require.NoError(t, p.Add(msg2, idx2))
att2 := p.attestations[primitives.PAYLOAD_PRESENT]
indices = att.AggregationBits.BitIndices()
require.DeepEqual(t, []int{int(idx), int(idx2)}, indices)
require.DeepEqual(t, att, att2)
// Test Seen
require.Equal(t, true, p.Seen(root, idx2))
require.Equal(t, false, p.Seen(root, idx2+1))
// Add another payload status for a different payload status
msg3 := &eth.PayloadAttestationMessage{
Signature: bls.NewAggregateSignature().Marshal(),
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: root[:],
Slot: 1,
PayloadStatus: primitives.PAYLOAD_WITHHELD,
},
}
idx3 := uint64(17)
require.NoError(t, p.Add(msg3, idx3))
att3 := p.attestations[primitives.PAYLOAD_WITHHELD]
indices3 := att3.AggregationBits.BitIndices()
require.DeepEqual(t, []int{int(idx3)}, indices3)
require.DeepEqual(t, singleSig, att3.Signature)
// Add a different root
root2 := [32]byte{'s'}
msg.Data.BeaconBlockRoot = root2[:]
require.NoError(t, p.Add(msg, idx))
require.Equal(t, root2, p.root)
require.Equal(t, true, p.Seen(root2, idx))
require.Equal(t, false, p.Seen(root, idx))
att = p.attestations[primitives.PAYLOAD_PRESENT]
indices = att.AggregationBits.BitIndices()
require.DeepEqual(t, []int{int(idx)}, indices)
}

View File

@@ -12,6 +12,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
"github.com/prysmaticlabs/prysm/v5/encoding/ssz"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
@@ -217,6 +218,42 @@ func NewGenesisBlockForState(ctx context.Context, st state.BeaconState) (interfa
},
Signature: params.BeaconConfig().EmptySignature[:],
})
case *ethpb.BeaconStateEPBS:
kzgs := make([][]byte, 0)
kzgRoot, err := ssz.KzgCommitmentsRoot(kzgs)
if err != nil {
return nil, err
}
return blocks.NewSignedBeaconBlock(&ethpb.SignedBeaconBlockEpbs{
Block: &ethpb.BeaconBlockEpbs{
ParentRoot: params.BeaconConfig().ZeroHash[:],
StateRoot: root[:],
Body: &ethpb.BeaconBlockBodyEpbs{
RandaoReveal: make([]byte, 96),
Eth1Data: &ethpb.Eth1Data{
DepositRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
},
Graffiti: make([]byte, 32),
SyncAggregate: &ethpb.SyncAggregate{
SyncCommitteeBits: make([]byte, fieldparams.SyncCommitteeLength/8),
SyncCommitteeSignature: make([]byte, fieldparams.BLSSignatureLength),
},
SignedExecutionPayloadHeader: &enginev1.SignedExecutionPayloadHeader{
Message: &enginev1.ExecutionPayloadHeaderEPBS{
ParentBlockHash: make([]byte, 32),
ParentBlockRoot: make([]byte, 32),
BlockHash: make([]byte, 32),
BlobKzgCommitmentsRoot: kzgRoot[:],
},
Signature: make([]byte, 96),
},
BlsToExecutionChanges: make([]*ethpb.SignedBLSToExecutionChange, 0),
PayloadAttestations: make([]*ethpb.PayloadAttestation, 0),
},
},
Signature: params.BeaconConfig().EmptySignature[:],
})
default:
return nil, ErrUnrecognizedState
}

View File

@@ -287,12 +287,45 @@ func ProcessPayloadHeader(st state.BeaconState, header interfaces.ExecutionData)
// GetBlockPayloadHash returns the hash of the execution payload of the block
func GetBlockPayloadHash(blk interfaces.ReadOnlyBeaconBlock) ([32]byte, error) {
var payloadHash [32]byte
if IsPreBellatrixVersion(blk.Version()) {
return payloadHash, nil
if blk.Version() >= version.EPBS {
header, err := blk.Body().SignedExecutionPayloadHeader()
if err != nil {
return payloadHash, err
}
if header.Message == nil {
return payloadHash, errors.New("nil execution header")
}
return [32]byte(header.Message.BlockHash), nil
}
payload, err := blk.Body().Execution()
if err != nil {
return payloadHash, err
if blk.Version() >= version.Bellatrix {
payload, err := blk.Body().Execution()
if err != nil {
return payloadHash, err
}
return bytesutil.ToBytes32(payload.BlockHash()), nil
}
return bytesutil.ToBytes32(payload.BlockHash()), nil
return payloadHash, nil
}
// GetBlockParentHash returns the hash of the parent execution payload
func GetBlockParentHash(blk interfaces.ReadOnlyBeaconBlock) ([32]byte, error) {
var parentHash [32]byte
if blk.Version() >= version.EPBS {
header, err := blk.Body().SignedExecutionPayloadHeader()
if err != nil {
return parentHash, err
}
if header.Message == nil {
return parentHash, errors.New("nil execution header")
}
return [32]byte(header.Message.ParentBlockHash), nil
}
if blk.Version() >= version.Bellatrix {
payload, err := blk.Body().Execution()
if err != nil {
return parentHash, err
}
return bytesutil.ToBytes32(payload.ParentHash()), nil
}
return parentHash, nil
}

View File

@@ -0,0 +1,37 @@
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"attestation.go",
"execution_payload_envelope.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/epbs",
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/core/electra:go_default_library",
"//beacon-chain/state:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//proto/engine/v1:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"attestation_test.go",
"execution_payload_envelope_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/altair:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/params:go_default_library",
"//consensus-types/epbs:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
],
)

View File

@@ -0,0 +1,13 @@
package epbs
import (
"fmt"
)
// RemoveValidatorFlag removes validator flag from existing one.
func RemoveValidatorFlag(flag, flagPosition uint8) (uint8, error) {
if flagPosition > 7 {
return flag, fmt.Errorf("flag position %d exceeds length", flagPosition)
}
return flag & ^(1 << flagPosition), nil
}

View File

@@ -0,0 +1,93 @@
package epbs_test
import (
"testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/altair"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/epbs"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestValidatorFlag_Remove(t *testing.T) {
tests := []struct {
name string
add []uint8
remove []uint8
expectedTrue []uint8
expectedFalse []uint8
}{
{
name: "none",
add: []uint8{},
remove: []uint8{},
expectedTrue: []uint8{},
expectedFalse: []uint8{params.BeaconConfig().TimelySourceFlagIndex, params.BeaconConfig().TimelyTargetFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
},
{
name: "source",
add: []uint8{params.BeaconConfig().TimelySourceFlagIndex},
remove: []uint8{params.BeaconConfig().TimelySourceFlagIndex},
expectedTrue: []uint8{},
expectedFalse: []uint8{params.BeaconConfig().TimelySourceFlagIndex, params.BeaconConfig().TimelyTargetFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
},
{
name: "source, target",
add: []uint8{params.BeaconConfig().TimelySourceFlagIndex, params.BeaconConfig().TimelyTargetFlagIndex},
remove: []uint8{params.BeaconConfig().TimelySourceFlagIndex},
expectedTrue: []uint8{params.BeaconConfig().TimelyTargetFlagIndex},
expectedFalse: []uint8{params.BeaconConfig().TimelySourceFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
},
{
name: "source, target, head",
add: []uint8{params.BeaconConfig().TimelySourceFlagIndex, params.BeaconConfig().TimelyTargetFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
remove: []uint8{params.BeaconConfig().TimelyTargetFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
expectedTrue: []uint8{params.BeaconConfig().TimelySourceFlagIndex},
expectedFalse: []uint8{params.BeaconConfig().TimelyTargetFlagIndex, params.BeaconConfig().TimelyHeadFlagIndex},
},
}
var err error
for _, test := range tests {
t.Run(test.name, func(t *testing.T) {
flag := uint8(0)
// Add flags.
for _, flagPosition := range test.add {
flag, err = altair.AddValidatorFlag(flag, flagPosition)
require.NoError(t, err)
has, err := altair.HasValidatorFlag(flag, flagPosition)
require.NoError(t, err)
require.Equal(t, true, has)
}
// Remove flags.
for _, flagPosition := range test.remove {
flag, err = epbs.RemoveValidatorFlag(flag, flagPosition)
require.NoError(t, err)
}
// Check if flags are set correctly.
for _, flagPosition := range test.expectedTrue {
has, err := altair.HasValidatorFlag(flag, flagPosition)
require.NoError(t, err)
require.Equal(t, true, has)
}
for _, flagPosition := range test.expectedFalse {
has, err := altair.HasValidatorFlag(flag, flagPosition)
require.NoError(t, err)
require.Equal(t, false, has)
}
})
}
}
func TestValidatorFlag_Remove_ExceedsLength(t *testing.T) {
_, err := epbs.RemoveValidatorFlag(0, 8)
require.ErrorContains(t, "flag position 8 exceeds length", err)
}
func TestValidatorFlag_Remove_NotSet(t *testing.T) {
_, err := epbs.RemoveValidatorFlag(0, 1)
require.NoError(t, err)
}

View File

@@ -0,0 +1,138 @@
package epbs
import (
"context"
"fmt"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/electra"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
)
// ValidatePayloadStateTransition performs the process_execution_payload
// function.
func ValidatePayloadStateTransition(
ctx context.Context,
preState state.BeaconState,
envelope interfaces.ROExecutionPayloadEnvelope,
) error {
if err := validateAgainstHeader(ctx, preState, envelope); err != nil {
return err
}
committedHeader, err := preState.LatestExecutionPayloadHeaderEPBS()
if err != nil {
return err
}
if err := validateAgainstCommittedBid(committedHeader, envelope); err != nil {
return err
}
if err := processPayloadStateTransition(ctx, preState, envelope); err != nil {
return err
}
return checkPostStateRoot(ctx, preState, envelope)
}
func processPayloadStateTransition(
ctx context.Context,
preState state.BeaconState,
envelope interfaces.ROExecutionPayloadEnvelope,
) error {
payload, err := envelope.Execution()
if err != nil {
return err
}
exe, ok := payload.(interfaces.ExecutionDataElectra)
if !ok {
return errors.New("could not cast execution data to electra execution data")
}
preState, err = electra.ProcessDepositRequests(ctx, preState, exe.DepositRequests())
if err != nil {
return errors.Wrap(err, "could not process deposit receipts")
}
preState, err = electra.ProcessWithdrawalRequests(ctx, preState, exe.WithdrawalRequests())
if err != nil {
return errors.Wrap(err, "could not process execution layer withdrawal requests")
}
if err := electra.ProcessConsolidationRequests(ctx, preState, exe.ConsolidationRequests()); err != nil {
return errors.Wrap(err, "could not process consolidation requests")
}
if err := preState.SetLatestBlockHash(payload.BlockHash()); err != nil {
return err
}
return preState.SetLatestFullSlot(preState.Slot())
}
func validateAgainstHeader(
ctx context.Context,
preState state.BeaconState,
envelope interfaces.ROExecutionPayloadEnvelope,
) error {
blockHeader := preState.LatestBlockHeader()
if blockHeader == nil {
return errors.New("invalid nil latest block header")
}
if len(blockHeader.StateRoot) == 0 || [32]byte(blockHeader.StateRoot) == [32]byte{} {
prevStateRoot, err := preState.HashTreeRoot(ctx)
if err != nil {
return errors.Wrap(err, "could not compute previous state root")
}
blockHeader.StateRoot = prevStateRoot[:]
if err := preState.SetLatestBlockHeader(blockHeader); err != nil {
return errors.Wrap(err, "could not set latest block header")
}
}
blockHeaderRoot, err := blockHeader.HashTreeRoot()
if err != nil {
return err
}
beaconBlockRoot, err := envelope.BeaconBlockRoot()
if err != nil {
return err
}
if blockHeaderRoot != beaconBlockRoot {
return fmt.Errorf("beacon block root does not match previous header, got: %#x wanted: %#x", beaconBlockRoot, blockHeaderRoot)
}
return nil
}
func validateAgainstCommittedBid(
committedHeader *enginev1.ExecutionPayloadHeaderEPBS,
envelope interfaces.ROExecutionPayloadEnvelope,
) error {
builderIndex, err := envelope.BuilderIndex()
if err != nil {
return err
}
if committedHeader.BuilderIndex != builderIndex {
return errors.New("builder index does not match committed header")
}
kzgRoot, err := envelope.BlobKzgCommitmentsRoot()
if err != nil {
return err
}
if [32]byte(committedHeader.BlobKzgCommitmentsRoot) != kzgRoot {
return errors.New("blob KZG commitments root does not match committed header")
}
return nil
}
func checkPostStateRoot(
ctx context.Context,
preState state.BeaconState,
envelope interfaces.ROExecutionPayloadEnvelope,
) error {
stateRoot, err := preState.HashTreeRoot(ctx)
if err != nil {
return err
}
envelopeStateRoot, err := envelope.StateRoot()
if err != nil {
return err
}
if stateRoot != envelopeStateRoot {
return errors.New("state root mismatch")
}
return nil
}

View File

@@ -0,0 +1,105 @@
package epbs
import (
"context"
"testing"
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
consensus_epbs "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
)
func TestProcessPayloadStateTransition(t *testing.T) {
bh := [32]byte{'h'}
payload := &enginev1.ExecutionPayloadElectra{BlockHash: bh[:]}
p := &enginev1.ExecutionPayloadEnvelope{Payload: payload}
e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
stpb := &ethpb.BeaconStateEPBS{Slot: 3}
st, err := state_native.InitializeFromProtoUnsafeEpbs(stpb)
require.NoError(t, err)
ctx := context.Background()
lbh, err := st.LatestBlockHash()
require.NoError(t, err)
require.Equal(t, [32]byte{}, [32]byte(lbh))
require.NoError(t, processPayloadStateTransition(ctx, st, e))
lbh, err = st.LatestBlockHash()
require.NoError(t, err)
require.Equal(t, bh, [32]byte(lbh))
lfs, err := st.LatestFullSlot()
require.NoError(t, err)
require.Equal(t, lfs, st.Slot())
}
func Test_validateAgainstHeader(t *testing.T) {
bh := [32]byte{'h'}
payload := &enginev1.ExecutionPayloadElectra{BlockHash: bh[:]}
p := &enginev1.ExecutionPayloadEnvelope{Payload: payload}
e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
stpb := &ethpb.BeaconStateEPBS{Slot: 3}
st, err := state_native.InitializeFromProtoUnsafeEpbs(stpb)
require.NoError(t, err)
ctx := context.Background()
require.ErrorContains(t, "invalid nil latest block header", validateAgainstHeader(ctx, st, e))
prest, _ := util.DeterministicGenesisStateEpbs(t, 64)
require.ErrorContains(t, "attempted to wrap nil object", validateAgainstHeader(ctx, prest, e))
br := [32]byte{'r'}
p.BeaconBlockRoot = br[:]
require.ErrorContains(t, "beacon block root does not match previous header", validateAgainstHeader(ctx, prest, e))
header := prest.LatestBlockHeader()
require.NoError(t, err)
headerRoot, err := header.HashTreeRoot()
require.NoError(t, err)
p.BeaconBlockRoot = headerRoot[:]
require.NoError(t, validateAgainstHeader(ctx, prest, e))
}
func Test_validateAgainstCommittedBid(t *testing.T) {
payload := &enginev1.ExecutionPayloadElectra{}
p := &enginev1.ExecutionPayloadEnvelope{Payload: payload, BuilderIndex: 1}
e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
h := &enginev1.ExecutionPayloadHeaderEPBS{}
require.ErrorContains(t, "builder index does not match committed header", validateAgainstCommittedBid(h, e))
h.BuilderIndex = 1
require.ErrorContains(t, "attempted to wrap nil object", validateAgainstCommittedBid(h, e))
p.BlobKzgCommitments = make([][]byte, 6)
for i := range p.BlobKzgCommitments {
p.BlobKzgCommitments[i] = make([]byte, 48)
}
h.BlobKzgCommitmentsRoot = make([]byte, 32)
require.ErrorContains(t, "blob KZG commitments root does not match committed header", validateAgainstCommittedBid(h, e))
root, err := e.BlobKzgCommitmentsRoot()
require.NoError(t, err)
h.BlobKzgCommitmentsRoot = root[:]
require.NoError(t, validateAgainstCommittedBid(h, e))
}
func TestCheckPostStateRoot(t *testing.T) {
payload := &enginev1.ExecutionPayloadElectra{}
p := &enginev1.ExecutionPayloadEnvelope{Payload: payload, BuilderIndex: 1}
e, err := consensus_epbs.WrappedROExecutionPayloadEnvelope(p)
require.NoError(t, err)
ctx := context.Background()
st, _ := util.DeterministicGenesisStateEpbs(t, 64)
require.ErrorContains(t, "attempted to wrap nil object", checkPostStateRoot(ctx, st, e))
p.StateRoot = make([]byte, 32)
require.ErrorContains(t, "state root mismatch", checkPostStateRoot(ctx, st, e))
root, err := st.HashTreeRoot(ctx)
require.NoError(t, err)
p.StateRoot = root[:]
require.NoError(t, checkPostStateRoot(ctx, st, e))
}

View File

@@ -8,6 +8,7 @@ go_library(
"block.go",
"genesis.go",
"metrics.go",
"payload_attestation.go",
"randao.go",
"rewards_penalties.go",
"shuffle.go",
@@ -20,11 +21,13 @@ go_library(
visibility = ["//visibility:public"],
deps = [
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/forkchoice/types:go_default_library",
"//beacon-chain/state:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/epbs:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/slice:go_default_library",
@@ -53,6 +56,8 @@ go_test(
"attestation_test.go",
"beacon_committee_test.go",
"block_test.go",
"exports_test.go",
"payload_attestation_test.go",
"private_access_fuzz_noop_test.go", # keep
"private_access_test.go",
"randao_test.go",
@@ -63,26 +68,33 @@ go_test(
"validators_test.go",
"weak_subjectivity_test.go",
],
data = glob(["testdata/**"]),
embed = [":go_default_library"],
shard_count = 2,
tags = ["CI_race_detection"],
deps = [
"//beacon-chain/cache:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/forkchoice/types:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/epbs:go_default_library",
"//consensus-types/primitives:go_default_library",
"//container/slice:go_default_library",
"//crypto/bls:go_default_library",
"//crypto/hash:go_default_library",
"//crypto/rand:go_default_library",
"//encoding/bytesutil:go_default_library",
"//math:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//testing/util/random:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",

View File

@@ -213,6 +213,7 @@ type CommitteeAssignment struct {
Committee []primitives.ValidatorIndex
AttesterSlot primitives.Slot
CommitteeIndex primitives.CommitteeIndex
PtcSlot primitives.Slot
}
// verifyAssignmentEpoch verifies if the given epoch is valid for assignment based on the provided state.
@@ -294,7 +295,7 @@ func CommitteeAssignments(ctx context.Context, state state.BeaconState, epoch pr
if err := verifyAssignmentEpoch(epoch, state); err != nil {
return nil, err
}
startSlot, err := slots.EpochStart(epoch)
slot, err := slots.EpochStart(epoch)
if err != nil {
return nil, err
}
@@ -303,14 +304,17 @@ func CommitteeAssignments(ctx context.Context, state state.BeaconState, epoch pr
vals[v] = struct{}{}
}
assignments := make(map[primitives.ValidatorIndex]*CommitteeAssignment)
committees, err := BeaconCommittees(ctx, state, slot)
if err != nil {
return nil, errors.Wrap(err, "could not compute beacon committees")
}
ptcPerSlot, ptcMembersPerCommittee := PtcAllocation(len(committees))
// Compute committee assignments for each slot in the epoch.
for slot := startSlot; slot < startSlot+params.BeaconConfig().SlotsPerEpoch; slot++ {
committees, err := BeaconCommittees(ctx, state, slot)
if err != nil {
return nil, errors.Wrap(err, "could not compute beacon committees")
}
endSlot := slot + params.BeaconConfig().SlotsPerEpoch
for {
for j, committee := range committees {
for _, vIndex := range committee {
for i, vIndex := range committee {
if _, ok := vals[vIndex]; !ok { // Skip if the validator is not in the provided validators slice.
continue
}
@@ -320,8 +324,19 @@ func CommitteeAssignments(ctx context.Context, state state.BeaconState, epoch pr
assignments[vIndex].Committee = committee
assignments[vIndex].AttesterSlot = slot
assignments[vIndex].CommitteeIndex = primitives.CommitteeIndex(j)
if uint64(j) < ptcPerSlot && uint64(i) < ptcMembersPerCommittee {
assignments[vIndex].PtcSlot = slot
}
}
}
slot++
if slot == endSlot {
break
}
committees, err = BeaconCommittees(ctx, state, slot)
if err != nil {
return nil, errors.Wrap(err, "could not compute beacon committees")
}
}
return assignments, nil
}

View File

@@ -3,6 +3,7 @@ package helpers_test
import (
"context"
"fmt"
"slices"
"strconv"
"testing"
@@ -10,6 +11,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
field_params "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/container/slice"
@@ -717,15 +719,26 @@ func TestCommitteeIndices(t *testing.T) {
assert.DeepEqual(t, []primitives.CommitteeIndex{0, 1, 3}, indices)
}
func TestAttestationCommittees(t *testing.T) {
validators := make([]*ethpb.Validator, params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().TargetCommitteeSize))
func TestCommitteeAssignments_PTC(t *testing.T) {
helpers.ClearCache()
// Create 10 committees. Total 40960 validators.
committeeCount := uint64(10)
validatorCount := committeeCount * params.BeaconConfig().TargetCommitteeSize * uint64(params.BeaconConfig().SlotsPerEpoch)
validators := make([]*ethpb.Validator, validatorCount)
validatorIndices := make([]primitives.ValidatorIndex, validatorCount)
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
PublicKey: k,
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
validatorIndices[i] = primitives.ValidatorIndex(i)
}
state, err := state_native.InitializeFromProtoPhase0(&ethpb.BeaconState{
state, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: validators,
RandaoMixes: make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector),
})
@@ -749,6 +762,31 @@ func TestAttestationCommittees(t *testing.T) {
assert.Equal(t, params.BeaconConfig().TargetCommitteeSize, uint64(len(committees[0])))
assert.Equal(t, params.BeaconConfig().TargetCommitteeSize, uint64(len(committees[1])))
})
as, err := helpers.CommitteeAssignments(context.Background(), state, 1, validatorIndices)
require.NoError(t, err)
// Capture all the slots and all the validator index that belonged in a PTC using a map for verification later.
slotValidatorMap := make(map[primitives.Slot][]primitives.ValidatorIndex)
for i, a := range as {
slotValidatorMap[a.PtcSlot] = append(slotValidatorMap[a.PtcSlot], i)
}
// Verify that all the slots have the correct number of PTC.
for s, v := range slotValidatorMap {
if s == 0 {
continue
}
// Make sure all the PTC are the correct size from the map.
require.Equal(t, len(v), field_params.PTCSize)
// Get the actual PTC from the beacon state using the helper function
ptc, err := helpers.GetPayloadTimelinessCommittee(context.Background(), state, s)
require.NoError(t, err)
for _, index := range ptc {
i := slices.Index(v, index)
require.NotEqual(t, -1, i) // PTC not found from the assignment map
}
}
}
func TestBeaconCommittees(t *testing.T) {

View File

@@ -0,0 +1,12 @@
package helpers
var (
ErrNilMessage = errNilMessage
ErrNilData = errNilData
ErrNilBeaconBlockRoot = errNilBeaconBlockRoot
ErrNilPayloadAttestation = errNilPayloadAttestation
ErrNilSignature = errNilSignature
ErrNilAggregationBits = errNilAggregationBits
ErrPreEPBSState = errPreEPBSState
ErrCommitteeOverflow = errCommitteeOverflow
)

View File

@@ -0,0 +1,296 @@
package helpers
import (
"context"
"slices"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/epbs"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/math"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
var (
errNilMessage = errors.New("nil PayloadAttestationMessage")
errNilData = errors.New("nil PayloadAttestationData")
errNilBeaconBlockRoot = errors.New("nil BeaconBlockRoot")
errNilPayloadAttestation = errors.New("nil PayloadAttestation")
errNilSignature = errors.New("nil Signature")
errNilAggregationBits = errors.New("nil AggregationBits")
errPreEPBSState = errors.New("beacon state pre ePBS fork")
errCommitteeOverflow = errors.New("beacon committee of insufficient size")
)
// ValidateNilPayloadAttestationData checks if any composite field of the
// payload attestation data is nil
func ValidateNilPayloadAttestationData(data *eth.PayloadAttestationData) error {
if data == nil {
return errNilData
}
if data.BeaconBlockRoot == nil {
return errNilBeaconBlockRoot
}
return nil
}
// ValidateNilPayloadAttestationMessage checks if any composite field of the
// payload attestation message is nil
func ValidateNilPayloadAttestationMessage(att *eth.PayloadAttestationMessage) error {
if att == nil {
return errNilMessage
}
if att.Signature == nil {
return errNilSignature
}
return ValidateNilPayloadAttestationData(att.Data)
}
// ValidateNilPayloadAttestation checks if any composite field of the
// payload attestation is nil
func ValidateNilPayloadAttestation(att *eth.PayloadAttestation) error {
if att == nil {
return errNilPayloadAttestation
}
if att.AggregationBits == nil {
return errNilAggregationBits
}
if att.Signature == nil {
return errNilSignature
}
return ValidateNilPayloadAttestationData(att.Data)
}
// InPayloadTimelinessCommittee returns whether the given index belongs to the
// PTC computed from the passed state.
func InPayloadTimelinessCommittee(ctx context.Context, state state.ReadOnlyBeaconState, slot primitives.Slot, idx primitives.ValidatorIndex) (bool, error) {
ptc, err := GetPayloadTimelinessCommittee(ctx, state, slot)
if err != nil {
return false, err
}
for _, i := range ptc {
if i == idx {
return true, nil
}
}
return false, nil
}
// GetPayloadTimelinessCommittee returns the PTC for the given slot, computed from the passed state as in the
// spec function `get_ptc`.
func GetPayloadTimelinessCommittee(ctx context.Context, state state.ReadOnlyBeaconState, slot primitives.Slot) (indices []primitives.ValidatorIndex, err error) {
if state.Version() < version.EPBS {
return nil, errPreEPBSState
}
committees, err := BeaconCommittees(ctx, state, slot)
if err != nil {
return nil, errors.Wrap(err, "could not get beacon committees")
}
committeesPerSlot, membersPerCommittee := PtcAllocation(len(committees))
for i, committee := range committees {
if uint64(i) >= committeesPerSlot {
return
}
if uint64(len(committee)) < membersPerCommittee {
return nil, errCommitteeOverflow
}
indices = append(indices, committee[:membersPerCommittee]...)
}
return
}
// PtcAllocation returns:
// 1. The number of beacon committees that PTC will borrow from in a slot.
// 2. The number of validators that PTC will borrow from in a beacon committee.
func PtcAllocation(slotCommittees int) (committeesPerSlot, membersPerCommittee uint64) {
committeesPerSlot = math.LargestPowerOfTwo(math.Min(uint64(slotCommittees), fieldparams.PTCSize))
membersPerCommittee = fieldparams.PTCSize / committeesPerSlot
return
}
// GetPayloadAttestingIndices returns the set of attester indices corresponding to the given PayloadAttestation.
//
// Spec pseudocode definition:
//
// def get_payload_attesting_indices(state: BeaconState, slot: Slot,
// payload_attestation: PayloadAttestation) -> Set[ValidatorIndex]:
// """
// Return the set of attesting indices corresponding to ``payload_attestation``.
// """
// ptc = get_ptc(state, slot)
// return set(index for i, index in enumerate(ptc) if payload_attestation.aggregation_bits[i])
func GetPayloadAttestingIndices(ctx context.Context, state state.ReadOnlyBeaconState, slot primitives.Slot, att *eth.PayloadAttestation) (indices []primitives.ValidatorIndex, err error) {
if state.Version() < version.EPBS {
return nil, errPreEPBSState
}
ptc, err := GetPayloadTimelinessCommittee(ctx, state, slot)
if err != nil {
return nil, err
}
for i, validatorIndex := range ptc {
if att.AggregationBits.BitAt(uint64(i)) {
indices = append(indices, validatorIndex)
}
}
return
}
// GetIndexedPayloadAttestation replaces a PayloadAttestation's AggregationBits with sorted AttestingIndices and returns an IndexedPayloadAttestation.
//
// Spec pseudocode definition:
//
// def get_indexed_payload_attestation(state: BeaconState, slot: Slot,
// payload_attestation: PayloadAttestation) -> IndexedPayloadAttestation:
// """
// Return the indexed payload attestation corresponding to ``payload_attestation``.
// """
// attesting_indices = get_payload_attesting_indices(state, slot, payload_attestation)
//
// return IndexedPayloadAttestation(
// attesting_indices=sorted(attesting_indices),
// data=payload_attestation.data,
// signature=payload_attestation.signature,
// )
func GetIndexedPayloadAttestation(ctx context.Context, state state.ReadOnlyBeaconState, slot primitives.Slot, att *eth.PayloadAttestation) (*epbs.IndexedPayloadAttestation, error) {
if state.Version() < version.EPBS {
return nil, errPreEPBSState
}
attestingIndices, err := GetPayloadAttestingIndices(ctx, state, slot, att)
if err != nil {
return nil, err
}
slices.Sort(attestingIndices)
return &epbs.IndexedPayloadAttestation{
AttestingIndices: attestingIndices,
Data: att.Data,
Signature: att.Signature,
}, nil
}
// IsValidIndexedPayloadAttestation validates the given IndexedPayloadAttestation.
//
// Spec pseudocode definition:
//
// def is_valid_indexed_payload_attestation(
// state: BeaconState,
// indexed_payload_attestation: IndexedPayloadAttestation) -> bool:
// """
// Check if ``indexed_payload_attestation`` is not empty, has sorted and unique indices and has
// a valid aggregate signature.
// """
// # Verify the data is valid
// if indexed_payload_attestation.data.payload_status >= PAYLOAD_INVALID_STATUS:
// return False
//
// # Verify indices are sorted and unique
// indices = indexed_payload_attestation.attesting_indices
// if len(indices) == 0 or not indices == sorted(set(indices)):
// return False
//
// # Verify aggregate signature
// pubkeys = [state.validators[i].pubkey for i in indices]
// domain = get_domain(state, DOMAIN_PTC_ATTESTER, None)
// signing_root = compute_signing_root(indexed_payload_attestation.data, domain)
// return bls.FastAggregateVerify(pubkeys, signing_root, indexed_payload_attestation.signature)
func IsValidIndexedPayloadAttestation(state state.ReadOnlyBeaconState, att *epbs.IndexedPayloadAttestation) (bool, error) {
if state.Version() < version.EPBS {
return false, errPreEPBSState
}
// Verify the data is valid.
if att.Data.PayloadStatus >= primitives.PAYLOAD_INVALID_STATUS {
return false, nil
}
// Verify indices are sorted and unique.
indices := att.AttestingIndices
slices.Sort(indices)
if len(indices) == 0 || !slices.Equal(att.AttestingIndices, indices) {
return false, nil
}
// Verify aggregate signature.
publicKeys := make([]bls.PublicKey, len(indices))
for i, index := range indices {
validator, err := state.ValidatorAtIndexReadOnly(index)
if err != nil {
return false, err
}
publicKeyBytes := validator.PublicKey()
publicKey, err := bls.PublicKeyFromBytes(publicKeyBytes[:])
if err != nil {
return false, err
}
publicKeys[i] = publicKey
}
domain, err := signing.Domain(
state.Fork(),
slots.ToEpoch(state.Slot()),
params.BeaconConfig().DomainPTCAttester,
state.GenesisValidatorsRoot(),
)
if err != nil {
return false, err
}
signingRoot, err := signing.ComputeSigningRoot(att.Data, domain)
if err != nil {
return false, err
}
signature, err := bls.SignatureFromBytes(att.Signature)
if err != nil {
return false, err
}
return signature.FastAggregateVerify(publicKeys, signingRoot), nil
}
// ValidatePayloadAttestationMessageSignature verifies the signature of a
// payload attestation message.
func ValidatePayloadAttestationMessageSignature(ctx context.Context, st state.ReadOnlyBeaconState, msg *eth.PayloadAttestationMessage) error {
if err := ValidateNilPayloadAttestationMessage(msg); err != nil {
return err
}
val, err := st.ValidatorAtIndex(msg.ValidatorIndex)
if err != nil {
return err
}
pub, err := bls.PublicKeyFromBytes(val.PublicKey)
if err != nil {
return err
}
sig, err := bls.SignatureFromBytes(msg.Signature)
if err != nil {
return err
}
currentEpoch := slots.ToEpoch(st.Slot())
domain, err := signing.Domain(st.Fork(), currentEpoch, params.BeaconConfig().DomainPTCAttester, st.GenesisValidatorsRoot())
if err != nil {
return err
}
root, err := signing.ComputeSigningRoot(msg.Data, domain)
if err != nil {
return err
}
if !sig.Verify(pub, root[:]) {
return signing.ErrSigFailedToVerify
}
return nil
}

View File

@@ -0,0 +1,363 @@
package helpers_test
import (
"context"
"slices"
"strconv"
"testing"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/epbs"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/crypto/rand"
"github.com/prysmaticlabs/prysm/v5/math"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
func TestValidateNilPayloadAttestation(t *testing.T) {
require.ErrorIs(t, helpers.ErrNilData, helpers.ValidateNilPayloadAttestationData(nil))
data := &eth.PayloadAttestationData{}
require.ErrorIs(t, helpers.ErrNilBeaconBlockRoot, helpers.ValidateNilPayloadAttestationData(data))
data.BeaconBlockRoot = make([]byte, 32)
require.NoError(t, helpers.ValidateNilPayloadAttestationData(data))
require.ErrorIs(t, helpers.ErrNilMessage, helpers.ValidateNilPayloadAttestationMessage(nil))
message := &eth.PayloadAttestationMessage{}
require.ErrorIs(t, helpers.ErrNilSignature, helpers.ValidateNilPayloadAttestationMessage(message))
message.Signature = make([]byte, 96)
require.ErrorIs(t, helpers.ErrNilData, helpers.ValidateNilPayloadAttestationMessage(message))
message.Data = data
require.NoError(t, helpers.ValidateNilPayloadAttestationMessage(message))
require.ErrorIs(t, helpers.ErrNilPayloadAttestation, helpers.ValidateNilPayloadAttestation(nil))
att := &eth.PayloadAttestation{}
require.ErrorIs(t, helpers.ErrNilAggregationBits, helpers.ValidateNilPayloadAttestation(att))
att.AggregationBits = bitfield.NewBitvector512()
require.ErrorIs(t, helpers.ErrNilSignature, helpers.ValidateNilPayloadAttestation(att))
att.Signature = message.Signature
require.ErrorIs(t, helpers.ErrNilData, helpers.ValidateNilPayloadAttestation(att))
att.Data = data
require.NoError(t, helpers.ValidateNilPayloadAttestation(att))
}
func TestGetPayloadTimelinessCommittee(t *testing.T) {
helpers.ClearCache()
// Create 10 committees
committeeCount := uint64(10)
validatorCount := committeeCount * params.BeaconConfig().TargetCommitteeSize * uint64(params.BeaconConfig().SlotsPerEpoch)
validators := make([]*ethpb.Validator, validatorCount)
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
}
state, err := state_native.InitializeFromProtoEpbs(random.BeaconState(t))
require.NoError(t, err)
require.NoError(t, state.SetValidators(validators))
require.NoError(t, state.SetSlot(200))
ctx := context.Background()
indices, err := helpers.BeaconCommitteeFromState(ctx, state, state.Slot(), 1)
require.NoError(t, err)
require.Equal(t, 128, len(indices))
epoch := slots.ToEpoch(state.Slot())
activeCount, err := helpers.ActiveValidatorCount(ctx, state, epoch)
require.NoError(t, err)
require.Equal(t, uint64(40960), activeCount)
computedCommitteeCount := helpers.SlotCommitteeCount(activeCount)
require.Equal(t, committeeCount, computedCommitteeCount)
committeesPerSlot := math.LargestPowerOfTwo(math.Min(committeeCount, fieldparams.PTCSize))
require.Equal(t, uint64(8), committeesPerSlot)
ptc, err := helpers.GetPayloadTimelinessCommittee(ctx, state, state.Slot())
require.NoError(t, err)
require.Equal(t, fieldparams.PTCSize, len(ptc))
committee1, err := helpers.BeaconCommitteeFromState(ctx, state, state.Slot(), 0)
require.NoError(t, err)
require.DeepEqual(t, committee1[:64], ptc[:64])
}
func Test_PtcAllocation(t *testing.T) {
tests := []struct {
committeeCount int
memberPerCommittee uint64
committeesPerSlot uint64
}{
{1, 512, 1},
{4, 128, 4},
{128, 4, 128},
{512, 1, 512},
{1024, 1, 512},
}
for _, test := range tests {
committeesPerSlot, memberPerCommittee := helpers.PtcAllocation(test.committeeCount)
if memberPerCommittee != test.memberPerCommittee {
t.Errorf("memberPerCommittee(%d) = %d; expected %d", test.committeeCount, memberPerCommittee, test.memberPerCommittee)
}
if committeesPerSlot != test.committeesPerSlot {
t.Errorf("committeesPerSlot(%d) = %d; expected %d", test.committeeCount, committeesPerSlot, test.committeesPerSlot)
}
}
}
func TestGetPayloadAttestingIndices(t *testing.T) {
helpers.ClearCache()
// Create 10 committees. Total 40960 validators.
committeeCount := uint64(10)
validatorCount := committeeCount * params.BeaconConfig().TargetCommitteeSize * uint64(params.BeaconConfig().SlotsPerEpoch)
validators := make([]*ethpb.Validator, validatorCount)
for i := 0; i < len(validators); i++ {
pubkey := make([]byte, 48)
copy(pubkey, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: pubkey,
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
}
// Create a beacon state.
state, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: validators,
RandaoMixes: make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector),
})
require.NoError(t, err)
// Get PTC.
ptc, err := helpers.GetPayloadTimelinessCommittee(context.Background(), state, state.Slot())
require.NoError(t, err)
require.Equal(t, fieldparams.PTCSize, len(ptc))
// Generate random indices. PTC members at the corresponding indices are considered attested.
randGen := rand.NewDeterministicGenerator()
attesterCount := randGen.Intn(fieldparams.PTCSize) + 1
indices := randGen.Perm(fieldparams.PTCSize)[:attesterCount]
slices.Sort(indices)
require.Equal(t, attesterCount, len(indices))
// Create a PayloadAttestation with AggregationBits set true at the indices.
aggregationBits := bitfield.NewBitvector512()
for _, index := range indices {
aggregationBits.SetBitAt(uint64(index), true)
}
payloadAttestation := &eth.PayloadAttestation{
AggregationBits: aggregationBits,
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, 32),
},
Signature: make([]byte, 96),
}
// Get attesting indices.
attesters, err := helpers.GetPayloadAttestingIndices(context.Background(), state, state.Slot(), payloadAttestation)
require.NoError(t, err)
require.Equal(t, len(indices), len(attesters))
// Check if each attester equals to the PTC member at the corresponding index.
for i, index := range indices {
require.Equal(t, attesters[i], ptc[index])
}
}
func TestGetIndexedPayloadAttestation(t *testing.T) {
helpers.ClearCache()
// Create 10 committees. Total 40960 validators.
committeeCount := uint64(10)
validatorCount := committeeCount * params.BeaconConfig().TargetCommitteeSize * uint64(params.BeaconConfig().SlotsPerEpoch)
validators := make([]*ethpb.Validator, validatorCount)
for i := 0; i < len(validators); i++ {
publicKey := make([]byte, 48)
copy(publicKey, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: publicKey,
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
}
// Create a beacon state.
state, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: validators,
RandaoMixes: make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector),
})
require.NoError(t, err)
// Get PTC.
ptc, err := helpers.GetPayloadTimelinessCommittee(context.Background(), state, state.Slot())
require.NoError(t, err)
require.Equal(t, fieldparams.PTCSize, len(ptc))
// Generate random indices. PTC members at the corresponding indices are considered attested.
randGen := rand.NewDeterministicGenerator()
attesterCount := randGen.Intn(fieldparams.PTCSize) + 1
indices := randGen.Perm(fieldparams.PTCSize)[:attesterCount]
slices.Sort(indices)
require.Equal(t, attesterCount, len(indices))
// Create a PayloadAttestation with AggregationBits set true at the indices.
aggregationBits := bitfield.NewBitvector512()
for _, index := range indices {
aggregationBits.SetBitAt(uint64(index), true)
}
payloadAttestation := &eth.PayloadAttestation{
AggregationBits: aggregationBits,
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, 32),
},
Signature: make([]byte, 96),
}
// Get attesting indices.
ctx := context.Background()
attesters, err := helpers.GetPayloadAttestingIndices(ctx, state, state.Slot(), payloadAttestation)
require.NoError(t, err)
require.Equal(t, len(indices), len(attesters))
// Get an IndexedPayloadAttestation.
indexedPayloadAttestation, err := helpers.GetIndexedPayloadAttestation(ctx, state, state.Slot(), payloadAttestation)
require.NoError(t, err)
require.Equal(t, len(indices), len(indexedPayloadAttestation.AttestingIndices))
require.DeepEqual(t, payloadAttestation.Data, indexedPayloadAttestation.Data)
require.DeepEqual(t, payloadAttestation.Signature, indexedPayloadAttestation.Signature)
// Check if the attesting indices are the same.
slices.Sort(attesters) // GetIndexedPayloadAttestation sorts attesting indices.
require.DeepEqual(t, attesters, indexedPayloadAttestation.AttestingIndices)
}
func TestIsValidIndexedPayloadAttestation(t *testing.T) {
helpers.ClearCache()
// Create validators.
validatorCount := uint64(350)
validators := make([]*ethpb.Validator, validatorCount)
_, secretKeys, err := util.DeterministicDepositsAndKeys(validatorCount)
require.NoError(t, err)
for i := 0; i < len(validators); i++ {
validators[i] = &ethpb.Validator{
PublicKey: secretKeys[i].PublicKey().Marshal(),
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
}
// Create a beacon state.
state, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: validators,
Fork: &ethpb.Fork{
Epoch: 0,
CurrentVersion: params.BeaconConfig().GenesisForkVersion,
PreviousVersion: params.BeaconConfig().GenesisForkVersion,
},
RandaoMixes: make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector),
})
require.NoError(t, err)
// Define test cases.
tests := []struct {
attestation *epbs.IndexedPayloadAttestation
}{
{
attestation: &epbs.IndexedPayloadAttestation{
AttestingIndices: []primitives.ValidatorIndex{1},
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
},
{
attestation: &epbs.IndexedPayloadAttestation{
AttestingIndices: []primitives.ValidatorIndex{13, 19},
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
},
{
attestation: &epbs.IndexedPayloadAttestation{
AttestingIndices: []primitives.ValidatorIndex{123, 234, 345},
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
},
{
attestation: &epbs.IndexedPayloadAttestation{
AttestingIndices: []primitives.ValidatorIndex{38, 46, 54, 62, 70, 78, 86, 194},
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
},
{
attestation: &epbs.IndexedPayloadAttestation{
AttestingIndices: []primitives.ValidatorIndex{5},
Data: &eth.PayloadAttestationData{
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
},
Signature: make([]byte, fieldparams.BLSSignatureLength),
},
},
}
// Run test cases.
for _, test := range tests {
signatures := make([]bls.Signature, len(test.attestation.AttestingIndices))
for i, index := range test.attestation.AttestingIndices {
signedBytes, err := signing.ComputeDomainAndSign(
state,
slots.ToEpoch(test.attestation.Data.Slot),
test.attestation.Data,
params.BeaconConfig().DomainPTCAttester,
secretKeys[index],
)
require.NoError(t, err)
signature, err := bls.SignatureFromBytes(signedBytes)
require.NoError(t, err)
signatures[i] = signature
}
aggregatedSignature := bls.AggregateSignatures(signatures)
test.attestation.Signature = aggregatedSignature.Marshal()
isValid, err := helpers.IsValidIndexedPayloadAttestation(state, test.attestation)
require.NoError(t, err)
require.Equal(t, true, isValid)
}
}

View File

@@ -22,7 +22,7 @@ func SlashingParamsPerVersion(v int) (slashingQuotient, proposerRewardQuotient,
slashingQuotient = cfg.MinSlashingPenaltyQuotientBellatrix
proposerRewardQuotient = cfg.ProposerRewardQuotient
whistleblowerRewardQuotient = cfg.WhistleBlowerRewardQuotient
case version.Electra:
case version.Electra, version.EPBS:
slashingQuotient = cfg.MinSlashingPenaltyQuotientElectra
proposerRewardQuotient = cfg.ProposerRewardQuotient
whistleblowerRewardQuotient = cfg.WhistleBlowerRewardQuotientElectra

View File

@@ -53,6 +53,7 @@ type ReadOnlyDatabase interface {
DepositContractAddress(ctx context.Context) ([]byte, error)
// ExecutionChainData operations.
ExecutionChainData(ctx context.Context) (*ethpb.ETH1ChainData, error)
SignedBlindPayloadEnvelope(ctx context.Context, blockRoot []byte) (*ethpb.SignedBlindPayloadEnvelope, error)
// Fee recipients operations.
FeeRecipientByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (common.Address, error)
RegistrationByValidatorID(ctx context.Context, id primitives.ValidatorIndex) (*ethpb.ValidatorRegistrationV1, error)
@@ -87,6 +88,7 @@ type NoHeadAccessDatabase interface {
SaveDepositContractAddress(ctx context.Context, addr common.Address) error
// SaveExecutionChainData operations.
SaveExecutionChainData(ctx context.Context, data *ethpb.ETH1ChainData) error
SaveBlindPayloadEnvelope(ctx context.Context, envelope *ethpb.SignedBlindPayloadEnvelope) error
// Run any required database migrations.
RunMigrations(ctx context.Context) error
// Fee recipients operations.

View File

@@ -6,6 +6,7 @@ go_library(
"archived_point.go",
"backfill.go",
"backup.go",
"blind_payload_envelope.go",
"blocks.go",
"checkpoint.go",
"deposit_contract.go",
@@ -78,6 +79,7 @@ go_test(
"archived_point_test.go",
"backfill_test.go",
"backup_test.go",
"blind_payload_envelope_test.go",
"blocks_test.go",
"checkpoint_test.go",
"deposit_contract_test.go",
@@ -119,6 +121,7 @@ go_test(
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//testing/util/random:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_golang_snappy//:go_default_library",
"@com_github_pkg_errors//:go_default_library",

View File

@@ -0,0 +1,45 @@
package kv
import (
"context"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
bolt "go.etcd.io/bbolt"
"go.opencensus.io/trace"
)
// SaveBlindPayloadEnvelope saves a signed execution payload envelope blind in the database.
func (s *Store) SaveBlindPayloadEnvelope(ctx context.Context, env *ethpb.SignedBlindPayloadEnvelope) error {
ctx, span := trace.StartSpan(ctx, "BeaconDB.SaveBlindPayloadEnvelope")
defer span.End()
enc, err := encode(ctx, env)
if err != nil {
return err
}
r := env.Message.BeaconBlockRoot
err = s.db.Update(func(tx *bolt.Tx) error {
bucket := tx.Bucket(executionPayloadEnvelopeBucket)
return bucket.Put(r, enc)
})
return err
}
// SignedBlindPayloadEnvelope retrieves a signed execution payload envelope blind from the database.
func (s *Store) SignedBlindPayloadEnvelope(ctx context.Context, blockRoot []byte) (*ethpb.SignedBlindPayloadEnvelope, error) {
ctx, span := trace.StartSpan(ctx, "BeaconDB.SignedBlindPayloadEnvelope")
defer span.End()
env := &ethpb.SignedBlindPayloadEnvelope{}
err := s.db.View(func(tx *bolt.Tx) error {
bkt := tx.Bucket(executionPayloadEnvelopeBucket)
enc := bkt.Get(blockRoot)
if enc == nil {
return ErrNotFound
}
return decode(ctx, enc, env)
})
return env, err
}

View File

@@ -0,0 +1,23 @@
package kv
import (
"context"
"testing"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
)
func TestStore_SignedBlindPayloadEnvelope(t *testing.T) {
db := setupDB(t)
ctx := context.Background()
_, err := db.SignedBlindPayloadEnvelope(ctx, []byte("test"))
require.ErrorIs(t, err, ErrNotFound)
env := random.SignedBlindPayloadEnvelope(t)
err = db.SaveBlindPayloadEnvelope(ctx, env)
require.NoError(t, err)
got, err := db.SignedBlindPayloadEnvelope(ctx, env.Message.BeaconBlockRoot)
require.NoError(t, err)
require.DeepEqual(t, got, env)
}

View File

@@ -823,6 +823,11 @@ func unmarshalBlock(_ context.Context, enc []byte) (interfaces.ReadOnlySignedBea
if err := rawBlock.UnmarshalSSZ(enc[len(electraBlindKey):]); err != nil {
return nil, errors.Wrap(err, "could not unmarshal blinded Electra block")
}
case hasEpbsKey(enc):
rawBlock = &ethpb.SignedBeaconBlockEpbs{}
if err := rawBlock.UnmarshalSSZ(enc[len(epbsKey):]); err != nil {
return nil, errors.Wrap(err, "could not unmarshal EPBS block")
}
default:
// Marshal block bytes to phase 0 beacon block.
rawBlock = &ethpb.SignedBeaconBlock{}
@@ -852,6 +857,8 @@ func encodeBlock(blk interfaces.ReadOnlySignedBeaconBlock) ([]byte, error) {
func keyForBlock(blk interfaces.ReadOnlySignedBeaconBlock) ([]byte, error) {
switch blk.Version() {
case version.EPBS:
return epbsKey, nil
case version.Electra:
if blk.IsBlinded() {
return electraBlindKey, nil

View File

@@ -18,6 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
"google.golang.org/protobuf/proto"
)
@@ -146,6 +147,17 @@ var blockTests = []struct {
}
return blocks.NewSignedBeaconBlock(b)
}},
{
name: "epbs",
newBlock: func(slot primitives.Slot, root []byte) (interfaces.ReadOnlySignedBeaconBlock, error) {
b := random.SignedBeaconBlock(&testing.T{})
b.Block.Slot = slot
if root != nil {
b.Block.ParentRoot = root
}
return blocks.NewSignedBeaconBlock(b)
},
},
}
func TestStore_SaveBlock_NoDuplicates(t *testing.T) {
@@ -202,7 +214,7 @@ func TestStore_BlocksCRUD(t *testing.T) {
retrievedBlock, err = db.Block(ctx, blockRoot)
require.NoError(t, err)
wanted := retrievedBlock
if retrievedBlock.Version() >= version.Bellatrix {
if retrievedBlock.Version() >= version.Bellatrix && retrievedBlock.Version() < version.EPBS {
wanted, err = retrievedBlock.ToBlinded()
require.NoError(t, err)
}
@@ -390,7 +402,7 @@ func TestStore_BlocksCRUD_NoCache(t *testing.T) {
require.NoError(t, err)
wanted := blk
if blk.Version() >= version.Bellatrix {
if blk.Version() >= version.Bellatrix && blk.Version() < version.EPBS {
wanted, err = blk.ToBlinded()
require.NoError(t, err)
}
@@ -609,7 +621,7 @@ func TestStore_SaveBlock_CanGetHighestAt(t *testing.T) {
b, err := db.Block(ctx, root)
require.NoError(t, err)
wanted := block1
if block1.Version() >= version.Bellatrix {
if block1.Version() >= version.Bellatrix && block1.Version() < version.EPBS {
wanted, err = wanted.ToBlinded()
require.NoError(t, err)
}
@@ -627,7 +639,7 @@ func TestStore_SaveBlock_CanGetHighestAt(t *testing.T) {
b, err = db.Block(ctx, root)
require.NoError(t, err)
wanted2 := block2
if block2.Version() >= version.Bellatrix {
if block2.Version() >= version.Bellatrix && block2.Version() < version.EPBS {
wanted2, err = block2.ToBlinded()
require.NoError(t, err)
}
@@ -645,7 +657,7 @@ func TestStore_SaveBlock_CanGetHighestAt(t *testing.T) {
b, err = db.Block(ctx, root)
require.NoError(t, err)
wanted = block3
if block3.Version() >= version.Bellatrix {
if block3.Version() >= version.Bellatrix && block3.Version() < version.EPBS {
wanted, err = wanted.ToBlinded()
require.NoError(t, err)
}
@@ -681,7 +693,7 @@ func TestStore_GenesisBlock_CanGetHighestAt(t *testing.T) {
b, err := db.Block(ctx, root)
require.NoError(t, err)
wanted := block1
if block1.Version() >= version.Bellatrix {
if block1.Version() >= version.Bellatrix && block1.Version() < version.EPBS {
wanted, err = block1.ToBlinded()
require.NoError(t, err)
}
@@ -698,7 +710,7 @@ func TestStore_GenesisBlock_CanGetHighestAt(t *testing.T) {
b, err = db.Block(ctx, root)
require.NoError(t, err)
wanted = genesisBlock
if genesisBlock.Version() >= version.Bellatrix {
if genesisBlock.Version() >= version.Bellatrix && genesisBlock.Version() < version.EPBS {
wanted, err = genesisBlock.ToBlinded()
require.NoError(t, err)
}
@@ -715,7 +727,7 @@ func TestStore_GenesisBlock_CanGetHighestAt(t *testing.T) {
b, err = db.Block(ctx, root)
require.NoError(t, err)
wanted = genesisBlock
if genesisBlock.Version() >= version.Bellatrix {
if genesisBlock.Version() >= version.Bellatrix && genesisBlock.Version() < version.EPBS {
wanted, err = genesisBlock.ToBlinded()
require.NoError(t, err)
}
@@ -811,7 +823,7 @@ func TestStore_BlocksBySlot_BlockRootsBySlot(t *testing.T) {
require.NoError(t, err)
wanted := b1
if b1.Version() >= version.Bellatrix {
if b1.Version() >= version.Bellatrix && b1.Version() < version.EPBS {
wanted, err = b1.ToBlinded()
require.NoError(t, err)
}
@@ -827,7 +839,7 @@ func TestStore_BlocksBySlot_BlockRootsBySlot(t *testing.T) {
t.Fatalf("Expected 2 blocks, received %d blocks", len(retrievedBlocks))
}
wanted = b2
if b2.Version() >= version.Bellatrix {
if b2.Version() >= version.Bellatrix && b2.Version() < version.EPBS {
wanted, err = b2.ToBlinded()
require.NoError(t, err)
}
@@ -837,7 +849,7 @@ func TestStore_BlocksBySlot_BlockRootsBySlot(t *testing.T) {
require.NoError(t, err)
assert.Equal(t, true, proto.Equal(wantedPb, retrieved0Pb), "Wanted: %v, received: %v", retrievedBlocks[0], wanted)
wanted = b3
if b3.Version() >= version.Bellatrix {
if b3.Version() >= version.Bellatrix && b3.Version() < version.EPBS {
wanted, err = b3.ToBlinded()
require.NoError(t, err)
}

View File

@@ -78,6 +78,8 @@ func isSSZStorageFormat(obj interface{}) bool {
return true
case *ethpb.VoluntaryExit:
return true
case *ethpb.SignedBlindPayloadEnvelope:
return true
case *ethpb.ValidatorRegistrationV1:
return true
default:

View File

@@ -65,3 +65,10 @@ func hasElectraBlindKey(enc []byte) bool {
}
return bytes.Equal(enc[:len(electraBlindKey)], electraBlindKey)
}
func hasEpbsKey(enc []byte) bool {
if len(epbsKey) >= len(enc) {
return false
}
return bytes.Equal(enc[:len(epbsKey)], epbsKey)
}

View File

@@ -118,6 +118,9 @@ var Buckets = [][]byte{
feeRecipientBucket,
registrationBucket,
// ePBS
executionPayloadEnvelopeBucket,
}
// KVStoreOption is a functional option that modifies a kv.Store.

View File

@@ -7,15 +7,16 @@ package kv
// it easy to scan for keys that have a certain shard number as a prefix and return those
// corresponding attestations.
var (
blocksBucket = []byte("blocks")
stateBucket = []byte("state")
stateSummaryBucket = []byte("state-summary")
chainMetadataBucket = []byte("chain-metadata")
checkpointBucket = []byte("check-point")
powchainBucket = []byte("powchain")
stateValidatorsBucket = []byte("state-validators")
feeRecipientBucket = []byte("fee-recipient")
registrationBucket = []byte("registration")
blocksBucket = []byte("blocks")
stateBucket = []byte("state")
stateSummaryBucket = []byte("state-summary")
chainMetadataBucket = []byte("chain-metadata")
checkpointBucket = []byte("check-point")
powchainBucket = []byte("powchain")
stateValidatorsBucket = []byte("state-validators")
feeRecipientBucket = []byte("fee-recipient")
registrationBucket = []byte("registration")
executionPayloadEnvelopeBucket = []byte("execution-payload-envelope")
// Deprecated: This bucket was migrated in PR 6461. Do not use, except for migrations.
slotsHasObjectBucket = []byte("slots-has-objects")
@@ -50,6 +51,7 @@ var (
denebBlindKey = []byte("blind-deneb")
electraKey = []byte("electra")
electraBlindKey = []byte("blind-electra")
epbsKey = []byte("epbs")
// block root included in the beacon state used by weak subjectivity initial sync
originCheckpointBlockRootKey = []byte("origin-checkpoint-block-root")

View File

@@ -252,6 +252,10 @@ func (s *Store) saveStatesEfficientInternal(ctx context.Context, tx *bolt.Tx, bl
if err := s.processElectra(ctx, rawType, rt[:], bucket, valIdxBkt, validatorKeys[i]); err != nil {
return err
}
case *ethpb.BeaconStateEPBS:
if err := s.processEPBS(ctx, rawType, rt[:], bucket, valIdxBkt, validatorKeys[i]); err != nil {
return err
}
default:
return errors.New("invalid state type")
}
@@ -367,6 +371,24 @@ func (s *Store) processElectra(ctx context.Context, pbState *ethpb.BeaconStateEl
return nil
}
func (s *Store) processEPBS(ctx context.Context, pbState *ethpb.BeaconStateEPBS, rootHash []byte, bucket, valIdxBkt *bolt.Bucket, validatorKey []byte) error {
valEntries := pbState.Validators
pbState.Validators = make([]*ethpb.Validator, 0)
rawObj, err := pbState.MarshalSSZ()
if err != nil {
return err
}
encodedState := snappy.Encode(nil, append(epbsKey, rawObj...))
if err := bucket.Put(rootHash, encodedState); err != nil {
return err
}
pbState.Validators = valEntries
if err := valIdxBkt.Put(rootHash, validatorKey); err != nil {
return err
}
return nil
}
func (s *Store) storeValidatorEntriesSeparately(ctx context.Context, tx *bolt.Tx, validatorsEntries map[string]*ethpb.Validator) error {
valBkt := tx.Bucket(stateValidatorsBucket)
for hashStr, validatorEntry := range validatorsEntries {
@@ -516,6 +538,19 @@ func (s *Store) unmarshalState(_ context.Context, enc []byte, validatorEntries [
}
switch {
case hasEpbsKey(enc):
protoState := &ethpb.BeaconStateEPBS{}
if err := protoState.UnmarshalSSZ(enc[len(epbsKey):]); err != nil {
return nil, errors.Wrap(err, "failed to unmarshal encoding for EPBS")
}
ok, err := s.isStateValidatorMigrationOver()
if err != nil {
return nil, err
}
if ok {
protoState.Validators = validatorEntries
}
return statenative.InitializeFromProtoEpbs(protoState)
case hasElectraKey(enc):
protoState := &ethpb.BeaconStateElectra{}
if err := protoState.UnmarshalSSZ(enc[len(electraKey):]); err != nil {
@@ -675,6 +710,19 @@ func marshalState(ctx context.Context, st state.ReadOnlyBeaconState) ([]byte, er
return nil, err
}
return snappy.Encode(nil, append(electraKey, rawObj...)), nil
case *ethpb.BeaconStateEPBS:
rState, ok := st.ToProtoUnsafe().(*ethpb.BeaconStateEPBS)
if !ok {
return nil, errors.New("non valid inner state")
}
if rState == nil {
return nil, errors.New("nil state")
}
rawObj, err := rState.MarshalSSZ()
if err != nil {
return nil, err
}
return snappy.Encode(nil, append(epbsKey, rawObj...)), nil
default:
return nil, errors.New("invalid inner state")
}

View File

@@ -9,6 +9,7 @@ import (
"time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
statenative "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v5/config/features"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
@@ -20,6 +21,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
bolt "go.etcd.io/bbolt"
)
@@ -159,6 +161,16 @@ func TestState_CanSaveRetrieve(t *testing.T) {
},
rootSeed: 'E',
},
{
name: "epbs",
s: func() state.BeaconState {
stPb := random.BeaconState(t)
st, err := statenative.InitializeFromProtoUnsafeEpbs(stPb)
require.NoError(t, err)
return st
},
rootSeed: 'F',
},
}
db := setupDB(t)
@@ -1113,6 +1125,26 @@ func TestDenebState_CanDelete(t *testing.T) {
require.Equal(t, state.ReadOnlyBeaconState(nil), savedS, "Unsaved state should've been nil")
}
func TestEpbsState_CanDelete(t *testing.T) {
db := setupDB(t)
r := [32]byte{'A'}
require.Equal(t, false, db.HasState(context.Background(), r))
s := random.BeaconState(t)
st, err := statenative.InitializeFromProtoUnsafeEpbs(s)
require.NoError(t, err)
require.NoError(t, db.SaveState(context.Background(), st, r))
require.Equal(t, true, db.HasState(context.Background(), r))
require.NoError(t, db.DeleteState(context.Background(), r))
savedS, err := db.State(context.Background(), r)
require.NoError(t, err)
require.Equal(t, state.ReadOnlyBeaconState(nil), savedS, "Unsaved state should've been nil")
}
func TestStateDeneb_CanSaveRetrieveValidatorEntries(t *testing.T) {
db := setupDB(t)

View File

@@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"doc.go",
"epbs.go",
"errors.go",
"forkchoice.go",
"last_root.go",
@@ -47,6 +48,7 @@ go_library(
go_test(
name = "go_default_test",
srcs = [
"epbs_test.go",
"ffg_update_test.go",
"forkchoice_test.go",
"last_root_test.go",

View File

@@ -0,0 +1,9 @@
package doublylinkedtree
func (n *Node) isParentFull() bool {
// Finalized checkpoint is considered full
if n.parent == nil || n.parent.parent == nil {
return true
}
return n.parent.payloadHash != [32]byte{}
}

View File

@@ -0,0 +1,79 @@
package doublylinkedtree
import (
"context"
"testing"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestStore_Insert_PayloadContent(t *testing.T) {
ctx := context.Background()
f := setup(0, 0)
s := f.store
// The tree root is full
fr := [32]byte{}
n := s.nodeByRoot[fr]
require.Equal(t, true, n.isParentFull())
// Insert a child with a payload
cr := [32]byte{'a'}
cp := [32]byte{'p'}
n, err := s.insert(ctx, 1, cr, fr, cp, fr, 0, 0)
require.NoError(t, err)
require.Equal(t, true, n.isParentFull())
require.Equal(t, s.treeRootNode, n.parent)
require.Equal(t, s.nodeByRoot[cr], n)
// Insert a grandchild without a payload
gr := [32]byte{'b'}
gn, err := s.insert(ctx, 2, gr, cr, fr, cp, 0, 0)
require.NoError(t, err)
require.Equal(t, true, gn.isParentFull())
require.Equal(t, n, gn.parent)
// Insert the payload of the same grandchild
gp := [32]byte{'q'}
gfn, err := s.insert(ctx, 2, gr, cr, gp, cp, 0, 0)
require.NoError(t, err)
require.Equal(t, true, gfn.isParentFull())
require.Equal(t, n, gfn.parent)
// Insert an empty great grandchild based on empty
ggr := [32]byte{'c'}
ggn, err := s.insert(ctx, 3, ggr, gr, fr, cp, 0, 0)
require.NoError(t, err)
require.Equal(t, false, ggn.isParentFull())
require.Equal(t, gn, ggn.parent)
// Insert an empty great grandchild based on full
ggfr := [32]byte{'d'}
ggfn, err := s.insert(ctx, 3, ggfr, gr, fr, gp, 0, 0)
require.NoError(t, err)
require.Equal(t, gfn, ggfn.parent)
require.Equal(t, true, ggfn.isParentFull())
// Insert the payload for the great grandchild based on empty
ggp := [32]byte{'r'}
n, err = s.insert(ctx, 3, ggr, gr, ggp, cp, 0, 0)
require.NoError(t, err)
require.Equal(t, false, n.isParentFull())
require.Equal(t, gn, n.parent)
// Insert the payload for the great grandchild based on full
ggfp := [32]byte{'s'}
n, err = s.insert(ctx, 3, ggfr, gr, ggfp, gp, 0, 0)
require.NoError(t, err)
require.Equal(t, true, n.isParentFull())
require.Equal(t, gfn, n.parent)
// Reinsert an empty node
ggfn2, err := s.insert(ctx, 3, ggfr, gr, fr, gp, 0, 0)
require.NoError(t, err)
require.Equal(t, ggfn, ggfn2)
// Reinsert a full node
n2, err := s.insert(ctx, 3, ggfr, gr, ggfp, gp, 0, 0)
require.NoError(t, err)
require.Equal(t, n, n2)
}

View File

@@ -80,7 +80,7 @@ func (f *ForkChoice) Head(
// ProcessAttestation processes attestation for vote accounting, it iterates around validator indices
// and update their votes accordingly.
func (f *ForkChoice) ProcessAttestation(ctx context.Context, validatorIndices []uint64, blockRoot [32]byte, targetEpoch primitives.Epoch) {
func (f *ForkChoice) ProcessAttestation(ctx context.Context, validatorIndices []uint64, blockRoot [32]byte, attSlot primitives.Slot) {
_, span := trace.StartSpan(ctx, "doublyLinkedForkchoice.ProcessAttestation")
defer span.End()
@@ -94,9 +94,9 @@ func (f *ForkChoice) ProcessAttestation(ctx context.Context, validatorIndices []
newVote := f.votes[index].nextRoot == params.BeaconConfig().ZeroHash &&
f.votes[index].currentRoot == params.BeaconConfig().ZeroHash
// Vote gets updated if it's newly allocated or high target epoch.
if newVote || targetEpoch > f.votes[index].nextEpoch {
f.votes[index].nextEpoch = targetEpoch
// Vote gets updated if it's newly allocated or higher attestation slot.
if newVote || attSlot > f.votes[index].slot {
f.votes[index].slot = attSlot
f.votes[index].nextRoot = blockRoot
}
}
@@ -115,8 +115,27 @@ func (f *ForkChoice) InsertNode(ctx context.Context, state state.BeaconState, ro
return errNilBlockHeader
}
parentRoot := bytesutil.ToBytes32(bh.ParentRoot)
var payloadHash [32]byte
if state.Version() >= version.Bellatrix {
var payloadHash, parentHash [32]byte
if state.Version() >= version.EPBS {
slot, err := state.LatestFullSlot()
if err != nil {
return err
}
if slot == state.Slot() {
latestHeader, err := state.LatestExecutionPayloadHeaderEPBS()
if err != nil {
return err
}
copy(payloadHash[:], latestHeader.BlockHash)
copy(parentHash[:], latestHeader.ParentBlockHash)
} else {
latestHash, err := state.LatestBlockHash()
if err != nil {
return err
}
copy(parentHash[:], latestHash)
}
} else if state.Version() >= version.Bellatrix {
ph, err := state.LatestExecutionPayloadHeader()
if err != nil {
return err
@@ -135,7 +154,7 @@ func (f *ForkChoice) InsertNode(ctx context.Context, state state.BeaconState, ro
return errInvalidNilCheckpoint
}
finalizedEpoch := fc.Epoch
node, err := f.store.insert(ctx, slot, root, parentRoot, payloadHash, justifiedEpoch, finalizedEpoch)
node, err := f.store.insert(ctx, slot, root, parentRoot, payloadHash, parentHash, justifiedEpoch, finalizedEpoch)
if err != nil {
return err
}
@@ -490,8 +509,12 @@ func (f *ForkChoice) InsertChain(ctx context.Context, chain []*forkchoicetypes.B
if err != nil {
return err
}
parentHash, err := blocks.GetBlockParentHash(b)
if err != nil {
return err
}
if _, err := f.store.insert(ctx,
b.Slot(), r, parentRoot, payloadHash,
b.Slot(), r, parentRoot, payloadHash, parentHash,
chain[i].JustifiedCheckpoint.Epoch, chain[i].FinalizedCheckpoint.Epoch); err != nil {
return err
}

View File

@@ -22,7 +22,8 @@ import (
)
// prepareForkchoiceState prepares a beacon State with the given data to mock
// insert into forkchoice
// insert into forkchoice. This method prepares full states and blocks for
// bellatrix, it cannot be used for ePBS tests.
func prepareForkchoiceState(
_ context.Context,
slot primitives.Slot,

View File

@@ -9,6 +9,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
// Helper function to simulate the block being on time or delayed for proposer
@@ -61,7 +62,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
)
require.NoError(t, err)
require.NoError(t, f.InsertNode(ctx, state, blkRoot))
f.ProcessAttestation(ctx, []uint64{0}, newRoot, fEpoch)
fSlot, err := slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, []uint64{0}, newRoot, fSlot)
headRoot, err = f.Head(ctx)
require.NoError(t, err)
assert.Equal(t, newRoot, headRoot, "Incorrect head for justified epoch at slot 1")
@@ -87,7 +90,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
)
require.NoError(t, err)
require.NoError(t, f.InsertNode(ctx, state, blkRoot))
f.ProcessAttestation(ctx, []uint64{1}, newRoot, fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, []uint64{1}, newRoot, fSlot)
headRoot, err = f.Head(ctx)
require.NoError(t, err)
assert.Equal(t, newRoot, headRoot, "Incorrect head for justified epoch at slot 2")
@@ -115,7 +120,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
)
require.NoError(t, err)
require.NoError(t, f.InsertNode(ctx, state, blkRoot))
f.ProcessAttestation(ctx, []uint64{2}, newRoot, fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, []uint64{2}, newRoot, fSlot)
headRoot, err = f.Head(ctx)
require.NoError(t, err)
assert.Equal(t, newRoot, headRoot, "Incorrect head for justified epoch at slot 3")
@@ -144,7 +151,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
)
require.NoError(t, err)
require.NoError(t, f.InsertNode(ctx, state, blkRoot))
f.ProcessAttestation(ctx, []uint64{3}, newRoot, fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, []uint64{3}, newRoot, fSlot)
headRoot, err = f.Head(ctx)
require.NoError(t, err)
assert.Equal(t, newRoot, headRoot, "Incorrect head for justified epoch at slot 3")
@@ -174,7 +183,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
// Regression: process attestations for C, check that it
// becomes head, we need two attestations to have C.weight = 30 > 24 = D.weight
f.ProcessAttestation(ctx, []uint64{4, 5}, indexToHash(3), fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, []uint64{4, 5}, indexToHash(3), fSlot)
headRoot, err = f.Head(ctx)
require.NoError(t, err)
assert.Equal(t, indexToHash(3), headRoot, "Incorrect head for justified epoch at slot 4")
@@ -235,10 +246,14 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
// The maliciously withheld block has one vote.
votes := []uint64{1}
f.ProcessAttestation(ctx, votes, maliciouslyWithheldBlock, fEpoch)
fSlot, err := slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, votes, maliciouslyWithheldBlock, fSlot)
// The honest block has one vote.
votes = []uint64{2}
f.ProcessAttestation(ctx, votes, honestBlock, fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, votes, honestBlock, fSlot)
// Ensure the head is STILL C, the honest block, as the honest block had proposer boost.
r, err = f.Head(ctx)
@@ -304,7 +319,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
// An attestation is received for B that has more voting power than C with the proposer boost,
// allowing B to then become the head if their attestation has enough adversarial votes.
votes := []uint64{1, 2}
f.ProcessAttestation(ctx, votes, maliciouslyWithheldBlock, fEpoch)
fSlot, err := slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, votes, maliciouslyWithheldBlock, fSlot)
// Expect the head to have switched to B.
r, err = f.Head(ctx)
@@ -379,7 +396,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
// An attestation for C is received at slot N+3.
votes := []uint64{1}
f.ProcessAttestation(ctx, votes, c, fEpoch)
fSlot, err := slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, votes, c, fSlot)
// A block D, building on B, is received at slot N+3. It should not be able to win without boosting.
dSlot := primitives.Slot(3)
@@ -419,7 +438,9 @@ func TestForkChoice_BoostProposerRoot_PreventsExAnteAttack(t *testing.T) {
require.NoError(t, f.InsertNode(ctx, state, blkRoot))
votes = []uint64{2}
f.ProcessAttestation(ctx, votes, d2, fEpoch)
fSlot, err = slots.EpochStart(fEpoch)
require.NoError(t, err)
f.ProcessAttestation(ctx, votes, d2, fSlot)
// Ensure D becomes the head thanks to boosting.
r, err = f.Head(ctx)
require.NoError(t, err)

View File

@@ -62,20 +62,32 @@ func (s *Store) head(ctx context.Context) ([32]byte, error) {
// insert registers a new block node to the fork choice store's node list.
// It then updates the new node's parent with the best child and descendant node.
func (s *Store) insert(ctx context.Context,
func (s *Store) insert(
ctx context.Context,
slot primitives.Slot,
root, parentRoot, payloadHash [fieldparams.RootLength]byte,
justifiedEpoch, finalizedEpoch primitives.Epoch) (*Node, error) {
root, parentRoot, payloadHash, parentHash [fieldparams.RootLength]byte,
justifiedEpoch, finalizedEpoch primitives.Epoch,
) (*Node, error) {
ctx, span := trace.StartSpan(ctx, "doublyLinkedForkchoice.insert")
defer span.End()
// Return if the block has been inserted into Store before.
if n, ok := s.nodeByRoot[root]; ok {
return n, nil
n, rootPresent := s.nodeByRoot[root]
m, hashPresent := s.nodeByPayload[payloadHash]
if rootPresent {
if payloadHash == [32]byte{} {
return n, nil
}
if hashPresent {
return m, nil
}
}
parent := s.nodeByRoot[parentRoot]
n := &Node{
fullParent := s.nodeByPayload[parentHash]
if fullParent != nil && parent != nil && fullParent.root == parent.root {
parent = fullParent
}
n = &Node{
slot: slot,
root: root,
parent: parent,
@@ -99,17 +111,22 @@ func (s *Store) insert(ctx context.Context,
}
}
s.nodeByPayload[payloadHash] = n
s.nodeByRoot[root] = n
if parent == nil {
if s.treeRootNode == nil {
s.treeRootNode = n
s.headNode = n
s.highestReceivedNode = n
} else {
} else if s.treeRootNode.root != n.root {
return n, errInvalidParentRoot
}
} else {
}
if !rootPresent {
s.nodeByRoot[root] = n
}
if !hashPresent {
s.nodeByPayload[payloadHash] = n
}
if parent != nil {
parent.children = append(parent.children, n)
// Apply proposer boost
timeNow := uint64(time.Now().Unix())

View File

@@ -133,7 +133,7 @@ func TestStore_Insert(t *testing.T) {
fc := &forkchoicetypes.Checkpoint{Epoch: 0}
s := &Store{nodeByRoot: nodeByRoot, treeRootNode: treeRootNode, nodeByPayload: nodeByPayload, justifiedCheckpoint: jc, finalizedCheckpoint: fc, highestReceivedNode: &Node{}}
payloadHash := [32]byte{'a'}
_, err := s.insert(context.Background(), 100, indexToHash(100), indexToHash(0), payloadHash, 1, 1)
_, err := s.insert(context.Background(), 100, indexToHash(100), indexToHash(0), payloadHash, payloadHash, 1, 1)
require.NoError(t, err)
assert.Equal(t, 2, len(s.nodeByRoot), "Did not insert block")
assert.Equal(t, (*Node)(nil), treeRootNode.parent, "Incorrect parent")
@@ -327,7 +327,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// Make sure it doesn't underflow
s.genesisTime = uint64(time.Now().Add(time.Duration(-1*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
_, err := s.insert(context.Background(), 1, [32]byte{'a'}, b, b, 1, 1)
_, err := s.insert(context.Background(), 1, [32]byte{'a'}, b, b, b, 1, 1)
require.NoError(t, err)
count, err := f.ReceivedBlocksLastEpoch()
require.NoError(t, err)
@@ -337,7 +337,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 64
// Received block last epoch is 1
_, err = s.insert(context.Background(), 64, [32]byte{'A'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 64, [32]byte{'A'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration((-64*int64(params.BeaconConfig().SecondsPerSlot))-1) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -348,7 +348,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 64 65
// Received block last epoch is 2
_, err = s.insert(context.Background(), 65, [32]byte{'B'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 65, [32]byte{'B'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-66*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -359,7 +359,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 64 65 66
// Received block last epoch is 3
_, err = s.insert(context.Background(), 66, [32]byte{'C'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 66, [32]byte{'C'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-66*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -370,7 +370,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 64 65 66
// 98
// Received block last epoch is 1
_, err = s.insert(context.Background(), 98, [32]byte{'D'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 98, [32]byte{'D'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-98*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -382,7 +382,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 98
// 132
// Received block last epoch is 1
_, err = s.insert(context.Background(), 132, [32]byte{'E'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 132, [32]byte{'E'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-132*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -395,7 +395,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 132
// 99
// Received block last epoch is still 1. 99 is outside the window
_, err = s.insert(context.Background(), 99, [32]byte{'F'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 99, [32]byte{'F'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-132*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -408,7 +408,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 132
// 99 100
// Received block last epoch is still 1. 100 is at the same position as 132
_, err = s.insert(context.Background(), 100, [32]byte{'G'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 100, [32]byte{'G'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-132*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()
@@ -421,7 +421,7 @@ func TestForkChoice_ReceivedBlocksLastEpoch(t *testing.T) {
// 132
// 99 100 101
// Received block last epoch is 2. 101 is within the window
_, err = s.insert(context.Background(), 101, [32]byte{'H'}, b, b, 1, 1)
_, err = s.insert(context.Background(), 101, [32]byte{'H'}, b, b, b, 1, 1)
require.NoError(t, err)
s.genesisTime = uint64(time.Now().Add(time.Duration(-132*int64(params.BeaconConfig().SecondsPerSlot)) * time.Second).Unix())
count, err = f.ReceivedBlocksLastEpoch()

View File

@@ -67,5 +67,5 @@ type Node struct {
type Vote struct {
currentRoot [fieldparams.RootLength]byte // current voting root.
nextRoot [fieldparams.RootLength]byte // next voting root.
nextEpoch primitives.Epoch // epoch of next voting period.
slot primitives.Slot // slot of the last vote by this validator
}

View File

@@ -47,7 +47,7 @@ type BlockProcessor interface {
// AttestationProcessor processes the attestation that's used for accounting fork choice.
type AttestationProcessor interface {
ProcessAttestation(context.Context, []uint64, [32]byte, primitives.Epoch)
ProcessAttestation(context.Context, []uint64, [32]byte, primitives.Slot)
}
// Getter returns fork choice related information.

View File

@@ -102,6 +102,7 @@ type BeaconNode struct {
blsToExecPool blstoexec.PoolManager
depositCache cache.DepositCache
trackedValidatorsCache *cache.TrackedValidatorsCache
payloadAttestationCache *cache.PayloadAttestationCache
payloadIDCache *cache.PayloadIDCache
stateFeed *event.Feed
blockFeed *event.Feed
@@ -152,6 +153,7 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
syncCommitteePool: synccommittee.NewPool(),
blsToExecPool: blstoexec.NewPool(),
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
payloadAttestationCache: &cache.PayloadAttestationCache{},
payloadIDCache: cache.NewPayloadIDCache(),
slasherBlockHeadersFeed: new(event.Feed),
slasherAttestationsFeed: new(event.Feed),
@@ -775,6 +777,7 @@ func (b *BeaconNode) registerBlockchainService(fc forkchoice.ForkChoicer, gs *st
blockchain.WithSyncComplete(syncComplete),
blockchain.WithBlobStorage(b.BlobStorage),
blockchain.WithTrackedValidatorsCache(b.trackedValidatorsCache),
blockchain.WithPayloadAttestationCache(b.payloadAttestationCache),
blockchain.WithPayloadIDCache(b.payloadIDCache),
blockchain.WithSyncChecker(b.syncChecker),
)
@@ -852,6 +855,7 @@ func (b *BeaconNode) registerSyncService(initialSyncComplete chan struct{}, bFil
regularsync.WithStateGen(b.stateGen),
regularsync.WithSlasherAttestationsFeed(b.slasherAttestationsFeed),
regularsync.WithSlasherBlockHeadersFeed(b.slasherBlockHeadersFeed),
regularsync.WithPayloadAttestationCache(b.payloadAttestationCache),
regularsync.WithPayloadReconstructor(web3Service),
regularsync.WithClockWaiter(b.clockWaiter),
regularsync.WithInitialSyncComplete(initialSyncComplete),

View File

@@ -30,6 +30,7 @@ go_library(
"service.go",
"subnets.go",
"topics.go",
"topics_epbs.go",
"utils.go",
"watch_peers.go",
],
@@ -68,6 +69,7 @@ go_library(
"//monitoring/tracing:go_default_library",
"//network:go_default_library",
"//network/forks:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/metadata:go_default_library",
"//runtime:go_default_library",
@@ -159,6 +161,7 @@ go_test(
"//encoding/bytesutil:go_default_library",
"//network:go_default_library",
"//network/forks:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/eth/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/testing:go_default_library",
@@ -166,6 +169,7 @@ go_test(
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//testing/util/random:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//crypto:go_default_library",
@@ -186,9 +190,11 @@ go_test(
"@com_github_libp2p_go_libp2p_pubsub//pb:go_default_library",
"@com_github_multiformats_go_multiaddr//:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prysmaticlabs_fastssz//:go_default_library",
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
"@org_golang_google_protobuf//proto:go_default_library",
"@org_golang_google_protobuf//reflect/protoreflect:go_default_library",
],
)

View File

@@ -12,6 +12,7 @@ import (
"github.com/ethereum/go-ethereum/p2p/discover"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/host"
ssz "github.com/prysmaticlabs/fastssz"
"github.com/prysmaticlabs/go-bitfield"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/peers"
@@ -25,7 +26,9 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
"google.golang.org/protobuf/proto"
"google.golang.org/protobuf/reflect/protoreflect"
)
func TestService_Broadcast(t *testing.T) {
@@ -520,3 +523,80 @@ func TestService_BroadcastBlob(t *testing.T) {
require.NoError(t, p.BroadcastBlob(ctx, subnet, blobSidecar))
require.Equal(t, false, util.WaitTimeout(&wg, 1*time.Second), "Failed to receive pubsub within 1s")
}
func TestService_BroadcastExecutionPayloadHeader(t *testing.T) {
msg := random.SignedExecutionPayloadHeader(t)
testBroadcast(t, SignedExecutionPayloadHeaderTopicFormat, msg)
}
func TestService_BroadcastExecutionPayloadEnvelope(t *testing.T) {
msg := random.SignedExecutionPayloadEnvelope(t)
testBroadcast(t, SignedExecutionPayloadEnvelopeTopicFormat, msg)
}
func TestService_BroadcastPayloadAttestationMessage(t *testing.T) {
msg := random.PayloadAttestationMessage(t)
testBroadcast(t, PayloadAttestationMessageTopicFormat, msg)
}
func testBroadcast(t *testing.T, topicFormat string, msg interface{}) {
// Create two peers and let them connect.
p1 := p2ptest.NewTestP2P(t)
p2 := p2ptest.NewTestP2P(t)
p1.Connect(p2)
if len(p1.BHost.Network().Peers()) == 0 {
t.Fatal("No peers")
}
// Create a `Service` for the first peer.
s1 := &Service{
host: p1.BHost,
pubsub: p1.PubSub(),
joinedTopics: map[string]*pubsub.Topic{},
cfg: &Config{},
genesisTime: time.Now(),
genesisValidatorsRoot: bytesutil.PadTo([]byte{'A'}, 32),
peers: peers.NewStatus(context.Background(), &peers.StatusConfig{
ScorerParams: &scorers.Config{},
}),
}
// The second peer subscribes to the topic.
digest, err := s1.currentForkDigest()
require.NoError(t, err)
topic := fmt.Sprintf(topicFormat, digest) + s1.Encoding().ProtocolSuffix()
subscription, err := p2.SubscribeToTopic(topic)
require.NoError(t, err)
time.Sleep(50 * time.Millisecond) // Wait for libp2p to be set up.
// Start a goroutine listening for a pubsub message.
var wg sync.WaitGroup
wg.Add(1)
go func() {
defer wg.Done()
ctx, cancel := context.WithTimeout(context.Background(), 1*time.Second)
defer cancel()
incomingMessage, err := subscription.Next(ctx)
require.NoError(t, err)
result, ok := msg.(ssz.Unmarshaler)
require.Equal(t, true, ok)
require.NoError(t, s1.Encoding().DecodeGossip(incomingMessage.Data, result))
require.DeepEqual(t, result, msg)
}()
// An attempt to broadcast a message unmapped to a topic should fail.
ctx := context.Background()
require.ErrorContains(t, "message type is not mapped to a PubSub topic", s1.Broadcast(ctx, nil))
// The first peer broadcasts the message to the second peer.
require.NoError(t, s1.Broadcast(ctx, msg.(protoreflect.ProtoMessage)))
// Wait for one second for the message to be delivered and processed.
if util.WaitTimeout(&wg, 1*time.Second) {
t.Error("Failed to receive pubsub within 1s")
}
}

View File

@@ -5,6 +5,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"google.golang.org/protobuf/proto"
)
@@ -22,6 +23,9 @@ var gossipTopicMappings = map[string]func() proto.Message{
SyncCommitteeSubnetTopicFormat: func() proto.Message { return &ethpb.SyncCommitteeMessage{} },
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return &ethpb.SignedBLSToExecutionChange{} },
BlobSubnetTopicFormat: func() proto.Message { return &ethpb.BlobSidecar{} },
SignedExecutionPayloadHeaderTopicFormat: func() proto.Message { return &enginev1.SignedExecutionPayloadHeader{} },
SignedExecutionPayloadEnvelopeTopicFormat: func() proto.Message { return &enginev1.SignedExecutionPayloadEnvelope{} },
PayloadAttestationMessageTopicFormat: func() proto.Message { return &ethpb.PayloadAttestationMessage{} },
}
// GossipTopicMappings is a function to return the assigned data type
@@ -104,4 +108,8 @@ func init() {
GossipTypeMapping[reflect.TypeOf(&ethpb.AttestationElectra{})] = AttestationSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.AttesterSlashingElectra{})] = AttesterSlashingSubnetTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.SignedAggregateAttestationAndProofElectra{})] = AggregateAndProofSubnetTopicFormat
// Handle ePBS objects.
GossipTypeMapping[reflect.TypeOf(&enginev1.SignedExecutionPayloadHeader{})] = SignedExecutionPayloadHeaderTopicFormat
GossipTypeMapping[reflect.TypeOf(&enginev1.SignedExecutionPayloadEnvelope{})] = SignedExecutionPayloadEnvelopeTopicFormat
GossipTypeMapping[reflect.TypeOf(&ethpb.PayloadAttestationMessage{})] = PayloadAttestationMessageTopicFormat
}

View File

@@ -7,8 +7,10 @@ import (
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/assert"
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
func TestMappingHasNoDuplicates(t *testing.T) {
@@ -30,17 +32,20 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
capellaForkEpoch := primitives.Epoch(300)
denebForkEpoch := primitives.Epoch(400)
electraForkEpoch := primitives.Epoch(500)
epbsForkEpoch := primitives.Epoch(600)
bCfg.AltairForkEpoch = altairForkEpoch
bCfg.BellatrixForkEpoch = bellatrixForkEpoch
bCfg.CapellaForkEpoch = capellaForkEpoch
bCfg.DenebForkEpoch = denebForkEpoch
bCfg.ElectraForkEpoch = electraForkEpoch
bCfg.EPBSForkEpoch = epbsForkEpoch
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.AltairForkVersion)] = primitives.Epoch(100)
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.BellatrixForkVersion)] = primitives.Epoch(200)
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.CapellaForkVersion)] = primitives.Epoch(300)
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.DenebForkVersion)] = primitives.Epoch(400)
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.ElectraForkVersion)] = primitives.Epoch(500)
bCfg.ForkVersionSchedule[bytesutil.ToBytes4(bCfg.EPBSForkVersion)] = primitives.Epoch(600)
params.OverrideBeaconConfig(bCfg)
// Phase 0
@@ -126,4 +131,15 @@ func TestGossipTopicMappings_CorrectType(t *testing.T) {
pMessage = GossipTopicMappings(AggregateAndProofSubnetTopicFormat, electraForkEpoch)
_, ok = pMessage.(*ethpb.SignedAggregateAttestationAndProofElectra)
assert.Equal(t, true, ok)
// Epbs fork
pMessage = GossipTopicMappings(SignedExecutionPayloadHeaderTopicFormat, epbsForkEpoch)
_, ok = pMessage.(*enginev1.SignedExecutionPayloadHeader)
require.Equal(t, true, ok)
pMessage = GossipTopicMappings(SignedExecutionPayloadEnvelopeTopicFormat, epbsForkEpoch)
_, ok = pMessage.(*enginev1.SignedExecutionPayloadEnvelope)
require.Equal(t, true, ok)
pMessage = GossipTopicMappings(PayloadAttestationMessageTopicFormat, epbsForkEpoch)
_, ok = pMessage.(*ethpb.PayloadAttestationMessage)
require.Equal(t, true, ok)
}

View File

@@ -0,0 +1,11 @@
package p2p
const (
GossipSignedExecutionPayloadHeader = "signed_execution_payload_header"
GossipSignedExecutionPayloadEnvelope = "signed_execution_payload_envelope"
GossipPayloadAttestationMessage = "payload_attestation_message"
SignedExecutionPayloadHeaderTopicFormat = GossipProtocolAndDigest + GossipSignedExecutionPayloadHeader
SignedExecutionPayloadEnvelopeTopicFormat = GossipProtocolAndDigest + GossipSignedExecutionPayloadEnvelope
PayloadAttestationMessageTopicFormat = GossipProtocolAndDigest + GossipPayloadAttestationMessage
)

View File

@@ -20,6 +20,10 @@ import (
"github.com/prysmaticlabs/prysm/v5/testing/require"
)
// Variables defined in the placeholderFields will not be tested in `TestGetSpec`.
// These are variables that we don't use in Prysm. (i.e. future hardfork, light client... etc)
var placeholderFields = []string{"DOMAIN_BEACON_BUILDER", "DOMAIN_PTC_ATTESTER"}
func TestGetDepositContract(t *testing.T) {
params.SetupTestConfigCleanup(t)
config := params.BeaconConfig().Copy()
@@ -192,7 +196,7 @@ func TestGetSpec(t *testing.T) {
data, ok := resp.Data.(map[string]interface{})
require.Equal(t, true, ok)
assert.Equal(t, 155, len(data))
assert.Equal(t, 157, len(data))
for k, v := range data {
t.Run(k, func(t *testing.T) {
switch k {
@@ -530,6 +534,12 @@ func TestGetSpec(t *testing.T) {
case "MAX_DEPOSIT_REQUESTS_PER_PAYLOAD":
assert.Equal(t, "93", v)
default:
for _, pf := range placeholderFields {
if k == pf {
t.Logf("Skipping placeholder field: %s", k)
return
}
}
t.Errorf("Incorrect key: %s", k)
}
})

View File

@@ -20,6 +20,7 @@ go_library(
"//beacon-chain/core/time:go_default_library",
"//beacon-chain/core/transition:go_default_library",
"//config/params:go_default_library",
"//consensus-types/primitives:go_default_library",
"//network/httputil:go_default_library",
"//proto/eth/v1:go_default_library",
"//proto/eth/v2:go_default_library",

View File

@@ -11,7 +11,6 @@ import (
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/api"
"github.com/prysmaticlabs/prysm/v5/api/server/structs"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
@@ -19,6 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/transition"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/network/httputil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/eth/v1"
ethpbv2 "github.com/prysmaticlabs/prysm/v5/proto/eth/v2"
@@ -217,7 +217,7 @@ func handleBlockOperationEvents(w http.ResponseWriter, flusher http.Flusher, req
if !ok {
return write(w, flusher, topicDataMismatch, event.Data, BlobSidecarTopic)
}
versionedHash := blockchain.ConvertKzgCommitmentToVersionedHash(blobData.Blob.KzgCommitment)
versionedHash := primitives.ConvertKzgCommitmentToVersionedHash(blobData.Blob.KzgCommitment)
blobEvent := &structs.BlobSidecarEvent{
BlockRoot: hexutil.Encode(blobData.Blob.BlockRootSlice()),
Index: fmt.Sprintf("%d", blobData.Blob.Index),

View File

@@ -25,6 +25,7 @@ go_library(
"proposer_exits.go",
"proposer_slashings.go",
"proposer_sync_aggregate.go",
"ptc_attester.go",
"server.go",
"status.go",
"sync_committee.go",

View File

@@ -108,6 +108,7 @@ func (vs *Server) duties(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.
assignment.Committee = ca.Committee
assignment.AttesterSlot = ca.AttesterSlot
assignment.CommitteeIndex = ca.CommitteeIndex
assignment.PtcSlot = ca.PtcSlot
}
// Save the next epoch assignments.
ca, ok = nextEpochAssignments[idx]
@@ -115,6 +116,7 @@ func (vs *Server) duties(ctx context.Context, req *ethpb.DutiesRequest) (*ethpb.
nextAssignment.Committee = ca.Committee
nextAssignment.AttesterSlot = ca.AttesterSlot
nextAssignment.CommitteeIndex = ca.CommitteeIndex
nextAssignment.PtcSlot = ca.PtcSlot
}
} else {
// If the validator isn't in the beacon state, try finding their deposit to determine their status.

View File

@@ -0,0 +1,17 @@
package validator
import (
"context"
"github.com/golang/protobuf/ptypes/empty"
"github.com/pkg/errors"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
func (vs *Server) GetPayloadAttestationData(ctx context.Context, req *ethpb.GetPayloadAttestationDataRequest) (*ethpb.PayloadAttestationData, error) {
return nil, errors.New("not implemented")
}
func (vs *Server) SubmitPayloadAttestation(ctx context.Context, in *ethpb.PayloadAttestationMessage) (*empty.Empty, error) {
return nil, errors.New("not implemented")
}

View File

@@ -5,6 +5,7 @@ go_library(
srcs = [
"error.go",
"interfaces.go",
"interfaces_epbs.go",
"prometheus.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/state",

View File

@@ -60,6 +60,7 @@ type ReadOnlyBeaconState interface {
ReadOnlySyncCommittee
ReadOnlyDeposits
ReadOnlyConsolidations
ReadOnlyEpbsFields
ToProtoUnsafe() interface{}
ToProto() interface{}
GenesisTime() uint64
@@ -94,6 +95,7 @@ type WriteOnlyBeaconState interface {
WriteOnlyConsolidations
WriteOnlyWithdrawals
WriteOnlyDeposits
WriteOnlyEpbsFields
SetGenesisTime(val uint64) error
SetGenesisValidatorsRoot(val []byte) error
SetSlot(val primitives.Slot) error

View File

@@ -0,0 +1,21 @@
package state
import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
)
type ReadOnlyEpbsFields interface {
IsParentBlockFull() (bool, error)
LatestExecutionPayloadHeaderEPBS() (*enginev1.ExecutionPayloadHeaderEPBS, error)
LatestBlockHash() ([]byte, error)
LatestFullSlot() (primitives.Slot, error)
LastWithdrawalsRoot() ([]byte, error)
}
type WriteOnlyEpbsFields interface {
SetLatestExecutionPayloadHeaderEPBS(val *enginev1.ExecutionPayloadHeaderEPBS) error
SetLatestBlockHash(val []byte) error
SetLatestFullSlot(val primitives.Slot) error
SetLastWithdrawalsRoot(val []byte) error
}

View File

@@ -13,11 +13,13 @@ go_library(
"getters_checkpoint.go",
"getters_consolidation.go",
"getters_deposit_requests.go",
"getters_epbs.go",
"getters_eth1.go",
"getters_exit.go",
"getters_misc.go",
"getters_participation.go",
"getters_payload_header.go",
"getters_payload_header_epbs.go",
"getters_randao.go",
"getters_state.go",
"getters_sync_committee.go",
@@ -34,6 +36,7 @@ go_library(
"setters_churn.go",
"setters_consolidation.go",
"setters_deposit_requests.go",
"setters_epbs.go",
"setters_eth1.go",
"setters_misc.go",
"setters_participation.go",
@@ -46,6 +49,7 @@ go_library(
"spec_parameters.go",
"ssz.go",
"state_trie.go",
"state_trie_epbs.go",
"types.go",
"validator_index_cache.go",
],
@@ -98,6 +102,7 @@ go_test(
"getters_deposit_requests_test.go",
"getters_exit_test.go",
"getters_participation_test.go",
"getters_setters_epbs_test.go",
"getters_test.go",
"getters_validator_test.go",
"getters_withdrawal_test.go",
@@ -119,6 +124,7 @@ go_test(
"setters_withdrawal_test.go",
"state_fuzz_test.go",
"state_test.go",
"state_trie_epbs_test.go",
"state_trie_test.go",
"types_test.go",
"validator_index_cache_test.go",
@@ -152,6 +158,7 @@ go_test(
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//testing/util/random:go_default_library",
"//time/slots:go_default_library",
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
"@com_github_golang_snappy//:go_default_library",

View File

@@ -60,6 +60,11 @@ type BeaconState struct {
latestExecutionPayloadHeaderElectra *enginev1.ExecutionPayloadHeaderElectra
nextWithdrawalIndex uint64
nextWithdrawalValidatorIndex primitives.ValidatorIndex
// ePBS fields
latestBlockHash [32]byte
latestFullSlot primitives.Slot
latestExecutionPayloadHeaderEPBS *enginev1.ExecutionPayloadHeaderEPBS
lastWithdrawalsRoot [32]byte
// Electra fields
depositRequestsStartIndex uint64

View File

@@ -60,6 +60,11 @@ type BeaconState struct {
latestExecutionPayloadHeaderElectra *enginev1.ExecutionPayloadHeaderElectra
nextWithdrawalIndex uint64
nextWithdrawalValidatorIndex primitives.ValidatorIndex
// ePBS fields
latestBlockHash [32]byte
latestFullSlot primitives.Slot
latestExecutionPayloadHeaderEPBS *enginev1.ExecutionPayloadHeaderEPBS
lastWithdrawalsRoot [32]byte
// Electra fields
depositRequestsStartIndex uint64

View File

@@ -0,0 +1,75 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// LatestExecutionPayloadHeaderEPBS retrieves a copy of the execution payload header from epbs state.
// It returns an error if the operation is not supported for the beacon state's version.
func (b *BeaconState) LatestExecutionPayloadHeaderEPBS() (*enginev1.ExecutionPayloadHeaderEPBS, error) {
if b.version < version.EPBS {
return nil, errNotSupported("LatestExecutionPayloadHeaderEPBS", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.executionPayloadHeaderVal(), nil
}
// IsParentBlockFull checks if the last committed payload header was fulfilled.
// Returns true if both the beacon block and payload were present.
// Call this function on a beacon state before processing the execution payload header.
// It returns an error if the operation is not supported for the beacon state's version.
func (b *BeaconState) IsParentBlockFull() (bool, error) {
if b.version < version.EPBS {
return false, errNotSupported("IsParentBlockFull", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
headerBlockHash := bytesutil.ToBytes32(b.latestExecutionPayloadHeaderEPBS.BlockHash)
return headerBlockHash == b.latestBlockHash, nil
}
// LatestBlockHash returns the latest block hash.
// It returns an error if the operation is not supported for the beacon state's version.
func (b *BeaconState) LatestBlockHash() ([]byte, error) {
if b.version < version.EPBS {
return nil, errNotSupported("LatestBlockHash", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.latestBlockHash[:], nil
}
// LatestFullSlot returns the slot of the latest full block.
// It returns an error if the operation is not supported for the beacon state's version.
func (b *BeaconState) LatestFullSlot() (primitives.Slot, error) {
if b.version < version.EPBS {
return 0, errNotSupported("LatestFullSlot", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.latestFullSlot, nil
}
// LastWithdrawalsRoot returns the latest withdrawal root.
// It returns an error if the operation is not supported for the beacon state's version.
func (b *BeaconState) LastWithdrawalsRoot() ([]byte, error) {
if b.version < version.EPBS {
return nil, errNotSupported("LastWithdrawalsRoot", b.version)
}
b.lock.RLock()
defer b.lock.RUnlock()
return b.lastWithdrawalsRoot[:], nil
}

View File

@@ -0,0 +1,10 @@
package state_native
import (
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
)
func (b *BeaconState) executionPayloadHeaderVal() *enginev1.ExecutionPayloadHeaderEPBS {
return eth.CopyExecutionPayloadHeaderEPBS(b.latestExecutionPayloadHeaderEPBS)
}

View File

@@ -0,0 +1,90 @@
package state_native
import (
"crypto/rand"
"testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
)
func Test_LatestExecutionPayloadHeaderEPBS(t *testing.T) {
s := &BeaconState{version: version.EPBS}
_, err := s.LatestExecutionPayloadHeader()
require.ErrorContains(t, "unsupported version (epbs) for latest execution payload header", err)
}
func Test_SetLatestExecutionPayloadHeader(t *testing.T) {
s := &BeaconState{version: version.EPBS}
require.ErrorContains(t, "SetLatestExecutionPayloadHeader is not supported for epbs", s.SetLatestExecutionPayloadHeader(nil))
}
func Test_SetLatestExecutionPayloadHeaderEPBS(t *testing.T) {
s := &BeaconState{version: version.EPBS, dirtyFields: make(map[types.FieldIndex]bool)}
header := random.ExecutionPayloadHeader(t)
require.NoError(t, s.SetLatestExecutionPayloadHeaderEPBS(header))
require.Equal(t, true, s.dirtyFields[types.ExecutionPayloadHeader])
got, err := s.LatestExecutionPayloadHeaderEPBS()
require.NoError(t, err)
require.DeepEqual(t, got, header)
}
func Test_SetLatestBlockHash(t *testing.T) {
s := &BeaconState{version: version.EPBS, dirtyFields: make(map[types.FieldIndex]bool)}
b := make([]byte, fieldparams.RootLength)
_, err := rand.Read(b)
require.NoError(t, err)
require.NoError(t, s.SetLatestBlockHash(b))
require.Equal(t, true, s.dirtyFields[types.LatestBlockHash])
got, err := s.LatestBlockHash()
require.NoError(t, err)
require.DeepEqual(t, got, b)
}
func Test_SetLatestFullSlot(t *testing.T) {
s := &BeaconState{version: version.EPBS, dirtyFields: make(map[types.FieldIndex]bool)}
require.NoError(t, s.SetLatestFullSlot(primitives.Slot(3)))
require.Equal(t, true, s.dirtyFields[types.LatestFullSlot])
got, err := s.LatestFullSlot()
require.NoError(t, err)
require.Equal(t, primitives.Slot(3), got)
}
func Test_SetLastWithdrawalsRoot(t *testing.T) {
s := &BeaconState{version: version.EPBS, dirtyFields: make(map[types.FieldIndex]bool)}
b := make([]byte, fieldparams.RootLength)
_, err := rand.Read(b)
require.NoError(t, err)
require.NoError(t, s.SetLastWithdrawalsRoot(b))
require.Equal(t, true, s.dirtyFields[types.LastWithdrawalsRoot])
got, err := s.LastWithdrawalsRoot()
require.NoError(t, err)
require.DeepEqual(t, got, b)
}
func Test_UnsupportedStateVersionEpbs(t *testing.T) {
s := &BeaconState{version: version.Electra}
_, err := s.IsParentBlockFull()
require.ErrorContains(t, "IsParentBlockFull is not supported for electra", err)
_, err = s.LatestBlockHash()
require.ErrorContains(t, "LatestBlockHash is not supported for electra", err)
_, err = s.LatestFullSlot()
require.ErrorContains(t, "LatestFullSlot is not supported for electra", err)
_, err = s.LastWithdrawalsRoot()
require.ErrorContains(t, "LastWithdrawalsRoot is not supported for electra", err)
_, err = s.LatestExecutionPayloadHeaderEPBS()
require.ErrorContains(t, "LatestExecutionPayloadHeaderEPBS is not supported for electra", err)
require.ErrorContains(t, "LastWithdrawalsRoot is not supported for electra", s.SetLastWithdrawalsRoot(nil))
require.ErrorContains(t, "SetLatestBlockHash is not supported for electra", s.SetLatestBlockHash(nil))
require.ErrorContains(t, "SetLatestFullSlot is not supported for electra", s.SetLatestFullSlot(0))
require.ErrorContains(t, "SetLatestExecutionPayloadHeaderEPBS is not supported for electra", s.SetLatestExecutionPayloadHeaderEPBS(nil))
}

View File

@@ -212,6 +212,49 @@ func (b *BeaconState) ToProtoUnsafe() interface{} {
PendingPartialWithdrawals: b.pendingPartialWithdrawals,
PendingConsolidations: b.pendingConsolidations,
}
case version.EPBS:
return &ethpb.BeaconStateEPBS{
GenesisTime: b.genesisTime,
GenesisValidatorsRoot: gvrCopy[:],
Slot: b.slot,
Fork: b.fork,
LatestBlockHeader: b.latestBlockHeader,
BlockRoots: br,
StateRoots: sr,
HistoricalRoots: b.historicalRoots.Slice(),
Eth1Data: b.eth1Data,
Eth1DataVotes: b.eth1DataVotes,
Eth1DepositIndex: b.eth1DepositIndex,
Validators: vals,
Balances: bals,
RandaoMixes: rm,
Slashings: b.slashings,
PreviousEpochParticipation: b.previousEpochParticipation,
CurrentEpochParticipation: b.currentEpochParticipation,
JustificationBits: b.justificationBits,
PreviousJustifiedCheckpoint: b.previousJustifiedCheckpoint,
CurrentJustifiedCheckpoint: b.currentJustifiedCheckpoint,
FinalizedCheckpoint: b.finalizedCheckpoint,
InactivityScores: b.inactivityScoresVal(),
CurrentSyncCommittee: b.currentSyncCommittee,
NextSyncCommittee: b.nextSyncCommittee,
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
HistoricalSummaries: b.historicalSummaries,
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,
ConsolidationBalanceToConsume: b.consolidationBalanceToConsume,
EarliestConsolidationEpoch: b.earliestConsolidationEpoch,
PendingBalanceDeposits: b.pendingBalanceDeposits,
PendingPartialWithdrawals: b.pendingPartialWithdrawals,
PendingConsolidations: b.pendingConsolidations,
LatestBlockHash: b.latestBlockHash[:],
LatestFullSlot: b.latestFullSlot,
LatestExecutionPayloadHeader: b.latestExecutionPayloadHeaderEPBS,
LastWithdrawalsRoot: b.lastWithdrawalsRoot[:],
}
default:
return nil
}
@@ -236,6 +279,9 @@ func (b *BeaconState) ToProto() interface{} {
inactivityScores = b.inactivityScoresVal()
}
LatestBlockHashCopy := b.latestBlockHash
lastWithdrawalsRootCopy := b.lastWithdrawalsRoot
switch b.version {
case version.Phase0:
return &ethpb.BeaconState{
@@ -418,6 +464,49 @@ func (b *BeaconState) ToProto() interface{} {
PendingPartialWithdrawals: b.pendingPartialWithdrawalsVal(),
PendingConsolidations: b.pendingConsolidationsVal(),
}
case version.EPBS:
return &ethpb.BeaconStateEPBS{
GenesisTime: b.genesisTime,
GenesisValidatorsRoot: gvrCopy[:],
Slot: b.slot,
Fork: b.forkVal(),
LatestBlockHeader: b.latestBlockHeaderVal(),
BlockRoots: br,
StateRoots: sr,
HistoricalRoots: b.historicalRoots.Slice(),
Eth1Data: b.eth1DataVal(),
Eth1DataVotes: b.eth1DataVotesVal(),
Eth1DepositIndex: b.eth1DepositIndex,
Validators: b.validatorsVal(),
Balances: b.balancesVal(),
RandaoMixes: rm,
Slashings: b.slashingsVal(),
PreviousEpochParticipation: b.previousEpochParticipationVal(),
CurrentEpochParticipation: b.currentEpochParticipationVal(),
JustificationBits: b.justificationBitsVal(),
PreviousJustifiedCheckpoint: b.previousJustifiedCheckpointVal(),
CurrentJustifiedCheckpoint: b.currentJustifiedCheckpointVal(),
FinalizedCheckpoint: b.finalizedCheckpointVal(),
InactivityScores: b.inactivityScoresVal(),
CurrentSyncCommittee: b.currentSyncCommitteeVal(),
NextSyncCommittee: b.nextSyncCommitteeVal(),
NextWithdrawalIndex: b.nextWithdrawalIndex,
NextWithdrawalValidatorIndex: b.nextWithdrawalValidatorIndex,
HistoricalSummaries: b.historicalSummariesVal(),
DepositRequestsStartIndex: b.depositRequestsStartIndex,
DepositBalanceToConsume: b.depositBalanceToConsume,
ExitBalanceToConsume: b.exitBalanceToConsume,
EarliestExitEpoch: b.earliestExitEpoch,
ConsolidationBalanceToConsume: b.consolidationBalanceToConsume,
EarliestConsolidationEpoch: b.earliestConsolidationEpoch,
PendingBalanceDeposits: b.pendingBalanceDepositsVal(),
PendingPartialWithdrawals: b.pendingPartialWithdrawalsVal(),
PendingConsolidations: b.pendingConsolidationsVal(),
LatestBlockHash: LatestBlockHashCopy[:],
LatestFullSlot: b.latestFullSlot,
LatestExecutionPayloadHeader: b.executionPayloadHeaderVal(),
LastWithdrawalsRoot: lastWithdrawalsRootCopy[:],
}
default:
return nil
}

View File

@@ -41,6 +41,8 @@ func ComputeFieldRootsWithHasher(ctx context.Context, state *BeaconState) ([][]b
fieldRoots = make([][]byte, params.BeaconConfig().BeaconStateDenebFieldCount)
case version.Electra:
fieldRoots = make([][]byte, params.BeaconConfig().BeaconStateElectraFieldCount)
case version.EPBS:
fieldRoots = make([][]byte, params.BeaconConfig().BeaconStateEpbsFieldCount)
default:
return nil, fmt.Errorf("unknown state version %s", version.String(state.version))
}
@@ -260,6 +262,14 @@ func ComputeFieldRootsWithHasher(ctx context.Context, state *BeaconState) ([][]b
}
fieldRoots[types.LatestExecutionPayloadHeaderElectra.RealPosition()] = executionPayloadRoot[:]
}
if state.version == version.EPBS {
// Execution payload header root.
executionPayloadRoot, err := state.latestExecutionPayloadHeaderEPBS.HashTreeRoot()
if err != nil {
return nil, err
}
fieldRoots[types.ExecutionPayloadHeader.RealPosition()] = executionPayloadRoot[:]
}
if state.version >= version.Capella {
// Next withdrawal index root.
@@ -327,5 +337,19 @@ func ComputeFieldRootsWithHasher(ctx context.Context, state *BeaconState) ([][]b
fieldRoots[types.PendingConsolidations.RealPosition()] = pcRoot[:]
}
if state.version >= version.EPBS {
// Latest block hash root.
latestBlockHashRoot := state.latestBlockHash[:]
fieldRoots[types.LatestBlockHash.RealPosition()] = latestBlockHashRoot
// Latest full slot root.
latestFullSlotRoot := ssz.Uint64Root(uint64(state.latestFullSlot))
fieldRoots[types.LatestFullSlot.RealPosition()] = latestFullSlotRoot[:]
// Last withdrawals root.
lastWithdrawalsRoot := state.lastWithdrawalsRoot[:]
fieldRoots[types.LastWithdrawalsRoot.RealPosition()] = lastWithdrawalsRoot
}
return fieldRoots, nil
}

View File

@@ -0,0 +1,69 @@
package state_native
import (
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// SetLatestExecutionPayloadHeaderEPBS sets the latest execution payload header for the epbs beacon state.
func (b *BeaconState) SetLatestExecutionPayloadHeaderEPBS(h *enginev1.ExecutionPayloadHeaderEPBS) error {
if b.version < version.EPBS {
return errNotSupported("SetLatestExecutionPayloadHeaderEPBS", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.latestExecutionPayloadHeaderEPBS = h
b.markFieldAsDirty(types.ExecutionPayloadHeader)
return nil
}
// SetLatestBlockHash sets the latest block hash for the beacon state.
func (b *BeaconState) SetLatestBlockHash(h []byte) error {
if b.version < version.EPBS {
return errNotSupported("SetLatestBlockHash", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.latestBlockHash = bytesutil.ToBytes32(h)
b.markFieldAsDirty(types.LatestBlockHash)
return nil
}
// SetLatestFullSlot sets the latest full slot for the beacon state.
func (b *BeaconState) SetLatestFullSlot(s primitives.Slot) error {
if b.version < version.EPBS {
return errNotSupported("SetLatestFullSlot", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.latestFullSlot = s
b.markFieldAsDirty(types.LatestFullSlot)
return nil
}
// SetLastWithdrawalsRoot sets the latest withdrawals root for the beacon state.
func (b *BeaconState) SetLastWithdrawalsRoot(r []byte) error {
if b.version < version.EPBS {
return errNotSupported("SetLastWithdrawalsRoot", b.version)
}
b.lock.Lock()
defer b.lock.Unlock()
b.lastWithdrawalsRoot = bytesutil.ToBytes32(r)
b.markFieldAsDirty(types.LastWithdrawalsRoot)
return nil
}

View File

@@ -17,7 +17,7 @@ func (b *BeaconState) SetLatestExecutionPayloadHeader(val interfaces.ExecutionDa
b.lock.Lock()
defer b.lock.Unlock()
if b.version < version.Bellatrix {
if b.version < version.Bellatrix || b.version >= version.EPBS {
return errNotSupported("SetLatestExecutionPayloadHeader", b.version)
}

View File

@@ -7,7 +7,7 @@ import (
func (b *BeaconState) ProportionalSlashingMultiplier() (uint64, error) {
switch b.version {
case version.Bellatrix, version.Capella, version.Deneb, version.Electra:
case version.Bellatrix, version.Capella, version.Deneb, version.Electra, version.EPBS:
return params.BeaconConfig().ProportionalSlashingMultiplierBellatrix, nil
case version.Altair:
return params.BeaconConfig().ProportionalSlashingMultiplierAltair, nil
@@ -19,7 +19,7 @@ func (b *BeaconState) ProportionalSlashingMultiplier() (uint64, error) {
func (b *BeaconState) InactivityPenaltyQuotient() (uint64, error) {
switch b.version {
case version.Bellatrix, version.Capella, version.Deneb, version.Electra:
case version.Bellatrix, version.Capella, version.Deneb, version.Electra, version.EPBS:
return params.BeaconConfig().InactivityPenaltyQuotientBellatrix, nil
case version.Altair:
return params.BeaconConfig().InactivityPenaltyQuotientAltair, nil

View File

@@ -111,6 +111,27 @@ var electraFields = append(
types.PendingConsolidations,
)
var epbsFields = append(
altairFields,
types.NextWithdrawalIndex,
types.NextWithdrawalValidatorIndex,
types.HistoricalSummaries,
types.ExecutionPayloadHeader, // new in ePBS
types.DepositRequestsStartIndex, // electra fields start here
types.DepositBalanceToConsume,
types.ExitBalanceToConsume,
types.EarliestExitEpoch,
types.ConsolidationBalanceToConsume,
types.EarliestConsolidationEpoch,
types.PendingBalanceDeposits,
types.PendingPartialWithdrawals,
types.PendingConsolidations,
types.LatestBlockHash, // ePBS fields start here
types.LatestFullSlot,
types.ExecutionPayloadHeader,
types.LastWithdrawalsRoot,
)
const (
phase0SharedFieldRefCount = 10
altairSharedFieldRefCount = 11
@@ -118,12 +139,14 @@ const (
capellaSharedFieldRefCount = 13
denebSharedFieldRefCount = 13
electraSharedFieldRefCount = 16
epbsSharedFieldRefCount = 16
experimentalStatePhase0SharedFieldRefCount = 5
experimentalStateAltairSharedFieldRefCount = 5
experimentalStateBellatrixSharedFieldRefCount = 6
experimentalStateCapellaSharedFieldRefCount = 7
experimentalStateDenebSharedFieldRefCount = 7
experimentalStateElectraSharedFieldRefCount = 10
experimentalStateEpbsSharedFieldRefCount = 10
)
// InitializeFromProtoPhase0 the beacon state from a protobuf representation.
@@ -155,6 +178,11 @@ func InitializeFromProtoElectra(st *ethpb.BeaconStateElectra) (state.BeaconState
return InitializeFromProtoUnsafeElectra(proto.Clone(st).(*ethpb.BeaconStateElectra))
}
// InitializeFromProtoEpbs initializes the beacon state from its protobuf representation.
func InitializeFromProtoEpbs(st *ethpb.BeaconStateEPBS) (state.BeaconState, error) {
return InitializeFromProtoUnsafeEpbs(proto.Clone(st).(*ethpb.BeaconStateEPBS))
}
// InitializeFromProtoUnsafePhase0 directly uses the beacon state protobuf fields
// and sets them as fields of the BeaconState type.
func InitializeFromProtoUnsafePhase0(st *ethpb.BeaconState) (state.BeaconState, error) {
@@ -857,6 +885,8 @@ func (b *BeaconState) Copy() state.BeaconState {
fieldCount = params.BeaconConfig().BeaconStateDenebFieldCount
case version.Electra:
fieldCount = params.BeaconConfig().BeaconStateElectraFieldCount
case version.EPBS:
fieldCount = params.BeaconConfig().BeaconStateEpbsFieldCount
}
dst := &BeaconState{
@@ -874,6 +904,9 @@ func (b *BeaconState) Copy() state.BeaconState {
earliestExitEpoch: b.earliestExitEpoch,
consolidationBalanceToConsume: b.consolidationBalanceToConsume,
earliestConsolidationEpoch: b.earliestConsolidationEpoch,
latestBlockHash: b.latestBlockHash,
latestFullSlot: b.latestFullSlot,
lastWithdrawalsRoot: b.lastWithdrawalsRoot,
// Large arrays, infrequently changed, constant size.
blockRoots: b.blockRoots,
@@ -917,6 +950,7 @@ func (b *BeaconState) Copy() state.BeaconState {
latestExecutionPayloadHeaderCapella: b.latestExecutionPayloadHeaderCapella.Copy(),
latestExecutionPayloadHeaderDeneb: b.latestExecutionPayloadHeaderDeneb.Copy(),
latestExecutionPayloadHeaderElectra: b.latestExecutionPayloadHeaderElectra.Copy(),
latestExecutionPayloadHeaderEPBS: b.executionPayloadHeaderVal(),
id: types.Enumerator.Inc(),
@@ -955,6 +989,8 @@ func (b *BeaconState) Copy() state.BeaconState {
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, experimentalStateDenebSharedFieldRefCount)
case version.Electra:
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, experimentalStateElectraSharedFieldRefCount)
case version.EPBS:
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, experimentalStateEpbsSharedFieldRefCount)
}
} else {
switch b.version {
@@ -970,6 +1006,8 @@ func (b *BeaconState) Copy() state.BeaconState {
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, denebSharedFieldRefCount)
case version.Electra:
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, electraSharedFieldRefCount)
case version.EPBS:
dst.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, epbsSharedFieldRefCount)
}
}
@@ -1064,6 +1102,8 @@ func (b *BeaconState) initializeMerkleLayers(ctx context.Context) error {
b.dirtyFields = make(map[types.FieldIndex]bool, params.BeaconConfig().BeaconStateDenebFieldCount)
case version.Electra:
b.dirtyFields = make(map[types.FieldIndex]bool, params.BeaconConfig().BeaconStateElectraFieldCount)
case version.EPBS:
b.dirtyFields = make(map[types.FieldIndex]bool, params.BeaconConfig().BeaconStateEpbsFieldCount)
default:
return fmt.Errorf("unknown state version (%s) when computing dirty fields in merklization", version.String(b.version))
}
@@ -1310,6 +1350,14 @@ func (b *BeaconState) rootSelector(ctx context.Context, field types.FieldIndex)
return stateutil.PendingPartialWithdrawalsRoot(b.pendingPartialWithdrawals)
case types.PendingConsolidations:
return stateutil.PendingConsolidationsRoot(b.pendingConsolidations)
case types.LatestBlockHash:
return b.latestBlockHash, nil
case types.LatestFullSlot:
return ssz.Uint64Root(uint64(b.latestFullSlot)), nil
case types.ExecutionPayloadHeader:
return b.latestExecutionPayloadHeaderEPBS.HashTreeRoot()
case types.LastWithdrawalsRoot:
return b.lastWithdrawalsRoot, nil
}
return [32]byte{}, errors.New("invalid field index provided")
}

View File

@@ -0,0 +1,148 @@
package state_native
import (
"runtime"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/fieldtrie"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native/types"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stateutil"
"github.com/prysmaticlabs/prysm/v5/config/features"
fieldparams "github.com/prysmaticlabs/prysm/v5/config/fieldparams"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/runtime/version"
)
// InitializeFromProtoUnsafeEpbs constructs a BeaconState from its protobuf representation.
func InitializeFromProtoUnsafeEpbs(st *ethpb.BeaconStateEPBS) (*BeaconState, error) {
if st == nil {
return nil, errors.New("received nil state")
}
// Process historical roots.
hRoots := make([][32]byte, len(st.HistoricalRoots))
for i, root := range st.HistoricalRoots {
hRoots[i] = bytesutil.ToBytes32(root)
}
// Define the number of fields to track changes.
fieldCount := params.BeaconConfig().BeaconStateEpbsFieldCount
b := &BeaconState{
version: version.EPBS,
genesisTime: st.GenesisTime,
genesisValidatorsRoot: bytesutil.ToBytes32(st.GenesisValidatorsRoot),
slot: st.Slot,
fork: st.Fork,
latestBlockHeader: st.LatestBlockHeader,
historicalRoots: hRoots,
eth1Data: st.Eth1Data,
eth1DataVotes: st.Eth1DataVotes,
eth1DepositIndex: st.Eth1DepositIndex,
slashings: st.Slashings,
previousEpochParticipation: st.PreviousEpochParticipation,
currentEpochParticipation: st.CurrentEpochParticipation,
justificationBits: st.JustificationBits,
previousJustifiedCheckpoint: st.PreviousJustifiedCheckpoint,
currentJustifiedCheckpoint: st.CurrentJustifiedCheckpoint,
finalizedCheckpoint: st.FinalizedCheckpoint,
currentSyncCommittee: st.CurrentSyncCommittee,
nextSyncCommittee: st.NextSyncCommittee,
nextWithdrawalIndex: st.NextWithdrawalIndex,
nextWithdrawalValidatorIndex: st.NextWithdrawalValidatorIndex,
historicalSummaries: st.HistoricalSummaries,
depositRequestsStartIndex: st.DepositRequestsStartIndex,
depositBalanceToConsume: st.DepositBalanceToConsume,
exitBalanceToConsume: st.ExitBalanceToConsume,
earliestExitEpoch: st.EarliestExitEpoch,
consolidationBalanceToConsume: st.ConsolidationBalanceToConsume,
earliestConsolidationEpoch: st.EarliestConsolidationEpoch,
pendingBalanceDeposits: st.PendingBalanceDeposits,
pendingPartialWithdrawals: st.PendingPartialWithdrawals,
pendingConsolidations: st.PendingConsolidations,
// ePBS fields
latestBlockHash: bytesutil.ToBytes32(st.LatestBlockHash),
latestFullSlot: st.LatestFullSlot,
latestExecutionPayloadHeaderEPBS: st.LatestExecutionPayloadHeader,
lastWithdrawalsRoot: bytesutil.ToBytes32(st.LastWithdrawalsRoot),
dirtyFields: make(map[types.FieldIndex]bool, fieldCount),
dirtyIndices: make(map[types.FieldIndex][]uint64, fieldCount),
stateFieldLeaves: make(map[types.FieldIndex]*fieldtrie.FieldTrie, fieldCount),
rebuildTrie: make(map[types.FieldIndex]bool, fieldCount),
valMapHandler: stateutil.NewValMapHandler(st.Validators),
validatorIndexCache: newFinalizedValidatorIndexCache(), // only used in post-electra and only populates when finalizing, otherwise it falls back to processing the full validator set
}
if features.Get().EnableExperimentalState {
b.blockRootsMultiValue = NewMultiValueBlockRoots(st.BlockRoots)
b.stateRootsMultiValue = NewMultiValueStateRoots(st.StateRoots)
b.randaoMixesMultiValue = NewMultiValueRandaoMixes(st.RandaoMixes)
b.balancesMultiValue = NewMultiValueBalances(st.Balances)
b.validatorsMultiValue = NewMultiValueValidators(st.Validators)
b.inactivityScoresMultiValue = NewMultiValueInactivityScores(st.InactivityScores)
b.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, experimentalStateEpbsSharedFieldRefCount)
} else {
bRoots := make([][32]byte, fieldparams.BlockRootsLength)
for i, r := range st.BlockRoots {
bRoots[i] = bytesutil.ToBytes32(r)
}
b.blockRoots = bRoots
sRoots := make([][32]byte, fieldparams.StateRootsLength)
for i, r := range st.StateRoots {
sRoots[i] = bytesutil.ToBytes32(r)
}
b.stateRoots = sRoots
mixes := make([][32]byte, fieldparams.RandaoMixesLength)
for i, m := range st.RandaoMixes {
mixes[i] = bytesutil.ToBytes32(m)
}
b.randaoMixes = mixes
b.balances = st.Balances
b.validators = st.Validators
b.inactivityScores = st.InactivityScores
b.sharedFieldReferences = make(map[types.FieldIndex]*stateutil.Reference, epbsSharedFieldRefCount)
}
for _, f := range epbsFields {
b.dirtyFields[f] = true
b.rebuildTrie[f] = true
b.dirtyIndices[f] = []uint64{}
trie, err := fieldtrie.NewFieldTrie(f, types.BasicArray, nil, 0)
if err != nil {
return nil, err
}
b.stateFieldLeaves[f] = trie
}
// Initialize field reference tracking for shared data.
b.sharedFieldReferences[types.HistoricalRoots] = stateutil.NewRef(1)
b.sharedFieldReferences[types.Eth1DataVotes] = stateutil.NewRef(1)
b.sharedFieldReferences[types.Slashings] = stateutil.NewRef(1)
b.sharedFieldReferences[types.PreviousEpochParticipationBits] = stateutil.NewRef(1)
b.sharedFieldReferences[types.CurrentEpochParticipationBits] = stateutil.NewRef(1)
b.sharedFieldReferences[types.HistoricalSummaries] = stateutil.NewRef(1)
b.sharedFieldReferences[types.PendingBalanceDeposits] = stateutil.NewRef(1)
b.sharedFieldReferences[types.PendingPartialWithdrawals] = stateutil.NewRef(1)
b.sharedFieldReferences[types.PendingConsolidations] = stateutil.NewRef(1)
if !features.Get().EnableExperimentalState {
b.sharedFieldReferences[types.BlockRoots] = stateutil.NewRef(1)
b.sharedFieldReferences[types.StateRoots] = stateutil.NewRef(1)
b.sharedFieldReferences[types.RandaoMixes] = stateutil.NewRef(1)
b.sharedFieldReferences[types.Balances] = stateutil.NewRef(1)
b.sharedFieldReferences[types.Validators] = stateutil.NewRef(1)
b.sharedFieldReferences[types.InactivityScores] = stateutil.NewRef(1)
}
state.Count.Inc()
// Finalizer runs when dst is being destroyed in garbage collection.
runtime.SetFinalizer(b, finalizerCleanup)
return b, nil
}

View File

@@ -0,0 +1,70 @@
package state_native
import (
"context"
"testing"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
)
func Test_InitializeFromProtoEpbs(t *testing.T) {
st := random.BeaconState(t)
// Cache initial values to check against after initialization.
latestBlockHash := st.LatestBlockHash
latestFullSlot := st.LatestFullSlot
header := st.LatestExecutionPayloadHeader
lastWithdrawalsRoot := st.LastWithdrawalsRoot
s, err := InitializeFromProtoEpbs(st)
require.NoError(t, err)
// Assert that initial values match those in the new state.
gotLatestBlockHash, err := s.LatestBlockHash()
require.NoError(t, err)
require.DeepEqual(t, latestBlockHash, gotLatestBlockHash)
gotLatestFullSlot, err := s.LatestFullSlot()
require.NoError(t, err)
require.Equal(t, latestFullSlot, gotLatestFullSlot)
gotHeader, err := s.LatestExecutionPayloadHeaderEPBS()
require.NoError(t, err)
require.DeepEqual(t, header, gotHeader)
gotLastWithdrawalsRoot, err := s.LastWithdrawalsRoot()
require.NoError(t, err)
require.DeepEqual(t, lastWithdrawalsRoot, gotLastWithdrawalsRoot)
}
func Test_CopyEpbs(t *testing.T) {
st := random.BeaconState(t)
s, err := InitializeFromProtoUnsafeEpbs(st)
require.NoError(t, err)
// Test shallow copy.
sNoCopy := s
require.DeepEqual(t, s.latestExecutionPayloadHeaderEPBS, sNoCopy.latestExecutionPayloadHeaderEPBS)
// Modify a field to check if it reflects in the shallow copy.
s.latestExecutionPayloadHeaderEPBS.Slot = 100
require.Equal(t, s.latestExecutionPayloadHeaderEPBS, sNoCopy.latestExecutionPayloadHeaderEPBS)
// Copy the state
sCopy := s.Copy()
require.NoError(t, err)
header, err := sCopy.LatestExecutionPayloadHeaderEPBS()
require.NoError(t, err)
require.DeepEqual(t, s.latestExecutionPayloadHeaderEPBS, header)
// Modify the original to check if the copied state is independent.
s.latestExecutionPayloadHeaderEPBS.Slot = 200
require.DeepNotEqual(t, s.latestExecutionPayloadHeaderEPBS, header)
}
func Test_HashTreeRootEpbs(t *testing.T) {
st := random.BeaconState(t)
s, err := InitializeFromProtoUnsafeEpbs(st)
require.NoError(t, err)
_, err = s.HashTreeRoot(context.Background())
require.NoError(t, err)
}

View File

@@ -114,6 +114,14 @@ func (f FieldIndex) String() string {
return "pendingPartialWithdrawals"
case PendingConsolidations:
return "pendingConsolidations"
case LatestBlockHash: // ePBS fields start here
return "LatestBlockHash"
case LatestFullSlot:
return "LatestFullSlot"
case ExecutionPayloadHeader:
return "ExecutionPayloadHeader"
case LastWithdrawalsRoot:
return "LastWithdrawalsRoot"
default:
return fmt.Sprintf("unknown field index number: %d", f)
}
@@ -171,7 +179,8 @@ func (f FieldIndex) RealPosition() int {
return 22
case NextSyncCommittee:
return 23
case LatestExecutionPayloadHeader, LatestExecutionPayloadHeaderCapella, LatestExecutionPayloadHeaderDeneb, LatestExecutionPayloadHeaderElectra:
// ExecutionPayloadHeader is from ePBS.
case LatestExecutionPayloadHeader, LatestExecutionPayloadHeaderCapella, LatestExecutionPayloadHeaderDeneb, LatestExecutionPayloadHeaderElectra, ExecutionPayloadHeader:
return 24
case NextWithdrawalIndex:
return 25
@@ -197,6 +206,12 @@ func (f FieldIndex) RealPosition() int {
return 35
case PendingConsolidations:
return 36
case LatestBlockHash: // ePBS fields start here
return 41
case LatestFullSlot:
return 42
case LastWithdrawalsRoot:
return 43
default:
return -1
}
@@ -250,6 +265,7 @@ const (
LatestExecutionPayloadHeaderCapella
LatestExecutionPayloadHeaderDeneb
LatestExecutionPayloadHeaderElectra
ExecutionPayloadHeader
NextWithdrawalIndex
NextWithdrawalValidatorIndex
HistoricalSummaries
@@ -262,6 +278,9 @@ const (
PendingBalanceDeposits // Electra: EIP-7251
PendingPartialWithdrawals // Electra: EIP-7251
PendingConsolidations // Electra: EIP-7251
LatestBlockHash // ePBS fields start here
LatestFullSlot
LastWithdrawalsRoot
)
// Enumerator keeps track of the number of states created since the node's start.

View File

@@ -16,6 +16,7 @@ go_library(
"log.go",
"metrics.go",
"options.go",
"payload_attestations.go",
"pending_attestations_queue.go",
"pending_blocks_queue.go",
"rate_limiter.go",
@@ -101,6 +102,7 @@ go_library(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/epbs/payload-attestation:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
@@ -155,6 +157,7 @@ go_test(
"decode_pubsub_test.go",
"error_test.go",
"fork_watcher_test.go",
"payload_attestations_test.go",
"pending_attestations_queue_test.go",
"pending_blocks_queue_test.go",
"rate_limiter_test.go",
@@ -228,6 +231,7 @@ go_test(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/epbs/payload-attestation:go_default_library",
"//consensus-types/interfaces:go_default_library",
"//consensus-types/primitives:go_default_library",
"//consensus-types/wrapper:go_default_library",
@@ -246,6 +250,7 @@ go_test(
"//testing/assert:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"//testing/util/random:go_default_library",
"//time:go_default_library",
"//time/slots:go_default_library",
"@com_github_d4l3k_messagediff//:go_default_library",

View File

@@ -2,6 +2,7 @@ package sync
import (
"github.com/prysmaticlabs/prysm/v5/async/event"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
@@ -127,6 +128,13 @@ func WithSlasherBlockHeadersFeed(slasherBlockHeadersFeed *event.Feed) Option {
}
}
func WithPayloadAttestationCache(r *cache.PayloadAttestationCache) Option {
return func(s *Service) error {
s.payloadAttestationCache = r
return nil
}
}
func WithPayloadReconstructor(r execution.PayloadReconstructor) Option {
return func(s *Service) error {
s.cfg.executionPayloadReconstructor = r

View File

@@ -0,0 +1,114 @@
package sync
import (
"context"
"slices"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/libp2p/go-libp2p/core/peer"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"go.opencensus.io/trace"
"google.golang.org/protobuf/proto"
)
var (
errAlreadySeenPayloadAttestation = errors.New("payload attestation already seen for validator index")
)
func (s *Service) validatePayloadAttestation(ctx context.Context, pid peer.ID, msg *pubsub.Message) (pubsub.ValidationResult, error) {
if pid == s.cfg.p2p.PeerID() {
return pubsub.ValidationAccept, nil
}
if s.cfg.initialSync.Syncing() {
return pubsub.ValidationIgnore, nil
}
ctx, span := trace.StartSpan(ctx, "sync.validatePayloadAttestation")
defer span.End()
if msg.Topic == nil {
return pubsub.ValidationReject, errInvalidTopic
}
m, err := s.decodePubsubMessage(msg)
if err != nil {
tracing.AnnotateError(span, err)
return pubsub.ValidationReject, err
}
att, ok := m.(*eth.PayloadAttestationMessage)
if !ok {
return pubsub.ValidationReject, errWrongMessage
}
pa, err := payloadattestation.NewReadOnly(att)
if err != nil {
log.WithError(err).Error("failed to create read only payload attestation")
return pubsub.ValidationIgnore, err
}
v := s.newPayloadAttestationVerifier(pa, verification.GossipPayloadAttestationMessageRequirements)
if err := v.VerifyCurrentSlot(); err != nil {
return pubsub.ValidationIgnore, err
}
if err := v.VerifyPayloadStatus(); err != nil {
return pubsub.ValidationReject, err
}
if err := v.VerifyBlockRootSeen(s.hasBadBlock); err != nil {
return pubsub.ValidationIgnore, err
}
if err := v.VerifyBlockRootValid(s.hasBadBlock); err != nil {
return pubsub.ValidationReject, err
}
st, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return pubsub.ValidationIgnore, err
}
if err := v.VerifyValidatorInPTC(ctx, st); err != nil {
return pubsub.ValidationReject, err
}
if err := v.VerifySignature(st); err != nil {
return pubsub.ValidationReject, err
}
if s.payloadAttestationCache.Seen(pa.BeaconBlockRoot(), uint64(pa.ValidatorIndex())) {
return pubsub.ValidationIgnore, errAlreadySeenPayloadAttestation
}
return pubsub.ValidationAccept, nil
}
func (s *Service) payloadAttestationSubscriber(ctx context.Context, msg proto.Message) error {
a, ok := msg.(*eth.PayloadAttestationMessage)
if !ok {
return errWrongMessage
}
if err := helpers.ValidateNilPayloadAttestationMessage(a); err != nil {
return err
}
root := [32]byte(a.Data.BeaconBlockRoot)
st, err := s.cfg.chain.HeadState(ctx)
if err != nil {
return err
}
ptc, err := helpers.GetPayloadTimelinessCommittee(ctx, st, a.Data.Slot)
if err != nil {
return err
}
idx := slices.Index(ptc, a.ValidatorIndex)
if idx == -1 {
return errInvalidValidatorIndex
}
if s.payloadAttestationCache.Seen(root, uint64(primitives.ValidatorIndex(idx))) {
return nil
}
return s.payloadAttestationCache.Add(a, uint64(idx))
}

View File

@@ -0,0 +1,162 @@
package sync
import (
"bytes"
"context"
"reflect"
"testing"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pb "github.com/libp2p/go-libp2p-pubsub/pb"
"github.com/pkg/errors"
mock "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
mockSync "github.com/prysmaticlabs/prysm/v5/beacon-chain/sync/initial-sync/testing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
"github.com/prysmaticlabs/prysm/v5/config/params"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
)
func TestValidatePayloadAttestationMessage_IncorrectTopic(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
s := &Service{
payloadAttestationCache: &cache.PayloadAttestationCache{},
cfg: &config{chain: chainService, p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
msg := random.PayloadAttestation(t) // Using payload attestation for message should fail.
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, msg)
require.NoError(t, err)
topic := p2p.GossipTypeMapping[reflect.TypeOf(msg)]
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
result, err := s.validatePayloadAttestation(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
require.ErrorContains(t, "extraction failed for topic", err)
require.Equal(t, result, pubsub.ValidationReject)
}
func TestValidatePayloadAttestationMessage_ErrorPathsWithMock(t *testing.T) {
tests := []struct {
error error
verifier verification.NewPayloadAttestationMsgVerifier
result pubsub.ValidationResult
}{
{
error: errors.New("incorrect slot"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrIncorrectPayloadAttSlot: errors.New("incorrect slot")}
},
result: pubsub.ValidationIgnore,
},
{
error: errors.New("incorrect status"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrIncorrectPayloadAttStatus: errors.New("incorrect status")}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("block root seen"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrPayloadAttBlockRootNotSeen: errors.New("block root seen")}
},
result: pubsub.ValidationIgnore,
},
{
error: errors.New("block root invalid"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrPayloadAttBlockRootInvalid: errors.New("block root invalid")}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("validator not in PTC"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrIncorrectPayloadAttValidator: errors.New("validator not in PTC")}
},
result: pubsub.ValidationReject,
},
{
error: errors.New("incorrect signature"),
verifier: func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{ErrInvalidMessageSignature: errors.New("incorrect signature")}
},
result: pubsub.ValidationReject,
},
}
for _, tt := range tests {
t.Run(tt.error.Error(), func(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
s := &Service{
payloadAttestationCache: &cache.PayloadAttestationCache{},
cfg: &config{chain: chainService, p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
s.newPayloadAttestationVerifier = tt.verifier
msg := random.PayloadAttestationMessage(t)
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, msg)
require.NoError(t, err)
topic := p2p.GossipTypeMapping[reflect.TypeOf(msg)]
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
result, err := s.validatePayloadAttestation(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
require.ErrorContains(t, tt.error.Error(), err)
require.Equal(t, result, tt.result)
})
}
}
func TestValidatePayloadAttestationMessage_Accept(t *testing.T) {
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
s := &Service{
payloadAttestationCache: &cache.PayloadAttestationCache{},
cfg: &config{chain: chainService, p2p: p, initialSync: &mockSync.Sync{}, clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot)}}
s.newPayloadAttestationVerifier = func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return &verification.MockPayloadAttestation{}
}
msg := random.PayloadAttestationMessage(t)
buf := new(bytes.Buffer)
_, err := p.Encoding().EncodeGossip(buf, msg)
require.NoError(t, err)
topic := p2p.GossipTypeMapping[reflect.TypeOf(msg)]
digest, err := s.currentForkDigest()
require.NoError(t, err)
topic = s.addDigestToTopic(topic, digest)
result, err := s.validatePayloadAttestation(ctx, "", &pubsub.Message{
Message: &pb.Message{
Data: buf.Bytes(),
Topic: &topic,
}})
require.NoError(t, err)
require.Equal(t, result, pubsub.ValidationAccept)
}

View File

@@ -19,6 +19,7 @@ import (
"github.com/prysmaticlabs/prysm/v5/async/abool"
"github.com/prysmaticlabs/prysm/v5/async/event"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
blockfeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/block"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/operation"
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
@@ -38,6 +39,7 @@ import (
lruwrpr "github.com/prysmaticlabs/prysm/v5/cache/lru"
"github.com/prysmaticlabs/prysm/v5/config/params"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
leakybucket "github.com/prysmaticlabs/prysm/v5/container/leaky-bucket"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
@@ -124,6 +126,7 @@ type Service struct {
seenPendingBlocks map[[32]byte]bool
blkRootToPendingAtts map[[32]byte][]ethpb.SignedAggregateAttAndProof
subHandler *subTopicHandler
payloadAttestationCache *cache.PayloadAttestationCache
pendingAttsLock sync.RWMutex
pendingQueueLock sync.RWMutex
chainStarted *abool.AtomicBool
@@ -156,6 +159,7 @@ type Service struct {
initialSyncComplete chan struct{}
verifierWaiter *verification.InitializerWaiter
newBlobVerifier verification.NewBlobVerifier
newPayloadAttestationVerifier verification.NewPayloadAttestationMsgVerifier
availableBlocker coverage.AvailableBlocker
ctxMap ContextByteVersions
}
@@ -214,6 +218,12 @@ func newBlobVerifierFromInitializer(ini *verification.Initializer) verification.
}
}
func newPayloadAttestationMessageFromInitializer(ini *verification.Initializer) verification.NewPayloadAttestationMsgVerifier {
return func(pa payloadattestation.ROMessage, reqs []verification.Requirement) verification.PayloadAttestationMsgVerifier {
return ini.NewPayloadAttestationMsgVerifier(pa, reqs)
}
}
// Start the regular sync service.
func (s *Service) Start() {
v, err := s.verifierWaiter.WaitForInitializer(s.ctx)
@@ -222,6 +232,7 @@ func (s *Service) Start() {
return
}
s.newBlobVerifier = newBlobVerifierFromInitializer(v)
s.newPayloadAttestationVerifier = newPayloadAttestationMessageFromInitializer(v)
go s.verifierRoutine()
go s.registerHandlers()

View File

@@ -145,6 +145,16 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
params.BeaconConfig().BlobsidecarSubnetCount,
)
}
// New Gossip Topic for ePBS
if epoch >= params.BeaconConfig().EPBSForkEpoch {
s.subscribe(
p2p.PayloadAttestationMessageTopicFormat,
s.validatePayloadAttestation,
s.payloadAttestationSubscriber,
digest,
)
}
}
// subscribe to a given topic with a given validator and subscription handler.

View File

@@ -9,9 +9,12 @@ go_library(
"error.go",
"fake.go",
"initializer.go",
"initializer_epbs.go",
"interface.go",
"metrics.go",
"mock.go",
"payload_attestation.go",
"payload_attestation_mock.go",
"result.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/verification",
@@ -28,6 +31,7 @@ go_library(
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/epbs/payload-attestation:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",
@@ -50,18 +54,22 @@ go_test(
"blob_test.go",
"cache_test.go",
"initializer_test.go",
"payload_attestation_test.go",
"result_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/forkchoice/types:go_default_library",
"//beacon-chain/startup:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/state-native:go_default_library",
"//config/fieldparams:go_default_library",
"//config/params:go_default_library",
"//consensus-types/blocks:go_default_library",
"//consensus-types/epbs/payload-attestation:go_default_library",
"//consensus-types/primitives:go_default_library",
"//crypto/bls:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -0,0 +1,15 @@
package verification
import (
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
)
// NewPayloadAttestationMsgVerifier creates a PayloadAttestationMsgVerifier for a single payload attestation message,
// with the given set of requirements.
func (ini *Initializer) NewPayloadAttestationMsgVerifier(pa payloadattestation.ROMessage, reqs []Requirement) *PayloadAttMsgVerifier {
return &PayloadAttMsgVerifier{
sharedResources: ini.shared,
results: newResults(reqs...),
pa: pa,
}
}

View File

@@ -3,7 +3,9 @@ package verification
import (
"context"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
)
// BlobVerifier defines the methods implemented by the ROBlobVerifier.
@@ -26,6 +28,23 @@ type BlobVerifier interface {
SatisfyRequirement(Requirement)
}
// PayloadAttestationMsgVerifier defines the methods implemented by the ROPayloadAttestation.
// It is similar to BlobVerifier, but for payload attestation messages.
type PayloadAttestationMsgVerifier interface {
VerifyCurrentSlot() error
VerifyPayloadStatus() error
VerifyBlockRootSeen(func([32]byte) bool) error
VerifyBlockRootValid(func([32]byte) bool) error
VerifyValidatorInPTC(context.Context, state.BeaconState) error
VerifySignature(state.BeaconState) error
VerifiedPayloadAttestation() (payloadattestation.VerifiedROMessage, error)
SatisfyRequirement(Requirement)
}
// NewBlobVerifier is a function signature that can be used by code that needs to be
// able to mock Initializer.NewBlobVerifier without complex setup.
type NewBlobVerifier func(b blocks.ROBlob, reqs []Requirement) BlobVerifier
// NewPayloadAttestationMsgVerifier is a function signature that can be used by code that needs to be
// able to mock Initializer.NewPayloadAttestationMsgVerifier without complex setup.
type NewPayloadAttestationMsgVerifier func(pa payloadattestation.ROMessage, reqs []Requirement) PayloadAttestationMsgVerifier

View File

@@ -0,0 +1,231 @@
package verification
import (
"context"
"fmt"
"slices"
"github.com/pkg/errors"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
"github.com/prysmaticlabs/prysm/v5/config/params"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/time/slots"
log "github.com/sirupsen/logrus"
)
// RequirementList defines a list of requirements.
type RequirementList []Requirement
const (
RequireCurrentSlot Requirement = iota
RequireMessageNotSeen
RequireKnownPayloadStatus
RequireValidatorInPTC
RequireBlockRootSeen
RequireBlockRootValid
RequireSignatureValid
)
// PayloadAttGossipRequirements defines the list of requirements for gossip payload attestation messages.
var PayloadAttGossipRequirements = []Requirement{
RequireCurrentSlot,
RequireMessageNotSeen,
RequireKnownPayloadStatus,
RequireValidatorInPTC,
RequireBlockRootSeen,
RequireBlockRootValid,
RequireSignatureValid,
}
// GossipPayloadAttestationMessageRequirements is a requirement list for gossip payload attestation messages.
var GossipPayloadAttestationMessageRequirements = RequirementList(PayloadAttGossipRequirements)
var (
ErrIncorrectPayloadAttSlot = errors.New("payload att slot does not match the current slot")
ErrIncorrectPayloadAttStatus = errors.New("unknown payload att status")
ErrPayloadAttBlockRootNotSeen = errors.New("block root not seen")
ErrPayloadAttBlockRootInvalid = errors.New("block root invalid")
ErrIncorrectPayloadAttValidator = errors.New("validator not present in payload timeliness committee")
ErrInvalidPayloadAttMessage = errors.New("invalid payload attestation message")
)
var _ PayloadAttestationMsgVerifier = &PayloadAttMsgVerifier{}
// PayloadAttMsgVerifier is a read-only verifier for payload attestation messages.
type PayloadAttMsgVerifier struct {
*sharedResources
results *results
pa payloadattestation.ROMessage
}
// VerifyCurrentSlot verifies if the current slot matches the expected slot.
// Represents the following spec verification:
// [IGNORE] data.slot is the current slot.
func (v *PayloadAttMsgVerifier) VerifyCurrentSlot() (err error) {
defer v.record(RequireCurrentSlot, &err)
if v.pa.Slot() != v.clock.CurrentSlot() {
log.WithFields(logFields(v.pa)).Errorf("does not match current slot %d", v.clock.CurrentSlot())
return ErrIncorrectPayloadAttSlot
}
return nil
}
// VerifyPayloadStatus verifies if the payload status is known.
// Represents the following spec verification:
// [REJECT] data.payload_status < PAYLOAD_INVALID_STATUS.
func (v *PayloadAttMsgVerifier) VerifyPayloadStatus() (err error) {
defer v.record(RequireKnownPayloadStatus, &err)
if v.pa.PayloadStatus() >= primitives.PAYLOAD_INVALID_STATUS {
log.WithFields(logFields(v.pa)).Error(ErrIncorrectPayloadAttStatus.Error())
return ErrIncorrectPayloadAttStatus
}
return nil
}
// VerifyBlockRootSeen verifies if the block root has been seen before.
// Represents the following spec verification:
// [IGNORE] The attestation's data.beacon_block_root has been seen (via both gossip and non-gossip sources).
func (v *PayloadAttMsgVerifier) VerifyBlockRootSeen(parentSeen func([32]byte) bool) (err error) {
defer v.record(RequireBlockRootSeen, &err)
if parentSeen != nil && parentSeen(v.pa.BeaconBlockRoot()) {
return nil
}
if v.fc.HasNode(v.pa.BeaconBlockRoot()) {
return nil
}
log.WithFields(logFields(v.pa)).Error(ErrPayloadAttBlockRootNotSeen.Error())
return ErrPayloadAttBlockRootNotSeen
}
// VerifyBlockRootValid verifies if the block root is valid.
// Represents the following spec verification:
// [REJECT] The beacon block with root data.beacon_block_root passes validation.
func (v *PayloadAttMsgVerifier) VerifyBlockRootValid(badBlock func([32]byte) bool) (err error) {
defer v.record(RequireBlockRootValid, &err)
if badBlock != nil && badBlock(v.pa.BeaconBlockRoot()) {
log.WithFields(logFields(v.pa)).Error(ErrPayloadAttBlockRootInvalid.Error())
return ErrPayloadAttBlockRootInvalid
}
return nil
}
// VerifyValidatorInPTC verifies if the validator is present.
// Represents the following spec verification:
// [REJECT] The validator index is within the payload committee in get_ptc(state, data.slot). For the current's slot head state.
func (v *PayloadAttMsgVerifier) VerifyValidatorInPTC(ctx context.Context, st state.BeaconState) (err error) {
defer v.record(RequireValidatorInPTC, &err)
ptc, err := helpers.GetPayloadTimelinessCommittee(ctx, st, v.pa.Slot())
if err != nil {
return err
}
idx := slices.Index(ptc, v.pa.ValidatorIndex())
if idx == -1 {
log.WithFields(logFields(v.pa)).Error(ErrIncorrectPayloadAttValidator.Error())
return ErrIncorrectPayloadAttValidator
}
return nil
}
// VerifySignature verifies the signature of the payload attestation message.
// Represents the following spec verification:
// [REJECT] The signature of payload_attestation_message.signature is valid with respect to the validator index.
func (v *PayloadAttMsgVerifier) VerifySignature(st state.BeaconState) (err error) {
defer v.record(RequireSignatureValid, &err)
err = validatePayloadAttestationMessageSignature(st, v.pa)
if err != nil {
if errors.Is(err, signing.ErrSigFailedToVerify) {
log.WithFields(logFields(v.pa)).Error("signature failed to validate")
} else {
log.WithFields(logFields(v.pa)).WithError(err).Error("could not validate signature")
}
return err
}
return nil
}
// VerifiedPayloadAttestation returns a verified payload attestation message by checking all requirements.
func (v *PayloadAttMsgVerifier) VerifiedPayloadAttestation() (payloadattestation.VerifiedROMessage, error) {
if v.results.allSatisfied() {
return payloadattestation.NewVerifiedROMessage(v.pa), nil
}
return payloadattestation.VerifiedROMessage{}, ErrInvalidPayloadAttMessage
}
// SatisfyRequirement allows the caller to manually mark a requirement as satisfied.
func (v *PayloadAttMsgVerifier) SatisfyRequirement(req Requirement) {
v.record(req, nil)
}
// ValidatePayloadAttestationMessageSignature verifies the signature of a payload attestation message.
func validatePayloadAttestationMessageSignature(st state.BeaconState, payloadAtt payloadattestation.ROMessage) error {
val, err := st.ValidatorAtIndex(payloadAtt.ValidatorIndex())
if err != nil {
return err
}
pub, err := bls.PublicKeyFromBytes(val.PublicKey)
if err != nil {
return err
}
s := payloadAtt.Signature()
sig, err := bls.SignatureFromBytes(s[:])
if err != nil {
return err
}
currentEpoch := slots.ToEpoch(st.Slot())
domain, err := signing.Domain(st.Fork(), currentEpoch, params.BeaconConfig().DomainPTCAttester, st.GenesisValidatorsRoot())
if err != nil {
return err
}
root, err := payloadAtt.SigningRoot(domain)
if err != nil {
return err
}
if !sig.Verify(pub, root[:]) {
return signing.ErrSigFailedToVerify
}
return nil
}
// record records the result of a requirement verification.
func (v *PayloadAttMsgVerifier) record(req Requirement, err *error) {
if err == nil || *err == nil {
v.results.record(req, nil)
return
}
v.results.record(req, *err)
}
// logFields returns log fields for a ROMessage instance.
func logFields(payload payloadattestation.ROMessage) log.Fields {
return log.Fields{
"slot": payload.Slot(),
"validatorIndex": payload.ValidatorIndex(),
"signature": fmt.Sprintf("%#x", payload.Signature()),
"beaconBlockRoot": fmt.Sprintf("%#x", payload.BeaconBlockRoot()),
"payloadStatus": payload.PayloadStatus(),
}
}

View File

@@ -0,0 +1,51 @@
package verification
import (
"context"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
)
type MockPayloadAttestation struct {
ErrIncorrectPayloadAttSlot error
ErrIncorrectPayloadAttStatus error
ErrIncorrectPayloadAttValidator error
ErrPayloadAttBlockRootNotSeen error
ErrPayloadAttBlockRootInvalid error
ErrInvalidPayloadAttMessage error
ErrInvalidMessageSignature error
ErrUnsatisfiedRequirement error
}
var _ PayloadAttestationMsgVerifier = &MockPayloadAttestation{}
func (m *MockPayloadAttestation) VerifyCurrentSlot() error {
return m.ErrIncorrectPayloadAttSlot
}
func (m *MockPayloadAttestation) VerifyPayloadStatus() error {
return m.ErrIncorrectPayloadAttStatus
}
func (m *MockPayloadAttestation) VerifyValidatorInPTC(ctx context.Context, st state.BeaconState) error {
return m.ErrIncorrectPayloadAttValidator
}
func (m *MockPayloadAttestation) VerifyBlockRootSeen(func([32]byte) bool) error {
return m.ErrPayloadAttBlockRootNotSeen
}
func (m *MockPayloadAttestation) VerifyBlockRootValid(func([32]byte) bool) error {
return m.ErrPayloadAttBlockRootInvalid
}
func (m *MockPayloadAttestation) VerifySignature(st state.BeaconState) (err error) {
return m.ErrInvalidMessageSignature
}
func (m *MockPayloadAttestation) VerifiedPayloadAttestation() (payloadattestation.VerifiedROMessage, error) {
return payloadattestation.VerifiedROMessage{}, nil
}
func (m *MockPayloadAttestation) SatisfyRequirement(req Requirement) {}

View File

@@ -0,0 +1,321 @@
package verification
import (
"context"
"strconv"
"testing"
"time"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/v5/beacon-chain/startup"
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
"github.com/prysmaticlabs/prysm/v5/config/params"
payloadattestation "github.com/prysmaticlabs/prysm/v5/consensus-types/epbs/payload-attestation"
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v5/testing/require"
"github.com/prysmaticlabs/prysm/v5/testing/util"
"github.com/prysmaticlabs/prysm/v5/time/slots"
)
func TestVerifyCurrentSlot(t *testing.T) {
now := time.Now()
// make genesis 1 slot in the past
genesis := now.Add(-1 * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second)
clock := startup.NewClock(genesis, [32]byte{}, startup.WithNower(func() time.Time { return now }))
init := Initializer{shared: &sharedResources{clock: clock}}
t.Run("incorrect slot", func(t *testing.T) {
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyCurrentSlot(), ErrIncorrectPayloadAttSlot)
require.Equal(t, true, pa.results.executed(RequireCurrentSlot))
require.Equal(t, ErrIncorrectPayloadAttSlot, pa.results.result(RequireCurrentSlot))
})
t.Run("current slot", func(t *testing.T) {
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
Slot: 1,
},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyCurrentSlot())
require.Equal(t, true, pa.results.executed(RequireCurrentSlot))
require.NoError(t, pa.results.result(RequireCurrentSlot))
})
}
func TestVerifyKnownPayloadStatus(t *testing.T) {
init := Initializer{shared: &sharedResources{clock: &startup.Clock{}}}
t.Run("unknown status", func(t *testing.T) {
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
PayloadStatus: primitives.PAYLOAD_INVALID_STATUS,
},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyPayloadStatus(), ErrIncorrectPayloadAttStatus)
require.Equal(t, true, pa.results.executed(RequireKnownPayloadStatus))
require.Equal(t, ErrIncorrectPayloadAttStatus, pa.results.result(RequireKnownPayloadStatus))
})
t.Run("known status", func(t *testing.T) {
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyPayloadStatus())
require.Equal(t, true, pa.results.executed(RequireKnownPayloadStatus))
require.NoError(t, pa.results.result(RequireKnownPayloadStatus))
})
}
func TestVerifyBlockRootSeen(t *testing.T) {
blockRoot := [32]byte{1}
fc := &mockForkchoicer{
HasNodeCB: func(parent [32]byte) bool {
return parent == blockRoot
},
}
t.Run("happy path", func(t *testing.T) {
init := Initializer{shared: &sharedResources{fc: fc}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
BeaconBlockRoot: blockRoot[:],
},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyBlockRootSeen(nil))
require.Equal(t, true, pa.results.executed(RequireBlockRootSeen))
require.NoError(t, pa.results.result(RequireBlockRootSeen))
})
t.Run("unknown block", func(t *testing.T) {
init := Initializer{shared: &sharedResources{fc: fc}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyBlockRootSeen(nil), ErrPayloadAttBlockRootNotSeen)
require.Equal(t, true, pa.results.executed(RequireBlockRootSeen))
require.Equal(t, ErrPayloadAttBlockRootNotSeen, pa.results.result(RequireBlockRootSeen))
})
t.Run("bad parent true", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
BeaconBlockRoot: blockRoot[:],
},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyBlockRootSeen(badParentCb(t, blockRoot, true)))
require.Equal(t, true, pa.results.executed(RequireBlockRootSeen))
require.NoError(t, pa.results.result(RequireBlockRootSeen))
})
t.Run("bad parent false, unknown block", func(t *testing.T) {
init := Initializer{shared: &sharedResources{fc: fc}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
BeaconBlockRoot: []byte{2},
},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyBlockRootSeen(badParentCb(t, [32]byte{2}, false)), ErrPayloadAttBlockRootNotSeen)
require.Equal(t, true, pa.results.executed(RequireBlockRootSeen))
require.Equal(t, ErrPayloadAttBlockRootNotSeen, pa.results.result(RequireBlockRootSeen))
})
}
func TestVerifyBlockRootValid(t *testing.T) {
blockRoot := [32]byte{1}
t.Run("good block", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
BeaconBlockRoot: blockRoot[:],
},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyBlockRootValid(badParentCb(t, blockRoot, false)))
require.Equal(t, true, pa.results.executed(RequireBlockRootValid))
require.NoError(t, pa.results.result(RequireBlockRootValid))
})
t.Run("bad block", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
BeaconBlockRoot: blockRoot[:],
},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyBlockRootValid(badParentCb(t, blockRoot, true)), ErrPayloadAttBlockRootInvalid)
require.Equal(t, true, pa.results.executed(RequireBlockRootValid))
require.Equal(t, ErrPayloadAttBlockRootInvalid, pa.results.result(RequireBlockRootValid))
})
}
func TestGetPayloadTimelinessCommittee(t *testing.T) {
validators := make([]*ethpb.Validator, 4*params.BeaconConfig().TargetCommitteeSize*uint64(params.BeaconConfig().SlotsPerEpoch))
validatorIndices := make([]primitives.ValidatorIndex, len(validators))
for i := 0; i < len(validators); i++ {
k := make([]byte, 48)
copy(k, strconv.Itoa(i))
validators[i] = &ethpb.Validator{
PublicKey: k,
WithdrawalCredentials: make([]byte, 32),
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
}
validatorIndices[i] = primitives.ValidatorIndex(i)
}
st, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: validators,
RandaoMixes: make([][]byte, params.BeaconConfig().EpochsPerHistoricalVector),
})
require.NoError(t, err)
slot := primitives.Slot(1)
ctx := context.Background()
ptc, err := helpers.GetPayloadTimelinessCommittee(ctx, st, slot)
require.NoError(t, err)
t.Run("in committee", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
ValidatorIndex: ptc[0],
Data: &ethpb.PayloadAttestationData{
Slot: slot,
},
Signature: make([]byte, 96),
}, init)
require.NoError(t, pa.VerifyValidatorInPTC(ctx, st))
require.Equal(t, true, pa.results.executed(RequireValidatorInPTC))
require.NoError(t, pa.results.result(RequireValidatorInPTC))
})
t.Run("not in committee", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{
Slot: slot,
},
Signature: make([]byte, 96),
}, init)
require.ErrorIs(t, pa.VerifyValidatorInPTC(ctx, st), ErrIncorrectPayloadAttValidator)
require.Equal(t, true, pa.results.executed(RequireValidatorInPTC))
require.Equal(t, ErrIncorrectPayloadAttValidator, pa.results.result(RequireValidatorInPTC))
})
}
func TestPayloadAttestationVerifySignature(t *testing.T) {
_, secretKeys, err := util.DeterministicDepositsAndKeys(2)
require.NoError(t, err)
st, err := state_native.InitializeFromProtoEpbs(&ethpb.BeaconStateEPBS{
Validators: []*ethpb.Validator{{PublicKey: secretKeys[0].PublicKey().Marshal()},
{PublicKey: secretKeys[1].PublicKey().Marshal()}},
Fork: &ethpb.Fork{
CurrentVersion: params.BeaconConfig().EPBSForkVersion,
PreviousVersion: params.BeaconConfig().ElectraForkVersion,
},
})
require.NoError(t, err)
t.Run("valid signature", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
d := &ethpb.PayloadAttestationData{
BeaconBlockRoot: bytesutil.PadTo([]byte{'a'}, 32),
Slot: 1,
PayloadStatus: primitives.PAYLOAD_WITHHELD,
}
signedBytes, err := signing.ComputeDomainAndSign(
st,
slots.ToEpoch(d.Slot),
d,
params.BeaconConfig().DomainPTCAttester,
secretKeys[0],
)
require.NoError(t, err)
sig, err := bls.SignatureFromBytes(signedBytes)
require.NoError(t, err)
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: d,
Signature: sig.Marshal(),
}, init)
require.NoError(t, pa.VerifySignature(st))
require.Equal(t, true, pa.results.executed(RequireSignatureValid))
require.NoError(t, pa.results.result(RequireSignatureValid))
})
t.Run("invalid signature", func(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
d := &ethpb.PayloadAttestationData{
BeaconBlockRoot: bytesutil.PadTo([]byte{'a'}, 32),
Slot: 1,
PayloadStatus: primitives.PAYLOAD_WITHHELD,
}
signedBytes, err := signing.ComputeDomainAndSign(
st,
slots.ToEpoch(d.Slot),
d,
params.BeaconConfig().DomainPTCAttester,
secretKeys[0],
)
require.NoError(t, err)
sig, err := bls.SignatureFromBytes(signedBytes)
require.NoError(t, err)
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
ValidatorIndex: 1,
Data: d,
Signature: sig.Marshal(),
}, init)
require.ErrorIs(t, pa.VerifySignature(st), signing.ErrSigFailedToVerify)
require.Equal(t, true, pa.results.executed(RequireSignatureValid))
require.Equal(t, signing.ErrSigFailedToVerify, pa.results.result(RequireSignatureValid))
})
}
func TestVerifiedPayloadAttestation(t *testing.T) {
init := Initializer{shared: &sharedResources{}}
pa := newPayloadAttestation(t, &ethpb.PayloadAttestationMessage{
Data: &ethpb.PayloadAttestationData{},
Signature: make([]byte, 96),
}, init)
t.Run("missing last requirement", func(t *testing.T) {
for _, requirement := range GossipPayloadAttestationMessageRequirements[:len(GossipPayloadAttestationMessageRequirements)-1] {
pa.SatisfyRequirement(requirement)
}
_, err := pa.VerifiedPayloadAttestation()
require.ErrorIs(t, err, ErrInvalidPayloadAttMessage)
})
t.Run("satisfy all the requirements", func(t *testing.T) {
for _, requirement := range GossipPayloadAttestationMessageRequirements {
pa.SatisfyRequirement(requirement)
}
_, err := pa.VerifiedPayloadAttestation()
require.NoError(t, err)
})
}
func newPayloadAttestation(t *testing.T, m *ethpb.PayloadAttestationMessage, init Initializer) *PayloadAttMsgVerifier {
ro, err := payloadattestation.NewReadOnly(m)
require.NoError(t, err)
return init.NewPayloadAttestationMsgVerifier(ro, GossipPayloadAttestationMessageRequirements)
}

View File

@@ -14,6 +14,7 @@ const (
CurrentEpochAttestationsLength = 4096 // MAX_ATTESTATIONS * SLOTS_PER_EPOCH
SlashingsLength = 8192 // EPOCHS_PER_SLASHINGS_VECTOR
SyncCommitteeLength = 512 // SYNC_COMMITTEE_SIZE
PTCSize = 512 // PTC_SIZE [New in ePBS]
RootLength = 32 // RootLength defines the byte length of a Merkle root.
BLSSignatureLength = 96 // BLSSignatureLength defines the byte length of a BLSSignature.
BLSPubkeyLength = 48 // BLSPubkeyLength defines the byte length of a BLSSignature.
@@ -28,6 +29,7 @@ const (
MaxWithdrawalsPerPayload = 16 // MaxWithdrawalsPerPayloadLength defines the maximum number of withdrawals that can be included in a payload.
MaxBlobsPerBlock = 6 // MaxBlobsPerBlock defines the maximum number of blobs with respect to consensus rule can be included in a block.
MaxBlobCommitmentsPerBlock = 4096 // MaxBlobCommitmentsPerBlock defines the theoretical limit of blobs can be included in a block.
MaxPayloadAttestationsPerBlock = 4 // MAX_PAYLOAD_ATTESTATIONS [New in ePBS]
LogMaxBlobCommitments = 12 // Log_2 of MaxBlobCommitmentsPerBlock
BlobLength = 131072 // BlobLength defines the byte length of a blob.
BlobSize = 131072 // defined to match blob.size in bazel ssz codegen

View File

@@ -14,6 +14,7 @@ const (
CurrentEpochAttestationsLength = 1024 // MAX_ATTESTATIONS * SLOTS_PER_EPOCH
SlashingsLength = 64 // EPOCHS_PER_SLASHINGS_VECTOR
SyncCommitteeLength = 32 // SYNC_COMMITTEE_SIZE
PTCSize = 32 // PTC_SIZE [New in ePBS]
RootLength = 32 // RootLength defines the byte length of a Merkle root.
BLSSignatureLength = 96 // BLSSignatureLength defines the byte length of a BLSSignature.
BLSPubkeyLength = 48 // BLSPubkeyLength defines the byte length of a BLSSignature.
@@ -28,6 +29,7 @@ const (
MaxWithdrawalsPerPayload = 4 // MaxWithdrawalsPerPayloadLength defines the maximum number of withdrawals that can be included in a payload.
MaxBlobsPerBlock = 6 // MaxBlobsPerBlock defines the maximum number of blobs with respect to consensus rule can be included in a block.
MaxBlobCommitmentsPerBlock = 16 // MaxBlobCommitmentsPerBlock defines the theoretical limit of blobs can be included in a block.
MaxPayloadAttestationsPerBlock = 4 // MAX_PAYLOAD_ATTESTATIONS [New in ePBS]
LogMaxBlobCommitments = 4 // Log_2 of MaxBlobCommitmentsPerBlock
BlobLength = 131072 // BlobLength defines the byte length of a blob.
BlobSize = 131072 // defined to match blob.size in bazel ssz codegen

View File

@@ -126,6 +126,8 @@ type BeaconChainConfig struct {
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
DomainConsolidation [4]byte `yaml:"DOMAIN_CONSOLIDATION" spec:"true"`
DomainBeaconBuilder [4]byte `yaml:"DOMAIN_BEACON_BUILDER" spec:"false"` // DomainBeaconBuilder defines the BLS signature domain used by builders [New in ePBS]
DomainPTCAttester [4]byte `yaml:"DOMAIN_PTC_ATTESTER" spec:"false"` // DomainPTCAttester defines the BLS signature domain used by PTC members [New in ePBS]
// Prysm constants.
GweiPerEth uint64 // GweiPerEth is the amount of gwei corresponding to 1 eth.
@@ -146,6 +148,7 @@ type BeaconChainConfig struct {
BeaconStateCapellaFieldCount int // BeaconStateCapellaFieldCount defines how many fields are in beacon state post upgrade to Capella.
BeaconStateDenebFieldCount int // BeaconStateDenebFieldCount defines how many fields are in beacon state post upgrade to Deneb.
BeaconStateElectraFieldCount int // BeaconStateElectraFieldCount defines how many fields are in beacon state post upgrade to Electra.
BeaconStateEpbsFieldCount int // BeaconStateEpbsFieldCount defines how many fields are in beacon state post upgrade to ePBS.
// Slasher constants.
WeakSubjectivityPeriod primitives.Epoch // WeakSubjectivityPeriod defines the time period expressed in number of epochs were proof of stake network should validate block headers and attestations for slashable events.
@@ -166,6 +169,8 @@ type BeaconChainConfig struct {
DenebForkEpoch primitives.Epoch `yaml:"DENEB_FORK_EPOCH" spec:"true"` // DenebForkEpoch is used to represent the assigned fork epoch for deneb.
ElectraForkVersion []byte `yaml:"ELECTRA_FORK_VERSION" spec:"true"` // ElectraForkVersion is used to represent the fork version for deneb.
ElectraForkEpoch primitives.Epoch `yaml:"ELECTRA_FORK_EPOCH" spec:"true"` // ElectraForkEpoch is used to represent the assigned fork epoch for deneb.
EPBSForkVersion []byte // EPBSForkVersion is used to represent the fork version for ePBS.
EPBSForkEpoch primitives.Epoch // EPBSForkEpoch is used to represent the assigned fork epoch for ePBS.
ForkVersionSchedule map[[fieldparams.VersionLength]byte]primitives.Epoch // Schedule of fork epochs by version.
ForkVersionNames map[[fieldparams.VersionLength]byte]string // Human-readable names of fork versions.
@@ -314,6 +319,7 @@ func ConfigForkVersions(b *BeaconChainConfig) map[[fieldparams.VersionLength]byt
bytesutil.ToBytes4(b.CapellaForkVersion): version.Capella,
bytesutil.ToBytes4(b.DenebForkVersion): version.Deneb,
bytesutil.ToBytes4(b.ElectraForkVersion): version.Electra,
bytesutil.ToBytes4(b.EPBSForkVersion): version.EPBS,
}
}

View File

@@ -29,6 +29,8 @@ const (
mainnetDenebForkEpoch = 269568 // March 13, 2024, 13:55:35 UTC
// Electra Fork Epoch for mainnet config
mainnetElectraForkEpoch = math.MaxUint64 // Far future / to be defined
// ePBS Fork Epoch for mainnet config.
mainnetEPBSForkEpoch = math.MaxUint64
)
var mainnetNetworkConfig = &NetworkConfig{
@@ -194,6 +196,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
BeaconStateCapellaFieldCount: 28,
BeaconStateDenebFieldCount: 28,
BeaconStateElectraFieldCount: 37,
BeaconStateEpbsFieldCount: 44,
// Slasher related values.
WeakSubjectivityPeriod: 54000,
@@ -216,6 +219,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
DenebForkEpoch: mainnetDenebForkEpoch,
ElectraForkVersion: []byte{5, 0, 0, 0},
ElectraForkEpoch: mainnetElectraForkEpoch,
EPBSForkVersion: []byte{6, 0, 0, 0},
EPBSForkEpoch: mainnetEPBSForkEpoch,
// New values introduced in Altair hard fork 1.
// Participation flag indices.
@@ -335,6 +340,7 @@ func FillTestVersions(c *BeaconChainConfig, b byte) {
c.CapellaForkVersion = make([]byte, fieldparams.VersionLength)
c.DenebForkVersion = make([]byte, fieldparams.VersionLength)
c.ElectraForkVersion = make([]byte, fieldparams.VersionLength)
c.EPBSForkVersion = make([]byte, fieldparams.VersionLength)
c.GenesisForkVersion[fieldparams.VersionLength-1] = b
c.AltairForkVersion[fieldparams.VersionLength-1] = b
@@ -342,6 +348,7 @@ func FillTestVersions(c *BeaconChainConfig, b byte) {
c.CapellaForkVersion[fieldparams.VersionLength-1] = b
c.DenebForkVersion[fieldparams.VersionLength-1] = b
c.ElectraForkVersion[fieldparams.VersionLength-1] = b
c.EPBSForkVersion[fieldparams.VersionLength-1] = b
c.GenesisForkVersion[0] = 0
c.AltairForkVersion[0] = 1
@@ -349,4 +356,5 @@ func FillTestVersions(c *BeaconChainConfig, b byte) {
c.CapellaForkVersion[0] = 3
c.DenebForkVersion[0] = 4
c.ElectraForkVersion[0] = 5
c.EPBSForkVersion[0] = 6
}

View File

@@ -7,11 +7,13 @@ go_library(
"factory.go",
"get_payload.go",
"getters.go",
"getters_epbs.go",
"kzg.go",
"proto.go",
"roblob.go",
"roblock.go",
"setters.go",
"setters_epbs.go",
"types.go",
],
importpath = "github.com/prysmaticlabs/prysm/v5/consensus-types/blocks",
@@ -41,11 +43,14 @@ go_test(
srcs = [
"execution_test.go",
"factory_test.go",
"getters_epbs_test.go",
"getters_test.go",
"kzg_test.go",
"proto_epbs_test.go",
"proto_test.go",
"roblob_test.go",
"roblock_test.go",
"setters_test.go",
],
embed = [":go_default_library"],
deps = [

Some files were not shown because too many files have changed in this diff Show More