Add and use SignedBeaconBlockWriteOnly (#11968)

This commit is contained in:
terencechain
2023-02-08 08:39:14 -08:00
committed by GitHub
parent 0d6e8718fd
commit e1f7c52e6b
19 changed files with 173 additions and 150 deletions

View File

@@ -127,7 +127,7 @@ func TestHeadSlot_CanRetrieve(t *testing.T) {
require.NoError(t, err)
b, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlock())
require.NoError(t, err)
b.Block().SetSlot(100)
b.SetSlot(100)
c.head = &head{block: b, state: s}
assert.Equal(t, primitives.Slot(100), c.HeadSlot())
}

View File

@@ -249,7 +249,7 @@ func Test_validateTerminalBlockHash(t *testing.T) {
require.NoError(t, err)
blk, err := blocks.NewSignedBeaconBlock(util.HydrateSignedBeaconBlockBellatrix(&ethpb.SignedBeaconBlockBellatrix{}))
require.NoError(t, err)
blk.Block().SetSlot(1)
require.NoError(t, blk.Block().Body().SetExecution(wrapped))
blk.SetSlot(1)
require.NoError(t, blk.SetExecution(wrapped))
require.NoError(t, service.validateMergeBlock(ctx, blk))
}

View File

@@ -65,10 +65,10 @@ type ForkchoiceUpdatedResponse struct {
type ExecutionPayloadReconstructor interface {
ReconstructFullBlock(
ctx context.Context, blindedBlock interfaces.SignedBeaconBlock,
) (interfaces.SignedBeaconBlock, error)
) (interfaces.SignedBeaconBlockWriteable, error)
ReconstructFullBellatrixBlockBatch(
ctx context.Context, blindedBlocks []interfaces.SignedBeaconBlock,
) ([]interfaces.SignedBeaconBlock, error)
) ([]interfaces.SignedBeaconBlockWriteable, error)
}
// EngineCaller defines a client that can interact with an Ethereum
@@ -443,7 +443,7 @@ func (s *Service) HeaderByNumber(ctx context.Context, number *big.Int) (*types.H
// a beacon block with a full execution payload via the engine API.
func (s *Service) ReconstructFullBlock(
ctx context.Context, blindedBlock interfaces.SignedBeaconBlock,
) (interfaces.SignedBeaconBlock, error) {
) (interfaces.SignedBeaconBlockWriteable, error) {
if err := blocks.BeaconBlockIsNil(blindedBlock); err != nil {
return nil, errors.Wrap(err, "cannot reconstruct bellatrix block from nil data")
}
@@ -494,9 +494,9 @@ func (s *Service) ReconstructFullBlock(
// them with a full execution payload for each block via the engine API.
func (s *Service) ReconstructFullBellatrixBlockBatch(
ctx context.Context, blindedBlocks []interfaces.SignedBeaconBlock,
) ([]interfaces.SignedBeaconBlock, error) {
) ([]interfaces.SignedBeaconBlockWriteable, error) {
if len(blindedBlocks) == 0 {
return []interfaces.SignedBeaconBlock{}, nil
return []interfaces.SignedBeaconBlockWriteable{}, nil
}
executionHashes := []common.Hash{}
validExecPayloads := []int{}
@@ -532,6 +532,7 @@ func (s *Service) ReconstructFullBellatrixBlockBatch(
// For each valid payload, we reconstruct the full block from it with the
// blinded block.
fullBlocks := make([]interfaces.SignedBeaconBlockWriteable, len(blindedBlocks))
for sliceIdx, realIdx := range validExecPayloads {
b := execBlocks[sliceIdx]
if b == nil {
@@ -549,7 +550,7 @@ func (s *Service) ReconstructFullBellatrixBlockBatch(
if err != nil {
return nil, err
}
blindedBlocks[realIdx] = fullBlock
fullBlocks[realIdx] = fullBlock
}
// For blocks that are pre-merge we simply reconstruct them via an empty
// execution payload.
@@ -559,10 +560,10 @@ func (s *Service) ReconstructFullBellatrixBlockBatch(
if err != nil {
return nil, err
}
blindedBlocks[realIdx] = fullBlock
fullBlocks[realIdx] = fullBlock
}
reconstructedExecutionPayloadCount.Add(float64(len(blindedBlocks)))
return blindedBlocks, nil
return fullBlocks, nil
}
func fullPayloadFromExecutionBlock(

View File

@@ -769,7 +769,7 @@ func TestReconstructFullBellatrixBlockBatch(t *testing.T) {
require.NoError(t, err)
reconstructed, err := service.ReconstructFullBellatrixBlockBatch(ctx, []interfaces.SignedBeaconBlock{wrapped})
require.NoError(t, err)
require.DeepEqual(t, []interfaces.SignedBeaconBlock{wantedWrapped}, reconstructed)
require.DeepEqual(t, []interfaces.SignedBeaconBlockWriteable{wantedWrapped}, reconstructed)
})
t.Run("properly reconstructs block batch with correct payload", func(t *testing.T) {
fix := fixtures()

View File

@@ -89,7 +89,7 @@ func (e *EngineClient) ExecutionBlockByHash(_ context.Context, h common.Hash, _
// ReconstructFullBlock --
func (e *EngineClient) ReconstructFullBlock(
_ context.Context, blindedBlock interfaces.SignedBeaconBlock,
) (interfaces.SignedBeaconBlock, error) {
) (interfaces.SignedBeaconBlockWriteable, error) {
if !blindedBlock.Block().IsBlinded() {
return nil, errors.New("block must be blinded")
}
@@ -108,8 +108,8 @@ func (e *EngineClient) ReconstructFullBlock(
// ReconstructFullBellatrixBlockBatch --
func (e *EngineClient) ReconstructFullBellatrixBlockBatch(
ctx context.Context, blindedBlocks []interfaces.SignedBeaconBlock,
) ([]interfaces.SignedBeaconBlock, error) {
fullBlocks := make([]interfaces.SignedBeaconBlock, 0, len(blindedBlocks))
) ([]interfaces.SignedBeaconBlockWriteable, error) {
fullBlocks := make([]interfaces.SignedBeaconBlockWriteable, 0, len(blindedBlocks))
for _, b := range blindedBlocks {
newBlock, err := e.ReconstructFullBlock(ctx, b)
if err != nil {

View File

@@ -1724,7 +1724,7 @@ func TestServer_GetValidatorParticipation_CurrentAndPrevEpochWithBits(t *testing
})
}
func runGetValidatorParticipationCurrentAndPrevEpoch(t *testing.T, genState state.BeaconState, gb interfaces.SignedBeaconBlock) {
func runGetValidatorParticipationCurrentAndPrevEpoch(t *testing.T, genState state.BeaconState, gb interfaces.SignedBeaconBlockWriteable) {
helpers.ClearCache()
beaconDB := dbTest.SetupDB(t)

View File

@@ -71,12 +71,11 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
return nil, status.Errorf(codes.Internal, "Could not process slots up to %d: %v", req.Slot, err)
}
blk := sBlk.Block()
// Set slot, graffiti, randao reveal, and parent root.
blk.SetSlot(req.Slot)
blk.Body().SetGraffiti(req.Graffiti)
blk.Body().SetRandaoReveal(req.RandaoReveal)
blk.SetParentRoot(parentRoot)
sBlk.SetSlot(req.Slot)
sBlk.SetGraffiti(req.Graffiti)
sBlk.SetRandaoReveal(req.RandaoReveal)
sBlk.SetParentRoot(parentRoot)
// Set eth1 data.
eth1Data, err := vs.eth1DataMajorityVote(ctx, head)
@@ -84,62 +83,62 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
eth1Data = &ethpb.Eth1Data{DepositRoot: params.BeaconConfig().ZeroHash[:], BlockHash: params.BeaconConfig().ZeroHash[:]}
log.WithError(err).Error("Could not get eth1data")
}
blk.Body().SetEth1Data(eth1Data)
sBlk.SetEth1Data(eth1Data)
// Set deposit and attestation.
deposits, atts, err := vs.packDepositsAndAttestations(ctx, head, eth1Data) // TODO: split attestations and deposits
if err != nil {
blk.Body().SetDeposits([]*ethpb.Deposit{})
blk.Body().SetAttestations([]*ethpb.Attestation{})
sBlk.SetDeposits([]*ethpb.Deposit{})
sBlk.SetAttestations([]*ethpb.Attestation{})
log.WithError(err).Error("Could not pack deposits and attestations")
}
blk.Body().SetDeposits(deposits)
blk.Body().SetAttestations(atts)
sBlk.SetDeposits(deposits)
sBlk.SetAttestations(atts)
// Set proposer index.
idx, err := helpers.BeaconProposerIndex(ctx, head)
if err != nil {
return nil, fmt.Errorf("could not calculate proposer index %v", err)
}
blk.SetProposerIndex(idx)
sBlk.SetProposerIndex(idx)
// Set slashings.
validProposerSlashings, validAttSlashings := vs.getSlashings(ctx, head)
blk.Body().SetProposerSlashings(validProposerSlashings)
blk.Body().SetAttesterSlashings(validAttSlashings)
sBlk.SetProposerSlashings(validProposerSlashings)
sBlk.SetAttesterSlashings(validAttSlashings)
// Set exits.
blk.Body().SetVoluntaryExits(vs.getExits(head, req.Slot))
sBlk.SetVoluntaryExits(vs.getExits(head, req.Slot))
// Set sync aggregate. New in Altair.
vs.setSyncAggregate(ctx, blk)
vs.setSyncAggregate(ctx, sBlk)
// Set execution data. New in Bellatrix.
if err := vs.setExecutionData(ctx, blk, head); err != nil {
if err := vs.setExecutionData(ctx, sBlk, head); err != nil {
return nil, status.Errorf(codes.Internal, "Could not set execution data: %v", err)
}
// Set bls to execution change. New in Capella.
vs.setBlsToExecData(blk, head)
vs.setBlsToExecData(sBlk, head)
sr, err := vs.computeStateRoot(ctx, sBlk)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not compute state root: %v", err)
}
blk.SetStateRoot(sr)
sBlk.SetStateRoot(sr)
pb, err := blk.Proto()
pb, err := sBlk.Block().Proto()
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not convert block to proto: %v", err)
}
if slots.ToEpoch(req.Slot) >= params.BeaconConfig().CapellaForkEpoch {
if blk.IsBlinded() {
if sBlk.IsBlinded() {
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedCapella{BlindedCapella: pb.(*ethpb.BlindedBeaconBlockCapella)}}, nil
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Capella{Capella: pb.(*ethpb.BeaconBlockCapella)}}, nil
}
if slots.ToEpoch(req.Slot) >= params.BeaconConfig().BellatrixForkEpoch {
if blk.IsBlinded() {
if sBlk.IsBlinded() {
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedBellatrix{BlindedBellatrix: pb.(*ethpb.BlindedBeaconBlockBellatrix)}}, nil
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_Bellatrix{Bellatrix: pb.(*ethpb.BeaconBlockBellatrix)}}, nil

View File

@@ -15,12 +15,12 @@ import (
"go.opencensus.io/trace"
)
func (vs *Server) setSyncAggregate(ctx context.Context, blk interfaces.BeaconBlock) {
func (vs *Server) setSyncAggregate(ctx context.Context, blk interfaces.SignedBeaconBlockWriteable) {
if blk.Version() < version.Altair {
return
}
syncAggregate, err := vs.getSyncAggregate(ctx, blk.Slot()-1, blk.ParentRoot())
syncAggregate, err := vs.getSyncAggregate(ctx, blk.Block().Slot()-1, blk.Block().ParentRoot())
if err != nil {
log.WithError(err).Error("Could not get sync aggregate")
emptySig := [96]byte{0xC0}
@@ -28,14 +28,14 @@ func (vs *Server) setSyncAggregate(ctx context.Context, blk interfaces.BeaconBlo
SyncCommitteeBits: make([]byte, params.BeaconConfig().SyncCommitteeSize),
SyncCommitteeSignature: emptySig[:],
}
if err := blk.Body().SetSyncAggregate(emptyAggregate); err != nil {
if err := blk.SetSyncAggregate(emptyAggregate); err != nil {
log.WithError(err).Error("Could not set sync aggregate")
}
return
}
// Can not error. We already filter block versioning at the top. Phase 0 is impossible.
if err := blk.Body().SetSyncAggregate(syncAggregate); err != nil {
if err := blk.SetSyncAggregate(syncAggregate); err != nil {
log.WithError(err).Error("Could not set sync aggregate")
}
}

View File

@@ -12,11 +12,11 @@ import (
)
func TestServer_SetSyncAggregate_EmptyCase(t *testing.T) {
b, err := blocks.NewBeaconBlock(util.NewBeaconBlockAltair().Block)
b, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlockAltair())
require.NoError(t, err)
s := &Server{} // Sever is not initialized with sync committee pool.
s.setSyncAggregate(context.Background(), b)
agg, err := b.Body().SyncAggregate()
agg, err := b.Block().Body().SyncAggregate()
require.NoError(t, err)
emptySig := [96]byte{0xC0}

View File

@@ -37,9 +37,9 @@ var builderGetPayloadMissCount = promauto.NewCounter(prometheus.CounterOpts{
const blockBuilderTimeout = 1 * time.Second
// Sets the execution data for the block. Execution data can come from local EL client or remote builder depends on validator registration and circuit breaker conditions.
func (vs *Server) setExecutionData(ctx context.Context, blk interfaces.BeaconBlock, headState state.BeaconState) error {
idx := blk.ProposerIndex()
slot := blk.Slot()
func (vs *Server) setExecutionData(ctx context.Context, blk interfaces.SignedBeaconBlockWriteable, headState state.BeaconState) error {
idx := blk.Block().ProposerIndex()
slot := blk.Block().Slot()
if slots.ToEpoch(slot) < params.BeaconConfig().BellatrixForkEpoch {
return nil
}
@@ -54,18 +54,18 @@ func (vs *Server) setExecutionData(ctx context.Context, blk interfaces.BeaconBlo
log.WithError(err).Warn("Proposer: failed to get payload header from builder")
} else {
blk.SetBlinded(true)
if err := blk.Body().SetExecution(h); err != nil {
if err := blk.SetExecution(h); err != nil {
log.WithError(err).Warn("Proposer: failed to set execution payload")
} else {
return nil
}
}
}
executionData, err := vs.getExecutionPayload(ctx, slot, idx, blk.ParentRoot(), headState)
executionData, err := vs.getExecutionPayload(ctx, slot, idx, blk.Block().ParentRoot(), headState)
if err != nil {
return errors.Wrap(err, "failed to get execution payload")
}
return blk.Body().SetExecution(executionData)
return blk.SetExecution(executionData)
}
// This function retrieves the payload header given the slot number and the validator index.

View File

@@ -15,11 +15,11 @@ import (
)
// Sets the bls to exec data for a block.
func (vs *Server) setBlsToExecData(blk interfaces.BeaconBlock, headState state.BeaconState) {
func (vs *Server) setBlsToExecData(blk interfaces.SignedBeaconBlockWriteable, headState state.BeaconState) {
if blk.Version() < version.Capella {
return
}
if err := blk.Body().SetBLSToExecutionChanges([]*ethpb.SignedBLSToExecutionChange{}); err != nil {
if err := blk.SetBLSToExecutionChanges([]*ethpb.SignedBLSToExecutionChange{}); err != nil {
log.WithError(err).Error("Could not set bls to execution data in block")
return
}
@@ -28,7 +28,7 @@ func (vs *Server) setBlsToExecData(blk interfaces.BeaconBlock, headState state.B
log.WithError(err).Error("Could not get bls to execution changes")
return
} else {
if err := blk.Body().SetBLSToExecutionChanges(changes); err != nil {
if err := blk.SetBLSToExecutionChanges(changes); err != nil {
log.WithError(err).Error("Could not set bls to execution changes")
return
}

View File

@@ -11,8 +11,8 @@ import (
"google.golang.org/grpc/status"
)
func getEmptyBlock(slot primitives.Slot) (interfaces.SignedBeaconBlock, error) {
var sBlk interfaces.SignedBeaconBlock
func getEmptyBlock(slot primitives.Slot) (interfaces.SignedBeaconBlockWriteable, error) {
var sBlk interfaces.SignedBeaconBlockWriteable
var err error
switch {
case slots.ToEpoch(slot) < params.BeaconConfig().AltairForkEpoch:

View File

@@ -174,20 +174,39 @@ func (s *Service) writeBlockRangeToStream(ctx context.Context, startSlot, endSlo
break
}
}
var reconstructedBlock []interfaces.SignedBeaconBlockWriteable
if blindedExists {
reconstructedBlks, err := s.cfg.executionPayloadReconstructor.ReconstructFullBellatrixBlockBatch(ctx, blks[blindedIndex:])
reconstructedBlock, err = s.cfg.executionPayloadReconstructor.ReconstructFullBellatrixBlockBatch(ctx, blks[blindedIndex:])
if err != nil {
log.WithError(err).Error("Could not reconstruct full bellatrix block batch from blinded bodies")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
return err
}
copy(blks[blindedIndex:], reconstructedBlks)
}
for _, b := range blks {
if err := blocks.BeaconBlockIsNil(b); err != nil {
continue
}
if b.IsBlinded() {
continue
}
if chunkErr := s.chunkBlockWriter(stream, b); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)
tracing.AnnotateError(span, chunkErr)
return chunkErr
}
}
for _, b := range reconstructedBlock {
if err := blocks.BeaconBlockIsNil(b); err != nil {
continue
}
if b.IsBlinded() {
continue
}
if chunkErr := s.chunkBlockWriter(stream, b); chunkErr != nil {
log.WithError(chunkErr).Debug("Could not send a chunked response")
s.writeErrorResponseToStream(responseCodeServerError, p2ptypes.ErrGeneric.Error(), stream)

View File

@@ -28,7 +28,7 @@ var (
)
// NewSignedBeaconBlock creates a signed beacon block from a protobuf signed beacon block.
func NewSignedBeaconBlock(i interface{}) (interfaces.SignedBeaconBlock, error) {
func NewSignedBeaconBlock(i interface{}) (interfaces.SignedBeaconBlockWriteable, error) {
switch b := i.(type) {
case nil:
return nil, ErrNilObject
@@ -120,7 +120,7 @@ func NewBeaconBlockBody(i interface{}) (interfaces.BeaconBlockBody, error) {
// BuildSignedBeaconBlock assembles a block.SignedBeaconBlock interface compatible struct from a
// given beacon block and the appropriate signature. This method may be used to easily create a
// signed beacon block.
func BuildSignedBeaconBlock(blk interfaces.BeaconBlock, signature []byte) (interfaces.SignedBeaconBlock, error) {
func BuildSignedBeaconBlock(blk interfaces.BeaconBlock, signature []byte) (interfaces.SignedBeaconBlockWriteable, error) {
pb, err := blk.Proto()
if err != nil {
return nil, err
@@ -174,7 +174,7 @@ func BuildSignedBeaconBlock(blk interfaces.BeaconBlock, signature []byte) (inter
// a full, signed beacon block by specifying an execution payload.
func BuildSignedBeaconBlockFromExecutionPayload(
blk interfaces.SignedBeaconBlock, payload interface{},
) (interfaces.SignedBeaconBlock, error) {
) (interfaces.SignedBeaconBlockWriteable, error) {
if err := BeaconBlockIsNil(blk); err != nil {
return nil, err
}

View File

@@ -140,27 +140,27 @@ func Test_SignedBeaconBlock_UnmarshalSSZ(t *testing.T) {
}
func Test_BeaconBlock_Slot(t *testing.T) {
b := &BeaconBlock{}
b := &SignedBeaconBlock{block: &BeaconBlock{}}
b.SetSlot(128)
assert.Equal(t, primitives.Slot(128), b.Slot())
assert.Equal(t, primitives.Slot(128), b.Block().Slot())
}
func Test_BeaconBlock_ProposerIndex(t *testing.T) {
b := &BeaconBlock{}
b := &SignedBeaconBlock{block: &BeaconBlock{}}
b.SetProposerIndex(128)
assert.Equal(t, primitives.ValidatorIndex(128), b.ProposerIndex())
assert.Equal(t, primitives.ValidatorIndex(128), b.Block().ProposerIndex())
}
func Test_BeaconBlock_ParentRoot(t *testing.T) {
b := &BeaconBlock{}
b := &SignedBeaconBlock{block: &BeaconBlock{}}
b.SetParentRoot([]byte("parentroot"))
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("parentroot")), b.ParentRoot())
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("parentroot")), b.Block().ParentRoot())
}
func Test_BeaconBlock_StateRoot(t *testing.T) {
b := &BeaconBlock{}
b := &SignedBeaconBlock{block: &BeaconBlock{}}
b.SetStateRoot([]byte("stateroot"))
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("stateroot")), b.StateRoot())
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("stateroot")), b.Block().StateRoot())
}
func Test_BeaconBlock_Body(t *testing.T) {
@@ -230,7 +230,7 @@ func Test_BeaconBlock_IsNil(t *testing.T) {
}
func Test_BeaconBlock_IsBlinded(t *testing.T) {
b := &BeaconBlock{body: &BeaconBlockBody{}}
b := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
assert.Equal(t, false, b.IsBlinded())
b.SetBlinded(true)
assert.Equal(t, true, b.IsBlinded())
@@ -309,73 +309,73 @@ func Test_BeaconBlockBody_IsNil(t *testing.T) {
}
func Test_BeaconBlockBody_RandaoReveal(t *testing.T) {
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetRandaoReveal([]byte("randaoreveal"))
assert.DeepEqual(t, bytesutil.ToBytes96([]byte("randaoreveal")), bb.RandaoReveal())
assert.DeepEqual(t, bytesutil.ToBytes96([]byte("randaoreveal")), bb.Block().Body().RandaoReveal())
}
func Test_BeaconBlockBody_Eth1Data(t *testing.T) {
e := &eth.Eth1Data{DepositRoot: []byte("depositroot")}
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetEth1Data(e)
assert.DeepEqual(t, e, bb.Eth1Data())
assert.DeepEqual(t, e, bb.Block().Body().Eth1Data())
}
func Test_BeaconBlockBody_Graffiti(t *testing.T) {
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetGraffiti([]byte("graffiti"))
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("graffiti")), bb.Graffiti())
assert.DeepEqual(t, bytesutil.ToBytes32([]byte("graffiti")), bb.Block().Body().Graffiti())
}
func Test_BeaconBlockBody_ProposerSlashings(t *testing.T) {
ps := make([]*eth.ProposerSlashing, 0)
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetProposerSlashings(ps)
assert.DeepSSZEqual(t, ps, bb.ProposerSlashings())
assert.DeepSSZEqual(t, ps, bb.Block().Body().ProposerSlashings())
}
func Test_BeaconBlockBody_AttesterSlashings(t *testing.T) {
as := make([]*eth.AttesterSlashing, 0)
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetAttesterSlashings(as)
assert.DeepSSZEqual(t, as, bb.AttesterSlashings())
assert.DeepSSZEqual(t, as, bb.Block().Body().AttesterSlashings())
}
func Test_BeaconBlockBody_Attestations(t *testing.T) {
a := make([]*eth.Attestation, 0)
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetAttestations(a)
assert.DeepSSZEqual(t, a, bb.Attestations())
assert.DeepSSZEqual(t, a, bb.Block().Body().Attestations())
}
func Test_BeaconBlockBody_Deposits(t *testing.T) {
d := make([]*eth.Deposit, 0)
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetDeposits(d)
assert.DeepSSZEqual(t, d, bb.Deposits())
assert.DeepSSZEqual(t, d, bb.Block().Body().Deposits())
}
func Test_BeaconBlockBody_VoluntaryExits(t *testing.T) {
ve := make([]*eth.SignedVoluntaryExit, 0)
bb := &BeaconBlockBody{}
bb := &SignedBeaconBlock{block: &BeaconBlock{body: &BeaconBlockBody{}}}
bb.SetVoluntaryExits(ve)
assert.DeepSSZEqual(t, ve, bb.VoluntaryExits())
assert.DeepSSZEqual(t, ve, bb.Block().Body().VoluntaryExits())
}
func Test_BeaconBlockBody_SyncAggregate(t *testing.T) {
sa := &eth.SyncAggregate{}
bb := &BeaconBlockBody{version: version.Altair}
bb := &SignedBeaconBlock{version: version.Altair, block: &BeaconBlock{version: version.Altair, body: &BeaconBlockBody{version: version.Altair}}}
require.NoError(t, bb.SetSyncAggregate(sa))
result, err := bb.SyncAggregate()
result, err := bb.Block().Body().SyncAggregate()
require.NoError(t, err)
assert.DeepEqual(t, result, sa)
}
func Test_BeaconBlockBody_BLSToExecutionChanges(t *testing.T) {
changes := []*eth.SignedBLSToExecutionChange{{Message: &eth.BLSToExecutionChange{ToExecutionAddress: []byte("address")}}}
bb := &BeaconBlockBody{version: version.Capella}
bb := &SignedBeaconBlock{version: version.Capella, block: &BeaconBlock{body: &BeaconBlockBody{version: version.Capella}}}
require.NoError(t, bb.SetBLSToExecutionChanges(changes))
result, err := bb.BLSToExecutionChanges()
result, err := bb.Block().Body().BLSToExecutionChanges()
require.NoError(t, err)
assert.DeepSSZEqual(t, result, changes)
}
@@ -384,27 +384,27 @@ func Test_BeaconBlockBody_Execution(t *testing.T) {
execution := &pb.ExecutionPayload{BlockNumber: 1}
e, err := WrappedExecutionPayload(execution)
require.NoError(t, err)
bb := &BeaconBlockBody{version: version.Bellatrix}
bb := &SignedBeaconBlock{version: version.Bellatrix, block: &BeaconBlock{body: &BeaconBlockBody{version: version.Bellatrix}}}
require.NoError(t, bb.SetExecution(e))
result, err := bb.Execution()
result, err := bb.Block().Body().Execution()
require.NoError(t, err)
assert.DeepEqual(t, result, e)
executionCapella := &pb.ExecutionPayloadCapella{BlockNumber: 1}
eCapella, err := WrappedExecutionPayloadCapella(executionCapella)
require.NoError(t, err)
bb = &BeaconBlockBody{version: version.Capella}
bb = &SignedBeaconBlock{version: version.Capella, block: &BeaconBlock{body: &BeaconBlockBody{version: version.Capella}}}
require.NoError(t, bb.SetExecution(eCapella))
result, err = bb.Execution()
result, err = bb.Block().Body().Execution()
require.NoError(t, err)
assert.DeepEqual(t, result, eCapella)
executionCapellaHeader := &pb.ExecutionPayloadHeaderCapella{BlockNumber: 1}
eCapellaHeader, err := WrappedExecutionPayloadHeaderCapella(executionCapellaHeader)
require.NoError(t, err)
bb = &BeaconBlockBody{version: version.Capella, isBlinded: true}
bb = &SignedBeaconBlock{version: version.Capella, block: &BeaconBlock{version: version.Capella, body: &BeaconBlockBody{version: version.Capella, isBlinded: true}}}
require.NoError(t, bb.SetExecution(eCapellaHeader))
result, err = bb.Execution()
result, err = bb.Block().Body().Execution()
require.NoError(t, err)
assert.DeepEqual(t, result, eCapellaHeader)
}

View File

@@ -15,112 +15,112 @@ func (b *SignedBeaconBlock) SetSignature(sig []byte) {
// SetSlot sets the respective slot of the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlock) SetSlot(slot primitives.Slot) {
b.slot = slot
func (b *SignedBeaconBlock) SetSlot(slot primitives.Slot) {
b.block.slot = slot
}
// SetProposerIndex sets the proposer index of the beacon block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlock) SetProposerIndex(proposerIndex primitives.ValidatorIndex) {
b.proposerIndex = proposerIndex
func (b *SignedBeaconBlock) SetProposerIndex(proposerIndex primitives.ValidatorIndex) {
b.block.proposerIndex = proposerIndex
}
// SetParentRoot sets the parent root of beacon block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlock) SetParentRoot(parentRoot []byte) {
copy(b.parentRoot[:], parentRoot)
func (b *SignedBeaconBlock) SetParentRoot(parentRoot []byte) {
copy(b.block.parentRoot[:], parentRoot)
}
// SetStateRoot sets the state root of the underlying beacon block
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlock) SetStateRoot(root []byte) {
copy(b.stateRoot[:], root)
func (b *SignedBeaconBlock) SetStateRoot(root []byte) {
copy(b.block.stateRoot[:], root)
}
// SetBlinded sets the blinded flag of the beacon block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlock) SetBlinded(blinded bool) {
b.body.isBlinded = blinded
func (b *SignedBeaconBlock) SetBlinded(blinded bool) {
b.block.body.isBlinded = blinded
}
// SetRandaoReveal sets the randao reveal in the block body.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetRandaoReveal(r []byte) {
copy(b.randaoReveal[:], r)
func (b *SignedBeaconBlock) SetRandaoReveal(r []byte) {
copy(b.block.body.randaoReveal[:], r)
}
// SetGraffiti sets the graffiti in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetGraffiti(g []byte) {
copy(b.graffiti[:], g)
func (b *SignedBeaconBlock) SetGraffiti(g []byte) {
copy(b.block.body.graffiti[:], g)
}
// SetEth1Data sets the eth1 data in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetEth1Data(e *eth.Eth1Data) {
b.eth1Data = e
func (b *SignedBeaconBlock) SetEth1Data(e *eth.Eth1Data) {
b.block.body.eth1Data = e
}
// SetProposerSlashings sets the proposer slashings in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetProposerSlashings(p []*eth.ProposerSlashing) {
b.proposerSlashings = p
func (b *SignedBeaconBlock) SetProposerSlashings(p []*eth.ProposerSlashing) {
b.block.body.proposerSlashings = p
}
// SetAttesterSlashings sets the attester slashings in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetAttesterSlashings(a []*eth.AttesterSlashing) {
b.attesterSlashings = a
func (b *SignedBeaconBlock) SetAttesterSlashings(a []*eth.AttesterSlashing) {
b.block.body.attesterSlashings = a
}
// SetAttestations sets the attestations in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetAttestations(a []*eth.Attestation) {
b.attestations = a
func (b *SignedBeaconBlock) SetAttestations(a []*eth.Attestation) {
b.block.body.attestations = a
}
// SetDeposits sets the deposits in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetDeposits(d []*eth.Deposit) {
b.deposits = d
func (b *SignedBeaconBlock) SetDeposits(d []*eth.Deposit) {
b.block.body.deposits = d
}
// SetVoluntaryExits sets the voluntary exits in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetVoluntaryExits(v []*eth.SignedVoluntaryExit) {
b.voluntaryExits = v
func (b *SignedBeaconBlock) SetVoluntaryExits(v []*eth.SignedVoluntaryExit) {
b.block.body.voluntaryExits = v
}
// SetSyncAggregate sets the sync aggregate in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetSyncAggregate(s *eth.SyncAggregate) error {
func (b *SignedBeaconBlock) SetSyncAggregate(s *eth.SyncAggregate) error {
if b.version == version.Phase0 {
return ErrNotSupported("SyncAggregate", b.version)
}
b.syncAggregate = s
b.block.body.syncAggregate = s
return nil
}
// SetExecution sets the execution payload of the block body.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetExecution(e interfaces.ExecutionData) error {
func (b *SignedBeaconBlock) SetExecution(e interfaces.ExecutionData) error {
if b.version == version.Phase0 || b.version == version.Altair {
return ErrNotSupported("Execution", b.version)
}
if b.isBlinded {
b.executionPayloadHeader = e
if b.block.body.isBlinded {
b.block.body.executionPayloadHeader = e
return nil
}
b.executionPayload = e
b.block.body.executionPayload = e
return nil
}
// SetBLSToExecutionChanges sets the BLS to execution changes in the block.
// This function is not thread safe, it is only used during block creation.
func (b *BeaconBlockBody) SetBLSToExecutionChanges(blsToExecutionChanges []*eth.SignedBLSToExecutionChange) error {
func (b *SignedBeaconBlock) SetBLSToExecutionChanges(blsToExecutionChanges []*eth.SignedBLSToExecutionChange) error {
if b.version < version.Capella {
return ErrNotSupported("BLSToExecutionChanges", b.version)
}
b.blsToExecutionChanges = blsToExecutionChanges
b.block.body.blsToExecutionChanges = blsToExecutionChanges
return nil
}

View File

@@ -15,7 +15,7 @@ type blockMutator struct {
Capella func(beaconBlock *eth.SignedBeaconBlockCapella)
}
func (m blockMutator) apply(b interfaces.SignedBeaconBlock) (interfaces.SignedBeaconBlock, error) {
func (m blockMutator) apply(b interfaces.SignedBeaconBlockWriteable) (interfaces.SignedBeaconBlockWriteable, error) {
switch b.Version() {
case version.Phase0:
bb, err := b.PbPhase0Block()
@@ -51,7 +51,7 @@ func (m blockMutator) apply(b interfaces.SignedBeaconBlock) (interfaces.SignedBe
}
// SetBlockStateRoot modifies the block's state root.
func SetBlockStateRoot(b interfaces.SignedBeaconBlock, sr [32]byte) (interfaces.SignedBeaconBlock, error) {
func SetBlockStateRoot(b interfaces.SignedBeaconBlockWriteable, sr [32]byte) (interfaces.SignedBeaconBlockWriteable, error) {
return blockMutator{
Phase0: func(bb *eth.SignedBeaconBlock) { bb.Block.StateRoot = sr[:] },
Altair: func(bb *eth.SignedBeaconBlockAltair) { bb.Block.StateRoot = sr[:] },
@@ -61,7 +61,7 @@ func SetBlockStateRoot(b interfaces.SignedBeaconBlock, sr [32]byte) (interfaces.
}
// SetBlockParentRoot modifies the block's parent root.
func SetBlockParentRoot(b interfaces.SignedBeaconBlock, pr [32]byte) (interfaces.SignedBeaconBlock, error) {
func SetBlockParentRoot(b interfaces.SignedBeaconBlockWriteable, pr [32]byte) (interfaces.SignedBeaconBlockWriteable, error) {
return blockMutator{
Phase0: func(bb *eth.SignedBeaconBlock) { bb.Block.ParentRoot = pr[:] },
Altair: func(bb *eth.SignedBeaconBlockAltair) { bb.Block.ParentRoot = pr[:] },
@@ -71,7 +71,7 @@ func SetBlockParentRoot(b interfaces.SignedBeaconBlock, pr [32]byte) (interfaces
}
// SetBlockSlot modifies the block's slot.
func SetBlockSlot(b interfaces.SignedBeaconBlock, s primitives.Slot) (interfaces.SignedBeaconBlock, error) {
func SetBlockSlot(b interfaces.SignedBeaconBlockWriteable, s primitives.Slot) (interfaces.SignedBeaconBlockWriteable, error) {
return blockMutator{
Phase0: func(bb *eth.SignedBeaconBlock) { bb.Block.Slot = s },
Altair: func(bb *eth.SignedBeaconBlockAltair) { bb.Block.Slot = s },
@@ -81,7 +81,7 @@ func SetBlockSlot(b interfaces.SignedBeaconBlock, s primitives.Slot) (interfaces
}
// SetProposerIndex modifies the block's proposer index.
func SetProposerIndex(b interfaces.SignedBeaconBlock, idx primitives.ValidatorIndex) (interfaces.SignedBeaconBlock, error) {
func SetProposerIndex(b interfaces.SignedBeaconBlockWriteable, idx primitives.ValidatorIndex) (interfaces.SignedBeaconBlockWriteable, error) {
return blockMutator{
Phase0: func(bb *eth.SignedBeaconBlock) { bb.Block.ProposerIndex = idx },
Altair: func(bb *eth.SignedBeaconBlockAltair) { bb.Block.ProposerIndex = idx },

View File

@@ -15,7 +15,6 @@ import (
type SignedBeaconBlock interface {
Block() BeaconBlock
Signature() [field_params.BLSSignatureLength]byte
SetSignature(sig []byte)
IsNil() bool
Copy() (SignedBeaconBlock, error)
Proto() (proto.Message, error)
@@ -38,17 +37,12 @@ type SignedBeaconBlock interface {
// employed by an object that is a beacon block.
type BeaconBlock interface {
Slot() primitives.Slot
SetSlot(slot primitives.Slot)
ProposerIndex() primitives.ValidatorIndex
SetProposerIndex(idx primitives.ValidatorIndex)
ParentRoot() [field_params.RootLength]byte
SetParentRoot([]byte)
StateRoot() [field_params.RootLength]byte
SetStateRoot([]byte)
Body() BeaconBlockBody
IsNil() bool
IsBlinded() bool
SetBlinded(bool)
HashTreeRoot() ([field_params.RootLength]byte, error)
Proto() (proto.Message, error)
ssz.Marshaler
@@ -63,30 +57,40 @@ type BeaconBlock interface {
// that is a beacon block body.
type BeaconBlockBody interface {
RandaoReveal() [field_params.BLSSignatureLength]byte
SetRandaoReveal([]byte)
Eth1Data() *ethpb.Eth1Data
SetEth1Data(*ethpb.Eth1Data)
Graffiti() [field_params.RootLength]byte
SetGraffiti([]byte)
ProposerSlashings() []*ethpb.ProposerSlashing
SetProposerSlashings([]*ethpb.ProposerSlashing)
AttesterSlashings() []*ethpb.AttesterSlashing
SetAttesterSlashings([]*ethpb.AttesterSlashing)
Attestations() []*ethpb.Attestation
SetAttestations([]*ethpb.Attestation)
Deposits() []*ethpb.Deposit
SetDeposits([]*ethpb.Deposit)
VoluntaryExits() []*ethpb.SignedVoluntaryExit
SetVoluntaryExits([]*ethpb.SignedVoluntaryExit)
SyncAggregate() (*ethpb.SyncAggregate, error)
SetSyncAggregate(*ethpb.SyncAggregate) error
IsNil() bool
HashTreeRoot() ([field_params.RootLength]byte, error)
Proto() (proto.Message, error)
Execution() (ExecutionData, error)
SetExecution(ExecutionData) error
BLSToExecutionChanges() ([]*ethpb.SignedBLSToExecutionChange, error)
}
type SignedBeaconBlockWriteable interface {
SignedBeaconBlock
SetExecution(ExecutionData) error
SetBLSToExecutionChanges([]*ethpb.SignedBLSToExecutionChange) error
SetSyncAggregate(*ethpb.SyncAggregate) error
SetVoluntaryExits([]*ethpb.SignedVoluntaryExit)
SetDeposits([]*ethpb.Deposit)
SetAttestations([]*ethpb.Attestation)
SetAttesterSlashings([]*ethpb.AttesterSlashing)
SetProposerSlashings([]*ethpb.ProposerSlashing)
SetGraffiti([]byte)
SetEth1Data(*ethpb.Eth1Data)
SetRandaoReveal([]byte)
SetBlinded(bool)
SetStateRoot([]byte)
SetParentRoot([]byte)
SetProposerIndex(idx primitives.ValidatorIndex)
SetSlot(slot primitives.Slot)
SetSignature(sig []byte)
}
// ExecutionData represents execution layer information that is contained

View File

@@ -1109,7 +1109,7 @@ func HydrateV2BlindedBeaconBlockBodyCapella(b *v2.BlindedBeaconBlockBodyCapella)
return b
}
func SaveBlock(tb assertions.AssertionTestingTB, ctx context.Context, db iface.NoHeadAccessDatabase, b interface{}) interfaces.SignedBeaconBlock {
func SaveBlock(tb assertions.AssertionTestingTB, ctx context.Context, db iface.NoHeadAccessDatabase, b interface{}) interfaces.SignedBeaconBlockWriteable {
wsb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(tb, err)
require.NoError(tb, db.SaveBlock(ctx, wsb))