diff --git a/consensus/beacon/consensus.go b/consensus/beacon/consensus.go index aed9e7cb5b..12fc6b1ad3 100644 --- a/consensus/beacon/consensus.go +++ b/consensus/beacon/consensus.go @@ -368,6 +368,10 @@ func (beacon *Beacon) Delay(_ consensus.ChainReader, _ *types.Header, _ *time.Du return nil } +func (beacon *Beacon) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) { + panic("not supported") +} + // Finalize implements consensus.Engine, setting the final state on the header func (beacon *Beacon) Finalize(chain consensus.ChainHeaderReader, header *types.Header, state vm.StateDB, txs *[]*types.Transaction, uncles []*types.Header, withdrawals []*types.Withdrawal, _ *[]*types.Receipt, _ *[]*types.Transaction, _ *uint64, tracer *tracing.Hooks) error { // Finalize is different with Prepare, it can be used in both block verification. diff --git a/consensus/clique/clique.go b/consensus/clique/clique.go index 4591642305..92a23675d9 100644 --- a/consensus/clique/clique.go +++ b/consensus/clique/clique.go @@ -628,6 +628,10 @@ func (c *Clique) Delay(chain consensus.ChainReader, header *types.Header, leftOv return nil } +func (c *Clique) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) { + return c.config.Period, nil +} + // Seal implements consensus.Engine, attempting to create a sealed block using // the local signing credentials. func (c *Clique) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error { diff --git a/consensus/consensus.go b/consensus/consensus.go index a58287f5e8..a806c9d6c0 100644 --- a/consensus/consensus.go +++ b/consensus/consensus.go @@ -147,6 +147,9 @@ type Engine interface { // Delay returns the max duration the miner can commit txs Delay(chain ChainReader, header *types.Header, leftOver *time.Duration) *time.Duration + // BlockInterval returns the block interval after given header applied + BlockInterval(chain ChainHeaderReader, header *types.Header) (uint64, error) + // Close terminates any background threads maintained by the consensus engine. Close() error } @@ -162,6 +165,5 @@ type PoSA interface { GetFinalizedHeader(chain ChainHeaderReader, header *types.Header) *types.Header VerifyVote(chain ChainHeaderReader, vote *types.VoteEnvelope) error IsActiveValidatorAt(chain ChainHeaderReader, header *types.Header, checkVoteKeyFn func(bLSPublicKey *types.BLSPublicKey) bool) bool - BlockInterval() uint64 NextProposalBlock(chain ChainHeaderReader, header *types.Header, proposer common.Address) (uint64, uint64, error) } diff --git a/consensus/ethash/ethash.go b/consensus/ethash/ethash.go index f37ec26056..fcf3afd2b4 100644 --- a/consensus/ethash/ethash.go +++ b/consensus/ethash/ethash.go @@ -83,3 +83,7 @@ func (ethash *Ethash) APIs(chain consensus.ChainHeaderReader) []rpc.API { func (ethash *Ethash) Seal(chain consensus.ChainHeaderReader, block *types.Block, results chan<- *types.Block, stop <-chan struct{}) error { panic("ethash (pow) sealing not supported any more") } + +func (ethash *Ethash) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) { + panic("not supported") +} diff --git a/consensus/misc/eip1559/eip1559_test.go b/consensus/misc/eip1559/eip1559_test.go index 54f91046e3..387ec7a65c 100644 --- a/consensus/misc/eip1559/eip1559_test.go +++ b/consensus/misc/eip1559/eip1559_test.go @@ -53,8 +53,7 @@ func config() *params.ChainConfig { config := copyConfig(params.TestChainConfig) config.Ethash = nil config.Parlia = ¶ms.ParliaConfig{ - Period: 3, - Epoch: 200, + Epoch: 200, } config.LondonBlock = big.NewInt(5) return config diff --git a/consensus/parlia/parlia.go b/consensus/parlia/parlia.go index 64b4b12a3c..f7c79ca9d1 100644 --- a/consensus/parlia/parlia.go +++ b/consensus/parlia/parlia.go @@ -53,9 +53,11 @@ const ( inMemorySignatures = 4096 // Number of recent block signatures to keep in memory inMemoryHeaders = 86400 // Number of recent headers to keep in memory for double sign detection, - checkpointInterval = 1024 // Number of blocks after which to save the snapshot to the database - defaultEpochLength = uint64(200) // Default number of blocks of checkpoint to update validatorSet from contract - defaultTurnLength = uint8(1) // Default consecutive number of blocks a validator receives priority for block production + checkpointInterval = 1024 // Number of blocks after which to save the snapshot to the database + defaultEpochLength = uint64(200) // Default number of blocks of checkpoint to update validatorSet from contract + defaultBlockInterval = uint8(3) // Default block interval + lorentzBlockInterval = uint8(1) // Block interval starting from the Lorentz hard fork + defaultTurnLength = uint8(1) // Default consecutive number of blocks a validator receives priority for block production extraVanity = 32 // Fixed number of extra-data prefix bytes reserved for signer vanity extraSeal = 65 // Fixed number of extra-data suffix bytes reserved for signer seal @@ -316,10 +318,6 @@ func New( return c } -func (p *Parlia) Period() uint64 { - return p.config.Period -} - func (p *Parlia) IsSystemTransaction(tx *types.Transaction, header *types.Header) (bool, error) { // deploy a contract if tx.To() == nil { @@ -779,6 +777,10 @@ func (p *Parlia) snapshot(chain consensus.ChainHeaderReader, number uint64, hash // new snapshot snap = newSnapshot(p.config, p.signatures, number, blockHash, validators, voteAddrs, p.ethAPI) + if p.chainConfig.IsLorentz(checkpoint.Number, checkpoint.Time) { + snap.BlockInterval = lorentzBlockInterval + } + // get turnLength from headers and use that for new turnLength turnLength, err := parseTurnLength(checkpoint, p.chainConfig, p.config) if err != nil { @@ -1539,9 +1541,9 @@ func (p *Parlia) Delay(chain consensus.ChainReader, header *types.Header, leftOv } delay := p.delayForRamanujanFork(snap, header) - if *leftOver >= time.Duration(p.config.Period)*time.Second { + if *leftOver >= time.Duration(snap.BlockInterval)*time.Second { // ignore invalid leftOver - log.Error("Delay invalid argument", "leftOver", leftOver.String(), "Period", p.config.Period) + log.Error("Delay invalid argument", "leftOver", leftOver.String(), "Period", snap.BlockInterval) } else if *leftOver >= delay { delay = time.Duration(0) return &delay @@ -1550,9 +1552,9 @@ func (p *Parlia) Delay(chain consensus.ChainReader, header *types.Header, leftOv } // The blocking time should be no more than half of period when snap.TurnLength == 1 - timeForMining := time.Duration(p.config.Period) * time.Second / 2 + timeForMining := time.Duration(snap.BlockInterval) * time.Second / 2 if !snap.lastBlockInOneTurn(header.Number.Uint64()) { - timeForMining = time.Duration(p.config.Period) * time.Second * 2 / 3 + timeForMining = time.Duration(snap.BlockInterval) * time.Second * 2 / 3 } if delay > timeForMining { delay = timeForMining @@ -1570,11 +1572,6 @@ func (p *Parlia) Seal(chain consensus.ChainHeaderReader, block *types.Block, res if number == 0 { return errUnknownBlock } - // For 0-period chains, refuse to seal empty blocks (no reward but would spin sealing) - if p.config.Period == 0 && len(block.Transactions()) == 0 { - log.Info("Sealing paused, waiting for transactions") - return nil - } // Don't hold the val fields for the entire sealing procedure p.lock.RLock() val, signFn := p.val, p.signFn @@ -2159,8 +2156,15 @@ func (p *Parlia) backOffTime(snap *Snapshot, header *types.Header, val common.Ad } } -func (p *Parlia) BlockInterval() uint64 { - return p.config.Period +func (p *Parlia) BlockInterval(chain consensus.ChainHeaderReader, header *types.Header) (uint64, error) { + if header == nil { + return uint64(defaultBlockInterval), errUnknownBlock + } + snap, err := p.snapshot(chain, header.Number.Uint64(), header.Hash(), nil) + if err != nil { + return uint64(defaultBlockInterval), err + } + return uint64(snap.BlockInterval), nil } func (p *Parlia) NextProposalBlock(chain consensus.ChainHeaderReader, header *types.Header, proposer common.Address) (uint64, uint64, error) { diff --git a/consensus/parlia/ramanujanfork.go b/consensus/parlia/ramanujanfork.go index ce9089debc..4c82ab08ee 100644 --- a/consensus/parlia/ramanujanfork.go +++ b/consensus/parlia/ramanujanfork.go @@ -27,7 +27,7 @@ func (p *Parlia) delayForRamanujanFork(snap *Snapshot, header *types.Header) tim } func (p *Parlia) blockTimeForRamanujanFork(snap *Snapshot, header, parent *types.Header) uint64 { - blockTime := parent.Time + p.config.Period + blockTime := parent.Time + uint64(snap.BlockInterval) if p.chainConfig.IsRamanujan(header.Number) { blockTime = blockTime + p.backOffTime(snap, header, p.val) } @@ -36,7 +36,7 @@ func (p *Parlia) blockTimeForRamanujanFork(snap *Snapshot, header, parent *types func (p *Parlia) blockTimeVerifyForRamanujanFork(snap *Snapshot, header, parent *types.Header) error { if p.chainConfig.IsRamanujan(header.Number) { - if header.Time < parent.Time+p.config.Period+p.backOffTime(snap, header, header.Coinbase) { + if header.Time < parent.Time+uint64(snap.BlockInterval)+p.backOffTime(snap, header, header.Coinbase) { return consensus.ErrFutureBlock } } diff --git a/consensus/parlia/snapshot.go b/consensus/parlia/snapshot.go index d474c06041..dc24c74cc5 100644 --- a/consensus/parlia/snapshot.go +++ b/consensus/parlia/snapshot.go @@ -44,6 +44,7 @@ type Snapshot struct { Number uint64 `json:"number"` // Block number where the snapshot was created Hash common.Hash `json:"hash"` // Block hash where the snapshot was created + BlockInterval uint8 `json:"block_interval"` // Block Interval TurnLength uint8 `json:"turn_length"` // Length of `turn`, meaning the consecutive number of blocks a validator receives priority for block production Validators map[common.Address]*ValidatorInfo `json:"validators"` // Set of authorized validators at this moment Recents map[uint64]common.Address `json:"recents"` // Set of recent validators for spam protections @@ -74,6 +75,7 @@ func newSnapshot( sigCache: sigCache, Number: number, Hash: hash, + BlockInterval: defaultBlockInterval, TurnLength: defaultTurnLength, Recents: make(map[uint64]common.Address), RecentForkHashes: make(map[uint64]string), @@ -117,6 +119,9 @@ func loadSnapshot(config *params.ParliaConfig, sigCache *lru.ARCCache, db ethdb. if err := json.Unmarshal(blob, snap); err != nil { return nil, err } + if snap.BlockInterval == 0 { // no BlockInterval field in old snapshots + snap.BlockInterval = defaultBlockInterval + } if snap.TurnLength == 0 { // no TurnLength field in old snapshots snap.TurnLength = defaultTurnLength } @@ -145,6 +150,7 @@ func (s *Snapshot) copy() *Snapshot { sigCache: s.sigCache, Number: s.Number, Hash: s.Hash, + BlockInterval: s.BlockInterval, TurnLength: s.TurnLength, Validators: make(map[common.Address]*ValidatorInfo), Recents: make(map[uint64]common.Address), @@ -308,6 +314,10 @@ func (s *Snapshot) apply(headers []*types.Header, chain consensus.ChainHeaderRea } } } + // It's better to set it based on IsOnLorentz, but in practice, the effect is the same as using IsLorentz. + if chainConfig.IsLorentz(header.Number, header.Time) { + snap.BlockInterval = lorentzBlockInterval + } snap.Recents[number] = validator snap.RecentForkHashes[number] = hex.EncodeToString(header.Extra[extraVanity-nextForkHashSize : extraVanity]) snap.updateAttestation(header, chainConfig, s.config) diff --git a/core/vote/vote_manager.go b/core/vote/vote_manager.go index aed7d29b41..26a14684b3 100644 --- a/core/vote/vote_manager.go +++ b/core/vote/vote_manager.go @@ -8,7 +8,6 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/consensus" - "github.com/ethereum/go-ethereum/consensus/parlia" "github.com/ethereum/go-ethereum/core" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/eth/downloader" @@ -147,13 +146,13 @@ func (voteManager *VoteManager) loop() { } curHead := cHead.Header - if p, ok := voteManager.engine.(*parlia.Parlia); ok { - nextBlockMinedTime := time.Unix(int64((curHead.Time + p.Period())), 0) - timeForBroadcast := 50 * time.Millisecond // enough to broadcast a vote - if time.Now().Add(timeForBroadcast).After(nextBlockMinedTime) { - log.Warn("too late to vote", "Head.Time(Second)", curHead.Time, "Now(Millisecond)", time.Now().UnixMilli()) - continue - } + parentHeader := voteManager.chain.GetHeaderByHash(curHead.ParentHash) + blockInterval, _ := voteManager.engine.BlockInterval(voteManager.chain, parentHeader) + nextBlockMinedTime := time.Unix(int64((curHead.Time + blockInterval)), 0) + timeForBroadcast := 50 * time.Millisecond // enough to broadcast a vote + if time.Now().Add(timeForBroadcast).After(nextBlockMinedTime) { + log.Warn("too late to vote", "Head.Time(Second)", curHead.Time, "Now(Millisecond)", time.Now().UnixMilli()) + continue } // Check if cur validator is within the validatorSet at curHead diff --git a/miner/bid_simulator.go b/miner/bid_simulator.go index a48407b7e2..61248b2409 100644 --- a/miner/bid_simulator.go +++ b/miner/bid_simulator.go @@ -398,7 +398,8 @@ func (b *bidSimulator) newBidLoop() { func (b *bidSimulator) bidBetterBefore(parentHash common.Hash) time.Time { parentHeader := b.chain.GetHeaderByHash(parentHash) - return bidutil.BidBetterBefore(parentHeader, b.chainConfig.Parlia.Period, b.delayLeftOver, b.config.BidSimulationLeftOver) + blockInterval, _ := b.engine.BlockInterval(b.chain, parentHeader) + return bidutil.BidBetterBefore(parentHeader, blockInterval, b.delayLeftOver, b.config.BidSimulationLeftOver) } func (b *bidSimulator) clearLoop() { diff --git a/miner/worker.go b/miner/worker.go index 31eb752901..34a03c1b88 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -469,7 +469,7 @@ func (w *worker) newWorkLoop(recommit time.Duration) { // If sealing is running resubmit a new work cycle periodically to pull in // higher priced transactions. Disable this overhead for pending blocks. if w.isRunning() && ((w.chainConfig.Clique != nil && - w.chainConfig.Clique.Period > 0) || (w.chainConfig.Parlia != nil && w.chainConfig.Parlia.Period > 0)) { + w.chainConfig.Clique.Period > 0) || (w.chainConfig.Parlia != nil)) { // Short circuit if no new transaction arrives. commit(commitInterruptResubmit) } @@ -1538,13 +1538,14 @@ func (w *worker) tryWaitProposalDoneWhenStopping() { log.Warn("next proposal end block has passed, ignore") return } - if startBlock > currentBlock && (startBlock-currentBlock)*posa.BlockInterval() > w.config.MaxWaitProposalInSecs { + blockInterval, _ := w.engine.BlockInterval(w.chain, currentHeader) + if startBlock > currentBlock && (startBlock-currentBlock)*blockInterval > w.config.MaxWaitProposalInSecs { log.Warn("the next proposal start block is too far, just skip waiting") return } // wait one more block for safety - waitSecs := (endBlock - currentBlock + 1) * posa.BlockInterval() + waitSecs := (endBlock - currentBlock + 1) * blockInterval log.Info("The miner will propose in later, waiting for the proposal to be done", "currentBlock", currentBlock, "nextProposalStart", startBlock, "nextProposalEnd", endBlock, "waitTime", waitSecs) time.Sleep(time.Duration(waitSecs) * time.Second) diff --git a/params/config.go b/params/config.go index 1312f644ae..827715bfba 100644 --- a/params/config.go +++ b/params/config.go @@ -189,8 +189,7 @@ var ( LorentzTime: nil, Parlia: &ParliaConfig{ - Period: 3, - Epoch: 200, + Epoch: 200, }, BlobScheduleConfig: &BlobScheduleConfig{ Cancun: DefaultCancunBlobConfig, @@ -238,8 +237,7 @@ var ( LorentzTime: nil, Parlia: &ParliaConfig{ - Period: 3, - Epoch: 200, + Epoch: 200, }, BlobScheduleConfig: &BlobScheduleConfig{ Cancun: DefaultCancunBlobConfig, @@ -288,8 +286,7 @@ var ( LorentzTime: nil, Parlia: &ParliaConfig{ - Period: 3, - Epoch: 200, + Epoch: 200, }, BlobScheduleConfig: &BlobScheduleConfig{ Cancun: DefaultCancunBlobConfig, @@ -330,8 +327,7 @@ var ( CancunTime: newUint64(0), Parlia: &ParliaConfig{ - Period: 3, - Epoch: 200, + Epoch: 200, }, BlobScheduleConfig: &BlobScheduleConfig{ Cancun: DefaultCancunBlobConfig, @@ -677,8 +673,7 @@ func (c CliqueConfig) String() string { // ParliaConfig is the consensus engine configs for proof-of-staked-authority based sealing. type ParliaConfig struct { - Period uint64 `json:"period"` // Number of seconds between blocks to enforce - Epoch uint64 `json:"epoch"` // Epoch length to update validatorSet + Epoch uint64 `json:"epoch"` // Epoch length to update validatorSet } // String implements the stringer interface, returning the consensus engine details.