diff --git a/consensus/dummy/consensus.go b/consensus/dummy/consensus.go index 1022dc873b..8f9f44eaaf 100644 --- a/consensus/dummy/consensus.go +++ b/consensus/dummy/consensus.go @@ -59,6 +59,7 @@ type ( block *types.Block, parent *types.Header, statedb *state.StateDB, + config *params.ChainConfig, ) ( blockFeeContribution *big.Int, extDataGasUsed *big.Int, @@ -371,7 +372,7 @@ func (eng *DummyEngine) Finalize(chain consensus.ChainHeaderReader, block *types err error ) if eng.cb.OnExtraStateChange != nil { - contribution, extDataGasUsed, err = eng.cb.OnExtraStateChange(block, parent, state) + contribution, extDataGasUsed, err = eng.cb.OnExtraStateChange(block, parent, state, chain.Config()) if err != nil { return err } diff --git a/core/blockchain_ext_test.go b/core/blockchain_ext_test.go index f03917d5f5..ed633e8878 100644 --- a/core/blockchain_ext_test.go +++ b/core/blockchain_ext_test.go @@ -22,7 +22,7 @@ import ( ) var TestCallbacks = dummy.ConsensusCallbacks{ - OnExtraStateChange: func(block *types.Block, _ *types.Header, sdb *state.StateDB) (*big.Int, *big.Int, error) { + OnExtraStateChange: func(block *types.Block, _ *types.Header, sdb *state.StateDB, _ *params.ChainConfig) (*big.Int, *big.Int, error) { sdb.AddBalanceMultiCoin(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), big.NewInt(block.Number().Int64())) return nil, nil, nil }, diff --git a/eth/gasprice/gasprice_test.go b/eth/gasprice/gasprice_test.go index 90de7177cd..a30d9b2293 100644 --- a/eth/gasprice/gasprice_test.go +++ b/eth/gasprice/gasprice_test.go @@ -134,7 +134,7 @@ func newTestBackend(t *testing.T, config *params.ChainConfig, numBlocks int, ext OnFinalizeAndAssemble: func(*types.Header, *types.Header, *state.StateDB, []*types.Transaction) ([]byte, *big.Int, *big.Int, error) { return nil, common.Big0, extDataGasUsage, nil }, - OnExtraStateChange: func(*types.Block, *types.Header, *state.StateDB) (*big.Int, *big.Int, error) { + OnExtraStateChange: func(*types.Block, *types.Header, *state.StateDB, *params.ChainConfig) (*big.Int, *big.Int, error) { return common.Big0, extDataGasUsage, nil }, }) diff --git a/metrics/metricstest/metrics.go b/metrics/metricstest/metrics.go index e4f8e7f87c..2bc55ac380 100644 --- a/metrics/metricstest/metrics.go +++ b/metrics/metricstest/metrics.go @@ -6,6 +6,8 @@ package metricstest import ( "testing" + avalanchemetrics "github.com/ava-labs/avalanchego/api/metrics" + "github.com/ava-labs/avalanchego/snow" "github.com/ava-labs/libevm/metrics" ) @@ -21,3 +23,9 @@ func WithMetrics(t *testing.T) { metrics.Enabled = false }) } + +// ResetMetrics resets the vm avalanchego metrics, and allows +// for the VM to be re-initialized in tests. +func ResetMetrics(snowCtx *snow.Context) { + snowCtx.Metrics = avalanchemetrics.NewPrefixGatherer() +} diff --git a/plugin/evm/api.go b/plugin/evm/api.go index d85c2294f2..305c3c4066 100644 --- a/plugin/evm/api.go +++ b/plugin/evm/api.go @@ -5,40 +5,12 @@ package evm import ( "context" - "errors" - "fmt" "math/big" - "net/http" - "github.com/ava-labs/avalanchego/api" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/utils/formatting" - "github.com/ava-labs/avalanchego/utils/json" - "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/coreth/plugin/evm/atomic" - "github.com/ava-labs/coreth/plugin/evm/client" - "github.com/ava-labs/coreth/plugin/evm/upgrade/ap3" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/log" ) -// test constants -const ( - GenesisTestAddr = "0x751a0b96e1042bee789452ecb20253fba40dbe85" - GenesisTestKey = "0xabd71b35d559563fea757f0f5edbde286fb8c043105b15abb7cd57189306d7d1" - - // Max number of addresses that can be passed in as argument to GetUTXOs - maxGetUTXOsAddrs = 1024 -) - -var ( - errNoAddresses = errors.New("no addresses provided") - errNoSourceChain = errors.New("no source chain provided") - errNilTxID = errors.New("nil transaction ID") - - initialBaseFee = big.NewInt(ap3.InitialBaseFee) -) - // SnowmanAPI introduces snowman specific functionality to the evm type SnowmanAPI struct{ vm *VM } @@ -65,202 +37,3 @@ func (api *SnowmanAPI) IssueBlock(ctx context.Context) error { api.vm.builder.signalTxsReady() return nil } - -// AvaxAPI offers Avalanche network related API methods -type AvaxAPI struct{ vm *VM } - -type VersionReply struct { - Version string `json:"version"` -} - -// ClientVersion returns the version of the VM running -func (service *AvaxAPI) Version(r *http.Request, _ *struct{}, reply *VersionReply) error { - reply.Version = Version - return nil -} - -// GetUTXOs gets all utxos for passed in addresses -func (service *AvaxAPI) GetUTXOs(r *http.Request, args *api.GetUTXOsArgs, reply *api.GetUTXOsReply) error { - log.Info("EVM: GetUTXOs called", "Addresses", args.Addresses) - - if len(args.Addresses) == 0 { - return errNoAddresses - } - if len(args.Addresses) > maxGetUTXOsAddrs { - return fmt.Errorf("number of addresses given, %d, exceeds maximum, %d", len(args.Addresses), maxGetUTXOsAddrs) - } - - if args.SourceChain == "" { - return errNoSourceChain - } - - chainID, err := service.vm.ctx.BCLookup.Lookup(args.SourceChain) - if err != nil { - return fmt.Errorf("problem parsing source chainID %q: %w", args.SourceChain, err) - } - sourceChain := chainID - - addrSet := set.Set[ids.ShortID]{} - for _, addrStr := range args.Addresses { - addr, err := service.vm.ParseServiceAddress(addrStr) - if err != nil { - return fmt.Errorf("couldn't parse address %q: %w", addrStr, err) - } - addrSet.Add(addr) - } - - startAddr := ids.ShortEmpty - startUTXO := ids.Empty - if args.StartIndex.Address != "" || args.StartIndex.UTXO != "" { - startAddr, err = service.vm.ParseServiceAddress(args.StartIndex.Address) - if err != nil { - return fmt.Errorf("couldn't parse start index address %q: %w", args.StartIndex.Address, err) - } - startUTXO, err = ids.FromString(args.StartIndex.UTXO) - if err != nil { - return fmt.Errorf("couldn't parse start index utxo: %w", err) - } - } - - service.vm.ctx.Lock.Lock() - defer service.vm.ctx.Lock.Unlock() - - utxos, endAddr, endUTXOID, err := service.vm.GetAtomicUTXOs( - sourceChain, - addrSet, - startAddr, - startUTXO, - int(args.Limit), - ) - if err != nil { - return fmt.Errorf("problem retrieving UTXOs: %w", err) - } - - reply.UTXOs = make([]string, len(utxos)) - for i, utxo := range utxos { - b, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) - if err != nil { - return fmt.Errorf("problem marshalling UTXO: %w", err) - } - str, err := formatting.Encode(args.Encoding, b) - if err != nil { - return fmt.Errorf("problem encoding utxo: %w", err) - } - reply.UTXOs[i] = str - } - - endAddress, err := service.vm.FormatLocalAddress(endAddr) - if err != nil { - return fmt.Errorf("problem formatting address: %w", err) - } - - reply.EndIndex.Address = endAddress - reply.EndIndex.UTXO = endUTXOID.String() - reply.NumFetched = json.Uint64(len(utxos)) - reply.Encoding = args.Encoding - return nil -} - -func (service *AvaxAPI) IssueTx(r *http.Request, args *api.FormattedTx, response *api.JSONTxID) error { - log.Info("EVM: IssueTx called") - - txBytes, err := formatting.Decode(args.Encoding, args.Tx) - if err != nil { - return fmt.Errorf("problem decoding transaction: %w", err) - } - - tx := &atomic.Tx{} - if _, err := atomic.Codec.Unmarshal(txBytes, tx); err != nil { - return fmt.Errorf("problem parsing transaction: %w", err) - } - if err := tx.Sign(atomic.Codec, nil); err != nil { - return fmt.Errorf("problem initializing transaction: %w", err) - } - - response.TxID = tx.ID() - - service.vm.ctx.Lock.Lock() - defer service.vm.ctx.Lock.Unlock() - - if err := service.vm.mempool.AddLocalTx(tx); err != nil { - return err - } - service.vm.atomicTxPushGossiper.Add(tx) - return nil -} - -// GetAtomicTxStatus returns the status of the specified transaction -func (service *AvaxAPI) GetAtomicTxStatus(r *http.Request, args *api.JSONTxID, reply *client.GetAtomicTxStatusReply) error { - log.Info("EVM: GetAtomicTxStatus called", "txID", args.TxID) - - if args.TxID == ids.Empty { - return errNilTxID - } - - service.vm.ctx.Lock.Lock() - defer service.vm.ctx.Lock.Unlock() - - _, status, height, _ := service.vm.getAtomicTx(args.TxID) - - reply.Status = status - if status == atomic.Accepted { - // Since chain state updates run asynchronously with VM block acceptance, - // avoid returning [Accepted] until the chain state reaches the block - // containing the atomic tx. - lastAccepted := service.vm.blockChain.LastAcceptedBlock() - if height > lastAccepted.NumberU64() { - reply.Status = atomic.Processing - return nil - } - - jsonHeight := json.Uint64(height) - reply.BlockHeight = &jsonHeight - } - return nil -} - -type FormattedTx struct { - api.FormattedTx - BlockHeight *json.Uint64 `json:"blockHeight,omitempty"` -} - -// GetAtomicTx returns the specified transaction -func (service *AvaxAPI) GetAtomicTx(r *http.Request, args *api.GetTxArgs, reply *FormattedTx) error { - log.Info("EVM: GetAtomicTx called", "txID", args.TxID) - - if args.TxID == ids.Empty { - return errNilTxID - } - - service.vm.ctx.Lock.Lock() - defer service.vm.ctx.Lock.Unlock() - - tx, status, height, err := service.vm.getAtomicTx(args.TxID) - if err != nil { - return err - } - - if status == atomic.Unknown { - return fmt.Errorf("could not find tx %s", args.TxID) - } - - txBytes, err := formatting.Encode(args.Encoding, tx.SignedBytes()) - if err != nil { - return err - } - reply.Tx = txBytes - reply.Encoding = args.Encoding - if status == atomic.Accepted { - // Since chain state updates run asynchronously with VM block acceptance, - // avoid returning [Accepted] until the chain state reaches the block - // containing the atomic tx. - lastAccepted := service.vm.blockChain.LastAcceptedBlock() - if height > lastAccepted.NumberU64() { - return nil - } - - jsonHeight := json.Uint64(height) - reply.BlockHeight = &jsonHeight - } - return nil -} diff --git a/plugin/evm/atomic/state/atomic_trie.go b/plugin/evm/atomic/state/atomic_trie.go index 2314213df9..dca8965974 100644 --- a/plugin/evm/atomic/state/atomic_trie.go +++ b/plugin/evm/atomic/state/atomic_trie.go @@ -131,8 +131,8 @@ func (a *AtomicTrie) OpenTrie(root common.Hash) (*trie.Trie, error) { return trie.New(trie.TrieID(root), a.trieDB) } -// Commit calls Commit on the underlying trieDB and updates metadata pointers. -func (a *AtomicTrie) Commit(height uint64, root common.Hash) error { +// commit calls commit on the underlying trieDB and updates metadata pointers. +func (a *AtomicTrie) commit(height uint64, root common.Hash) error { if err := a.trieDB.Commit(root, false); err != nil { return err } @@ -265,7 +265,7 @@ func (a *AtomicTrie) AcceptTrie(height uint64, root common.Hash) (bool, error) { // Because we do not accept the trie at every height, we may need to // populate roots at prior commit heights that were skipped. for nextCommitHeight := a.lastCommittedHeight + a.commitInterval; nextCommitHeight < height; nextCommitHeight += a.commitInterval { - if err := a.Commit(nextCommitHeight, a.lastAcceptedRoot); err != nil { + if err := a.commit(nextCommitHeight, a.lastAcceptedRoot); err != nil { return false, err } hasCommitted = true @@ -273,7 +273,7 @@ func (a *AtomicTrie) AcceptTrie(height uint64, root common.Hash) (bool, error) { // Commit this root if we have reached the [commitInterval]. if height%a.commitInterval == 0 { - if err := a.Commit(height, root); err != nil { + if err := a.commit(height, root); err != nil { return false, err } hasCommitted = true diff --git a/plugin/evm/atomic/sync/atomic_leaf_handler.go b/plugin/evm/atomic/sync/atomic_leaf_handler.go new file mode 100644 index 0000000000..327f79924c --- /dev/null +++ b/plugin/evm/atomic/sync/atomic_leaf_handler.go @@ -0,0 +1,44 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package sync + +import ( + "context" + "errors" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/sync/handlers" + "github.com/ava-labs/coreth/sync/handlers/stats" + + "github.com/ava-labs/libevm/metrics" + "github.com/ava-labs/libevm/triedb" +) + +var errUninitialized = errors.New("uninitialized handler") + +type uninitializedHandler struct{} + +func (h *uninitializedHandler) OnLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { + return nil, errUninitialized +} + +// atomicLeafHandler is a wrapper around handlers.LeafRequestHandler that allows for initialization after creation +type atomicLeafHandler struct { + handlers.LeafRequestHandler +} + +// NewAtomicLeafHandler returns a new uninitialzied atomicLeafHandler that can be later initialized +func NewAtomicLeafHandler() *atomicLeafHandler { + return &atomicLeafHandler{ + LeafRequestHandler: &uninitializedHandler{}, + } +} + +// Initialize initializes the atomicLeafHandler with the provided atomicTrieDB, trieKeyLength, and networkCodec +func (a *atomicLeafHandler) Initialize(atomicTrieDB *triedb.Database, trieKeyLength int, networkCodec codec.Manager) { + handlerStats := stats.GetOrRegisterHandlerStats(metrics.Enabled) + a.LeafRequestHandler = handlers.NewLeafsRequestHandler(atomicTrieDB, trieKeyLength, nil, networkCodec, handlerStats) +} diff --git a/plugin/evm/atomic/sync/atomic_summary_provider.go b/plugin/evm/atomic/sync/atomic_summary_provider.go new file mode 100644 index 0000000000..d642d707e6 --- /dev/null +++ b/plugin/evm/atomic/sync/atomic_summary_provider.go @@ -0,0 +1,44 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package sync + +import ( + "fmt" + + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + + "github.com/ava-labs/coreth/plugin/evm/atomic/state" + "github.com/ava-labs/coreth/plugin/evm/sync" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" +) + +var _ sync.SummaryProvider = (*AtomicSummaryProvider)(nil) + +type AtomicSummaryProvider struct { + atomicTrie *state.AtomicTrie +} + +func (a *AtomicSummaryProvider) Initialize(atomicTrie *state.AtomicTrie) { + a.atomicTrie = atomicTrie +} + +// StateSummaryAtBlock returns the block state summary at [blk] if valid. +func (a *AtomicSummaryProvider) StateSummaryAtBlock(blk *types.Block) (block.StateSummary, error) { + height := blk.NumberU64() + atomicRoot, err := a.atomicTrie.Root(height) + if err != nil { + return nil, fmt.Errorf("failed to retrieve atomic trie root for height (%d): %w", height, err) + } + + if atomicRoot == (common.Hash{}) { + return nil, fmt.Errorf("atomic trie root not found for height (%d)", height) + } + + summary, err := NewAtomicSyncSummary(blk.Hash(), height, blk.Root(), atomicRoot) + if err != nil { + return nil, fmt.Errorf("failed to construct syncable block at height %d: %w", height, err) + } + return summary, nil +} diff --git a/plugin/evm/atomic/sync/atomic_sync_extender.go b/plugin/evm/atomic/sync/atomic_sync_extender.go new file mode 100644 index 0000000000..ee9a6b89dd --- /dev/null +++ b/plugin/evm/atomic/sync/atomic_sync_extender.go @@ -0,0 +1,81 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package sync + +import ( + "context" + "fmt" + + "github.com/ava-labs/avalanchego/database/versiondb" + syncclient "github.com/ava-labs/coreth/sync/client" + + "github.com/ava-labs/coreth/plugin/evm/atomic/state" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/sync" + + "github.com/ava-labs/libevm/log" +) + +var _ sync.Extender = (*AtomicSyncExtender)(nil) + +type AtomicSyncExtender struct { + backend *state.AtomicBackend + atomicTrie *state.AtomicTrie + stateSyncRequestSize uint16 +} + +// Initialize initializes the atomic sync extender with the atomic backend and atomic trie. +func (a *AtomicSyncExtender) Initialize(backend *state.AtomicBackend, atomicTrie *state.AtomicTrie, stateSyncRequestSize uint16) { + a.backend = backend + a.atomicTrie = atomicTrie + a.stateSyncRequestSize = stateSyncRequestSize +} + +func (a *AtomicSyncExtender) Sync(ctx context.Context, client syncclient.LeafClient, verDB *versiondb.Database, syncSummary message.Syncable) error { + atomicSyncSummary, ok := syncSummary.(*AtomicSyncSummary) + if !ok { + return fmt.Errorf("expected *AtomicBlockSyncSummary, got %T", syncSummary) + } + log.Info("atomic tx: sync starting", "root", atomicSyncSummary) + atomicSyncer, err := newAtomicSyncer( + client, + verDB, + a.atomicTrie, + atomicSyncSummary.AtomicRoot, + atomicSyncSummary.BlockNumber, + a.stateSyncRequestSize, + ) + if err != nil { + return fmt.Errorf("failed to create atomic syncer: %w", err) + } + if err := atomicSyncer.Start(ctx); err != nil { + return fmt.Errorf("failed to start atomic syncer: %w", err) + } + err = <-atomicSyncer.Done() + log.Info("atomic tx: sync finished", "root", atomicSyncSummary.AtomicRoot, "err", err) + return err +} + +func (a *AtomicSyncExtender) OnFinishBeforeCommit(lastAcceptedHeight uint64, syncSummary message.Syncable) error { + // Mark the previously last accepted block for the shared memory cursor, so that we will execute shared + // memory operations from the previously last accepted block when ApplyToSharedMemory + // is called. + if err := a.backend.MarkApplyToSharedMemoryCursor(lastAcceptedHeight); err != nil { + return fmt.Errorf("failed to mark apply to shared memory cursor before commit: %w", err) + } + a.backend.SetLastAccepted(syncSummary.GetBlockHash()) + return nil +} + +func (a *AtomicSyncExtender) OnFinishAfterCommit(summaryHeight uint64) error { + // the chain state is already restored, and, from this point on, + // the block synced to is the accepted block. The last operation + // is updating shared memory with the atomic trie. + // ApplyToSharedMemory does this, and, even if the VM is stopped + // (gracefully or ungracefully), since MarkApplyToSharedMemoryCursor + // is called, VM will resume ApplyToSharedMemory on Initialize. + if err := a.backend.ApplyToSharedMemory(summaryHeight); err != nil { + return fmt.Errorf("failed to apply atomic trie to shared memory after commit: %w", err) + } + return nil +} diff --git a/plugin/evm/atomic/sync/atomic_sync_summary.go b/plugin/evm/atomic/sync/atomic_sync_summary.go new file mode 100644 index 0000000000..0e4fa6712c --- /dev/null +++ b/plugin/evm/atomic/sync/atomic_sync_summary.go @@ -0,0 +1,126 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package sync + +import ( + "context" + "fmt" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/coreth/plugin/evm/message" + + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/crypto" +) + +var ( + _ message.Syncable = (*AtomicSyncSummary)(nil) + _ message.SyncableParser = (*AtomicSyncSummaryParser)(nil) +) + +// codecWithAtomicSync is the codec manager that contains the codec for AtomicBlockSyncSummary and +// other message types that are used in the network protocol. This is to ensure that the codec +// version is consistent across all message types and includes the codec for AtomicBlockSyncSummary. +var codecWithAtomicSync codec.Manager + +func init() { + var err error + codecWithAtomicSync, err = message.NewCodec(AtomicSyncSummary{}) + if err != nil { + panic(fmt.Errorf("failed to create codec manager: %w", err)) + } +} + +// AtomicSyncSummary provides the information necessary to sync a node starting +// at the given block. +type AtomicSyncSummary struct { + BlockNumber uint64 `serialize:"true"` + BlockHash common.Hash `serialize:"true"` + BlockRoot common.Hash `serialize:"true"` + AtomicRoot common.Hash `serialize:"true"` + + summaryID ids.ID + bytes []byte + acceptImpl message.AcceptImplFn +} + +type AtomicSyncSummaryParser struct{} + +func NewAtomicSyncSummaryParser() *AtomicSyncSummaryParser { + return &AtomicSyncSummaryParser{} +} + +func (a *AtomicSyncSummaryParser) ParseFromBytes(summaryBytes []byte, acceptImpl message.AcceptImplFn) (message.Syncable, error) { + summary := AtomicSyncSummary{} + if codecVersion, err := codecWithAtomicSync.Unmarshal(summaryBytes, &summary); err != nil { + return nil, fmt.Errorf("failed to parse syncable summary: %w", err) + } else if codecVersion != message.Version { + return nil, fmt.Errorf("failed to parse syncable summary due to unexpected codec version (got %d, expected %d)", codecVersion, message.Version) + } + + summary.bytes = summaryBytes + summaryID, err := ids.ToID(crypto.Keccak256(summaryBytes)) + if err != nil { + return nil, fmt.Errorf("failed to compute summary ID: %w", err) + } + summary.summaryID = summaryID + summary.acceptImpl = acceptImpl + return &summary, nil +} + +func NewAtomicSyncSummary(blockHash common.Hash, blockNumber uint64, blockRoot common.Hash, atomicRoot common.Hash) (*AtomicSyncSummary, error) { + summary := AtomicSyncSummary{ + BlockNumber: blockNumber, + BlockHash: blockHash, + BlockRoot: blockRoot, + AtomicRoot: atomicRoot, + } + bytes, err := codecWithAtomicSync.Marshal(message.Version, &summary) + if err != nil { + return nil, fmt.Errorf("failed to marshal syncable summary: %w", err) + } + + summary.bytes = bytes + summaryID, err := ids.ToID(crypto.Keccak256(bytes)) + if err != nil { + return nil, fmt.Errorf("failed to compute summary ID: %w", err) + } + summary.summaryID = summaryID + + return &summary, nil +} + +func (a *AtomicSyncSummary) GetBlockHash() common.Hash { + return a.BlockHash +} + +func (a *AtomicSyncSummary) GetBlockRoot() common.Hash { + return a.BlockRoot +} + +func (a *AtomicSyncSummary) Bytes() []byte { + return a.bytes +} + +func (a *AtomicSyncSummary) Height() uint64 { + return a.BlockNumber +} + +func (a *AtomicSyncSummary) ID() ids.ID { + return a.summaryID +} + +func (a *AtomicSyncSummary) String() string { + return fmt.Sprintf("AtomicBlockSyncSummary(BlockHash=%s, BlockNumber=%d, BlockRoot=%s, AtomicRoot=%s)", a.BlockHash, a.BlockNumber, a.BlockRoot, a.AtomicRoot) +} + +func (a *AtomicSyncSummary) Accept(context.Context) (block.StateSyncMode, error) { + if a.acceptImpl == nil { + return block.StateSyncSkipped, fmt.Errorf("accept implementation not specified for summary: %s", a) + } + return a.acceptImpl(a) +} diff --git a/plugin/evm/atomic/sync/atomic_sync_summary_test.go b/plugin/evm/atomic/sync/atomic_sync_summary_test.go new file mode 100644 index 0000000000..b1e999de2c --- /dev/null +++ b/plugin/evm/atomic/sync/atomic_sync_summary_test.go @@ -0,0 +1,46 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package sync + +import ( + "context" + "encoding/base64" + "testing" + + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/libevm/common" + "github.com/stretchr/testify/require" +) + +func TestMarshalAtomicSyncSummary(t *testing.T) { + atomicSyncSummary, err := NewAtomicSyncSummary(common.Hash{1}, 2, common.Hash{3}, common.Hash{4}) + require.NoError(t, err) + + require.Equal(t, common.Hash{1}, atomicSyncSummary.GetBlockHash()) + require.Equal(t, uint64(2), atomicSyncSummary.Height()) + require.Equal(t, common.Hash{3}, atomicSyncSummary.GetBlockRoot()) + + expectedBase64Bytes := "AAAAAAAAAAAAAgEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" + require.Equal(t, expectedBase64Bytes, base64.StdEncoding.EncodeToString(atomicSyncSummary.Bytes())) + + parser := NewAtomicSyncSummaryParser() + called := false + acceptImplTest := func(message.Syncable) (block.StateSyncMode, error) { + called = true + return block.StateSyncSkipped, nil + } + s, err := parser.ParseFromBytes(atomicSyncSummary.Bytes(), acceptImplTest) + require.NoError(t, err) + require.Equal(t, atomicSyncSummary.GetBlockHash(), s.GetBlockHash()) + require.Equal(t, atomicSyncSummary.Height(), s.Height()) + require.Equal(t, atomicSyncSummary.GetBlockRoot(), s.GetBlockRoot()) + require.Equal(t, atomicSyncSummary.AtomicRoot, s.(*AtomicSyncSummary).AtomicRoot) + require.Equal(t, atomicSyncSummary.Bytes(), s.Bytes()) + + mode, err := s.Accept(context.TODO()) + require.NoError(t, err) + require.Equal(t, block.StateSyncSkipped, mode) + require.True(t, called) +} diff --git a/plugin/evm/atomic_syncer.go b/plugin/evm/atomic/sync/atomic_syncer.go similarity index 93% rename from plugin/evm/atomic_syncer.go rename to plugin/evm/atomic/sync/atomic_syncer.go index 9a2ec1aa63..fef9b76eb4 100644 --- a/plugin/evm/atomic_syncer.go +++ b/plugin/evm/atomic/sync/atomic_syncer.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package sync import ( "bytes" @@ -17,14 +17,29 @@ import ( atomicstate "github.com/ava-labs/coreth/plugin/evm/atomic/state" "github.com/ava-labs/coreth/plugin/evm/message" syncclient "github.com/ava-labs/coreth/sync/client" + "github.com/ava-labs/libevm/trie" ) +const ( + // AtomicTrieNode represents a leaf node that belongs to the atomic trie. + AtomicTrieNode message.NodeType = 2 +) + var ( _ Syncer = (*atomicSyncer)(nil) _ syncclient.LeafSyncTask = (*atomicSyncerLeafTask)(nil) ) +// Syncer represents a step in state sync, +// along with Start/Done methods to control +// and monitor progress. +// Error returns an error if any was encountered. +type Syncer interface { + Start(ctx context.Context) error + Done() <-chan error +} + // atomicSyncer is used to sync the atomic trie from the network. The CallbackLeafSyncer // is responsible for orchestrating the sync while atomicSyncer is responsible for maintaining // the state of progress and writing the actual atomic trie to the trieDB. @@ -167,7 +182,7 @@ type atomicSyncerLeafTask struct { func (a *atomicSyncerLeafTask) Start() []byte { return addZeroes(a.atomicSyncer.lastHeight + 1) } func (a *atomicSyncerLeafTask) End() []byte { return nil } -func (a *atomicSyncerLeafTask) NodeType() message.NodeType { return message.AtomicTrieNode } +func (a *atomicSyncerLeafTask) NodeType() message.NodeType { return AtomicTrieNode } func (a *atomicSyncerLeafTask) OnFinish(context.Context) error { return a.atomicSyncer.onFinish() } func (a *atomicSyncerLeafTask) OnStart() (bool, error) { return false, nil } func (a *atomicSyncerLeafTask) Root() common.Hash { return a.atomicSyncer.targetRoot } diff --git a/plugin/evm/atomic_syncer_test.go b/plugin/evm/atomic/sync/atomic_syncer_test.go similarity index 96% rename from plugin/evm/atomic_syncer_test.go rename to plugin/evm/atomic/sync/atomic_syncer_test.go index ed03854579..9503513427 100644 --- a/plugin/evm/atomic_syncer_test.go +++ b/plugin/evm/atomic/sync/atomic_syncer_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package sync import ( "bytes" @@ -17,6 +17,7 @@ import ( "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/coreth/plugin/evm/atomic" "github.com/ava-labs/coreth/plugin/evm/atomic/atomictest" atomicstate "github.com/ava-labs/coreth/plugin/evm/atomic/state" "github.com/ava-labs/coreth/plugin/evm/config" @@ -48,14 +49,14 @@ func testAtomicSyncer(t *testing.T, serverTrieDB *triedb.Database, targetHeight numLeaves := 0 mockClient := syncclient.NewTestClient( - message.Codec, - handlers.NewLeafsRequestHandler(serverTrieDB, nil, message.Codec, handlerstats.NewNoopHandlerStats()), + atomic.Codec, + handlers.NewLeafsRequestHandler(serverTrieDB, atomicstate.TrieKeyLength, nil, codecWithAtomicSync, handlerstats.NewNoopHandlerStats()), nil, nil, ) clientDB := versiondb.New(memdb.New()) - repo, err := atomicstate.NewAtomicTxRepository(clientDB, message.Codec, 0) + repo, err := atomicstate.NewAtomicTxRepository(clientDB, atomic.Codec, 0) if err != nil { t.Fatal("could not initialize atomix tx repository", err) } diff --git a/plugin/evm/atomic/txpool/mempool.go b/plugin/evm/atomic/txpool/mempool.go index bff2725f77..f54fcf8f8f 100644 --- a/plugin/evm/atomic/txpool/mempool.go +++ b/plugin/evm/atomic/txpool/mempool.go @@ -85,30 +85,34 @@ type Mempool struct { verify func(tx *atomic.Tx) error } -// NewMempool returns a Mempool with [maxSize] -func NewMempool(ctx *snow.Context, registerer prometheus.Registerer, maxSize int, verify func(tx *atomic.Tx) error) (*Mempool, error) { +// Initialize initializes the Mempool with `maxSize` +func (m *Mempool) Initialize(ctx *snow.Context, registerer prometheus.Registerer, maxSize int, verify func(tx *atomic.Tx) error) error { bloom, err := gossip.NewBloomFilter(registerer, "atomic_mempool_bloom_filter", config.TxGossipBloomMinTargetElements, config.TxGossipBloomTargetFalsePositiveRate, config.TxGossipBloomResetFalsePositiveRate, ) if err != nil { - return nil, fmt.Errorf("failed to initialize bloom filter: %w", err) - } - - return &Mempool{ - ctx: ctx, - issuedTxs: make(map[ids.ID]*atomic.Tx), - discardedTxs: lru.NewCache[ids.ID, *atomic.Tx](discardedTxsCacheSize), - currentTxs: make(map[ids.ID]*atomic.Tx), - pending: make(chan struct{}, 1), - txHeap: newTxHeap(maxSize), - maxSize: maxSize, - utxoSpenders: make(map[ids.ID]*atomic.Tx), - bloom: bloom, - metrics: newMempoolMetrics(), - verify: verify, - }, nil + return fmt.Errorf("failed to initialize bloom filter: %w", err) + } + + m.ctx = ctx + m.issuedTxs = make(map[ids.ID]*atomic.Tx) + m.discardedTxs = lru.NewCache[ids.ID, *atomic.Tx](discardedTxsCacheSize) + m.currentTxs = make(map[ids.ID]*atomic.Tx) + m.pending = make(chan struct{}, 1) + m.txHeap = newTxHeap(maxSize) + m.maxSize = maxSize + m.utxoSpenders = make(map[ids.ID]*atomic.Tx) + m.bloom = bloom + m.metrics = newMempoolMetrics() + m.verify = verify + return nil +} + +// PendingLen returns the number of pending transactions in the mempool +func (m *Mempool) PendingLen() int { + return m.Len() } // Len returns the number of transactions in the mempool diff --git a/plugin/evm/atomic/txpool/mempool_test.go b/plugin/evm/atomic/txpool/mempool_test.go index 511d2b6db2..e487e6b602 100644 --- a/plugin/evm/atomic/txpool/mempool_test.go +++ b/plugin/evm/atomic/txpool/mempool_test.go @@ -16,9 +16,9 @@ import ( func TestMempoolAddTx(t *testing.T) { require := require.New(t) + m := &Mempool{} ctx := snowtest.Context(t, snowtest.CChainID) - m, err := NewMempool(ctx, prometheus.NewRegistry(), 5_000, nil) - require.NoError(err) + require.NoError(m.Initialize(ctx, prometheus.NewRegistry(), 5_000, nil)) txs := make([]*atomic.Tx, 0) for i := 0; i < 3_000; i++ { @@ -40,9 +40,9 @@ func TestMempoolAddTx(t *testing.T) { // Add should return an error if a tx is already known func TestMempoolAdd(t *testing.T) { require := require.New(t) + m := &Mempool{} ctx := snowtest.Context(t, snowtest.CChainID) - m, err := NewMempool(ctx, prometheus.NewRegistry(), 5_000, nil) - require.NoError(err) + require.NoError(m.Initialize(ctx, prometheus.NewRegistry(), 5_000, nil)) tx := &atomic.Tx{ UnsignedAtomicTx: &atomictest.TestUnsignedTx{ @@ -51,7 +51,7 @@ func TestMempoolAdd(t *testing.T) { } require.NoError(m.Add(tx)) - err = m.Add(tx) + err := m.Add(tx) require.ErrorIs(err, errTxAlreadyKnown) } @@ -105,8 +105,8 @@ func TestAtomicMempoolIterate(t *testing.T) { t.Run(tt.name, func(t *testing.T) { require := require.New(t) ctx := snowtest.Context(t, snowtest.CChainID) - m, err := NewMempool(ctx, prometheus.NewRegistry(), 10, nil) - require.NoError(err) + m := &Mempool{} + require.NoError(m.Initialize(ctx, prometheus.NewRegistry(), 10, nil)) for _, add := range tt.add { require.NoError(m.Add(add)) @@ -136,8 +136,8 @@ func TestMempoolMaxSizeHandling(t *testing.T) { require := require.New(t) ctx := snowtest.Context(t, snowtest.CChainID) - mempool, err := NewMempool(ctx, prometheus.NewRegistry(), 1, nil) - require.NoError(err) + mempool := &Mempool{} + require.NoError(mempool.Initialize(ctx, prometheus.NewRegistry(), 5_000, nil)) // create candidate tx (we will drop before validation) tx := atomictest.GenerateTestImportTx() @@ -150,7 +150,7 @@ func TestMempoolMaxSizeHandling(t *testing.T) { // try to add one more tx tx2 := atomictest.GenerateTestImportTx() - err = mempool.AddRemoteTx(tx2) + err := mempool.AddRemoteTx(tx2) require.ErrorIs(err, ErrTooManyAtomicTx) require.False(mempool.Has(tx2.ID())) } @@ -160,15 +160,15 @@ func TestMempoolPriorityDrop(t *testing.T) { require := require.New(t) ctx := snowtest.Context(t, snowtest.CChainID) - mempool, err := NewMempool(ctx, prometheus.NewRegistry(), 1, nil) - require.NoError(err) + mempool := &Mempool{} + require.NoError(mempool.Initialize(ctx, prometheus.NewRegistry(), 5_000, nil)) tx1 := atomictest.GenerateTestImportTxWithGas(1, 2) // lower fee require.NoError(mempool.AddRemoteTx(tx1)) require.True(mempool.Has(tx1.ID())) tx2 := atomictest.GenerateTestImportTxWithGas(1, 2) // lower fee - err = mempool.AddRemoteTx(tx2) + err := mempool.AddRemoteTx(tx2) require.ErrorIs(err, ErrInsufficientAtomicTxFee) require.True(mempool.Has(tx1.ID())) require.False(mempool.Has(tx2.ID())) diff --git a/plugin/evm/atomic/vm/api.go b/plugin/evm/atomic/vm/api.go new file mode 100644 index 0000000000..1a51c1de37 --- /dev/null +++ b/plugin/evm/atomic/vm/api.go @@ -0,0 +1,234 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package vm + +import ( + "context" + "errors" + "fmt" + "net/http" + + "github.com/ava-labs/avalanchego/api" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/formatting" + "github.com/ava-labs/avalanchego/utils/json" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/client" + + "github.com/ava-labs/libevm/log" +) + +const ( + // Max number of addresses that can be passed in as argument to GetUTXOs + maxGetUTXOsAddrs = 1024 +) + +var ( + errNoAddresses = errors.New("no addresses provided") + errNoSourceChain = errors.New("no source chain provided") + errNilTxID = errors.New("nil transaction ID") +) + +// AvaxAPI offers Avalanche network related API methods +type AvaxAPI struct{ vm *VM } + +type VersionReply struct { + Version string `json:"version"` +} + +// ClientVersion returns the version of the VM running +func (service *AvaxAPI) Version(r *http.Request, _ *struct{}, reply *VersionReply) error { + version, err := service.vm.Version(context.Background()) + if err != nil { + return err + } + reply.Version = version + return nil +} + +// GetUTXOs gets all utxos for passed in addresses +func (service *AvaxAPI) GetUTXOs(r *http.Request, args *api.GetUTXOsArgs, reply *api.GetUTXOsReply) error { + log.Info("EVM: GetUTXOs called", "Addresses", args.Addresses) + + if len(args.Addresses) == 0 { + return errNoAddresses + } + if len(args.Addresses) > maxGetUTXOsAddrs { + return fmt.Errorf("number of addresses given, %d, exceeds maximum, %d", len(args.Addresses), maxGetUTXOsAddrs) + } + + if args.SourceChain == "" { + return errNoSourceChain + } + + chainID, err := service.vm.ctx.BCLookup.Lookup(args.SourceChain) + if err != nil { + return fmt.Errorf("problem parsing source chainID %q: %w", args.SourceChain, err) + } + sourceChain := chainID + + addrSet := set.Set[ids.ShortID]{} + for _, addrStr := range args.Addresses { + addr, err := ParseServiceAddress(service.vm.ctx, addrStr) + if err != nil { + return fmt.Errorf("couldn't parse address %q: %w", addrStr, err) + } + addrSet.Add(addr) + } + + startAddr := ids.ShortEmpty + startUTXO := ids.Empty + if args.StartIndex.Address != "" || args.StartIndex.UTXO != "" { + startAddr, err = ParseServiceAddress(service.vm.ctx, args.StartIndex.Address) + if err != nil { + return fmt.Errorf("couldn't parse start index address %q: %w", args.StartIndex.Address, err) + } + startUTXO, err = ids.FromString(args.StartIndex.UTXO) + if err != nil { + return fmt.Errorf("couldn't parse start index utxo: %w", err) + } + } + + service.vm.ctx.Lock.Lock() + defer service.vm.ctx.Lock.Unlock() + + utxos, endAddr, endUTXOID, err := service.vm.GetAtomicUTXOs( + sourceChain, + addrSet, + startAddr, + startUTXO, + int(args.Limit), + ) + if err != nil { + return fmt.Errorf("problem retrieving UTXOs: %w", err) + } + + reply.UTXOs = make([]string, len(utxos)) + for i, utxo := range utxos { + b, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + if err != nil { + return fmt.Errorf("problem marshalling UTXO: %w", err) + } + str, err := formatting.Encode(args.Encoding, b) + if err != nil { + return fmt.Errorf("problem encoding utxo: %w", err) + } + reply.UTXOs[i] = str + } + + endAddress, err := FormatLocalAddress(service.vm.ctx, endAddr) + if err != nil { + return fmt.Errorf("problem formatting address: %w", err) + } + + reply.EndIndex.Address = endAddress + reply.EndIndex.UTXO = endUTXOID.String() + reply.NumFetched = json.Uint64(len(utxos)) + reply.Encoding = args.Encoding + return nil +} + +func (service *AvaxAPI) IssueTx(r *http.Request, args *api.FormattedTx, response *api.JSONTxID) error { + log.Info("EVM: IssueTx called") + + txBytes, err := formatting.Decode(args.Encoding, args.Tx) + if err != nil { + return fmt.Errorf("problem decoding transaction: %w", err) + } + + tx := &atomic.Tx{} + if _, err := atomic.Codec.Unmarshal(txBytes, tx); err != nil { + return fmt.Errorf("problem parsing transaction: %w", err) + } + if err := tx.Sign(atomic.Codec, nil); err != nil { + return fmt.Errorf("problem initializing transaction: %w", err) + } + + response.TxID = tx.ID() + + service.vm.ctx.Lock.Lock() + defer service.vm.ctx.Lock.Unlock() + + if err := service.vm.mempool.AddLocalTx(tx); err != nil { + return err + } + service.vm.atomicTxPushGossiper.Add(tx) + return nil +} + +// GetAtomicTxStatus returns the status of the specified transaction +func (service *AvaxAPI) GetAtomicTxStatus(r *http.Request, args *api.JSONTxID, reply *client.GetAtomicTxStatusReply) error { + log.Info("EVM: GetAtomicTxStatus called", "txID", args.TxID) + + if args.TxID == ids.Empty { + return errNilTxID + } + + service.vm.ctx.Lock.Lock() + defer service.vm.ctx.Lock.Unlock() + + _, status, height, _ := service.vm.getAtomicTx(args.TxID) + + reply.Status = status + if status == atomic.Accepted { + // Since chain state updates run asynchronously with VM block acceptance, + // avoid returning [Accepted] until the chain state reaches the block + // containing the atomic tx. + lastAccepted := service.vm.Ethereum().BlockChain().LastAcceptedBlock() + if height > lastAccepted.NumberU64() { + reply.Status = atomic.Processing + return nil + } + + jsonHeight := json.Uint64(height) + reply.BlockHeight = &jsonHeight + } + return nil +} + +type FormattedTx struct { + api.FormattedTx + BlockHeight *json.Uint64 `json:"blockHeight,omitempty"` +} + +// GetAtomicTx returns the specified transaction +func (service *AvaxAPI) GetAtomicTx(r *http.Request, args *api.GetTxArgs, reply *FormattedTx) error { + log.Info("EVM: GetAtomicTx called", "txID", args.TxID) + + if args.TxID == ids.Empty { + return errNilTxID + } + + service.vm.ctx.Lock.Lock() + defer service.vm.ctx.Lock.Unlock() + + tx, status, height, err := service.vm.getAtomicTx(args.TxID) + if err != nil { + return err + } + + if status == atomic.Unknown { + return fmt.Errorf("could not find tx %s", args.TxID) + } + + txBytes, err := formatting.Encode(args.Encoding, tx.SignedBytes()) + if err != nil { + return err + } + reply.Tx = txBytes + reply.Encoding = args.Encoding + if status == atomic.Accepted { + // Since chain state updates run asynchronously with VM block acceptance, + // avoid returning [Accepted] until the chain state reaches the block + // containing the atomic tx. + lastAccepted := service.vm.Ethereum().BlockChain().LastAcceptedBlock() + if height > lastAccepted.NumberU64() { + return nil + } + + jsonHeight := json.Uint64(height) + reply.BlockHeight = &jsonHeight + } + return nil +} diff --git a/plugin/evm/atomic/vm/atomic_leaf_handler.go b/plugin/evm/atomic/vm/atomic_leaf_handler.go new file mode 100644 index 0000000000..e257cbaf33 --- /dev/null +++ b/plugin/evm/atomic/vm/atomic_leaf_handler.go @@ -0,0 +1,44 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "context" + "errors" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/sync/handlers" + "github.com/ava-labs/coreth/sync/handlers/stats" + + "github.com/ava-labs/libevm/metrics" + "github.com/ava-labs/libevm/triedb" +) + +var errUninitialized = errors.New("uninitialized handler") + +type uninitializedHandler struct{} + +func (h *uninitializedHandler) OnLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { + return nil, errUninitialized +} + +// atomicLeafHandler is a wrapper around handlers.LeafRequestHandler that allows for initialization after creation +type atomicLeafHandler struct { + handlers.LeafRequestHandler +} + +// NewAtomicLeafHandler returns a new uninitialzied atomicLeafHandler that can be later initialized +func NewAtomicLeafHandler() *atomicLeafHandler { + return &atomicLeafHandler{ + LeafRequestHandler: &uninitializedHandler{}, + } +} + +// Initialize initializes the atomicLeafHandler with the provided atomicTrieDB, trieKeyLength, and networkCodec +func (a *atomicLeafHandler) Initialize(atomicTrieDB *triedb.Database, trieKeyLength int, networkCodec codec.Manager) { + handlerStats := stats.GetOrRegisterHandlerStats(metrics.Enabled) + a.LeafRequestHandler = handlers.NewLeafsRequestHandler(atomicTrieDB, trieKeyLength, nil, networkCodec, handlerStats) +} diff --git a/plugin/evm/atomic/vm/block_extension.go b/plugin/evm/atomic/vm/block_extension.go new file mode 100644 index 0000000000..95e45c5b7e --- /dev/null +++ b/plugin/evm/atomic/vm/block_extension.go @@ -0,0 +1,261 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "errors" + "fmt" + + "github.com/ava-labs/avalanchego/database" + safemath "github.com/ava-labs/avalanchego/utils/math" + + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/customtypes" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap5" + "github.com/ava-labs/coreth/utils" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/log" +) + +var ( + _ extension.BlockExtension = (*blockExtension)(nil) + _ extension.BlockExtender = (*blockExtender)(nil) +) + +var ( + errNilEthBlock = errors.New("nil ethBlock") + errMissingUTXOs = errors.New("missing UTXOs") + errEmptyBlock = errors.New("empty block") +) + +type blockExtender struct { + extDataHashes map[common.Hash]common.Hash + vm *VM +} + +type blockExtension struct { + atomicTxs []*atomic.Tx + blockExtender *blockExtender + block extension.VMBlock +} + +// newBlockExtender returns a new block extender. +func newBlockExtender( + extDataHashes map[common.Hash]common.Hash, + vm *VM, +) *blockExtender { + return &blockExtender{ + extDataHashes: extDataHashes, + // Note: we need VM here to access the atomic backend that + // could be initialized later in the VM. + vm: vm, + } +} + +// NewBlockExtension returns a new block extension. +func (be *blockExtender) NewBlockExtension(b extension.VMBlock) (extension.BlockExtension, error) { + ethBlock := b.GetEthBlock() + if ethBlock == nil { + return nil, errNilEthBlock + } + // Extract atomic transactions from the block + isApricotPhase5 := be.vm.chainConfigExtra().IsApricotPhase5(ethBlock.Time()) + atomicTxs, err := atomic.ExtractAtomicTxs(customtypes.BlockExtData(ethBlock), isApricotPhase5, atomic.Codec) + if err != nil { + return nil, err + } + + return &blockExtension{ + atomicTxs: atomicTxs, + blockExtender: be, + block: b, + }, nil +} + +// SyntacticVerify checks the syntactic validity of the block. This is called by the wrapper +// block manager's SyntacticVerify method. +func (be *blockExtension) SyntacticVerify(rules params.Rules) error { + b := be.block + ethBlock := b.GetEthBlock() + blockExtender := be.blockExtender + // should not happen + if ethBlock == nil { + return errNilEthBlock + } + ethHeader := ethBlock.Header() + blockHash := ethBlock.Hash() + headerExtra := customtypes.GetHeaderExtra(ethHeader) + + rulesExtra := params.GetRulesExtra(rules) + if !rulesExtra.IsApricotPhase1 { + if blockExtender.extDataHashes != nil { + extData := customtypes.BlockExtData(ethBlock) + extDataHash := customtypes.CalcExtDataHash(extData) + // If there is no extra data, check that there is no extra data in the hash map either to ensure we do not + // have a block that is unexpectedly missing extra data. + expectedExtDataHash, ok := blockExtender.extDataHashes[blockHash] + if len(extData) == 0 { + if ok { + return fmt.Errorf("found block with unexpected missing extra data (%s, %d), expected extra data hash: %s", blockHash, b.Height(), expectedExtDataHash) + } + } else { + // If there is extra data, check to make sure that the extra data hash matches the expected extra data hash for this + // block + if extDataHash != expectedExtDataHash { + return fmt.Errorf("extra data hash in block (%s, %d): %s, did not match the expected extra data hash: %s", blockHash, b.Height(), extDataHash, expectedExtDataHash) + } + } + } + } + + // Verify the ExtDataHash field + if rulesExtra.IsApricotPhase1 { + extraData := customtypes.BlockExtData(ethBlock) + hash := customtypes.CalcExtDataHash(extraData) + if headerExtra.ExtDataHash != hash { + return fmt.Errorf("extra data hash mismatch: have %x, want %x", headerExtra.ExtDataHash, hash) + } + } else { + if headerExtra.ExtDataHash != (common.Hash{}) { + return fmt.Errorf( + "expected ExtDataHash to be empty but got %x", + headerExtra.ExtDataHash, + ) + } + } + + // Block must not be empty + txs := ethBlock.Transactions() + atomicTxs := be.atomicTxs + if len(txs) == 0 && len(atomicTxs) == 0 { + return errEmptyBlock + } + + // If we are in ApricotPhase4, ensure that ExtDataGasUsed is populated correctly. + if rulesExtra.IsApricotPhase4 { + // After the F upgrade, the extDataGasUsed field is validated by + // [header.VerifyGasUsed]. + if !rulesExtra.IsFortuna && rulesExtra.IsApricotPhase5 { + if !utils.BigLessOrEqualUint64(headerExtra.ExtDataGasUsed, ap5.AtomicGasLimit) { + return fmt.Errorf("too large extDataGasUsed: %d", headerExtra.ExtDataGasUsed) + } + } + var totalGasUsed uint64 + for _, atomicTx := range atomicTxs { + // We perform this check manually here to avoid the overhead of having to + // reparse the atomicTx in `CalcExtDataGasUsed`. + fixedFee := rulesExtra.IsApricotPhase5 // Charge the atomic tx fixed fee as of ApricotPhase5 + gasUsed, err := atomicTx.GasUsed(fixedFee) + if err != nil { + return err + } + totalGasUsed, err = safemath.Add(totalGasUsed, gasUsed) + if err != nil { + return err + } + } + + if !utils.BigEqualUint64(headerExtra.ExtDataGasUsed, totalGasUsed) { + return fmt.Errorf("invalid extDataGasUsed: have %d, want %d", headerExtra.ExtDataGasUsed, totalGasUsed) + } + } + + return nil +} + +// SemanticVerify checks the semantic validity of the block. This is called by the wrapper +// block manager's SemanticVerify method. +func (be *blockExtension) SemanticVerify() error { + vm := be.blockExtender.vm + if vm.IsBootstrapped() { + // Verify that the UTXOs named in import txs are present in shared + // memory. + // + // This does not fully verify that this block can spend these UTXOs. + // However, it guarantees that any block that fails the later checks was + // built by an incorrect block proposer. This ensures that we only mark + // blocks as BAD BLOCKs if they were incorrectly generated. + if err := be.verifyUTXOsPresent(be.atomicTxs); err != nil { + return err + } + } + return nil +} + +// OnAccept is called when the block is accepted. This is called by the wrapper +// block manager's Accept method. The acceptedBatch contains the changes that +// were made to the database as a result of accepting the block, and it's flushed +// to the database in this method. +func (be *blockExtension) OnAccept(acceptedBatch database.Batch) error { + vm := be.blockExtender.vm + for _, tx := range be.atomicTxs { + // Remove the accepted transaction from the mempool + vm.mempool.RemoveTx(tx) + } + + // Update VM state for atomic txs in this block. This includes updating the + // atomic tx repo, atomic trie, and shared memory. + atomicState, err := vm.atomicBackend.GetVerifiedAtomicState(common.Hash(be.block.ID())) + if err != nil { + // should never occur since [b] must be verified before calling Accept + return err + } + // Apply any shared memory changes atomically with other pending batched changes + return atomicState.Accept(acceptedBatch) +} + +// OnReject is called when the block is rejected. This is called by the wrapper +// block manager's Reject method. +func (be *blockExtension) OnReject() error { + vm := be.blockExtender.vm + for _, tx := range be.atomicTxs { + // Re-issue the transaction in the mempool, continue even if it fails + vm.mempool.RemoveTx(tx) + if err := vm.mempool.AddRemoteTx(tx); err != nil { + log.Debug("Failed to re-issue transaction in rejected block", "txID", tx.ID(), "err", err) + } + } + atomicState, err := vm.atomicBackend.GetVerifiedAtomicState(common.Hash(be.block.ID())) + if err != nil { + // should never occur since [b] must be verified before calling Reject + return err + } + return atomicState.Reject() +} + +// CleanupVerified is called when the block is cleaned up after a failed insertion. +func (be *blockExtension) CleanupVerified() { + vm := be.blockExtender.vm + if atomicState, err := vm.atomicBackend.GetVerifiedAtomicState(be.block.GetEthBlock().Hash()); err == nil { + atomicState.Reject() + } +} + +// verifyUTXOsPresent verifies all atomic UTXOs consumed by the block are +// present in shared memory. +func (be *blockExtension) verifyUTXOsPresent(atomicTxs []*atomic.Tx) error { + b := be.block + blockHash := common.Hash(b.ID()) + vm := be.blockExtender.vm + if vm.atomicBackend.IsBonus(b.Height(), blockHash) { + log.Info("skipping atomic tx verification on bonus block", "block", blockHash) + return nil + } + + // verify UTXOs named in import txs are present in shared memory. + for _, atomicTx := range atomicTxs { + utx := atomicTx.UnsignedAtomicTx + chainID, requests, err := utx.AtomicOps() + if err != nil { + return err + } + if _, err := vm.ctx.SharedMemory.Get(chainID, requests.RemoveRequests); err != nil { + return fmt.Errorf("%w: %s", errMissingUTXOs, err) + } + } + return nil +} diff --git a/plugin/evm/atomic/vm/bonus_blocks.go b/plugin/evm/atomic/vm/bonus_blocks.go new file mode 100644 index 0000000000..a81ceb9850 --- /dev/null +++ b/plugin/evm/atomic/vm/bonus_blocks.go @@ -0,0 +1,79 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package vm + +import "github.com/ava-labs/avalanchego/ids" + +// readMainnetBonusBlocks returns maps of bonus block numbers to block IDs. +// Note bonus blocks are indexed in the atomic trie. +func readMainnetBonusBlocks() (map[uint64]ids.ID, error) { + mainnetBonusBlocks := map[uint64]string{ + 102972: "Njm9TcLUXRojZk8YhEM6ksvfiPdC1TME4zJvGaDXgzMCyB6oB", + 103105: "BYqLB6xpqy7HsAgP2XNfGE8Ubg1uEzse5mBPTSJH9z5s8pvMa", + 103143: "AfWvJH3rB2fdHuPWQp6qYNCFVT29MooQPRigD88rKKwUDEDhq", + 103183: "2KPW9G5tiNF14tZNfG4SqHuQrtUYVZyxuof37aZ7AnTKrQdsHn", + 103197: "pE93VXY3N5QKfwsEFcM9i59UpPFgeZ8nxpJNaGaDQyDgsscNf", + 103203: "2czmtnBS44VCWNRFUM89h4Fe9m3ZeZVYyh7Pe3FhNqjRNgPXhZ", + 103208: "esx5J962LtYm2aSrskpLai5e4CMMsaS1dsu9iuLGJ3KWgSu2M", + 103209: "DK9NqAJGry1wAo767uuYc1dYXAjUhzwka6vi8d9tNheqzGUTd", + 103259: "i1HoerJ1axognkUKKL58FvF9aLrbZKtv7TdKLkT5kgzoeU1vB", + 103261: "2DpCuBaH94zKKFNY2XTs4GeJcwsEv6qT2DHc59S8tdg97GZpcJ", + 103266: "2ez4CA7w4HHr8SSobHQUAwFgj2giRNjNFUZK9JvrZFa1AuRj6X", + 103287: "2QBNMMFJmhVHaGF45GAPszKyj1gK6ToBERRxYvXtM7yfrdUGPK", + 103339: "2pSjfo7rkFCfZ2CqAxqfw8vqM2CU2nVLHrFZe3rwxz43gkVuGo", + 103346: "2SiSziHHqPjb1qkw7CdGYupokiYpd2b7mMqRiyszurctcA5AKr", + 103350: "2F5tSQbdTfhZxvkxZqdFp7KR3FrJPKEsDLQK7KtPhNXj1EZAh4", + 103358: "2tCe88ur6MLQcVgwE5XxoaHiTGtSrthwKN3SdbHE4kWiQ7MSTV", + 103437: "21o2fVTnzzmtgXqkV1yuQeze7YEQhR5JB31jVVD9oVUnaaV8qm", + 103472: "2nG4exd9eUoAGzELfksmBR8XDCKhohY1uDKRFzEXJG4M8p3qA7", + 103478: "63YLdYXfXc5tY3mwWLaDsbXzQHYmwWVxMP7HKbRh4Du3C2iM1", + 103493: "soPweZ8DGaoUMjrnzjH3V2bypa7ZvvfqBan4UCsMUxMP759gw", + 103514: "2dNkpQF4mooveyUDfBYQTBfsGDV4wkncQPpEw4kHKfSTSTo5x", + 103536: "PJTkRrHvKZ1m4AQdPND1MBpUXpCrGN4DDmXmJQAiUrsxPoLQX", + 103545: "22ck2Z7cC38hmBfX2v3jMWxun8eD8psNaicfYeokS67DxwmPTx", + 103547: "pTf7gfk1ksj7bqMrLyMCij8FBKth1uRqQrtfykMFeXhx5xnrL", + 103554: "9oZh4qyBCcVwSGyDoUzRAuausvPJN3xH6nopKS6bwYzMfLoQ2", + 103555: "MjExz2z1qhwugc1tAyiGxRsCq4GvJwKfyyS29nr4tRVB8ooic", + 103559: "cwJusfmn98TW3DjAbfLRN9utYR24KAQ82qpAXmVSvjHyJZuM2", + 103561: "2YgxGHns7Z2hMMHJsPCgVXuJaL7x1b3gnHbmSCfCdyAcYGr6mx", + 103563: "2AXxT3PSEnaYHNtBTnYrVTf24TtKDWjky9sqoFEhydrGXE9iKH", + 103564: "Ry2sfjFfGEnJxRkUGFSyZNn7GR3m4aKAf1scDW2uXSNQB568Y", + 103569: "21Jys8UNURmtckKSV89S2hntEWymJszrLQbdLaNcbXcxDAsQSa", + 103570: "sg6wAwFBsPQiS5Yfyh41cVkCRQbrrXsxXmeNyQ1xkunf2sdyv", + 103575: "z3BgePPpCXq1mRBRvUi28rYYxnEtJizkUEHnDBrcZeVA7MFVk", + 103577: "uK5Ff9iBfDtREpVv9NgCQ1STD1nzLJG3yrfibHG4mGvmybw6f", + 103578: "Qv5v5Ru8ArfnWKB1w6s4G5EYPh7TybHJtF6UsVwAkfvZFoqmj", + 103582: "7KCZKBpxovtX9opb7rMRie9WmW5YbZ8A4HwBBokJ9eSHpZPqx", + 103587: "2AfTQ2FXNj9bkSUQnud9pFXULx6EbF7cbbw6i3ayvc2QNhgxfF", + 103590: "2gTygYckZgFZfN5QQWPaPBD3nabqjidV55mwy1x1Nd4JmJAwaM", + 103591: "2cUPPHy1hspr2nAKpQrrAEisLKkaWSS9iF2wjNFyFRs8vnSkKK", + 103594: "5MptSdP6dBMPSwk9GJjeVe39deZJTRh9i82cgNibjeDffrrTf", + 103597: "2J8z7HNv4nwh82wqRGyEHqQeuw4wJ6mCDCSvUgusBu35asnshK", + 103598: "2i2FP6nJyvhX9FR15qN2D9AVoK5XKgBD2i2AQ7FoSpfowxvQDX", + 103603: "2v3smb35s4GLACsK4Zkd2RcLBLdWA4huqrvq8Y3VP4CVe8kfTM", + 103604: "b7XfDDLgwB12DfL7UTWZoxwBpkLPL5mdHtXngD94Y2RoeWXSh", + 103607: "PgaRk1UAoUvRybhnXsrLq5t6imWhEa6ksNjbN6hWgs4qPrSzm", + 103612: "2oueNTj4dUE2FFtGyPpawnmCCsy6EUQeVHVLZy8NHeQmkAciP4", + 103614: "2YHZ1KymFjiBhpXzgt6HXJhLSt5SV9UQ4tJuUNjfN1nQQdm5zz", + 103617: "amgH2C1s9H3Av7vSW4y7n7TXb9tKyKHENvrDXutgNN6nsejgc", + 103618: "fV8k1U8oQDmfVwK66kAwN73aSsWiWhm8quNpVnKmSznBycV2W", + 103621: "Nzs93kFTvcXanFUp9Y8VQkKYnzmH8xykxVNFJTkdyAEeuxWbP", + 103623: "2rAsBj3emqQa13CV8r5fTtHogs4sXnjvbbXVzcKPi3WmzhpK9D", + 103624: "2JbuExUGKW5mYz5KfXATwq1ibRDimgks9wEdYGNSC6Ttey1R4U", + 103627: "tLLijh7oKfvWT1yk9zRv4FQvuQ5DAiuvb5kHCNN9zh4mqkFMG", + 103628: "dWBsRYRwFrcyi3DPdLoHsL67QkZ5h86hwtVfP94ZBaY18EkmF", + 103629: "XMoEsew2DhSgQaydcJFJUQAQYP8BTNTYbEJZvtbrV2QsX7iE3", + 103630: "2db2wMbVAoCc5EUJrsBYWvNZDekqyY8uNpaaVapdBAQZ5oRaou", + 103633: "2QiHZwLhQ3xLuyyfcdo5yCUfoSqWDvRZox5ECU19HiswfroCGp", + } + + bonusBlockMainnetHeights := make(map[uint64]ids.ID) + for height, blkIDStr := range mainnetBonusBlocks { + blkID, err := ids.FromString(blkIDStr) + if err != nil { + return nil, err + } + bonusBlockMainnetHeights[height] = blkID + } + return bonusBlockMainnetHeights, nil +} diff --git a/plugin/evm/export_tx_test.go b/plugin/evm/atomic/vm/export_tx_test.go similarity index 76% rename from plugin/evm/export_tx_test.go rename to plugin/evm/atomic/vm/export_tx_test.go index d0a3aed252..3acb218a8c 100644 --- a/plugin/evm/export_tx_test.go +++ b/plugin/evm/atomic/vm/export_tx_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( "bytes" @@ -22,14 +22,19 @@ import ( "github.com/ava-labs/avalanchego/vms/secp256k1fx" "github.com/ava-labs/coreth/params/extras" "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/utils" "github.com/ava-labs/libevm/common" "github.com/holiman/uint256" + "github.com/stretchr/testify/require" ) // createExportTxOptions adds funds to shared memory, imports them, and returns a list of export transactions // that attempt to send the funds to each of the test keys (list of length 3). func createExportTxOptions(t *testing.T, vm *VM, issuer chan engCommon.Message, sharedMemory *avalancheatomic.Memory) []*atomic.Tx { + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddr := key.EthAddress() // Add a UTXO to shared memory utxo := &avax.UTXO{ UTXOID: avax.UTXOID{TxID: ids.GenerateTestID()}, @@ -38,7 +43,7 @@ func createExportTxOptions(t *testing.T, vm *VM, issuer chan engCommon.Message, Amt: uint64(50000000), OutputOwners: secp256k1fx.OutputOwners{ Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, + Addrs: []ids.ShortID{key.Address()}, }, }, } @@ -53,14 +58,14 @@ func createExportTxOptions(t *testing.T, vm *VM, issuer chan engCommon.Message, Key: inputID[:], Value: utxoBytes, Traits: [][]byte{ - testKeys[0].Address().Bytes(), + key.Address().Bytes(), }, }}}}); err != nil { t.Fatal(err) } // Import the funds - importTx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + importTx, err := vm.newImportTx(vm.ctx.XChainID, ethAddr, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } @@ -90,12 +95,12 @@ func createExportTxOptions(t *testing.T, vm *VM, issuer chan engCommon.Message, // Use the funds to create 3 conflicting export transactions sending the funds to each of the test addresses exportTxs := make([]*atomic.Tx, 0, 3) - state, err := vm.blockChain.State() + state, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - for _, addr := range testShortIDAddrs { - exportTx, err := atomic.NewExportTx(vm.ctx, vm.currentRules(), state, vm.ctx.AVAXAssetID, uint64(5000000), vm.ctx.XChainID, addr, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + for _, addr := range vmtest.TestShortIDAddrs { + exportTx, err := atomic.NewExportTx(vm.ctx, vm.currentRules(), state, vm.ctx.AVAXAssetID, uint64(5000000), vm.ctx.XChainID, addr, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } @@ -106,7 +111,8 @@ func createExportTxOptions(t *testing.T, vm *VM, issuer chan engCommon.Message, } func TestExportTxEVMStateTransfer(t *testing.T) { - key := testKeys[0] + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) addr := key.Address() ethAddr := key.EthAddress() @@ -328,18 +334,19 @@ func TestExportTxEVMStateTransfer(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() avaxUTXO := &avax.UTXO{ UTXOID: avaxUTXOID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, Out: &secp256k1fx.TransferOutput{ Amt: avaxAmount, OutputOwners: secp256k1fx.OutputOwners{ @@ -359,8 +366,8 @@ func TestExportTxEVMStateTransfer(t *testing.T) { t.Fatal(err) } - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{ + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{ { Key: avaxInputID[:], Value: avaxUTXOBytes, @@ -379,18 +386,18 @@ func TestExportTxEVMStateTransfer(t *testing.T) { t.Fatal(err) } - tx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err := vm.newImportTx(vm.ctx.XChainID, ethAddr, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(tx); err != nil { + if err := vm.mempool.AddLocalTx(tx); err != nil { t.Fatal(err) } - <-tvm.toEngine + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -399,7 +406,7 @@ func TestExportTxEVMStateTransfer(t *testing.T) { t.Fatal(err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { t.Fatal(err) } @@ -411,12 +418,12 @@ func TestExportTxEVMStateTransfer(t *testing.T) { Ins: test.tx, } - stateDB, err := tvm.vm.blockChain.State() + stateDB, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - err = newTx.EVMStateTransfer(tvm.vm.ctx, stateDB) + err = newTx.EVMStateTransfer(vm.ctx, stateDB) if test.shouldErr { if err == nil { t.Fatal("expected EVMStateTransfer to fail") @@ -448,20 +455,21 @@ func TestExportTxEVMStateTransfer(t *testing.T) { func TestExportTxSemanticVerify(t *testing.T) { fork := upgradetest.NoUpgrades - vm := newVM(t, testVMConfig{ - fork: &fork, - }).vm + vm := newAtomicTestVM() + _ = vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &fork, + }) defer func() { if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - parent := vm.LastAcceptedBlockInternal().(*Block) + parent := vm.LastAcceptedVMBlock() - key := testKeys[0] + key := vmtest.TestKeys[0] addr := key.Address() - ethAddr := testEthAddrs[0] + ethAddr := vmtest.TestEthAddrs[0] var ( avaxBalance = 10 * units.Avax @@ -541,7 +549,7 @@ func TestExportTxSemanticVerify(t *testing.T) { tx *atomic.Tx signers [][]*secp256k1.PrivateKey baseFee *big.Int - rules extras.Rules + rules *extras.Rules shouldErr bool }{ { @@ -552,8 +560,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: false, }, { @@ -566,8 +574,8 @@ func TestExportTxSemanticVerify(t *testing.T) { signers: [][]*secp256k1.PrivateKey{ {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -580,8 +588,8 @@ func TestExportTxSemanticVerify(t *testing.T) { signers: [][]*secp256k1.PrivateKey{ {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase5, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), shouldErr: false, }, { @@ -594,8 +602,8 @@ func TestExportTxSemanticVerify(t *testing.T) { signers: [][]*secp256k1.PrivateKey{ {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase5, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), shouldErr: true, }, { @@ -610,8 +618,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -626,8 +634,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase5, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), shouldErr: true, }, { @@ -642,8 +650,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase5, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), shouldErr: true, }, { @@ -658,8 +666,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -674,8 +682,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -690,8 +698,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -707,8 +715,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -732,8 +740,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -773,8 +781,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -790,8 +798,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -817,8 +825,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -844,8 +852,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -857,8 +865,8 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -868,20 +876,20 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { name: "too many signatures on credential", tx: &atomic.Tx{UnsignedAtomicTx: validExportTx}, signers: [][]*secp256k1.PrivateKey{ - {key, testKeys[1]}, + {key, vmtest.TestKeys[1]}, {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { @@ -892,28 +900,28 @@ func TestExportTxSemanticVerify(t *testing.T) { {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { name: "wrong signature on credential", tx: &atomic.Tx{UnsignedAtomicTx: validExportTx}, signers: [][]*secp256k1.PrivateKey{ - {testKeys[1]}, + {vmtest.TestKeys[1]}, {key}, {key}, }, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, { name: "no signatures", tx: &atomic.Tx{UnsignedAtomicTx: validExportTx}, signers: [][]*secp256k1.PrivateKey{}, - baseFee: initialBaseFee, - rules: apricotRulesPhase3, + baseFee: vmtest.InitialBaseFee, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), shouldErr: true, }, } @@ -922,24 +930,24 @@ func TestExportTxSemanticVerify(t *testing.T) { t.Fatal(err) } - backend := &atomic.VerifierBackend{ - Ctx: vm.ctx, - Fx: &vm.fx, - Rules: test.rules, - Bootstrapped: vm.bootstrapped.Get(), - BlockFetcher: vm, - SecpCache: vm.secpCache, + backend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: *test.rules, + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, } t.Run(test.name, func(t *testing.T) { tx := test.tx exportTx := tx.UnsignedAtomicTx - err := exportTx.Visit(&atomic.SemanticVerifier{ - Backend: backend, - Tx: tx, - Parent: parent, - BaseFee: test.baseFee, + err := exportTx.Visit(&semanticVerifier{ + backend: backend, + tx: tx, + parent: parent, + baseFee: test.baseFee, }) if test.shouldErr && err == nil { t.Fatalf("should have errored but returned valid") @@ -953,20 +961,21 @@ func TestExportTxSemanticVerify(t *testing.T) { func TestExportTxAccept(t *testing.T) { fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) - key := testKeys[0] + key := vmtest.TestKeys[0] addr := key.Address() - ethAddr := testEthAddrs[0] + ethAddr := vmtest.TestEthAddrs[0] var ( avaxBalance = 10 * units.Avax @@ -975,14 +984,14 @@ func TestExportTxAccept(t *testing.T) { ) exportTx := &atomic.UnsignedExportTx{ - NetworkID: tvm.vm.ctx.NetworkID, - BlockchainID: tvm.vm.ctx.ChainID, - DestinationChain: tvm.vm.ctx.XChainID, + NetworkID: vm.ctx.NetworkID, + BlockchainID: vm.ctx.ChainID, + DestinationChain: vm.ctx.XChainID, Ins: []atomic.EVMInput{ { Address: ethAddr, Amount: avaxBalance, - AssetID: tvm.vm.ctx.AVAXAssetID, + AssetID: vm.ctx.AVAXAssetID, Nonce: 0, }, { @@ -994,7 +1003,7 @@ func TestExportTxAccept(t *testing.T) { }, ExportedOutputs: []*avax.TransferableOutput{ { - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, Out: &secp256k1fx.TransferOutput{ Amt: avaxBalance, OutputOwners: secp256k1fx.OutputOwners{ @@ -1028,7 +1037,7 @@ func TestExportTxAccept(t *testing.T) { t.Fatal(err) } - commitBatch, err := tvm.vm.versiondb.CommitBatch() + commitBatch, err := vm.VersionDB().CommitBatch() if err != nil { t.Fatalf("Failed to create commit batch for VM due to %s", err) } @@ -1037,10 +1046,10 @@ func TestExportTxAccept(t *testing.T) { t.Fatalf("Failed to accept export transaction due to: %s", err) } - if err := tvm.vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { + if err := vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { t.Fatal(err) } - indexedValues, _, _, err := xChainSharedMemory.Indexed(tvm.vm.ctx.ChainID, [][]byte{addr.Bytes()}, nil, nil, 3) + indexedValues, _, _, err := xChainSharedMemory.Indexed(vm.ctx.ChainID, [][]byte{addr.Bytes()}, nil, nil, 3) if err != nil { t.Fatal(err) } @@ -1061,7 +1070,7 @@ func TestExportTxAccept(t *testing.T) { } customInputID := customUTXOID.InputID() - fetchedValues, err := xChainSharedMemory.Get(tvm.vm.ctx.ChainID, [][]byte{ + fetchedValues, err := xChainSharedMemory.Get(vm.ctx.ChainID, [][]byte{ customInputID[:], avaxInputID[:], }) @@ -1087,7 +1096,7 @@ func TestExportTxAccept(t *testing.T) { avaxUTXOBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, &avax.UTXO{ UTXOID: avaxUTXOID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, Out: exportTx.ExportedOutputs[0].Out, }) if err != nil { @@ -1110,13 +1119,13 @@ func TestExportTxVerify(t *testing.T) { DestinationChain: snowtest.XChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: snowtest.AVAXAssetID, Nonce: 0, }, { - Address: testEthAddrs[2], + Address: vmtest.TestEthAddrs[2], Amount: exportAmount, AssetID: snowtest.AVAXAssetID, Nonce: 0, @@ -1130,7 +1139,7 @@ func TestExportTxVerify(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, @@ -1141,7 +1150,7 @@ func TestExportTxVerify(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[1]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[1]}, }, }, }, @@ -1163,7 +1172,7 @@ func TestExportTxVerify(t *testing.T) { return (*atomic.UnsignedExportTx)(nil) }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrNilTx.Error(), }, "valid export tx": { @@ -1171,7 +1180,7 @@ func TestExportTxVerify(t *testing.T) { return exportTx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", }, "valid export tx banff": { @@ -1179,7 +1188,7 @@ func TestExportTxVerify(t *testing.T) { return exportTx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: "", }, "incorrect networkID": { @@ -1189,7 +1198,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongNetworkID.Error(), }, "incorrect blockchainID": { @@ -1199,7 +1208,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongChainID.Error(), }, "incorrect destination chain": { @@ -1209,7 +1218,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongChainID.Error(), // TODO make this error more specific to destination not just chainID }, "no exported outputs": { @@ -1219,7 +1228,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrNoExportOutputs.Error(), }, "unsorted outputs": { @@ -1232,7 +1241,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrOutputsNotSorted.Error(), }, "invalid exported output": { @@ -1242,7 +1251,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "nil transferable output is not valid", }, "unsorted EVM inputs before AP1": { @@ -1255,7 +1264,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", }, "unsorted EVM inputs after AP1": { @@ -1268,7 +1277,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase1, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase1), expectedErr: atomic.ErrInputsNotSortedUnique.Error(), }, "EVM input with amount 0": { @@ -1276,7 +1285,7 @@ func TestExportTxVerify(t *testing.T) { tx := *exportTx tx.Ins = []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: 0, AssetID: snowtest.AVAXAssetID, Nonce: 0, @@ -1285,7 +1294,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrNoValueInput.Error(), }, "non-unique EVM input before AP1": { @@ -1295,7 +1304,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", }, "non-unique EVM input after AP1": { @@ -1305,7 +1314,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase1, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase1), expectedErr: atomic.ErrInputsNotSortedUnique.Error(), }, "non-AVAX input Apricot Phase 6": { @@ -1313,7 +1322,7 @@ func TestExportTxVerify(t *testing.T) { tx := *exportTx tx.Ins = []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: 1, AssetID: ids.GenerateTestID(), Nonce: 0, @@ -1322,7 +1331,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase6, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase6), expectedErr: "", }, "non-AVAX output Apricot Phase 6": { @@ -1336,7 +1345,7 @@ func TestExportTxVerify(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, @@ -1344,7 +1353,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase6, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase6), expectedErr: "", }, "non-AVAX input Banff": { @@ -1352,7 +1361,7 @@ func TestExportTxVerify(t *testing.T) { tx := *exportTx tx.Ins = []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: 1, AssetID: ids.GenerateTestID(), Nonce: 0, @@ -1361,7 +1370,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: atomic.ErrExportNonAVAXInputBanff.Error(), }, "non-AVAX output Banff": { @@ -1375,7 +1384,7 @@ func TestExportTxVerify(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, @@ -1383,7 +1392,7 @@ func TestExportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: atomic.ErrExportNonAVAXOutputBanff.Error(), }, } @@ -1420,7 +1429,7 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1434,13 +1443,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 1230, ExpectedFee: 1, BaseFee: big.NewInt(1), @@ -1452,7 +1461,7 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1466,13 +1475,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 11230, ExpectedFee: 1, BaseFee: big.NewInt(1), @@ -1485,7 +1494,7 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1499,13 +1508,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 1230, ExpectedFee: 30750, BaseFee: big.NewInt(25 * utils.GWei), @@ -1517,7 +1526,7 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1531,13 +1540,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 1230, ExpectedFee: 276750, BaseFee: big.NewInt(225 * utils.GWei), @@ -1549,19 +1558,19 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, }, { - Address: testEthAddrs[1], + Address: vmtest.TestEthAddrs[1], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, }, { - Address: testEthAddrs[2], + Address: vmtest.TestEthAddrs[2], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1575,13 +1584,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0], testKeys[0], testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0], vmtest.TestKeys[0], vmtest.TestKeys[0]}}, ExpectedGasUsed: 3366, ExpectedFee: 84150, BaseFee: big.NewInt(25 * utils.GWei), @@ -1593,19 +1602,19 @@ func TestExportTxGasCost(t *testing.T) { DestinationChain: xChainID, Ins: []atomic.EVMInput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, }, { - Address: testEthAddrs[1], + Address: vmtest.TestEthAddrs[1], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, }, { - Address: testEthAddrs[2], + Address: vmtest.TestEthAddrs[2], Amount: exportAmount, AssetID: avaxAssetID, Nonce: 0, @@ -1619,13 +1628,13 @@ func TestExportTxGasCost(t *testing.T) { OutputOwners: secp256k1fx.OutputOwners{ Locktime: 0, Threshold: 1, - Addrs: []ids.ShortID{testShortIDAddrs[0]}, + Addrs: []ids.ShortID{vmtest.TestShortIDAddrs[0]}, }, }, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0], testKeys[0], testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0], vmtest.TestKeys[0], vmtest.TestKeys[0]}}, ExpectedGasUsed: 3366, ExpectedFee: 757350, BaseFee: big.NewInt(225 * utils.GWei), @@ -1661,6 +1670,9 @@ func TestExportTxGasCost(t *testing.T) { } func TestNewExportTx(t *testing.T) { + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddress := key.PublicKey().EthAddress() tests := []struct { fork upgradetest.Fork bal uint64 @@ -1699,27 +1711,28 @@ func TestNewExportTx(t *testing.T) { } for _, test := range tests { t.Run(test.fork.String(), func(t *testing.T) { - tvm := newVM(t, testVMConfig{ - fork: &test.fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &test.fork, }) defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - parent := tvm.vm.LastAcceptedBlockInternal().(*Block) + parent := vm.LastAcceptedVMBlock() importAmount := uint64(50000000) utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} utxo := &avax.UTXO{ UTXOID: utxoID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, Out: &secp256k1fx.TransferOutput{ Amt: importAmount, OutputOwners: secp256k1fx.OutputOwners{ Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, + Addrs: []ids.ShortID{key.Address()}, }, }, } @@ -1728,30 +1741,30 @@ func TestNewExportTx(t *testing.T) { t.Fatal(err) } - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) inputID := utxo.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ Key: inputID[:], Value: utxoBytes, Traits: [][]byte{ - testKeys[0].Address().Bytes(), + key.Address().Bytes(), }, }}}}); err != nil { t.Fatal(err) } - tx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err := vm.newImportTx(vm.ctx.XChainID, ethAddress, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(tx); err != nil { + if err := vm.mempool.AddLocalTx(tx); err != nil { t.Fatal(err) } - <-tvm.toEngine + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -1760,7 +1773,7 @@ func TestNewExportTx(t *testing.T) { t.Fatal(err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { t.Fatal(err) } @@ -1768,40 +1781,40 @@ func TestNewExportTx(t *testing.T) { t.Fatal(err) } - parent = tvm.vm.LastAcceptedBlockInternal().(*Block) + parent = vm.LastAcceptedVMBlock() exportAmount := uint64(5000000) - state, err := tvm.vm.blockChain.State() + state, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - tx, err = atomic.NewExportTx(tvm.vm.ctx, tvm.vm.currentRules(), state, tvm.vm.ctx.AVAXAssetID, exportAmount, tvm.vm.ctx.XChainID, testShortIDAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err = atomic.NewExportTx(vm.ctx, vm.currentRules(), state, vm.ctx.AVAXAssetID, exportAmount, vm.ctx.XChainID, key.Address(), vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } exportTx := tx.UnsignedAtomicTx - backend := &atomic.VerifierBackend{ - Ctx: tvm.vm.ctx, - Fx: &tvm.vm.fx, - Rules: tvm.vm.currentRules(), - Bootstrapped: tvm.vm.bootstrapped.Get(), - BlockFetcher: tvm.vm, - SecpCache: tvm.vm.secpCache, + backend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: vm.currentRules(), + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, } - if err := exportTx.Visit(&atomic.SemanticVerifier{ - Backend: backend, - Tx: tx, - Parent: parent, - BaseFee: parent.ethBlock.BaseFee(), + if err := exportTx.Visit(&semanticVerifier{ + backend: backend, + tx: tx, + parent: parent, + baseFee: parent.GetEthBlock().BaseFee(), }); err != nil { t.Fatal("newExportTx created an invalid transaction", err) } - burnedAVAX, err := exportTx.Burned(tvm.vm.ctx.AVAXAssetID) + burnedAVAX, err := exportTx.Burned(vm.ctx.AVAXAssetID) if err != nil { t.Fatal(err) } @@ -1809,7 +1822,7 @@ func TestNewExportTx(t *testing.T) { t.Fatalf("burned wrong amount of AVAX - expected %d burned %d", test.expectedBurnedAVAX, burnedAVAX) } - commitBatch, err := tvm.vm.versiondb.CommitBatch() + commitBatch, err := vm.VersionDB().CommitBatch() if err != nil { t.Fatalf("Failed to create commit batch for VM due to %s", err) } @@ -1818,28 +1831,30 @@ func TestNewExportTx(t *testing.T) { t.Fatalf("Failed to accept export transaction due to: %s", err) } - if err := tvm.vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { + if err := vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { t.Fatal(err) } - sdb, err := tvm.vm.blockChain.State() + sdb, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - err = exportTx.EVMStateTransfer(tvm.vm.ctx, sdb) + err = exportTx.EVMStateTransfer(vm.ctx, sdb) if err != nil { t.Fatal(err) } - addr := testKeys[0].EthAddress() - if sdb.GetBalance(addr).Cmp(uint256.NewInt(test.bal*units.Avax)) != 0 { - t.Fatalf("address balance %s equal %s not %s", addr.String(), sdb.GetBalance(addr), new(big.Int).SetUint64(test.bal*units.Avax)) + if sdb.GetBalance(ethAddress).Cmp(uint256.NewInt(test.bal*units.Avax)) != 0 { + t.Fatalf("address balance %s equal %s not %s", ethAddress.String(), sdb.GetBalance(ethAddress), new(big.Int).SetUint64(test.bal*units.Avax)) } }) } } func TestNewExportTxMulticoin(t *testing.T) { + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddress := key.PublicKey().EthAddress() tests := []struct { fork upgradetest.Fork bal uint64 @@ -1868,27 +1883,28 @@ func TestNewExportTxMulticoin(t *testing.T) { } for _, test := range tests { t.Run(test.fork.String(), func(t *testing.T) { - tvm := newVM(t, testVMConfig{ - fork: &test.fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &test.fork, }) defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - parent := tvm.vm.LastAcceptedBlockInternal().(*Block) + parent := vm.LastAcceptedVMBlock() importAmount := uint64(50000000) utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} utxo := &avax.UTXO{ UTXOID: utxoID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, Out: &secp256k1fx.TransferOutput{ Amt: importAmount, OutputOwners: secp256k1fx.OutputOwners{ Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, + Addrs: []ids.ShortID{key.Address()}, }, }, } @@ -1909,7 +1925,7 @@ func TestNewExportTxMulticoin(t *testing.T) { Amt: importAmount2, OutputOwners: secp256k1fx.OutputOwners{ Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, + Addrs: []ids.ShortID{key.Address()}, }, }, } @@ -1918,39 +1934,39 @@ func TestNewExportTxMulticoin(t *testing.T) { t.Fatal(err) } - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) inputID2 := utxo2.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{ + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{ { Key: inputID[:], Value: utxoBytes, Traits: [][]byte{ - testKeys[0].Address().Bytes(), + key.Address().Bytes(), }, }, { Key: inputID2[:], Value: utxoBytes2, Traits: [][]byte{ - testKeys[0].Address().Bytes(), + key.Address().Bytes(), }, }, }}}); err != nil { t.Fatal(err) } - tx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err := vm.newImportTx(vm.ctx.XChainID, ethAddress, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddRemoteTx(tx); err != nil { + if err := vm.mempool.AddRemoteTx(tx); err != nil { t.Fatal(err) } - <-tvm.toEngine + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -1959,7 +1975,7 @@ func TestNewExportTxMulticoin(t *testing.T) { t.Fatal(err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { t.Fatal(err) } @@ -1967,45 +1983,45 @@ func TestNewExportTxMulticoin(t *testing.T) { t.Fatal(err) } - parent = tvm.vm.LastAcceptedBlockInternal().(*Block) + parent = vm.LastAcceptedVMBlock() exportAmount := uint64(5000000) - testKeys0Addr := testKeys[0].EthAddress() + testKeys0Addr := vmtest.TestKeys[0].EthAddress() exportId, err := ids.ToShortID(testKeys0Addr[:]) if err != nil { t.Fatal(err) } - state, err := tvm.vm.blockChain.State() + state, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - tx, err = atomic.NewExportTx(tvm.vm.ctx, tvm.vm.currentRules(), state, tid, exportAmount, tvm.vm.ctx.XChainID, exportId, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err = atomic.NewExportTx(vm.ctx, vm.currentRules(), state, tid, exportAmount, vm.ctx.XChainID, exportId, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } exportTx := tx.UnsignedAtomicTx - backend := &atomic.VerifierBackend{ - Ctx: tvm.vm.ctx, - Fx: &tvm.vm.fx, - Rules: tvm.vm.currentRules(), - Bootstrapped: tvm.vm.bootstrapped.Get(), - BlockFetcher: tvm.vm, - SecpCache: tvm.vm.secpCache, - } - - if err := exportTx.Visit(&atomic.SemanticVerifier{ - Backend: backend, - Tx: tx, - Parent: parent, - BaseFee: parent.ethBlock.BaseFee(), + backend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: vm.currentRules(), + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, + } + + if err := exportTx.Visit(&semanticVerifier{ + backend: backend, + tx: tx, + parent: parent, + baseFee: parent.GetEthBlock().BaseFee(), }); err != nil { t.Fatal("newExportTx created an invalid transaction", err) } - commitBatch, err := tvm.vm.versiondb.CommitBatch() + commitBatch, err := vm.VersionDB().CommitBatch() if err != nil { t.Fatalf("Failed to create commit batch for VM due to %s", err) } @@ -2014,25 +2030,24 @@ func TestNewExportTxMulticoin(t *testing.T) { t.Fatalf("Failed to accept export transaction due to: %s", err) } - if err := tvm.vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { + if err := vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{chainID: {PutRequests: atomicRequests.PutRequests}}, commitBatch); err != nil { t.Fatal(err) } - stdb, err := tvm.vm.blockChain.State() + stdb, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } - err = exportTx.EVMStateTransfer(tvm.vm.ctx, stdb) + err = exportTx.EVMStateTransfer(vm.ctx, stdb) if err != nil { t.Fatal(err) } - addr := testKeys[0].EthAddress() - if stdb.GetBalance(addr).Cmp(uint256.NewInt(test.bal*units.Avax)) != 0 { - t.Fatalf("address balance %s equal %s not %s", addr.String(), stdb.GetBalance(addr), new(big.Int).SetUint64(test.bal*units.Avax)) + if stdb.GetBalance(ethAddress).Cmp(uint256.NewInt(test.bal*units.Avax)) != 0 { + t.Fatalf("address balance %s equal %s not %s", ethAddress.String(), stdb.GetBalance(ethAddress), new(big.Int).SetUint64(test.bal*units.Avax)) } - if stdb.GetBalanceMultiCoin(addr, common.BytesToHash(tid[:])).Cmp(new(big.Int).SetUint64(test.balmc)) != 0 { - t.Fatalf("address balance multicoin %s equal %s not %s", addr.String(), stdb.GetBalanceMultiCoin(addr, common.BytesToHash(tid[:])), new(big.Int).SetUint64(test.balmc)) + if stdb.GetBalanceMultiCoin(ethAddress, common.BytesToHash(tid[:])).Cmp(new(big.Int).SetUint64(test.balmc)) != 0 { + t.Fatalf("address balance multicoin %s equal %s not %s", ethAddress.String(), stdb.GetBalanceMultiCoin(ethAddress, common.BytesToHash(tid[:])), new(big.Int).SetUint64(test.balmc)) } }) } diff --git a/plugin/evm/ext_data_hashes.go b/plugin/evm/atomic/vm/ext_data_hashes.go similarity index 98% rename from plugin/evm/ext_data_hashes.go rename to plugin/evm/atomic/vm/ext_data_hashes.go index 27a9d741f6..af310db66a 100644 --- a/plugin/evm/ext_data_hashes.go +++ b/plugin/evm/atomic/vm/ext_data_hashes.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( _ "embed" diff --git a/plugin/evm/atomic/vm/formatting.go b/plugin/evm/atomic/vm/formatting.go new file mode 100644 index 0000000000..57adaa3d9e --- /dev/null +++ b/plugin/evm/atomic/vm/formatting.go @@ -0,0 +1,73 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "fmt" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/utils/constants" + "github.com/ava-labs/avalanchego/utils/formatting/address" +) + +// ParseServiceAddress get address ID from address string, being it either localized (using address manager, +// doing also components validations), or not localized. +// If both attempts fail, reports error from localized address parsing +func ParseServiceAddress(ctx *snow.Context, addrStr string) (ids.ShortID, error) { + addr, err := ids.ShortFromString(addrStr) + if err == nil { + return addr, nil + } + return ParseLocalAddress(ctx, addrStr) +} + +// ParseLocalAddress takes in an address for this chain and produces the ID +func ParseLocalAddress(ctx *snow.Context, addrStr string) (ids.ShortID, error) { + chainID, addr, err := ParseAddress(ctx, addrStr) + if err != nil { + return ids.ShortID{}, err + } + if chainID != ctx.ChainID { + return ids.ShortID{}, fmt.Errorf("expected chainID to be %q but was %q", + ctx.ChainID, chainID) + } + return addr, nil +} + +// FormatLocalAddress takes in a raw address and produces the formatted address +func FormatLocalAddress(ctx *snow.Context, addr ids.ShortID) (string, error) { + chainIDAlias, err := ctx.BCLookup.PrimaryAlias(ctx.ChainID) + if err != nil { + return "", err + } + hrp := constants.GetHRP(ctx.NetworkID) + return address.Format(chainIDAlias, hrp, addr.Bytes()) +} + +// ParseAddress takes in an address and produces the ID of the chain it's for +// the ID of the address +func ParseAddress(ctx *snow.Context, addrStr string) (ids.ID, ids.ShortID, error) { + chainIDAlias, hrp, addrBytes, err := address.Parse(addrStr) + if err != nil { + return ids.ID{}, ids.ShortID{}, err + } + + chainID, err := ctx.BCLookup.Lookup(chainIDAlias) + if err != nil { + return ids.ID{}, ids.ShortID{}, err + } + + expectedHRP := constants.GetHRP(ctx.NetworkID) + if hrp != expectedHRP { + return ids.ID{}, ids.ShortID{}, fmt.Errorf("expected hrp %q but got %q", + expectedHRP, hrp) + } + + addr, err := ids.ToShortID(addrBytes) + if err != nil { + return ids.ID{}, ids.ShortID{}, err + } + return chainID, addr, nil +} diff --git a/plugin/evm/fuji_ext_data_hashes.json b/plugin/evm/atomic/vm/fuji_ext_data_hashes.json similarity index 100% rename from plugin/evm/fuji_ext_data_hashes.json rename to plugin/evm/atomic/vm/fuji_ext_data_hashes.json diff --git a/plugin/evm/gossiper_atomic_gossiping_test.go b/plugin/evm/atomic/vm/gossiper_atomic_gossiping_test.go similarity index 74% rename from plugin/evm/gossiper_atomic_gossiping_test.go rename to plugin/evm/atomic/vm/gossiper_atomic_gossiping_test.go index 917fe5b91f..0de16e209f 100644 --- a/plugin/evm/gossiper_atomic_gossiping_test.go +++ b/plugin/evm/atomic/vm/gossiper_atomic_gossiping_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( "context" @@ -20,6 +20,7 @@ import ( commonEng "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/vmtest" ) // show that a txID discovered from gossip is requested to the same node only if @@ -27,9 +28,10 @@ import ( func TestMempoolAtmTxsAppGossipHandling(t *testing.T) { assert := assert.New(t) - tvm := newVM(t, testVMConfig{}) + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{}) defer func() { - assert.NoError(tvm.vm.Shutdown(context.Background())) + assert.NoError(vm.Shutdown(context.Background())) }() nodeID := ids.GenerateTestNodeID() @@ -39,50 +41,50 @@ func TestMempoolAtmTxsAppGossipHandling(t *testing.T) { txGossipedLock sync.Mutex txRequested bool ) - tvm.appSender.CantSendAppGossip = false - tvm.appSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { + tvm.AppSender.CantSendAppGossip = false + tvm.AppSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { txGossipedLock.Lock() defer txGossipedLock.Unlock() txGossiped++ return nil } - tvm.appSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { + tvm.AppSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { txRequested = true return nil } // Create conflicting transactions - importTxs := createImportTxOptions(t, tvm.vm, tvm.atomicMemory) + importTxs := createImportTxOptions(t, vm, tvm.AtomicMemory) tx, conflictingTx := importTxs[0], importTxs[1] // gossip tx and check it is accepted and gossiped marshaller := atomic.TxMarshaller{} txBytes, err := marshaller.MarshalGossip(tx) assert.NoError(err) - tvm.vm.ctx.Lock.Unlock() + vm.ctx.Lock.Unlock() msgBytes, err := buildAtomicPushGossip(txBytes) assert.NoError(err) // show that no txID is requested - assert.NoError(tvm.vm.AppGossip(context.Background(), nodeID, msgBytes)) + assert.NoError(vm.AppGossip(context.Background(), nodeID, msgBytes)) time.Sleep(500 * time.Millisecond) - tvm.vm.ctx.Lock.Lock() + vm.ctx.Lock.Lock() assert.False(txRequested, "tx should not have been requested") txGossipedLock.Lock() assert.Equal(0, txGossiped, "tx should not have been gossiped") txGossipedLock.Unlock() - assert.True(tvm.vm.mempool.Has(tx.ID())) + assert.True(vm.mempool.Has(tx.ID())) - tvm.vm.ctx.Lock.Unlock() + vm.ctx.Lock.Unlock() // show that tx is not re-gossiped - assert.NoError(tvm.vm.AppGossip(context.Background(), nodeID, msgBytes)) + assert.NoError(vm.AppGossip(context.Background(), nodeID, msgBytes)) - tvm.vm.ctx.Lock.Lock() + vm.ctx.Lock.Lock() txGossipedLock.Lock() assert.Equal(0, txGossiped, "tx should not have been gossiped") @@ -93,51 +95,52 @@ func TestMempoolAtmTxsAppGossipHandling(t *testing.T) { txBytes, err = marshaller.MarshalGossip(conflictingTx) assert.NoError(err) - tvm.vm.ctx.Lock.Unlock() + vm.ctx.Lock.Unlock() msgBytes, err = buildAtomicPushGossip(txBytes) assert.NoError(err) - assert.NoError(tvm.vm.AppGossip(context.Background(), nodeID, msgBytes)) + assert.NoError(vm.AppGossip(context.Background(), nodeID, msgBytes)) - tvm.vm.ctx.Lock.Lock() + vm.ctx.Lock.Lock() assert.False(txRequested, "tx should not have been requested") txGossipedLock.Lock() assert.Equal(0, txGossiped, "tx should not have been gossiped") txGossipedLock.Unlock() - assert.False(tvm.vm.mempool.Has(conflictingTx.ID()), "conflicting tx should not be in the atomic mempool") + assert.False(vm.mempool.Has(conflictingTx.ID()), "conflicting tx should not be in the atomic mempool") } // show that txs already marked as invalid are not re-requested on gossiping func TestMempoolAtmTxsAppGossipHandlingDiscardedTx(t *testing.T) { assert := assert.New(t) - tvm := newVM(t, testVMConfig{}) + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{}) defer func() { - assert.NoError(tvm.vm.Shutdown(context.Background())) + assert.NoError(vm.Shutdown(context.Background())) }() - mempool := tvm.vm.mempool + mempool := vm.mempool var ( txGossiped int txGossipedLock sync.Mutex txRequested bool ) - tvm.appSender.CantSendAppGossip = false - tvm.appSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { + tvm.AppSender.CantSendAppGossip = false + tvm.AppSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { txGossipedLock.Lock() defer txGossipedLock.Unlock() txGossiped++ return nil } - tvm.appSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { + tvm.AppSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { txRequested = true return nil } // Create a transaction and mark it as invalid by discarding it - importTxs := createImportTxOptions(t, tvm.vm, tvm.atomicMemory) + importTxs := createImportTxOptions(t, vm, tvm.AtomicMemory) tx, conflictingTx := importTxs[0], importTxs[1] txID := tx.ID() @@ -155,13 +158,13 @@ func TestMempoolAtmTxsAppGossipHandlingDiscardedTx(t *testing.T) { txBytes, err := marshaller.MarshalGossip(tx) assert.NoError(err) - tvm.vm.ctx.Lock.Unlock() + vm.ctx.Lock.Unlock() msgBytes, err := buildAtomicPushGossip(txBytes) assert.NoError(err) - assert.NoError(tvm.vm.AppGossip(context.Background(), nodeID, msgBytes)) + assert.NoError(vm.AppGossip(context.Background(), nodeID, msgBytes)) - tvm.vm.ctx.Lock.Lock() + vm.ctx.Lock.Lock() assert.False(txRequested, "tx shouldn't be requested") txGossipedLock.Lock() @@ -170,16 +173,16 @@ func TestMempoolAtmTxsAppGossipHandlingDiscardedTx(t *testing.T) { assert.False(mempool.Has(txID)) - tvm.vm.ctx.Lock.Unlock() + vm.ctx.Lock.Unlock() // Conflicting tx must be submitted over the API to be included in push gossip. // (i.e., txs received via p2p are not included in push gossip) // This test adds it directly to the mempool + gossiper to simulate that. - tvm.vm.mempool.AddRemoteTx(conflictingTx) - tvm.vm.atomicTxPushGossiper.Add(conflictingTx) + vm.mempool.AddRemoteTx(conflictingTx) + vm.atomicTxPushGossiper.Add(conflictingTx) time.Sleep(500 * time.Millisecond) - tvm.vm.ctx.Lock.Lock() + vm.ctx.Lock.Lock() assert.False(txRequested, "tx shouldn't be requested") txGossipedLock.Lock() diff --git a/plugin/evm/import_tx_test.go b/plugin/evm/atomic/vm/import_tx_test.go similarity index 84% rename from plugin/evm/import_tx_test.go rename to plugin/evm/atomic/vm/import_tx_test.go index a634176e42..4ec322b526 100644 --- a/plugin/evm/import_tx_test.go +++ b/plugin/evm/atomic/vm/import_tx_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( "math/big" @@ -9,9 +9,11 @@ import ( "github.com/ava-labs/coreth/plugin/evm/atomic" "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/utils" "github.com/ava-labs/libevm/common" "github.com/holiman/uint256" + "github.com/stretchr/testify/require" avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" "github.com/ava-labs/avalanchego/ids" @@ -35,7 +37,7 @@ func createImportTxOptions(t *testing.T, vm *VM, sharedMemory *avalancheatomic.M Amt: uint64(50000000), OutputOwners: secp256k1fx.OutputOwners{ Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, + Addrs: []ids.ShortID{vmtest.TestKeys[0].Address()}, }, }, } @@ -50,15 +52,15 @@ func createImportTxOptions(t *testing.T, vm *VM, sharedMemory *avalancheatomic.M Key: inputID[:], Value: utxoBytes, Traits: [][]byte{ - testKeys[0].Address().Bytes(), + vmtest.TestKeys[0].Address().Bytes(), }, }}}}); err != nil { t.Fatal(err) } importTxs := make([]*atomic.Tx, 0, 3) - for _, ethAddr := range testEthAddrs { - importTx, err := vm.newImportTx(vm.ctx.XChainID, ethAddr, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + for _, ethAddr := range vmtest.TestEthAddrs { + importTx, err := vm.newImportTx(vm.ctx.XChainID, ethAddr, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) if err != nil { t.Fatal(err) } @@ -107,12 +109,12 @@ func TestImportTxVerify(t *testing.T) { }, Outs: []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount - ap0.AtomicTxFee, AssetID: ctx.AVAXAssetID, }, { - Address: testEthAddrs[1], + Address: vmtest.TestEthAddrs[1], Amount: importAmount, AssetID: ctx.AVAXAssetID, }, @@ -130,7 +132,7 @@ func TestImportTxVerify(t *testing.T) { return importTx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrNilTx.Error(), }, "valid import tx": { @@ -138,7 +140,7 @@ func TestImportTxVerify(t *testing.T) { return importTx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", // Expect this transaction to be valid in Apricot Phase 0 }, "valid import tx banff": { @@ -146,7 +148,7 @@ func TestImportTxVerify(t *testing.T) { return importTx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: "", // Expect this transaction to be valid in Banff }, "invalid network ID": { @@ -156,7 +158,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongNetworkID.Error(), }, "invalid blockchain ID": { @@ -166,7 +168,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongChainID.Error(), }, "P-chain source before AP5": { @@ -176,7 +178,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrWrongChainID.Error(), }, "P-chain source after AP5": { @@ -186,7 +188,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase5, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), }, "invalid source chain ID": { generate: func(t *testing.T) atomic.UnsignedAtomicTx { @@ -195,7 +197,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase5, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase5), expectedErr: atomic.ErrWrongChainID.Error(), }, "no inputs": { @@ -205,7 +207,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrNoImportInputs.Error(), }, "inputs sorted incorrectly": { @@ -218,7 +220,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: atomic.ErrInputsNotSortedUnique.Error(), }, "invalid input": { @@ -231,7 +233,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "atomic input failed verification", }, "unsorted outputs phase 0 passes verification": { @@ -244,7 +246,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", }, "non-unique outputs phase 0 passes verification": { @@ -257,7 +259,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "", }, "unsorted outputs phase 1 fails verification": { @@ -270,7 +272,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase1, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase1), expectedErr: atomic.ErrOutputsNotSorted.Error(), }, "non-unique outputs phase 1 passes verification": { @@ -283,7 +285,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase1, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase1), expectedErr: "", }, "outputs not sorted and unique phase 2 fails verification": { @@ -296,7 +298,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase2, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase2), expectedErr: atomic.ErrOutputsNotSortedUnique.Error(), }, "outputs not sorted phase 2 fails verification": { @@ -309,7 +311,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase2, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase2), expectedErr: atomic.ErrOutputsNotSortedUnique.Error(), }, "invalid EVMOutput fails verification": { @@ -317,7 +319,7 @@ func TestImportTxVerify(t *testing.T) { tx := *importTx tx.Outs = []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: 0, AssetID: snowtest.AVAXAssetID, }, @@ -325,7 +327,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase0, + rules: vmtest.ForkToRules(upgradetest.NoUpgrades), expectedErr: "EVM Output failed verification", }, "no outputs apricot phase 3": { @@ -335,7 +337,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase3, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase3), expectedErr: atomic.ErrNoEVMOutputs.Error(), }, "non-AVAX input Apricot Phase 6": { @@ -359,7 +361,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase6, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase6), expectedErr: "", }, "non-AVAX output Apricot Phase 6": { @@ -375,7 +377,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: apricotRulesPhase6, + rules: vmtest.ForkToRules(upgradetest.ApricotPhase6), expectedErr: "", }, "non-AVAX input Banff": { @@ -399,7 +401,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: atomic.ErrImportNonAVAXInputBanff.Error(), }, "non-AVAX output Banff": { @@ -415,7 +417,7 @@ func TestImportTxVerify(t *testing.T) { return &tx }, ctx: ctx, - rules: banffRules, + rules: vmtest.ForkToRules(upgradetest.Banff), expectedErr: atomic.ErrImportNonAVAXOutputBanff.Error(), }, } @@ -428,16 +430,19 @@ func TestImportTxVerify(t *testing.T) { func TestNewImportTx(t *testing.T) { importAmount := uint64(5000000) + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddress := key.EthAddress() // createNewImportAVAXTx adds a UTXO to shared memory and then constructs a new import transaction // and checks that it has the correct fee for the base fee that has been used createNewImportAVAXTx := func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - _, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, importAmount, testShortIDAddrs[0]) + _, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, importAmount, key.Address()) if err != nil { t.Fatal(err) } - tx, err := vm.newImportTx(vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx, err := vm.newImportTx(vm.ctx.XChainID, ethAddress, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) if err != nil { t.Fatal(err) } @@ -454,7 +459,7 @@ func TestNewImportTx(t *testing.T) { if err != nil { t.Fatal(err) } - actualFee, err = atomic.CalculateDynamicFee(actualCost, initialBaseFee) + actualFee, err = atomic.CalculateDynamicFee(actualCost, vmtest.InitialBaseFee) if err != nil { t.Fatal(err) } @@ -471,7 +476,10 @@ func TestNewImportTx(t *testing.T) { return tx } checkState := func(t *testing.T, vm *VM) { - txs := vm.LastAcceptedBlockInternal().(*Block).atomicTxs + blk := vm.LastAcceptedVMBlock() + blockExtension, ok := blk.GetBlockExtension().(*blockExtension) + require.True(t, ok) + txs := blockExtension.atomicTxs if len(txs) != 1 { t.Fatalf("Expected one import tx to be in the last accepted block, but found %d", len(txs)) } @@ -484,7 +492,7 @@ func TestNewImportTx(t *testing.T) { // Ensure that the UTXO has been removed from shared memory within Accept addrSet := set.Set[ids.ShortID]{} - addrSet.Add(testShortIDAddrs[0]) + addrSet.Add(key.Address()) utxos, _, _, err := vm.GetAtomicUTXOs(vm.ctx.XChainID, addrSet, ids.ShortEmpty, ids.Empty, -1) if err != nil { t.Fatal(err) @@ -494,16 +502,15 @@ func TestNewImportTx(t *testing.T) { } // Ensure that the call to EVMStateTransfer correctly updates the balance of [addr] - sdb, err := vm.blockChain.State() + sdb, err := vm.Ethereum().BlockChain().State() if err != nil { t.Fatal(err) } expectedRemainingBalance := new(uint256.Int).Mul( uint256.NewInt(importAmount-actualAVAXBurned), atomic.X2CRate) - addr := testKeys[0].EthAddress() - if actualBalance := sdb.GetBalance(addr); actualBalance.Cmp(expectedRemainingBalance) != 0 { - t.Fatalf("address remaining balance %s equal %s not %s", addr.String(), actualBalance, expectedRemainingBalance) + if actualBalance := sdb.GetBalance(ethAddress); actualBalance.Cmp(expectedRemainingBalance) != 0 { + t.Fatalf("address remaining balance %s equal %s not %s", ethAddress.String(), actualBalance, expectedRemainingBalance) } } tests2 := map[string]atomicTxTest{ @@ -569,12 +576,12 @@ func TestImportTxGasCost(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: avaxAssetID, }}, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 1230, ExpectedFee: 30750, BaseFee: big.NewInt(25 * utils.GWei), @@ -593,12 +600,12 @@ func TestImportTxGasCost(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: avaxAssetID, }}, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 1230, ExpectedFee: 1, BaseFee: big.NewInt(1), @@ -617,12 +624,12 @@ func TestImportTxGasCost(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: avaxAssetID, }}, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}, ExpectedGasUsed: 11230, ExpectedFee: 1, BaseFee: big.NewInt(1), @@ -653,13 +660,13 @@ func TestImportTxGasCost(t *testing.T) { }, Outs: []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: antAssetID, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}, {testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}, {vmtest.TestKeys[0]}}, ExpectedGasUsed: 2318, ExpectedFee: 57950, BaseFee: big.NewInt(25 * utils.GWei), @@ -689,18 +696,18 @@ func TestImportTxGasCost(t *testing.T) { }, Outs: []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: avaxAssetID, }, { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: antAssetID, }, }, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0]}, {testKeys[0]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}, {vmtest.TestKeys[0]}}, ExpectedGasUsed: 2378, ExpectedFee: 59450, BaseFee: big.NewInt(25 * utils.GWei), @@ -719,12 +726,12 @@ func TestImportTxGasCost(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount, AssetID: avaxAssetID, }}, }, - Keys: [][]*secp256k1.PrivateKey{{testKeys[0], testKeys[1]}}, + Keys: [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0], vmtest.TestKeys[1]}}, ExpectedGasUsed: 2234, ExpectedFee: 55850, BaseFee: big.NewInt(25 * utils.GWei), @@ -818,23 +825,23 @@ func TestImportTxGasCost(t *testing.T) { }, Outs: []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: vmtest.TestEthAddrs[0], Amount: importAmount * 10, AssetID: avaxAssetID, }, }, }, Keys: [][]*secp256k1.PrivateKey{ - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, - {testKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, + {vmtest.TestKeys[0]}, }, ExpectedGasUsed: 11022, ExpectedFee: 275550, @@ -871,6 +878,9 @@ func TestImportTxGasCost(t *testing.T) { } func TestImportTxSemanticVerify(t *testing.T) { + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddress := key.PublicKey().EthAddress() tests := map[string]atomicTxTest{ "UTXO not present during bootstrapping": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { @@ -889,12 +899,12 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx @@ -918,12 +928,12 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx @@ -939,7 +949,7 @@ func TestImportTxSemanticVerify(t *testing.T) { Key: inputID[:], Value: []byte("hey there"), Traits: [][]byte{ - testShortIDAddrs[0].Bytes(), + key.Address().Bytes(), }, }}}}); err != nil { t.Fatal(err) @@ -958,12 +968,12 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx @@ -974,7 +984,7 @@ func TestImportTxSemanticVerify(t *testing.T) { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() expectedAssetID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, expectedAssetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, expectedAssetID, 1, key.Address()) if err != nil { t.Fatal(err) } @@ -992,22 +1002,22 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx }, - semanticVerifyErr: atomic.ErrAssetIDMismatch.Error(), + semanticVerifyErr: errAssetIDMismatch.Error(), }, "insufficient AVAX funds": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, key.Address()) if err != nil { t.Fatal(err) } @@ -1025,12 +1035,12 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 2, // Produce more output than is consumed by the transaction AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx @@ -1041,7 +1051,7 @@ func TestImportTxSemanticVerify(t *testing.T) { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() assetID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, assetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, assetID, 1, vmtest.TestShortIDAddrs[0]) if err != nil { t.Fatal(err) } @@ -1059,12 +1069,12 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 2, // Produce more output than is consumed by the transaction AssetID: assetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx @@ -1074,7 +1084,7 @@ func TestImportTxSemanticVerify(t *testing.T) { "no signatures": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, vmtest.TestShortIDAddrs[0]) if err != nil { t.Fatal(err) } @@ -1092,7 +1102,7 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, @@ -1107,7 +1117,7 @@ func TestImportTxSemanticVerify(t *testing.T) { "incorrect signature": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, vmtest.TestShortIDAddrs[0]) if err != nil { t.Fatal(err) } @@ -1125,13 +1135,13 @@ func TestImportTxSemanticVerify(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} // Sign the transaction with the incorrect key - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[1]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{vmtest.TestKeys[1]}}); err != nil { t.Fatal(err) } return tx @@ -1141,7 +1151,7 @@ func TestImportTxSemanticVerify(t *testing.T) { "non-unique EVM Outputs": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 2, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 2, vmtest.TestShortIDAddrs[0]) if err != nil { t.Fatal(err) } @@ -1160,18 +1170,18 @@ func TestImportTxSemanticVerify(t *testing.T) { }}, Outs: []atomic.EVMOutput{ { - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }, { - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }, }, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{vmtest.TestKeys[0]}}); err != nil { t.Fatal(err) } return tx @@ -1190,11 +1200,14 @@ func TestImportTxSemanticVerify(t *testing.T) { func TestImportTxEVMStateTransfer(t *testing.T) { assetID := ids.GenerateTestID() + key, err := secp256k1.NewPrivateKey() + require.NoError(t, err) + ethAddress := key.EthAddress() tests := map[string]atomicTxTest{ "AVAX UTXO": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, 1, key.Address()) if err != nil { t.Fatal(err) } @@ -1212,25 +1225,25 @@ func TestImportTxEVMStateTransfer(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: vm.ctx.AVAXAssetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx }, checkState: func(t *testing.T, vm *VM) { - lastAcceptedBlock := vm.LastAcceptedBlockInternal().(*Block) + lastAcceptedBlock := vm.LastAcceptedVMBlock() - sdb, err := vm.blockChain.StateAt(lastAcceptedBlock.ethBlock.Root()) + sdb, err := vm.Ethereum().BlockChain().StateAt(lastAcceptedBlock.GetEthBlock().Root()) if err != nil { t.Fatal(err) } - avaxBalance := sdb.GetBalance(testEthAddrs[0]) + avaxBalance := sdb.GetBalance(ethAddress) if avaxBalance.Cmp(atomic.X2CRate) != 0 { t.Fatalf("Expected AVAX balance to be %d, found balance: %d", *atomic.X2CRate, avaxBalance) } @@ -1239,7 +1252,7 @@ func TestImportTxEVMStateTransfer(t *testing.T) { "non-AVAX UTXO": { setup: func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) *atomic.Tx { txID := ids.GenerateTestID() - utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, assetID, 1, testShortIDAddrs[0]) + utxo, err := addUTXO(sharedMemory, vm.ctx, txID, 0, assetID, 1, key.Address()) if err != nil { t.Fatal(err) } @@ -1257,29 +1270,28 @@ func TestImportTxEVMStateTransfer(t *testing.T) { }, }}, Outs: []atomic.EVMOutput{{ - Address: testEthAddrs[0], + Address: ethAddress, Amount: 1, AssetID: assetID, }}, }} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{testKeys[0]}}); err != nil { + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key}}); err != nil { t.Fatal(err) } return tx }, checkState: func(t *testing.T, vm *VM) { - lastAcceptedBlock := vm.LastAcceptedBlockInternal().(*Block) - - sdb, err := vm.blockChain.StateAt(lastAcceptedBlock.ethBlock.Root()) + lastAcceptedBlock := vm.LastAcceptedVMBlock() + sdb, err := vm.Ethereum().BlockChain().StateAt(lastAcceptedBlock.GetEthBlock().Root()) if err != nil { t.Fatal(err) } - assetBalance := sdb.GetBalanceMultiCoin(testEthAddrs[0], common.Hash(assetID)) + assetBalance := sdb.GetBalanceMultiCoin(ethAddress, common.Hash(assetID)) if assetBalance.Cmp(common.Big1) != 0 { t.Fatalf("Expected asset balance to be %d, found balance: %d", common.Big1, assetBalance) } - avaxBalance := sdb.GetBalance(testEthAddrs[0]) + avaxBalance := sdb.GetBalance(ethAddress) if avaxBalance.Cmp(common.U2560) != 0 { t.Fatalf("Expected AVAX balance to be 0, found balance: %d", avaxBalance) } diff --git a/plugin/evm/mainnet_ext_data_hashes.json b/plugin/evm/atomic/vm/mainnet_ext_data_hashes.json similarity index 100% rename from plugin/evm/mainnet_ext_data_hashes.json rename to plugin/evm/atomic/vm/mainnet_ext_data_hashes.json diff --git a/plugin/evm/mempool_atomic_gossiping_test.go b/plugin/evm/atomic/vm/mempool_atomic_gossiping_test.go similarity index 69% rename from plugin/evm/mempool_atomic_gossiping_test.go rename to plugin/evm/atomic/vm/mempool_atomic_gossiping_test.go index aade07b741..64df4148af 100644 --- a/plugin/evm/mempool_atomic_gossiping_test.go +++ b/plugin/evm/atomic/vm/mempool_atomic_gossiping_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( "context" @@ -10,6 +10,9 @@ import ( "github.com/ava-labs/avalanchego/upgrade/upgradetest" "github.com/ava-labs/avalanchego/vms/components/chain" "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/vmtest" + atomictxpool "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" "github.com/stretchr/testify/assert" @@ -24,55 +27,61 @@ func TestMempoolAddLocallyCreateAtomicTx(t *testing.T) { // we use AP3 here to not trip any block fees fork := upgradetest.ApricotPhase3 - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - err := tvm.vm.Shutdown(context.Background()) + err := vm.Shutdown(context.Background()) assert.NoError(err) }() - mempool := tvm.vm.mempool + mempool := vm.mempool // generate a valid and conflicting tx var ( tx, conflictingTx *atomic.Tx ) if name == "import" { - importTxs := createImportTxOptions(t, tvm.vm, tvm.atomicMemory) + importTxs := createImportTxOptions(t, vm, tvm.AtomicMemory) tx, conflictingTx = importTxs[0], importTxs[1] } else { - exportTxs := createExportTxOptions(t, tvm.vm, tvm.toEngine, tvm.atomicMemory) + exportTxs := createExportTxOptions(t, vm, tvm.ToEngine, tvm.AtomicMemory) tx, conflictingTx = exportTxs[0], exportTxs[1] } txID := tx.ID() conflictingTxID := conflictingTx.ID() // add a tx to the mempool - err := tvm.vm.mempool.AddLocalTx(tx) + err := vm.mempool.AddLocalTx(tx) assert.NoError(err) has := mempool.Has(txID) assert.True(has, "valid tx not recorded into mempool") // try to add a conflicting tx - err = tvm.vm.mempool.AddLocalTx(conflictingTx) + err = vm.mempool.AddLocalTx(conflictingTx) assert.ErrorIs(err, atomictxpool.ErrConflictingAtomicTx) has = mempool.Has(conflictingTxID) assert.False(has, "conflicting tx in mempool") - <-tvm.toEngine + <-tvm.ToEngine has = mempool.Has(txID) assert.True(has, "valid tx not recorded into mempool") // Show that BuildBlock generates a block containing [txID] and that it is // still present in the mempool. - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) assert.NoError(err, "could not build block out of mempool") - evmBlk, ok := blk.(*chain.BlockWrapper).Block.(*Block) + wrappedBlock, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) assert.True(ok, "unknown block type") - assert.Equal(txID, evmBlk.atomicTxs[0].ID(), "block does not include expected transaction") + blockExtension, ok := wrappedBlock.GetBlockExtension().(*blockExtension) + assert.True(ok, "unknown block extension type") + + atomicTxs := blockExtension.atomicTxs + assert.NoError(err) + assert.Equal(txID, atomicTxs[0].ID(), "block does not include expected transaction") has = mempool.Has(txID) assert.True(has, "tx should stay in mempool until block is accepted") diff --git a/plugin/evm/atomic/vm/syncervm_test.go b/plugin/evm/atomic/vm/syncervm_test.go new file mode 100644 index 0000000000..9d10434ed7 --- /dev/null +++ b/plugin/evm/atomic/vm/syncervm_test.go @@ -0,0 +1,129 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "testing" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/utils/hashing" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/ava-labs/coreth/consensus/dummy" + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/atomic/atomictest" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/vmtest" + "github.com/ava-labs/coreth/predicate" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" + + "github.com/stretchr/testify/require" +) + +func TestAtomicSyncerVM(t *testing.T) { + importAmount := 2000000 * units.Avax // 2M avax + for _, test := range vmtest.SyncerVMTests { + includedAtomicTxs := make([]*atomic.Tx, 0) + + t.Run(test.Name, func(t *testing.T) { + genFn := func(i int, vm extension.InnerVM, gen *core.BlockGen) { + atomicVM, ok := vm.(*VM) + require.True(t, ok) + b, err := predicate.NewResults().Bytes() + if err != nil { + t.Fatal(err) + } + gen.AppendExtra(b) + switch i { + case 0: + // spend the UTXOs from shared memory + importTx, err := atomicVM.newImportTx(atomicVM.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, vmtest.TestKeys[0:1]) + require.NoError(t, err) + require.NoError(t, atomicVM.mempool.AddLocalTx(importTx)) + includedAtomicTxs = append(includedAtomicTxs, importTx) + case 1: + // export some of the imported UTXOs to test exportTx is properly synced + state, err := vm.Ethereum().BlockChain().State() + if err != nil { + t.Fatal(err) + } + exportTx, err := atomic.NewExportTx( + atomicVM.ctx, + atomicVM.currentRules(), + state, + atomicVM.ctx.AVAXAssetID, + importAmount/2, + atomicVM.ctx.XChainID, + vmtest.TestShortIDAddrs[0], + vmtest.InitialBaseFee, + vmtest.TestKeys[0:1], + ) + require.NoError(t, err) + require.NoError(t, atomicVM.mempool.AddLocalTx(exportTx)) + includedAtomicTxs = append(includedAtomicTxs, exportTx) + default: // Generate simple transfer transactions. + pk := vmtest.TestKeys[0].ToECDSA() + tx := types.NewTransaction(gen.TxNonce(vmtest.TestEthAddrs[0]), vmtest.TestEthAddrs[1], common.Big1, params.TxGas, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), pk) + require.NoError(t, err) + gen.AddTx(signedTx) + } + } + newVMFn := func() (extension.InnerVM, dummy.ConsensusCallbacks) { + vm := newAtomicTestVM() + return vm, vm.createConsensusCallbacks() + } + + afterInit := func(t *testing.T, params vmtest.SyncTestParams, vmSetup vmtest.SyncVMSetup, isServer bool) { + atomicVM, ok := vmSetup.VM.(*VM) + require.True(t, ok) + + alloc := map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + } + + for addr, avaxAmount := range alloc { + txID, err := ids.ToID(hashing.ComputeHash256(addr.Bytes())) + if err != nil { + t.Fatalf("Failed to generate txID from addr: %s", err) + } + if _, err := addUTXO(vmSetup.AtomicMemory, vmSetup.SnowCtx, txID, 0, vmSetup.SnowCtx.AVAXAssetID, avaxAmount, addr); err != nil { + t.Fatalf("Failed to add UTXO to shared memory: %s", err) + } + } + if isServer { + serverAtomicTrie := atomicVM.atomicBackend.AtomicTrie() + // Calling AcceptTrie with SyncableInterval creates a commit for the atomic trie + committed, err := serverAtomicTrie.AcceptTrie(params.SyncableInterval, serverAtomicTrie.LastAcceptedRoot()) + require.NoError(t, err) + require.True(t, committed) + require.NoError(t, atomicVM.VersionDB().Commit()) + } + } + + testSetup := &vmtest.SyncTestSetup{ + NewVM: newVMFn, + GenFn: genFn, + AfterInit: afterInit, + ExtraSyncerVMTest: func(t *testing.T, syncerVMSetup vmtest.SyncVMSetup) { + // check atomic memory was synced properly + syncerVM := syncerVMSetup.VM + atomicVM, ok := syncerVM.(*VM) + require.True(t, ok) + syncerSharedMemories := atomictest.NewSharedMemories(syncerVMSetup.AtomicMemory, atomicVM.ctx.ChainID, atomicVM.ctx.XChainID) + + for _, tx := range includedAtomicTxs { + atomicOps, err := atomictest.ConvertToAtomicOps(tx) + require.NoError(t, err) + syncerSharedMemories.AssertOpsApplied(t, atomicOps) + } + }, + } + test.TestFunc(t, testSetup) + }) + } +} diff --git a/plugin/evm/atomic/vm/tx_gossip_test.go b/plugin/evm/atomic/vm/tx_gossip_test.go new file mode 100644 index 0000000000..b2d521e0d7 --- /dev/null +++ b/plugin/evm/atomic/vm/tx_gossip_test.go @@ -0,0 +1,316 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "context" + "encoding/binary" + "sync" + "testing" + "time" + + avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" + "github.com/ava-labs/avalanchego/database/memdb" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/network/p2p/gossip" + "github.com/ava-labs/avalanchego/proto/pb/sdk" + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/engine/enginetest" + "github.com/ava-labs/avalanchego/snow/snowtest" + "github.com/ava-labs/avalanchego/snow/validators" + agoUtils "github.com/ava-labs/avalanchego/utils" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/prometheus/client_golang/prometheus" + "github.com/stretchr/testify/require" + + "github.com/ava-labs/avalanchego/vms/components/avax" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + + "google.golang.org/protobuf/proto" + + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/config" + "github.com/ava-labs/coreth/plugin/evm/vmtest" + "github.com/ava-labs/coreth/utils" +) + +func TestAtomicTxGossip(t *testing.T) { + require := require.New(t) + ctx := context.Background() + snowCtx := snowtest.Context(t, snowtest.CChainID) + snowCtx.AVAXAssetID = ids.GenerateTestID() + validatorState := utils.NewTestValidatorState() + snowCtx.ValidatorState = validatorState + memory := avalancheatomic.NewMemory(memdb.New()) + snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) + + pk, err := secp256k1.NewPrivateKey() + require.NoError(err) + address := pk.EthAddress() + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) + genesisBytes, err := genesis.MarshalJSON() + require.NoError(err) + + responseSender := &enginetest.SenderStub{ + SentAppResponse: make(chan []byte, 1), + } + vm := newAtomicTestVM() + + require.NoError(vm.Initialize( + ctx, + snowCtx, + memdb.New(), + genesisBytes, + nil, + nil, + make(chan common.Message), + nil, + responseSender, + )) + require.NoError(vm.SetState(ctx, snow.NormalOp)) + + defer func() { + require.NoError(vm.Shutdown(ctx)) + }() + + // sender for the peer requesting gossip from [vm] + peerSender := &enginetest.SenderStub{ + SentAppRequest: make(chan []byte, 1), + } + network, err := p2p.NewNetwork(logging.NoLog{}, peerSender, prometheus.NewRegistry(), "") + require.NoError(err) + client := network.NewClient(p2p.AtomicTxGossipHandlerID) + + // we only accept gossip requests from validators + requestingNodeID := ids.GenerateTestNodeID() + require.NoError(vm.Connected(ctx, requestingNodeID, nil)) + validatorState.GetCurrentHeightF = func(context.Context) (uint64, error) { + return 0, nil + } + validatorState.GetValidatorSetF = func(context.Context, uint64, ids.ID) (map[ids.NodeID]*validators.GetValidatorOutput, error) { + return map[ids.NodeID]*validators.GetValidatorOutput{ + requestingNodeID: { + NodeID: requestingNodeID, + Weight: 1, + }, + }, nil + } + + // Ask the VM for any new transactions. We should get nothing at first. + emptyBloomFilter, err := gossip.NewBloomFilter( + prometheus.NewRegistry(), + "", + config.TxGossipBloomMinTargetElements, + config.TxGossipBloomTargetFalsePositiveRate, + config.TxGossipBloomResetFalsePositiveRate, + ) + require.NoError(err) + emptyBloomFilterBytes, _ := emptyBloomFilter.Marshal() + request := &sdk.PullGossipRequest{ + Filter: emptyBloomFilterBytes, + Salt: agoUtils.RandomBytes(32), + } + + requestBytes, err := proto.Marshal(request) + require.NoError(err) + + wg := &sync.WaitGroup{} + wg.Add(1) + onResponse := func(_ context.Context, nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := &sdk.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Empty(response.Gossip) + wg.Done() + } + require.NoError(client.AppRequest(ctx, set.Of(vm.ctx.NodeID), requestBytes, onResponse)) + require.NoError(vm.AppRequest(ctx, requestingNodeID, 1, time.Time{}, <-peerSender.SentAppRequest)) + require.NoError(network.AppResponse(ctx, snowCtx.NodeID, 1, <-responseSender.SentAppResponse)) + wg.Wait() + + // Issue a tx to the VM + utxo, err := addUTXO( + memory, + snowCtx, + ids.GenerateTestID(), + 0, + snowCtx.AVAXAssetID, + 100_000_000_000, + pk.Address(), + ) + require.NoError(err) + tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, vmtest.InitialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) + require.NoError(err) + require.NoError(vm.mempool.AddLocalTx(tx)) + + // wait so we aren't throttled by the vm + time.Sleep(5 * time.Second) + + // Ask the VM for new transactions. We should get the newly issued tx. + wg.Add(1) + + marshaller := atomic.TxMarshaller{} + onResponse = func(_ context.Context, nodeID ids.NodeID, responseBytes []byte, err error) { + require.NoError(err) + + response := &sdk.PullGossipResponse{} + require.NoError(proto.Unmarshal(responseBytes, response)) + require.Len(response.Gossip, 1) + + gotTx, err := marshaller.UnmarshalGossip(response.Gossip[0]) + require.NoError(err) + require.Equal(tx.ID(), gotTx.GossipID()) + + wg.Done() + } + require.NoError(client.AppRequest(ctx, set.Of(vm.ctx.NodeID), requestBytes, onResponse)) + require.NoError(vm.AppRequest(ctx, requestingNodeID, 3, time.Time{}, <-peerSender.SentAppRequest)) + require.NoError(network.AppResponse(ctx, snowCtx.NodeID, 3, <-responseSender.SentAppResponse)) + wg.Wait() +} + +// Tests that a tx is gossiped when it is issued +func TestAtomicTxPushGossipOutbound(t *testing.T) { + require := require.New(t) + ctx := context.Background() + snowCtx := snowtest.Context(t, snowtest.CChainID) + snowCtx.AVAXAssetID = ids.GenerateTestID() + validatorState := utils.NewTestValidatorState() + snowCtx.ValidatorState = validatorState + memory := avalancheatomic.NewMemory(memdb.New()) + snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) + + pk, err := secp256k1.NewPrivateKey() + require.NoError(err) + address := pk.EthAddress() + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) + genesisBytes, err := genesis.MarshalJSON() + require.NoError(err) + + sender := &enginetest.SenderStub{ + SentAppGossip: make(chan []byte, 1), + } + vm := newAtomicTestVM() + vm.atomicTxPullGossiper = gossip.NoOpGossiper{} + + require.NoError(vm.Initialize( + ctx, + snowCtx, + memdb.New(), + genesisBytes, + nil, + nil, + make(chan common.Message), + nil, + sender, + )) + require.NoError(vm.SetState(ctx, snow.NormalOp)) + + defer func() { + require.NoError(vm.Shutdown(ctx)) + }() + + // Issue a tx to the VM + utxo, err := addUTXO( + memory, + snowCtx, + ids.GenerateTestID(), + 0, + snowCtx.AVAXAssetID, + 100_000_000_000, + pk.Address(), + ) + require.NoError(err) + tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, vmtest.InitialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) + require.NoError(err) + require.NoError(vm.mempool.AddLocalTx(tx)) + vm.atomicTxPushGossiper.Add(tx) + + gossipedBytes := <-sender.SentAppGossip + require.Equal(byte(p2p.AtomicTxGossipHandlerID), gossipedBytes[0]) + + outboundGossipMsg := &sdk.PushGossip{} + require.NoError(proto.Unmarshal(gossipedBytes[1:], outboundGossipMsg)) + require.Len(outboundGossipMsg.Gossip, 1) + + marshaller := atomic.TxMarshaller{} + gossipedTx, err := marshaller.UnmarshalGossip(outboundGossipMsg.Gossip[0]) + require.NoError(err) + require.Equal(tx.ID(), gossipedTx.ID()) +} + +// Tests that a tx is gossiped when it is issued +func TestAtomicTxPushGossipInbound(t *testing.T) { + require := require.New(t) + ctx := context.Background() + snowCtx := snowtest.Context(t, snowtest.CChainID) + snowCtx.AVAXAssetID = ids.GenerateTestID() + validatorState := utils.NewTestValidatorState() + snowCtx.ValidatorState = validatorState + memory := avalancheatomic.NewMemory(memdb.New()) + snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) + + pk, err := secp256k1.NewPrivateKey() + require.NoError(err) + address := pk.EthAddress() + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) + genesisBytes, err := genesis.MarshalJSON() + require.NoError(err) + + sender := &enginetest.Sender{} + vm := newAtomicTestVM() + vm.atomicTxPullGossiper = gossip.NoOpGossiper{} + + require.NoError(vm.Initialize( + ctx, + snowCtx, + memdb.New(), + genesisBytes, + nil, + nil, + make(chan common.Message), + nil, + sender, + )) + require.NoError(vm.SetState(ctx, snow.NormalOp)) + + defer func() { + require.NoError(vm.Shutdown(ctx)) + }() + + // issue a tx to the vm + utxo, err := addUTXO( + memory, + snowCtx, + ids.GenerateTestID(), + 0, + snowCtx.AVAXAssetID, + 100_000_000_000, + pk.Address(), + ) + require.NoError(err) + tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, vmtest.InitialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) + require.NoError(err) + require.NoError(vm.mempool.AddLocalTx(tx)) + + marshaller := atomic.TxMarshaller{} + gossipBytes, err := marshaller.MarshalGossip(tx) + require.NoError(err) + + inboundGossip := &sdk.PushGossip{ + Gossip: [][]byte{gossipBytes}, + } + inboundGossipBytes, err := proto.Marshal(inboundGossip) + require.NoError(err) + + inboundGossipMsg := append(binary.AppendUvarint(nil, p2p.AtomicTxGossipHandlerID), inboundGossipBytes...) + + require.NoError(vm.AppGossip(ctx, ids.EmptyNodeID, inboundGossipMsg)) + require.True(vm.mempool.Has(tx.ID())) +} diff --git a/plugin/evm/atomic/vm/tx_semantic_verifier.go b/plugin/evm/atomic/vm/tx_semantic_verifier.go new file mode 100644 index 0000000000..374ffd7752 --- /dev/null +++ b/plugin/evm/atomic/vm/tx_semantic_verifier.go @@ -0,0 +1,243 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "context" + "errors" + "fmt" + "math/big" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/vms/components/avax" + "github.com/ava-labs/avalanchego/vms/platformvm/fx" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" +) + +var _ atomic.Visitor = (*semanticVerifier)(nil) + +var ( + errAssetIDMismatch = errors.New("asset IDs in the input don't match the utxo") + errConflictingAtomicInputs = errors.New("invalid block due to conflicting atomic inputs") + errRejectedParent = errors.New("rejected parent") + errPublicKeySignatureMismatch = errors.New("signature doesn't match public key") +) + +type BlockFetcher interface { + LastAcceptedVMBlock() extension.VMBlock + GetVMBlock(context.Context, ids.ID) (extension.VMBlock, error) +} + +type verifierBackend struct { + ctx *snow.Context + fx fx.Fx + rules extras.Rules + bootstrapped bool + blockFetcher BlockFetcher + secpCache *secp256k1.RecoverCache +} + +// semanticVerifier is a visitor that checks the semantic validity of atomic transactions. +type semanticVerifier struct { + backend *verifierBackend + tx *atomic.Tx + parent extension.VMBlock + baseFee *big.Int +} + +// ImportTx verifies this transaction is valid. +func (s *semanticVerifier) ImportTx(utx *atomic.UnsignedImportTx) error { + backend := s.backend + ctx := backend.ctx + rules := backend.rules + stx := s.tx + if err := utx.Verify(ctx, rules); err != nil { + return err + } + + // Check the transaction consumes and produces the right amounts + fc := avax.NewFlowChecker() + switch { + // Apply dynamic fees to import transactions as of Apricot Phase 3 + case rules.IsApricotPhase3: + gasUsed, err := stx.GasUsed(rules.IsApricotPhase5) + if err != nil { + return err + } + txFee, err := atomic.CalculateDynamicFee(gasUsed, s.baseFee) + if err != nil { + return err + } + fc.Produce(ctx.AVAXAssetID, txFee) + + // Apply fees to import transactions as of Apricot Phase 2 + case rules.IsApricotPhase2: + fc.Produce(ctx.AVAXAssetID, ap0.AtomicTxFee) + } + for _, out := range utx.Outs { + fc.Produce(out.AssetID, out.Amount) + } + for _, in := range utx.ImportedInputs { + fc.Consume(in.AssetID(), in.Input().Amount()) + } + + if err := fc.Verify(); err != nil { + return fmt.Errorf("import tx flow check failed due to: %w", err) + } + + if len(stx.Creds) != len(utx.ImportedInputs) { + return fmt.Errorf("import tx contained mismatched number of inputs/credentials (%d vs. %d)", len(utx.ImportedInputs), len(stx.Creds)) + } + + if !backend.bootstrapped { + // Allow for force committing during bootstrapping + return nil + } + + utxoIDs := make([][]byte, len(utx.ImportedInputs)) + for i, in := range utx.ImportedInputs { + inputID := in.UTXOID.InputID() + utxoIDs[i] = inputID[:] + } + // allUTXOBytes is guaranteed to be the same length as utxoIDs + allUTXOBytes, err := ctx.SharedMemory.Get(utx.SourceChain, utxoIDs) + if err != nil { + return fmt.Errorf("failed to fetch import UTXOs from %s due to: %w", utx.SourceChain, err) + } + + for i, in := range utx.ImportedInputs { + utxoBytes := allUTXOBytes[i] + + utxo := &avax.UTXO{} + if _, err := atomic.Codec.Unmarshal(utxoBytes, utxo); err != nil { + return fmt.Errorf("failed to unmarshal UTXO: %w", err) + } + + cred := stx.Creds[i] + + utxoAssetID := utxo.AssetID() + inAssetID := in.AssetID() + if utxoAssetID != inAssetID { + return errAssetIDMismatch + } + + if err := backend.fx.VerifyTransfer(utx, in.In, cred, utxo.Out); err != nil { + return fmt.Errorf("import tx transfer failed verification: %w", err) + } + } + + return conflicts(backend, utx.InputUTXOs(), s.parent) +} + +// conflicts returns an error if [inputs] conflicts with any of the atomic inputs contained in [ancestor] +// or any of its ancestor blocks going back to the last accepted block in its ancestry. If [ancestor] is +// accepted, then nil will be returned immediately. +// If the ancestry of [ancestor] cannot be fetched, then [errRejectedParent] may be returned. +func conflicts(backend *verifierBackend, inputs set.Set[ids.ID], ancestor extension.VMBlock) error { + lastAcceptedBlock := backend.blockFetcher.LastAcceptedVMBlock() + lastAcceptedHeight := lastAcceptedBlock.Height() + for ancestor.Height() > lastAcceptedHeight { + blockExtension, ok := ancestor.GetBlockExtension().(*blockExtension) + if !ok { + return fmt.Errorf("expected block extension to be of type *blockExtension but got %T", ancestor.GetBlockExtension()) + } + // If any of the atomic transactions in the ancestor conflict with [inputs] + // return an error. + for _, atomicTx := range blockExtension.atomicTxs { + if inputs.Overlaps(atomicTx.InputUTXOs()) { + return errConflictingAtomicInputs + } + } + + // Move up the chain. + nextAncestorID := ancestor.Parent() + // If the ancestor is unknown, then the parent failed + // verification when it was called. + // If the ancestor is rejected, then this block shouldn't be + // inserted into the canonical chain because the parent is + // will be missing. + // If the ancestor is processing, then the block may have + // been verified. + nextAncestor, err := backend.blockFetcher.GetVMBlock(context.TODO(), nextAncestorID) + if err != nil { + return errRejectedParent + } + ancestor = nextAncestor + } + + return nil +} + +// ExportTx verifies this transaction is valid. +func (s *semanticVerifier) ExportTx(utx *atomic.UnsignedExportTx) error { + ctx := s.backend.ctx + rules := s.backend.rules + stx := s.tx + if err := utx.Verify(ctx, rules); err != nil { + return err + } + + // Check the transaction consumes and produces the right amounts + fc := avax.NewFlowChecker() + switch { + // Apply dynamic fees to export transactions as of Apricot Phase 3 + case rules.IsApricotPhase3: + gasUsed, err := stx.GasUsed(rules.IsApricotPhase5) + if err != nil { + return err + } + txFee, err := atomic.CalculateDynamicFee(gasUsed, s.baseFee) + if err != nil { + return err + } + fc.Produce(ctx.AVAXAssetID, txFee) + // Apply fees to export transactions before Apricot Phase 3 + default: + fc.Produce(ctx.AVAXAssetID, ap0.AtomicTxFee) + } + for _, out := range utx.ExportedOutputs { + fc.Produce(out.AssetID(), out.Output().Amount()) + } + for _, in := range utx.Ins { + fc.Consume(in.AssetID, in.Amount) + } + + if err := fc.Verify(); err != nil { + return fmt.Errorf("export tx flow check failed due to: %w", err) + } + + if len(utx.Ins) != len(stx.Creds) { + return fmt.Errorf("export tx contained mismatched number of inputs/credentials (%d vs. %d)", len(utx.Ins), len(stx.Creds)) + } + + for i, input := range utx.Ins { + cred, ok := stx.Creds[i].(*secp256k1fx.Credential) + if !ok { + return fmt.Errorf("expected *secp256k1fx.Credential but got %T", cred) + } + if err := cred.Verify(); err != nil { + return err + } + + if len(cred.Sigs) != 1 { + return fmt.Errorf("expected one signature for EVM Input Credential, but found: %d", len(cred.Sigs)) + } + pubKey, err := s.backend.secpCache.RecoverPublicKey(utx.Bytes(), cred.Sigs[0][:]) + if err != nil { + return err + } + if input.Address != pubKey.EthAddress() { + return errPublicKeySignatureMismatch + } + } + + return nil +} diff --git a/plugin/evm/tx_test.go b/plugin/evm/atomic/vm/tx_test.go similarity index 83% rename from plugin/evm/tx_test.go rename to plugin/evm/atomic/vm/tx_test.go index 6017f69205..6bcaccbe8e 100644 --- a/plugin/evm/tx_test.go +++ b/plugin/evm/atomic/vm/tx_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package vm import ( "context" @@ -15,6 +15,7 @@ import ( "github.com/ava-labs/coreth/params/extras" "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/utils" avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" @@ -63,7 +64,7 @@ func TestCalculateDynamicFee(t *testing.T) { type atomicTxVerifyTest struct { ctx *snow.Context generate func(t *testing.T) atomic.UnsignedAtomicTx - rules extras.Rules + rules *extras.Rules expectedErr string } @@ -71,7 +72,7 @@ type atomicTxVerifyTest struct { func executeTxVerifyTest(t *testing.T, test atomicTxVerifyTest) { require := require.New(t) atomicTx := test.generate(t) - err := atomicTx.Verify(test.ctx, test.rules) + err := atomicTx.Verify(test.ctx, *test.rules) if len(test.expectedErr) == 0 { require.NoError(err) } else { @@ -98,36 +99,38 @@ type atomicTxTest struct { } func executeTxTest(t *testing.T, test atomicTxTest) { - tvm := newVM(t, testVMConfig{ - isSyncing: test.bootstrapping, - fork: &test.fork, + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + IsSyncing: test.bootstrapping, + Fork: &test.fork, }) - rules := tvm.vm.currentRules() + rules := vm.currentRules() - tx := test.setup(t, tvm.vm, tvm.atomicMemory) + tx := test.setup(t, vm, tvm.AtomicMemory) var baseFee *big.Int // If ApricotPhase3 is active, use the initial base fee for the atomic transaction switch { case rules.IsApricotPhase3: - baseFee = initialBaseFee + baseFee = vmtest.InitialBaseFee } - lastAcceptedBlock := tvm.vm.LastAcceptedBlockInternal().(*Block) - backend := &atomic.VerifierBackend{ - Ctx: tvm.vm.ctx, - Fx: &tvm.vm.fx, - Rules: rules, - Bootstrapped: tvm.vm.bootstrapped.Get(), - BlockFetcher: tvm.vm, - SecpCache: tvm.vm.secpCache, + lastAcceptedBlock := vm.LastAcceptedVMBlock() + backend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: rules, + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, } - if err := tx.UnsignedAtomicTx.Visit(&atomic.SemanticVerifier{ - Backend: backend, - Tx: tx, - Parent: lastAcceptedBlock, - BaseFee: baseFee, - }); len(test.semanticVerifyErr) == 0 && err != nil { + if err := tx.UnsignedAtomicTx.Visit( + &semanticVerifier{ + backend: backend, + tx: tx, + parent: lastAcceptedBlock, + baseFee: baseFee, + }); len(test.semanticVerifyErr) == 0 && err != nil { t.Fatalf("SemanticVerify failed unexpectedly due to: %s", err) } else if len(test.semanticVerifyErr) != 0 { if err == nil { @@ -141,11 +144,11 @@ func executeTxTest(t *testing.T, test atomicTxTest) { } // Retrieve dummy state to test that EVMStateTransfer works correctly - sdb, err := tvm.vm.blockChain.StateAt(lastAcceptedBlock.ethBlock.Root()) + sdb, err := vm.Ethereum().BlockChain().StateAt(lastAcceptedBlock.GetEthBlock().Root()) if err != nil { t.Fatal(err) } - if err := tx.UnsignedAtomicTx.EVMStateTransfer(tvm.vm.ctx, sdb); len(test.evmStateTransferErr) == 0 && err != nil { + if err := tx.UnsignedAtomicTx.EVMStateTransfer(vm.ctx, sdb); len(test.evmStateTransferErr) == 0 && err != nil { t.Fatalf("EVMStateTransfer failed unexpectedly due to: %s", err) } else if len(test.evmStateTransferErr) != 0 { if err == nil { @@ -158,22 +161,17 @@ func executeTxTest(t *testing.T, test atomicTxTest) { return } - if test.bootstrapping { - // If this test simulates processing txs during bootstrapping (where some verification is skipped), - // initialize the block building goroutines normally initialized in SetState(snow.NormalOps). - // This ensures that the VM can build a block correctly during the test. - if err := tvm.vm.initBlockBuilding(); err != nil { - t.Fatal(err) - } + if err := vm.mempool.AddLocalTx(tx); err != nil { + t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(tx); err != nil { - t.Fatal(err) + if test.bootstrapping { + return } - <-tvm.toEngine + <-tvm.ToEngine // If we've reached this point, we expect to be able to build and verify the block without any errors - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -196,7 +194,7 @@ func executeTxTest(t *testing.T, test atomicTxTest) { } if test.checkState != nil { - test.checkState(t, tvm.vm) + test.checkState(t, vm) } } diff --git a/plugin/evm/atomic/vm/vm.go b/plugin/evm/atomic/vm/vm.go new file mode 100644 index 0000000000..4c91289607 --- /dev/null +++ b/plugin/evm/atomic/vm/vm.go @@ -0,0 +1,826 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "context" + "errors" + "fmt" + "math/big" + "net/http" + "sync" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/codec/linearcodec" + avalanchedatabase "github.com/ava-labs/avalanchego/database" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + avalanchegossip "github.com/ava-labs/avalanchego/network/p2p/gossip" + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/snow/consensus/snowman" + avalanchecommon "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/avalanchego/utils/constants" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/logging" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/utils/timer/mockable" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/ava-labs/avalanchego/vms/components/avax" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + + "github.com/ava-labs/coreth/consensus/dummy" + "github.com/ava-labs/coreth/core/state" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/atomic" + atomicstate "github.com/ava-labs/coreth/plugin/evm/atomic/state" + atomicsync "github.com/ava-labs/coreth/plugin/evm/atomic/sync" + "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" + "github.com/ava-labs/coreth/plugin/evm/config" + "github.com/ava-labs/coreth/plugin/evm/customtypes" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/gossip" + customheader "github.com/ava-labs/coreth/plugin/evm/header" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap5" + "github.com/ava-labs/coreth/plugin/evm/vmerrors" + "github.com/ava-labs/coreth/utils" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" + "github.com/ava-labs/libevm/log" +) + +var ( + _ secp256k1fx.VM = (*VM)(nil) + _ block.ChainVM = (*VM)(nil) + _ block.BuildBlockWithContextChainVM = (*VM)(nil) + _ block.StateSyncableVM = (*VM)(nil) +) + +const ( + secpCacheSize = 1024 + defaultMempoolSize = 4096 + targetAtomicTxsSize = 40 * units.KiB + // maxAtomicTxMempoolGas is the maximum amount of gas that is allowed to be + // used by an atomic transaction in the mempool. It is allowed to build + // blocks with larger atomic transactions, but they will not be accepted + // into the mempool. + maxAtomicTxMempoolGas = ap5.AtomicGasLimit + maxUTXOsToFetch = 1024 + atomicTxGossipNamespace = "atomic_tx_gossip" + avaxEndpoint = "/avax" +) + +type VM struct { + // TODO: decide if we want to directly import the evm package and VM struct + extension.InnerVM + + secpCache *secp256k1.RecoverCache + baseCodec codec.Registry + mempool *txpool.Mempool + fx secp256k1fx.Fx + ctx *snow.Context + + // [atomicTxRepository] maintains two indexes on accepted atomic txs. + // - txID to accepted atomic tx + // - block height to list of atomic txs accepted on block at that height + atomicTxRepository *atomicstate.AtomicRepository + // [atomicBackend] abstracts verification and processing of atomic transactions + atomicBackend *atomicstate.AtomicBackend + + atomicTxGossipHandler p2p.Handler + atomicTxPushGossiper *avalanchegossip.PushGossiper[*atomic.Tx] + atomicTxPullGossiper avalanchegossip.Gossiper + + // [cancel] may be nil until [snow.NormalOp] starts + cancel context.CancelFunc + shutdownWg sync.WaitGroup + + clock mockable.Clock +} + +func WrapVM(vm extension.InnerVM) *VM { + return &VM{InnerVM: vm} +} + +// Initialize implements the snowman.ChainVM interface +func (vm *VM) Initialize( + ctx context.Context, + chainCtx *snow.Context, + db avalanchedatabase.Database, + genesisBytes []byte, + upgradeBytes []byte, + configBytes []byte, + toEngine chan<- avalanchecommon.Message, + fxs []*avalanchecommon.Fx, + appSender avalanchecommon.AppSender, +) error { + vm.ctx = chainCtx + + var extDataHashes map[common.Hash]common.Hash + // Set the chain config for mainnet/fuji chain IDs + switch chainCtx.NetworkID { + case constants.MainnetID: + extDataHashes = mainnetExtDataHashes + case constants.FujiID: + extDataHashes = fujiExtDataHashes + } + // Free the memory of the extDataHash map + fujiExtDataHashes = nil + mainnetExtDataHashes = nil + + networkCodec, err := message.NewCodec(atomicsync.AtomicSyncSummary{}) + if err != nil { + return fmt.Errorf("failed to create codec manager: %w", err) + } + + // Create the atomic extension structs + // some of them need to be initialized after the inner VM is initialized + blockExtender := newBlockExtender(extDataHashes, vm) + syncExtender := &atomicsync.AtomicSyncExtender{} + syncProvider := &atomicsync.AtomicSummaryProvider{} + // Create and pass the leaf handler to the atomic extension + // it will be initialized after the inner VM is initialized + leafHandler := atomicsync.NewAtomicLeafHandler() + atomicLeafTypeConfig := &extension.LeafRequestConfig{ + LeafType: atomicsync.AtomicTrieNode, + MetricName: "sync_atomic_trie_leaves", + Handler: leafHandler, + } + vm.mempool = &txpool.Mempool{} + + extensionConfig := &extension.Config{ + NetworkCodec: networkCodec, + ConsensusCallbacks: vm.createConsensusCallbacks(), + BlockExtender: blockExtender, + SyncableParser: atomicsync.NewAtomicSyncSummaryParser(), + SyncExtender: syncExtender, + SyncSummaryProvider: syncProvider, + ExtraSyncLeafHandlerConfig: atomicLeafTypeConfig, + ExtraMempool: vm.mempool, + Clock: &vm.clock, + } + if err := vm.SetExtensionConfig(extensionConfig); err != nil { + return fmt.Errorf("failed to set extension config: %w", err) + } + + // Initialize inner vm with the provided parameters + if err := vm.InnerVM.Initialize( + ctx, + chainCtx, + db, + genesisBytes, + upgradeBytes, + configBytes, + toEngine, + fxs, + appSender, + ); err != nil { + return fmt.Errorf("failed to initialize inner VM: %w", err) + } + + // Now we can initialize the mempool and so + err = vm.mempool.Initialize(chainCtx, vm.MetricRegistry(), defaultMempoolSize, vm.verifyTxAtTip) + if err != nil { + return fmt.Errorf("failed to initialize mempool: %w", err) + } + + // initialize bonus blocks on mainnet + var ( + bonusBlockHeights map[uint64]ids.ID + ) + if vm.ctx.NetworkID == constants.MainnetID { + bonusBlockHeights, err = readMainnetBonusBlocks() + if err != nil { + return fmt.Errorf("failed to read mainnet bonus blocks: %w", err) + } + } + + // initialize atomic repository + lastAcceptedHash, lastAcceptedHeight, err := vm.ReadLastAccepted() + if err != nil { + return fmt.Errorf("failed to read last accepted block: %w", err) + } + vm.atomicTxRepository, err = atomicstate.NewAtomicTxRepository(vm.VersionDB(), atomic.Codec, lastAcceptedHeight) + if err != nil { + return fmt.Errorf("failed to create atomic repository: %w", err) + } + vm.atomicBackend, err = atomicstate.NewAtomicBackend( + vm.ctx.SharedMemory, bonusBlockHeights, + vm.atomicTxRepository, lastAcceptedHeight, lastAcceptedHash, + vm.Config().CommitInterval, + ) + if err != nil { + return fmt.Errorf("failed to create atomic backend: %w", err) + } + + // Atomic backend is available now, we can initialize structs that depend on it + syncProvider.Initialize(vm.atomicBackend.AtomicTrie()) + syncExtender.Initialize(vm.atomicBackend, vm.atomicBackend.AtomicTrie(), vm.Config().StateSyncRequestSize) + leafHandler.Initialize(vm.atomicBackend.AtomicTrie().TrieDB(), atomicstate.TrieKeyLength, networkCodec) + vm.secpCache = secp256k1.NewRecoverCache(secpCacheSize) + + // so [vm.baseCodec] is a dummy codec use to fulfill the secp256k1fx VM + // interface. The fx will register all of its types, which can be safely + // ignored by the VM's codec. + vm.baseCodec = linearcodec.NewDefault() + return vm.fx.Initialize(vm) +} + +func (vm *VM) SetState(ctx context.Context, state snow.State) error { + switch state { + case snow.Bootstrapping: + if err := vm.onBootstrapStarted(); err != nil { + return err + } + case snow.NormalOp: + if err := vm.onNormalOperationsStarted(); err != nil { + return err + } + } + + return vm.InnerVM.SetState(ctx, state) +} + +func (vm *VM) onBootstrapStarted() error { + return vm.fx.Bootstrapping() +} + +func (vm *VM) onNormalOperationsStarted() error { + if vm.IsBootstrapped() { + return nil + } + if err := vm.fx.Bootstrapped(); err != nil { + return err + } + + ctx, cancel := context.WithCancel(context.TODO()) + vm.cancel = cancel + atomicTxGossipMarshaller := atomic.TxMarshaller{} + atomicTxGossipClient := vm.NewClient(p2p.AtomicTxGossipHandlerID, p2p.WithValidatorSampling(vm.P2PValidators())) + atomicTxGossipMetrics, err := avalanchegossip.NewMetrics(vm.MetricRegistry(), atomicTxGossipNamespace) + if err != nil { + return fmt.Errorf("failed to initialize atomic tx gossip metrics: %w", err) + } + + pushGossipParams := avalanchegossip.BranchingFactor{ + StakePercentage: vm.Config().PushGossipPercentStake, + Validators: vm.Config().PushGossipNumValidators, + Peers: vm.Config().PushGossipNumPeers, + } + pushRegossipParams := avalanchegossip.BranchingFactor{ + Validators: vm.Config().PushRegossipNumValidators, + Peers: vm.Config().PushRegossipNumPeers, + } + + if vm.atomicTxPushGossiper == nil { + vm.atomicTxPushGossiper, err = avalanchegossip.NewPushGossiper[*atomic.Tx]( + &atomicTxGossipMarshaller, + vm.mempool, + vm.P2PValidators(), + atomicTxGossipClient, + atomicTxGossipMetrics, + pushGossipParams, + pushRegossipParams, + config.PushGossipDiscardedElements, + config.TxGossipTargetMessageSize, + vm.Config().RegossipFrequency.Duration, + ) + if err != nil { + return fmt.Errorf("failed to initialize atomic tx push gossiper: %w", err) + } + } + + if vm.atomicTxGossipHandler == nil { + vm.atomicTxGossipHandler = gossip.NewTxGossipHandler[*atomic.Tx]( + vm.ctx.Log, + &atomicTxGossipMarshaller, + vm.mempool, + atomicTxGossipMetrics, + config.TxGossipTargetMessageSize, + config.TxGossipThrottlingPeriod, + config.TxGossipThrottlingLimit, + vm.P2PValidators(), + ) + } + + if err := vm.AddHandler(p2p.AtomicTxGossipHandlerID, vm.atomicTxGossipHandler); err != nil { + return fmt.Errorf("failed to add atomic tx gossip handler: %w", err) + } + + if vm.atomicTxPullGossiper == nil { + atomicTxPullGossiper := avalanchegossip.NewPullGossiper[*atomic.Tx]( + vm.ctx.Log, + &atomicTxGossipMarshaller, + vm.mempool, + atomicTxGossipClient, + atomicTxGossipMetrics, + config.TxGossipPollSize, + ) + + vm.atomicTxPullGossiper = &avalanchegossip.ValidatorGossiper{ + Gossiper: atomicTxPullGossiper, + NodeID: vm.ctx.NodeID, + Validators: vm.P2PValidators(), + } + } + + vm.shutdownWg.Add(1) + go func() { + avalanchegossip.Every(ctx, vm.ctx.Log, vm.atomicTxPushGossiper, vm.Config().PushGossipFrequency.Duration) + vm.shutdownWg.Done() + }() + + vm.shutdownWg.Add(1) + go func() { + avalanchegossip.Every(ctx, vm.ctx.Log, vm.atomicTxPullGossiper, vm.Config().PullGossipFrequency.Duration) + vm.shutdownWg.Done() + }() + + return nil +} + +func (vm *VM) Shutdown(context.Context) error { + if vm.ctx == nil { + return nil + } + if vm.cancel != nil { + vm.cancel() + } + if err := vm.InnerVM.Shutdown(context.Background()); err != nil { + log.Error("failed to shutdown inner VM", "err", err) + } + vm.shutdownWg.Wait() + return nil +} + +func (vm *VM) CreateHandlers(ctx context.Context) (map[string]http.Handler, error) { + apis, err := vm.InnerVM.CreateHandlers(ctx) + if err != nil { + return nil, err + } + avaxAPI, err := utils.NewHandler("avax", &AvaxAPI{vm}) + if err != nil { + return nil, fmt.Errorf("failed to register service for AVAX API due to %w", err) + } + log.Info("AVAX API enabled") + apis[avaxEndpoint] = avaxAPI + return apis, nil +} + +// verifyTxAtTip verifies that [tx] is valid to be issued on top of the currently preferred block +func (vm *VM) verifyTxAtTip(tx *atomic.Tx) error { + if txByteLen := len(tx.SignedBytes()); txByteLen > targetAtomicTxsSize { + return fmt.Errorf("tx size (%d) exceeds total atomic txs size target (%d)", txByteLen, targetAtomicTxsSize) + } + gasUsed, err := tx.GasUsed(true) + if err != nil { + return err + } + if gasUsed > maxAtomicTxMempoolGas { + return fmt.Errorf("tx gas usage (%d) exceeds maximum allowed mempool gas usage (%d)", gasUsed, maxAtomicTxMempoolGas) + } + blockchain := vm.Ethereum().BlockChain() + // Note: we fetch the current block and then the state at that block instead of the current state directly + // since we need the header of the current block below. + preferredBlock := blockchain.CurrentBlock() + preferredState, err := blockchain.StateAt(preferredBlock.Root) + if err != nil { + return fmt.Errorf("failed to retrieve block state at tip while verifying atomic tx: %w", err) + } + extraConfig := params.GetExtra(blockchain.Config()) + extraRules := params.GetRulesExtra(blockchain.Config().Rules(preferredBlock.Number, params.IsMergeTODO, preferredBlock.Time)) + parentHeader := preferredBlock + var nextBaseFee *big.Int + timestamp := uint64(vm.clock.Time().Unix()) + if extraConfig.IsApricotPhase3(timestamp) { + nextBaseFee, err = customheader.EstimateNextBaseFee(extraConfig, parentHeader, timestamp) + if err != nil { + // Return extremely detailed error since CalcBaseFee should never encounter an issue here + return fmt.Errorf("failed to calculate base fee with parent timestamp (%d), parent ExtraData: (0x%x), and current timestamp (%d): %w", parentHeader.Time, parentHeader.Extra, timestamp, err) + } + } + + // We don’t need to revert the state here in case verifyTx errors, because + // [preferredState] is thrown away either way. + return vm.verifyTx(tx, parentHeader.Hash(), nextBaseFee, preferredState, *extraRules) +} + +// verifyTx verifies that [tx] is valid to be issued into a block with parent block [parentHash] +// and validated at [state] using [rules] as the current rule set. +// Note: verifyTx may modify [state]. If [state] needs to be properly maintained, the caller is responsible +// for reverting to the correct snapshot after calling this function. If this function is called with a +// throwaway state, then this is not necessary. +func (vm *VM) verifyTx(tx *atomic.Tx, parentHash common.Hash, baseFee *big.Int, state *state.StateDB, rules extras.Rules) error { + parent, err := vm.InnerVM.GetVMBlock(context.TODO(), ids.ID(parentHash)) + if err != nil { + return fmt.Errorf("failed to get parent block: %w", err) + } + atomicBackend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: rules, + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, + } + if err := tx.UnsignedAtomicTx.Visit(&semanticVerifier{ + backend: atomicBackend, + tx: tx, + parent: parent, + baseFee: baseFee, + }); err != nil { + return err + } + return tx.UnsignedAtomicTx.EVMStateTransfer(vm.ctx, state) +} + +// verifyTxs verifies that [txs] are valid to be issued into a block with parent block [parentHash] +// using [rules] as the current rule set. +func (vm *VM) verifyTxs(txs []*atomic.Tx, parentHash common.Hash, baseFee *big.Int, height uint64, rules extras.Rules) error { + // Ensure that the parent was verified and inserted correctly. + if !vm.Ethereum().BlockChain().HasBlock(parentHash, height-1) { + return errRejectedParent + } + + ancestorID := ids.ID(parentHash) + // If the ancestor is unknown, then the parent failed verification when + // it was called. + // If the ancestor is rejected, then this block shouldn't be inserted + // into the canonical chain because the parent will be missing. + ancestor, err := vm.GetVMBlock(context.TODO(), ancestorID) + if err != nil { + return errRejectedParent + } + + // Ensure each tx in [txs] doesn't conflict with any other atomic tx in + // a processing ancestor block. + inputs := set.Set[ids.ID]{} + atomicBackend := &verifierBackend{ + ctx: vm.ctx, + fx: &vm.fx, + rules: rules, + bootstrapped: vm.IsBootstrapped(), + blockFetcher: vm, + secpCache: vm.secpCache, + } + + for _, atomicTx := range txs { + utx := atomicTx.UnsignedAtomicTx + if err := utx.Visit(&semanticVerifier{ + backend: atomicBackend, + tx: atomicTx, + parent: ancestor, + baseFee: baseFee, + }); err != nil { + return fmt.Errorf("invalid block due to failed semantic verify: %w at height %d", err, height) + } + txInputs := utx.InputUTXOs() + if inputs.Overlaps(txInputs) { + return errConflictingAtomicInputs + } + inputs.Union(txInputs) + } + return nil +} + +// CodecRegistry implements the secp256k1fx interface +func (vm *VM) CodecRegistry() codec.Registry { return vm.baseCodec } + +// Clock implements the secp256k1fx interface +func (vm *VM) Clock() *mockable.Clock { return &vm.clock } + +// Logger implements the secp256k1fx interface +func (vm *VM) Logger() logging.Logger { return vm.ctx.Log } + +func (vm *VM) createConsensusCallbacks() dummy.ConsensusCallbacks { + return dummy.ConsensusCallbacks{ + OnFinalizeAndAssemble: vm.onFinalizeAndAssemble, + OnExtraStateChange: vm.onExtraStateChange, + } +} + +func (vm *VM) preBatchOnFinalizeAndAssemble(header *types.Header, state *state.StateDB, txs []*types.Transaction) ([]byte, *big.Int, *big.Int, error) { + for { + tx, exists := vm.mempool.NextTx() + if !exists { + break + } + // Take a snapshot of [state] before calling verifyTx so that if the transaction fails verification + // we can revert to [snapshot]. + // Note: snapshot is taken inside the loop because you cannot revert to the same snapshot more than + // once. + snapshot := state.Snapshot() + rules := vm.rules(header.Number, header.Time) + if err := vm.verifyTx(tx, header.ParentHash, header.BaseFee, state, rules); err != nil { + // Discard the transaction from the mempool on failed verification. + log.Debug("discarding tx from mempool on failed verification", "txID", tx.ID(), "err", err) + vm.mempool.DiscardCurrentTx(tx.ID()) + state.RevertToSnapshot(snapshot) + continue + } + + atomicTxBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, tx) + if err != nil { + // Discard the transaction from the mempool and error if the transaction + // cannot be marshalled. This should never happen. + log.Debug("discarding tx due to unmarshal err", "txID", tx.ID(), "err", err) + vm.mempool.DiscardCurrentTx(tx.ID()) + return nil, nil, nil, fmt.Errorf("failed to marshal atomic transaction %s due to %w", tx.ID(), err) + } + var contribution, gasUsed *big.Int + if rules.IsApricotPhase4 { + contribution, gasUsed, err = tx.BlockFeeContribution(rules.IsApricotPhase5, vm.ctx.AVAXAssetID, header.BaseFee) + if err != nil { + return nil, nil, nil, err + } + } + return atomicTxBytes, contribution, gasUsed, nil + } + + if len(txs) == 0 { + // this could happen due to the async logic of geth tx pool + return nil, nil, nil, errEmptyBlock + } + + return nil, nil, nil, nil +} + +// assumes that we are in at least Apricot Phase 5. +func (vm *VM) postBatchOnFinalizeAndAssemble( + header *types.Header, + parent *types.Header, + state *state.StateDB, + txs []*types.Transaction, +) ([]byte, *big.Int, *big.Int, error) { + var ( + batchAtomicTxs []*atomic.Tx + batchAtomicUTXOs set.Set[ids.ID] + batchContribution *big.Int = new(big.Int).Set(common.Big0) + batchGasUsed *big.Int = new(big.Int).Set(common.Big0) + rules = vm.rules(header.Number, header.Time) + size int + ) + + atomicGasLimit, err := customheader.RemainingAtomicGasCapacity(vm.chainConfigExtra(), parent, header) + if err != nil { + return nil, nil, nil, err + } + + for { + tx, exists := vm.mempool.NextTx() + if !exists { + break + } + + // Ensure that adding [tx] to the block will not exceed the block size soft limit. + txSize := len(tx.SignedBytes()) + if size+txSize > targetAtomicTxsSize { + vm.mempool.CancelCurrentTx(tx.ID()) + break + } + + var ( + txGasUsed, txContribution *big.Int + err error + ) + + // Note: we do not need to check if we are in at least ApricotPhase4 here because + // we assume that this function will only be called when the block is in at least + // ApricotPhase5. + txContribution, txGasUsed, err = tx.BlockFeeContribution(true, vm.ctx.AVAXAssetID, header.BaseFee) + if err != nil { + return nil, nil, nil, err + } + // ensure `gasUsed + batchGasUsed` doesn't exceed `atomicGasLimit` + if totalGasUsed := new(big.Int).Add(batchGasUsed, txGasUsed); !utils.BigLessOrEqualUint64(totalGasUsed, atomicGasLimit) { + // Send [tx] back to the mempool's tx heap. + vm.mempool.CancelCurrentTx(tx.ID()) + break + } + + if batchAtomicUTXOs.Overlaps(tx.InputUTXOs()) { + // Discard the transaction from the mempool since it will fail verification + // after this block has been accepted. + // Note: if the proposed block is not accepted, the transaction may still be + // valid, but we discard it early here based on the assumption that the proposed + // block will most likely be accepted. + // Discard the transaction from the mempool on failed verification. + log.Debug("discarding tx due to overlapping input utxos", "txID", tx.ID()) + vm.mempool.DiscardCurrentTx(tx.ID()) + continue + } + + snapshot := state.Snapshot() + if err := vm.verifyTx(tx, header.ParentHash, header.BaseFee, state, rules); err != nil { + // Discard the transaction from the mempool and reset the state to [snapshot] + // if it fails verification here. + // Note: prior to this point, we have not modified [state] so there is no need to + // revert to a snapshot if we discard the transaction prior to this point. + log.Debug("discarding tx from mempool due to failed verification", "txID", tx.ID(), "err", err) + vm.mempool.DiscardCurrentTx(tx.ID()) + state.RevertToSnapshot(snapshot) + continue + } + + batchAtomicTxs = append(batchAtomicTxs, tx) + batchAtomicUTXOs.Union(tx.InputUTXOs()) + // Add the [txGasUsed] to the [batchGasUsed] when the [tx] has passed verification + batchGasUsed.Add(batchGasUsed, txGasUsed) + batchContribution.Add(batchContribution, txContribution) + size += txSize + } + + // If there is a non-zero number of transactions, marshal them and return the byte slice + // for the block's extra data along with the contribution and gas used. + if len(batchAtomicTxs) > 0 { + atomicTxBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, batchAtomicTxs) + if err != nil { + // If we fail to marshal the batch of atomic transactions for any reason, + // discard the entire set of current transactions. + log.Debug("discarding txs due to error marshaling atomic transactions", "err", err) + vm.mempool.DiscardCurrentTxs() + return nil, nil, nil, fmt.Errorf("failed to marshal batch of atomic transactions due to %w", err) + } + return atomicTxBytes, batchContribution, batchGasUsed, nil + } + + // If there are no regular transactions and there were also no atomic transactions to be included, + // then the block is empty and should be considered invalid. + if len(txs) == 0 { + // this could happen due to the async logic of geth tx pool + return nil, nil, nil, errEmptyBlock + } + + // If there are no atomic transactions, but there is a non-zero number of regular transactions, then + // we return a nil slice with no contribution from the atomic transactions and a nil error. + return nil, nil, nil, nil +} + +func (vm *VM) onFinalizeAndAssemble( + header *types.Header, + parent *types.Header, + state *state.StateDB, + txs []*types.Transaction, +) ([]byte, *big.Int, *big.Int, error) { + if !vm.chainConfigExtra().IsApricotPhase5(header.Time) { + return vm.preBatchOnFinalizeAndAssemble(header, state, txs) + } + return vm.postBatchOnFinalizeAndAssemble(header, parent, state, txs) +} + +func (vm *VM) onExtraStateChange(block *types.Block, parent *types.Header, state *state.StateDB, chainConfig *params.ChainConfig) (*big.Int, *big.Int, error) { + var ( + batchContribution *big.Int = big.NewInt(0) + batchGasUsed *big.Int = big.NewInt(0) + header = block.Header() + // We cannot use chain config from InnerVM since it's not available when this function is called for the first time (bc.loadLastState). + rules = chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) + rulesExtra = *params.GetRulesExtra(rules) + ) + + txs, err := atomic.ExtractAtomicTxs(customtypes.BlockExtData(block), rulesExtra.IsApricotPhase5, atomic.Codec) + if err != nil { + return nil, nil, err + } + + // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. + if vm.atomicBackend != nil { + if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { + log.Info("skipping atomic tx verification on bonus block", "block", block.Hash()) + } else { + // Verify [txs] do not conflict with themselves or ancestor blocks. + if err := vm.verifyTxs(txs, block.ParentHash(), block.BaseFee(), block.NumberU64(), rulesExtra); err != nil { + return nil, nil, err + } + } + // Update the atomic backend with [txs] from this block. + // + // Note: The atomic trie canonically contains the duplicate operations + // from any bonus blocks. + _, err := vm.atomicBackend.InsertTxs(block.Hash(), block.NumberU64(), block.ParentHash(), txs) + if err != nil { + return nil, nil, err + } + } + + // If there are no transactions, we can return early. + if len(txs) == 0 { + return nil, nil, nil + } + + for _, tx := range txs { + if err := tx.UnsignedAtomicTx.EVMStateTransfer(vm.ctx, state); err != nil { + return nil, nil, err + } + // If ApricotPhase4 is enabled, calculate the block fee contribution + if rulesExtra.IsApricotPhase4 { + contribution, gasUsed, err := tx.BlockFeeContribution(rulesExtra.IsApricotPhase5, vm.ctx.AVAXAssetID, block.BaseFee()) + if err != nil { + return nil, nil, err + } + + batchContribution.Add(batchContribution, contribution) + batchGasUsed.Add(batchGasUsed, gasUsed) + } + } + + // If ApricotPhase5 is enabled, enforce that the atomic gas used does not exceed the + // atomic gas limit. + if rulesExtra.IsApricotPhase5 { + chainConfigExtra := params.GetExtra(chainConfig) + atomicGasLimit, err := customheader.RemainingAtomicGasCapacity(chainConfigExtra, parent, header) + if err != nil { + return nil, nil, err + } + + if !utils.BigLessOrEqualUint64(batchGasUsed, atomicGasLimit) { + return nil, nil, fmt.Errorf("atomic gas used (%d) by block (%s), exceeds atomic gas limit (%d)", batchGasUsed, block.Hash().Hex(), atomicGasLimit) + } + } + return batchContribution, batchGasUsed, nil +} + +// getAtomicTx returns the requested transaction, status, and height. +// If the status is Unknown, then the returned transaction will be nil. +func (vm *VM) getAtomicTx(txID ids.ID) (*atomic.Tx, atomic.Status, uint64, error) { + if tx, height, err := vm.atomicTxRepository.GetByTxID(txID); err == nil { + return tx, atomic.Accepted, height, nil + } else if err != avalanchedatabase.ErrNotFound { + return nil, atomic.Unknown, 0, err + } + tx, dropped, found := vm.mempool.GetTx(txID) + switch { + case found && dropped: + return tx, atomic.Dropped, 0, nil + case found: + return tx, atomic.Processing, 0, nil + default: + return nil, atomic.Unknown, 0, nil + } +} + +// GetAtomicUTXOs returns the utxos that at least one of the provided addresses is +// referenced in. +func (vm *VM) GetAtomicUTXOs( + chainID ids.ID, + addrs set.Set[ids.ShortID], + startAddr ids.ShortID, + startUTXOID ids.ID, + limit int, +) ([]*avax.UTXO, ids.ShortID, ids.ID, error) { + if limit <= 0 || limit > maxUTXOsToFetch { + limit = maxUTXOsToFetch + } + + return avax.GetAtomicUTXOs( + vm.ctx.SharedMemory, + atomic.Codec, + chainID, + addrs, + startAddr, + startUTXOID, + limit, + ) +} + +func (vm *VM) BuildBlock(ctx context.Context) (snowman.Block, error) { + return vm.BuildBlockWithContext(ctx, nil) +} + +func (vm *VM) BuildBlockWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) (snowman.Block, error) { + blk, err := vm.InnerVM.BuildBlockWithContext(ctx, proposerVMBlockCtx) + + // Handle errors and signal the mempool to take appropriate action + switch { + case err == nil: + // Marks the current transactions from the mempool as being successfully issued + // into a block. + vm.mempool.IssueCurrentTxs() + case errors.Is(err, vmerrors.ErrGenerateBlockFailed), errors.Is(err, vmerrors.ErrBlockVerificationFailed): + log.Debug("cancelling txs due to error generating block", "err", err) + vm.mempool.CancelCurrentTxs() + case errors.Is(err, vmerrors.ErrWrapBlockFailed): + log.Debug("discarding txs due to error making new block", "err", err) + vm.mempool.DiscardCurrentTxs() + } + return blk, err +} + +func (vm *VM) chainConfigExtra() *extras.ChainConfig { + return params.GetExtra(vm.Ethereum().BlockChain().Config()) +} + +func (vm *VM) rules(number *big.Int, time uint64) extras.Rules { + ethrules := vm.Ethereum().BlockChain().Config().Rules(number, params.IsMergeTODO, time) + return *params.GetRulesExtra(ethrules) +} + +// currentRules returns the chain rules for the current block. +func (vm *VM) currentRules() extras.Rules { + header := vm.Ethereum().BlockChain().CurrentHeader() + return vm.rules(header.Number, header.Time) +} diff --git a/plugin/evm/atomic/vm/vm_test.go b/plugin/evm/atomic/vm/vm_test.go new file mode 100644 index 0000000000..426fc4269c --- /dev/null +++ b/plugin/evm/atomic/vm/vm_test.go @@ -0,0 +1,1709 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vm + +import ( + "context" + "errors" + "fmt" + "math/big" + "strings" + "testing" + "time" + + avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" + "github.com/ava-labs/avalanchego/upgrade/upgradetest" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + "github.com/ava-labs/avalanchego/utils/hashing" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/utils/units" + "github.com/ava-labs/avalanchego/vms/components/avax" + "github.com/ava-labs/avalanchego/vms/components/chain" + "github.com/ava-labs/avalanchego/vms/secp256k1fx" + + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm" + "github.com/ava-labs/coreth/plugin/evm/atomic" + "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" + "github.com/ava-labs/coreth/plugin/evm/customtypes" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/header" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap1" + "github.com/ava-labs/coreth/plugin/evm/vmtest" + "github.com/ava-labs/coreth/utils/utilstest" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" + "github.com/ava-labs/libevm/rlp" + "github.com/ava-labs/libevm/trie" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func newAtomicTestVM() *VM { + return WrapVM(&evm.VM{}) +} + +func setupAtomicTestVM(t *testing.T, cfg vmtest.TestVMConfig) (*VM, *vmtest.TestVMSuite) { + vm := newAtomicTestVM() + tvm := vmtest.SetupTestVM(t, vm, cfg) + return vm, tvm +} + +func setupAtomicTestVMWithUtxos(t *testing.T, cfg vmtest.TestVMConfig, utxos map[ids.ShortID]uint64) (*VM, *vmtest.TestVMSuite) { + vm, tvm := setupAtomicTestVM(t, cfg) + require.NoError(t, addUTXOs(tvm.AtomicMemory, vm.ctx, utxos)) + return vm, tvm +} + +func (vm *VM) newImportTx( + chainID ids.ID, // chain to import from + to common.Address, // Address of recipient + baseFee *big.Int, // fee to use post-AP3 + keys []*secp256k1.PrivateKey, // Keys to import the funds +) (*atomic.Tx, error) { + kc := secp256k1fx.NewKeychain() + for _, key := range keys { + kc.Add(key) + } + + atomicUTXOs, _, _, err := vm.GetAtomicUTXOs(chainID, kc.Addresses(), ids.ShortEmpty, ids.Empty, -1) + if err != nil { + return nil, fmt.Errorf("problem retrieving atomic UTXOs: %w", err) + } + + return atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), chainID, to, baseFee, kc, atomicUTXOs) +} + +func addUTXO(sharedMemory *avalancheatomic.Memory, ctx *snow.Context, txID ids.ID, index uint32, assetID ids.ID, amount uint64, addr ids.ShortID) (*avax.UTXO, error) { + utxo := &avax.UTXO{ + UTXOID: avax.UTXOID{ + TxID: txID, + OutputIndex: index, + }, + Asset: avax.Asset{ID: assetID}, + Out: &secp256k1fx.TransferOutput{ + Amt: amount, + OutputOwners: secp256k1fx.OutputOwners{ + Threshold: 1, + Addrs: []ids.ShortID{addr}, + }, + }, + } + utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + if err != nil { + return nil, err + } + + xChainSharedMemory := sharedMemory.NewSharedMemory(ctx.XChainID) + inputID := utxo.InputID() + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ + Key: inputID[:], + Value: utxoBytes, + Traits: [][]byte{ + addr.Bytes(), + }, + }}}}); err != nil { + return nil, err + } + + return utxo, nil +} + +func addUTXOs(sharedMemory *avalancheatomic.Memory, ctx *snow.Context, utxos map[ids.ShortID]uint64) error { + for addr, avaxAmount := range utxos { + txID, err := ids.ToID(hashing.ComputeHash256(addr.Bytes())) + if err != nil { + return fmt.Errorf("Failed to generate txID from addr: %s", err) + } + if _, err := addUTXO(sharedMemory, ctx, txID, 0, ctx.AVAXAssetID, avaxAmount, addr); err != nil { + return fmt.Errorf("Failed to add UTXO to shared memory: %s", err) + } + } + return nil +} + +func TestImportMissingUTXOs(t *testing.T) { + // make a VM with a shared memory that has an importable UTXO to build a block + importAmount := uint64(50000000) + fork := upgradetest.ApricotPhase2 + vm1, tvm1 := setupAtomicTestVMWithUtxos( + t, + vmtest.TestVMConfig{ + Fork: &fork, + }, + map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + }, + ) + defer func() { + require.NoError(t, vm1.Shutdown(context.Background())) + }() + + importTx, err := vm1.newImportTx(vm1.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, vmtest.TestKeys[0:1]) + require.NoError(t, err) + require.NoError(t, vm1.mempool.AddLocalTx(importTx)) + <-tvm1.ToEngine + blk, err := vm1.BuildBlock(context.Background()) + require.NoError(t, err) + + // make another VM which is missing the UTXO in shared memory + vm2, _ := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + defer func() { + require.NoError(t, vm2.Shutdown(context.Background())) + }() + + vm2Blk, err := vm2.ParseBlock(context.Background(), blk.Bytes()) + require.NoError(t, err) + err = vm2Blk.Verify(context.Background()) + require.ErrorIs(t, err, errMissingUTXOs) + + // This should not result in a bad block since the missing UTXO should + // prevent InsertBlockManual from being called. + badBlocks, _ := vm2.Ethereum().BlockChain().BadBlocks() + require.Len(t, badBlocks, 0) +} + +// Simple test to ensure we can issue an import transaction followed by an export transaction +// and they will be indexed correctly when accepted. +func TestIssueAtomicTxs(t *testing.T) { + importAmount := uint64(50000000) + vm := newAtomicTestVM() + fork := upgradetest.ApricotPhase2 + tvm := vmtest.SetupTestVM(t, vm, vmtest.TestVMConfig{ + Fork: &fork, + }) + utxos := map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + } + addUTXOs(tvm.AtomicMemory, vm.ctx, utxos) + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, vmtest.TestKeys[0:1]) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + if lastAcceptedID, err := vm.LastAccepted(context.Background()); err != nil { + t.Fatal(err) + } else if lastAcceptedID != blk.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + } + vm.Ethereum().BlockChain().DrainAcceptorQueue() + + state, err := vm.Ethereum().BlockChain().State() + if err != nil { + t.Fatal(err) + } + + exportTx, err := atomic.NewExportTx(vm.ctx, vm.currentRules(), state, vm.ctx.AVAXAssetID, importAmount-(2*ap0.AtomicTxFee), vm.ctx.XChainID, vmtest.TestShortIDAddrs[0], vmtest.InitialBaseFee, vmtest.TestKeys[0:1]) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(exportTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk2, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk2.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := blk2.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + if lastAcceptedID, err := vm.LastAccepted(context.Background()); err != nil { + t.Fatal(err) + } else if lastAcceptedID != blk2.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk2.ID(), lastAcceptedID) + } + + // Check that both atomic transactions were indexed as expected. + indexedImportTx, status, height, err := vm.getAtomicTx(importTx.ID()) + assert.NoError(t, err) + assert.Equal(t, atomic.Accepted, status) + assert.Equal(t, uint64(1), height, "expected height of indexed import tx to be 1") + assert.Equal(t, indexedImportTx.ID(), importTx.ID(), "expected ID of indexed import tx to match original txID") + + indexedExportTx, status, height, err := vm.getAtomicTx(exportTx.ID()) + assert.NoError(t, err) + assert.Equal(t, atomic.Accepted, status) + assert.Equal(t, uint64(2), height, "expected height of indexed export tx to be 2") + assert.Equal(t, indexedExportTx.ID(), exportTx.ID(), "expected ID of indexed import tx to match original txID") +} + +func testConflictingImportTxs(t *testing.T, fork upgradetest.Fork) { + importAmount := uint64(10000000) + vm, tvm := setupAtomicTestVMWithUtxos(t, vmtest.TestVMConfig{ + Fork: &fork, + }, map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + vmtest.TestShortIDAddrs[1]: importAmount, + vmtest.TestShortIDAddrs[2]: importAmount, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + importTxs := make([]*atomic.Tx, 0, 3) + conflictTxs := make([]*atomic.Tx, 0, 3) + for i, key := range vmtest.TestKeys { + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[i], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) + if err != nil { + t.Fatal(err) + } + importTxs = append(importTxs, importTx) + + conflictAddr := vmtest.TestEthAddrs[(i+1)%len(vmtest.TestEthAddrs)] + conflictTx, err := vm.newImportTx(vm.ctx.XChainID, conflictAddr, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) + if err != nil { + t.Fatal(err) + } + conflictTxs = append(conflictTxs, conflictTx) + } + + expectedParentBlkID, err := vm.LastAccepted(context.Background()) + if err != nil { + t.Fatal(err) + } + for _, tx := range importTxs[:2] { + if err := vm.mempool.AddLocalTx(tx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + vm.clock.Set(vm.clock.Time().Add(2 * time.Second)) + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if parentID := blk.Parent(); parentID != expectedParentBlkID { + t.Fatalf("Expected parent to have blockID %s, but found %s", expectedParentBlkID, parentID) + } + + expectedParentBlkID = blk.ID() + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + } + + // Check that for each conflict tx (whose conflict is in the chain ancestry) + // the VM returns an error when it attempts to issue the conflict into the mempool + // and when it attempts to build a block with the conflict force added to the mempool. + for i, tx := range conflictTxs[:2] { + if err := vm.mempool.AddLocalTx(tx); err == nil { + t.Fatal("Expected issueTx to fail due to conflicting transaction") + } + // Force issue transaction directly to the mempool + if err := vm.mempool.ForceAddTx(tx); err != nil { + t.Fatal(err) + } + <-tvm.ToEngine + + vm.clock.Set(vm.clock.Time().Add(2 * time.Second)) + _, err = vm.BuildBlock(context.Background()) + // The new block is verified in BuildBlock, so + // BuildBlock should fail due to an attempt to + // double spend an atomic UTXO. + if err == nil { + t.Fatalf("Block verification should have failed in BuildBlock %d due to double spending atomic UTXO", i) + } + } + + // Generate one more valid block so that we can copy the header to create an invalid block + // with modified extra data. This new block will be invalid for more than one reason (invalid merkle root) + // so we check to make sure that the expected error is returned from block verification. + if err := vm.mempool.AddLocalTx(importTxs[2]); err != nil { + t.Fatal(err) + } + <-tvm.ToEngine + vm.clock.Set(vm.clock.Time().Add(2 * time.Second)) + + validBlock, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := validBlock.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + validEthBlock := validBlock.(*chain.BlockWrapper).Block.(extension.VMBlock).GetEthBlock() + + rules := vm.currentRules() + var extraData []byte + switch { + case rules.IsApricotPhase5: + extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{conflictTxs[1]}) + default: + extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, conflictTxs[1]) + } + if err != nil { + t.Fatal(err) + } + + conflictingAtomicTxBlock := customtypes.NewBlockWithExtData( + types.CopyHeader(validEthBlock.Header()), + nil, + nil, + nil, + new(trie.Trie), + extraData, + true, + ) + + blockBytes, err := rlp.EncodeToBytes(conflictingAtomicTxBlock) + if err != nil { + t.Fatal(err) + } + + parsedBlock, err := vm.ParseBlock(context.Background(), blockBytes) + if err != nil { + t.Fatal(err) + } + + if err := parsedBlock.Verify(context.Background()); !errors.Is(err, errConflictingAtomicInputs) { + t.Fatalf("Expected to fail with err: %s, but found err: %s", errConflictingAtomicInputs, err) + } + + if !rules.IsApricotPhase5 { + return + } + + extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{importTxs[2], conflictTxs[2]}) + if err != nil { + t.Fatal(err) + } + + header := types.CopyHeader(validEthBlock.Header()) + headerExtra := customtypes.GetHeaderExtra(header) + headerExtra.ExtDataGasUsed.Mul(common.Big2, headerExtra.ExtDataGasUsed) + + internalConflictBlock := customtypes.NewBlockWithExtData( + header, + nil, + nil, + nil, + new(trie.Trie), + extraData, + true, + ) + + blockBytes, err = rlp.EncodeToBytes(internalConflictBlock) + if err != nil { + t.Fatal(err) + } + + parsedBlock, err = vm.ParseBlock(context.Background(), blockBytes) + if err != nil { + t.Fatal(err) + } + + if err := parsedBlock.Verify(context.Background()); !errors.Is(err, errConflictingAtomicInputs) { + t.Fatalf("Expected to fail with err: %s, but found err: %s", errConflictingAtomicInputs, err) + } +} + +func TestReissueAtomicTxHigherGasPrice(t *testing.T) { + kc := secp256k1fx.NewKeychain(vmtest.TestKeys...) + + for name, issueTxs := range map[string]func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, discarded []*atomic.Tx){ + "single UTXO override": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { + utxo, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, vmtest.TestShortIDAddrs[0]) + if err != nil { + t.Fatal(err) + } + tx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, kc, []*avax.UTXO{utxo}) + if err != nil { + t.Fatal(err) + } + tx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(common.Big2, vmtest.InitialBaseFee), kc, []*avax.UTXO{utxo}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(tx1); err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(tx2); err != nil { + t.Fatal(err) + } + + return []*atomic.Tx{tx2}, []*atomic.Tx{tx1} + }, + "one of two UTXOs overrides": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { + utxo1, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, vmtest.TestShortIDAddrs[0]) + if err != nil { + t.Fatal(err) + } + utxo2, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, vmtest.TestShortIDAddrs[0]) + if err != nil { + t.Fatal(err) + } + tx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, kc, []*avax.UTXO{utxo1, utxo2}) + if err != nil { + t.Fatal(err) + } + tx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(common.Big2, vmtest.InitialBaseFee), kc, []*avax.UTXO{utxo1}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(tx1); err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(tx2); err != nil { + t.Fatal(err) + } + + return []*atomic.Tx{tx2}, []*atomic.Tx{tx1} + }, + "hola": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { + utxo1, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, vmtest.TestShortIDAddrs[0]) + if err != nil { + t.Fatal(err) + } + utxo2, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, vmtest.TestShortIDAddrs[0]) + if err != nil { + t.Fatal(err) + } + + importTx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, kc, []*avax.UTXO{utxo1}) + if err != nil { + t.Fatal(err) + } + + importTx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(big.NewInt(3), vmtest.InitialBaseFee), kc, []*avax.UTXO{utxo2}) + if err != nil { + t.Fatal(err) + } + + reissuanceTx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(big.NewInt(2), vmtest.InitialBaseFee), kc, []*avax.UTXO{utxo1, utxo2}) + if err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(importTx1); err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx2); err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(reissuanceTx1); !errors.Is(err, txpool.ErrConflictingAtomicTx) { + t.Fatalf("Expected to fail with err: %s, but found err: %s", txpool.ErrConflictingAtomicTx, err) + } + + assert.True(t, vm.mempool.Has(importTx1.ID())) + assert.True(t, vm.mempool.Has(importTx2.ID())) + assert.False(t, vm.mempool.Has(reissuanceTx1.ID())) + + reissuanceTx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(big.NewInt(4), vmtest.InitialBaseFee), kc, []*avax.UTXO{utxo1, utxo2}) + if err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(reissuanceTx2); err != nil { + t.Fatal(err) + } + + return []*atomic.Tx{reissuanceTx2}, []*atomic.Tx{importTx1, importTx2} + }, + } { + t.Run(name, func(t *testing.T) { + fork := upgradetest.ApricotPhase5 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + issuedTxs, evictedTxs := issueTxs(t, vm, tvm.AtomicMemory) + + for i, tx := range issuedTxs { + _, issued := vm.mempool.GetPendingTx(tx.ID()) + assert.True(t, issued, "expected issued tx at index %d to be issued", i) + } + + for i, tx := range evictedTxs { + _, discarded, _ := vm.mempool.GetTx(tx.ID()) + assert.True(t, discarded, "expected discarded tx at index %d to be discarded", i) + } + }) + } +} + +func TestConflictingImportTxsAcrossBlocks(t *testing.T) { + for _, fork := range []upgradetest.Fork{ + upgradetest.ApricotPhase1, + upgradetest.ApricotPhase2, + upgradetest.ApricotPhase3, + upgradetest.ApricotPhase4, + upgradetest.ApricotPhase5, + } { + t.Run(fork.String(), func(t *testing.T) { + testConflictingImportTxs(t, fork) + }) + } +} + +func TestConflictingTransitiveAncestryWithGap(t *testing.T) { + key := utilstest.NewKey(t) + + key0 := vmtest.TestKeys[0] + addr0 := key0.Address() + + key1 := vmtest.TestKeys[1] + addr1 := key1.Address() + + importAmount := uint64(1000000000) + fork := upgradetest.NoUpgrades + vm, tvm := setupAtomicTestVMWithUtxos(t, vmtest.TestVMConfig{ + Fork: &fork, + }, map[ids.ShortID]uint64{ + addr0: importAmount, + addr1: importAmount, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) + vm.Ethereum().TxPool().SubscribeNewReorgEvent(newTxPoolHeadChan) + + importTx0A, err := vm.newImportTx(vm.ctx.XChainID, key.Address, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key0}) + if err != nil { + t.Fatal(err) + } + // Create a conflicting transaction + importTx0B, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[2], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key0}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx0A); err != nil { + t.Fatalf("Failed to issue importTx0A: %s", err) + } + + <-tvm.ToEngine + + blk0, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build block with import transaction: %s", err) + } + + if err := blk0.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification: %s", err) + } + + if err := vm.SetPreference(context.Background(), blk0.ID()); err != nil { + t.Fatal(err) + } + + newHead := <-newTxPoolHeadChan + if newHead.Head.Hash() != common.Hash(blk0.ID()) { + t.Fatalf("Expected new block to match") + } + + tx := types.NewTransaction(0, key.Address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), key.PrivateKey) + if err != nil { + t.Fatal(err) + } + + // Add the remote transactions, build the block, and set VM1's preference for block A + errs := vm.Ethereum().TxPool().AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) + } + } + + <-tvm.ToEngine + + blk1, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build blk1: %s", err) + } + + if err := blk1.Verify(context.Background()); err != nil { + t.Fatalf("blk1 failed verification due to %s", err) + } + + if err := vm.SetPreference(context.Background(), blk1.ID()); err != nil { + t.Fatal(err) + } + + importTx1, err := vm.newImportTx(vm.ctx.XChainID, key.Address, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key1}) + if err != nil { + t.Fatalf("Failed to issue importTx1 due to: %s", err) + } + + if err := vm.mempool.AddLocalTx(importTx1); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk2, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build block with import transaction: %s", err) + } + + if err := blk2.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification: %s", err) + } + + if err := vm.SetPreference(context.Background(), blk2.ID()); err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx0B); err == nil { + t.Fatalf("Should not have been able to issue import tx with conflict") + } + // Force issue transaction directly into the mempool + if err := vm.mempool.ForceAddTx(importTx0B); err != nil { + t.Fatal(err) + } + <-tvm.ToEngine + + _, err = vm.BuildBlock(context.Background()) + if err == nil { + t.Fatal("Shouldn't have been able to build an invalid block") + } +} + +func TestBonusBlocksTxs(t *testing.T) { + fork := upgradetest.NoUpgrades + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + importAmount := uint64(10000000) + utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} + + utxo := &avax.UTXO{ + UTXOID: utxoID, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, + Out: &secp256k1fx.TransferOutput{ + Amt: importAmount, + OutputOwners: secp256k1fx.OutputOwners{ + Threshold: 1, + Addrs: []ids.ShortID{vmtest.TestKeys[0].Address()}, + }, + }, + } + utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + if err != nil { + t.Fatal(err) + } + + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) + inputID := utxo.InputID() + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ + Key: inputID[:], + Value: utxoBytes, + Traits: [][]byte{ + vmtest.TestKeys[0].Address().Bytes(), + }, + }}}}); err != nil { + t.Fatal(err) + } + + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + // Make [blk] a bonus block. + vm.atomicBackend.AddBonusBlock(blk.Height(), blk.ID()) + + // Remove the UTXOs from shared memory, so that non-bonus blocks will fail verification + if err := vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.XChainID: {RemoveRequests: [][]byte{inputID[:]}}}); err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + lastAcceptedID, err := vm.LastAccepted(context.Background()) + if err != nil { + t.Fatal(err) + } + if lastAcceptedID != blk.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + } +} + +// Builds [blkA] with a virtuous import transaction and [blkB] with a separate import transaction +// that does not conflict. Accepts [blkB] and rejects [blkA], then asserts that the virtuous atomic +// transaction in [blkA] is correctly re-issued into the atomic transaction mempool. +func TestReissueAtomicTx(t *testing.T) { + fork := upgradetest.ApricotPhase1 + vm, tvm := setupAtomicTestVMWithUtxos(t, vmtest.TestVMConfig{ + Fork: &fork, + }, map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: 10000000, + vmtest.TestShortIDAddrs[1]: 10000000, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + genesisBlkID, err := vm.LastAccepted(context.Background()) + if err != nil { + t.Fatal(err) + } + + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blkA, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blkA.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blkA.ID()); err != nil { + t.Fatal(err) + } + + // SetPreference to parent before rejecting (will rollback state to genesis + // so that atomic transaction can be reissued, otherwise current block will + // conflict with UTXO to be reissued) + if err := vm.SetPreference(context.Background(), genesisBlkID); err != nil { + t.Fatal(err) + } + + // Rejecting [blkA] should cause [importTx] to be re-issued into the mempool. + if err := blkA.Reject(context.Background()); err != nil { + t.Fatal(err) + } + + // Sleep for a minimum of two seconds to ensure that [blkB] will have a different timestamp + // than [blkA] so that the block will be unique. This is necessary since we have marked [blkA] + // as Rejected. + time.Sleep(2 * time.Second) + <-tvm.ToEngine + blkB, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if blkB.Height() != blkA.Height() { + t.Fatalf("Expected blkB (%d) to have the same height as blkA (%d)", blkB.Height(), blkA.Height()) + } + + if err := blkB.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blkB.ID()); err != nil { + t.Fatal(err) + } + + if err := blkB.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + if lastAcceptedID, err := vm.LastAccepted(context.Background()); err != nil { + t.Fatal(err) + } else if lastAcceptedID != blkB.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blkB.ID(), lastAcceptedID) + } + + // Check that [importTx] has been indexed correctly after [blkB] is accepted. + _, height, err := vm.atomicTxRepository.GetByTxID(importTx.ID()) + if err != nil { + t.Fatal(err) + } else if height != blkB.Height() { + t.Fatalf("Expected indexed height of import tx to be %d, but found %d", blkB.Height(), height) + } +} + +func TestAtomicTxFailsEVMStateTransferBuildBlock(t *testing.T) { + fork := upgradetest.ApricotPhase1 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + exportTxs := createExportTxOptions(t, vm, tvm.ToEngine, tvm.AtomicMemory) + exportTx1, exportTx2 := exportTxs[0], exportTxs[1] + + if err := vm.mempool.AddLocalTx(exportTx1); err != nil { + t.Fatal(err) + } + <-tvm.ToEngine + exportBlk1, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + if err := exportBlk1.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), exportBlk1.ID()); err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(exportTx2); err == nil { + t.Fatal("Should have failed to issue due to an invalid export tx") + } + + if err := vm.mempool.AddRemoteTx(exportTx2); err == nil { + t.Fatal("Should have failed to add because conflicting") + } + + // Manually add transaction to mempool to bypass validation + if err := vm.mempool.ForceAddTx(exportTx2); err != nil { + t.Fatal(err) + } + <-tvm.ToEngine + + _, err = vm.BuildBlock(context.Background()) + if err == nil { + t.Fatal("BuildBlock should have returned an error due to invalid export transaction") + } +} + +// This is a regression test to ensure that if two consecutive atomic transactions fail verification +// in onFinalizeAndAssemble it will not cause a panic due to calling RevertToSnapshot(revID) on the +// same revision ID twice. +func TestConsecutiveAtomicTransactionsRevertSnapshot(t *testing.T) { + fork := upgradetest.ApricotPhase1 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) + vm.Ethereum().TxPool().SubscribeNewReorgEvent(newTxPoolHeadChan) + + // Create three conflicting import transactions + importTxs := createImportTxOptions(t, vm, tvm.AtomicMemory) + + // Issue the first import transaction, build, and accept the block. + if err := vm.mempool.AddLocalTx(importTxs[0]); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + newHead := <-newTxPoolHeadChan + if newHead.Head.Hash() != common.Hash(blk.ID()) { + t.Fatalf("Expected new block to match") + } + + // Add the two conflicting transactions directly to the mempool, so that two consecutive transactions + // will fail verification when build block is called. + vm.mempool.AddRemoteTx(importTxs[1]) + vm.mempool.AddRemoteTx(importTxs[2]) + + if _, err := vm.BuildBlock(context.Background()); err == nil { + t.Fatal("Expected build block to fail due to empty block") + } +} + +func TestAtomicTxBuildBlockDropsConflicts(t *testing.T) { + importAmount := uint64(10000000) + fork := upgradetest.ApricotPhase5 + vm, tvm := setupAtomicTestVMWithUtxos(t, vmtest.TestVMConfig{ + Fork: &fork, + }, map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + vmtest.TestShortIDAddrs[1]: importAmount, + vmtest.TestShortIDAddrs[2]: importAmount, + }) + conflictKey := utilstest.NewKey(t) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + // Create a conflict set for each pair of transactions + conflictSets := make([]set.Set[ids.ID], len(vmtest.TestKeys)) + for index, key := range vmtest.TestKeys { + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[index], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) + if err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + conflictSets[index].Add(importTx.ID()) + conflictTx, err := vm.newImportTx(vm.ctx.XChainID, conflictKey.Address, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{key}) + if err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(conflictTx); err == nil { + t.Fatal("should conflict with the utxoSet in the mempool") + } + // force add the tx + vm.mempool.ForceAddTx(conflictTx) + conflictSets[index].Add(conflictTx.ID()) + } + <-tvm.ToEngine + // Note: this only checks the path through OnFinalizeAndAssemble, we should make sure to add a test + // that verifies blocks received from the network will also fail verification + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + wrappedBlk, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + blockExtension, ok := wrappedBlk.GetBlockExtension().(*blockExtension) + require.True(t, ok, "expected block to be a blockExtension") + atomicTxs := blockExtension.atomicTxs + assert.True(t, len(atomicTxs) == len(vmtest.TestKeys), "Conflict transactions should be out of the batch") + atomicTxIDs := set.Set[ids.ID]{} + for _, tx := range atomicTxs { + atomicTxIDs.Add(tx.ID()) + } + + // Check that removing the txIDs actually included in the block from each conflict set + // leaves one item remaining for each conflict set ie. only one tx from each conflict set + // has been included in the block. + for _, conflictSet := range conflictSets { + conflictSet.Difference(atomicTxIDs) + assert.Equal(t, 1, conflictSet.Len()) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } +} + +func TestBuildBlockDoesNotExceedAtomicGasLimit(t *testing.T) { + importAmount := uint64(10000000) + fork := upgradetest.ApricotPhase5 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + kc := secp256k1fx.NewKeychain() + kc.Add(vmtest.TestKeys[0]) + txID, err := ids.ToID(hashing.ComputeHash256(vmtest.TestShortIDAddrs[0][:])) + assert.NoError(t, err) + + mempoolTxs := 200 + for i := 0; i < mempoolTxs; i++ { + utxo, err := addUTXO(tvm.AtomicMemory, vm.ctx, txID, uint32(i), vm.ctx.AVAXAssetID, importAmount, vmtest.TestShortIDAddrs[0]) + assert.NoError(t, err) + + importTx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, kc, []*avax.UTXO{utxo}) + if err != nil { + t.Fatal(err) + } + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + } + + <-tvm.ToEngine + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + wrappedBlk, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + blockExtension, ok := wrappedBlk.GetBlockExtension().(*blockExtension) + require.True(t, ok, "expected block to be a blockExtension") + // Need to ensure that not all of the transactions in the mempool are included in the block. + // This ensures that we hit the atomic gas limit while building the block before we hit the + // upper limit on the size of the codec for marshalling the atomic transactions. + atomicTxs := blockExtension.atomicTxs + if len(atomicTxs) >= mempoolTxs { + t.Fatalf("Expected number of atomic transactions included in the block (%d) to be less than the number of transactions added to the mempool (%d)", len(atomicTxs), mempoolTxs) + } +} + +func TestExtraStateChangeAtomicGasLimitExceeded(t *testing.T) { + importAmount := uint64(10000000) + // We create two VMs one in ApriotPhase4 and one in ApricotPhase5, so that we can construct a block + // containing a large enough atomic transaction that it will exceed the atomic gas limit in + // ApricotPhase5. + fork1 := upgradetest.ApricotPhase4 + vm1, tvm1 := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork1, + }) + fork2 := upgradetest.ApricotPhase5 + vm2, tvm2 := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork2, + }) + defer func() { + if err := vm1.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + if err := vm2.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + kc := secp256k1fx.NewKeychain() + kc.Add(vmtest.TestKeys[0]) + txID, err := ids.ToID(hashing.ComputeHash256(vmtest.TestShortIDAddrs[0][:])) + assert.NoError(t, err) + + // Add enough UTXOs, such that the created import transaction will attempt to consume more gas than allowed + // in ApricotPhase5. + for i := 0; i < 100; i++ { + _, err := addUTXO(tvm1.AtomicMemory, vm1.ctx, txID, uint32(i), vm1.ctx.AVAXAssetID, importAmount, vmtest.TestShortIDAddrs[0]) + assert.NoError(t, err) + + _, err = addUTXO(tvm2.AtomicMemory, vm2.ctx, txID, uint32(i), vm2.ctx.AVAXAssetID, importAmount, vmtest.TestShortIDAddrs[0]) + assert.NoError(t, err) + } + + // Double the initial base fee used when estimating the cost of this transaction to ensure that when it is + // used in ApricotPhase5 it still pays a sufficient fee with the fixed fee per atomic transaction. + importTx, err := vm1.newImportTx(vm1.ctx.XChainID, vmtest.TestEthAddrs[0], new(big.Int).Mul(common.Big2, vmtest.InitialBaseFee), []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + if err := vm1.mempool.ForceAddTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm1.ToEngine + blk1, err := vm1.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + if err := blk1.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + wrappedBlk, ok := blk1.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + validEthBlock := wrappedBlk.GetEthBlock() + extraData, err := atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{importTx}) + if err != nil { + t.Fatal(err) + } + + // Construct the new block with the extra data in the new format (slice of atomic transactions). + ethBlk2 := customtypes.NewBlockWithExtData( + types.CopyHeader(validEthBlock.Header()), + nil, + nil, + nil, + new(trie.Trie), + extraData, + true, + ) + + state, err := vm2.Ethereum().BlockChain().State() + if err != nil { + t.Fatal(err) + } + + // Hack: test [onExtraStateChange] directly to ensure it catches the atomic gas limit error correctly. + if _, _, err := vm2.onExtraStateChange(ethBlk2, nil, state, vm2.Ethereum().BlockChain().Config()); err == nil || !strings.Contains(err.Error(), "exceeds atomic gas limit") { + t.Fatalf("Expected block to fail verification due to exceeded atomic gas limit, but found error: %v", err) + } +} + +// Regression test to ensure that a VM that is not able to parse a block that +// contains no transactions. +func TestEmptyBlock(t *testing.T) { + importAmount := uint64(1000000000) + fork := upgradetest.NoUpgrades + vm, tvm := setupAtomicTestVMWithUtxos(t, vmtest.TestVMConfig{ + Fork: &fork, + }, map[ids.ShortID]uint64{ + vmtest.TestShortIDAddrs[0]: importAmount, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + importTx, err := vm.newImportTx(vm.ctx.XChainID, vmtest.TestEthAddrs[0], vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build block with import transaction: %s", err) + } + + // Create empty block from blkA + wrappedBlk, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + ethBlock := wrappedBlk.GetEthBlock() + + emptyEthBlock := customtypes.NewBlockWithExtData( + types.CopyHeader(ethBlock.Header()), + nil, + nil, + nil, + new(trie.Trie), + nil, + false, + ) + + if len(customtypes.BlockExtData(emptyEthBlock)) != 0 || customtypes.GetHeaderExtra(emptyEthBlock.Header()).ExtDataHash != (common.Hash{}) { + t.Fatalf("emptyEthBlock should not have any extra data") + } + + emptyBlockBytes, err := rlp.EncodeToBytes(emptyEthBlock) + require.NoError(t, err) + + if _, err := vm.ParseBlock(context.Background(), emptyBlockBytes); !errors.Is(err, errEmptyBlock) { + t.Fatalf("VM should have failed with errEmptyBlock but got %s", err.Error()) + } +} + +// Regression test to ensure we can build blocks if we are starting with the +// Apricot Phase 5 ruleset in genesis. +func TestBuildApricotPhase5Block(t *testing.T) { + fork := upgradetest.ApricotPhase5 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) + vm.Ethereum().TxPool().SubscribeNewReorgEvent(newTxPoolHeadChan) + + key := vmtest.TestKeys[0].ToECDSA() + address := vmtest.TestEthAddrs[0] + + importAmount := uint64(1000000000) + utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} + + utxo := &avax.UTXO{ + UTXOID: utxoID, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, + Out: &secp256k1fx.TransferOutput{ + Amt: importAmount, + OutputOwners: secp256k1fx.OutputOwners{ + Threshold: 1, + Addrs: []ids.ShortID{vmtest.TestKeys[0].Address()}, + }, + }, + } + utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + if err != nil { + t.Fatal(err) + } + + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) + inputID := utxo.InputID() + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ + Key: inputID[:], + Value: utxoBytes, + Traits: [][]byte{ + vmtest.TestKeys[0].Address().Bytes(), + }, + }}}}); err != nil { + t.Fatal(err) + } + + importTx, err := vm.newImportTx(vm.ctx.XChainID, address, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + wrappedBlk, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + ethBlk := wrappedBlk.GetEthBlock() + if eBlockGasCost := customtypes.BlockGasCost(ethBlk); eBlockGasCost == nil || eBlockGasCost.Cmp(common.Big0) != 0 { + t.Fatalf("expected blockGasCost to be 0 but got %d", eBlockGasCost) + } + if eExtDataGasUsed := customtypes.BlockExtDataGasUsed(ethBlk); eExtDataGasUsed == nil || eExtDataGasUsed.Cmp(big.NewInt(11230)) != 0 { + t.Fatalf("expected extDataGasUsed to be 11230 but got %d", eExtDataGasUsed) + } + minRequiredTip, err := header.EstimateRequiredTip(vm.chainConfigExtra(), ethBlk.Header()) + if err != nil { + t.Fatal(err) + } + if minRequiredTip == nil || minRequiredTip.Cmp(common.Big0) != 0 { + t.Fatalf("expected minRequiredTip to be 0 but got %d", minRequiredTip) + } + + newHead := <-newTxPoolHeadChan + if newHead.Head.Hash() != common.Hash(blk.ID()) { + t.Fatalf("Expected new block to match") + } + + txs := make([]*types.Transaction, 10) + for i := 0; i < 10; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice*3), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), key) + if err != nil { + t.Fatal(err) + } + txs[i] = signedTx + } + errs := vm.Ethereum().TxPool().Add(txs, false, false) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } + } + + <-tvm.ToEngine + + blk, err = vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + wrappedBlk, ok = blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + ethBlk = wrappedBlk.GetEthBlock() + if customtypes.BlockGasCost(ethBlk) == nil || customtypes.BlockGasCost(ethBlk).Cmp(big.NewInt(100)) < 0 { + t.Fatalf("expected blockGasCost to be at least 100 but got %d", customtypes.BlockGasCost(ethBlk)) + } + if customtypes.BlockExtDataGasUsed(ethBlk) == nil || customtypes.BlockExtDataGasUsed(ethBlk).Cmp(common.Big0) != 0 { + t.Fatalf("expected extDataGasUsed to be 0 but got %d", customtypes.BlockExtDataGasUsed(ethBlk)) + } + minRequiredTip, err = header.EstimateRequiredTip(vm.chainConfigExtra(), ethBlk.Header()) + if err != nil { + t.Fatal(err) + } + if minRequiredTip == nil || minRequiredTip.Cmp(big.NewInt(0.05*params.GWei)) < 0 { + t.Fatalf("expected minRequiredTip to be at least 0.05 gwei but got %d", minRequiredTip) + } + + lastAcceptedID, err := vm.LastAccepted(context.Background()) + if err != nil { + t.Fatal(err) + } + if lastAcceptedID != blk.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + } + + // Confirm all txs are present + ethBlkTxs := vm.Ethereum().BlockChain().GetBlockByNumber(2).Transactions() + for i, tx := range txs { + if len(ethBlkTxs) <= i { + t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) + } + if ethBlkTxs[i].Hash() != tx.Hash() { + t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) + } + } +} + +// Regression test to ensure we can build blocks if we are starting with the +// Apricot Phase 4 ruleset in genesis. +func TestBuildApricotPhase4Block(t *testing.T) { + fork := upgradetest.ApricotPhase4 + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) + vm.Ethereum().TxPool().SubscribeNewReorgEvent(newTxPoolHeadChan) + + key := vmtest.TestKeys[0].ToECDSA() + address := vmtest.TestEthAddrs[0] + + importAmount := uint64(1000000000) + utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} + + utxo := &avax.UTXO{ + UTXOID: utxoID, + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, + Out: &secp256k1fx.TransferOutput{ + Amt: importAmount, + OutputOwners: secp256k1fx.OutputOwners{ + Threshold: 1, + Addrs: []ids.ShortID{vmtest.TestKeys[0].Address()}, + }, + }, + } + utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + if err != nil { + t.Fatal(err) + } + + xChainSharedMemory := tvm.AtomicMemory.NewSharedMemory(vm.ctx.XChainID) + inputID := utxo.InputID() + if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ + Key: inputID[:], + Value: utxoBytes, + Traits: [][]byte{ + vmtest.TestKeys[0].Address().Bytes(), + }, + }}}}); err != nil { + t.Fatal(err) + } + + importTx, err := vm.newImportTx(vm.ctx.XChainID, address, vmtest.InitialBaseFee, []*secp256k1.PrivateKey{vmtest.TestKeys[0]}) + if err != nil { + t.Fatal(err) + } + + if err := vm.mempool.AddLocalTx(importTx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + wrappedBlk, ok := blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + ethBlk := wrappedBlk.GetEthBlock() + if eBlockGasCost := customtypes.BlockGasCost(ethBlk); eBlockGasCost == nil || eBlockGasCost.Cmp(common.Big0) != 0 { + t.Fatalf("expected blockGasCost to be 0 but got %d", eBlockGasCost) + } + if eExtDataGasUsed := customtypes.BlockExtDataGasUsed(ethBlk); eExtDataGasUsed == nil || eExtDataGasUsed.Cmp(big.NewInt(1230)) != 0 { + t.Fatalf("expected extDataGasUsed to be 1000 but got %d", eExtDataGasUsed) + } + minRequiredTip, err := header.EstimateRequiredTip(vm.chainConfigExtra(), ethBlk.Header()) + if err != nil { + t.Fatal(err) + } + if minRequiredTip == nil || minRequiredTip.Cmp(common.Big0) != 0 { + t.Fatalf("expected minRequiredTip to be 0 but got %d", minRequiredTip) + } + + newHead := <-newTxPoolHeadChan + if newHead.Head.Hash() != common.Hash(blk.ID()) { + t.Fatalf("Expected new block to match") + } + + txs := make([]*types.Transaction, 10) + chainID := vm.Ethereum().BlockChain().Config().ChainID + for i := 0; i < 5; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(chainID), key) + if err != nil { + t.Fatal(err) + } + txs[i] = signedTx + } + for i := 5; i < 10; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap1.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(chainID), key) + if err != nil { + t.Fatal(err) + } + txs[i] = signedTx + } + errs := vm.Ethereum().TxPool().AddRemotesSync(txs) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } + } + + <-tvm.ToEngine + + blk, err = vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) + } + + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + wrappedBlk, ok = blk.(*chain.BlockWrapper).Block.(extension.VMBlock) + require.True(t, ok, "expected block to be a VMBlock") + ethBlk = wrappedBlk.GetEthBlock() + if customtypes.BlockGasCost(ethBlk) == nil || customtypes.BlockGasCost(ethBlk).Cmp(big.NewInt(100)) < 0 { + t.Fatalf("expected blockGasCost to be at least 100 but got %d", customtypes.BlockGasCost(ethBlk)) + } + if customtypes.BlockExtDataGasUsed(ethBlk) == nil || customtypes.BlockExtDataGasUsed(ethBlk).Cmp(common.Big0) != 0 { + t.Fatalf("expected extDataGasUsed to be 0 but got %d", customtypes.BlockExtDataGasUsed(ethBlk)) + } + minRequiredTip, err = header.EstimateRequiredTip(vm.chainConfigExtra(), ethBlk.Header()) + if err != nil { + t.Fatal(err) + } + if minRequiredTip == nil || minRequiredTip.Cmp(big.NewInt(0.05*params.GWei)) < 0 { + t.Fatalf("expected minRequiredTip to be at least 0.05 gwei but got %d", minRequiredTip) + } + + lastAcceptedID, err := vm.LastAccepted(context.Background()) + if err != nil { + t.Fatal(err) + } + if lastAcceptedID != blk.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + } + + // Confirm all txs are present + ethBlkTxs := vm.Ethereum().BlockChain().GetBlockByNumber(2).Transactions() + for i, tx := range txs { + if len(ethBlkTxs) <= i { + t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) + } + if ethBlkTxs[i].Hash() != tx.Hash() { + t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) + } + } +} + +func TestBuildInvalidBlockHead(t *testing.T) { + fork := upgradetest.NoUpgrades + vm, tvm := setupAtomicTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + key0 := vmtest.TestKeys[0] + addr0 := key0.Address() + + // Create the transaction + utx := &atomic.UnsignedImportTx{ + NetworkID: vm.ctx.NetworkID, + BlockchainID: vm.ctx.ChainID, + Outs: []atomic.EVMOutput{{ + Address: common.Address(addr0), + Amount: 1 * units.Avax, + AssetID: vm.ctx.AVAXAssetID, + }}, + ImportedInputs: []*avax.TransferableInput{ + { + Asset: avax.Asset{ID: vm.ctx.AVAXAssetID}, + In: &secp256k1fx.TransferInput{ + Amt: 1 * units.Avax, + Input: secp256k1fx.Input{ + SigIndices: []uint32{0}, + }, + }, + }, + }, + SourceChain: vm.ctx.XChainID, + } + tx := &atomic.Tx{UnsignedAtomicTx: utx} + if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key0}}); err != nil { + t.Fatal(err) + } + + currentBlock := vm.Ethereum().BlockChain().CurrentBlock() + + // Verify that the transaction fails verification when attempting to issue + // it into the atomic mempool. + if err := vm.mempool.AddLocalTx(tx); err == nil { + t.Fatal("Should have failed to issue invalid transaction") + } + // Force issue the transaction directly to the mempool + if err := vm.mempool.ForceAddTx(tx); err != nil { + t.Fatal(err) + } + + <-tvm.ToEngine + + if _, err := vm.BuildBlock(context.Background()); err == nil { + t.Fatalf("Unexpectedly created a block") + } + + newCurrentBlock := vm.Ethereum().BlockChain().CurrentBlock() + + if currentBlock.Hash() != newCurrentBlock.Hash() { + t.Fatal("current block changed") + } +} diff --git a/plugin/evm/block.go b/plugin/evm/block.go deleted file mode 100644 index ffa182c881..0000000000 --- a/plugin/evm/block.go +++ /dev/null @@ -1,447 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package evm - -import ( - "bytes" - "context" - "errors" - "fmt" - "time" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/log" - "github.com/ava-labs/libevm/rlp" - - "github.com/ava-labs/coreth/core" - "github.com/ava-labs/coreth/params" - "github.com/ava-labs/coreth/params/extras" - "github.com/ava-labs/coreth/plugin/evm/atomic" - "github.com/ava-labs/coreth/plugin/evm/customtypes" - "github.com/ava-labs/coreth/plugin/evm/header" - "github.com/ava-labs/coreth/precompile/precompileconfig" - "github.com/ava-labs/coreth/predicate" - "github.com/ava-labs/libevm/core/rawdb" - "github.com/ava-labs/libevm/core/types" - - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow/consensus/snowman" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" -) - -var ( - _ snowman.Block = (*Block)(nil) - _ block.WithVerifyContext = (*Block)(nil) -) - -var errMissingUTXOs = errors.New("missing UTXOs") - -// readMainnetBonusBlocks returns maps of bonus block numbers to block IDs. -// Note bonus blocks are indexed in the atomic trie. -func readMainnetBonusBlocks() (map[uint64]ids.ID, error) { - mainnetBonusBlocks := map[uint64]string{ - 102972: "Njm9TcLUXRojZk8YhEM6ksvfiPdC1TME4zJvGaDXgzMCyB6oB", - 103105: "BYqLB6xpqy7HsAgP2XNfGE8Ubg1uEzse5mBPTSJH9z5s8pvMa", - 103143: "AfWvJH3rB2fdHuPWQp6qYNCFVT29MooQPRigD88rKKwUDEDhq", - 103183: "2KPW9G5tiNF14tZNfG4SqHuQrtUYVZyxuof37aZ7AnTKrQdsHn", - 103197: "pE93VXY3N5QKfwsEFcM9i59UpPFgeZ8nxpJNaGaDQyDgsscNf", - 103203: "2czmtnBS44VCWNRFUM89h4Fe9m3ZeZVYyh7Pe3FhNqjRNgPXhZ", - 103208: "esx5J962LtYm2aSrskpLai5e4CMMsaS1dsu9iuLGJ3KWgSu2M", - 103209: "DK9NqAJGry1wAo767uuYc1dYXAjUhzwka6vi8d9tNheqzGUTd", - 103259: "i1HoerJ1axognkUKKL58FvF9aLrbZKtv7TdKLkT5kgzoeU1vB", - 103261: "2DpCuBaH94zKKFNY2XTs4GeJcwsEv6qT2DHc59S8tdg97GZpcJ", - 103266: "2ez4CA7w4HHr8SSobHQUAwFgj2giRNjNFUZK9JvrZFa1AuRj6X", - 103287: "2QBNMMFJmhVHaGF45GAPszKyj1gK6ToBERRxYvXtM7yfrdUGPK", - 103339: "2pSjfo7rkFCfZ2CqAxqfw8vqM2CU2nVLHrFZe3rwxz43gkVuGo", - 103346: "2SiSziHHqPjb1qkw7CdGYupokiYpd2b7mMqRiyszurctcA5AKr", - 103350: "2F5tSQbdTfhZxvkxZqdFp7KR3FrJPKEsDLQK7KtPhNXj1EZAh4", - 103358: "2tCe88ur6MLQcVgwE5XxoaHiTGtSrthwKN3SdbHE4kWiQ7MSTV", - 103437: "21o2fVTnzzmtgXqkV1yuQeze7YEQhR5JB31jVVD9oVUnaaV8qm", - 103472: "2nG4exd9eUoAGzELfksmBR8XDCKhohY1uDKRFzEXJG4M8p3qA7", - 103478: "63YLdYXfXc5tY3mwWLaDsbXzQHYmwWVxMP7HKbRh4Du3C2iM1", - 103493: "soPweZ8DGaoUMjrnzjH3V2bypa7ZvvfqBan4UCsMUxMP759gw", - 103514: "2dNkpQF4mooveyUDfBYQTBfsGDV4wkncQPpEw4kHKfSTSTo5x", - 103536: "PJTkRrHvKZ1m4AQdPND1MBpUXpCrGN4DDmXmJQAiUrsxPoLQX", - 103545: "22ck2Z7cC38hmBfX2v3jMWxun8eD8psNaicfYeokS67DxwmPTx", - 103547: "pTf7gfk1ksj7bqMrLyMCij8FBKth1uRqQrtfykMFeXhx5xnrL", - 103554: "9oZh4qyBCcVwSGyDoUzRAuausvPJN3xH6nopKS6bwYzMfLoQ2", - 103555: "MjExz2z1qhwugc1tAyiGxRsCq4GvJwKfyyS29nr4tRVB8ooic", - 103559: "cwJusfmn98TW3DjAbfLRN9utYR24KAQ82qpAXmVSvjHyJZuM2", - 103561: "2YgxGHns7Z2hMMHJsPCgVXuJaL7x1b3gnHbmSCfCdyAcYGr6mx", - 103563: "2AXxT3PSEnaYHNtBTnYrVTf24TtKDWjky9sqoFEhydrGXE9iKH", - 103564: "Ry2sfjFfGEnJxRkUGFSyZNn7GR3m4aKAf1scDW2uXSNQB568Y", - 103569: "21Jys8UNURmtckKSV89S2hntEWymJszrLQbdLaNcbXcxDAsQSa", - 103570: "sg6wAwFBsPQiS5Yfyh41cVkCRQbrrXsxXmeNyQ1xkunf2sdyv", - 103575: "z3BgePPpCXq1mRBRvUi28rYYxnEtJizkUEHnDBrcZeVA7MFVk", - 103577: "uK5Ff9iBfDtREpVv9NgCQ1STD1nzLJG3yrfibHG4mGvmybw6f", - 103578: "Qv5v5Ru8ArfnWKB1w6s4G5EYPh7TybHJtF6UsVwAkfvZFoqmj", - 103582: "7KCZKBpxovtX9opb7rMRie9WmW5YbZ8A4HwBBokJ9eSHpZPqx", - 103587: "2AfTQ2FXNj9bkSUQnud9pFXULx6EbF7cbbw6i3ayvc2QNhgxfF", - 103590: "2gTygYckZgFZfN5QQWPaPBD3nabqjidV55mwy1x1Nd4JmJAwaM", - 103591: "2cUPPHy1hspr2nAKpQrrAEisLKkaWSS9iF2wjNFyFRs8vnSkKK", - 103594: "5MptSdP6dBMPSwk9GJjeVe39deZJTRh9i82cgNibjeDffrrTf", - 103597: "2J8z7HNv4nwh82wqRGyEHqQeuw4wJ6mCDCSvUgusBu35asnshK", - 103598: "2i2FP6nJyvhX9FR15qN2D9AVoK5XKgBD2i2AQ7FoSpfowxvQDX", - 103603: "2v3smb35s4GLACsK4Zkd2RcLBLdWA4huqrvq8Y3VP4CVe8kfTM", - 103604: "b7XfDDLgwB12DfL7UTWZoxwBpkLPL5mdHtXngD94Y2RoeWXSh", - 103607: "PgaRk1UAoUvRybhnXsrLq5t6imWhEa6ksNjbN6hWgs4qPrSzm", - 103612: "2oueNTj4dUE2FFtGyPpawnmCCsy6EUQeVHVLZy8NHeQmkAciP4", - 103614: "2YHZ1KymFjiBhpXzgt6HXJhLSt5SV9UQ4tJuUNjfN1nQQdm5zz", - 103617: "amgH2C1s9H3Av7vSW4y7n7TXb9tKyKHENvrDXutgNN6nsejgc", - 103618: "fV8k1U8oQDmfVwK66kAwN73aSsWiWhm8quNpVnKmSznBycV2W", - 103621: "Nzs93kFTvcXanFUp9Y8VQkKYnzmH8xykxVNFJTkdyAEeuxWbP", - 103623: "2rAsBj3emqQa13CV8r5fTtHogs4sXnjvbbXVzcKPi3WmzhpK9D", - 103624: "2JbuExUGKW5mYz5KfXATwq1ibRDimgks9wEdYGNSC6Ttey1R4U", - 103627: "tLLijh7oKfvWT1yk9zRv4FQvuQ5DAiuvb5kHCNN9zh4mqkFMG", - 103628: "dWBsRYRwFrcyi3DPdLoHsL67QkZ5h86hwtVfP94ZBaY18EkmF", - 103629: "XMoEsew2DhSgQaydcJFJUQAQYP8BTNTYbEJZvtbrV2QsX7iE3", - 103630: "2db2wMbVAoCc5EUJrsBYWvNZDekqyY8uNpaaVapdBAQZ5oRaou", - 103633: "2QiHZwLhQ3xLuyyfcdo5yCUfoSqWDvRZox5ECU19HiswfroCGp", - } - - bonusBlockMainnetHeights := make(map[uint64]ids.ID) - for height, blkIDStr := range mainnetBonusBlocks { - blkID, err := ids.FromString(blkIDStr) - if err != nil { - return nil, err - } - bonusBlockMainnetHeights[height] = blkID - } - return bonusBlockMainnetHeights, nil -} - -// Block implements the snowman.Block interface -type Block struct { - id ids.ID - ethBlock *types.Block - vm *VM - atomicTxs []*atomic.Tx -} - -// newBlock returns a new Block wrapping the ethBlock type and implementing the snowman.Block interface -func (vm *VM) newBlock(ethBlock *types.Block) (*Block, error) { - isApricotPhase5 := vm.chainConfigExtra().IsApricotPhase5(ethBlock.Time()) - atomicTxs, err := atomic.ExtractAtomicTxs(customtypes.BlockExtData(ethBlock), isApricotPhase5, atomic.Codec) - if err != nil { - return nil, err - } - - return &Block{ - id: ids.ID(ethBlock.Hash()), - ethBlock: ethBlock, - vm: vm, - atomicTxs: atomicTxs, - }, nil -} - -// ID implements the snowman.Block interface -func (b *Block) ID() ids.ID { return b.id } - -func (b *Block) AtomicTxs() []*atomic.Tx { return b.atomicTxs } - -// Accept implements the snowman.Block interface -func (b *Block) Accept(context.Context) error { - vm := b.vm - - // Although returning an error from Accept is considered fatal, it is good - // practice to cleanup the batch we were modifying in the case of an error. - defer vm.versiondb.Abort() - - blkID := b.ID() - log.Debug("accepting block", - "hash", blkID.Hex(), - "id", blkID, - "height", b.Height(), - ) - - // Call Accept for relevant precompile logs. Note we do this prior to - // calling Accept on the blockChain so any side effects (eg warp signatures) - // take place before the accepted log is emitted to subscribers. - rules := b.vm.rules(b.ethBlock.Number(), b.ethBlock.Time()) - if err := b.handlePrecompileAccept(rules); err != nil { - return err - } - if err := vm.blockChain.Accept(b.ethBlock); err != nil { - return fmt.Errorf("chain could not accept %s: %w", blkID, err) - } - - if err := vm.acceptedBlockDB.Put(lastAcceptedKey, blkID[:]); err != nil { - return fmt.Errorf("failed to put %s as the last accepted block: %w", blkID, err) - } - - for _, tx := range b.atomicTxs { - // Remove the accepted transaction from the mempool - vm.mempool.RemoveTx(tx) - } - - // Update VM state for atomic txs in this block. This includes updating the - // atomic tx repo, atomic trie, and shared memory. - atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(blkID)) - if err != nil { - // should never occur since [b] must be verified before calling Accept - return err - } - // Get pending operations on the vm's versionDB so we can apply them atomically - // with the shared memory changes. - vdbBatch, err := b.vm.versiondb.CommitBatch() - if err != nil { - return fmt.Errorf("could not create commit batch processing block[%s]: %w", blkID, err) - } - - // Apply any shared memory changes atomically with other pending changes to - // the vm's versionDB. - return atomicState.Accept(vdbBatch) -} - -// handlePrecompileAccept calls Accept on any logs generated with an active precompile address that implements -// contract.Accepter -func (b *Block) handlePrecompileAccept(rules extras.Rules) error { - // Short circuit early if there are no precompile accepters to execute - if len(rules.AccepterPrecompiles) == 0 { - return nil - } - - // Read receipts from disk - receipts := rawdb.ReadReceipts(b.vm.chaindb, b.ethBlock.Hash(), b.ethBlock.NumberU64(), b.ethBlock.Time(), b.vm.chainConfig) - // If there are no receipts, ReadReceipts may be nil, so we check the length and confirm the ReceiptHash - // is empty to ensure that missing receipts results in an error on accept. - if len(receipts) == 0 && b.ethBlock.ReceiptHash() != types.EmptyRootHash { - return fmt.Errorf("failed to fetch receipts for accepted block with non-empty root hash (%s) (Block: %s, Height: %d)", b.ethBlock.ReceiptHash(), b.ethBlock.Hash(), b.ethBlock.NumberU64()) - } - acceptCtx := &precompileconfig.AcceptContext{ - SnowCtx: b.vm.ctx, - Warp: b.vm.warpBackend, - } - for _, receipt := range receipts { - for logIdx, log := range receipt.Logs { - accepter, ok := rules.AccepterPrecompiles[log.Address] - if !ok { - continue - } - if err := accepter.Accept(acceptCtx, log.BlockHash, log.BlockNumber, log.TxHash, logIdx, log.Topics, log.Data); err != nil { - return err - } - } - } - - return nil -} - -// Reject implements the snowman.Block interface -// If [b] contains an atomic transaction, attempt to re-issue it -func (b *Block) Reject(context.Context) error { - blkID := b.ID() - log.Debug("rejecting block", - "hash", blkID.Hex(), - "id", blkID, - "height", b.Height(), - ) - - for _, tx := range b.atomicTxs { - // Re-issue the transaction in the mempool, continue even if it fails - b.vm.mempool.RemoveTx(tx) - if err := b.vm.mempool.AddRemoteTx(tx); err != nil { - log.Debug("Failed to re-issue transaction in rejected block", "txID", tx.ID(), "err", err) - } - } - atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(common.Hash(blkID)) - if err != nil { - // should never occur since [b] must be verified before calling Reject - return err - } - if err := atomicState.Reject(); err != nil { - return err - } - return b.vm.blockChain.Reject(b.ethBlock) -} - -// Parent implements the snowman.Block interface -func (b *Block) Parent() ids.ID { - return ids.ID(b.ethBlock.ParentHash()) -} - -// Height implements the snowman.Block interface -func (b *Block) Height() uint64 { - return b.ethBlock.NumberU64() -} - -// Timestamp implements the snowman.Block interface -func (b *Block) Timestamp() time.Time { - return time.Unix(int64(b.ethBlock.Time()), 0) -} - -// syntacticVerify verifies that a *Block is well-formed. -func (b *Block) syntacticVerify() error { - if b == nil || b.ethBlock == nil { - return errInvalidBlock - } - - header := b.ethBlock.Header() - rules := b.vm.chainConfig.Rules(header.Number, params.IsMergeTODO, header.Time) - return b.vm.syntacticBlockValidator.SyntacticVerify(b, rules) -} - -// Verify implements the snowman.Block interface -func (b *Block) Verify(context.Context) error { - return b.verify(&precompileconfig.PredicateContext{ - SnowCtx: b.vm.ctx, - ProposerVMBlockCtx: nil, - }, true) -} - -// ShouldVerifyWithContext implements the block.WithVerifyContext interface -func (b *Block) ShouldVerifyWithContext(context.Context) (bool, error) { - rules := b.vm.rules(b.ethBlock.Number(), b.ethBlock.Time()) - predicates := rules.Predicaters - // Short circuit early if there are no predicates to verify - if len(predicates) == 0 { - return false, nil - } - - // Check if any of the transactions in the block specify a precompile that enforces a predicate, which requires - // the ProposerVMBlockCtx. - for _, tx := range b.ethBlock.Transactions() { - for _, accessTuple := range tx.AccessList() { - if _, ok := predicates[accessTuple.Address]; ok { - log.Debug("Block verification requires proposerVM context", "block", b.ID(), "height", b.Height()) - return true, nil - } - } - } - - log.Debug("Block verification does not require proposerVM context", "block", b.ID(), "height", b.Height()) - return false, nil -} - -// VerifyWithContext implements the block.WithVerifyContext interface -func (b *Block) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) error { - return b.verify(&precompileconfig.PredicateContext{ - SnowCtx: b.vm.ctx, - ProposerVMBlockCtx: proposerVMBlockCtx, - }, true) -} - -// Verify the block is valid. -// Enforces that the predicates are valid within [predicateContext]. -// Writes the block details to disk and the state to the trie manager iff writes=true. -func (b *Block) verify(predicateContext *precompileconfig.PredicateContext, writes bool) error { - if predicateContext.ProposerVMBlockCtx != nil { - log.Debug("Verifying block with context", "block", b.ID(), "height", b.Height()) - } else { - log.Debug("Verifying block without context", "block", b.ID(), "height", b.Height()) - } - if err := b.syntacticVerify(); err != nil { - return fmt.Errorf("syntactic block verification failed: %w", err) - } - - // If the VM is not marked as bootstrapped the other chains may also be - // bootstrapping and not have populated the required indices. Since - // bootstrapping only verifies blocks that have been canonically accepted by - // the network, these checks would be guaranteed to pass on a synced node. - if b.vm.bootstrapped.Get() { - // Verify that the UTXOs named in import txs are present in shared - // memory. - // - // This does not fully verify that this block can spend these UTXOs. - // However, it guarantees that any block that fails the later checks was - // built by an incorrect block proposer. This ensures that we only mark - // blocks as BAD BLOCKs if they were incorrectly generated. - if err := b.verifyUTXOsPresent(); err != nil { - return err - } - - // Verify that all the ICM messages are correctly marked as either valid - // or invalid. - if err := b.verifyPredicates(predicateContext); err != nil { - return fmt.Errorf("failed to verify predicates: %w", err) - } - } - - // The engine may call VerifyWithContext multiple times on the same block with different contexts. - // Since the engine will only call Accept/Reject once, we should only call InsertBlockManual once. - // Additionally, if a block is already in processing, then it has already passed verification and - // at this point we have checked the predicates are still valid in the different context so we - // can return nil. - if b.vm.State.IsProcessing(b.id) { - return nil - } - - err := b.vm.blockChain.InsertBlockManual(b.ethBlock, writes) - if err != nil || !writes { - // if an error occurred inserting the block into the chain - // or if we are not pinning to memory, unpin the atomic trie - // changes from memory (if they were pinned). - if atomicState, err := b.vm.atomicBackend.GetVerifiedAtomicState(b.ethBlock.Hash()); err == nil { - _ = atomicState.Reject() // ignore this error so we can return the original error instead. - } - } - return err -} - -// verifyPredicates verifies the predicates in the block are valid according to predicateContext. -func (b *Block) verifyPredicates(predicateContext *precompileconfig.PredicateContext) error { - rules := b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Time()) - rulesExtra := params.GetRulesExtra(rules) - - switch { - case !rulesExtra.IsDurango && rulesExtra.PredicatersExist(): - return errors.New("cannot enable predicates before Durango activation") - case !rulesExtra.IsDurango: - return nil - } - - predicateResults := predicate.NewResults() - for _, tx := range b.ethBlock.Transactions() { - results, err := core.CheckPredicates(rules, predicateContext, tx) - if err != nil { - return err - } - predicateResults.SetTxResults(tx.Hash(), results) - } - // TODO: document required gas constraints to ensure marshalling predicate results does not error - predicateResultsBytes, err := predicateResults.Bytes() - if err != nil { - return fmt.Errorf("failed to marshal predicate results: %w", err) - } - extraData := b.ethBlock.Extra() - avalancheRules := rulesExtra.AvalancheRules - headerPredicateResultsBytes := header.PredicateBytesFromExtra(avalancheRules, extraData) - if !bytes.Equal(headerPredicateResultsBytes, predicateResultsBytes) { - return fmt.Errorf("%w (remote: %x local: %x)", errInvalidHeaderPredicateResults, headerPredicateResultsBytes, predicateResultsBytes) - } - return nil -} - -// verifyUTXOsPresent verifies all atomic UTXOs consumed by the block are -// present in shared memory. -func (b *Block) verifyUTXOsPresent() error { - blockHash := common.Hash(b.ID()) - if b.vm.atomicBackend.IsBonus(b.Height(), blockHash) { - log.Info("skipping atomic tx verification on bonus block", "block", blockHash) - return nil - } - - for _, atomicTx := range b.atomicTxs { - utx := atomicTx.UnsignedAtomicTx - chainID, requests, err := utx.AtomicOps() - if err != nil { - return err - } - if _, err := b.vm.ctx.SharedMemory.Get(chainID, requests.RemoveRequests); err != nil { - return fmt.Errorf("%w: %s", errMissingUTXOs, err) - } - } - return nil -} - -// Bytes implements the snowman.Block interface -func (b *Block) Bytes() []byte { - res, err := rlp.EncodeToBytes(b.ethBlock) - if err != nil { - panic(err) - } - return res -} - -func (b *Block) String() string { return fmt.Sprintf("EVM block, ID = %s", b.ID()) } diff --git a/plugin/evm/block_builder.go b/plugin/evm/block_builder.go index a2ad306b58..a73fd9c755 100644 --- a/plugin/evm/block_builder.go +++ b/plugin/evm/block_builder.go @@ -10,7 +10,7 @@ import ( "github.com/ava-labs/avalanchego/utils/timer" "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/txpool" - atomictxpool "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" + "github.com/ava-labs/coreth/plugin/evm/extension" "github.com/holiman/uint256" "github.com/ava-labs/avalanchego/snow" @@ -27,8 +27,8 @@ const ( type blockBuilder struct { ctx *snow.Context - txPool *txpool.TxPool - mempool *atomictxpool.Mempool + txPool *txpool.TxPool + extraMempool extension.BuilderMempool shutdownChan <-chan struct{} shutdownWg *sync.WaitGroup @@ -51,11 +51,13 @@ type blockBuilder struct { buildBlockTimer *timer.Timer } -func (vm *VM) NewBlockBuilder(notifyBuildBlockChan chan<- commonEng.Message) *blockBuilder { +// NewBlockBuilder creates a new block builder. extraMempool is an optional mempool (can be nil) that +// can be used to add transactions to the block builder, in addition to the txPool. +func (vm *VM) NewBlockBuilder(notifyBuildBlockChan chan<- commonEng.Message, extraMempool extension.BuilderMempool) *blockBuilder { b := &blockBuilder{ ctx: vm.ctx, txPool: vm.txPool, - mempool: vm.mempool, + extraMempool: extraMempool, shutdownChan: vm.shutdownChan, shutdownWg: &vm.shutdownWg, notifyBuildBlockChan: notifyBuildBlockChan, @@ -102,7 +104,7 @@ func (b *blockBuilder) needToBuild() bool { size := b.txPool.PendingSize(txpool.PendingFilter{ MinTip: uint256.MustFromBig(b.txPool.GasTip()), }) - return size > 0 || b.mempool.Len() > 0 + return size > 0 || (b.extraMempool != nil && b.extraMempool.PendingLen() > 0) } // markBuilding adds a PendingTxs message to the toEngine channel. @@ -147,6 +149,11 @@ func (b *blockBuilder) awaitSubmittedTxs() { txSubmitChan := make(chan core.NewTxsEvent) b.txPool.SubscribeTransactions(txSubmitChan, true) + var extraChan <-chan struct{} + if b.extraMempool != nil { + extraChan = b.extraMempool.SubscribePendingTxs() + } + b.shutdownWg.Add(1) go b.ctx.Log.RecoverAndPanic(func() { defer b.shutdownWg.Done() @@ -156,8 +163,8 @@ func (b *blockBuilder) awaitSubmittedTxs() { case <-txSubmitChan: log.Trace("New tx detected, trying to generate a block") b.signalTxsReady() - case <-b.mempool.SubscribePendingTxs(): - log.Trace("New atomic Tx detected, trying to generate a block") + case <-extraChan: + log.Trace("New extra Tx detected, trying to generate a block") b.signalTxsReady() case <-b.shutdownChan: b.buildBlockTimer.Stop() diff --git a/plugin/evm/block_verification.go b/plugin/evm/block_verification.go deleted file mode 100644 index e439c2db16..0000000000 --- a/plugin/evm/block_verification.go +++ /dev/null @@ -1,256 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package evm - -import ( - "errors" - "fmt" - "math/big" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/trie" - - safemath "github.com/ava-labs/avalanchego/utils/math" - - "github.com/ava-labs/coreth/constants" - "github.com/ava-labs/coreth/params" - "github.com/ava-labs/coreth/plugin/evm/customtypes" - "github.com/ava-labs/coreth/plugin/evm/header" - "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" - "github.com/ava-labs/coreth/plugin/evm/upgrade/ap1" - "github.com/ava-labs/coreth/plugin/evm/upgrade/ap5" - "github.com/ava-labs/coreth/utils" - "github.com/ava-labs/libevm/core/types" -) - -var ( - ap0MinGasPrice = big.NewInt(ap0.MinGasPrice) - ap1MinGasPrice = big.NewInt(ap1.MinGasPrice) -) - -type BlockValidator interface { - SyntacticVerify(b *Block, rules params.Rules) error -} - -type blockValidator struct { - extDataHashes map[common.Hash]common.Hash -} - -func NewBlockValidator(extDataHashes map[common.Hash]common.Hash) BlockValidator { - return &blockValidator{ - extDataHashes: extDataHashes, - } -} - -func (v blockValidator) SyntacticVerify(b *Block, rules params.Rules) error { - rulesExtra := params.GetRulesExtra(rules) - if b == nil || b.ethBlock == nil { - return errInvalidBlock - } - - ethHeader := b.ethBlock.Header() - blockHash := b.ethBlock.Hash() - - if !rulesExtra.IsApricotPhase1 { - if v.extDataHashes != nil { - extData := customtypes.BlockExtData(b.ethBlock) - extDataHash := customtypes.CalcExtDataHash(extData) - // If there is no extra data, check that there is no extra data in the hash map either to ensure we do not - // have a block that is unexpectedly missing extra data. - expectedExtDataHash, ok := v.extDataHashes[blockHash] - if len(extData) == 0 { - if ok { - return fmt.Errorf("found block with unexpected missing extra data (%s, %d), expected extra data hash: %s", blockHash, b.Height(), expectedExtDataHash) - } - } else { - // If there is extra data, check to make sure that the extra data hash matches the expected extra data hash for this - // block - if extDataHash != expectedExtDataHash { - return fmt.Errorf("extra data hash in block (%s, %d): %s, did not match the expected extra data hash: %s", blockHash, b.Height(), extDataHash, expectedExtDataHash) - } - } - } - } - - // Skip verification of the genesis block since it should already be marked as accepted. - if blockHash == b.vm.genesisHash { - return nil - } - - // Verify the ExtDataHash field - headerExtra := customtypes.GetHeaderExtra(ethHeader) - if rulesExtra.IsApricotPhase1 { - extraData := customtypes.BlockExtData(b.ethBlock) - hash := customtypes.CalcExtDataHash(extraData) - if headerExtra.ExtDataHash != hash { - return fmt.Errorf("extra data hash mismatch: have %x, want %x", headerExtra.ExtDataHash, hash) - } - } else { - if headerExtra.ExtDataHash != (common.Hash{}) { - return fmt.Errorf( - "expected ExtDataHash to be empty but got %x", - headerExtra.ExtDataHash, - ) - } - } - - // Perform block and header sanity checks - if !ethHeader.Number.IsUint64() { - return fmt.Errorf("invalid block number: %v", ethHeader.Number) - } - if !ethHeader.Difficulty.IsUint64() || ethHeader.Difficulty.Cmp(common.Big1) != 0 { - return fmt.Errorf("invalid difficulty: %d", ethHeader.Difficulty) - } - if ethHeader.Nonce.Uint64() != 0 { - return fmt.Errorf( - "expected nonce to be 0 but got %d: %w", - ethHeader.Nonce.Uint64(), errInvalidNonce, - ) - } - - if ethHeader.MixDigest != (common.Hash{}) { - return fmt.Errorf("invalid mix digest: %v", ethHeader.MixDigest) - } - - // Verify the extra data is well-formed. - if err := header.VerifyExtra(rulesExtra.AvalancheRules, ethHeader.Extra); err != nil { - return err - } - - if version := customtypes.BlockVersion(b.ethBlock); version != 0 { - return fmt.Errorf("invalid version: %d", version) - } - - // Check that the tx hash in the header matches the body - txsHash := types.DeriveSha(b.ethBlock.Transactions(), trie.NewStackTrie(nil)) - if txsHash != ethHeader.TxHash { - return fmt.Errorf("invalid txs hash %v does not match calculated txs hash %v", ethHeader.TxHash, txsHash) - } - // Check that the uncle hash in the header matches the body - uncleHash := types.CalcUncleHash(b.ethBlock.Uncles()) - if uncleHash != ethHeader.UncleHash { - return fmt.Errorf("invalid uncle hash %v does not match calculated uncle hash %v", ethHeader.UncleHash, uncleHash) - } - // Coinbase must match the BlackholeAddr on C-Chain - if ethHeader.Coinbase != constants.BlackholeAddr { - return fmt.Errorf("invalid coinbase %v does not match required blackhole address %v", ethHeader.Coinbase, constants.BlackholeAddr) - } - // Block must not have any uncles - if len(b.ethBlock.Uncles()) > 0 { - return errUnclesUnsupported - } - - // Block must not be empty - txs := b.ethBlock.Transactions() - if len(txs) == 0 && len(b.atomicTxs) == 0 { - return errEmptyBlock - } - - // Enforce minimum gas prices here prior to dynamic fees going into effect. - switch { - case !rulesExtra.IsApricotPhase1: - // If we are in ApricotPhase0, enforce each transaction has a minimum gas price of at least the LaunchMinGasPrice - for _, tx := range b.ethBlock.Transactions() { - if tx.GasPrice().Cmp(ap0MinGasPrice) < 0 { - return fmt.Errorf("block contains tx %s with gas price too low (%d < %d)", tx.Hash(), tx.GasPrice(), ap0.MinGasPrice) - } - } - case !rulesExtra.IsApricotPhase3: - // If we are prior to ApricotPhase3, enforce each transaction has a minimum gas price of at least the ApricotPhase1MinGasPrice - for _, tx := range b.ethBlock.Transactions() { - if tx.GasPrice().Cmp(ap1MinGasPrice) < 0 { - return fmt.Errorf("block contains tx %s with gas price too low (%d < %d)", tx.Hash(), tx.GasPrice(), ap1.MinGasPrice) - } - } - } - - // Make sure the block isn't too far in the future - // TODO: move this to only be part of semantic verification. - blockTimestamp := b.ethBlock.Time() - if maxBlockTime := uint64(b.vm.clock.Time().Add(maxFutureBlockTime).Unix()); blockTimestamp > maxBlockTime { - return fmt.Errorf("block timestamp is too far in the future: %d > allowed %d", blockTimestamp, maxBlockTime) - } - - // Ensure BaseFee is non-nil as of ApricotPhase3. - if rulesExtra.IsApricotPhase3 { - if ethHeader.BaseFee == nil { - return errNilBaseFeeApricotPhase3 - } - // TODO: this should be removed as 256 is the maximum possible bit length of a big int - if bfLen := ethHeader.BaseFee.BitLen(); bfLen > 256 { - return fmt.Errorf("too large base fee: bitlen %d", bfLen) - } - } - - // If we are in ApricotPhase4, ensure that ExtDataGasUsed is populated correctly. - if rulesExtra.IsApricotPhase4 { - // After the F upgrade, the extDataGasUsed field is validated by - // [header.VerifyGasUsed]. - if !rulesExtra.IsFortuna && rulesExtra.IsApricotPhase5 { - if !utils.BigLessOrEqualUint64(headerExtra.ExtDataGasUsed, ap5.AtomicGasLimit) { - return fmt.Errorf("too large extDataGasUsed: %d", headerExtra.ExtDataGasUsed) - } - } - var totalGasUsed uint64 - for _, atomicTx := range b.atomicTxs { - // We perform this check manually here to avoid the overhead of having to - // reparse the atomicTx in `CalcExtDataGasUsed`. - fixedFee := rulesExtra.IsApricotPhase5 // Charge the atomic tx fixed fee as of ApricotPhase5 - gasUsed, err := atomicTx.GasUsed(fixedFee) - if err != nil { - return err - } - totalGasUsed, err = safemath.Add(totalGasUsed, gasUsed) - if err != nil { - return err - } - } - - switch { - case !utils.BigEqualUint64(headerExtra.ExtDataGasUsed, totalGasUsed): - return fmt.Errorf("invalid extDataGasUsed: have %d, want %d", headerExtra.ExtDataGasUsed, totalGasUsed) - - // Make sure BlockGasCost is not nil - // NOTE: ethHeader.BlockGasCost correctness is checked in header verification - case headerExtra.BlockGasCost == nil: - return errNilBlockGasCostApricotPhase4 - case !headerExtra.BlockGasCost.IsUint64(): - return fmt.Errorf("too large blockGasCost: %d", headerExtra.BlockGasCost) - } - } - - // Verify the existence / non-existence of excessBlobGas - cancun := rules.IsCancun - if !cancun && ethHeader.ExcessBlobGas != nil { - return fmt.Errorf("invalid excessBlobGas: have %d, expected nil", *ethHeader.ExcessBlobGas) - } - if !cancun && ethHeader.BlobGasUsed != nil { - return fmt.Errorf("invalid blobGasUsed: have %d, expected nil", *ethHeader.BlobGasUsed) - } - if cancun && ethHeader.ExcessBlobGas == nil { - return errors.New("header is missing excessBlobGas") - } - if cancun && ethHeader.BlobGasUsed == nil { - return errors.New("header is missing blobGasUsed") - } - if !cancun && ethHeader.ParentBeaconRoot != nil { - return fmt.Errorf("invalid parentBeaconRoot: have %x, expected nil", *ethHeader.ParentBeaconRoot) - } - // TODO: decide what to do after Cancun - // currently we are enforcing it to be empty hash - if cancun { - switch { - case ethHeader.ParentBeaconRoot == nil: - return errors.New("header is missing parentBeaconRoot") - case *ethHeader.ParentBeaconRoot != (common.Hash{}): - return fmt.Errorf("invalid parentBeaconRoot: have %x, expected empty hash", ethHeader.ParentBeaconRoot) - } - if ethHeader.BlobGasUsed == nil { - return fmt.Errorf("blob gas used must not be nil in Cancun") - } else if *ethHeader.BlobGasUsed > 0 { - return fmt.Errorf("blobs not enabled on avalanche networks: used %d blob gas, expected 0", *ethHeader.BlobGasUsed) - } - } - return nil -} diff --git a/plugin/evm/client/client.go b/plugin/evm/client/client.go index 2f704d6df2..e4d24e358c 100644 --- a/plugin/evm/client/client.go +++ b/plugin/evm/client/client.go @@ -5,7 +5,6 @@ package client import ( "context" - "errors" "fmt" "golang.org/x/exp/slog" @@ -20,11 +19,10 @@ import ( "github.com/ava-labs/coreth/plugin/evm/config" ) +// TODO: this should be defined per chain vm (coreth, subnet-evm) in their relative pkgs. // Interface compliance var _ Client = (*client)(nil) -var errInvalidAddr = errors.New("invalid hex address") - // Client interface for interacting with EVM [chain] type Client interface { IssueTx(ctx context.Context, txBytes []byte, options ...rpc.Option) (ids.ID, error) diff --git a/plugin/evm/client/utils.go b/plugin/evm/client/utils.go deleted file mode 100644 index 64b375d84d..0000000000 --- a/plugin/evm/client/utils.go +++ /dev/null @@ -1,13 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package client - -import "github.com/ava-labs/libevm/common" - -func ParseEthAddress(addrStr string) (common.Address, error) { - if !common.IsHexAddress(addrStr) { - return common.Address{}, errInvalidAddr - } - return common.HexToAddress(addrStr), nil -} diff --git a/plugin/evm/config/constants.go b/plugin/evm/config/constants.go index 336102a548..46caa1b8ce 100644 --- a/plugin/evm/config/constants.go +++ b/plugin/evm/config/constants.go @@ -3,9 +3,20 @@ package config +import ( + "time" + + "github.com/ava-labs/avalanchego/utils/units" +) + const ( TxGossipBloomMinTargetElements = 8 * 1024 TxGossipBloomTargetFalsePositiveRate = 0.01 TxGossipBloomResetFalsePositiveRate = 0.05 TxGossipBloomChurnMultiplier = 3 + PushGossipDiscardedElements = 16_384 + TxGossipTargetMessageSize = 20 * units.KiB + TxGossipThrottlingPeriod = 10 * time.Second + TxGossipThrottlingLimit = 2 + TxGossipPollSize = 1 ) diff --git a/plugin/evm/gossip.go b/plugin/evm/eth_gossiper.go similarity index 76% rename from plugin/evm/gossip.go rename to plugin/evm/eth_gossiper.go index b28439dc40..324c5e1b70 100644 --- a/plugin/evm/gossip.go +++ b/plugin/evm/eth_gossiper.go @@ -10,17 +10,13 @@ import ( "fmt" "sync" "sync/atomic" - "time" ethcommon "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/log" "github.com/prometheus/client_golang/prometheus" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/network/p2p" "github.com/ava-labs/avalanchego/network/p2p/gossip" - "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/core/txpool" @@ -32,8 +28,6 @@ import ( const pendingTxsBuffer = 10 var ( - _ p2p.Handler = (*txGossipHandler)(nil) - _ gossip.Gossipable = (*GossipEthTx)(nil) _ gossip.Marshaller[*GossipEthTx] = (*GossipEthTxMarshaller)(nil) _ gossip.Set[*GossipEthTx] = (*GossipEthTxPool)(nil) @@ -41,56 +35,6 @@ var ( _ eth.PushGossiper = (*EthPushGossiper)(nil) ) -func newTxGossipHandler[T gossip.Gossipable]( - log logging.Logger, - marshaller gossip.Marshaller[T], - mempool gossip.Set[T], - metrics gossip.Metrics, - maxMessageSize int, - throttlingPeriod time.Duration, - throttlingLimit int, - validators p2p.ValidatorSet, -) txGossipHandler { - // push gossip messages can be handled from any peer - handler := gossip.NewHandler( - log, - marshaller, - mempool, - metrics, - maxMessageSize, - ) - - // pull gossip requests are filtered by validators and are throttled - // to prevent spamming - validatorHandler := p2p.NewValidatorHandler( - p2p.NewThrottlerHandler( - handler, - p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), - log, - ), - validators, - log, - ) - - return txGossipHandler{ - appGossipHandler: handler, - appRequestHandler: validatorHandler, - } -} - -type txGossipHandler struct { - appGossipHandler p2p.Handler - appRequestHandler p2p.Handler -} - -func (t txGossipHandler) AppGossip(ctx context.Context, nodeID ids.NodeID, gossipBytes []byte) { - t.appGossipHandler.AppGossip(ctx, nodeID, gossipBytes) -} - -func (t txGossipHandler) AppRequest(ctx context.Context, nodeID ids.NodeID, deadline time.Time, requestBytes []byte) ([]byte, *common.AppError) { - return t.appRequestHandler.AppRequest(ctx, nodeID, deadline, requestBytes) -} - func NewGossipEthTxPool(mempool *txpool.TxPool, registerer prometheus.Registerer) (*GossipEthTxPool, error) { bloom, err := gossip.NewBloomFilter( registerer, diff --git a/plugin/evm/extension/config.go b/plugin/evm/extension/config.go new file mode 100644 index 0000000000..22e76a93d4 --- /dev/null +++ b/plugin/evm/extension/config.go @@ -0,0 +1,185 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package extension + +import ( + "context" + "errors" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/database" + "github.com/ava-labs/avalanchego/database/versiondb" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/snow/consensus/snowman" + avalanchecommon "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils/timer/mockable" + + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/prometheus/client_golang/prometheus" + + "github.com/ava-labs/coreth/consensus/dummy" + "github.com/ava-labs/coreth/eth" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm/config" + "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/sync" + "github.com/ava-labs/coreth/sync/handlers" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" +) + +var ( + errNilConfig = errors.New("nil extension config") + errNilNetworkCodec = errors.New("nil network codec") + errNilSyncSummaryProvider = errors.New("nil sync summary provider") + errNilSyncableParser = errors.New("nil syncable parser") +) + +type ExtensibleVM interface { + // SetExtensionConfig sets the configuration for the VM extension + // Should be called before any other method and only once + SetExtensionConfig(config *Config) error + + // NewClient returns a client to send messages with for the given protocol + NewClient(protocol uint64, options ...p2p.ClientOption) *p2p.Client + // AddHandler registers a server handler for an application protocol + AddHandler(protocol uint64, handler p2p.Handler) error + // SetLastAcceptedBlock sets the last accepted block + SetLastAcceptedBlock(lastAcceptedBlock snowman.Block) error + // GetVMBlock returns the VMBlock for the given ID or an error if the block is not found + GetVMBlock(context.Context, ids.ID) (VMBlock, error) + // LastAcceptedVMBlock returns the last accepted VM block + LastAcceptedVMBlock() VMBlock + // IsBootstrapped returns true if the VM is bootstrapped + IsBootstrapped() bool + + // P2PValidators returns the validators for the network + P2PValidators() *p2p.Validators + // Ethereum returns the Ethereum client + Ethereum() *eth.Ethereum + // Config returns the configuration for the VM + Config() config.Config + // MetricRegistry returns the metric registry for the VM + MetricRegistry() *prometheus.Registry + // ReadLastAccepted returns the last accepted block hash and height + ReadLastAccepted() (common.Hash, uint64, error) + // VersionDB returns the versioned database for the VM + VersionDB() *versiondb.Database + // SyncerClient returns the syncer client for the VM + SyncerClient() sync.Client +} + +// InnerVM is the interface that must be implemented by the VM +// that's being wrapped by the extension +type InnerVM interface { + ExtensibleVM + avalanchecommon.VM + block.ChainVM + block.BuildBlockWithContextChainVM + block.StateSyncableVM +} + +// VMBlock is a block that can be used by the extension +type VMBlock interface { + snowman.Block + GetEthBlock() *types.Block + GetBlockExtension() BlockExtension +} + +type BlockExtender interface { + // NewBlockExtension is called when a new block is created + NewBlockExtension(b VMBlock) (BlockExtension, error) +} + +// BlockExtension allows the VM extension to handle block processing events. +type BlockExtension interface { + // SyntacticVerify verifies the block syntactically + // it can be implemented to extend inner block verification + SyntacticVerify(rules params.Rules) error + // SemanticVerify verifies the block semantically + // it can be implemented to extend inner block verification + SemanticVerify() error + // CleanupVerified is called when a block has passed SemanticVerify and SynctacticVerify, + // and should be cleaned up due to error or verification runs under non-write mode. This + // does not return an error because the block has already been verified. + CleanupVerified() + // OnAccept is called when a block is accepted by the block manager. OnAccept takes a + // database.Batch that contains the changes that were made to the database as a result + // of accepting the block. The changes in the batch should be flushed to the database in this method. + OnAccept(acceptedBatch database.Batch) error + // OnReject is called when a block is rejected by the block manager + OnReject() error +} + +// BuilderMempool is a mempool that's used in the block builder +type BuilderMempool interface { + // PendingLen returns the number of pending transactions + // that are waiting to be included in a block + PendingLen() int + // SubscribePendingTxs returns a channel that's signaled when there are pending transactions + SubscribePendingTxs() <-chan struct{} +} + +// LeafRequestConfig is the configuration to handle leaf requests +// in the network and syncer +type LeafRequestConfig struct { + // LeafType is the type of the leaf node + LeafType message.NodeType + // MetricName is the name of the metric to use for the leaf request + MetricName string + // Handler is the handler to use for the leaf request + Handler handlers.LeafRequestHandler +} + +// Config is the configuration for the VM extension +type Config struct { + // NetworkCodec is the codec manager to use + // for encoding and decoding network messages. + // It's required and should be non-nil + NetworkCodec codec.Manager + // ConsensusCallbacks is the consensus callbacks to use + // for the VM to be used in consensus engine. + // Callback functions can be nil. + ConsensusCallbacks dummy.ConsensusCallbacks + // SyncSummaryProvider is the sync summary provider to use + // for the VM to be used in syncer. + // It's required and should be non-nil + SyncSummaryProvider sync.SummaryProvider + // SyncExtender can extend the syncer to handle custom sync logic. + // It's optional and can be nil + SyncExtender sync.Extender + // SyncableParser is to parse summary messages from the network. + // It's required and should be non-nil + SyncableParser message.SyncableParser + // BlockExtender allows the VM extension to create an extension to handle block processing events. + // It's optional and can be nil + BlockExtender BlockExtender + // ExtraSyncLeafHandlerConfig is the extra configuration to handle leaf requests + // in the network and syncer. It's optional and can be nil + ExtraSyncLeafHandlerConfig *LeafRequestConfig + // ExtraMempool is the mempool to be used in the block builder. + // It's optional and can be nil + ExtraMempool BuilderMempool + // Clock is the clock to use for time related operations. + // It's optional and can be nil + Clock *mockable.Clock +} + +func (c *Config) Validate() error { + if c == nil { + return errNilConfig + } + if c.NetworkCodec == nil { + return errNilNetworkCodec + } + if c.SyncSummaryProvider == nil { + return errNilSyncSummaryProvider + } + if c.SyncableParser == nil { + return errNilSyncableParser + } + return nil +} diff --git a/plugin/evm/formatting.go b/plugin/evm/formatting.go deleted file mode 100644 index 3243635203..0000000000 --- a/plugin/evm/formatting.go +++ /dev/null @@ -1,52 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package evm - -import ( - "fmt" - - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/utils/constants" - "github.com/ava-labs/avalanchego/utils/formatting/address" -) - -// ParseServiceAddress get address ID from address string, being it either localized (using address manager, -// doing also components validations), or not localized. -// If both attempts fail, reports error from localized address parsing -func (vm *VM) ParseServiceAddress(addrStr string) (ids.ShortID, error) { - addr, err := ids.ShortFromString(addrStr) - if err == nil { - return addr, nil - } - return vm.ParseLocalAddress(addrStr) -} - -// ParseLocalAddress takes in an address for this chain and produces the ID -func (vm *VM) ParseLocalAddress(addrStr string) (ids.ShortID, error) { - chainID, addr, err := vm.ParseAddress(addrStr) - if err != nil { - return ids.ShortID{}, err - } - if chainID != vm.ctx.ChainID { - return ids.ShortID{}, fmt.Errorf("expected chainID to be %q but was %q", - vm.ctx.ChainID, chainID) - } - return addr, nil -} - -// FormatLocalAddress takes in a raw address and produces the formatted address -func (vm *VM) FormatLocalAddress(addr ids.ShortID) (string, error) { - return vm.FormatAddress(vm.ctx.ChainID, addr) -} - -// FormatAddress takes in a chainID and a raw address and produces the formatted -// address -func (vm *VM) FormatAddress(chainID ids.ID, addr ids.ShortID) (string, error) { - chainIDAlias, err := vm.ctx.BCLookup.PrimaryAlias(chainID) - if err != nil { - return "", err - } - hrp := constants.GetHRP(vm.ctx.NetworkID) - return address.Format(chainIDAlias, hrp, addr.Bytes()) -} diff --git a/plugin/evm/gossip/handler.go b/plugin/evm/gossip/handler.go new file mode 100644 index 0000000000..ff31f01989 --- /dev/null +++ b/plugin/evm/gossip/handler.go @@ -0,0 +1,67 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package gossip + +import ( + "context" + "time" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/network/p2p/gossip" + "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/utils/logging" +) + +var _ p2p.Handler = (*txGossipHandler)(nil) + +func NewTxGossipHandler[T gossip.Gossipable]( + log logging.Logger, + marshaller gossip.Marshaller[T], + mempool gossip.Set[T], + metrics gossip.Metrics, + maxMessageSize int, + throttlingPeriod time.Duration, + throttlingLimit int, + validators p2p.ValidatorSet, +) *txGossipHandler { + // push gossip messages can be handled from any peer + handler := gossip.NewHandler( + log, + marshaller, + mempool, + metrics, + maxMessageSize, + ) + + // pull gossip requests are filtered by validators and are throttled + // to prevent spamming + validatorHandler := p2p.NewValidatorHandler( + p2p.NewThrottlerHandler( + handler, + p2p.NewSlidingWindowThrottler(throttlingPeriod, throttlingLimit), + log, + ), + validators, + log, + ) + + return &txGossipHandler{ + appGossipHandler: handler, + appRequestHandler: validatorHandler, + } +} + +type txGossipHandler struct { + appGossipHandler p2p.Handler + appRequestHandler p2p.Handler +} + +func (t *txGossipHandler) AppGossip(ctx context.Context, nodeID ids.NodeID, gossipBytes []byte) { + t.appGossipHandler.AppGossip(ctx, nodeID, gossipBytes) +} + +func (t *txGossipHandler) AppRequest(ctx context.Context, nodeID ids.NodeID, deadline time.Time, requestBytes []byte) ([]byte, *common.AppError) { + return t.appRequestHandler.AppRequest(ctx, nodeID, deadline, requestBytes) +} diff --git a/plugin/evm/gossiper_eth_gossiping_test.go b/plugin/evm/gossiper_eth_gossiping_test.go index fefe1f3a10..f1fb573d53 100644 --- a/plugin/evm/gossiper_eth_gossiping_test.go +++ b/plugin/evm/gossiper_eth_gossiping_test.go @@ -25,6 +25,7 @@ import ( "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/libevm/core/types" ) @@ -83,27 +84,28 @@ func TestMempoolEthTxsAppGossipHandling(t *testing.T) { genesisJSON, err := fundAddressByGenesis([]common.Address{addr}) assert.NoError(err) - tvm := newVM(t, testVMConfig{ - genesisJSON: genesisJSON, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + GenesisJSON: genesisJSON, }) + defer func() { - err := tvm.vm.Shutdown(context.Background()) + err := vm.Shutdown(context.Background()) assert.NoError(err) }() - tvm.vm.txPool.SetGasTip(common.Big1) - tvm.vm.txPool.SetMinFee(common.Big0) + vm.txPool.SetGasTip(common.Big1) + vm.txPool.SetMinFee(common.Big0) var ( wg sync.WaitGroup txRequested bool ) - tvm.appSender.CantSendAppGossip = false - tvm.appSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { + tvm.AppSender.CantSendAppGossip = false + tvm.AppSender.SendAppRequestF = func(context.Context, set.Set[ids.NodeID], uint32, []byte) error { txRequested = true return nil } wg.Add(1) - tvm.appSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { + tvm.AppSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { wg.Done() return nil } @@ -113,7 +115,7 @@ func TestMempoolEthTxsAppGossipHandling(t *testing.T) { // Txs must be submitted over the API to be included in push gossip. // (i.e., txs received via p2p are not included in push gossip) - err = tvm.vm.eth.APIBackend.SendTx(context.Background(), tx) + err = vm.eth.APIBackend.SendTx(context.Background(), tx) assert.NoError(err) assert.False(txRequested, "tx should not be requested") diff --git a/plugin/evm/message/block_request_test.go b/plugin/evm/message/block_request_test.go index 17650a74c7..10cbebda20 100644 --- a/plugin/evm/message/block_request_test.go +++ b/plugin/evm/message/block_request_test.go @@ -23,12 +23,12 @@ func TestMarshalBlockRequest(t *testing.T) { base64BlockRequest := "AAAAAAAAAAAAAAAAAABzb21lIGhhc2ggaXMgaGVyZSB5bwAAAAAAAAU5AEA=" - blockRequestBytes, err := Codec.Marshal(Version, blockRequest) + blockRequestBytes, err := codecWithBlockSync.Marshal(Version, blockRequest) assert.NoError(t, err) assert.Equal(t, base64BlockRequest, base64.StdEncoding.EncodeToString(blockRequestBytes)) var b BlockRequest - _, err = Codec.Unmarshal(blockRequestBytes, &b) + _, err = codecWithBlockSync.Unmarshal(blockRequestBytes, &b) assert.NoError(t, err) assert.Equal(t, blockRequest.Hash, b.Hash) assert.Equal(t, blockRequest.Height, b.Height) @@ -54,12 +54,12 @@ func TestMarshalBlockResponse(t *testing.T) { base64BlockResponse := "AAAAAAAgAAAAIU8WP18PmmIdcpVmx00QA3xNe7sEB9HixkmBhVrYaB0NhgAAADnR6ZTSxCKs0gigByk5SH9pmeudGKRHhARdh/PGfPInRumVr1olNnlRuqL/bNRxxIPxX7kLrbN8WCEAAAA6tmgLTnyLdjobHUnUlVyEhiFjJSU/7HON16nii/khEZwWDwcCRIYVu9oIMT9qjrZo0gv1BZh1kh5migAAACtb3yx/xIRo0tbFL1BU4tCDa/hMcXTLdHY2TMPb2Wiw9xcu2FeUuzWLDDtSAAAAO12heG+f69ehnQ97usvgJVqlt9RL7ED4TIkrm//UNimwIjvupfT3Q5H0RdFa/UKUBAN09pJLmMv4cT+NAAAAMpYtJOLK/Mrjph+1hrFDI6a8j5598dkpMz/5k5M76m9bOvbeA3Q2bEcZ5DobBn2JvH8BAAAAOfHxekxyFaO1OeseWEnGB327VyL1cXoomiZvl2R5gZmOvqicC0s3OXARXoLtb0ElyPpzEeTX3vqSLQAAACc2zU8kq/ffhmuqVgODZ61hRd4e6PSosJk+vfiIOgrYvpw5eLBIg+UAAAAkahVqnexqQOmh0AfwM8KCMGG90Oqln45NpkMBBSINCyloi3NLAAAAKI6gENd8luqAp6Zl9gb2pjt/Pf0lZ8GJeeTWDyZobZvy+ybJAf81TN4AAAA8FgfuKbpk+Eq0PKDG5rkcH9O+iZBDQXnTr0SRo2kBLbktGE/DnRc0/1cWQolTu2hl/PkrDDoXyQKL6ZFOAAAAMwl50YMDVvKlTD3qsqS0R11jr76PtWmHx39YGFJvGBS+gjNQ6rE5NfMdhEhFF+kkrveK4QAAADhRwAdVkgww7CmjcDk0v1CijaECl13tp351hXnqPf5BNqv3UrO4Jx0D6USzyds2a3UEX479adIq5QAAADpBGUfLVbzqQGsy1hCL1oWE9X43yqxuM/6qMmOjmUNwJLqcmxRniidPAakQrilfbvv+X1q/RMzeJjtWAAAAKAZjPn05Bp8BojnENlhUw69/a0HWMfkrmo0S9BJXMl//My91drBiBVYAAAAqMEo+Pq6QGlJyDahcoeSzjq8/RMbG74Ni8vVPwA4J1vwlZAhUwV38rKqKAAAAOyzszlo6lLTTOKUUPmNAjYcksM8/rhej95vhBy+2PDXWBCxBYPOO6eKp8/tP+wAZtFTVIrX/oXYEGT+4AAAAMpZnz1PD9SDIibeb9QTPtXx2ASMtWJuszqnW4mPiXCd0HT9sYsu7FdmvvL9/faQasECOAAAALzk4vxd0rOdwmk8JHpqD/erg7FXrIzqbU5TLPHhWtUbTE8ijtMHA4FRH9Lo3DrNtAAAAPLz97PUi4qbx7Qr+wfjiD6q+32sWLnF9OnSKWGd6DFY0j4khomaxHQ8zTGL+UrpTrxl3nLKUi2Vw/6C3cwAAADqWPBMK15dRJSEPDvHDFAkPB8eab1ccJG8+msC3QT7xEL1YsAznO/9wb3/0tvRAkKMnEfMgjk5LictRAAAAJ2XOZAA98kaJKNWiO5ynQPgMk4LZxgNK0pYMeWUD4c4iFyX1DK8fvwAAADtcR6U9v459yvyeE4ZHpLRO1LzpZO1H90qllEaM7TI8t28NP6xHbJ+wP8kij7roj9WAZjoEVLaDEiB/CgAAADc7WExi1QJ84VpPClglDY+1Dnfyv08BUuXUlDWAf51Ll75vt3lwRmpWJv4zQIz56I4seXQIoy0pAAAAKkFrryBqmDIJgsharXA4SFnAWksTodWy9b/vWm7ZLaSCyqlWjltv6dip3QAAAC7Z6wkne1AJRMvoAKCxUn6mRymoYdL2SXoyNcN/QZJ3nsHZazscVCT84LcnsDByAAAAI+ZAq8lEj93rIZHZRcBHZ6+Eev0O212IV7eZrLGOSv+r4wN/AAAAL/7MQW5zTTc8Xr68nNzFlbzOPHvT2N+T+rfhJd3rr+ZaMb1dQeLSzpwrF4kvD+oZAAAAMTGikNy/poQG6HcHP/CINOGXpANKpIr6P4W4picIyuu6yIC1uJuT2lOBAWRAIQTmSLYAAAA1ImobDzE6id38RUxfj3KsibOLGfU3hMGem+rAPIdaJ9sCneN643pCMYgTSHaFkpNZyoxeuU4AAAA9FS3Br0LquOKSXG2u5N5e+fnc8I38vQK4CAk5hYWSig995QvhptwdV2joU3mI/dzlYum5SMkYu6PpM+XEAAAAAC3Nrne6HSWbGIpLIchvvCPXKLRTR+raZQryTFbQgAqGkTMgiKgFvVXERuJesHU=" - blockResponseBytes, err := Codec.Marshal(Version, blockResponse) + blockResponseBytes, err := codecWithBlockSync.Marshal(Version, blockResponse) assert.NoError(t, err) assert.Equal(t, base64BlockResponse, base64.StdEncoding.EncodeToString(blockResponseBytes)) var b BlockResponse - _, err = Codec.Unmarshal(blockResponseBytes, &b) + _, err = codecWithBlockSync.Unmarshal(blockResponseBytes, &b) assert.NoError(t, err) assert.Equal(t, blockResponse.Blocks, b.Blocks) } diff --git a/plugin/evm/message/block_sync_summary.go b/plugin/evm/message/block_sync_summary.go new file mode 100644 index 0000000000..da9df7aefc --- /dev/null +++ b/plugin/evm/message/block_sync_summary.go @@ -0,0 +1,119 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package message + +import ( + "context" + "fmt" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/crypto" +) + +var _ Syncable = (*BlockSyncSummary)(nil) + +// codecWithBlockSync is the codec manager that contains the codec for BlockSyncSummary and +// other message types that are used in the network protocol. This is to ensure that the codec +// version is consistent across all message types and includes the codec for BlockSyncSummary. +var codecWithBlockSync codec.Manager + +func init() { + var err error + codecWithBlockSync, err = NewCodec(BlockSyncSummary{}) + if err != nil { + panic(fmt.Errorf("failed to create codec manager: %w", err)) + } +} + +// BlockSyncSummary provides the information necessary to sync a node starting +// at the given block. +type BlockSyncSummary struct { + BlockNumber uint64 `serialize:"true"` + BlockHash common.Hash `serialize:"true"` + BlockRoot common.Hash `serialize:"true"` + + summaryID ids.ID + bytes []byte + acceptImpl AcceptImplFn +} + +type BlockSyncSummaryParser struct{} + +func NewBlockSyncSummaryParser() *BlockSyncSummaryParser { + return &BlockSyncSummaryParser{} +} + +func (b *BlockSyncSummaryParser) ParseFromBytes(summaryBytes []byte, acceptImpl AcceptImplFn) (Syncable, error) { + summary := BlockSyncSummary{} + if codecVersion, err := codecWithBlockSync.Unmarshal(summaryBytes, &summary); err != nil { + return nil, fmt.Errorf("failed to parse syncable summary: %w", err) + } else if codecVersion != Version { + return nil, fmt.Errorf("failed to parse syncable summary due to unexpected codec version (%d != %d)", codecVersion, Version) + } + + summary.bytes = summaryBytes + summaryID, err := ids.ToID(crypto.Keccak256(summaryBytes)) + if err != nil { + return nil, fmt.Errorf("failed to compute summary ID: %w", err) + } + summary.summaryID = summaryID + summary.acceptImpl = acceptImpl + return &summary, nil +} + +func NewBlockSyncSummary(blockHash common.Hash, blockNumber uint64, blockRoot common.Hash) (*BlockSyncSummary, error) { + summary := BlockSyncSummary{ + BlockNumber: blockNumber, + BlockHash: blockHash, + BlockRoot: blockRoot, + } + bytes, err := codecWithBlockSync.Marshal(Version, &summary) + if err != nil { + return nil, fmt.Errorf("failed to marshal syncable summary: %w", err) + } + + summary.bytes = bytes + summaryID, err := ids.ToID(crypto.Keccak256(bytes)) + if err != nil { + return nil, fmt.Errorf("failed to compute summary ID: %w", err) + } + summary.summaryID = summaryID + + return &summary, nil +} + +func (s *BlockSyncSummary) GetBlockHash() common.Hash { + return s.BlockHash +} + +func (s *BlockSyncSummary) GetBlockRoot() common.Hash { + return s.BlockRoot +} + +func (s *BlockSyncSummary) Bytes() []byte { + return s.bytes +} + +func (s *BlockSyncSummary) Height() uint64 { + return s.BlockNumber +} + +func (s *BlockSyncSummary) ID() ids.ID { + return s.summaryID +} + +func (s *BlockSyncSummary) String() string { + return fmt.Sprintf("BlockSyncSummary(BlockHash=%s, BlockNumber=%d, BlockRoot=%s)", s.BlockHash, s.BlockNumber, s.BlockRoot) +} + +func (s *BlockSyncSummary) Accept(context.Context) (block.StateSyncMode, error) { + if s.acceptImpl == nil { + return block.StateSyncSkipped, fmt.Errorf("accept implementation not specified for summary: %s", s) + } + return s.acceptImpl(s) +} diff --git a/plugin/evm/message/block_sync_summary_provider.go b/plugin/evm/message/block_sync_summary_provider.go new file mode 100644 index 0000000000..959ae4d0b7 --- /dev/null +++ b/plugin/evm/message/block_sync_summary_provider.go @@ -0,0 +1,16 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package message + +import ( + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + + "github.com/ava-labs/libevm/core/types" +) + +type BlockSyncSummaryProvider struct{} + +// StateSummaryAtBlock returns the block state summary at [block] if valid. +func (a *BlockSyncSummaryProvider) StateSummaryAtBlock(blk *types.Block) (block.StateSummary, error) { + return NewBlockSyncSummary(blk.Hash(), blk.NumberU64(), blk.Root()) +} diff --git a/plugin/evm/message/block_sync_summary_test.go b/plugin/evm/message/block_sync_summary_test.go new file mode 100644 index 0000000000..e84e08fad1 --- /dev/null +++ b/plugin/evm/message/block_sync_summary_test.go @@ -0,0 +1,44 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package message + +import ( + "context" + "encoding/base64" + "testing" + + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/libevm/common" + "github.com/stretchr/testify/require" +) + +func TestMarshalBlockSyncSummary(t *testing.T) { + blockSyncSummary, err := NewBlockSyncSummary(common.Hash{1}, 2, common.Hash{3}) + require.NoError(t, err) + + require.Equal(t, common.Hash{1}, blockSyncSummary.GetBlockHash()) + require.Equal(t, uint64(2), blockSyncSummary.Height()) + require.Equal(t, common.Hash{3}, blockSyncSummary.GetBlockRoot()) + + expectedBase64Bytes := "AAAAAAAAAAAAAgEAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAwAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA=" + require.Equal(t, expectedBase64Bytes, base64.StdEncoding.EncodeToString(blockSyncSummary.Bytes())) + + parser := NewBlockSyncSummaryParser() + called := false + acceptImplTest := func(Syncable) (block.StateSyncMode, error) { + called = true + return block.StateSyncSkipped, nil + } + s, err := parser.ParseFromBytes(blockSyncSummary.Bytes(), acceptImplTest) + require.NoError(t, err) + require.Equal(t, blockSyncSummary.GetBlockHash(), s.GetBlockHash()) + require.Equal(t, blockSyncSummary.Height(), s.Height()) + require.Equal(t, blockSyncSummary.GetBlockRoot(), s.GetBlockRoot()) + require.Equal(t, blockSyncSummary.Bytes(), s.Bytes()) + + mode, err := s.Accept(context.TODO()) + require.NoError(t, err) + require.Equal(t, block.StateSyncSkipped, mode) + require.True(t, called) +} diff --git a/plugin/evm/message/code_request_test.go b/plugin/evm/message/code_request_test.go index c8825453fe..23d4912e28 100644 --- a/plugin/evm/message/code_request_test.go +++ b/plugin/evm/message/code_request_test.go @@ -21,12 +21,12 @@ func TestMarshalCodeRequest(t *testing.T) { base64CodeRequest := "AAAAAAABAAAAAAAAAAAAAAAAAAAAAAAAAHNvbWUgY29kZSBwbHM=" - codeRequestBytes, err := Codec.Marshal(Version, codeRequest) + codeRequestBytes, err := codecWithBlockSync.Marshal(Version, codeRequest) assert.NoError(t, err) assert.Equal(t, base64CodeRequest, base64.StdEncoding.EncodeToString(codeRequestBytes)) var c CodeRequest - _, err = Codec.Unmarshal(codeRequestBytes, &c) + _, err = codecWithBlockSync.Unmarshal(codeRequestBytes, &c) assert.NoError(t, err) assert.Equal(t, codeRequest.Hashes, c.Hashes) } @@ -47,12 +47,12 @@ func TestMarshalCodeResponse(t *testing.T) { base64CodeResponse := "AAAAAAABAAAAMlL9/AchgmVPFj9fD5piHXKVZsdNEAN8TXu7BAfR4sZJgYVa2GgdDYbR6R4AFnk5y2aU" - codeResponseBytes, err := Codec.Marshal(Version, codeResponse) + codeResponseBytes, err := codecWithBlockSync.Marshal(Version, codeResponse) assert.NoError(t, err) assert.Equal(t, base64CodeResponse, base64.StdEncoding.EncodeToString(codeResponseBytes)) var c CodeResponse - _, err = Codec.Unmarshal(codeResponseBytes, &c) + _, err = codecWithBlockSync.Unmarshal(codeResponseBytes, &c) assert.NoError(t, err) assert.Equal(t, codeResponse.Data, c.Data) } diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 780c2a7653..1774738237 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -15,10 +15,11 @@ const ( maxMessageSize = 2*units.MiB - 64*units.KiB // Subtract 64 KiB from p2p network cap to leave room for encoding overhead from AvalancheGo ) -var Codec codec.Manager - -func init() { - Codec = codec.NewManager(maxMessageSize) +// NewCodec returns a codec manager that can be used to marshal and unmarshal +// messages, including the provided syncSummaryType. syncSummaryType can be used +// to register a type for sync summaries. +func NewCodec(syncSummaryType interface{}) (codec.Manager, error) { + codec := codec.NewManager(maxMessageSize) c := linearcodec.NewDefault() errs := wrappers.Errs{} @@ -26,7 +27,7 @@ func init() { c.SkipRegistrations(2) errs.Add( // Types for state sync frontier consensus - c.RegisterType(SyncSummary{}), + c.RegisterType(syncSummaryType), // state sync types c.RegisterType(BlockRequest{}), @@ -41,10 +42,12 @@ func init() { c.RegisterType(BlockSignatureRequest{}), c.RegisterType(SignatureResponse{}), - Codec.RegisterCodec(Version, c), + codec.RegisterCodec(Version, c), ) if errs.Errored() { - panic(errs.Err) + return nil, errs.Err } + + return codec, nil } diff --git a/plugin/evm/message/handler.go b/plugin/evm/message/handler.go index 0df91a481f..ac39bdc761 100644 --- a/plugin/evm/message/handler.go +++ b/plugin/evm/message/handler.go @@ -15,10 +15,8 @@ var _ RequestHandler = NoopRequestHandler{} // Must have methods in format of handleType(context.Context, ids.NodeID, uint32, request Type) error // so that the Request object of relevant Type can invoke its respective handle method // on this struct. -// Also see GossipHandler for implementation style. type RequestHandler interface { - HandleStateTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) - HandleAtomicTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) + HandleLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) HandleBlockRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, request BlockRequest) ([]byte, error) HandleCodeRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, codeRequest CodeRequest) ([]byte, error) HandleMessageSignatureRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, signatureRequest MessageSignatureRequest) ([]byte, error) @@ -36,11 +34,7 @@ type ResponseHandler interface { type NoopRequestHandler struct{} -func (NoopRequestHandler) HandleStateTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) { - return nil, nil -} - -func (NoopRequestHandler) HandleAtomicTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) { +func (NoopRequestHandler) HandleLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest LeafsRequest) ([]byte, error) { return nil, nil } diff --git a/plugin/evm/message/leafs_request.go b/plugin/evm/message/leafs_request.go index 696f74c4fc..e21e94988a 100644 --- a/plugin/evm/message/leafs_request.go +++ b/plugin/evm/message/leafs_request.go @@ -9,7 +9,6 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/log" ) const MaxCodeHashesPerRequest = 5 @@ -18,27 +17,14 @@ var _ Request = LeafsRequest{} // NodeType outlines the trie that a leaf node belongs to // handlers.LeafsRequestHandler uses this information to determine -// which of the two tries (state/atomic) to fetch the information from +// which trie type to fetch the information from type NodeType uint8 const ( - // StateTrieNode represents a leaf node that belongs to the coreth State trie - StateTrieNode NodeType = iota + 1 - // AtomicTrieNode represents a leaf node that belongs to the coreth evm.AtomicTrie - AtomicTrieNode + StateTrieNode = NodeType(1) + StateTrieKeyLength = common.HashLength ) -func (nt NodeType) String() string { - switch nt { - case StateTrieNode: - return "StateTrie" - case AtomicTrieNode: - return "AtomicTrie" - default: - return "Unknown" - } -} - // LeafsRequest is a request to receive trie leaves at specified Root within Start and End byte range // Limit outlines maximum number of leaves to returns starting at Start // NodeType outlines which trie to read from state/atomic. @@ -53,21 +39,13 @@ type LeafsRequest struct { func (l LeafsRequest) String() string { return fmt.Sprintf( - "LeafsRequest(Root=%s, Account=%s, Start=%s, End=%s, Limit=%d, NodeType=%s)", + "LeafsRequest(Root=%s, Account=%s, Start=%s, End=%s, Limit=%d, NodeType=%d)", l.Root, l.Account, common.Bytes2Hex(l.Start), common.Bytes2Hex(l.End), l.Limit, l.NodeType, ) } func (l LeafsRequest) Handle(ctx context.Context, nodeID ids.NodeID, requestID uint32, handler RequestHandler) ([]byte, error) { - switch l.NodeType { - case StateTrieNode: - return handler.HandleStateTrieLeafsRequest(ctx, nodeID, requestID, l) - case AtomicTrieNode: - return handler.HandleAtomicTrieLeafsRequest(ctx, nodeID, requestID, l) - } - - log.Debug("node type is not recognised, dropping request", "nodeID", nodeID, "requestID", requestID, "nodeType", l.NodeType) - return nil, nil + return handler.HandleLeafsRequest(ctx, nodeID, requestID, l) } // LeafsResponse is a response to a LeafsRequest diff --git a/plugin/evm/message/leafs_request_test.go b/plugin/evm/message/leafs_request_test.go index 36614dd2cb..ce1c806db6 100644 --- a/plugin/evm/message/leafs_request_test.go +++ b/plugin/evm/message/leafs_request_test.go @@ -4,14 +4,10 @@ package message import ( - "bytes" - "context" "encoding/base64" "math/rand" "testing" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/libevm/common" "github.com/stretchr/testify/assert" ) @@ -37,17 +33,17 @@ func TestMarshalLeafsRequest(t *testing.T) { Start: startBytes, End: endBytes, Limit: 1024, - NodeType: StateTrieNode, + NodeType: NodeType(1), } base64LeafsRequest := "AAAAAAAAAAAAAAAAAAAAAABpbSBST09UaW5nIGZvciB5YQAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAIFL9/AchgmVPFj9fD5piHXKVZsdNEAN8TXu7BAfR4sZJAAAAIIGFWthoHQ2G0ekeABZ5OctmlNLEIqzSCKAHKTlIf2mZBAAB" - leafsRequestBytes, err := Codec.Marshal(Version, leafsRequest) + leafsRequestBytes, err := codecWithBlockSync.Marshal(Version, leafsRequest) assert.NoError(t, err) assert.Equal(t, base64LeafsRequest, base64.StdEncoding.EncodeToString(leafsRequestBytes)) var l LeafsRequest - _, err = Codec.Unmarshal(leafsRequestBytes, &l) + _, err = codecWithBlockSync.Unmarshal(leafsRequestBytes, &l) assert.NoError(t, err) assert.Equal(t, leafsRequest.Root, l.Root) assert.Equal(t, leafsRequest.Start, l.Start) @@ -96,125 +92,15 @@ func TestMarshalLeafsResponse(t *testing.T) { base64LeafsResponse := "AAAAAAAQAAAAIE8WP18PmmIdcpVmx00QA3xNe7sEB9HixkmBhVrYaB0NAAAAIGagByk5SH9pmeudGKRHhARdh/PGfPInRumVr1olNnlRAAAAIK2zfFghtmgLTnyLdjobHUnUlVyEhiFjJSU/7HON16niAAAAIIYVu9oIMfUFmHWSHmaKW98sf8SERZLSVyvNBmjS1sUvAAAAIHHb2Wiw9xcu2FeUuzWLDDtSXaF4b5//CUJ52xlE69ehAAAAIPhMiSs77qX090OR9EXRWv1ClAQDdPaSS5jL+HE/jZYtAAAAIMr8yuOmvI+effHZKTM/+ZOTO+pvWzr23gN0NmxHGeQ6AAAAIBZZpE856x5YScYHfbtXIvVxeiiaJm+XZHmBmY6+qJwLAAAAIHOq53hmZ/fpNs1PJKv334ZrqlYDg2etYUXeHuj0qLCZAAAAIHiN5WOvpGfUnexqQOmh0AfwM8KCMGG90Oqln45NpkMBAAAAIKAQ13yW6oCnpmX2BvamO389/SVnwYl55NYPJmhtm/L7AAAAIAfuKbpk+Eq0PKDG5rkcH9O+iZBDQXnTr0SRo2kBLbktAAAAILsXyQKL6ZFOt2ScbJNHgAl50YMDVvKlTD3qsqS0R11jAAAAIOqxOTXzHYRIRRfpJK73iuFRwAdVklg2twdYhWUMMOwpAAAAIHnqPf5BNqv3UrO4Jx0D6USzyds2a3UEX479adIq5UEZAAAAIDLWEMqsbjP+qjJjo5lDcCS6nJsUZ4onTwGpEK4pX277AAAAEAAAAAmG0ekeABZ5OcsAAAAMuqL/bNRxxIPxX7kLAAAACov5IRGcFg8HAkQAAAAIUFTi0INr+EwAAAAOnQ97usvgJVqlt9RL7EAAAAAJfI0BkZLCQiTiAAAACxsGfYm8fwHx9XOYAAAADUs3OXARXoLtb0ElyPoAAAAKPr34iDoK2L6cOQAAAAoFIg0LKWiLc0uOAAAACCbJAf81TN4WAAAADBhPw50XNP9XFkKJUwAAAAuvvo+1aYfHf1gYUgAAAAqjcDk0v1CijaECAAAADkfLVT12lCZ670686kBrAAAADf5fWr9EzN4mO1YGYz4AAAAEAAAADlcyXwVWMEo+Pq4Uwo0MAAAADeo50qHks46vP0TGxu8AAAAOg2Ly9WQIVMFd/KyqiiwAAAAL7M5aOpS00zilFD4=" - leafsResponseBytes, err := Codec.Marshal(Version, leafsResponse) + leafsResponseBytes, err := codecWithBlockSync.Marshal(Version, leafsResponse) assert.NoError(t, err) assert.Equal(t, base64LeafsResponse, base64.StdEncoding.EncodeToString(leafsResponseBytes)) var l LeafsResponse - _, err = Codec.Unmarshal(leafsResponseBytes, &l) + _, err = codecWithBlockSync.Unmarshal(leafsResponseBytes, &l) assert.NoError(t, err) assert.Equal(t, leafsResponse.Keys, l.Keys) assert.Equal(t, leafsResponse.Vals, l.Vals) assert.False(t, l.More) // make sure it is not serialized assert.Equal(t, leafsResponse.ProofVals, l.ProofVals) } - -func TestLeafsRequestValidation(t *testing.T) { - mockRequestHandler := &mockHandler{} - - tests := map[string]struct { - request LeafsRequest - assertResponse func(t *testing.T) - }{ - "node type StateTrieNode": { - request: LeafsRequest{ - Root: common.BytesToHash([]byte("some hash goes here")), - Start: bytes.Repeat([]byte{0x00}, common.HashLength), - End: bytes.Repeat([]byte{0xff}, common.HashLength), - Limit: 10, - NodeType: StateTrieNode, - }, - assertResponse: func(t *testing.T) { - assert.True(t, mockRequestHandler.handleStateTrieCalled) - assert.False(t, mockRequestHandler.handleAtomicTrieCalled) - assert.False(t, mockRequestHandler.handleBlockRequestCalled) - assert.False(t, mockRequestHandler.handleCodeRequestCalled) - }, - }, - "node type AtomicTrieNode": { - request: LeafsRequest{ - Root: common.BytesToHash([]byte("some hash goes here")), - Start: bytes.Repeat([]byte{0x00}, common.HashLength), - End: bytes.Repeat([]byte{0xff}, common.HashLength), - Limit: 10, - NodeType: AtomicTrieNode, - }, - assertResponse: func(t *testing.T) { - assert.False(t, mockRequestHandler.handleStateTrieCalled) - assert.True(t, mockRequestHandler.handleAtomicTrieCalled) - assert.False(t, mockRequestHandler.handleBlockRequestCalled) - assert.False(t, mockRequestHandler.handleCodeRequestCalled) - }, - }, - "unknown node type": { - request: LeafsRequest{ - Root: common.BytesToHash([]byte("some hash goes here")), - Start: bytes.Repeat([]byte{0x00}, common.HashLength), - End: bytes.Repeat([]byte{0xff}, common.HashLength), - Limit: 10, - NodeType: NodeType(11), - }, - assertResponse: func(t *testing.T) { - assert.False(t, mockRequestHandler.handleStateTrieCalled) - assert.False(t, mockRequestHandler.handleAtomicTrieCalled) - assert.False(t, mockRequestHandler.handleBlockRequestCalled) - assert.False(t, mockRequestHandler.handleCodeRequestCalled) - }, - }, - } - for name, test := range tests { - t.Run(name, func(t *testing.T) { - _, _ = test.request.Handle(context.Background(), ids.GenerateTestNodeID(), 1, mockRequestHandler) - test.assertResponse(t) - mockRequestHandler.reset() - }) - } -} - -var _ RequestHandler = (*mockHandler)(nil) - -type mockHandler struct { - handleStateTrieCalled, - handleAtomicTrieCalled, - handleBlockRequestCalled, - handleCodeRequestCalled, - handleMessageSignatureCalled, - handleBlockSignatureCalled bool -} - -func (m *mockHandler) HandleStateTrieLeafsRequest(context.Context, ids.NodeID, uint32, LeafsRequest) ([]byte, error) { - m.handleStateTrieCalled = true - return nil, nil -} - -func (m *mockHandler) HandleAtomicTrieLeafsRequest(context.Context, ids.NodeID, uint32, LeafsRequest) ([]byte, error) { - m.handleAtomicTrieCalled = true - return nil, nil -} - -func (m *mockHandler) HandleBlockRequest(context.Context, ids.NodeID, uint32, BlockRequest) ([]byte, error) { - m.handleBlockRequestCalled = true - return nil, nil -} - -func (m *mockHandler) HandleCodeRequest(context.Context, ids.NodeID, uint32, CodeRequest) ([]byte, error) { - m.handleCodeRequestCalled = true - return nil, nil -} - -func (m *mockHandler) HandleMessageSignatureRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, signatureRequest MessageSignatureRequest) ([]byte, error) { - m.handleMessageSignatureCalled = true - return nil, nil -} - -func (m *mockHandler) HandleBlockSignatureRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, signatureRequest BlockSignatureRequest) ([]byte, error) { - m.handleBlockSignatureCalled = true - return nil, nil -} - -func (m *mockHandler) reset() { - m.handleStateTrieCalled = false - m.handleAtomicTrieCalled = false - m.handleBlockRequestCalled = false - m.handleCodeRequestCalled = false -} diff --git a/plugin/evm/message/messagetest/test_block_sync_summary.go b/plugin/evm/message/messagetest/test_block_sync_summary.go new file mode 100644 index 0000000000..aaf4ca9da2 --- /dev/null +++ b/plugin/evm/message/messagetest/test_block_sync_summary.go @@ -0,0 +1,18 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package messagetest + +import ( + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/coreth/plugin/evm/message" +) + +var BlockSyncSummaryCodec codec.Manager + +func init() { + var err error + BlockSyncSummaryCodec, err = message.NewCodec(message.BlockSyncSummary{}) + if err != nil { + panic(err) + } +} diff --git a/plugin/evm/message/signature_request_test.go b/plugin/evm/message/signature_request_test.go index dea154e16b..a3f7894eb7 100644 --- a/plugin/evm/message/signature_request_test.go +++ b/plugin/evm/message/signature_request_test.go @@ -21,12 +21,12 @@ func TestMarshalMessageSignatureRequest(t *testing.T) { } base64MessageSignatureRequest := "AABET0ZBSElAawAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - signatureRequestBytes, err := Codec.Marshal(Version, signatureRequest) + signatureRequestBytes, err := codecWithBlockSync.Marshal(Version, signatureRequest) require.NoError(t, err) require.Equal(t, base64MessageSignatureRequest, base64.StdEncoding.EncodeToString(signatureRequestBytes)) var s MessageSignatureRequest - _, err = Codec.Unmarshal(signatureRequestBytes, &s) + _, err = codecWithBlockSync.Unmarshal(signatureRequestBytes, &s) require.NoError(t, err) require.Equal(t, signatureRequest.MessageID, s.MessageID) } @@ -39,12 +39,12 @@ func TestMarshalBlockSignatureRequest(t *testing.T) { } base64BlockSignatureRequest := "AABET0ZBSElAawAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAAA==" - signatureRequestBytes, err := Codec.Marshal(Version, signatureRequest) + signatureRequestBytes, err := codecWithBlockSync.Marshal(Version, signatureRequest) require.NoError(t, err) require.Equal(t, base64BlockSignatureRequest, base64.StdEncoding.EncodeToString(signatureRequestBytes)) var s BlockSignatureRequest - _, err = Codec.Unmarshal(signatureRequestBytes, &s) + _, err = codecWithBlockSync.Unmarshal(signatureRequestBytes, &s) require.NoError(t, err) require.Equal(t, signatureRequest.BlockID, s.BlockID) } @@ -62,12 +62,12 @@ func TestMarshalSignatureResponse(t *testing.T) { } base64SignatureResponse := "AAABI0VniavN7wEjRWeJq83vASNFZ4mrze8BI0VniavN7wEjRWeJq83vASNFZ4mrze8BI0VniavN7wEjRWeJq83vASNFZ4mrze8BI0VniavN7wEjRWeJq83vASNFZ4mrze8=" - signatureResponseBytes, err := Codec.Marshal(Version, signatureResponse) + signatureResponseBytes, err := codecWithBlockSync.Marshal(Version, signatureResponse) require.NoError(t, err) require.Equal(t, base64SignatureResponse, base64.StdEncoding.EncodeToString(signatureResponseBytes)) var s SignatureResponse - _, err = Codec.Unmarshal(signatureResponseBytes, &s) + _, err = codecWithBlockSync.Unmarshal(signatureResponseBytes, &s) require.NoError(t, err) require.Equal(t, signatureResponse.Signature, s.Signature) } diff --git a/plugin/evm/message/syncable.go b/plugin/evm/message/syncable.go index 2d729ab956..4fa5cdfbc3 100644 --- a/plugin/evm/message/syncable.go +++ b/plugin/evm/message/syncable.go @@ -4,90 +4,19 @@ package message import ( - "context" - "fmt" - - "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/crypto" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" ) -var _ block.StateSummary = (*SyncSummary)(nil) - -// SyncSummary provides the information necessary to sync a node starting -// at the given block. -type SyncSummary struct { - BlockNumber uint64 `serialize:"true"` - BlockHash common.Hash `serialize:"true"` - BlockRoot common.Hash `serialize:"true"` - AtomicRoot common.Hash `serialize:"true"` - - summaryID ids.ID - bytes []byte - acceptImpl func(SyncSummary) (block.StateSyncMode, error) +type Syncable interface { + block.StateSummary + GetBlockHash() common.Hash + GetBlockRoot() common.Hash } -func NewSyncSummaryFromBytes(summaryBytes []byte, acceptImpl func(SyncSummary) (block.StateSyncMode, error)) (SyncSummary, error) { - summary := SyncSummary{} - if codecVersion, err := Codec.Unmarshal(summaryBytes, &summary); err != nil { - return SyncSummary{}, err - } else if codecVersion != Version { - return SyncSummary{}, fmt.Errorf("failed to parse syncable summary due to unexpected codec version (%d != %d)", codecVersion, Version) - } - - summary.bytes = summaryBytes - summaryID, err := ids.ToID(crypto.Keccak256(summaryBytes)) - if err != nil { - return SyncSummary{}, err - } - summary.summaryID = summaryID - summary.acceptImpl = acceptImpl - return summary, nil +type SyncableParser interface { + ParseFromBytes(summaryBytes []byte, acceptImpl AcceptImplFn) (Syncable, error) } -func NewSyncSummary(blockHash common.Hash, blockNumber uint64, blockRoot common.Hash, atomicRoot common.Hash) (SyncSummary, error) { - summary := SyncSummary{ - BlockNumber: blockNumber, - BlockHash: blockHash, - BlockRoot: blockRoot, - AtomicRoot: atomicRoot, - } - bytes, err := Codec.Marshal(Version, &summary) - if err != nil { - return SyncSummary{}, err - } - - summary.bytes = bytes - summaryID, err := ids.ToID(crypto.Keccak256(bytes)) - if err != nil { - return SyncSummary{}, err - } - summary.summaryID = summaryID - - return summary, nil -} - -func (s SyncSummary) Bytes() []byte { - return s.bytes -} - -func (s SyncSummary) Height() uint64 { - return s.BlockNumber -} - -func (s SyncSummary) ID() ids.ID { - return s.summaryID -} - -func (s SyncSummary) String() string { - return fmt.Sprintf("SyncSummary(BlockHash=%s, BlockNumber=%d, BlockRoot=%s, AtomicRoot=%s)", s.BlockHash, s.BlockNumber, s.BlockRoot, s.AtomicRoot) -} - -func (s SyncSummary) Accept(context.Context) (block.StateSyncMode, error) { - if s.acceptImpl == nil { - return block.StateSyncSkipped, fmt.Errorf("accept implementation not specified for summary: %s", s) - } - return s.acceptImpl(s) -} +type AcceptImplFn func(Syncable) (block.StateSyncMode, error) diff --git a/plugin/evm/network_handler.go b/plugin/evm/network_handler.go index b1b32b2f50..fb5162e17e 100644 --- a/plugin/evm/network_handler.go +++ b/plugin/evm/network_handler.go @@ -10,49 +10,57 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/coreth/plugin/evm/message" syncHandlers "github.com/ava-labs/coreth/sync/handlers" - syncStats "github.com/ava-labs/coreth/sync/handlers/stats" + "github.com/ava-labs/coreth/sync/handlers/stats" "github.com/ava-labs/coreth/warp" warpHandlers "github.com/ava-labs/coreth/warp/handlers" "github.com/ava-labs/libevm/ethdb" - "github.com/ava-labs/libevm/metrics" + "github.com/ava-labs/libevm/log" "github.com/ava-labs/libevm/triedb" ) var _ message.RequestHandler = (*networkHandler)(nil) +type LeafHandlers map[message.NodeType]syncHandlers.LeafRequestHandler + type networkHandler struct { - stateTrieLeafsRequestHandler *syncHandlers.LeafsRequestHandler - atomicTrieLeafsRequestHandler *syncHandlers.LeafsRequestHandler - blockRequestHandler *syncHandlers.BlockRequestHandler - codeRequestHandler *syncHandlers.CodeRequestHandler - signatureRequestHandler *warpHandlers.SignatureRequestHandler + leafRequestHandlers LeafHandlers + blockRequestHandler *syncHandlers.BlockRequestHandler + codeRequestHandler *syncHandlers.CodeRequestHandler + signatureRequestHandler *warpHandlers.SignatureRequestHandler +} + +type LeafRequestTypeConfig struct { + NodeType message.NodeType + NodeKeyLen int + TrieDB *triedb.Database + UseSnapshots bool + MetricName string } // newNetworkHandler constructs the handler for serving network requests. func newNetworkHandler( provider syncHandlers.SyncDataProvider, diskDB ethdb.KeyValueReader, - evmTrieDB *triedb.Database, - atomicTrieDB *triedb.Database, warpBackend warp.Backend, networkCodec codec.Manager, -) message.RequestHandler { - syncStats := syncStats.NewHandlerStats(metrics.Enabled) + leafRequestHandlers LeafHandlers, + syncStats stats.HandlerStats, +) *networkHandler { return &networkHandler{ - stateTrieLeafsRequestHandler: syncHandlers.NewLeafsRequestHandler(evmTrieDB, provider, networkCodec, syncStats), - atomicTrieLeafsRequestHandler: syncHandlers.NewLeafsRequestHandler(atomicTrieDB, nil, networkCodec, syncStats), - blockRequestHandler: syncHandlers.NewBlockRequestHandler(provider, networkCodec, syncStats), - codeRequestHandler: syncHandlers.NewCodeRequestHandler(diskDB, networkCodec, syncStats), - signatureRequestHandler: warpHandlers.NewSignatureRequestHandler(warpBackend, networkCodec), + leafRequestHandlers: leafRequestHandlers, + blockRequestHandler: syncHandlers.NewBlockRequestHandler(provider, networkCodec, syncStats), + codeRequestHandler: syncHandlers.NewCodeRequestHandler(diskDB, networkCodec, syncStats), + signatureRequestHandler: warpHandlers.NewSignatureRequestHandler(warpBackend, networkCodec), } } -func (n networkHandler) HandleStateTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { - return n.stateTrieLeafsRequestHandler.OnLeafsRequest(ctx, nodeID, requestID, leafsRequest) -} - -func (n networkHandler) HandleAtomicTrieLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { - return n.atomicTrieLeafsRequestHandler.OnLeafsRequest(ctx, nodeID, requestID, leafsRequest) +func (n networkHandler) HandleLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { + handler, ok := n.leafRequestHandlers[leafsRequest.NodeType] + if !ok { + log.Debug("node type is not recognised, dropping request", "nodeID", nodeID, "requestID", requestID, "nodeType", leafsRequest.NodeType) + return nil, nil + } + return handler.OnLeafsRequest(ctx, nodeID, requestID, leafsRequest) } func (n networkHandler) HandleBlockRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, blockRequest message.BlockRequest) ([]byte, error) { diff --git a/plugin/evm/syncervm_client.go b/plugin/evm/sync/syncervm_client.go similarity index 68% rename from plugin/evm/syncervm_client.go rename to plugin/evm/sync/syncervm_client.go index 1cb0cfd638..ee8d76c4e6 100644 --- a/plugin/evm/syncervm_client.go +++ b/plugin/evm/sync/syncervm_client.go @@ -1,13 +1,14 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. - -package evm +package sync import ( "context" "fmt" "sync" + syncclient "github.com/ava-labs/coreth/sync/client" + "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/ids" @@ -17,69 +18,91 @@ import ( "github.com/ava-labs/coreth/core/state/snapshot" "github.com/ava-labs/coreth/eth" "github.com/ava-labs/coreth/params" - "github.com/ava-labs/coreth/plugin/evm/atomic/state" "github.com/ava-labs/coreth/plugin/evm/message" - syncclient "github.com/ava-labs/coreth/sync/client" "github.com/ava-labs/coreth/sync/statesync" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/rawdb" + "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" ) -const ( - // State sync fetches [parentsToGet] parents of the block it syncs to. - // The last 256 block hashes are necessary to support the BLOCKHASH opcode. - parentsToGet = 256 -) +// ParentsToFetch is the number of the block parents the state syncs to. +// The last 256 block hashes are necessary to support the BLOCKHASH opcode. +const ParentsToFetch = 256 var stateSyncSummaryKey = []byte("stateSyncSummary") -// stateSyncClientConfig defines the options and dependencies needed to construct a StateSyncerClient -type stateSyncClientConfig struct { - enabled bool - skipResume bool +type BlockAcceptor interface { + PutLastAcceptedID(ids.ID) error +} + +// EthBlockWrapper can be implemented by a concrete block wrapper type to +// return *types.Block, which is needed to update chain pointers at the +// end of the sync operation. +type EthBlockWrapper interface { + GetEthBlock() *types.Block +} + +// Extender is an interface that allows for extending the state sync process. +type Extender interface { + // Sync is called to perform any extension-specific state sync logic. + Sync(ctx context.Context, client syncclient.LeafClient, verdb *versiondb.Database, syncSummary message.Syncable) error + // OnFinishBeforeCommit is called after the state sync process has completed but before the state sync summary is committed. + OnFinishBeforeCommit(lastAcceptedHeight uint64, syncSummary message.Syncable) error + // OnFinishAfterCommit is called after the state sync process has completed and the state sync summary is committed. + OnFinishAfterCommit(summaryHeight uint64) error +} + +// ClientConfig defines the options and dependencies needed to construct a Client +type ClientConfig struct { + Enabled bool + SkipResume bool // Specifies the number of blocks behind the latest state summary that the chain must be // in order to prefer performing state sync over falling back to the normal bootstrapping // algorithm. - stateSyncMinBlocks uint64 - stateSyncRequestSize uint16 // number of key/value pairs to ask peers for per request + StateSyncMinBlocks uint64 + StateSyncRequestSize uint16 // number of key/value pairs to ask peers for per request + + LastAcceptedHeight uint64 - lastAcceptedHeight uint64 + Chain *eth.Ethereum + State *chain.State + ChaindDB ethdb.Database + Acceptor BlockAcceptor + VerDB *versiondb.Database + MetadataDB database.Database - chain *eth.Ethereum - state *chain.State - chaindb ethdb.Database - metadataDB database.Database - acceptedBlockDB database.Database - db *versiondb.Database - atomicBackend *state.AtomicBackend + // Extension points + SyncableParser message.SyncableParser + // SyncExtender is an optional extension point for the state sync process, and can be nil. + SyncExtender Extender - client syncclient.Client + Client syncclient.Client - toEngine chan<- commonEng.Message + ToEngine chan<- commonEng.Message } type stateSyncerClient struct { - *stateSyncClientConfig + *ClientConfig - resumableSummary message.SyncSummary + resumableSummary message.Syncable cancel context.CancelFunc wg sync.WaitGroup // State Sync results - syncSummary message.SyncSummary + syncSummary message.Syncable stateSyncErr error } -func NewStateSyncClient(config *stateSyncClientConfig) StateSyncClient { +func NewClient(config *ClientConfig) Client { return &stateSyncerClient{ - stateSyncClientConfig: config, + ClientConfig: config, } } -type StateSyncClient interface { +type Client interface { // methods that implement the client side of [block.StateSyncableVM] StateSyncEnabled(context.Context) (bool, error) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) @@ -102,23 +125,23 @@ type Syncer interface { // StateSyncEnabled returns [client.enabled], which is set in the chain's config file. func (client *stateSyncerClient) StateSyncEnabled(context.Context) (bool, error) { - return client.enabled, nil + return client.Enabled, nil } // GetOngoingSyncStateSummary returns a state summary that was previously started // and not finished, and sets [resumableSummary] if one was found. // Returns [database.ErrNotFound] if no ongoing summary is found or if [client.skipResume] is true. func (client *stateSyncerClient) GetOngoingSyncStateSummary(context.Context) (block.StateSummary, error) { - if client.skipResume { + if client.SkipResume { return nil, database.ErrNotFound } - summaryBytes, err := client.metadataDB.Get(stateSyncSummaryKey) + summaryBytes, err := client.MetadataDB.Get(stateSyncSummaryKey) if err != nil { return nil, err // includes the [database.ErrNotFound] case } - summary, err := message.NewSyncSummaryFromBytes(summaryBytes, client.acceptSyncSummary) + summary, err := client.SyncableParser.ParseFromBytes(summaryBytes, client.acceptSyncSummary) if err != nil { return nil, fmt.Errorf("failed to parse saved state sync summary to SyncSummary: %w", err) } @@ -128,10 +151,10 @@ func (client *stateSyncerClient) GetOngoingSyncStateSummary(context.Context) (bl // ClearOngoingSummary clears any marker of an ongoing state sync summary func (client *stateSyncerClient) ClearOngoingSummary() error { - if err := client.metadataDB.Delete(stateSyncSummaryKey); err != nil { + if err := client.MetadataDB.Delete(stateSyncSummaryKey); err != nil { return fmt.Errorf("failed to clear ongoing summary: %w", err) } - if err := client.db.Commit(); err != nil { + if err := client.VerDB.Commit(); err != nil { return fmt.Errorf("failed to commit db while clearing ongoing summary: %w", err) } @@ -140,37 +163,40 @@ func (client *stateSyncerClient) ClearOngoingSummary() error { // ParseStateSummary parses [summaryBytes] to [commonEng.Summary] func (client *stateSyncerClient) ParseStateSummary(_ context.Context, summaryBytes []byte) (block.StateSummary, error) { - return message.NewSyncSummaryFromBytes(summaryBytes, client.acceptSyncSummary) + return client.SyncableParser.ParseFromBytes(summaryBytes, client.acceptSyncSummary) } // stateSync blockingly performs the state sync for the EVM state and the atomic state // to [client.syncSummary]. returns an error if one occurred. func (client *stateSyncerClient) stateSync(ctx context.Context) error { - if err := client.syncBlocks(ctx, client.syncSummary.BlockHash, client.syncSummary.BlockNumber, parentsToGet); err != nil { + if err := client.syncBlocks(ctx, client.syncSummary.GetBlockHash(), client.syncSummary.Height(), ParentsToFetch); err != nil { return err } - // Sync the EVM trie and then the atomic trie. These steps could be done - // in parallel or in the opposite order. Keeping them serial for simplicity for now. + // Sync the EVM trie. if err := client.syncStateTrie(ctx); err != nil { return err } - return client.syncAtomicTrie(ctx) + if client.SyncExtender != nil { + return client.SyncExtender.Sync(ctx, client.Client, client.VerDB, client.syncSummary) + } + return nil } // acceptSyncSummary returns true if sync will be performed and launches the state sync process // in a goroutine. -func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncSummary) (block.StateSyncMode, error) { - isResume := proposedSummary.BlockHash == client.resumableSummary.BlockHash +func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.Syncable) (block.StateSyncMode, error) { + isResume := client.resumableSummary != nil && + proposedSummary.GetBlockHash() == client.resumableSummary.GetBlockHash() if !isResume { // Skip syncing if the blockchain is not significantly ahead of local state, // since bootstrapping would be faster. // (Also ensures we don't sync to a height prior to local state.) - if client.lastAcceptedHeight+client.stateSyncMinBlocks > proposedSummary.Height() { + if client.LastAcceptedHeight+client.StateSyncMinBlocks > proposedSummary.Height() { log.Info( "last accepted too close to most recent syncable block, skipping state sync", - "lastAccepted", client.lastAcceptedHeight, + "lastAccepted", client.LastAcceptedHeight, "syncableHeight", proposedSummary.Height(), ) return block.StateSyncSkipped, nil @@ -182,21 +208,21 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS // sync marker will be wiped, so we do not accidentally resume progress from an incorrect version // of the snapshot. (if switching between versions that come before this change and back this could // lead to the snapshot not being cleaned up correctly) - <-snapshot.WipeSnapshot(client.chaindb, true) + <-snapshot.WipeSnapshot(client.ChaindDB, true) // Reset the snapshot generator here so that when state sync completes, snapshots will not attempt to read an // invalid generator. // Note: this must be called after WipeSnapshot is called so that we do not invalidate a partially generated snapshot. - snapshot.ResetSnapshotGeneration(client.chaindb) + snapshot.ResetSnapshotGeneration(client.ChaindDB) } client.syncSummary = proposedSummary // Update the current state sync summary key in the database // Note: this must be performed after WipeSnapshot finishes so that we do not start a state sync // session from a partially wiped snapshot. - if err := client.metadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { + if err := client.MetadataDB.Put(stateSyncSummaryKey, proposedSummary.Bytes()); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to write state sync summary key to disk: %w", err) } - if err := client.db.Commit(); err != nil { + if err := client.VerDB.Commit(); err != nil { return block.StateSyncSkipped, fmt.Errorf("failed to commit db: %w", err) } @@ -219,7 +245,7 @@ func (client *stateSyncerClient) acceptSyncSummary(proposedSummary message.SyncS // this error will be propagated to the engine when it calls // vm.SetState(snow.Bootstrapping) log.Info("stateSync completed, notifying engine", "err", client.stateSyncErr) - client.toEngine <- commonEng.StateSyncDone + client.ToEngine <- commonEng.StateSyncDone }() return block.StateSyncStatic, nil } @@ -236,7 +262,7 @@ func (client *stateSyncerClient) syncBlocks(ctx context.Context, fromHash common // first, check for blocks already available on disk so we don't // request them from peers. for parentsToGet >= 0 { - blk := rawdb.ReadBlock(client.chaindb, nextHash, nextHeight) + blk := rawdb.ReadBlock(client.ChaindDB, nextHash, nextHeight) if blk != nil { // block exists nextHash = blk.ParentHash() @@ -251,12 +277,12 @@ func (client *stateSyncerClient) syncBlocks(ctx context.Context, fromHash common // get any blocks we couldn't find on disk from peers and write // them to disk. - batch := client.chaindb.NewBatch() + batch := client.ChaindDB.NewBatch() for i := parentsToGet - 1; i >= 0 && (nextHash != common.Hash{}); { if err := ctx.Err(); err != nil { return err } - blocks, err := client.client.GetBlocks(ctx, nextHash, nextHeight, parentsPerRequest) + blocks, err := client.Client.GetBlocks(ctx, nextHash, nextHeight, parentsPerRequest) if err != nil { log.Error("could not get blocks from peer", "err", err, "nextHash", nextHash, "remaining", i+1) return err @@ -275,37 +301,16 @@ func (client *stateSyncerClient) syncBlocks(ctx context.Context, fromHash common return batch.Write() } -func (client *stateSyncerClient) syncAtomicTrie(ctx context.Context) error { - log.Info("atomic tx: sync starting", "root", client.syncSummary.AtomicRoot) - atomicSyncer, err := newAtomicSyncer( - client.client, - client.db, - client.atomicBackend.AtomicTrie(), - client.syncSummary.AtomicRoot, - client.syncSummary.BlockNumber, - client.stateSyncRequestSize, - ) - if err != nil { - return err - } - if err := atomicSyncer.Start(ctx); err != nil { - return err - } - err = <-atomicSyncer.Done() - log.Info("atomic tx: sync finished", "root", client.syncSummary.AtomicRoot, "err", err) - return err -} - func (client *stateSyncerClient) syncStateTrie(ctx context.Context) error { - log.Info("state sync: sync starting", "root", client.syncSummary.BlockRoot) + log.Info("state sync: sync starting", "root", client.syncSummary.GetBlockRoot()) evmSyncer, err := statesync.NewStateSyncer(&statesync.StateSyncerConfig{ - Client: client.client, - Root: client.syncSummary.BlockRoot, + Client: client.Client, + Root: client.syncSummary.GetBlockRoot(), BatchSize: ethdb.IdealBatchSize, - DB: client.chaindb, + DB: client.ChaindDB, MaxOutstandingCodeHashes: statesync.DefaultMaxOutstandingCodeHashes, NumCodeFetchingWorkers: statesync.DefaultNumCodeFetchingWorkers, - RequestSize: client.stateSyncRequestSize, + RequestSize: client.StateSyncRequestSize, }) if err != nil { return err @@ -314,7 +319,7 @@ func (client *stateSyncerClient) syncStateTrie(ctx context.Context) error { return err } err = <-evmSyncer.Done() - log.Info("state sync: sync finished", "root", client.syncSummary.BlockRoot, "err", err) + log.Info("state sync: sync finished", "root", client.syncSummary.GetBlockRoot(), "err", err) return err } @@ -329,27 +334,29 @@ func (client *stateSyncerClient) Shutdown() error { // finishSync is responsible for updating disk and memory pointers so the VM is prepared // for bootstrapping. Executes any shared memory operations from the atomic trie to shared memory. func (client *stateSyncerClient) finishSync() error { - stateBlock, err := client.state.GetBlock(context.TODO(), ids.ID(client.syncSummary.BlockHash)) + stateBlock, err := client.State.GetBlock(context.TODO(), ids.ID(client.syncSummary.GetBlockHash())) if err != nil { - return fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.BlockHash) + return fmt.Errorf("could not get block by hash from client state: %s", client.syncSummary.GetBlockHash()) } wrapper, ok := stateBlock.(*chain.BlockWrapper) if !ok { return fmt.Errorf("could not convert block(%T) to *chain.BlockWrapper", wrapper) } - evmBlock, ok := wrapper.Block.(*Block) + wrappedBlock := wrapper.Block + + evmBlockGetter, ok := wrappedBlock.(EthBlockWrapper) if !ok { - return fmt.Errorf("could not convert block(%T) to evm.Block", stateBlock) + return fmt.Errorf("could not convert block(%T) to evm.EthBlockWrapper", stateBlock) } - block := evmBlock.ethBlock + block := evmBlockGetter.GetEthBlock() - if block.Hash() != client.syncSummary.BlockHash { - return fmt.Errorf("attempted to set last summary block to unexpected block hash: (%s != %s)", block.Hash(), client.syncSummary.BlockHash) + if block.Hash() != client.syncSummary.GetBlockHash() { + return fmt.Errorf("attempted to set last summary block to unexpected block hash: (%s != %s)", block.Hash(), client.syncSummary.GetBlockHash()) } - if block.NumberU64() != client.syncSummary.BlockNumber { - return fmt.Errorf("attempted to set last summary block to unexpected block number: (%d != %d)", block.NumberU64(), client.syncSummary.BlockNumber) + if block.NumberU64() != client.syncSummary.Height() { + return fmt.Errorf("attempted to set last summary block to unexpected block number: (%d != %d)", block.NumberU64(), client.syncSummary.Height()) } // BloomIndexer needs to know that some parts of the chain are not available @@ -362,50 +369,54 @@ func (client *stateSyncerClient) finishSync() error { // by [params.BloomBitsBlocks]. parentHeight := block.NumberU64() - 1 parentHash := block.ParentHash() - client.chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) + client.Chain.BloomIndexer().AddCheckpoint(parentHeight/params.BloomBitsBlocks, parentHash) - if err := client.chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { + if err := client.Chain.BlockChain().ResetToStateSyncedBlock(block); err != nil { return err } - if err := client.updateVMMarkers(); err != nil { + if client.SyncExtender != nil { + if err := client.SyncExtender.OnFinishBeforeCommit(client.LastAcceptedHeight, client.syncSummary); err != nil { + return err + } + } + + if err := client.commitVMMarkers(); err != nil { return fmt.Errorf("error updating vm markers, height=%d, hash=%s, err=%w", block.NumberU64(), block.Hash(), err) } - if err := client.state.SetLastAcceptedBlock(evmBlock); err != nil { + if err := client.State.SetLastAcceptedBlock(wrappedBlock); err != nil { return err } - // the chain state is already restored, and from this point on - // the block synced to is the accepted block. the last operation - // is updating shared memory with the atomic trie. - // ApplyToSharedMemory does this, and even if the VM is stopped - // (gracefully or ungracefully), since MarkApplyToSharedMemoryCursor - // is called, VM will resume ApplyToSharedMemory on Initialize. - return client.atomicBackend.ApplyToSharedMemory(block.NumberU64()) + if client.SyncExtender != nil { + return client.SyncExtender.OnFinishAfterCommit(block.NumberU64()) + } + + return nil } -// updateVMMarkers updates the following markers in the VM's database +// commitVMMarkers updates the following markers in the VM's database // and commits them atomically: // - updates atomic trie so it will have necessary metadata for the last committed root // - updates atomic trie so it will resume applying operations to shared memory on initialize // - updates lastAcceptedKey // - removes state sync progress markers -func (client *stateSyncerClient) updateVMMarkers() error { +func (client *stateSyncerClient) commitVMMarkers() error { // Mark the previously last accepted block for the shared memory cursor, so that we will execute shared // memory operations from the previously last accepted block to [vm.syncSummary] when ApplyToSharedMemory // is called. - if err := client.atomicBackend.MarkApplyToSharedMemoryCursor(client.lastAcceptedHeight); err != nil { + id, err := ids.ToID(client.syncSummary.GetBlockHash().Bytes()) + if err != nil { return err } - client.atomicBackend.SetLastAccepted(client.syncSummary.BlockHash) - if err := client.acceptedBlockDB.Put(lastAcceptedKey, client.syncSummary.BlockHash[:]); err != nil { + if err := client.Acceptor.PutLastAcceptedID(id); err != nil { return err } - if err := client.metadataDB.Delete(stateSyncSummaryKey); err != nil { + if err := client.MetadataDB.Delete(stateSyncSummaryKey); err != nil { return err } - return client.db.Commit() + return client.VerDB.Commit() } // Error returns a non-nil error if one occurred during the sync. diff --git a/plugin/evm/syncervm_server.go b/plugin/evm/sync/syncervm_server.go similarity index 60% rename from plugin/evm/syncervm_server.go rename to plugin/evm/sync/syncervm_server.go index a29efdc907..5b742dccaa 100644 --- a/plugin/evm/syncervm_server.go +++ b/plugin/evm/sync/syncervm_server.go @@ -1,7 +1,6 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. - -package evm +package sync import ( "context" @@ -11,65 +10,35 @@ import ( "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/coreth/core" - "github.com/ava-labs/coreth/plugin/evm/atomic/state" - "github.com/ava-labs/coreth/plugin/evm/message" - "github.com/ava-labs/libevm/common" + + "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/log" ) -type stateSyncServerConfig struct { - Chain *core.BlockChain - AtomicTrie *state.AtomicTrie +var errProviderNotSet = fmt.Errorf("provider not set") - // SyncableInterval is the interval at which blocks are eligible to provide syncable block summaries. - SyncableInterval uint64 +type SummaryProvider interface { + StateSummaryAtBlock(ethBlock *types.Block) (block.StateSummary, error) } type stateSyncServer struct { - chain *core.BlockChain - atomicTrie *state.AtomicTrie + chain *core.BlockChain + provider SummaryProvider syncableInterval uint64 } -type StateSyncServer interface { +type Server interface { GetLastStateSummary(context.Context) (block.StateSummary, error) GetStateSummary(context.Context, uint64) (block.StateSummary, error) } -func NewStateSyncServer(config *stateSyncServerConfig) StateSyncServer { +func SyncServer(chain *core.BlockChain, provider SummaryProvider, syncableInterval uint64) Server { return &stateSyncServer{ - chain: config.Chain, - atomicTrie: config.AtomicTrie, - syncableInterval: config.SyncableInterval, - } -} - -// stateSummaryAtHeight returns the SyncSummary at [height] if valid and available. -func (server *stateSyncServer) stateSummaryAtHeight(height uint64) (message.SyncSummary, error) { - atomicRoot, err := server.atomicTrie.Root(height) - if err != nil { - return message.SyncSummary{}, fmt.Errorf("error getting atomic trie root for height (%d): %w", height, err) - } - - if (atomicRoot == common.Hash{}) { - return message.SyncSummary{}, fmt.Errorf("atomic trie root not found for height (%d)", height) - } - - blk := server.chain.GetBlockByNumber(height) - if blk == nil { - return message.SyncSummary{}, fmt.Errorf("block not found for height (%d)", height) - } - - if !server.chain.HasState(blk.Root()) { - return message.SyncSummary{}, fmt.Errorf("block root does not exist for height (%d), root (%s)", height, blk.Root()) - } - - summary, err := message.NewSyncSummary(blk.Hash(), height, blk.Root(), atomicRoot) - if err != nil { - return message.SyncSummary{}, fmt.Errorf("failed to construct syncable block at height %d: %w", height, err) + chain: chain, + syncableInterval: syncableInterval, + provider: provider, } - return summary, nil } // GetLastStateSummary returns the latest state summary. @@ -109,3 +78,18 @@ func (server *stateSyncServer) GetStateSummary(_ context.Context, height uint64) log.Debug("Serving syncable block at requested height", "height", height, "summary", summary) return summary, nil } + +func (server *stateSyncServer) stateSummaryAtHeight(height uint64) (block.StateSummary, error) { + blk := server.chain.GetBlockByNumber(height) + if blk == nil { + return nil, fmt.Errorf("block not found for height (%d)", height) + } + + if !server.chain.HasState(blk.Root()) { + return nil, fmt.Errorf("block root does not exist for height (%d), root (%s)", height, blk.Root()) + } + if server.provider == nil { + return nil, errProviderNotSet + } + return server.provider.StateSummaryAtBlock(blk) +} diff --git a/plugin/evm/syncervm_test.go b/plugin/evm/syncervm_test.go index 1148b8398e..c891987408 100644 --- a/plugin/evm/syncervm_test.go +++ b/plugin/evm/syncervm_test.go @@ -4,681 +4,47 @@ package evm import ( - "context" - "fmt" - "math/big" - "math/rand" - "sync" "testing" - "time" - - "github.com/stretchr/testify/assert" - "github.com/stretchr/testify/require" - - "github.com/ava-labs/avalanchego/api/metrics" - avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" - avalanchedatabase "github.com/ava-labs/avalanchego/database" - "github.com/ava-labs/avalanchego/database/prefixdb" - "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow" - commonEng "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/snow/engine/enginetest" - "github.com/ava-labs/avalanchego/snow/engine/snowman/block" - "github.com/ava-labs/avalanchego/upgrade/upgradetest" - "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" - "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/coreth/consensus/dummy" - "github.com/ava-labs/coreth/constants" "github.com/ava-labs/coreth/core" - "github.com/ava-labs/coreth/core/coretest" "github.com/ava-labs/coreth/params" - "github.com/ava-labs/coreth/plugin/evm/atomic" - "github.com/ava-labs/coreth/plugin/evm/atomic/atomictest" - "github.com/ava-labs/coreth/plugin/evm/customrawdb" - "github.com/ava-labs/coreth/plugin/evm/customtypes" - "github.com/ava-labs/coreth/plugin/evm/database" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/predicate" - statesyncclient "github.com/ava-labs/coreth/sync/client" - "github.com/ava-labs/coreth/sync/statesync/statesynctest" - "github.com/ava-labs/coreth/utils/utilstest" + "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/rawdb" "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/libevm/ethdb" - "github.com/ava-labs/libevm/log" - "github.com/ava-labs/libevm/rlp" - "github.com/ava-labs/libevm/trie" - "github.com/ava-labs/libevm/triedb" -) - -func TestSkipStateSync(t *testing.T) { - rand.Seed(1) - test := syncTest{ - syncableInterval: 256, - stateSyncMinBlocks: 300, // must be greater than [syncableInterval] to skip sync - syncMode: block.StateSyncSkipped, - } - vmSetup := createSyncServerAndClientVMs(t, test, parentsToGet) - testSyncerVM(t, vmSetup, test) -} - -func TestStateSyncFromScratch(t *testing.T) { - rand.Seed(1) - test := syncTest{ - syncableInterval: 256, - stateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync - syncMode: block.StateSyncStatic, - } - vmSetup := createSyncServerAndClientVMs(t, test, parentsToGet) - - testSyncerVM(t, vmSetup, test) -} - -func TestStateSyncFromScratchExceedParent(t *testing.T) { - rand.Seed(1) - numToGen := parentsToGet + uint64(32) - test := syncTest{ - syncableInterval: numToGen, - stateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync - syncMode: block.StateSyncStatic, - } - vmSetup := createSyncServerAndClientVMs(t, test, int(numToGen)) - - testSyncerVM(t, vmSetup, test) -} - -func TestStateSyncToggleEnabledToDisabled(t *testing.T) { - rand.Seed(1) - - var lock sync.Mutex - reqCount := 0 - test := syncTest{ - syncableInterval: 256, - stateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync - syncMode: block.StateSyncStatic, - responseIntercept: func(syncerVM *VM, nodeID ids.NodeID, requestID uint32, response []byte) { - lock.Lock() - defer lock.Unlock() + "github.com/stretchr/testify/require" +) - reqCount++ - // Fail all requests after number 50 to interrupt the sync - if reqCount > 50 { - if err := syncerVM.AppRequestFailed(context.Background(), nodeID, requestID, commonEng.ErrTimeout); err != nil { - panic(err) - } - cancel := syncerVM.StateSyncClient.(*stateSyncerClient).cancel - if cancel != nil { - cancel() - } else { - t.Fatal("state sync client not populated correctly") +func TestEVMSyncerVM(t *testing.T) { + for _, test := range vmtest.SyncerVMTests { + t.Run(test.Name, func(t *testing.T) { + genFn := func(i int, vm extension.InnerVM, gen *core.BlockGen) { + b, err := predicate.NewResults().Bytes() + if err != nil { + t.Fatal(err) } - } else { - syncerVM.AppResponse(context.Background(), nodeID, requestID, response) - } - }, - expectedErr: context.Canceled, - } - vmSetup := createSyncServerAndClientVMs(t, test, parentsToGet) - - // Perform sync resulting in early termination. - testSyncerVM(t, vmSetup, test) + gen.AppendExtra(b) - test.syncMode = block.StateSyncStatic - test.responseIntercept = nil - test.expectedErr = nil - - syncDisabledVM := &VM{} - appSender := &enginetest.Sender{T: t} - appSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { return nil } - appSender.SendAppRequestF = func(ctx context.Context, nodeSet set.Set[ids.NodeID], requestID uint32, request []byte) error { - nodeID, hasItem := nodeSet.Pop() - if !hasItem { - t.Fatal("expected nodeSet to contain at least 1 nodeID") - } - go vmSetup.serverVM.AppRequest(ctx, nodeID, requestID, time.Now().Add(1*time.Second), request) - return nil - } - // Reset metrics to allow re-initialization - vmSetup.syncerVM.ctx.Metrics = metrics.NewPrefixGatherer() - stateSyncDisabledConfigJSON := `{"state-sync-enabled":false}` - genesisJSON := []byte(genesisJSON(forkToChainConfig[upgradetest.Latest])) - if err := syncDisabledVM.Initialize( - context.Background(), - vmSetup.syncerVM.ctx, - vmSetup.syncerDB, - genesisJSON, - nil, - []byte(stateSyncDisabledConfigJSON), - vmSetup.syncerVM.toEngine, - []*commonEng.Fx{}, - appSender, - ); err != nil { - t.Fatal(err) - } - - defer func() { - if err := syncDisabledVM.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - if height := syncDisabledVM.LastAcceptedBlockInternal().Height(); height != 0 { - t.Fatalf("Unexpected last accepted height: %d", height) - } - - enabled, err := syncDisabledVM.StateSyncEnabled(context.Background()) - assert.NoError(t, err) - assert.False(t, enabled, "sync should be disabled") - - // Process the first 10 blocks from the serverVM - for i := uint64(1); i < 10; i++ { - ethBlock := vmSetup.serverVM.blockChain.GetBlockByNumber(i) - if ethBlock == nil { - t.Fatalf("VM Server did not have a block available at height %d", i) - } - b, err := rlp.EncodeToBytes(ethBlock) - if err != nil { - t.Fatal(err) - } - blk, err := syncDisabledVM.ParseBlock(context.Background(), b) - if err != nil { - t.Fatal(err) - } - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - } - // Verify the snapshot disk layer matches the last block root - lastRoot := syncDisabledVM.blockChain.CurrentBlock().Root - if err := syncDisabledVM.blockChain.Snapshots().Verify(lastRoot); err != nil { - t.Fatal(err) - } - syncDisabledVM.blockChain.DrainAcceptorQueue() - - // Create a new VM from the same database with state sync enabled. - syncReEnabledVM := &VM{} - // Enable state sync in configJSON - configJSON := fmt.Sprintf( - `{"state-sync-enabled":true, "state-sync-min-blocks":%d}`, - test.stateSyncMinBlocks, - ) - // Reset metrics to allow re-initialization - vmSetup.syncerVM.ctx.Metrics = metrics.NewPrefixGatherer() - if err := syncReEnabledVM.Initialize( - context.Background(), - vmSetup.syncerVM.ctx, - vmSetup.syncerDB, - genesisJSON, - nil, - []byte(configJSON), - vmSetup.syncerVM.toEngine, - []*commonEng.Fx{}, - appSender, - ); err != nil { - t.Fatal(err) - } - - // override [serverVM]'s SendAppResponse function to trigger AppResponse on [syncerVM] - vmSetup.serverAppSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { - if test.responseIntercept == nil { - go syncReEnabledVM.AppResponse(ctx, nodeID, requestID, response) - } else { - go test.responseIntercept(syncReEnabledVM, nodeID, requestID, response) - } - - return nil - } - - // connect peer to [syncerVM] - assert.NoError(t, syncReEnabledVM.Connected( - context.Background(), - vmSetup.serverVM.ctx.NodeID, - statesyncclient.StateSyncVersion, - )) - - enabled, err = syncReEnabledVM.StateSyncEnabled(context.Background()) - assert.NoError(t, err) - assert.True(t, enabled, "sync should be enabled") - - vmSetup.syncerVM = syncReEnabledVM - testSyncerVM(t, vmSetup, test) -} - -func TestVMShutdownWhileSyncing(t *testing.T) { - var ( - lock sync.Mutex - vmSetup *syncVMSetup - ) - reqCount := 0 - test := syncTest{ - syncableInterval: 256, - stateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync - syncMode: block.StateSyncStatic, - responseIntercept: func(syncerVM *VM, nodeID ids.NodeID, requestID uint32, response []byte) { - lock.Lock() - defer lock.Unlock() - - reqCount++ - // Shutdown the VM after 50 requests to interrupt the sync - if reqCount == 50 { - // Note this verifies the VM shutdown does not time out while syncing. - require.NoError(t, vmSetup.shutdownOnceSyncerVM.Shutdown(context.Background())) - } else if reqCount < 50 { - require.NoError(t, syncerVM.AppResponse(context.Background(), nodeID, requestID, response)) + tx := types.NewTransaction(gen.TxNonce(vmtest.TestEthAddrs[0]), vmtest.TestEthAddrs[1], common.Big1, params.TxGas, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), vmtest.TestKeys[0].ToECDSA()) + require.NoError(t, err) + gen.AddTx(signedTx) } - }, - expectedErr: context.Canceled, - } - vmSetup = createSyncServerAndClientVMs(t, test, parentsToGet) - // Perform sync resulting in early termination. - testSyncerVM(t, vmSetup, test) -} - -func createSyncServerAndClientVMs(t *testing.T, test syncTest, numBlocks int) *syncVMSetup { - var ( - require = require.New(t) - importAmount = 2000000 * units.Avax // 2M avax - alloc = map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - } - ) - configJSON := fmt.Sprintf(`{"commit-interval": %d, "state-sync-commit-interval": %d}`, test.syncableInterval, test.syncableInterval) - server := newVM(t, testVMConfig{ - utxos: alloc, - configJSON: configJSON, - }) - t.Cleanup(func() { - log.Info("Shutting down server VM") - require.NoError(server.vm.Shutdown(context.Background())) - }) - var ( - importTx, exportTx *atomic.Tx - err error - ) - generateAndAcceptBlocks(t, server.vm, numBlocks, func(i int, gen *core.BlockGen) { - b, err := predicate.NewResults().Bytes() - if err != nil { - t.Fatal(err) - } - gen.AppendExtra(b) - switch i { - case 0: - // spend the UTXOs from shared memory - importTx, err = server.vm.newImportTx(server.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - require.NoError(err) - require.NoError(server.vm.mempool.AddLocalTx(importTx)) - case 1: - // export some of the imported UTXOs to test exportTx is properly synced - exportTx, err = server.vm.newExportTx( - server.vm.ctx.AVAXAssetID, - importAmount/2, - server.vm.ctx.XChainID, - testShortIDAddrs[0], - initialBaseFee, - []*secp256k1.PrivateKey{testKeys[0]}, - ) - require.NoError(err) - require.NoError(server.vm.mempool.AddLocalTx(exportTx)) - default: // Generate simple transfer transactions. - pk := testKeys[0].ToECDSA() - tx := types.NewTransaction(gen.TxNonce(testEthAddrs[0]), testEthAddrs[1], common.Big1, params.TxGas, initialBaseFee, nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(server.vm.chainID), pk) - require.NoError(err) - gen.AddTx(signedTx) - } - }, nil) - - // override serverAtomicTrie's commitInterval so the call to [serverAtomicTrie.Index] - // creates a commit at the height [syncableInterval]. This is necessary to support - // fetching a state summary. - serverAtomicTrie := server.vm.atomicBackend.AtomicTrie() - require.NoError(serverAtomicTrie.Commit(test.syncableInterval, serverAtomicTrie.LastAcceptedRoot())) - require.NoError(server.vm.versiondb.Commit()) - - serverSharedMemories := atomictest.NewSharedMemories(server.atomicMemory, server.vm.ctx.ChainID, server.vm.ctx.XChainID) - importOps, err := atomictest.ConvertToAtomicOps(importTx) - require.NoError(err) - exportOps, err := atomictest.ConvertToAtomicOps(exportTx) - require.NoError(err) - serverSharedMemories.AssertOpsApplied(t, importOps) - serverSharedMemories.AssertOpsApplied(t, exportOps) - - // make some accounts - trieDB := triedb.NewDatabase(server.vm.chaindb, nil) - root, accounts := statesynctest.FillAccountsWithOverlappingStorage(t, trieDB, types.EmptyRootHash, 1000, 16) - - // patch serverVM's lastAcceptedBlock to have the new root - // and update the vm's state so the trie with accounts will - // be returned by StateSyncGetLastSummary - lastAccepted := server.vm.blockChain.LastAcceptedBlock() - patchedBlock := patchBlock(lastAccepted, root, server.vm.chaindb) - blockBytes, err := rlp.EncodeToBytes(patchedBlock) - require.NoError(err) - internalBlock, err := server.vm.parseBlock(context.Background(), blockBytes) - require.NoError(err) - require.NoError(server.vm.State.SetLastAcceptedBlock(internalBlock)) - - // patch syncableInterval for test - server.vm.StateSyncServer.(*stateSyncServer).syncableInterval = test.syncableInterval - - // initialise [syncerVM] with blank genesis state - stateSyncEnabledJSON := fmt.Sprintf(`{"state-sync-enabled":true, "state-sync-min-blocks": %d, "tx-lookup-limit": %d, "commit-interval": %d}`, test.stateSyncMinBlocks, 4, test.syncableInterval) - syncer := newVM(t, testVMConfig{ - isSyncing: true, - configJSON: stateSyncEnabledJSON, - utxos: alloc, - }) - shutdownOnceSyncerVM := &shutdownOnceVM{VM: syncer.vm} - t.Cleanup(func() { - require.NoError(shutdownOnceSyncerVM.Shutdown(context.Background())) - }) - require.NoError(syncer.vm.SetState(context.Background(), snow.StateSyncing)) - enabled, err := syncer.vm.StateSyncEnabled(context.Background()) - require.NoError(err) - require.True(enabled) - - // override [serverVM]'s SendAppResponse function to trigger AppResponse on [syncerVM] - server.appSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { - if test.responseIntercept == nil { - go syncer.vm.AppResponse(ctx, nodeID, requestID, response) - } else { - go test.responseIntercept(syncer.vm, nodeID, requestID, response) - } - - return nil - } - - // connect peer to [syncerVM] - require.NoError( - syncer.vm.Connected( - context.Background(), - server.vm.ctx.NodeID, - statesyncclient.StateSyncVersion, - ), - ) - - // override [syncerVM]'s SendAppRequest function to trigger AppRequest on [serverVM] - syncer.appSender.SendAppRequestF = func(ctx context.Context, nodeSet set.Set[ids.NodeID], requestID uint32, request []byte) error { - nodeID, hasItem := nodeSet.Pop() - require.True(hasItem, "expected nodeSet to contain at least 1 nodeID") - require.NoError(server.vm.AppRequest(ctx, nodeID, requestID, time.Now().Add(1*time.Second), request)) - return nil - } - - return &syncVMSetup{ - serverVM: server.vm, - serverAppSender: server.appSender, - includedAtomicTxs: []*atomic.Tx{ - importTx, - exportTx, - }, - fundedAccounts: accounts, - syncerVM: syncer.vm, - syncerDB: syncer.db, - syncerEngineChan: syncer.toEngine, - syncerAtomicMemory: syncer.atomicMemory, - shutdownOnceSyncerVM: shutdownOnceSyncerVM, - } -} - -// syncVMSetup contains the required set up for a client VM to perform state sync -// off of a server VM. -type syncVMSetup struct { - serverVM *VM - serverAppSender *enginetest.Sender - - includedAtomicTxs []*atomic.Tx - fundedAccounts map[*utilstest.Key]*types.StateAccount - - syncerVM *VM - syncerDB avalanchedatabase.Database - syncerEngineChan <-chan commonEng.Message - syncerAtomicMemory *avalancheatomic.Memory - shutdownOnceSyncerVM *shutdownOnceVM -} - -type shutdownOnceVM struct { - *VM - shutdownOnce sync.Once -} - -func (vm *shutdownOnceVM) Shutdown(ctx context.Context) error { - var err error - vm.shutdownOnce.Do(func() { err = vm.VM.Shutdown(ctx) }) - return err -} - -// syncTest contains both the actual VMs as well as the parameters with the expected output. -type syncTest struct { - responseIntercept func(vm *VM, nodeID ids.NodeID, requestID uint32, response []byte) - stateSyncMinBlocks uint64 - syncableInterval uint64 - syncMode block.StateSyncMode - expectedErr error -} - -func testSyncerVM(t *testing.T, vmSetup *syncVMSetup, test syncTest) { - t.Helper() - var ( - require = require.New(t) - serverVM = vmSetup.serverVM - includedAtomicTxs = vmSetup.includedAtomicTxs - fundedAccounts = vmSetup.fundedAccounts - syncerVM = vmSetup.syncerVM - syncerEngineChan = vmSetup.syncerEngineChan - syncerAtomicMemory = vmSetup.syncerAtomicMemory - ) - // get last summary and test related methods - summary, err := serverVM.GetLastStateSummary(context.Background()) - require.NoError(err, "error getting state sync last summary") - parsedSummary, err := syncerVM.ParseStateSummary(context.Background(), summary.Bytes()) - require.NoError(err, "error parsing state summary") - retrievedSummary, err := serverVM.GetStateSummary(context.Background(), parsedSummary.Height()) - require.NoError(err, "error getting state sync summary at height") - require.Equal(summary, retrievedSummary) - - syncMode, err := parsedSummary.Accept(context.Background()) - require.NoError(err, "error accepting state summary") - require.Equal(test.syncMode, syncMode) - if syncMode == block.StateSyncSkipped { - return - } - - msg := <-syncerEngineChan - require.Equal(commonEng.StateSyncDone, msg) - - // If the test is expected to error, assert the correct error is returned and finish the test. - err = syncerVM.StateSyncClient.Error() - if test.expectedErr != nil { - require.ErrorIs(err, test.expectedErr) - // Note we re-open the database here to avoid a closed error when the test is for a shutdown VM. - chaindb := database.WrapDatabase(prefixdb.NewNested(ethDBPrefix, syncerVM.db)) - assertSyncPerformedHeights(t, chaindb, map[uint64]struct{}{}) - return - } - require.NoError(err, "state sync failed") - - // set [syncerVM] to bootstrapping and verify the last accepted block has been updated correctly - // and that we can bootstrap and process some blocks. - require.NoError(syncerVM.SetState(context.Background(), snow.Bootstrapping)) - require.Equal(serverVM.LastAcceptedBlock().Height(), syncerVM.LastAcceptedBlock().Height(), "block height mismatch between syncer and server") - require.Equal(serverVM.LastAcceptedBlock().ID(), syncerVM.LastAcceptedBlock().ID(), "blockID mismatch between syncer and server") - require.True(syncerVM.blockChain.HasState(syncerVM.blockChain.LastAcceptedBlock().Root()), "unavailable state for last accepted block") - assertSyncPerformedHeights(t, syncerVM.chaindb, map[uint64]struct{}{retrievedSummary.Height(): {}}) - - lastNumber := syncerVM.blockChain.LastAcceptedBlock().NumberU64() - // check the last block is indexed - lastSyncedBlock := rawdb.ReadBlock(syncerVM.chaindb, rawdb.ReadCanonicalHash(syncerVM.chaindb, lastNumber), lastNumber) - for _, tx := range lastSyncedBlock.Transactions() { - index := rawdb.ReadTxLookupEntry(syncerVM.chaindb, tx.Hash()) - require.NotNilf(index, "Miss transaction indices, number %d hash %s", lastNumber, tx.Hash().Hex()) - } - - // tail should be the last block synced - if syncerVM.ethConfig.TransactionHistory != 0 { - tail := lastSyncedBlock.NumberU64() - - coretest.CheckTxIndices(t, &tail, tail, tail, tail, syncerVM.chaindb, true) - } - - blocksToBuild := 10 - txsPerBlock := 10 - toAddress := testEthAddrs[1] // arbitrary choice - generateAndAcceptBlocks(t, syncerVM, blocksToBuild, func(_ int, gen *core.BlockGen) { - b, err := predicate.NewResults().Bytes() - if err != nil { - t.Fatal(err) - } - gen.AppendExtra(b) - i := 0 - for k := range fundedAccounts { - tx := types.NewTransaction(gen.TxNonce(k.Address), toAddress, big.NewInt(1), 21000, initialBaseFee, nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(serverVM.chainConfig.ChainID), k.PrivateKey) - require.NoError(err) - gen.AddTx(signedTx) - i++ - if i >= txsPerBlock { - break + newVMFn := func() (extension.InnerVM, dummy.ConsensusCallbacks) { + vm := newDefaultTestVM() + return vm, vm.extensionConfig.ConsensusCallbacks } - } - }, - func(block *types.Block) { - if syncerVM.ethConfig.TransactionHistory != 0 { - tail := block.NumberU64() - syncerVM.ethConfig.TransactionHistory + 1 - // tail should be the minimum last synced block, since we skipped it to the last block - if tail < lastSyncedBlock.NumberU64() { - tail = lastSyncedBlock.NumberU64() - } - coretest.CheckTxIndices(t, &tail, tail, block.NumberU64(), block.NumberU64(), syncerVM.chaindb, true) - } - }, - ) - - // check we can transition to [NormalOp] state and continue to process blocks. - require.NoError(syncerVM.SetState(context.Background(), snow.NormalOp)) - require.True(syncerVM.bootstrapped.Get()) - - // check atomic memory was synced properly - syncerSharedMemories := atomictest.NewSharedMemories(syncerAtomicMemory, syncerVM.ctx.ChainID, syncerVM.ctx.XChainID) - - for _, tx := range includedAtomicTxs { - atomicOps, err := atomictest.ConvertToAtomicOps(tx) - require.NoError(err) - syncerSharedMemories.AssertOpsApplied(t, atomicOps) - } - // Generate blocks after we have entered normal consensus as well - generateAndAcceptBlocks(t, syncerVM, blocksToBuild, func(_ int, gen *core.BlockGen) { - b, err := predicate.NewResults().Bytes() - if err != nil { - t.Fatal(err) - } - gen.AppendExtra(b) - i := 0 - for k := range fundedAccounts { - tx := types.NewTransaction(gen.TxNonce(k.Address), toAddress, big.NewInt(1), 21000, initialBaseFee, nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(serverVM.chainConfig.ChainID), k.PrivateKey) - require.NoError(err) - gen.AddTx(signedTx) - i++ - if i >= txsPerBlock { - break + testSetup := &vmtest.SyncTestSetup{ + NewVM: newVMFn, + GenFn: genFn, + ExtraSyncerVMTest: nil, } - } - }, - func(block *types.Block) { - if syncerVM.ethConfig.TransactionHistory != 0 { - tail := block.NumberU64() - syncerVM.ethConfig.TransactionHistory + 1 - // tail should be the minimum last synced block, since we skipped it to the last block - if tail < lastSyncedBlock.NumberU64() { - tail = lastSyncedBlock.NumberU64() - } - coretest.CheckTxIndices(t, &tail, tail, block.NumberU64(), block.NumberU64(), syncerVM.chaindb, true) - } - }, - ) -} - -// patchBlock returns a copy of [blk] with [root] and updates [db] to -// include the new block as canonical for [blk]'s height. -// This breaks the digestibility of the chain since after this call -// [blk] does not necessarily define a state transition from its parent -// state to the new state root. -func patchBlock(blk *types.Block, root common.Hash, db ethdb.Database) *types.Block { - header := blk.Header() - header.Root = root - receipts := rawdb.ReadRawReceipts(db, blk.Hash(), blk.NumberU64()) - newBlk := customtypes.NewBlockWithExtData( - header, blk.Transactions(), blk.Uncles(), receipts, trie.NewStackTrie(nil), customtypes.BlockExtData(blk), true, - ) - rawdb.WriteBlock(db, newBlk) - rawdb.WriteCanonicalHash(db, newBlk.Hash(), newBlk.NumberU64()) - return newBlk -} - -// generateAndAcceptBlocks uses [core.GenerateChain] to generate blocks, then -// calls Verify and Accept on each generated block -// TODO: consider using this helper function in vm_test.go and elsewhere in this package to clean up tests -func generateAndAcceptBlocks(t *testing.T, vm *VM, numBlocks int, gen func(int, *core.BlockGen), accepted func(*types.Block)) { - t.Helper() - - // acceptExternalBlock defines a function to parse, verify, and accept a block once it has been - // generated by GenerateChain - acceptExternalBlock := func(block *types.Block) { - bytes, err := rlp.EncodeToBytes(block) - if err != nil { - t.Fatal(err) - } - vmBlock, err := vm.ParseBlock(context.Background(), bytes) - if err != nil { - t.Fatal(err) - } - if err := vmBlock.Verify(context.Background()); err != nil { - t.Fatal(err) - } - if err := vmBlock.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - if accepted != nil { - accepted(block) - } - } - _, _, err := core.GenerateChain( - vm.chainConfig, - vm.blockChain.LastAcceptedBlock(), - dummy.NewFakerWithCallbacks(vm.createConsensusCallbacks()), - vm.chaindb, - numBlocks, - 10, - func(i int, g *core.BlockGen) { - g.SetOnBlockGenerated(acceptExternalBlock) - g.SetCoinbase(constants.BlackholeAddr) // necessary for syntactic validation of the block - gen(i, g) - }, - ) - if err != nil { - t.Fatal(err) - } - vm.blockChain.DrainAcceptorQueue() -} - -// assertSyncPerformedHeights iterates over all heights the VM has synced to and -// verifies it matches [expected]. -func assertSyncPerformedHeights(t *testing.T, db ethdb.Iteratee, expected map[uint64]struct{}) { - it := customrawdb.NewSyncPerformedIterator(db) - defer it.Release() - - found := make(map[uint64]struct{}, len(expected)) - for it.Next() { - found[customrawdb.UnpackSyncPerformedKey(it.Key())] = struct{}{} + test.TestFunc(t, testSetup) + }) } - require.NoError(t, it.Error()) - require.Equal(t, expected, found) } diff --git a/plugin/evm/tx_gossip_test.go b/plugin/evm/tx_gossip_test.go index 6427668eb3..ba777d2d8d 100644 --- a/plugin/evm/tx_gossip_test.go +++ b/plugin/evm/tx_gossip_test.go @@ -11,7 +11,6 @@ import ( "testing" "time" - avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" "github.com/ava-labs/avalanchego/database/memdb" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" @@ -29,14 +28,11 @@ import ( "github.com/prometheus/client_golang/prometheus" "github.com/stretchr/testify/require" - "github.com/ava-labs/avalanchego/vms/components/avax" - "github.com/ava-labs/avalanchego/vms/secp256k1fx" - "google.golang.org/protobuf/proto" - "github.com/ava-labs/coreth/plugin/evm/atomic" "github.com/ava-labs/coreth/plugin/evm/config" "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/utils" "github.com/ava-labs/libevm/core/types" ) @@ -51,17 +47,14 @@ func TestEthTxGossip(t *testing.T) { pk, err := secp256k1.NewPrivateKey() require.NoError(err) address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) genesisBytes, err := genesis.MarshalJSON() require.NoError(err) responseSender := &enginetest.SenderStub{ SentAppResponse: make(chan []byte, 1), } - vm := &VM{ - atomicTxGossipHandler: &p2p.NoOpHandler{}, - atomicTxPullGossiper: &gossip.NoOpGossiper{}, - } + vm := newDefaultTestVM() require.NoError(vm.Initialize( ctx, @@ -171,145 +164,6 @@ func TestEthTxGossip(t *testing.T) { wg.Wait() } -func TestAtomicTxGossip(t *testing.T) { - require := require.New(t) - ctx := context.Background() - snowCtx := snowtest.Context(t, snowtest.CChainID) - snowCtx.AVAXAssetID = ids.GenerateTestID() - validatorState := utils.NewTestValidatorState() - snowCtx.ValidatorState = validatorState - memory := avalancheatomic.NewMemory(memdb.New()) - snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) - - pk, err := secp256k1.NewPrivateKey() - require.NoError(err) - address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) - genesisBytes, err := genesis.MarshalJSON() - require.NoError(err) - - responseSender := &enginetest.SenderStub{ - SentAppResponse: make(chan []byte, 1), - } - vm := &VM{ - ethTxGossipHandler: &p2p.NoOpHandler{}, - ethTxPullGossiper: &gossip.NoOpGossiper{}, - } - - require.NoError(vm.Initialize( - ctx, - snowCtx, - memdb.New(), - genesisBytes, - nil, - nil, - make(chan common.Message), - nil, - responseSender, - )) - require.NoError(vm.SetState(ctx, snow.NormalOp)) - - defer func() { - require.NoError(vm.Shutdown(ctx)) - }() - - // sender for the peer requesting gossip from [vm] - peerSender := &enginetest.SenderStub{ - SentAppRequest: make(chan []byte, 1), - } - network, err := p2p.NewNetwork(logging.NoLog{}, peerSender, prometheus.NewRegistry(), "") - require.NoError(err) - client := network.NewClient(p2p.AtomicTxGossipHandlerID) - - // we only accept gossip requests from validators - requestingNodeID := ids.GenerateTestNodeID() - require.NoError(vm.Network.Connected(ctx, requestingNodeID, nil)) - validatorState.GetCurrentHeightF = func(context.Context) (uint64, error) { - return 0, nil - } - validatorState.GetValidatorSetF = func(context.Context, uint64, ids.ID) (map[ids.NodeID]*validators.GetValidatorOutput, error) { - return map[ids.NodeID]*validators.GetValidatorOutput{ - requestingNodeID: { - NodeID: requestingNodeID, - Weight: 1, - }, - }, nil - } - - // Ask the VM for any new transactions. We should get nothing at first. - emptyBloomFilter, err := gossip.NewBloomFilter( - prometheus.NewRegistry(), - "", - config.TxGossipBloomMinTargetElements, - config.TxGossipBloomTargetFalsePositiveRate, - config.TxGossipBloomResetFalsePositiveRate, - ) - require.NoError(err) - emptyBloomFilterBytes, _ := emptyBloomFilter.Marshal() - request := &sdk.PullGossipRequest{ - Filter: emptyBloomFilterBytes, - Salt: agoUtils.RandomBytes(32), - } - - requestBytes, err := proto.Marshal(request) - require.NoError(err) - - wg := &sync.WaitGroup{} - wg.Add(1) - onResponse := func(_ context.Context, nodeID ids.NodeID, responseBytes []byte, err error) { - require.NoError(err) - - response := &sdk.PullGossipResponse{} - require.NoError(proto.Unmarshal(responseBytes, response)) - require.Empty(response.Gossip) - wg.Done() - } - require.NoError(client.AppRequest(ctx, set.Of(vm.ctx.NodeID), requestBytes, onResponse)) - require.NoError(vm.AppRequest(ctx, requestingNodeID, 1, time.Time{}, <-peerSender.SentAppRequest)) - require.NoError(network.AppResponse(ctx, snowCtx.NodeID, 1, <-responseSender.SentAppResponse)) - wg.Wait() - - // Issue a tx to the VM - utxo, err := addUTXO( - memory, - snowCtx, - ids.GenerateTestID(), - 0, - snowCtx.AVAXAssetID, - 100_000_000_000, - pk.Address(), - ) - require.NoError(err) - tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, initialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) - require.NoError(err) - require.NoError(vm.mempool.AddLocalTx(tx)) - - // wait so we aren't throttled by the vm - time.Sleep(5 * time.Second) - - // Ask the VM for new transactions. We should get the newly issued tx. - wg.Add(1) - - marshaller := atomic.TxMarshaller{} - onResponse = func(_ context.Context, nodeID ids.NodeID, responseBytes []byte, err error) { - require.NoError(err) - - response := &sdk.PullGossipResponse{} - require.NoError(proto.Unmarshal(responseBytes, response)) - require.Len(response.Gossip, 1) - - gotTx, err := marshaller.UnmarshalGossip(response.Gossip[0]) - require.NoError(err) - require.Equal(tx.ID(), gotTx.GossipID()) - - wg.Done() - } - require.NoError(client.AppRequest(ctx, set.Of(vm.ctx.NodeID), requestBytes, onResponse)) - require.NoError(vm.AppRequest(ctx, requestingNodeID, 3, time.Time{}, <-peerSender.SentAppRequest)) - require.NoError(network.AppResponse(ctx, snowCtx.NodeID, 3, <-responseSender.SentAppResponse)) - wg.Wait() -} - // Tests that a tx is gossiped when it is issued func TestEthTxPushGossipOutbound(t *testing.T) { require := require.New(t) @@ -319,15 +173,12 @@ func TestEthTxPushGossipOutbound(t *testing.T) { SentAppGossip: make(chan []byte, 1), } - vm := &VM{ - ethTxPullGossiper: gossip.NoOpGossiper{}, - atomicTxPullGossiper: gossip.NoOpGossiper{}, - } + vm := newDefaultTestVM() pk, err := secp256k1.NewPrivateKey() require.NoError(err) address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) genesisBytes, err := genesis.MarshalJSON() require.NoError(err) @@ -378,15 +229,13 @@ func TestEthTxPushGossipInbound(t *testing.T) { snowCtx := snowtest.Context(t, snowtest.CChainID) sender := &enginetest.Sender{} - vm := &VM{ - ethTxPullGossiper: gossip.NoOpGossiper{}, - atomicTxPullGossiper: gossip.NoOpGossiper{}, - } + vm := newDefaultTestVM() + vm.ethTxPullGossiper = gossip.NoOpGossiper{} pk, err := secp256k1.NewPrivateKey() require.NoError(err) address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) + genesis := vmtest.NewPrefundedGenesis(100_000_000_000_000_000, address) genesisBytes, err := genesis.MarshalJSON() require.NoError(err) @@ -430,147 +279,3 @@ func TestEthTxPushGossipInbound(t *testing.T) { require.True(vm.txPool.Has(signedTx.Hash())) } - -// Tests that a tx is gossiped when it is issued -func TestAtomicTxPushGossipOutbound(t *testing.T) { - require := require.New(t) - ctx := context.Background() - snowCtx := snowtest.Context(t, snowtest.CChainID) - snowCtx.AVAXAssetID = ids.GenerateTestID() - validatorState := utils.NewTestValidatorState() - snowCtx.ValidatorState = validatorState - memory := avalancheatomic.NewMemory(memdb.New()) - snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) - - pk, err := secp256k1.NewPrivateKey() - require.NoError(err) - address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) - genesisBytes, err := genesis.MarshalJSON() - require.NoError(err) - - sender := &enginetest.SenderStub{ - SentAppGossip: make(chan []byte, 1), - } - vm := &VM{ - ethTxPullGossiper: gossip.NoOpGossiper{}, - atomicTxPullGossiper: gossip.NoOpGossiper{}, - } - - require.NoError(vm.Initialize( - ctx, - snowCtx, - memdb.New(), - genesisBytes, - nil, - nil, - make(chan common.Message), - nil, - sender, - )) - require.NoError(vm.SetState(ctx, snow.NormalOp)) - - defer func() { - require.NoError(vm.Shutdown(ctx)) - }() - - // Issue a tx to the VM - utxo, err := addUTXO( - memory, - snowCtx, - ids.GenerateTestID(), - 0, - snowCtx.AVAXAssetID, - 100_000_000_000, - pk.Address(), - ) - require.NoError(err) - tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, initialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) - require.NoError(err) - require.NoError(vm.mempool.AddLocalTx(tx)) - vm.atomicTxPushGossiper.Add(tx) - - gossipedBytes := <-sender.SentAppGossip - require.Equal(byte(p2p.AtomicTxGossipHandlerID), gossipedBytes[0]) - - outboundGossipMsg := &sdk.PushGossip{} - require.NoError(proto.Unmarshal(gossipedBytes[1:], outboundGossipMsg)) - require.Len(outboundGossipMsg.Gossip, 1) - - marshaller := atomic.TxMarshaller{} - gossipedTx, err := marshaller.UnmarshalGossip(outboundGossipMsg.Gossip[0]) - require.NoError(err) - require.Equal(tx.ID(), gossipedTx.GossipID()) -} - -// Tests that a tx is gossiped when it is issued -func TestAtomicTxPushGossipInbound(t *testing.T) { - require := require.New(t) - ctx := context.Background() - snowCtx := snowtest.Context(t, snowtest.CChainID) - snowCtx.AVAXAssetID = ids.GenerateTestID() - validatorState := utils.NewTestValidatorState() - snowCtx.ValidatorState = validatorState - memory := avalancheatomic.NewMemory(memdb.New()) - snowCtx.SharedMemory = memory.NewSharedMemory(snowCtx.ChainID) - - pk, err := secp256k1.NewPrivateKey() - require.NoError(err) - address := pk.EthAddress() - genesis := newPrefundedGenesis(100_000_000_000_000_000, address) - genesisBytes, err := genesis.MarshalJSON() - require.NoError(err) - - sender := &enginetest.Sender{} - vm := &VM{ - ethTxPullGossiper: gossip.NoOpGossiper{}, - atomicTxPullGossiper: gossip.NoOpGossiper{}, - } - - require.NoError(vm.Initialize( - ctx, - snowCtx, - memdb.New(), - genesisBytes, - nil, - nil, - make(chan common.Message), - nil, - sender, - )) - require.NoError(vm.SetState(ctx, snow.NormalOp)) - - defer func() { - require.NoError(vm.Shutdown(ctx)) - }() - - // issue a tx to the vm - utxo, err := addUTXO( - memory, - snowCtx, - ids.GenerateTestID(), - 0, - snowCtx.AVAXAssetID, - 100_000_000_000, - pk.Address(), - ) - require.NoError(err) - tx, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, address, initialBaseFee, secp256k1fx.NewKeychain(pk), []*avax.UTXO{utxo}) - require.NoError(err) - require.NoError(vm.mempool.AddLocalTx(tx)) - - marshaller := atomic.TxMarshaller{} - gossipBytes, err := marshaller.MarshalGossip(tx) - require.NoError(err) - - inboundGossip := &sdk.PushGossip{ - Gossip: [][]byte{gossipBytes}, - } - inboundGossipBytes, err := proto.Marshal(inboundGossip) - require.NoError(err) - - inboundGossipMsg := append(binary.AppendUvarint(nil, p2p.AtomicTxGossipHandlerID), inboundGossipBytes...) - - require.NoError(vm.AppGossip(ctx, ids.EmptyNodeID, inboundGossipMsg)) - require.True(vm.mempool.Has(tx.ID())) -} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index d1bdd21996..daf8faadcd 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -17,96 +17,79 @@ import ( "sync" "time" - "github.com/ava-labs/avalanchego/cache/metercacher" - "github.com/ava-labs/avalanchego/network/p2p" - "github.com/ava-labs/avalanchego/network/p2p/acp118" - "github.com/ava-labs/avalanchego/network/p2p/gossip" - avalanchegoConstants "github.com/ava-labs/avalanchego/utils/constants" - "github.com/ava-labs/coreth/network" - "github.com/ava-labs/coreth/plugin/evm/customtypes" "github.com/prometheus/client_golang/prometheus" "github.com/ava-labs/coreth/consensus/dummy" "github.com/ava-labs/coreth/constants" "github.com/ava-labs/coreth/core" - "github.com/ava-labs/coreth/core/state" "github.com/ava-labs/coreth/core/txpool" "github.com/ava-labs/coreth/eth" "github.com/ava-labs/coreth/eth/ethconfig" corethprometheus "github.com/ava-labs/coreth/metrics/prometheus" "github.com/ava-labs/coreth/miner" + "github.com/ava-labs/coreth/network" "github.com/ava-labs/coreth/node" "github.com/ava-labs/coreth/params" "github.com/ava-labs/coreth/params/extras" - "github.com/ava-labs/coreth/plugin/evm/atomic" - atomicstate "github.com/ava-labs/coreth/plugin/evm/atomic/state" - atomictxpool "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" "github.com/ava-labs/coreth/plugin/evm/config" - customheader "github.com/ava-labs/coreth/plugin/evm/header" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/gossip" corethlog "github.com/ava-labs/coreth/plugin/evm/log" "github.com/ava-labs/coreth/plugin/evm/message" + vmsync "github.com/ava-labs/coreth/plugin/evm/sync" "github.com/ava-labs/coreth/plugin/evm/upgrade/acp176" - "github.com/ava-labs/coreth/plugin/evm/upgrade/ap5" - "github.com/ava-labs/coreth/triedb/hashdb" - "github.com/ava-labs/coreth/utils" - - "github.com/ava-labs/libevm/core/rawdb" - "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/libevm/metrics" - ethparams "github.com/ava-labs/libevm/params" - "github.com/ava-labs/libevm/triedb" - + "github.com/ava-labs/coreth/plugin/evm/vmerrors" warpcontract "github.com/ava-labs/coreth/precompile/contracts/warp" + "github.com/ava-labs/coreth/precompile/precompileconfig" "github.com/ava-labs/coreth/rpc" statesyncclient "github.com/ava-labs/coreth/sync/client" "github.com/ava-labs/coreth/sync/client/stats" + "github.com/ava-labs/coreth/sync/handlers" + handlerstats "github.com/ava-labs/coreth/sync/handlers/stats" + "github.com/ava-labs/coreth/triedb/hashdb" + "github.com/ava-labs/coreth/utils" "github.com/ava-labs/coreth/warp" - // Force-load tracer engine to trigger registration - // - // We must import this package (not referenced elsewhere) so that the native "callTracer" - // is added to a map of client-accessible tracers. In geth, this is done - // inside of cmd/geth. - _ "github.com/ava-labs/libevm/eth/tracers/js" - _ "github.com/ava-labs/libevm/eth/tracers/native" - - "github.com/ava-labs/coreth/precompile/precompileconfig" - // Force-load precompiles to trigger registration - _ "github.com/ava-labs/coreth/precompile/registry" - "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/rawdb" + "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/log" + "github.com/ava-labs/libevm/metrics" + ethparams "github.com/ava-labs/libevm/params" "github.com/ava-labs/libevm/rlp" - - avalancheRPC "github.com/gorilla/rpc/v2" + "github.com/ava-labs/libevm/triedb" "github.com/ava-labs/avalanchego/cache/lru" - "github.com/ava-labs/avalanchego/codec" - "github.com/ava-labs/avalanchego/codec/linearcodec" - "github.com/ava-labs/avalanchego/database" + "github.com/ava-labs/avalanchego/cache/metercacher" + avalanchedatabase "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/versiondb" "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/network/p2p/acp118" + avalanchegossip "github.com/ava-labs/avalanchego/network/p2p/gossip" "github.com/ava-labs/avalanchego/snow" "github.com/ava-labs/avalanchego/snow/consensus/snowman" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" - "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" - "github.com/ava-labs/avalanchego/utils/formatting/address" - "github.com/ava-labs/avalanchego/utils/logging" + avalancheUtils "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/perms" "github.com/ava-labs/avalanchego/utils/profiler" - "github.com/ava-labs/avalanchego/utils/set" "github.com/ava-labs/avalanchego/utils/timer/mockable" "github.com/ava-labs/avalanchego/utils/units" - "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/components/chain" "github.com/ava-labs/avalanchego/vms/components/gas" - "github.com/ava-labs/avalanchego/vms/secp256k1fx" - commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + // Force-load tracer engine to trigger registration + // + // We must import this package (not referenced elsewhere) so that the native "callTracer" + // is added to a map of client-accessible tracers. In geth, this is done + // inside of cmd/geth. + _ "github.com/ava-labs/libevm/eth/tracers/js" + _ "github.com/ava-labs/libevm/eth/tracers/native" - avalancheUtils "github.com/ava-labs/avalanchego/utils" - avalancheJSON "github.com/ava-labs/avalanchego/utils/json" + // Force-load precompiles to trigger registration + _ "github.com/ava-labs/coreth/precompile/registry" ) var ( @@ -114,17 +97,14 @@ var ( _ block.BuildBlockWithContextChainVM = (*VM)(nil) _ block.StateSyncableVM = (*VM)(nil) _ statesyncclient.EthBlockParser = (*VM)(nil) - _ secp256k1fx.VM = (*VM)(nil) + _ vmsync.BlockAcceptor = (*VM)(nil) ) const ( // Max time from current time allowed for blocks, before they're considered future blocks // and fail verification maxFutureBlockTime = 10 * time.Second - maxUTXOsToFetch = 1024 - defaultMempoolSize = 4096 - secpCacheSize = 1024 decidedCacheSize = 10 * units.MiB missingCacheSize = 50 unverifiedCacheSize = 5 * units.MiB @@ -135,31 +115,14 @@ const ( ethMetricsPrefix = "eth" sdkMetricsPrefix = "sdk" chainStateMetricsPrefix = "chain_state" - - targetAtomicTxsSize = 40 * units.KiB - - // maxAtomicTxMempoolGas is the maximum amount of gas that is allowed to be - // used by an atomic transaction in the mempool. It is allowed to build - // blocks with larger atomic transactions, but they will not be accepted - // into the mempool. - maxAtomicTxMempoolGas = ap5.AtomicGasLimit - - // gossip constants - pushGossipDiscardedElements = 16_384 - txGossipTargetMessageSize = 20 * units.KiB - txGossipThrottlingPeriod = 10 * time.Second - txGossipThrottlingLimit = 2 - txGossipPollSize = 1 ) // Define the API endpoints for the VM const ( - avaxEndpoint = "/avax" - adminEndpoint = "/admin" - ethRPCEndpoint = "/rpc" - ethWSEndpoint = "/ws" - ethTxGossipNamespace = "eth_tx_gossip" - atomicTxGossipNamespace = "atomic_tx_gossip" + adminEndpoint = "/admin" + ethRPCEndpoint = "/rpc" + ethWSEndpoint = "/ws" + ethTxGossipNamespace = "eth_tx_gossip" ) var ( @@ -172,11 +135,9 @@ var ( ) var ( - errEmptyBlock = errors.New("empty block") errInvalidBlock = errors.New("invalid block") errInvalidNonce = errors.New("invalid nonce") errUnclesUnsupported = errors.New("uncles unsupported") - errRejectedParent = errors.New("rejected parent") errNilBaseFeeApricotPhase3 = errors.New("nil base fee is invalid after apricotPhase3") errNilBlockGasCostApricotPhase4 = errors.New("nil blockGasCost is invalid after apricotPhase4") errInvalidHeaderPredicateResults = errors.New("invalid header predicate results") @@ -227,6 +188,9 @@ type VM struct { chainConfig *params.ChainConfig ethConfig ethconfig.Config + // Extension Points + extensionConfig *extension.Config + // pointers to eth constructs eth *eth.Ethereum txPool *txpool.TxPool @@ -237,50 +201,34 @@ type VM struct { versiondb *versiondb.Database // [db] is the VM's current database - db database.Database - - // metadataDB is used to store one off keys. - metadataDB database.Database + db avalanchedatabase.Database // [chaindb] is the database supplied to the Ethereum backend chaindb ethdb.Database // [acceptedBlockDB] is the database to store the last accepted // block. - acceptedBlockDB database.Database + acceptedBlockDB avalanchedatabase.Database // [warpDB] is used to store warp message signatures // set to a prefixDB with the prefix [warpPrefix] - warpDB database.Database - - toEngine chan<- commonEng.Message + warpDB avalanchedatabase.Database - syntacticBlockValidator BlockValidator + metadataDB avalanchedatabase.Database - // [atomicTxRepository] maintains two indexes on accepted atomic txs. - // - txID to accepted atomic tx - // - block height to list of atomic txs accepted on block at that height - atomicTxRepository *atomicstate.AtomicRepository - // [atomicBackend] abstracts verification and processing of atomic transactions - atomicBackend *atomicstate.AtomicBackend + toEngine chan<- commonEng.Message builder *blockBuilder - baseCodec codec.Registry - clock mockable.Clock - mempool *atomictxpool.Mempool + clock *mockable.Clock shutdownChan chan struct{} shutdownWg sync.WaitGroup - fx secp256k1fx.Fx - secpCache *secp256k1.RecoverCache - // Continuous Profiler profiler profiler.ContinuousProfiler network.Network - networkCodec codec.Manager // Metrics sdkMetrics *prometheus.Registry @@ -290,40 +238,28 @@ type VM struct { logger corethlog.Logger // State sync server and client - StateSyncServer - StateSyncClient + vmsync.Server + vmsync.Client // Avalanche Warp Messaging backend // Used to serve BLS signatures of warp messages over RPC warpBackend warp.Backend // Initialize only sets these if nil so they can be overridden in tests - ethTxGossipHandler p2p.Handler - ethTxPushGossiper avalancheUtils.Atomic[*gossip.PushGossiper[*GossipEthTx]] - ethTxPullGossiper gossip.Gossiper - atomicTxGossipHandler p2p.Handler - atomicTxPushGossiper *gossip.PushGossiper[*atomic.Tx] - atomicTxPullGossiper gossip.Gossiper + ethTxGossipHandler p2p.Handler + ethTxPushGossiper avalancheUtils.Atomic[*avalanchegossip.PushGossiper[*GossipEthTx]] + ethTxPullGossiper avalanchegossip.Gossiper chainAlias string // RPC handlers (should be stopped before closing chaindb) rpcHandlers []interface{ Stop() } } -// CodecRegistry implements the secp256k1fx interface -func (vm *VM) CodecRegistry() codec.Registry { return vm.baseCodec } - -// Clock implements the secp256k1fx interface -func (vm *VM) Clock() *mockable.Clock { return &vm.clock } - -// Logger implements the secp256k1fx interface -func (vm *VM) Logger() logging.Logger { return vm.ctx.Log } - // Initialize implements the snowman.ChainVM interface func (vm *VM) Initialize( _ context.Context, chainCtx *snow.Context, - db database.Database, + db avalanchedatabase.Database, genesisBytes []byte, _ []byte, configBytes []byte, @@ -331,6 +267,15 @@ func (vm *VM) Initialize( _ []*commonEng.Fx, appSender commonEng.AppSender, ) error { + if err := vm.extensionConfig.Validate(); err != nil { + return fmt.Errorf("failed to validate extension config: %w", err) + } + + vm.clock = &mockable.Clock{} + if vm.extensionConfig.Clock != nil { + vm.clock = vm.extensionConfig.Clock + } + vm.config.SetDefaults(defaultTxPoolConfig) if len(configBytes) > 0 { if err := json.Unmarshal(configBytes, &vm.config); err != nil { @@ -397,28 +342,13 @@ func (vm *VM) Initialize( return err } - var extDataHashes map[common.Hash]common.Hash - // Set the chain config for mainnet/fuji chain IDs - switch chainCtx.NetworkID { - case avalanchegoConstants.MainnetID: - extDataHashes = mainnetExtDataHashes - case avalanchegoConstants.FujiID: - extDataHashes = fujiExtDataHashes - } - vm.syntacticBlockValidator = NewBlockValidator(extDataHashes) - - // Free the memory of the extDataHash map that is not used (i.e. if mainnet - // config, free fuji) - fujiExtDataHashes = nil - mainnetExtDataHashes = nil - vm.chainID = g.Config.ChainID vm.ethConfig = ethconfig.NewDefaultConfig() vm.ethConfig.Genesis = g vm.ethConfig.NetworkId = vm.chainID.Uint64() - vm.genesisHash = vm.ethConfig.Genesis.ToBlock().Hash() // must create genesis hash before [vm.readLastAccepted] - lastAcceptedHash, lastAcceptedHeight, err := vm.readLastAccepted() + vm.genesisHash = vm.ethConfig.Genesis.ToBlock().Hash() // must create genesis hash before [vm.ReadLastAccepted] + lastAcceptedHash, lastAcceptedHeight, err := vm.ReadLastAccepted() if err != nil { return err } @@ -481,16 +411,8 @@ func (vm *VM) Initialize( } vm.chainConfig = g.Config - vm.secpCache = secp256k1.NewRecoverCache(secpCacheSize) - // TODO: read size from settings - vm.mempool, err = atomictxpool.NewMempool(chainCtx, vm.sdkMetrics, defaultMempoolSize, vm.verifyTxAtTip) - if err != nil { - return fmt.Errorf("failed to initialize mempool: %w", err) - } - - vm.networkCodec = message.Codec - vm.Network, err = network.NewNetwork(vm.ctx, appSender, vm.networkCodec, vm.config.MaxOutboundActiveRequests, vm.sdkMetrics) + vm.Network, err = network.NewNetwork(vm.ctx, appSender, vm.extensionConfig.NetworkCodec, vm.config.MaxOutboundActiveRequests, vm.sdkMetrics) if err != nil { return fmt.Errorf("failed to create network: %w", err) } @@ -508,7 +430,7 @@ func (vm *VM) Initialize( // clear warpdb on initialization if config enabled if vm.config.PruneWarpDB { - if err := database.Clear(vm.warpDB, ethdb.IdealBatchSize); err != nil { + if err := avalanchedatabase.Clear(vm.warpDB, ethdb.IdealBatchSize); err != nil { return fmt.Errorf("failed to prune warpDB: %w", err) } } @@ -528,54 +450,14 @@ func (vm *VM) Initialize( if err := vm.initializeChain(lastAcceptedHash); err != nil { return err } - // initialize bonus blocks on mainnet - var ( - bonusBlockHeights map[uint64]ids.ID - ) - if vm.ctx.NetworkID == avalanchegoConstants.MainnetID { - bonusBlockHeights, err = readMainnetBonusBlocks() - if err != nil { - return fmt.Errorf("failed to read mainnet bonus blocks: %w", err) - } - } - - // initialize atomic repository - vm.atomicTxRepository, err = atomicstate.NewAtomicTxRepository(vm.versiondb, atomic.Codec, lastAcceptedHeight) - if err != nil { - return fmt.Errorf("failed to create atomic repository: %w", err) - } - vm.atomicBackend, err = atomicstate.NewAtomicBackend( - vm.ctx.SharedMemory, bonusBlockHeights, - vm.atomicTxRepository, lastAcceptedHeight, lastAcceptedHash, - vm.config.CommitInterval, - ) - if err != nil { - return fmt.Errorf("failed to create atomic backend: %w", err) - } go vm.ctx.Log.RecoverAndPanic(vm.startContinuousProfiler) - // so [vm.baseCodec] is a dummy codec use to fulfill the secp256k1fx VM - // interface. The fx will register all of its types, which can be safely - // ignored by the VM's codec. - vm.baseCodec = linearcodec.NewDefault() - - if err := vm.fx.Initialize(vm); err != nil { - return err - } - // Add p2p warp message warpHandler warpHandler := acp118.NewCachedHandler(meteredCache, vm.warpBackend, vm.ctx.WarpSigner) vm.Network.AddHandler(p2p.SignatureRequestHandlerID, warpHandler) - vm.setAppRequestHandlers() - - vm.StateSyncServer = NewStateSyncServer(&stateSyncServerConfig{ - Chain: vm.blockChain, - AtomicTrie: vm.atomicBackend.AtomicTrie(), - SyncableInterval: vm.config.StateSyncCommitInterval, - }) - return vm.initializeStateSyncClient(lastAcceptedHeight) + return vm.initializeStateSync(lastAcceptedHeight) } func parseGenesis(ctx *snow.Context, bytes []byte) (*core.Genesis, error) { @@ -629,7 +511,6 @@ func (vm *VM) initializeChain(lastAcceptedHash common.Hash) error { if err != nil { return err } - callbacks := vm.createConsensusCallbacks() // If the gas target is specified, calculate the desired target excess and // use it during block creation. @@ -647,12 +528,12 @@ func (vm *VM) initializeChain(lastAcceptedHash common.Hash) error { eth.Settings{MaxBlocksPerRequest: vm.config.MaxBlocksPerRequest}, lastAcceptedHash, dummy.NewDummyEngine( - callbacks, + vm.extensionConfig.ConsensusCallbacks, dummy.Mode{}, - &vm.clock, + vm.clock, desiredTargetExcess, ), - &vm.clock, + vm.clock, ) if err != nil { return err @@ -671,10 +552,56 @@ func (vm *VM) initializeChain(lastAcceptedHash common.Hash) error { return vm.initChainState(vm.blockChain.LastAcceptedBlock()) } -// initializeStateSyncClient initializes the client for performing state sync. +// initializeStateSync initializes the vm for performing state sync and responding to peer requests. // If state sync is disabled, this function will wipe any ongoing summary from // disk to ensure that we do not continue syncing from an invalid snapshot. -func (vm *VM) initializeStateSyncClient(lastAcceptedHeight uint64) error { +func (vm *VM) initializeStateSync(lastAcceptedHeight uint64) error { + // Create standalone EVM TrieDB (read only) for serving leafs requests. + // We create a standalone TrieDB here, so that it has a standalone cache from the one + // used by the node when processing blocks. + evmTrieDB := triedb.NewDatabase( + vm.chaindb, + &triedb.Config{ + DBOverride: hashdb.Config{ + CleanCacheSize: vm.config.StateSyncServerTrieCache * units.MiB, + }.BackendConstructor, + }, + ) + var leafHandlerConfigs []*extension.LeafRequestConfig + syncStats := handlerstats.GetOrRegisterHandlerStats(metrics.Enabled) + // register default leaf request handler for state trie + leafHandlerConfigs = append(leafHandlerConfigs, &extension.LeafRequestConfig{ + LeafType: message.StateTrieNode, + MetricName: "sync_state_trie_leaves", + Handler: handlers.NewLeafsRequestHandler(evmTrieDB, + message.StateTrieKeyLength, + vm.blockChain, vm.extensionConfig.NetworkCodec, + syncStats, + ), + }) + + if vm.extensionConfig.ExtraSyncLeafHandlerConfig != nil { + leafHandlerConfigs = append(leafHandlerConfigs, vm.extensionConfig.ExtraSyncLeafHandlerConfig) + } + + leafHandlers := make(LeafHandlers, len(leafHandlerConfigs)) + for _, leafConfig := range leafHandlerConfigs { + if _, exists := leafHandlers[leafConfig.LeafType]; exists { + return fmt.Errorf("duplicate leaf type %v", leafConfig.LeafType) + } + leafHandlers[leafConfig.LeafType] = leafConfig.Handler + } + networkHandler := newNetworkHandler( + vm.blockChain, + vm.chaindb, + vm.warpBackend, + vm.extensionConfig.NetworkCodec, + leafHandlers, + syncStats, + ) + vm.Network.SetRequestHandler(networkHandler) + + vm.Server = vmsync.SyncServer(vm.blockChain, vm.extensionConfig.SyncSummaryProvider, vm.config.StateSyncCommitInterval) stateSyncEnabled := vm.stateSyncEnabled(lastAcceptedHeight) // parse nodeIDs from state sync IDs in vm config var stateSyncIDs []ids.NodeID @@ -690,42 +617,48 @@ func (vm *VM) initializeStateSyncClient(lastAcceptedHeight uint64) error { } } - vm.StateSyncClient = NewStateSyncClient(&stateSyncClientConfig{ - chain: vm.eth, - state: vm.State, - client: statesyncclient.NewClient( + // Initialize the state sync client + leafMetricsNames := make(map[message.NodeType]string, len(leafHandlerConfigs)) + for _, leafConfig := range leafHandlerConfigs { + leafMetricsNames[leafConfig.LeafType] = leafConfig.MetricName + } + vm.Client = vmsync.NewClient(&vmsync.ClientConfig{ + Chain: vm.eth, + State: vm.State, + Client: statesyncclient.NewClient( &statesyncclient.ClientConfig{ NetworkClient: vm.Network, - Codec: vm.networkCodec, - Stats: stats.NewClientSyncerStats(), + Codec: vm.extensionConfig.NetworkCodec, + Stats: stats.NewClientSyncerStats(leafMetricsNames), StateSyncNodeIDs: stateSyncIDs, BlockParser: vm, }, ), - enabled: stateSyncEnabled, - skipResume: vm.config.StateSyncSkipResume, - stateSyncMinBlocks: vm.config.StateSyncMinBlocks, - stateSyncRequestSize: vm.config.StateSyncRequestSize, - lastAcceptedHeight: lastAcceptedHeight, // TODO clean up how this is passed around - chaindb: vm.chaindb, - metadataDB: vm.metadataDB, - acceptedBlockDB: vm.acceptedBlockDB, - db: vm.versiondb, - atomicBackend: vm.atomicBackend, - toEngine: vm.toEngine, + Enabled: stateSyncEnabled, + SkipResume: vm.config.StateSyncSkipResume, + StateSyncMinBlocks: vm.config.StateSyncMinBlocks, + StateSyncRequestSize: vm.config.StateSyncRequestSize, + LastAcceptedHeight: lastAcceptedHeight, // TODO clean up how this is passed around + ChaindDB: vm.chaindb, + VerDB: vm.versiondb, + MetadataDB: vm.metadataDB, + ToEngine: vm.toEngine, + Acceptor: vm, + SyncableParser: vm.extensionConfig.SyncableParser, + SyncExtender: vm.extensionConfig.SyncExtender, }) // If StateSync is disabled, clear any ongoing summary so that we will not attempt to resume // sync using a snapshot that has been modified by the node running normal operations. if !stateSyncEnabled { - return vm.StateSyncClient.ClearOngoingSummary() + return vm.Client.ClearOngoingSummary() } return nil } func (vm *VM) initChainState(lastAcceptedBlock *types.Block) error { - block, err := vm.newBlock(lastAcceptedBlock) + block, err := wrapBlock(lastAcceptedBlock, vm) if err != nil { return fmt.Errorf("failed to create block wrapper for the last accepted block: %w", err) } @@ -757,251 +690,6 @@ func (vm *VM) initChainState(lastAcceptedBlock *types.Block) error { return vm.ctx.Metrics.Register(chainStateMetricsPrefix, chainStateRegisterer) } -func (vm *VM) createConsensusCallbacks() dummy.ConsensusCallbacks { - return dummy.ConsensusCallbacks{ - OnFinalizeAndAssemble: vm.onFinalizeAndAssemble, - OnExtraStateChange: vm.onExtraStateChange, - } -} - -func (vm *VM) preBatchOnFinalizeAndAssemble(header *types.Header, state *state.StateDB, txs []*types.Transaction) ([]byte, *big.Int, *big.Int, error) { - for { - tx, exists := vm.mempool.NextTx() - if !exists { - break - } - // Take a snapshot of [state] before calling verifyTx so that if the transaction fails verification - // we can revert to [snapshot]. - // Note: snapshot is taken inside the loop because you cannot revert to the same snapshot more than - // once. - snapshot := state.Snapshot() - rules := vm.rules(header.Number, header.Time) - if err := vm.verifyTx(tx, header.ParentHash, header.BaseFee, state, rules); err != nil { - // Discard the transaction from the mempool on failed verification. - log.Debug("discarding tx from mempool on failed verification", "txID", tx.ID(), "err", err) - vm.mempool.DiscardCurrentTx(tx.ID()) - state.RevertToSnapshot(snapshot) - continue - } - - atomicTxBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, tx) - if err != nil { - // Discard the transaction from the mempool and error if the transaction - // cannot be marshalled. This should never happen. - log.Debug("discarding tx due to unmarshal err", "txID", tx.ID(), "err", err) - vm.mempool.DiscardCurrentTx(tx.ID()) - return nil, nil, nil, fmt.Errorf("failed to marshal atomic transaction %s due to %w", tx.ID(), err) - } - var contribution, gasUsed *big.Int - if rules.IsApricotPhase4 { - contribution, gasUsed, err = tx.BlockFeeContribution(rules.IsApricotPhase5, vm.ctx.AVAXAssetID, header.BaseFee) - if err != nil { - return nil, nil, nil, err - } - } - return atomicTxBytes, contribution, gasUsed, nil - } - - if len(txs) == 0 { - // this could happen due to the async logic of geth tx pool - return nil, nil, nil, errEmptyBlock - } - - return nil, nil, nil, nil -} - -// assumes that we are in at least Apricot Phase 5. -func (vm *VM) postBatchOnFinalizeAndAssemble( - header *types.Header, - parent *types.Header, - state *state.StateDB, - txs []*types.Transaction, -) ([]byte, *big.Int, *big.Int, error) { - var ( - batchAtomicTxs []*atomic.Tx - batchAtomicUTXOs set.Set[ids.ID] - batchContribution *big.Int = new(big.Int).Set(common.Big0) - batchGasUsed *big.Int = new(big.Int).Set(common.Big0) - rules = vm.rules(header.Number, header.Time) - size int - ) - - atomicGasLimit, err := customheader.RemainingAtomicGasCapacity(vm.chainConfigExtra(), parent, header) - if err != nil { - return nil, nil, nil, err - } - - for { - tx, exists := vm.mempool.NextTx() - if !exists { - break - } - - // Ensure that adding [tx] to the block will not exceed the block size soft limit. - txSize := len(tx.SignedBytes()) - if size+txSize > targetAtomicTxsSize { - vm.mempool.CancelCurrentTx(tx.ID()) - break - } - - var ( - txGasUsed, txContribution *big.Int - err error - ) - - // Note: we do not need to check if we are in at least ApricotPhase4 here because - // we assume that this function will only be called when the block is in at least - // ApricotPhase5. - txContribution, txGasUsed, err = tx.BlockFeeContribution(true, vm.ctx.AVAXAssetID, header.BaseFee) - if err != nil { - return nil, nil, nil, err - } - // ensure `gasUsed + batchGasUsed` doesn't exceed `atomicGasLimit` - if totalGasUsed := new(big.Int).Add(batchGasUsed, txGasUsed); !utils.BigLessOrEqualUint64(totalGasUsed, atomicGasLimit) { - // Send [tx] back to the mempool's tx heap. - vm.mempool.CancelCurrentTx(tx.ID()) - break - } - - if batchAtomicUTXOs.Overlaps(tx.InputUTXOs()) { - // Discard the transaction from the mempool since it will fail verification - // after this block has been accepted. - // Note: if the proposed block is not accepted, the transaction may still be - // valid, but we discard it early here based on the assumption that the proposed - // block will most likely be accepted. - // Discard the transaction from the mempool on failed verification. - log.Debug("discarding tx due to overlapping input utxos", "txID", tx.ID()) - vm.mempool.DiscardCurrentTx(tx.ID()) - continue - } - - snapshot := state.Snapshot() - if err := vm.verifyTx(tx, header.ParentHash, header.BaseFee, state, rules); err != nil { - // Discard the transaction from the mempool and reset the state to [snapshot] - // if it fails verification here. - // Note: prior to this point, we have not modified [state] so there is no need to - // revert to a snapshot if we discard the transaction prior to this point. - log.Debug("discarding tx from mempool due to failed verification", "txID", tx.ID(), "err", err) - vm.mempool.DiscardCurrentTx(tx.ID()) - state.RevertToSnapshot(snapshot) - continue - } - - batchAtomicTxs = append(batchAtomicTxs, tx) - batchAtomicUTXOs.Union(tx.InputUTXOs()) - // Add the [txGasUsed] to the [batchGasUsed] when the [tx] has passed verification - batchGasUsed.Add(batchGasUsed, txGasUsed) - batchContribution.Add(batchContribution, txContribution) - size += txSize - } - - // If there is a non-zero number of transactions, marshal them and return the byte slice - // for the block's extra data along with the contribution and gas used. - if len(batchAtomicTxs) > 0 { - atomicTxBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, batchAtomicTxs) - if err != nil { - // If we fail to marshal the batch of atomic transactions for any reason, - // discard the entire set of current transactions. - log.Debug("discarding txs due to error marshaling atomic transactions", "err", err) - vm.mempool.DiscardCurrentTxs() - return nil, nil, nil, fmt.Errorf("failed to marshal batch of atomic transactions due to %w", err) - } - return atomicTxBytes, batchContribution, batchGasUsed, nil - } - - // If there are no regular transactions and there were also no atomic transactions to be included, - // then the block is empty and should be considered invalid. - if len(txs) == 0 { - // this could happen due to the async logic of geth tx pool - return nil, nil, nil, errEmptyBlock - } - - // If there are no atomic transactions, but there is a non-zero number of regular transactions, then - // we return a nil slice with no contribution from the atomic transactions and a nil error. - return nil, nil, nil, nil -} - -func (vm *VM) onFinalizeAndAssemble( - header *types.Header, - parent *types.Header, - state *state.StateDB, - txs []*types.Transaction, -) ([]byte, *big.Int, *big.Int, error) { - if !vm.chainConfigExtra().IsApricotPhase5(header.Time) { - return vm.preBatchOnFinalizeAndAssemble(header, state, txs) - } - return vm.postBatchOnFinalizeAndAssemble(header, parent, state, txs) -} - -func (vm *VM) onExtraStateChange(block *types.Block, parent *types.Header, state *state.StateDB) (*big.Int, *big.Int, error) { - var ( - batchContribution *big.Int = big.NewInt(0) - batchGasUsed *big.Int = big.NewInt(0) - header = block.Header() - rules = vm.rules(header.Number, header.Time) - ) - - txs, err := atomic.ExtractAtomicTxs(customtypes.BlockExtData(block), rules.IsApricotPhase5, atomic.Codec) - if err != nil { - return nil, nil, err - } - - // If [atomicBackend] is nil, the VM is still initializing and is reprocessing accepted blocks. - if vm.atomicBackend != nil { - if vm.atomicBackend.IsBonus(block.NumberU64(), block.Hash()) { - log.Info("skipping atomic tx verification on bonus block", "block", block.Hash()) - } else { - // Verify [txs] do not conflict with themselves or ancestor blocks. - if err := vm.verifyTxs(txs, block.ParentHash(), block.BaseFee(), block.NumberU64(), rules); err != nil { - return nil, nil, err - } - } - // Update the atomic backend with [txs] from this block. - // - // Note: The atomic trie canonically contains the duplicate operations - // from any bonus blocks. - _, err := vm.atomicBackend.InsertTxs(block.Hash(), block.NumberU64(), block.ParentHash(), txs) - if err != nil { - return nil, nil, err - } - } - - // If there are no transactions, we can return early. - if len(txs) == 0 { - return nil, nil, nil - } - - for _, tx := range txs { - if err := tx.UnsignedAtomicTx.EVMStateTransfer(vm.ctx, state); err != nil { - return nil, nil, err - } - // If ApricotPhase4 is enabled, calculate the block fee contribution - if rules.IsApricotPhase4 { - contribution, gasUsed, err := tx.BlockFeeContribution(rules.IsApricotPhase5, vm.ctx.AVAXAssetID, block.BaseFee()) - if err != nil { - return nil, nil, err - } - - batchContribution.Add(batchContribution, contribution) - batchGasUsed.Add(batchGasUsed, gasUsed) - } - } - - // If ApricotPhase5 is enabled, enforce that the atomic gas used does not exceed the - // atomic gas limit. - if rules.IsApricotPhase5 { - atomicGasLimit, err := customheader.RemainingAtomicGasCapacity(vm.chainConfigExtra(), parent, header) - if err != nil { - return nil, nil, err - } - - if !utils.BigLessOrEqualUint64(batchGasUsed, atomicGasLimit) { - return nil, nil, fmt.Errorf("atomic gas used (%d) by block (%s), exceeds atomic gas limit (%d)", batchGasUsed, block.Hash().Hex(), atomicGasLimit) - } - } - return batchContribution, batchGasUsed, nil -} - func (vm *VM) SetState(_ context.Context, state snow.State) error { switch state { case snow.StateSyncing: @@ -1019,18 +707,17 @@ func (vm *VM) SetState(_ context.Context, state snow.State) error { // onBootstrapStarted marks this VM as bootstrapping func (vm *VM) onBootstrapStarted() error { vm.bootstrapped.Set(false) - if err := vm.StateSyncClient.Error(); err != nil { + if err := vm.Client.Error(); err != nil { return err } // After starting bootstrapping, do not attempt to resume a previous state sync. - if err := vm.StateSyncClient.ClearOngoingSummary(); err != nil { + if err := vm.Client.ClearOngoingSummary(); err != nil { return err } // Ensure snapshots are initialized before bootstrapping (i.e., if state sync is skipped). // Note calling this function has no effect if snapshots are already initialized. vm.blockChain.InitializeSnapshots() - - return vm.fx.Bootstrapping() + return nil } // onNormalOperationsStarted marks this VM as bootstrapped @@ -1039,9 +726,6 @@ func (vm *VM) onNormalOperationsStarted() error { return nil } vm.bootstrapped.Set(true) - if err := vm.fx.Bootstrapped(); err != nil { - return err - } // Initialize goroutines related to block building // once we enter normal operation as there is no need to handle mempool gossip before this point. return vm.initBlockBuilding() @@ -1054,7 +738,7 @@ func (vm *VM) initBlockBuilding() error { ethTxGossipMarshaller := GossipEthTxMarshaller{} ethTxGossipClient := vm.Network.NewClient(p2p.TxGossipHandlerID, p2p.WithValidatorSampling(vm.P2PValidators())) - ethTxGossipMetrics, err := gossip.NewMetrics(vm.sdkMetrics, ethTxGossipNamespace) + ethTxGossipMetrics, err := avalanchegossip.NewMetrics(vm.sdkMetrics, ethTxGossipNamespace) if err != nil { return fmt.Errorf("failed to initialize eth tx gossip metrics: %w", err) } @@ -1067,27 +751,19 @@ func (vm *VM) initBlockBuilding() error { ethTxPool.Subscribe(ctx) vm.shutdownWg.Done() }() - - atomicTxGossipMarshaller := &atomic.TxMarshaller{} - atomicTxGossipClient := vm.Network.NewClient(p2p.AtomicTxGossipHandlerID, p2p.WithValidatorSampling(vm.P2PValidators())) - atomicTxGossipMetrics, err := gossip.NewMetrics(vm.sdkMetrics, atomicTxGossipNamespace) - if err != nil { - return fmt.Errorf("failed to initialize atomic tx gossip metrics: %w", err) - } - - pushGossipParams := gossip.BranchingFactor{ + pushGossipParams := avalanchegossip.BranchingFactor{ StakePercentage: vm.config.PushGossipPercentStake, Validators: vm.config.PushGossipNumValidators, Peers: vm.config.PushGossipNumPeers, } - pushRegossipParams := gossip.BranchingFactor{ + pushRegossipParams := avalanchegossip.BranchingFactor{ Validators: vm.config.PushRegossipNumValidators, Peers: vm.config.PushRegossipNumPeers, } ethTxPushGossiper := vm.ethTxPushGossiper.Get() if ethTxPushGossiper == nil { - ethTxPushGossiper, err = gossip.NewPushGossiper[*GossipEthTx]( + ethTxPushGossiper, err = avalanchegossip.NewPushGossiper[*GossipEthTx]( ethTxGossipMarshaller, ethTxPool, vm.P2PValidators(), @@ -1095,8 +771,8 @@ func (vm *VM) initBlockBuilding() error { ethTxGossipMetrics, pushGossipParams, pushRegossipParams, - pushGossipDiscardedElements, - txGossipTargetMessageSize, + config.PushGossipDiscardedElements, + config.TxGossipTargetMessageSize, vm.config.RegossipFrequency.Duration, ) if err != nil { @@ -1105,37 +781,19 @@ func (vm *VM) initBlockBuilding() error { vm.ethTxPushGossiper.Set(ethTxPushGossiper) } - if vm.atomicTxPushGossiper == nil { - vm.atomicTxPushGossiper, err = gossip.NewPushGossiper[*atomic.Tx]( - atomicTxGossipMarshaller, - vm.mempool, - vm.P2PValidators(), - atomicTxGossipClient, - atomicTxGossipMetrics, - pushGossipParams, - pushRegossipParams, - pushGossipDiscardedElements, - txGossipTargetMessageSize, - vm.config.RegossipFrequency.Duration, - ) - if err != nil { - return fmt.Errorf("failed to initialize atomic tx push gossiper: %w", err) - } - } - // NOTE: gossip network must be initialized first otherwise ETH tx gossip will not work. - vm.builder = vm.NewBlockBuilder(vm.toEngine) + vm.builder = vm.NewBlockBuilder(vm.toEngine, vm.extensionConfig.ExtraMempool) vm.builder.awaitSubmittedTxs() if vm.ethTxGossipHandler == nil { - vm.ethTxGossipHandler = newTxGossipHandler[*GossipEthTx]( + vm.ethTxGossipHandler = gossip.NewTxGossipHandler[*GossipEthTx]( vm.ctx.Log, ethTxGossipMarshaller, ethTxPool, ethTxGossipMetrics, - txGossipTargetMessageSize, - txGossipThrottlingPeriod, - txGossipThrottlingLimit, + config.TxGossipTargetMessageSize, + config.TxGossipThrottlingPeriod, + config.TxGossipThrottlingLimit, vm.P2PValidators(), ) } @@ -1144,34 +802,17 @@ func (vm *VM) initBlockBuilding() error { return fmt.Errorf("failed to add eth tx gossip handler: %w", err) } - if vm.atomicTxGossipHandler == nil { - vm.atomicTxGossipHandler = newTxGossipHandler[*atomic.Tx]( - vm.ctx.Log, - atomicTxGossipMarshaller, - vm.mempool, - atomicTxGossipMetrics, - txGossipTargetMessageSize, - txGossipThrottlingPeriod, - txGossipThrottlingLimit, - vm.P2PValidators(), - ) - } - - if err := vm.Network.AddHandler(p2p.AtomicTxGossipHandlerID, vm.atomicTxGossipHandler); err != nil { - return fmt.Errorf("failed to add atomic tx gossip handler: %w", err) - } - if vm.ethTxPullGossiper == nil { - ethTxPullGossiper := gossip.NewPullGossiper[*GossipEthTx]( + ethTxPullGossiper := avalanchegossip.NewPullGossiper[*GossipEthTx]( vm.ctx.Log, ethTxGossipMarshaller, ethTxPool, ethTxGossipClient, ethTxGossipMetrics, - txGossipPollSize, + config.TxGossipPollSize, ) - vm.ethTxPullGossiper = gossip.ValidatorGossiper{ + vm.ethTxPullGossiper = avalanchegossip.ValidatorGossiper{ Gossiper: ethTxPullGossiper, NodeID: vm.ctx.NodeID, Validators: vm.P2PValidators(), @@ -1180,71 +821,29 @@ func (vm *VM) initBlockBuilding() error { vm.shutdownWg.Add(1) go func() { - gossip.Every(ctx, vm.ctx.Log, ethTxPushGossiper, vm.config.PushGossipFrequency.Duration) + avalanchegossip.Every(ctx, vm.ctx.Log, ethTxPushGossiper, vm.config.PushGossipFrequency.Duration) vm.shutdownWg.Done() }() vm.shutdownWg.Add(1) go func() { - gossip.Every(ctx, vm.ctx.Log, vm.ethTxPullGossiper, vm.config.PullGossipFrequency.Duration) + avalanchegossip.Every(ctx, vm.ctx.Log, vm.ethTxPullGossiper, vm.config.PullGossipFrequency.Duration) vm.shutdownWg.Done() }() - if vm.atomicTxPullGossiper == nil { - atomicTxPullGossiper := gossip.NewPullGossiper[*atomic.Tx]( - vm.ctx.Log, - atomicTxGossipMarshaller, - vm.mempool, - atomicTxGossipClient, - atomicTxGossipMetrics, - txGossipPollSize, - ) - - vm.atomicTxPullGossiper = &gossip.ValidatorGossiper{ - Gossiper: atomicTxPullGossiper, - NodeID: vm.ctx.NodeID, - Validators: vm.P2PValidators(), - } - } - vm.shutdownWg.Add(1) go func() { - gossip.Every(ctx, vm.ctx.Log, vm.atomicTxPushGossiper, vm.config.PushGossipFrequency.Duration) + avalanchegossip.Every(ctx, vm.ctx.Log, ethTxPushGossiper, vm.config.PushGossipFrequency.Duration) vm.shutdownWg.Done() }() vm.shutdownWg.Add(1) go func() { - gossip.Every(ctx, vm.ctx.Log, vm.atomicTxPullGossiper, vm.config.PullGossipFrequency.Duration) + avalanchegossip.Every(ctx, vm.ctx.Log, vm.ethTxPullGossiper, vm.config.PullGossipFrequency.Duration) vm.shutdownWg.Done() }() return nil } -// setAppRequestHandlers sets the request handlers for the VM to serve state sync -// requests. -func (vm *VM) setAppRequestHandlers() { - // Create standalone EVM TrieDB (read only) for serving leafs requests. - // We create a standalone TrieDB here, so that it has a standalone cache from the one - // used by the node when processing blocks. - evmTrieDB := triedb.NewDatabase( - vm.chaindb, - &triedb.Config{ - DBOverride: hashdb.Config{ - CleanCacheSize: vm.config.StateSyncServerTrieCache * units.MiB, - }.BackendConstructor, - }, - ) - networkHandler := newNetworkHandler( - vm.blockChain, - vm.chaindb, - evmTrieDB, - vm.atomicBackend.AtomicTrie().TrieDB(), - vm.warpBackend, - vm.networkCodec, - ) - vm.Network.SetRequestHandler(networkHandler) -} - // Shutdown implements the snowman.ChainVM interface func (vm *VM) Shutdown(context.Context) error { if vm.ctx == nil { @@ -1254,7 +853,7 @@ func (vm *VM) Shutdown(context.Context) error { vm.cancel() } vm.Network.Shutdown() - if err := vm.StateSyncClient.Shutdown(); err != nil { + if err := vm.Client.Shutdown(); err != nil { log.Error("error stopping state syncer", "err", err) } close(vm.shutdownChan) @@ -1286,18 +885,14 @@ func (vm *VM) buildBlockWithContext(ctx context.Context, proposerVMBlockCtx *blo block, err := vm.miner.GenerateBlock(predicateCtx) vm.builder.handleGenerateBlock() if err != nil { - vm.mempool.CancelCurrentTxs() - return nil, err + return nil, fmt.Errorf("%w: %w", vmerrors.ErrGenerateBlockFailed, err) } // Note: the status of block is set by ChainState - blk, err := vm.newBlock(block) + blk, err := wrapBlock(block, vm) if err != nil { - log.Debug("discarding txs due to error making new block", "err", err) - vm.mempool.DiscardCurrentTxs() - return nil, err + return nil, fmt.Errorf("%w: %w", vmerrors.ErrWrapBlockFailed, err) } - // Verify is called on a non-wrapped block here, such that this // does not add [blk] to the processing blocks map in ChainState. // @@ -1311,16 +906,12 @@ func (vm *VM) buildBlockWithContext(ctx context.Context, proposerVMBlockCtx *blo // to the blk state root in the triedb when we are going to call verify // again from the consensus engine with writes enabled. if err := blk.verify(predicateCtx, false /*=writes*/); err != nil { - vm.mempool.CancelCurrentTxs() - return nil, fmt.Errorf("block failed verification due to: %w", err) + return nil, fmt.Errorf("%w: %w", vmerrors.ErrBlockVerificationFailed, err) } log.Debug("built block", "id", blk.ID(), ) - // Marks the current transactions from the mempool as being successfully issued - // into a block. - vm.mempool.IssueCurrentTxs() return blk, nil } @@ -1332,7 +923,7 @@ func (vm *VM) parseBlock(_ context.Context, b []byte) (snowman.Block, error) { } // Note: the status of block is set by ChainState - block, err := vm.newBlock(ethBlock) + block, err := wrapBlock(ethBlock, vm) if err != nil { return nil, err } @@ -1350,20 +941,20 @@ func (vm *VM) ParseEthBlock(b []byte) (*types.Block, error) { return nil, err } - return block.(*Block).ethBlock, nil + return block.(*wrappedBlock).ethBlock, nil } // getBlock attempts to retrieve block [id] from the VM to be wrapped // by ChainState. func (vm *VM) getBlock(_ context.Context, id ids.ID) (snowman.Block, error) { ethBlock := vm.blockChain.GetBlockByHash(common.Hash(id)) - // If [ethBlock] is nil, return [database.ErrNotFound] here + // If [ethBlock] is nil, return [avalanchedatabase.ErrNotFound] here // so that the miss is considered cacheable. if ethBlock == nil { - return nil, database.ErrNotFound + return nil, avalanchedatabase.ErrNotFound } // Note: the status of block is set by ChainState - return vm.newBlock(ethBlock) + return wrapBlock(ethBlock, vm) } // GetAcceptedBlock attempts to retrieve block [blkID] from the VM. This method @@ -1382,56 +973,38 @@ func (vm *VM) GetAcceptedBlock(ctx context.Context, blkID ids.ID) (snowman.Block if acceptedBlkID != blkID { // The provided block is not accepted. - return nil, database.ErrNotFound + return nil, avalanchedatabase.ErrNotFound } return blk, nil } // SetPreference sets what the current tail of the chain is func (vm *VM) SetPreference(ctx context.Context, blkID ids.ID) error { - // Since each internal handler used by [vm.State] always returns a block - // with non-nil ethBlock value, GetBlockInternal should never return a - // (*Block) with a nil ethBlock value. - block, err := vm.GetBlockInternal(ctx, blkID) + block, err := vm.GetVMBlock(ctx, blkID) if err != nil { return fmt.Errorf("failed to set preference to %s: %w", blkID, err) } - return vm.blockChain.SetPreference(block.(*Block).ethBlock) + return vm.blockChain.SetPreference(block.GetEthBlock()) } // GetBlockIDAtHeight returns the canonical block at [height]. // Note: the engine assumes that if a block is not found at [height], then -// [database.ErrNotFound] will be returned. This indicates that the VM has state +// [avalanchedatabase.ErrNotFound] will be returned. This indicates that the VM has state // synced and does not have all historical blocks available. func (vm *VM) GetBlockIDAtHeight(_ context.Context, height uint64) (ids.ID, error) { lastAcceptedBlock := vm.LastAcceptedBlock() if lastAcceptedBlock.Height() < height { - return ids.ID{}, database.ErrNotFound + return ids.ID{}, avalanchedatabase.ErrNotFound } hash := vm.blockChain.GetCanonicalHash(height) if hash == (common.Hash{}) { - return ids.ID{}, database.ErrNotFound + return ids.ID{}, avalanchedatabase.ErrNotFound } return ids.ID(hash), nil } -func (vm *VM) Version(context.Context) (string, error) { - return Version, nil -} - -// NewHandler returns a new Handler for a service where: -// - The handler's functionality is defined by [service] -// [service] should be a gorilla RPC service (see https://www.gorillatoolkit.org/pkg/rpc/v2) -// - The name of the service is [name] -func newHandler(name string, service interface{}) (http.Handler, error) { - server := avalancheRPC.NewServer() - server.RegisterCodec(avalancheJSON.NewCodec(), "application/json") - server.RegisterCodec(avalancheJSON.NewCodec(), "application/json;charset=UTF-8") - return server, server.RegisterService(service, name) -} - // CreateHandlers makes new http handlers that can handle API calls func (vm *VM) CreateHandlers(context.Context) (map[string]http.Handler, error) { handler := rpc.NewServer(vm.config.APIMaxDuration.Duration) @@ -1445,15 +1018,9 @@ func (vm *VM) CreateHandlers(context.Context) (map[string]http.Handler, error) { } apis := make(map[string]http.Handler) - avaxAPI, err := newHandler("avax", &AvaxAPI{vm}) - if err != nil { - return nil, fmt.Errorf("failed to register service for AVAX API due to %w", err) - } - enabledAPIs = append(enabledAPIs, "avax") - apis[avaxEndpoint] = avaxAPI if vm.config.AdminAPIEnabled { - adminAPI, err := newHandler("admin", NewAdminService(vm, os.ExpandEnv(fmt.Sprintf("%s_coreth_performance_%s", vm.config.AdminAPIDir, vm.chainAlias)))) + adminAPI, err := utils.NewHandler("admin", NewAdminService(vm, os.ExpandEnv(fmt.Sprintf("%s_coreth_performance_%s", vm.config.AdminAPIDir, vm.chainAlias)))) if err != nil { return nil, fmt.Errorf("failed to register service for admin API due to %w", err) } @@ -1470,7 +1037,7 @@ func (vm *VM) CreateHandlers(context.Context) (map[string]http.Handler, error) { } if vm.config.WarpAPIEnabled { - warpAPI := warp.NewAPI(vm.ctx, vm.networkCodec, vm.warpBackend, vm.Network, vm.requirePrimaryNetworkSigners) + warpAPI := warp.NewAPI(vm.ctx, vm.extensionConfig.NetworkCodec, vm.warpBackend, vm.Network, vm.requirePrimaryNetworkSigners) if err := handler.RegisterName("warp", warpAPI); err != nil { return nil, err } @@ -1492,203 +1059,6 @@ func (vm *VM) CreateHandlers(context.Context) (map[string]http.Handler, error) { return apis, nil } -/* - ****************************************************************************** - *********************************** Helpers ********************************** - ****************************************************************************** - */ - -// getAtomicTx returns the requested transaction, status, and height. -// If the status is Unknown, then the returned transaction will be nil. -func (vm *VM) getAtomicTx(txID ids.ID) (*atomic.Tx, atomic.Status, uint64, error) { - if tx, height, err := vm.atomicTxRepository.GetByTxID(txID); err == nil { - return tx, atomic.Accepted, height, nil - } else if err != database.ErrNotFound { - return nil, atomic.Unknown, 0, err - } - tx, dropped, found := vm.mempool.GetTx(txID) - switch { - case found && dropped: - return tx, atomic.Dropped, 0, nil - case found: - return tx, atomic.Processing, 0, nil - default: - return nil, atomic.Unknown, 0, nil - } -} - -// ParseAddress takes in an address and produces the ID of the chain it's for -// the ID of the address -func (vm *VM) ParseAddress(addrStr string) (ids.ID, ids.ShortID, error) { - chainIDAlias, hrp, addrBytes, err := address.Parse(addrStr) - if err != nil { - return ids.ID{}, ids.ShortID{}, err - } - - chainID, err := vm.ctx.BCLookup.Lookup(chainIDAlias) - if err != nil { - return ids.ID{}, ids.ShortID{}, err - } - - expectedHRP := avalanchegoConstants.GetHRP(vm.ctx.NetworkID) - if hrp != expectedHRP { - return ids.ID{}, ids.ShortID{}, fmt.Errorf("expected hrp %q but got %q", - expectedHRP, hrp) - } - - addr, err := ids.ToShortID(addrBytes) - if err != nil { - return ids.ID{}, ids.ShortID{}, err - } - return chainID, addr, nil -} - -// verifyTxAtTip verifies that [tx] is valid to be issued on top of the currently preferred block -func (vm *VM) verifyTxAtTip(tx *atomic.Tx) error { - if txByteLen := len(tx.SignedBytes()); txByteLen > targetAtomicTxsSize { - return fmt.Errorf("tx size (%d) exceeds total atomic txs size target (%d)", txByteLen, targetAtomicTxsSize) - } - gasUsed, err := tx.GasUsed(true) - if err != nil { - return err - } - if gasUsed > maxAtomicTxMempoolGas { - return fmt.Errorf("tx gas usage (%d) exceeds maximum allowed mempool gas usage (%d)", gasUsed, maxAtomicTxMempoolGas) - } - - // Note: we fetch the current block and then the state at that block instead of the current state directly - // since we need the header of the current block below. - preferredBlock := vm.blockChain.CurrentBlock() - preferredState, err := vm.blockChain.StateAt(preferredBlock.Root) - if err != nil { - return fmt.Errorf("failed to retrieve block state at tip while verifying atomic tx: %w", err) - } - rules := vm.currentRules() - parentHeader := preferredBlock - var nextBaseFee *big.Int - timestamp := uint64(vm.clock.Time().Unix()) - if vm.chainConfigExtra().IsApricotPhase3(timestamp) { - nextBaseFee, err = customheader.EstimateNextBaseFee(vm.chainConfigExtra(), parentHeader, timestamp) - if err != nil { - // Return extremely detailed error since CalcBaseFee should never encounter an issue here - return fmt.Errorf("failed to calculate base fee with parent timestamp (%d), parent ExtraData: (0x%x), and current timestamp (%d): %w", parentHeader.Time, parentHeader.Extra, timestamp, err) - } - } - - // We don’t need to revert the state here in case verifyTx errors, because - // [preferredState] is thrown away either way. - return vm.verifyTx(tx, parentHeader.Hash(), nextBaseFee, preferredState, rules) -} - -// verifyTx verifies that [tx] is valid to be issued into a block with parent block [parentHash] -// and validated at [state] using [rules] as the current rule set. -// Note: verifyTx may modify [state]. If [state] needs to be properly maintained, the caller is responsible -// for reverting to the correct snapshot after calling this function. If this function is called with a -// throwaway state, then this is not necessary. -func (vm *VM) verifyTx(tx *atomic.Tx, parentHash common.Hash, baseFee *big.Int, state *state.StateDB, rules extras.Rules) error { - parentIntf, err := vm.GetBlockInternal(context.TODO(), ids.ID(parentHash)) - if err != nil { - return fmt.Errorf("failed to get parent block: %w", err) - } - parent, ok := parentIntf.(*Block) - if !ok { - return fmt.Errorf("parent block %s had unexpected type %T", parentIntf.ID(), parentIntf) - } - atomicBackend := &atomic.VerifierBackend{ - Ctx: vm.ctx, - Fx: &vm.fx, - Rules: rules, - Bootstrapped: vm.bootstrapped.Get(), - BlockFetcher: vm, - SecpCache: vm.secpCache, - } - if err := tx.UnsignedAtomicTx.Visit(&atomic.SemanticVerifier{ - Backend: atomicBackend, - Tx: tx, - Parent: parent, - BaseFee: baseFee, - }); err != nil { - return err - } - return tx.UnsignedAtomicTx.EVMStateTransfer(vm.ctx, state) -} - -// verifyTxs verifies that [txs] are valid to be issued into a block with parent block [parentHash] -// using [rules] as the current rule set. -func (vm *VM) verifyTxs(txs []*atomic.Tx, parentHash common.Hash, baseFee *big.Int, height uint64, rules extras.Rules) error { - // Ensure that the parent was verified and inserted correctly. - if !vm.blockChain.HasBlock(parentHash, height-1) { - return errRejectedParent - } - - ancestorID := ids.ID(parentHash) - // If the ancestor is unknown, then the parent failed verification when - // it was called. - // If the ancestor is rejected, then this block shouldn't be inserted - // into the canonical chain because the parent will be missing. - ancestorInf, err := vm.GetBlockInternal(context.TODO(), ancestorID) - if err != nil { - return errRejectedParent - } - ancestor, ok := ancestorInf.(*Block) - if !ok { - return fmt.Errorf("expected parent block %s, to be *Block but is %T", ancestor.ID(), ancestorInf) - } - - // Ensure each tx in [txs] doesn't conflict with any other atomic tx in - // a processing ancestor block. - inputs := set.Set[ids.ID]{} - atomicBackend := &atomic.VerifierBackend{ - Ctx: vm.ctx, - Fx: &vm.fx, - Rules: rules, - Bootstrapped: vm.bootstrapped.Get(), - BlockFetcher: vm, - SecpCache: vm.secpCache, - } - for _, atomicTx := range txs { - utx := atomicTx.UnsignedAtomicTx - if err := utx.Visit(&atomic.SemanticVerifier{ - Backend: atomicBackend, - Tx: atomicTx, - Parent: ancestor, - BaseFee: baseFee, - }); err != nil { - return fmt.Errorf("invalid block due to failed semanatic verify: %w at height %d", err, height) - } - txInputs := utx.InputUTXOs() - if inputs.Overlaps(txInputs) { - return atomic.ErrConflictingAtomicInputs - } - inputs.Union(txInputs) - } - return nil -} - -// GetAtomicUTXOs returns the utxos that at least one of the provided addresses is -// referenced in. -func (vm *VM) GetAtomicUTXOs( - chainID ids.ID, - addrs set.Set[ids.ShortID], - startAddr ids.ShortID, - startUTXOID ids.ID, - limit int, -) ([]*avax.UTXO, ids.ShortID, ids.ID, error) { - if limit <= 0 || limit > maxUTXOsToFetch { - limit = maxUTXOsToFetch - } - - return avax.GetAtomicUTXOs( - vm.ctx.SharedMemory, - atomic.Codec, - chainID, - addrs, - startAddr, - startUTXOID, - limit, - ) -} - func (vm *VM) chainConfigExtra() *extras.ChainConfig { return params.GetExtra(vm.chainConfig) } @@ -1742,16 +1112,16 @@ func (vm *VM) startContinuousProfiler() { <-vm.shutdownChan } -// readLastAccepted reads the last accepted hash from [acceptedBlockDB] and returns the +// ReadLastAccepted reads the last accepted hash from [acceptedBlockDB] and returns the // last accepted block hash and height by reading directly from [vm.chaindb] instead of relying // on [chain]. // Note: assumes [vm.chaindb] and [vm.genesisHash] have been initialized. -func (vm *VM) readLastAccepted() (common.Hash, uint64, error) { +func (vm *VM) ReadLastAccepted() (common.Hash, uint64, error) { // Attempt to load last accepted block to determine if it is necessary to // initialize state with the genesis block. lastAcceptedBytes, lastAcceptedErr := vm.acceptedBlockDB.Get(lastAcceptedKey) switch { - case lastAcceptedErr == database.ErrNotFound: + case lastAcceptedErr == avalanchedatabase.ErrNotFound: // If there is nothing in the database, return the genesis block hash and height return vm.genesisHash, 0, nil case lastAcceptedErr != nil: @@ -1815,54 +1185,6 @@ func (vm *VM) stateSyncEnabled(lastAcceptedHeight uint64) bool { return lastAcceptedHeight == 0 } -func (vm *VM) newImportTx( - chainID ids.ID, // chain to import from - to common.Address, // Address of recipient - baseFee *big.Int, // fee to use post-AP3 - keys []*secp256k1.PrivateKey, // Keys to import the funds -) (*atomic.Tx, error) { - kc := secp256k1fx.NewKeychain() - for _, key := range keys { - kc.Add(key) - } - - atomicUTXOs, _, _, err := vm.GetAtomicUTXOs(chainID, kc.Addresses(), ids.ShortEmpty, ids.Empty, -1) - if err != nil { - return nil, fmt.Errorf("problem retrieving atomic UTXOs: %w", err) - } - - return atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), chainID, to, baseFee, kc, atomicUTXOs) -} - -// newExportTx returns a new ExportTx -func (vm *VM) newExportTx( - assetID ids.ID, // AssetID of the tokens to export - amount uint64, // Amount of tokens to export - chainID ids.ID, // Chain to send the UTXOs to - to ids.ShortID, // Address of chain recipient - baseFee *big.Int, // fee to use post-AP3 - keys []*secp256k1.PrivateKey, // Pay the fee and provide the tokens -) (*atomic.Tx, error) { - state, err := vm.blockChain.State() - if err != nil { - return nil, err - } - - // Create the transaction - tx, err := atomic.NewExportTx( - vm.ctx, // Context - vm.currentRules(), // VM rules - state, - assetID, // AssetID - amount, // Amount - chainID, // ID of the chain to send the funds to - to, // Address - baseFee, - keys, // Private keys - ) - if err != nil { - return nil, err - } - - return tx, nil +func (vm *VM) PutLastAcceptedID(ID ids.ID) error { + return vm.acceptedBlockDB.Put(lastAcceptedKey, ID[:]) } diff --git a/plugin/evm/vm_extensible.go b/plugin/evm/vm_extensible.go new file mode 100644 index 0000000000..b5da80dbea --- /dev/null +++ b/plugin/evm/vm_extensible.go @@ -0,0 +1,97 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "context" + "errors" + + "github.com/ava-labs/avalanchego/database/versiondb" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/coreth/eth" + "github.com/ava-labs/coreth/plugin/evm/config" + "github.com/ava-labs/coreth/plugin/evm/extension" + vmsync "github.com/ava-labs/coreth/plugin/evm/sync" + "github.com/prometheus/client_golang/prometheus" + + "github.com/ava-labs/libevm/ethdb" +) + +var _ extension.InnerVM = (*VM)(nil) + +var ( + errVMAlreadyInitialized = errors.New("vm already initialized") + errExtensionConfigAlreadySet = errors.New("extension config already set") +) + +func (vm *VM) SetExtensionConfig(config *extension.Config) error { + if vm.ctx != nil { + return errVMAlreadyInitialized + } + if vm.extensionConfig != nil { + return errExtensionConfigAlreadySet + } + vm.extensionConfig = config + return nil +} + +// All these methods below assumes that VM is already initialized + +func (vm *VM) GetVMBlock(ctx context.Context, blkID ids.ID) (extension.VMBlock, error) { + // Since each internal handler used by [vm.State] always returns a block + // with non-nil ethBlock value, GetBlockInternal should never return a + // (*Block) with a nil ethBlock value. + blk, err := vm.GetBlockInternal(ctx, blkID) + if err != nil { + return nil, err + } + + return blk.(*wrappedBlock), nil +} + +func (vm *VM) LastAcceptedVMBlock() extension.VMBlock { + lastAcceptedBlock := vm.LastAcceptedBlockInternal() + if lastAcceptedBlock == nil { + return nil + } + return lastAcceptedBlock.(*wrappedBlock) +} + +// IsBootstrapped returns true if the VM has finished bootstrapping +func (vm *VM) IsBootstrapped() bool { + return vm.bootstrapped.Get() +} + +func (vm *VM) Ethereum() *eth.Ethereum { + return vm.eth +} + +func (vm *VM) Config() config.Config { + return vm.config +} + +func (vm *VM) MetricRegistry() *prometheus.Registry { + return vm.sdkMetrics +} + +func (vm *VM) Validators() *p2p.Validators { + return vm.P2PValidators() +} + +func (vm *VM) VersionDB() *versiondb.Database { + return vm.versiondb +} + +func (vm *VM) EthChainDB() ethdb.Database { + return vm.chaindb +} + +func (vm *VM) SyncerClient() vmsync.Client { + return vm.Client +} + +func (vm *VM) Version(context.Context) (string, error) { + return Version, nil +} diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index 2ddc132b65..23f54b573a 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -17,87 +17,39 @@ import ( "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/log" - "github.com/ava-labs/libevm/rlp" "github.com/holiman/uint256" "github.com/ava-labs/coreth/constants" - "github.com/ava-labs/coreth/eth/filters" - "github.com/ava-labs/coreth/plugin/evm/atomic" - "github.com/ava-labs/coreth/plugin/evm/config" - "github.com/ava-labs/coreth/plugin/evm/header" + "github.com/ava-labs/coreth/metrics/metricstest" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/message" "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" "github.com/ava-labs/coreth/plugin/evm/upgrade/ap1" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/utils" - "github.com/ava-labs/coreth/utils/utilstest" "github.com/ava-labs/libevm/trie" - "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - "github.com/ava-labs/avalanchego/api/metrics" - avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" "github.com/ava-labs/avalanchego/database" - "github.com/ava-labs/avalanchego/database/memdb" - "github.com/ava-labs/avalanchego/database/prefixdb" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/snow" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/snowtest" "github.com/ava-labs/avalanchego/upgrade" "github.com/ava-labs/avalanchego/upgrade/upgradetest" - "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" - "github.com/ava-labs/avalanchego/utils/hashing" - "github.com/ava-labs/avalanchego/utils/set" - "github.com/ava-labs/avalanchego/utils/units" - "github.com/ava-labs/avalanchego/vms/components/avax" "github.com/ava-labs/avalanchego/vms/components/chain" - "github.com/ava-labs/avalanchego/vms/secp256k1fx" - - commonEng "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ava-labs/avalanchego/snow/engine/enginetest" - "github.com/ava-labs/avalanchego/snow/snowtest" "github.com/ava-labs/coreth/consensus/dummy" "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/eth" "github.com/ava-labs/coreth/params" - atomictxpool "github.com/ava-labs/coreth/plugin/evm/atomic/txpool" "github.com/ava-labs/coreth/plugin/evm/customtypes" "github.com/ava-labs/coreth/rpc" "github.com/ava-labs/libevm/core/types" ) var ( - testKeys = secp256k1.TestKeys()[:3] - testEthAddrs []common.Address // testEthAddrs[i] corresponds to testKeys[i] - testShortIDAddrs []ids.ShortID - - genesisJSON = func(cfg *params.ChainConfig) string { - g := new(core.Genesis) - g.Difficulty = big.NewInt(0) - g.GasLimit = 0x5f5e100 - g.Timestamp = uint64(upgrade.InitiallyActiveTime.Unix()) - - // Use chainId: 43111, so that it does not overlap with any Avalanche ChainIDs, which may have their - // config overridden in vm.Initialize. - cpy := *cfg - cpy.ChainID = big.NewInt(43111) - g.Config = &cpy - - allocStr := `{"0100000000000000000000000000000000000000":{"code":"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033","balance":"0x0"}}` - json.Unmarshal([]byte(allocStr), &g.Alloc) - // After Durango, an additional account is funded in tests to use - // with warp messages. - if params.GetExtra(cfg).IsDurango(0) { - addr := common.HexToAddress("0x99b9DEA54C48Dfea6aA9A4Ca4623633EE04ddbB5") - balance := new(big.Int).Mul(big.NewInt(params.Ether), big.NewInt(10)) - g.Alloc[addr] = types.Account{Balance: balance} - } - - b, err := json.Marshal(g) - if err != nil { - panic(err) - } - return string(b) - } + genesisJSONCancun = vmtest.GenesisJSON(activateCancun(params.TestChainConfig)) activateCancun = func(cfg *params.ChainConfig) *params.ChainConfig { cpy := *cfg @@ -105,255 +57,57 @@ var ( cpy.CancunTime = utils.NewUint64(0) return &cpy } - - forkToChainConfig = map[upgradetest.Fork]*params.ChainConfig{ - upgradetest.NoUpgrades: params.TestLaunchConfig, - upgradetest.ApricotPhase1: params.TestApricotPhase1Config, - upgradetest.ApricotPhase2: params.TestApricotPhase2Config, - upgradetest.ApricotPhase3: params.TestApricotPhase3Config, - upgradetest.ApricotPhase4: params.TestApricotPhase4Config, - upgradetest.ApricotPhase5: params.TestApricotPhase5Config, - upgradetest.ApricotPhasePre6: params.TestApricotPhasePre6Config, - upgradetest.ApricotPhase6: params.TestApricotPhase6Config, - upgradetest.ApricotPhasePost6: params.TestApricotPhasePost6Config, - upgradetest.Banff: params.TestBanffChainConfig, - upgradetest.Cortina: params.TestCortinaChainConfig, - upgradetest.Durango: params.TestDurangoChainConfig, - upgradetest.Etna: params.TestEtnaChainConfig, - upgradetest.Fortuna: params.TestFortunaChainConfig, - } - - genesisJSONCancun = genesisJSON(activateCancun(params.TestChainConfig)) - - apricotRulesPhase0 = *params.GetRulesExtra(params.TestLaunchConfig.Rules(common.Big0, params.IsMergeTODO, 0)) - apricotRulesPhase1 = *params.GetRulesExtra(params.TestApricotPhase1Config.Rules(common.Big0, params.IsMergeTODO, 0)) - apricotRulesPhase2 = *params.GetRulesExtra(params.TestApricotPhase2Config.Rules(common.Big0, params.IsMergeTODO, 0)) - apricotRulesPhase3 = *params.GetRulesExtra(params.TestApricotPhase3Config.Rules(common.Big0, params.IsMergeTODO, 0)) - apricotRulesPhase5 = *params.GetRulesExtra(params.TestApricotPhase5Config.Rules(common.Big0, params.IsMergeTODO, 0)) - apricotRulesPhase6 = *params.GetRulesExtra(params.TestApricotPhase6Config.Rules(common.Big0, params.IsMergeTODO, 0)) - banffRules = *params.GetRulesExtra(params.TestBanffChainConfig.Rules(common.Big0, params.IsMergeTODO, 0)) ) -func init() { - for _, key := range testKeys { - testEthAddrs = append(testEthAddrs, key.EthAddress()) - testShortIDAddrs = append(testShortIDAddrs, key.Address()) - } -} - -func newPrefundedGenesis( - balance int, - addresses ...common.Address, -) *core.Genesis { - alloc := types.GenesisAlloc{} - for _, address := range addresses { - alloc[address] = types.Account{ - Balance: big.NewInt(int64(balance)), - } - } - - return &core.Genesis{ - Config: params.TestChainConfig, - Difficulty: big.NewInt(0), - Alloc: alloc, +func defaultExtensions() (*extension.Config, error) { + codecManager, err := message.NewCodec(message.BlockSyncSummary{}) + if err != nil { + return nil, err } + return &extension.Config{ + NetworkCodec: codecManager, + SyncSummaryProvider: &message.BlockSyncSummaryProvider{}, + SyncableParser: &message.BlockSyncSummaryParser{}, + ConsensusCallbacks: dummy.ConsensusCallbacks{ + OnFinalizeAndAssemble: nil, + OnExtraStateChange: nil, + }, + SyncExtender: nil, + BlockExtender: nil, + ExtraMempool: nil, + }, nil } -type testVMConfig struct { - isSyncing bool - fork *upgradetest.Fork - // If genesisJSON is empty, defaults to the genesis corresponding to the - // fork. - genesisJSON string - configJSON string - // the VM will start with UTXOs in the X-Chain Shared Memory containing - // AVAX based on the map - // The UTXOIDs are generated by using a hash of the address in the map such - // that the UTXOs will be generated deterministically. - utxos map[ids.ShortID]uint64 -} - -type testVM struct { - vm *VM - toEngine chan commonEng.Message - db *prefixdb.Database - atomicMemory *avalancheatomic.Memory - appSender *enginetest.Sender -} - -func newVM(t *testing.T, config testVMConfig) *testVM { - ctx := snowtest.Context(t, snowtest.CChainID) - fork := upgradetest.Latest - if config.fork != nil { - fork = *config.fork - } - ctx.NetworkUpgrades = upgradetest.GetConfig(fork) - - if len(config.genesisJSON) == 0 { - config.genesisJSON = genesisJSON(forkToChainConfig[fork]) - } - - baseDB := memdb.New() - - // initialize the atomic memory - atomicMemory := avalancheatomic.NewMemory(prefixdb.New([]byte{0}, baseDB)) - ctx.SharedMemory = atomicMemory.NewSharedMemory(ctx.ChainID) - - // NB: this lock is intentionally left locked when this function returns. - // The caller of this function is responsible for unlocking. - ctx.Lock.Lock() - - issuer := make(chan commonEng.Message, 1) - prefixedDB := prefixdb.New([]byte{1}, baseDB) - +// newDefaultTestVM returns a new instance of the VM with default extensions +// This should not be called if the VM is being extended +func newDefaultTestVM() *VM { vm := &VM{} - appSender := &enginetest.Sender{ - T: t, - CantSendAppGossip: true, - SendAppGossipF: func(context.Context, commonEng.SendConfig, []byte) error { return nil }, - } - require.NoError(t, vm.Initialize( - context.Background(), - ctx, - prefixedDB, - []byte(config.genesisJSON), - nil, - []byte(config.configJSON), - issuer, - nil, - appSender, - ), "error initializing vm") - - if !config.isSyncing { - require.NoError(t, vm.SetState(context.Background(), snow.Bootstrapping)) - require.NoError(t, vm.SetState(context.Background(), snow.NormalOp)) - } - - for addr, avaxAmount := range config.utxos { - txID, err := ids.ToID(hashing.ComputeHash256(addr.Bytes())) - if err != nil { - t.Fatalf("Failed to generate txID from addr: %s", err) - } - if _, err := addUTXO(atomicMemory, vm.ctx, txID, 0, vm.ctx.AVAXAssetID, avaxAmount, addr); err != nil { - t.Fatalf("Failed to add UTXO to shared memory: %s", err) - } - } - - return &testVM{ - vm: vm, - toEngine: issuer, - db: prefixedDB, - atomicMemory: atomicMemory, - appSender: appSender, - } -} - -// setupGenesis sets up the genesis -func setupGenesis( - t *testing.T, - fork upgradetest.Fork, -) (*snow.Context, - *prefixdb.Database, - []byte, - chan commonEng.Message, - *avalancheatomic.Memory, -) { - ctx := snowtest.Context(t, snowtest.CChainID) - ctx.NetworkUpgrades = upgradetest.GetConfig(fork) - - baseDB := memdb.New() - - // initialize the atomic memory - atomicMemory := avalancheatomic.NewMemory(prefixdb.New([]byte{0}, baseDB)) - ctx.SharedMemory = atomicMemory.NewSharedMemory(ctx.ChainID) - - // NB: this lock is intentionally left locked when this function returns. - // The caller of this function is responsible for unlocking. - ctx.Lock.Lock() - - issuer := make(chan commonEng.Message, 1) - prefixedDB := prefixdb.New([]byte{1}, baseDB) - genesisJSON := genesisJSON(forkToChainConfig[fork]) - return ctx, prefixedDB, []byte(genesisJSON), issuer, atomicMemory -} - -func addUTXO(sharedMemory *avalancheatomic.Memory, ctx *snow.Context, txID ids.ID, index uint32, assetID ids.ID, amount uint64, addr ids.ShortID) (*avax.UTXO, error) { - utxo := &avax.UTXO{ - UTXOID: avax.UTXOID{ - TxID: txID, - OutputIndex: index, - }, - Asset: avax.Asset{ID: assetID}, - Out: &secp256k1fx.TransferOutput{ - Amt: amount, - OutputOwners: secp256k1fx.OutputOwners{ - Threshold: 1, - Addrs: []ids.ShortID{addr}, - }, - }, - } - utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + exts, err := defaultExtensions() if err != nil { - return nil, err + panic(err) } - xChainSharedMemory := sharedMemory.NewSharedMemory(ctx.XChainID) - inputID := utxo.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ - Key: inputID[:], - Value: utxoBytes, - Traits: [][]byte{ - addr.Bytes(), - }, - }}}}); err != nil { - return nil, err + if err := vm.SetExtensionConfig(exts); err != nil { + panic(err) } - - return utxo, nil -} - -func TestVMConfig(t *testing.T) { - txFeeCap := float64(11) - enabledEthAPIs := []string{"debug"} - vm := newVM(t, testVMConfig{ - configJSON: fmt.Sprintf(`{"rpc-tx-fee-cap": %g,"eth-apis": [%q]}`, txFeeCap, enabledEthAPIs[0]), - }).vm - require.Equal(t, vm.config.RPCTxFeeCap, txFeeCap, "Tx Fee Cap should be set") - require.Equal(t, vm.config.EthAPIs(), enabledEthAPIs, "EnabledEthAPIs should be set") - require.NoError(t, vm.Shutdown(context.Background())) -} - -func TestVMConfigDefaults(t *testing.T) { - txFeeCap := float64(11) - enabledEthAPIs := []string{"debug"} - vm := newVM(t, testVMConfig{ - configJSON: fmt.Sprintf(`{"rpc-tx-fee-cap": %g,"eth-apis": [%q]}`, txFeeCap, enabledEthAPIs[0]), - }).vm - - var vmConfig config.Config - vmConfig.SetDefaults(defaultTxPoolConfig) - vmConfig.RPCTxFeeCap = txFeeCap - vmConfig.EnabledEthAPIs = enabledEthAPIs - require.Equal(t, vmConfig, vm.config, "VM Config should match default with overrides") - require.NoError(t, vm.Shutdown(context.Background())) + return vm } -func TestVMNilConfig(t *testing.T) { - vm := newVM(t, testVMConfig{}).vm - - // VM Config should match defaults if no config is passed in - var vmConfig config.Config - vmConfig.SetDefaults(defaultTxPoolConfig) - require.Equal(t, vmConfig, vm.config, "VM Config should match default config") - require.NoError(t, vm.Shutdown(context.Background())) +func setupDefaultTestVM(t *testing.T, cfg vmtest.TestVMConfig) (*VM, *vmtest.TestVMSuite) { + vm := newDefaultTestVM() + tvm := vmtest.SetupTestVM(t, vm, cfg) + return vm, tvm } func TestVMContinuousProfiler(t *testing.T) { profilerDir := t.TempDir() profilerFrequency := 500 * time.Millisecond - vm := newVM(t, testVMConfig{ - configJSON: fmt.Sprintf(`{"continuous-profiler-dir": %q,"continuous-profiler-frequency": "500ms"}`, profilerDir), - }).vm + configJSON := fmt.Sprintf(`{"continuous-profiler-dir": %q,"continuous-profiler-frequency": "500ms"}`, profilerDir) + fork := upgradetest.Latest + vm, _ := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: configJSON, + }) require.Equal(t, vm.config.ContinuousProfilerDir, profilerDir, "profiler dir should be set") require.Equal(t, vm.config.ContinuousProfilerFrequency.Duration, profilerFrequency, "profiler frequency should be set") @@ -413,10 +167,10 @@ func TestVMUpgrades(t *testing.T) { for _, test := range genesisTests { t.Run(test.fork.String(), func(t *testing.T) { require := require.New(t) + vm, _ := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &test.fork, + }) - vm := newVM(t, testVMConfig{ - fork: &test.fork, - }).vm defer func() { require.NoError(vm.Shutdown(context.Background())) }() @@ -438,189 +192,37 @@ func TestVMUpgrades(t *testing.T) { } } -func TestImportMissingUTXOs(t *testing.T) { - // make a VM with a shared memory that has an importable UTXO to build a block - importAmount := uint64(50000000) - fork := upgradetest.ApricotPhase2 - tvm1 := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - require.NoError(t, tvm1.vm.Shutdown(context.Background())) - }() - - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - require.NoError(t, err) - require.NoError(t, tvm1.vm.mempool.AddLocalTx(importTx)) - <-tvm1.toEngine - blk, err := tvm1.vm.BuildBlock(context.Background()) - require.NoError(t, err) - - // make another VM which is missing the UTXO in shared memory - vm2 := newVM(t, testVMConfig{ - fork: &fork, - }).vm - defer func() { - require.NoError(t, vm2.Shutdown(context.Background())) - }() - - vm2Blk, err := vm2.ParseBlock(context.Background(), blk.Bytes()) - require.NoError(t, err) - err = vm2Blk.Verify(context.Background()) - require.ErrorIs(t, err, errMissingUTXOs) - - // This should not result in a bad block since the missing UTXO should - // prevent InsertBlockManual from being called. - badBlocks, _ := vm2.blockChain.BadBlocks() - require.Len(t, badBlocks, 0) -} - -// Simple test to ensure we can issue an import transaction followed by an export transaction -// and they will be indexed correctly when accepted. -func TestIssueAtomicTxs(t *testing.T) { - importAmount := uint64(50000000) - fork := upgradetest.ApricotPhase2 - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - if lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()); err != nil { - t.Fatal(err) - } else if lastAcceptedID != blk.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) - } - tvm.vm.blockChain.DrainAcceptorQueue() - filterAPI := filters.NewFilterAPI(filters.NewFilterSystem(tvm.vm.eth.APIBackend, filters.Config{ - Timeout: 5 * time.Minute, - })) - blockHash := common.Hash(blk.ID()) - logs, err := filterAPI.GetLogs(context.Background(), filters.FilterCriteria{ - BlockHash: &blockHash, - }) - if err != nil { - t.Fatal(err) - } - if len(logs) != 0 { - t.Fatalf("Expected log length to be 0, but found %d", len(logs)) - } - if logs == nil { - t.Fatal("Expected logs to be non-nil") - } - - exportTx, err := tvm.vm.newExportTx(tvm.vm.ctx.AVAXAssetID, importAmount-(2*ap0.AtomicTxFee), tvm.vm.ctx.XChainID, testShortIDAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(exportTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk2, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk2.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := blk2.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - if lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()); err != nil { - t.Fatal(err) - } else if lastAcceptedID != blk2.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk2.ID(), lastAcceptedID) - } - - // Check that both atomic transactions were indexed as expected. - indexedImportTx, status, height, err := tvm.vm.getAtomicTx(importTx.ID()) - assert.NoError(t, err) - assert.Equal(t, atomic.Accepted, status) - assert.Equal(t, uint64(1), height, "expected height of indexed import tx to be 1") - assert.Equal(t, indexedImportTx.ID(), importTx.ID(), "expected ID of indexed import tx to match original txID") - - indexedExportTx, status, height, err := tvm.vm.getAtomicTx(exportTx.ID()) - assert.NoError(t, err) - assert.Equal(t, atomic.Accepted, status) - assert.Equal(t, uint64(2), height, "expected height of indexed export tx to be 2") - assert.Equal(t, indexedExportTx.ID(), exportTx.ID(), "expected ID of indexed import tx to match original txID") -} - func TestBuildEthTxBlock(t *testing.T) { - importAmount := uint64(20000000) fork := upgradetest.ApricotPhase2 - tvm := newVM(t, testVMConfig{ - fork: &fork, - configJSON: `{"pruning-enabled":true}`, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, }) + defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) + vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm.toEngine + <-tvm.ToEngine - blk1, err := tvm.vm.BuildBlock(context.Background()) + blk1, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -629,7 +231,7 @@ func TestBuildEthTxBlock(t *testing.T) { t.Fatal(err) } - if err := tvm.vm.SetPreference(context.Background(), blk1.ID()); err != nil { + if err := vm.SetPreference(context.Background(), blk1.ID()); err != nil { t.Fatal(err) } @@ -644,23 +246,23 @@ func TestBuildEthTxBlock(t *testing.T) { txs := make([]*types.Transaction, 10) for i := 0; i < 10; i++ { - tx := types.NewTransaction(uint64(i), testEthAddrs[0], big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), testKeys[0].ToECDSA()) + tx := types.NewTransaction(uint64(i), vmtest.TestEthAddrs[0], big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainID), vmtest.TestKeys[1].ToECDSA()) if err != nil { t.Fatal(err) } txs[i] = signedTx } - errs := tvm.vm.txPool.AddRemotesSync(txs) + errs = vm.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { t.Fatalf("Failed to add tx at index %d: %s", i, err) } } - <-tvm.toEngine + <-tvm.ToEngine - blk2, err := tvm.vm.BuildBlock(context.Background()) + blk2, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -678,7 +280,7 @@ func TestBuildEthTxBlock(t *testing.T) { t.Fatalf("Expected new block to match") } - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) + lastAcceptedID, err := vm.LastAccepted(context.Background()) if err != nil { t.Fatal(err) } @@ -686,20 +288,20 @@ func TestBuildEthTxBlock(t *testing.T) { t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk2.ID(), lastAcceptedID) } - ethBlk1 := blk1.(*chain.BlockWrapper).Block.(*Block).ethBlock - if ethBlk1Root := ethBlk1.Root(); !tvm.vm.blockChain.HasState(ethBlk1Root) { + ethBlk1 := blk1.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock + if ethBlk1Root := ethBlk1.Root(); !vm.blockChain.HasState(ethBlk1Root) { t.Fatalf("Expected blk1 state root to not yet be pruned after blk2 was accepted because of tip buffer") } // Clear the cache and ensure that GetBlock returns internal blocks with the correct status - tvm.vm.State.Flush() - blk2Refreshed, err := tvm.vm.GetBlockInternal(context.Background(), blk2.ID()) + vm.State.Flush() + blk2Refreshed, err := vm.GetBlockInternal(context.Background(), blk2.ID()) if err != nil { t.Fatal(err) } blk1RefreshedID := blk2Refreshed.Parent() - blk1Refreshed, err := tvm.vm.GetBlockInternal(context.Background(), blk1RefreshedID) + blk1Refreshed, err := vm.GetBlockInternal(context.Background(), blk1RefreshedID) if err != nil { t.Fatal(err) } @@ -708,17 +310,17 @@ func TestBuildEthTxBlock(t *testing.T) { t.Fatalf("Found unexpected blkID for parent of blk2") } - restartedVM := &VM{} + restartedVM := newDefaultTestVM() newCTX := snowtest.Context(t, snowtest.CChainID) newCTX.NetworkUpgrades = upgradetest.GetConfig(fork) if err := restartedVM.Initialize( context.Background(), newCTX, - tvm.db, - []byte(genesisJSON(forkToChainConfig[fork])), + tvm.DB, + []byte(vmtest.GenesisJSON(vmtest.ForkToChainConfig[fork])), []byte(""), []byte(`{"pruning-enabled":true}`), - tvm.toEngine, + tvm.ToEngine, []*commonEng.Fx{}, nil, ); err != nil { @@ -731,404 +333,320 @@ func TestBuildEthTxBlock(t *testing.T) { } // State root should be committed when accepted tip on shutdown - ethBlk2 := blk2.(*chain.BlockWrapper).Block.(*Block).ethBlock + ethBlk2 := blk2.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock if ethBlk2Root := ethBlk2.Root(); !restartedVM.blockChain.HasState(ethBlk2Root) { t.Fatalf("Expected blk2 state root to not be pruned after shutdown (last accepted tip should be committed)") } } -func testConflictingImportTxs(t *testing.T, fork upgradetest.Fork) { - importAmount := uint64(10000000) - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - testShortIDAddrs[1]: importAmount, - testShortIDAddrs[2]: importAmount, - }, - }) +// Regression test to ensure that after accepting block A +// then calling SetPreference on block B (when it becomes preferred) +// and the head of a longer chain (block D) does not corrupt the +// canonical chain. +// +// A +// / \ +// B C +// | +// D +func TestSetPreferenceRace(t *testing.T) { + // Create two VMs which will agree on block A and then + // build the two distinct preferred chains above + fork := upgradetest.NoUpgrades + conf := vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, + } + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) + defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm1.Shutdown(context.Background()); err != nil { t.Fatal(err) } - }() - importTxs := make([]*atomic.Tx, 0, 3) - conflictTxs := make([]*atomic.Tx, 0, 3) - for i, key := range testKeys { - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[i], initialBaseFee, []*secp256k1.PrivateKey{key}) - if err != nil { + if err := vm2.Shutdown(context.Background()); err != nil { t.Fatal(err) } - importTxs = append(importTxs, importTx) + }() - conflictAddr := testEthAddrs[(i+1)%len(testEthAddrs)] - conflictTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, conflictAddr, initialBaseFee, []*secp256k1.PrivateKey{key}) - if err != nil { - t.Fatal(err) - } - conflictTxs = append(conflictTxs, conflictTx) - } + newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - expectedParentBlkID, err := tvm.vm.LastAccepted(context.Background()) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - for _, tx := range importTxs[:2] { - if err := tvm.vm.mempool.AddLocalTx(tx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - tvm.vm.clock.Set(tvm.vm.clock.Time().Add(2 * time.Second)) - blk, err := tvm.vm.BuildBlock(context.Background()) + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) + t.Fatalf("Failed to add tx at index %d: %s", i, err) } + } - if parentID := blk.Parent(); parentID != expectedParentBlkID { - t.Fatalf("Expected parent to have blockID %s, but found %s", expectedParentBlkID, parentID) - } + <-tvm1.ToEngine - expectedParentBlkID = blk.ID() - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) - } + vm1BlkA, err := vm1.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build block with transaction: %s", err) } - // Check that for each conflict tx (whose conflict is in the chain ancestry) - // the VM returns an error when it attempts to issue the conflict into the mempool - // and when it attempts to build a block with the conflict force added to the mempool. - for i, tx := range conflictTxs[:2] { - if err := tvm.vm.mempool.AddLocalTx(tx); err == nil { - t.Fatal("Expected issueTx to fail due to conflicting transaction") - } - // Force issue transaction directly to the mempool - if err := tvm.vm.mempool.ForceAddTx(tx); err != nil { - t.Fatal(err) - } - <-tvm.toEngine - - tvm.vm.clock.Set(tvm.vm.clock.Time().Add(2 * time.Second)) - _, err = tvm.vm.BuildBlock(context.Background()) - // The new block is verified in BuildBlock, so - // BuildBlock should fail due to an attempt to - // double spend an atomic UTXO. - if err == nil { - t.Fatalf("Block verification should have failed in BuildBlock %d due to double spending atomic UTXO", i) - } + if err := vm1BlkA.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - // Generate one more valid block so that we can copy the header to create an invalid block - // with modified extra data. This new block will be invalid for more than one reason (invalid merkle root) - // so we check to make sure that the expected error is returned from block verification. - if err := tvm.vm.mempool.AddLocalTx(importTxs[2]); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - <-tvm.toEngine - tvm.vm.clock.Set(tvm.vm.clock.Time().Add(2 * time.Second)) - validBlock, err := tvm.vm.BuildBlock(context.Background()) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) if err != nil { - t.Fatal(err) + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - - if err := validBlock.Verify(context.Background()); err != nil { + if err := vm2BlkA.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM2: %s", err) + } + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } - validEthBlock := validBlock.(*chain.BlockWrapper).Block.(*Block).ethBlock + if err := vm1BlkA.Accept(context.Background()); err != nil { + t.Fatalf("VM1 failed to accept block: %s", err) + } + if err := vm2BlkA.Accept(context.Background()); err != nil { + t.Fatalf("VM2 failed to accept block: %s", err) + } - rules := tvm.vm.currentRules() - var extraData []byte - switch { - case rules.IsApricotPhase5: - extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{conflictTxs[1]}) - default: - extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, conflictTxs[1]) + newHead := <-newTxPoolHeadChan1 + if newHead.Head.Hash() != common.Hash(vm1BlkA.ID()) { + t.Fatalf("Expected new block to match") } - if err != nil { - t.Fatal(err) + newHead = <-newTxPoolHeadChan2 + if newHead.Head.Hash() != common.Hash(vm2BlkA.ID()) { + t.Fatalf("Expected new block to match") } - conflictingAtomicTxBlock := customtypes.NewBlockWithExtData( - types.CopyHeader(validEthBlock.Header()), - nil, - nil, - nil, - new(trie.Trie), - extraData, - true, - ) + // Create list of 10 successive transactions to build block A on vm1 + // and to be split into two separate blocks on VM2 + txs := make([]*types.Transaction, 10) + for i := 0; i < 10; i++ { + tx := types.NewTransaction(uint64(i), vmtest.TestEthAddrs[1], big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), vmtest.TestKeys[1].ToECDSA()) + if err != nil { + t.Fatal(err) + } + txs[i] = signedTx + } - blockBytes, err := rlp.EncodeToBytes(conflictingAtomicTxBlock) - if err != nil { - t.Fatal(err) + // Add the remote transactions, build the block, and set VM1's preference for block A + errs = vm1.txPool.AddRemotesSync(txs) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) + } } - parsedBlock, err := tvm.vm.ParseBlock(context.Background(), blockBytes) + <-tvm1.ToEngine + + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } - if err := parsedBlock.Verify(context.Background()); !errors.Is(err, atomic.ErrConflictingAtomicInputs) { - t.Fatalf("Expected to fail with err: %s, but found err: %s", atomic.ErrConflictingAtomicInputs, err) - } - - if !rules.IsApricotPhase5 { - return + if err := vm1BlkB.Verify(context.Background()); err != nil { + t.Fatal(err) } - extraData, err = atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{importTxs[2], conflictTxs[2]}) - if err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } - header := types.CopyHeader(validEthBlock.Header()) - headerExtra := customtypes.GetHeaderExtra(header) - headerExtra.ExtDataGasUsed.Mul(common.Big2, headerExtra.ExtDataGasUsed) - - internalConflictBlock := customtypes.NewBlockWithExtData( - header, - nil, - nil, - nil, - new(trie.Trie), - extraData, - true, - ) - - blockBytes, err = rlp.EncodeToBytes(internalConflictBlock) - if err != nil { - t.Fatal(err) + // Split the transactions over two blocks, and set VM2's preference to them in sequence + // after building each block + // Block C + errs = vm2.txPool.AddRemotesSync(txs[0:5]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) + } } - parsedBlock, err = tvm.vm.ParseBlock(context.Background(), blockBytes) + <-tvm2.ToEngine + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) + t.Fatalf("Failed to build BlkC on VM2: %s", err) } - if err := parsedBlock.Verify(context.Background()); !errors.Is(err, atomic.ErrConflictingAtomicInputs) { - t.Fatalf("Expected to fail with err: %s, but found err: %s", atomic.ErrConflictingAtomicInputs, err) + if err := vm2BlkC.Verify(context.Background()); err != nil { + t.Fatalf("BlkC failed verification on VM2: %s", err) } -} - -func TestReissueAtomicTxHigherGasPrice(t *testing.T) { - kc := secp256k1fx.NewKeychain(testKeys...) - - for name, issueTxs := range map[string]func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, discarded []*atomic.Tx){ - "single UTXO override": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { - utxo, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, testShortIDAddrs[0]) - if err != nil { - t.Fatal(err) - } - tx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, kc, []*avax.UTXO{utxo}) - if err != nil { - t.Fatal(err) - } - tx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(common.Big2, initialBaseFee), kc, []*avax.UTXO{utxo}) - if err != nil { - t.Fatal(err) - } - if err := vm.mempool.AddLocalTx(tx1); err != nil { - t.Fatal(err) - } - if err := vm.mempool.AddLocalTx(tx2); err != nil { - t.Fatal(err) - } - - return []*atomic.Tx{tx2}, []*atomic.Tx{tx1} - }, - "one of two UTXOs overrides": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { - utxo1, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, testShortIDAddrs[0]) - if err != nil { - t.Fatal(err) - } - utxo2, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, testShortIDAddrs[0]) - if err != nil { - t.Fatal(err) - } - tx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, kc, []*avax.UTXO{utxo1, utxo2}) - if err != nil { - t.Fatal(err) - } - tx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(common.Big2, initialBaseFee), kc, []*avax.UTXO{utxo1}) - if err != nil { - t.Fatal(err) - } - - if err := vm.mempool.AddLocalTx(tx1); err != nil { - t.Fatal(err) - } - if err := vm.mempool.AddLocalTx(tx2); err != nil { - t.Fatal(err) - } + if err := vm2.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { + t.Fatal(err) + } - return []*atomic.Tx{tx2}, []*atomic.Tx{tx1} - }, - "hola": func(t *testing.T, vm *VM, sharedMemory *avalancheatomic.Memory) (issued []*atomic.Tx, evicted []*atomic.Tx) { - utxo1, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, testShortIDAddrs[0]) - if err != nil { - t.Fatal(err) - } - utxo2, err := addUTXO(sharedMemory, vm.ctx, ids.GenerateTestID(), 0, vm.ctx.AVAXAssetID, units.Avax, testShortIDAddrs[0]) - if err != nil { - t.Fatal(err) - } + newHead = <-newTxPoolHeadChan2 + if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { + t.Fatalf("Expected new block to match") + } - importTx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, kc, []*avax.UTXO{utxo1}) - if err != nil { - t.Fatal(err) - } + // Block D + errs = vm2.txPool.AddRemotesSync(txs[5:10]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) + } + } - importTx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(big.NewInt(3), initialBaseFee), kc, []*avax.UTXO{utxo2}) - if err != nil { - t.Fatal(err) - } + <-tvm2.ToEngine + vm2BlkD, err := vm2.BuildBlock(context.Background()) + if err != nil { + t.Fatalf("Failed to build BlkD on VM2: %s", err) + } - reissuanceTx1, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(big.NewInt(2), initialBaseFee), kc, []*avax.UTXO{utxo1, utxo2}) - if err != nil { - t.Fatal(err) - } - if err := vm.mempool.AddLocalTx(importTx1); err != nil { - t.Fatal(err) - } + if err := vm2BlkD.Verify(context.Background()); err != nil { + t.Fatalf("BlkD failed verification on VM2: %s", err) + } - if err := vm.mempool.AddLocalTx(importTx2); err != nil { - t.Fatal(err) - } + if err := vm2.SetPreference(context.Background(), vm2BlkD.ID()); err != nil { + t.Fatal(err) + } - if err := vm.mempool.AddLocalTx(reissuanceTx1); !errors.Is(err, atomictxpool.ErrConflictingAtomicTx) { - t.Fatalf("Expected to fail with err: %s, but found err: %s", atomictxpool.ErrConflictingAtomicTx, err) - } + // VM1 receives blkC and blkD from VM1 + // and happens to call SetPreference on blkD without ever calling SetPreference + // on blkC + // Here we parse them in reverse order to simulate receiving a chain from the tip + // back to the last accepted block as would typically be the case in the consensus + // engine + vm1BlkD, err := vm1.ParseBlock(context.Background(), vm2BlkD.Bytes()) + if err != nil { + t.Fatalf("VM1 errored parsing blkD: %s", err) + } + vm1BlkC, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()) + if err != nil { + t.Fatalf("VM1 errored parsing blkC: %s", err) + } - assert.True(t, vm.mempool.Has(importTx1.ID())) - assert.True(t, vm.mempool.Has(importTx2.ID())) - assert.False(t, vm.mempool.Has(reissuanceTx1.ID())) + // The blocks must be verified in order. This invariant is maintained + // in the consensus engine. + if err := vm1BlkC.Verify(context.Background()); err != nil { + t.Fatalf("VM1 BlkC failed verification: %s", err) + } + if err := vm1BlkD.Verify(context.Background()); err != nil { + t.Fatalf("VM1 BlkD failed verification: %s", err) + } - reissuanceTx2, err := atomic.NewImportTx(vm.ctx, vm.currentRules(), vm.clock.Unix(), vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(big.NewInt(4), initialBaseFee), kc, []*avax.UTXO{utxo1, utxo2}) - if err != nil { - t.Fatal(err) - } - if err := vm.mempool.AddLocalTx(reissuanceTx2); err != nil { - t.Fatal(err) - } + // Set VM1's preference to blockD, skipping blockC + if err := vm1.SetPreference(context.Background(), vm1BlkD.ID()); err != nil { + t.Fatal(err) + } - return []*atomic.Tx{reissuanceTx2}, []*atomic.Tx{importTx1, importTx2} - }, - } { - t.Run(name, func(t *testing.T) { - fork := upgradetest.ApricotPhase5 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - issuedTxs, evictedTxs := issueTxs(t, tvm.vm, tvm.atomicMemory) + // Accept the longer chain on both VMs and ensure there are no errors + // VM1 Accepts the blocks in order + if err := vm1BlkC.Accept(context.Background()); err != nil { + t.Fatalf("VM1 BlkC failed on accept: %s", err) + } + if err := vm1BlkD.Accept(context.Background()); err != nil { + t.Fatalf("VM1 BlkC failed on accept: %s", err) + } - for i, tx := range issuedTxs { - _, issued := tvm.vm.mempool.GetPendingTx(tx.ID()) - assert.True(t, issued, "expected issued tx at index %d to be issued", i) - } + // VM2 Accepts the blocks in order + if err := vm2BlkC.Accept(context.Background()); err != nil { + t.Fatalf("VM2 BlkC failed on accept: %s", err) + } + if err := vm2BlkD.Accept(context.Background()); err != nil { + t.Fatalf("VM2 BlkC failed on accept: %s", err) + } - for i, tx := range evictedTxs { - _, discarded, _ := tvm.vm.mempool.GetTx(tx.ID()) - assert.True(t, discarded, "expected discarded tx at index %d to be discarded", i) - } - }) + log.Info("Validating canonical chain") + // Verify the Canonical Chain for Both VMs + if err := vm2.blockChain.ValidateCanonicalChain(); err != nil { + t.Fatalf("VM2 failed canonical chain verification due to: %s", err) } -} -func TestConflictingImportTxsAcrossBlocks(t *testing.T) { - for _, fork := range []upgradetest.Fork{ - upgradetest.ApricotPhase1, - upgradetest.ApricotPhase2, - upgradetest.ApricotPhase3, - upgradetest.ApricotPhase4, - upgradetest.ApricotPhase5, - } { - t.Run(fork.String(), func(t *testing.T) { - testConflictingImportTxs(t, fork) - }) + if err := vm1.blockChain.ValidateCanonicalChain(); err != nil { + t.Fatalf("VM1 failed canonical chain verification due to: %s", err) } } -// Regression test to ensure that after accepting block A -// then calling SetPreference on block B (when it becomes preferred) -// and the head of a longer chain (block D) does not corrupt the -// canonical chain. +// Regression test to ensure that a VM that accepts block A and B +// will not attempt to orphan either when verifying blocks C and D +// from another VM (which have a common ancestor under the finalized +// frontier). // // A // / \ // B C -// | -// D -func TestSetPreferenceRace(t *testing.T) { - // Create two VMs which will agree on block A and then - // build the two distinct preferred chains above - importAmount := uint64(1000000000) +// +// verifies block B and C, then Accepts block B. Then we test to ensure +// that the VM defends against any attempt to set the preference or to +// accept block C, which should be an orphaned block at this point and +// get rejected. +func TestReorgProtection(t *testing.T) { fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - configJSON: `{"pruning-enabled":true}`, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + conf := vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { + if err := vm1.Shutdown(context.Background()); err != nil { t.Fatal(err) } - if err := tvm2.vm.Shutdown(context.Background()); err != nil { + if err := vm2.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, testEthAddrs[1], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] + + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkA, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } if err := vm1BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) if err != nil { t.Fatalf("Unexpected error parsing block from vm2: %s", err) } if err := vm2BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } @@ -1152,27 +670,25 @@ func TestSetPreferenceRace(t *testing.T) { // and to be split into two separate blocks on VM2 txs := make([]*types.Transaction, 10) for i := 0; i < 10; i++ { - tx := types.NewTransaction(uint64(i), testEthAddrs[1], big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), testKeys[1].ToECDSA()) + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), key) if err != nil { t.Fatal(err) } txs[i] = signedTx } - var errs []error - // Add the remote transactions, build the block, and set VM1's preference for block A - errs = tvm1.vm.txPool.AddRemotesSync(txs) + errs = vm1.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -1181,410 +697,320 @@ func TestSetPreferenceRace(t *testing.T) { t.Fatal(err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } // Split the transactions over two blocks, and set VM2's preference to them in sequence // after building each block // Block C - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) + errs = vm2.txPool.AddRemotesSync(txs[0:5]) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } } - <-tvm2.toEngine - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) + <-tvm2.ToEngine + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { t.Fatalf("Failed to build BlkC on VM2: %s", err) } if err := vm2BlkC.Verify(context.Background()); err != nil { - t.Fatalf("BlkC failed verification on VM2: %s", err) + t.Fatalf("Block failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { - t.Fatal(err) + vm1BlkC, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()) + if err != nil { + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - newHead = <-newTxPoolHeadChan2 - if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { - t.Fatalf("Expected new block to match") + if err := vm1BlkC.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - // Block D - errs = tvm2.vm.txPool.AddRemotesSync(txs[5:10]) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } + // Accept B, such that block C should get Rejected. + if err := vm1BlkB.Accept(context.Background()); err != nil { + t.Fatalf("VM1 failed to accept block: %s", err) } - <-tvm2.toEngine - vm2BlkD, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkD on VM2: %s", err) + // The below (setting preference blocks that have a common ancestor + // with the preferred chain lower than the last finalized block) + // should NEVER happen. However, the VM defends against this + // just in case. + if err := vm1.SetPreference(context.Background(), vm1BlkC.ID()); !strings.Contains(err.Error(), "cannot orphan finalized block") { + t.Fatalf("Unexpected error when setting preference that would trigger reorg: %s", err) } - if err := vm2BlkD.Verify(context.Background()); err != nil { - t.Fatalf("BlkD failed verification on VM2: %s", err) + if err := vm1BlkC.Accept(context.Background()); !strings.Contains(err.Error(), "expected accepted block to have parent") { + t.Fatalf("Unexpected error when setting block at finalized height: %s", err) } +} - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkD.ID()); err != nil { - t.Fatal(err) +// Regression test to ensure that a VM that accepts block C while preferring +// block B will trigger a reorg. +// +// A +// / \ +// B C +func TestNonCanonicalAccept(t *testing.T) { + fork := upgradetest.NoUpgrades + conf := vmtest.TestVMConfig{ + Fork: &fork, } + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) - // VM1 receives blkC and blkD from VM1 - // and happens to call SetPreference on blkD without ever calling SetPreference - // on blkC - // Here we parse them in reverse order to simulate receiving a chain from the tip - // back to the last accepted block as would typically be the case in the consensus - // engine - vm1BlkD, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkD.Bytes()) + defer func() { + if err := vm1.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + + if err := vm2.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) + + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] + + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { - t.Fatalf("VM1 errored parsing blkD: %s", err) + t.Fatal(err) + } + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - vm1BlkC, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()) + + <-tvm1.ToEngine + + vm1BlkA, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("VM1 errored parsing blkC: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } - // The blocks must be verified in order. This invariant is maintained - // in the consensus engine. - if err := vm1BlkC.Verify(context.Background()); err != nil { - t.Fatalf("VM1 BlkC failed verification: %s", err) + if err := vm1BlkA.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - if err := vm1BlkD.Verify(context.Background()); err != nil { - t.Fatalf("VM1 BlkD failed verification: %s", err) + + if _, err := vm1.GetBlockIDAtHeight(context.Background(), vm1BlkA.Height()); err != database.ErrNotFound { + t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) } - // Set VM1's preference to blockD, skipping blockC - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkD.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - // Accept the longer chain on both VMs and ensure there are no errors - // VM1 Accepts the blocks in order - if err := vm1BlkC.Accept(context.Background()); err != nil { - t.Fatalf("VM1 BlkC failed on accept: %s", err) - } - if err := vm1BlkD.Accept(context.Background()); err != nil { - t.Fatalf("VM1 BlkC failed on accept: %s", err) - } - - // VM2 Accepts the blocks in order - if err := vm2BlkC.Accept(context.Background()); err != nil { - t.Fatalf("VM2 BlkC failed on accept: %s", err) - } - if err := vm2BlkD.Accept(context.Background()); err != nil { - t.Fatalf("VM2 BlkC failed on accept: %s", err) - } - - log.Info("Validating canonical chain") - // Verify the Canonical Chain for Both VMs - if err := tvm2.vm.blockChain.ValidateCanonicalChain(); err != nil { - t.Fatalf("VM2 failed canonical chain verification due to: %s", err) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) + if err != nil { + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - - if err := tvm1.vm.blockChain.ValidateCanonicalChain(); err != nil { - t.Fatalf("VM1 failed canonical chain verification due to: %s", err) + if err := vm2BlkA.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM2: %s", err) } -} - -func TestConflictingTransitiveAncestryWithGap(t *testing.T) { - key := utilstest.NewKey(t) - - key0 := testKeys[0] - addr0 := key0.Address() - - key1 := testKeys[1] - addr1 := key1.Address() - - importAmount := uint64(1000000000) - - fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - addr0: importAmount, - addr1: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - - importTx0A, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, key.Address, initialBaseFee, []*secp256k1.PrivateKey{key0}) - if err != nil { - t.Fatal(err) + if _, err := vm2.GetBlockIDAtHeight(context.Background(), vm2BlkA.Height()); err != database.ErrNotFound { + t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) } - // Create a conflicting transaction - importTx0B, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[2], initialBaseFee, []*secp256k1.PrivateKey{key0}) - if err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(importTx0A); err != nil { - t.Fatalf("Failed to issue importTx0A: %s", err) + if err := vm1BlkA.Accept(context.Background()); err != nil { + t.Fatalf("VM1 failed to accept block: %s", err) } - - <-tvm.toEngine - - blk0, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + if blkID, err := vm1.GetBlockIDAtHeight(context.Background(), vm1BlkA.Height()); err != nil { + t.Fatalf("Height lookuped failed on accepted block: %s", err) + } else if blkID != vm1BlkA.ID() { + t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) } - - if err := blk0.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification: %s", err) + if err := vm2BlkA.Accept(context.Background()); err != nil { + t.Fatalf("VM2 failed to accept block: %s", err) } - - if err := tvm.vm.SetPreference(context.Background(), blk0.ID()); err != nil { - t.Fatal(err) + if blkID, err := vm2.GetBlockIDAtHeight(context.Background(), vm2BlkA.Height()); err != nil { + t.Fatalf("Height lookuped failed on accepted block: %s", err) + } else if blkID != vm2BlkA.ID() { + t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) } - newHead := <-newTxPoolHeadChan - if newHead.Head.Hash() != common.Hash(blk0.ID()) { + newHead := <-newTxPoolHeadChan1 + if newHead.Head.Hash() != common.Hash(vm1BlkA.ID()) { + t.Fatalf("Expected new block to match") + } + newHead = <-newTxPoolHeadChan2 + if newHead.Head.Hash() != common.Hash(vm2BlkA.ID()) { t.Fatalf("Expected new block to match") } - tx := types.NewTransaction(0, key.Address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key.PrivateKey) - if err != nil { - t.Fatal(err) + // Create list of 10 successive transactions to build block A on vm1 + // and to be split into two separate blocks on VM2 + txs := make([]*types.Transaction, 10) + for i := 0; i < 10; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), key) + if err != nil { + t.Fatal(err) + } + txs[i] = signedTx } // Add the remote transactions, build the block, and set VM1's preference for block A - errs := tvm.vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + errs = vm1.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) } } - <-tvm.toEngine + <-tvm1.ToEngine - blk1, err := tvm.vm.BuildBlock(context.Background()) + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build blk1: %s", err) - } - - if err := blk1.Verify(context.Background()); err != nil { - t.Fatalf("blk1 failed verification due to %s", err) - } - - if err := tvm.vm.SetPreference(context.Background(), blk1.ID()); err != nil { t.Fatal(err) } - importTx1, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, key.Address, initialBaseFee, []*secp256k1.PrivateKey{key1}) - if err != nil { - t.Fatalf("Failed to issue importTx1 due to: %s", err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx1); err != nil { + if err := vm1BlkB.Verify(context.Background()); err != nil { t.Fatal(err) } - <-tvm.toEngine - - blk2, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) - } - - if err := blk2.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification: %s", err) + if _, err := vm1.GetBlockIDAtHeight(context.Background(), vm1BlkB.Height()); err != database.ErrNotFound { + t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) } - if err := tvm.vm.SetPreference(context.Background(), blk2.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(importTx0B); err == nil { - t.Fatalf("Should not have been able to issue import tx with conflict") - } - // Force issue transaction directly into the mempool - if err := tvm.vm.mempool.ForceAddTx(importTx0B); err != nil { - t.Fatal(err) - } - <-tvm.toEngine + vm1.eth.APIBackend.SetAllowUnfinalizedQueries(true) - _, err = tvm.vm.BuildBlock(context.Background()) - if err == nil { - t.Fatal("Shouldn't have been able to build an invalid block") + blkBHeight := vm1BlkB.Height() + blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) } -} -func TestBonusBlocksTxs(t *testing.T) { - fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) + errs = vm2.txPool.AddRemotesSync(txs[0:5]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } - }() - - importAmount := uint64(10000000) - utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} - - utxo := &avax.UTXO{ - UTXOID: utxoID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, - Out: &secp256k1fx.TransferOutput{ - Amt: importAmount, - OutputOwners: secp256k1fx.OutputOwners{ - Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, - }, - }, - } - utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) - if err != nil { - t.Fatal(err) - } - - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) - inputID := utxo.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ - Key: inputID[:], - Value: utxoBytes, - Traits: [][]byte{ - testKeys[0].Address().Bytes(), - }, - }}}}); err != nil { - t.Fatal(err) } - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + <-tvm2.ToEngine + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + t.Fatalf("Failed to build BlkC on VM2: %s", err) } - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) + vm1BlkC, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()) if err != nil { - t.Fatal(err) + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - // Make [blk] a bonus block. - tvm.vm.atomicBackend.AddBonusBlock(blk.Height(), blk.ID()) - - // Remove the UTXOs from shared memory, so that non-bonus blocks will fail verification - if err := tvm.vm.ctx.SharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.XChainID: {RemoveRequests: [][]byte{inputID[:]}}}); err != nil { - t.Fatal(err) + if err := vm1BlkC.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) + if _, err := vm1.GetBlockIDAtHeight(context.Background(), vm1BlkC.Height()); err != database.ErrNotFound { + t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) + if err := vm1BlkC.Accept(context.Background()); err != nil { + t.Fatalf("VM1 failed to accept block: %s", err) } - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) + if blkID, err := vm1.GetBlockIDAtHeight(context.Background(), vm1BlkC.Height()); err != nil { + t.Fatalf("Height lookuped failed on accepted block: %s", err) + } else if blkID != vm1BlkC.ID() { + t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) } - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) - if err != nil { - t.Fatal(err) - } - if lastAcceptedID != blk.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) } } -// Regression test to ensure that a VM that accepts block A and B -// will not attempt to orphan either when verifying blocks C and D -// from another VM (which have a common ancestor under the finalized -// frontier). +// Regression test to ensure that a VM that verifies block B, C, then +// D (preferring block B) does not trigger a reorg through the re-verification +// of block C or D. // // A // / \ // B C -// -// verifies block B and C, then Accepts block B. Then we test to ensure -// that the VM defends against any attempt to set the preference or to -// accept block C, which should be an orphaned block at this point and -// get rejected. -func TestReorgProtection(t *testing.T) { - importAmount := uint64(1000000000) +// | +// D +func TestStickyPreference(t *testing.T) { fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - configJSON: `{"pruning-enabled":false}`, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + conf := vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) + defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { + if err := vm1.Shutdown(context.Background()); err != nil { t.Fatal(err) } - if err := tvm2.vm.Shutdown(context.Background()); err != nil { + if err := vm2.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkA, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } if err := vm1BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) if err != nil { t.Fatalf("Unexpected error parsing block from vm2: %s", err) } if err := vm2BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } @@ -1609,26 +1035,24 @@ func TestReorgProtection(t *testing.T) { txs := make([]*types.Transaction, 10) for i := 0; i < 10; i++ { tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), key) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), key) if err != nil { t.Fatal(err) } txs[i] = signedTx } - var errs []error - // Add the remote transactions, build the block, and set VM1's preference for block A - errs = tvm1.vm.txPool.AddRemotesSync(txs) + errs = vm1.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -1637,321 +1061,214 @@ func TestReorgProtection(t *testing.T) { t.Fatal(err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } - // Split the transactions over two blocks, and set VM2's preference to them in sequence - // after building each block - // Block C - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) + vm1.eth.APIBackend.SetAllowUnfinalizedQueries(true) + + blkBHeight := vm1BlkB.Height() + blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) + } + + errs = vm2.txPool.AddRemotesSync(txs[0:5]) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } } - <-tvm2.toEngine - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) + <-tvm2.ToEngine + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { t.Fatalf("Failed to build BlkC on VM2: %s", err) } if err := vm2BlkC.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM2: %s", err) - } - - vm1BlkC, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - - if err := vm1BlkC.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - - // Accept B, such that block C should get Rejected. - if err := vm1BlkB.Accept(context.Background()); err != nil { - t.Fatalf("VM1 failed to accept block: %s", err) - } - - // The below (setting preference blocks that have a common ancestor - // with the preferred chain lower than the last finalized block) - // should NEVER happen. However, the VM defends against this - // just in case. - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkC.ID()); !strings.Contains(err.Error(), "cannot orphan finalized block") { - t.Fatalf("Unexpected error when setting preference that would trigger reorg: %s", err) + t.Fatalf("BlkC failed verification on VM2: %s", err) } - if err := vm1BlkC.Accept(context.Background()); !strings.Contains(err.Error(), "expected accepted block to have parent") { - t.Fatalf("Unexpected error when setting block at finalized height: %s", err) + if err := vm2.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { + t.Fatal(err) } -} -// Regression test to ensure that a VM that accepts block C while preferring -// block B will trigger a reorg. -// -// A -// / \ -// B C -func TestNonCanonicalAccept(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + newHead = <-newTxPoolHeadChan2 + if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { + t.Fatalf("Expected new block to match") } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) - defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - if err := tvm2.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) + errs = vm2.txPool.AddRemotesSync(txs[5:]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } - }() - - newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) - newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] + } - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + <-tvm2.ToEngine + vm2BlkD, err := vm2.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) + t.Fatalf("Failed to build BlkD on VM2: %s", err) } - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + // Parse blocks produced in vm2 + vm1BlkC, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()) + if err != nil { + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } + blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() - <-tvm1.toEngine - - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkD, err := vm1.ParseBlock(context.Background(), vm2BlkD.Bytes()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } + blkDHeight := vm1BlkD.Height() + blkDHash := vm1BlkD.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() - if err := vm1BlkA.Verify(context.Background()); err != nil { + // Should be no-ops + if err := vm1BlkC.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - - if _, err := tvm1.vm.GetBlockIDAtHeight(context.Background(), vm1BlkA.Height()); err != database.ErrNotFound { - t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) + if err := vm1BlkD.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { - t.Fatal(err) + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) } - - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) + if b := vm1.blockChain.GetBlockByNumber(blkDHeight); b != nil { + t.Fatalf("expected block at %d to be nil but got %s", blkDHeight, b.Hash().Hex()) } - if err := vm2BlkA.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM2: %s", err) - } - if _, err := tvm2.vm.GetBlockIDAtHeight(context.Background(), vm2BlkA.Height()); err != database.ErrNotFound { - t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) - } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { - t.Fatal(err) - } - - if err := vm1BlkA.Accept(context.Background()); err != nil { - t.Fatalf("VM1 failed to accept block: %s", err) - } - if blkID, err := tvm1.vm.GetBlockIDAtHeight(context.Background(), vm1BlkA.Height()); err != nil { - t.Fatalf("Height lookuped failed on accepted block: %s", err) - } else if blkID != vm1BlkA.ID() { - t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) - } - if err := vm2BlkA.Accept(context.Background()); err != nil { - t.Fatalf("VM2 failed to accept block: %s", err) - } - if blkID, err := tvm2.vm.GetBlockIDAtHeight(context.Background(), vm2BlkA.Height()); err != nil { - t.Fatalf("Height lookuped failed on accepted block: %s", err) - } else if blkID != vm2BlkA.ID() { - t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) - } - - newHead := <-newTxPoolHeadChan1 - if newHead.Head.Hash() != common.Hash(vm1BlkA.ID()) { - t.Fatalf("Expected new block to match") - } - newHead = <-newTxPoolHeadChan2 - if newHead.Head.Hash() != common.Hash(vm2BlkA.ID()) { - t.Fatalf("Expected new block to match") + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkBHash { + t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) } - // Create list of 10 successive transactions to build block A on vm1 - // and to be split into two separate blocks on VM2 - txs := make([]*types.Transaction, 10) - for i := 0; i < 10; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx + // Should still be no-ops on re-verify + if err := vm1BlkC.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - - var errs []error - - // Add the remote transactions, build the block, and set VM1's preference for block A - errs = tvm1.vm.txPool.AddRemotesSync(txs) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) - } + if err := vm1BlkD.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - - <-tvm1.toEngine - - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) } - - if err := vm1BlkB.Verify(context.Background()); err != nil { - t.Fatal(err) + if b := vm1.blockChain.GetBlockByNumber(blkDHeight); b != nil { + t.Fatalf("expected block at %d to be nil but got %s", blkDHeight, b.Hash().Hex()) } - - if _, err := tvm1.vm.GetBlockIDAtHeight(context.Background(), vm1BlkB.Height()); err != database.ErrNotFound { - t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkBHash { + t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { + // Should be queryable after setting preference to side chain + if err := vm1.SetPreference(context.Background(), vm1BlkD.ID()); err != nil { t.Fatal(err) } - tvm1.vm.eth.APIBackend.SetAllowUnfinalizedQueries(true) - - blkBHeight := vm1BlkB.Height() - blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } + if b := vm1.blockChain.GetBlockByNumber(blkDHeight); b.Hash() != blkDHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkDHeight, blkDHash.Hex(), b.Hash().Hex()) } - - <-tvm2.toEngine - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkC on VM2: %s", err) + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkDHash { + t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) } - vm1BlkC, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } + // Attempt to accept out of order + err = vm1BlkD.Accept(context.Background()) + require.ErrorContains(t, err, "expected accepted block to have parent") - if err := vm1BlkC.Verify(context.Background()); err != nil { + // Accept in order + if err := vm1BlkC.Accept(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - - if _, err := tvm1.vm.GetBlockIDAtHeight(context.Background(), vm1BlkC.Height()); err != database.ErrNotFound { - t.Fatalf("Expected unaccepted block not to be indexed by height, but found %s", err) + if err := vm1BlkD.Accept(context.Background()); err != nil { + t.Fatalf("Block failed acceptance on VM1: %s", err) } - if err := vm1BlkC.Accept(context.Background()); err != nil { - t.Fatalf("VM1 failed to accept block: %s", err) + // Ensure queryable after accepting + if b := vm1.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) } - - if blkID, err := tvm1.vm.GetBlockIDAtHeight(context.Background(), vm1BlkC.Height()); err != nil { - t.Fatalf("Height lookuped failed on accepted block: %s", err) - } else if blkID != vm1BlkC.ID() { - t.Fatalf("Expected accepted block to be indexed by height, but found %s", blkID) + if b := vm1.blockChain.GetBlockByNumber(blkDHeight); b.Hash() != blkDHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkDHeight, blkDHash.Hex(), b.Hash().Hex()) } - - blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkDHash { + t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) } } -// Regression test to ensure that a VM that verifies block B, C, then -// D (preferring block B) does not trigger a reorg through the re-verification -// of block C or D. +// Regression test to ensure that a VM that prefers block B is able to parse +// block C but unable to parse block D because it names B as an uncle, which +// are not supported. // // A // / \ // B C // | // D -func TestStickyPreference(t *testing.T) { - importAmount := uint64(1000000000) +func TestUncleBlock(t *testing.T) { fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + conf := vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) + defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { + if err := vm1.Shutdown(context.Background()); err != nil { t.Fatal(err) } - - if err := tvm2.vm.Shutdown(context.Background()); err != nil { + if err := vm2.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkA, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } if err := vm1BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) if err != nil { t.Fatalf("Unexpected error parsing block from vm2: %s", err) } if err := vm2BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } @@ -1971,31 +1288,26 @@ func TestStickyPreference(t *testing.T) { t.Fatalf("Expected new block to match") } - // Create list of 10 successive transactions to build block A on vm1 - // and to be split into two separate blocks on VM2 txs := make([]*types.Transaction, 10) for i := 0; i < 10; i++ { tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), key) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), key) if err != nil { t.Fatal(err) } txs[i] = signedTx } - var errs []error - - // Add the remote transactions, build the block, and set VM1's preference for block A - errs = tvm1.vm.txPool.AddRemotesSync(txs) + errs = vm1.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -2004,27 +1316,19 @@ func TestStickyPreference(t *testing.T) { t.Fatal(err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } - tvm1.vm.eth.APIBackend.SetAllowUnfinalizedQueries(true) - - blkBHeight := vm1BlkB.Height() - blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) - } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) + errs = vm2.txPool.AddRemotesSync(txs[0:5]) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } } - <-tvm2.toEngine - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) + <-tvm2.ToEngine + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { t.Fatalf("Failed to build BlkC on VM2: %s", err) } @@ -2033,7 +1337,7 @@ func TestStickyPreference(t *testing.T) { t.Fatalf("BlkC failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { t.Fatal(err) } @@ -2042,176 +1346,117 @@ func TestStickyPreference(t *testing.T) { t.Fatalf("Expected new block to match") } - errs = tvm2.vm.txPool.AddRemotesSync(txs[5:]) + errs = vm2.txPool.AddRemotesSync(txs[5:10]) for i, err := range errs { if err != nil { t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) } } - <-tvm2.toEngine - vm2BlkD, err := tvm2.vm.BuildBlock(context.Background()) + <-tvm2.ToEngine + vm2BlkD, err := vm2.BuildBlock(context.Background()) if err != nil { t.Fatalf("Failed to build BlkD on VM2: %s", err) } - // Parse blocks produced in vm2 - vm1BlkC, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - - vm1BlkD, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkD.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - blkDHeight := vm1BlkD.Height() - blkDHash := vm1BlkD.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - - // Should be no-ops - if err := vm1BlkC.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if err := vm1BlkD.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkDHeight); b != nil { - t.Fatalf("expected block at %d to be nil but got %s", blkDHeight, b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkBHash { - t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) - } - - // Should still be no-ops on re-verify - if err := vm1BlkC.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if err := vm1BlkD.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkBHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkBHash.Hex(), b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkDHeight); b != nil { - t.Fatalf("expected block at %d to be nil but got %s", blkDHeight, b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkBHash { - t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) - } - - // Should be queryable after setting preference to side chain - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkD.ID()); err != nil { - t.Fatal(err) - } - - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkDHeight); b.Hash() != blkDHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkDHeight, blkDHash.Hex(), b.Hash().Hex()) - } - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkDHash { - t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) - } - - // Attempt to accept out of order - if err := vm1BlkD.Accept(context.Background()); !strings.Contains(err.Error(), "expected accepted block to have parent") { - t.Fatalf("unexpected error when accepting out of order block: %s", err) - } - - // Accept in order - if err := vm1BlkC.Accept(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if err := vm1BlkD.Accept(context.Background()); err != nil { - t.Fatalf("Block failed acceptance on VM1: %s", err) - } + // Create uncle block from blkD + blkDEthBlock := vm2BlkD.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock + uncles := []*types.Header{vm1BlkB.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Header()} + uncleBlockHeader := types.CopyHeader(blkDEthBlock.Header()) + uncleBlockHeader.UncleHash = types.CalcUncleHash(uncles) - // Ensure queryable after accepting - if b := tvm1.vm.blockChain.GetBlockByNumber(blkBHeight); b.Hash() != blkCHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkBHeight, blkCHash.Hex(), b.Hash().Hex()) + uncleEthBlock := customtypes.NewBlockWithExtData( + uncleBlockHeader, + blkDEthBlock.Transactions(), + uncles, + nil, + trie.NewStackTrie(nil), + customtypes.BlockExtData(blkDEthBlock), + false, + ) + uncleBlock, err := wrapBlock(uncleEthBlock, vm2) + require.NoError(t, err) + if err := uncleBlock.Verify(context.Background()); !errors.Is(err, errUnclesUnsupported) { + t.Fatalf("VM2 should have failed with %q but got %q", errUnclesUnsupported, err.Error()) } - if b := tvm1.vm.blockChain.GetBlockByNumber(blkDHeight); b.Hash() != blkDHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkDHeight, blkDHash.Hex(), b.Hash().Hex()) + if _, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()); err != nil { + t.Fatalf("VM1 errored parsing blkC: %s", err) } - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkDHash { - t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) + if _, err := vm1.ParseBlock(context.Background(), uncleBlock.Bytes()); !errors.Is(err, errUnclesUnsupported) { + t.Fatalf("VM1 should have failed with %q but got %q", errUnclesUnsupported, err.Error()) } } -// Regression test to ensure that a VM that prefers block B is able to parse -// block C but unable to parse block D because it names B as an uncle, which -// are not supported. +// Regression test to ensure that a VM that verifies block B, C, then +// D (preferring block B) reorgs when C and then D are accepted. // // A // / \ // B C // | // D -func TestUncleBlock(t *testing.T) { - importAmount := uint64(1000000000) +func TestAcceptReorg(t *testing.T) { fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + conf := vmtest.TestVMConfig{ + Fork: &fork, + ConfigJSON: `{"pruning-enabled":true}`, } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) + vm1, tvm1 := setupDefaultTestVM(t, conf) + vm2, tvm2 := setupDefaultTestVM(t, conf) + defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { + if err := vm1.Shutdown(context.Background()); err != nil { t.Fatal(err) } - if err := tvm2.vm.Shutdown(context.Background()); err != nil { + + if err := vm2.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) + vm1.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) + vm2.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm1.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm1.toEngine + <-tvm1.ToEngine - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) + vm1BlkA, err := vm1.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } if err := vm1BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM1: %s", err) } - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { t.Fatal(err) } - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) + vm2BlkA, err := vm2.ParseBlock(context.Background(), vm1BlkA.Bytes()) if err != nil { t.Fatalf("Unexpected error parsing block from vm2: %s", err) } if err := vm2BlkA.Verify(context.Background()); err != nil { t.Fatalf("Block failed verification on VM2: %s", err) } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { t.Fatal(err) } @@ -2231,1206 +1476,218 @@ func TestUncleBlock(t *testing.T) { t.Fatalf("Expected new block to match") } + // Create list of 10 successive transactions to build block A on vm1 + // and to be split into two separate blocks on VM2 txs := make([]*types.Transaction, 10) for i := 0; i < 10; i++ { tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), key) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm1.chainID), key) if err != nil { t.Fatal(err) } txs[i] = signedTx } - var errs []error - - errs = tvm1.vm.txPool.AddRemotesSync(txs) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) - } - } - - <-tvm1.toEngine - - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := vm1BlkB.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { - t.Fatal(err) - } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) + // Add the remote transactions, build the block, and set VM1's preference + // for block B + errs = vm1.txPool.AddRemotesSync(txs) for i, err := range errs { if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } - } - - <-tvm2.toEngine - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkC on VM2: %s", err) - } - - if err := vm2BlkC.Verify(context.Background()); err != nil { - t.Fatalf("BlkC failed verification on VM2: %s", err) - } - - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { - t.Fatal(err) - } - - newHead = <-newTxPoolHeadChan2 - if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { - t.Fatalf("Expected new block to match") - } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[5:10]) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } - } - - <-tvm2.toEngine - vm2BlkD, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkD on VM2: %s", err) - } - - // Create uncle block from blkD - blkDEthBlock := vm2BlkD.(*chain.BlockWrapper).Block.(*Block).ethBlock - uncles := []*types.Header{vm1BlkB.(*chain.BlockWrapper).Block.(*Block).ethBlock.Header()} - uncleBlockHeader := types.CopyHeader(blkDEthBlock.Header()) - uncleBlockHeader.UncleHash = types.CalcUncleHash(uncles) - - uncleEthBlock := customtypes.NewBlockWithExtData( - uncleBlockHeader, - blkDEthBlock.Transactions(), - uncles, - nil, - trie.NewStackTrie(nil), - customtypes.BlockExtData(blkDEthBlock), - false, - ) - uncleBlock, err := tvm2.vm.newBlock(uncleEthBlock) - if err != nil { - t.Fatal(err) - } - if err := uncleBlock.Verify(context.Background()); !errors.Is(err, errUnclesUnsupported) { - t.Fatalf("VM2 should have failed with %q but got %q", errUnclesUnsupported, err.Error()) - } - if _, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()); err != nil { - t.Fatalf("VM1 errored parsing blkC: %s", err) - } - if _, err := tvm1.vm.ParseBlock(context.Background(), uncleBlock.Bytes()); !errors.Is(err, errUnclesUnsupported) { - t.Fatalf("VM1 should have failed with %q but got %q", errUnclesUnsupported, err.Error()) - } -} - -// Regression test to ensure that a VM that is not able to parse a block that -// contains no transactions. -func TestEmptyBlock(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) - } - - // Create empty block from blkA - ethBlock := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock - - emptyEthBlock := customtypes.NewBlockWithExtData( - types.CopyHeader(ethBlock.Header()), - nil, - nil, - nil, - new(trie.Trie), - nil, - false, - ) - - if len(customtypes.BlockExtData(emptyEthBlock)) != 0 || customtypes.GetHeaderExtra(emptyEthBlock.Header()).ExtDataHash != (common.Hash{}) { - t.Fatalf("emptyEthBlock should not have any extra data") - } - - emptyBlock, err := tvm.vm.newBlock(emptyEthBlock) - if err != nil { - t.Fatal(err) - } - - if _, err := tvm.vm.ParseBlock(context.Background(), emptyBlock.Bytes()); !errors.Is(err, errEmptyBlock) { - t.Fatalf("VM should have failed with errEmptyBlock but got %s", err.Error()) - } - if err := emptyBlock.Verify(context.Background()); !errors.Is(err, errEmptyBlock) { - t.Fatalf("block should have failed verification with errEmptyBlock but got %s", err.Error()) - } -} - -// Regression test to ensure that a VM that verifies block B, C, then -// D (preferring block B) reorgs when C and then D are accepted. -// -// A -// / \ -// B C -// | -// D -func TestAcceptReorg(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.NoUpgrades - tvmConfig := testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - } - tvm1 := newVM(t, tvmConfig) - tvm2 := newVM(t, tvmConfig) - defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm2.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - newTxPoolHeadChan1 := make(chan core.NewTxPoolReorgEvent, 1) - tvm1.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan1) - newTxPoolHeadChan2 := make(chan core.NewTxPoolReorgEvent, 1) - tvm2.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan2) - - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] - - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm1.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm1.toEngine - - vm1BlkA, err := tvm1.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) - } - - if err := vm1BlkA.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkA.ID()); err != nil { - t.Fatal(err) - } - - vm2BlkA, err := tvm2.vm.ParseBlock(context.Background(), vm1BlkA.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - if err := vm2BlkA.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM2: %s", err) - } - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkA.ID()); err != nil { - t.Fatal(err) - } - - if err := vm1BlkA.Accept(context.Background()); err != nil { - t.Fatalf("VM1 failed to accept block: %s", err) - } - if err := vm2BlkA.Accept(context.Background()); err != nil { - t.Fatalf("VM2 failed to accept block: %s", err) - } - - newHead := <-newTxPoolHeadChan1 - if newHead.Head.Hash() != common.Hash(vm1BlkA.ID()) { - t.Fatalf("Expected new block to match") - } - newHead = <-newTxPoolHeadChan2 - if newHead.Head.Hash() != common.Hash(vm2BlkA.ID()) { - t.Fatalf("Expected new block to match") - } - - // Create list of 10 successive transactions to build block A on vm1 - // and to be split into two separate blocks on VM2 - txs := make([]*types.Transaction, 10) - for i := 0; i < 10; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm1.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx - } - - // Add the remote transactions, build the block, and set VM1's preference - // for block B - errs := tvm1.vm.txPool.AddRemotesSync(txs) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) - } - } - - <-tvm1.toEngine - - vm1BlkB, err := tvm1.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := vm1BlkB.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm1.vm.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { - t.Fatal(err) - } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[0:5]) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } - } - - <-tvm2.toEngine - - vm2BlkC, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkC on VM2: %s", err) - } - - if err := vm2BlkC.Verify(context.Background()); err != nil { - t.Fatalf("BlkC failed verification on VM2: %s", err) - } - - if err := tvm2.vm.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { - t.Fatal(err) - } - - newHead = <-newTxPoolHeadChan2 - if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { - t.Fatalf("Expected new block to match") - } - - errs = tvm2.vm.txPool.AddRemotesSync(txs[5:]) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) - } - } - - <-tvm2.toEngine - - vm2BlkD, err := tvm2.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build BlkD on VM2: %s", err) - } - - // Parse blocks produced in vm2 - vm1BlkC, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkC.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - - vm1BlkD, err := tvm1.vm.ParseBlock(context.Background(), vm2BlkD.Bytes()) - if err != nil { - t.Fatalf("Unexpected error parsing block from vm2: %s", err) - } - - if err := vm1BlkC.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - if err := vm1BlkD.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM1: %s", err) - } - - blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkBHash { - t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) - } - - if err := vm1BlkC.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkCHash { - t.Fatalf("expected current block to have hash %s but got %s", blkCHash.Hex(), b.Hash().Hex()) - } - if err := vm1BlkB.Reject(context.Background()); err != nil { - t.Fatal(err) - } - - if err := vm1BlkD.Accept(context.Background()); err != nil { - t.Fatal(err) - } - blkDHash := vm1BlkD.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - if b := tvm1.vm.blockChain.CurrentBlock(); b.Hash() != blkDHash { - t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) - } -} - -func TestFutureBlock(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blkA, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) - } - - // Create empty block from blkA - internalBlkA := blkA.(*chain.BlockWrapper).Block.(*Block) - modifiedHeader := types.CopyHeader(internalBlkA.ethBlock.Header()) - // Set the VM's clock to the time of the produced block - tvm.vm.clock.Set(time.Unix(int64(modifiedHeader.Time), 0)) - // Set the modified time to exceed the allowed future time - modifiedTime := modifiedHeader.Time + uint64(maxFutureBlockTime.Seconds()+1) - modifiedHeader.Time = modifiedTime - modifiedBlock := customtypes.NewBlockWithExtData( - modifiedHeader, - nil, - nil, - nil, - new(trie.Trie), - customtypes.BlockExtData(internalBlkA.ethBlock), - false, - ) - - futureBlock, err := tvm.vm.newBlock(modifiedBlock) - if err != nil { - t.Fatal(err) - } - - if err := futureBlock.Verify(context.Background()); err == nil { - t.Fatal("Future block should have failed verification due to block timestamp too far in the future") - } else if !strings.Contains(err.Error(), "block timestamp is too far in the future") { - t.Fatalf("Expected error to be block timestamp too far in the future but found %s", err) - } -} - -// Regression test to ensure we can build blocks if we are starting with the -// Apricot Phase 1 ruleset in genesis. -func TestBuildApricotPhase1Block(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.ApricotPhase1 - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - newHead := <-newTxPoolHeadChan - if newHead.Head.Hash() != common.Hash(blk.ID()) { - t.Fatalf("Expected new block to match") - } - - txs := make([]*types.Transaction, 10) - for i := 0; i < 5; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx - } - for i := 5; i < 10; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap1.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx - } - errs := tvm.vm.txPool.AddRemotesSync(txs) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add tx at index %d: %s", i, err) - } - } - - <-tvm.toEngine - - blk, err = tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) - if err != nil { - t.Fatal(err) - } - if lastAcceptedID != blk.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) - } - - // Confirm all txs are present - ethBlkTxs := tvm.vm.blockChain.GetBlockByNumber(2).Transactions() - for i, tx := range txs { - if len(ethBlkTxs) <= i { - t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) - } - if ethBlkTxs[i].Hash() != tx.Hash() { - t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) - } - } -} - -func TestLastAcceptedBlockNumberAllow(t *testing.T) { - importAmount := uint64(1000000000) - fork := upgradetest.NoUpgrades - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatalf("Block failed verification on VM: %s", err) - } - - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) - } - - blkHeight := blk.Height() - blkHash := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock.Hash() - - tvm.vm.eth.APIBackend.SetAllowUnfinalizedQueries(true) - - ctx := context.Background() - b, err := tvm.vm.eth.APIBackend.BlockByNumber(ctx, rpc.BlockNumber(blkHeight)) - if err != nil { - t.Fatal(err) - } - if b.Hash() != blkHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkHeight, blkHash.Hex(), b.Hash().Hex()) - } - - tvm.vm.eth.APIBackend.SetAllowUnfinalizedQueries(false) - - _, err = tvm.vm.eth.APIBackend.BlockByNumber(ctx, rpc.BlockNumber(blkHeight)) - if !errors.Is(err, eth.ErrUnfinalizedData) { - t.Fatalf("expected ErrUnfinalizedData but got %s", err.Error()) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatalf("VM failed to accept block: %s", err) - } - - if b := tvm.vm.blockChain.GetBlockByNumber(blkHeight); b.Hash() != blkHash { - t.Fatalf("expected block at %d to have hash %s but got %s", blkHeight, blkHash.Hex(), b.Hash().Hex()) - } -} - -// Builds [blkA] with a virtuous import transaction and [blkB] with a separate import transaction -// that does not conflict. Accepts [blkB] and rejects [blkA], then asserts that the virtuous atomic -// transaction in [blkA] is correctly re-issued into the atomic transaction mempool. -func TestReissueAtomicTx(t *testing.T) { - fork := upgradetest.ApricotPhase1 - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: 10000000, - testShortIDAddrs[1]: 10000000, - }, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - genesisBlkID, err := tvm.vm.LastAccepted(context.Background()) - if err != nil { - t.Fatal(err) - } - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blkA, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blkA.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), blkA.ID()); err != nil { - t.Fatal(err) - } - - // SetPreference to parent before rejecting (will rollback state to genesis - // so that atomic transaction can be reissued, otherwise current block will - // conflict with UTXO to be reissued) - if err := tvm.vm.SetPreference(context.Background(), genesisBlkID); err != nil { - t.Fatal(err) - } - - // Rejecting [blkA] should cause [importTx] to be re-issued into the mempool. - if err := blkA.Reject(context.Background()); err != nil { - t.Fatal(err) - } - - // Sleep for a minimum of two seconds to ensure that [blkB] will have a different timestamp - // than [blkA] so that the block will be unique. This is necessary since we have marked [blkA] - // as Rejected. - time.Sleep(2 * time.Second) - <-tvm.toEngine - blkB, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if blkB.Height() != blkA.Height() { - t.Fatalf("Expected blkB (%d) to have the same height as blkA (%d)", blkB.Height(), blkA.Height()) - } - - if err := blkB.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), blkB.ID()); err != nil { - t.Fatal(err) - } - - if err := blkB.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - if lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()); err != nil { - t.Fatal(err) - } else if lastAcceptedID != blkB.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blkB.ID(), lastAcceptedID) - } - - // Check that [importTx] has been indexed correctly after [blkB] is accepted. - _, height, err := tvm.vm.atomicTxRepository.GetByTxID(importTx.ID()) - if err != nil { - t.Fatal(err) - } else if height != blkB.Height() { - t.Fatalf("Expected indexed height of import tx to be %d, but found %d", blkB.Height(), height) - } -} - -func TestAtomicTxFailsEVMStateTransferBuildBlock(t *testing.T) { - fork := upgradetest.ApricotPhase1 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - exportTxs := createExportTxOptions(t, tvm.vm, tvm.toEngine, tvm.atomicMemory) - exportTx1, exportTx2 := exportTxs[0], exportTxs[1] - - if err := tvm.vm.mempool.AddLocalTx(exportTx1); err != nil { - t.Fatal(err) - } - <-tvm.toEngine - exportBlk1, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - if err := exportBlk1.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), exportBlk1.ID()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(exportTx2); err == nil { - t.Fatal("Should have failed to issue due to an invalid export tx") - } - - if err := tvm.vm.mempool.AddRemoteTx(exportTx2); err == nil { - t.Fatal("Should have failed to add because conflicting") - } - - // Manually add transaction to mempool to bypass validation - if err := tvm.vm.mempool.ForceAddTx(exportTx2); err != nil { - t.Fatal(err) - } - <-tvm.toEngine - - _, err = tvm.vm.BuildBlock(context.Background()) - if err == nil { - t.Fatal("BuildBlock should have returned an error due to invalid export transaction") - } -} - -func TestBuildInvalidBlockHead(t *testing.T) { - fork := upgradetest.ApricotPhase1 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - key0 := testKeys[0] - addr0 := key0.Address() - - // Create the transaction - utx := &atomic.UnsignedImportTx{ - NetworkID: tvm.vm.ctx.NetworkID, - BlockchainID: tvm.vm.ctx.ChainID, - Outs: []atomic.EVMOutput{{ - Address: common.Address(addr0), - Amount: 1 * units.Avax, - AssetID: tvm.vm.ctx.AVAXAssetID, - }}, - ImportedInputs: []*avax.TransferableInput{ - { - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, - In: &secp256k1fx.TransferInput{ - Amt: 1 * units.Avax, - Input: secp256k1fx.Input{ - SigIndices: []uint32{0}, - }, - }, - }, - }, - SourceChain: tvm.vm.ctx.XChainID, - } - tx := &atomic.Tx{UnsignedAtomicTx: utx} - if err := tx.Sign(atomic.Codec, [][]*secp256k1.PrivateKey{{key0}}); err != nil { - t.Fatal(err) - } - - currentBlock := tvm.vm.blockChain.CurrentBlock() - - // Verify that the transaction fails verification when attempting to issue - // it into the atomic mempool. - if err := tvm.vm.mempool.AddLocalTx(tx); err == nil { - t.Fatal("Should have failed to issue invalid transaction") - } - // Force issue the transaction directly to the mempool - if err := tvm.vm.mempool.ForceAddTx(tx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - if _, err := tvm.vm.BuildBlock(context.Background()); err == nil { - t.Fatalf("Unexpectedly created a block") - } - - newCurrentBlock := tvm.vm.blockChain.CurrentBlock() - - if currentBlock.Hash() != newCurrentBlock.Hash() { - t.Fatal("current block changed") - } -} - -// Regression test to ensure we can build blocks if we are starting with the -// Apricot Phase 4 ruleset in genesis. -func TestBuildApricotPhase4Block(t *testing.T) { - fork := upgradetest.ApricotPhase4 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] - - importAmount := uint64(1000000000) - utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} - - utxo := &avax.UTXO{ - UTXOID: utxoID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, - Out: &secp256k1fx.TransferOutput{ - Amt: importAmount, - OutputOwners: secp256k1fx.OutputOwners{ - Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, - }, - }, - } - utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) - if err != nil { - t.Fatal(err) - } - - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) - inputID := utxo.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ - Key: inputID[:], - Value: utxoBytes, - Traits: [][]byte{ - testKeys[0].Address().Bytes(), - }, - }}}}); err != nil { - t.Fatal(err) - } - - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { - t.Fatal(err) - } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - - <-tvm.toEngine - - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { - t.Fatal(err) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - ethBlk := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock - if eBlockGasCost := customtypes.BlockGasCost(ethBlk); eBlockGasCost == nil || eBlockGasCost.Cmp(common.Big0) != 0 { - t.Fatalf("expected blockGasCost to be 0 but got %d", eBlockGasCost) - } - if eExtDataGasUsed := customtypes.BlockExtDataGasUsed(ethBlk); eExtDataGasUsed == nil || eExtDataGasUsed.Cmp(big.NewInt(1230)) != 0 { - t.Fatalf("expected extDataGasUsed to be 1000 but got %d", eExtDataGasUsed) - } - minRequiredTip, err := header.EstimateRequiredTip(tvm.vm.chainConfigExtra(), ethBlk.Header()) - if err != nil { - t.Fatal(err) - } - if minRequiredTip == nil || minRequiredTip.Cmp(common.Big0) != 0 { - t.Fatalf("expected minRequiredTip to be 0 but got %d", minRequiredTip) - } - - newHead := <-newTxPoolHeadChan - if newHead.Head.Hash() != common.Hash(blk.ID()) { - t.Fatalf("Expected new block to match") - } - - txs := make([]*types.Transaction, 10) - for i := 0; i < 5; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx - } - for i := 5; i < 10; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap1.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx - } - errs := tvm.vm.txPool.AddRemotesSync(txs) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add tx at index %d: %s", i, err) - } - } - - <-tvm.toEngine - - blk, err = tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) - } - - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) - } - - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) - } - - ethBlk = blk.(*chain.BlockWrapper).Block.(*Block).ethBlock - if customtypes.BlockGasCost(ethBlk) == nil || customtypes.BlockGasCost(ethBlk).Cmp(big.NewInt(100)) < 0 { - t.Fatalf("expected blockGasCost to be at least 100 but got %d", customtypes.BlockGasCost(ethBlk)) - } - if customtypes.BlockExtDataGasUsed(ethBlk) == nil || customtypes.BlockExtDataGasUsed(ethBlk).Cmp(common.Big0) != 0 { - t.Fatalf("expected extDataGasUsed to be 0 but got %d", customtypes.BlockExtDataGasUsed(ethBlk)) - } - minRequiredTip, err = header.EstimateRequiredTip(tvm.vm.chainConfigExtra(), ethBlk.Header()) - if err != nil { - t.Fatal(err) - } - if minRequiredTip == nil || minRequiredTip.Cmp(big.NewInt(0.05*utils.GWei)) < 0 { - t.Fatalf("expected minRequiredTip to be at least 0.05 gwei but got %d", minRequiredTip) - } - - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) - if err != nil { - t.Fatal(err) - } - if lastAcceptedID != blk.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) - } - - // Confirm all txs are present - ethBlkTxs := tvm.vm.blockChain.GetBlockByNumber(2).Transactions() - for i, tx := range txs { - if len(ethBlkTxs) <= i { - t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) - } - if ethBlkTxs[i].Hash() != tx.Hash() { - t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) - } - } -} - -// Regression test to ensure we can build blocks if we are starting with the -// Apricot Phase 5 ruleset in genesis. -func TestBuildApricotPhase5Block(t *testing.T) { - fork := upgradetest.ApricotPhase5 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) + t.Fatalf("Failed to add transaction to VM1 at index %d: %s", i, err) } - }() - - newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - - key := testKeys[0].ToECDSA() - address := testEthAddrs[0] - - importAmount := uint64(1000000000) - utxoID := avax.UTXOID{TxID: ids.GenerateTestID()} - - utxo := &avax.UTXO{ - UTXOID: utxoID, - Asset: avax.Asset{ID: tvm.vm.ctx.AVAXAssetID}, - Out: &secp256k1fx.TransferOutput{ - Amt: importAmount, - OutputOwners: secp256k1fx.OutputOwners{ - Threshold: 1, - Addrs: []ids.ShortID{testKeys[0].Address()}, - }, - }, } - utxoBytes, err := atomic.Codec.Marshal(atomic.CodecVersion, utxo) + + <-tvm1.ToEngine + + vm1BlkB, err := vm1.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } - xChainSharedMemory := tvm.atomicMemory.NewSharedMemory(tvm.vm.ctx.XChainID) - inputID := utxo.InputID() - if err := xChainSharedMemory.Apply(map[ids.ID]*avalancheatomic.Requests{tvm.vm.ctx.ChainID: {PutRequests: []*avalancheatomic.Element{{ - Key: inputID[:], - Value: utxoBytes, - Traits: [][]byte{ - testKeys[0].Address().Bytes(), - }, - }}}}); err != nil { + if err := vm1BlkB.Verify(context.Background()); err != nil { t.Fatal(err) } - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, address, initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - if err != nil { + if err := vm1.SetPreference(context.Background(), vm1BlkB.ID()); err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs = vm2.txPool.AddRemotesSync(txs[0:5]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) + } } - <-tvm.toEngine + <-tvm2.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + vm2BlkC, err := vm2.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) + t.Fatalf("Failed to build BlkC on VM2: %s", err) } - if err := blk.Verify(context.Background()); err != nil { - t.Fatal(err) + if err := vm2BlkC.Verify(context.Background()); err != nil { + t.Fatalf("BlkC failed verification on VM2: %s", err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { + if err := vm2.SetPreference(context.Background(), vm2BlkC.ID()); err != nil { t.Fatal(err) } - if err := blk.Accept(context.Background()); err != nil { - t.Fatal(err) + newHead = <-newTxPoolHeadChan2 + if newHead.Head.Hash() != common.Hash(vm2BlkC.ID()) { + t.Fatalf("Expected new block to match") } - ethBlk := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock - if eBlockGasCost := customtypes.BlockGasCost(ethBlk); eBlockGasCost == nil || eBlockGasCost.Cmp(common.Big0) != 0 { - t.Fatalf("expected blockGasCost to be 0 but got %d", eBlockGasCost) - } - if eExtDataGasUsed := customtypes.BlockExtDataGasUsed(ethBlk); eExtDataGasUsed == nil || eExtDataGasUsed.Cmp(big.NewInt(11230)) != 0 { - t.Fatalf("expected extDataGasUsed to be 11230 but got %d", eExtDataGasUsed) + errs = vm2.txPool.AddRemotesSync(txs[5:]) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add transaction to VM2 at index %d: %s", i, err) + } } - minRequiredTip, err := header.EstimateRequiredTip(tvm.vm.chainConfigExtra(), ethBlk.Header()) + + <-tvm2.ToEngine + + vm2BlkD, err := vm2.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) + t.Fatalf("Failed to build BlkD on VM2: %s", err) } - if minRequiredTip == nil || minRequiredTip.Cmp(common.Big0) != 0 { - t.Fatalf("expected minRequiredTip to be 0 but got %d", minRequiredTip) + + // Parse blocks produced in vm2 + vm1BlkC, err := vm1.ParseBlock(context.Background(), vm2BlkC.Bytes()) + if err != nil { + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - newHead := <-newTxPoolHeadChan - if newHead.Head.Hash() != common.Hash(blk.ID()) { - t.Fatalf("Expected new block to match") + vm1BlkD, err := vm1.ParseBlock(context.Background(), vm2BlkD.Bytes()) + if err != nil { + t.Fatalf("Unexpected error parsing block from vm2: %s", err) } - txs := make([]*types.Transaction, 10) - for i := 0; i < 10; i++ { - tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(3*ap0.MinGasPrice), nil) - signedTx, err := types.SignTx(tx, types.NewEIP155Signer(tvm.vm.chainID), key) - if err != nil { - t.Fatal(err) - } - txs[i] = signedTx + if err := vm1BlkC.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - errs := tvm.vm.txPool.Add(txs, false, false) - for i, err := range errs { - if err != nil { - t.Fatalf("Failed to add tx at index %d: %s", i, err) - } + if err := vm1BlkD.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM1: %s", err) } - <-tvm.toEngine - - blk, err = tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) + blkBHash := vm1BlkB.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkBHash { + t.Fatalf("expected current block to have hash %s but got %s", blkBHash.Hex(), b.Hash().Hex()) } - if err := blk.Verify(context.Background()); err != nil { + if err := vm1BlkC.Accept(context.Background()); err != nil { t.Fatal(err) } - if err := blk.Accept(context.Background()); err != nil { + blkCHash := vm1BlkC.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkCHash { + t.Fatalf("expected current block to have hash %s but got %s", blkCHash.Hex(), b.Hash().Hex()) + } + if err := vm1BlkB.Reject(context.Background()); err != nil { t.Fatal(err) } - ethBlk = blk.(*chain.BlockWrapper).Block.(*Block).ethBlock - if customtypes.BlockGasCost(ethBlk) == nil || customtypes.BlockGasCost(ethBlk).Cmp(big.NewInt(100)) < 0 { - t.Fatalf("expected blockGasCost to be at least 100 but got %d", customtypes.BlockGasCost(ethBlk)) + if err := vm1BlkD.Accept(context.Background()); err != nil { + t.Fatal(err) } - if customtypes.BlockExtDataGasUsed(ethBlk) == nil || customtypes.BlockExtDataGasUsed(ethBlk).Cmp(common.Big0) != 0 { - t.Fatalf("expected extDataGasUsed to be 0 but got %d", customtypes.BlockExtDataGasUsed(ethBlk)) + blkDHash := vm1BlkD.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + if b := vm1.blockChain.CurrentBlock(); b.Hash() != blkDHash { + t.Fatalf("expected current block to have hash %s but got %s", blkDHash.Hex(), b.Hash().Hex()) } - minRequiredTip, err = header.EstimateRequiredTip(tvm.vm.chainConfigExtra(), ethBlk.Header()) +} + +func TestFutureBlock(t *testing.T) { + fork := upgradetest.NoUpgrades + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, + }) + + defer func() { + if err := vm.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - if minRequiredTip == nil || minRequiredTip.Cmp(big.NewInt(0.05*utils.GWei)) < 0 { - t.Fatalf("expected minRequiredTip to be at least 0.05 gwei but got %d", minRequiredTip) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } + <-tvm.ToEngine - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) + blkA, err := vm.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) - } - if lastAcceptedID != blk.ID() { - t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + t.Fatalf("Failed to build block with transaction: %s", err) } - // Confirm all txs are present - ethBlkTxs := tvm.vm.blockChain.GetBlockByNumber(2).Transactions() - for i, tx := range txs { - if len(ethBlkTxs) <= i { - t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) - } - if ethBlkTxs[i].Hash() != tx.Hash() { - t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) - } + // Create empty block from blkA + internalBlkA := blkA.(*chain.BlockWrapper).Block.(*wrappedBlock) + modifiedHeader := types.CopyHeader(internalBlkA.ethBlock.Header()) + // Set the VM's clock to the time of the produced block + vm.clock.Set(time.Unix(int64(modifiedHeader.Time), 0)) + // Set the modified time to exceed the allowed future time + modifiedTime := modifiedHeader.Time + uint64(maxFutureBlockTime.Seconds()+1) + modifiedHeader.Time = modifiedTime + modifiedBlock := customtypes.NewBlockWithExtData( + modifiedHeader, + nil, + nil, + nil, + new(trie.Trie), + customtypes.BlockExtData(internalBlkA.ethBlock), + false, + ) + + futureBlock, err := wrapBlock(modifiedBlock, vm) + require.NoError(t, err) + if err := futureBlock.Verify(context.Background()); err == nil { + t.Fatal("Future block should have failed verification due to block timestamp too far in the future") + } else if !strings.Contains(err.Error(), "block timestamp is too far in the future") { + t.Fatalf("Expected error to be block timestamp too far in the future but found %s", err) } } -// This is a regression test to ensure that if two consecutive atomic transactions fail verification -// in onFinalizeAndAssemble it will not cause a panic due to calling RevertToSnapshot(revID) on the -// same revision ID twice. -func TestConsecutiveAtomicTransactionsRevertSnapshot(t *testing.T) { +// Regression test to ensure we can build blocks if we are starting with the +// Apricot Phase 1 ruleset in genesis. +func TestBuildApricotPhase1Block(t *testing.T) { fork := upgradetest.ApricotPhase1 - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() newTxPoolHeadChan := make(chan core.NewTxPoolReorgEvent, 1) - tvm.vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) + vm.txPool.SubscribeNewReorgEvent(newTxPoolHeadChan) - // Create three conflicting import transactions - importTxs := createImportTxOptions(t, tvm.vm, tvm.atomicMemory) + key := vmtest.TestKeys[1].ToECDSA() + address := vmtest.TestEthAddrs[1] - // Issue the first import transaction, build, and accept the block. - if err := tvm.vm.mempool.AddLocalTx(importTxs[0]); err != nil { + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) + if err != nil { t.Fatal(err) } + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } + } - <-tvm.toEngine + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { t.Fatal(err) } @@ -3439,7 +1696,7 @@ func TestConsecutiveAtomicTransactionsRevertSnapshot(t *testing.T) { t.Fatal(err) } - if err := tvm.vm.SetPreference(context.Background(), blk.ID()); err != nil { + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { t.Fatal(err) } @@ -3452,268 +1709,179 @@ func TestConsecutiveAtomicTransactionsRevertSnapshot(t *testing.T) { t.Fatalf("Expected new block to match") } - // Add the two conflicting transactions directly to the mempool, so that two consecutive transactions - // will fail verification when build block is called. - tvm.vm.mempool.AddRemoteTx(importTxs[1]) - tvm.vm.mempool.AddRemoteTx(importTxs[2]) - - if _, err := tvm.vm.BuildBlock(context.Background()); err == nil { - t.Fatal("Expected build block to fail due to empty block") - } -} - -func TestAtomicTxBuildBlockDropsConflicts(t *testing.T) { - importAmount := uint64(10000000) - fork := upgradetest.ApricotPhase5 - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - testShortIDAddrs[1]: importAmount, - testShortIDAddrs[2]: importAmount, - }, - }) - conflictKey := utilstest.NewKey(t) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - // Create a conflict set for each pair of transactions - conflictSets := make([]set.Set[ids.ID], len(testKeys)) - for index, key := range testKeys { - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[index], initialBaseFee, []*secp256k1.PrivateKey{key}) + txs := make([]*types.Transaction, 10) + for i := 0; i < 5; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainID), key) if err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - conflictSets[index].Add(importTx.ID()) - conflictTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, conflictKey.Address, initialBaseFee, []*secp256k1.PrivateKey{key}) + txs[i] = signedTx + } + for i := 5; i < 10; i++ { + tx := types.NewTransaction(uint64(i), address, big.NewInt(10), 21000, big.NewInt(ap1.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainID), key) if err != nil { t.Fatal(err) } - if err := tvm.vm.mempool.AddLocalTx(conflictTx); err == nil { - t.Fatal("should conflict with the utxoSet in the mempool") - } - // force add the tx - tvm.vm.mempool.ForceAddTx(conflictTx) - conflictSets[index].Add(conflictTx.ID()) - } - <-tvm.toEngine - // Note: this only checks the path through OnFinalizeAndAssemble, we should make sure to add a test - // that verifies blocks received from the network will also fail verification - blk, err := tvm.vm.BuildBlock(context.Background()) - if err != nil { - t.Fatal(err) + txs[i] = signedTx } - atomicTxs := blk.(*chain.BlockWrapper).Block.(*Block).atomicTxs - assert.True(t, len(atomicTxs) == len(testKeys), "Conflict transactions should be out of the batch") - atomicTxIDs := set.Set[ids.ID]{} - for _, tx := range atomicTxs { - atomicTxIDs.Add(tx.ID()) + errs = vm.txPool.AddRemotesSync(txs) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - // Check that removing the txIDs actually included in the block from each conflict set - // leaves one item remaining for each conflict set ie. only one tx from each conflict set - // has been included in the block. - for _, conflictSet := range conflictSets { - conflictSet.Difference(atomicTxIDs) - assert.Equal(t, 1, conflictSet.Len()) + <-tvm.ToEngine + + blk, err = vm.BuildBlock(context.Background()) + if err != nil { + t.Fatal(err) } if err := blk.Verify(context.Background()); err != nil { t.Fatal(err) } + if err := blk.Accept(context.Background()); err != nil { t.Fatal(err) } -} - -func TestBuildBlockDoesNotExceedAtomicGasLimit(t *testing.T) { - importAmount := uint64(10000000) - fork := upgradetest.ApricotPhase5 - tvm := newVM(t, testVMConfig{ - fork: &fork, - }) - defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - }() - - kc := secp256k1fx.NewKeychain() - kc.Add(testKeys[0]) - txID, err := ids.ToID(hashing.ComputeHash256(testShortIDAddrs[0][:])) - assert.NoError(t, err) - - mempoolTxs := 200 - for i := 0; i < mempoolTxs; i++ { - utxo, err := addUTXO(tvm.atomicMemory, tvm.vm.ctx, txID, uint32(i), tvm.vm.ctx.AVAXAssetID, importAmount, testShortIDAddrs[0]) - assert.NoError(t, err) - - importTx, err := atomic.NewImportTx(tvm.vm.ctx, tvm.vm.currentRules(), tvm.vm.clock.Unix(), tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, kc, []*avax.UTXO{utxo}) - if err != nil { - t.Fatal(err) - } - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) - } - } - <-tvm.toEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + lastAcceptedID, err := vm.LastAccepted(context.Background()) if err != nil { t.Fatal(err) } + if lastAcceptedID != blk.ID() { + t.Fatalf("Expected last accepted blockID to be the accepted block: %s, but found %s", blk.ID(), lastAcceptedID) + } - atomicTxs := blk.(*chain.BlockWrapper).Block.(*Block).atomicTxs - - // Need to ensure that not all of the transactions in the mempool are included in the block. - // This ensures that we hit the atomic gas limit while building the block before we hit the - // upper limit on the size of the codec for marshalling the atomic transactions. - if len(atomicTxs) >= mempoolTxs { - t.Fatalf("Expected number of atomic transactions included in the block (%d) to be less than the number of transactions added to the mempool (%d)", len(atomicTxs), mempoolTxs) + // Confirm all txs are present + ethBlkTxs := vm.blockChain.GetBlockByNumber(2).Transactions() + for i, tx := range txs { + if len(ethBlkTxs) <= i { + t.Fatalf("missing transactions expected: %d but found: %d", len(txs), len(ethBlkTxs)) + } + if ethBlkTxs[i].Hash() != tx.Hash() { + t.Fatalf("expected tx at index %d to have hash: %x but has: %x", i, txs[i].Hash(), tx.Hash()) + } } } -func TestExtraStateChangeAtomicGasLimitExceeded(t *testing.T) { - importAmount := uint64(10000000) - // We create two VMs one in ApriotPhase4 and one in ApricotPhase5, so that we can construct a block - // containing a large enough atomic transaction that it will exceed the atomic gas limit in - // ApricotPhase5. - ap4 := upgradetest.ApricotPhase4 - tvm1 := newVM(t, testVMConfig{ - fork: &ap4, - }) - ap5 := upgradetest.ApricotPhase5 - tvm2 := newVM(t, testVMConfig{ - fork: &ap5, +func TestLastAcceptedBlockNumberAllow(t *testing.T) { + fork := upgradetest.NoUpgrades + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) + defer func() { - if err := tvm1.vm.Shutdown(context.Background()); err != nil { - t.Fatal(err) - } - if err := tvm2.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - kc := secp256k1fx.NewKeychain() - kc.Add(testKeys[0]) - txID, err := ids.ToID(hashing.ComputeHash256(testShortIDAddrs[0][:])) - assert.NoError(t, err) - - // Add enough UTXOs, such that the created import transaction will attempt to consume more gas than allowed - // in ApricotPhase5. - for i := 0; i < 100; i++ { - _, err := addUTXO(tvm1.atomicMemory, tvm1.vm.ctx, txID, uint32(i), tvm1.vm.ctx.AVAXAssetID, importAmount, testShortIDAddrs[0]) - assert.NoError(t, err) - - _, err = addUTXO(tvm2.atomicMemory, tvm2.vm.ctx, txID, uint32(i), tvm2.vm.ctx.AVAXAssetID, importAmount, testShortIDAddrs[0]) - assert.NoError(t, err) - } - - // Double the initial base fee used when estimating the cost of this transaction to ensure that when it is - // used in ApricotPhase5 it still pays a sufficient fee with the fixed fee per atomic transaction. - importTx, err := tvm1.vm.newImportTx(tvm1.vm.ctx.XChainID, testEthAddrs[0], new(big.Int).Mul(common.Big2, initialBaseFee), []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, big.NewInt(ap0.MinGasPrice), nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - if err := tvm1.vm.mempool.ForceAddTx(importTx); err != nil { - t.Fatal(err) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm1.toEngine - blk1, err := tvm1.vm.BuildBlock(context.Background()) + <-tvm.ToEngine + + blk, err := vm.BuildBlock(context.Background()) if err != nil { - t.Fatal(err) + t.Fatalf("Failed to build block with transaction: %s", err) } - if err := blk1.Verify(context.Background()); err != nil { + + if err := blk.Verify(context.Background()); err != nil { + t.Fatalf("Block failed verification on VM: %s", err) + } + + if err := vm.SetPreference(context.Background(), blk.ID()); err != nil { t.Fatal(err) } - validEthBlock := blk1.(*chain.BlockWrapper).Block.(*Block).ethBlock + blkHeight := blk.Height() + blkHash := blk.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock.Hash() + + vm.eth.APIBackend.SetAllowUnfinalizedQueries(true) - extraData, err := atomic.Codec.Marshal(atomic.CodecVersion, []*atomic.Tx{importTx}) + ctx := context.Background() + b, err := vm.eth.APIBackend.BlockByNumber(ctx, rpc.BlockNumber(blkHeight)) if err != nil { t.Fatal(err) } + if b.Hash() != blkHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkHeight, blkHash.Hex(), b.Hash().Hex()) + } - // Construct the new block with the extra data in the new format (slice of atomic transactions). - ethBlk2 := customtypes.NewBlockWithExtData( - types.CopyHeader(validEthBlock.Header()), - nil, - nil, - nil, - new(trie.Trie), - extraData, - true, - ) + vm.eth.APIBackend.SetAllowUnfinalizedQueries(false) - state, err := tvm2.vm.blockChain.State() - if err != nil { - t.Fatal(err) + _, err = vm.eth.APIBackend.BlockByNumber(ctx, rpc.BlockNumber(blkHeight)) + if !errors.Is(err, eth.ErrUnfinalizedData) { + t.Fatalf("expected ErrUnfinalizedData but got %s", err.Error()) + } + + if err := blk.Accept(context.Background()); err != nil { + t.Fatalf("VM failed to accept block: %s", err) } - // Hack: test [onExtraStateChange] directly to ensure it catches the atomic gas limit error correctly. - if _, _, err := tvm2.vm.onExtraStateChange(ethBlk2, nil, state); err == nil || !strings.Contains(err.Error(), "exceeds atomic gas limit") { - t.Fatalf("Expected block to fail verification due to exceeded atomic gas limit, but found error: %v", err) + if b := vm.blockChain.GetBlockByNumber(blkHeight); b.Hash() != blkHash { + t.Fatalf("expected block at %d to have hash %s but got %s", blkHeight, blkHash.Hex(), b.Hash().Hex()) } } func TestSkipChainConfigCheckCompatible(t *testing.T) { - importAmount := uint64(50000000) fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) - defer func() { require.NoError(t, tvm.vm.Shutdown(context.Background())) }() // Since rewinding is permitted for last accepted height of 0, we must // accept one block to test the SkipUpgradeCheck functionality. - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) - require.NoError(t, err) - require.NoError(t, tvm.vm.mempool.AddLocalTx(importTx)) - <-tvm.toEngine + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) + if err != nil { + t.Fatal(err) + } + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } + } + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) require.NoError(t, err) require.NoError(t, blk.Verify(context.Background())) - require.NoError(t, tvm.vm.SetPreference(context.Background(), blk.ID())) + require.NoError(t, vm.SetPreference(context.Background(), blk.ID())) require.NoError(t, blk.Accept(context.Background())) - reinitVM := &VM{} + require.NoError(t, vm.Shutdown(context.Background())) + + reinitVM := newDefaultTestVM() // use the block's timestamp instead of 0 since rewind to genesis // is hardcoded to be allowed in core/genesis.go. - newCTX := snowtest.Context(t, tvm.vm.ctx.ChainID) + newCTX := snowtest.Context(t, vm.ctx.ChainID) upgradetest.SetTimesTo(&newCTX.NetworkUpgrades, upgradetest.Latest, upgrade.UnscheduledActivationTime) upgradetest.SetTimesTo(&newCTX.NetworkUpgrades, fork+1, blk.Timestamp()) upgradetest.SetTimesTo(&newCTX.NetworkUpgrades, fork, upgrade.InitiallyActiveTime) - genesis := []byte(genesisJSON(forkToChainConfig[fork])) - err = reinitVM.Initialize(context.Background(), newCTX, tvm.db, genesis, []byte{}, []byte{}, tvm.toEngine, []*commonEng.Fx{}, tvm.appSender) + genesis := []byte(vmtest.GenesisJSON(vmtest.ForkToChainConfig[fork])) + err = reinitVM.Initialize(context.Background(), newCTX, tvm.DB, genesis, []byte{}, []byte{}, tvm.ToEngine, []*commonEng.Fx{}, tvm.AppSender) require.ErrorContains(t, err, "mismatching Cancun fork timestamp in database") - reinitVM = &VM{} - newCTX.Metrics = metrics.NewPrefixGatherer() - // try again with skip-upgrade-check + reinitVM = newDefaultTestVM() + metricstest.ResetMetrics(newCTX) config := []byte(`{"skip-upgrade-check": true}`) - require.NoError(t, reinitVM.Initialize( - context.Background(), - newCTX, - tvm.db, - genesis, - []byte{}, - config, - tvm.toEngine, - []*commonEng.Fx{}, - tvm.appSender)) + require.NoError(t, reinitVM.Initialize(context.Background(), newCTX, tvm.DB, genesis, []byte{}, config, tvm.ToEngine, []*commonEng.Fx{}, tvm.AppSender)) require.NoError(t, reinitVM.Shutdown(context.Background())) } @@ -3768,53 +1936,44 @@ func TestParentBeaconRootBlock(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { - importAmount := uint64(1000000000) - tvm := newVM(t, testVMConfig{ - fork: &test.fork, - utxos: map[ids.ShortID]uint64{ - testShortIDAddrs[0]: importAmount, - }, + fork := test.fork + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) + defer func() { - if err := tvm.vm.Shutdown(context.Background()); err != nil { + if err := vm.Shutdown(context.Background()); err != nil { t.Fatal(err) } }() - importTx, err := tvm.vm.newImportTx(tvm.vm.ctx.XChainID, testEthAddrs[0], initialBaseFee, []*secp256k1.PrivateKey{testKeys[0]}) + tx := types.NewTransaction(uint64(0), vmtest.TestEthAddrs[1], big.NewInt(1), 21000, vmtest.InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) if err != nil { t.Fatal(err) } - - if err := tvm.vm.mempool.AddLocalTx(importTx); err != nil { - t.Fatal(err) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx}) + for i, err := range errs { + if err != nil { + t.Fatalf("Failed to add tx at index %d: %s", i, err) + } } - <-tvm.toEngine + <-tvm.ToEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + blk, err := vm.BuildBlock(context.Background()) if err != nil { - t.Fatalf("Failed to build block with import transaction: %s", err) + t.Fatalf("Failed to build block with transaction: %s", err) } // Modify the block to have a parent beacon root - ethBlock := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock + ethBlock := blk.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock header := types.CopyHeader(ethBlock.Header()) header.ParentBeaconRoot = test.beaconRoot - parentBeaconEthBlock := customtypes.NewBlockWithExtData( - header, - nil, - nil, - nil, - new(trie.Trie), - customtypes.BlockExtData(ethBlock), - false, - ) - - parentBeaconBlock, err := tvm.vm.newBlock(parentBeaconEthBlock) - if err != nil { - t.Fatal(err) - } + parentBeaconEthBlock := ethBlock.WithSeal(header) + + parentBeaconBlock, err := wrapBlock(parentBeaconEthBlock, vm) + require.NoError(t, err) errCheck := func(err error) { if test.expectedError { @@ -3828,7 +1987,7 @@ func TestParentBeaconRootBlock(t *testing.T) { } } - _, err = tvm.vm.ParseBlock(context.Background(), parentBeaconBlock.Bytes()) + _, err = vm.ParseBlock(context.Background(), parentBeaconBlock.Bytes()) errCheck(err) err = parentBeaconBlock.Verify(context.Background()) errCheck(err) @@ -3854,11 +2013,11 @@ func TestNoBlobsAllowed(t *testing.T) { GasTipCap: uint256.NewInt(1), GasFeeCap: uint256.MustFromBig(fee), Gas: params.TxGas, - To: testEthAddrs[0], + To: vmtest.TestEthAddrs[0], BlobFeeCap: uint256.NewInt(1), BlobHashes: []common.Hash{{1}}, // This blob is expected to cause verification to fail Value: new(uint256.Int), - }), signer, testKeys[0].ToECDSA()) + }), signer, vmtest.TestKeys[0].ToECDSA()) require.NoError(err) b.AddTx(tx) } @@ -3867,13 +2026,13 @@ func TestNoBlobsAllowed(t *testing.T) { require.NoError(err) // Create a VM with the genesis (will use header verification) - vm := newVM(t, testVMConfig{ - genesisJSON: genesisJSONCancun, - }).vm + vm, _ := setupDefaultTestVM(t, vmtest.TestVMConfig{ + GenesisJSON: genesisJSONCancun, + }) defer func() { require.NoError(vm.Shutdown(ctx)) }() // Verification should fail - vmBlock, err := vm.newBlock(blocks[0]) + vmBlock, err := wrapBlock(blocks[0], vm) require.NoError(err) _, err = vm.ParseBlock(ctx, vmBlock.Bytes()) require.ErrorContains(err, "blobs not enabled on avalanche networks") diff --git a/plugin/evm/vm_warp_test.go b/plugin/evm/vm_warp_test.go index d01777c9ac..7591f5e0b8 100644 --- a/plugin/evm/vm_warp_test.go +++ b/plugin/evm/vm_warp_test.go @@ -30,11 +30,14 @@ import ( avalancheWarp "github.com/ava-labs/avalanchego/vms/platformvm/warp" "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" "github.com/ava-labs/coreth/eth/tracers" + "github.com/ava-labs/coreth/metrics/metricstest" "github.com/ava-labs/coreth/params" "github.com/ava-labs/coreth/params/extras" customheader "github.com/ava-labs/coreth/plugin/evm/header" "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/message/messagetest" "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" + "github.com/ava-labs/coreth/plugin/evm/vmtest" "github.com/ava-labs/coreth/precompile/contract" warpcontract "github.com/ava-labs/coreth/precompile/contracts/warp" "github.com/ava-labs/coreth/predicate" @@ -68,18 +71,20 @@ const ( signersPrimary ) +var networkCodec = messagetest.BlockSyncSummaryCodec + func TestSendWarpMessage(t *testing.T) { require := require.New(t) fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - require.NoError(tvm.vm.Shutdown(context.Background())) + require.NoError(vm.Shutdown(context.Background())) }() acceptedLogsChan := make(chan []*types.Log, 10) - logsSub := tvm.vm.eth.APIBackend.SubscribeAcceptedLogsEvent(acceptedLogsChan) + logsSub := vm.eth.APIBackend.SubscribeAcceptedLogsEvent(acceptedLogsChan) defer logsSub.Unsubscribe() payloadData := avagoUtils.RandomBytes(100) @@ -87,41 +92,41 @@ func TestSendWarpMessage(t *testing.T) { warpSendMessageInput, err := warpcontract.PackSendWarpMessage(payloadData) require.NoError(err) addressedPayload, err := payload.NewAddressedCall( - testEthAddrs[0].Bytes(), + vmtest.TestEthAddrs[0].Bytes(), payloadData, ) require.NoError(err) expectedUnsignedMessage, err := avalancheWarp.NewUnsignedMessage( - tvm.vm.ctx.NetworkID, - tvm.vm.ctx.ChainID, + vm.ctx.NetworkID, + vm.ctx.ChainID, addressedPayload.Bytes(), ) require.NoError(err) // Submit a transaction to trigger sending a warp message tx0 := types.NewTransaction(uint64(0), warpcontract.ContractAddress, big.NewInt(1), 100_000, big.NewInt(ap0.MinGasPrice), warpSendMessageInput) - signedTx0, err := types.SignTx(tx0, types.LatestSignerForChainID(tvm.vm.chainConfig.ChainID), testKeys[0].ToECDSA()) + signedTx0, err := types.SignTx(tx0, types.LatestSignerForChainID(vm.chainConfig.ChainID), vmtest.TestKeys[0].ToECDSA()) require.NoError(err) - errs := tvm.vm.txPool.AddRemotesSync([]*types.Transaction{signedTx0}) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{signedTx0}) require.NoError(errs[0]) - <-tvm.toEngine - blk, err := tvm.vm.BuildBlock(context.Background()) + <-tvm.ToEngine + blk, err := vm.BuildBlock(context.Background()) require.NoError(err) require.NoError(blk.Verify(context.Background())) // Verify that the constructed block contains the expected log with an unsigned warp message in the log data - ethBlock1 := blk.(*chain.BlockWrapper).Block.(*Block).ethBlock + ethBlock1 := blk.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock require.Len(ethBlock1.Transactions(), 1) - receipts := rawdb.ReadReceipts(tvm.vm.chaindb, ethBlock1.Hash(), ethBlock1.NumberU64(), ethBlock1.Time(), tvm.vm.chainConfig) + receipts := rawdb.ReadReceipts(vm.chaindb, ethBlock1.Hash(), ethBlock1.NumberU64(), ethBlock1.Time(), vm.chainConfig) require.Len(receipts, 1) require.Len(receipts[0].Logs, 1) expectedTopics := []common.Hash{ warpcontract.WarpABI.Events["SendWarpMessage"].ID, - common.BytesToHash(testEthAddrs[0].Bytes()), + common.BytesToHash(vmtest.TestEthAddrs[0].Bytes()), common.Hash(expectedUnsignedMessage.ID()), } require.Equal(expectedTopics, receipts[0].Logs[0].Topics) @@ -130,17 +135,17 @@ func TestSendWarpMessage(t *testing.T) { require.NoError(err) // Verify the signature cannot be fetched before the block is accepted - _, err = tvm.vm.warpBackend.GetMessageSignature(context.TODO(), unsignedMessage) + _, err = vm.warpBackend.GetMessageSignature(context.TODO(), unsignedMessage) require.Error(err) - _, err = tvm.vm.warpBackend.GetBlockSignature(context.TODO(), blk.ID()) + _, err = vm.warpBackend.GetBlockSignature(context.TODO(), blk.ID()) require.Error(err) - require.NoError(tvm.vm.SetPreference(context.Background(), blk.ID())) + require.NoError(vm.SetPreference(context.Background(), blk.ID())) require.NoError(blk.Accept(context.Background())) - tvm.vm.blockChain.DrainAcceptorQueue() + vm.blockChain.DrainAcceptorQueue() // Verify the message signature after accepting the block. - rawSignatureBytes, err := tvm.vm.warpBackend.GetMessageSignature(context.TODO(), unsignedMessage) + rawSignatureBytes, err := vm.warpBackend.GetMessageSignature(context.TODO(), unsignedMessage) require.NoError(err) blsSignature, err := bls.SignatureFromBytes(rawSignatureBytes[:]) require.NoError(err) @@ -154,21 +159,21 @@ func TestSendWarpMessage(t *testing.T) { } // Verify the produced message signature is valid - require.True(bls.Verify(tvm.vm.ctx.PublicKey, blsSignature, unsignedMessage.Bytes())) + require.True(bls.Verify(vm.ctx.PublicKey, blsSignature, unsignedMessage.Bytes())) // Verify the blockID will now be signed by the backend and produces a valid signature. - rawSignatureBytes, err = tvm.vm.warpBackend.GetBlockSignature(context.TODO(), blk.ID()) + rawSignatureBytes, err = vm.warpBackend.GetBlockSignature(context.TODO(), blk.ID()) require.NoError(err) blsSignature, err = bls.SignatureFromBytes(rawSignatureBytes[:]) require.NoError(err) blockHashPayload, err := payload.NewHash(blk.ID()) require.NoError(err) - unsignedMessage, err = avalancheWarp.NewUnsignedMessage(tvm.vm.ctx.NetworkID, tvm.vm.ctx.ChainID, blockHashPayload.Bytes()) + unsignedMessage, err = avalancheWarp.NewUnsignedMessage(vm.ctx.NetworkID, vm.ctx.ChainID, blockHashPayload.Bytes()) require.NoError(err) // Verify the produced message signature is valid - require.True(bls.Verify(tvm.vm.ctx.PublicKey, blsSignature, unsignedMessage.Bytes())) + require.True(bls.Verify(vm.ctx.PublicKey, blsSignature, unsignedMessage.Bytes())) } func TestValidateWarpMessage(t *testing.T) { @@ -263,15 +268,15 @@ func TestValidateInvalidWarpBlockHash(t *testing.T) { func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.UnsignedMessage, validSignature bool, txPayload []byte) { require := require.New(t) fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - require.NoError(tvm.vm.Shutdown(context.Background())) + require.NoError(vm.Shutdown(context.Background())) }() acceptedLogsChan := make(chan []*types.Log, 10) - logsSub := tvm.vm.eth.APIBackend.SubscribeAcceptedLogsEvent(acceptedLogsChan) + logsSub := vm.eth.APIBackend.SubscribeAcceptedLogsEvent(acceptedLogsChan) defer logsSub.Unsubscribe() nodeID1 := ids.GenerateTestNodeID() @@ -294,7 +299,7 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned minimumValidPChainHeight := uint64(10) getValidatorSetTestErr := errors.New("can't get validator set test error") - tvm.vm.ctx.ValidatorState = &validatorstest.State{ + vm.ctx.ValidatorState = &validatorstest.State{ // TODO: test both Primary Network / C-Chain and non-Primary Network GetSubnetIDF: func(ctx context.Context, chainID ids.ID) (ids.ID, error) { return ids.Empty, nil @@ -337,15 +342,15 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned createTx, err := types.SignTx( types.NewContractCreation(0, common.Big0, 7_000_000, big.NewInt(225*utils.GWei), common.Hex2Bytes(exampleWarpBin)), - types.LatestSignerForChainID(tvm.vm.chainConfig.ChainID), - testKeys[0].ToECDSA(), + types.LatestSignerForChainID(vm.chainConfig.ChainID), + vmtest.TestKeys[0].ToECDSA(), ) require.NoError(err) - exampleWarpAddress := crypto.CreateAddress(testEthAddrs[0], 0) + exampleWarpAddress := crypto.CreateAddress(vmtest.TestEthAddrs[0], 0) tx, err := types.SignTx( predicate.NewPredicateTx( - tvm.vm.chainConfig.ChainID, + vm.chainConfig.ChainID, 1, &exampleWarpAddress, 1_000_000, @@ -357,11 +362,11 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned warpcontract.ContractAddress, signedMessage.Bytes(), ), - types.LatestSignerForChainID(tvm.vm.chainConfig.ChainID), - testKeys[0].ToECDSA(), + types.LatestSignerForChainID(vm.chainConfig.ChainID), + vmtest.TestKeys[0].ToECDSA(), ) require.NoError(err) - errs := tvm.vm.txPool.AddRemotesSync([]*types.Transaction{createTx, tx}) + errs := vm.txPool.AddRemotesSync([]*types.Transaction{createTx, tx}) for i, err := range errs { require.NoError(err, "failed to add tx at index %d", i) } @@ -373,10 +378,10 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned if validSignature { blockCtx.PChainHeight = minimumValidPChainHeight } - tvm.vm.clock.Set(tvm.vm.clock.Time().Add(2 * time.Second)) - <-tvm.toEngine + vm.clock.Set(vm.clock.Time().Add(2 * time.Second)) + <-tvm.ToEngine - warpBlock, err := tvm.vm.BuildBlockWithContext(context.Background(), blockCtx) + warpBlock, err := vm.BuildBlockWithContext(context.Background(), blockCtx) require.NoError(err) warpBlockVerifyWithCtx, ok := warpBlock.(block.WithVerifyContext) @@ -385,18 +390,18 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned require.NoError(err) require.True(shouldVerifyWithCtx) require.NoError(warpBlockVerifyWithCtx.VerifyWithContext(context.Background(), blockCtx)) - require.NoError(tvm.vm.SetPreference(context.Background(), warpBlock.ID())) + require.NoError(vm.SetPreference(context.Background(), warpBlock.ID())) require.NoError(warpBlock.Accept(context.Background())) - tvm.vm.blockChain.DrainAcceptorQueue() + vm.blockChain.DrainAcceptorQueue() - ethBlock := warpBlock.(*chain.BlockWrapper).Block.(*Block).ethBlock - verifiedMessageReceipts := tvm.vm.blockChain.GetReceiptsByHash(ethBlock.Hash()) + ethBlock := warpBlock.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock + verifiedMessageReceipts := vm.blockChain.GetReceiptsByHash(ethBlock.Hash()) require.Len(verifiedMessageReceipts, 2) for i, receipt := range verifiedMessageReceipts { require.Equal(types.ReceiptStatusSuccessful, receipt.Status, "index: %d", i) } - tracerAPI := tracers.NewAPI(tvm.vm.eth.APIBackend) + tracerAPI := tracers.NewAPI(vm.eth.APIBackend) txTraceResults, err := tracerAPI.TraceBlockByHash(context.Background(), ethBlock.Hash(), nil) require.NoError(err) require.Len(txTraceResults, 2) @@ -416,11 +421,11 @@ func testWarpVMTransaction(t *testing.T, unsignedMessage *avalancheWarp.Unsigned func TestReceiveWarpMessage(t *testing.T) { require := require.New(t) fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - require.NoError(tvm.vm.Shutdown(context.Background())) + require.NoError(vm.Shutdown(context.Background())) }() // enable warp at the default genesis time @@ -439,7 +444,7 @@ func TestReceiveWarpMessage(t *testing.T) { true, // RequirePrimaryNetworkSigners ) - tvm.vm.chainConfigExtra().UpgradeConfig = extras.UpgradeConfig{ + vm.chainConfigExtra().UpgradeConfig = extras.UpgradeConfig{ PrecompileUpgrades: []extras.PrecompileUpgrade{ {Config: enableConfig}, {Config: disableConfig}, @@ -459,7 +464,7 @@ func TestReceiveWarpMessage(t *testing.T) { tests := []test{ { name: "subnet message should be signed by subnet without RequirePrimaryNetworkSigners", - sourceChainID: tvm.vm.ctx.ChainID, + sourceChainID: vm.ctx.ChainID, msgFrom: fromSubnet, useSigners: signersSubnet, blockTime: upgrade.InitiallyActiveTime, @@ -473,7 +478,7 @@ func TestReceiveWarpMessage(t *testing.T) { }, { name: "C-Chain message should be signed by subnet without RequirePrimaryNetworkSigners", - sourceChainID: tvm.vm.ctx.CChainID, + sourceChainID: vm.ctx.CChainID, msgFrom: fromPrimary, useSigners: signersSubnet, blockTime: upgrade.InitiallyActiveTime.Add(2 * blockGap), @@ -482,7 +487,7 @@ func TestReceiveWarpMessage(t *testing.T) { // by using reEnableTime. { name: "subnet message should be signed by subnet with RequirePrimaryNetworkSigners (unimpacted)", - sourceChainID: tvm.vm.ctx.ChainID, + sourceChainID: vm.ctx.ChainID, msgFrom: fromSubnet, useSigners: signersSubnet, blockTime: reEnableTime, @@ -496,7 +501,7 @@ func TestReceiveWarpMessage(t *testing.T) { }, { name: "C-Chain message should be signed by primary with RequirePrimaryNetworkSigners (impacted)", - sourceChainID: tvm.vm.ctx.CChainID, + sourceChainID: vm.ctx.CChainID, msgFrom: fromPrimary, useSigners: signersPrimary, blockTime: reEnableTime.Add(2 * blockGap), @@ -507,7 +512,7 @@ func TestReceiveWarpMessage(t *testing.T) { for _, test := range tests { t.Run(test.name, func(t *testing.T) { testReceiveWarpMessage( - t, tvm.toEngine, tvm.vm, test.sourceChainID, test.msgFrom, test.useSigners, test.blockTime, + t, tvm.ToEngine, vm, test.sourceChainID, test.msgFrom, test.useSigners, test.blockTime, ) }) } @@ -522,7 +527,7 @@ func testReceiveWarpMessage( require := require.New(t) payloadData := avagoUtils.RandomBytes(100) addressedPayload, err := payload.NewAddressedCall( - testEthAddrs[0].Bytes(), + vmtest.TestEthAddrs[0].Bytes(), payloadData, ) require.NoError(err) @@ -630,7 +635,7 @@ func testReceiveWarpMessage( getVerifiedWarpMessageTx, err := types.SignTx( predicate.NewPredicateTx( vm.chainConfig.ChainID, - vm.txPool.Nonce(testEthAddrs[0]), + vm.txPool.Nonce(vmtest.TestEthAddrs[0]), &warpcontract.Module.Address, 1_000_000, big.NewInt(225*utils.GWei), @@ -642,7 +647,7 @@ func testReceiveWarpMessage( signedMessage.Bytes(), ), types.LatestSignerForChainID(vm.chainConfig.ChainID), - testKeys[0].ToECDSA(), + vmtest.TestKeys[0].ToECDSA(), ) require.NoError(err) errs := vm.txPool.AddRemotesSync([]*types.Transaction{getVerifiedWarpMessageTx}) @@ -661,7 +666,7 @@ func testReceiveWarpMessage( require.NoError(err) // Require the block was built with a successful predicate result - ethBlock := block2.(*chain.BlockWrapper).Block.(*Block).ethBlock + ethBlock := block2.(*chain.BlockWrapper).Block.(*wrappedBlock).ethBlock rules := params.GetExtra(vm.chainConfig).GetAvalancheRules(ethBlock.Time()) headerPredicateResultsBytes := customheader.PredicateBytesFromExtra(rules, ethBlock.Extra()) results, err := predicate.ParseResults(headerPredicateResultsBytes) @@ -707,7 +712,7 @@ func testReceiveWarpMessage( expectedOutput, err := warpcontract.PackGetVerifiedWarpMessageOutput(warpcontract.GetVerifiedWarpMessageOutput{ Message: warpcontract.WarpMessage{ SourceChainID: common.Hash(sourceChainID), - OriginSenderAddress: testEthAddrs[0], + OriginSenderAddress: vmtest.TestEthAddrs[0], Payload: payloadData, }, Valid: true, @@ -733,20 +738,20 @@ func testReceiveWarpMessage( func TestMessageSignatureRequestsToVM(t *testing.T) { fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - require.NoError(t, tvm.vm.Shutdown(context.Background())) + require.NoError(t, vm.Shutdown(context.Background())) }() // Generate a new warp unsigned message and add to warp backend - warpMessage, err := avalancheWarp.NewUnsignedMessage(tvm.vm.ctx.NetworkID, tvm.vm.ctx.ChainID, []byte{1, 2, 3}) + warpMessage, err := avalancheWarp.NewUnsignedMessage(vm.ctx.NetworkID, vm.ctx.ChainID, []byte{1, 2, 3}) require.NoError(t, err) // Add the known message and get its signature to confirm. - require.NoError(t, tvm.vm.warpBackend.AddMessage(warpMessage)) - signature, err := tvm.vm.warpBackend.GetMessageSignature(context.TODO(), warpMessage) + require.NoError(t, vm.warpBackend.AddMessage(warpMessage)) + signature, err := vm.warpBackend.GetMessageSignature(context.TODO(), warpMessage) require.NoError(t, err) var knownSignature [bls.SignatureLen]byte copy(knownSignature[:], signature) @@ -767,10 +772,10 @@ func TestMessageSignatureRequestsToVM(t *testing.T) { for name, test := range tests { calledSendAppResponseFn := false - tvm.appSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, responseBytes []byte) error { + tvm.AppSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, responseBytes []byte) error { calledSendAppResponseFn = true var response message.SignatureResponse - _, err := message.Codec.Unmarshal(responseBytes, &response) + _, err := networkCodec.Unmarshal(responseBytes, &response) require.NoError(t, err) require.Equal(t, test.expectedResponse, response.Signature) @@ -781,12 +786,12 @@ func TestMessageSignatureRequestsToVM(t *testing.T) { MessageID: test.messageID, } - requestBytes, err := message.Codec.Marshal(message.Version, &signatureRequest) + requestBytes, err := networkCodec.Marshal(message.Version, &signatureRequest) require.NoError(t, err) // Send the app request and make sure we called SendAppResponseFn deadline := time.Now().Add(60 * time.Second) - require.NoError(t, tvm.vm.Network.AppRequest(context.Background(), ids.GenerateTestNodeID(), 1, deadline, requestBytes)) + require.NoError(t, vm.Network.AppRequest(context.Background(), ids.GenerateTestNodeID(), 1, deadline, requestBytes)) require.True(t, calledSendAppResponseFn) }) } @@ -794,17 +799,17 @@ func TestMessageSignatureRequestsToVM(t *testing.T) { func TestBlockSignatureRequestsToVM(t *testing.T) { fork := upgradetest.Durango - tvm := newVM(t, testVMConfig{ - fork: &fork, + vm, tvm := setupDefaultTestVM(t, vmtest.TestVMConfig{ + Fork: &fork, }) defer func() { - require.NoError(t, tvm.vm.Shutdown(context.Background())) + require.NoError(t, vm.Shutdown(context.Background())) }() - lastAcceptedID, err := tvm.vm.LastAccepted(context.Background()) + lastAcceptedID, err := vm.LastAccepted(context.Background()) require.NoError(t, err) - signature, err := tvm.vm.warpBackend.GetBlockSignature(context.TODO(), lastAcceptedID) + signature, err := vm.warpBackend.GetBlockSignature(context.TODO(), lastAcceptedID) require.NoError(t, err) var knownSignature [bls.SignatureLen]byte copy(knownSignature[:], signature) @@ -825,10 +830,10 @@ func TestBlockSignatureRequestsToVM(t *testing.T) { for name, test := range tests { calledSendAppResponseFn := false - tvm.appSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, responseBytes []byte) error { + tvm.AppSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, responseBytes []byte) error { calledSendAppResponseFn = true var response message.SignatureResponse - _, err := message.Codec.Unmarshal(responseBytes, &response) + _, err := networkCodec.Unmarshal(responseBytes, &response) require.NoError(t, err) require.Equal(t, test.expectedResponse, response.Signature) @@ -839,30 +844,21 @@ func TestBlockSignatureRequestsToVM(t *testing.T) { BlockID: test.blockID, } - requestBytes, err := message.Codec.Marshal(message.Version, &signatureRequest) + requestBytes, err := networkCodec.Marshal(message.Version, &signatureRequest) require.NoError(t, err) // Send the app request and make sure we called SendAppResponseFn deadline := time.Now().Add(60 * time.Second) - require.NoError(t, tvm.vm.Network.AppRequest(context.Background(), ids.GenerateTestNodeID(), 1, deadline, requestBytes)) + require.NoError(t, vm.Network.AppRequest(context.Background(), ids.GenerateTestNodeID(), 1, deadline, requestBytes)) require.True(t, calledSendAppResponseFn) }) } } func TestClearWarpDB(t *testing.T) { - ctx, db, genesisBytes, issuer, _ := setupGenesis(t, upgradetest.Latest) - vm := &VM{} - require.NoError(t, vm.Initialize( - context.Background(), - ctx, - db, - genesisBytes, - []byte{}, - []byte{}, - issuer, - []*commonEng.Fx{}, - &enginetest.Sender{})) + ctx, db, genesisBytes, issuer, _ := vmtest.SetupGenesis(t, upgradetest.Latest) + vm := newDefaultTestVM() + require.NoError(t, vm.Initialize(context.Background(), ctx, db, genesisBytes, []byte{}, []byte{}, issuer, []*commonEng.Fx{}, &enginetest.Sender{})) // use multiple messages to test that all messages get cleared payloads := [][]byte{[]byte("test1"), []byte("test2"), []byte("test3"), []byte("test4"), []byte("test5")} @@ -882,19 +878,10 @@ func TestClearWarpDB(t *testing.T) { require.NoError(t, vm.Shutdown(context.Background())) // Restart VM with the same database default should not prune the warp db - vm = &VM{} - // we need new context since the previous one has registered metrics. - ctx, _, _, _, _ = setupGenesis(t, upgradetest.Latest) - require.NoError(t, vm.Initialize( - context.Background(), - ctx, - db, - genesisBytes, - []byte{}, - []byte{}, - issuer, - []*commonEng.Fx{}, - &enginetest.Sender{})) + vm = newDefaultTestVM() + // we need to reset context since the previous one has registered metrics. + metricstest.ResetMetrics(ctx) + require.NoError(t, vm.Initialize(context.Background(), ctx, db, genesisBytes, []byte{}, []byte{}, issuer, []*commonEng.Fx{}, &enginetest.Sender{})) // check messages are still present for _, message := range messages { @@ -906,19 +893,10 @@ func TestClearWarpDB(t *testing.T) { require.NoError(t, vm.Shutdown(context.Background())) // restart the VM with pruning enabled - vm = &VM{} + vm = newDefaultTestVM() config := `{"prune-warp-db-enabled": true}` - ctx, _, _, _, _ = setupGenesis(t, upgradetest.Latest) - require.NoError(t, vm.Initialize( - context.Background(), - ctx, - db, - genesisBytes, - []byte{}, - []byte(config), - issuer, - []*commonEng.Fx{}, - &enginetest.Sender{})) + metricstest.ResetMetrics(ctx) + require.NoError(t, vm.Initialize(context.Background(), ctx, db, genesisBytes, []byte{}, []byte(config), issuer, []*commonEng.Fx{}, &enginetest.Sender{})) it := vm.warpDB.NewIterator() require.False(t, it.Next()) diff --git a/plugin/evm/vmerrors/errors.go b/plugin/evm/vmerrors/errors.go new file mode 100644 index 0000000000..444ead5dd6 --- /dev/null +++ b/plugin/evm/vmerrors/errors.go @@ -0,0 +1,12 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmerrors + +import "errors" + +var ( + ErrGenerateBlockFailed = errors.New("failed to generate block") + ErrBlockVerificationFailed = errors.New("failed to verify block") + ErrWrapBlockFailed = errors.New("failed to wrap block") +) diff --git a/plugin/evm/vmtest/genesis.go b/plugin/evm/vmtest/genesis.go new file mode 100644 index 0000000000..f0f17c7a65 --- /dev/null +++ b/plugin/evm/vmtest/genesis.go @@ -0,0 +1,149 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package vmtest + +import ( + "encoding/json" + "math/big" + "testing" + + avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" + "github.com/ava-labs/avalanchego/database/memdb" + "github.com/ava-labs/avalanchego/database/prefixdb" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/snowtest" + "github.com/ava-labs/avalanchego/upgrade" + "github.com/ava-labs/avalanchego/upgrade/upgradetest" + "github.com/ava-labs/avalanchego/utils/crypto/secp256k1" + + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap3" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/types" +) + +var ( + TestKeys = secp256k1.TestKeys()[:3] + TestEthAddrs []common.Address // testEthAddrs[i] corresponds to testKeys[i] + TestShortIDAddrs []ids.ShortID + InitialBaseFee = big.NewInt(ap3.InitialBaseFee) + + ForkToChainConfig = map[upgradetest.Fork]*params.ChainConfig{ + upgradetest.NoUpgrades: params.TestLaunchConfig, + upgradetest.ApricotPhase1: params.TestApricotPhase1Config, + upgradetest.ApricotPhase2: params.TestApricotPhase2Config, + upgradetest.ApricotPhase3: params.TestApricotPhase3Config, + upgradetest.ApricotPhase4: params.TestApricotPhase4Config, + upgradetest.ApricotPhase5: params.TestApricotPhase5Config, + upgradetest.ApricotPhasePre6: params.TestApricotPhasePre6Config, + upgradetest.ApricotPhase6: params.TestApricotPhase6Config, + upgradetest.ApricotPhasePost6: params.TestApricotPhasePost6Config, + upgradetest.Banff: params.TestBanffChainConfig, + upgradetest.Cortina: params.TestCortinaChainConfig, + upgradetest.Durango: params.TestDurangoChainConfig, + upgradetest.Etna: params.TestEtnaChainConfig, + upgradetest.Fortuna: params.TestFortunaChainConfig, + } +) + +func init() { + for _, pk := range TestKeys { + TestEthAddrs = append(TestEthAddrs, pk.EthAddress()) + TestShortIDAddrs = append(TestShortIDAddrs, pk.Address()) + } +} + +// GenesisJSON returns the JSON representation of the genesis block +// for the given chain configuration, with pre-funded accounts. +func GenesisJSON(cfg *params.ChainConfig) string { + g := new(core.Genesis) + g.Difficulty = big.NewInt(0) + g.GasLimit = 0x5f5e100 + g.Timestamp = uint64(upgrade.InitiallyActiveTime.Unix()) + + // Use chainId: 43111, so that it does not overlap with any Avalanche ChainIDs, which may have their + // config overridden in vm.Initialize. + cpy := *cfg + cpy.ChainID = big.NewInt(43111) + g.Config = &cpy + + allocStr := `{"0100000000000000000000000000000000000000":{"code":"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033","balance":"0x0"}}` + json.Unmarshal([]byte(allocStr), &g.Alloc) + // After Durango, an additional account is funded in tests to use + // with warp messages. + if params.GetExtra(cfg).IsDurango(0) { + addr := common.HexToAddress("0x99b9DEA54C48Dfea6aA9A4Ca4623633EE04ddbB5") + balance := new(big.Int).Mul(big.NewInt(params.Ether), big.NewInt(10)) + g.Alloc[addr] = types.Account{Balance: balance} + } + + // Fund the test keys + var b []byte + for _, ethAddr := range TestEthAddrs { + balance := new(big.Int).Mul(big.NewInt(params.Ether), big.NewInt(10)) + g.Alloc[ethAddr] = types.Account{Balance: balance} + } + + b, err := json.Marshal(g) + if err != nil { + panic(err) + } + return string(b) +} + +func NewPrefundedGenesis( + balance int, + addresses ...common.Address, +) *core.Genesis { + alloc := types.GenesisAlloc{} + for _, address := range addresses { + alloc[address] = types.Account{ + Balance: big.NewInt(int64(balance)), + } + } + + return &core.Genesis{ + Config: params.TestChainConfig, + Difficulty: big.NewInt(0), + Alloc: alloc, + } +} + +// SetupGenesis sets up the genesis +func SetupGenesis( + t *testing.T, + fork upgradetest.Fork, +) (*snow.Context, + *prefixdb.Database, + []byte, + chan commonEng.Message, + *avalancheatomic.Memory, +) { + ctx := snowtest.Context(t, snowtest.CChainID) + ctx.NetworkUpgrades = upgradetest.GetConfig(fork) + + baseDB := memdb.New() + + // initialize the atomic memory + atomicMemory := avalancheatomic.NewMemory(prefixdb.New([]byte{0}, baseDB)) + ctx.SharedMemory = atomicMemory.NewSharedMemory(ctx.ChainID) + + // NB: this lock is intentionally left locked when this function returns. + // The caller of this function is responsible for unlocking. + ctx.Lock.Lock() + + issuer := make(chan commonEng.Message, 1) + prefixedDB := prefixdb.New([]byte{1}, baseDB) + genesisJSON := GenesisJSON(ForkToChainConfig[fork]) + return ctx, prefixedDB, []byte(genesisJSON), issuer, atomicMemory +} + +func ForkToRules(fork upgradetest.Fork) *extras.Rules { + chainConfig := ForkToChainConfig[fork] + return params.GetRulesExtra(chainConfig.Rules(common.Big0, params.IsMergeTODO, 0)) +} diff --git a/plugin/evm/vmtest/test_syncervm.go b/plugin/evm/vmtest/test_syncervm.go new file mode 100644 index 0000000000..e41da9ed1b --- /dev/null +++ b/plugin/evm/vmtest/test_syncervm.go @@ -0,0 +1,690 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package vmtest + +import ( + "context" + "fmt" + "math/big" + "math/rand" + "sync" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" + avalanchedatabase "github.com/ava-labs/avalanchego/database" + "github.com/ava-labs/avalanchego/database/prefixdb" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/engine/enginetest" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" + "github.com/ava-labs/avalanchego/upgrade/upgradetest" + "github.com/ava-labs/avalanchego/utils/set" + "github.com/ava-labs/avalanchego/vms/components/chain" + + "github.com/ava-labs/coreth/consensus/dummy" + "github.com/ava-labs/coreth/constants" + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/core/coretest" + "github.com/ava-labs/coreth/metrics/metricstest" + "github.com/ava-labs/coreth/plugin/evm/customrawdb" + "github.com/ava-labs/coreth/plugin/evm/customtypes" + "github.com/ava-labs/coreth/plugin/evm/database" + "github.com/ava-labs/coreth/plugin/evm/extension" + vmsync "github.com/ava-labs/coreth/plugin/evm/sync" + "github.com/ava-labs/coreth/predicate" + statesyncclient "github.com/ava-labs/coreth/sync/client" + "github.com/ava-labs/coreth/sync/statesync/statesynctest" + "github.com/ava-labs/coreth/utils/utilstest" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/rawdb" + "github.com/ava-labs/libevm/core/types" + "github.com/ava-labs/libevm/ethdb" + "github.com/ava-labs/libevm/log" + "github.com/ava-labs/libevm/rlp" + "github.com/ava-labs/libevm/trie" +) + +type SyncerVMTest struct { + Name string + TestFunc func( + t *testing.T, + testSetup *SyncTestSetup, + ) +} + +var SyncerVMTests = []SyncerVMTest{ + { + Name: "SkipStateSyncTest", + TestFunc: SkipStateSyncTest, + }, + { + Name: "StateSyncFromScratchTest", + TestFunc: StateSyncFromScratchTest, + }, + { + Name: "StateSyncFromScratchExceedParentTest", + TestFunc: StateSyncFromScratchExceedParentTest, + }, + { + Name: "StateSyncToggleEnabledToDisabledTest", + TestFunc: StateSyncToggleEnabledToDisabledTest, + }, + { + Name: "VMShutdownWhileSyncingTest", + TestFunc: VMShutdownWhileSyncingTest, + }, +} + +func SkipStateSyncTest(t *testing.T, testSetup *SyncTestSetup) { + rand.Seed(1) + test := SyncTestParams{ + SyncableInterval: 256, + StateSyncMinBlocks: 300, // must be greater than [syncableInterval] to skip sync + SyncMode: block.StateSyncSkipped, + } + testSyncVMSetup := initSyncServerAndClientVMs(t, test, vmsync.ParentsToFetch, testSetup) + + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) +} + +func StateSyncFromScratchTest(t *testing.T, testSetup *SyncTestSetup) { + rand.Seed(1) + test := SyncTestParams{ + SyncableInterval: 256, + StateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync + SyncMode: block.StateSyncStatic, + } + testSyncVMSetup := initSyncServerAndClientVMs(t, test, vmsync.ParentsToFetch, testSetup) + + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) +} + +func StateSyncFromScratchExceedParentTest(t *testing.T, testSetup *SyncTestSetup) { + rand.Seed(1) + numToGen := vmsync.ParentsToFetch + uint64(32) + test := SyncTestParams{ + SyncableInterval: numToGen, + StateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync + SyncMode: block.StateSyncStatic, + } + testSyncVMSetup := initSyncServerAndClientVMs(t, test, int(numToGen), testSetup) + + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) +} + +func StateSyncToggleEnabledToDisabledTest(t *testing.T, testSetup *SyncTestSetup) { + rand.Seed(1) + var lock sync.Mutex + reqCount := 0 + test := SyncTestParams{ + SyncableInterval: 256, + StateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync + SyncMode: block.StateSyncStatic, + responseIntercept: func(syncerVM extension.InnerVM, nodeID ids.NodeID, requestID uint32, response []byte) { + lock.Lock() + defer lock.Unlock() + + reqCount++ + // Fail all requests after number 50 to interrupt the sync + if reqCount > 50 { + if err := syncerVM.AppRequestFailed(context.Background(), nodeID, requestID, commonEng.ErrTimeout); err != nil { + panic(err) + } + if err := syncerVM.SyncerClient().Shutdown(); err != nil { + panic(err) + } + } else { + require.NoError(t, syncerVM.AppResponse(context.Background(), nodeID, requestID, response)) + } + }, + expectedErr: context.Canceled, + } + testSyncVMSetup := initSyncServerAndClientVMs(t, test, vmsync.ParentsToFetch, testSetup) + + // Perform sync resulting in early termination. + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) + + test.SyncMode = block.StateSyncStatic + test.responseIntercept = nil + test.expectedErr = nil + + syncDisabledVM, _ := testSetup.NewVM() + appSender := &enginetest.Sender{T: t} + appSender.SendAppGossipF = func(context.Context, commonEng.SendConfig, []byte) error { return nil } + appSender.SendAppRequestF = func(ctx context.Context, nodeSet set.Set[ids.NodeID], requestID uint32, request []byte) error { + nodeID, hasItem := nodeSet.Pop() + if !hasItem { + t.Fatal("expected nodeSet to contain at least 1 nodeID") + } + go testSyncVMSetup.serverVM.VM.AppRequest(ctx, nodeID, requestID, time.Now().Add(1*time.Second), request) + return nil + } + metricstest.ResetMetrics(testSyncVMSetup.syncerVM.SnowCtx) + stateSyncDisabledConfigJSON := `{"state-sync-enabled":false}` + genesisJSON := []byte(GenesisJSON(ForkToChainConfig[upgradetest.Latest])) + if err := syncDisabledVM.Initialize( + context.Background(), + testSyncVMSetup.syncerVM.SnowCtx, + testSyncVMSetup.syncerVM.DB, + genesisJSON, + nil, + []byte(stateSyncDisabledConfigJSON), + testSyncVMSetup.syncerVM.EngineChan, + []*commonEng.Fx{}, + appSender, + ); err != nil { + t.Fatal(err) + } + + defer func() { + if err := syncDisabledVM.Shutdown(context.Background()); err != nil { + t.Fatal(err) + } + }() + + if height := syncDisabledVM.LastAcceptedVMBlock().Height(); height != 0 { + t.Fatalf("Unexpected last accepted height: %d", height) + } + + enabled, err := syncDisabledVM.StateSyncEnabled(context.Background()) + assert.NoError(t, err) + assert.False(t, enabled, "sync should be disabled") + + // Process the first 10 blocks from the serverVM + for i := uint64(1); i < 10; i++ { + ethBlock := testSyncVMSetup.serverVM.VM.Ethereum().BlockChain().GetBlockByNumber(i) + if ethBlock == nil { + t.Fatalf("VM Server did not have a block available at height %d", i) + } + b, err := rlp.EncodeToBytes(ethBlock) + if err != nil { + t.Fatal(err) + } + blk, err := syncDisabledVM.ParseBlock(context.Background(), b) + if err != nil { + t.Fatal(err) + } + if err := blk.Verify(context.Background()); err != nil { + t.Fatal(err) + } + if err := blk.Accept(context.Background()); err != nil { + t.Fatal(err) + } + } + // Verify the snapshot disk layer matches the last block root + lastRoot := syncDisabledVM.Ethereum().BlockChain().CurrentBlock().Root + if err := syncDisabledVM.Ethereum().BlockChain().Snapshots().Verify(lastRoot); err != nil { + t.Fatal(err) + } + syncDisabledVM.Ethereum().BlockChain().DrainAcceptorQueue() + + // Create a new VM from the same database with state sync enabled. + syncReEnabledVM, _ := testSetup.NewVM() + // Enable state sync in configJSON + configJSON := fmt.Sprintf( + `{"state-sync-enabled":true, "state-sync-min-blocks":%d}`, + test.StateSyncMinBlocks, + ) + metricstest.ResetMetrics(testSyncVMSetup.syncerVM.SnowCtx) + if err := syncReEnabledVM.Initialize( + context.Background(), + testSyncVMSetup.syncerVM.SnowCtx, + testSyncVMSetup.syncerVM.DB, + genesisJSON, + nil, + []byte(configJSON), + testSyncVMSetup.syncerVM.EngineChan, + []*commonEng.Fx{}, + appSender, + ); err != nil { + t.Fatal(err) + } + + // override [serverVM]'s SendAppResponse function to trigger AppResponse on [syncerVM] + testSyncVMSetup.serverVM.AppSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { + if test.responseIntercept == nil { + go syncReEnabledVM.AppResponse(ctx, nodeID, requestID, response) + } else { + go test.responseIntercept(syncReEnabledVM, nodeID, requestID, response) + } + + return nil + } + + // connect peer to [syncerVM] + assert.NoError(t, syncReEnabledVM.Connected( + context.Background(), + testSyncVMSetup.serverVM.SnowCtx.NodeID, + statesyncclient.StateSyncVersion, + )) + + enabled, err = syncReEnabledVM.StateSyncEnabled(context.Background()) + assert.NoError(t, err) + assert.True(t, enabled, "sync should be enabled") + + testSyncVMSetup.syncerVM.VM = syncReEnabledVM + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) +} + +func VMShutdownWhileSyncingTest(t *testing.T, testSetup *SyncTestSetup) { + var ( + lock sync.Mutex + testSyncVMSetup *testSyncVMSetup + ) + reqCount := 0 + test := SyncTestParams{ + SyncableInterval: 256, + StateSyncMinBlocks: 50, // must be less than [syncableInterval] to perform sync + SyncMode: block.StateSyncStatic, + responseIntercept: func(syncerVM extension.InnerVM, nodeID ids.NodeID, requestID uint32, response []byte) { + lock.Lock() + defer lock.Unlock() + + reqCount++ + // Shutdown the VM after 50 requests to interrupt the sync + if reqCount == 50 { + // Note this verifies the VM shutdown does not time out while syncing. + require.NoError(t, testSyncVMSetup.syncerVM.shutdownOnceSyncerVM.Shutdown(context.Background())) + } else if reqCount < 50 { + require.NoError(t, syncerVM.AppResponse(context.Background(), nodeID, requestID, response)) + } + }, + expectedErr: context.Canceled, + } + testSyncVMSetup = initSyncServerAndClientVMs(t, test, vmsync.ParentsToFetch, testSetup) + // Perform sync resulting in early termination. + testSyncerVM(t, testSyncVMSetup, test, testSetup.ExtraSyncerVMTest) +} + +type SyncTestSetup struct { + NewVM func() (extension.InnerVM, dummy.ConsensusCallbacks) // should not be initialized + AfterInit func(t *testing.T, testParams SyncTestParams, vmSetup SyncVMSetup, isServer bool) + GenFn func(i int, vm extension.InnerVM, gen *core.BlockGen) + ExtraSyncerVMTest func(t *testing.T, syncerVM SyncVMSetup) +} + +func initSyncServerAndClientVMs(t *testing.T, test SyncTestParams, numBlocks int, testSetup *SyncTestSetup) *testSyncVMSetup { + require := require.New(t) + + // override commitInterval so the call to trie creates a commit at the height [syncableInterval]. + // This is necessary to support fetching a state summary. + config := fmt.Sprintf(`{"commit-interval": %d, "state-sync-commit-interval": %d}`, test.SyncableInterval, test.SyncableInterval) + serverVM, cb := testSetup.NewVM() + fork := upgradetest.Latest + serverTest := SetupTestVM(t, serverVM, TestVMConfig{ + Fork: &fork, + ConfigJSON: config, + }) + t.Cleanup(func() { + log.Info("Shutting down server VM") + require.NoError(serverVM.Shutdown(context.Background())) + }) + serverVmSetup := SyncVMSetup{ + VM: serverVM, + AppSender: serverTest.AppSender, + SnowCtx: serverTest.Ctx, + ConsensusCallbacks: cb, + DB: serverTest.DB, + EngineChan: serverTest.ToEngine, + AtomicMemory: serverTest.AtomicMemory, + } + var err error + if testSetup.AfterInit != nil { + testSetup.AfterInit(t, test, serverVmSetup, true) + } + generateAndAcceptBlocks(t, serverVM, numBlocks, testSetup.GenFn, nil, cb) + + // make some accounts + root, accounts := statesynctest.FillAccountsWithOverlappingStorage(t, serverVM.Ethereum().BlockChain().TrieDB(), types.EmptyRootHash, 1000, 16) + + // patch serverVM's lastAcceptedBlock to have the new root + // and update the vm's state so the trie with accounts will + // be returned by StateSyncGetLastSummary + lastAccepted := serverVM.Ethereum().BlockChain().LastAcceptedBlock() + patchedBlock := patchBlock(lastAccepted, root, serverVM.Ethereum().ChainDb()) + blockBytes, err := rlp.EncodeToBytes(patchedBlock) + require.NoError(err) + internalWrappedBlock, err := serverVM.ParseBlock(context.Background(), blockBytes) + require.NoError(err) + internalBlock, ok := internalWrappedBlock.(*chain.BlockWrapper) + require.True(ok) + require.NoError(serverVM.SetLastAcceptedBlock(internalBlock.Block)) + + // initialise [syncerVM] with blank genesis state + // we also override [syncerVM]'s commit interval so the atomic trie works correctly. + stateSyncEnabledJSON := fmt.Sprintf(`{"state-sync-enabled":true, "state-sync-min-blocks": %d, "tx-lookup-limit": %d, "commit-interval": %d}`, test.StateSyncMinBlocks, 4, test.SyncableInterval) + + syncerVM, syncerCB := testSetup.NewVM() + syncerTest := SetupTestVM(t, syncerVM, TestVMConfig{ + Fork: &fork, + ConfigJSON: stateSyncEnabledJSON, + IsSyncing: true, + }) + shutdownOnceSyncerVM := &shutdownOnceVM{InnerVM: syncerVM} + t.Cleanup(func() { + require.NoError(shutdownOnceSyncerVM.Shutdown(context.Background())) + }) + syncerVmSetup := syncerVMSetup{ + SyncVMSetup: SyncVMSetup{ + VM: syncerVM, + ConsensusCallbacks: syncerCB, + SnowCtx: syncerTest.Ctx, + DB: syncerTest.DB, + EngineChan: syncerTest.ToEngine, + AtomicMemory: syncerTest.AtomicMemory, + }, + shutdownOnceSyncerVM: shutdownOnceSyncerVM, + } + if testSetup.AfterInit != nil { + testSetup.AfterInit(t, test, syncerVmSetup.SyncVMSetup, false) + } + require.NoError(syncerVM.SetState(context.Background(), snow.StateSyncing)) + enabled, err := syncerVM.StateSyncEnabled(context.Background()) + require.NoError(err) + require.True(enabled) + + // override [serverVM]'s SendAppResponse function to trigger AppResponse on [syncerVM] + serverTest.AppSender.SendAppResponseF = func(ctx context.Context, nodeID ids.NodeID, requestID uint32, response []byte) error { + if test.responseIntercept == nil { + go syncerVM.AppResponse(ctx, nodeID, requestID, response) + } else { + go test.responseIntercept(syncerVM, nodeID, requestID, response) + } + + return nil + } + + // connect peer to [syncerVM] + require.NoError( + syncerVM.Connected( + context.Background(), + serverTest.Ctx.NodeID, + statesyncclient.StateSyncVersion, + ), + ) + + // override [syncerVM]'s SendAppRequest function to trigger AppRequest on [serverVM] + syncerTest.AppSender.SendAppRequestF = func(ctx context.Context, nodeSet set.Set[ids.NodeID], requestID uint32, request []byte) error { + nodeID, hasItem := nodeSet.Pop() + require.True(hasItem, "expected nodeSet to contain at least 1 nodeID") + require.NoError(serverVM.AppRequest(ctx, nodeID, requestID, time.Now().Add(1*time.Second), request)) + return nil + } + + return &testSyncVMSetup{ + serverVM: SyncVMSetup{ + VM: serverVM, + AppSender: serverTest.AppSender, + SnowCtx: serverTest.Ctx, + }, + fundedAccounts: accounts, + syncerVM: syncerVmSetup, + } +} + +// testSyncVMSetup contains the required set up for a client VM to perform state sync +// off of a server VM. +type testSyncVMSetup struct { + serverVM SyncVMSetup + syncerVM syncerVMSetup + + fundedAccounts map[*utilstest.Key]*types.StateAccount +} + +type SyncVMSetup struct { + VM extension.InnerVM + SnowCtx *snow.Context + ConsensusCallbacks dummy.ConsensusCallbacks + DB avalanchedatabase.Database + EngineChan chan commonEng.Message + AtomicMemory *avalancheatomic.Memory + AppSender *enginetest.Sender +} + +type syncerVMSetup struct { + SyncVMSetup + shutdownOnceSyncerVM *shutdownOnceVM +} + +type shutdownOnceVM struct { + extension.InnerVM + shutdownOnce sync.Once +} + +func (vm *shutdownOnceVM) Shutdown(ctx context.Context) error { + var err error + vm.shutdownOnce.Do(func() { err = vm.InnerVM.Shutdown(ctx) }) + return err +} + +// SyncTestParams contains both the actual VMs as well as the parameters with the expected output. +type SyncTestParams struct { + responseIntercept func(vm extension.InnerVM, nodeID ids.NodeID, requestID uint32, response []byte) + StateSyncMinBlocks uint64 + SyncableInterval uint64 + SyncMode block.StateSyncMode + expectedErr error +} + +func testSyncerVM(t *testing.T, testSyncVMSetup *testSyncVMSetup, test SyncTestParams, extraSyncerVMTest func(t *testing.T, syncerVMSetup SyncVMSetup)) { + t.Helper() + var ( + require = require.New(t) + serverVM = testSyncVMSetup.serverVM.VM + fundedAccounts = testSyncVMSetup.fundedAccounts + syncerVM = testSyncVMSetup.syncerVM.VM + syncerEngineChan = testSyncVMSetup.syncerVM.EngineChan + ) + // get last summary and test related methods + summary, err := serverVM.GetLastStateSummary(context.Background()) + require.NoError(err, "error getting state sync last summary") + parsedSummary, err := syncerVM.ParseStateSummary(context.Background(), summary.Bytes()) + require.NoError(err, "error parsing state summary") + retrievedSummary, err := serverVM.GetStateSummary(context.Background(), parsedSummary.Height()) + require.NoError(err, "error getting state sync summary at height") + require.Equal(summary, retrievedSummary) + + syncMode, err := parsedSummary.Accept(context.Background()) + require.NoError(err, "error accepting state summary") + require.Equal(test.SyncMode, syncMode) + if syncMode == block.StateSyncSkipped { + return + } + + msg := <-syncerEngineChan + require.Equal(commonEng.StateSyncDone, msg) + + // If the test is expected to error, assert the correct error is returned and finish the test. + err = syncerVM.SyncerClient().Error() + if test.expectedErr != nil { + require.ErrorIs(err, test.expectedErr) + // Note we re-open the database here to avoid a closed error when the test is for a shutdown VM. + // TODO: this avoids circular dependencies but is not ideal. + ethDBPrefix := []byte("ethdb") + chaindb := database.WrapDatabase(prefixdb.NewNested(ethDBPrefix, testSyncVMSetup.syncerVM.DB)) + assertSyncPerformedHeights(t, chaindb, map[uint64]struct{}{}) + return + } + require.NoError(err, "state sync failed") + + // set [syncerVM] to bootstrapping and verify the last accepted block has been updated correctly + // and that we can bootstrap and process some blocks. + require.NoError(syncerVM.SetState(context.Background(), snow.Bootstrapping)) + require.Equal(serverVM.LastAcceptedVMBlock().Height(), syncerVM.LastAcceptedVMBlock().Height(), "block height mismatch between syncer and server") + require.Equal(serverVM.LastAcceptedVMBlock().ID(), syncerVM.LastAcceptedVMBlock().ID(), "blockID mismatch between syncer and server") + require.True(syncerVM.Ethereum().BlockChain().HasState(syncerVM.Ethereum().BlockChain().LastAcceptedBlock().Root()), "unavailable state for last accepted block") + assertSyncPerformedHeights(t, syncerVM.Ethereum().ChainDb(), map[uint64]struct{}{retrievedSummary.Height(): {}}) + + lastNumber := syncerVM.Ethereum().BlockChain().LastAcceptedBlock().NumberU64() + // check the last block is indexed + lastSyncedBlock := rawdb.ReadBlock(syncerVM.Ethereum().ChainDb(), rawdb.ReadCanonicalHash(syncerVM.Ethereum().ChainDb(), lastNumber), lastNumber) + require.NotNil(lastSyncedBlock, "last synced block not found") + for _, tx := range lastSyncedBlock.Transactions() { + index := rawdb.ReadTxLookupEntry(syncerVM.Ethereum().ChainDb(), tx.Hash()) + require.NotNilf(index, "Miss transaction indices, number %d hash %s", lastNumber, tx.Hash().Hex()) + } + + // tail should be the last block synced + if syncerVM.Ethereum().BlockChain().CacheConfig().TransactionHistory != 0 { + tail := lastSyncedBlock.NumberU64() + + coretest.CheckTxIndices(t, &tail, tail, tail, tail, syncerVM.Ethereum().ChainDb(), true) + } + + blocksToBuild := 10 + txsPerBlock := 10 + toAddress := TestEthAddrs[1] // arbitrary choice + generateAndAcceptBlocks(t, syncerVM, blocksToBuild, func(_ int, vm extension.InnerVM, gen *core.BlockGen) { + b, err := predicate.NewResults().Bytes() + if err != nil { + t.Fatal(err) + } + gen.AppendExtra(b) + i := 0 + for k := range fundedAccounts { + tx := types.NewTransaction(gen.TxNonce(k.Address), toAddress, big.NewInt(1), 21000, InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), k.PrivateKey) + require.NoError(err) + gen.AddTx(signedTx) + i++ + if i >= txsPerBlock { + break + } + } + }, + func(block *types.Block) { + if syncerVM.Ethereum().BlockChain().CacheConfig().TransactionHistory != 0 { + tail := block.NumberU64() - syncerVM.Ethereum().BlockChain().CacheConfig().TransactionHistory + 1 + // tail should be the minimum last synced block, since we skipped it to the last block + if tail < lastSyncedBlock.NumberU64() { + tail = lastSyncedBlock.NumberU64() + } + coretest.CheckTxIndices(t, &tail, tail, block.NumberU64(), block.NumberU64(), syncerVM.Ethereum().ChainDb(), true) + } + }, + testSyncVMSetup.syncerVM.ConsensusCallbacks, + ) + + // check we can transition to [NormalOp] state and continue to process blocks. + require.NoError(syncerVM.SetState(context.Background(), snow.NormalOp)) + + // Generate blocks after we have entered normal consensus as well + generateAndAcceptBlocks(t, syncerVM, blocksToBuild, func(_ int, vm extension.InnerVM, gen *core.BlockGen) { + b, err := predicate.NewResults().Bytes() + if err != nil { + t.Fatal(err) + } + gen.AppendExtra(b) + i := 0 + for k := range fundedAccounts { + tx := types.NewTransaction(gen.TxNonce(k.Address), toAddress, big.NewInt(1), 21000, InitialBaseFee, nil) + signedTx, err := types.SignTx(tx, types.NewEIP155Signer(vm.Ethereum().BlockChain().Config().ChainID), k.PrivateKey) + require.NoError(err) + gen.AddTx(signedTx) + i++ + if i >= txsPerBlock { + break + } + } + }, + func(block *types.Block) { + if syncerVM.Ethereum().BlockChain().CacheConfig().TransactionHistory != 0 { + tail := block.NumberU64() - syncerVM.Ethereum().BlockChain().CacheConfig().TransactionHistory + 1 + // tail should be the minimum last synced block, since we skipped it to the last block + if tail < lastSyncedBlock.NumberU64() { + tail = lastSyncedBlock.NumberU64() + } + coretest.CheckTxIndices(t, &tail, tail, block.NumberU64(), block.NumberU64(), syncerVM.Ethereum().ChainDb(), true) + } + }, + testSyncVMSetup.syncerVM.ConsensusCallbacks, + ) + + if extraSyncerVMTest != nil { + extraSyncerVMTest(t, testSyncVMSetup.syncerVM.SyncVMSetup) + } +} + +// patchBlock returns a copy of [blk] with [root] and updates [db] to +// include the new block as canonical for [blk]'s height. +// This breaks the digestibility of the chain since after this call +// [blk] does not necessarily define a state transition from its parent +// state to the new state root. +func patchBlock(blk *types.Block, root common.Hash, db ethdb.Database) *types.Block { + header := blk.Header() + header.Root = root + receipts := rawdb.ReadRawReceipts(db, blk.Hash(), blk.NumberU64()) + newBlk := customtypes.NewBlockWithExtData( + header, blk.Transactions(), blk.Uncles(), receipts, trie.NewStackTrie(nil), customtypes.BlockExtData(blk), true, + ) + rawdb.WriteBlock(db, newBlk) + rawdb.WriteCanonicalHash(db, newBlk.Hash(), newBlk.NumberU64()) + return newBlk +} + +// generateAndAcceptBlocks uses [core.GenerateChain] to generate blocks, then +// calls Verify and Accept on each generated block +// TODO: consider using this helper function in vm_test.go and elsewhere in this package to clean up tests +func generateAndAcceptBlocks(t *testing.T, vm extension.InnerVM, numBlocks int, gen func(int, extension.InnerVM, *core.BlockGen), accepted func(*types.Block), cb dummy.ConsensusCallbacks) { + t.Helper() + + // acceptExternalBlock defines a function to parse, verify, and accept a block once it has been + // generated by GenerateChain + acceptExternalBlock := func(block *types.Block) { + bytes, err := rlp.EncodeToBytes(block) + if err != nil { + t.Fatal(err) + } + vmBlock, err := vm.ParseBlock(context.Background(), bytes) + if err != nil { + t.Fatal(err) + } + if err := vmBlock.Verify(context.Background()); err != nil { + t.Fatal(err) + } + if err := vmBlock.Accept(context.Background()); err != nil { + t.Fatal(err) + } + + if accepted != nil { + accepted(block) + } + } + _, _, err := core.GenerateChain( + vm.Ethereum().BlockChain().Config(), + vm.Ethereum().BlockChain().LastAcceptedBlock(), + dummy.NewFakerWithCallbacks(cb), + vm.Ethereum().ChainDb(), + numBlocks, + 10, + func(i int, g *core.BlockGen) { + g.SetOnBlockGenerated(acceptExternalBlock) + g.SetCoinbase(constants.BlackholeAddr) // necessary for syntactic validation of the block + gen(i, vm, g) + }, + ) + if err != nil { + t.Fatal(err) + } + vm.Ethereum().BlockChain().DrainAcceptorQueue() +} + +// assertSyncPerformedHeights iterates over all heights the VM has synced to and +// verifies they all match the heights present in `expected`. +func assertSyncPerformedHeights(t *testing.T, db ethdb.Iteratee, expected map[uint64]struct{}) { + it := customrawdb.NewSyncPerformedIterator(db) + defer it.Release() + + found := make(map[uint64]struct{}, len(expected)) + for it.Next() { + found[customrawdb.UnpackSyncPerformedKey(it.Key())] = struct{}{} + } + require.NoError(t, it.Error()) + require.Equal(t, expected, found) +} diff --git a/plugin/evm/vmtest/test_vm.go b/plugin/evm/vmtest/test_vm.go new file mode 100644 index 0000000000..751551d5a9 --- /dev/null +++ b/plugin/evm/vmtest/test_vm.go @@ -0,0 +1,81 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package vmtest + +import ( + "context" + "testing" + + avalancheatomic "github.com/ava-labs/avalanchego/chains/atomic" + "github.com/ava-labs/avalanchego/database/prefixdb" + "github.com/ava-labs/avalanchego/snow" + commonEng "github.com/ava-labs/avalanchego/snow/engine/common" + commoneng "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/snow/engine/enginetest" + "github.com/ava-labs/avalanchego/upgrade/upgradetest" + "github.com/stretchr/testify/require" +) + +type TestVMConfig struct { + IsSyncing bool + Fork *upgradetest.Fork + // If genesisJSON is empty, defaults to the genesis corresponding to the + // fork. + GenesisJSON string + ConfigJSON string +} + +type TestVMSuite struct { + VM commoneng.VM + ToEngine chan commonEng.Message + DB *prefixdb.Database + AtomicMemory *avalancheatomic.Memory + AppSender *enginetest.Sender + Ctx *snow.Context +} + +// SetupTestVM initializes a VM for testing. It sets up the genesis and returns the +// issuer channel, database, atomic memory, app sender, and context. +// Expects the passed VM to be a uninitialized VM. +func SetupTestVM(t *testing.T, vm commoneng.VM, config TestVMConfig) *TestVMSuite { + fork := upgradetest.Latest + if config.Fork != nil { + fork = *config.Fork + } + ctx, dbManager, genesisBytes, issuer, m := SetupGenesis(t, fork) + if len(config.GenesisJSON) != 0 { + genesisBytes = []byte(config.GenesisJSON) + } + appSender := &enginetest.Sender{ + T: t, + CantSendAppGossip: true, + SendAppGossipF: func(context.Context, commonEng.SendConfig, []byte) error { return nil }, + } + err := vm.Initialize( + context.Background(), + ctx, + dbManager, + genesisBytes, + nil, + []byte(config.ConfigJSON), + issuer, + nil, + appSender, + ) + require.NoError(t, err, "error initializing GenesisVM") + + if !config.IsSyncing { + require.NoError(t, vm.SetState(context.Background(), snow.Bootstrapping)) + require.NoError(t, vm.SetState(context.Background(), snow.NormalOp)) + } + + return &TestVMSuite{ + VM: vm, + ToEngine: issuer, + DB: dbManager, + AtomicMemory: m, + AppSender: appSender, + Ctx: ctx, + } +} diff --git a/plugin/evm/wrapped_block.go b/plugin/evm/wrapped_block.go new file mode 100644 index 0000000000..c11494d8bb --- /dev/null +++ b/plugin/evm/wrapped_block.go @@ -0,0 +1,481 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "bytes" + "context" + "errors" + "fmt" + "math/big" + "time" + + "github.com/ava-labs/coreth/constants" + "github.com/ava-labs/coreth/core" + "github.com/ava-labs/coreth/params" + "github.com/ava-labs/coreth/params/extras" + "github.com/ava-labs/coreth/plugin/evm/customtypes" + "github.com/ava-labs/coreth/plugin/evm/extension" + "github.com/ava-labs/coreth/plugin/evm/header" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap0" + "github.com/ava-labs/coreth/plugin/evm/upgrade/ap1" + "github.com/ava-labs/coreth/precompile/precompileconfig" + "github.com/ava-labs/coreth/predicate" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/rawdb" + "github.com/ava-labs/libevm/core/types" + "github.com/ava-labs/libevm/log" + "github.com/ava-labs/libevm/rlp" + "github.com/ava-labs/libevm/trie" + + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/consensus/snowman" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" +) + +var ( + _ snowman.Block = (*wrappedBlock)(nil) + _ block.WithVerifyContext = (*wrappedBlock)(nil) + _ extension.VMBlock = (*wrappedBlock)(nil) +) + +var ( + ap0MinGasPrice = big.NewInt(ap0.MinGasPrice) + ap1MinGasPrice = big.NewInt(ap1.MinGasPrice) +) + +// wrappedBlock implements the snowman.wrappedBlock interface +type wrappedBlock struct { + id ids.ID + ethBlock *types.Block + extension extension.BlockExtension + vm *VM +} + +// wrapBlock returns a new Block wrapping the ethBlock type and implementing the snowman.Block interface +func wrapBlock(ethBlock *types.Block, vm *VM) (*wrappedBlock, error) { + b := &wrappedBlock{ + id: ids.ID(ethBlock.Hash()), + ethBlock: ethBlock, + vm: vm, + } + if vm.extensionConfig.BlockExtender != nil { + extension, err := vm.extensionConfig.BlockExtender.NewBlockExtension(b) + if err != nil { + return nil, fmt.Errorf("failed to create block extension: %w", err) + } + b.extension = extension + } + return b, nil +} + +// ID implements the snowman.Block interface +func (b *wrappedBlock) ID() ids.ID { return b.id } + +// Accept implements the snowman.Block interface +func (b *wrappedBlock) Accept(context.Context) error { + vm := b.vm + // Although returning an error from Accept is considered fatal, it is good + // practice to cleanup the batch we were modifying in the case of an error. + defer vm.versiondb.Abort() + + blkID := b.ID() + log.Debug("accepting block", + "hash", blkID.Hex(), + "id", blkID, + "height", b.Height(), + ) + // Call Accept for relevant precompile logs. Note we do this prior to + // calling Accept on the blockChain so any side effects (eg warp signatures) + // take place before the accepted log is emitted to subscribers. + rules := b.vm.rules(b.ethBlock.Number(), b.ethBlock.Time()) + if err := b.handlePrecompileAccept(rules); err != nil { + return err + } + if err := vm.blockChain.Accept(b.ethBlock); err != nil { + return fmt.Errorf("chain could not accept %s: %w", blkID, err) + } + + if err := vm.PutLastAcceptedID(blkID); err != nil { + return fmt.Errorf("failed to put %s as the last accepted block: %w", blkID, err) + } + + // Get pending operations on the vm's versionDB so we can apply them atomically + // with the block extension's changes. + vdbBatch, err := vm.versiondb.CommitBatch() + if err != nil { + return fmt.Errorf("could not create commit batch processing block[%s]: %w", blkID, err) + } + + if b.extension != nil { + // Apply any changes atomically with other pending changes to + // the vm's versionDB. + // OnAccept flushes the changes in the batch to the database. + return b.extension.OnAccept(vdbBatch) + } + + // If there is no extension, we still need to apply the changes to the versionDB + return vdbBatch.Write() +} + +// handlePrecompileAccept calls Accept on any logs generated with an active precompile address that implements +// contract.Accepter +func (b *wrappedBlock) handlePrecompileAccept(rules extras.Rules) error { + // Short circuit early if there are no precompile accepters to execute + if len(rules.AccepterPrecompiles) == 0 { + return nil + } + + // Read receipts from disk + receipts := rawdb.ReadReceipts(b.vm.chaindb, b.ethBlock.Hash(), b.ethBlock.NumberU64(), b.ethBlock.Time(), b.vm.chainConfig) + // If there are no receipts, ReadReceipts may be nil, so we check the length and confirm the ReceiptHash + // is empty to ensure that missing receipts results in an error on accept. + if len(receipts) == 0 && b.ethBlock.ReceiptHash() != types.EmptyRootHash { + return fmt.Errorf("failed to fetch receipts for accepted block with non-empty root hash (%s) (Block: %s, Height: %d)", b.ethBlock.ReceiptHash(), b.ethBlock.Hash(), b.ethBlock.NumberU64()) + } + acceptCtx := &precompileconfig.AcceptContext{ + SnowCtx: b.vm.ctx, + Warp: b.vm.warpBackend, + } + for _, receipt := range receipts { + for logIdx, log := range receipt.Logs { + accepter, ok := rules.AccepterPrecompiles[log.Address] + if !ok { + continue + } + if err := accepter.Accept(acceptCtx, log.BlockHash, log.BlockNumber, log.TxHash, logIdx, log.Topics, log.Data); err != nil { + return err + } + } + } + + return nil +} + +// Reject implements the snowman.Block interface +// If [b] contains an atomic transaction, attempt to re-issue it +func (b *wrappedBlock) Reject(context.Context) error { + blkID := b.ID() + log.Debug("rejecting block", + "hash", blkID.Hex(), + "id", blkID, + "height", b.Height(), + ) + + if b.extension != nil { + if err := b.extension.OnReject(); err != nil { + return err + } + } + return b.vm.blockChain.Reject(b.ethBlock) +} + +// Parent implements the snowman.Block interface +func (b *wrappedBlock) Parent() ids.ID { + return ids.ID(b.ethBlock.ParentHash()) +} + +// Height implements the snowman.Block interface +func (b *wrappedBlock) Height() uint64 { + return b.ethBlock.NumberU64() +} + +// Timestamp implements the snowman.Block interface +func (b *wrappedBlock) Timestamp() time.Time { + return time.Unix(int64(b.ethBlock.Time()), 0) +} + +// Verify implements the snowman.Block interface +func (b *wrappedBlock) Verify(context.Context) error { + return b.verify(&precompileconfig.PredicateContext{ + SnowCtx: b.vm.ctx, + ProposerVMBlockCtx: nil, + }, true) +} + +// ShouldVerifyWithContext implements the block.WithVerifyContext interface +func (b *wrappedBlock) ShouldVerifyWithContext(context.Context) (bool, error) { + rules := b.vm.rules(b.ethBlock.Number(), b.ethBlock.Time()) + predicates := rules.Predicaters + // Short circuit early if there are no predicates to verify + if len(predicates) == 0 { + return false, nil + } + + // Check if any of the transactions in the block specify a precompile that enforces a predicate, which requires + // the ProposerVMBlockCtx. + for _, tx := range b.ethBlock.Transactions() { + for _, accessTuple := range tx.AccessList() { + if _, ok := predicates[accessTuple.Address]; ok { + log.Debug("Block verification requires proposerVM context", "block", b.ID(), "height", b.Height()) + return true, nil + } + } + } + + log.Debug("Block verification does not require proposerVM context", "block", b.ID(), "height", b.Height()) + return false, nil +} + +// VerifyWithContext implements the block.WithVerifyContext interface +func (b *wrappedBlock) VerifyWithContext(ctx context.Context, proposerVMBlockCtx *block.Context) error { + return b.verify(&precompileconfig.PredicateContext{ + SnowCtx: b.vm.ctx, + ProposerVMBlockCtx: proposerVMBlockCtx, + }, true) +} + +// Verify the block is valid. +// Enforces that the predicates are valid within [predicateContext]. +// Writes the block details to disk and the state to the trie manager iff writes=true. +func (b *wrappedBlock) verify(predicateContext *precompileconfig.PredicateContext, writes bool) error { + if predicateContext.ProposerVMBlockCtx != nil { + log.Debug("Verifying block with context", "block", b.ID(), "height", b.Height()) + } else { + log.Debug("Verifying block without context", "block", b.ID(), "height", b.Height()) + } + if err := b.syntacticVerify(); err != nil { + return fmt.Errorf("syntactic block verification failed: %w", err) + } + + if err := b.semanticVerify(); err != nil { + return fmt.Errorf("failed to verify block: %w", err) + } + + // If the VM is not marked as bootstrapped the other chains may also be + // bootstrapping and not have populated the required indices. Since + // bootstrapping only verifies blocks that have been canonically accepted by + // the network, these checks would be guaranteed to pass on a synced node. + if b.vm.bootstrapped.Get() { + // Verify that all the ICM messages are correctly marked as either valid + // or invalid. + if err := b.verifyPredicates(predicateContext); err != nil { + return fmt.Errorf("failed to verify predicates: %w", err) + } + } + + // The engine may call VerifyWithContext multiple times on the same block with different contexts. + // Since the engine will only call Accept/Reject once, we should only call InsertBlockManual once. + // Additionally, if a block is already in processing, then it has already passed verification and + // at this point we have checked the predicates are still valid in the different context so we + // can return nil. + if b.vm.State.IsProcessing(b.id) { + return nil + } + + err := b.vm.blockChain.InsertBlockManual(b.ethBlock, writes) + if b.extension != nil && (err != nil || !writes) { + b.extension.CleanupVerified() + } + return err +} + +// semanticVerify verifies that a *Block is internally consistent. +func (b *wrappedBlock) semanticVerify() error { + // Make sure the block isn't too far in the future + blockTimestamp := b.ethBlock.Time() + if maxBlockTime := uint64(b.vm.clock.Time().Add(maxFutureBlockTime).Unix()); blockTimestamp > maxBlockTime { + return fmt.Errorf("block timestamp is too far in the future: %d > allowed %d", blockTimestamp, maxBlockTime) + } + + if b.extension != nil { + if err := b.extension.SemanticVerify(); err != nil { + return err + } + } + return nil +} + +// syntacticVerify verifies that a *Block is well-formed. +func (b *wrappedBlock) syntacticVerify() error { + if b == nil || b.ethBlock == nil { + return errInvalidBlock + } + + // Skip verification of the genesis block since it should already be marked as accepted. + if b.ethBlock.Hash() == b.vm.genesisHash { + return nil + } + + ethHeader := b.ethBlock.Header() + rules := b.vm.chainConfig.Rules(ethHeader.Number, params.IsMergeTODO, ethHeader.Time) + rulesExtra := params.GetRulesExtra(rules) + // Perform block and header sanity checks + if !ethHeader.Number.IsUint64() { + return fmt.Errorf("invalid block number: %v", ethHeader.Number) + } + if !ethHeader.Difficulty.IsUint64() || ethHeader.Difficulty.Cmp(common.Big1) != 0 { + return fmt.Errorf("invalid difficulty: %d", ethHeader.Difficulty) + } + if ethHeader.Nonce.Uint64() != 0 { + return fmt.Errorf( + "expected nonce to be 0 but got %d: %w", + ethHeader.Nonce.Uint64(), errInvalidNonce, + ) + } + + if ethHeader.MixDigest != (common.Hash{}) { + return fmt.Errorf("invalid mix digest: %v", ethHeader.MixDigest) + } + + // Verify the extra data is well-formed. + if err := header.VerifyExtra(rulesExtra.AvalancheRules, ethHeader.Extra); err != nil { + return err + } + + if version := customtypes.BlockVersion(b.ethBlock); version != 0 { + return fmt.Errorf("invalid version: %d", version) + } + + // Check that the tx hash in the header matches the body + txsHash := types.DeriveSha(b.ethBlock.Transactions(), trie.NewStackTrie(nil)) + if txsHash != ethHeader.TxHash { + return fmt.Errorf("invalid txs hash %v does not match calculated txs hash %v", ethHeader.TxHash, txsHash) + } + // Check that the uncle hash in the header matches the body + uncleHash := types.CalcUncleHash(b.ethBlock.Uncles()) + if uncleHash != ethHeader.UncleHash { + return fmt.Errorf("invalid uncle hash %v does not match calculated uncle hash %v", ethHeader.UncleHash, uncleHash) + } + // Coinbase must match the BlackholeAddr on C-Chain + if ethHeader.Coinbase != constants.BlackholeAddr { + return fmt.Errorf("invalid coinbase %v does not match required blackhole address %v", ethHeader.Coinbase, constants.BlackholeAddr) + } + // Block must not have any uncles + if len(b.ethBlock.Uncles()) > 0 { + return errUnclesUnsupported + } + + // Enforce minimum gas prices here prior to dynamic fees going into effect. + switch { + case !rulesExtra.IsApricotPhase1: + // If we are in ApricotPhase0, enforce each transaction has a minimum gas price of at least the LaunchMinGasPrice + for _, tx := range b.ethBlock.Transactions() { + if tx.GasPrice().Cmp(ap0MinGasPrice) < 0 { + return fmt.Errorf("block contains tx %s with gas price too low (%d < %d)", tx.Hash(), tx.GasPrice(), ap0.MinGasPrice) + } + } + case !rulesExtra.IsApricotPhase3: + // If we are prior to ApricotPhase3, enforce each transaction has a minimum gas price of at least the ApricotPhase1MinGasPrice + for _, tx := range b.ethBlock.Transactions() { + if tx.GasPrice().Cmp(ap1MinGasPrice) < 0 { + return fmt.Errorf("block contains tx %s with gas price too low (%d < %d)", tx.Hash(), tx.GasPrice(), ap1.MinGasPrice) + } + } + } + + // Ensure BaseFee is non-nil as of ApricotPhase3. + if rulesExtra.IsApricotPhase3 { + if ethHeader.BaseFee == nil { + return errNilBaseFeeApricotPhase3 + } + if bfLen := ethHeader.BaseFee.BitLen(); bfLen > 256 { + return fmt.Errorf("too large base fee: bitlen %d", bfLen) + } + } + + headerExtra := customtypes.GetHeaderExtra(ethHeader) + if rulesExtra.IsApricotPhase4 { + switch { + // Make sure BlockGasCost is not nil + // NOTE: ethHeader.BlockGasCost correctness is checked in header verification + case headerExtra.BlockGasCost == nil: + return errNilBlockGasCostApricotPhase4 + case !headerExtra.BlockGasCost.IsUint64(): + return fmt.Errorf("too large blockGasCost: %d", headerExtra.BlockGasCost) + } + } + + // Verify the existence / non-existence of excessBlobGas + cancun := rules.IsCancun + if !cancun && ethHeader.ExcessBlobGas != nil { + return fmt.Errorf("invalid excessBlobGas: have %d, expected nil", *ethHeader.ExcessBlobGas) + } + if !cancun && ethHeader.BlobGasUsed != nil { + return fmt.Errorf("invalid blobGasUsed: have %d, expected nil", *ethHeader.BlobGasUsed) + } + if cancun && ethHeader.ExcessBlobGas == nil { + return errors.New("header is missing excessBlobGas") + } + if cancun && ethHeader.BlobGasUsed == nil { + return errors.New("header is missing blobGasUsed") + } + if !cancun && ethHeader.ParentBeaconRoot != nil { + return fmt.Errorf("invalid parentBeaconRoot: have %x, expected nil", *ethHeader.ParentBeaconRoot) + } + if cancun { + switch { + case ethHeader.ParentBeaconRoot == nil: + return errors.New("header is missing parentBeaconRoot") + case *ethHeader.ParentBeaconRoot != (common.Hash{}): + return fmt.Errorf("invalid parentBeaconRoot: have %x, expected empty hash", ethHeader.ParentBeaconRoot) + } + if ethHeader.BlobGasUsed == nil { + return fmt.Errorf("blob gas used must not be nil in Cancun") + } else if *ethHeader.BlobGasUsed > 0 { + return fmt.Errorf("blobs not enabled on avalanche networks: used %d blob gas, expected 0", *ethHeader.BlobGasUsed) + } + } + + if b.extension != nil { + if err := b.extension.SyntacticVerify(rules); err != nil { + return err + } + } + return nil +} + +// verifyPredicates verifies the predicates in the block are valid according to predicateContext. +func (b *wrappedBlock) verifyPredicates(predicateContext *precompileconfig.PredicateContext) error { + rules := b.vm.chainConfig.Rules(b.ethBlock.Number(), params.IsMergeTODO, b.ethBlock.Time()) + rulesExtra := params.GetRulesExtra(rules) + + switch { + case !rulesExtra.IsDurango && rulesExtra.PredicatersExist(): + return errors.New("cannot enable predicates before Durango activation") + case !rulesExtra.IsDurango: + return nil + } + + predicateResults := predicate.NewResults() + for _, tx := range b.ethBlock.Transactions() { + results, err := core.CheckPredicates(rules, predicateContext, tx) + if err != nil { + return err + } + predicateResults.SetTxResults(tx.Hash(), results) + } + // TODO: document required gas constraints to ensure marshalling predicate results does not error + predicateResultsBytes, err := predicateResults.Bytes() + if err != nil { + return fmt.Errorf("failed to marshal predicate results: %w", err) + } + extraData := b.ethBlock.Extra() + avalancheRules := rulesExtra.AvalancheRules + headerPredicateResultsBytes := header.PredicateBytesFromExtra(avalancheRules, extraData) + if !bytes.Equal(headerPredicateResultsBytes, predicateResultsBytes) { + return fmt.Errorf("%w (remote: %x local: %x)", errInvalidHeaderPredicateResults, headerPredicateResultsBytes, predicateResultsBytes) + } + return nil +} + +// Bytes implements the snowman.Block interface +func (b *wrappedBlock) Bytes() []byte { + res, err := rlp.EncodeToBytes(b.ethBlock) + if err != nil { + panic(err) + } + return res +} + +func (b *wrappedBlock) String() string { return fmt.Sprintf("EVM block, ID = %s", b.ID()) } + +func (b *wrappedBlock) GetEthBlock() *types.Block { + return b.ethBlock +} + +func (b *wrappedBlock) GetBlockExtension() extension.BlockExtension { + return b.extension +} diff --git a/plugin/evm/factory.go b/plugin/factory/factory.go similarity index 59% rename from plugin/evm/factory.go rename to plugin/factory/factory.go index 8e5d0ca4c6..3c668db3c7 100644 --- a/plugin/evm/factory.go +++ b/plugin/factory/factory.go @@ -1,12 +1,16 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package evm +package factory import ( "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/engine/snowman/block" "github.com/ava-labs/avalanchego/utils/logging" "github.com/ava-labs/avalanchego/vms" + + "github.com/ava-labs/coreth/plugin/evm" + atomicvm "github.com/ava-labs/coreth/plugin/evm/atomic/vm" ) var ( @@ -19,5 +23,9 @@ var ( type Factory struct{} func (*Factory) New(logging.Logger) (interface{}, error) { - return &VM{}, nil + return atomicvm.WrapVM(&evm.VM{}), nil +} + +func NewPluginVM() block.ChainVM { + return atomicvm.WrapVM(&evm.VM{IsPlugin: true}) } diff --git a/plugin/main.go b/plugin/main.go index d841f453ab..28711dfbd7 100644 --- a/plugin/main.go +++ b/plugin/main.go @@ -13,12 +13,13 @@ import ( "github.com/ava-labs/avalanchego/vms/rpcchainvm" "github.com/ava-labs/coreth/plugin/evm" + "github.com/ava-labs/coreth/plugin/factory" ) func main() { version, err := PrintVersion() if err != nil { - fmt.Printf("couldn't get config: %s", err) + fmt.Printf("couldn't get config: %s\n", err) os.Exit(1) } if version { @@ -26,9 +27,8 @@ func main() { os.Exit(0) } if err := ulimit.Set(ulimit.DefaultFDLimit, logging.NoLog{}); err != nil { - fmt.Printf("failed to set fd limit correctly due to: %s", err) + fmt.Printf("failed to set fd limit correctly due to: %s\n", err) os.Exit(1) } - - rpcchainvm.Serve(context.Background(), &evm.VM{IsPlugin: true}) + rpcchainvm.Serve(context.Background(), factory.NewPluginVM()) } diff --git a/scripts/update_avalanchego_version.sh b/scripts/update_avalanchego_version.sh index 892f9d152d..9b326308e9 100755 --- a/scripts/update_avalanchego_version.sh +++ b/scripts/update_avalanchego_version.sh @@ -26,7 +26,8 @@ if [[ -n "${GITHUB_TOKEN:-}" ]]; then CURL_ARGS+=(-H "Authorization: token ${GITHUB_TOKEN}") fi CURL_URL="https://api.github.com/repos/ava-labs/avalanchego/commits/${AVALANCHE_VERSION}" -FULL_AVALANCHE_VERSION="$("${CURL_ARGS[@]}" "${CURL_URL}" | grep '"sha":' | head -n1 | cut -d'"' -f4)" + +FULL_AVALANCHE_VERSION="$("${CURL_ARGS[@]}" "${CURL_URL}" | jq -r .sha)" # Ensure the custom action version matches the avalanche version WORKFLOW_PATH=".github/workflows/ci.yml" diff --git a/scripts/upstream_licensed_folders.txt b/scripts/upstream_licensed_folders.txt new file mode 100644 index 0000000000..373149ef1a --- /dev/null +++ b/scripts/upstream_licensed_folders.txt @@ -0,0 +1,26 @@ +core/* +eth/* +node/* +internal/* + +!internal/ethapi/api_extra_test.go +!core/blockchain_ext.go +!core/blockchain_log_test.go +!core/blockchain_test.go +!core/bounded_buffer.go +!core/extstate/statedb.go +!core/extstate/test_statedb.go +!core/fifo_cache.go +!core/main_test.go +!core/predicate_check.go +!core/predicate_check_test.go +!core/state/snapshot/snapshot_ext.go +!core/state/statedb_multicoin_test.go +!core/state_manager_test.go +!core/state_processor_ext.go +!core/test_blockchain.go +!eth/chain_with_final_block.go +!eth/gasprice/fee_info_provider_test.go +!internal/ethapi/api.coreth.go +!internal/ethapi/api.coreth_test.go +!internal/ethapi/api_extra.go \ No newline at end of file diff --git a/sync/README.md b/sync/README.md index 2e6437a71f..5cf88bcd2e 100644 --- a/sync/README.md +++ b/sync/README.md @@ -65,8 +65,8 @@ The following steps are executed by the VM to sync its state from peers (see `st Steps 3 and 4 involve syncing tries. To sync trie data, the VM will send a series of `LeafRequests` to its peers. Each request specifies: - Type of trie (`NodeType`): - - `message.StateTrieNode` (account trie and storage tries share the same database) - - `message.AtomicTrieNode` (atomic trie has an independent database) + - `statesync.StateTrieNode` (account trie and storage tries share the same database) + - `atomic.AtomicTrieNode` (atomic trie has an independent database) - `Root` of the trie to sync, - `Start` and `End` specify a range of keys. diff --git a/sync/client/client_test.go b/sync/client/client_test.go index 9d5b3acc8a..9cb5e62a5c 100644 --- a/sync/client/client_test.go +++ b/sync/client/client_test.go @@ -19,6 +19,7 @@ import ( "github.com/ava-labs/coreth/core" "github.com/ava-labs/coreth/params" "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/message/messagetest" clientstats "github.com/ava-labs/coreth/sync/client/stats" "github.com/ava-labs/coreth/sync/handlers" handlerstats "github.com/ava-labs/coreth/sync/handlers/stats" @@ -30,6 +31,8 @@ import ( "github.com/ava-labs/libevm/triedb" ) +var networkCodec = messagetest.BlockSyncSummaryCodec + func TestGetCode(t *testing.T) { testNetClient := &testNetwork{} @@ -86,7 +89,7 @@ func TestGetCode(t *testing.T) { stateSyncClient := NewClient(&ClientConfig{ NetworkClient: testNetClient, - Codec: message.Codec, + Codec: networkCodec, Stats: clientstats.NewNoOpStats(), StateSyncNodeIDs: nil, BlockParser: newTestBlockParser(), @@ -98,7 +101,7 @@ func TestGetCode(t *testing.T) { defer cancel() codeHashes, res, expectedCode := test.setupRequest() - responseBytes, err := message.Codec.Marshal(message.Version, res) + responseBytes, err := networkCodec.Marshal(message.Version, res) if err != nil { t.Fatal(err) } @@ -157,13 +160,13 @@ func TestGetBlocks(t *testing.T) { testNetClient := &testNetwork{} stateSyncClient := NewClient(&ClientConfig{ NetworkClient: testNetClient, - Codec: message.Codec, + Codec: networkCodec, Stats: clientstats.NewNoOpStats(), StateSyncNodeIDs: nil, BlockParser: newTestBlockParser(), }) - blocksRequestHandler := handlers.NewBlockRequestHandler(buildGetter(blocks), message.Codec, handlerstats.NewNoopHandlerStats()) + blocksRequestHandler := handlers.NewBlockRequestHandler(buildGetter(blocks), networkCodec, handlerstats.NewNoopHandlerStats()) // encodeBlockSlice takes a slice of blocks that are ordered in increasing height order // and returns a slice of byte slices with those blocks encoded in reverse order @@ -254,12 +257,12 @@ func TestGetBlocks(t *testing.T) { t.Fatalf("failed to get block response: %s", err) } var blockResponse message.BlockResponse - if _, err = message.Codec.Unmarshal(response, &blockResponse); err != nil { + if _, err = networkCodec.Unmarshal(response, &blockResponse); err != nil { t.Fatalf("failed to marshal block response: %s", err) } // Replace middle value with garbage data blockResponse.Blocks[10] = []byte("invalid value replacing block bytes") - responseBytes, err := message.Codec.Marshal(message.Version, blockResponse) + responseBytes, err := networkCodec.Marshal(message.Version, blockResponse) if err != nil { t.Fatalf("failed to marshal block response: %s", err) } @@ -308,7 +311,7 @@ func TestGetBlocks(t *testing.T) { blockResponse := message.BlockResponse{ Blocks: blockBytes, } - responseBytes, err := message.Codec.Marshal(message.Version, blockResponse) + responseBytes, err := networkCodec.Marshal(message.Version, blockResponse) if err != nil { t.Fatalf("failed to marshal block response: %s", err) } @@ -327,7 +330,7 @@ func TestGetBlocks(t *testing.T) { blockResponse := message.BlockResponse{ Blocks: nil, } - responseBytes, err := message.Codec.Marshal(message.Version, blockResponse) + responseBytes, err := networkCodec.Marshal(message.Version, blockResponse) if err != nil { t.Fatalf("failed to marshal block response: %s", err) } @@ -348,7 +351,7 @@ func TestGetBlocks(t *testing.T) { blockResponse := message.BlockResponse{ Blocks: blockBytes, } - responseBytes, err := message.Codec.Marshal(message.Version, blockResponse) + responseBytes, err := networkCodec.Marshal(message.Version, blockResponse) if err != nil { t.Fatalf("failed to marshal block response: %s", err) } @@ -415,10 +418,10 @@ func TestGetLeafs(t *testing.T) { largeTrieRoot, largeTrieKeys, _ := statesynctest.GenerateTrie(t, trieDB, 100_000, common.HashLength) smallTrieRoot, _, _ := statesynctest.GenerateTrie(t, trieDB, leafsLimit, common.HashLength) - handler := handlers.NewLeafsRequestHandler(trieDB, nil, message.Codec, handlerstats.NewNoopHandlerStats()) + handler := handlers.NewLeafsRequestHandler(trieDB, message.StateTrieKeyLength, nil, networkCodec, handlerstats.NewNoopHandlerStats()) client := NewClient(&ClientConfig{ NetworkClient: &testNetwork{}, - Codec: message.Codec, + Codec: networkCodec, Stats: clientstats.NewNoOpStats(), StateSyncNodeIDs: nil, BlockParser: newTestBlockParser(), @@ -594,13 +597,13 @@ func TestGetLeafs(t *testing.T) { t.Fatal("Failed to create valid response") } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } leafResponse.Keys = leafResponse.Keys[1:] leafResponse.Vals = leafResponse.Vals[1:] - modifiedResponse, err := message.Codec.Marshal(message.Version, leafResponse) + modifiedResponse, err := networkCodec.Marshal(message.Version, leafResponse) if err != nil { t.Fatal(err) } @@ -625,7 +628,7 @@ func TestGetLeafs(t *testing.T) { t.Fatal("Failed to create valid response") } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } modifiedRequest := request @@ -655,13 +658,13 @@ func TestGetLeafs(t *testing.T) { t.Fatal("Failed to create valid response") } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } leafResponse.Keys = leafResponse.Keys[:len(leafResponse.Keys)-2] leafResponse.Vals = leafResponse.Vals[:len(leafResponse.Vals)-2] - modifiedResponse, err := message.Codec.Marshal(message.Version, leafResponse) + modifiedResponse, err := networkCodec.Marshal(message.Version, leafResponse) if err != nil { t.Fatal(err) } @@ -686,14 +689,14 @@ func TestGetLeafs(t *testing.T) { t.Fatal("Failed to create valid response") } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } // Remove middle key-value pair response leafResponse.Keys = append(leafResponse.Keys[:100], leafResponse.Keys[101:]...) leafResponse.Vals = append(leafResponse.Vals[:100], leafResponse.Vals[101:]...) - modifiedResponse, err := message.Codec.Marshal(message.Version, leafResponse) + modifiedResponse, err := networkCodec.Marshal(message.Version, leafResponse) if err != nil { t.Fatal(err) } @@ -718,13 +721,13 @@ func TestGetLeafs(t *testing.T) { t.Fatal("Failed to create valid response") } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } // Remove middle key-value pair response leafResponse.Vals[100] = []byte("garbage value data") - modifiedResponse, err := message.Codec.Marshal(message.Version, leafResponse) + modifiedResponse, err := networkCodec.Marshal(message.Version, leafResponse) if err != nil { t.Fatal(err) } @@ -750,13 +753,13 @@ func TestGetLeafs(t *testing.T) { } var leafResponse message.LeafsResponse - if _, err := message.Codec.Unmarshal(response, &leafResponse); err != nil { + if _, err := networkCodec.Unmarshal(response, &leafResponse); err != nil { t.Fatal(err) } // Remove the proof leafResponse.ProofVals = nil - modifiedResponse, err := message.Codec.Marshal(message.Version, leafResponse) + modifiedResponse, err := networkCodec.Marshal(message.Version, leafResponse) if err != nil { t.Fatal(err) } @@ -797,13 +800,13 @@ func TestGetLeafsRetries(t *testing.T) { trieDB := triedb.NewDatabase(rawdb.NewMemoryDatabase(), nil) root, _, _ := statesynctest.GenerateTrie(t, trieDB, 100_000, common.HashLength) - handler := handlers.NewLeafsRequestHandler(trieDB, nil, message.Codec, handlerstats.NewNoopHandlerStats()) + handler := handlers.NewLeafsRequestHandler(trieDB, message.StateTrieKeyLength, nil, networkCodec, handlerstats.NewNoopHandlerStats()) testNetClient := &testNetwork{} const maxAttempts = 8 client := NewClient(&ClientConfig{ NetworkClient: testNetClient, - Codec: message.Codec, + Codec: networkCodec, Stats: clientstats.NewNoOpStats(), StateSyncNodeIDs: nil, BlockParser: newTestBlockParser(), @@ -865,7 +868,7 @@ func TestStateSyncNodes(t *testing.T) { } client := NewClient(&ClientConfig{ NetworkClient: testNetClient, - Codec: message.Codec, + Codec: networkCodec, Stats: clientstats.NewNoOpStats(), StateSyncNodeIDs: stateSyncNodes, BlockParser: newTestBlockParser(), diff --git a/sync/client/stats/stats.go b/sync/client/stats/stats.go index 61822e40c1..cd9b3eb0e7 100644 --- a/sync/client/stats/stats.go +++ b/sync/client/stats/stats.go @@ -75,19 +75,21 @@ func (m *messageMetric) UpdateRequestLatency(duration time.Duration) { } type clientSyncerStats struct { - atomicTrieLeavesMetric, - stateTrieLeavesMetric, + leafMetrics map[message.NodeType]MessageMetric codeRequestMetric, blockRequestMetric MessageMetric } // NewClientSyncerStats returns stats for the client syncer -func NewClientSyncerStats() ClientSyncerStats { +func NewClientSyncerStats(leafMetricNames map[message.NodeType]string) *clientSyncerStats { + leafMetrics := make(map[message.NodeType]MessageMetric, len(leafMetricNames)) + for nodeType, name := range leafMetricNames { + leafMetrics[nodeType] = NewMessageMetric(name) + } return &clientSyncerStats{ - atomicTrieLeavesMetric: NewMessageMetric("sync_atomic_trie_leaves"), - stateTrieLeavesMetric: NewMessageMetric("sync_state_trie_leaves"), - codeRequestMetric: NewMessageMetric("sync_code"), - blockRequestMetric: NewMessageMetric("sync_blocks"), + leafMetrics: leafMetrics, + codeRequestMetric: NewMessageMetric("sync_code"), + blockRequestMetric: NewMessageMetric("sync_blocks"), } } @@ -99,14 +101,11 @@ func (c *clientSyncerStats) GetMetric(msgIntf message.Request) (MessageMetric, e case message.CodeRequest: return c.codeRequestMetric, nil case message.LeafsRequest: - switch msg.NodeType { - case message.StateTrieNode: - return c.stateTrieLeavesMetric, nil - case message.AtomicTrieNode: - return c.atomicTrieLeavesMetric, nil - default: + metric, ok := c.leafMetrics[msg.NodeType] + if !ok { return nil, fmt.Errorf("invalid leafs request for node type: %T", msg.NodeType) } + return metric, nil default: return nil, fmt.Errorf("attempted to get metric for invalid request with type %T", msg) } @@ -133,12 +132,3 @@ func NewNoOpStats() ClientSyncerStats { func (n noopStats) GetMetric(_ message.Request) (MessageMetric, error) { return n.noop, nil } - -// NewStats returns syncer stats if enabled or a no-op version if disabled. -func NewStats(enabled bool) ClientSyncerStats { - if enabled { - return NewClientSyncerStats() - } else { - return NewNoOpStats() - } -} diff --git a/sync/client/test_client.go b/sync/client/test_client.go index 7c0d6d9c58..81b8c12e5f 100644 --- a/sync/client/test_client.go +++ b/sync/client/test_client.go @@ -24,7 +24,7 @@ var ( type TestClient struct { codec codec.Manager - leafsHandler *handlers.LeafsRequestHandler + leafsHandler handlers.LeafRequestHandler leavesReceived int32 codesHandler *handlers.CodeRequestHandler codeReceived int32 @@ -43,7 +43,7 @@ type TestClient struct { func NewTestClient( codec codec.Manager, - leafHandler *handlers.LeafsRequestHandler, + leafHandler handlers.LeafRequestHandler, codesHandler *handlers.CodeRequestHandler, blocksHandler *handlers.BlockRequestHandler, ) *TestClient { diff --git a/sync/handlers/block_request_test.go b/sync/handlers/block_request_test.go index 6db3ac489d..b16b5e2056 100644 --- a/sync/handlers/block_request_test.go +++ b/sync/handlers/block_request_test.go @@ -56,7 +56,7 @@ func executeBlockRequestTest(t testing.TB, test blockRequestTest, blocks []*type return blk }, } - blockRequestHandler := NewBlockRequestHandler(blockProvider, message.Codec, testHandlerStats) + blockRequestHandler := NewBlockRequestHandler(blockProvider, networkCodec, testHandlerStats) var blockRequest message.BlockRequest if test.startBlockHash != (common.Hash{}) { @@ -85,7 +85,7 @@ func executeBlockRequestTest(t testing.TB, test blockRequestTest, blocks []*type assert.NotEmpty(t, responseBytes) var response message.BlockResponse - if _, err = message.Codec.Unmarshal(responseBytes, &response); err != nil { + if _, err = networkCodec.Unmarshal(responseBytes, &response); err != nil { t.Fatal("error unmarshalling", err) } assert.Len(t, response.Blocks, test.expectedBlocks) @@ -253,7 +253,7 @@ func TestBlockRequestHandlerCtxExpires(t *testing.T) { return blk }, } - blockRequestHandler := NewBlockRequestHandler(blockProvider, message.Codec, stats.NewNoopHandlerStats()) + blockRequestHandler := NewBlockRequestHandler(blockProvider, networkCodec, stats.NewNoopHandlerStats()) responseBytes, err := blockRequestHandler.OnBlockRequest(ctx, ids.GenerateTestNodeID(), 1, message.BlockRequest{ Hash: blocks[10].Hash(), @@ -266,7 +266,7 @@ func TestBlockRequestHandlerCtxExpires(t *testing.T) { assert.NotEmpty(t, responseBytes) var response message.BlockResponse - if _, err = message.Codec.Unmarshal(responseBytes, &response); err != nil { + if _, err = networkCodec.Unmarshal(responseBytes, &response); err != nil { t.Fatal("error unmarshalling", err) } // requested 8 blocks, received cancelAfterNumRequests because of timeout diff --git a/sync/handlers/code_request_test.go b/sync/handlers/code_request_test.go index e7166c4cf5..45963f3da3 100644 --- a/sync/handlers/code_request_test.go +++ b/sync/handlers/code_request_test.go @@ -35,7 +35,7 @@ func TestCodeRequestHandler(t *testing.T) { rawdb.WriteCode(database, maxSizeCodeHash, maxSizeCodeBytes) testHandlerStats := &statstest.TestHandlerStats{} - codeRequestHandler := NewCodeRequestHandler(database, message.Codec, testHandlerStats) + codeRequestHandler := NewCodeRequestHandler(database, networkCodec, testHandlerStats) tests := map[string]struct { setup func() (request message.CodeRequest, expectedCodeResponse [][]byte) @@ -100,7 +100,7 @@ func TestCodeRequestHandler(t *testing.T) { return } var response message.CodeResponse - if _, err = message.Codec.Unmarshal(responseBytes, &response); err != nil { + if _, err = networkCodec.Unmarshal(responseBytes, &response); err != nil { t.Fatal("error unmarshalling CodeResponse", err) } if len(expectedResponse) != len(response.Data) { diff --git a/sync/handlers/leafs_request.go b/sync/handlers/leafs_request.go index 3ede906a37..d50c210c59 100644 --- a/sync/handlers/leafs_request.go +++ b/sync/handlers/leafs_request.go @@ -6,13 +6,11 @@ package handlers import ( "bytes" "context" - "fmt" "sync" "time" "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/ids" - "github.com/ava-labs/avalanchego/utils/wrappers" "github.com/ava-labs/coreth/core/state/snapshot" "github.com/ava-labs/coreth/plugin/evm/message" "github.com/ava-labs/coreth/sync/handlers/stats" @@ -27,6 +25,8 @@ import ( "github.com/ava-labs/libevm/triedb" ) +var _ LeafRequestHandler = (*leafsRequestHandler)(nil) + const ( // Maximum number of leaves to return in a message.LeafsResponse // This parameter overrides any other Limit specified @@ -40,22 +40,28 @@ const ( segmentLen = 64 // divide data from snapshot to segments of this size ) -// LeafsRequestHandler is a peer.RequestHandler for types.LeafsRequest +type LeafRequestHandler interface { + OnLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) +} + +// leafsRequestHandler is a peer.RequestHandler for types.LeafsRequest // serving requested trie data -type LeafsRequestHandler struct { +type leafsRequestHandler struct { trieDB *triedb.Database snapshotProvider SnapshotProvider codec codec.Manager stats stats.LeafsRequestHandlerStats pool sync.Pool + trieKeyLength int } -func NewLeafsRequestHandler(trieDB *triedb.Database, snapshotProvider SnapshotProvider, codec codec.Manager, syncerStats stats.LeafsRequestHandlerStats) *LeafsRequestHandler { - return &LeafsRequestHandler{ +func NewLeafsRequestHandler(trieDB *triedb.Database, trieKeyLength int, snapshotProvider SnapshotProvider, codec codec.Manager, syncerStats stats.LeafsRequestHandlerStats) *leafsRequestHandler { + return &leafsRequestHandler{ trieDB: trieDB, snapshotProvider: snapshotProvider, codec: codec, stats: syncerStats, + trieKeyLength: trieKeyLength, pool: sync.Pool{ New: func() interface{} { return make([][]byte, 0, maxLeavesLimit) }, }, @@ -70,10 +76,9 @@ func NewLeafsRequestHandler(trieDB *triedb.Database, snapshotProvider SnapshotPr // Specified Limit in message.LeafsRequest is overridden to maxLeavesLimit if it is greater than maxLeavesLimit // Expects returned errors to be treated as FATAL // Never returns errors -// Expects NodeType to be one of message.AtomicTrieNode or message.StateTrieNode // Returns nothing if NodeType is invalid or requested trie root is not found // Assumes ctx is active -func (lrh *LeafsRequestHandler) OnLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { +func (lrh *leafsRequestHandler) OnLeafsRequest(ctx context.Context, nodeID ids.NodeID, requestID uint32, leafsRequest message.LeafsRequest) ([]byte, error) { startTime := time.Now() lrh.stats.IncLeafsRequest() @@ -85,16 +90,9 @@ func (lrh *LeafsRequestHandler) OnLeafsRequest(ctx context.Context, nodeID ids.N lrh.stats.IncInvalidLeafsRequest() return nil, nil } - keyLength, err := getKeyLength(leafsRequest.NodeType) - if err != nil { - // Note: LeafsRequest.Handle checks NodeType's validity so clients cannot cause the server to spam this error - log.Error("Failed to get key length for leafs request", "err", err) - lrh.stats.IncInvalidLeafsRequest() - return nil, nil - } - if len(leafsRequest.Start) != 0 && len(leafsRequest.Start) != keyLength || - len(leafsRequest.End) != 0 && len(leafsRequest.End) != keyLength { - log.Debug("invalid length for leafs request range, dropping request", "startLen", len(leafsRequest.Start), "endLen", len(leafsRequest.End), "expected", keyLength) + if (len(leafsRequest.Start) != 0 && len(leafsRequest.Start) != lrh.trieKeyLength) || + (len(leafsRequest.End) != 0 && len(leafsRequest.End) != lrh.trieKeyLength) { + log.Debug("invalid length for leafs request range, dropping request", "startLen", len(leafsRequest.Start), "endLen", len(leafsRequest.End), "expected", lrh.trieKeyLength) lrh.stats.IncInvalidLeafsRequest() return nil, nil } @@ -134,7 +132,7 @@ func (lrh *LeafsRequestHandler) OnLeafsRequest(ctx context.Context, nodeID ids.N request: &leafsRequest, response: &leafsResponse, t: t, - keyLength: keyLength, + keyLength: lrh.trieKeyLength, limit: limit, stats: lrh.stats, } @@ -455,18 +453,6 @@ func (rb *responseBuilder) fillFromTrie(ctx context.Context, end []byte) (bool, return more, it.Err } -// getKeyLength returns trie key length for given nodeType -// expects nodeType to be one of message.AtomicTrieNode or message.StateTrieNode -func getKeyLength(nodeType message.NodeType) (int, error) { - switch nodeType { - case message.AtomicTrieNode: - return wrappers.LongLen + common.HashLength, nil - case message.StateTrieNode: - return common.HashLength, nil - } - return 0, fmt.Errorf("cannot get key length for unknown node type: %s", nodeType) -} - // readLeafsFromSnapshot iterates the storage snapshot of the requested account // (or the main account trie if account is empty). Returns up to [rb.limit] key/value // pairs for keys that are in the request's range (inclusive). diff --git a/sync/handlers/leafs_request_test.go b/sync/handlers/leafs_request_test.go index b31d95252e..2d5e46ab9c 100644 --- a/sync/handlers/leafs_request_test.go +++ b/sync/handlers/leafs_request_test.go @@ -12,6 +12,7 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/coreth/core/state/snapshot" "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/message/messagetest" "github.com/ava-labs/coreth/sync/handlers/stats/statstest" "github.com/ava-labs/coreth/sync/statesync/statesynctest" "github.com/ava-labs/libevm/common" @@ -24,6 +25,8 @@ import ( "github.com/stretchr/testify/assert" ) +var networkCodec = messagetest.BlockSyncSummaryCodec + func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { rand.Seed(1) testHandlerStats := &statstest.TestHandlerStats{} @@ -74,7 +77,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { } } snapshotProvider := &TestSnapshotProvider{} - leafsHandler := NewLeafsRequestHandler(trieDB, snapshotProvider, message.Codec, testHandlerStats) + leafsHandler := NewLeafsRequestHandler(trieDB, message.StateTrieKeyLength, snapshotProvider, networkCodec, testHandlerStats) snapConfig := snapshot.Config{ CacheSize: 64, AsyncBuild: false, @@ -228,7 +231,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, _ message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.Len(t, leafsResponse.Keys, 500) assert.Len(t, leafsResponse.Vals, 500) @@ -248,7 +251,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, _ message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.Len(t, leafsResponse.Keys, 500) assert.Len(t, leafsResponse.Vals, 500) @@ -302,7 +305,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, _ message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, len(leafsResponse.Keys), maxLeavesLimit) assert.EqualValues(t, len(leafsResponse.Vals), maxLeavesLimit) @@ -323,7 +326,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, len(leafsResponse.Keys), maxLeavesLimit) assert.EqualValues(t, len(leafsResponse.Vals), maxLeavesLimit) @@ -345,7 +348,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, len(leafsResponse.Keys), maxLeavesLimit) assert.EqualValues(t, len(leafsResponse.Vals), maxLeavesLimit) @@ -370,7 +373,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, 40, len(leafsResponse.Keys)) assert.EqualValues(t, 40, len(leafsResponse.Vals)) @@ -392,7 +395,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, 600, len(leafsResponse.Keys)) assert.EqualValues(t, 600, len(leafsResponse.Vals)) @@ -414,7 +417,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, len(leafsResponse.Keys), 0) assert.EqualValues(t, len(leafsResponse.Vals), 0) @@ -437,7 +440,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assert.NotEmpty(t, response) var leafsResponse message.LeafsResponse - if _, err = message.Codec.Unmarshal(response, &leafsResponse); err != nil { + if _, err = networkCodec.Unmarshal(response, &leafsResponse); err != nil { t.Fatalf("unexpected error when unmarshalling LeafsResponse: %v", err) } @@ -465,7 +468,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Keys)) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Vals)) @@ -513,7 +516,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Keys)) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Vals)) @@ -546,7 +549,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Keys)) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Vals)) @@ -592,7 +595,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Keys)) assert.EqualValues(t, maxLeavesLimit, len(leafsResponse.Vals)) @@ -633,7 +636,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, 500, len(leafsResponse.Keys)) assert.EqualValues(t, 500, len(leafsResponse.Vals)) @@ -670,7 +673,7 @@ func TestLeafsRequestHandler_OnLeafsRequest(t *testing.T) { assertResponseFn: func(t *testing.T, request message.LeafsRequest, response []byte, err error) { assert.NoError(t, err) var leafsResponse message.LeafsResponse - _, err = message.Codec.Unmarshal(response, &leafsResponse) + _, err = networkCodec.Unmarshal(response, &leafsResponse) assert.NoError(t, err) assert.EqualValues(t, 1, len(leafsResponse.Keys)) assert.EqualValues(t, 1, len(leafsResponse.Vals)) diff --git a/sync/handlers/stats/stats.go b/sync/handlers/stats/stats.go index 9acf088d8b..277320e4ed 100644 --- a/sync/handlers/stats/stats.go +++ b/sync/handlers/stats/stats.go @@ -166,7 +166,10 @@ func (h *handlerStats) IncSnapshotReadSuccess() { h.snapshotReadSuccess.Inc(1 func (h *handlerStats) IncSnapshotSegmentValid() { h.snapshotSegmentValid.Inc(1) } func (h *handlerStats) IncSnapshotSegmentInvalid() { h.snapshotSegmentInvalid.Inc(1) } -func NewHandlerStats(enabled bool) HandlerStats { +// GetOrRegisterHandlerStats returns a [HandlerStats] to track state sync handler metrics. +// If `enabled` is false, a no-op implementation is returned. +// if `enabled` is true, calling this multiple times will return the same registered metrics. +func GetOrRegisterHandlerStats(enabled bool) HandlerStats { if !enabled { return NewNoopHandlerStats() } diff --git a/sync/statesync/code_syncer_test.go b/sync/statesync/code_syncer_test.go index 2562b7db4d..75369a9149 100644 --- a/sync/statesync/code_syncer_test.go +++ b/sync/statesync/code_syncer_test.go @@ -9,15 +9,17 @@ import ( "testing" "github.com/ava-labs/avalanchego/utils" + "github.com/ava-labs/coreth/plugin/evm/customrawdb" - "github.com/ava-labs/coreth/plugin/evm/message" statesyncclient "github.com/ava-labs/coreth/sync/client" "github.com/ava-labs/coreth/sync/handlers" handlerstats "github.com/ava-labs/coreth/sync/handlers/stats" + "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/rawdb" "github.com/ava-labs/libevm/crypto" "github.com/ava-labs/libevm/ethdb/memorydb" + "github.com/stretchr/testify/assert" ) @@ -41,8 +43,8 @@ func testCodeSyncer(t *testing.T, test codeSyncerTest) { } // Set up mockClient - codeRequestHandler := handlers.NewCodeRequestHandler(serverDB, message.Codec, handlerstats.NewNoopHandlerStats()) - mockClient := statesyncclient.NewTestClient(message.Codec, nil, codeRequestHandler, nil) + codeRequestHandler := handlers.NewCodeRequestHandler(serverDB, networkCodec, handlerstats.NewNoopHandlerStats()) + mockClient := statesyncclient.NewTestClient(networkCodec, nil, codeRequestHandler, nil) mockClient.GetCodeIntercept = test.getCodeIntercept clientDB := rawdb.NewMemoryDatabase() diff --git a/sync/statesync/sync_test.go b/sync/statesync/sync_test.go index c47abd1388..9de9875c62 100644 --- a/sync/statesync/sync_test.go +++ b/sync/statesync/sync_test.go @@ -16,6 +16,7 @@ import ( "github.com/ava-labs/coreth/core/state/snapshot" "github.com/ava-labs/coreth/plugin/evm/customrawdb" "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/message/messagetest" statesyncclient "github.com/ava-labs/coreth/sync/client" "github.com/ava-labs/coreth/sync/handlers" handlerstats "github.com/ava-labs/coreth/sync/handlers/stats" @@ -35,6 +36,8 @@ const testSyncTimeout = 30 * time.Second var errInterrupted = errors.New("interrupted sync") +var networkCodec = messagetest.BlockSyncSummaryCodec + type syncTest struct { ctx context.Context prepareForTest func(t *testing.T) (clientDB ethdb.Database, serverDB ethdb.Database, serverTrieDB *triedb.Database, syncRoot common.Hash) @@ -50,9 +53,9 @@ func testSync(t *testing.T, test syncTest) { ctx = test.ctx } clientDB, serverDB, serverTrieDB, root := test.prepareForTest(t) - leafsRequestHandler := handlers.NewLeafsRequestHandler(serverTrieDB, nil, message.Codec, handlerstats.NewNoopHandlerStats()) - codeRequestHandler := handlers.NewCodeRequestHandler(serverDB, message.Codec, handlerstats.NewNoopHandlerStats()) - testClient := statesyncclient.NewTestClient(message.Codec, leafsRequestHandler, codeRequestHandler, nil) + leafsRequestHandler := handlers.NewLeafsRequestHandler(serverTrieDB, message.StateTrieKeyLength, nil, networkCodec, handlerstats.NewNoopHandlerStats()) + codeRequestHandler := handlers.NewCodeRequestHandler(serverDB, networkCodec, handlerstats.NewNoopHandlerStats()) + testClient := statesyncclient.NewTestClient(networkCodec, leafsRequestHandler, codeRequestHandler, nil) // Set intercept functions for the test client testClient.GetLeafsIntercept = test.GetLeafsIntercept testClient.GetCodeIntercept = test.GetCodeIntercept diff --git a/utils/handler.go b/utils/handler.go new file mode 100644 index 0000000000..194bf8a09d --- /dev/null +++ b/utils/handler.go @@ -0,0 +1,21 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. +package utils + +import ( + "net/http" + + "github.com/ava-labs/avalanchego/utils/json" + "github.com/gorilla/rpc/v2" +) + +// NewHandler returns a new Handler for a service where: +// - The handler's functionality is defined by [service] +// [service] should be a gorilla RPC service (see https://www.gorillatoolkit.org/pkg/rpc/v2) +// - The name of the service is [name] +func NewHandler(name string, service interface{}) (http.Handler, error) { + server := rpc.NewServer() + server.RegisterCodec(json.NewCodec(), "application/json") + server.RegisterCodec(json.NewCodec(), "application/json;charset=UTF-8") + return server, server.RegisterService(service, name) +} diff --git a/warp/handlers/signature_request_test.go b/warp/handlers/signature_request_test.go index 404759d8d4..e62fad8051 100644 --- a/warp/handlers/signature_request_test.go +++ b/warp/handlers/signature_request_test.go @@ -16,13 +16,14 @@ import ( "github.com/ava-labs/avalanchego/vms/platformvm/warp/payload" "github.com/ava-labs/coreth/metrics/metricstest" "github.com/ava-labs/coreth/plugin/evm/message" + "github.com/ava-labs/coreth/plugin/evm/message/messagetest" "github.com/ava-labs/coreth/warp" "github.com/ava-labs/coreth/warp/warptest" "github.com/stretchr/testify/require" ) -var networkCodec = message.Codec +var networkCodec = messagetest.BlockSyncSummaryCodec func TestMessageSignatureHandler(t *testing.T) { metricstest.WithMetrics(t)