From aa9a2cc9fb5e242f2d499444abd85d05eead2cae Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Fri, 3 Oct 2025 16:16:10 +0300 Subject: [PATCH 1/2] apply min blok delay to block builder --- RELEASES.md | 8 +- plugin/evm/block_builder.go | 111 ++++++++++++++--- plugin/evm/block_builder_test.go | 182 ++++++++++++++++++++++++++++ plugin/evm/customtypes/block_ext.go | 8 ++ plugin/evm/vm_test.go | 24 ++-- 5 files changed, 303 insertions(+), 30 deletions(-) create mode 100644 plugin/evm/block_builder_test.go diff --git a/RELEASES.md b/RELEASES.md index 45bc5a4ba1..31ed556bc5 100644 --- a/RELEASES.md +++ b/RELEASES.md @@ -4,9 +4,11 @@ - Removed deprecated flags `coreth-admin-api-enabled`, `coreth-admin-api-dir`, `tx-regossip-frequency`, `tx-lookup-limit`. Use `admin-api-enabled`, `admin-api-dir`, `regossip-frequency`, `transaction-history` instead. - Enabled RPC batch limits by default, and configurable with `batch-request-limit` and `batch-max-response-size`. -- Implement ACP-226: Set expected block gas cost to 0 in Granite network upgrade, removing block gas cost requirements for block building. -- Implement ACP-226: Add `timeMilliseconds` (Unix uint64) timestamp to block header for Granite upgrade. -- Implement ACP-226: Add `minDelayExcess` (uint64) to block header for Granite upgrade. +- ACP-226: + - Set expected block gas cost to 0 in Granite network upgrade, removing block gas cost requirements for block building. + - Add `timeMilliseconds` (Unix uint64) timestamp to block header for Granite upgrade. + - Add `minDelayExcess` (uint64) to block header for Granite upgrade. + - Add minimum block building delays to conform ACP-226 requirements to the block builder. - Update go version to 1.24.7 ## [v0.15.3](https://github.com/ava-labs/coreth/releases/tag/v0.15.3) diff --git a/plugin/evm/block_builder.go b/plugin/evm/block_builder.go index 45e99198cf..72d41595ef 100644 --- a/plugin/evm/block_builder.go +++ b/plugin/evm/block_builder.go @@ -10,6 +10,7 @@ import ( "github.com/ava-labs/avalanchego/snow" "github.com/ava-labs/avalanchego/utils/lock" + "github.com/ava-labs/avalanchego/utils/timer/mockable" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/log" @@ -18,17 +19,29 @@ import ( "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/txpool" + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/customheader" + "github.com/ava-labs/coreth/plugin/evm/customtypes" "github.com/ava-labs/coreth/plugin/evm/extension" commonEng "github.com/ava-labs/avalanchego/snow/engine/common" ) -// Minimum amount of time to wait after building a block before attempting to build a block -// a second time without changing the contents of the mempool. -const MinBlockBuildingRetryDelay = 500 * time.Millisecond +const ( + // Minimum amount of time to wait after building a block before attempting to build a block + // a second time without changing the contents of the mempool. + PreGraniteMinBlockBuildingRetryDelay = 500 * time.Millisecond + // Minimum amount of time to wait after attempting/build a block before attempting to build another block + // This is only applied for retrying to build a block after a minimum delay has passed. + // The initial minimum delay is applied according to parent minDelayExcess (if available) + // TODO (ceyonur): Decide whether this a correct value. + PostGraniteMinBlockBuildingRetryDelay = 100 * time.Millisecond +) type blockBuilder struct { - ctx *snow.Context + clock *mockable.Clock + ctx *snow.Context + chainConfig *extras.ChainConfig txPool *txpool.TxPool extraMempool extension.BuilderMempool @@ -51,10 +64,12 @@ type blockBuilder struct { func (vm *VM) NewBlockBuilder(extraMempool extension.BuilderMempool) *blockBuilder { b := &blockBuilder{ ctx: vm.ctx, + chainConfig: vm.chainConfigExtra(), txPool: vm.txPool, extraMempool: extraMempool, shutdownChan: vm.shutdownChan, shutdownWg: &vm.shutdownWg, + clock: vm.clock, } b.pendingSignal = lock.NewCond(&b.buildBlockLock) return b @@ -64,7 +79,7 @@ func (vm *VM) NewBlockBuilder(extraMempool extension.BuilderMempool) *blockBuild func (b *blockBuilder) handleGenerateBlock(currentParentHash common.Hash) { b.buildBlockLock.Lock() defer b.buildBlockLock.Unlock() - b.lastBuildTime = time.Now() + b.lastBuildTime = b.clock.Time() b.lastBuildParentHash = currentParentHash } @@ -124,20 +139,19 @@ func (b *blockBuilder) waitForEvent(ctx context.Context, currentHeader *types.He if err != nil { return 0, err } - timeSinceLastBuildTime := time.Since(lastBuildTime) - isRetry := lastBuildParentHash == currentHeader.ParentHash - // 1. if this is not a retry - // 2. if this is the first time we try to build a block - // 3. if the time since the last build is greater than the minimum retry delay - // then we can build a block immediately. - if !isRetry || lastBuildTime.IsZero() || timeSinceLastBuildTime >= MinBlockBuildingRetryDelay { - b.ctx.Log.Debug("Last time we built a block was long enough ago or this is not a retry, no need to wait", - zap.Duration("timeSinceLastBuildTime", timeSinceLastBuildTime), - zap.Bool("isRetry", isRetry), - ) + timeUntilNextBuild, shouldBuildImmediately, err := b.calculateBlockBuildingDelay( + lastBuildTime, + lastBuildParentHash, + currentHeader, + ) + if err != nil { + return 0, err + } + if shouldBuildImmediately { + b.ctx.Log.Debug("Last time we built a block was long enough ago or this is not a retry, no need to wait") return commonEng.PendingTxs, nil } - timeUntilNextBuild := MinBlockBuildingRetryDelay - timeSinceLastBuildTime + b.ctx.Log.Debug("Last time we built a block was too recent, waiting", zap.Duration("timeUntilNextBuild", timeUntilNextBuild), ) @@ -161,3 +175,66 @@ func (b *blockBuilder) waitForNeedToBuild(ctx context.Context) (time.Time, commo } return b.lastBuildTime, b.lastBuildParentHash, nil } + +// getMinBlockBuildingDelays returns the initial min block building delay and the minimum retry delay. +// It implements the following logic: +// 1. If the current header is in Granite, return the remaining ACP-226 delay after the parent block time and the minimum retry delay. +// 2. If the current header is not in Granite, return 0 and the minimum retry delay. +func (b *blockBuilder) getMinBlockBuildingDelays(currentHeader *types.Header, config *extras.ChainConfig) (time.Duration, time.Duration, error) { + // TODO Cleanup (ceyonur): this check can be removed after Granite is activated. + currentTimestamp := b.clock.Unix() + if !config.IsGranite(currentTimestamp) { + return 0, PreGraniteMinBlockBuildingRetryDelay, nil // Pre-Granite: no initial delay + } + + acp226DelayExcess, err := customheader.MinDelayExcess(config, currentHeader, currentTimestamp, nil) + if err != nil { + return 0, 0, err + } + acp226Delay := time.Duration(acp226DelayExcess.Delay()) * time.Millisecond + + // Calculate initial delay: time since parent minus ACP-226 delay (clamped to 0) + parentBlockTime := customtypes.BlockTime(currentHeader) + timeSinceParentBlock := b.clock.Time().Sub(parentBlockTime) + // TODO question (ceyonur): should we just use acp226Delay if timeSinceParentBlock is negative? + initialMinBlockBuildingDelay := acp226Delay - timeSinceParentBlock + if initialMinBlockBuildingDelay < 0 { + initialMinBlockBuildingDelay = 0 + } + + return initialMinBlockBuildingDelay, PostGraniteMinBlockBuildingRetryDelay, nil +} + +// calculateBlockBuildingDelay calculates the delay needed before building the next block. +// It returns the time to wait, a boolean indicating whether to build immediately, and any error. +// It implements the following logic: +// 1. If there is no initial min block building delay +// 2. if this is not a retry +// 3. if the time since the last build is greater than the minimum retry delay +// then we can build a block immediately. +func (b *blockBuilder) calculateBlockBuildingDelay( + lastBuildTime time.Time, + lastBuildParentHash common.Hash, + currentHeader *types.Header, +) (time.Duration, bool, error) { + initialMinBlockBuildingDelay, minBlockBuildingRetryDelay, err := b.getMinBlockBuildingDelays(currentHeader, b.chainConfig) + if err != nil { + return 0, false, err + } + + isRetry := lastBuildParentHash == currentHeader.ParentHash && !lastBuildTime.IsZero() // if last build time is zero, this is not a retry + + timeSinceLastBuildTime := b.clock.Time().Sub(lastBuildTime) + var remainingMinDelay time.Duration + if minBlockBuildingRetryDelay > timeSinceLastBuildTime { + remainingMinDelay = minBlockBuildingRetryDelay - timeSinceLastBuildTime + } + + if initialMinBlockBuildingDelay > 0 { + remainingMinDelay = max(initialMinBlockBuildingDelay, remainingMinDelay) + } else if !isRetry || remainingMinDelay == 0 { + return 0, true, nil // Build immediately + } + + return remainingMinDelay, false, nil // Need to wait +} diff --git a/plugin/evm/block_builder_test.go b/plugin/evm/block_builder_test.go new file mode 100644 index 0000000000..481094e4ba --- /dev/null +++ b/plugin/evm/block_builder_test.go @@ -0,0 +1,182 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "testing" + "time" + + "github.com/ava-labs/avalanchego/utils/timer/mockable" + "github.com/ava-labs/avalanchego/vms/evm/acp226" + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" + "github.com/stretchr/testify/require" + + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/customtypes" +) + +func TestCalculateBlockBuildingDelay(t *testing.T) { + now := time.UnixMilli(10000) + nowSecUint64 := uint64(now.Unix()) + nowMilliUint64 := uint64(now.UnixMilli()) + clock := &mockable.Clock{} + clock.Set(now) + tests := []struct { + name string + config *extras.ChainConfig + currentHeader *types.Header + lastBuildTime time.Time + lastBuildParentHash common.Hash + expectedTimeToWait time.Duration + expectedShouldBuildNow bool + }{ + { + name: "pre_granite_returns_build_immediately_zero_time", + config: extras.TestFortunaChainConfig, // Pre-Granite config + currentHeader: &types.Header{ + ParentHash: common.Hash{1}, + Time: nowSecUint64, + }, + lastBuildTime: time.Time{}, // Zero time means not a retry + lastBuildParentHash: common.Hash{1}, + expectedShouldBuildNow: true, + }, + { + name: "pre_granite_returns_build_immediately_different_parent_hash", + config: extras.TestFortunaChainConfig, // Pre-Granite config + currentHeader: &types.Header{ + ParentHash: common.Hash{2}, + Time: nowSecUint64, + }, + lastBuildTime: now, + lastBuildParentHash: common.Hash{1}, + expectedShouldBuildNow: true, + }, + { + name: "pre_granite_returns_build_delays_with_same_parent_hash", + config: extras.TestFortunaChainConfig, // Pre-Granite config + currentHeader: &types.Header{ + ParentHash: common.Hash{1}, + Time: nowSecUint64, + }, + lastBuildTime: now, + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: PreGraniteMinBlockBuildingRetryDelay, + expectedShouldBuildNow: false, + }, + { + name: "pre_granite_returns_build_returns_immediately_if_enough_time_passed", + config: extras.TestFortunaChainConfig, // Pre-Granite config + currentHeader: &types.Header{ + ParentHash: common.Hash{1}, + Time: nowSecUint64, + }, + lastBuildTime: now.Add(-PreGraniteMinBlockBuildingRetryDelay), // Less than retry delay ago + lastBuildParentHash: common.Hash{1}, // Same as current parent + expectedTimeToWait: 0, + expectedShouldBuildNow: true, + }, + { + name: "pre_granite_returns_build_delays_only_remaining_min_delay", + config: extras.TestFortunaChainConfig, // Pre-Granite config + currentHeader: &types.Header{ + ParentHash: common.Hash{1}, + Time: nowSecUint64, + }, + lastBuildTime: now.Add(-PreGraniteMinBlockBuildingRetryDelay / 2), // Less than retry delay ago + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: PreGraniteMinBlockBuildingRetryDelay / 2, + expectedShouldBuildNow: false, + }, + { + name: "granite_block_with_now_time", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64, acp226.InitialDelayExcess), + lastBuildTime: time.Time{}, + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: 2000 * time.Millisecond, // should wait for initial delay + expectedShouldBuildNow: false, + }, + { + name: "granite_block_with_2_seconds_before_clock_no_retry", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64-2000, acp226.InitialDelayExcess), + lastBuildTime: time.Time{}, // Zero time means not a retry + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: 0, // should not wait for initial delay + expectedShouldBuildNow: true, + }, + { + name: "granite_block_with_2_seconds_before_clock_with_retry", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64-2000, acp226.InitialDelayExcess), + lastBuildTime: now, + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: PostGraniteMinBlockBuildingRetryDelay, + expectedShouldBuildNow: false, + }, + { + name: "granite_with_2_seconds_before_clock_only_waits_for_retry_delay", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64-2000, 0), // 0 means min delay excess which is 1 + lastBuildTime: now, + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: PostGraniteMinBlockBuildingRetryDelay, + expectedShouldBuildNow: false, + }, + { + name: "granite_with_2_seconds_before_clock_only_waits_for_remaining_retry_delay", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64-2000, 0), // 0 means min delay excess which is 1 + lastBuildTime: now.Add(-PostGraniteMinBlockBuildingRetryDelay / 2), // Less than retry delay ago + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: PostGraniteMinBlockBuildingRetryDelay / 2, + expectedShouldBuildNow: false, + }, + { + name: "granite_with_2_seconds_after_clock", + config: extras.TestGraniteChainConfig, + currentHeader: createGraniteTestHeader(common.Hash{1}, nowMilliUint64+2000, acp226.InitialDelayExcess), + lastBuildTime: time.Time{}, // Zero time means not a retry + lastBuildParentHash: common.Hash{1}, + expectedTimeToWait: 4000 * time.Millisecond, + expectedShouldBuildNow: false, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + b := &blockBuilder{ + clock: clock, + chainConfig: tt.config, + } + + timeToWait, shouldBuildNow, err := b.calculateBlockBuildingDelay( + tt.lastBuildTime, + tt.lastBuildParentHash, + tt.currentHeader, + ) + + require.NoError(t, err) + require.Equal(t, tt.expectedTimeToWait, timeToWait) + require.Equal(t, tt.expectedShouldBuildNow, shouldBuildNow) + }) + } +} + +func createGraniteTestHeader(parentHash common.Hash, timeMilliseconds uint64, minDelayExcess acp226.DelayExcess) *types.Header { + header := &types.Header{ + Time: timeMilliseconds / 1000, + } + header.ParentHash = parentHash + + extra := &customtypes.HeaderExtra{ + TimeMilliseconds: &timeMilliseconds, + MinDelayExcess: &minDelayExcess, + } + customtypes.SetHeaderExtra(header, extra) + + return header +} diff --git a/plugin/evm/customtypes/block_ext.go b/plugin/evm/customtypes/block_ext.go index 4e66b132ce..669d00d4c0 100644 --- a/plugin/evm/customtypes/block_ext.go +++ b/plugin/evm/customtypes/block_ext.go @@ -6,6 +6,7 @@ package customtypes import ( "math/big" "slices" + "time" "github.com/ava-labs/avalanchego/vms/evm/acp226" "github.com/ava-labs/libevm/common" @@ -155,6 +156,13 @@ func CalcExtDataHash(extdata []byte) common.Hash { return ethtypes.RLPHash(extdata) } +func BlockTime(eth *ethtypes.Header) time.Time { + if t := GetHeaderExtra(eth).TimeMilliseconds; t != nil { + return time.UnixMilli(int64(*t)) + } + return time.Unix(int64(eth.Time), 0) +} + func NewBlockWithExtData( header *ethtypes.Header, txs []*ethtypes.Transaction, uncles []*ethtypes.Header, receipts []*ethtypes.Receipt, hasher ethtypes.TrieHasher, extdata []byte, recalc bool, diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index 3edffbff12..b034be1c39 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -1549,8 +1549,10 @@ func TestBuildBlockLargeTxStarvation(t *testing.T) { } func TestWaitForEvent(t *testing.T) { + fortunaFork := upgradetest.Fortuna for _, testCase := range []struct { name string + Fork *upgradetest.Fork testCase func(*testing.T, *VM) }{ { @@ -1614,8 +1616,10 @@ func TestWaitForEvent(t *testing.T) { wg.Wait() }, }, + // TODO Cleanup (ceyonur): remove this test after Granite is activated. { - name: "WaitForEvent does not wait to build on a new block", + name: "WaitForEvent does not wait for new block to be built in fortuna", + Fork: &fortunaFork, testCase: func(t *testing.T, vm *VM) { signedTx := newSignedLegacyTx(t, vm.chainConfig, vmtest.TestKeys[0].ToECDSA(), 0, &vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) blk, err := vmtest.IssueTxsAndSetPreference([]*types.Transaction{signedTx}, vm) @@ -1641,13 +1645,15 @@ func TestWaitForEvent(t *testing.T) { wg.Wait() }, }, + // TODO Cleanup (ceyonur): remove this test after Granite is activated. { - name: "WaitForEvent waits for a delay with a retry", + name: "WaitForEvent waits for a delay with a retry in fortuna", + Fork: &fortunaFork, testCase: func(t *testing.T, vm *VM) { lastBuildBlockTime := time.Now() _, err := vm.BuildBlock(context.Background()) require.NoError(t, err) - // we haven't advanced the tip to include the previous built block, so this is a retry + // we haven't accepted the previous built block, so this should be a retry signedTx := newSignedLegacyTx(t, vm.chainConfig, vmtest.TestKeys[0].ToECDSA(), 0, &vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) for _, err := range vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) { require.NoError(t, err) @@ -1660,7 +1666,7 @@ func TestWaitForEvent(t *testing.T) { msg, err := vm.WaitForEvent(context.Background()) assert.NoError(t, err) assert.Equal(t, commonEng.PendingTxs, msg) - assert.GreaterOrEqual(t, time.Since(lastBuildBlockTime), MinBlockBuildingRetryDelay) + assert.GreaterOrEqual(t, time.Since(lastBuildBlockTime), PreGraniteMinBlockBuildingRetryDelay) }() wg.Wait() }, @@ -1668,6 +1674,9 @@ func TestWaitForEvent(t *testing.T) { } { t.Run(testCase.name, func(t *testing.T) { fork := upgradetest.Latest + if testCase.Fork != nil { + fork = *testCase.Fork + } vm := newDefaultTestVM() vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ Fork: &fork, @@ -1901,14 +1910,9 @@ func TestDelegatePrecompile_BehaviorAcrossUpgrades(t *testing.T) { data := crypto.Keccak256([]byte("delegateSendHello()"))[:4] nonce := vm.txPool.Nonce(vmtest.TestEthAddrs[0]) signedTx := newSignedLegacyTx(t, vm.chainConfig, vmtest.TestKeys[0].ToECDSA(), nonce, &contractAddr, big.NewInt(0), 100000, tt.txGasPrice, data) - for _, err := range vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) { - require.NoError(t, err) - } - blk, err := vm.BuildBlock(ctx) + blk, err := vmtest.IssueTxsAndSetPreference([]*types.Transaction{signedTx}, vm) require.NoError(t, err) - require.NoError(t, blk.Verify(ctx)) - require.NoError(t, vm.SetPreference(ctx, blk.ID())) require.NoError(t, blk.Accept(ctx)) ethBlock := blk.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock From 6a3b9331cabd18d24c705d1146915c118cc29f02 Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Fri, 3 Oct 2025 16:21:13 +0300 Subject: [PATCH 2/2] add delay verification --- miner/worker.go | 5 +- plugin/evm/customheader/min_delay_excess.go | 2 +- plugin/evm/customheader/time.go | 77 +++++-- plugin/evm/customheader/time_test.go | 220 ++++++++++++++++++-- 4 files changed, 266 insertions(+), 38 deletions(-) diff --git a/miner/worker.go b/miner/worker.go index 783c1cedd6..946f9f5c63 100644 --- a/miner/worker.go +++ b/miner/worker.go @@ -149,7 +149,10 @@ func (w *worker) commitNewWork(predicateContext *precompileconfig.PredicateConte chainExtra = params.GetExtra(w.chainConfig) ) - timestamp, timestampMS := customheader.GetNextTimestamp(parent, tstart) + timestamp, timestampMS, err := customheader.GetNextTimestamp(chainExtra, parent, tstart) + if err != nil { + return nil, fmt.Errorf("failed to get next timestamp: %w", err) + } header := &types.Header{ ParentHash: parent.Hash(), diff --git a/plugin/evm/customheader/min_delay_excess.go b/plugin/evm/customheader/min_delay_excess.go index 0498fdc32c..a4cd4975d0 100644 --- a/plugin/evm/customheader/min_delay_excess.go +++ b/plugin/evm/customheader/min_delay_excess.go @@ -77,7 +77,7 @@ func VerifyMinDelayExcess( } default: // Prior to Granite there was no expected min delay excess. - // TODO (ceyonur): this can be removed after Granite is activated. + // TODO Cleanup (ceyonur): this can be removed after Granite is activated. if customtypes.GetHeaderExtra(header).MinDelayExcess != nil { return fmt.Errorf("%w: %s", errRemoteMinDelayExcessSet, header.Hash()) } diff --git a/plugin/evm/customheader/time.go b/plugin/evm/customheader/time.go index 75075d1ee4..c92fdd3ff7 100644 --- a/plugin/evm/customheader/time.go +++ b/plugin/evm/customheader/time.go @@ -24,30 +24,34 @@ var ( ErrTimeMillisecondsRequired = errors.New("TimeMilliseconds is required after Granite activation") ErrTimeMillisecondsMismatched = errors.New("TimeMilliseconds does not match header.Time") ErrTimeMillisecondsBeforeGranite = errors.New("TimeMilliseconds should be nil before Granite activation") + ErrMinDelayNotMet = errors.New("minimum block delay not met") + ErrGraniteClockBehindParent = errors.New("current timestamp is not allowed to be behind than parent timestamp in Granite") ) -// GetNextTimestamp calculates the timestamp (in seconds and milliseconds) for the next child block based on the parent's timestamp and the current time. +// GetNextTimestamp calculates the timestamp (in seconds and milliseconds) for the header based on the parent's timestamp and the current time. // First return value is the timestamp in seconds, second return value is the timestamp in milliseconds. -func GetNextTimestamp(parent *types.Header, now time.Time) (uint64, uint64) { +func GetNextTimestamp(config *extras.ChainConfig, parent *types.Header, now time.Time) (uint64, uint64, error) { var ( timestamp = uint64(now.Unix()) timestampMS = uint64(now.UnixMilli()) ) - // Note: in order to support asynchronous block production, blocks are allowed to have - // the same timestamp as their parent. This allows more than one block to be produced - // per second. - parentExtra := customtypes.GetHeaderExtra(parent) - if parent.Time >= timestamp || - (parentExtra.TimeMilliseconds != nil && *parentExtra.TimeMilliseconds >= timestampMS) { - timestamp = parent.Time - // If the parent has a TimeMilliseconds, use it. Otherwise, use the parent time * 1000. - if parentExtra.TimeMilliseconds != nil { - timestampMS = *parentExtra.TimeMilliseconds - } else { - timestampMS = parent.Time * 1000 // TODO: establish minimum time + if parent.Time < timestamp { + return timestamp, timestampMS, nil + } + // In Granite, there is a minimum delay enforced, and if the timestamp is the same as the parent, + // the block will be rejected. + // The block builder should have already waited enough time to meet the minimum delay. + // This is to early-exit from the block building. + if config.IsGranite(timestamp) { + if uint64(customtypes.BlockTime(parent).UnixMilli()) >= timestampMS { + return 0, 0, ErrGraniteClockBehindParent } + return timestamp, timestampMS, nil } - return timestamp, timestampMS + // In pre-Granite, blocks are allowed to have the same timestamp as their parent + // Actually we don't need to return modified timestampMS, because it will be not be set if this is not + // Granite, but setting here for consistency. + return parent.Time, parent.Time * 1000, nil } // VerifyTime verifies that the header's Time and TimeMilliseconds fields are @@ -100,7 +104,6 @@ func VerifyTime(extraConfig *extras.ChainConfig, parent *types.Header, header *t } // Verify TimeMilliseconds is not earlier than parent's TimeMilliseconds - // TODO: Ensure minimum block delay is enforced if parentExtra.TimeMilliseconds != nil && *headerExtra.TimeMilliseconds < *parentExtra.TimeMilliseconds { return fmt.Errorf("%w: %d < parent %d", errBlockTooOld, @@ -109,7 +112,14 @@ func VerifyTime(extraConfig *extras.ChainConfig, parent *types.Header, header *t ) } + // Verify minimum block delay is enforced + if err := verifyMinDelay(parent, header); err != nil { + return err + } + // Verify TimeMilliseconds is not too far in the future + // Q: This still is a problem especially if someone issues a block in the future + // then the next builder will wait until potentially MaxFutureBlockTime + delay if maxBlockTimeMillis := uint64(now.Add(MaxFutureBlockTime).UnixMilli()); *headerExtra.TimeMilliseconds > maxBlockTimeMillis { return fmt.Errorf("%w: %d > allowed %d", ErrBlockTooFarInFuture, @@ -120,3 +130,38 @@ func VerifyTime(extraConfig *extras.ChainConfig, parent *types.Header, header *t return nil } + +// verifyMinDelay verifies that the minimum block delay is enforced based on the min delay excess. +func verifyMinDelay(parent *types.Header, header *types.Header) error { + headerExtra := customtypes.GetHeaderExtra(header) + parentExtra := customtypes.GetHeaderExtra(parent) + + // if parent has no TimeMilliseconds, no min delay is required + if parentExtra.TimeMilliseconds == nil { + return nil + } + + // Calculate the actual time difference in milliseconds + actualDelayMillis := *headerExtra.TimeMilliseconds - *parentExtra.TimeMilliseconds + + // Parent might not have a min delay excess if this is the first Granite block + // in this case we cannot verify the min delay, + // Otherwise parent should have been verified in VerifyMinDelayExcess + minDelayExcess := customtypes.GetHeaderExtra(parent).MinDelayExcess + if minDelayExcess == nil { + return nil + } + + minRequiredDelayMillis := minDelayExcess.Delay() + + // Check if the actual delay meets the minimum requirement + if actualDelayMillis < minRequiredDelayMillis { + return fmt.Errorf("%w: actual delay %dms < required %dms", + ErrMinDelayNotMet, + actualDelayMillis, + minRequiredDelayMillis, + ) + } + + return nil +} diff --git a/plugin/evm/customheader/time_test.go b/plugin/evm/customheader/time_test.go index 375110e082..284a2a8365 100644 --- a/plugin/evm/customheader/time_test.go +++ b/plugin/evm/customheader/time_test.go @@ -7,6 +7,7 @@ import ( "testing" "time" + "github.com/ava-labs/avalanchego/vms/evm/acp226" "github.com/ava-labs/libevm/core/types" "github.com/stretchr/testify/require" @@ -15,17 +16,6 @@ import ( "github.com/ava-labs/coreth/utils" ) -func generateHeader(timeSeconds uint64, timeMilliseconds *uint64) *types.Header { - return customtypes.WithHeaderExtra( - &types.Header{ - Time: timeSeconds, - }, - &customtypes.HeaderExtra{ - TimeMilliseconds: timeMilliseconds, - }, - ) -} - func TestVerifyTime(t *testing.T) { var ( now = time.Unix(1714339200, 123_456_789) @@ -138,6 +128,139 @@ func TestVerifyTime(t *testing.T) { parentHeader: generateHeader(timeSeconds, nil), extraConfig: extras.TestGraniteChainConfig, }, + // Min delay verification tests + { + name: "pre_granite_no_min_delay_verification", + header: generateHeader(timeSeconds, nil), + parentHeader: generateHeader(timeSeconds-1, nil), + extraConfig: extras.TestFortunaChainConfig, + }, + { + name: "granite_first_block_no_parent_min_delay_excess", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(acp226.InitialDelayExcess), + ), + parentHeader: generateHeader(timeSeconds-1, nil), // Pre-Granite parent + extraConfig: extras.TestGraniteChainConfig, + }, + { + name: "granite_min_delay_met", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(acp226.InitialDelayExcess), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-uint64(acp226.InitialDelayExcess)), // Exact minimum delay + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + }, + { + name: "granite_min_delay_not_met", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(acp226.InitialDelayExcess), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-100), // Only 100ms delay, less than required + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + expectedErr: ErrMinDelayNotMet, + }, + { + name: "granite_future_timestamp_within_limits", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds+5, // 5 seconds in future + utils.NewUint64(timeMillis+5000), + utils.NewUint64(acp226.InitialDelayExcess), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-uint64(acp226.InitialDelayExcess)), + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + }, + { + name: "granite_future_timestamp_abuse", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds+15, // 15 seconds in future, exceeds MaxFutureBlockTime + utils.NewUint64(timeMillis+15000), + utils.NewUint64(acp226.InitialDelayExcess), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-uint64(acp226.InitialDelayExcess)), + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + expectedErr: ErrBlockTooFarInFuture, + }, + { + name: "granite_min_delay_excess_updated", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(acp226.InitialDelayExcess+acp226.MaxDelayExcessDiff), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-uint64(acp226.InitialDelayExcess+acp226.MaxDelayExcessDiff)), // Meets increased requirement + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + }, + { + name: "granite_min_delay_excess_updated_but_delay_insufficient", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(acp226.InitialDelayExcess+acp226.MaxDelayExcessDiff), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-1000), // 1000ms delay, insufficient for increased requirement + utils.NewUint64(acp226.InitialDelayExcess), + ), + extraConfig: extras.TestGraniteChainConfig, + expectedErr: ErrMinDelayNotMet, + }, + { + name: "granite_zero_delay_excess", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(0), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds-1, + utils.NewUint64(timeMillis-1), // 1ms delay, meets zero requirement + utils.NewUint64(0), // Parent has zero delay excess + ), + extraConfig: extras.TestGraniteChainConfig, + }, + { + name: "granite_zero_delay_excess_but_zero_delay", + header: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), + utils.NewUint64(0), + ), + parentHeader: generateHeaderWithMinDelayExcessAndTime( + timeSeconds, + utils.NewUint64(timeMillis), // Same timestamp, zero delay + utils.NewUint64(0), // Parent has zero delay excess + ), + extraConfig: extras.TestGraniteChainConfig, + expectedErr: ErrMinDelayNotMet, + }, } for _, test := range tests { @@ -162,9 +285,11 @@ func TestGetNextTimestamp(t *testing.T) { tests := []struct { name string parent *types.Header + extraConfig *extras.ChainConfig now time.Time expectedSec uint64 expectedMillis uint64 + expectedErr error }{ { name: "current_time_after_parent_time_no_milliseconds", @@ -181,47 +306,102 @@ func TestGetNextTimestamp(t *testing.T) { expectedMillis: nowMillis, }, { - name: "current_time_equals_parent_time_no_milliseconds", + name: "current_time_equals_parent_time_no_milliseconds_pre_granite", parent: generateHeader(nowSeconds, nil), + extraConfig: extras.TestFortunaChainConfig, now: now, expectedSec: nowSeconds, expectedMillis: nowSeconds * 1000, // parent.Time * 1000 }, { - name: "current_time_equals_parent_time_with_milliseconds", + name: "current_time_equals_parent_time_with_milliseconds_pre_granite", parent: generateHeader(nowSeconds, utils.NewUint64(nowMillis)), + extraConfig: extras.TestFortunaChainConfig, now: now, expectedSec: nowSeconds, - expectedMillis: nowMillis, // parent's TimeMilliseconds + expectedMillis: nowSeconds * 1000, // parent.Time * 1000 }, { - name: "current_time_before_parent_time", + name: "current_time_before_parent_time_pre_granite", parent: generateHeader(nowSeconds+10, nil), + extraConfig: extras.TestFortunaChainConfig, now: now, expectedSec: nowSeconds + 10, expectedMillis: (nowSeconds + 10) * 1000, // parent.Time * 1000 }, { - name: "current_time_before_parent_time_with_milliseconds", + name: "current_time_before_parent_time_with_milliseconds_pre_granite", parent: generateHeader(nowSeconds+10, utils.NewUint64(nowMillis)), + extraConfig: extras.TestFortunaChainConfig, now: now, expectedSec: nowSeconds + 10, - expectedMillis: nowMillis, // parent's TimeMilliseconds + expectedMillis: (nowSeconds + 10) * 1000, // parent.Time * 1000 }, { - name: "current_time_milliseconds_before_parent_time_milliseconds", + name: "current_time_milliseconds_before_parent_time_milliseconds_pre_granite", parent: generateHeader(nowSeconds, utils.NewUint64(nowMillis+10)), + extraConfig: extras.TestFortunaChainConfig, + now: now, + expectedSec: nowSeconds, + expectedMillis: nowSeconds * 1000, // parent.Time * 1000 + }, + { + name: "current_time_before_parent_time_granite", + parent: generateHeader(nowSeconds+10, utils.NewUint64(nowMillis)), + extraConfig: extras.TestGraniteChainConfig, + now: now, + expectedErr: ErrGraniteClockBehindParent, + }, + { + name: "current_time_equals_parent_time_with_milliseconds_granite", + parent: generateHeader(nowSeconds, utils.NewUint64(nowMillis)), + extraConfig: extras.TestGraniteChainConfig, + now: now, + expectedErr: ErrGraniteClockBehindParent, + }, + { + name: "current_timesec_equals_parent_time_with_different_milliseconds_granite", + parent: generateHeader(nowSeconds, utils.NewUint64(nowMillis-1000)), + extraConfig: extras.TestGraniteChainConfig, now: now, + expectedErr: nil, expectedSec: nowSeconds, - expectedMillis: nowMillis + 10, // parent's TimeMilliseconds + expectedMillis: nowMillis, }, } for _, test := range tests { t.Run(test.name, func(t *testing.T) { - sec, millis := GetNextTimestamp(test.parent, test.now) + if test.extraConfig == nil { + test.extraConfig = extras.TestChainConfig + } + sec, millis, err := GetNextTimestamp(test.extraConfig, test.parent, test.now) + require.ErrorIs(t, err, test.expectedErr) require.Equal(t, test.expectedSec, sec) require.Equal(t, test.expectedMillis, millis) }) } } + +func generateHeader(timeSeconds uint64, timeMilliseconds *uint64) *types.Header { + return customtypes.WithHeaderExtra( + &types.Header{ + Time: timeSeconds, + }, + &customtypes.HeaderExtra{ + TimeMilliseconds: timeMilliseconds, + }, + ) +} + +func generateHeaderWithMinDelayExcessAndTime(timeSeconds uint64, timeMilliseconds *uint64, minDelayExcess *uint64) *types.Header { + return customtypes.WithHeaderExtra( + &types.Header{ + Time: timeSeconds, + }, + &customtypes.HeaderExtra{ + TimeMilliseconds: timeMilliseconds, + MinDelayExcess: (*acp226.DelayExcess)(minDelayExcess), + }, + ) +}