From fbbc928c5bf96fdb1bb9e056cf224f512f12a785 Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Wed, 9 Feb 2022 00:35:35 +0300 Subject: [PATCH 1/8] merge coreth-0.8.5-rc.2 --- consensus/dummy/dynamic_fees.go | 12 + consensus/dummy/test_consensus.go | 53 +++ core/blockchain.go | 77 +++- core/blockchain_test.go | 85 ++++ core/genesis.go | 12 +- core/genesis_test.go | 71 ++-- core/rawdb/accessors_metadata.go | 34 ++ core/rawdb/schema.go | 3 + core/state/pruner/bloom.go | 142 +++++++ core/state/pruner/pruner.go | 417 +++++++++++++++++++ core/test_blockchain.go | 80 ++-- core/tx_pool.go | 4 +- core/vm/evm.go | 9 + eth/backend.go | 62 ++- eth/ethconfig/config.go | 46 +-- eth/gasprice/gasprice.go | 9 +- go.mod | 5 +- go.sum | 13 +- metrics/prometheus/prometheus.go | 3 +- params/config.go | 34 +- params/config_test.go | 138 +++++++ peer/client.go | 52 +++ peer/network.go | 343 ++++++++++++++++ peer/network_test.go | 478 ++++++++++++++++++++++ peer/waiting_handler.go | 39 ++ plugin/evm/block_builder.go | 26 +- plugin/evm/block_builder_test.go | 2 +- plugin/evm/config.go | 42 +- plugin/evm/{network.go => gossiper.go} | 279 ++++--------- plugin/evm/gossiper_eth_gossiping_test.go | 375 +++++++++++++++++ plugin/evm/message/codec.go | 26 +- plugin/evm/message/handler.go | 30 +- plugin/evm/message/handler_test.go | 14 +- plugin/evm/message/message.go | 20 +- plugin/evm/message/message_test.go | 18 +- plugin/evm/message/request.go | 36 ++ plugin/evm/vm.go | 154 +++++-- plugin/evm/vm_test.go | 85 ++-- rpc/handler.go | 5 +- scripts/lint.sh | 0 40 files changed, 2865 insertions(+), 468 deletions(-) create mode 100644 consensus/dummy/test_consensus.go create mode 100644 core/state/pruner/bloom.go create mode 100644 core/state/pruner/pruner.go create mode 100644 params/config_test.go create mode 100644 peer/client.go create mode 100644 peer/network.go create mode 100644 peer/network_test.go create mode 100644 peer/waiting_handler.go rename plugin/evm/{network.go => gossiper.go} (51%) create mode 100644 plugin/evm/gossiper_eth_gossiping_test.go create mode 100644 plugin/evm/message/request.go mode change 100644 => 100755 scripts/lint.sh diff --git a/consensus/dummy/dynamic_fees.go b/consensus/dummy/dynamic_fees.go index 9f8408dfa9..d66d73d64d 100644 --- a/consensus/dummy/dynamic_fees.go +++ b/consensus/dummy/dynamic_fees.go @@ -107,6 +107,18 @@ func CalcBaseFee(config *params.ChainConfig, parent *types.Header, timestamp uin return newRollupWindow, baseFee, nil } +// EstiamteNextBaseFee attempts to estimate the next base fee based on a block with [parent] being built at +// [timestamp]. +// If [timestamp] is less than the timestamp of [parent], then it uses the same timestamp as parent. +// Warning: This function should only be used in estimation and should not be used when calculating the canonical +// base fee for a subsequent block. +func EstimateNextBaseFee(config *params.ChainConfig, parent *types.Header, timestamp uint64) ([]byte, *big.Int, error) { + if timestamp < parent.Time { + timestamp = parent.Time + } + return CalcBaseFee(config, parent, timestamp) +} + // selectBigWithinBounds returns [value] if it is within the bounds: // lowerBound <= value <= upperBound or the bound at either end if [value] // is outside of the defined boundaries. diff --git a/consensus/dummy/test_consensus.go b/consensus/dummy/test_consensus.go new file mode 100644 index 0000000000..b2e0f7e6d1 --- /dev/null +++ b/consensus/dummy/test_consensus.go @@ -0,0 +1,53 @@ +// (c) 2019-2020, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package dummy + +import ( + "github.com/ava-labs/subnet-evm/consensus" + "github.com/ava-labs/subnet-evm/core/state" + "github.com/ava-labs/subnet-evm/core/types" +) + +type ( + OnFinalizeAndAssembleCallbackType = func(header *types.Header, state *state.StateDB, txs []*types.Transaction) (err error) + OnExtraStateChangeType = func(block *types.Block, statedb *state.StateDB) (err error) + + ConsensusCallbacks struct { + OnFinalizeAndAssemble OnFinalizeAndAssembleCallbackType + OnExtraStateChange OnExtraStateChangeType + } + + DummyEngineCB struct { + cb *ConsensusCallbacks + DummyEngine + } +) + +func NewTestConsensusCB(cb *ConsensusCallbacks) *DummyEngineCB { + return &DummyEngineCB{ + cb: cb, + } +} + +func (self *DummyEngineCB) Finalize(chain consensus.ChainHeaderReader, block *types.Block, parent *types.Header, state *state.StateDB, receipts []*types.Receipt) error { + if self.cb.OnExtraStateChange != nil { + err := self.cb.OnExtraStateChange(block, state) + if err != nil { + return err + } + } + + return self.DummyEngine.Finalize(chain, block, parent, state, receipts) +} + +func (self *DummyEngineCB) FinalizeAndAssemble(chain consensus.ChainHeaderReader, header *types.Header, parent *types.Header, state *state.StateDB, txs []*types.Transaction, + uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) { + if self.cb.OnFinalizeAndAssemble != nil { + err := self.cb.OnFinalizeAndAssemble(header, state, txs) + if err != nil { + return nil, err + } + } + return self.DummyEngine.FinalizeAndAssemble(chain, header, parent, state, txs, uncles, receipts) +} diff --git a/core/blockchain.go b/core/blockchain.go index c25a659b47..216354ddf1 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1080,15 +1080,15 @@ func (bc *BlockChain) reorg(oldBlock, newBlock *types.Block) error { // Ensure the user sees large reorgs if len(oldChain) > 0 && len(newChain) > 0 { logFn := log.Info - msg := "Chain reorg detected" + msg := "Resetting chain preference" if len(oldChain) > 63 { - msg = "Large chain reorg detected" + msg = "Large chain preference change detected" logFn = log.Warn } logFn(msg, "number", commonBlock.Number(), "hash", commonBlock.Hash(), "drop", len(oldChain), "dropfrom", oldChain[0].Hash(), "add", len(newChain), "addfrom", newChain[0].Hash()) } else { - log.Warn("Unlikely reorg (rewind to ancestor) occurred", "oldnum", oldHead.Number(), "oldhash", oldHead.Hash(), "newnum", newHead.Number(), "newhash", newHead.Hash()) + log.Warn("Unlikely preference change (rewind to ancestor) occurred", "oldnum", oldHead.Number(), "oldhash", oldHead.Hash(), "newnum", newHead.Number(), "newhash", newHead.Hash()) } // Insert the new chain(except the head block(reverse order)), // taking care of the proper incremental order. @@ -1293,3 +1293,74 @@ func (bc *BlockChain) reprocessState(current *types.Block, reexec uint64) error } return nil } + +// CleanBlockRootsAboveLastAccepted gathers the blocks that may have previously been in processing above the +// last accepted block and wipes their block roots from disk to mark their tries as inaccessible. +// This is used prior to pruning to ensure that all of the tries that may still be in processing are marked +// as inaccessible and mirrors the handling of middle roots in the geth offline pruning implementation. +// This is not strictly necessary, but maintains a soft assumption. +func (bc *BlockChain) CleanBlockRootsAboveLastAccepted() error { + targetRoot := bc.LastAcceptedBlock().Root() + + // Clean up any block roots above the last accepted block before we start pruning. + // Note: this takes the place of middleRoots in the geth implementation since we do not + // track processing block roots via snapshot journals in the same way. + processingRoots := bc.gatherBlockRootsAboveLastAccepted() + // If there is a block above the last accepted block with an identical state root, we + // explicitly remove it from the set to ensure we do not corrupt the last accepted trie. + delete(processingRoots, targetRoot) + for processingRoot := range processingRoots { + // Delete the processing root from disk to mark the trie as inaccessible (no need to handle this in a batch). + if err := bc.db.Delete(processingRoot[:]); err != nil { + return fmt.Errorf("failed to remove processing root (%s) preparing for offline pruning: %w", processingRoot, err) + } + } + + return nil +} + +// gatherBlockRootsAboveLastAccepted iterates forward from the last accepted block and returns a list of all block roots +// for any blocks that were inserted above the last accepted block. +// Given that we never insert a block into the chain unless all of its ancestors have been inserted, this should gather +// all of the block roots for blocks inserted above the last accepted block that may have been in processing at some point +// in the past and are therefore potentially still acceptable. +// Note: there is an edge case where the node dies while the consensus engine is rejecting a branch of blocks since the +// consensus engine will reject the lowest ancestor first. In this case, these blocks will not be considered acceptable in +// the future. +// Ex. +// A +// / \ +// B C +// | +// D +// | +// E +// | +// F +// +// The consensus engine accepts block C and proceeds to reject the other branch in order (B, D, E, F). +// If the consensus engine dies after rejecting block D, block D will be deleted, such that the forward iteration +// may not find any blocks at this height and will not reach the previously processing blocks E and F. +func (bc *BlockChain) gatherBlockRootsAboveLastAccepted() map[common.Hash]struct{} { + blockRoots := make(map[common.Hash]struct{}) + for height := bc.lastAccepted.NumberU64() + 1; ; height++ { + blockHashes := rawdb.ReadAllHashes(bc.db, height) + // If there are no block hashes at [height], then there should be no further acceptable blocks + // past this point. + if len(blockHashes) == 0 { + break + } + + // Fetch the blocks and append their roots. + for _, blockHash := range blockHashes { + block := bc.GetBlockByHash(blockHash) + if block == nil { + continue + } + + blockRoots[block.Root()] = struct{}{} + } + } + + return blockRoots +} diff --git a/core/blockchain_test.go b/core/blockchain_test.go index e7d13fca00..b070af0171 100644 --- a/core/blockchain_test.go +++ b/core/blockchain_test.go @@ -4,10 +4,14 @@ package core import ( + "fmt" "testing" "github.com/ava-labs/subnet-evm/consensus/dummy" "github.com/ava-labs/subnet-evm/core/rawdb" + "github.com/ava-labs/subnet-evm/core/state" + "github.com/ava-labs/subnet-evm/core/state/pruner" + "github.com/ava-labs/subnet-evm/core/types" "github.com/ava-labs/subnet-evm/core/vm" "github.com/ava-labs/subnet-evm/ethdb" "github.com/ava-labs/subnet-evm/params" @@ -248,3 +252,84 @@ func TestCorruptSnapshots(t *testing.T) { }) } } + +func TestBlockChainOfflinePruningUngracefulShutdown(t *testing.T) { + create := func(db ethdb.Database, chainConfig *params.ChainConfig, lastAcceptedHash common.Hash) (*BlockChain, error) { + // Import the chain. This runs all block validation rules. + blockchain, err := NewBlockChain( + db, + &CacheConfig{ + TrieCleanLimit: 256, + TrieDirtyLimit: 256, + Pruning: true, // Enable pruning + SnapshotLimit: 256, + }, + chainConfig, + dummy.NewTestConsensusCB(&dummy.ConsensusCallbacks{ + OnExtraStateChange: func(block *types.Block, sdb *state.StateDB) error { + sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) + return nil + }, + OnFinalizeAndAssemble: func(header *types.Header, sdb *state.StateDB, txs []*types.Transaction) error { + sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) + return nil + }, + }), + vm.Config{}, + lastAcceptedHash, + ) + if err != nil { + return nil, err + } + + // Overwrite state manager, so that Shutdown is not called. + // This tests to ensure that the state manager handles an ungraceful shutdown correctly. + blockchain.stateManager = &wrappedStateManager{TrieWriter: blockchain.stateManager} + + if lastAcceptedHash == (common.Hash{}) { + return blockchain, nil + } + + tempDir := t.TempDir() + if err := blockchain.CleanBlockRootsAboveLastAccepted(); err != nil { + return nil, err + } + pruner, err := pruner.NewPruner(db, tempDir, 256) + if err != nil { + return nil, fmt.Errorf("offline pruning failed (%s, %d): %w", tempDir, 256, err) + } + + targetRoot := blockchain.LastAcceptedBlock().Root() + if err := pruner.Prune(targetRoot); err != nil { + return nil, fmt.Errorf("failed to prune blockchain with target root: %s due to: %w", targetRoot, err) + } + // Re-initialize the blockchain after pruning + return NewBlockChain( + db, + &CacheConfig{ + TrieCleanLimit: 256, + TrieDirtyLimit: 256, + Pruning: true, // Enable pruning + SnapshotLimit: 256, + }, + chainConfig, + dummy.NewTestConsensusCB(&dummy.ConsensusCallbacks{ + OnExtraStateChange: func(block *types.Block, sdb *state.StateDB) error { + sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) + return nil + }, + OnFinalizeAndAssemble: func(header *types.Header, sdb *state.StateDB, txs []*types.Transaction) error { + sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) + return nil + }, + }), + vm.Config{}, + lastAcceptedHash, + ) + } + for _, tt := range tests { + t.Run(tt.Name, func(t *testing.T) { + tt.testFunc(t, create) + }) + } +} diff --git a/core/genesis.go b/core/genesis.go index 2d61685890..08e5b616bf 100644 --- a/core/genesis.go +++ b/core/genesis.go @@ -224,12 +224,14 @@ func SetupGenesisBlock(db ethdb.Database, genesis *Genesis) (*params.ChainConfig // Check config compatibility and write the config. Compatibility errors // are returned to the caller unless we're already at block zero. - height := rawdb.ReadHeaderNumber(db, rawdb.ReadHeadHeaderHash(db)) - if height == nil { - return newcfg, fmt.Errorf("missing block number for head header hash") + headBlock := rawdb.ReadHeadBlock(db) + if headBlock == nil { + return newcfg, fmt.Errorf("missing head block") } - compatErr := storedcfg.CheckCompatible(newcfg, *height) - if compatErr != nil && *height != 0 && compatErr.RewindTo != 0 { + height := headBlock.NumberU64() + timestamp := headBlock.Time() + compatErr := storedcfg.CheckCompatible(newcfg, height, timestamp) + if compatErr != nil && height != 0 && compatErr.RewindTo != 0 { return newcfg, compatErr } rawdb.WriteChainConfig(db, stored, newcfg) diff --git a/core/genesis_test.go b/core/genesis_test.go index f9b81968ed..cb50d3340d 100644 --- a/core/genesis_test.go +++ b/core/genesis_test.go @@ -31,7 +31,6 @@ import ( "math/big" "reflect" "testing" - "time" "github.com/ava-labs/subnet-evm/consensus/dummy" "github.com/ava-labs/subnet-evm/core/rawdb" @@ -57,23 +56,22 @@ func TestGenesisBlockForTesting(t *testing.T) { } func TestSetupGenesis(t *testing.T) { + preSubnetConfig := *params.TestPreSubnetEVMConfig + preSubnetConfig.SubnetEVMTimestamp = big.NewInt(100) var ( customghash = common.HexToHash("0x89c99d90b79719238d2645c7642f2c9295246e80775b38cfd162b696817fbd50") customg = Genesis{ - Config: ¶ms.ChainConfig{ - HomesteadBlock: big.NewInt(0), - SubnetEVMTimestamp: big.NewInt(time.Date(2021, time.July, 31, 14, 0, 0, 0, time.UTC).Unix()), - }, + Config: &preSubnetConfig, Alloc: GenesisAlloc{ {1}: {Balance: big.NewInt(1), Storage: map[common.Hash]common.Hash{{1}: {1}}}, }, } oldcustomg = customg ) - oldcustomg.Config = ¶ms.ChainConfig{ - HomesteadBlock: big.NewInt(0), - SubnetEVMTimestamp: big.NewInt(time.Date(2021, time.March, 31, 14, 0, 0, 0, time.UTC).Unix()), - } + + rollbackpreSubnetConfig := preSubnetConfig + rollbackpreSubnetConfig.SubnetEVMTimestamp = big.NewInt(90) + oldcustomg.Config = &rollbackpreSubnetConfig tests := []struct { name string fn func(ethdb.Database) (*params.ChainConfig, common.Hash, error) @@ -117,46 +115,53 @@ func TestSetupGenesis(t *testing.T) { wantConfig: customg.Config, }, { - name: "incompatible config in DB", + name: "incompatible config for avalanche fork in DB", fn: func(db ethdb.Database) (*params.ChainConfig, common.Hash, error) { + // Commit the 'old' genesis block with SubnetEVM transition at 90. + // Advance to block #4, past the SubnetEVM transition block of customg. genesis := oldcustomg.MustCommit(db) + bc, _ := NewBlockChain(db, DefaultCacheConfig, oldcustomg.Config, dummy.NewFullFaker(), vm.Config{}, common.Hash{}) defer bc.Stop() - blocks, _, _ := GenerateChain(oldcustomg.Config, genesis, dummy.NewFaker(), db, 4, 10, nil) + + blocks, _, _ := GenerateChain(oldcustomg.Config, genesis, dummy.NewFullFaker(), db, 4, 25, nil) bc.InsertChain(blocks) bc.CurrentBlock() + // This should return a compatibility error. return setupGenesisBlock(db, &customg) }, wantHash: customghash, wantConfig: customg.Config, wantErr: ¶ms.ConfigCompatError{ - What: "SubnetEVM fork block", - StoredConfig: big.NewInt(1617199200), - NewConfig: big.NewInt(1627740000), - RewindTo: 1617199199, + What: "SubnetEVM fork block timestamp", + StoredConfig: big.NewInt(90), + NewConfig: big.NewInt(100), + RewindTo: 89, }, }, } for _, test := range tests { - db := rawdb.NewMemoryDatabase() - config, hash, err := test.fn(db) - // Check the return values. - if !reflect.DeepEqual(err, test.wantErr) { - spew := spew.ConfigState{DisablePointerAddresses: true, DisableCapacities: true} - t.Errorf("%s: returned error %#v, want %#v", test.name, spew.NewFormatter(err), spew.NewFormatter(test.wantErr)) - } - if !reflect.DeepEqual(config, test.wantConfig) { - t.Errorf("%s:\nreturned %v\nwant %v", test.name, config, test.wantConfig) - } - if hash != test.wantHash { - t.Errorf("%s: returned hash %s, want %s", test.name, hash.Hex(), test.wantHash.Hex()) - } else if err == nil { - // Check database content. - stored := rawdb.ReadBlock(db, test.wantHash, 0) - if stored.Hash() != test.wantHash { - t.Errorf("%s: block in DB has hash %s, want %s", test.name, stored.Hash(), test.wantHash) + t.Run(test.name, func(t *testing.T) { + db := rawdb.NewMemoryDatabase() + config, hash, err := test.fn(db) + // Check the return values. + if !reflect.DeepEqual(err, test.wantErr) { + spew := spew.ConfigState{DisablePointerAddresses: true, DisableCapacities: true} + t.Errorf("returned error %#v, want %#v", spew.NewFormatter(err), spew.NewFormatter(test.wantErr)) } - } + if !reflect.DeepEqual(config, test.wantConfig) { + t.Errorf("returned %v\nwant %v", config, test.wantConfig) + } + if hash != test.wantHash { + t.Errorf("returned hash %s, want %s", hash.Hex(), test.wantHash.Hex()) + } else if err == nil { + // Check database content. + stored := rawdb.ReadBlock(db, test.wantHash, 0) + if stored.Hash() != test.wantHash { + t.Errorf("block in DB has hash %s, want %s", stored.Hash(), test.wantHash) + } + } + }) } } diff --git a/core/rawdb/accessors_metadata.go b/core/rawdb/accessors_metadata.go index c8da18cb66..f932bf4869 100644 --- a/core/rawdb/accessors_metadata.go +++ b/core/rawdb/accessors_metadata.go @@ -170,3 +170,37 @@ func UpdateUncleanShutdownMarker(db ethdb.KeyValueStore) { log.Warn("Failed to write unclean-shutdown marker", "err", err) } } + +// WriteOfflinePruning writes a marker of the last attempt to run offline pruning +// The marker is written when offline pruning completes and is deleted when the node +// is started successfully with offline pruning disabled. This ensures users must +// disable offline pruning and start their node successfully between runs of offline +// pruning. +func WriteOfflinePruning(db ethdb.KeyValueStore) error { + data, err := rlp.EncodeToBytes(uint64(time.Now().Unix())) + if err != nil { + return err + } + return db.Put(offlinePruningKey, data) +} + +// ReadOfflinePruning reads to check if there is a marker of the last attempt +// to run offline pruning. +func ReadOfflinePruning(db ethdb.KeyValueStore) (uint64, error) { + data, err := db.Get(offlinePruningKey) + if err != nil { + return 0, err + } + + var offlinePruningRun uint64 + if err := rlp.DecodeBytes(data, &offlinePruningRun); err != nil { + return 0, err + } + + return offlinePruningRun, nil +} + +// DeleteOfflinePruning deletes any marker of the last attempt to run offline pruning. +func DeleteOfflinePruning(db ethdb.KeyValueStore) error { + return db.Delete(offlinePruningKey) +} diff --git a/core/rawdb/schema.go b/core/rawdb/schema.go index 484f0da796..6f6701fc79 100644 --- a/core/rawdb/schema.go +++ b/core/rawdb/schema.go @@ -58,6 +58,9 @@ var ( // uncleanShutdownKey tracks the list of local crashes uncleanShutdownKey = []byte("unclean-shutdown") // config prefix for the db + // offlinePruningKey tracks runs of offline pruning + offlinePruningKey = []byte("OfflinePruning") + // Data item prefixes (use single byte to avoid mixing data types, avoid `i`, used for indexes). headerPrefix = []byte("h") // headerPrefix + num (uint64 big endian) + hash -> header headerHashSuffix = []byte("n") // headerPrefix + num (uint64 big endian) + headerHashSuffix -> hash diff --git a/core/state/pruner/bloom.go b/core/state/pruner/bloom.go new file mode 100644 index 0000000000..9b2c90e884 --- /dev/null +++ b/core/state/pruner/bloom.go @@ -0,0 +1,142 @@ +// (c) 2019-2020, Ava Labs, Inc. +// +// This file is a derived work, based on the go-ethereum library whose original +// notices appear below. +// +// It is distributed under a license compatible with the licensing terms of the +// original code from which it is derived. +// +// Much love to the original authors for their work. +// ********** +// Copyright 2020 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package pruner + +import ( + "encoding/binary" + "errors" + "os" + + "github.com/ava-labs/subnet-evm/core/rawdb" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/log" + bloomfilter "github.com/holiman/bloomfilter/v2" +) + +// stateBloomHasher is a wrapper around a byte blob to satisfy the interface API +// requirements of the bloom library used. It's used to convert a trie hash or +// contract code hash into a 64 bit mini hash. +type stateBloomHasher []byte + +func (f stateBloomHasher) Write(p []byte) (n int, err error) { panic("not implemented") } +func (f stateBloomHasher) Sum(b []byte) []byte { panic("not implemented") } +func (f stateBloomHasher) Reset() { panic("not implemented") } +func (f stateBloomHasher) BlockSize() int { panic("not implemented") } +func (f stateBloomHasher) Size() int { return 8 } +func (f stateBloomHasher) Sum64() uint64 { return binary.BigEndian.Uint64(f) } + +// stateBloom is a bloom filter used during the state convesion(snapshot->state). +// The keys of all generated entries will be recorded here so that in the pruning +// stage the entries belong to the specific version can be avoided for deletion. +// +// The false-positive is allowed here. The "false-positive" entries means they +// actually don't belong to the specific version but they are not deleted in the +// pruning. The downside of the false-positive allowance is we may leave some "dangling" +// nodes in the disk. But in practice the it's very unlike the dangling node is +// state root. So in theory this pruned state shouldn't be visited anymore. Another +// potential issue is for fast sync. If we do another fast sync upon the pruned +// database, it's problematic which will stop the expansion during the syncing. +// TODO address it @rjl493456442 @holiman @karalabe. +// +// After the entire state is generated, the bloom filter should be persisted into +// the disk. It indicates the whole generation procedure is finished. +type stateBloom struct { + bloom *bloomfilter.Filter +} + +// newStateBloomWithSize creates a brand new state bloom for state generation. +// The bloom filter will be created by the passing bloom filter size. According +// to the https://hur.st/bloomfilter/?n=600000000&p=&m=2048MB&k=4, the parameters +// are picked so that the false-positive rate for mainnet is low enough. +func newStateBloomWithSize(size uint64) (*stateBloom, error) { + bloom, err := bloomfilter.New(size*1024*1024*8, 4) + if err != nil { + return nil, err + } + log.Info("Initialized state bloom", "size", common.StorageSize(float64(bloom.M()/8))) + return &stateBloom{bloom: bloom}, nil +} + +// NewStateBloomFromDisk loads the state bloom from the given file. +// In this case the assumption is held the bloom filter is complete. +func NewStateBloomFromDisk(filename string) (*stateBloom, error) { + bloom, _, err := bloomfilter.ReadFile(filename) + if err != nil { + return nil, err + } + return &stateBloom{bloom: bloom}, nil +} + +// Commit flushes the bloom filter content into the disk and marks the bloom +// as complete. +func (bloom *stateBloom) Commit(filename, tempname string) error { + // Write the bloom out into a temporary file + _, err := bloom.bloom.WriteFile(tempname) + if err != nil { + return err + } + // Ensure the file is synced to disk + f, err := os.OpenFile(tempname, os.O_RDWR, 0666) + if err != nil { + return err + } + if err := f.Sync(); err != nil { + f.Close() + return err + } + f.Close() + + // Move the teporary file into it's final location + return os.Rename(tempname, filename) +} + +// Put implements the KeyValueWriter interface. But here only the key is needed. +func (bloom *stateBloom) Put(key []byte, value []byte) error { + // If the key length is not 32bytes, ensure it's contract code + // entry with new scheme. + if len(key) != common.HashLength { + isCode, codeKey := rawdb.IsCodeKey(key) + if !isCode { + return errors.New("invalid entry") + } + bloom.bloom.Add(stateBloomHasher(codeKey)) + return nil + } + bloom.bloom.Add(stateBloomHasher(key)) + return nil +} + +// Delete removes the key from the key-value data store. +func (bloom *stateBloom) Delete(key []byte) error { panic("not supported") } + +// Contain is the wrapper of the underlying contains function which +// reports whether the key is contained. +// - If it says yes, the key may be contained +// - If it says no, the key is definitely not contained. +func (bloom *stateBloom) Contain(key []byte) (bool, error) { + return bloom.bloom.Contains(stateBloomHasher(key)), nil +} diff --git a/core/state/pruner/pruner.go b/core/state/pruner/pruner.go new file mode 100644 index 0000000000..4092042220 --- /dev/null +++ b/core/state/pruner/pruner.go @@ -0,0 +1,417 @@ +// (c) 2019-2020, Ava Labs, Inc. +// +// This file is a derived work, based on the go-ethereum library whose original +// notices appear below. +// +// It is distributed under a license compatible with the licensing terms of the +// original code from which it is derived. +// +// Much love to the original authors for their work. +// ********** +// Copyright 2020 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package pruner + +import ( + "bytes" + "encoding/binary" + "errors" + "fmt" + "math" + "os" + "path/filepath" + "strings" + "time" + + "github.com/ava-labs/subnet-evm/core/rawdb" + "github.com/ava-labs/subnet-evm/core/state/snapshot" + "github.com/ava-labs/subnet-evm/core/types" + "github.com/ava-labs/subnet-evm/ethdb" + "github.com/ava-labs/subnet-evm/trie" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/rlp" +) + +const ( + // stateBloomFilePrefix is the filename prefix of state bloom filter. + stateBloomFilePrefix = "statebloom" + + // stateBloomFilePrefix is the filename suffix of state bloom filter. + stateBloomFileSuffix = "bf.gz" + + // stateBloomFileTempSuffix is the filename suffix of state bloom filter + // while it is being written out to detect write aborts. + stateBloomFileTempSuffix = ".tmp" + + // rangeCompactionThreshold is the minimal deleted entry number for + // triggering range compaction. It's a quite arbitrary number but just + // to avoid triggering range compaction because of small deletion. + rangeCompactionThreshold = 100000 +) + +var ( + // emptyRoot is the known root hash of an empty trie. + emptyRoot = common.HexToHash("56e81f171bcc55a6ff8345e692c0f86e5b48e01b996cadc001622fb5e363b421") + + // emptyCode is the known hash of the empty EVM bytecode. + emptyCode = crypto.Keccak256(nil) +) + +// Pruner is an offline tool to prune the stale state with the +// help of the snapshot. The workflow of pruner is very simple: +// +// - iterate the snapshot, reconstruct the relevant state +// - iterate the database, delete all other state entries which +// don't belong to the target state and the genesis state +// +// It can take several hours(around 2 hours for mainnet) to finish +// the whole pruning work. It's recommended to run this offline tool +// periodically in order to release the disk usage and improve the +// disk read performance to some extent. +type Pruner struct { + db ethdb.Database + stateBloom *stateBloom + datadir string + headHeader *types.Header + snaptree *snapshot.Tree +} + +// NewPruner creates the pruner instance. +func NewPruner(db ethdb.Database, datadir string, bloomSize uint64) (*Pruner, error) { + headBlock := rawdb.ReadHeadBlock(db) + if headBlock == nil { + return nil, errors.New("Failed to load head block") + } + // Note: we refuse to start a pruning session unless the snapshot disk layer exists, which should prevent + // us from ever needing to enter RecoverPruning in an invalid pruning session (a session where we do not have + // the protected trie in the triedb and in the snapshot disk layer). + snaptree, err := snapshot.New(db, trie.NewDatabase(db), 256, headBlock.Hash(), headBlock.Root(), false, false, false) + if err != nil { + return nil, fmt.Errorf("failed to create snapshot for pruning, must restart without offline pruning disabled to recover: %w", err) // The relevant snapshot(s) might not exist + } + // Sanitize the bloom filter size if it's too small. + if bloomSize < 256 { + log.Warn("Sanitizing bloomfilter size", "provided(MB)", bloomSize, "updated(MB)", 256) + bloomSize = 256 + } + stateBloom, err := newStateBloomWithSize(bloomSize) + if err != nil { + return nil, err + } + return &Pruner{ + db: db, + stateBloom: stateBloom, + datadir: datadir, + headHeader: headBlock.Header(), + snaptree: snaptree, + }, nil +} + +func prune(maindb ethdb.Database, stateBloom *stateBloom, bloomPath string, start time.Time) error { + // Delete all stale trie nodes in the disk. With the help of state bloom + // the trie nodes(and codes) belong to the active state will be filtered + // out. A very small part of stale tries will also be filtered because of + // the false-positive rate of bloom filter. But the assumption is held here + // that the false-positive is low enough(~0.05%). The probablity of the + // dangling node is the state root is super low. So the dangling nodes in + // theory will never ever be visited again. + var ( + count int + size common.StorageSize + pstart = time.Now() + logged = time.Now() + batch = maindb.NewBatch() + iter = maindb.NewIterator(nil, nil) + ) + // We wrap iter.Release() in an anonymous function so that the [iter] + // value captured is the value of [iter] at the end of the function as opposed + // to incorrectly capturing the first iterator immediately. + defer func() { + iter.Release() + }() + + for iter.Next() { + key := iter.Key() + + // All state entries don't belong to specific state and genesis are deleted here + // - trie node + // - legacy contract code + // - new-scheme contract code + isCode, codeKey := rawdb.IsCodeKey(key) + if len(key) == common.HashLength || isCode { + checkKey := key + if isCode { + checkKey = codeKey + } + if ok, err := stateBloom.Contain(checkKey); err != nil { + return err + } else if ok { + continue + } + count += 1 + size += common.StorageSize(len(key) + len(iter.Value())) + if err := batch.Delete(key); err != nil { + return err + } + + var eta time.Duration // Realistically will never remain uninited + if done := binary.BigEndian.Uint64(key[:8]); done > 0 { + var ( + left = math.MaxUint64 - binary.BigEndian.Uint64(key[:8]) + speed = done/uint64(time.Since(pstart)/time.Millisecond+1) + 1 // +1s to avoid division by zero + ) + eta = time.Duration(left/speed) * time.Millisecond + } + if time.Since(logged) > 8*time.Second { + log.Info("Pruning state data", "nodes", count, "size", size, + "elapsed", common.PrettyDuration(time.Since(pstart)), "eta", common.PrettyDuration(eta)) + logged = time.Now() + } + // Recreate the iterator after every batch commit in order + // to allow the underlying compactor to delete the entries. + if batch.ValueSize() >= ethdb.IdealBatchSize { + if err := batch.Write(); err != nil { + return err + } + batch.Reset() + + iter.Release() + iter = maindb.NewIterator(nil, key) + } + } + } + if err := iter.Error(); err != nil { + return fmt.Errorf("failed to iterate db during pruning: %w", err) + } + if batch.ValueSize() > 0 { + if err := batch.Write(); err != nil { + return err + } + batch.Reset() + } + iter.Release() + log.Info("Pruned state data", "nodes", count, "size", size, "elapsed", common.PrettyDuration(time.Since(pstart))) + + // Write marker to DB to indicate offline pruning finished successfully. We write before calling os.RemoveAll + // to guarantee that if the node dies midway through pruning, then this will run during RecoverPruning. + if err := rawdb.WriteOfflinePruning(maindb); err != nil { + return fmt.Errorf("failed to write offline pruning success marker: %w", err) + } + + // Delete the state bloom, it marks the entire pruning procedure is + // finished. If any crashes or manual exit happens before this, + // `RecoverPruning` will pick it up in the next restarts to redo all + // the things. + if err := os.RemoveAll(bloomPath); err != nil { + return fmt.Errorf("failed to remove bloom filter from disk: %w", err) + } + + // Start compactions, will remove the deleted data from the disk immediately. + // Note for small pruning, the compaction is skipped. + if count >= rangeCompactionThreshold { + cstart := time.Now() + for b := 0x00; b <= 0xf0; b += 0x10 { + var ( + start = []byte{byte(b)} + end = []byte{byte(b + 0x10)} + ) + if b == 0xf0 { + end = nil + } + log.Info("Compacting database", "range", fmt.Sprintf("%#x-%#x", start, end), "elapsed", common.PrettyDuration(time.Since(cstart))) + if err := maindb.Compact(start, end); err != nil { + log.Error("Database compaction failed", "error", err) + return err + } + } + log.Info("Database compaction finished", "elapsed", common.PrettyDuration(time.Since(cstart))) + } + log.Info("State pruning successful", "pruned", size, "elapsed", common.PrettyDuration(time.Since(start))) + return nil +} + +// Prune deletes all historical state nodes except the nodes belong to the +// specified state version. If user doesn't specify the state version, use +// the bottom-most snapshot diff layer as the target. +func (p *Pruner) Prune(root common.Hash) error { + // If the state bloom filter is already committed previously, + // reuse it for pruning instead of generating a new one. It's + // mandatory because a part of state may already be deleted, + // the recovery procedure is necessary. + _, stateBloomRoot, err := findBloomFilter(p.datadir) + if err != nil { + return err + } + if stateBloomRoot != (common.Hash{}) { + return RecoverPruning(p.datadir, p.db) + } + + // If the target state root is not specified, return a fatal error. + if root == (common.Hash{}) { + return fmt.Errorf("cannot prune with an empty root: %s", root) + } + // Ensure the root is really present. The weak assumption + // is the presence of root can indicate the presence of the + // entire trie. + if blob := rawdb.ReadTrieNode(p.db, root); len(blob) == 0 { + return fmt.Errorf("associated state[%x] is not present", root) + } else { + log.Info("Selecting last accepted block root as the pruning target", "root", root) + } + + // Traverse the target state, re-construct the whole state trie and + // commit to the given bloom filter. + start := time.Now() + if err := snapshot.GenerateTrie(p.snaptree, root, p.db, p.stateBloom); err != nil { + return err + } + // Traverse the genesis, put all genesis state entries into the + // bloom filter too. + if err := extractGenesis(p.db, p.stateBloom); err != nil { + return err + } + filterName := bloomFilterName(p.datadir, root) + + log.Info("Writing state bloom to disk", "name", filterName) + if err := p.stateBloom.Commit(filterName, filterName+stateBloomFileTempSuffix); err != nil { + return err + } + log.Info("State bloom filter committed", "name", filterName) + return prune(p.db, p.stateBloom, filterName, start) +} + +// RecoverPruning will resume the pruning procedure during the system restart. +// This function is used in this case: user tries to prune state data, but the +// system was interrupted midway because of crash or manual-kill. In this case +// if the bloom filter for filtering active state is already constructed, the +// pruning can be resumed. What's more if the bloom filter is constructed, the +// pruning **has to be resumed**. Otherwise a lot of dangling nodes may be left +// in the disk. +func RecoverPruning(datadir string, db ethdb.Database) error { + stateBloomPath, stateBloomRoot, err := findBloomFilter(datadir) + if err != nil { + return err + } + if stateBloomPath == "" { + return nil // nothing to recover + } + headBlock := rawdb.ReadHeadBlock(db) + if headBlock == nil { + return errors.New("Failed to load head block") + } + stateBloom, err := NewStateBloomFromDisk(stateBloomPath) + if err != nil { + return err + } + log.Info("Loaded state bloom filter", "path", stateBloomPath) + + // All the state roots of the middle layers should be forcibly pruned, + // otherwise the dangling state will be left. + if stateBloomRoot != headBlock.Root() { + return fmt.Errorf("cannot recover pruning to state bloom root: %s, with head block root: %s", stateBloomRoot, headBlock.Root()) + } + + return prune(db, stateBloom, stateBloomPath, time.Now()) +} + +// extractGenesis loads the genesis state and commits all the state entries +// into the given bloomfilter. +func extractGenesis(db ethdb.Database, stateBloom *stateBloom) error { + genesisHash := rawdb.ReadCanonicalHash(db, 0) + if genesisHash == (common.Hash{}) { + return errors.New("missing genesis hash") + } + genesis := rawdb.ReadBlock(db, genesisHash, 0) + if genesis == nil { + return errors.New("missing genesis block") + } + t, err := trie.NewSecure(genesis.Root(), trie.NewDatabase(db)) + if err != nil { + return err + } + accIter := t.NodeIterator(nil) + for accIter.Next(true) { + hash := accIter.Hash() + + // Embedded nodes don't have hash. + if hash != (common.Hash{}) { + stateBloom.Put(hash.Bytes(), nil) + } + // If it's a leaf node, yes we are touching an account, + // dig into the storage trie further. + if accIter.Leaf() { + var acc types.StateAccount + if err := rlp.DecodeBytes(accIter.LeafBlob(), &acc); err != nil { + return err + } + if acc.Root != emptyRoot { + storageTrie, err := trie.NewSecure(acc.Root, trie.NewDatabase(db)) + if err != nil { + return err + } + storageIter := storageTrie.NodeIterator(nil) + for storageIter.Next(true) { + hash := storageIter.Hash() + if hash != (common.Hash{}) { + stateBloom.Put(hash.Bytes(), nil) + } + } + if storageIter.Error() != nil { + return storageIter.Error() + } + } + if !bytes.Equal(acc.CodeHash, emptyCode) { + stateBloom.Put(acc.CodeHash, nil) + } + } + } + return accIter.Error() +} + +func bloomFilterName(datadir string, hash common.Hash) string { + return filepath.Join(datadir, fmt.Sprintf("%s.%s.%s", stateBloomFilePrefix, hash.Hex(), stateBloomFileSuffix)) +} + +func isBloomFilter(filename string) (bool, common.Hash) { + filename = filepath.Base(filename) + if strings.HasPrefix(filename, stateBloomFilePrefix) && strings.HasSuffix(filename, stateBloomFileSuffix) { + return true, common.HexToHash(filename[len(stateBloomFilePrefix)+1 : len(filename)-len(stateBloomFileSuffix)-1]) + } + return false, common.Hash{} +} + +func findBloomFilter(datadir string) (string, common.Hash, error) { + var ( + stateBloomPath string + stateBloomRoot common.Hash + ) + if err := filepath.Walk(datadir, func(path string, info os.FileInfo, err error) error { + if info != nil && !info.IsDir() { + ok, root := isBloomFilter(path) + if ok { + stateBloomPath = path + stateBloomRoot = root + } + } + return nil + }); err != nil { + return "", common.Hash{}, err + } + return stateBloomPath, stateBloomRoot, nil +} diff --git a/core/test_blockchain.go b/core/test_blockchain.go index 9c9156703a..aae6fa322c 100644 --- a/core/test_blockchain.go +++ b/core/test_blockchain.go @@ -28,50 +28,50 @@ type ChainTest struct { } var tests = []ChainTest{ - { - "InsertChainAcceptSingleBlock", - TestInsertChainAcceptSingleBlock, - }, - { - "InsertForkedChain", - TestInsertLongForkedChain, - }, - { - "AcceptNonCanonicalBlock", - TestAcceptNonCanonicalBlock, - }, - { - "SetPreferenceRewind", - TestSetPreferenceRewind, - }, - { - "BuildOnVariousStages", - TestBuildOnVariousStages, - }, + // { + // "InsertChainAcceptSingleBlock", + // TestInsertChainAcceptSingleBlock, + // }, + // { + // "InsertForkedChain", + // TestInsertLongForkedChain, + // }, + // { + // "AcceptNonCanonicalBlock", + // TestAcceptNonCanonicalBlock, + // }, + // { + // "SetPreferenceRewind", + // TestSetPreferenceRewind, + // }, + // { + // "BuildOnVariousStages", + // TestBuildOnVariousStages, + // }, { "EmptyBlocks", TestEmptyBlocks, }, - { - "AcceptBlockIdenticalStateRoot", - TestAcceptBlockIdenticalStateRoot, - }, - { - "ReprocessAcceptBlockIdenticalStateRoot", - TestReprocessAcceptBlockIdenticalStateRoot, - }, - { - "GenerateChainInvalidBlockFee", - TestGenerateChainInvalidBlockFee, - }, - { - "InsertChainInvalidBlockFee", - TestInsertChainInvalidBlockFee, - }, - { - "InsertChainValidBlockFee", - TestInsertChainValidBlockFee, - }, + // { + // "AcceptBlockIdenticalStateRoot", + // TestAcceptBlockIdenticalStateRoot, + // }, + // { + // "ReprocessAcceptBlockIdenticalStateRoot", + // TestReprocessAcceptBlockIdenticalStateRoot, + // }, + // { + // "GenerateChainInvalidBlockFee", + // TestGenerateChainInvalidBlockFee, + // }, + // { + // "InsertChainInvalidBlockFee", + // TestInsertChainInvalidBlockFee, + // }, + // { + // "InsertChainValidBlockFee", + // TestInsertChainValidBlockFee, + // }, } func copyMemDB(db ethdb.Database) (ethdb.Database, error) { diff --git a/core/tx_pool.go b/core/tx_pool.go index 5fef2d71a8..5de3eb4617 100644 --- a/core/tx_pool.go +++ b/core/tx_pool.go @@ -1253,7 +1253,7 @@ func (pool *TxPool) runReorg(done chan struct{}, reset *txpoolResetRequest, dirt if reset != nil { pool.demoteUnexecutables() if reset.newHead != nil && pool.chainconfig.IsSubnetEVM(new(big.Int).SetUint64(reset.newHead.Time)) { - _, baseFeeEstimate, err := dummy.CalcBaseFee(pool.chainconfig, reset.newHead, uint64(time.Now().Unix())) + _, baseFeeEstimate, err := dummy.EstimateNextBaseFee(pool.chainconfig, reset.newHead, uint64(time.Now().Unix())) if err == nil { pool.priced.SetBaseFee(baseFeeEstimate) } @@ -1696,7 +1696,7 @@ func (pool *TxPool) updateBaseFee() { pool.mu.Lock() defer pool.mu.Unlock() - _, baseFeeEstimate, err := dummy.CalcBaseFee(pool.chainconfig, pool.currentHead, uint64(time.Now().Unix())) + _, baseFeeEstimate, err := dummy.EstimateNextBaseFee(pool.chainconfig, pool.currentHead, uint64(time.Now().Unix())) if err == nil { pool.priced.SetBaseFee(baseFeeEstimate) } else { diff --git a/core/vm/evm.go b/core/vm/evm.go index f264487718..4fa3a6cc3c 100644 --- a/core/vm/evm.go +++ b/core/vm/evm.go @@ -180,6 +180,9 @@ func (evm *EVM) Call(caller ContractRef, addr common.Address, input []byte, gas return nil, gas, ErrDepth } // Fail if we're trying to transfer more than the available balance + // Note: it is not possible for a negative value to be passed in here due to the fact + // that [value] will be popped from the stack and decoded to a *big.Int, which will + // always yield a positive result. if value.Sign() != 0 && !evm.Context.CanTransfer(evm.StateDB, caller.Address(), value) { return nil, gas, ErrInsufficientBalance } @@ -269,6 +272,9 @@ func (evm *EVM) CallCode(caller ContractRef, addr common.Address, input []byte, // Note although it's noop to transfer X ether to caller itself. But // if caller doesn't have enough balance, it would be an error to allow // over-charging itself. So the check here is necessary. + // Note: it is not possible for a negative value to be passed in here due to the fact + // that [value] will be popped from the stack and decoded to a *big.Int, which will + // always yield a positive result. if !evm.Context.CanTransfer(evm.StateDB, caller.Address(), value) { return nil, gas, ErrInsufficientBalance } @@ -418,6 +424,9 @@ func (evm *EVM) create(caller ContractRef, codeAndHash *codeAndHash, gas uint64, if evm.depth > int(params.CallCreateDepth) { return nil, common.Address{}, gas, ErrDepth } + // Note: it is not possible for a negative value to be passed in here due to the fact + // that [value] will be popped from the stack and decoded to a *big.Int, which will + // always yield a positive result. if !evm.Context.CanTransfer(evm.StateDB, caller.Address(), value) { return nil, common.Address{}, gas, ErrInsufficientBalance } diff --git a/eth/backend.go b/eth/backend.go index ec7d734400..4734bf219f 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -40,6 +40,7 @@ import ( "github.com/ava-labs/subnet-evm/core" "github.com/ava-labs/subnet-evm/core/bloombits" "github.com/ava-labs/subnet-evm/core/rawdb" + "github.com/ava-labs/subnet-evm/core/state/pruner" "github.com/ava-labs/subnet-evm/core/types" "github.com/ava-labs/subnet-evm/core/vm" "github.com/ava-labs/subnet-evm/eth/ethconfig" @@ -140,6 +141,15 @@ func New( // Free airdrop data to save memory usage core.AirdropData = nil + // Note: RecoverPruning must be called to handle the case that we are midway through offline pruning. + // If the data directory is changed in between runs preventing RecoverPruning from performing its job correctly, + // it may cause DB corruption. + // Since RecoverPruning will only continue a pruning run that already began, we do not need to ensure that + // reprocessState has already been called and completed successfully. To ensure this, we must maintain + // that Prune is only run after reprocessState has finished successfully. + if err := pruner.RecoverPruning(config.OfflinePruningDataDirectory, chainDb); err != nil { + log.Error("Failed to recover state", "error", err) + } eth := &Ethereum{ config: config, chainDb: chainDb, @@ -190,12 +200,13 @@ func New( if err != nil { return nil, err } + + if err := eth.handleOfflinePruning(cacheConfig, chainConfig, vmConfig, lastAcceptedHash); err != nil { + return nil, err + } + eth.bloomIndexer.Start(eth.blockchain) - // Original code (requires disk): - // if config.TxPool.Journal != "" { - // config.TxPool.Journal = stack.ResolvePath(config.TxPool.Journal) - // } config.TxPool.Journal = "" eth.txPool = core.NewTxPool(config.TxPool, chainConfig, eth.blockchain) @@ -354,3 +365,46 @@ func (s *Ethereum) Stop() error { func (s *Ethereum) LastAcceptedBlock() *types.Block { return s.blockchain.LastAcceptedBlock() } + +func (s *Ethereum) handleOfflinePruning(cacheConfig *core.CacheConfig, chainConfig *params.ChainConfig, vmConfig vm.Config, lastAcceptedHash common.Hash) error { + if !s.config.OfflinePruning { + // Delete the offline pruning marker to indicate that the node started with offline pruning disabled. + if err := rawdb.DeleteOfflinePruning(s.chainDb); err != nil { + return fmt.Errorf("failed to write offline pruning disabled marker: %w", err) + } + return nil + } + + // Perform offline pruning after NewBlockChain has been called to ensure that we have rolled back the chain + // to the last accepted block before pruning begins. + // If offline pruning marker is on disk, then we force the node to be started with offline pruning disabled + // before allowing another run of offline pruning. + if _, err := rawdb.ReadOfflinePruning(s.chainDb); err == nil { + log.Error("Offline pruning is not meant to be left enabled permanently. Please disable offline pruning and allow your node to start successfully before running offline pruning again.") + return errors.New("cannot start chain with offline pruning enabled on consecutive starts") + } + + // Clean up middle roots + if err := s.blockchain.CleanBlockRootsAboveLastAccepted(); err != nil { + return err + } + targetRoot := s.blockchain.LastAcceptedBlock().Root() + + // Allow the blockchain to be garbage collected immediately, since we will shut down the chain after offline pruning completes. + s.blockchain.Stop() + s.blockchain = nil + log.Info("Starting offline pruning", "dataDir", s.config.OfflinePruningDataDirectory, "bloomFilterSize", s.config.OfflinePruningBloomFilterSize) + pruner, err := pruner.NewPruner(s.chainDb, s.config.OfflinePruningDataDirectory, s.config.OfflinePruningBloomFilterSize) + if err != nil { + return fmt.Errorf("failed to create new pruner with data directory: %s, size: %d, due to: %w", s.config.OfflinePruningDataDirectory, s.config.OfflinePruningBloomFilterSize, err) + } + if err := pruner.Prune(targetRoot); err != nil { + return fmt.Errorf("failed to prune blockchain with target root: %s due to: %w", targetRoot, err) + } + s.blockchain, err = core.NewBlockChain(s.chainDb, cacheConfig, chainConfig, s.engine, vmConfig, lastAcceptedHash) + if err != nil { + return fmt.Errorf("failed to re-initialize blockchain after offline pruning: %w", err) + } + + return nil +} diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 0ffb727f51..aeea88f634 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -51,22 +51,19 @@ var DefaultConfig = NewDefaultConfig() func NewDefaultConfig() Config { return Config{ - NetworkId: 1, - LightPeers: 100, - UltraLightFraction: 75, - DatabaseCache: 512, - TrieCleanCache: 75, - TrieCleanCacheJournal: "triecache", - TrieCleanCacheRejournal: 60 * time.Minute, - TrieDirtyCache: 256, - TrieTimeout: 60 * time.Minute, - SnapshotCache: 128, - Miner: miner.Config{}, - TxPool: core.DefaultTxPoolConfig, - RPCGasCap: 25000000, - RPCEVMTimeout: 5 * time.Second, - GPO: DefaultFullGPOConfig, - RPCTxFeeCap: 1, // 1 AVAX + NetworkId: 1, + LightPeers: 100, + UltraLightFraction: 75, + DatabaseCache: 512, + TrieCleanCache: 75, + TrieDirtyCache: 256, + SnapshotCache: 128, + Miner: miner.Config{}, + TxPool: core.DefaultTxPoolConfig, + RPCGasCap: 25000000, + RPCEVMTimeout: 5 * time.Second, + GPO: DefaultFullGPOConfig, + RPCTxFeeCap: 1, } } @@ -110,13 +107,10 @@ type Config struct { DatabaseCache int // DatabaseFreezer string - TrieCleanCache int - TrieCleanCacheJournal string `toml:",omitempty"` // Disk journal directory for trie cache to survive node restarts - TrieCleanCacheRejournal time.Duration `toml:",omitempty"` // Time interval to regenerate the journal for clean cache - TrieDirtyCache int - TrieTimeout time.Duration - SnapshotCache int - Preimages bool + TrieCleanCache int + TrieDirtyCache int + SnapshotCache int + Preimages bool // Mining options Miner miner.Config @@ -150,4 +144,10 @@ type Config struct { // Unprotected transactions are transactions that are signed without EIP-155 // replay protection. AllowUnprotectedTxs bool + + // OfflinePruning enables offline pruning on startup of the node. If a node is started + // with this configuration option, it must finish pruning before resuming normal operation. + OfflinePruning bool + OfflinePruningBloomFilterSize uint64 + OfflinePruningDataDirectory string } diff --git a/eth/gasprice/gasprice.go b/eth/gasprice/gasprice.go index 722f1857ac..a92ba8f625 100644 --- a/eth/gasprice/gasprice.go +++ b/eth/gasprice/gasprice.go @@ -211,17 +211,10 @@ func (oracle *Oracle) estimateNextBaseFee(ctx context.Context) (*big.Int, error) return nil, nil } - // If the current time is prior to the parent timestamp, then we use the parent - // timestamp instead. - header := block.Header() - timestamp := oracle.clock.Unix() - if timestamp < header.Time { - timestamp = header.Time - } // If the block does have a baseFee, calculate the next base fee // based on the current time and add it to the tip to estimate the // total gas price estimate. - _, nextBaseFee, err := dummy.CalcBaseFee(oracle.backend.ChainConfig(), header, timestamp) + _, nextBaseFee, err := dummy.EstimateNextBaseFee(oracle.backend.ChainConfig(), block.Header(), oracle.clock.Unix()) return nextBaseFee, err } diff --git a/go.mod b/go.mod index affdf4257c..c52d212c64 100644 --- a/go.mod +++ b/go.mod @@ -4,7 +4,8 @@ go 1.16 require ( github.com/VictoriaMetrics/fastcache v1.6.0 - github.com/ava-labs/avalanchego v1.7.4 + github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377 + github.com/btcsuite/btcd v0.21.0-beta // indirect github.com/cespare/cp v0.1.0 github.com/davecgh/go-spew v1.1.1 github.com/deckarep/golang-set v1.7.1 @@ -24,6 +25,7 @@ require ( github.com/olekukonko/tablewriter v0.0.5 github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 + github.com/prometheus/tsdb v0.10.0 // indirect github.com/rjeczalik/notify v0.9.2 github.com/spf13/cast v1.3.1 github.com/spf13/pflag v1.0.5 @@ -33,6 +35,7 @@ require ( github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 github.com/tyler-smith/go-bip39 v1.0.2 golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2 + golang.org/x/sync v0.0.0-20210220032951-036812b2e83c golang.org/x/text v0.3.6 golang.org/x/time v0.0.0-20210723032227-1f47c861a9ac gopkg.in/olebedev/go-duktape.v3 v3.0.0-20200619000410-60c24ae608a6 diff --git a/go.sum b/go.sum index 9ee5543e17..771133b823 100644 --- a/go.sum +++ b/go.sum @@ -20,7 +20,6 @@ cloud.google.com/go/storage v1.5.0/go.mod h1:tpKbwo567HUNpVclU5sGELwQWBDZ8gh0Zeo collectd.org v0.3.0/go.mod h1:A/8DzQBkF6abtvrT2j/AU/4tiBgJWYyh0y/oB/4MlWE= dmitri.shuralyov.com/gpu/mtl v0.0.0-20190408044501-666a987793e9/go.mod h1:H6x//7gZCb22OMCxBHrMx7a5I7Hp++hsVxbQ4BYO7hU= gioui.org v0.0.0-20210308172011-57750fc8a0a6/go.mod h1:RSH6KIUZ0p2xy5zHDxgAM4zumjgTw83q2ge/PI+yyw8= -github.com/AppsFlyer/go-sundheit v0.2.0/go.mod h1:rCRkVTMQo7/krF7xQ9X0XEF1an68viFR6/Gy02q+4ds= github.com/Azure/azure-pipeline-go v0.2.1/go.mod h1:UGSo8XybXnIGZ3epmeBw7Jdz+HiUVpqIlpz/HKHylF4= github.com/Azure/azure-pipeline-go v0.2.2/go.mod h1:4rQ/NZncSvGqNkkOsNpOU1tgoNuIlp9AfUH5G1tvCHc= github.com/Azure/azure-storage-blob-go v0.7.0/go.mod h1:f9YQKtsG1nMisotuTPpO0tjNuEjKRYAcJU8/ydDI++4= @@ -58,12 +57,8 @@ github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VT github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/ava-labs/avalanchego v1.7.4-0.20211222191707-3163be793cc9/go.mod h1:YPIP9GyzXKjscsWlvbPWvV0TQcgV+0t8rMe8sLvc0xQ= -github.com/ava-labs/avalanchego v1.7.4-0.20220113231534-ccb8039d74b9/go.mod h1:hm1T3Nui/qRALnNngXmC+lWeTEPtYyPBKxaWMWuRBrs= -github.com/ava-labs/avalanchego v1.7.4 h1:NTLDw9oh+Y+acH2xTsFtxMxPMDTISkpJReL+lhABYfo= -github.com/ava-labs/avalanchego v1.7.4/go.mod h1:+xnmwjlkeNHhs0v5p2yvdv7cw0jgyq4SyU9O+S4EUPs= -github.com/ava-labs/coreth v0.8.4-rc.1/go.mod h1:bP2Atm7pCJdx8fwzsPT3xU/kWOdHFklpja7aRNT++Qo= -github.com/ava-labs/coreth v0.8.4-rc.3/go.mod h1:9TgpLJVY9ot6RV8Lh66F356S4MfalvaL3sAqw4+miTU= +github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377 h1:suLHw7lxX8m8UfmZxatQQYEgkHUVWrXjN96GLdpATFg= +github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377/go.mod h1:sUhn77bV5EBGXwqcFz1FzFs94MR6SRgrU+Ob7lb+pC4= github.com/aws/aws-sdk-go-v2 v1.2.0/go.mod h1:zEQs02YRBw1DjK0PoJv3ygDYOFTre1ejlJWl8FwAuQo= github.com/aws/aws-sdk-go-v2/config v1.1.1/go.mod h1:0XsVy9lBI/BCXm+2Tuvt39YmdHwS5unDQmxZOYe8F5Y= github.com/aws/aws-sdk-go-v2/credentials v1.1.1/go.mod h1:mM2iIjwl7LULWtS6JCACyInboHirisUUdkBPoTHMOUo= @@ -158,7 +153,6 @@ github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5 h1:FtmdgXiUlNeRsoNMFlK github.com/fjl/memsize v0.0.0-20190710130421-bcb5799ab5e5/go.mod h1:VvhXpOYNQvB+uIk2RvXzuaQtkQJzzIx6lSBe1xv7hi0= github.com/fogleman/gg v1.2.1-0.20190220221249-0403632d5b90/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= github.com/fogleman/gg v1.3.0/go.mod h1:R/bRT+9gY/C5z7JzPU0zXsXHKM4/ayA+zqcVNZzPa1k= -github.com/fortytw2/leaktest v1.3.0/go.mod h1:jDsjWgpAGjm2CA7WthBh/CdZYEPF31XHquHwclZch5g= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/fsnotify/fsnotify v1.4.9 h1:hsms1Qyu0jgnwNXIxa+/V/PDsU6CfLf6CNO8H7IWoS4= github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= @@ -205,6 +199,7 @@ github.com/golang/groupcache v0.0.0-20190702054246-869f871628b6/go.mod h1:cIg4er github.com/golang/groupcache v0.0.0-20191227052852-215e87163ea7/go.mod h1:cIg4eruTrX1D+g88fzRXU5OdNfaM+9IcxsU14FzY7Hc= github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +github.com/golang/mock v1.3.1 h1:qGJ6qTW+x6xX/my+8YUVl4WNpX9B7+/l2tRsHGZ7f2s= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= @@ -229,6 +224,7 @@ github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEW github.com/golangci/lint-1 v0.0.0-20181222135242-d2cdd8c08219/go.mod h1:/X8TswGSh1pIozq4ZwCfxS0WA5JGXguxk94ar/4c87Y= github.com/google/btree v0.0.0-20180813153112-4030bb1f1f0c/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= +github.com/google/btree v1.0.1/go.mod h1:xXMiIv4Fb/0kKde4SpL7qlzvu5cMJDRkFDxJfI9uaxA= github.com/google/flatbuffers v1.11.0/go.mod h1:1AeVuKshWv4vARoZatz6mlQ0JxURH0Kv5+zNeJKJCa8= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.3.0/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= @@ -568,7 +564,6 @@ github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9dec go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.opencensus.io v0.21.0/go.mod h1:mSImk1erAIZhrmZN+AvHh14ztQfjbGwt4TtuofqLduU= go.opencensus.io v0.22.0/go.mod h1:+kGneAE2xo2IficOXnaByMWTGM9T73dGwxeWcUqIpI8= -go.opencensus.io v0.22.1/go.mod h1:Ap50jQcDJrx6rB6VgeeFPtuPIf3wMRvRfrfYDO6+BmA= go.opencensus.io v0.22.2/go.mod h1:yxeiOL68Rb0Xd1ddK5vPZ/oVn4vY4Ynel7k9FzqtOIw= go.uber.org/atomic v1.3.2/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= diff --git a/metrics/prometheus/prometheus.go b/metrics/prometheus/prometheus.go index 576a3786c6..0dbb87065f 100644 --- a/metrics/prometheus/prometheus.go +++ b/metrics/prometheus/prometheus.go @@ -7,7 +7,6 @@ import ( "sort" "strings" - "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" "github.com/prometheus/client_golang/prometheus" @@ -178,7 +177,7 @@ func (g gatherer) Gather() ([]*dto.MetricFamily, error) { }) } } - log.Error("called", "names", names, "mfs", mfs) + return mfs, nil } diff --git a/params/config.go b/params/config.go index 898634e92d..0a350e6274 100644 --- a/params/config.go +++ b/params/config.go @@ -213,13 +213,14 @@ func (c *ChainConfig) GetFeeConfig() *FeeConfig { // CheckCompatible checks whether scheduled fork transitions have been imported // with a mismatching chain configuration. -func (c *ChainConfig) CheckCompatible(newcfg *ChainConfig, height uint64) *ConfigCompatError { +func (c *ChainConfig) CheckCompatible(newcfg *ChainConfig, height uint64, timestamp uint64) *ConfigCompatError { bhead := new(big.Int).SetUint64(height) + bheadTimestamp := new(big.Int).SetUint64(timestamp) // Iterate checkCompatible to find the lowest conflict. var lasterr *ConfigCompatError for { - err := c.checkCompatible(newcfg, bhead) + err := c.checkCompatible(newcfg, bhead, bheadTimestamp) if err == nil || (lasterr != nil && err.RewindTo == lasterr.RewindTo) { break } @@ -301,44 +302,45 @@ func (c *ChainConfig) CheckConfigForkOrder() error { return nil } -func (c *ChainConfig) checkCompatible(newcfg *ChainConfig, head *big.Int) *ConfigCompatError { - if isForkIncompatible(c.HomesteadBlock, newcfg.HomesteadBlock, head) { +func (c *ChainConfig) checkCompatible(newcfg *ChainConfig, headHeight *big.Int, headTimestamp *big.Int) *ConfigCompatError { + if isForkIncompatible(c.HomesteadBlock, newcfg.HomesteadBlock, headHeight) { return newCompatError("Homestead fork block", c.HomesteadBlock, newcfg.HomesteadBlock) } - if isForkIncompatible(c.EIP150Block, newcfg.EIP150Block, head) { + if isForkIncompatible(c.EIP150Block, newcfg.EIP150Block, headHeight) { return newCompatError("EIP150 fork block", c.EIP150Block, newcfg.EIP150Block) } - if isForkIncompatible(c.EIP155Block, newcfg.EIP155Block, head) { + if isForkIncompatible(c.EIP155Block, newcfg.EIP155Block, headHeight) { return newCompatError("EIP155 fork block", c.EIP155Block, newcfg.EIP155Block) } - if isForkIncompatible(c.EIP158Block, newcfg.EIP158Block, head) { + if isForkIncompatible(c.EIP158Block, newcfg.EIP158Block, headHeight) { return newCompatError("EIP158 fork block", c.EIP158Block, newcfg.EIP158Block) } - if c.IsEIP158(head) && !configNumEqual(c.ChainID, newcfg.ChainID) { + if c.IsEIP158(headHeight) && !configNumEqual(c.ChainID, newcfg.ChainID) { return newCompatError("EIP158 chain ID", c.EIP158Block, newcfg.EIP158Block) } - if isForkIncompatible(c.ByzantiumBlock, newcfg.ByzantiumBlock, head) { + if isForkIncompatible(c.ByzantiumBlock, newcfg.ByzantiumBlock, headHeight) { return newCompatError("Byzantium fork block", c.ByzantiumBlock, newcfg.ByzantiumBlock) } - if isForkIncompatible(c.ConstantinopleBlock, newcfg.ConstantinopleBlock, head) { + if isForkIncompatible(c.ConstantinopleBlock, newcfg.ConstantinopleBlock, headHeight) { return newCompatError("Constantinople fork block", c.ConstantinopleBlock, newcfg.ConstantinopleBlock) } - if isForkIncompatible(c.PetersburgBlock, newcfg.PetersburgBlock, head) { + if isForkIncompatible(c.PetersburgBlock, newcfg.PetersburgBlock, headHeight) { // the only case where we allow Petersburg to be set in the past is if it is equal to Constantinople // mainly to satisfy fork ordering requirements which state that Petersburg fork be set if Constantinople fork is set - if isForkIncompatible(c.ConstantinopleBlock, newcfg.PetersburgBlock, head) { + if isForkIncompatible(c.ConstantinopleBlock, newcfg.PetersburgBlock, headHeight) { return newCompatError("Petersburg fork block", c.PetersburgBlock, newcfg.PetersburgBlock) } } - if isForkIncompatible(c.IstanbulBlock, newcfg.IstanbulBlock, head) { + if isForkIncompatible(c.IstanbulBlock, newcfg.IstanbulBlock, headHeight) { return newCompatError("Istanbul fork block", c.IstanbulBlock, newcfg.IstanbulBlock) } - if isForkIncompatible(c.MuirGlacierBlock, newcfg.MuirGlacierBlock, head) { + if isForkIncompatible(c.MuirGlacierBlock, newcfg.MuirGlacierBlock, headHeight) { return newCompatError("Muir Glacier fork block", c.MuirGlacierBlock, newcfg.MuirGlacierBlock) } - if !configNumEqual(c.SubnetEVMTimestamp, newcfg.SubnetEVMTimestamp) { - return newCompatError("SubnetEVM fork block", c.SubnetEVMTimestamp, newcfg.SubnetEVMTimestamp) + if isForkIncompatible(c.SubnetEVMTimestamp, newcfg.SubnetEVMTimestamp, headTimestamp) { + return newCompatError("SubnetEVM fork block timestamp", c.SubnetEVMTimestamp, newcfg.SubnetEVMTimestamp) } + return nil } diff --git a/params/config_test.go b/params/config_test.go new file mode 100644 index 0000000000..8778ed6cf3 --- /dev/null +++ b/params/config_test.go @@ -0,0 +1,138 @@ +// (c) 2019-2020, Ava Labs, Inc. +// +// This file is a derived work, based on the go-ethereum library whose original +// notices appear below. +// +// It is distributed under a license compatible with the licensing terms of the +// original code from which it is derived. +// +// Much love to the original authors for their work. +// ********** +// Copyright 2017 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package params + +import ( + "math/big" + "reflect" + "testing" +) + +func TestCheckCompatible(t *testing.T) { + type test struct { + stored, new *ChainConfig + headHeight, headTimestamp uint64 + wantErr *ConfigCompatError + } + tests := []test{ + {stored: TestChainConfig, new: TestChainConfig, headHeight: 0, headTimestamp: 0, wantErr: nil}, + {stored: TestChainConfig, new: TestChainConfig, headHeight: 100, headTimestamp: 1000, wantErr: nil}, + { + stored: &ChainConfig{EIP150Block: big.NewInt(10)}, + new: &ChainConfig{EIP150Block: big.NewInt(20)}, + headHeight: 9, + headTimestamp: 90, + wantErr: nil, + }, + { + stored: TestChainConfig, + new: &ChainConfig{HomesteadBlock: nil}, + headHeight: 3, + headTimestamp: 30, + wantErr: &ConfigCompatError{ + What: "Homestead fork block", + StoredConfig: big.NewInt(0), + NewConfig: nil, + RewindTo: 0, + }, + }, + { + stored: TestChainConfig, + new: &ChainConfig{HomesteadBlock: big.NewInt(1)}, + headHeight: 3, + headTimestamp: 30, + wantErr: &ConfigCompatError{ + What: "Homestead fork block", + StoredConfig: big.NewInt(0), + NewConfig: big.NewInt(1), + RewindTo: 0, + }, + }, + { + stored: &ChainConfig{HomesteadBlock: big.NewInt(30), EIP150Block: big.NewInt(10)}, + new: &ChainConfig{HomesteadBlock: big.NewInt(25), EIP150Block: big.NewInt(20)}, + headHeight: 25, + headTimestamp: 250, + wantErr: &ConfigCompatError{ + What: "EIP150 fork block", + StoredConfig: big.NewInt(10), + NewConfig: big.NewInt(20), + RewindTo: 9, + }, + }, + { + stored: &ChainConfig{ConstantinopleBlock: big.NewInt(30)}, + new: &ChainConfig{ConstantinopleBlock: big.NewInt(30), PetersburgBlock: big.NewInt(30)}, + headHeight: 40, + headTimestamp: 400, + wantErr: nil, + }, + { + stored: &ChainConfig{ConstantinopleBlock: big.NewInt(30)}, + new: &ChainConfig{ConstantinopleBlock: big.NewInt(30), PetersburgBlock: big.NewInt(31)}, + headHeight: 40, + headTimestamp: 400, + wantErr: &ConfigCompatError{ + What: "Petersburg fork block", + StoredConfig: nil, + NewConfig: big.NewInt(31), + RewindTo: 30, + }, + }, + { + stored: TestChainConfig, + new: TestPreSubnetEVMConfig, + headHeight: 0, + headTimestamp: 0, + wantErr: &ConfigCompatError{ + What: "SubnetEVM fork block timestamp", + StoredConfig: big.NewInt(0), + NewConfig: nil, + RewindTo: 0, + }, + }, + { + stored: TestChainConfig, + new: TestPreSubnetEVMConfig, + headHeight: 10, + headTimestamp: 100, + wantErr: &ConfigCompatError{ + What: "SubnetEVM fork block timestamp", + StoredConfig: big.NewInt(0), + NewConfig: nil, + RewindTo: 0, + }, + }, + } + + for _, test := range tests { + err := test.stored.CheckCompatible(test.new, test.headHeight, test.headTimestamp) + if !reflect.DeepEqual(err, test.wantErr) { + t.Errorf("error mismatch:\nstored: %v\nnew: %v\nheadHeight: %v\nerr: %v\nwant: %v", test.stored, test.new, test.headHeight, err, test.wantErr) + } + } +} diff --git a/peer/client.go b/peer/client.go new file mode 100644 index 0000000000..2dcfac8972 --- /dev/null +++ b/peer/client.go @@ -0,0 +1,52 @@ +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package peer + +import ( + "github.com/ava-labs/avalanchego/version" +) + +var _ Client = &client{} + +// Client defines ability to send request / response through the Network +type Client interface { + // RequestAny synchronously sends request to the first connected peer that matches the specified minVersion in + // random order. + // A peer is considered a match if its version is greater than or equal to the specified minVersion + // Returns errNoPeersMatchingVersion if no peer could be found matching specified version + RequestAny(minVersion version.Application, request []byte) ([]byte, bool, error) + + // Gossip sends given gossip message to peers + Gossip(gossip []byte) error +} + +// client implements Client interface +// provides ability to send request / responses through the Network +type client struct { + network Network +} + +// RequestAny synchronously sends request to the first connected peer that matches the specified minVersion in +// random order. +// Returns response bytes, whether the request failed and optional error +// Returns errNoPeersMatchingVersion if no peer could be found matching specified version +// This function is blocks until a response is received from the peer +func (c *client) RequestAny(minVersion version.Application, request []byte) ([]byte, bool, error) { + waitingHandler := newWaitingResponseHandler() + if err := c.network.RequestAny(minVersion, request, waitingHandler); err != nil { + return nil, true, err + } + return <-waitingHandler.responseChan, waitingHandler.failed, nil +} + +func (c *client) Gossip(gossip []byte) error { + return c.network.Gossip(gossip) +} + +// NewClient returns Client for a given network +func NewClient(network Network) Client { + return &client{ + network: network, + } +} diff --git a/peer/network.go b/peer/network.go new file mode 100644 index 0000000000..9a6b3edf4c --- /dev/null +++ b/peer/network.go @@ -0,0 +1,343 @@ +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package peer + +import ( + "context" + "errors" + "fmt" + "sync" + "time" + + "github.com/ava-labs/subnet-evm/plugin/evm/message" + + "github.com/ava-labs/avalanchego/snow/validators" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/snow/engine/common" + "github.com/ava-labs/avalanchego/version" + "github.com/ethereum/go-ethereum/log" + "golang.org/x/sync/semaphore" +) + +// Minimum amount of time to handle a request +const minRequestHandlingDuration = 100 * time.Millisecond + +var ( + errAcquiringSemaphore = errors.New("error acquiring semaphore") + _ Network = &network{} + _ validators.Connector = &network{} + _ common.AppHandler = &network{} +) + +type Network interface { + validators.Connector + common.AppHandler + + // RequestAny synchronously sends request to the first connected peer that matches the specified minVersion in + // random order. + // A peer is considered a match if its version is greater than or equal to the specified minVersion + // Returns errNoPeersMatchingVersion if no peer could be found matching specified version + RequestAny(minVersion version.Application, message []byte, handler message.ResponseHandler) error + + // Gossip sends given gossip message to peers + Gossip(gossip []byte) error + + // Shutdown stops all peer channel listeners and marks the node to have stopped + // n.Start() can be called again but the peers will have to be reconnected + // by calling OnPeerConnected for each peer + Shutdown() + + // SetGossipHandler sets the provided gossip handler as the gossip handler + SetGossipHandler(handler message.GossipHandler) + + // SetRequestHandler sets the provided request handler as the request handler + SetRequestHandler(handler message.RequestHandler) + + // Size returns the size of the network in number of connected peers + Size() uint32 +} + +// network is an implementation of Network that processes message requests for +// each peer in linear fashion +type network struct { + lock sync.RWMutex // lock for mutating state of this Network struct + self ids.ShortID // NodeID of this node + requestIDGen uint32 // requestID counter used to track outbound requests + outstandingResponseHandlerMap map[uint32]message.ResponseHandler // maps avalanchego requestID => response handler + activeRequests *semaphore.Weighted // controls maximum number of active outbound requests + appSender common.AppSender // avalanchego AppSender for sending messages + codec codec.Manager // Codec used for parsing messages + requestHandler message.RequestHandler // maps request type => handler + gossipHandler message.GossipHandler // maps gossip type => handler + peers map[ids.ShortID]version.Application // maps nodeID => version.Version +} + +func NewNetwork(appSender common.AppSender, codec codec.Manager, self ids.ShortID, maxActiveRequests int64) Network { + return &network{ + appSender: appSender, + codec: codec, + self: self, + outstandingResponseHandlerMap: make(map[uint32]message.ResponseHandler), + peers: make(map[ids.ShortID]version.Application), + activeRequests: semaphore.NewWeighted(maxActiveRequests), + } +} + +// RequestAny sends given request to the first connected peer that matches the specified minVersion +// A peer is considered a match if its version is greater than or equal to the specified minVersion +// If minVersion is nil, then the request will be sent to any peer regardless of their version +// Returns a non-nil error if we were not able to send a request to a peer with >= [minVersion] +// or we fail to send a request to the selected peer. +func (n *network) RequestAny(minVersion version.Application, request []byte, handler message.ResponseHandler) error { + // Take a slot from total [activeRequests] and block until a slot becomes available. + if err := n.activeRequests.Acquire(context.Background(), 1); err != nil { + return errAcquiringSemaphore + } + + n.lock.Lock() + defer n.lock.Unlock() + + for nodeID, nodeVersion := range n.peers { + // map iteration is sufficiently random to avoid hitting same peer so here + // we get a random peerID key that we compare minimum version that + // we have + if minVersion == nil || nodeVersion.Compare(minVersion) >= 0 { + return n.request(nodeID, request, handler) + } + } + + n.activeRequests.Release(1) + return fmt.Errorf("no peers found matching version %s out of %d peers", minVersion, len(n.peers)) +} + +// Request sends request message bytes to specified nodeID and adds [responseHandler] to [outstandingResponseHandlerMap] +// so that it can be invoked when the network receives either a response or failure message. +// Assumes [nodeID] is never [self] since we guarantee [self] will not be added to the [peers] map. +// Returns an error if [appSender] is unable to make the request. +// Assumes write lock is held +func (n *network) request(nodeID ids.ShortID, request []byte, responseHandler message.ResponseHandler) error { + log.Debug("sending request to peer", "nodeID", nodeID, "requestLen", len(request)) + + // generate requestID + requestID := n.requestIDGen + n.requestIDGen++ + + n.outstandingResponseHandlerMap[requestID] = responseHandler + + nodeIDs := ids.NewShortSet(1) + nodeIDs.Add(nodeID) + + // send app request to the peer + // on failure: release the activeRequests slot, mark message as processed and return fatal error + // Send app request to [nodeID]. + // On failure, release the slot from active requests and [outstandingResponseHandlerMap]. + if err := n.appSender.SendAppRequest(nodeIDs, requestID, request); err != nil { + n.activeRequests.Release(1) + delete(n.outstandingResponseHandlerMap, requestID) + log.Error("could not send app message", "err", err, "nodeID", nodeID, "requestID", requestID) + return err + } + + log.Debug("sent request message to peer", "nodeID", nodeID, "requestID", requestID) + return nil +} + +// AppRequest is called by avalanchego -> VM when there is an incoming AppRequest from a peer +// error returned by this function is expected to be treated as fatal by the engine +// returns error if the requestHandler returns an error +// sends a response back to the sender if length of response returned by the handler is >0 +// expects the deadline to not have been passed +func (n *network) AppRequest(nodeID ids.ShortID, requestID uint32, deadline time.Time, request []byte) error { + n.lock.RLock() + defer n.lock.RUnlock() + + log.Debug("received AppRequest from node", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request)) + + var req message.Request + if _, err := n.codec.Unmarshal(request, &req); err != nil { + log.Debug("failed to unmarshal app request", "nodeID", nodeID, "requestID", requestID, "requestLen", len(request), "err", err) + return nil + } + + // calculate how much time is left until the deadline + timeTillDeadline := time.Until(deadline) + + // bufferedDeadline is half the time till actual deadline so that the message has a reasonable chance + // of completing its processing and sending the response to the peer. + timeTillDeadline = time.Duration(timeTillDeadline.Nanoseconds() / 2) + bufferedDeadline := time.Now().Add(timeTillDeadline) + + // check if we have enough time to handle this request + if time.Until(bufferedDeadline) < minRequestHandlingDuration { + // Drop the request if we already missed the deadline to respond. + log.Debug("deadline to process AppRequest has expired, skipping", "nodeID", nodeID, "requestID", requestID, "type", req.Type()) + return nil + } + + log.Debug("processing incoming request", "nodeID", nodeID, "requestID", requestID, "type", req.Type()) + ctx, cancel := context.WithDeadline(context.Background(), bufferedDeadline) + defer cancel() + + responseBytes, err := req.Handle(ctx, nodeID, requestID, n.requestHandler) + switch { + case err != nil && err != context.DeadlineExceeded: + return err // Return a fatal error + case responseBytes != nil: + return n.appSender.SendAppResponse(nodeID, requestID, responseBytes) // Propagate fatal error + default: + return nil + } +} + +// AppResponse is invoked when there is a response received from a peer regarding a request +// Error returned by this function is expected to be treated as fatal by the engine +// If [requestID] is not known, this function will emit a log and return a nil error. +// If the response handler returns an error it is propagated as a fatal error. +func (n *network) AppResponse(nodeID ids.ShortID, requestID uint32, response []byte) error { + n.lock.Lock() + defer n.lock.Unlock() + + log.Debug("received AppResponse from peer", "nodeID", nodeID, "requestID", requestID) + + handler, exists := n.getRequestHandler(requestID) + if !exists { + // Should never happen since the engine should be managing outstanding requests + log.Error("received response to unknown request", "nodeID", nodeID, "requestID", requestID, "responseLen", len(response)) + return nil + } + + return handler.OnResponse(nodeID, requestID, response) +} + +// AppRequestFailed can be called by the avalanchego -> VM in following cases: +// - node is benched +// - failed to send message to [nodeID] due to a network issue +// - timeout +// error returned by this function is expected to be treated as fatal by the engine +// returns error only when the response handler returns an error +func (n *network) AppRequestFailed(nodeID ids.ShortID, requestID uint32) error { + n.lock.Lock() + defer n.lock.Unlock() + log.Debug("received AppRequestFailed from peer", "nodeID", nodeID, "requestID", requestID) + + handler, exists := n.getRequestHandler(requestID) + if !exists { + // Should never happen since the engine should be managing outstanding requests + log.Error("received request failed to unknown request", "nodeID", nodeID, "requestID", requestID) + return nil + } + + return handler.OnFailure(nodeID, requestID) +} + +// getRequestHandler fetches the handler for [requestID] and marks the request with [requestID] as having been fulfilled. +// This is called by either [AppResponse] or [AppRequestFailed]. +// assumes that the write lock is held. +func (n *network) getRequestHandler(requestID uint32) (message.ResponseHandler, bool) { + handler, exists := n.outstandingResponseHandlerMap[requestID] + if !exists { + return nil, false + } + // mark message as processed, release activeRequests slot + delete(n.outstandingResponseHandlerMap, requestID) + n.activeRequests.Release(1) + return handler, true +} + +// Gossip sends given gossip message to peers +func (n *network) Gossip(gossip []byte) error { + return n.appSender.SendAppGossip(gossip) +} + +// AppGossip is called by avalanchego -> VM when there is an incoming AppGossip from a peer +// error returned by this function is expected to be treated as fatal by the engine +// returns error if request could not be parsed as message.Request or when the requestHandler returns an error +func (n *network) AppGossip(nodeID ids.ShortID, gossipBytes []byte) error { + var gossipMsg message.Message + if _, err := n.codec.Unmarshal(gossipBytes, &gossipMsg); err != nil { + log.Debug("could not parse app gossip", "nodeID", nodeID, "gossipLen", len(gossipBytes), "err", err) + return nil + } + + log.Debug("processing AppGossip from node", "nodeID", nodeID, "gossipLen", len(gossipBytes)) + return gossipMsg.Handle(n.gossipHandler, nodeID) +} + +// Connected adds the given nodeID to the peer list so that it can receive messages +func (n *network) Connected(nodeID ids.ShortID, nodeVersion version.Application) error { + log.Debug("adding new peer", "nodeID", nodeID) + + n.lock.Lock() + defer n.lock.Unlock() + + if nodeID == n.self { + log.Debug("skipping registering self as peer") + return nil + } + + if storedVersion, exists := n.peers[nodeID]; exists { + // Peer is already connected, update the version if it has changed. + // Log a warning message since the consensus engine should never call Connected on a peer + // that we have already marked as Connected. + if nodeVersion.Compare(storedVersion) != 0 { + n.peers[nodeID] = nodeVersion + log.Warn("received Connected message for already connected peer, updating node version", "nodeID", nodeID, "storedVersion", storedVersion, "nodeVersion", nodeVersion) + } else { + log.Warn("ignoring peer connected event for already connected peer with identical version", "nodeID", nodeID) + } + return nil + } + + n.peers[nodeID] = nodeVersion + return nil +} + +// Disconnected removes given [nodeID] from the peer list +func (n *network) Disconnected(nodeID ids.ShortID) error { + log.Debug("disconnecting peer", "nodeID", nodeID) + n.lock.Lock() + defer n.lock.Unlock() + + // if this peer already exists, log a warning and ignore the request + if _, exists := n.peers[nodeID]; !exists { + // we're not connected to this peer, nothing to do here + log.Warn("received peer disconnect request to unconnected peer", "nodeID", nodeID) + return nil + } + + delete(n.peers, nodeID) + return nil +} + +// Shutdown disconnects all peers +func (n *network) Shutdown() { + n.lock.Lock() + defer n.lock.Unlock() + + // reset peers map + n.peers = make(map[ids.ShortID]version.Application) +} + +func (n *network) SetGossipHandler(handler message.GossipHandler) { + n.lock.Lock() + defer n.lock.Unlock() + + n.gossipHandler = handler +} + +func (n *network) SetRequestHandler(handler message.RequestHandler) { + n.lock.Lock() + defer n.lock.Unlock() + + n.requestHandler = handler +} + +func (n *network) Size() uint32 { + n.lock.RLock() + defer n.lock.RUnlock() + + return uint32(len(n.peers)) +} diff --git a/peer/network_test.go b/peer/network_test.go new file mode 100644 index 0000000000..ff3c9909cb --- /dev/null +++ b/peer/network_test.go @@ -0,0 +1,478 @@ +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package peer + +import ( + "context" + "errors" + "fmt" + "sync" + "sync/atomic" + "testing" + "time" + + "github.com/ava-labs/avalanchego/snow/engine/common" + + "github.com/ava-labs/subnet-evm/plugin/evm/message" + + "github.com/ava-labs/avalanchego/codec" + "github.com/ava-labs/avalanchego/codec/linearcodec" + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/avalanchego/version" + "github.com/stretchr/testify/assert" +) + +var ( + defaultPeerVersion = version.NewDefaultApplication("subnet-evmtest", 1, 0, 0) + + _ message.Request = &HelloRequest{} + _ = &HelloResponse{} + _ = &GreetingRequest{} + _ = &GreetingResponse{} + _ = &TestMessage{} + + _ message.RequestHandler = &HelloGreetingRequestHandler{} + _ message.RequestHandler = &testRequestHandler{} + + _ common.AppSender = testAppSender{} + _ message.Message = HelloGossip{} + _ message.GossipHandler = &testGossipHandler{} +) + +func TestNetworkDoesNotConnectToItself(t *testing.T) { + selfNodeID := ids.GenerateTestShortID() + n := NewNetwork(nil, nil, selfNodeID, 1) + assert.NoError(t, n.Connected(selfNodeID, version.NewDefaultApplication("avalanchego", 1, 0, 0))) + assert.EqualValues(t, 0, n.Size()) +} + +func TestRequestsRoutingAndResponse(t *testing.T) { + callNum := uint32(0) + senderWg := &sync.WaitGroup{} + var net Network + sender := testAppSender{ + sendAppRequestFn: func(nodes ids.ShortSet, requestID uint32, requestBytes []byte) error { + nodeID, _ := nodes.Pop() + senderWg.Add(1) + go func() { + defer senderWg.Done() + if err := net.AppRequest(nodeID, requestID, time.Now().Add(5*time.Second), requestBytes); err != nil { + panic(err) + } + }() + return nil + }, + sendAppResponseFn: func(nodeID ids.ShortID, requestID uint32, responseBytes []byte) error { + senderWg.Add(1) + go func() { + defer senderWg.Done() + if err := net.AppResponse(nodeID, requestID, responseBytes); err != nil { + panic(err) + } + atomic.AddUint32(&callNum, 1) + }() + return nil + }, + } + + codecManager := buildCodec(t, HelloRequest{}, HelloResponse{}) + net = NewNetwork(sender, codecManager, ids.ShortEmpty, 16) + net.SetRequestHandler(&HelloGreetingRequestHandler{codec: codecManager}) + client := NewClient(net) + nodeID := ids.GenerateTestShortID() + assert.NoError(t, net.Connected(nodeID, defaultPeerVersion)) + + requestMessage := HelloRequest{Message: "this is a request"} + + defer net.Shutdown() + assert.NoError(t, net.Connected(nodeID, defaultPeerVersion)) + + totalRequests := 5000 + numCallsPerRequest := 1 // on sending response + totalCalls := totalRequests * numCallsPerRequest + + requestWg := &sync.WaitGroup{} + requestWg.Add(totalCalls) + for i := 0; i < totalCalls; i++ { + go func(wg *sync.WaitGroup) { + defer wg.Done() + requestBytes, err := message.RequestToBytes(codecManager, requestMessage) + assert.NoError(t, err) + responseBytes, failed, err := client.RequestAny(defaultPeerVersion, requestBytes) + assert.NoError(t, err) + assert.False(t, failed) + assert.NotNil(t, responseBytes) + + var response TestMessage + if _, err = codecManager.Unmarshal(responseBytes, &response); err != nil { + panic(fmt.Errorf("unexpected error during unmarshal: %w", err)) + } + assert.Equal(t, "Hi", response.Message) + }(requestWg) + } + + requestWg.Wait() + senderWg.Wait() + assert.Equal(t, totalCalls, int(atomic.LoadUint32(&callNum))) +} + +func TestRequestMinVersion(t *testing.T) { + callNum := uint32(0) + nodeID := ids.GenerateTestShortID() + codecManager := buildCodec(t, TestMessage{}) + + var net Network + sender := testAppSender{ + sendAppRequestFn: func(nodes ids.ShortSet, reqID uint32, messageBytes []byte) error { + atomic.AddUint32(&callNum, 1) + assert.True(t, nodes.Contains(nodeID), "request nodes should contain expected nodeID") + assert.Len(t, nodes, 1, "request nodes should contain exactly one node") + + go func() { + time.Sleep(200 * time.Millisecond) + atomic.AddUint32(&callNum, 1) + responseBytes, err := codecManager.Marshal(message.Version, TestMessage{Message: "this is a response"}) + if err != nil { + panic(err) + } + err = net.AppResponse(nodeID, reqID, responseBytes) + assert.NoError(t, err) + }() + return nil + }, + } + + // passing nil as codec works because the net.AppRequest is never called + net = NewNetwork(sender, codecManager, ids.ShortEmpty, 1) + client := NewClient(net) + requestMessage := TestMessage{Message: "this is a request"} + requestBytes, err := message.RequestToBytes(codecManager, requestMessage) + assert.NoError(t, err) + assert.NoError(t, net.Connected(nodeID, version.NewDefaultApplication("subnet-evmtest", 1, 7, 1))) + + // ensure version does not match + responseBytes, failed, err := client.RequestAny(version.NewDefaultApplication("subnet-evmtest", 2, 0, 0), requestBytes) + assert.Equal(t, err.Error(), "no peers found matching version subnet-evmtest/2.0.0 out of 1 peers") + assert.True(t, failed) + assert.Nil(t, responseBytes) + + // ensure version matches and the request goes through + responseBytes, failed, err = client.RequestAny(version.NewDefaultApplication("subnet-evmtest", 1, 0, 0), requestBytes) + assert.NoError(t, err) + assert.False(t, failed) + + var response TestMessage + if _, err = codecManager.Unmarshal(responseBytes, &response); err != nil { + t.Fatal("unexpected error during unmarshal", err) + } + assert.Equal(t, "this is a response", response.Message) +} + +func TestOnRequestHonoursDeadline(t *testing.T) { + var net Network + responded := false + sender := testAppSender{ + sendAppRequestFn: func(nodes ids.ShortSet, reqID uint32, message []byte) error { + return nil + }, + sendAppResponseFn: func(nodeID ids.ShortID, reqID uint32, message []byte) error { + responded = true + return nil + }, + } + + codecManager := buildCodec(t, TestMessage{}) + + requestBytes, err := marshalStruct(codecManager, TestMessage{Message: "hello there"}) + assert.NoError(t, err) + + requestHandler := &testRequestHandler{ + processingDuration: 500 * time.Millisecond, + } + net = NewNetwork(sender, codecManager, ids.ShortEmpty, 1) + net.SetRequestHandler(requestHandler) + nodeID := ids.GenerateTestShortID() + + requestHandler.response, err = marshalStruct(codecManager, TestMessage{Message: "hi there"}) + assert.NoError(t, err) + err = net.AppRequest(nodeID, 1, time.Now().Add(1*time.Millisecond), requestBytes) + assert.NoError(t, err) + // ensure the handler didn't get called (as peer.Network would've dropped the request) + assert.EqualValues(t, requestHandler.calls, 0) + + requestHandler.processingDuration = 0 + err = net.AppRequest(nodeID, 2, time.Now().Add(250*time.Millisecond), requestBytes) + assert.NoError(t, err) + assert.True(t, responded) + assert.EqualValues(t, requestHandler.calls, 1) +} + +func TestGossip(t *testing.T) { + codecManager := buildCodec(t, HelloGossip{}) + + nodeID := ids.GenerateTestShortID() + var clientNetwork Network + wg := &sync.WaitGroup{} + sentGossip := false + wg.Add(1) + sender := testAppSender{ + sendAppGossipFn: func(msg []byte) error { + go func() { + defer wg.Done() + err := clientNetwork.AppGossip(nodeID, msg) + assert.NoError(t, err) + }() + sentGossip = true + return nil + }, + } + + gossipHandler := &testGossipHandler{} + clientNetwork = NewNetwork(sender, codecManager, ids.ShortEmpty, 1) + clientNetwork.SetGossipHandler(gossipHandler) + + assert.NoError(t, clientNetwork.Connected(nodeID, defaultPeerVersion)) + + client := NewClient(clientNetwork) + defer clientNetwork.Shutdown() + + b, err := buildGossip(codecManager, HelloGossip{Msg: "hello there!"}) + assert.NoError(t, err) + + err = client.Gossip(b) + assert.NoError(t, err) + + wg.Wait() + assert.True(t, sentGossip) + assert.True(t, gossipHandler.received) +} + +func TestHandleInvalidMessages(t *testing.T) { + codecManager := buildCodec(t, HelloGossip{}, TestMessage{}) + + nodeID := ids.GenerateTestShortID() + requestID := uint32(1) + sender := testAppSender{} + + clientNetwork := NewNetwork(sender, codecManager, ids.ShortEmpty, 1) + clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) + clientNetwork.SetRequestHandler(&testRequestHandler{}) + + assert.NoError(t, clientNetwork.Connected(nodeID, defaultPeerVersion)) + + defer clientNetwork.Shutdown() + + // Ensure a valid gossip message sent as any App specific message type does not trigger a fatal error + gossipMsg, err := buildGossip(codecManager, HelloGossip{Msg: "hello there!"}) + assert.NoError(t, err) + + // Ensure a valid request message sent as any App specific message type does not trigger a fatal error + requestMessage, err := marshalStruct(codecManager, TestMessage{Message: "Hello"}) + assert.NoError(t, err) + + // Ensure a random message sent as any App specific message type does not trigger a fatal error + garbageResponse := make([]byte, 10) + // Ensure a zero-length message sent as any App specific message type does not trigger a fatal error + emptyResponse := make([]byte, 0) + // Ensure a nil byte slice sent as any App specific message type does not trigger a fatal error + var nilResponse []byte + + // Check for edge cases + assert.NoError(t, clientNetwork.AppGossip(nodeID, gossipMsg)) + assert.NoError(t, clientNetwork.AppGossip(nodeID, requestMessage)) + assert.NoError(t, clientNetwork.AppGossip(nodeID, garbageResponse)) + assert.NoError(t, clientNetwork.AppGossip(nodeID, emptyResponse)) + assert.NoError(t, clientNetwork.AppGossip(nodeID, nilResponse)) + assert.NoError(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), gossipMsg)) + assert.NoError(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), requestMessage)) + assert.NoError(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), garbageResponse)) + assert.NoError(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), emptyResponse)) + assert.NoError(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), nilResponse)) + assert.NoError(t, clientNetwork.AppResponse(nodeID, requestID, gossipMsg)) + assert.NoError(t, clientNetwork.AppResponse(nodeID, requestID, requestMessage)) + assert.NoError(t, clientNetwork.AppResponse(nodeID, requestID, garbageResponse)) + assert.NoError(t, clientNetwork.AppResponse(nodeID, requestID, emptyResponse)) + assert.NoError(t, clientNetwork.AppResponse(nodeID, requestID, nilResponse)) + assert.NoError(t, clientNetwork.AppRequestFailed(nodeID, requestID)) +} + +func TestNetworkPropagatesRequestHandlerError(t *testing.T) { + codecManager := buildCodec(t, TestMessage{}) + + nodeID := ids.GenerateTestShortID() + requestID := uint32(1) + sender := testAppSender{} + + clientNetwork := NewNetwork(sender, codecManager, ids.ShortEmpty, 1) + clientNetwork.SetGossipHandler(message.NoopMempoolGossipHandler{}) + clientNetwork.SetRequestHandler(&testRequestHandler{err: errors.New("fail")}) // Return an error from the request handler + + assert.NoError(t, clientNetwork.Connected(nodeID, defaultPeerVersion)) + + defer clientNetwork.Shutdown() + + // Ensure a valid request message sent as any App specific message type does not trigger a fatal error + requestMessage, err := marshalStruct(codecManager, TestMessage{Message: "Hello"}) + assert.NoError(t, err) + + // Check that if the request handler returns an error, it is propagated as a fatal error. + assert.Error(t, clientNetwork.AppRequest(nodeID, requestID, time.Now().Add(time.Second), requestMessage)) +} + +func buildCodec(t *testing.T, types ...interface{}) codec.Manager { + codecManager := codec.NewDefaultManager() + c := linearcodec.NewDefault() + for _, typ := range types { + assert.NoError(t, c.RegisterType(typ)) + } + assert.NoError(t, codecManager.RegisterCodec(message.Version, c)) + return codecManager +} + +// marshalStruct is a helper method used to marshal an object as `interface{}` +// so that the codec is able to include the TypeID in the resulting bytes +func marshalStruct(codec codec.Manager, obj interface{}) ([]byte, error) { + return codec.Marshal(message.Version, &obj) +} + +func buildGossip(codec codec.Manager, msg message.Message) ([]byte, error) { + return codec.Marshal(message.Version, &msg) +} + +type testAppSender struct { + sendAppRequestFn func(ids.ShortSet, uint32, []byte) error + sendAppResponseFn func(ids.ShortID, uint32, []byte) error + sendAppGossipFn func([]byte) error +} + +func (t testAppSender) SendAppGossipSpecific(ids.ShortSet, []byte) error { + panic("not implemented") +} + +func (t testAppSender) SendAppRequest(nodeIDs ids.ShortSet, requestID uint32, message []byte) error { + return t.sendAppRequestFn(nodeIDs, requestID, message) +} + +func (t testAppSender) SendAppResponse(nodeID ids.ShortID, requestID uint32, message []byte) error { + return t.sendAppResponseFn(nodeID, requestID, message) +} + +func (t testAppSender) SendAppGossip(message []byte) error { + return t.sendAppGossipFn(message) +} + +type HelloRequest struct { + Message string `serialize:"true"` +} + +func (h HelloRequest) Handle(ctx context.Context, nodeID ids.ShortID, requestID uint32, handler message.RequestHandler) ([]byte, error) { + // casting is only necessary for test since RequestHandler does not implement anything at the moment + return handler.(TestRequestHandler).HandleHelloRequest(ctx, nodeID, requestID, &h) +} + +func (h HelloRequest) Type() string { + return "hello-request" +} + +type GreetingRequest struct { + Greeting string `serialize:"true"` +} + +func (g GreetingRequest) Handle(ctx context.Context, nodeID ids.ShortID, requestID uint32, handler message.RequestHandler) ([]byte, error) { + // casting is only necessary for test since RequestHandler does not implement anything at the moment + return handler.(TestRequestHandler).HandleGreetingRequest(ctx, nodeID, requestID, &g) +} + +func (g GreetingRequest) Type() string { + return "greeting-request" +} + +type HelloResponse struct { + Response string `serialize:"true"` +} + +type GreetingResponse struct { + Greet string `serialize:"true"` +} + +type TestRequestHandler interface { + HandleHelloRequest(ctx context.Context, nodeID ids.ShortID, requestID uint32, request *HelloRequest) ([]byte, error) + HandleGreetingRequest(ctx context.Context, nodeID ids.ShortID, requestID uint32, request *GreetingRequest) ([]byte, error) +} + +type HelloGreetingRequestHandler struct { + codec codec.Manager +} + +func (h *HelloGreetingRequestHandler) HandleHelloRequest(ctx context.Context, nodeID ids.ShortID, requestID uint32, request *HelloRequest) ([]byte, error) { + return h.codec.Marshal(message.Version, HelloResponse{Response: "Hi"}) +} + +func (h *HelloGreetingRequestHandler) HandleGreetingRequest(ctx context.Context, nodeID ids.ShortID, requestID uint32, request *GreetingRequest) ([]byte, error) { + return h.codec.Marshal(message.Version, GreetingResponse{Greet: "Hey there"}) +} + +type TestMessage struct { + Message string `serialize:"true"` +} + +func (t TestMessage) Handle(ctx context.Context, nodeID ids.ShortID, requestID uint32, handler message.RequestHandler) ([]byte, error) { + return handler.(*testRequestHandler).handleTestRequest(ctx, nodeID, requestID, &t) +} + +func (t TestMessage) Type() string { + return "test-message" +} + +type HelloGossip struct { + message.Message + Msg string `serialize:"true"` +} + +func (h HelloGossip) Handle(handler message.GossipHandler, nodeID ids.ShortID) error { + return handler.HandleTxs(nodeID, nil) +} + +func (h HelloGossip) Type() string { + return "hello-gossip" +} + +func (h HelloGossip) initialize(_ []byte) { + // no op +} + +func (h HelloGossip) Bytes() []byte { + // no op + return nil +} + +type testGossipHandler struct { + received bool + nodeID ids.ShortID + msg []byte +} + +func (t *testGossipHandler) HandleTxs(nodeID ids.ShortID, _ *message.Txs) error { + t.received = true + t.nodeID = nodeID + return nil +} + +type testRequestHandler struct { + calls uint32 + processingDuration time.Duration + response []byte + err error +} + +func (r *testRequestHandler) handleTestRequest(ctx context.Context, _ ids.ShortID, _ uint32, _ *TestMessage) ([]byte, error) { + r.calls++ + select { + case <-time.After(r.processingDuration): + break + case <-ctx.Done(): + return nil, ctx.Err() + } + return r.response, r.err +} diff --git a/peer/waiting_handler.go b/peer/waiting_handler.go new file mode 100644 index 0000000000..34471f5cdd --- /dev/null +++ b/peer/waiting_handler.go @@ -0,0 +1,39 @@ +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package peer + +import ( + "github.com/ava-labs/avalanchego/ids" + "github.com/ava-labs/subnet-evm/plugin/evm/message" +) + +var _ message.ResponseHandler = &waitingResponseHandler{} + +// waitingResponseHandler implements the ResponseHandler interface +// Internally used to wait for response after making a request synchronously +// responseChan may contain response bytes if the original request has not failed +// responseChan is closed in either fail or success scenario +type waitingResponseHandler struct { + responseChan chan []byte // blocking channel with response bytes + failed bool // whether the original request is failed +} + +// OnResponse passes the response bytes to the responseChan and closes the channel +func (w *waitingResponseHandler) OnResponse(_ ids.ShortID, _ uint32, response []byte) error { + w.responseChan <- response + close(w.responseChan) + return nil +} + +// OnFailure sets the failed flag to true and closes the channel +func (w *waitingResponseHandler) OnFailure(ids.ShortID, uint32) error { + w.failed = true + close(w.responseChan) + return nil +} + +// newWaitingResponseHandler returns new instance of the waitingResponseHandler +func newWaitingResponseHandler() *waitingResponseHandler { + return &waitingResponseHandler{responseChan: make(chan []byte)} +} diff --git a/plugin/evm/block_builder.go b/plugin/evm/block_builder.go index 28d3671b2b..d6d8259f2e 100644 --- a/plugin/evm/block_builder.go +++ b/plugin/evm/block_builder.go @@ -52,8 +52,8 @@ type blockBuilder struct { ctx *snow.Context chainConfig *params.ChainConfig - chain *subnetEVM.ETHChain - network Network + chain *subnetEVM.ETHChain + gossiper Gossiper shutdownChan <-chan struct{} shutdownWg *sync.WaitGroup @@ -80,7 +80,7 @@ type blockBuilder struct { // isSC is a boolean indicating if SubnetEVM is activated. This prevents us from // getting the current time and comparing it to the *params.chainConfig more // than once. - isSC bool + isSE bool } func (vm *VM) NewBlockBuilder(notifyBuildBlockChan chan<- commonEng.Message) *blockBuilder { @@ -88,7 +88,7 @@ func (vm *VM) NewBlockBuilder(notifyBuildBlockChan chan<- commonEng.Message) *bl ctx: vm.ctx, chainConfig: vm.chainConfig, chain: vm.chain, - network: vm.network, + gossiper: vm.gossiper, shutdownChan: vm.shutdownChan, shutdownWg: &vm.shutdownWg, notifyBuildBlockChan: notifyBuildBlockChan, @@ -107,7 +107,7 @@ func (b *blockBuilder) handleBlockBuilding() { b.shutdownWg.Add(1) go b.ctx.Log.RecoverAndPanic(b.migrateSC) } else { - b.isSC = true + b.isSE = true } } @@ -126,7 +126,7 @@ func (b *blockBuilder) migrateSC() { duration := time.Until(timestamp) select { case <-time.After(duration): - b.isSC = true + b.isSE = true b.buildBlockLock.Lock() // Flush any invalid statuses leftover from legacy block timer builder if b.buildStatus == conditionalBuild { @@ -147,7 +147,7 @@ func (b *blockBuilder) handleGenerateBlock() { b.buildBlockLock.Lock() defer b.buildBlockLock.Unlock() - if !b.isSC { + if !b.isSE { // Set the buildStatus before calling Cancel or Issue on // the mempool and after generating the block. // This prevents [needToBuild] from returning true when the @@ -244,7 +244,7 @@ func (b *blockBuilder) signalTxsReady() { return } - if !b.isSC { + if !b.isSE { b.buildStatus = conditionalBuild b.buildBlockTimer.SetTimeoutIn(minBlockTime) return @@ -272,18 +272,18 @@ func (b *blockBuilder) awaitSubmittedTxs() { txSubmitChan := b.chain.GetTxSubmitCh() for { select { - case ethTxsEvent := <-txSubmitChan: + case txsEvent := <-txSubmitChan: log.Trace("New tx detected, trying to generate a block") b.signalTxsReady() - // We only attempt to invoke [GossipEthTxs] once SC is activated - if b.isSC && b.network != nil && len(ethTxsEvent.Txs) > 0 { + // We only attempt to invoke [GossipTxs] once AP4 is activated + if b.isSE && b.gossiper != nil && len(txsEvent.Txs) > 0 { // Give time for this node to build a block before attempting to // gossip time.Sleep(waitBlockTime) - // [GossipEthTxs] will block unless [pushNetwork.ethTxsToGossipChan] (an + // [GossipTxs] will block unless [gossiper.txsToGossipChan] (an // unbuffered channel) is listened on - if err := b.network.GossipEthTxs(ethTxsEvent.Txs); err != nil { + if err := b.gossiper.GossipTxs(txsEvent.Txs); err != nil { log.Warn( "failed to gossip new eth transactions", "err", err, diff --git a/plugin/evm/block_builder_test.go b/plugin/evm/block_builder_test.go index 96f59c3b97..e61455c078 100644 --- a/plugin/evm/block_builder_test.go +++ b/plugin/evm/block_builder_test.go @@ -83,7 +83,7 @@ func TestBlockBuilderStopsTimer(t *testing.T) { if builder.buildStatus != mayBuild { t.Fatalf("expected build status to be %d but got %d", dontBuild, builder.buildStatus) } - if !builder.isSC { + if !builder.isSE { t.Fatal("expected isSC to be true") } } diff --git a/plugin/evm/config.go b/plugin/evm/config.go index 65a7548c1a..e33ad28c70 100644 --- a/plugin/evm/config.go +++ b/plugin/evm/config.go @@ -12,21 +12,23 @@ import ( ) const ( - defaultPruningEnabled = true - defaultSnapshotAsync = true - defaultRpcGasCap = 50_000_000 // Default to 50M Gas Limit - defaultRpcTxFeeCap = 100 // 100 AVAX - defaultMetricsEnabled = false - defaultMetricsExpensiveEnabled = false - defaultApiMaxDuration = 0 // Default to no maximum API call duration - defaultWsCpuRefillRate = 0 // Default to no maximum WS CPU usage - defaultWsCpuMaxStored = 0 // Default to no maximum WS CPU usage - defaultMaxBlocksPerRequest = 0 // Default to no maximum on the number of blocks per getLogs request - defaultContinuousProfilerFrequency = 15 * time.Minute - defaultContinuousProfilerMaxFiles = 5 - defaultTxRegossipFrequency = 1 * time.Minute - defaultTxRegossipMaxSize = 15 - defaultLogLevel = "info" + defaultPruningEnabled = true + defaultSnapshotAsync = true + defaultRpcGasCap = 50_000_000 // Default to 50M Gas Limit + defaultRpcTxFeeCap = 100 // 100 AVAX + defaultMetricsEnabled = true + defaultMetricsExpensiveEnabled = false + defaultApiMaxDuration = 0 // Default to no maximum API call duration + defaultWsCpuRefillRate = 0 // Default to no maximum WS CPU usage + defaultWsCpuMaxStored = 0 // Default to no maximum WS CPU usage + defaultMaxBlocksPerRequest = 0 // Default to no maximum on the number of blocks per getLogs request + defaultContinuousProfilerFrequency = 15 * time.Minute + defaultContinuousProfilerMaxFiles = 5 + defaultTxRegossipFrequency = 1 * time.Minute + defaultTxRegossipMaxSize = 15 + defaultOfflinePruningBloomFilterSize uint64 = 512 // Default size (MB) for the offline pruner to use + defaultLogLevel = "info" + defaultMaxOutboundActiveRequests = 8 ) var defaultEnabledAPIs = []string{ @@ -97,6 +99,14 @@ type Config struct { // Address for Tx Fees (must be empty if not supported by blockchain) FeeRecipient string `json:"feeRecipient"` + + // Offline Pruning Settings + OfflinePruning bool `json:"offline-pruning-enabled"` + OfflinePruningBloomFilterSize uint64 `json:"offline-pruning-bloom-filter-size"` + OfflinePruningDataDirectory string `json:"offline-pruning-data-directory"` + + // VM2VM network + MaxOutboundActiveRequests int64 `json:"max-outbound-active-requests"` } // EthAPIs returns an array of strings representing the Eth APIs that should be enabled @@ -124,7 +134,9 @@ func (c *Config) SetDefaults() { c.SnapshotAsync = defaultSnapshotAsync c.TxRegossipFrequency.Duration = defaultTxRegossipFrequency c.TxRegossipMaxSize = defaultTxRegossipMaxSize + c.OfflinePruningBloomFilterSize = defaultOfflinePruningBloomFilterSize c.LogLevel = defaultLogLevel + c.MaxOutboundActiveRequests = defaultMaxOutboundActiveRequests } func (d *Duration) UnmarshalJSON(data []byte) (err error) { diff --git a/plugin/evm/network.go b/plugin/evm/gossiper.go similarity index 51% rename from plugin/evm/network.go rename to plugin/evm/gossiper.go index 0cf7bd6199..10709155e4 100644 --- a/plugin/evm/network.go +++ b/plugin/evm/gossiper.go @@ -9,12 +9,14 @@ import ( "sync" "time" + "github.com/ava-labs/avalanchego/codec" + + "github.com/ava-labs/subnet-evm/peer" + "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/snow" - commonEng "github.com/ava-labs/avalanchego/snow/engine/common" - "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/rlp" @@ -23,8 +25,6 @@ import ( "github.com/ava-labs/subnet-evm/core/state" "github.com/ava-labs/subnet-evm/core/types" "github.com/ava-labs/subnet-evm/plugin/evm/message" - - subnetEVM "github.com/ava-labs/subnet-evm/chain" ) const ( @@ -32,96 +32,58 @@ const ( // in the cache, not entire transactions. recentCacheSize = 512 - // [ethTxsGossipInterval] is how often we attempt to gossip newly seen + // [txsGossipInterval] is how often we attempt to gossip newly seen // transactions to other nodes. - ethTxsGossipInterval = 500 * time.Millisecond + txsGossipInterval = 500 * time.Millisecond ) -type Network interface { - // Message handling - AppRequestFailed(nodeID ids.ShortID, requestID uint32) error - AppRequest(nodeID ids.ShortID, requestID uint32, deadline time.Time, msgBytes []byte) error - AppResponse(nodeID ids.ShortID, requestID uint32, msgBytes []byte) error - AppGossip(nodeID ids.ShortID, msgBytes []byte) error - - // Gossip entrypoints - GossipEthTxs(txs []*types.Transaction) error -} - -func (vm *VM) AppRequest(nodeID ids.ShortID, requestID uint32, deadline time.Time, request []byte) error { - return vm.network.AppRequest(nodeID, requestID, deadline, request) -} - -func (vm *VM) AppResponse(nodeID ids.ShortID, requestID uint32, response []byte) error { - return vm.network.AppResponse(nodeID, requestID, response) +// Gossiper handles outgoing gossip of transactions +type Gossiper interface { + // GossipTxs sends AppGossip message containing the given [txs] + GossipTxs(txs []*types.Transaction) error } -func (vm *VM) AppRequestFailed(nodeID ids.ShortID, requestID uint32) error { - return vm.network.AppRequestFailed(nodeID, requestID) -} - -func (vm *VM) AppGossip(nodeID ids.ShortID, msg []byte) error { - return vm.network.AppGossip(nodeID, msg) -} - -// NewNetwork creates a new Network based on the [vm.chainConfig]. -func (vm *VM) NewNetwork(appSender commonEng.AppSender) Network { - if vm.chainConfig.SubnetEVMTimestamp != nil { - return vm.newPushNetwork( - time.Unix(vm.chainConfig.SubnetEVMTimestamp.Int64(), 0), - vm.config, - appSender, - vm.chain, - ) - } - - return &noopNetwork{} -} - -type pushNetwork struct { +// pushGossiper is used to gossip transactions to the network +type pushGossiper struct { ctx *snow.Context gossipActivationTime time.Time config Config - appSender commonEng.AppSender - chain *subnetEVM.ETHChain - - gossipHandler message.Handler + client peer.Client + blockchain *core.BlockChain + txPool *core.TxPool // We attempt to batch transactions we need to gossip to avoid runaway // amplification of mempol chatter. - ethTxsToGossipChan chan []*types.Transaction - ethTxsToGossip map[common.Hash]*types.Transaction - lastGossiped time.Time - shutdownChan chan struct{} - shutdownWg *sync.WaitGroup + txsToGossipChan chan []*types.Transaction + txsToGossip map[common.Hash]*types.Transaction + lastGossiped time.Time + shutdownChan chan struct{} + shutdownWg *sync.WaitGroup - // [recentEthTxs] prevent us from over-gossiping the + // [recentTxs] prevent us from over-gossiping the // same transaction in a short period of time. - recentEthTxs *cache.LRU + recentTxs *cache.LRU + + codec codec.Manager } -func (vm *VM) newPushNetwork( - activationTime time.Time, - config Config, - appSender commonEng.AppSender, - chain *subnetEVM.ETHChain, -) Network { - net := &pushNetwork{ +// newPushGossiper constructs and returns a pushGossiper +// assumes vm.chainConfig.SubnetEVMTimestamp is set +func (vm *VM) newPushGossiper() Gossiper { + net := &pushGossiper{ ctx: vm.ctx, - gossipActivationTime: activationTime, - config: config, - appSender: appSender, - chain: chain, - ethTxsToGossipChan: make(chan []*types.Transaction), - ethTxsToGossip: make(map[common.Hash]*types.Transaction), + gossipActivationTime: time.Unix(vm.chainConfig.SubnetEVMTimestamp.Int64(), 0), + config: vm.config, + client: vm.client, + blockchain: vm.chain.BlockChain(), + txPool: vm.chain.GetTxPool(), + txsToGossipChan: make(chan []*types.Transaction), + txsToGossip: make(map[common.Hash]*types.Transaction), shutdownChan: vm.shutdownChan, shutdownWg: &vm.shutdownWg, - recentEthTxs: &cache.LRU{Size: recentCacheSize}, - } - net.gossipHandler = &GossipHandler{ - vm: vm, - net: net, + recentTxs: &cache.LRU{Size: recentCacheSize}, + codec: vm.networkCodec, } net.awaitEthTxGossip() return net @@ -133,7 +95,7 @@ func (vm *VM) newPushNetwork( // We assume that [txs] contains an array of nonce-ordered transactions for a given // account. This array of transactions can have gaps and start at a nonce lower // than the current state of an account. -func (n *pushNetwork) queueExecutableTxs(state *state.StateDB, baseFee *big.Int, txs map[common.Address]types.Transactions, maxTxs int) types.Transactions { +func (n *pushGossiper) queueExecutableTxs(state *state.StateDB, baseFee *big.Int, txs map[common.Address]types.Transactions, maxTxs int) types.Transactions { // Setup heap for transactions heads := make(types.TxByPriceAndTime, 0, len(txs)) for addr, accountTxs := range txs { @@ -197,17 +159,15 @@ func (n *pushNetwork) queueExecutableTxs(state *state.StateDB, baseFee *big.Int, } // queueRegossipTxs finds the best transactions in the mempool and adds up to -// [TxRegossipMaxSize] of them to [ethTxsToGossip]. -func (n *pushNetwork) queueRegossipTxs() types.Transactions { - txPool := n.chain.GetTxPool() - +// [TxRegossipMaxSize] of them to [txsToGossip]. +func (n *pushGossiper) queueRegossipTxs() types.Transactions { // Fetch all pending transactions - pending := txPool.Pending(true) + pending := n.txPool.Pending(true) // Split the pending transactions into locals and remotes localTxs := make(map[common.Address]types.Transactions) remoteTxs := pending - for _, account := range txPool.Locals() { + for _, account := range n.txPool.Locals() { if txs := remoteTxs[account]; len(txs) > 0 { delete(remoteTxs, account) localTxs[account] = txs @@ -215,8 +175,8 @@ func (n *pushNetwork) queueRegossipTxs() types.Transactions { } // Add best transactions to be gossiped (preferring local txs) - tip := n.chain.BlockChain().CurrentBlock() - state, err := n.chain.BlockChain().StateAt(tip.Root()) + tip := n.blockchain.CurrentBlock() + state, err := n.blockchain.StateAt(tip.Root()) if err != nil || state == nil { log.Debug( "could not get state at tip", @@ -235,21 +195,21 @@ func (n *pushNetwork) queueRegossipTxs() types.Transactions { } // awaitEthTxGossip periodically gossips transactions that have been queued for -// gossip at least once every [ethTxsGossipInterval]. -func (n *pushNetwork) awaitEthTxGossip() { +// gossip at least once every [txsGossipInterval]. +func (n *pushGossiper) awaitEthTxGossip() { n.shutdownWg.Add(1) go n.ctx.Log.RecoverAndPanic(func() { defer n.shutdownWg.Done() var ( - gossipTicker = time.NewTicker(ethTxsGossipInterval) + gossipTicker = time.NewTicker(txsGossipInterval) regossipTicker = time.NewTicker(n.config.TxRegossipFrequency.Duration) ) for { select { case <-gossipTicker.C: - if attempted, err := n.gossipEthTxs(false); err != nil { + if attempted, err := n.gossipTxs(false); err != nil { log.Warn( "failed to send eth transactions", "len(txs)", attempted, @@ -258,20 +218,20 @@ func (n *pushNetwork) awaitEthTxGossip() { } case <-regossipTicker.C: for _, tx := range n.queueRegossipTxs() { - n.ethTxsToGossip[tx.Hash()] = tx + n.txsToGossip[tx.Hash()] = tx } - if attempted, err := n.gossipEthTxs(true); err != nil { + if attempted, err := n.gossipTxs(true); err != nil { log.Warn( "failed to send eth transactions", "len(txs)", attempted, "err", err, ) } - case txs := <-n.ethTxsToGossipChan: + case txs := <-n.txsToGossipChan: for _, tx := range txs { - n.ethTxsToGossip[tx.Hash()] = tx + n.txsToGossip[tx.Hash()] = tx } - if attempted, err := n.gossipEthTxs(false); err != nil { + if attempted, err := n.gossipTxs(false); err != nil { log.Warn( "failed to send eth transactions", "len(txs)", attempted, @@ -285,29 +245,7 @@ func (n *pushNetwork) awaitEthTxGossip() { }) } -func (n *pushNetwork) AppRequestFailed(nodeID ids.ShortID, requestID uint32) error { - return nil -} - -func (n *pushNetwork) AppRequest(nodeID ids.ShortID, requestID uint32, deadline time.Time, msgBytes []byte) error { - return nil -} - -func (n *pushNetwork) AppResponse(nodeID ids.ShortID, requestID uint32, msgBytes []byte) error { - return nil -} - -func (n *pushNetwork) AppGossip(nodeID ids.ShortID, msgBytes []byte) error { - return n.handle( - n.gossipHandler, - "Gossip", - nodeID, - 0, - msgBytes, - ) -} - -func (n *pushNetwork) sendEthTxs(txs []*types.Transaction) error { +func (n *pushGossiper) sendTxs(txs []*types.Transaction) error { if len(txs) == 0 { return nil } @@ -319,7 +257,7 @@ func (n *pushNetwork) sendEthTxs(txs []*types.Transaction) error { msg := message.Txs{ Txs: txBytes, } - msgBytes, err := message.Build(&msg) + msgBytes, err := message.BuildMessage(n.codec, &msg) if err != nil { return err } @@ -329,41 +267,40 @@ func (n *pushNetwork) sendEthTxs(txs []*types.Transaction) error { "len(txs)", len(txs), "size(txs)", len(msg.Txs), ) - return n.appSender.SendAppGossip(msgBytes) + return n.client.Gossip(msgBytes) } -func (n *pushNetwork) gossipEthTxs(force bool) (int, error) { - if (!force && time.Since(n.lastGossiped) < ethTxsGossipInterval) || len(n.ethTxsToGossip) == 0 { +func (n *pushGossiper) gossipTxs(force bool) (int, error) { + if (!force && time.Since(n.lastGossiped) < txsGossipInterval) || len(n.txsToGossip) == 0 { return 0, nil } n.lastGossiped = time.Now() - txs := make([]*types.Transaction, 0, len(n.ethTxsToGossip)) - for _, tx := range n.ethTxsToGossip { + txs := make([]*types.Transaction, 0, len(n.txsToGossip)) + for _, tx := range n.txsToGossip { txs = append(txs, tx) - delete(n.ethTxsToGossip, tx.Hash()) + delete(n.txsToGossip, tx.Hash()) } - pool := n.chain.GetTxPool() selectedTxs := make([]*types.Transaction, 0) for _, tx := range txs { txHash := tx.Hash() - txStatus := pool.Status([]common.Hash{txHash})[0] + txStatus := n.txPool.Status([]common.Hash{txHash})[0] if txStatus != core.TxStatusPending { continue } - if n.config.RemoteTxGossipOnlyEnabled && pool.HasLocal(txHash) { + if n.config.RemoteTxGossipOnlyEnabled && n.txPool.HasLocal(txHash) { continue } // We check [force] outside of the if statement to avoid an unnecessary // cache lookup. if !force { - if _, has := n.recentEthTxs.Get(txHash); has { + if _, has := n.recentTxs.Get(txHash); has { continue } } - n.recentEthTxs.Put(txHash, nil) + n.recentTxs.Put(txHash, nil) selectedTxs = append(selectedTxs, tx) } @@ -378,7 +315,7 @@ func (n *pushNetwork) gossipEthTxs(force bool) (int, error) { for _, tx := range selectedTxs { size := tx.Size() if msgTxsSize+size > message.EthMsgSoftCapSize { - if err := n.sendEthTxs(msgTxs); err != nil { + if err := n.sendTxs(msgTxs); err != nil { return len(selectedTxs), err } msgTxs = msgTxs[:0] @@ -389,16 +326,16 @@ func (n *pushNetwork) gossipEthTxs(force bool) (int, error) { } // Send any remaining [msgTxs] - return len(selectedTxs), n.sendEthTxs(msgTxs) + return len(selectedTxs), n.sendTxs(msgTxs) } -// GossipEthTxs enqueues the provided [txs] for gossiping. At some point, the -// [pushNetwork] will attempt to gossip the provided txs to other nodes +// GossipTxs enqueues the provided [txs] for gossiping. At some point, the +// [pushGossiper] will attempt to gossip the provided txs to other nodes // (usually right away if not under load). // // NOTE: We never return a non-nil error from this function but retain the // option to do so in case it becomes useful. -func (n *pushNetwork) GossipEthTxs(txs []*types.Transaction) error { +func (n *pushGossiper) GossipTxs(txs []*types.Transaction) error { if time.Now().Before(n.gossipActivationTime) { log.Trace( "not gossiping eth txs before the gossiping activation time", @@ -408,61 +345,35 @@ func (n *pushNetwork) GossipEthTxs(txs []*types.Transaction) error { } select { - case n.ethTxsToGossipChan <- txs: + case n.txsToGossipChan <- txs: case <-n.shutdownChan: } return nil } -func (n *pushNetwork) handle( - handler message.Handler, - handlerName string, - nodeID ids.ShortID, - requestID uint32, - msgBytes []byte, -) error { - log.Trace( - "App message handler called", - "handler", handlerName, - "peerID", nodeID, - "requestID", requestID, - "len(msg)", len(msgBytes), - ) - - if time.Now().Before(n.gossipActivationTime) { - log.Trace("App message called before activation time") - return nil - } - - msg, err := message.Parse(msgBytes) - if err != nil { - log.Trace( - "dropping App message due to failing to parse message", - "err", err, - ) - return nil - } - - return msg.Handle(handler, nodeID, requestID) -} - +// GossipHandler handles incoming gossip messages type GossipHandler struct { - message.NoopHandler + vm *VM + txPool *core.TxPool +} - vm *VM - net *pushNetwork +func NewGossipHandler(vm *VM) *GossipHandler { + return &GossipHandler{ + vm: vm, + txPool: vm.chain.GetTxPool(), + } } -func (h *GossipHandler) HandleTxs(nodeID ids.ShortID, _ uint32, msg *message.Txs) error { +func (h *GossipHandler) HandleTxs(nodeID ids.ShortID, msg *message.Txs) error { log.Trace( - "AppGossip called with EthTxs", + "AppGossip called with Txs", "peerID", nodeID, "size(txs)", len(msg.Txs), ) if len(msg.Txs) == 0 { log.Trace( - "AppGossip received empty EthTxs Message", + "AppGossip received empty Txs Message", "peerID", nodeID, ) return nil @@ -478,7 +389,7 @@ func (h *GossipHandler) HandleTxs(nodeID ids.ShortID, _ uint32, msg *message.Txs ) return nil } - errs := h.net.chain.GetTxPool().AddRemotes(txs) + errs := h.txPool.AddRemotes(txs) for i, err := range errs { if err != nil { log.Trace( @@ -491,25 +402,9 @@ func (h *GossipHandler) HandleTxs(nodeID ids.ShortID, _ uint32, msg *message.Txs return nil } -// noopNetwork should be used when gossip communication is not supported -type noopNetwork struct{} - -func (n *noopNetwork) AppRequestFailed(nodeID ids.ShortID, requestID uint32) error { - return nil -} - -func (n *noopNetwork) AppRequest(nodeID ids.ShortID, requestID uint32, deadline time.Time, msgBytes []byte) error { - return nil -} - -func (n *noopNetwork) AppResponse(nodeID ids.ShortID, requestID uint32, msgBytes []byte) error { - return nil -} - -func (n *noopNetwork) AppGossip(nodeID ids.ShortID, msgBytes []byte) error { - return nil -} +// noopGossiper should be used when gossip communication is not supported +type noopGossiper struct{} -func (n *noopNetwork) GossipEthTxs(txs []*types.Transaction) error { +func (n *noopGossiper) GossipTxs([]*types.Transaction) error { return nil } diff --git a/plugin/evm/gossiper_eth_gossiping_test.go b/plugin/evm/gossiper_eth_gossiping_test.go new file mode 100644 index 0000000000..50dad3ce53 --- /dev/null +++ b/plugin/evm/gossiper_eth_gossiping_test.go @@ -0,0 +1,375 @@ +// (c) 2019-2021, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package evm + +import ( + "crypto/ecdsa" + "encoding/json" + "math/big" + "strings" + "sync" + "testing" + "time" + + "github.com/ava-labs/avalanchego/ids" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/rlp" + + "github.com/stretchr/testify/assert" + + "github.com/ava-labs/subnet-evm/core" + "github.com/ava-labs/subnet-evm/core/types" + "github.com/ava-labs/subnet-evm/params" + "github.com/ava-labs/subnet-evm/plugin/evm/message" +) + +func fundAddressByGenesis(addrs []common.Address) (string, error) { + balance := big.NewInt(0xffffffffffffff) + genesis := &core.Genesis{ + Difficulty: common.Big0, + GasLimit: uint64(5000000), + } + funds := make(map[common.Address]core.GenesisAccount) + for _, addr := range addrs { + funds[addr] = core.GenesisAccount{ + Balance: balance, + } + } + genesis.Alloc = funds + + genesis.Config = ¶ms.ChainConfig{ + ChainID: params.TestChainConfig.ChainID, + SubnetEVMTimestamp: big.NewInt(0), + } + + bytes, err := json.Marshal(genesis) + return string(bytes), err +} + +func getValidTxs(key *ecdsa.PrivateKey, count int, gasPrice *big.Int) []*types.Transaction { + res := make([]*types.Transaction, count) + + to := common.Address{} + amount := big.NewInt(10000) + gasLimit := uint64(100000) + + for i := 0; i < count; i++ { + tx, _ := types.SignTx( + types.NewTransaction( + uint64(i), + to, + amount, + gasLimit, + gasPrice, + []byte(strings.Repeat("aaaaaaaaaa", 100))), + types.HomesteadSigner{}, key) + tx.SetFirstSeen(time.Now().Add(-1 * time.Minute)) + res[i] = tx + } + return res +} + +// show that locally issued eth txs are gossiped +// Note: channel through which subnet-evm mempool push txs to vm is injected here +// to ease up UT, which target only VM behaviors in response to subnet-evm mempool +// signals +func TestMempoolTxsAddedTxsGossipedAfterActivation(t *testing.T) { + assert := assert.New(t) + + key, err := crypto.GenerateKey() + assert.NoError(err) + + addr := crypto.PubkeyToAddress(key.PublicKey) + cfgJson, err := fundAddressByGenesis([]common.Address{addr}) + assert.NoError(err) + + _, vm, _, sender := GenesisVM(t, true, cfgJson, "", "") + defer func() { + err := vm.Shutdown() + assert.NoError(err) + }() + vm.chain.GetTxPool().SetGasPrice(common.Big1) + vm.chain.GetTxPool().SetMinFee(common.Big0) + + // create eth txes + ethTxs := getValidTxs(key, 3, common.Big1) + + var wg sync.WaitGroup + var wg2 sync.WaitGroup + wg.Add(2) + wg2.Add(1) + sender.CantSendAppGossip = false + seen := 0 + sender.SendAppGossipF = func(gossipedBytes []byte) error { + if seen == 0 { + notifyMsgIntf, err := message.ParseMessage(vm.networkCodec, gossipedBytes) + assert.NoError(err) + + requestMsg, ok := notifyMsgIntf.(*message.Txs) + assert.True(ok) + assert.NotEmpty(requestMsg.Txs) + + txs := make([]*types.Transaction, 0) + assert.NoError(rlp.DecodeBytes(requestMsg.Txs, &txs)) + assert.Len(txs, 2) + assert.ElementsMatch( + []common.Hash{ethTxs[0].Hash(), txs[1].Hash()}, + []common.Hash{txs[0].Hash(), txs[1].Hash()}, + ) + seen++ + wg2.Done() + } else if seen == 1 { + notifyMsgIntf, err := message.ParseMessage(vm.networkCodec, gossipedBytes) + assert.NoError(err) + + requestMsg, ok := notifyMsgIntf.(*message.Txs) + assert.True(ok) + assert.NotEmpty(requestMsg.Txs) + + txs := make([]*types.Transaction, 0) + assert.NoError(rlp.DecodeBytes(requestMsg.Txs, &txs)) + assert.Len(txs, 1) + assert.Equal(ethTxs[2].Hash(), txs[0].Hash()) + + seen++ + } else { + t.Fatal("should not be seen 3 times") + } + wg.Done() + return nil + } + + // Notify VM about eth txs + errs := vm.chain.GetTxPool().AddRemotesSync(ethTxs[:2]) + for _, err := range errs { + assert.NoError(err, "failed adding subnet-evm tx to mempool") + } + + // Gossip txs again (shouldn't gossip hashes) + attemptAwait(t, &wg2, 5*time.Second) // wait until reorg processed + assert.NoError(vm.gossiper.GossipTxs(ethTxs[:2])) + + errs = vm.chain.GetTxPool().AddRemotesSync(ethTxs) + assert.Contains(errs[0].Error(), "already known") + assert.Contains(errs[1].Error(), "already known") + assert.NoError(errs[2], "failed adding subnet-evm tx to mempool") + + attemptAwait(t, &wg, 5*time.Second) +} + +// show that locally issued eth txs are chunked correctly +func TestMempoolTxsAddedTxsGossipedAfterActivationChunking(t *testing.T) { + assert := assert.New(t) + + key, err := crypto.GenerateKey() + assert.NoError(err) + + addr := crypto.PubkeyToAddress(key.PublicKey) + + cfgJson, err := fundAddressByGenesis([]common.Address{addr}) + assert.NoError(err) + + _, vm, _, sender := GenesisVM(t, true, cfgJson, "", "") + defer func() { + err := vm.Shutdown() + assert.NoError(err) + }() + vm.chain.GetTxPool().SetGasPrice(common.Big1) + vm.chain.GetTxPool().SetMinFee(common.Big0) + + // create eth txes + txs := getValidTxs(key, 100, common.Big1) + + var wg sync.WaitGroup + wg.Add(2) + sender.CantSendAppGossip = false + seen := map[common.Hash]struct{}{} + sender.SendAppGossipF = func(gossipedBytes []byte) error { + notifyMsgIntf, err := message.ParseMessage(vm.networkCodec, gossipedBytes) + assert.NoError(err) + + requestMsg, ok := notifyMsgIntf.(*message.Txs) + assert.True(ok) + assert.NotEmpty(requestMsg.Txs) + + txs := make([]*types.Transaction, 0) + assert.NoError(rlp.DecodeBytes(requestMsg.Txs, &txs)) + for _, tx := range txs { + seen[tx.Hash()] = struct{}{} + } + wg.Done() + return nil + } + + // Notify VM about eth txs + errs := vm.chain.GetTxPool().AddRemotesSync(txs) + for _, err := range errs { + assert.NoError(err, "failed adding subnet-evm tx to mempool") + } + + attemptAwait(t, &wg, 5*time.Second) + + for _, tx := range txs { + _, ok := seen[tx.Hash()] + assert.True(ok, "missing hash: %v", tx.Hash()) + } +} + +// show that a geth tx discovered from gossip is requested to the same node that +// gossiped it +func TestMempoolTxsAppGossipHandling(t *testing.T) { + assert := assert.New(t) + + key, err := crypto.GenerateKey() + assert.NoError(err) + + addr := crypto.PubkeyToAddress(key.PublicKey) + + cfgJson, err := fundAddressByGenesis([]common.Address{addr}) + assert.NoError(err) + + _, vm, _, sender := GenesisVM(t, true, cfgJson, "", "") + defer func() { + err := vm.Shutdown() + assert.NoError(err) + }() + vm.chain.GetTxPool().SetGasPrice(common.Big1) + vm.chain.GetTxPool().SetMinFee(common.Big0) + + var ( + wg sync.WaitGroup + txRequested bool + ) + sender.CantSendAppGossip = false + sender.SendAppRequestF = func(_ ids.ShortSet, _ uint32, _ []byte) error { + txRequested = true + return nil + } + wg.Add(1) + sender.SendAppGossipF = func(_ []byte) error { + wg.Done() + return nil + } + + // prepare a tx + tx := getValidTxs(key, 1, common.Big1)[0] + + // show that unknown subnet-evm hashes is requested + txBytes, err := rlp.EncodeToBytes([]*types.Transaction{tx}) + assert.NoError(err) + msg := message.Txs{ + Txs: txBytes, + } + msgBytes, err := message.BuildMessage(vm.networkCodec, &msg) + assert.NoError(err) + + nodeID := ids.GenerateTestShortID() + err = vm.AppGossip(nodeID, msgBytes) + assert.NoError(err) + assert.False(txRequested, "tx should not be requested") + + // wait for transaction to be re-gossiped + attemptAwait(t, &wg, 5*time.Second) +} + +func TestMempoolTxsRegossipSingleAccount(t *testing.T) { + assert := assert.New(t) + + key, err := crypto.GenerateKey() + assert.NoError(err) + + addr := crypto.PubkeyToAddress(key.PublicKey) + + cfgJson, err := fundAddressByGenesis([]common.Address{addr}) + assert.NoError(err) + + _, vm, _, _ := GenesisVM(t, true, cfgJson, `{"local-txs-enabled":true}`, "") + defer func() { + err := vm.Shutdown() + assert.NoError(err) + }() + vm.chain.GetTxPool().SetGasPrice(common.Big1) + vm.chain.GetTxPool().SetMinFee(common.Big0) + + // create eth txes + txs := getValidTxs(key, 10, big.NewInt(226*params.GWei)) + + // Notify VM about eth txs + errs := vm.chain.GetTxPool().AddRemotesSync(txs) + for _, err := range errs { + assert.NoError(err, "failed adding subnet-evm tx to remote mempool") + } + + // Only 1 transaction will be regossiped for an address (should be lowest + // nonce) + pushNetwork := vm.gossiper.(*pushGossiper) + queued := pushNetwork.queueRegossipTxs() + assert.Len(queued, 1, "unexpected length of queued txs") + assert.Equal(txs[0].Hash(), queued[0].Hash()) +} + +func TestMempoolTxsRegossip(t *testing.T) { + assert := assert.New(t) + + keys := make([]*ecdsa.PrivateKey, 20) + addrs := make([]common.Address, 20) + for i := 0; i < 20; i++ { + key, err := crypto.GenerateKey() + assert.NoError(err) + keys[i] = key + addrs[i] = crypto.PubkeyToAddress(key.PublicKey) + } + + cfgJson, err := fundAddressByGenesis(addrs) + assert.NoError(err) + + _, vm, _, _ := GenesisVM(t, true, cfgJson, `{"local-txs-enabled":true}`, "") + defer func() { + err := vm.Shutdown() + assert.NoError(err) + }() + vm.chain.GetTxPool().SetGasPrice(common.Big1) + vm.chain.GetTxPool().SetMinFee(common.Big0) + + // create eth txes + ethTxs := make([]*types.Transaction, 20) + ethTxHashes := make([]common.Hash, 20) + for i := 0; i < 20; i++ { + txs := getValidTxs(keys[i], 1, big.NewInt(226*params.GWei)) + tx := txs[0] + ethTxs[i] = tx + ethTxHashes[i] = tx.Hash() + } + + // Notify VM about eth txs + errs := vm.chain.GetTxPool().AddRemotesSync(ethTxs[:10]) + for _, err := range errs { + assert.NoError(err, "failed adding subnet-evm tx to remote mempool") + } + errs = vm.chain.GetTxPool().AddLocals(ethTxs[10:]) + for _, err := range errs { + assert.NoError(err, "failed adding subnet-evm tx to local mempool") + } + + // We expect 15 transactions (the default max number of transactions to + // regossip) comprised of 10 local txs and 5 remote txs (we prioritize local + // txs over remote). + pushNetwork := vm.gossiper.(*pushGossiper) + queued := pushNetwork.queueRegossipTxs() + assert.Len(queued, 15, "unexpected length of queued txs") + + // Confirm queued transactions (should be ordered based on + // timestamp submitted, with local priorized over remote) + queuedTxHashes := make([]common.Hash, 15) + for i, tx := range queued { + queuedTxHashes[i] = tx.Hash() + } + assert.ElementsMatch(queuedTxHashes[:10], ethTxHashes[10:], "missing local transactions") + + // NOTE: We don't care which remote transactions are included in this test + // (due to the non-deterministic way pending transactions are surfaced, this can be difficult + // to assert as well). +} diff --git a/plugin/evm/message/codec.go b/plugin/evm/message/codec.go index 62b7848c82..64ef2e6721 100644 --- a/plugin/evm/message/codec.go +++ b/plugin/evm/message/codec.go @@ -1,4 +1,4 @@ -// (c) 2019-2021, Ava Labs, Inc. All rights reserved. +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. package message @@ -6,30 +6,22 @@ package message import ( "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/codec/linearcodec" - "github.com/ava-labs/avalanchego/codec/reflectcodec" "github.com/ava-labs/avalanchego/utils/units" "github.com/ava-labs/avalanchego/utils/wrappers" ) const ( - codecVersion uint16 = 0 - maxMessageSize = 512 * units.KiB - maxSliceLen = maxMessageSize + Version = uint16(0) + maxMessageSize = 1 * units.MiB ) -// Codec does serialization and deserialization -var c codec.Manager - -func init() { - c = codec.NewManager(maxMessageSize) - lc := linearcodec.New(reflectcodec.DefaultTagName, maxSliceLen) - +func BuildCodec() (codec.Manager, error) { + codecManager := codec.NewManager(maxMessageSize) + c := linearcodec.NewDefault() errs := wrappers.Errs{} errs.Add( - lc.RegisterType(&Txs{}), - c.RegisterCodec(codecVersion, lc), + c.RegisterType(&Txs{}), ) - if errs.Errored() { - panic(errs.Err) - } + errs.Add(codecManager.RegisterCodec(Version, c)) + return codecManager, errs.Err } diff --git a/plugin/evm/message/handler.go b/plugin/evm/message/handler.go index 89d72f6c52..39c81879a7 100644 --- a/plugin/evm/message/handler.go +++ b/plugin/evm/message/handler.go @@ -9,15 +9,33 @@ import ( "github.com/ava-labs/avalanchego/ids" ) -var _ Handler = NoopHandler{} +var _ GossipHandler = NoopMempoolGossipHandler{} -type Handler interface { - HandleTxs(nodeID ids.ShortID, requestID uint32, msg *Txs) error +// GossipHandler handles incoming gossip messages +type GossipHandler interface { + HandleTxs(nodeID ids.ShortID, msg *Txs) error } -type NoopHandler struct{} +type NoopMempoolGossipHandler struct{} -func (NoopHandler) HandleTxs(nodeID ids.ShortID, requestID uint32, _ *Txs) error { - log.Debug("dropping unexpected EthTxs message", "peerID", nodeID, "requestID", requestID) +func (NoopMempoolGossipHandler) HandleTxs(nodeID ids.ShortID, _ *Txs) error { + log.Debug("dropping unexpected Txs message", "peerID", nodeID) return nil } + +// RequestHandler interface handles incoming requests from peers +// Must have methods in format of handleType(context.Context, ids.ShortID, 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{} + +// ResponseHandler handles response for a sent request +// Only one of OnResponse or OnFailure is called for a given requestID, not both +type ResponseHandler interface { + // OnResponse is invoked when the peer responded to a request + OnResponse(nodeID ids.ShortID, requestID uint32, response []byte) error + // OnFailure is invoked when there was a failure in processing a request + // The FailureReason outlines the underlying cause. + OnFailure(nodeID ids.ShortID, requestID uint32) error +} diff --git a/plugin/evm/message/handler_test.go b/plugin/evm/message/handler_test.go index 2be3dceddc..38a9530896 100644 --- a/plugin/evm/message/handler_test.go +++ b/plugin/evm/message/handler_test.go @@ -12,11 +12,11 @@ import ( ) type CounterHandler struct { - EthTxs int + Txs int } -func (h *CounterHandler) HandleTxs(ids.ShortID, uint32, *Txs) error { - h.EthTxs++ +func (h *CounterHandler) HandleTxs(ids.ShortID, *Txs) error { + h.Txs++ return nil } @@ -26,16 +26,16 @@ func TestHandleTxs(t *testing.T) { handler := CounterHandler{} msg := Txs{} - err := msg.Handle(&handler, ids.ShortEmpty, 0) + err := msg.Handle(&handler, ids.ShortEmpty) assert.NoError(err) - assert.Equal(1, handler.EthTxs) + assert.Equal(1, handler.Txs) } func TestNoopHandler(t *testing.T) { assert := assert.New(t) - handler := NoopHandler{} + handler := NoopMempoolGossipHandler{} - err := handler.HandleTxs(ids.ShortEmpty, 0, nil) + err := handler.HandleTxs(ids.ShortEmpty, nil) assert.NoError(err) } diff --git a/plugin/evm/message/message.go b/plugin/evm/message/message.go index c4e18b5251..0248db646c 100644 --- a/plugin/evm/message/message.go +++ b/plugin/evm/message/message.go @@ -6,6 +6,8 @@ package message import ( "errors" + "github.com/ava-labs/avalanchego/codec" + "github.com/ethereum/go-ethereum/common" "github.com/ava-labs/avalanchego/ids" @@ -14,7 +16,7 @@ import ( const ( // EthMsgSoftCapSize is the ideal size of encoded transaction bytes we send in - // any [EthTxs] message. We do not limit inbound messages to + // any [Txs] message. We do not limit inbound messages to // this size, however. Max inbound message size is enforced by the codec // (512KB). EthMsgSoftCapSize = common.StorageSize(64 * units.KiB) @@ -28,7 +30,7 @@ var ( type Message interface { // Handle this message with the correct message handler - Handle(handler Handler, nodeID ids.ShortID, requestID uint32) error + Handle(handler GossipHandler, nodeID ids.ShortID) error // initialize should be called whenever a message is built or parsed initialize([]byte) @@ -50,25 +52,25 @@ type Txs struct { Txs []byte `serialize:"true"` } -func (msg *Txs) Handle(handler Handler, nodeID ids.ShortID, requestID uint32) error { - return handler.HandleTxs(nodeID, requestID, msg) +func (msg *Txs) Handle(handler GossipHandler, nodeID ids.ShortID) error { + return handler.HandleTxs(nodeID, msg) } -func Parse(bytes []byte) (Message, error) { +func ParseMessage(codec codec.Manager, bytes []byte) (Message, error) { var msg Message - version, err := c.Unmarshal(bytes, &msg) + version, err := codec.Unmarshal(bytes, &msg) if err != nil { return nil, err } - if version != codecVersion { + if version != Version { return nil, errUnexpectedCodecVersion } msg.initialize(bytes) return msg, nil } -func Build(msg Message) ([]byte, error) { - bytes, err := c.Marshal(codecVersion, &msg) +func BuildMessage(codec codec.Manager, msg Message) ([]byte, error) { + bytes, err := codec.Marshal(Version, &msg) msg.initialize(bytes) return bytes, err } diff --git a/plugin/evm/message/message_test.go b/plugin/evm/message/message_test.go index b1273872b7..56c6d0d2f7 100644 --- a/plugin/evm/message/message_test.go +++ b/plugin/evm/message/message_test.go @@ -12,18 +12,20 @@ import ( "github.com/stretchr/testify/assert" ) -func TestEthTxs(t *testing.T) { +func TestTxs(t *testing.T) { assert := assert.New(t) msg := []byte("blah") builtMsg := Txs{ Txs: msg, } - builtMsgBytes, err := Build(&builtMsg) + codec, err := BuildCodec() + assert.NoError(err) + builtMsgBytes, err := BuildMessage(codec, &builtMsg) assert.NoError(err) assert.Equal(builtMsgBytes, builtMsg.Bytes()) - parsedMsgIntf, err := Parse(builtMsgBytes) + parsedMsgIntf, err := ParseMessage(codec, builtMsgBytes) assert.NoError(err) assert.Equal(builtMsgBytes, parsedMsgIntf.Bytes()) @@ -33,20 +35,24 @@ func TestEthTxs(t *testing.T) { assert.Equal(msg, parsedMsg.Txs) } -func TestEthTxsTooLarge(t *testing.T) { +func TestTxsTooLarge(t *testing.T) { assert := assert.New(t) builtMsg := Txs{ Txs: utils.RandomBytes(1024 * units.KiB), } - _, err := Build(&builtMsg) + codec, err := BuildCodec() + assert.NoError(err) + _, err = BuildMessage(codec, &builtMsg) assert.Error(err) } func TestParseGibberish(t *testing.T) { assert := assert.New(t) + codec, err := BuildCodec() + assert.NoError(err) randomBytes := utils.RandomBytes(256 * units.KiB) - _, err := Parse(randomBytes) + _, err = ParseMessage(codec, randomBytes) assert.Error(err) } diff --git a/plugin/evm/message/request.go b/plugin/evm/message/request.go new file mode 100644 index 0000000000..d931284b28 --- /dev/null +++ b/plugin/evm/message/request.go @@ -0,0 +1,36 @@ +// (c) 2019-2022, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package message + +import ( + "context" + + "github.com/ava-labs/avalanchego/codec" + + "github.com/ava-labs/avalanchego/ids" +) + +// Request represents a Network request type +type Request interface { + // Handle allows `Request` to call respective methods on handler to handle + // this particular request type + Handle(ctx context.Context, nodeID ids.ShortID, requestID uint32, handler RequestHandler) ([]byte, error) + + // Type returns user-friendly name for this object that can be used for logging + Type() string +} + +// BytesToRequest unmarshals the given requestBytes into Request object +func BytesToRequest(codec codec.Manager, requestBytes []byte) (Request, error) { + var request Request + if _, err := codec.Unmarshal(requestBytes, &request); err != nil { + return nil, err + } + return request, nil +} + +// RequestToBytes marshals the given request object into bytes +func RequestToBytes(codec codec.Manager, request Request) ([]byte, error) { + return codec.Marshal(Version, &request) +} diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 3b72732c37..79af4ea290 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -4,6 +4,7 @@ package evm import ( + "encoding/binary" "encoding/json" "errors" "fmt" @@ -22,6 +23,8 @@ import ( "github.com/ava-labs/subnet-evm/metrics/prometheus" "github.com/ava-labs/subnet-evm/node" "github.com/ava-labs/subnet-evm/params" + "github.com/ava-labs/subnet-evm/peer" + "github.com/ava-labs/subnet-evm/plugin/evm/message" // Force-load tracer engine to trigger registration // @@ -38,6 +41,7 @@ import ( avalancheRPC "github.com/gorilla/rpc/v2" + "github.com/ava-labs/avalanchego/codec" "github.com/ava-labs/avalanchego/database" "github.com/ava-labs/avalanchego/database/manager" "github.com/ava-labs/avalanchego/database/prefixdb" @@ -48,9 +52,9 @@ import ( "github.com/ava-labs/avalanchego/snow/consensus/snowman" "github.com/ava-labs/avalanchego/snow/engine/snowman/block" cjson "github.com/ava-labs/avalanchego/utils/json" + "github.com/ava-labs/avalanchego/utils/perms" "github.com/ava-labs/avalanchego/utils/profiler" "github.com/ava-labs/avalanchego/utils/timer/mockable" - "github.com/ava-labs/avalanchego/version" "github.com/ava-labs/avalanchego/vms/components/chain" commonEng "github.com/ava-labs/avalanchego/snow/engine/common" @@ -58,7 +62,10 @@ import ( avalancheJSON "github.com/ava-labs/avalanchego/utils/json" ) -var _ block.ChainVM = &VM{} +var ( + _ block.ChainVM = &VM{} + _ block.HeightIndexedChainVM = &VM{} +) const ( // Max time from current time allowed for blocks, before they're considered future blocks @@ -79,9 +86,10 @@ const ( var ( // Set last accepted key to be longer than the keys used to store accepted block IDs. - lastAcceptedKey = []byte("last_accepted_key") - acceptedPrefix = []byte("snowman_accepted") - ethDBPrefix = []byte("ethdb") + lastAcceptedKey = []byte("last_accepted_key") + acceptedPrefix = []byte("snowman_accepted") + ethDBPrefix = []byte("ethdb") + pruneRejectedBlocksKey = []byte("pruned_rejected_blocks") ) var ( @@ -92,6 +100,7 @@ var ( errUnclesUnsupported = errors.New("uncles unsupported") errTxHashMismatch = errors.New("txs hash does not match header") errUncleHashMismatch = errors.New("uncle hash mismatch") + errRejectedParent = errors.New("rejected parent") errInvalidDifficulty = errors.New("invalid difficulty") errInvalidMixDigest = errors.New("invalid mix digest") errHeaderExtraDataTooBig = errors.New("header extra data too big") @@ -132,7 +141,7 @@ type VM struct { builder *blockBuilder - network Network + gossiper Gossiper clock mockable.Clock @@ -142,15 +151,11 @@ type VM struct { // Continuous Profiler profiler profiler.ContinuousProfiler - bootstrapped bool -} + peer.Network + client peer.Client + networkCodec codec.Manager -func (vm *VM) Connected(id ids.ShortID, nodeVersion version.Application) error { - return nil // noop -} - -func (vm *VM) Disconnected(nodeID ids.ShortID) error { - return nil // noop + bootstrapped bool } // setLogLevel sets the log level with the original [os.StdErr] interface along @@ -250,6 +255,8 @@ func (vm *VM) Initialize( ethConfig.Pruning = vm.config.Pruning ethConfig.SnapshotAsync = vm.config.SnapshotAsync ethConfig.SnapshotVerify = vm.config.SnapshotVerify + + // Handle custom fee recipient ethConfig.Miner.Etherbase = schain.BlackholeAddr switch { case common.IsHexAddress(vm.config.FeeRecipient): @@ -264,6 +271,17 @@ func (vm *VM) Initialize( log.Warn("Chain enabled `AllowFeeRecipients`, but chain config has not specified any coinbase address. Defaulting to the blackhole address.") } + // Handle offline pruning + ethConfig.OfflinePruning = vm.config.OfflinePruning + ethConfig.OfflinePruningBloomFilterSize = vm.config.OfflinePruningBloomFilterSize + ethConfig.OfflinePruningDataDirectory = vm.config.OfflinePruningDataDirectory + if len(ethConfig.OfflinePruningDataDirectory) != 0 { + if err := os.MkdirAll(ethConfig.OfflinePruningDataDirectory, perms.ReadWriteExecute); err != nil { + log.Error("failed to create offline pruning data directory", "error", err) + return err + } + } + vm.chainConfig = g.Config vm.networkID = ethConfig.NetworkId @@ -299,8 +317,15 @@ func (vm *VM) Initialize( // start goroutines to update the tx pool gas minimum gas price when upgrades go into effect vm.handleGasPriceUpdates() - // initialize new gossip network - vm.network = vm.NewNetwork(appSender) + vm.networkCodec, err = message.BuildCodec() + if err != nil { + return err + } + + // initialize peer network + vm.Network = peer.NewNetwork(appSender, vm.networkCodec, ctx.NodeID, vm.config.MaxOutboundActiveRequests) + vm.client = peer.NewClient(vm.Network) + vm.initGossipHandling() // start goroutines to manage block building // @@ -323,7 +348,7 @@ func (vm *VM) Initialize( vm: vm, status: choices.Accepted, }, - GetBlockIDAtHeight: vm.getBlockIDAtHeight, + GetBlockIDAtHeight: vm.GetBlockIDAtHeight, GetBlock: vm.getBlock, UnmarshalBlock: vm.parseBlock, BuildBlock: vm.buildBlock, @@ -343,18 +368,51 @@ func (vm *VM) Initialize( return nil } -// Bootstrapping notifies this VM that the consensus engine is performing -// bootstrapping -func (vm *VM) Bootstrapping() error { - vm.bootstrapped = false - return nil +func (vm *VM) initGossipHandling() { + if vm.chainConfig.SubnetEVMTimestamp != nil { + vm.gossiper = vm.newPushGossiper() + vm.Network.SetGossipHandler(NewGossipHandler(vm)) + } else { + vm.gossiper = &noopGossiper{} + vm.Network.SetGossipHandler(message.NoopMempoolGossipHandler{}) + } } -// Bootstrapped notifies this VM that the consensus engine has finished -// bootstrapping -func (vm *VM) Bootstrapped() error { - vm.bootstrapped = true - return nil +func (vm *VM) pruneChain() error { + if !vm.config.Pruning { + return nil + } + pruned, err := vm.db.Has(pruneRejectedBlocksKey) + if err != nil { + return fmt.Errorf("failed to check if the VM has pruned rejected blocks: %w", err) + } + if pruned { + return nil + } + + lastAcceptedHeight := vm.LastAcceptedBlock().Height() + if err := vm.chain.RemoveRejectedBlocks(0, lastAcceptedHeight); err != nil { + return err + } + heightBytes := make([]byte, 8) + binary.PutUvarint(heightBytes, lastAcceptedHeight) + if err := vm.db.Put(pruneRejectedBlocksKey, heightBytes); err != nil { + return err + } + return vm.db.Commit() +} + +func (vm *VM) SetState(state snow.State) error { + switch state { + case snow.Bootstrapping: + vm.bootstrapped = false + return nil + case snow.NormalOp: + vm.bootstrapped = true + return nil + default: + return snow.ErrUnknownState + } } // Shutdown implements the snowman.ChainVM interface @@ -458,10 +516,15 @@ func (vm *VM) SetPreference(blkID ids.ID) error { return vm.chain.SetPreference(block.(*Block).ethBlock) } -// getBlockIDAtHeight retrieves the blkID of the canonical block at [blkHeight] +func (vm *VM) VerifyHeightIndex() error { + // our index is vm.chain.GetBlockByNumber + return nil +} + +// GetBlockIDAtHeight retrieves the blkID of the canonical block at [blkHeight] // if [blkHeight] is less than the height of the last accepted block, this will return // a canonical block. Otherwise, it may return a blkID that has not yet been accepted. -func (vm *VM) getBlockIDAtHeight(blkHeight uint64) (ids.ID, error) { +func (vm *VM) GetBlockIDAtHeight(blkHeight uint64) (ids.ID, error) { ethBlock := vm.chain.GetBlockByNumber(blkHeight) if ethBlock == nil { return ids.ID{}, fmt.Errorf("could not find block at height: %d", blkHeight) @@ -565,6 +628,39 @@ func (vm *VM) CreateStaticHandlers() (map[string]*commonEng.HTTPHandler, error) ****************************************************************************** */ +// 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 (vm *VM) conflicts(inputs ids.Set, ancestor *Block) error { + for ancestor.Status() != choices.Accepted { + // 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. + nextAncestorIntf, err := vm.GetBlockInternal(nextAncestorID) + if err != nil { + return errRejectedParent + } + + if blkStatus := nextAncestorIntf.Status(); blkStatus == choices.Unknown || blkStatus == choices.Rejected { + return errRejectedParent + } + nextAncestor, ok := nextAncestorIntf.(*Block) + if !ok { + return fmt.Errorf("ancestor block %s had unexpected type %T", nextAncestor.ID(), nextAncestorIntf) + } + ancestor = nextAncestor + } + + return nil +} + // GetCurrentNonce returns the nonce associated with the address at the // preferred block func (vm *VM) GetCurrentNonce(address common.Address) (uint64, error) { diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index efbfa4f6a7..13876b4b41 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -79,19 +79,30 @@ func init() { genesisBalance = new(big.Int).Mul(big.NewInt(testMinGasPrice), big.NewInt(21000*1000)) } -// BuildGenesisTest returns the genesis bytes for Subnet EVM VM to be used in testing -func BuildGenesisTestWithAlloc(t *testing.T, genesisJSON string) []byte { - ss := CreateStaticService() - +func fundGenesis(genesisJSON string) (string, error) { genesis := &core.Genesis{} if err := json.Unmarshal([]byte(genesisJSON), genesis); err != nil { - t.Fatalf("Problem unmarshaling genesis JSON: %s", err) + return "", fmt.Errorf("Problem unmarshaling genesis JSON: %s", err) } + // add test allocs genesis.Alloc = core.GenesisAlloc{ testEthAddrs[0]: core.GenesisAccount{Balance: genesisBalance}, testEthAddrs[1]: core.GenesisAccount{Balance: genesisBalance}, } + + bytes, err := json.Marshal(genesis) + return string(bytes), err +} + +// BuildGenesisTest returns the genesis bytes for Subnet EVM VM to be used in testing +func buildGenesisTest(t *testing.T, genesisJSON string) []byte { + ss := CreateStaticService() + + genesis := &core.Genesis{} + if err := json.Unmarshal([]byte(genesisJSON), genesis); err != nil { + t.Fatalf("Problem unmarshaling genesis JSON: %s", err) + } args := &BuildGenesisArgs{GenesisData: genesis} reply := &BuildGenesisReply{} err := ss.BuildGenesis(nil, args, reply) @@ -144,7 +155,7 @@ func setupGenesis(t *testing.T, manager.Manager, []byte, chan engCommon.Message) { - genesisBytes := BuildGenesisTestWithAlloc(t, genesisJSON) + genesisBytes := buildGenesisTest(t, genesisJSON) ctx := NewContext() baseDBManager := manager.NewMemDB(version.NewDefaultVersion(1, 4, 5)) @@ -193,8 +204,8 @@ func GenesisVM(t *testing.T, } if finishBootstrapping { - assert.NoError(t, vm.Bootstrapping()) - assert.NoError(t, vm.Bootstrapped()) + assert.NoError(t, vm.SetState(snow.Bootstrapping)) + assert.NoError(t, vm.SetState(snow.NormalOp)) } return issuer, vm, dbManager, appSender @@ -324,7 +335,9 @@ func TestVMUpgrades(t *testing.T) { func TestBuildEthTxBlock(t *testing.T) { // reduce block gas cost - issuer, vm, dbManager, _ := GenesisVM(t, true, genesisJSONSubnetEVM, "{\"pruning-enabled\":true}", "") + genesisString, err := fundGenesis(genesisJSONSubnetEVM) + assert.NoError(t, err) + issuer, vm, dbManager, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -463,7 +476,7 @@ func TestBuildEthTxBlock(t *testing.T) { } restartedVM := &VM{} - genesisBytes := BuildGenesisTestWithAlloc(t, genesisJSONSubnetEVM) + genesisBytes := buildGenesisTest(t, genesisString) if err := restartedVM.Initialize( NewContext(), @@ -502,8 +515,10 @@ func TestBuildEthTxBlock(t *testing.T) { func TestSetPreferenceRace(t *testing.T) { // Create two VMs which will agree on block A and then // build the two distinct preferred chains above - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":true}", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":true}", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -757,8 +772,10 @@ func TestSetPreferenceRace(t *testing.T) { // accept block C, which should be an orphaned block at this point and // get rejected. func TestReorgProtection(t *testing.T) { - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":false}", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":false}", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":false}", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":false}", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -939,8 +956,10 @@ func TestReorgProtection(t *testing.T) { // / \ // B C func TestNonCanonicalAccept(t *testing.T) { - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1114,8 +1133,10 @@ func TestNonCanonicalAccept(t *testing.T) { // | // D func TestStickyPreference(t *testing.T) { - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1388,8 +1409,10 @@ func TestStickyPreference(t *testing.T) { // | // D func TestUncleBlock(t *testing.T) { - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1585,7 +1608,9 @@ func TestUncleBlock(t *testing.T) { // Regression test to ensure that a VM that is not able to parse a block that // contains no transactions. func TestEmptyBlock(t *testing.T) { - issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -1651,8 +1676,10 @@ func TestEmptyBlock(t *testing.T) { // | // D func TestAcceptReorg(t *testing.T) { - issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1863,7 +1890,9 @@ func TestAcceptReorg(t *testing.T) { } func TestFutureBlock(t *testing.T) { - issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -1926,7 +1955,9 @@ func TestFutureBlock(t *testing.T) { } func TestLastAcceptedBlockNumberAllow(t *testing.T) { - issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + genesisString, err := fundGenesis(genesisJSONMuirGlacier) + assert.NoError(t, err) + issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -2070,7 +2101,9 @@ func TestConfigureLogLevel(t *testing.T) { // Regression test to ensure we can build blocks if we are starting with the // Subnet EVM ruleset in genesis. func TestBuildSubnetEVMBlock(t *testing.T) { - issuer, vm, _, _ := GenesisVM(t, true, genesisJSONSubnetEVM, "", "") + genesisString, err := fundGenesis(genesisJSONSubnetEVM) + assert.NoError(t, err) + issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") defer func() { if err := vm.Shutdown(); err != nil { diff --git a/rpc/handler.go b/rpc/handler.go index 539132cb24..82d9d66450 100644 --- a/rpc/handler.go +++ b/rpc/handler.go @@ -36,6 +36,7 @@ import ( "time" "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" "golang.org/x/time/rate" ) @@ -443,7 +444,9 @@ func (h *handler) handleCall(cp *callProc, msg *jsonrpcMessage) *jsonrpcMessage successfulRequestGauge.Inc(1) } rpcServingTimer.UpdateSince(start) - newRPCServingTimer(msg.Method, answer.Error == nil).UpdateSince(start) + if metrics.EnabledExpensive { + newRPCServingTimer(msg.Method, answer.Error == nil).UpdateSince(start) + } } return answer } diff --git a/scripts/lint.sh b/scripts/lint.sh old mode 100644 new mode 100755 From 2487ed761b9c3d2efd1e0da5e414c9c3804c6d97 Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Thu, 10 Feb 2022 14:11:19 +0300 Subject: [PATCH 2/8] bump to avalanchego 1.7.5 --- go.mod | 4 +--- go.sum | 4 +++- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index c52d212c64..e7ad29e289 100644 --- a/go.mod +++ b/go.mod @@ -4,8 +4,7 @@ go 1.16 require ( github.com/VictoriaMetrics/fastcache v1.6.0 - github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377 - github.com/btcsuite/btcd v0.21.0-beta // indirect + github.com/ava-labs/avalanchego v1.7.5 github.com/cespare/cp v0.1.0 github.com/davecgh/go-spew v1.1.1 github.com/deckarep/golang-set v1.7.1 @@ -25,7 +24,6 @@ require ( github.com/olekukonko/tablewriter v0.0.5 github.com/prometheus/client_golang v1.7.1 github.com/prometheus/client_model v0.2.0 - github.com/prometheus/tsdb v0.10.0 // indirect github.com/rjeczalik/notify v0.9.2 github.com/spf13/cast v1.3.1 github.com/spf13/pflag v1.0.5 diff --git a/go.sum b/go.sum index 771133b823..1d409ed747 100644 --- a/go.sum +++ b/go.sum @@ -57,8 +57,10 @@ github.com/apache/arrow/go/arrow v0.0.0-20191024131854-af6fa24be0db/go.mod h1:VT github.com/armon/circbuf v0.0.0-20150827004946-bbbad097214e/go.mod h1:3U/XgcO3hCbHZ8TKRvWD2dDTCfh9M9ya+I9JpbB7O8o= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/armon/go-radix v0.0.0-20180808171621-7fddfc383310/go.mod h1:ufUuZ+zHj4x4TnLV4JWEpy2hxWSpsRywHrMgIH9cCH8= -github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377 h1:suLHw7lxX8m8UfmZxatQQYEgkHUVWrXjN96GLdpATFg= github.com/ava-labs/avalanchego v1.7.5-0.20220202014036-7c45dd1e2377/go.mod h1:sUhn77bV5EBGXwqcFz1FzFs94MR6SRgrU+Ob7lb+pC4= +github.com/ava-labs/avalanchego v1.7.5 h1:iY6PawL/V3h3UzdNYjYOyevxBDTeXu+7fPBGfSch1rw= +github.com/ava-labs/avalanchego v1.7.5/go.mod h1:Uo9V940al0S2JFYjAxpJ9IF6Yik6BITPxdZ68RY0+cQ= +github.com/ava-labs/coreth v0.8.5-rc.2/go.mod h1:uO87Zjp3bFkBvPrnvaq0huiz9diGbO2viTTB2PxzEV4= github.com/aws/aws-sdk-go-v2 v1.2.0/go.mod h1:zEQs02YRBw1DjK0PoJv3ygDYOFTre1ejlJWl8FwAuQo= github.com/aws/aws-sdk-go-v2/config v1.1.1/go.mod h1:0XsVy9lBI/BCXm+2Tuvt39YmdHwS5unDQmxZOYe8F5Y= github.com/aws/aws-sdk-go-v2/credentials v1.1.1/go.mod h1:mM2iIjwl7LULWtS6JCACyInboHirisUUdkBPoTHMOUo= From b8b9daedaedf604629c61ca4d72b86e254b715ba Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Tue, 15 Feb 2022 16:13:57 +0300 Subject: [PATCH 3/8] fix nits --- core/test_blockchain.go | 80 +++++++++---------- plugin/evm/block_builder.go | 25 +++--- plugin/evm/block_builder_test.go | 2 +- plugin/evm/gossiper.go | 2 +- plugin/evm/message/message.go | 4 +- ...gossiping_test.go => tx_gossiping_test.go} | 0 plugin/evm/vm.go | 65 +-------------- plugin/evm/vm_test.go | 78 +++++------------- 8 files changed, 79 insertions(+), 177 deletions(-) rename plugin/evm/{gossiper_eth_gossiping_test.go => tx_gossiping_test.go} (100%) diff --git a/core/test_blockchain.go b/core/test_blockchain.go index aae6fa322c..9c9156703a 100644 --- a/core/test_blockchain.go +++ b/core/test_blockchain.go @@ -28,50 +28,50 @@ type ChainTest struct { } var tests = []ChainTest{ - // { - // "InsertChainAcceptSingleBlock", - // TestInsertChainAcceptSingleBlock, - // }, - // { - // "InsertForkedChain", - // TestInsertLongForkedChain, - // }, - // { - // "AcceptNonCanonicalBlock", - // TestAcceptNonCanonicalBlock, - // }, - // { - // "SetPreferenceRewind", - // TestSetPreferenceRewind, - // }, - // { - // "BuildOnVariousStages", - // TestBuildOnVariousStages, - // }, + { + "InsertChainAcceptSingleBlock", + TestInsertChainAcceptSingleBlock, + }, + { + "InsertForkedChain", + TestInsertLongForkedChain, + }, + { + "AcceptNonCanonicalBlock", + TestAcceptNonCanonicalBlock, + }, + { + "SetPreferenceRewind", + TestSetPreferenceRewind, + }, + { + "BuildOnVariousStages", + TestBuildOnVariousStages, + }, { "EmptyBlocks", TestEmptyBlocks, }, - // { - // "AcceptBlockIdenticalStateRoot", - // TestAcceptBlockIdenticalStateRoot, - // }, - // { - // "ReprocessAcceptBlockIdenticalStateRoot", - // TestReprocessAcceptBlockIdenticalStateRoot, - // }, - // { - // "GenerateChainInvalidBlockFee", - // TestGenerateChainInvalidBlockFee, - // }, - // { - // "InsertChainInvalidBlockFee", - // TestInsertChainInvalidBlockFee, - // }, - // { - // "InsertChainValidBlockFee", - // TestInsertChainValidBlockFee, - // }, + { + "AcceptBlockIdenticalStateRoot", + TestAcceptBlockIdenticalStateRoot, + }, + { + "ReprocessAcceptBlockIdenticalStateRoot", + TestReprocessAcceptBlockIdenticalStateRoot, + }, + { + "GenerateChainInvalidBlockFee", + TestGenerateChainInvalidBlockFee, + }, + { + "InsertChainInvalidBlockFee", + TestInsertChainInvalidBlockFee, + }, + { + "InsertChainValidBlockFee", + TestInsertChainValidBlockFee, + }, } func copyMemDB(db ethdb.Database) (ethdb.Database, error) { diff --git a/plugin/evm/block_builder.go b/plugin/evm/block_builder.go index d6d8259f2e..11d92045a0 100644 --- a/plugin/evm/block_builder.go +++ b/plugin/evm/block_builder.go @@ -21,12 +21,11 @@ import ( type buildingBlkStatus uint8 var ( - // Subnet EVM Params + // Pre-Subnet EVM Params minBlockTime = 2 * time.Second maxBlockTime = 3 * time.Second - - // SC Params - minBlockTimeSC = 500 * time.Millisecond + // Subnet EVM Params + minBlockTimeSE = 500 * time.Millisecond ) const ( @@ -43,7 +42,7 @@ const ( waitBlockTime = 100 * time.Millisecond dontBuild buildingBlkStatus = iota - conditionalBuild // Only used prior to SC + conditionalBuild // Only used prior to SubnetEVM mayBuild building ) @@ -77,7 +76,7 @@ type blockBuilder struct { // [building] indicates the VM has sent a request to the engine to build a block. buildStatus buildingBlkStatus - // isSC is a boolean indicating if SubnetEVM is activated. This prevents us from + // isSE is a boolean indicating if SubnetEVM is activated. This prevents us from // getting the current time and comparing it to the *params.chainConfig more // than once. isSE bool @@ -105,16 +104,16 @@ func (b *blockBuilder) handleBlockBuilding() { if !b.chainConfig.IsSubnetEVM(big.NewInt(time.Now().Unix())) { b.shutdownWg.Add(1) - go b.ctx.Log.RecoverAndPanic(b.migrateSC) + go b.ctx.Log.RecoverAndPanic(b.migrateSE) } else { b.isSE = true } } -func (b *blockBuilder) migrateSC() { +func (b *blockBuilder) migrateSE() { defer b.shutdownWg.Done() - // In some tests, the SC timestamp is not populated. If this is the case, we + // In some tests, the SubnetEVM timestamp is not populated. If this is the case, we // should only stop [buildBlockTwoStageTimer] on shutdown. if b.chainConfig.SubnetEVMTimestamp == nil { <-b.shutdownChan @@ -162,13 +161,13 @@ func (b *blockBuilder) handleGenerateBlock() { } } else { // If we still need to build a block immediately after building, we let the - // engine know it [mayBuild] in [minBlockTimeSC]. + // engine know it [mayBuild] in [minBlockTimeSE]. // - // It is often the case in SC that a block (with the same txs) could be built + // It is often the case in SubnetEVM that a block (with the same txs) could be built // after a few seconds of delay as the [baseFee] and/or [blockGasCost] decrease. if b.needToBuild() { b.buildStatus = mayBuild - b.buildBlockTimer.SetTimeoutIn(minBlockTimeSC) + b.buildBlockTimer.SetTimeoutIn(minBlockTimeSE) } else { b.buildStatus = dontBuild } @@ -185,7 +184,7 @@ func (b *blockBuilder) needToBuild() bool { // buildEarly returns true if there are sufficient outstanding transactions to // be issued into a block to build a block early. // -// NOTE: Only used prior to SC. +// NOTE: Only used prior to SubnetEVM. func (b *blockBuilder) buildEarly() bool { size := b.chain.PendingSize() return size > batchSize diff --git a/plugin/evm/block_builder_test.go b/plugin/evm/block_builder_test.go index e61455c078..9663dbccaf 100644 --- a/plugin/evm/block_builder_test.go +++ b/plugin/evm/block_builder_test.go @@ -84,6 +84,6 @@ func TestBlockBuilderStopsTimer(t *testing.T) { t.Fatalf("expected build status to be %d but got %d", dontBuild, builder.buildStatus) } if !builder.isSE { - t.Fatal("expected isSC to be true") + t.Fatal("expected isSE to be true") } } diff --git a/plugin/evm/gossiper.go b/plugin/evm/gossiper.go index 10709155e4..0f938d2272 100644 --- a/plugin/evm/gossiper.go +++ b/plugin/evm/gossiper.go @@ -314,7 +314,7 @@ func (n *pushGossiper) gossipTxs(force bool) (int, error) { msgTxsSize := common.StorageSize(0) for _, tx := range selectedTxs { size := tx.Size() - if msgTxsSize+size > message.EthMsgSoftCapSize { + if msgTxsSize+size > message.TxMsgSoftCapSize { if err := n.sendTxs(msgTxs); err != nil { return len(selectedTxs), err } diff --git a/plugin/evm/message/message.go b/plugin/evm/message/message.go index 0248db646c..43fa22acc5 100644 --- a/plugin/evm/message/message.go +++ b/plugin/evm/message/message.go @@ -15,11 +15,11 @@ import ( ) const ( - // EthMsgSoftCapSize is the ideal size of encoded transaction bytes we send in + // TxMsgSoftCapSize is the ideal size of encoded transaction bytes we send in // any [Txs] message. We do not limit inbound messages to // this size, however. Max inbound message size is enforced by the codec // (512KB). - EthMsgSoftCapSize = common.StorageSize(64 * units.KiB) + TxMsgSoftCapSize = common.StorageSize(64 * units.KiB) ) var ( diff --git a/plugin/evm/gossiper_eth_gossiping_test.go b/plugin/evm/tx_gossiping_test.go similarity index 100% rename from plugin/evm/gossiper_eth_gossiping_test.go rename to plugin/evm/tx_gossiping_test.go diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index 79af4ea290..c0b407328c 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -4,7 +4,6 @@ package evm import ( - "encoding/binary" "encoding/json" "errors" "fmt" @@ -86,10 +85,9 @@ const ( var ( // Set last accepted key to be longer than the keys used to store accepted block IDs. - lastAcceptedKey = []byte("last_accepted_key") - acceptedPrefix = []byte("snowman_accepted") - ethDBPrefix = []byte("ethdb") - pruneRejectedBlocksKey = []byte("pruned_rejected_blocks") + lastAcceptedKey = []byte("last_accepted_key") + acceptedPrefix = []byte("snowman_accepted") + ethDBPrefix = []byte("ethdb") ) var ( @@ -378,30 +376,6 @@ func (vm *VM) initGossipHandling() { } } -func (vm *VM) pruneChain() error { - if !vm.config.Pruning { - return nil - } - pruned, err := vm.db.Has(pruneRejectedBlocksKey) - if err != nil { - return fmt.Errorf("failed to check if the VM has pruned rejected blocks: %w", err) - } - if pruned { - return nil - } - - lastAcceptedHeight := vm.LastAcceptedBlock().Height() - if err := vm.chain.RemoveRejectedBlocks(0, lastAcceptedHeight); err != nil { - return err - } - heightBytes := make([]byte, 8) - binary.PutUvarint(heightBytes, lastAcceptedHeight) - if err := vm.db.Put(pruneRejectedBlocksKey, heightBytes); err != nil { - return err - } - return vm.db.Commit() -} - func (vm *VM) SetState(state snow.State) error { switch state { case snow.Bootstrapping: @@ -628,39 +602,6 @@ func (vm *VM) CreateStaticHandlers() (map[string]*commonEng.HTTPHandler, error) ****************************************************************************** */ -// 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 (vm *VM) conflicts(inputs ids.Set, ancestor *Block) error { - for ancestor.Status() != choices.Accepted { - // 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. - nextAncestorIntf, err := vm.GetBlockInternal(nextAncestorID) - if err != nil { - return errRejectedParent - } - - if blkStatus := nextAncestorIntf.Status(); blkStatus == choices.Unknown || blkStatus == choices.Rejected { - return errRejectedParent - } - nextAncestor, ok := nextAncestorIntf.(*Block) - if !ok { - return fmt.Errorf("ancestor block %s had unexpected type %T", nextAncestor.ID(), nextAncestorIntf) - } - ancestor = nextAncestor - } - - return nil -} - // GetCurrentNonce returns the nonce associated with the address at the // preferred block func (vm *VM) GetCurrentNonce(address common.Address) (uint64, error) { diff --git a/plugin/evm/vm_test.go b/plugin/evm/vm_test.go index 13876b4b41..84c9f2f1c0 100644 --- a/plugin/evm/vm_test.go +++ b/plugin/evm/vm_test.go @@ -59,8 +59,8 @@ var ( password = "CjasdjhiPeirbSenfeI13" // #nosec G101 // Use chainId: 43111, so that it does not overlap with any Avalanche ChainIDs, which may have their // config overridden in vm.Initialize. - genesisJSONMuirGlacier = "{\"config\":{\"chainId\":43111,\"homesteadBlock\":0,\"eip150Block\":0,\"eip150Hash\":\"0x2086799aeebeae135c246c65021c82b4e15a2c451340993aacfd2751886514f0\",\"eip155Block\":0,\"eip158Block\":0,\"byzantiumBlock\":0,\"constantinopleBlock\":0,\"petersburgBlock\":0,\"istanbulBlock\":0,\"muirGlacierBlock\":0},\"nonce\":\"0x0\",\"timestamp\":\"0x0\",\"extraData\":\"0x00\",\"gasLimit\":\"0x5f5e100\",\"difficulty\":\"0x0\",\"mixHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\",\"coinbase\":\"0x0000000000000000000000000000000000000000\",\"alloc\":{\"0100000000000000000000000000000000000000\":{\"code\":\"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033\",\"balance\":\"0x0\"}},\"number\":\"0x0\",\"gasUsed\":\"0x0\",\"parentHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\"}" - genesisJSONSubnetEVM = "{\"config\":{\"chainId\":43111,\"homesteadBlock\":0,\"eip150Block\":0,\"eip150Hash\":\"0x2086799aeebeae135c246c65021c82b4e15a2c451340993aacfd2751886514f0\",\"eip155Block\":0,\"eip158Block\":0,\"byzantiumBlock\":0,\"constantinopleBlock\":0,\"petersburgBlock\":0,\"istanbulBlock\":0,\"muirGlacierBlock\":0,\"subnetEVMTimestamp\":0},\"nonce\":\"0x0\",\"timestamp\":\"0x0\",\"extraData\":\"0x00\",\"gasLimit\":\"0x5f5e100\",\"difficulty\":\"0x0\",\"mixHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\",\"coinbase\":\"0x0000000000000000000000000000000000000000\",\"alloc\":{\"0100000000000000000000000000000000000000\":{\"code\":\"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033\",\"balance\":\"0x0\"}},\"number\":\"0x0\",\"gasUsed\":\"0x0\",\"parentHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\"}" + genesisJSONMuirGlacier = "{\"config\":{\"chainId\":43111,\"homesteadBlock\":0,\"eip150Block\":0,\"eip150Hash\":\"0x2086799aeebeae135c246c65021c82b4e15a2c451340993aacfd2751886514f0\",\"eip155Block\":0,\"eip158Block\":0,\"byzantiumBlock\":0,\"constantinopleBlock\":0,\"petersburgBlock\":0,\"istanbulBlock\":0,\"muirGlacierBlock\":0},\"nonce\":\"0x0\",\"timestamp\":\"0x0\",\"extraData\":\"0x00\",\"gasLimit\":\"0x5f5e100\",\"difficulty\":\"0x0\",\"mixHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\",\"coinbase\":\"0x0000000000000000000000000000000000000000\",\"alloc\":{\"0100000000000000000000000000000000000000\":{\"code\":\"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033\",\"balance\":\"0x0\"}, \"0x71562b71999873DB5b286dF957af199Ec94617F7\": {\"balance\":\"0x4192927743b88000\"}, \"0x703c4b2bD70c169f5717101CaeE543299Fc946C7\": {\"balance\":\"0x4192927743b88000\"}},\"number\":\"0x0\",\"gasUsed\":\"0x0\",\"parentHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\"}" + genesisJSONSubnetEVM = "{\"config\":{\"chainId\":43111,\"homesteadBlock\":0,\"eip150Block\":0,\"eip150Hash\":\"0x2086799aeebeae135c246c65021c82b4e15a2c451340993aacfd2751886514f0\",\"eip155Block\":0,\"eip158Block\":0,\"byzantiumBlock\":0,\"constantinopleBlock\":0,\"petersburgBlock\":0,\"istanbulBlock\":0,\"muirGlacierBlock\":0,\"subnetEVMTimestamp\":0},\"nonce\":\"0x0\",\"timestamp\":\"0x0\",\"extraData\":\"0x00\",\"gasLimit\":\"0x5f5e100\",\"difficulty\":\"0x0\",\"mixHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\",\"coinbase\":\"0x0000000000000000000000000000000000000000\",\"alloc\":{\"0100000000000000000000000000000000000000\":{\"code\":\"0x7300000000000000000000000000000000000000003014608060405260043610603d5760003560e01c80631e010439146042578063b6510bb314606e575b600080fd5b605c60048036036020811015605657600080fd5b503560b1565b60408051918252519081900360200190f35b818015607957600080fd5b5060af60048036036080811015608e57600080fd5b506001600160a01b03813516906020810135906040810135906060013560b6565b005b30cd90565b836001600160a01b031681836108fc8690811502906040516000604051808303818888878c8acf9550505050505015801560f4573d6000803e3d6000fd5b505050505056fea26469706673582212201eebce970fe3f5cb96bf8ac6ba5f5c133fc2908ae3dcd51082cfee8f583429d064736f6c634300060a0033\",\"balance\":\"0x0\"}, \"0x71562b71999873DB5b286dF957af199Ec94617F7\": {\"balance\":\"0x4192927743b88000\"}, \"0x703c4b2bD70c169f5717101CaeE543299Fc946C7\": {\"balance\":\"0x4192927743b88000\"}},\"number\":\"0x0\",\"gasUsed\":\"0x0\",\"parentHash\":\"0x0000000000000000000000000000000000000000000000000000000000000000\"}" firstTxAmount *big.Int genesisBalance *big.Int ) @@ -79,22 +79,6 @@ func init() { genesisBalance = new(big.Int).Mul(big.NewInt(testMinGasPrice), big.NewInt(21000*1000)) } -func fundGenesis(genesisJSON string) (string, error) { - genesis := &core.Genesis{} - if err := json.Unmarshal([]byte(genesisJSON), genesis); err != nil { - return "", fmt.Errorf("Problem unmarshaling genesis JSON: %s", err) - } - - // add test allocs - genesis.Alloc = core.GenesisAlloc{ - testEthAddrs[0]: core.GenesisAccount{Balance: genesisBalance}, - testEthAddrs[1]: core.GenesisAccount{Balance: genesisBalance}, - } - - bytes, err := json.Marshal(genesis) - return string(bytes), err -} - // BuildGenesisTest returns the genesis bytes for Subnet EVM VM to be used in testing func buildGenesisTest(t *testing.T, genesisJSON string) []byte { ss := CreateStaticService() @@ -335,9 +319,7 @@ func TestVMUpgrades(t *testing.T) { func TestBuildEthTxBlock(t *testing.T) { // reduce block gas cost - genesisString, err := fundGenesis(genesisJSONSubnetEVM) - assert.NoError(t, err) - issuer, vm, dbManager, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") + issuer, vm, dbManager, _ := GenesisVM(t, true, genesisJSONSubnetEVM, "{\"pruning-enabled\":true}", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -476,7 +458,7 @@ func TestBuildEthTxBlock(t *testing.T) { } restartedVM := &VM{} - genesisBytes := buildGenesisTest(t, genesisString) + genesisBytes := buildGenesisTest(t, genesisJSONSubnetEVM) if err := restartedVM.Initialize( NewContext(), @@ -515,10 +497,8 @@ func TestBuildEthTxBlock(t *testing.T) { func TestSetPreferenceRace(t *testing.T) { // Create two VMs which will agree on block A and then // build the two distinct preferred chains above - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":true}", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":true}", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":true}", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -772,10 +752,8 @@ func TestSetPreferenceRace(t *testing.T) { // accept block C, which should be an orphaned block at this point and // get rejected. func TestReorgProtection(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":false}", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "{\"pruning-enabled\":false}", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":false}", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "{\"pruning-enabled\":false}", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -956,10 +934,8 @@ func TestReorgProtection(t *testing.T) { // / \ // B C func TestNonCanonicalAccept(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1133,10 +1109,8 @@ func TestNonCanonicalAccept(t *testing.T) { // | // D func TestStickyPreference(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1409,10 +1383,8 @@ func TestStickyPreference(t *testing.T) { // | // D func TestUncleBlock(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1608,9 +1580,7 @@ func TestUncleBlock(t *testing.T) { // Regression test to ensure that a VM that is not able to parse a block that // contains no transactions. func TestEmptyBlock(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -1676,10 +1646,8 @@ func TestEmptyBlock(t *testing.T) { // | // D func TestAcceptReorg(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer1, vm1, _, _ := GenesisVM(t, true, genesisString, "", "") - issuer2, vm2, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer1, vm1, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") + issuer2, vm2, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm1.Shutdown(); err != nil { @@ -1890,9 +1858,7 @@ func TestAcceptReorg(t *testing.T) { } func TestFutureBlock(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -1955,9 +1921,7 @@ func TestFutureBlock(t *testing.T) { } func TestLastAcceptedBlockNumberAllow(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONMuirGlacier) - assert.NoError(t, err) - issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer, vm, _, _ := GenesisVM(t, true, genesisJSONMuirGlacier, "", "") defer func() { if err := vm.Shutdown(); err != nil { @@ -2101,9 +2065,7 @@ func TestConfigureLogLevel(t *testing.T) { // Regression test to ensure we can build blocks if we are starting with the // Subnet EVM ruleset in genesis. func TestBuildSubnetEVMBlock(t *testing.T) { - genesisString, err := fundGenesis(genesisJSONSubnetEVM) - assert.NoError(t, err) - issuer, vm, _, _ := GenesisVM(t, true, genesisString, "", "") + issuer, vm, _, _ := GenesisVM(t, true, genesisJSONSubnetEVM, "", "") defer func() { if err := vm.Shutdown(); err != nil { From ef3999b1b62c0ac6c5a9c746ddef8a57675eefa1 Mon Sep 17 00:00:00 2001 From: Ceyhun Onur Date: Tue, 15 Feb 2022 16:32:44 +0300 Subject: [PATCH 4/8] remove unused err var --- plugin/evm/vm.go | 1 - 1 file changed, 1 deletion(-) diff --git a/plugin/evm/vm.go b/plugin/evm/vm.go index c0b407328c..c8532bad23 100644 --- a/plugin/evm/vm.go +++ b/plugin/evm/vm.go @@ -98,7 +98,6 @@ var ( errUnclesUnsupported = errors.New("uncles unsupported") errTxHashMismatch = errors.New("txs hash does not match header") errUncleHashMismatch = errors.New("uncle hash mismatch") - errRejectedParent = errors.New("rejected parent") errInvalidDifficulty = errors.New("invalid difficulty") errInvalidMixDigest = errors.New("invalid mix digest") errHeaderExtraDataTooBig = errors.New("header extra data too big") From db98452b8dd14eaec8ad935e146aa77c42b36048 Mon Sep 17 00:00:00 2001 From: Patrick O'Grady Date: Tue, 15 Feb 2022 14:15:21 -0800 Subject: [PATCH 5/8] add compatibility to README --- README.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/README.md b/README.md index 8ff26b8e81..fb48357f3f 100644 --- a/README.md +++ b/README.md @@ -12,6 +12,12 @@ This chain implements the Ethereum Virtual Machine and supports Solidity smart c The Subnet EVM runs in a separate process from the main AvalancheGo process and communicates with it over a local gRPC connection. +### AvalancheGo Compatibility +``` +[v0.1.0] AvalancheGo@v1.7.0-v1.7.4 +[v0.1.1] AvalancheGo@v1.7.5 +``` + ## API The Subnet EVM supports the following API namespaces: From 8a7487c785770f80c831b95eb277fad30fa94eed Mon Sep 17 00:00:00 2001 From: Patrick O'Grady Date: Tue, 15 Feb 2022 14:27:18 -0800 Subject: [PATCH 6/8] update versions --- README.md | 4 ++-- scripts/versions.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index fb48357f3f..dd3d683e17 100644 --- a/README.md +++ b/README.md @@ -74,12 +74,12 @@ and creates a `subnet-evm` genesis file. ```bash # to startup a local cluster (good for development) cd ${HOME}/go/src/github.com/ava-labs/subnet-evm -./scripts/run.sh 1.7.4 +./scripts/run.sh 1.7.5 ``` ```bash # inspect cluster endpoints when ready -cat /tmp/avalanchego-v1.7.4/output.yaml +cat /tmp/avalanchego-v1.7.5/output.yaml < Date: Tue, 15 Feb 2022 17:53:54 -0800 Subject: [PATCH 7/8] remove unnecessary testing harness --- consensus/dummy/test_consensus.go | 53 ------------------------------- core/blockchain_test.go | 31 +++++------------- 2 files changed, 8 insertions(+), 76 deletions(-) delete mode 100644 consensus/dummy/test_consensus.go diff --git a/consensus/dummy/test_consensus.go b/consensus/dummy/test_consensus.go deleted file mode 100644 index b2e0f7e6d1..0000000000 --- a/consensus/dummy/test_consensus.go +++ /dev/null @@ -1,53 +0,0 @@ -// (c) 2019-2020, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package dummy - -import ( - "github.com/ava-labs/subnet-evm/consensus" - "github.com/ava-labs/subnet-evm/core/state" - "github.com/ava-labs/subnet-evm/core/types" -) - -type ( - OnFinalizeAndAssembleCallbackType = func(header *types.Header, state *state.StateDB, txs []*types.Transaction) (err error) - OnExtraStateChangeType = func(block *types.Block, statedb *state.StateDB) (err error) - - ConsensusCallbacks struct { - OnFinalizeAndAssemble OnFinalizeAndAssembleCallbackType - OnExtraStateChange OnExtraStateChangeType - } - - DummyEngineCB struct { - cb *ConsensusCallbacks - DummyEngine - } -) - -func NewTestConsensusCB(cb *ConsensusCallbacks) *DummyEngineCB { - return &DummyEngineCB{ - cb: cb, - } -} - -func (self *DummyEngineCB) Finalize(chain consensus.ChainHeaderReader, block *types.Block, parent *types.Header, state *state.StateDB, receipts []*types.Receipt) error { - if self.cb.OnExtraStateChange != nil { - err := self.cb.OnExtraStateChange(block, state) - if err != nil { - return err - } - } - - return self.DummyEngine.Finalize(chain, block, parent, state, receipts) -} - -func (self *DummyEngineCB) FinalizeAndAssemble(chain consensus.ChainHeaderReader, header *types.Header, parent *types.Header, state *state.StateDB, txs []*types.Transaction, - uncles []*types.Header, receipts []*types.Receipt) (*types.Block, error) { - if self.cb.OnFinalizeAndAssemble != nil { - err := self.cb.OnFinalizeAndAssemble(header, state, txs) - if err != nil { - return nil, err - } - } - return self.DummyEngine.FinalizeAndAssemble(chain, header, parent, state, txs, uncles, receipts) -} diff --git a/core/blockchain_test.go b/core/blockchain_test.go index b070af0171..172d355229 100644 --- a/core/blockchain_test.go +++ b/core/blockchain_test.go @@ -9,9 +9,7 @@ import ( "github.com/ava-labs/subnet-evm/consensus/dummy" "github.com/ava-labs/subnet-evm/core/rawdb" - "github.com/ava-labs/subnet-evm/core/state" "github.com/ava-labs/subnet-evm/core/state/pruner" - "github.com/ava-labs/subnet-evm/core/types" "github.com/ava-labs/subnet-evm/core/vm" "github.com/ava-labs/subnet-evm/ethdb" "github.com/ava-labs/subnet-evm/params" @@ -265,16 +263,7 @@ func TestBlockChainOfflinePruningUngracefulShutdown(t *testing.T) { SnapshotLimit: 256, }, chainConfig, - dummy.NewTestConsensusCB(&dummy.ConsensusCallbacks{ - OnExtraStateChange: func(block *types.Block, sdb *state.StateDB) error { - sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) - return nil - }, - OnFinalizeAndAssemble: func(header *types.Header, sdb *state.StateDB, txs []*types.Transaction) error { - sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) - return nil - }, - }), + dummy.NewFaker(), vm.Config{}, lastAcceptedHash, ) @@ -290,6 +279,11 @@ func TestBlockChainOfflinePruningUngracefulShutdown(t *testing.T) { return blockchain, nil } + targetRoot := blockchain.LastAcceptedBlock().Root() + if targetRoot == blockchain.Genesis().Root() { + return blockchain, nil + } + tempDir := t.TempDir() if err := blockchain.CleanBlockRootsAboveLastAccepted(); err != nil { return nil, err @@ -299,10 +293,10 @@ func TestBlockChainOfflinePruningUngracefulShutdown(t *testing.T) { return nil, fmt.Errorf("offline pruning failed (%s, %d): %w", tempDir, 256, err) } - targetRoot := blockchain.LastAcceptedBlock().Root() if err := pruner.Prune(targetRoot); err != nil { return nil, fmt.Errorf("failed to prune blockchain with target root: %s due to: %w", targetRoot, err) } + // Re-initialize the blockchain after pruning return NewBlockChain( db, @@ -313,16 +307,7 @@ func TestBlockChainOfflinePruningUngracefulShutdown(t *testing.T) { SnapshotLimit: 256, }, chainConfig, - dummy.NewTestConsensusCB(&dummy.ConsensusCallbacks{ - OnExtraStateChange: func(block *types.Block, sdb *state.StateDB) error { - sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) - return nil - }, - OnFinalizeAndAssemble: func(header *types.Header, sdb *state.StateDB, txs []*types.Transaction) error { - sdb.SetState(common.HexToAddress("0xdeadbeef"), common.HexToHash("0xdeadbeef"), common.HexToHash("0xdeadbeef")) - return nil - }, - }), + dummy.NewFaker(), vm.Config{}, lastAcceptedHash, ) From c14727ab85c7d86d84501a61ed2a3509b7dda2cc Mon Sep 17 00:00:00 2001 From: Patrick O'Grady Date: Tue, 15 Feb 2022 18:19:54 -0800 Subject: [PATCH 8/8] fix test --- plugin/evm/tx_gossiping_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plugin/evm/tx_gossiping_test.go b/plugin/evm/tx_gossiping_test.go index 50dad3ce53..a5f845cea9 100644 --- a/plugin/evm/tx_gossiping_test.go +++ b/plugin/evm/tx_gossiping_test.go @@ -116,7 +116,7 @@ func TestMempoolTxsAddedTxsGossipedAfterActivation(t *testing.T) { assert.NoError(rlp.DecodeBytes(requestMsg.Txs, &txs)) assert.Len(txs, 2) assert.ElementsMatch( - []common.Hash{ethTxs[0].Hash(), txs[1].Hash()}, + []common.Hash{ethTxs[0].Hash(), ethTxs[1].Hash()}, []common.Hash{txs[0].Hash(), txs[1].Hash()}, ) seen++