Compare commits

...

1 Commits

Author SHA1 Message Date
terence tsao
5736617492 Change slot time 2025-05-29 08:34:30 -07:00
18 changed files with 208 additions and 50 deletions

View File

@@ -32,7 +32,7 @@ func newSubnetIDs() *subnetIDs {
cacheSize := int(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().MaxCommitteesPerSlot * 2)) // lint:ignore uintcast -- constant values that would panic on startup if negative.
attesterCache := lruwrpr.New(cacheSize)
aggregatorCache := lruwrpr.New(cacheSize)
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().DeprecatedSecondsPerSlot)) // TODO: without fixing it, the cache will live longer than necessary.
subLength := epochDuration * time.Duration(params.BeaconConfig().EpochsPerRandomSubnetSubscription)
persistentCache := cache.New(subLength*time.Second, epochDuration*time.Second)
return &subnetIDs{attester: attesterCache, aggregator: aggregatorCache, persistentSubnets: persistentCache}

View File

@@ -21,7 +21,7 @@ type syncSubnetIDs struct {
var SyncSubnetIDs = newSyncSubnetIDs()
func newSyncSubnetIDs() *syncSubnetIDs {
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().DeprecatedSecondsPerSlot)) // TODO: without fixing it, the cache will live longer than necessary.
// Set the default duration of a sync subnet index as the whole sync committee period.
subLength := epochDuration * time.Duration(params.BeaconConfig().EpochsPerSyncCommitteePeriod)
persistentCache := cache.New(subLength*time.Second, epochDuration*time.Second)

View File

@@ -66,7 +66,7 @@ func New(ctx context.Context, db iface.Database, genesisTime uint64, initSyncWai
db: db,
ps: pruneStartSlotFunc(helpers.MinEpochsForBlockRequests() + 1), // Default retention epochs is MIN_EPOCHS_FOR_BLOCK_REQUESTS + 1 from the current slot.
done: make(chan struct{}),
slotTicker: slots.NewSlotTicker(slots.StartTime(genesisTime, 0), params.BeaconConfig().SecondsPerSlot),
slotTicker: slots.NewSlotTicker(slots.StartTime(genesisTime, 0), params.BeaconConfig().DeprecatedSecondsPerSlot),
initSyncWaiter: initSyncWaiter,
backfillWaiter: backfillWaiter,
}

View File

@@ -133,7 +133,12 @@ func (n *Node) setNodeAndParentValidated(ctx context.Context) error {
// slot will have secs = 3 below.
func (n *Node) arrivedEarly(genesisTime uint64) (bool, error) {
secs, err := slots.SecondsSinceSlotStart(n.slot, genesisTime, n.timestamp)
votingWindow := params.BeaconConfig().SecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
e := slots.ToEpoch(n.slot)
if e >= params.BeaconConfig().FuluForkEpoch {
votingWindow := params.BeaconConfig().DeprecatedSecondsPerSlotXYZ / params.BeaconConfig().IntervalsPerSlot
return secs < votingWindow, err
}
votingWindow := params.BeaconConfig().DeprecatedSecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
return secs < votingWindow, err
}

View File

@@ -132,9 +132,14 @@ func (s *Store) insert(ctx context.Context,
if timeNow < s.genesisTime {
return n, nil
}
secondsIntoSlot := (timeNow - s.genesisTime) % params.BeaconConfig().SecondsPerSlot
sps := params.BeaconConfig().DeprecatedSecondsPerSlot
e := slots.ToEpoch(roblock.Block().Slot())
if e >= params.BeaconConfig().FuluForkEpoch {
sps = params.BeaconConfig().DeprecatedSecondsPerSlotXYZ
}
secondsIntoSlot := (timeNow - s.genesisTime) % sps
currentSlot := slots.CurrentSlot(s.genesisTime)
boostThreshold := params.BeaconConfig().SecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
boostThreshold := sps / params.BeaconConfig().IntervalsPerSlot
isFirstBlock := s.proposerBoostRoot == [32]byte{}
if currentSlot == slot && secondsIntoSlot < boostThreshold && isFirstBlock {
s.proposerBoostRoot = root
@@ -278,7 +283,7 @@ func (f *ForkChoice) HighestReceivedBlockDelay() primitives.Slot {
if err != nil {
return 0
}
return primitives.Slot(secs / params.BeaconConfig().SecondsPerSlot)
return primitives.Slot(secs / params.BeaconConfig().DeprecatedSecondsPerSlot) // TODO: fix this, it's only used in test now
}
// ReceivedBlocksLastEpoch returns the number of blocks received in the last epoch

View File

@@ -31,7 +31,7 @@ type AttCaches struct {
// NewAttCaches initializes a new attestation pool consists of multiple KV store in cache for
// various kind of attestations.
func NewAttCaches() *AttCaches {
secsInEpoch := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
secsInEpoch := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().DeprecatedSecondsPerSlot))
c := cache.New(2*secsInEpoch*time.Second, 2*secsInEpoch*time.Second)
pool := &AttCaches{
unAggregatedAtt: make(map[attestation.Id]ethpb.Att),

View File

@@ -34,7 +34,7 @@ func (s *Service) Broadcast(ctx context.Context, msg proto.Message) error {
ctx, span := trace.StartSpan(ctx, "p2p.Broadcast")
defer span.End()
twoSlots := time.Duration(2*params.BeaconConfig().SecondsPerSlot) * time.Second
twoSlots := time.Duration(2*slots.CurrentSecondsPerSlot(uint64(s.genesisTime.Unix()))) * time.Second
ctx, cancel := context.WithTimeout(ctx, twoSlots)
defer cancel()
@@ -104,7 +104,7 @@ func (s *Service) internalBroadcastAttestation(ctx context.Context, subnet uint6
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
oneEpoch := time.Duration(1*params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot)) * time.Second
oneEpoch := time.Duration(1*params.BeaconConfig().SlotsPerEpoch.Mul(slots.CurrentSecondsPerSlot(uint64(s.genesisTime.Unix())))) * time.Second
ctx, cancel := context.WithTimeout(ctx, oneEpoch)
defer cancel()
@@ -160,7 +160,7 @@ func (s *Service) broadcastSyncCommittee(ctx context.Context, subnet uint64, sMs
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
oneSlot := time.Duration(1*params.BeaconConfig().SecondsPerSlot) * time.Second
oneSlot := time.Duration(1*slots.CurrentSecondsPerSlot(uint64(s.genesisTime.Unix()))) * time.Second
ctx, cancel := context.WithTimeout(ctx, oneSlot)
defer cancel()
@@ -236,7 +236,7 @@ func (s *Service) internalBroadcastBlob(ctx context.Context, subnet uint64, blob
defer span.End()
ctx = trace.NewContext(context.Background(), span) // clear parent context / deadline.
oneSlot := time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
oneSlot := time.Duration(slots.CurrentSecondsPerSlot(uint64(s.genesisTime.Unix()))) * time.Second
ctx, cancel := context.WithTimeout(ctx, oneSlot)
defer cancel()

View File

@@ -9,7 +9,7 @@ import (
// updates the node's discovery service to reflect any new fork version
// changes.
func (s *Service) forkWatcher() {
slotTicker := slots.NewSlotTicker(s.genesisTime, params.BeaconConfig().SecondsPerSlot)
slotTicker := slots.NewSlotTicker(s.genesisTime, slots.CurrentSecondsPerSlot(uint64(s.genesisTime.Unix())))
for {
select {
case currSlot := <-slotTicker.C():

View File

@@ -558,7 +558,7 @@ func defaultLightClientFinalityUpdateTopicParams() *pubsub.TopicScoreParams {
}
func oneSlotDuration() time.Duration {
return time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
return time.Duration(params.BeaconConfig().DeprecatedSecondsPerSlot) * time.Second
}
func oneEpochDuration() time.Duration {

View File

@@ -195,7 +195,7 @@ func pubsubGossipParam() pubsub.GossipSubParams {
// to configure our message id time-cache rather than instantiating
// it with a router instance.
func setPubSubParameters() {
seenTtl := 2 * time.Second * time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
seenTtl := 2 * time.Second * time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().DeprecatedSecondsPerSlot))
pubsub.TimeCacheDuration = seenTtl
}

View File

@@ -439,7 +439,7 @@ func computeSubscriptionExpirationTime(nodeID enode.ID, epoch primitives.Epoch)
nodeOffset, _ := computeOffsetAndPrefix(nodeID)
pastEpochs := (nodeOffset + uint64(epoch)) % params.BeaconConfig().EpochsPerSubnetSubscription
remEpochs := params.BeaconConfig().EpochsPerSubnetSubscription - pastEpochs
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().SecondsPerSlot))
epochDuration := time.Duration(params.BeaconConfig().SlotsPerEpoch.Mul(params.BeaconConfig().DeprecatedSecondsPerSlot))
epochTime := time.Duration(remEpochs) * epochDuration
return epochTime * time.Second
}

View File

@@ -56,7 +56,8 @@ type BeaconChainConfig struct {
// Time parameters constants.
GenesisDelay uint64 `yaml:"GENESIS_DELAY" spec:"true"` // GenesisDelay is the minimum number of seconds to delay starting the Ethereum Beacon Chain genesis. Must be at least 1 second.
MinAttestationInclusionDelay primitives.Slot `yaml:"MIN_ATTESTATION_INCLUSION_DELAY" spec:"true"` // MinAttestationInclusionDelay defines how many slots validator has to wait to include attestation for beacon block.
SecondsPerSlot uint64 `yaml:"SECONDS_PER_SLOT" spec:"true"` // SecondsPerSlot is how many seconds are in a single slot.
DeprecatedSecondsPerSlot uint64 `yaml:"SECONDS_PER_SLOT" spec:"true"` // DeprecatedSecondsPerSlot is how many seconds are in a single slot.
DeprecatedSecondsPerSlotXYZ uint64 `yaml:"SECONDS_PER_SLOT_XYZ" spec:"true"` // DeprecatedSecondsPerSlotXYZ is how many seconds are in a single slot.
SlotsPerEpoch primitives.Slot `yaml:"SLOTS_PER_EPOCH" spec:"true"` // SlotsPerEpoch is the number of slots in an epoch.
SqrRootSlotsPerEpoch primitives.Slot // SqrRootSlotsPerEpoch is a hard coded value where we take the square root of `SlotsPerEpoch` and round down.
MinSeedLookahead primitives.Epoch `yaml:"MIN_SEED_LOOKAHEAD" spec:"true"` // MinSeedLookahead is the duration of randao look ahead seed.
@@ -480,3 +481,11 @@ func FuluEnabled() bool {
func WithinDAPeriod(block, current primitives.Epoch) bool {
return block+BeaconConfig().MinEpochsForBlobsSidecarsRequest >= current
}
func SecondsPerSlot(slot primitives.Slot) uint64 {
epoch := primitives.Epoch(slot / BeaconConfig().SlotsPerEpoch)
if epoch >= BeaconConfig().FuluForkEpoch {
return BeaconConfig().DeprecatedSecondsPerSlotXYZ
}
return BeaconConfig().DeprecatedSecondsPerSlot
}

View File

@@ -186,7 +186,7 @@ func ConfigToYaml(cfg *BeaconChainConfig) []byte {
fmt.Sprintf("MIN_GENESIS_TIME: %d", cfg.MinGenesisTime),
fmt.Sprintf("GENESIS_FORK_VERSION: %#x", cfg.GenesisForkVersion),
fmt.Sprintf("CHURN_LIMIT_QUOTIENT: %d", cfg.ChurnLimitQuotient),
fmt.Sprintf("SECONDS_PER_SLOT: %d", cfg.SecondsPerSlot),
fmt.Sprintf("SECONDS_PER_SLOT: %d", cfg.DeprecatedSecondsPerSlot),
fmt.Sprintf("SLOTS_PER_EPOCH: %d", cfg.SlotsPerEpoch),
fmt.Sprintf("SECONDS_PER_ETH1_BLOCK: %d", cfg.SecondsPerETH1Block),
fmt.Sprintf("ETH1_FOLLOW_DISTANCE: %d", cfg.Eth1FollowDistance),

View File

@@ -98,7 +98,8 @@ var mainnetBeaconConfig = &BeaconChainConfig{
// Time parameter constants.
MinAttestationInclusionDelay: 1,
SecondsPerSlot: 12,
DeprecatedSecondsPerSlot: 12,
DeprecatedSecondsPerSlotXYZ: 8,
SlotsPerEpoch: 32,
SqrRootSlotsPerEpoch: 5,
MinSeedLookahead: 1,

View File

@@ -33,7 +33,7 @@ func MinimalSpecConfig() *BeaconChainConfig {
minimalConfig.ETH1AddressWithdrawalPrefixByte = byte(1)
// Time parameters
minimalConfig.SecondsPerSlot = 6
minimalConfig.DeprecatedSecondsPerSlot = 6
minimalConfig.MinAttestationInclusionDelay = 1
minimalConfig.SlotsPerEpoch = 8
minimalConfig.SqrRootSlotsPerEpoch = 2

View File

@@ -26,7 +26,7 @@ func E2ETestConfig() *BeaconChainConfig {
e2eConfig.MaxValidatorsPerWithdrawalsSweep = 128
// Time parameters.
e2eConfig.SecondsPerSlot = 10
e2eConfig.DeprecatedSecondsPerSlot = 10
e2eConfig.SlotsPerEpoch = 6
e2eConfig.SqrRootSlotsPerEpoch = 2
e2eConfig.SecondsPerETH1Block = 2
@@ -80,7 +80,7 @@ func E2EMainnetTestConfig() *BeaconChainConfig {
e2eConfig.ChurnLimitQuotient = 65536
// Time parameters.
e2eConfig.SecondsPerSlot = 6
e2eConfig.DeprecatedSecondsPerSlot = 6
e2eConfig.SqrRootSlotsPerEpoch = 5
e2eConfig.SecondsPerETH1Block = 2
e2eConfig.ShardCommitteePeriod = 4

View File

@@ -189,7 +189,7 @@ func NewSlotTickerWithIntervals(genesisTime time.Time, intervals []time.Duration
if len(intervals) == 0 {
panic("at least one interval has to be entered")
}
slotDuration := time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Second
slotDuration := time.Duration(params.SecondsPerSlot(CurrentSlot(uint64(genesisTime.Unix())))) * time.Second
lastOffset := time.Duration(0)
for _, offset := range intervals {
if offset < lastOffset {

View File

@@ -3,6 +3,7 @@ package slots
import (
"fmt"
"math"
"math/bits"
"time"
"github.com/OffchainLabs/prysm/v6/config/params"
@@ -20,8 +21,22 @@ const MaxSlotBuffer = uint64(1 << 7)
// startFromTime returns the slot start in terms of genesis time.Time
func startFromTime(genesis time.Time, slot primitives.Slot) time.Time {
duration := time.Second * time.Duration(slot.Mul(params.BeaconConfig().SecondsPerSlot))
return genesis.Add(duration) // lint:ignore uintcast -- Genesis timestamp will not exceed int64 in your lifetime.
cfg := params.BeaconConfig()
epoch := ToEpoch(slot)
var duration time.Duration
if epoch >= cfg.FuluForkEpoch {
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
adjustedSlot := slot.SubSlot(upgradeSlot)
duration += time.Duration(adjustedSlot.Mul(cfg.DeprecatedSecondsPerSlotXYZ)) * time.Second
duration += time.Duration(upgradeSlot.Mul(cfg.DeprecatedSecondsPerSlot)) * time.Second
} else {
duration += time.Duration(slot.Mul(cfg.DeprecatedSecondsPerSlot)) * time.Second
}
return genesis.Add(duration)
}
// StartTime returns the start time in terms of its unix epoch
@@ -34,10 +49,25 @@ func StartTime(genesis uint64, slot primitives.Slot) time.Time {
// SinceGenesis returns the number of slots since
// the provided genesis time.
func SinceGenesis(genesis time.Time) primitives.Slot {
if genesis.After(prysmTime.Now()) { // Genesis has not occurred yet.
now := prysmTime.Now()
if genesis.After(now) {
return 0
}
return primitives.Slot(uint64(prysmTime.Since(genesis).Seconds()) / params.BeaconConfig().SecondsPerSlot)
cfg := params.BeaconConfig()
elapsedSeconds := uint64(now.Sub(genesis).Seconds())
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
upgradeTime := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
if elapsedSeconds <= upgradeTime {
return primitives.Slot(elapsedSeconds / cfg.DeprecatedSecondsPerSlot)
}
postUpgradeElapsed := elapsedSeconds - upgradeTime
postUpgradeSlots := primitives.Slot(postUpgradeElapsed / cfg.DeprecatedSecondsPerSlotXYZ)
return upgradeSlot + postUpgradeSlots
}
// EpochsSinceGenesis returns the number of epochs since
@@ -50,15 +80,21 @@ func EpochsSinceGenesis(genesis time.Time) primitives.Epoch {
// parameter by a specified number. It returns a value of time.Duration
// in milliseconds, useful for dividing values such as 1 second into
// millisecond-based durations.
func DivideSlotBy(timesPerSlot int64) time.Duration {
return time.Duration(int64(params.BeaconConfig().SecondsPerSlot*1000)/timesPerSlot) * time.Millisecond
func DivideSlotBy(timesPerSlot int64, slot primitives.Slot) time.Duration {
if ToEpoch(slot) >= params.BeaconConfig().FuluForkEpoch {
return time.Duration(int64(params.BeaconConfig().DeprecatedSecondsPerSlotXYZ*1000)/timesPerSlot) * time.Millisecond
}
return time.Duration(int64(params.BeaconConfig().DeprecatedSecondsPerSlot*1000)/timesPerSlot) * time.Millisecond
}
// MultiplySlotBy multiplies the SECONDS_PER_SLOT configuration
// parameter by a specified number. It returns a value of time.Duration
// in millisecond-based durations.
func MultiplySlotBy(times int64) time.Duration {
return time.Duration(int64(params.BeaconConfig().SecondsPerSlot)*times) * time.Second
func MultiplySlotBy(times int64, slot primitives.Slot) time.Duration {
if ToEpoch(slot) >= params.BeaconConfig().FuluForkEpoch {
return time.Duration(int64(params.BeaconConfig().DeprecatedSecondsPerSlotXYZ)*times) * time.Second
}
return time.Duration(int64(params.BeaconConfig().DeprecatedSecondsPerSlot)*times) * time.Second
}
// AbsoluteValueSlotDifference between two slots.
@@ -183,23 +219,64 @@ func VerifyTime(genesisTime uint64, slot primitives.Slot, timeTolerance time.Dur
return nil
}
// ToTime takes the given slot and genesis time to determine the start time of the slot.
func ToTime(genesisTimeSec uint64, slot primitives.Slot) (time.Time, error) {
timeSinceGenesis, err := slot.SafeMul(params.BeaconConfig().SecondsPerSlot)
if err != nil {
return time.Unix(0, 0), fmt.Errorf("slot (%d) is in the far distant future: %w", slot, err)
cfg := params.BeaconConfig()
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
var timeSinceGenesis primitives.Slot
var err error
if slot < upgradeSlot {
timeSinceGenesis, err = slot.SafeMul(cfg.DeprecatedSecondsPerSlot)
if err != nil {
return time.Time{}, fmt.Errorf("slot (%d) is in the far distant future: %w", slot, err)
}
} else {
adjustedSlot := slot - upgradeSlot
preUpgradeTime, err := upgradeSlot.SafeMul(cfg.DeprecatedSecondsPerSlot)
if err != nil {
return time.Time{}, fmt.Errorf("slot (%d) pre-upgrade mul overflow: %w", slot, err)
}
postUpgradeTime, err := adjustedSlot.SafeMul(cfg.DeprecatedSecondsPerSlotXYZ)
if err != nil {
return time.Time{}, fmt.Errorf("slot (%d) post-upgrade mul overflow: %w", slot, err)
}
timeSinceGenesis, err = preUpgradeTime.SafeAddSlot(postUpgradeTime)
if err != nil {
return time.Time{}, fmt.Errorf("slot (%d) addition overflow: %w", slot, err)
}
}
sTime, err := timeSinceGenesis.SafeAdd(genesisTimeSec)
absoluteTime, err := timeSinceGenesis.SafeAdd(genesisTimeSec)
if err != nil {
return time.Unix(0, 0), fmt.Errorf("slot (%d) is in the far distant future: %w", slot, err)
return time.Time{}, fmt.Errorf("slot (%d) genesis time addition overflow: %w", slot, err)
}
return time.Unix(int64(sTime), 0), nil // lint:ignore uintcast -- A timestamp will not exceed int64 in your lifetime.
if bits.Len64(uint64(absoluteTime)) >= 63 {
return time.Time{}, fmt.Errorf("slot (%d) resulting timestamp overflows int64: %d", slot, absoluteTime)
}
return time.Unix(0, 0).Add(time.Duration(absoluteTime) * time.Second), nil
}
// BeginsAt computes the timestamp where the given slot begins, relative to the genesis timestamp.
func BeginsAt(slot primitives.Slot, genesis time.Time) time.Time {
sd := time.Second * time.Duration(params.BeaconConfig().SecondsPerSlot) * time.Duration(slot)
return genesis.Add(sd)
cfg := params.BeaconConfig()
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
var seconds uint64
if slot < upgradeSlot {
seconds = uint64(slot) * cfg.DeprecatedSecondsPerSlot
} else {
preUpgrade := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
postUpgrade := uint64(slot-upgradeSlot) * cfg.DeprecatedSecondsPerSlotXYZ
seconds = preUpgrade + postUpgrade
}
return genesis.Add(time.Duration(seconds) * time.Second)
}
// Since computes the number of time slots that have occurred since the given timestamp.
@@ -207,14 +284,26 @@ func Since(time time.Time) primitives.Slot {
return CurrentSlot(uint64(time.Unix()))
}
// CurrentSlot returns the current slot as determined by the local clock and
// provided genesis time.
func CurrentSlot(genesisTimeSec uint64) primitives.Slot {
now := uint64(prysmTime.Now().Unix())
if now < genesisTimeSec {
return 0
}
return primitives.Slot((now - genesisTimeSec) / params.BeaconConfig().SecondsPerSlot)
cfg := params.BeaconConfig()
elapsed := now - genesisTimeSec
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
upgradeTime := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
if elapsed <= upgradeTime {
return primitives.Slot(elapsed / cfg.DeprecatedSecondsPerSlot)
}
postUpgradeElapsed := elapsed - upgradeTime
postUpgradeSlots := primitives.Slot(postUpgradeElapsed / cfg.DeprecatedSecondsPerSlotXYZ)
return upgradeSlot + postUpgradeSlots
}
// Duration computes the span of time between two instants, represented as Slots.
@@ -222,7 +311,21 @@ func Duration(start, end time.Time) primitives.Slot {
if end.Before(start) {
return 0
}
return primitives.Slot(uint64(end.Unix()-start.Unix()) / params.BeaconConfig().SecondsPerSlot)
cfg := params.BeaconConfig()
elapsed := uint64(end.Sub(start).Seconds())
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
upgradeTime := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
if elapsed <= upgradeTime {
return primitives.Slot(elapsed / cfg.DeprecatedSecondsPerSlot)
}
postUpgradeElapsed := elapsed - upgradeTime
postUpgradeSlots := primitives.Slot(postUpgradeElapsed / cfg.DeprecatedSecondsPerSlotXYZ)
return upgradeSlot + postUpgradeSlots
}
// ValidateClock validates a provided slot against the local
@@ -251,7 +354,7 @@ func RoundUpToNearestEpoch(slot primitives.Slot) primitives.Slot {
// depending on the provided genesis and current slot.
func VotingPeriodStartTime(genesis uint64, slot primitives.Slot) uint64 {
slots := params.BeaconConfig().SlotsPerEpoch.Mul(uint64(params.BeaconConfig().EpochsPerEth1VotingPeriod))
startTime := uint64((slot - slot.ModSlot(slots)).Mul(params.BeaconConfig().SecondsPerSlot))
startTime := uint64((slot - slot.ModSlot(slots)).Mul(params.BeaconConfig().DeprecatedSecondsPerSlot)) // Can ignore this as it's used before Pectra. See EIP-6110
return genesis + startTime
}
@@ -286,11 +389,23 @@ func SyncCommitteePeriodStartEpoch(e primitives.Epoch) (primitives.Epoch, error)
// SecondsSinceSlotStart returns the number of seconds elapsed since the
// given slot start time
func SecondsSinceSlotStart(s primitives.Slot, genesisTime, timeStamp uint64) (uint64, error) {
limit := genesisTime + uint64(s)*params.BeaconConfig().SecondsPerSlot
if timeStamp < limit {
return 0, fmt.Errorf("could not compute seconds since slot %d start: invalid timestamp, got %d < want %d", s, timeStamp, limit)
cfg := params.BeaconConfig()
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
var slotStart uint64
if s < upgradeSlot {
slotStart = genesisTime + uint64(s)*cfg.DeprecatedSecondsPerSlot
} else {
preUpgrade := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
postUpgrade := uint64(s-upgradeSlot) * cfg.DeprecatedSecondsPerSlotXYZ
slotStart = genesisTime + preUpgrade + postUpgrade
}
return timeStamp - genesisTime - uint64(s)*params.BeaconConfig().SecondsPerSlot, nil
if timeStamp < slotStart {
return 0, fmt.Errorf("could not compute seconds since slot %d start: invalid timestamp, got %d < want %d", s, timeStamp, slotStart)
}
return timeStamp - slotStart, nil
}
// TimeIntoSlot returns the time duration elapsed between the current time and
@@ -302,7 +417,12 @@ func TimeIntoSlot(genesisTime uint64) time.Duration {
// WithinVotingWindow returns whether the current time is within the voting window
// (eg. 4 seconds on mainnet) of the current slot.
func WithinVotingWindow(genesisTime uint64, slot primitives.Slot) bool {
votingWindow := params.BeaconConfig().SecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
e := ToEpoch(slot)
if e >= params.BeaconConfig().FuluForkEpoch {
votingWindow := params.BeaconConfig().DeprecatedSecondsPerSlotXYZ / params.BeaconConfig().IntervalsPerSlot
return time.Since(StartTime(genesisTime, slot)) < time.Duration(votingWindow)*time.Second
}
votingWindow := params.BeaconConfig().DeprecatedSecondsPerSlot / params.BeaconConfig().IntervalsPerSlot
return time.Since(StartTime(genesisTime, slot)) < time.Duration(votingWindow)*time.Second
}
@@ -332,3 +452,21 @@ func SecondsUntilNextEpochStart(genesisTimeSec uint64) (uint64, error) {
}).Debugf("%d seconds until next epoch", waitTime)
return waitTime, nil
}
func CurrentSecondsPerSlot(genesisTimeSec uint64) uint64 {
now := uint64(prysmTime.Now().Unix())
if now < genesisTimeSec {
return 0
}
cfg := params.BeaconConfig()
elapsed := now - genesisTimeSec
upgradeSlot := primitives.Slot(cfg.FuluForkEpoch) * cfg.SlotsPerEpoch
upgradeTime := uint64(upgradeSlot) * cfg.DeprecatedSecondsPerSlot
if elapsed <= upgradeTime {
return cfg.DeprecatedSecondsPerSlot
}
return cfg.DeprecatedSecondsPerSlotXYZ
}