Compare commits

...

7 Commits

Author SHA1 Message Date
terence tsao
5c01b6112c No more cache 2023-03-13 13:28:30 -07:00
terence tsao
107de1d66f rename 2023-03-13 12:45:08 -07:00
terence tsao
aa2b0aeb1f Cache block instead of payload 2023-03-13 12:36:48 -07:00
terence tsao
9babc7da2f Fix test 2023-03-13 11:53:20 -07:00
terence tsao
244d64745d Move cache 2023-03-13 11:52:04 -07:00
terence tsao
e94e16fbbc Add usages 2023-03-13 09:25:22 -07:00
terence tsao
79fb746eff Can receive blind block post capella 2023-03-10 15:45:09 -08:00
6 changed files with 278 additions and 3 deletions

View File

@@ -1,6 +1,7 @@
package beacon
import (
"bytes"
"context"
"github.com/pkg/errors"
@@ -584,6 +585,21 @@ func (bs *Server) submitBlindedBellatrixBlock(ctx context.Context, blindedBellat
}
func (bs *Server) submitBlindedCapellaBlock(ctx context.Context, blindedCapellaBlk *ethpbv2.BlindedBeaconBlockCapella, sig []byte) error {
// If we have the full payload in cache, we do not have to goto the relayer for full payload.
if bytes.Equal(blindedCapellaBlk.Body.ExecutionPayloadHeader.BlockHash, bs.LastServedBlock.Body.ExecutionPayload.BlockHash) {
_, err := bs.V1Alpha1ValidatorServer.ProposeBeaconBlock(ctx, &eth.GenericSignedBeaconBlock{
Block: &eth.GenericSignedBeaconBlock_Capella{
Capella: &eth.SignedBeaconBlockCapella{
Block: bs.LastServedBlock,
Signature: sig,
},
},
})
if err != nil {
return status.Errorf(codes.Internal, "Could not propose blinded block: %v", err)
}
}
b, err := migration.BlindedCapellaToV1Alpha1SignedBlock(&ethpbv2.SignedBlindedBeaconBlockCapella{
Message: blindedCapellaBlk,
Signature: sig,

View File

@@ -18,6 +18,7 @@ import (
"github.com/prysmaticlabs/prysm/v3/beacon-chain/rpc/statefetcher"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/sync"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
// Server defines a server implementation of the gRPC Beacon Chain service,
@@ -43,4 +44,5 @@ type Server struct {
ExecutionPayloadReconstructor execution.ExecutionPayloadReconstructor
FinalizationFetcher blockchain.FinalizationFetcher
BLSChangesPool blstoexec.PoolManager
LastServedBlock *ethpb.BeaconBlockCapella
}

View File

@@ -9,6 +9,7 @@ import (
v1alpha1validator "github.com/prysmaticlabs/prysm/v3/beacon-chain/rpc/prysm/v1alpha1/validator"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/rpc/statefetcher"
"github.com/prysmaticlabs/prysm/v3/beacon-chain/sync"
ethpb "github.com/prysmaticlabs/prysm/v3/proto/prysm/v1alpha1"
)
// Server defines a server implementation of the gRPC Validator service,
@@ -26,4 +27,5 @@ type Server struct {
SyncCommitteePool synccommittee.Pool
V1Alpha1Server *v1alpha1validator.Server
ProposerSlotIndexCache *cache.ProposerPayloadIDsCache
LastServedBlock *ethpb.BeaconBlockCapella
}

View File

@@ -539,9 +539,9 @@ func (vs *Server) ProduceBlindedBlock(ctx context.Context, req *ethpbv1.ProduceB
},
}, nil
}
capellaBlock, ok := v1alpha1resp.Block.(*ethpbalpha.GenericBeaconBlock_BlindedCapella)
capellaBlindBlock, ok := v1alpha1resp.Block.(*ethpbalpha.GenericBeaconBlock_BlindedCapella)
if ok {
blk, err := migration.V1Alpha1BeaconBlockBlindedCapellaToV2Blinded(capellaBlock.BlindedCapella)
blk, err := migration.V1Alpha1BeaconBlockBlindedCapellaToV2Blinded(capellaBlindBlock.BlindedCapella)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not prepare beacon block: %v", err)
}
@@ -552,6 +552,21 @@ func (vs *Server) ProduceBlindedBlock(ctx context.Context, req *ethpbv1.ProduceB
},
}, nil
}
capellaBlock, ok := v1alpha1resp.Block.(*ethpbalpha.GenericBeaconBlock_Capella)
if ok {
blind, err := migration.V1Alpha1BeaconBlockCapellaToV2Blinded(capellaBlock.Capella)
if err != nil {
return nil, status.Errorf(codes.Internal, "Could not prepare beacon block: %v", err)
}
vs.LastServedBlock = capellaBlock.Capella
return &ethpbv2.ProduceBlindedBlockResponse{
Version: ethpbv2.Version_CAPELLA,
Data: &ethpbv2.BlindedBeaconBlockContainer{
Block: &ethpbv2.BlindedBeaconBlockContainer_CapellaBlock{CapellaBlock: blind},
},
}, nil
}
return nil, status.Error(codes.InvalidArgument, "Unsupported block type")
}

View File

@@ -2666,7 +2666,7 @@ func TestProduceBlindedBlock(t *testing.T) {
assert.DeepEqual(t, aggregatedSig, blk.Body.SyncAggregate.SyncCommitteeSignature)
})
t.Run("Capella", func(t *testing.T) {
t.Run("Blind Capella", func(t *testing.T) {
db := dbutil.SetupDB(t)
ctx := context.Background()
@@ -2889,6 +2889,242 @@ func TestProduceBlindedBlock(t *testing.T) {
assert.DeepEqual(t, expectedBits, blk.Body.SyncAggregate.SyncCommitteeBits)
assert.DeepEqual(t, aggregatedSig, blk.Body.SyncAggregate.SyncCommitteeSignature)
})
t.Run("Capella", func(t *testing.T) {
db := dbutil.SetupDB(t)
ctx := context.Background()
params.SetupTestConfigCleanup(t)
bc := params.BeaconConfig().Copy()
bc.AltairForkEpoch = primitives.Epoch(0)
bc.BellatrixForkEpoch = primitives.Epoch(1)
bc.CapellaForkEpoch = primitives.Epoch(2)
bc.MaxBuilderConsecutiveMissedSlots = params.BeaconConfig().SlotsPerEpoch*2 + 1
bc.MaxBuilderEpochMissedSlots = params.BeaconConfig().SlotsPerEpoch * 2
params.OverrideBeaconConfig(bc)
beaconState, privKeys := util.DeterministicGenesisStateCapella(t, params.BeaconConfig().SyncCommitteeSize)
require.NoError(t, beaconState.SetSlot(params.BeaconConfig().SlotsPerEpoch*2))
syncCommittee, err := altair.NextSyncCommittee(context.Background(), beaconState)
require.NoError(t, err)
require.NoError(t, beaconState.SetCurrentSyncCommittee(syncCommittee))
require.NoError(t, beaconState.SetNextSyncCommittee(syncCommittee))
ts := time.Now()
ti := ts.Add(-time.Duration(uint64(2*params.BeaconConfig().SlotsPerEpoch+1)*params.BeaconConfig().SecondsPerSlot) * time.Second)
require.NoError(t, beaconState.SetGenesisTime(uint64(ti.Unix())))
stateRoot, err := beaconState.HashTreeRoot(ctx)
require.NoError(t, err, "Could not hash genesis state")
genesisBlock := util.NewBeaconBlockCapella()
genesisBlock.Block.StateRoot = stateRoot[:]
util.SaveBlock(t, ctx, db, genesisBlock)
parentRoot, err := genesisBlock.Block.HashTreeRoot()
require.NoError(t, err)
require.NoError(t, db.SaveState(ctx, beaconState, parentRoot), "Could not save genesis state")
require.NoError(t, db.SaveHeadBlockRoot(ctx, parentRoot), "Could not save genesis state")
fb := util.HydrateSignedBeaconBlockCapella(&ethpbalpha.SignedBeaconBlockCapella{})
fb.Block.Body.ExecutionPayload.GasLimit = 123
wfb, err := blocks.NewSignedBeaconBlock(fb)
require.NoError(t, err)
require.NoError(t, db.SaveBlock(ctx, wfb), "Could not save block")
r, err := wfb.Block().HashTreeRoot()
require.NoError(t, err)
sk, err := bls.RandKey()
require.NoError(t, err)
random, err := helpers.RandaoMix(beaconState, coreTime.CurrentEpoch(beaconState))
require.NoError(t, err)
wds, err := beaconState.ExpectedWithdrawals()
require.NoError(t, err)
wr, err := ssz.WithdrawalSliceRoot(hash.CustomSHA256Hasher(), wds, fieldparams.MaxWithdrawalsPerPayload)
require.NoError(t, err)
bid := &ethpbalpha.BuilderBidCapella{
Header: &enginev1.ExecutionPayloadHeaderCapella{
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: random,
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
TransactionsRoot: make([]byte, fieldparams.RootLength),
BlockNumber: 1,
Timestamp: uint64(ts.Unix()),
WithdrawalsRoot: wr[:],
},
Pubkey: sk.PublicKey().Marshal(),
Value: bytesutil.PadTo([]byte{0}, 32),
}
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 := &ethpbalpha.SignedBuilderBidCapella{
Message: bid,
Signature: sk.Sign(sr[:]).Marshal(),
}
id := &enginev1.PayloadIDBytes{0x1}
chainSlot := 2*params.BeaconConfig().SlotsPerEpoch + 1
fcs := doublylinkedtree.New()
fcs.SetGenesisTime(uint64(ti.Unix()))
mockChainService := &mockChain.ChainService{Slot: &chainSlot, Genesis: ti, State: beaconState, Root: parentRoot[:], ForkChoiceStore: fcs, Block: wfb}
v1Alpha1Server := &v1alpha1validator.Server{
ExecutionEngineCaller: &mockExecution.EngineClient{PayloadIDBytes: id, ExecutionPayloadCapella: &enginev1.ExecutionPayloadCapella{
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: random,
BaseFeePerGas: make([]byte, fieldparams.RootLength),
BlockHash: make([]byte, fieldparams.RootLength),
Timestamp: uint64(ts.Unix()),
BlockNumber: 1,
Withdrawals: wds},
BlockValue: big.NewInt(0)},
BeaconDB: db,
ForkFetcher: mockChainService,
TimeFetcher: mockChainService,
HeadFetcher: mockChainService,
OptimisticModeFetcher: mockChainService,
SyncChecker: &mockSync.Sync{IsSyncing: false},
BlockReceiver: mockChainService,
HeadUpdater: mockChainService,
ChainStartFetcher: &mockExecution.Chain{},
Eth1InfoFetcher: &mockExecution.Chain{},
Eth1BlockFetcher: &mockExecution.Chain{},
MockEth1Votes: true,
AttPool: attestations.NewPool(),
SlashingsPool: slashings.NewPool(),
ExitPool: voluntaryexits.NewPool(),
BLSChangesPool: blstoexec.NewPool(),
StateGen: stategen.New(db, fcs),
SyncCommitteePool: synccommittee.NewStore(),
ProposerSlotIndexCache: cache.NewProposerPayloadIDsCache(),
BlockBuilder: &builderTest.MockBuilderService{
HasConfigured: true,
BidCapella: sBid,
},
FinalizationFetcher: &mockChain.ChainService{
FinalizedCheckPoint: &ethpbalpha.Checkpoint{
Root: r[:],
},
},
}
proposerSlashings := make([]*ethpbalpha.ProposerSlashing, params.BeaconConfig().MaxProposerSlashings)
for i := primitives.ValidatorIndex(0); uint64(i) < params.BeaconConfig().MaxProposerSlashings; i++ {
proposerSlashing, err := util.GenerateProposerSlashingForValidator(
beaconState,
privKeys[i],
i,
)
require.NoError(t, err)
proposerSlashings[i] = proposerSlashing
err = v1Alpha1Server.SlashingsPool.InsertProposerSlashing(context.Background(), beaconState, proposerSlashing)
require.NoError(t, err)
}
attSlashings := make([]*ethpbalpha.AttesterSlashing, params.BeaconConfig().MaxAttesterSlashings)
for i := uint64(0); i < params.BeaconConfig().MaxAttesterSlashings; i++ {
attesterSlashing, err := util.GenerateAttesterSlashingForValidator(
beaconState,
privKeys[i+params.BeaconConfig().MaxProposerSlashings],
primitives.ValidatorIndex(i+params.BeaconConfig().MaxProposerSlashings), /* validator index */
)
require.NoError(t, err)
attSlashings[i] = attesterSlashing
err = v1Alpha1Server.SlashingsPool.InsertAttesterSlashing(context.Background(), beaconState, attesterSlashing)
require.NoError(t, err)
}
aggregationBits := bitfield.NewBitvector128()
for i := range aggregationBits {
aggregationBits[i] = 0xAA
}
syncCommitteeIndices, err := altair.NextSyncCommitteeIndices(context.Background(), beaconState)
require.NoError(t, err)
sigs := make([]bls.Signature, 0, len(syncCommitteeIndices))
for i, indice := range syncCommitteeIndices {
if aggregationBits.BitAt(uint64(i)) {
b := p2pType.SSZBytes(parentRoot[:])
sb, err := signing.ComputeDomainAndSign(beaconState, coreTime.CurrentEpoch(beaconState), &b, params.BeaconConfig().DomainSyncCommittee, privKeys[indice])
require.NoError(t, err)
sig, err := bls.SignatureFromBytes(sb)
require.NoError(t, err)
sigs = append(sigs, sig)
}
}
aggregatedSig := bls.AggregateSignatures(sigs).Marshal()
contribution := &ethpbalpha.SyncCommitteeContribution{
Slot: params.BeaconConfig().SlotsPerEpoch * 2,
BlockRoot: parentRoot[:],
SubcommitteeIndex: 0,
AggregationBits: aggregationBits,
Signature: aggregatedSig,
}
require.NoError(t, v1Alpha1Server.SyncCommitteePool.SaveSyncCommitteeContribution(contribution))
v1Server := &Server{
V1Alpha1Server: v1Alpha1Server,
SyncChecker: &mockSync.Sync{IsSyncing: false},
TimeFetcher: &mockChain.ChainService{},
OptimisticModeFetcher: &mockChain.ChainService{},
}
randaoReveal, err := util.RandaoReveal(beaconState, 1, privKeys)
require.NoError(t, err)
graffiti := bytesutil.ToBytes32([]byte("eth2"))
copied := beaconState.Copy()
require.NoError(t, copied.SetSlot(params.BeaconConfig().SlotsPerEpoch*2+1))
idx, err := helpers.BeaconProposerIndex(ctx, copied)
require.NoError(t, err)
require.NoError(t,
db.SaveRegistrationsByValidatorIDs(ctx, []primitives.ValidatorIndex{idx},
[]*ethpbalpha.ValidatorRegistrationV1{{FeeRecipient: make([]byte, 20), Pubkey: make([]byte, 48)}}))
req := &ethpbv1.ProduceBlockRequest{
Slot: params.BeaconConfig().SlotsPerEpoch*2 + 1,
RandaoReveal: randaoReveal,
Graffiti: graffiti[:],
}
resp, err := v1Server.ProduceBlindedBlock(ctx, req)
require.NoError(t, err)
assert.Equal(t, ethpbv2.Version_CAPELLA, resp.Version)
containerBlock, ok := resp.Data.Block.(*ethpbv2.BlindedBeaconBlockContainer_CapellaBlock)
require.Equal(t, true, ok)
blk := containerBlock.CapellaBlock
assert.Equal(t, req.Slot, blk.Slot, "Expected block to have slot of 1")
assert.DeepEqual(t, parentRoot[:], blk.ParentRoot, "Expected block to have correct parent root")
assert.DeepEqual(t, randaoReveal, blk.Body.RandaoReveal, "Expected block to have correct randao reveal")
assert.DeepEqual(t, req.Graffiti, blk.Body.Graffiti, "Expected block to have correct graffiti")
assert.Equal(t, params.BeaconConfig().MaxProposerSlashings, uint64(len(blk.Body.ProposerSlashings)))
expectedPropSlashings := make([]*ethpbv1.ProposerSlashing, len(proposerSlashings))
for i, slash := range proposerSlashings {
expectedPropSlashings[i] = migration.V1Alpha1ProposerSlashingToV1(slash)
}
assert.DeepEqual(t, expectedPropSlashings, blk.Body.ProposerSlashings)
assert.Equal(t, params.BeaconConfig().MaxAttesterSlashings, uint64(len(blk.Body.AttesterSlashings)))
expectedAttSlashings := make([]*ethpbv1.AttesterSlashing, len(attSlashings))
for i, slash := range attSlashings {
expectedAttSlashings[i] = migration.V1Alpha1AttSlashingToV1(slash)
}
assert.DeepEqual(t, expectedAttSlashings, blk.Body.AttesterSlashings)
expectedBits := bitfield.NewBitvector512()
for i := 0; i <= 15; i++ {
expectedBits[i] = 0xAA
}
assert.DeepEqual(t, expectedBits, blk.Body.SyncAggregate.SyncCommitteeBits)
assert.DeepEqual(t, aggregatedSig, blk.Body.SyncAggregate.SyncCommitteeSignature)
})
}
func TestProduceBlindedBlockSSZ(t *testing.T) {

View File

@@ -190,6 +190,8 @@ func (s *Service) Start() {
withCache := stategen.WithCache(stateCache)
ch := stategen.NewCanonicalHistory(s.cfg.BeaconDB, s.cfg.ChainInfoFetcher, s.cfg.ChainInfoFetcher, withCache)
var lastServedBlock *ethpbv1alpha1.BeaconBlockCapella
validatorServer := &validatorv1alpha1.Server{
Ctx: s.ctx,
AttestationCache: cache.NewAttestationCache(),
@@ -244,6 +246,7 @@ func (s *Service) Start() {
},
SyncCommitteePool: s.cfg.SyncCommitteeObjectPool,
ProposerSlotIndexCache: s.cfg.ProposerIdsCache,
LastServedBlock: lastServedBlock,
}
nodeServer := &nodev1alpha1.Server{
@@ -324,6 +327,7 @@ func (s *Service) Start() {
ExecutionPayloadReconstructor: s.cfg.ExecutionPayloadReconstructor,
BLSChangesPool: s.cfg.BLSChangesPool,
FinalizationFetcher: s.cfg.FinalizationFetcher,
LastServedBlock: lastServedBlock,
}
ethpbv1alpha1.RegisterNodeServer(s.grpcServer, nodeServer)
ethpbservice.RegisterBeaconNodeServer(s.grpcServer, nodeServerV1)