diff --git a/RELEASES.md b/RELEASES.md index 45bc5a4ba1..8fd3b0dea9 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 the block builder to ACP-226 requirements. - 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..1513d4907f 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, err := b.calculateBlockBuildingDelay( + lastBuildTime, + lastBuildParentHash, + currentHeader, + ) + if err != nil { + return 0, err + } + if timeUntilNextBuild == 0 { + 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 } + +// currentMinBlockBuildingDelays 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) currentMinBlockBuildingDelays(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, error) { + initialDelay, retryDelay, err := b.currentMinBlockBuildingDelays(currentHeader, b.chainConfig) + if err != nil { + return 0, 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 retryDelay > timeSinceLastBuildTime { + remainingMinDelay = retryDelay - timeSinceLastBuildTime + } + + if initialDelay > 0 { + remainingMinDelay = max(initialDelay, remainingMinDelay) + } else if !isRetry || remainingMinDelay == 0 { + return 0, nil // Build immediately + } + + return remainingMinDelay, 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..33f11dd155 --- /dev/null +++ b/plugin/evm/block_builder_test.go @@ -0,0 +1,171 @@ +// 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 + }{ + { + 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}, + expectedTimeToWait: 0, + }, + { + 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}, + expectedTimeToWait: 0, + }, + { + 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, + }, + { + 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, + }, + { + 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, + }, + { + 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 + }, + { + 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 + }, + { + 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, + }, + { + 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, + }, + { + 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, + }, + { + 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, + }, + } + + for _, tt := range tests { + t.Run(tt.name, func(t *testing.T) { + b := &blockBuilder{ + clock: clock, + chainConfig: tt.config, + } + + timeToWait, err := b.calculateBlockBuildingDelay( + tt.lastBuildTime, + tt.lastBuildParentHash, + tt.currentHeader, + ) + + require.NoError(t, err) + require.Equal(t, tt.expectedTimeToWait, timeToWait) + }) + } +} + +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