mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-09 21:38:05 -05:00
Compare commits
35 Commits
v6.0.5-rel
...
p2p-header
| Author | SHA1 | Date | |
|---|---|---|---|
|
|
2e31e8eeaf | ||
|
|
ff6c1c159b | ||
|
|
26bf6abbd5 | ||
|
|
79f371857f | ||
|
|
7dff10cadd | ||
|
|
d594939204 | ||
|
|
a7e1d2a7a5 | ||
|
|
e41a270726 | ||
|
|
69ad52b9e3 | ||
|
|
94c15d2238 | ||
|
|
b9856b5602 | ||
|
|
0d529ce238 | ||
|
|
1f0c1f7150 | ||
|
|
aa04def0f0 | ||
|
|
bce9d7cbfa | ||
|
|
88f217f562 | ||
|
|
ba149f2968 | ||
|
|
237488c8d9 | ||
|
|
32b59713ca | ||
|
|
6340ed6bfb | ||
|
|
59264ab06b | ||
|
|
a2e16236fa | ||
|
|
f73925b242 | ||
|
|
0d64f0d093 | ||
|
|
a02e1a5e53 | ||
|
|
7124b8b23c | ||
|
|
9cd8f96acd | ||
|
|
e43c57d273 | ||
|
|
512e781a76 | ||
|
|
8bf6a6983c | ||
|
|
88bf7b2a86 | ||
|
|
1f91bb0ab4 | ||
|
|
c66a17297a | ||
|
|
6d18057cb3 | ||
|
|
d95ac2e9d4 |
@@ -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",
|
||||
@@ -180,6 +184,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//common:go_default_library",
|
||||
|
||||
@@ -118,6 +118,12 @@ type OptimisticModeFetcher interface {
|
||||
IsOptimisticForRoot(ctx context.Context, root [32]byte) (bool, error)
|
||||
}
|
||||
|
||||
// ExecutionPayloadFetcher defines a common interface that returns forkchoice
|
||||
// information about payload block hashes
|
||||
type ExecutionPayloadFetcher interface {
|
||||
HashInForkchoice([32]byte) bool
|
||||
}
|
||||
|
||||
// FinalizedCheckpt returns the latest finalized checkpoint from chain store.
|
||||
func (s *Service) FinalizedCheckpt() *ethpb.Checkpoint {
|
||||
s.cfg.ForkChoiceStore.RLock()
|
||||
@@ -399,6 +405,14 @@ func (s *Service) InForkchoice(root [32]byte) bool {
|
||||
return s.cfg.ForkChoiceStore.HasNode(root)
|
||||
}
|
||||
|
||||
// HashInForkchoice returns true if the given payload block hash is found in
|
||||
// forkchoice
|
||||
func (s *Service) HashInForkchoice(hash [32]byte) bool {
|
||||
s.cfg.ForkChoiceStore.RLock()
|
||||
defer s.cfg.ForkChoiceStore.RUnlock()
|
||||
return s.cfg.ForkChoiceStore.HasHash(hash)
|
||||
}
|
||||
|
||||
// IsOptimisticForRoot takes the root as argument instead of the current head
|
||||
// and returns true if it is optimistic.
|
||||
func (s *Service) IsOptimisticForRoot(ctx context.Context, root [32]byte) (bool, error) {
|
||||
@@ -530,6 +544,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)
|
||||
|
||||
@@ -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
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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()",
|
||||
|
||||
@@ -1,6 +1,8 @@
|
||||
package blockchain
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/async/event"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/cache"
|
||||
statefeed "github.com/prysmaticlabs/prysm/v5/beacon-chain/core/feed/state"
|
||||
@@ -69,6 +71,22 @@ 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
|
||||
}
|
||||
}
|
||||
|
||||
// WithPayloadEnvelopeCache for payload envelope cache.
|
||||
func WithPayloadEnvelopeCache(c *sync.Map) Option {
|
||||
return func(s *Service) error {
|
||||
s.cfg.PayloadEnvelopeCache = c
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
// WithPayloadIDCache for payload ID cache.
|
||||
func WithPayloadIDCache(c *cache.PayloadIDCache) Option {
|
||||
return func(s *Service) error {
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -39,6 +39,7 @@ var epochsSinceFinalityExpandCache = primitives.Epoch(4)
|
||||
// BlockReceiver interface defines the methods of chain service for receiving and processing new blocks.
|
||||
type BlockReceiver interface {
|
||||
ReceiveBlock(ctx context.Context, block interfaces.ReadOnlySignedBeaconBlock, blockRoot [32]byte, avs das.AvailabilityStore) error
|
||||
ReceiveExecutionPayloadEnvelope(ctx context.Context, env interfaces.ROExecutionPayloadEnvelope, avs das.AvailabilityStore) error
|
||||
ReceiveBlockBatch(ctx context.Context, blocks []blocks.ROBlock, avs das.AvailabilityStore) error
|
||||
HasBlock(ctx context.Context, root [32]byte) bool
|
||||
RecentBlockSlot(root [32]byte) (primitives.Slot, error)
|
||||
|
||||
149
beacon-chain/blockchain/receive_execution_payload_envelope.go
Normal file
149
beacon-chain/blockchain/receive_execution_payload_envelope.go
Normal file
@@ -0,0 +1,149 @@
|
||||
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 := envelope.BeaconBlockRoot()
|
||||
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")
|
||||
}
|
||||
|
||||
versionedHashes := envelope.VersionedHashes()
|
||||
root := envelope.BeaconBlockRoot()
|
||||
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 := e.BeaconBlockRoot()
|
||||
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 := e.BeaconBlockRoot()
|
||||
// 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
|
||||
}
|
||||
@@ -0,0 +1,99 @@
|
||||
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/blocks"
|
||||
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
)
|
||||
|
||||
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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.BeaconBlockRoot = service.originBlockRoot[:]
|
||||
e, err := blocks.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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.BeaconBlockRoot = service.originBlockRoot[:]
|
||||
e, err := blocks.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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.BeaconBlockRoot = service.originBlockRoot[:]
|
||||
e, err := blocks.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 := blocks.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))
|
||||
}
|
||||
@@ -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,8 @@ type config struct {
|
||||
ChainStartFetcher execution.ChainStartFetcher
|
||||
BeaconDB db.HeadAccessDatabase
|
||||
DepositCache cache.DepositCache
|
||||
PayloadAttestationCache *cache.PayloadAttestationCache
|
||||
PayloadEnvelopeCache *sync.Map
|
||||
PayloadIDCache *cache.PayloadIDCache
|
||||
TrackedValidatorsCache *cache.TrackedValidatorsCache
|
||||
AttPool attestations.Pool
|
||||
@@ -179,6 +182,7 @@ func NewService(ctx context.Context, opts ...Option) (*Service, error) {
|
||||
blobNotifiers: bn,
|
||||
cfg: &config{},
|
||||
blockBeingSynced: ¤tlySyncingBlock{roots: make(map[[32]byte]struct{})},
|
||||
payloadBeingSynced: ¤tlySyncingPayload{roots: make(map[[32]byte]struct{})},
|
||||
}
|
||||
for _, opt := range opts {
|
||||
if err := opt(srv); err != nil {
|
||||
@@ -544,7 +548,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
|
||||
}
|
||||
|
||||
@@ -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) {
|
||||
|
||||
@@ -3,7 +3,10 @@ load("@prysm//tools/go:def.bzl", "go_library")
|
||||
go_library(
|
||||
name = "go_default_library",
|
||||
testonly = True,
|
||||
srcs = ["mock.go"],
|
||||
srcs = [
|
||||
"mock.go",
|
||||
"mock_epbs.go",
|
||||
],
|
||||
importpath = "github.com/prysmaticlabs/prysm/v5/beacon-chain/blockchain/testing",
|
||||
visibility = [
|
||||
"//beacon-chain:__subpackages__",
|
||||
|
||||
@@ -54,6 +54,7 @@ type ChainService struct {
|
||||
DB db.Database
|
||||
State state.BeaconState
|
||||
Block interfaces.ReadOnlySignedBeaconBlock
|
||||
ExecutionPayloadEnvelope interfaces.ROExecutionPayloadEnvelope
|
||||
VerifyBlkDescendantErr error
|
||||
stateNotifier statefeed.Notifier
|
||||
BlocksReceived []interfaces.ReadOnlySignedBeaconBlock
|
||||
@@ -68,6 +69,7 @@ type ChainService struct {
|
||||
Genesis time.Time
|
||||
ForkChoiceStore forkchoice.ForkChoicer
|
||||
ReceiveBlockMockErr error
|
||||
ReceiveEnvelopeMockErr error
|
||||
OptimisticCheckRootReceived [32]byte
|
||||
FinalizedRoots map[[32]byte]bool
|
||||
OptimisticRoots map[[32]byte]bool
|
||||
|
||||
25
beacon-chain/blockchain/testing/mock_epbs.go
Normal file
25
beacon-chain/blockchain/testing/mock_epbs.go
Normal file
@@ -0,0 +1,25 @@
|
||||
package testing
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/das"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
)
|
||||
|
||||
// ReceiveExecutionPayloadEnvelope mocks the method in chain service.
|
||||
func (s *ChainService) ReceiveExecutionPayloadEnvelope(ctx context.Context, env interfaces.ROExecutionPayloadEnvelope, _ das.AvailabilityStore) error {
|
||||
if s.ReceiveBlockMockErr != nil {
|
||||
return s.ReceiveBlockMockErr
|
||||
}
|
||||
if s.State == nil {
|
||||
return ErrNilState
|
||||
}
|
||||
if s.State.Slot() == env.Slot() {
|
||||
if err := s.State.SetLatestFullSlot(s.State.Slot()); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
s.ExecutionPayloadEnvelope = env
|
||||
return nil
|
||||
}
|
||||
8
beacon-chain/cache/BUILD.bazel
vendored
8
beacon-chain/cache/BUILD.bazel
vendored
@@ -15,11 +15,13 @@ go_library(
|
||||
"doc.go",
|
||||
"error.go",
|
||||
"interfaces.go",
|
||||
"payload_attestation.go",
|
||||
"payload_id.go",
|
||||
"proposer_indices.go",
|
||||
"proposer_indices_disabled.go", # keep
|
||||
"proposer_indices_type.go",
|
||||
"registration.go",
|
||||
"signed_execution_payload_header.go",
|
||||
"skip_slot_cache.go",
|
||||
"subnet_ids.go",
|
||||
"sync_committee.go",
|
||||
@@ -42,10 +44,12 @@ 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",
|
||||
"//math:go_default_library",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//runtime/version:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common:go_default_library",
|
||||
@@ -70,10 +74,12 @@ 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",
|
||||
"registration_test.go",
|
||||
"signed_execution_payload_header_test.go",
|
||||
"skip_slot_cache_test.go",
|
||||
"subnet_ids_test.go",
|
||||
"sync_committee_head_state_test.go",
|
||||
@@ -88,11 +94,13 @@ 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",
|
||||
"//testing/require:go_default_library",
|
||||
"//testing/util:go_default_library",
|
||||
"//testing/util/random:go_default_library",
|
||||
"@com_github_ethereum_go_ethereum//common/hexutil:go_default_library",
|
||||
"@com_github_google_gofuzz//:go_default_library",
|
||||
"@com_github_hashicorp_golang_lru//:go_default_library",
|
||||
|
||||
132
beacon-chain/cache/payload_attestation.go
vendored
Normal file
132
beacon-chain/cache/payload_attestation.go
vendored
Normal file
@@ -0,0 +1,132 @@
|
||||
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 := ð.PayloadAttestationData{
|
||||
BeaconBlockRoot: bytesutil.SafeCopyBytes(att.Data.BeaconBlockRoot),
|
||||
Slot: att.Data.Slot,
|
||||
PayloadStatus: att.Data.PayloadStatus,
|
||||
}
|
||||
return ð.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
|
||||
}
|
||||
|
||||
// Get returns the aggregated PayloadAttestation for the given root and status
|
||||
// if the root doesn't exist or status is invalid, the function returns nil.
|
||||
func (p *PayloadAttestationCache) Get(root [32]byte, status primitives.PTCStatus) *eth.PayloadAttestation {
|
||||
p.Lock()
|
||||
defer p.Unlock()
|
||||
|
||||
if p.root != root {
|
||||
return nil
|
||||
}
|
||||
if status >= primitives.PAYLOAD_INVALID_STATUS {
|
||||
return nil
|
||||
}
|
||||
|
||||
return eth.CopyPayloadAttestation(p.attestations[status])
|
||||
}
|
||||
|
||||
// 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{}
|
||||
}
|
||||
143
beacon-chain/cache/payload_attestation_test.go
vendored
Normal file
143
beacon-chain/cache/payload_attestation_test.go
vendored
Normal file
@@ -0,0 +1,143 @@
|
||||
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 := ð.PayloadAttestationMessage{
|
||||
Signature: bls.NewAggregateSignature().Marshal(),
|
||||
Data: ð.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 := ð.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 := ð.PayloadAttestationMessage{
|
||||
Signature: bls.NewAggregateSignature().Marshal(),
|
||||
Data: ð.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)
|
||||
}
|
||||
|
||||
func TestPayloadAttestationCache_Get(t *testing.T) {
|
||||
root := [32]byte{1, 2, 3}
|
||||
wrongRoot := [32]byte{4, 5, 6}
|
||||
status := primitives.PAYLOAD_PRESENT
|
||||
invalidStatus := primitives.PAYLOAD_INVALID_STATUS
|
||||
|
||||
cache := &PayloadAttestationCache{
|
||||
root: root,
|
||||
attestations: [primitives.PAYLOAD_INVALID_STATUS]*eth.PayloadAttestation{
|
||||
{
|
||||
Signature: []byte{1},
|
||||
},
|
||||
{
|
||||
Signature: []byte{2},
|
||||
},
|
||||
{
|
||||
Signature: []byte{3},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
t.Run("valid root and status", func(t *testing.T) {
|
||||
result := cache.Get(root, status)
|
||||
require.NotNil(t, result, "Expected a non-nil result")
|
||||
require.DeepEqual(t, cache.attestations[status], result)
|
||||
})
|
||||
|
||||
t.Run("invalid root", func(t *testing.T) {
|
||||
result := cache.Get(wrongRoot, status)
|
||||
require.IsNil(t, result)
|
||||
})
|
||||
|
||||
t.Run("status out of bound", func(t *testing.T) {
|
||||
result := cache.Get(root, invalidStatus)
|
||||
require.IsNil(t, result)
|
||||
})
|
||||
|
||||
t.Run("no attestation", func(t *testing.T) {
|
||||
emptyCache := &PayloadAttestationCache{
|
||||
root: root,
|
||||
attestations: [primitives.PAYLOAD_INVALID_STATUS]*eth.PayloadAttestation{},
|
||||
}
|
||||
|
||||
result := emptyCache.Get(root, status)
|
||||
require.IsNil(t, result)
|
||||
})
|
||||
}
|
||||
25
beacon-chain/cache/signed_execution_payload_header.go
vendored
Normal file
25
beacon-chain/cache/signed_execution_payload_header.go
vendored
Normal file
@@ -0,0 +1,25 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
enginev1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
|
||||
eth "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
)
|
||||
|
||||
var (
|
||||
mu sync.RWMutex
|
||||
cachedSignedExecutionPayloadHeader *enginev1.SignedExecutionPayloadHeader
|
||||
)
|
||||
|
||||
func SaveSignedExecutionPayloadHeader(header *enginev1.SignedExecutionPayloadHeader) {
|
||||
mu.Lock()
|
||||
defer mu.Unlock()
|
||||
cachedSignedExecutionPayloadHeader = header
|
||||
}
|
||||
|
||||
func SignedExecutionPayloadHeader() *enginev1.SignedExecutionPayloadHeader {
|
||||
mu.RLock()
|
||||
defer mu.RUnlock()
|
||||
return eth.CopySignedExecutionPayloadHeader(cachedSignedExecutionPayloadHeader)
|
||||
}
|
||||
16
beacon-chain/cache/signed_execution_payload_header_test.go
vendored
Normal file
16
beacon-chain/cache/signed_execution_payload_header_test.go
vendored
Normal file
@@ -0,0 +1,16 @@
|
||||
package cache
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
)
|
||||
|
||||
func TestSignedExecutionPayloadHeader(t *testing.T) {
|
||||
require.IsNil(t, SignedExecutionPayloadHeader())
|
||||
|
||||
h := random.SignedExecutionPayloadHeader(t)
|
||||
SaveSignedExecutionPayloadHeader(h)
|
||||
require.DeepEqual(t, h, SignedExecutionPayloadHeader())
|
||||
}
|
||||
@@ -107,6 +107,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_google_gofuzz//:go_default_library",
|
||||
"@com_github_prysmaticlabs_go_bitfield//:go_default_library",
|
||||
|
||||
@@ -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(ðpb.SignedBeaconBlockEpbs{
|
||||
Block: ðpb.BeaconBlockEpbs{
|
||||
ParentRoot: params.BeaconConfig().ZeroHash[:],
|
||||
StateRoot: root[:],
|
||||
Body: ðpb.BeaconBlockBodyEpbs{
|
||||
RandaoReveal: make([]byte, 96),
|
||||
Eth1Data: ðpb.Eth1Data{
|
||||
DepositRoot: make([]byte, 32),
|
||||
BlockHash: make([]byte, 32),
|
||||
},
|
||||
Graffiti: make([]byte, 32),
|
||||
SyncAggregate: ðpb.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
|
||||
}
|
||||
|
||||
@@ -239,12 +239,47 @@ func verifyBlobCommitmentCount(body interfaces.ReadOnlyBeaconBlockBody) error {
|
||||
// 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
|
||||
}
|
||||
payload, err := header.Header()
|
||||
if err != nil {
|
||||
return payloadHash, err
|
||||
}
|
||||
return payload.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
|
||||
}
|
||||
payload, err := header.Header()
|
||||
if err != nil {
|
||||
return parentHash, err
|
||||
}
|
||||
return payload.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
|
||||
}
|
||||
|
||||
@@ -14,8 +14,8 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v5/crypto/bls"
|
||||
"github.com/prysmaticlabs/prysm/v5/crypto/hash"
|
||||
"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"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
)
|
||||
@@ -115,15 +115,97 @@ func ValidateBLSToExecutionChange(st state.ReadOnlyBeaconState, signed *ethpb.Si
|
||||
return val, nil
|
||||
}
|
||||
|
||||
func checkWithdrawalsAgainstPayload(
|
||||
executionData interfaces.ExecutionData,
|
||||
numExpected int,
|
||||
expectedRoot [32]byte,
|
||||
) error {
|
||||
var wdRoot [32]byte
|
||||
if executionData.IsBlinded() {
|
||||
r, err := executionData.WithdrawalsRoot()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not get withdrawals root")
|
||||
}
|
||||
copy(wdRoot[:], r)
|
||||
} else {
|
||||
wds, err := executionData.Withdrawals()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not get withdrawals")
|
||||
}
|
||||
|
||||
if len(wds) != numExpected {
|
||||
return fmt.Errorf("execution payload header has %d withdrawals when %d were expected", len(wds), numExpected)
|
||||
}
|
||||
|
||||
wdRoot, err = ssz.WithdrawalSliceRoot(wds, fieldparams.MaxWithdrawalsPerPayload)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not get withdrawals root")
|
||||
}
|
||||
}
|
||||
if expectedRoot != wdRoot {
|
||||
return fmt.Errorf("expected withdrawals root %#x, got %#x", expectedRoot, wdRoot)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func processWithdrawalStateTransition(
|
||||
st state.BeaconState,
|
||||
expectedWithdrawals []*enginev1.Withdrawal,
|
||||
partialWithdrawalsCount uint64,
|
||||
) (err error) {
|
||||
for _, withdrawal := range expectedWithdrawals {
|
||||
err := helpers.DecreaseBalance(st, withdrawal.ValidatorIndex, withdrawal.Amount)
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not decrease balance")
|
||||
}
|
||||
}
|
||||
if st.Version() >= version.Electra {
|
||||
if err := st.DequeuePartialWithdrawals(partialWithdrawalsCount); err != nil {
|
||||
return fmt.Errorf("unable to dequeue partial withdrawals from state: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
if len(expectedWithdrawals) > 0 {
|
||||
if err := st.SetNextWithdrawalIndex(expectedWithdrawals[len(expectedWithdrawals)-1].Index + 1); err != nil {
|
||||
return errors.Wrap(err, "could not set next withdrawal index")
|
||||
}
|
||||
}
|
||||
var nextValidatorIndex primitives.ValidatorIndex
|
||||
if uint64(len(expectedWithdrawals)) < params.BeaconConfig().MaxWithdrawalsPerPayload {
|
||||
nextValidatorIndex, err = st.NextWithdrawalValidatorIndex()
|
||||
if err != nil {
|
||||
return errors.Wrap(err, "could not get next withdrawal validator index")
|
||||
}
|
||||
nextValidatorIndex += primitives.ValidatorIndex(params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep)
|
||||
nextValidatorIndex = nextValidatorIndex % primitives.ValidatorIndex(st.NumValidators())
|
||||
} else {
|
||||
nextValidatorIndex = expectedWithdrawals[len(expectedWithdrawals)-1].ValidatorIndex + 1
|
||||
if nextValidatorIndex == primitives.ValidatorIndex(st.NumValidators()) {
|
||||
nextValidatorIndex = 0
|
||||
}
|
||||
}
|
||||
if err := st.SetNextWithdrawalValidatorIndex(nextValidatorIndex); err != nil {
|
||||
return errors.Wrap(err, "could not set next withdrawal validator index")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// ProcessWithdrawals processes the validator withdrawals from the provided execution payload
|
||||
// into the beacon state.
|
||||
//
|
||||
// Spec pseudocode definition:
|
||||
//
|
||||
// def process_withdrawals(state: BeaconState, payload: ExecutionPayload) -> None:
|
||||
// if state.fork.current_version >= EIP7732_FORK_VERSION :
|
||||
// if not is_parent_block_full(state): # [New in EIP-7732]
|
||||
// return
|
||||
//
|
||||
// expected_withdrawals, partial_withdrawals_count = get_expected_withdrawals(state) # [Modified in Electra:EIP7251]
|
||||
//
|
||||
// assert len(payload.withdrawals) == len(expected_withdrawals)
|
||||
// if state.fork.current_version >= EIP7732_FORK_VERSION :
|
||||
// state.latest_withdrawals_root = hash_tree_root(expected_withdrawals) # [New in EIP-7732]
|
||||
// else :
|
||||
// assert len(payload.withdrawals) == len(expected_withdrawals)
|
||||
//
|
||||
// for expected_withdrawal, withdrawal in zip(expected_withdrawals, payload.withdrawals):
|
||||
// assert withdrawal == expected_withdrawal
|
||||
@@ -148,76 +230,39 @@ func ValidateBLSToExecutionChange(st state.ReadOnlyBeaconState, signed *ethpb.Si
|
||||
// next_validator_index = ValidatorIndex(next_index % len(state.validators))
|
||||
// state.next_withdrawal_validator_index = next_validator_index
|
||||
func ProcessWithdrawals(st state.BeaconState, executionData interfaces.ExecutionData) (state.BeaconState, error) {
|
||||
if st.Version() >= version.EPBS {
|
||||
IsParentBlockFull, err := st.IsParentBlockFull()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not check if parent block is full")
|
||||
}
|
||||
|
||||
if !IsParentBlockFull {
|
||||
return nil, nil
|
||||
}
|
||||
}
|
||||
|
||||
expectedWithdrawals, partialWithdrawalsCount, err := st.ExpectedWithdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get expected withdrawals")
|
||||
}
|
||||
|
||||
var wdRoot [32]byte
|
||||
if executionData.IsBlinded() {
|
||||
r, err := executionData.WithdrawalsRoot()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get withdrawals root")
|
||||
}
|
||||
wdRoot = bytesutil.ToBytes32(r)
|
||||
} else {
|
||||
wds, err := executionData.Withdrawals()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get withdrawals")
|
||||
}
|
||||
|
||||
if len(wds) != len(expectedWithdrawals) {
|
||||
return nil, fmt.Errorf("execution payload header has %d withdrawals when %d were expected", len(wds), len(expectedWithdrawals))
|
||||
}
|
||||
|
||||
wdRoot, err = ssz.WithdrawalSliceRoot(wds, fieldparams.MaxWithdrawalsPerPayload)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get withdrawals root")
|
||||
}
|
||||
}
|
||||
|
||||
expectedRoot, err := ssz.WithdrawalSliceRoot(expectedWithdrawals, fieldparams.MaxWithdrawalsPerPayload)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get expected withdrawals root")
|
||||
}
|
||||
if expectedRoot != wdRoot {
|
||||
return nil, fmt.Errorf("expected withdrawals root %#x, got %#x", expectedRoot, wdRoot)
|
||||
}
|
||||
|
||||
for _, withdrawal := range expectedWithdrawals {
|
||||
err := helpers.DecreaseBalance(st, withdrawal.ValidatorIndex, withdrawal.Amount)
|
||||
if st.Version() >= version.EPBS {
|
||||
err = st.SetLastWithdrawalsRoot(expectedRoot[:])
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not decrease balance")
|
||||
return nil, errors.Wrap(err, "could not set withdrawals root")
|
||||
}
|
||||
}
|
||||
|
||||
if st.Version() >= version.Electra {
|
||||
if err := st.DequeuePartialWithdrawals(partialWithdrawalsCount); err != nil {
|
||||
return nil, fmt.Errorf("unable to dequeue partial withdrawals from state: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
if len(expectedWithdrawals) > 0 {
|
||||
if err := st.SetNextWithdrawalIndex(expectedWithdrawals[len(expectedWithdrawals)-1].Index + 1); err != nil {
|
||||
return nil, errors.Wrap(err, "could not set next withdrawal index")
|
||||
}
|
||||
}
|
||||
var nextValidatorIndex primitives.ValidatorIndex
|
||||
if uint64(len(expectedWithdrawals)) < params.BeaconConfig().MaxWithdrawalsPerPayload {
|
||||
nextValidatorIndex, err = st.NextWithdrawalValidatorIndex()
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "could not get next withdrawal validator index")
|
||||
}
|
||||
nextValidatorIndex += primitives.ValidatorIndex(params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep)
|
||||
nextValidatorIndex = nextValidatorIndex % primitives.ValidatorIndex(st.NumValidators())
|
||||
} else {
|
||||
nextValidatorIndex = expectedWithdrawals[len(expectedWithdrawals)-1].ValidatorIndex + 1
|
||||
if nextValidatorIndex == primitives.ValidatorIndex(st.NumValidators()) {
|
||||
nextValidatorIndex = 0
|
||||
if err := checkWithdrawalsAgainstPayload(executionData, len(expectedWithdrawals), expectedRoot); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
if err := st.SetNextWithdrawalValidatorIndex(nextValidatorIndex); err != nil {
|
||||
return nil, errors.Wrap(err, "could not set next withdrawal validator index")
|
||||
if err := processWithdrawalStateTransition(st, expectedWithdrawals, partialWithdrawalsCount); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return st, nil
|
||||
}
|
||||
|
||||
@@ -22,6 +22,7 @@ import (
|
||||
ethpb "github.com/prysmaticlabs/prysm/v5/proto/prysm/v1alpha1"
|
||||
"github.com/prysmaticlabs/prysm/v5/runtime/version"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
"github.com/prysmaticlabs/prysm/v5/time/slots"
|
||||
)
|
||||
|
||||
@@ -1132,13 +1133,407 @@ func TestProcessWithdrawals(t *testing.T) {
|
||||
checkPostState(t, test.Control, post)
|
||||
}
|
||||
params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep = saved
|
||||
|
||||
})
|
||||
}
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestProcessWithdrawalsEPBS(t *testing.T) {
|
||||
const (
|
||||
currentEpoch = primitives.Epoch(10)
|
||||
epochInFuture = primitives.Epoch(12)
|
||||
epochInPast = primitives.Epoch(8)
|
||||
numValidators = 128
|
||||
notWithdrawableIndex = 127
|
||||
notPartiallyWithdrawable = 126
|
||||
maxSweep = uint64(80)
|
||||
)
|
||||
maxEffectiveBalance := params.BeaconConfig().MaxEffectiveBalance
|
||||
|
||||
type args struct {
|
||||
Name string
|
||||
NextWithdrawalValidatorIndex primitives.ValidatorIndex
|
||||
NextWithdrawalIndex uint64
|
||||
FullWithdrawalIndices []primitives.ValidatorIndex
|
||||
PendingPartialWithdrawalIndices []primitives.ValidatorIndex
|
||||
Withdrawals []*enginev1.Withdrawal
|
||||
PendingPartialWithdrawals []*ethpb.PendingPartialWithdrawal // Electra
|
||||
LatestBlockHash []byte // EIP-7732
|
||||
}
|
||||
type control struct {
|
||||
NextWithdrawalValidatorIndex primitives.ValidatorIndex
|
||||
NextWithdrawalIndex uint64
|
||||
Balances map[uint64]uint64
|
||||
}
|
||||
type Test struct {
|
||||
Args args
|
||||
Control control
|
||||
}
|
||||
executionAddress := func(i primitives.ValidatorIndex) []byte {
|
||||
wc := make([]byte, 20)
|
||||
wc[19] = byte(i)
|
||||
return wc
|
||||
}
|
||||
withdrawalAmount := func(i primitives.ValidatorIndex) uint64 {
|
||||
return maxEffectiveBalance + uint64(i)*100000
|
||||
}
|
||||
fullWithdrawal := func(i primitives.ValidatorIndex, idx uint64) *enginev1.Withdrawal {
|
||||
return &enginev1.Withdrawal{
|
||||
Index: idx,
|
||||
ValidatorIndex: i,
|
||||
Address: executionAddress(i),
|
||||
Amount: withdrawalAmount(i),
|
||||
}
|
||||
}
|
||||
PendingPartialWithdrawal := func(i primitives.ValidatorIndex, idx uint64) *enginev1.Withdrawal {
|
||||
return &enginev1.Withdrawal{
|
||||
Index: idx,
|
||||
ValidatorIndex: i,
|
||||
Address: executionAddress(i),
|
||||
Amount: withdrawalAmount(i) - maxEffectiveBalance,
|
||||
}
|
||||
}
|
||||
tests := []Test{
|
||||
{
|
||||
Args: args{
|
||||
Name: "success no withdrawals",
|
||||
NextWithdrawalValidatorIndex: 10,
|
||||
NextWithdrawalIndex: 3,
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 90,
|
||||
NextWithdrawalIndex: 3,
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success one full withdrawal",
|
||||
NextWithdrawalIndex: 3,
|
||||
NextWithdrawalValidatorIndex: 5,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{70},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(70, 3),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 85,
|
||||
NextWithdrawalIndex: 4,
|
||||
Balances: map[uint64]uint64{70: 0},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success one partial withdrawal",
|
||||
NextWithdrawalIndex: 21,
|
||||
NextWithdrawalValidatorIndex: 120,
|
||||
PendingPartialWithdrawalIndices: []primitives.ValidatorIndex{7},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
PendingPartialWithdrawal(7, 21),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 72,
|
||||
NextWithdrawalIndex: 22,
|
||||
Balances: map[uint64]uint64{7: maxEffectiveBalance},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success many full withdrawals",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 4,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{7, 19, 28, 1},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(7, 22), fullWithdrawal(19, 23), fullWithdrawal(28, 24),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 84,
|
||||
NextWithdrawalIndex: 25,
|
||||
Balances: map[uint64]uint64{7: 0, 19: 0, 28: 0},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "less than max sweep at end",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 4,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{80, 81, 82, 83},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(80, 22), fullWithdrawal(81, 23), fullWithdrawal(82, 24),
|
||||
fullWithdrawal(83, 25),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 84,
|
||||
NextWithdrawalIndex: 26,
|
||||
Balances: map[uint64]uint64{80: 0, 81: 0, 82: 0, 83: 0},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "less than max sweep and beginning",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 4,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{4, 5, 6},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(4, 22), fullWithdrawal(5, 23), fullWithdrawal(6, 24),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 84,
|
||||
NextWithdrawalIndex: 25,
|
||||
Balances: map[uint64]uint64{4: 0, 5: 0, 6: 0},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success many partial withdrawals",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 4,
|
||||
PendingPartialWithdrawalIndices: []primitives.ValidatorIndex{7, 19, 28},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
PendingPartialWithdrawal(7, 22), PendingPartialWithdrawal(19, 23), PendingPartialWithdrawal(28, 24),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 84,
|
||||
NextWithdrawalIndex: 25,
|
||||
Balances: map[uint64]uint64{
|
||||
7: maxEffectiveBalance,
|
||||
19: maxEffectiveBalance,
|
||||
28: maxEffectiveBalance,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success many withdrawals",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 88,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{7, 19, 28},
|
||||
PendingPartialWithdrawalIndices: []primitives.ValidatorIndex{2, 1, 89, 15},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
PendingPartialWithdrawal(89, 22), PendingPartialWithdrawal(1, 23), PendingPartialWithdrawal(2, 24),
|
||||
fullWithdrawal(7, 25), PendingPartialWithdrawal(15, 26), fullWithdrawal(19, 27),
|
||||
fullWithdrawal(28, 28),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 40,
|
||||
NextWithdrawalIndex: 29,
|
||||
Balances: map[uint64]uint64{
|
||||
7: 0, 19: 0, 28: 0,
|
||||
2: maxEffectiveBalance, 1: maxEffectiveBalance, 89: maxEffectiveBalance,
|
||||
15: maxEffectiveBalance,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success many withdrawals with pending partial withdrawals in state",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 88,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{7, 19, 28},
|
||||
PendingPartialWithdrawalIndices: []primitives.ValidatorIndex{2, 1, 89, 15},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
PendingPartialWithdrawal(89, 22), PendingPartialWithdrawal(1, 23), PendingPartialWithdrawal(2, 24),
|
||||
fullWithdrawal(7, 25), PendingPartialWithdrawal(15, 26), fullWithdrawal(19, 27),
|
||||
fullWithdrawal(28, 28),
|
||||
},
|
||||
PendingPartialWithdrawals: []*ethpb.PendingPartialWithdrawal{
|
||||
{
|
||||
Index: 11,
|
||||
Amount: withdrawalAmount(11) - maxEffectiveBalance,
|
||||
},
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 40,
|
||||
NextWithdrawalIndex: 29,
|
||||
Balances: map[uint64]uint64{
|
||||
7: 0, 19: 0, 28: 0,
|
||||
2: maxEffectiveBalance, 1: maxEffectiveBalance, 89: maxEffectiveBalance,
|
||||
15: maxEffectiveBalance,
|
||||
},
|
||||
},
|
||||
},
|
||||
|
||||
{
|
||||
Args: args{
|
||||
Name: "success more than max fully withdrawals",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 0,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{1, 2, 3, 4, 5, 6, 7, 8, 9, 21, 22, 23, 24, 25, 26, 27, 29, 35, 89},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(1, 22), fullWithdrawal(2, 23), fullWithdrawal(3, 24),
|
||||
fullWithdrawal(4, 25), fullWithdrawal(5, 26), fullWithdrawal(6, 27),
|
||||
fullWithdrawal(7, 28), fullWithdrawal(8, 29), fullWithdrawal(9, 30),
|
||||
fullWithdrawal(21, 31), fullWithdrawal(22, 32), fullWithdrawal(23, 33),
|
||||
fullWithdrawal(24, 34), fullWithdrawal(25, 35), fullWithdrawal(26, 36),
|
||||
fullWithdrawal(27, 37),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 28,
|
||||
NextWithdrawalIndex: 38,
|
||||
Balances: map[uint64]uint64{
|
||||
1: 0, 2: 0, 3: 0, 4: 0, 5: 0, 6: 0, 7: 0, 8: 0, 9: 0,
|
||||
21: 0, 22: 0, 23: 0, 24: 0, 25: 0, 26: 0, 27: 0,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "success more than max partially withdrawals",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 0,
|
||||
PendingPartialWithdrawalIndices: []primitives.ValidatorIndex{1, 2, 3, 4, 5, 6, 7, 8, 9, 21, 22, 23, 24, 25, 26, 27, 29, 35, 89},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
PendingPartialWithdrawal(1, 22), PendingPartialWithdrawal(2, 23), PendingPartialWithdrawal(3, 24),
|
||||
PendingPartialWithdrawal(4, 25), PendingPartialWithdrawal(5, 26), PendingPartialWithdrawal(6, 27),
|
||||
PendingPartialWithdrawal(7, 28), PendingPartialWithdrawal(8, 29), PendingPartialWithdrawal(9, 30),
|
||||
PendingPartialWithdrawal(21, 31), PendingPartialWithdrawal(22, 32), PendingPartialWithdrawal(23, 33),
|
||||
PendingPartialWithdrawal(24, 34), PendingPartialWithdrawal(25, 35), PendingPartialWithdrawal(26, 36),
|
||||
PendingPartialWithdrawal(27, 37),
|
||||
},
|
||||
},
|
||||
Control: control{
|
||||
NextWithdrawalValidatorIndex: 28,
|
||||
NextWithdrawalIndex: 38,
|
||||
Balances: map[uint64]uint64{
|
||||
1: maxEffectiveBalance,
|
||||
2: maxEffectiveBalance,
|
||||
3: maxEffectiveBalance,
|
||||
4: maxEffectiveBalance,
|
||||
5: maxEffectiveBalance,
|
||||
6: maxEffectiveBalance,
|
||||
7: maxEffectiveBalance,
|
||||
8: maxEffectiveBalance,
|
||||
9: maxEffectiveBalance,
|
||||
21: maxEffectiveBalance,
|
||||
22: maxEffectiveBalance,
|
||||
23: maxEffectiveBalance,
|
||||
24: maxEffectiveBalance,
|
||||
25: maxEffectiveBalance,
|
||||
26: maxEffectiveBalance,
|
||||
27: maxEffectiveBalance,
|
||||
},
|
||||
},
|
||||
},
|
||||
{
|
||||
Args: args{
|
||||
Name: "Parent Node is not full",
|
||||
NextWithdrawalIndex: 22,
|
||||
NextWithdrawalValidatorIndex: 4,
|
||||
FullWithdrawalIndices: []primitives.ValidatorIndex{7, 19, 28, 1},
|
||||
Withdrawals: []*enginev1.Withdrawal{
|
||||
fullWithdrawal(7, 22), fullWithdrawal(19, 23), fullWithdrawal(28, 24),
|
||||
},
|
||||
LatestBlockHash: []byte{1, 2, 3},
|
||||
},
|
||||
},
|
||||
}
|
||||
|
||||
checkPostState := func(t *testing.T, expected control, st state.BeaconState) {
|
||||
l, err := st.NextWithdrawalValidatorIndex()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expected.NextWithdrawalValidatorIndex, l)
|
||||
|
||||
n, err := st.NextWithdrawalIndex()
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expected.NextWithdrawalIndex, n)
|
||||
balances := st.Balances()
|
||||
for idx, bal := range expected.Balances {
|
||||
require.Equal(t, bal, balances[idx])
|
||||
}
|
||||
}
|
||||
|
||||
prepareValidators := func(st state.BeaconState, arguments args) error {
|
||||
validators := make([]*ethpb.Validator, numValidators)
|
||||
if err := st.SetBalances(make([]uint64, numValidators)); err != nil {
|
||||
return err
|
||||
}
|
||||
for i := range validators {
|
||||
v := ðpb.Validator{}
|
||||
v.EffectiveBalance = maxEffectiveBalance
|
||||
v.WithdrawableEpoch = epochInFuture
|
||||
v.WithdrawalCredentials = make([]byte, 32)
|
||||
v.WithdrawalCredentials[31] = byte(i)
|
||||
if err := st.UpdateBalancesAtIndex(primitives.ValidatorIndex(i), v.EffectiveBalance-uint64(rand.Intn(1000))); err != nil {
|
||||
return err
|
||||
}
|
||||
validators[i] = v
|
||||
}
|
||||
for _, idx := range arguments.FullWithdrawalIndices {
|
||||
if idx != notWithdrawableIndex {
|
||||
validators[idx].WithdrawableEpoch = epochInPast
|
||||
}
|
||||
if err := st.UpdateBalancesAtIndex(idx, withdrawalAmount(idx)); err != nil {
|
||||
return err
|
||||
}
|
||||
validators[idx].WithdrawalCredentials[0] = params.BeaconConfig().ETH1AddressWithdrawalPrefixByte
|
||||
}
|
||||
for _, idx := range arguments.PendingPartialWithdrawalIndices {
|
||||
validators[idx].WithdrawalCredentials[0] = params.BeaconConfig().ETH1AddressWithdrawalPrefixByte
|
||||
if err := st.UpdateBalancesAtIndex(idx, withdrawalAmount(idx)); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
return st.SetValidators(validators)
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
t.Run(test.Args.Name, func(t *testing.T) {
|
||||
saved := params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep
|
||||
params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep = maxSweep
|
||||
if test.Args.Withdrawals == nil {
|
||||
test.Args.Withdrawals = make([]*enginev1.Withdrawal, 0)
|
||||
}
|
||||
if test.Args.FullWithdrawalIndices == nil {
|
||||
test.Args.FullWithdrawalIndices = make([]primitives.ValidatorIndex, 0)
|
||||
}
|
||||
if test.Args.PendingPartialWithdrawalIndices == nil {
|
||||
test.Args.PendingPartialWithdrawalIndices = make([]primitives.ValidatorIndex, 0)
|
||||
}
|
||||
slot, err := slots.EpochStart(currentEpoch)
|
||||
require.NoError(t, err)
|
||||
var st state.BeaconState
|
||||
var p interfaces.ExecutionData
|
||||
spb := ðpb.BeaconStateEPBS{
|
||||
Slot: slot,
|
||||
NextWithdrawalValidatorIndex: test.Args.NextWithdrawalValidatorIndex,
|
||||
NextWithdrawalIndex: test.Args.NextWithdrawalIndex,
|
||||
PendingPartialWithdrawals: test.Args.PendingPartialWithdrawals,
|
||||
LatestExecutionPayloadHeader: &enginev1.ExecutionPayloadHeaderEPBS{
|
||||
BlockHash: []byte{},
|
||||
},
|
||||
LatestBlockHash: test.Args.LatestBlockHash,
|
||||
}
|
||||
st, err = state_native.InitializeFromProtoUnsafeEpbs(spb)
|
||||
require.NoError(t, err)
|
||||
env := random.ExecutionPayloadEnvelope(t)
|
||||
env.Payload.Withdrawals = test.Args.Withdrawals
|
||||
wp, err := consensusblocks.WrappedROExecutionPayloadEnvelope(env)
|
||||
require.NoError(t, err)
|
||||
p, err = wp.Execution()
|
||||
require.NoError(t, err)
|
||||
err = prepareValidators(st, test.Args)
|
||||
require.NoError(t, err)
|
||||
post, err := blocks.ProcessWithdrawals(st, p)
|
||||
if test.Args.Name == "Parent Node is not full" {
|
||||
require.IsNil(t, post)
|
||||
require.IsNil(t, err)
|
||||
} else {
|
||||
require.NoError(t, err)
|
||||
checkPostState(t, test.Control, post)
|
||||
}
|
||||
params.BeaconConfig().MaxValidatorsPerWithdrawalsSweep = saved
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
func TestProcessBLSToExecutionChanges(t *testing.T) {
|
||||
spb := ðpb.BeaconStateCapella{
|
||||
Fork: ðpb.Fork{
|
||||
|
||||
38
beacon-chain/core/epbs/BUILD.bazel
Normal file
38
beacon-chain/core/epbs/BUILD.bazel
Normal file
@@ -0,0 +1,38 @@
|
||||
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/blocks: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",
|
||||
"//testing/util/random:go_default_library",
|
||||
],
|
||||
)
|
||||
13
beacon-chain/core/epbs/attestation.go
Normal file
13
beacon-chain/core/epbs/attestation.go
Normal 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
|
||||
}
|
||||
93
beacon-chain/core/epbs/attestation_test.go
Normal file
93
beacon-chain/core/epbs/attestation_test.go
Normal 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)
|
||||
}
|
||||
129
beacon-chain/core/epbs/execution_payload_envelope.go
Normal file
129
beacon-chain/core/epbs/execution_payload_envelope.go
Normal file
@@ -0,0 +1,129 @@
|
||||
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 := envelope.BeaconBlockRoot()
|
||||
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 := envelope.BuilderIndex()
|
||||
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 := envelope.StateRoot()
|
||||
if stateRoot != envelopeStateRoot {
|
||||
return errors.New("state root mismatch")
|
||||
}
|
||||
return nil
|
||||
}
|
||||
108
beacon-chain/core/epbs/execution_payload_envelope_test.go
Normal file
108
beacon-chain/core/epbs/execution_payload_envelope_test.go
Normal file
@@ -0,0 +1,108 @@
|
||||
package epbs
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
state_native "github.com/prysmaticlabs/prysm/v5/beacon-chain/state/state-native"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
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"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
)
|
||||
|
||||
func TestProcessPayloadStateTransition(t *testing.T) {
|
||||
bh := [32]byte{'h'}
|
||||
p := random.ExecutionPayloadEnvelope(t)
|
||||
p.Payload.BlockHash = bh[:]
|
||||
e, err := blocks.WrappedROExecutionPayloadEnvelope(p)
|
||||
require.NoError(t, err)
|
||||
validators := make([]*ethpb.Validator, 0)
|
||||
stpb := ðpb.BeaconStateEPBS{Slot: 3, Validators: validators}
|
||||
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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.Payload = payload
|
||||
e, err := blocks.WrappedROExecutionPayloadEnvelope(p)
|
||||
require.NoError(t, err)
|
||||
stpb := ðpb.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)
|
||||
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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.Payload = payload
|
||||
p.BuilderIndex = 1
|
||||
e, err := blocks.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
|
||||
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 := random.ExecutionPayloadEnvelope(t)
|
||||
p.Payload = payload
|
||||
p.BuilderIndex = 1
|
||||
e, err := blocks.WrappedROExecutionPayloadEnvelope(p)
|
||||
require.NoError(t, err)
|
||||
ctx := context.Background()
|
||||
st, _ := util.DeterministicGenesisStateEpbs(t, 64)
|
||||
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))
|
||||
}
|
||||
@@ -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",
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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] = ðpb.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(ðpb.BeaconState{
|
||||
state, err := state_native.InitializeFromProtoEpbs(ðpb.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) {
|
||||
|
||||
12
beacon-chain/core/helpers/exports_test.go
Normal file
12
beacon-chain/core/helpers/exports_test.go
Normal file
@@ -0,0 +1,12 @@
|
||||
package helpers
|
||||
|
||||
var (
|
||||
ErrNilMessage = errNilMessage
|
||||
ErrNilData = errNilData
|
||||
ErrNilBeaconBlockRoot = errNilBeaconBlockRoot
|
||||
ErrNilPayloadAttestation = errNilPayloadAttestation
|
||||
ErrNilSignature = errNilSignature
|
||||
ErrNilAggregationBits = errNilAggregationBits
|
||||
ErrPreEPBSState = errPreEPBSState
|
||||
ErrCommitteeOverflow = errCommitteeOverflow
|
||||
)
|
||||
296
beacon-chain/core/helpers/payload_attestation.go
Normal file
296
beacon-chain/core/helpers/payload_attestation.go
Normal 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
|
||||
}
|
||||
363
beacon-chain/core/helpers/payload_attestation_test.go
Normal file
363
beacon-chain/core/helpers/payload_attestation_test.go
Normal 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 := ð.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 := ð.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 := ð.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] = ðpb.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] = ðpb.Validator{
|
||||
PublicKey: pubkey,
|
||||
WithdrawalCredentials: make([]byte, 32),
|
||||
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
|
||||
}
|
||||
}
|
||||
|
||||
// Create a beacon state.
|
||||
state, err := state_native.InitializeFromProtoEpbs(ðpb.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 := ð.PayloadAttestation{
|
||||
AggregationBits: aggregationBits,
|
||||
Data: ð.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] = ðpb.Validator{
|
||||
PublicKey: publicKey,
|
||||
WithdrawalCredentials: make([]byte, 32),
|
||||
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
|
||||
}
|
||||
}
|
||||
|
||||
// Create a beacon state.
|
||||
state, err := state_native.InitializeFromProtoEpbs(ðpb.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 := ð.PayloadAttestation{
|
||||
AggregationBits: aggregationBits,
|
||||
Data: ð.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] = ðpb.Validator{
|
||||
PublicKey: secretKeys[i].PublicKey().Marshal(),
|
||||
WithdrawalCredentials: make([]byte, 32),
|
||||
ExitEpoch: params.BeaconConfig().FarFutureEpoch,
|
||||
}
|
||||
}
|
||||
|
||||
// Create a beacon state.
|
||||
state, err := state_native.InitializeFromProtoEpbs(ðpb.BeaconStateEPBS{
|
||||
Validators: validators,
|
||||
Fork: ðpb.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: ð.PayloadAttestationData{
|
||||
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
},
|
||||
{
|
||||
attestation: &epbs.IndexedPayloadAttestation{
|
||||
AttestingIndices: []primitives.ValidatorIndex{13, 19},
|
||||
Data: ð.PayloadAttestationData{
|
||||
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
},
|
||||
{
|
||||
attestation: &epbs.IndexedPayloadAttestation{
|
||||
AttestingIndices: []primitives.ValidatorIndex{123, 234, 345},
|
||||
Data: ð.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: ð.PayloadAttestationData{
|
||||
BeaconBlockRoot: make([]byte, fieldparams.RootLength),
|
||||
},
|
||||
Signature: make([]byte, fieldparams.BLSSignatureLength),
|
||||
},
|
||||
},
|
||||
{
|
||||
attestation: &epbs.IndexedPayloadAttestation{
|
||||
AttestingIndices: []primitives.ValidatorIndex{5},
|
||||
Data: ð.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)
|
||||
}
|
||||
}
|
||||
@@ -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
|
||||
|
||||
@@ -23,6 +23,7 @@ import (
|
||||
type ReadOnlyDatabase interface {
|
||||
// Block related methods.
|
||||
Block(ctx context.Context, blockRoot [32]byte) (interfaces.ReadOnlySignedBeaconBlock, error)
|
||||
SignedExecutionPayloadHeader(ctx context.Context, blockRoot [32]byte) (interfaces.ROSignedExecutionPayloadHeader, error)
|
||||
Blocks(ctx context.Context, f *filters.QueryFilter) ([]interfaces.ReadOnlySignedBeaconBlock, [][32]byte, error)
|
||||
BlockRoots(ctx context.Context, f *filters.QueryFilter) ([][32]byte, error)
|
||||
BlocksBySlot(ctx context.Context, slot primitives.Slot) ([]interfaces.ReadOnlySignedBeaconBlock, error)
|
||||
@@ -53,6 +54,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 +89,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.
|
||||
|
||||
@@ -6,10 +6,12 @@ go_library(
|
||||
"archived_point.go",
|
||||
"backfill.go",
|
||||
"backup.go",
|
||||
"blind_payload_envelope.go",
|
||||
"blocks.go",
|
||||
"checkpoint.go",
|
||||
"deposit_contract.go",
|
||||
"encoding.go",
|
||||
"epbs.go",
|
||||
"error.go",
|
||||
"execution_chain.go",
|
||||
"finalized_block_roots.go",
|
||||
@@ -78,10 +80,12 @@ 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",
|
||||
"encoding_test.go",
|
||||
"epbs_test.go",
|
||||
"execution_chain_test.go",
|
||||
"finalized_block_roots_test.go",
|
||||
"genesis_test.go",
|
||||
@@ -119,6 +123,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",
|
||||
|
||||
45
beacon-chain/db/kv/blind_payload_envelope.go
Normal file
45
beacon-chain/db/kv/blind_payload_envelope.go
Normal 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 := ðpb.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
|
||||
}
|
||||
23
beacon-chain/db/kv/blind_payload_envelope_test.go
Normal file
23
beacon-chain/db/kv/blind_payload_envelope_test.go
Normal 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)
|
||||
}
|
||||
@@ -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 = ðpb.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 = ðpb.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
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -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:
|
||||
|
||||
18
beacon-chain/db/kv/epbs.go
Normal file
18
beacon-chain/db/kv/epbs.go
Normal file
@@ -0,0 +1,18 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
)
|
||||
|
||||
func (s *Store) SignedExecutionPayloadHeader(ctx context.Context, blockRoot [32]byte) (interfaces.ROSignedExecutionPayloadHeader, error) {
|
||||
b, err := s.Block(ctx, blockRoot)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if b.IsNil() {
|
||||
return nil, ErrNotFound
|
||||
}
|
||||
return b.Block().Body().SignedExecutionPayloadHeader()
|
||||
}
|
||||
28
beacon-chain/db/kv/epbs_test.go
Normal file
28
beacon-chain/db/kv/epbs_test.go
Normal file
@@ -0,0 +1,28 @@
|
||||
package kv
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
)
|
||||
|
||||
func Test_SignedExecutionPayloadHeader(t *testing.T) {
|
||||
db := setupDB(t)
|
||||
ctx := context.Background()
|
||||
|
||||
b := random.SignedBeaconBlock(t)
|
||||
blk, err := blocks.NewSignedBeaconBlock(b)
|
||||
require.NoError(t, err)
|
||||
blockRoot, err := blk.Block().HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
|
||||
require.NoError(t, db.SaveBlock(ctx, blk))
|
||||
retrievedHeader, err := db.SignedExecutionPayloadHeader(ctx, blockRoot)
|
||||
require.NoError(t, err)
|
||||
wantedHeader, err := blk.Block().Body().SignedExecutionPayloadHeader()
|
||||
require.NoError(t, err)
|
||||
require.DeepEqual(t, wantedHeader, retrievedHeader)
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
@@ -118,6 +118,9 @@ var Buckets = [][]byte{
|
||||
|
||||
feeRecipientBucket,
|
||||
registrationBucket,
|
||||
|
||||
// ePBS
|
||||
executionPayloadEnvelopeBucket,
|
||||
}
|
||||
|
||||
// KVStoreOption is a functional option that modifies a kv.Store.
|
||||
|
||||
@@ -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")
|
||||
|
||||
@@ -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 := ðpb.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 := ðpb.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")
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
|
||||
|
||||
@@ -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",
|
||||
|
||||
9
beacon-chain/forkchoice/doubly-linked-tree/epbs.go
Normal file
9
beacon-chain/forkchoice/doubly-linked-tree/epbs.go
Normal 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{}
|
||||
}
|
||||
79
beacon-chain/forkchoice/doubly-linked-tree/epbs_test.go
Normal file
79
beacon-chain/forkchoice/doubly-linked-tree/epbs_test.go
Normal 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)
|
||||
}
|
||||
@@ -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
|
||||
}
|
||||
@@ -170,6 +189,13 @@ func (f *ForkChoice) HasNode(root [32]byte) bool {
|
||||
return ok
|
||||
}
|
||||
|
||||
// HasHash returns true if the node with the given payload hash exists in fork choice store,
|
||||
// false else wise.
|
||||
func (f *ForkChoice) HasHash(hash [32]byte) bool {
|
||||
_, ok := f.store.nodeByPayload[hash]
|
||||
return ok
|
||||
}
|
||||
|
||||
// IsCanonical returns true if the given root is part of the canonical chain.
|
||||
func (f *ForkChoice) IsCanonical(root [32]byte) bool {
|
||||
node, ok := f.store.nodeByRoot[root]
|
||||
@@ -490,8 +516,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
|
||||
}
|
||||
|
||||
@@ -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,
|
||||
|
||||
@@ -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)
|
||||
|
||||
@@ -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())
|
||||
|
||||
@@ -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()
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
@@ -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.
|
||||
@@ -63,6 +63,7 @@ type FastGetter interface {
|
||||
FinalizedCheckpoint() *forkchoicetypes.Checkpoint
|
||||
FinalizedPayloadBlockHash() [32]byte
|
||||
HasNode([32]byte) bool
|
||||
HasHash([32]byte) bool
|
||||
HighestReceivedBlockSlot() primitives.Slot
|
||||
HighestReceivedBlockDelay() primitives.Slot
|
||||
IsCanonical(root [32]byte) bool
|
||||
|
||||
@@ -37,6 +37,13 @@ func (ro *ROForkChoice) HasNode(root [32]byte) bool {
|
||||
return ro.getter.HasNode(root)
|
||||
}
|
||||
|
||||
// HasHash delegates to the underlying forkchoice call, under a lock.
|
||||
func (ro *ROForkChoice) HasHash(root [32]byte) bool {
|
||||
ro.l.RLock()
|
||||
defer ro.l.RUnlock()
|
||||
return ro.getter.HasHash(root)
|
||||
}
|
||||
|
||||
// ProposerBoost delegates to the underlying forkchoice call, under a lock.
|
||||
func (ro *ROForkChoice) ProposerBoost() [fieldparams.RootLength]byte {
|
||||
ro.l.RLock()
|
||||
|
||||
@@ -18,6 +18,7 @@ const (
|
||||
rlockCalled
|
||||
runlockCalled
|
||||
hasNodeCalled
|
||||
hasHashCalled
|
||||
proposerBoostCalled
|
||||
isCanonicalCalled
|
||||
finalizedCheckpointCalled
|
||||
@@ -197,6 +198,11 @@ func (ro *mockROForkchoice) HasNode(_ [32]byte) bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (ro *mockROForkchoice) HasHash(_ [32]byte) bool {
|
||||
ro.calls = append(ro.calls, hasHashCalled)
|
||||
return false
|
||||
}
|
||||
|
||||
func (ro *mockROForkchoice) ProposerBoost() [fieldparams.RootLength]byte {
|
||||
ro.calls = append(ro.calls, proposerBoostCalled)
|
||||
return [fieldparams.RootLength]byte{}
|
||||
|
||||
@@ -102,6 +102,8 @@ type BeaconNode struct {
|
||||
blsToExecPool blstoexec.PoolManager
|
||||
depositCache cache.DepositCache
|
||||
trackedValidatorsCache *cache.TrackedValidatorsCache
|
||||
payloadAttestationCache *cache.PayloadAttestationCache
|
||||
payloadEnvelopeCache *sync.Map
|
||||
payloadIDCache *cache.PayloadIDCache
|
||||
stateFeed *event.Feed
|
||||
blockFeed *event.Feed
|
||||
@@ -152,6 +154,8 @@ func New(cliCtx *cli.Context, cancel context.CancelFunc, opts ...Option) (*Beaco
|
||||
syncCommitteePool: synccommittee.NewPool(),
|
||||
blsToExecPool: blstoexec.NewPool(),
|
||||
trackedValidatorsCache: cache.NewTrackedValidatorsCache(),
|
||||
payloadAttestationCache: &cache.PayloadAttestationCache{},
|
||||
payloadEnvelopeCache: &sync.Map{},
|
||||
payloadIDCache: cache.NewPayloadIDCache(),
|
||||
slasherBlockHeadersFeed: new(event.Feed),
|
||||
slasherAttestationsFeed: new(event.Feed),
|
||||
@@ -775,6 +779,8 @@ 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.WithPayloadEnvelopeCache(b.payloadEnvelopeCache),
|
||||
blockchain.WithPayloadIDCache(b.payloadIDCache),
|
||||
blockchain.WithSyncChecker(b.syncChecker),
|
||||
)
|
||||
@@ -852,6 +858,8 @@ 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.WithPayloadEnvelopeCache(b.payloadEnvelopeCache),
|
||||
regularsync.WithPayloadReconstructor(web3Service),
|
||||
regularsync.WithClockWaiter(b.clockWaiter),
|
||||
regularsync.WithInitialSyncComplete(initialSyncComplete),
|
||||
|
||||
@@ -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",
|
||||
],
|
||||
)
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
}
|
||||
|
||||
@@ -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 ðpb.SyncCommitteeMessage{} },
|
||||
BlsToExecutionChangeSubnetTopicFormat: func() proto.Message { return ðpb.SignedBLSToExecutionChange{} },
|
||||
BlobSubnetTopicFormat: func() proto.Message { return ðpb.BlobSidecar{} },
|
||||
SignedExecutionPayloadHeaderTopicFormat: func() proto.Message { return &enginev1.SignedExecutionPayloadHeader{} },
|
||||
SignedExecutionPayloadEnvelopeTopicFormat: func() proto.Message { return &enginev1.SignedExecutionPayloadEnvelope{} },
|
||||
PayloadAttestationMessageTopicFormat: func() proto.Message { return ðpb.PayloadAttestationMessage{} },
|
||||
}
|
||||
|
||||
// GossipTopicMappings is a function to return the assigned data type
|
||||
@@ -104,4 +108,8 @@ func init() {
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.AttestationElectra{})] = AttestationSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.AttesterSlashingElectra{})] = AttesterSlashingSubnetTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.SignedAggregateAttestationAndProofElectra{})] = AggregateAndProofSubnetTopicFormat
|
||||
// Handle ePBS objects.
|
||||
GossipTypeMapping[reflect.TypeOf(&enginev1.SignedExecutionPayloadHeader{})] = SignedExecutionPayloadHeaderTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(&enginev1.SignedExecutionPayloadEnvelope{})] = SignedExecutionPayloadEnvelopeTopicFormat
|
||||
GossipTypeMapping[reflect.TypeOf(ðpb.PayloadAttestationMessage{})] = PayloadAttestationMessageTopicFormat
|
||||
}
|
||||
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
11
beacon-chain/p2p/topics_epbs.go
Normal file
11
beacon-chain/p2p/topics_epbs.go
Normal 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
|
||||
)
|
||||
@@ -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)
|
||||
}
|
||||
})
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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),
|
||||
|
||||
@@ -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",
|
||||
|
||||
@@ -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.
|
||||
|
||||
17
beacon-chain/rpc/prysm/v1alpha1/validator/ptc_attester.go
Normal file
17
beacon-chain/rpc/prysm/v1alpha1/validator/ptc_attester.go
Normal 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")
|
||||
}
|
||||
@@ -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",
|
||||
|
||||
@@ -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
|
||||
|
||||
21
beacon-chain/state/interfaces_epbs.go
Normal file
21
beacon-chain/state/interfaces_epbs.go
Normal 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
|
||||
}
|
||||
@@ -12,11 +12,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",
|
||||
@@ -33,6 +35,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",
|
||||
@@ -45,6 +48,7 @@ go_library(
|
||||
"spec_parameters.go",
|
||||
"ssz.go",
|
||||
"state_trie.go",
|
||||
"state_trie_epbs.go",
|
||||
"types.go",
|
||||
"validator_index_cache.go",
|
||||
],
|
||||
@@ -97,6 +101,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",
|
||||
@@ -118,6 +123,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",
|
||||
@@ -151,6 +157,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",
|
||||
|
||||
@@ -58,6 +58,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
|
||||
|
||||
75
beacon-chain/state/state-native/getters_epbs.go
Normal file
75
beacon-chain/state/state-native/getters_epbs.go
Normal 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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
90
beacon-chain/state/state-native/getters_setters_epbs_test.go
Normal file
90
beacon-chain/state/state-native/getters_setters_epbs_test.go
Normal 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))
|
||||
}
|
||||
@@ -212,6 +212,49 @@ func (b *BeaconState) ToProtoUnsafe() interface{} {
|
||||
PendingPartialWithdrawals: b.pendingPartialWithdrawals,
|
||||
PendingConsolidations: b.pendingConsolidations,
|
||||
}
|
||||
case version.EPBS:
|
||||
return ðpb.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 ðpb.BeaconState{
|
||||
@@ -418,6 +464,49 @@ func (b *BeaconState) ToProto() interface{} {
|
||||
PendingPartialWithdrawals: b.pendingPartialWithdrawalsVal(),
|
||||
PendingConsolidations: b.pendingConsolidationsVal(),
|
||||
}
|
||||
case version.EPBS:
|
||||
return ðpb.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
|
||||
}
|
||||
|
||||
@@ -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
|
||||
}
|
||||
|
||||
69
beacon-chain/state/state-native/setters_epbs.go
Normal file
69
beacon-chain/state/state-native/setters_epbs.go
Normal 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
|
||||
}
|
||||
@@ -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)
|
||||
}
|
||||
|
||||
|
||||
@@ -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
|
||||
|
||||
@@ -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")
|
||||
}
|
||||
|
||||
148
beacon-chain/state/state-native/state_trie_epbs.go
Normal file
148
beacon-chain/state/state-native/state_trie_epbs.go
Normal 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
|
||||
}
|
||||
70
beacon-chain/state/state-native/state_trie_epbs_test.go
Normal file
70
beacon-chain/state/state-native/state_trie_epbs_test.go
Normal 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)
|
||||
}
|
||||
@@ -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.
|
||||
|
||||
@@ -11,11 +11,13 @@ go_library(
|
||||
"decode_pubsub.go",
|
||||
"doc.go",
|
||||
"error.go",
|
||||
"execution_payload_envelope.go",
|
||||
"fork_watcher.go",
|
||||
"fuzz_exports.go", # keep
|
||||
"log.go",
|
||||
"metrics.go",
|
||||
"options.go",
|
||||
"payload_attestations.go",
|
||||
"pending_attestations_queue.go",
|
||||
"pending_blocks_queue.go",
|
||||
"rate_limiter.go",
|
||||
@@ -101,6 +103,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",
|
||||
@@ -113,6 +116,7 @@ go_library(
|
||||
"//io/file:go_default_library",
|
||||
"//monitoring/tracing:go_default_library",
|
||||
"//network/forks:go_default_library",
|
||||
"//proto/engine/v1:go_default_library",
|
||||
"//proto/prysm/v1alpha1:go_default_library",
|
||||
"//proto/prysm/v1alpha1/attestation:go_default_library",
|
||||
"//proto/prysm/v1alpha1/metadata:go_default_library",
|
||||
@@ -154,7 +158,9 @@ go_test(
|
||||
"context_test.go",
|
||||
"decode_pubsub_test.go",
|
||||
"error_test.go",
|
||||
"execution_payload_envelope_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 +234,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 +253,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",
|
||||
|
||||
102
beacon-chain/sync/execution_payload_envelope.go
Normal file
102
beacon-chain/sync/execution_payload_envelope.go
Normal file
@@ -0,0 +1,102 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"context"
|
||||
|
||||
pubsub "github.com/libp2p/go-libp2p-pubsub"
|
||||
"github.com/libp2p/go-libp2p/core/peer"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/verification"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
|
||||
v1 "github.com/prysmaticlabs/prysm/v5/proto/engine/v1"
|
||||
"go.opencensus.io/trace"
|
||||
"google.golang.org/protobuf/proto"
|
||||
)
|
||||
|
||||
func (s *Service) validateExecutionPayloadEnvelope(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.validateExecutionPayloadEnvelope")
|
||||
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
|
||||
}
|
||||
signedEnvelope, ok := m.(*v1.SignedExecutionPayloadEnvelope)
|
||||
if !ok {
|
||||
return pubsub.ValidationReject, errWrongMessage
|
||||
}
|
||||
e, err := blocks.WrappedROSignedExecutionPayloadEnvelope(signedEnvelope)
|
||||
if err != nil {
|
||||
log.WithError(err).Error("failed to create read only signed payload execution envelope")
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
v := s.newExecutionPayloadEnvelopeVerifier(e, verification.GossipExecutionPayloadEnvelopeRequirements)
|
||||
|
||||
if err := v.VerifyBlockRootSeen(s.seenBlockRoot); err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
root := [32]byte(signedEnvelope.Message.BeaconBlockRoot)
|
||||
_, seen := s.payloadEnvelopeCache.Load(root)
|
||||
if seen {
|
||||
return pubsub.ValidationIgnore, nil
|
||||
}
|
||||
if err := v.VerifyBlockRootValid(s.hasBadBlock); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
signedHeader, err := s.cfg.beaconDB.SignedExecutionPayloadHeader(ctx, root)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
res, err := verifyAgainstHeader(v, signedHeader)
|
||||
if err != nil {
|
||||
return res, err
|
||||
}
|
||||
st, err := s.cfg.stateGen.StateByRoot(ctx, root)
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
if err := v.VerifySignature(st); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
s.payloadEnvelopeCache.Store(root, struct{}{})
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
func verifyAgainstHeader(v verification.ExecutionPayloadEnvelopeVerifier, signed interfaces.ROSignedExecutionPayloadHeader) (pubsub.ValidationResult, error) {
|
||||
header, err := signed.Header()
|
||||
if err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
if err := v.SetSlot(header.Slot()); err != nil {
|
||||
return pubsub.ValidationIgnore, err
|
||||
}
|
||||
if err := v.VerifyBuilderValid(header); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
if err := v.VerifyPayloadHash(header); err != nil {
|
||||
return pubsub.ValidationReject, err
|
||||
}
|
||||
return pubsub.ValidationAccept, nil
|
||||
}
|
||||
|
||||
func (s *Service) executionPayloadEnvelopeSubscriber(ctx context.Context, msg proto.Message) error {
|
||||
e, ok := msg.(*v1.SignedExecutionPayloadEnvelope)
|
||||
if !ok {
|
||||
return errWrongMessage
|
||||
}
|
||||
env, err := blocks.WrappedROExecutionPayloadEnvelope(e.Message)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
return s.cfg.chain.ReceiveExecutionPayloadEnvelope(ctx, env, nil)
|
||||
}
|
||||
177
beacon-chain/sync/execution_payload_envelope_test.go
Normal file
177
beacon-chain/sync/execution_payload_envelope_test.go
Normal file
@@ -0,0 +1,177 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"reflect"
|
||||
"sync"
|
||||
"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"
|
||||
testDB "github.com/prysmaticlabs/prysm/v5/beacon-chain/db/testing"
|
||||
doublylinkedtree "github.com/prysmaticlabs/prysm/v5/beacon-chain/forkchoice/doubly-linked-tree"
|
||||
"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"
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/state/stategen"
|
||||
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"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/blocks"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/interfaces"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/require"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util"
|
||||
"github.com/prysmaticlabs/prysm/v5/testing/util/random"
|
||||
)
|
||||
|
||||
func TestValidateExecutionPayloadEnvelope_ErrorPathsWithMock(t *testing.T) {
|
||||
tests := []struct {
|
||||
error error
|
||||
verifier verification.NewExecutionPayloadEnvelopeVerifier
|
||||
result pubsub.ValidationResult
|
||||
}{
|
||||
{
|
||||
error: errors.New("block root seen"),
|
||||
verifier: func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{ErrBlockRootNotSeen: errors.New("block root seen")}
|
||||
},
|
||||
result: pubsub.ValidationIgnore,
|
||||
},
|
||||
{
|
||||
error: errors.New("block root invalid"),
|
||||
verifier: func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{ErrBlockRootInvalid: errors.New("block root invalid")}
|
||||
},
|
||||
result: pubsub.ValidationReject,
|
||||
},
|
||||
{
|
||||
error: errors.New("invalid builder index"),
|
||||
verifier: func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{ErrBuilderIndexInvalid: errors.New("invalid builder index")}
|
||||
},
|
||||
result: pubsub.ValidationReject,
|
||||
},
|
||||
{
|
||||
error: errors.New("invalid block hash"),
|
||||
verifier: func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{ErrBlockHashInvalid: errors.New("invalid block hash")}
|
||||
},
|
||||
result: pubsub.ValidationReject,
|
||||
},
|
||||
{
|
||||
error: errors.New("incorrect signature"),
|
||||
verifier: func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{ErrSignatureInvalid: errors.New("incorrect signature")}
|
||||
},
|
||||
result: pubsub.ValidationReject,
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.error.Error(), func(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
db := testDB.SetupDB(t)
|
||||
fcs := doublylinkedtree.New()
|
||||
sg := stategen.New(db, fcs)
|
||||
p := p2ptest.NewTestP2P(t)
|
||||
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
|
||||
|
||||
s := &Service{
|
||||
payloadEnvelopeCache: &sync.Map{},
|
||||
cfg: &config{
|
||||
chain: chainService,
|
||||
p2p: p,
|
||||
initialSync: &mockSync.Sync{},
|
||||
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
stateGen: sg,
|
||||
},
|
||||
}
|
||||
s.newExecutionPayloadEnvelopeVerifier = tt.verifier
|
||||
|
||||
blk := random.SignedBeaconBlock(t)
|
||||
blkRoot, err := blk.Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
msg := random.SignedExecutionPayloadEnvelope(t)
|
||||
msg.Message.BeaconBlockRoot = blkRoot[:]
|
||||
wblk, err := blocks.NewSignedBeaconBlock(blk)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, db.SaveBlock(ctx, wblk))
|
||||
st, err := util.NewBeaconStateEpbs()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, sg.SaveState(ctx, blkRoot, st))
|
||||
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.validateExecutionPayloadEnvelope(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 TestValidateExecutionPayloadEnvelope_Accept(t *testing.T) {
|
||||
ctx := context.Background()
|
||||
p := p2ptest.NewTestP2P(t)
|
||||
db := testDB.SetupDB(t)
|
||||
fcs := doublylinkedtree.New()
|
||||
sg := stategen.New(db, fcs)
|
||||
chainService := &mock.ChainService{Genesis: time.Unix(time.Now().Unix()-int64(params.BeaconConfig().SecondsPerSlot), 0)}
|
||||
s := &Service{
|
||||
payloadEnvelopeCache: &sync.Map{},
|
||||
cfg: &config{
|
||||
chain: chainService,
|
||||
p2p: p,
|
||||
initialSync: &mockSync.Sync{},
|
||||
clock: startup.NewClock(chainService.Genesis, chainService.ValidatorsRoot),
|
||||
beaconDB: db,
|
||||
stateGen: sg,
|
||||
},
|
||||
}
|
||||
s.newExecutionPayloadEnvelopeVerifier = func(e interfaces.ROSignedExecutionPayloadEnvelope, reqs []verification.Requirement) verification.ExecutionPayloadEnvelopeVerifier {
|
||||
return &verification.MockExecutionPayloadEnvelope{}
|
||||
}
|
||||
|
||||
blk := random.SignedBeaconBlock(t)
|
||||
blkRoot, err := blk.Block.HashTreeRoot()
|
||||
require.NoError(t, err)
|
||||
msg := random.SignedExecutionPayloadEnvelope(t)
|
||||
msg.Message.BeaconBlockRoot = blkRoot[:]
|
||||
wblk, err := blocks.NewSignedBeaconBlock(blk)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, db.SaveBlock(ctx, wblk))
|
||||
st, err := util.NewBeaconStateEpbs()
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, sg.SaveState(ctx, blkRoot, st))
|
||||
|
||||
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.validateExecutionPayloadEnvelope(ctx, "", &pubsub.Message{
|
||||
Message: &pb.Message{
|
||||
Data: buf.Bytes(),
|
||||
Topic: &topic,
|
||||
}})
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, result, pubsub.ValidationAccept)
|
||||
}
|
||||
@@ -1,7 +1,10 @@
|
||||
package sync
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"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 +130,20 @@ func WithSlasherBlockHeadersFeed(slasherBlockHeadersFeed *event.Feed) Option {
|
||||
}
|
||||
}
|
||||
|
||||
func WithPayloadAttestationCache(r *cache.PayloadAttestationCache) Option {
|
||||
return func(s *Service) error {
|
||||
s.payloadAttestationCache = r
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithPayloadEnvelopeCache(r *sync.Map) Option {
|
||||
return func(s *Service) error {
|
||||
s.payloadEnvelopeCache = r
|
||||
return nil
|
||||
}
|
||||
}
|
||||
|
||||
func WithPayloadReconstructor(r execution.PayloadReconstructor) Option {
|
||||
return func(s *Service) error {
|
||||
s.cfg.executionPayloadReconstructor = r
|
||||
|
||||
115
beacon-chain/sync/payload_attestations.go
Normal file
115
beacon-chain/sync/payload_attestations.go
Normal file
@@ -0,0 +1,115 @@
|
||||
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 (
|
||||
errInvalidValidatorIndex = errors.New("invalid validator index")
|
||||
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.seenBlockRoot); 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))
|
||||
}
|
||||
Some files were not shown because too many files have changed in this diff Show More
Reference in New Issue
Block a user