mirror of
https://github.com/OffchainLabs/prysm.git
synced 2026-01-08 23:18:15 -05:00
Implement EIP7691: Blob throughput increase (#14750)
* Add EIP-7691 blob throughput increase * Review feedback * Factorize blobs subscriptions. --------- Co-authored-by: Manu NALEPA <enalepa@offchainlabs.com>
This commit is contained in:
@@ -16,6 +16,7 @@ The format is based on Keep a Changelog, and this project adheres to Semantic Ve
|
||||
- Log before blob filesystem cache warm-up.
|
||||
- New design for the attestation pool. [PR](https://github.com/prysmaticlabs/prysm/pull/14324)
|
||||
- Add field param placeholder for Electra blob target and max to pass spec tests.
|
||||
- Add EIP-7691: Blob throughput increase.
|
||||
|
||||
### Changed
|
||||
|
||||
|
||||
@@ -136,30 +136,18 @@ func (s *Service) NewPayload(ctx context.Context, payload interfaces.ExecutionDa
|
||||
defer cancel()
|
||||
result := &pb.PayloadStatus{}
|
||||
|
||||
switch payload.Proto().(type) {
|
||||
switch payloadPb := payload.Proto().(type) {
|
||||
case *pb.ExecutionPayload:
|
||||
payloadPb, ok := payload.Proto().(*pb.ExecutionPayload)
|
||||
if !ok {
|
||||
return nil, errors.New("execution data must be a Bellatrix or Capella execution payload")
|
||||
}
|
||||
err := s.rpcClient.CallContext(ctx, result, NewPayloadMethod, payloadPb)
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
case *pb.ExecutionPayloadCapella:
|
||||
payloadPb, ok := payload.Proto().(*pb.ExecutionPayloadCapella)
|
||||
if !ok {
|
||||
return nil, errors.New("execution data must be a Capella execution payload")
|
||||
}
|
||||
err := s.rpcClient.CallContext(ctx, result, NewPayloadMethodV2, payloadPb)
|
||||
if err != nil {
|
||||
return nil, handleRPCError(err)
|
||||
}
|
||||
case *pb.ExecutionPayloadDeneb:
|
||||
payloadPb, ok := payload.Proto().(*pb.ExecutionPayloadDeneb)
|
||||
if !ok {
|
||||
return nil, errors.New("execution data must be a Deneb execution payload")
|
||||
}
|
||||
if executionRequests == nil {
|
||||
err := s.rpcClient.CallContext(ctx, result, NewPayloadMethodV3, payloadPb, versionedHashes, parentBlockRoot)
|
||||
if err != nil {
|
||||
|
||||
@@ -73,6 +73,9 @@ const (
|
||||
RPCBlocksByRootTopicV2 = protocolPrefix + BeaconBlocksByRootsMessageName + SchemaVersionV2
|
||||
// RPCMetaDataTopicV2 defines the v2 topic for the metadata rpc method.
|
||||
RPCMetaDataTopicV2 = protocolPrefix + MetadataMessageName + SchemaVersionV2
|
||||
|
||||
RPCBlobSidecarsByRangeTopicV2 = protocolPrefix + BlobSidecarsByRangeName + SchemaVersionV2
|
||||
RPCBlobSidecarsByRootTopicV2 = protocolPrefix + BlobSidecarsByRootName + SchemaVersionV2
|
||||
)
|
||||
|
||||
// RPC errors for topic parsing.
|
||||
|
||||
@@ -190,7 +190,7 @@ func TestGetSpec(t *testing.T) {
|
||||
data, ok := resp.Data.(map[string]interface{})
|
||||
require.Equal(t, true, ok)
|
||||
|
||||
assert.Equal(t, 159, len(data))
|
||||
assert.Equal(t, 160, len(data))
|
||||
for k, v := range data {
|
||||
t.Run(k, func(t *testing.T) {
|
||||
switch k {
|
||||
@@ -533,6 +533,8 @@ func TestGetSpec(t *testing.T) {
|
||||
assert.Equal(t, "6", v)
|
||||
case "MAX_BLOBS_PER_BLOCK_ELECTRA":
|
||||
assert.Equal(t, "9", v)
|
||||
case "MAX_REQUEST_BLOB_SIDECARS_ELECTRA":
|
||||
assert.Equal(t, "1152", v)
|
||||
default:
|
||||
t.Errorf("Incorrect key: %s", k)
|
||||
}
|
||||
|
||||
@@ -220,8 +220,12 @@ func (c *blobsTestCase) setup(t *testing.T) (*Service, []blocks.ROBlob, func())
|
||||
|
||||
byRootRate := params.BeaconConfig().MaxRequestBlobSidecars * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
|
||||
byRangeRate := params.BeaconConfig().MaxRequestBlobSidecars * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
|
||||
byRootRateElectra := params.BeaconConfig().MaxRequestBlobSidecarsElectra * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
|
||||
byRangeRateElectra := params.BeaconConfig().MaxRequestBlobSidecarsElectra * uint64(params.BeaconConfig().MaxBlobsPerBlock(0))
|
||||
s.setRateCollector(p2p.RPCBlobSidecarsByRootTopicV1, leakybucket.NewCollector(0.000001, int64(byRootRate), time.Second, false))
|
||||
s.setRateCollector(p2p.RPCBlobSidecarsByRangeTopicV1, leakybucket.NewCollector(0.000001, int64(byRangeRate), time.Second, false))
|
||||
s.setRateCollector(p2p.RPCBlobSidecarsByRootTopicV2, leakybucket.NewCollector(0.000001, int64(byRootRateElectra), time.Second, false))
|
||||
s.setRateCollector(p2p.RPCBlobSidecarsByRangeTopicV2, leakybucket.NewCollector(0.000001, int64(byRangeRateElectra), time.Second, false))
|
||||
|
||||
return s, sidecars, cleanup
|
||||
}
|
||||
|
||||
@@ -188,6 +188,50 @@ func TestService_CheckForNextEpochFork(t *testing.T) {
|
||||
assert.Equal(t, true, rpcMap[p2p.RPCBlobSidecarsByRootTopicV1+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
|
||||
},
|
||||
},
|
||||
{
|
||||
name: "electra fork in the next epoch",
|
||||
svcCreator: func(t *testing.T) *Service {
|
||||
peer2peer := p2ptest.NewTestP2P(t)
|
||||
gt := time.Now().Add(-4 * oneEpoch())
|
||||
vr := [32]byte{'A'}
|
||||
chainService := &mockChain.ChainService{
|
||||
Genesis: gt,
|
||||
ValidatorsRoot: vr,
|
||||
}
|
||||
bCfg := params.BeaconConfig().Copy()
|
||||
bCfg.ElectraForkEpoch = 5
|
||||
params.OverrideBeaconConfig(bCfg)
|
||||
params.BeaconConfig().InitializeForkSchedule()
|
||||
ctx, cancel := context.WithCancel(context.Background())
|
||||
r := &Service{
|
||||
ctx: ctx,
|
||||
cancel: cancel,
|
||||
cfg: &config{
|
||||
p2p: peer2peer,
|
||||
chain: chainService,
|
||||
clock: startup.NewClock(gt, vr),
|
||||
initialSync: &mockSync.Sync{IsSyncing: false},
|
||||
},
|
||||
chainStarted: abool.New(),
|
||||
subHandler: newSubTopicHandler(),
|
||||
}
|
||||
return r
|
||||
},
|
||||
currEpoch: 4,
|
||||
wantErr: false,
|
||||
postSvcCheck: func(t *testing.T, s *Service) {
|
||||
genRoot := s.cfg.clock.GenesisValidatorsRoot()
|
||||
digest, err := forks.ForkDigestFromEpoch(5, genRoot[:])
|
||||
assert.NoError(t, err)
|
||||
assert.Equal(t, true, s.subHandler.digestExists(digest))
|
||||
rpcMap := make(map[string]bool)
|
||||
for _, p := range s.cfg.p2p.Host().Mux().Protocols() {
|
||||
rpcMap[string(p)] = true
|
||||
}
|
||||
assert.Equal(t, true, rpcMap[p2p.RPCBlobSidecarsByRangeTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
|
||||
assert.Equal(t, true, rpcMap[p2p.RPCBlobSidecarsByRootTopicV2+s.cfg.p2p.Encoding().ProtocolSuffix()], "topic doesn't exist")
|
||||
},
|
||||
},
|
||||
}
|
||||
for _, tt := range tests {
|
||||
t.Run(tt.name, func(t *testing.T) {
|
||||
|
||||
@@ -79,6 +79,8 @@ func newRateLimiter(p2pProvider p2p.P2P) *limiter {
|
||||
topicMap[addEncoding(p2p.RPCBlobSidecarsByRootTopicV1)] = blobCollector
|
||||
// BlobSidecarsByRangeV1
|
||||
topicMap[addEncoding(p2p.RPCBlobSidecarsByRangeTopicV1)] = blobCollector
|
||||
topicMap[addEncoding(p2p.RPCBlobSidecarsByRootTopicV2)] = blobCollector
|
||||
topicMap[addEncoding(p2p.RPCBlobSidecarsByRangeTopicV2)] = blobCollector
|
||||
|
||||
// General topic for all rpc requests.
|
||||
topicMap[rpcLimiterTopic] = leakybucket.NewCollector(5, defaultBurstLimit*2, leakyBucketPeriod, false /* deleteEmptyBuckets */)
|
||||
|
||||
@@ -18,7 +18,7 @@ import (
|
||||
|
||||
func TestNewRateLimiter(t *testing.T) {
|
||||
rlimiter := newRateLimiter(mockp2p.NewTestP2P(t))
|
||||
assert.Equal(t, len(rlimiter.limiterMap), 12, "correct number of topics not registered")
|
||||
assert.Equal(t, len(rlimiter.limiterMap), 14, "correct number of topics not registered")
|
||||
}
|
||||
|
||||
func TestNewRateLimiter_FreeCorrectly(t *testing.T) {
|
||||
|
||||
@@ -65,7 +65,7 @@ func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandle
|
||||
|
||||
// Deneb: https://github.com/ethereum/consensus-specs/blob/dev/specs/deneb/p2p-interface.md#messages
|
||||
// Electra: https://github.com/ethereum/consensus-specs/blob/dev/specs/electra/p2p-interface.md#messages
|
||||
case version.Deneb, version.Electra:
|
||||
case version.Deneb:
|
||||
return map[string]rpcHandler{
|
||||
p2p.RPCStatusTopicV1: s.statusRPCHandler,
|
||||
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
|
||||
@@ -76,7 +76,17 @@ func (s *Service) rpcHandlerByTopicFromFork(forkIndex int) (map[string]rpcHandle
|
||||
p2p.RPCBlobSidecarsByRootTopicV1: s.blobSidecarByRootRPCHandler, // Added in Deneb
|
||||
p2p.RPCBlobSidecarsByRangeTopicV1: s.blobSidecarsByRangeRPCHandler, // Added in Deneb
|
||||
}, nil
|
||||
|
||||
case version.Electra:
|
||||
return map[string]rpcHandler{
|
||||
p2p.RPCStatusTopicV1: s.statusRPCHandler,
|
||||
p2p.RPCGoodByeTopicV1: s.goodbyeRPCHandler,
|
||||
p2p.RPCBlocksByRangeTopicV2: s.beaconBlocksByRangeRPCHandler,
|
||||
p2p.RPCBlocksByRootTopicV2: s.beaconBlocksRootRPCHandler,
|
||||
p2p.RPCPingTopicV1: s.pingHandler,
|
||||
p2p.RPCMetaDataTopicV2: s.metaDataHandler,
|
||||
p2p.RPCBlobSidecarsByRootTopicV2: s.blobSidecarByRootRPCHandler, // Added in Electra
|
||||
p2p.RPCBlobSidecarsByRangeTopicV2: s.blobSidecarsByRangeRPCHandler, // Added in Electra
|
||||
}, nil
|
||||
default:
|
||||
return nil, errors.Errorf("RPC handler not found for fork index %d", forkIndex)
|
||||
}
|
||||
|
||||
@@ -100,6 +100,9 @@ func (s *Service) blobSidecarsByRangeRPCHandler(ctx context.Context, msg interfa
|
||||
var batch blockBatch
|
||||
|
||||
wQuota := params.BeaconConfig().MaxRequestBlobSidecars
|
||||
if slots.ToEpoch(s.cfg.chain.CurrentSlot()) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
wQuota = params.BeaconConfig().MaxRequestBlobSidecarsElectra
|
||||
}
|
||||
for batch, ok = batcher.next(ctx, stream); ok; batch, ok = batcher.next(ctx, stream) {
|
||||
batchStart := time.Now()
|
||||
wQuota, err = s.streamBlobBatch(ctx, batch, wQuota, stream)
|
||||
|
||||
@@ -13,9 +13,11 @@ import (
|
||||
"github.com/prysmaticlabs/prysm/v5/beacon-chain/p2p/types"
|
||||
"github.com/prysmaticlabs/prysm/v5/cmd/beacon-chain/flags"
|
||||
"github.com/prysmaticlabs/prysm/v5/config/params"
|
||||
"github.com/prysmaticlabs/prysm/v5/consensus-types/primitives"
|
||||
"github.com/prysmaticlabs/prysm/v5/encoding/bytesutil"
|
||||
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing"
|
||||
"github.com/prysmaticlabs/prysm/v5/monitoring/tracing/trace"
|
||||
"github.com/prysmaticlabs/prysm/v5/time/slots"
|
||||
"github.com/sirupsen/logrus"
|
||||
)
|
||||
|
||||
@@ -34,7 +36,8 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
|
||||
}
|
||||
|
||||
blobIdents := *ref
|
||||
if err := validateBlobByRootRequest(blobIdents); err != nil {
|
||||
cs := s.cfg.clock.CurrentSlot()
|
||||
if err := validateBlobByRootRequest(blobIdents, cs); err != nil {
|
||||
s.cfg.p2p.Peers().Scorers().BadResponsesScorer().Increment(stream.Conn().RemotePeer())
|
||||
s.writeErrorResponseToStream(responseCodeInvalidRequest, err.Error(), stream)
|
||||
return err
|
||||
@@ -49,7 +52,6 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
|
||||
}
|
||||
|
||||
// Compute the oldest slot we'll allow a peer to request, based on the current slot.
|
||||
cs := s.cfg.clock.CurrentSlot()
|
||||
minReqSlot, err := BlobRPCMinValidSlot(cs)
|
||||
if err != nil {
|
||||
return errors.Wrapf(err, "unexpected error computing min valid blob request slot, current_slot=%d", cs)
|
||||
@@ -104,9 +106,15 @@ func (s *Service) blobSidecarByRootRPCHandler(ctx context.Context, msg interface
|
||||
return nil
|
||||
}
|
||||
|
||||
func validateBlobByRootRequest(blobIdents types.BlobSidecarsByRootReq) error {
|
||||
if uint64(len(blobIdents)) > params.BeaconConfig().MaxRequestBlobSidecars {
|
||||
return types.ErrMaxBlobReqExceeded
|
||||
func validateBlobByRootRequest(blobIdents types.BlobSidecarsByRootReq, slot primitives.Slot) error {
|
||||
if slots.ToEpoch(slot) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
if uint64(len(blobIdents)) > params.BeaconConfig().MaxRequestBlobSidecarsElectra {
|
||||
return types.ErrMaxBlobReqExceeded
|
||||
}
|
||||
} else {
|
||||
if uint64(len(blobIdents)) > params.BeaconConfig().MaxRequestBlobSidecars {
|
||||
return types.ErrMaxBlobReqExceeded
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
@@ -171,6 +171,9 @@ func SendBlobsByRangeRequest(ctx context.Context, tor blockchain.TemporalOracle,
|
||||
|
||||
maxBlobsPerBlock := uint64(params.BeaconConfig().MaxBlobsPerBlock(req.StartSlot))
|
||||
max := params.BeaconConfig().MaxRequestBlobSidecars
|
||||
if slots.ToEpoch(req.StartSlot) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
max = params.BeaconConfig().MaxRequestBlobSidecarsElectra
|
||||
}
|
||||
if max > req.Count*maxBlobsPerBlock {
|
||||
max = req.Count * maxBlobsPerBlock
|
||||
}
|
||||
@@ -201,6 +204,9 @@ func SendBlobSidecarByRoot(
|
||||
defer closeStream(stream, log)
|
||||
|
||||
max := params.BeaconConfig().MaxRequestBlobSidecars
|
||||
if slots.ToEpoch(slot) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
max = params.BeaconConfig().MaxRequestBlobSidecarsElectra
|
||||
}
|
||||
maxBlobCount := params.BeaconConfig().MaxBlobsPerBlock(slot)
|
||||
if max > uint64(len(*req)*maxBlobCount) {
|
||||
max = uint64(len(*req) * maxBlobCount)
|
||||
|
||||
@@ -152,12 +152,23 @@ func (s *Service) registerSubscribers(epoch primitives.Epoch, digest [4]byte) {
|
||||
s.validateBlob,
|
||||
s.blobSubscriber,
|
||||
digest,
|
||||
func(primitives.Slot) []uint64 { return sliceFromCount(params.BeaconConfig().BlobsidecarSubnetCount) },
|
||||
blobSubnetSlice,
|
||||
func(currentSlot primitives.Slot) []uint64 { return []uint64{} },
|
||||
)
|
||||
}
|
||||
}
|
||||
|
||||
// blobSubnetSlice returns the blob subnet slice for the given slot.
|
||||
func blobSubnetSlice(currentSlot primitives.Slot) []uint64 {
|
||||
currentEpoch := slots.ToEpoch(currentSlot)
|
||||
|
||||
if currentEpoch >= params.BeaconConfig().ElectraForkEpoch {
|
||||
return sliceFromCount(params.BeaconConfig().BlobsidecarSubnetCountElectra)
|
||||
}
|
||||
|
||||
return sliceFromCount(params.BeaconConfig().BlobsidecarSubnetCount)
|
||||
}
|
||||
|
||||
// subscribe to a given topic with a given validator and subscription handler.
|
||||
// The base protobuf message is used to initialize new messages for decoding.
|
||||
func (s *Service) subscribe(topic string, validator wrappedVal, handle subHandler, digest [4]byte) *pubsub.Subscription {
|
||||
|
||||
@@ -58,7 +58,7 @@ func (s *Service) validateBlob(ctx context.Context, pid peer.ID, msg *pubsub.Mes
|
||||
}
|
||||
|
||||
// [REJECT] The sidecar is for the correct subnet -- i.e. compute_subnet_for_blob_sidecar(sidecar.index) == subnet_id.
|
||||
want := fmt.Sprintf("blob_sidecar_%d", computeSubnetForBlobSidecar(blob.Index))
|
||||
want := fmt.Sprintf("blob_sidecar_%d", computeSubnetForBlobSidecar(blob.Index, blob.Slot()))
|
||||
if !strings.Contains(*msg.Topic, want) {
|
||||
log.WithFields(blobFields(blob)).Debug("Sidecar index does not match topic")
|
||||
return pubsub.ValidationReject, fmt.Errorf("wrong topic name: %s", *msg.Topic)
|
||||
@@ -169,8 +169,12 @@ func blobFields(b blocks.ROBlob) logrus.Fields {
|
||||
}
|
||||
}
|
||||
|
||||
func computeSubnetForBlobSidecar(index uint64) uint64 {
|
||||
return index % params.BeaconConfig().BlobsidecarSubnetCount
|
||||
func computeSubnetForBlobSidecar(index uint64, slot primitives.Slot) uint64 {
|
||||
subnetCount := params.BeaconConfig().BlobsidecarSubnetCount
|
||||
if slots.ToEpoch(slot) >= params.BeaconConfig().ElectraForkEpoch {
|
||||
subnetCount = params.BeaconConfig().BlobsidecarSubnetCountElectra
|
||||
}
|
||||
return index % subnetCount
|
||||
}
|
||||
|
||||
// saveInvalidBlobToTemp as a block ssz. Writes to temp directory.
|
||||
|
||||
@@ -229,12 +229,14 @@ type BeaconChainConfig struct {
|
||||
ExecutionEngineTimeoutValue uint64 // ExecutionEngineTimeoutValue defines the seconds to wait before timing out engine endpoints with execution payload execution semantics (newPayload, forkchoiceUpdated).
|
||||
|
||||
// Subnet value
|
||||
BlobsidecarSubnetCount uint64 `yaml:"BLOB_SIDECAR_SUBNET_COUNT"` // BlobsidecarSubnetCount is the number of blobsidecar subnets used in the gossipsub protocol.
|
||||
BlobsidecarSubnetCount uint64 `yaml:"BLOB_SIDECAR_SUBNET_COUNT"` // BlobsidecarSubnetCount is the number of blobsidecar subnets used in the gossipsub protocol.
|
||||
BlobsidecarSubnetCountElectra uint64 `yaml:"BLOB_SIDECAR_SUBNET_COUNT_ELECTRA"` // BlobsidecarSubnetCountElectra is the number of blobsidecar subnets used in the gossipsub protocol post Electra hard fork.
|
||||
|
||||
// Values introduced in Deneb hard fork
|
||||
MaxPerEpochActivationChurnLimit uint64 `yaml:"MAX_PER_EPOCH_ACTIVATION_CHURN_LIMIT" spec:"true"` // MaxPerEpochActivationChurnLimit is the maximum amount of churn allotted for validator activation.
|
||||
MinEpochsForBlobsSidecarsRequest primitives.Epoch `yaml:"MIN_EPOCHS_FOR_BLOB_SIDECARS_REQUESTS" spec:"true"` // MinEpochsForBlobsSidecarsRequest is the minimum number of epochs the node will keep the blobs for.
|
||||
MaxRequestBlobSidecars uint64 `yaml:"MAX_REQUEST_BLOB_SIDECARS" spec:"true"` // MaxRequestBlobSidecars is the maximum number of blobs to request in a single request.
|
||||
MaxRequestBlobSidecarsElectra uint64 `yaml:"MAX_REQUEST_BLOB_SIDECARS_ELECTRA" spec:"true"` // MaxRequestBlobSidecarsElectra is the maximum number of blobs to request in a single request.
|
||||
MaxRequestBlocksDeneb uint64 `yaml:"MAX_REQUEST_BLOCKS_DENEB" spec:"true"` // MaxRequestBlocksDeneb is the maximum number of blocks in a single request after the deneb epoch.
|
||||
|
||||
// Values introduce in Electra upgrade
|
||||
|
||||
@@ -25,7 +25,6 @@ import (
|
||||
// IMPORTANT: Use one field per line and sort these alphabetically to reduce conflicts.
|
||||
var placeholderFields = []string{
|
||||
"BLOB_SIDECAR_SUBNET_COUNT_EIP7594",
|
||||
"BLOB_SIDECAR_SUBNET_COUNT_ELECTRA",
|
||||
"BYTES_PER_LOGS_BLOOM", // Compile time constant on ExecutionPayload.logs_bloom.
|
||||
"EIP6110_FORK_EPOCH",
|
||||
"EIP6110_FORK_VERSION",
|
||||
@@ -44,7 +43,6 @@ var placeholderFields = []string{
|
||||
"MAX_BYTES_PER_TRANSACTION", // Used for ssz of EL transactions. Unused in Prysm.
|
||||
"MAX_EXTRA_DATA_BYTES", // Compile time constant on ExecutionPayload.extra_data.
|
||||
"MAX_REQUEST_BLOB_SIDECARS_EIP7594",
|
||||
"MAX_REQUEST_BLOB_SIDECARS_ELECTRA",
|
||||
"MAX_REQUEST_BLOB_SIDECARS_FULU",
|
||||
"MAX_REQUEST_PAYLOADS", // Compile time constant on BeaconBlockBody.ExecutionRequests
|
||||
"MAX_TRANSACTIONS_PER_PAYLOAD", // Compile time constant on ExecutionPayload.transactions.
|
||||
|
||||
@@ -268,7 +268,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
|
||||
ExecutionEngineTimeoutValue: 8, // 8 seconds default based on: https://github.com/ethereum/execution-apis/blob/main/src/engine/specification.md#core
|
||||
|
||||
// Subnet value
|
||||
BlobsidecarSubnetCount: 6,
|
||||
BlobsidecarSubnetCount: 6,
|
||||
BlobsidecarSubnetCountElectra: 9,
|
||||
|
||||
MaxPerEpochActivationChurnLimit: 8,
|
||||
MinEpochsForBlobsSidecarsRequest: 4096,
|
||||
@@ -323,6 +324,7 @@ var mainnetBeaconConfig = &BeaconChainConfig{
|
||||
DeprecatedMaxBlobsPerBlock: 6,
|
||||
DeprecatedMaxBlobsPerBlockElectra: 9,
|
||||
DeprecatedTargetBlobsPerBlockElectra: 6,
|
||||
MaxRequestBlobSidecarsElectra: 1152,
|
||||
}
|
||||
|
||||
// MainnetTestConfig provides a version of the mainnet config that has a different name
|
||||
|
||||
Reference in New Issue
Block a user