feat(deneb): proposer rpc to handle builder flow (#12554)

This commit is contained in:
terencechain
2023-06-26 08:59:02 -07:00
committed by Preston Van Loon
parent 03fb7ed20c
commit 590309e3fc
11 changed files with 538 additions and 99 deletions

View File

@@ -16,6 +16,7 @@ go_library(
"//consensus-types/primitives:go_default_library",
"//proto/engine/v1:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//runtime/version:go_default_library",
"//time/slots:go_default_library",
"@com_github_pkg_errors//:go_default_library",
],

View File

@@ -13,6 +13,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/consensus-types/primitives"
v1 "github.com/prysmaticlabs/prysm/v4/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
"github.com/prysmaticlabs/prysm/v4/time/slots"
)
@@ -44,27 +45,29 @@ func (s *MockBuilderService) Configured() bool {
}
// SubmitBlindedBlock for mocking.
func (s *MockBuilderService) SubmitBlindedBlock(_ context.Context, _ interfaces.ReadOnlySignedBeaconBlock, _ []*ethpb.SignedBlindedBlobSidecar) (interfaces.ExecutionData, *v1.BlobsBundle, error) {
if s.Payload != nil {
func (s *MockBuilderService) SubmitBlindedBlock(_ context.Context, b interfaces.ReadOnlySignedBeaconBlock, _ []*ethpb.SignedBlindedBlobSidecar) (interfaces.ExecutionData, *v1.BlobsBundle, error) {
switch b.Version() {
case version.Bellatrix:
w, err := blocks.WrappedExecutionPayload(s.Payload)
if err != nil {
return nil, nil, errors.Wrap(err, "could not wrap payload")
}
return w, nil, s.ErrSubmitBlindedBlock
}
if s.PayloadCapella != nil {
case version.Capella:
w, err := blocks.WrappedExecutionPayloadCapella(s.PayloadCapella, 0)
if err != nil {
return nil, nil, errors.Wrap(err, "could not wrap capella payload")
}
return w, nil, s.ErrSubmitBlindedBlock
case version.Deneb:
w, err := blocks.WrappedExecutionPayloadDeneb(s.PayloadDeneb, 0)
if err != nil {
return nil, nil, errors.Wrap(err, "could not wrap deneb payload")
}
return w, s.BlobBundle, s.ErrSubmitBlindedBlock
default:
return nil, nil, errors.New("unknown block version for mocking")
}
w, err := blocks.WrappedExecutionPayloadDeneb(s.PayloadDeneb, 0)
if err != nil {
return nil, nil, errors.Wrap(err, "could not wrap deneb payload")
}
return w, s.BlobBundle, s.ErrSubmitBlindedBlock
}
// GetHeader for mocking.

View File

@@ -148,7 +148,7 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
return nil, status.Errorf(codes.Internal, "Could not get local payload: %v", err)
}
builderPayload, err := vs.getBuilderPayload(ctx, sBlk.Block().Slot(), sBlk.Block().ProposerIndex())
builderPayload, blindBlobsBundle, err := vs.getBuilderPayloadAndBlobs(ctx, sBlk.Block().Slot(), sBlk.Block().ProposerIndex())
if err != nil {
builderGetPayloadMissCount.Inc()
log.WithError(err).Error("Could not get builder payload")
@@ -158,7 +158,7 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
return nil, status.Errorf(codes.Internal, "Could not set execution data: %v", err)
}
if err := setKzgCommitments(sBlk, blobsBundle); err != nil {
if err := setKzgCommitments(sBlk, blobsBundle, blindBlobsBundle); err != nil {
return nil, status.Errorf(codes.Internal, "Could not set kzg commitment: %v", err)
}
@@ -181,7 +181,18 @@ func (vs *Server) GetBeaconBlock(ctx context.Context, req *ethpb.BlockRequest) (
return nil, status.Errorf(codes.Internal, "Could not convert block to proto: %v", err)
}
if slots.ToEpoch(req.Slot) >= params.BeaconConfig().DenebForkEpoch {
// TODO: Handle blind case
if sBlk.IsBlinded() {
scs, err := blindBlobsBundleToSidecars(blindBlobsBundle, sBlk.Block())
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not convert blind blobs bundle to sidecar: %v", err)
}
blockAndBlobs := &ethpb.BlindedBeaconBlockAndBlobsDeneb{
Block: pb.(*ethpb.BlindedBeaconBlockDeneb),
Blobs: scs,
}
return &ethpb.GenericBeaconBlock{Block: &ethpb.GenericBeaconBlock_BlindedDeneb{BlindedDeneb: blockAndBlobs}}, nil
}
scs, err := blobsBundleToSidecars(blobsBundle, sBlk.Block())
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not convert blobs bundle to sidecar: %v", err)
@@ -222,11 +233,18 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
return nil, status.Errorf(codes.InvalidArgument, "%s: %v", CouldNotDecodeBlock, err)
}
unblinder, err := newUnblinder(blk, vs.BlockBuilder)
var blindSidecars []*ethpb.SignedBlindedBlobSidecar
if blk.Version() >= version.Deneb && blk.IsBlinded() {
blindSidecars = req.GetBlindedDeneb().Blobs
}
unblinder, err := newUnblinder(blk, blindSidecars, vs.BlockBuilder)
if err != nil {
return nil, errors.Wrap(err, "could not create unblinder")
}
blk, err = unblinder.unblindBuilderBlock(ctx)
blinded := unblinder.b.IsBlinded() //
blk, unblindedSidecars, err := unblinder.unblindBuilderBlock(ctx)
if err != nil {
return nil, errors.Wrap(err, "could not unblind builder block")
}
@@ -240,23 +258,25 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
return nil, fmt.Errorf("could not broadcast block: %v", err)
}
var scs []*ethpb.SignedBlobSidecar
if blk.Version() >= version.Deneb {
b, ok := req.GetBlock().(*ethpb.GenericSignedBeaconBlock_Deneb)
if !ok {
return nil, status.Error(codes.Internal, "Could not cast block to Deneb")
}
if len(b.Deneb.Blobs) > fieldparams.MaxBlobsPerBlock {
return nil, status.Errorf(codes.InvalidArgument, "Too many blobs in block: %d", len(b.Deneb.Blobs))
}
scs := make([]*ethpb.BlobSidecar, len(b.Deneb.Blobs))
for i, blob := range b.Deneb.Blobs {
if err := vs.P2P.BroadcastBlob(ctx, blob.Message.Index, blob); err != nil {
log.WithError(err).Errorf("Could not broadcast blob index %d / %d", i, len(b.Deneb.Blobs))
if blinded {
scs = unblindedSidecars // Use sidecars from unblinder if the block was blinded.
} else {
scs, err = extraSidecars(req) // Use sidecars from the request if the block was not blinded.
if err != nil {
return nil, errors.Wrap(err, "could not extract blobs")
}
scs[i] = blob.Message
}
sidecars := make([]*ethpb.BlobSidecar, len(scs))
for i, sc := range scs {
if err := vs.P2P.BroadcastBlob(ctx, sc.Message.Index, sc); err != nil {
log.WithError(err).Errorf("Could not broadcast blob sidecar index %d / %d", i, len(scs))
}
sidecars[i] = sc.Message
}
if len(scs) > 0 {
if err := vs.BeaconDB.SaveBlobSidecar(ctx, scs); err != nil {
if err := vs.BeaconDB.SaveBlobSidecar(ctx, sidecars); err != nil {
return nil, err
}
}
@@ -286,6 +306,19 @@ func (vs *Server) ProposeBeaconBlock(ctx context.Context, req *ethpb.GenericSign
}, nil
}
// extraSidecars extracts the sidecars from the request.
// return error if there are too many sidecars.
func extraSidecars(req *ethpb.GenericSignedBeaconBlock) ([]*ethpb.SignedBlobSidecar, error) {
b, ok := req.GetBlock().(*ethpb.GenericSignedBeaconBlock_Deneb)
if !ok {
return nil, errors.New("Could not cast block to Deneb")
}
if len(b.Deneb.Blobs) > fieldparams.MaxBlobsPerBlock {
return nil, fmt.Errorf("too many blobs in block: %d", len(b.Deneb.Blobs))
}
return b.Deneb.Blobs, nil
}
// PrepareBeaconProposer caches and updates the fee recipient for the given proposer.
func (vs *Server) PrepareBeaconProposer(
ctx context.Context, request *ethpb.PrepareBeaconProposerRequest,

View File

@@ -20,6 +20,7 @@ import (
"github.com/prysmaticlabs/prysm/v4/encoding/ssz"
"github.com/prysmaticlabs/prysm/v4/monitoring/tracing"
"github.com/prysmaticlabs/prysm/v4/network/forks"
enginev1 "github.com/prysmaticlabs/prysm/v4/proto/engine/v1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
"github.com/prysmaticlabs/prysm/v4/time/slots"
"github.com/sirupsen/logrus"
@@ -123,26 +124,26 @@ func setExecutionData(ctx context.Context, blk interfaces.SignedBeaconBlock, loc
// This function retrieves the payload header given the slot number and the validator index.
// It's a no-op if the latest head block is not versioned bellatrix.
func (vs *Server) getPayloadHeaderFromBuilder(ctx context.Context, slot primitives.Slot, idx primitives.ValidatorIndex) (interfaces.ExecutionData, error) {
func (vs *Server) getPayloadHeaderFromBuilder(ctx context.Context, slot primitives.Slot, idx primitives.ValidatorIndex) (interfaces.ExecutionData, *enginev1.BlindedBlobsBundle, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.getPayloadHeaderFromBuilder")
defer span.End()
if slots.ToEpoch(slot) < params.BeaconConfig().BellatrixForkEpoch {
return nil, errors.New("can't get payload header from builder before bellatrix epoch")
return nil, nil, errors.New("can't get payload header from builder before bellatrix epoch")
}
b, err := vs.HeadFetcher.HeadBlock(ctx)
if err != nil {
return nil, err
return nil, nil, err
}
h, err := b.Block().Body().Execution()
if err != nil {
return nil, errors.Wrap(err, "failed to get execution header")
return nil, nil, errors.Wrap(err, "failed to get execution header")
}
pk, err := vs.HeadFetcher.HeadValidatorIndexToPublicKey(ctx, idx)
if err != nil {
return nil, err
return nil, nil, err
}
ctx, cancel := context.WithTimeout(ctx, blockBuilderTimeout)
@@ -150,62 +151,71 @@ func (vs *Server) getPayloadHeaderFromBuilder(ctx context.Context, slot primitiv
signedBid, err := vs.BlockBuilder.GetHeader(ctx, slot, bytesutil.ToBytes32(h.BlockHash()), pk)
if err != nil {
return nil, err
return nil, nil, err
}
if signedBid.IsNil() {
return nil, errors.New("builder returned nil bid")
return nil, nil, errors.New("builder returned nil bid")
}
fork, err := forks.Fork(slots.ToEpoch(slot))
if err != nil {
return nil, errors.Wrap(err, "unable to get fork information")
return nil, nil, errors.Wrap(err, "unable to get fork information")
}
forkName, ok := params.BeaconConfig().ForkVersionNames[bytesutil.ToBytes4(fork.CurrentVersion)]
if !ok {
return nil, errors.New("unable to find current fork in schedule")
return nil, nil, errors.New("unable to find current fork in schedule")
}
if !strings.EqualFold(version.String(signedBid.Version()), forkName) {
return nil, fmt.Errorf("builder bid response version: %d is different from head block version: %d for epoch %d", signedBid.Version(), b.Version(), slots.ToEpoch(slot))
return nil, nil, fmt.Errorf("builder bid response version: %d is different from head block version: %d for epoch %d", signedBid.Version(), b.Version(), slots.ToEpoch(slot))
}
bid, err := signedBid.Message()
if err != nil {
return nil, errors.Wrap(err, "could not get bid")
return nil, nil, errors.Wrap(err, "could not get bid")
}
if bid.IsNil() {
return nil, errors.New("builder returned nil bid")
return nil, nil, errors.New("builder returned nil bid")
}
v := bytesutil.LittleEndianBytesToBigInt(bid.Value())
if v.String() == "0" {
return nil, errors.New("builder returned header with 0 bid amount")
return nil, nil, errors.New("builder returned header with 0 bid amount")
}
header, err := bid.Header()
if err != nil {
return nil, errors.Wrap(err, "could not get bid header")
return nil, nil, errors.Wrap(err, "could not get bid header")
}
txRoot, err := header.TransactionsRoot()
if err != nil {
return nil, errors.Wrap(err, "could not get transaction root")
return nil, nil, errors.Wrap(err, "could not get transaction root")
}
if bytesutil.ToBytes32(txRoot) == emptyTransactionsRoot {
return nil, errors.New("builder returned header with an empty tx root")
return nil, nil, errors.New("builder returned header with an empty tx root")
}
if !bytes.Equal(header.ParentHash(), h.BlockHash()) {
return nil, fmt.Errorf("incorrect parent hash %#x != %#x", header.ParentHash(), h.BlockHash())
return nil, nil, fmt.Errorf("incorrect parent hash %#x != %#x", header.ParentHash(), h.BlockHash())
}
t, err := slots.ToTime(uint64(vs.TimeFetcher.GenesisTime().Unix()), slot)
if err != nil {
return nil, err
return nil, nil, err
}
if header.Timestamp() != uint64(t.Unix()) {
return nil, fmt.Errorf("incorrect timestamp %d != %d", header.Timestamp(), uint64(t.Unix()))
return nil, nil, fmt.Errorf("incorrect timestamp %d != %d", header.Timestamp(), uint64(t.Unix()))
}
if err := validateBuilderSignature(signedBid); err != nil {
return nil, errors.Wrap(err, "could not validate builder signature")
return nil, nil, errors.Wrap(err, "could not validate builder signature")
}
var bundle *enginev1.BlindedBlobsBundle
if bid.Version() >= version.Deneb {
bundle, err = bid.BlindedBlobsBundle()
if err != nil {
return nil, nil, errors.Wrap(err, "could not get blinded blobs bundle")
}
log.WithField("blindBlobCount", len(bundle.BlobRoots))
}
log.WithFields(logrus.Fields{
@@ -223,7 +233,7 @@ func (vs *Server) getPayloadHeaderFromBuilder(ctx context.Context, slot primitiv
trace.StringAttribute("blockHash", fmt.Sprintf("%#x", header.BlockHash())),
)
return header, nil
return header, bundle, nil
}
// Validates builder signature and returns an error if the signature is invalid.

View File

@@ -40,7 +40,6 @@ func TestServer_setExecutionData(t *testing.T) {
cfg := params.BeaconConfig().Copy()
cfg.BellatrixForkEpoch = 0
cfg.CapellaForkEpoch = 0
cfg.DenebForkEpoch = 1
params.OverrideBeaconConfig(cfg)
params.SetupTestConfigCleanup(t)
@@ -81,7 +80,7 @@ func TestServer_setExecutionData(t *testing.T) {
b := blk.Block()
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.NoError(t, err)
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
@@ -140,7 +139,7 @@ func TestServer_setExecutionData(t *testing.T) {
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.NoError(t, err)
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
@@ -202,7 +201,7 @@ func TestServer_setExecutionData(t *testing.T) {
b := blk.Block()
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.NoError(t, err)
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
@@ -216,7 +215,7 @@ func TestServer_setExecutionData(t *testing.T) {
b := blk.Block()
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.NoError(t, err)
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
@@ -236,7 +235,7 @@ func TestServer_setExecutionData(t *testing.T) {
b := blk.Block()
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.NoError(t, err)
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
@@ -257,14 +256,19 @@ func TestServer_setExecutionData(t *testing.T) {
b := blk.Block()
localPayload, _, err := vs.getLocalPayloadAndBlobs(ctx, b, capellaTransitionState)
require.NoError(t, err)
builderPayload, err := vs.getBuilderPayload(ctx, b.Slot(), b.ProposerIndex())
builderPayload, _, err := vs.getBuilderPayloadAndBlobs(ctx, b.Slot(), b.ProposerIndex())
require.ErrorIs(t, consensus_types.ErrNilObjectWrapped, err) // Builder returns fault. Use local block
require.NoError(t, setExecutionData(context.Background(), blk, localPayload, builderPayload))
e, err := blk.Block().Body().Execution()
require.NoError(t, err)
require.Equal(t, uint64(4), e.BlockNumber()) // Local block
})
t.Run("Can get payload and blobs Deneb", func(t *testing.T) {
t.Run("Can get local payload and blobs Deneb", func(t *testing.T) {
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = 0
params.OverrideBeaconConfig(cfg)
params.SetupTestConfigCleanup(t)
blk, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
vs.BlockBuilder = &builderTest.MockBuilderService{
@@ -286,6 +290,81 @@ func TestServer_setExecutionData(t *testing.T) {
require.Equal(t, uint64(4), localPayload.BlockNumber())
require.DeepEqual(t, bb, blobsBundle)
})
t.Run("Can get builder payload and blobs in Deneb", func(t *testing.T) {
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = 0
params.OverrideBeaconConfig(cfg)
params.SetupTestConfigCleanup(t)
blk, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
ti, err := slots.ToTime(uint64(time.Now().Unix()), 0)
require.NoError(t, err)
sk, err := bls.RandKey()
require.NoError(t, err)
wr, err := ssz.WithdrawalSliceRoot(withdrawals, fieldparams.MaxWithdrawalsPerPayload)
require.NoError(t, err)
builderValue := bytesutil.ReverseByteOrder(big.NewInt(1e9).Bytes())
bid := &ethpb.BuilderBidDeneb{
Header: &v1.ExecutionPayloadHeaderDeneb{
FeeRecipient: make([]byte, fieldparams.FeeRecipientLength),
StateRoot: make([]byte, fieldparams.RootLength),
ReceiptsRoot: make([]byte, fieldparams.RootLength),
LogsBloom: make([]byte, fieldparams.LogsBloomLength),
PrevRandao: make([]byte, fieldparams.RootLength),
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
TransactionsRoot: bytesutil.PadTo([]byte{1}, fieldparams.RootLength),
ParentHash: params.BeaconConfig().ZeroHash[:],
Timestamp: uint64(ti.Unix()),
BlockNumber: 2,
WithdrawalsRoot: wr[:],
DataGasUsed: 123,
ExcessDataGas: 456,
},
Pubkey: sk.PublicKey().Marshal(),
Value: bytesutil.PadTo(builderValue, 32),
BlindedBlobsBundle: &v1.BlindedBlobsBundle{
KzgCommitments: [][]byte{bytesutil.PadTo([]byte{1}, fieldparams.BLSPubkeyLength), bytesutil.PadTo([]byte{4}, fieldparams.BLSPubkeyLength)},
Proofs: [][]byte{bytesutil.PadTo([]byte{2}, fieldparams.BLSPubkeyLength), bytesutil.PadTo([]byte{5}, fieldparams.BLSPubkeyLength)},
BlobRoots: [][]byte{bytesutil.PadTo([]byte{3}, fieldparams.RootLength), bytesutil.PadTo([]byte{6}, fieldparams.RootLength)},
},
}
d := params.BeaconConfig().DomainApplicationBuilder
domain, err := signing.ComputeDomain(d, nil, nil)
require.NoError(t, err)
sr, err := signing.ComputeSigningRoot(bid, domain)
require.NoError(t, err)
sBid := &ethpb.SignedBuilderBidDeneb{
Message: bid,
Signature: sk.Sign(sr[:]).Marshal(),
}
vs.BlockBuilder = &builderTest.MockBuilderService{
BidDeneb: sBid,
HasConfigured: true,
Cfg: &builderTest.Config{BeaconDB: beaconDB},
}
require.NoError(t, beaconDB.SaveRegistrationsByValidatorIDs(ctx, []primitives.ValidatorIndex{blk.Block().ProposerIndex()},
[]*ethpb.ValidatorRegistrationV1{{FeeRecipient: make([]byte, fieldparams.FeeRecipientLength), Timestamp: uint64(time.Now().Unix()), Pubkey: make([]byte, fieldparams.BLSPubkeyLength)}}))
wb, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
chain := &blockchainTest.ChainService{ForkChoiceStore: doublylinkedtree.New(), Genesis: time.Now(), Block: wb}
vs.ForkFetcher = chain
vs.ForkchoiceFetcher.SetForkChoiceGenesisTime(uint64(time.Now().Unix()))
vs.TimeFetcher = chain
vs.HeadFetcher = chain
require.NoError(t, err)
blk.SetSlot(primitives.Slot(params.BeaconConfig().DenebForkEpoch) * params.BeaconConfig().SlotsPerEpoch)
require.NoError(t, err)
builderPayload, bb, err := vs.getBuilderPayloadAndBlobs(ctx, blk.Block().Slot(), blk.Block().ProposerIndex())
require.NoError(t, err)
require.Equal(t, bid.Header.BlockNumber, builderPayload.BlockNumber()) // header should be the same from block
require.DeepEqual(t, bb, bid.BlindedBlobsBundle) // blind blobs should be the same from block
})
}
func TestServer_getPayloadHeader(t *testing.T) {
genesis := time.Now().Add(-time.Duration(params.BeaconConfig().SlotsPerEpoch) * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second)
@@ -502,7 +581,7 @@ func TestServer_getPayloadHeader(t *testing.T) {
}}
hb, err := vs.HeadFetcher.HeadBlock(context.Background())
require.NoError(t, err)
h, err := vs.getPayloadHeaderFromBuilder(context.Background(), hb.Block().Slot(), 0)
h, _, err := vs.getPayloadHeaderFromBuilder(context.Background(), hb.Block().Slot(), 0)
if tc.err != "" {
require.ErrorContains(t, tc.err, err)
} else {

View File

@@ -11,7 +11,8 @@ import (
// setKzgCommitments sets the KZG commitment on the block.
// Return early if the block version is older than deneb or block slot has not passed deneb epoch.
func setKzgCommitments(blk interfaces.SignedBeaconBlock, bundle *enginev1.BlobsBundle) error {
// Depends on the blk is blind or not, set the KZG commitment from the corresponding bundle.
func setKzgCommitments(blk interfaces.SignedBeaconBlock, bundle *enginev1.BlobsBundle, blindBundle *enginev1.BlindedBlobsBundle) error {
if blk.Version() < version.Deneb {
return nil
}
@@ -19,6 +20,11 @@ func setKzgCommitments(blk interfaces.SignedBeaconBlock, bundle *enginev1.BlobsB
if slots.ToEpoch(slot) < params.BeaconConfig().DenebForkEpoch {
return nil
}
if blk.IsBlinded() {
return blk.SetBlobKzgCommitments(blindBundle.KzgCommitments)
}
return blk.SetBlobKzgCommitments(bundle.KzgCommitments)
}
@@ -46,3 +52,28 @@ func blobsBundleToSidecars(bundle *enginev1.BlobsBundle, blk interfaces.ReadOnly
return sidecars, nil
}
// coverts a blinds blobs bundle to a sidecar format.
func blindBlobsBundleToSidecars(bundle *enginev1.BlindedBlobsBundle, blk interfaces.ReadOnlyBeaconBlock) ([]*ethpb.BlindedBlobSidecar, error) {
r, err := blk.HashTreeRoot()
if err != nil {
return nil, err
}
pr := blk.ParentRoot()
sidecars := make([]*ethpb.BlindedBlobSidecar, len(bundle.BlobRoots))
for i := 0; i < len(bundle.BlobRoots); i++ {
sidecars[i] = &ethpb.BlindedBlobSidecar{
BlockRoot: r[:],
Index: uint64(i),
Slot: blk.Slot(),
BlockParentRoot: pr[:],
ProposerIndex: blk.ProposerIndex(),
BlobRoot: bundle.BlobRoots[i],
KzgCommitment: bundle.KzgCommitments[i],
KzgProof: bundle.Proofs[i],
}
}
return sidecars, nil
}

View File

@@ -14,10 +14,10 @@ import (
func Test_setKzgCommitments(t *testing.T) {
b, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlock())
require.NoError(t, err)
require.NoError(t, setKzgCommitments(b, nil))
require.NoError(t, setKzgCommitments(b, nil, nil))
b, err = blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
require.NoError(t, setKzgCommitments(b, nil))
require.NoError(t, setKzgCommitments(b, nil, nil))
cfg := params.BeaconConfig().Copy()
cfg.DenebForkEpoch = 0
@@ -25,10 +25,20 @@ func Test_setKzgCommitments(t *testing.T) {
kcs := [][]byte{[]byte("kzg"), []byte("kzg1"), []byte("kzg2")}
bundle := &enginev1.BlobsBundle{KzgCommitments: kcs}
require.NoError(t, setKzgCommitments(b, bundle))
bkcs := [][]byte{[]byte("bkzg"), []byte("bkzg1"), []byte("bkzg2")}
blindBundle := &enginev1.BlindedBlobsBundle{KzgCommitments: bkcs}
require.NoError(t, setKzgCommitments(b, bundle, blindBundle))
got, err := b.Block().Body().BlobKzgCommitments()
require.NoError(t, err)
require.DeepEqual(t, got, kcs)
b, err = blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
b.SetBlinded(true)
require.NoError(t, setKzgCommitments(b, bundle, blindBundle))
got, err = b.Block().Body().BlobKzgCommitments()
require.NoError(t, err)
require.DeepEqual(t, got, bkcs)
}
func Test_blobsBundleToSidecars(t *testing.T) {
@@ -62,3 +72,35 @@ func Test_blobsBundleToSidecars(t *testing.T) {
require.DeepEqual(t, sidecars[i].KzgCommitment, kcs[i])
}
}
func Test_blindBlobsBundleToSidecars(t *testing.T) {
b, err := blocks.NewSignedBeaconBlock(util.NewBeaconBlockDeneb())
require.NoError(t, err)
b.SetSlot(1)
b.SetProposerIndex(2)
b.SetParentRoot(bytesutil.PadTo([]byte("parentRoot"), 32))
kcs := [][]byte{[]byte("kzg"), []byte("kzg1"), []byte("kzg2")}
proofs := [][]byte{[]byte("proof"), []byte("proof1"), []byte("proof2")}
blobRoots := [][]byte{[]byte("blob"), []byte("blob1"), []byte("blob2")}
bundle := &enginev1.BlindedBlobsBundle{KzgCommitments: kcs, Proofs: proofs, BlobRoots: blobRoots}
sidecars, err := blindBlobsBundleToSidecars(bundle, b.Block())
require.NoError(t, err)
r, err := b.Block().HashTreeRoot()
require.NoError(t, err)
require.Equal(t, len(sidecars), 3)
for i := 0; i < len(sidecars); i++ {
require.DeepEqual(t, sidecars[i].BlockRoot, r[:])
require.Equal(t, sidecars[i].Index, uint64(i))
require.Equal(t, sidecars[i].Slot, b.Block().Slot())
pr := b.Block().ParentRoot()
require.DeepEqual(t, sidecars[i].BlockParentRoot, pr[:])
require.Equal(t, sidecars[i].ProposerIndex, b.Block().ProposerIndex())
require.DeepEqual(t, sidecars[i].BlobRoot, blobRoots[i])
require.DeepEqual(t, sidecars[i].KzgProof, proofs[i])
require.DeepEqual(t, sidecars[i].KzgCommitment, kcs[i])
}
}

View File

@@ -236,22 +236,22 @@ func (vs *Server) getTerminalBlockHashIfExists(ctx context.Context, transitionTi
return vs.ExecutionEngineCaller.GetTerminalBlockHash(ctx, transitionTime)
}
func (vs *Server) getBuilderPayload(ctx context.Context,
func (vs *Server) getBuilderPayloadAndBlobs(ctx context.Context,
slot primitives.Slot,
vIdx primitives.ValidatorIndex) (interfaces.ExecutionData, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.getBuilderPayload")
vIdx primitives.ValidatorIndex) (interfaces.ExecutionData, *enginev1.BlindedBlobsBundle, error) {
ctx, span := trace.StartSpan(ctx, "ProposerServer.getBuilderPayloadAndBlobs")
defer span.End()
if slots.ToEpoch(slot) < params.BeaconConfig().BellatrixForkEpoch {
return nil, nil
return nil, nil, nil
}
canUseBuilder, err := vs.canUseBuilder(ctx, slot, vIdx)
if err != nil {
return nil, errors.Wrap(err, "failed to check if we can use the builder")
return nil, nil, errors.Wrap(err, "failed to check if we can use the builder")
}
span.AddAttributes(trace.BoolAttribute("canUseBuilder", canUseBuilder))
if !canUseBuilder {
return nil, nil
return nil, nil, nil
}
return vs.getPayloadHeaderFromBuilder(ctx, slot, vIdx)
@@ -300,3 +300,18 @@ func emptyPayloadCapella() *enginev1.ExecutionPayloadCapella {
Withdrawals: make([]*enginev1.Withdrawal, 0),
}
}
func emptyPayloadDeneb() *enginev1.ExecutionPayloadDeneb {
return &enginev1.ExecutionPayloadDeneb{
ParentHash: make([]byte, fieldparams.RootLength),
FeeRecipient: make([]byte, fieldparams.FeeRecipientLength),
StateRoot: make([]byte, fieldparams.RootLength),
ReceiptsRoot: make([]byte, fieldparams.RootLength),
LogsBloom: make([]byte, fieldparams.LogsBloomLength),
PrevRandao: make([]byte, fieldparams.RootLength),
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
Transactions: make([][]byte, 0),
Withdrawals: make([]*enginev1.Withdrawal, 0),
}
}

View File

@@ -721,7 +721,7 @@ func TestProposer_ProposeBlock_OK(t *testing.T) {
},
{
name: "deneb block has too many blobs",
err: "Too many blobs in block: 7",
err: "too many blobs in block: 7",
block: func(parent [32]byte) *ethpb.GenericSignedBeaconBlock {
blockToPropose := util.NewBeaconBlockDeneb()
blockToPropose.Block.Slot = 5
@@ -741,6 +741,38 @@ func TestProposer_ProposeBlock_OK(t *testing.T) {
return &ethpb.GenericSignedBeaconBlock{Block: blk}
},
},
{
name: "blind capella",
block: func(parent [32]byte) *ethpb.GenericSignedBeaconBlock {
blockToPropose := util.NewBlindedBeaconBlockDeneb()
blockToPropose.Block.Slot = 5
blockToPropose.Block.ParentRoot = parent[:]
txRoot, err := ssz.TransactionsRoot([][]byte{})
require.NoError(t, err)
withdrawalsRoot, err := ssz.WithdrawalSliceRoot([]*enginev1.Withdrawal{}, fieldparams.MaxWithdrawalsPerPayload)
require.NoError(t, err)
blockToPropose.Block.Body.ExecutionPayloadHeader.TransactionsRoot = txRoot[:]
blockToPropose.Block.Body.ExecutionPayloadHeader.WithdrawalsRoot = withdrawalsRoot[:]
blk := &ethpb.GenericSignedBeaconBlock_BlindedDeneb{BlindedDeneb: &ethpb.SignedBlindedBeaconBlockAndBlobsDeneb{
Block: blockToPropose,
Blobs: []*ethpb.SignedBlindedBlobSidecar{
{
Message: &ethpb.BlindedBlobSidecar{
BlockRoot: []byte{0x01},
Slot: 2,
BlockParentRoot: []byte{0x03},
ProposerIndex: 3,
BlobRoot: []byte{0x04},
KzgCommitment: []byte{0x05},
KzgProof: []byte{0x06},
},
Signature: []byte{0x07},
},
},
}}
return &ethpb.GenericSignedBeaconBlock{Block: blk}
},
},
}
for _, tt := range tests {
@@ -758,7 +790,7 @@ func TestProposer_ProposeBlock_OK(t *testing.T) {
BlockReceiver: c,
BlockNotifier: c.BlockNotifier(),
P2P: mockp2p.NewTestP2P(t),
BlockBuilder: &builderTest.MockBuilderService{HasConfigured: true, PayloadCapella: emptyPayloadCapella()},
BlockBuilder: &builderTest.MockBuilderService{HasConfigured: true, PayloadCapella: emptyPayloadCapella(), PayloadDeneb: emptyPayloadDeneb(), BlobBundle: &enginev1.BlobsBundle{KzgCommitments: [][]byte{{0x01}}, Proofs: [][]byte{{0x02}}, Blobs: [][]byte{{0x03}}}},
BeaconDB: db,
}
blockToPropose := tt.block(bsRoot)
@@ -2817,3 +2849,19 @@ func TestProposer_GetFeeRecipientByPubKey(t *testing.T) {
require.Equal(t, common.HexToAddress("0x055Fb65722E7b2455012BFEBf6177F1D2e9728D8").Hex(), common.BytesToAddress(resp.FeeRecipient).Hex())
}
func Test_extractBlobs(t *testing.T) {
blobs := []*ethpb.SignedBlobSidecar{
{Message: &ethpb.BlobSidecar{Index: 0}}, {Message: &ethpb.BlobSidecar{Index: 1}},
{Message: &ethpb.BlobSidecar{Index: 2}}, {Message: &ethpb.BlobSidecar{Index: 3}},
{Message: &ethpb.BlobSidecar{Index: 4}}, {Message: &ethpb.BlobSidecar{Index: 5}}}
req := &ethpb.GenericSignedBeaconBlock{Block: &ethpb.GenericSignedBeaconBlock_Deneb{
Deneb: &ethpb.SignedBeaconBlockAndBlobsDeneb{
Blobs: blobs,
},
},
}
bs, err := extraSidecars(req)
require.NoError(t, err)
require.DeepEqual(t, blobs, bs)
}

View File

@@ -9,6 +9,8 @@ import (
consensus_types "github.com/prysmaticlabs/prysm/v4/consensus-types"
consensusblocks "github.com/prysmaticlabs/prysm/v4/consensus-types/blocks"
"github.com/prysmaticlabs/prysm/v4/consensus-types/interfaces"
"github.com/prysmaticlabs/prysm/v4/encoding/bytesutil"
enginev1 "github.com/prysmaticlabs/prysm/v4/proto/engine/v1"
ethpb "github.com/prysmaticlabs/prysm/v4/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/v4/runtime/version"
"github.com/sirupsen/logrus"
@@ -17,10 +19,11 @@ import (
type unblinder struct {
b interfaces.SignedBeaconBlock
blobs []*ethpb.SignedBlindedBlobSidecar
builder builder.BlockBuilder
}
func newUnblinder(b interfaces.SignedBeaconBlock, builder builder.BlockBuilder) (*unblinder, error) {
func newUnblinder(b interfaces.SignedBeaconBlock, blobs []*ethpb.SignedBlindedBlobSidecar, builder builder.BlockBuilder) (*unblinder, error) {
if err := consensusblocks.BeaconBlockIsNil(b); err != nil {
return nil, err
}
@@ -29,74 +32,78 @@ func newUnblinder(b interfaces.SignedBeaconBlock, builder builder.BlockBuilder)
}
return &unblinder{
b: b,
blobs: blobs,
builder: builder,
}, nil
}
func (u *unblinder) unblindBuilderBlock(ctx context.Context) (interfaces.SignedBeaconBlock, error) {
func (u *unblinder) unblindBuilderBlock(ctx context.Context) (interfaces.SignedBeaconBlock, []*ethpb.SignedBlobSidecar, error) {
if !u.b.IsBlinded() || u.b.Version() < version.Bellatrix {
return u.b, nil
return u.b, nil, nil
}
if u.b.IsBlinded() && !u.builder.Configured() {
return nil, errors.New("builder not configured")
return nil, nil, errors.New("builder not configured")
}
psb, err := u.blindedProtoBlock()
if err != nil {
return nil, errors.Wrap(err, "could not get blinded proto block")
return nil, nil, errors.Wrap(err, "could not get blinded proto block")
}
sb, err := consensusblocks.NewSignedBeaconBlock(psb)
if err != nil {
return nil, errors.Wrap(err, "could not create signed block")
return nil, nil, errors.Wrap(err, "could not create signed block")
}
if err = copyBlockData(u.b, sb); err != nil {
return nil, errors.Wrap(err, "could not copy block data")
return nil, nil, errors.Wrap(err, "could not copy block data")
}
h, err := u.b.Block().Body().Execution()
if err != nil {
return nil, errors.Wrap(err, "could not get execution")
return nil, nil, errors.Wrap(err, "could not get execution")
}
if err = sb.SetExecution(h); err != nil {
return nil, errors.Wrap(err, "could not set execution")
return nil, nil, errors.Wrap(err, "could not set execution")
}
// TODO: replace nil with proper variable for signed blinded blob sidecars
// replace _ with blob bundle and use it in the response.
payload, _, err := u.builder.SubmitBlindedBlock(ctx, sb, nil)
payload, blobsBundle, err := u.builder.SubmitBlindedBlock(ctx, sb, u.blobs)
if err != nil {
return nil, errors.Wrap(err, "could not submit blinded block")
return nil, nil, errors.Wrap(err, "could not submit blinded block")
}
headerRoot, err := h.HashTreeRoot()
if err != nil {
return nil, errors.Wrap(err, "could not get header root")
return nil, nil, errors.Wrap(err, "could not get header root")
}
payloadRoot, err := payload.HashTreeRoot()
if err != nil {
return nil, errors.Wrap(err, "could not get payload root")
return nil, nil, errors.Wrap(err, "could not get payload root")
}
if headerRoot != payloadRoot {
return nil, fmt.Errorf("header and payload root do not match, consider disconnect from relay to avoid further issues, "+
return nil, nil, fmt.Errorf("header and payload root do not match, consider disconnect from relay to avoid further issues, "+
"%#x != %#x", headerRoot, payloadRoot)
}
bb, err := u.protoBlock()
if err != nil {
return nil, errors.Wrap(err, "could not get proto block")
return nil, nil, errors.Wrap(err, "could not get proto block")
}
wb, err := consensusblocks.NewSignedBeaconBlock(bb)
if err != nil {
return nil, errors.Wrap(err, "could not create signed block")
return nil, nil, errors.Wrap(err, "could not create signed block")
}
if err = copyBlockData(sb, wb); err != nil {
return nil, errors.Wrap(err, "could not copy block data")
return nil, nil, errors.Wrap(err, "could not copy block data")
}
if err = wb.SetExecution(payload); err != nil {
return nil, errors.Wrap(err, "could not set execution")
return nil, nil, errors.Wrap(err, "could not set execution")
}
txs, err := payload.Transactions()
if err != nil {
return nil, errors.Wrap(err, "could not get transactions from payload")
return nil, nil, errors.Wrap(err, "could not get transactions from payload")
}
if wb.Version() >= version.Bellatrix && blobsBundle != nil {
log.WithField("blobCount", len(blobsBundle.Blobs))
}
log.WithFields(logrus.Fields{
"blockHash": fmt.Sprintf("%#x", h.BlockHash()),
"feeRecipient": fmt.Sprintf("%#x", h.FeeRecipient()),
@@ -105,7 +112,36 @@ func (u *unblinder) unblindBuilderBlock(ctx context.Context) (interfaces.SignedB
"txs": len(txs),
}).Info("Retrieved full payload from builder")
return wb, nil
bundle, err := unblindBlobsSidecars(u.blobs, blobsBundle)
if err != nil {
return nil, nil, errors.Wrap(err, "could not unblind blobs sidecars")
}
return wb, bundle, nil
}
func unblindBlobsSidecars(blindSidecars []*ethpb.SignedBlindedBlobSidecar, bundle *enginev1.BlobsBundle) ([]*ethpb.SignedBlobSidecar, error) {
if bundle == nil {
return nil, nil
}
sidecars := make([]*ethpb.SignedBlobSidecar, len(blindSidecars))
for i, b := range blindSidecars {
sidecars[i] = &ethpb.SignedBlobSidecar{
Message: &ethpb.BlobSidecar{
BlockRoot: bytesutil.SafeCopyBytes(b.Message.BlockRoot),
Index: b.Message.Index,
Slot: b.Message.Slot,
BlockParentRoot: bytesutil.SafeCopyBytes(b.Message.BlockParentRoot),
ProposerIndex: b.Message.ProposerIndex,
Blob: bytesutil.SafeCopyBytes(bundle.Blobs[i]),
KzgCommitment: bytesutil.SafeCopyBytes(b.Message.KzgCommitment),
KzgProof: bytesutil.SafeCopyBytes(b.Message.KzgProof),
},
Signature: bytesutil.SafeCopyBytes(b.Signature),
}
}
return sidecars, nil
}
func copyBlockData(src interfaces.SignedBeaconBlock, dst interfaces.SignedBeaconBlock) error {
@@ -160,6 +196,12 @@ func (u *unblinder) blindedProtoBlock() (proto.Message, error) {
Body: &ethpb.BlindedBeaconBlockBodyCapella{},
},
}, nil
case version.Deneb:
return &ethpb.SignedBlindedBeaconBlockDeneb{
Block: &ethpb.BlindedBeaconBlockDeneb{
Body: &ethpb.BlindedBeaconBlockBodyDeneb{},
},
}, nil
default:
return nil, fmt.Errorf("invalid version %s", version.String(u.b.Version()))
}
@@ -179,6 +221,12 @@ func (u *unblinder) protoBlock() (proto.Message, error) {
Body: &ethpb.BeaconBlockBodyCapella{},
},
}, nil
case version.Deneb:
return &ethpb.SignedBeaconBlockDeneb{
Block: &ethpb.BeaconBlockDeneb{
Body: &ethpb.BeaconBlockBodyDeneb{},
},
}, nil
default:
return nil, fmt.Errorf("invalid version %s", version.String(u.b.Version()))
}

View File

@@ -21,13 +21,19 @@ func Test_unblindBuilderBlock(t *testing.T) {
p.GasLimit = 123
pCapella := emptyPayloadCapella()
pCapella.GasLimit = 123
pDeneb := emptyPayloadDeneb()
pDeneb.GasLimit = 123
pDeneb.ExcessDataGas = 456
pDeneb.DataGasUsed = 789
tests := []struct {
name string
blk interfaces.SignedBeaconBlock
mock *builderTest.MockBuilderService
err string
returnedBlk interfaces.SignedBeaconBlock
name string
blk interfaces.SignedBeaconBlock
blindBlobs []*eth.SignedBlindedBlobSidecar
mock *builderTest.MockBuilderService
err string
returnedBlk interfaces.SignedBeaconBlock
returnedBlobSidecars []*eth.SignedBlobSidecar
}{
{
name: "old block version",
@@ -247,17 +253,140 @@ func Test_unblindBuilderBlock(t *testing.T) {
return wb
}(),
},
{
name: "can get payload and blobs Deneb",
blindBlobs: func() []*eth.SignedBlindedBlobSidecar {
blobs := make([]*eth.SignedBlindedBlobSidecar, fieldparams.MaxBlobsPerBlock)
for i := 0; i < fieldparams.MaxBlobsPerBlock; i++ {
blobs[i] = &eth.SignedBlindedBlobSidecar{
Message: &eth.BlindedBlobSidecar{
BlockRoot: []byte{'a'},
Index: uint64(i),
Slot: 1,
BlockParentRoot: []byte{'b'},
ProposerIndex: 2,
BlobRoot: []byte{'c', byte(i)}, // Add i for uniqueness
KzgCommitment: []byte{'d', byte(i)},
KzgProof: []byte{'e', byte(i)},
},
Signature: []byte("sig"),
}
}
return blobs
}(),
blk: func() interfaces.SignedBeaconBlock {
b := util.NewBlindedBeaconBlockDeneb()
b.Block.Slot = 1
b.Block.ProposerIndex = 2
b.Block.Body.BlsToExecutionChanges = []*eth.SignedBLSToExecutionChange{
{
Message: &eth.BLSToExecutionChange{
ValidatorIndex: 123,
FromBlsPubkey: []byte{'a'},
ToExecutionAddress: []byte{'a'},
},
Signature: []byte("sig123"),
},
{
Message: &eth.BLSToExecutionChange{
ValidatorIndex: 456,
FromBlsPubkey: []byte{'b'},
ToExecutionAddress: []byte{'b'},
},
Signature: []byte("sig456"),
},
}
txRoot, err := ssz.TransactionsRoot([][]byte{})
require.NoError(t, err)
withdrawalsRoot, err := ssz.WithdrawalSliceRoot([]*v1.Withdrawal{}, fieldparams.MaxWithdrawalsPerPayload)
require.NoError(t, err)
b.Block.Body.ExecutionPayloadHeader = &v1.ExecutionPayloadHeaderDeneb{
ParentHash: make([]byte, fieldparams.RootLength),
FeeRecipient: make([]byte, fieldparams.FeeRecipientLength),
StateRoot: make([]byte, fieldparams.RootLength),
ReceiptsRoot: make([]byte, fieldparams.RootLength),
LogsBloom: make([]byte, fieldparams.LogsBloomLength),
PrevRandao: make([]byte, fieldparams.RootLength),
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
TransactionsRoot: txRoot[:],
WithdrawalsRoot: withdrawalsRoot[:],
GasLimit: 123,
ExcessDataGas: 456,
DataGasUsed: 789,
}
wb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return wb
}(),
mock: &builderTest.MockBuilderService{
HasConfigured: true,
PayloadDeneb: pDeneb,
BlobBundle: &v1.BlobsBundle{
KzgCommitments: [][]byte{{'c', 0}, {'c', 1}, {'c', 2}, {'c', 3}, {'c', 4}, {'c', 5}},
Proofs: [][]byte{{'d', 0}, {'d', 1}, {'d', 2}, {'d', 3}, {'d', 4}, {'d', 5}},
Blobs: [][]byte{{'f', 0}, {'f', 1}, {'f', 2}, {'f', 3}, {'f', 4}, {'f', 5}},
},
},
returnedBlk: func() interfaces.SignedBeaconBlock {
b := util.NewBeaconBlockDeneb()
b.Block.Slot = 1
b.Block.ProposerIndex = 2
b.Block.Body.BlsToExecutionChanges = []*eth.SignedBLSToExecutionChange{
{
Message: &eth.BLSToExecutionChange{
ValidatorIndex: 123,
FromBlsPubkey: []byte{'a'},
ToExecutionAddress: []byte{'a'},
},
Signature: []byte("sig123"),
},
{
Message: &eth.BLSToExecutionChange{
ValidatorIndex: 456,
FromBlsPubkey: []byte{'b'},
ToExecutionAddress: []byte{'b'},
},
Signature: []byte("sig456"),
},
}
b.Block.Body.ExecutionPayload = pDeneb
wb, err := blocks.NewSignedBeaconBlock(b)
require.NoError(t, err)
return wb
}(),
returnedBlobSidecars: func() []*eth.SignedBlobSidecar {
blobs := make([]*eth.SignedBlobSidecar, fieldparams.MaxBlobsPerBlock)
for i := 0; i < fieldparams.MaxBlobsPerBlock; i++ {
blobs[i] = &eth.SignedBlobSidecar{
Message: &eth.BlobSidecar{
BlockRoot: []byte{'a'},
Index: uint64(i),
Slot: 1,
BlockParentRoot: []byte{'b'},
ProposerIndex: 2,
Blob: []byte{'f', byte(i)},
KzgCommitment: []byte{'d', byte(i)},
KzgProof: []byte{'e', byte(i)},
},
Signature: []byte("sig"),
}
}
return blobs
}(),
},
}
for _, tc := range tests {
t.Run(tc.name, func(t *testing.T) {
unblinder, err := newUnblinder(tc.blk, tc.mock)
unblinder, err := newUnblinder(tc.blk, tc.blindBlobs, tc.mock)
require.NoError(t, err)
gotBlk, err := unblinder.unblindBuilderBlock(context.Background())
gotBlk, gotBlobs, err := unblinder.unblindBuilderBlock(context.Background())
if tc.err != "" {
require.ErrorContains(t, tc.err, err)
} else {
require.NoError(t, err)
require.DeepEqual(t, tc.returnedBlk, gotBlk)
require.DeepEqual(t, tc.returnedBlobSidecars, gotBlobs)
}
})
}