Skip to content
Merged
Show file tree
Hide file tree
Changes from 32 commits
Commits
Show all changes
38 commits
Select commit Hold shift + click to select a range
ef12dfb
add is retry logic
ceyonur Sep 29, 2025
ef76b58
Merge branch 'master' into ceyonur/wait-for-event-refactor
ceyonur Sep 29, 2025
cf66f59
update waitForEvent by using parent hash + last build time
ceyonur Sep 30, 2025
9c0d64c
Merge branch 'ceyonur/wait-for-event-refactor' of github.com:ava-labs…
ceyonur Sep 30, 2025
da7a8cb
rename var
ceyonur Sep 30, 2025
5402155
pass current header to waitForEvent
ceyonur Sep 30, 2025
a35d6fb
add delay tests
ceyonur Sep 30, 2025
4f5b78b
add tests
ceyonur Sep 30, 2025
27da7a6
Merge branch 'master' into ceyonur/wait-for-event-refactor
ceyonur Sep 30, 2025
3c2d944
ensure mempool is empty
ceyonur Sep 30, 2025
67c62cd
Merge branch 'ceyonur/wait-for-event-refactor' of github.com:ava-labs…
ceyonur Sep 30, 2025
eef3650
comment
ceyonur Sep 30, 2025
49d1a78
remove reduntant and flaky test
ceyonur Sep 30, 2025
e998652
use timeout for no-delay return test
ceyonur Oct 1, 2025
ac83a0c
Merge branch 'master' into ceyonur/wait-for-event-refactor
ceyonur Oct 1, 2025
10e3bc0
add acp-226 constraints to builder
ceyonur Oct 2, 2025
6d05981
nit comments
ceyonur Oct 2, 2025
3385ef1
refactor GetNextTimestamp and add test for same sec different ms
ceyonur Oct 2, 2025
a2a6946
Merge branch 'master' into ceyonur/acp-226-min-block-delay-builder
ceyonur Oct 3, 2025
76111f5
nits
ceyonur Oct 3, 2025
049aefa
nits
ceyonur Oct 3, 2025
ce38349
revert worker changes
ceyonur Oct 3, 2025
aa9a2cc
apply min blok delay to block builder
ceyonur Oct 3, 2025
29aee71
Merge branch 'master' into ceyonur/acp-226-min-block-delay-builder
ceyonur Oct 3, 2025
1503d87
Update RELEASES.md
ceyonur Oct 4, 2025
cf2697e
Merge branch 'ceyonur/acp-226-min-block-delay-builder' of github.com:…
ceyonur Oct 4, 2025
d96886b
comments
ceyonur Oct 4, 2025
8e8205b
use a single retry delay
ceyonur Oct 6, 2025
42be3f3
simplify var
ceyonur Oct 6, 2025
c447907
check for negative as well
ceyonur Oct 6, 2025
d55eba3
refactor block delay calculations
ceyonur Oct 7, 2025
68c33a4
Merge branch 'master' into ceyonur/acp-226-min-block-delay-builder
ceyonur Oct 7, 2025
50a0628
use excess from parent directly
ceyonur Oct 7, 2025
5a22df5
add comment
ceyonur Oct 7, 2025
29c102a
Merge branch 'ceyonur/acp-226-min-block-delay-builder' of github.com:…
ceyonur Oct 7, 2025
d5b5e6d
linter
ceyonur Oct 7, 2025
49a28d8
Update plugin/evm/block_builder.go
ceyonur Oct 8, 2025
e722c53
Merge branch 'master' into ceyonur/acp-226-min-block-delay-builder
ceyonur Oct 8, 2025
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
8 changes: 5 additions & 3 deletions RELEASES.md
Original file line number Diff line number Diff line change
Expand Up @@ -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)
Expand Down
90 changes: 73 additions & 17 deletions plugin/evm/block_builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand All @@ -18,17 +19,25 @@ 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 attempting/build a block before attempting to build another block
// This is only applied for retrying to build a block after a initial minimum delay has passed.
// The initial minimum delay is applied according to parent minDelayExcess (if available)
RetryDelay = 100 * time.Millisecond
)

type blockBuilder struct {
ctx *snow.Context
clock *mockable.Clock
ctx *snow.Context
chainConfig *extras.ChainConfig

txPool *txpool.TxPool
extraMempool extension.BuilderMempool
Expand All @@ -51,10 +60,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
Expand All @@ -64,7 +75,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
}

Expand Down Expand Up @@ -124,20 +135,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),
)
Expand All @@ -161,3 +171,49 @@ func (b *blockBuilder) waitForNeedToBuild(ctx context.Context) (time.Time, commo
}
return b.lastBuildTime, b.lastBuildParentHash, nil
}

// minNextBlockTime calculates the minimum next block time based on the current header and the chain config.
func (b *blockBuilder) minNextBlockTime(parent *types.Header, config *extras.ChainConfig) (time.Time, error) {
parentExtra := customtypes.GetHeaderExtra(parent)
// If the parent header has no min delay excess, there is nothing to wait for, because the rule does not apply
// to the block to be built.
if parentExtra.MinDelayExcess == nil {
return time.Time{}, nil
}
acp226DelayExcess, err := customheader.MinDelayExcess(config, parent, b.clock.Unix(), nil)
if err != nil {
return time.Time{}, err
}
// This should not happen because parent header is in Granite,
// but just checking nil pointer to be safe.
if acp226DelayExcess == nil {
return time.Time{}, nil
}
parentTime := customtypes.BlockTime(parent)
requiredDelay := time.Duration(acp226DelayExcess.Delay()) * time.Millisecond
return parentTime.Add(requiredDelay), nil
}

// calculateBlockBuildingDelay calculates the delay needed before building the next block.
func (b *blockBuilder) calculateBlockBuildingDelay(
lastBuildTime time.Time,
lastBuildParentHash common.Hash,
currentHeader *types.Header,
) (time.Duration, error) {
var nextBuildTime time.Time
isRetry := lastBuildParentHash == currentHeader.ParentHash && !lastBuildTime.IsZero() // if last build time is zero, this is not a retry
// If this is a retry, we already have waited for the minimum next block time in a previous attempt,
// and only need to wait for the retry delay.
if isRetry {
nextBuildTime = lastBuildTime.Add(RetryDelay)
} else {
// If this is not a retry, we need to wait for the minimum next block time.
var err error
nextBuildTime, err = b.minNextBlockTime(currentHeader, b.chainConfig)
if err != nil {
return 0, err
}
}
remainingDelay := nextBuildTime.Sub(b.clock.Time())
return max(remainingDelay, 0), nil
}
182 changes: 182 additions & 0 deletions plugin/evm/block_builder_test.go
Original file line number Diff line number Diff line change
@@ -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
}{
{
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: RetryDelay,
},
{
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(-RetryDelay), // 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(-RetryDelay / 2), // Less than retry delay ago
lastBuildParentHash: common.Hash{1},
expectedTimeToWait: RetryDelay / 2,
},
{
name: "first_granite_block_build_immediately",
config: extras.TestGraniteChainConfig,
currentHeader: &types.Header{
ParentHash: common.Hash{1},
Time: nowSecUint64,
},
lastBuildTime: now,
lastBuildParentHash: common.Hash{2},
expectedTimeToWait: 0,
},
{
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: RetryDelay,
},
{
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: RetryDelay,
},
{
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(-RetryDelay / 2), // Less than retry delay ago
lastBuildParentHash: common.Hash{1},
expectedTimeToWait: RetryDelay / 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
}
2 changes: 1 addition & 1 deletion plugin/evm/customheader/min_delay_excess.go
Original file line number Diff line number Diff line change
Expand Up @@ -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 (ceyonur): this can be removed after Granite is activated. (See https://github.com/ava-labs/coreth/issues/1318)
if customtypes.GetHeaderExtra(header).MinDelayExcess != nil {
return fmt.Errorf("%w: %s", errRemoteMinDelayExcessSet, header.Hash())
}
Expand Down
8 changes: 8 additions & 0 deletions plugin/evm/customtypes/block_ext.go
Original file line number Diff line number Diff line change
Expand Up @@ -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"
Expand Down Expand Up @@ -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,
Expand Down
Loading
Loading