diff --git a/graft/coreth/core/blockchain.go b/graft/coreth/core/blockchain.go index b1f6a70921c6..fddf901fabdf 100644 --- a/graft/coreth/core/blockchain.go +++ b/graft/coreth/core/blockchain.go @@ -47,9 +47,9 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/internal/version" "github.com/ava-labs/avalanchego/graft/coreth/params" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/customtypes" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" "github.com/ava-labs/avalanchego/graft/coreth/triedb/hashdb" "github.com/ava-labs/avalanchego/graft/coreth/triedb/pathdb" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/vms/evm/acp176" "github.com/ava-labs/avalanchego/vms/evm/sync/customrawdb" "github.com/ava-labs/libevm/common" diff --git a/graft/coreth/core/extstate/database.go b/graft/coreth/core/extstate/database.go index 8a74602ccc07..3d013a16e3d7 100644 --- a/graft/coreth/core/extstate/database.go +++ b/graft/coreth/core/extstate/database.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/triedb" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" + "github.com/ava-labs/avalanchego/graft/evm/firewood" ) func NewDatabaseWithConfig(db ethdb.Database, config *triedb.Config) state.Database { @@ -22,12 +22,9 @@ func NewDatabaseWithNodeDB(db ethdb.Database, triedb *triedb.Database) state.Dat } func wrapIfFirewood(db state.Database) state.Database { - fw, ok := db.TrieDB().Backend().(*firewood.Database) + fw, ok := db.TrieDB().Backend().(*firewood.TrieDB) if !ok { return db } - return &firewoodAccessorDB{ - Database: db, - fw: fw, - } + return firewood.NewStateAccessor(db, fw) } diff --git a/graft/coreth/core/extstate/firewood_database.go b/graft/coreth/core/extstate/firewood_database.go deleted file mode 100644 index 7499ef576813..000000000000 --- a/graft/coreth/core/extstate/firewood_database.go +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package extstate - -import ( - "fmt" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/state" - - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" -) - -var ( - _ state.Database = (*firewoodAccessorDB)(nil) - _ state.Trie = (*firewood.AccountTrie)(nil) - _ state.Trie = (*firewood.StorageTrie)(nil) -) - -type firewoodAccessorDB struct { - state.Database - fw *firewood.Database -} - -// OpenTrie opens the main account trie. -func (db *firewoodAccessorDB) OpenTrie(root common.Hash) (state.Trie, error) { - return firewood.NewAccountTrie(root, db.fw) -} - -// OpenStorageTrie opens a wrapped version of the account trie. -// -//nolint:revive // removing names loses context. -func (*firewoodAccessorDB) OpenStorageTrie(stateRoot common.Hash, addr common.Address, accountRoot common.Hash, self state.Trie) (state.Trie, error) { - accountTrie, ok := self.(*firewood.AccountTrie) - if !ok { - return nil, fmt.Errorf("invalid account trie type: %T", self) - } - return firewood.NewStorageTrie(accountTrie) -} - -// CopyTrie returns a deep copy of the given trie. -// It can be altered by the caller. -func (*firewoodAccessorDB) CopyTrie(t state.Trie) state.Trie { - switch t := t.(type) { - case *firewood.AccountTrie: - return t.Copy() - case *firewood.StorageTrie: - return nil // The storage trie just wraps the account trie, so we must re-open it separately. - default: - panic(fmt.Errorf("unknown trie type %T", t)) - } -} diff --git a/graft/coreth/core/genesis_test.go b/graft/coreth/core/genesis_test.go index cb8aa520c314..37e4aaf1e180 100644 --- a/graft/coreth/core/genesis_test.go +++ b/graft/coreth/core/genesis_test.go @@ -40,9 +40,9 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/params/extras" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/upgrade/ap3" "github.com/ava-labs/avalanchego/graft/coreth/precompile/contracts/warp" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" "github.com/ava-labs/avalanchego/graft/coreth/triedb/pathdb" "github.com/ava-labs/avalanchego/graft/coreth/utils" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/vms/evm/sync/customrawdb" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/rawdb" diff --git a/graft/coreth/internal/ethapi/api.go b/graft/coreth/internal/ethapi/api.go index 1c09de177118..9d6409918ef5 100644 --- a/graft/coreth/internal/ethapi/api.go +++ b/graft/coreth/internal/ethapi/api.go @@ -42,7 +42,7 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/params" "github.com/ava-labs/avalanchego/graft/coreth/plugin/evm/customtypes" "github.com/ava-labs/avalanchego/graft/coreth/rpc" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/libevm/accounts" "github.com/ava-labs/libevm/accounts/keystore" "github.com/ava-labs/libevm/accounts/scwallet" @@ -704,7 +704,7 @@ func (s *BlockChainAPI) GetProof(ctx context.Context, address common.Address, st if statedb == nil || err != nil { return nil, err } - if _, ok := statedb.Database().TrieDB().Backend().(*firewood.Database); ok { + if _, ok := statedb.Database().TrieDB().Backend().(*firewood.TrieDB); ok { return nil, errors.New("firewood database does not yet support getProof") } codeHash := statedb.GetCodeHash(address) diff --git a/graft/coreth/tests/state_test_util.go b/graft/coreth/tests/state_test_util.go index fd805da620ae..a90044765a22 100644 --- a/graft/coreth/tests/state_test_util.go +++ b/graft/coreth/tests/state_test_util.go @@ -32,9 +32,9 @@ import ( "github.com/ava-labs/avalanchego/graft/coreth/core/extstate" "github.com/ava-labs/avalanchego/graft/coreth/core/state/snapshot" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" "github.com/ava-labs/avalanchego/graft/coreth/triedb/hashdb" "github.com/ava-labs/avalanchego/graft/coreth/triedb/pathdb" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/vms/evm/sync/customrawdb" "github.com/ava-labs/libevm/common" "github.com/ava-labs/libevm/core/rawdb" diff --git a/graft/coreth/triedb/firewood/database.go b/graft/coreth/triedb/firewood/database.go deleted file mode 100644 index f95104260f35..000000000000 --- a/graft/coreth/triedb/firewood/database.go +++ /dev/null @@ -1,601 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package firewood - -import ( - "context" - "errors" - "fmt" - "os" - "path/filepath" - "sync" - "time" - - "github.com/ava-labs/firewood-go-ethhash/ffi" - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/rawdb" - "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/libevm/ethdb" - "github.com/ava-labs/libevm/libevm/stateconf" - "github.com/ava-labs/libevm/log" - "github.com/ava-labs/libevm/metrics" - "github.com/ava-labs/libevm/trie/trienode" - "github.com/ava-labs/libevm/trie/triestate" - "github.com/ava-labs/libevm/triedb" - "github.com/ava-labs/libevm/triedb/database" -) - -const ( - // Directory where all Firewood state lives. - firewoodDir = "firewood" - firewoodFileName = "firewood.db" - firewoodRootStoreDir = "root_store" -) - -var ( - _ proposable = (*ffi.Database)(nil) - _ proposable = (*ffi.Proposal)(nil) - - // FFI triedb operation metrics - ffiProposeCount = metrics.GetOrRegisterCounter("firewood/triedb/propose/count", nil) - ffiProposeTimer = metrics.GetOrRegisterCounter("firewood/triedb/propose/time", nil) - ffiCommitCount = metrics.GetOrRegisterCounter("firewood/triedb/commit/count", nil) - ffiCommitTimer = metrics.GetOrRegisterCounter("firewood/triedb/commit/time", nil) - ffiCleanupTimer = metrics.GetOrRegisterCounter("firewood/triedb/cleanup/time", nil) - ffiOutstandingProposals = metrics.GetOrRegisterGauge("firewood/triedb/propose/outstanding", nil) - - // FFI Trie operation metrics - ffiHashCount = metrics.GetOrRegisterCounter("firewood/triedb/hash/count", nil) - ffiHashTimer = metrics.GetOrRegisterCounter("firewood/triedb/hash/time", nil) - ffiReadCount = metrics.GetOrRegisterCounter("firewood/triedb/read/count", nil) - ffiReadTimer = metrics.GetOrRegisterCounter("firewood/triedb/read/time", nil) -) - -type proposable interface { - // Propose creates a new proposal from the current state with the given keys and values. - Propose(keys, values [][]byte) (*ffi.Proposal, error) -} - -// ProposalContext represents a proposal in the Firewood database. -// This tracks all outstanding proposals to allow dereferencing upon commit. -type ProposalContext struct { - Proposal *ffi.Proposal - Hashes map[common.Hash]struct{} // All corresponding block hashes - Root common.Hash - Block uint64 - Parent *ProposalContext - Children []*ProposalContext -} - -type Config struct { - ChainDataDir string - CleanCacheSize int // Size of the clean cache in bytes - FreeListCacheEntries uint // Number of free list entries to cache - Revisions uint // Number of revisions to keep in memory (must be >= 2) - ReadCacheStrategy ffi.CacheStrategy - ArchiveMode bool -} - -// Note that `FilePath` is not specified, and must always be set by the user. -var Defaults = Config{ - CleanCacheSize: 1024 * 1024, // 1MB - FreeListCacheEntries: 40_000, - Revisions: 100, - ReadCacheStrategy: ffi.CacheAllReads, -} - -func (c Config) BackendConstructor(ethdb.Database) triedb.DBOverride { - db, err := New(c) - if err != nil { - log.Crit("firewood: error creating database", "error", err) - } - return db -} - -type Database struct { - fwDisk *ffi.Database // The underlying Firewood database, used for storing proposals and revisions. - proposalLock sync.RWMutex - // proposalMap provides O(1) access by state root to all proposals stored in the proposalTree - proposalMap map[common.Hash][]*ProposalContext - // The proposal tree tracks the structure of the current proposals, and which proposals are children of which. - // This is used to ensure that we can dereference proposals correctly and commit the correct ones - // in the case of duplicate state roots. - // The root of the tree is stored here, and represents the top-most layer on disk. - proposalTree *ProposalContext -} - -// New creates a new Firewood database with the given disk database and configuration. -// Any error during creation will cause the program to exit. -func New(config Config) (*Database, error) { - firewoodDir := filepath.Join(config.ChainDataDir, firewoodDir) - filePath := filepath.Join(firewoodDir, firewoodFileName) - if err := validatePath(filePath); err != nil { - return nil, err - } - - var rootStoreDir string - if config.ArchiveMode { - rootStoreDir = filepath.Join(firewoodDir, firewoodRootStoreDir) - } - - fw, err := ffi.New(filePath, &ffi.Config{ - NodeCacheEntries: uint(config.CleanCacheSize) / 256, // TODO: estimate 256 bytes per node - FreeListCacheEntries: config.FreeListCacheEntries, - Revisions: config.Revisions, - ReadCacheStrategy: config.ReadCacheStrategy, - RootStoreDir: rootStoreDir, - }) - if err != nil { - return nil, err - } - - currentRoot, err := fw.Root() - if err != nil { - return nil, err - } - - return &Database{ - fwDisk: fw, - proposalMap: make(map[common.Hash][]*ProposalContext), - proposalTree: &ProposalContext{ - Root: common.Hash(currentRoot), - }, - }, nil -} - -func validatePath(path string) error { - if path == "" { - return errors.New("firewood database file path must be set") - } - - // Check that the directory exists - dir := filepath.Dir(path) - switch info, err := os.Stat(dir); { - case os.IsNotExist(err): - log.Info("Database directory not found, creating", "path", dir) - if err := os.MkdirAll(dir, 0o755); err != nil { - return fmt.Errorf("error creating database directory: %w", err) - } - return nil - case err != nil: - return fmt.Errorf("error checking database directory: %w", err) - case !info.IsDir(): - return fmt.Errorf("database directory path is not a directory: %s", dir) - } - - return nil -} - -// Scheme returns the scheme of the database. -// This is only used in some API calls -// and in StateDB to avoid iterating through deleted storage tries. -// WARNING: If cherry-picking anything from upstream that uses this, -// it must be overwritten to use something like: -// `_, ok := db.(*Database); if !ok { return "" }` -// to recognize the Firewood database. -func (*Database) Scheme() string { - return rawdb.HashScheme -} - -// Initialized checks whether a non-empty genesis block has been written. -func (db *Database) Initialized(common.Hash) bool { - root, err := db.fwDisk.Root() - if err != nil { - log.Error("firewood: error getting current root", "error", err) - return false - } - - // If the current root isn't empty, then unless the database is empty, we have a genesis block recorded. - return common.Hash(root) != types.EmptyRootHash -} - -// Update takes a root and a set of keys-values and creates a new proposal. -// It will not be committed until the Commit method is called. -// This function should be called even if there are no changes to the state to ensure proper tracking of block hashes. -func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint64, nodes *trienode.MergedNodeSet, _ *triestate.Set, opts ...stateconf.TrieDBUpdateOption) error { - // We require block hashes to be provided for all blocks in production. - // However, many tests cannot reasonably provide a block hash for genesis, so we allow it to be omitted. - parentHash, hash, ok := stateconf.ExtractTrieDBUpdatePayload(opts...) - if !ok { - log.Error("firewood: no block hash provided for block %d", block) - } - - // The rest of the operations except key-value arranging must occur with a lock - db.proposalLock.Lock() - defer db.proposalLock.Unlock() - - // Check if this proposal already exists. - // During reorgs, we may have already created this proposal. - // Additionally, we may have already created this proposal with a different block hash. - if existingProposals, ok := db.proposalMap[root]; ok { - for _, existing := range existingProposals { - // If the block hash is already tracked, we can skip proposing this again. - if _, exists := existing.Hashes[hash]; exists { - log.Debug("firewood: proposal already exists", "root", root.Hex(), "parent", parentRoot.Hex(), "block", block, "hash", hash.Hex()) - return nil - } - // We already have this proposal, but should create a new context with the correct hash. - // This solves the case of a unique block hash, but the same underlying proposal. - if _, exists := existing.Parent.Hashes[parentHash]; exists { - log.Debug("firewood: proposal already exists, updating hash", "root", root.Hex(), "parent", parentRoot.Hex(), "block", block, "hash", hash.Hex()) - existing.Hashes[hash] = struct{}{} - return nil - } - } - } - - keys, values := arrangeKeyValuePairs(nodes) // may return nil, nil if no changes - return db.propose(root, parentRoot, hash, parentHash, block, keys, values) -} - -// propose creates a new proposal for every possible parent with the given keys and values. -// If the parent cannot be found, an error will be returned. -// -// To avoid having to create a new proposal for each valid state root, the block hashes are -// provided to ensure uniqueness. When this method is called, we can guarantee that the proposalContext -// must be created and tracked. -// -// Should only be accessed with the proposal lock held. -func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash common.Hash, parentHash common.Hash, block uint64, keys [][]byte, values [][]byte) error { - // Find the parent proposal with the correct hash. - // We assume the number of proposals at a given root is small, so we can iterate through them. - for _, parentProposal := range db.proposalMap[parentRoot] { - // If we know this proposal cannot be the parent, we can skip it. - // Since the only possible block that won't have a parent hash is block 1, - // and that will always be proposed from the database root, - // we can guarantee that the parent hash will be present in one of the proposals. - if _, exists := parentProposal.Hashes[parentHash]; !exists { - continue - } - log.Debug("firewood: proposing from parent proposal", "parent", parentProposal.Root.Hex(), "root", root.Hex(), "height", block) - p, err := createProposal(parentProposal.Proposal, root, keys, values) - if err != nil { - return err - } - pCtx := &ProposalContext{ - Proposal: p, - Hashes: map[common.Hash]struct{}{hash: {}}, - Root: root, - Block: block, - Parent: parentProposal, - } - - db.proposalMap[root] = append(db.proposalMap[root], pCtx) - parentProposal.Children = append(parentProposal.Children, pCtx) - return nil - } - - // Since we were unable to find a parent proposal with the given parent hash, - // we must create a new proposal from the database root. - // We must avoid the case in which we are reexecuting blocks upon startup, and haven't yet stored the parent block. - if _, exists := db.proposalTree.Hashes[parentHash]; db.proposalTree.Block != 0 && !exists { - return fmt.Errorf("firewood: parent hash %s not found for block %s at height %d", parentHash.Hex(), hash.Hex(), block) - } else if db.proposalTree.Root != parentRoot { - return fmt.Errorf("firewood: parent root %s does not match proposal tree root %s for root %s at height %d", parentRoot.Hex(), db.proposalTree.Root.Hex(), root.Hex(), block) - } - - log.Debug("firewood: proposing from database root", "root", root.Hex(), "height", block) - p, err := createProposal(db.fwDisk, root, keys, values) - if err != nil { - return err - } - pCtx := &ProposalContext{ - Proposal: p, - Hashes: map[common.Hash]struct{}{hash: {}}, // This may be common.Hash{} for genesis blocks. - Root: root, - Block: block, - Parent: db.proposalTree, - } - db.proposalMap[root] = append(db.proposalMap[root], pCtx) - db.proposalTree.Children = append(db.proposalTree.Children, pCtx) - - return nil -} - -// Commit persists a proposal as a revision to the database. -// -// Any time this is called, we expect either: -// 1. The root is the same as the current root of the database (empty block during bootstrapping) -// 2. We have created a valid propsal with that root, and it is of height +1 above the proposal tree root. -// Additionally, this should be unique. -// -// Afterward, we know that no other proposal at this height can be committed, so we can dereference all -// children in the the other branches of the proposal tree. -func (db *Database) Commit(root common.Hash, report bool) error { - // We need to lock the proposal tree to prevent concurrent writes. - db.proposalLock.Lock() - defer db.proposalLock.Unlock() - - // Find the proposal with the given root. - var pCtx *ProposalContext - for _, possible := range db.proposalMap[root] { - if possible.Parent.Root == db.proposalTree.Root && possible.Parent.Block == db.proposalTree.Block { - // We found the proposal with the correct parent. - if pCtx != nil { - // This should never happen, as we ensure that we don't create duplicate proposals in `propose`. - return fmt.Errorf("firewood: multiple proposals found for %s", root.Hex()) - } - pCtx = possible - } - } - if pCtx == nil { - return fmt.Errorf("firewood: committable proposal not found for %s", root.Hex()) - } - - start := time.Now() - // Commit the proposal to the database. - if err := pCtx.Proposal.Commit(); err != nil { - db.dereference(pCtx) // no longer committable - return fmt.Errorf("firewood: error committing proposal %s: %w", root.Hex(), err) - } - ffiCommitCount.Inc(1) - ffiCommitTimer.Inc(time.Since(start).Milliseconds()) - ffiOutstandingProposals.Dec(1) - // Now that the proposal is committed, we should clean up the proposal tree on return. - defer db.cleanupCommittedProposal(pCtx) - - // Assert that the root of the database matches the committed proposal root. - currentRoot, err := db.fwDisk.Root() - if err != nil { - return fmt.Errorf("firewood: error getting current root after commit: %w", err) - } - - currentRootHash := common.Hash(currentRoot) - if currentRootHash != root { - return fmt.Errorf("firewood: current root %s does not match expected root %s", currentRootHash.Hex(), root.Hex()) - } - - if report { - log.Info("Persisted proposal to firewood database", "root", root) - } else { - log.Debug("Persisted proposal to firewood database", "root", root) - } - return nil -} - -// Size returns the storage size of diff layer nodes above the persistent disk -// layer and the dirty nodes buffered within the disk layer -// Only used for metrics and Commit intervals in APIs. -// This will be implemented in the firewood database eventually. -// Currently, Firewood stores all revisions in disk and proposals in memory. -func (*Database) Size() (common.StorageSize, common.StorageSize) { - return 0, 0 -} - -// Reference is a no-op. -func (*Database) Reference(common.Hash, common.Hash) {} - -// Dereference is a no-op since Firewood handles unused state roots internally. -func (*Database) Dereference(common.Hash) {} - -// Firewood does not support this. -func (*Database) Cap(common.StorageSize) error { - return nil -} - -func (db *Database) Close() error { - db.proposalLock.Lock() - defer db.proposalLock.Unlock() - - // before closing, we must deference any outstanding proposals to free the - // memory owned by firewood (outside of go's memory management) - for _, pCtx := range db.proposalTree.Children { - db.dereference(pCtx) - } - - db.proposalMap = nil - db.proposalTree.Children = nil - - // Close the database - // This may block momentarily while finalizers for Firewood objects run. - return db.fwDisk.Close(context.Background()) -} - -// createProposal creates a new proposal from the given layer -// If there are no changes, it will return nil. -func createProposal(layer proposable, root common.Hash, keys, values [][]byte) (p *ffi.Proposal, err error) { - // If there's an error after creating the proposal, we must drop it. - defer func() { - if err != nil && p != nil { - if dropErr := p.Drop(); dropErr != nil { - // We should still return the original error. - log.Error("firewood: error dropping proposal after error", "root", root.Hex(), "error", dropErr) - } - p = nil - } - }() - - if len(keys) != len(values) { - return nil, fmt.Errorf("firewood: keys and values must have the same length, got %d keys and %d values", len(keys), len(values)) - } - - start := time.Now() - p, err = layer.Propose(keys, values) - if err != nil { - return nil, fmt.Errorf("firewood: unable to create proposal for root %s: %w", root.Hex(), err) - } - ffiProposeCount.Inc(1) - ffiProposeTimer.Inc(time.Since(start).Milliseconds()) - ffiOutstandingProposals.Inc(1) - - currentRoot, err := p.Root() - if err != nil { - return nil, fmt.Errorf("firewood: error getting root of proposal %s: %w", root, err) - } - - currentRootHash := common.Hash(currentRoot) - if root != currentRootHash { - return nil, fmt.Errorf("firewood: proposed root %s does not match expected root %s", currentRootHash.Hex(), root.Hex()) - } - - return p, nil -} - -// cleanupCommittedProposal dereferences the proposal and removes it from the proposal map. -// It also recursively dereferences all children of the proposal. -func (db *Database) cleanupCommittedProposal(pCtx *ProposalContext) { - start := time.Now() - oldChildren := db.proposalTree.Children - db.proposalTree = pCtx - db.proposalTree.Parent = nil - - db.removeProposalFromMap(pCtx) - - for _, childCtx := range oldChildren { - // Don't dereference the recently commit proposal. - if childCtx != pCtx { - db.dereference(childCtx) - } - } - ffiCleanupTimer.Inc(time.Since(start).Milliseconds()) -} - -// Internally removes all references of the proposal from the database. -// Should only be accessed with the proposal lock held. -// Consumer must not be iterating the proposal map at this root. -func (db *Database) dereference(pCtx *ProposalContext) { - // Base case: if there are children, we need to dereference them as well. - for _, child := range pCtx.Children { - db.dereference(child) - } - pCtx.Children = nil - - // Remove the proposal from the map. - db.removeProposalFromMap(pCtx) - - // Drop the proposal in the backend. - if err := pCtx.Proposal.Drop(); err != nil { - log.Error("firewood: error dropping proposal", "root", pCtx.Root.Hex(), "error", err) - } - ffiOutstandingProposals.Dec(1) -} - -// removeProposalFromMap removes the proposal from the proposal map. -// The proposal lock must be held when calling this function. -func (db *Database) removeProposalFromMap(pCtx *ProposalContext) { - rootList := db.proposalMap[pCtx.Root] - for i, p := range rootList { - if p == pCtx { // pointer comparison - guaranteed to be unique - rootList[i] = rootList[len(rootList)-1] - rootList[len(rootList)-1] = nil - rootList = rootList[:len(rootList)-1] - break - } - } - if len(rootList) == 0 { - delete(db.proposalMap, pCtx.Root) - } else { - db.proposalMap[pCtx.Root] = rootList - } -} - -// Reader retrieves a node reader belonging to the given state root. -// An error will be returned if the requested state is not available. -func (db *Database) Reader(root common.Hash) (database.Reader, error) { - if _, err := db.fwDisk.GetFromRoot(ffi.Hash(root), []byte{}); err != nil { - return nil, fmt.Errorf("firewood: unable to retrieve from root %s: %w", root.Hex(), err) - } - return &reader{db: db, root: ffi.Hash(root)}, nil -} - -// reader is a state reader of Database which implements the Reader interface. -type reader struct { - db *Database - root ffi.Hash // The root of the state this reader is reading. -} - -// Node retrieves the trie node with the given node hash. No error will be -// returned if the node is not found. -func (reader *reader) Node(_ common.Hash, path []byte, _ common.Hash) ([]byte, error) { - // This function relies on Firewood's internal locking to ensure concurrent reads are safe. - // This is safe even if a proposal is being committed concurrently. - start := time.Now() - result, err := reader.db.fwDisk.GetFromRoot(reader.root, path) - if metrics.EnabledExpensive { - ffiReadCount.Inc(1) - ffiReadTimer.Inc(time.Since(start).Milliseconds()) - } - return result, err -} - -// getProposalHash calculates the hash if the set of keys and values are -// proposed from the given parent root. -func (db *Database) getProposalHash(parentRoot common.Hash, keys, values [][]byte) (common.Hash, error) { - // This function only reads from existing tracked proposals, so we can use a read lock. - db.proposalLock.RLock() - defer db.proposalLock.RUnlock() - - var ( - p *ffi.Proposal - err error - ) - start := time.Now() - if db.proposalTree.Root == parentRoot { - // Propose from the database root. - p, err = db.fwDisk.Propose(keys, values) - if err != nil { - return common.Hash{}, fmt.Errorf("firewood: error proposing from root %s: %w", parentRoot.Hex(), err) - } - } else { - // Find any proposal with the given parent root. - // Since we are only using the proposal to find the root hash, - // we can use the first proposal found. - proposals, ok := db.proposalMap[parentRoot] - if !ok || len(proposals) == 0 { - return common.Hash{}, fmt.Errorf("firewood: no proposal found for parent root %s", parentRoot.Hex()) - } - rootProposal := proposals[0].Proposal - - p, err = rootProposal.Propose(keys, values) - if err != nil { - return common.Hash{}, fmt.Errorf("firewood: error proposing from parent proposal %s: %w", parentRoot.Hex(), err) - } - } - ffiHashCount.Inc(1) - ffiHashTimer.Inc(time.Since(start).Milliseconds()) - - // We succesffuly created a proposal, so we must drop it after use. - defer func() { - if err := p.Drop(); err != nil { - log.Error("firewood: error dropping proposal after hash computation", "parentRoot", parentRoot.Hex(), "error", err) - } - }() - - root, err := p.Root() - if err != nil { - return common.Hash{}, err - } - return common.Hash(root), nil -} - -func arrangeKeyValuePairs(nodes *trienode.MergedNodeSet) ([][]byte, [][]byte) { - if nodes == nil { - return nil, nil // No changes to propose - } - // Create key-value pairs for the nodes in bytes. - var ( - acctKeys [][]byte - acctValues [][]byte - storageKeys [][]byte - storageValues [][]byte - ) - - flattenedNodes := nodes.Flatten() - - for _, nodeset := range flattenedNodes { - for str, node := range nodeset { - if len(str) == common.HashLength { - // This is an account node. - acctKeys = append(acctKeys, []byte(str)) - acctValues = append(acctValues, node.Blob) - } else { - storageKeys = append(storageKeys, []byte(str)) - storageValues = append(storageValues, node.Blob) - } - } - } - - // We need to do all storage operations first, so prefix-deletion works for accounts. - return append(storageKeys, acctKeys...), append(storageValues, acctValues...) -} diff --git a/graft/coreth/triedb/firewood/account_trie.go b/graft/evm/firewood/account_trie.go similarity index 88% rename from graft/coreth/triedb/firewood/account_trie.go rename to graft/evm/firewood/account_trie.go index 7be34f34e89a..c1952a548cd0 100644 --- a/graft/coreth/triedb/firewood/account_trie.go +++ b/graft/evm/firewood/account_trie.go @@ -7,6 +7,7 @@ import ( "errors" "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/state" "github.com/ava-labs/libevm/core/types" "github.com/ava-labs/libevm/crypto" "github.com/ava-labs/libevm/ethdb" @@ -17,17 +18,19 @@ import ( "github.com/ava-labs/libevm/triedb/database" ) -// AccountTrie implements state.Trie for managing account states. +var _ state.Trie = (*accountTrie)(nil) + +// accountTrie implements state.Trie for managing account states. // There are a couple caveats to the current implementation: // 1. `Commit` is not used as expected in the state package. The `StorageTrie` doesn't return -// values, and we thus rely on the `AccountTrie`. +// values, and we thus rely on the `accountTrie`. // 2. The `Hash` method actually creates the proposal, since Firewood cannot calculate // the hash of the trie without committing it. It is immediately dropped, and this // can likely be optimized. // // Note this is not concurrent safe. -type AccountTrie struct { - fw *Database +type accountTrie struct { + fw *TrieDB parentRoot common.Hash root common.Hash reader database.Reader @@ -37,12 +40,12 @@ type AccountTrie struct { hasChanges bool } -func NewAccountTrie(root common.Hash, db *Database) (*AccountTrie, error) { +func newAccountTrie(root common.Hash, db *TrieDB) (*accountTrie, error) { reader, err := db.Reader(root) if err != nil { return nil, err } - return &AccountTrie{ + return &accountTrie{ fw: db, parentRoot: root, reader: reader, @@ -55,7 +58,7 @@ func NewAccountTrie(root common.Hash, db *Database) (*AccountTrie, error) { // - If the account has been updated, the new value is returned. // - If the account has been deleted, (nil, nil) is returned. // - If the account does not exist, (nil, nil) is returned. -func (a *AccountTrie) GetAccount(addr common.Address) (*types.StateAccount, error) { +func (a *accountTrie) GetAccount(addr common.Address) (*types.StateAccount, error) { key := crypto.Keccak256Hash(addr.Bytes()).Bytes() // First check if there's a pending update for this account @@ -91,7 +94,7 @@ func (a *AccountTrie) GetAccount(addr common.Address) (*types.StateAccount, erro // - If the storage slot has been updated, the new value is returned. // - If the storage slot has been deleted, (nil, nil) is returned. // - If the storage slot does not exist, (nil, nil) is returned. -func (a *AccountTrie) GetStorage(addr common.Address, key []byte) ([]byte, error) { +func (a *accountTrie) GetStorage(addr common.Address, key []byte) ([]byte, error) { // If the account has been deleted, we should return nil accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() if val, exists := a.dirtyKeys[string(accountKey)]; exists && len(val) == 0 { @@ -127,7 +130,7 @@ func (a *AccountTrie) GetStorage(addr common.Address, key []byte) ([]byte, error // UpdateAccount replaces or creates the state account associated with an address. // This new value will be returned for subsequent `GetAccount` calls. -func (a *AccountTrie) UpdateAccount(addr common.Address, account *types.StateAccount) error { +func (a *accountTrie) UpdateAccount(addr common.Address, account *types.StateAccount) error { // Queue the keys and values for later commit key := crypto.Keccak256Hash(addr.Bytes()).Bytes() data, err := rlp.EncodeToBytes(account) @@ -143,7 +146,7 @@ func (a *AccountTrie) UpdateAccount(addr common.Address, account *types.StateAcc // UpdateStorage replaces or creates the value associated with a storage key for a given account address. // This new value will be returned for subsequent `GetStorage` calls. -func (a *AccountTrie) UpdateStorage(addr common.Address, key []byte, value []byte) error { +func (a *accountTrie) UpdateStorage(addr common.Address, key []byte, value []byte) error { var combinedKey [2 * common.HashLength]byte accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() storageKey := crypto.Keccak256Hash(key).Bytes() @@ -164,7 +167,7 @@ func (a *AccountTrie) UpdateStorage(addr common.Address, key []byte, value []byt } // DeleteAccount removes the state account associated with an address. -func (a *AccountTrie) DeleteAccount(addr common.Address) error { +func (a *accountTrie) DeleteAccount(addr common.Address) error { key := crypto.Keccak256Hash(addr.Bytes()).Bytes() // Queue the key for deletion a.dirtyKeys[string(key)] = nil @@ -175,7 +178,7 @@ func (a *AccountTrie) DeleteAccount(addr common.Address) error { } // DeleteStorage removes the value associated with a storage key for a given account address. -func (a *AccountTrie) DeleteStorage(addr common.Address, key []byte) error { +func (a *accountTrie) DeleteStorage(addr common.Address, key []byte) error { var combinedKey [2 * common.HashLength]byte accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() storageKey := crypto.Keccak256Hash(key).Bytes() @@ -193,7 +196,7 @@ func (a *AccountTrie) DeleteStorage(addr common.Address, key []byte) error { // Hash returns the current hash of the state trie. // This will create a proposal and drop it, so it is not efficient to call for each transaction. // If there are no changes since the last call, the cached root is returned. -func (a *AccountTrie) Hash() common.Hash { +func (a *accountTrie) Hash() common.Hash { hash, err := a.hash() if err != nil { log.Error("Failed to hash account trie", "error", err) @@ -202,7 +205,7 @@ func (a *AccountTrie) Hash() common.Hash { return hash } -func (a *AccountTrie) hash() (common.Hash, error) { +func (a *accountTrie) hash() (common.Hash, error) { // If we haven't already hashed, we need to do so. if a.hasChanges { root, err := a.fw.getProposalHash(a.parentRoot, a.updateKeys, a.updateValues) @@ -218,7 +221,7 @@ func (a *AccountTrie) hash() (common.Hash, error) { // Commit returns the new root hash of the trie and a NodeSet containing all modified accounts and storage slots. // The format of the NodeSet is different than in go-ethereum's trie implementation due to Firewood's design. // This boolean is ignored, as it is a relic of the StateTrie implementation. -func (a *AccountTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { +func (a *accountTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { // Get the hash of the trie. hash, err := a.hash() if err != nil { @@ -238,31 +241,31 @@ func (a *AccountTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { // UpdateContractCode implements state.Trie. // Contract code is controlled by rawdb, so we don't need to do anything here. -func (*AccountTrie) UpdateContractCode(common.Address, common.Hash, []byte) error { +func (*accountTrie) UpdateContractCode(common.Address, common.Hash, []byte) error { return nil } // GetKey implements state.Trie. // This should not be used, since any user should not be accessing by raw key. -func (*AccountTrie) GetKey([]byte) []byte { +func (*accountTrie) GetKey([]byte) []byte { return nil } // NodeIterator implements state.Trie. // Firewood does not support iterating over internal nodes. -func (*AccountTrie) NodeIterator([]byte) (trie.NodeIterator, error) { +func (*accountTrie) NodeIterator([]byte) (trie.NodeIterator, error) { return nil, errors.New("NodeIterator not implemented for Firewood") } // Prove implements state.Trie. // Firewood does not yet support providing key proofs. -func (*AccountTrie) Prove([]byte, ethdb.KeyValueWriter) error { +func (*accountTrie) Prove([]byte, ethdb.KeyValueWriter) error { return errors.New("Prove not implemented for Firewood") } -func (a *AccountTrie) Copy() *AccountTrie { +func (a *accountTrie) Copy() *accountTrie { // Create a new AccountTrie with the same root and reader - newTrie := &AccountTrie{ + newTrie := &accountTrie{ fw: a.fw, parentRoot: a.parentRoot, root: a.root, diff --git a/graft/coreth/core/extstate/database_test.go b/graft/evm/firewood/hash_test.go similarity index 96% rename from graft/coreth/core/extstate/database_test.go rename to graft/evm/firewood/hash_test.go index a37b5b67d353..b160dbe7e41c 100644 --- a/graft/coreth/core/extstate/database_test.go +++ b/graft/evm/firewood/hash_test.go @@ -1,7 +1,7 @@ // Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. // See the file LICENSE for licensing terms. -package extstate +package firewood import ( "encoding/binary" @@ -19,9 +19,6 @@ import ( "github.com/ava-labs/libevm/triedb" "github.com/holiman/uint256" "github.com/stretchr/testify/require" - - "github.com/ava-labs/avalanchego/graft/coreth/triedb/firewood" - "github.com/ava-labs/avalanchego/graft/coreth/triedb/hashdb" ) const ( @@ -68,11 +65,10 @@ type merkleTrie struct { func newFuzzState(t *testing.T) *fuzzState { r := require.New(t) - hashState := NewDatabaseWithConfig( + hashState := state.NewDatabaseWithConfig( rawdb.NewMemoryDatabase(), - &triedb.Config{ - DBOverride: hashdb.Defaults.BackendConstructor, - }) + &triedb.Config{}, + ) ethRoot := types.EmptyRootHash hashTr, err := hashState.OpenTrie(ethRoot) r.NoError(err) @@ -81,9 +77,9 @@ func newFuzzState(t *testing.T) *fuzzState { }) firewoodMemdb := rawdb.NewMemoryDatabase() - fwCfg := firewood.Defaults // copy the defaults + fwCfg := Defaults // copy the defaults fwCfg.ChainDataDir = t.TempDir() // Use a temporary directory for the Firewood - firewoodState := NewDatabaseWithConfig( + firewoodState := state.NewDatabaseWithConfig( firewoodMemdb, &triedb.Config{ DBOverride: fwCfg.BackendConstructor, @@ -148,7 +144,7 @@ func (fs *fuzzState) commit() { } // HashDB/PathDB only allows updating the triedb if there have been changes. - if _, ok := tr.ethDatabase.TrieDB().Backend().(*firewood.Database); ok { + if _, ok := tr.ethDatabase.TrieDB().Backend().(*TrieDB); ok { triedbopt := stateconf.WithTrieDBUpdatePayload(common.Hash{byte(int64(fs.blockNumber - 1))}, common.Hash{byte(int64(fs.blockNumber))}) fs.require.NoError(tr.ethDatabase.TrieDB().Update(updatedRoot, tr.lastRoot, fs.blockNumber, mergedNodeSet, nil, triedbopt), "failed to update triedb in %s", tr.name) tr.lastRoot = updatedRoot diff --git a/graft/evm/firewood/state.go b/graft/evm/firewood/state.go new file mode 100644 index 000000000000..3328cef6763f --- /dev/null +++ b/graft/evm/firewood/state.go @@ -0,0 +1,54 @@ +// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. +// See the file LICENSE for licensing terms. + +package firewood + +import ( + "fmt" + + "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/state" +) + +var _ state.Database = (*stateAccessor)(nil) + +type stateAccessor struct { + state.Database + triedb *TrieDB +} + +func NewStateAccessor(db state.Database, fw *TrieDB) state.Database { + return &stateAccessor{ + Database: db, + triedb: fw, + } +} + +// OpenTrie opens the main account trie. +func (s *stateAccessor) OpenTrie(root common.Hash) (state.Trie, error) { + return newAccountTrie(root, s.triedb) +} + +// OpenStorageTrie opens a wrapped version of the account trie. +// +//nolint:revive // removing names loses context. +func (*stateAccessor) OpenStorageTrie(stateRoot common.Hash, addr common.Address, accountRoot common.Hash, self state.Trie) (state.Trie, error) { + accountTrie, ok := self.(*accountTrie) + if !ok { + return nil, fmt.Errorf("invalid account trie type: %T", self) + } + return newStorageTrie(accountTrie), nil +} + +// CopyTrie returns a deep copy of the given trie. +// It can be altered by the caller. +func (*stateAccessor) CopyTrie(t state.Trie) state.Trie { + switch t := t.(type) { + case *accountTrie: + return t.Copy() + case *storageTrie: + return nil // The storage trie just wraps the account trie, so we must re-open it separately. + default: + panic(fmt.Errorf("unknown trie type %T", t)) + } +} diff --git a/graft/coreth/triedb/firewood/storage_trie.go b/graft/evm/firewood/storage_trie.go similarity index 68% rename from graft/coreth/triedb/firewood/storage_trie.go rename to graft/evm/firewood/storage_trie.go index 9e6367183b16..54bad04aedda 100644 --- a/graft/coreth/triedb/firewood/storage_trie.go +++ b/graft/evm/firewood/storage_trie.go @@ -5,37 +5,40 @@ package firewood import ( "github.com/ava-labs/libevm/common" + "github.com/ava-labs/libevm/core/state" "github.com/ava-labs/libevm/trie/trienode" ) -type StorageTrie struct { - *AccountTrie +var _ state.Trie = (*storageTrie)(nil) + +type storageTrie struct { + *accountTrie } -// `NewStorageTrie` returns a wrapper around an `AccountTrie` since Firewood +// `newStorageTrie` returns a wrapper around an `accountTrie` since Firewood // does not require a separate storage trie. All changes are managed by the account trie. -func NewStorageTrie(accountTrie *AccountTrie) (*StorageTrie, error) { - return &StorageTrie{ - AccountTrie: accountTrie, - }, nil +func newStorageTrie(accountTrie *accountTrie) *storageTrie { + return &storageTrie{ + accountTrie: accountTrie, + } } // Actual commit is handled by the account trie. // Return the old storage root as if there was no change since Firewood // will manage the hash calculations without it. // All changes are managed by the account trie. -func (*StorageTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { +func (*storageTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { return common.Hash{}, nil, nil } // Firewood doesn't require tracking storage roots inside of an account. // They will be updated in place when hashing of the proposal takes place. -func (*StorageTrie) Hash() common.Hash { +func (*storageTrie) Hash() common.Hash { return common.Hash{} } // Copy should never be called on a storage trie, as it is just a wrapper around the account trie. // Each storage trie should be re-opened with the account trie separately. -func (*StorageTrie) Copy() *StorageTrie { +func (*storageTrie) Copy() *storageTrie { return nil } diff --git a/graft/subnet-evm/triedb/firewood/database.go b/graft/evm/firewood/triedb.go similarity index 85% rename from graft/subnet-evm/triedb/firewood/database.go rename to graft/evm/firewood/triedb.go index f95104260f35..12f615086db8 100644 --- a/graft/subnet-evm/triedb/firewood/database.go +++ b/graft/evm/firewood/triedb.go @@ -93,7 +93,7 @@ func (c Config) BackendConstructor(ethdb.Database) triedb.DBOverride { return db } -type Database struct { +type TrieDB struct { fwDisk *ffi.Database // The underlying Firewood database, used for storing proposals and revisions. proposalLock sync.RWMutex // proposalMap provides O(1) access by state root to all proposals stored in the proposalTree @@ -107,7 +107,7 @@ type Database struct { // New creates a new Firewood database with the given disk database and configuration. // Any error during creation will cause the program to exit. -func New(config Config) (*Database, error) { +func New(config Config) (*TrieDB, error) { firewoodDir := filepath.Join(config.ChainDataDir, firewoodDir) filePath := filepath.Join(firewoodDir, firewoodFileName) if err := validatePath(filePath); err != nil { @@ -135,7 +135,7 @@ func New(config Config) (*Database, error) { return nil, err } - return &Database{ + return &TrieDB{ fwDisk: fw, proposalMap: make(map[common.Hash][]*ProposalContext), proposalTree: &ProposalContext{ @@ -174,13 +174,13 @@ func validatePath(path string) error { // it must be overwritten to use something like: // `_, ok := db.(*Database); if !ok { return "" }` // to recognize the Firewood database. -func (*Database) Scheme() string { +func (*TrieDB) Scheme() string { return rawdb.HashScheme } // Initialized checks whether a non-empty genesis block has been written. -func (db *Database) Initialized(common.Hash) bool { - root, err := db.fwDisk.Root() +func (t *TrieDB) Initialized(common.Hash) bool { + root, err := t.fwDisk.Root() if err != nil { log.Error("firewood: error getting current root", "error", err) return false @@ -193,7 +193,7 @@ func (db *Database) Initialized(common.Hash) bool { // Update takes a root and a set of keys-values and creates a new proposal. // It will not be committed until the Commit method is called. // This function should be called even if there are no changes to the state to ensure proper tracking of block hashes. -func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint64, nodes *trienode.MergedNodeSet, _ *triestate.Set, opts ...stateconf.TrieDBUpdateOption) error { +func (t *TrieDB) Update(root common.Hash, parentRoot common.Hash, block uint64, nodes *trienode.MergedNodeSet, _ *triestate.Set, opts ...stateconf.TrieDBUpdateOption) error { // We require block hashes to be provided for all blocks in production. // However, many tests cannot reasonably provide a block hash for genesis, so we allow it to be omitted. parentHash, hash, ok := stateconf.ExtractTrieDBUpdatePayload(opts...) @@ -202,13 +202,13 @@ func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint6 } // The rest of the operations except key-value arranging must occur with a lock - db.proposalLock.Lock() - defer db.proposalLock.Unlock() + t.proposalLock.Lock() + defer t.proposalLock.Unlock() // Check if this proposal already exists. // During reorgs, we may have already created this proposal. // Additionally, we may have already created this proposal with a different block hash. - if existingProposals, ok := db.proposalMap[root]; ok { + if existingProposals, ok := t.proposalMap[root]; ok { for _, existing := range existingProposals { // If the block hash is already tracked, we can skip proposing this again. if _, exists := existing.Hashes[hash]; exists { @@ -226,7 +226,7 @@ func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint6 } keys, values := arrangeKeyValuePairs(nodes) // may return nil, nil if no changes - return db.propose(root, parentRoot, hash, parentHash, block, keys, values) + return t.propose(root, parentRoot, hash, parentHash, block, keys, values) } // propose creates a new proposal for every possible parent with the given keys and values. @@ -237,10 +237,10 @@ func (db *Database) Update(root common.Hash, parentRoot common.Hash, block uint6 // must be created and tracked. // // Should only be accessed with the proposal lock held. -func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash common.Hash, parentHash common.Hash, block uint64, keys [][]byte, values [][]byte) error { +func (t *TrieDB) propose(root common.Hash, parentRoot common.Hash, hash common.Hash, parentHash common.Hash, block uint64, keys [][]byte, values [][]byte) error { // Find the parent proposal with the correct hash. // We assume the number of proposals at a given root is small, so we can iterate through them. - for _, parentProposal := range db.proposalMap[parentRoot] { + for _, parentProposal := range t.proposalMap[parentRoot] { // If we know this proposal cannot be the parent, we can skip it. // Since the only possible block that won't have a parent hash is block 1, // and that will always be proposed from the database root, @@ -261,7 +261,7 @@ func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash commo Parent: parentProposal, } - db.proposalMap[root] = append(db.proposalMap[root], pCtx) + t.proposalMap[root] = append(t.proposalMap[root], pCtx) parentProposal.Children = append(parentProposal.Children, pCtx) return nil } @@ -269,14 +269,14 @@ func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash commo // Since we were unable to find a parent proposal with the given parent hash, // we must create a new proposal from the database root. // We must avoid the case in which we are reexecuting blocks upon startup, and haven't yet stored the parent block. - if _, exists := db.proposalTree.Hashes[parentHash]; db.proposalTree.Block != 0 && !exists { + if _, exists := t.proposalTree.Hashes[parentHash]; t.proposalTree.Block != 0 && !exists { return fmt.Errorf("firewood: parent hash %s not found for block %s at height %d", parentHash.Hex(), hash.Hex(), block) - } else if db.proposalTree.Root != parentRoot { - return fmt.Errorf("firewood: parent root %s does not match proposal tree root %s for root %s at height %d", parentRoot.Hex(), db.proposalTree.Root.Hex(), root.Hex(), block) + } else if t.proposalTree.Root != parentRoot { + return fmt.Errorf("firewood: parent root %s does not match proposal tree root %s for root %s at height %d", parentRoot.Hex(), t.proposalTree.Root.Hex(), root.Hex(), block) } log.Debug("firewood: proposing from database root", "root", root.Hex(), "height", block) - p, err := createProposal(db.fwDisk, root, keys, values) + p, err := createProposal(t.fwDisk, root, keys, values) if err != nil { return err } @@ -285,10 +285,10 @@ func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash commo Hashes: map[common.Hash]struct{}{hash: {}}, // This may be common.Hash{} for genesis blocks. Root: root, Block: block, - Parent: db.proposalTree, + Parent: t.proposalTree, } - db.proposalMap[root] = append(db.proposalMap[root], pCtx) - db.proposalTree.Children = append(db.proposalTree.Children, pCtx) + t.proposalMap[root] = append(t.proposalMap[root], pCtx) + t.proposalTree.Children = append(t.proposalTree.Children, pCtx) return nil } @@ -302,15 +302,15 @@ func (db *Database) propose(root common.Hash, parentRoot common.Hash, hash commo // // Afterward, we know that no other proposal at this height can be committed, so we can dereference all // children in the the other branches of the proposal tree. -func (db *Database) Commit(root common.Hash, report bool) error { +func (t *TrieDB) Commit(root common.Hash, report bool) error { // We need to lock the proposal tree to prevent concurrent writes. - db.proposalLock.Lock() - defer db.proposalLock.Unlock() + t.proposalLock.Lock() + defer t.proposalLock.Unlock() // Find the proposal with the given root. var pCtx *ProposalContext - for _, possible := range db.proposalMap[root] { - if possible.Parent.Root == db.proposalTree.Root && possible.Parent.Block == db.proposalTree.Block { + for _, possible := range t.proposalMap[root] { + if possible.Parent.Root == t.proposalTree.Root && possible.Parent.Block == t.proposalTree.Block { // We found the proposal with the correct parent. if pCtx != nil { // This should never happen, as we ensure that we don't create duplicate proposals in `propose`. @@ -326,17 +326,17 @@ func (db *Database) Commit(root common.Hash, report bool) error { start := time.Now() // Commit the proposal to the database. if err := pCtx.Proposal.Commit(); err != nil { - db.dereference(pCtx) // no longer committable + t.dereference(pCtx) // no longer committable return fmt.Errorf("firewood: error committing proposal %s: %w", root.Hex(), err) } ffiCommitCount.Inc(1) ffiCommitTimer.Inc(time.Since(start).Milliseconds()) ffiOutstandingProposals.Dec(1) // Now that the proposal is committed, we should clean up the proposal tree on return. - defer db.cleanupCommittedProposal(pCtx) + defer t.cleanupCommittedProposal(pCtx) // Assert that the root of the database matches the committed proposal root. - currentRoot, err := db.fwDisk.Root() + currentRoot, err := t.fwDisk.Root() if err != nil { return fmt.Errorf("firewood: error getting current root after commit: %w", err) } @@ -359,37 +359,37 @@ func (db *Database) Commit(root common.Hash, report bool) error { // Only used for metrics and Commit intervals in APIs. // This will be implemented in the firewood database eventually. // Currently, Firewood stores all revisions in disk and proposals in memory. -func (*Database) Size() (common.StorageSize, common.StorageSize) { +func (*TrieDB) Size() (common.StorageSize, common.StorageSize) { return 0, 0 } // Reference is a no-op. -func (*Database) Reference(common.Hash, common.Hash) {} +func (*TrieDB) Reference(common.Hash, common.Hash) {} // Dereference is a no-op since Firewood handles unused state roots internally. -func (*Database) Dereference(common.Hash) {} +func (*TrieDB) Dereference(common.Hash) {} // Firewood does not support this. -func (*Database) Cap(common.StorageSize) error { +func (*TrieDB) Cap(common.StorageSize) error { return nil } -func (db *Database) Close() error { - db.proposalLock.Lock() - defer db.proposalLock.Unlock() +func (t *TrieDB) Close() error { + t.proposalLock.Lock() + defer t.proposalLock.Unlock() // before closing, we must deference any outstanding proposals to free the // memory owned by firewood (outside of go's memory management) - for _, pCtx := range db.proposalTree.Children { - db.dereference(pCtx) + for _, pCtx := range t.proposalTree.Children { + t.dereference(pCtx) } - db.proposalMap = nil - db.proposalTree.Children = nil + t.proposalMap = nil + t.proposalTree.Children = nil // Close the database // This may block momentarily while finalizers for Firewood objects run. - return db.fwDisk.Close(context.Background()) + return t.fwDisk.Close(context.Background()) } // createProposal creates a new proposal from the given layer @@ -434,18 +434,18 @@ func createProposal(layer proposable, root common.Hash, keys, values [][]byte) ( // cleanupCommittedProposal dereferences the proposal and removes it from the proposal map. // It also recursively dereferences all children of the proposal. -func (db *Database) cleanupCommittedProposal(pCtx *ProposalContext) { +func (t *TrieDB) cleanupCommittedProposal(pCtx *ProposalContext) { start := time.Now() - oldChildren := db.proposalTree.Children - db.proposalTree = pCtx - db.proposalTree.Parent = nil + oldChildren := t.proposalTree.Children + t.proposalTree = pCtx + t.proposalTree.Parent = nil - db.removeProposalFromMap(pCtx) + t.removeProposalFromMap(pCtx) for _, childCtx := range oldChildren { // Don't dereference the recently commit proposal. if childCtx != pCtx { - db.dereference(childCtx) + t.dereference(childCtx) } } ffiCleanupTimer.Inc(time.Since(start).Milliseconds()) @@ -454,15 +454,15 @@ func (db *Database) cleanupCommittedProposal(pCtx *ProposalContext) { // Internally removes all references of the proposal from the database. // Should only be accessed with the proposal lock held. // Consumer must not be iterating the proposal map at this root. -func (db *Database) dereference(pCtx *ProposalContext) { +func (t *TrieDB) dereference(pCtx *ProposalContext) { // Base case: if there are children, we need to dereference them as well. for _, child := range pCtx.Children { - db.dereference(child) + t.dereference(child) } pCtx.Children = nil // Remove the proposal from the map. - db.removeProposalFromMap(pCtx) + t.removeProposalFromMap(pCtx) // Drop the proposal in the backend. if err := pCtx.Proposal.Drop(); err != nil { @@ -473,8 +473,8 @@ func (db *Database) dereference(pCtx *ProposalContext) { // removeProposalFromMap removes the proposal from the proposal map. // The proposal lock must be held when calling this function. -func (db *Database) removeProposalFromMap(pCtx *ProposalContext) { - rootList := db.proposalMap[pCtx.Root] +func (t *TrieDB) removeProposalFromMap(pCtx *ProposalContext) { + rootList := t.proposalMap[pCtx.Root] for i, p := range rootList { if p == pCtx { // pointer comparison - guaranteed to be unique rootList[i] = rootList[len(rootList)-1] @@ -484,24 +484,24 @@ func (db *Database) removeProposalFromMap(pCtx *ProposalContext) { } } if len(rootList) == 0 { - delete(db.proposalMap, pCtx.Root) + delete(t.proposalMap, pCtx.Root) } else { - db.proposalMap[pCtx.Root] = rootList + t.proposalMap[pCtx.Root] = rootList } } // Reader retrieves a node reader belonging to the given state root. // An error will be returned if the requested state is not available. -func (db *Database) Reader(root common.Hash) (database.Reader, error) { - if _, err := db.fwDisk.GetFromRoot(ffi.Hash(root), []byte{}); err != nil { +func (t *TrieDB) Reader(root common.Hash) (database.Reader, error) { + if _, err := t.fwDisk.GetFromRoot(ffi.Hash(root), []byte{}); err != nil { return nil, fmt.Errorf("firewood: unable to retrieve from root %s: %w", root.Hex(), err) } - return &reader{db: db, root: ffi.Hash(root)}, nil + return &reader{db: t, root: ffi.Hash(root)}, nil } // reader is a state reader of Database which implements the Reader interface. type reader struct { - db *Database + db *TrieDB root ffi.Hash // The root of the state this reader is reading. } @@ -521,19 +521,19 @@ func (reader *reader) Node(_ common.Hash, path []byte, _ common.Hash) ([]byte, e // getProposalHash calculates the hash if the set of keys and values are // proposed from the given parent root. -func (db *Database) getProposalHash(parentRoot common.Hash, keys, values [][]byte) (common.Hash, error) { +func (t *TrieDB) getProposalHash(parentRoot common.Hash, keys, values [][]byte) (common.Hash, error) { // This function only reads from existing tracked proposals, so we can use a read lock. - db.proposalLock.RLock() - defer db.proposalLock.RUnlock() + t.proposalLock.RLock() + defer t.proposalLock.RUnlock() var ( p *ffi.Proposal err error ) start := time.Now() - if db.proposalTree.Root == parentRoot { + if t.proposalTree.Root == parentRoot { // Propose from the database root. - p, err = db.fwDisk.Propose(keys, values) + p, err = t.fwDisk.Propose(keys, values) if err != nil { return common.Hash{}, fmt.Errorf("firewood: error proposing from root %s: %w", parentRoot.Hex(), err) } @@ -541,7 +541,7 @@ func (db *Database) getProposalHash(parentRoot common.Hash, keys, values [][]byt // Find any proposal with the given parent root. // Since we are only using the proposal to find the root hash, // we can use the first proposal found. - proposals, ok := db.proposalMap[parentRoot] + proposals, ok := t.proposalMap[parentRoot] if !ok || len(proposals) == 0 { return common.Hash{}, fmt.Errorf("firewood: no proposal found for parent root %s", parentRoot.Hex()) } diff --git a/graft/evm/go.mod b/graft/evm/go.mod index 855ca12e1a9a..0eb612cc9f9e 100644 --- a/graft/evm/go.mod +++ b/graft/evm/go.mod @@ -2,12 +2,67 @@ module github.com/ava-labs/avalanchego/graft/evm go 1.24.9 -require github.com/ava-labs/libevm v1.13.15-0.20251016142715-1bccf4f2ddb2 +require ( + github.com/ava-labs/firewood-go-ethhash/ffi v0.0.16 + github.com/ava-labs/libevm v1.13.15-0.20251016142715-1bccf4f2ddb2 + github.com/holiman/uint256 v1.2.4 + github.com/stretchr/testify v1.10.0 +) require ( - github.com/holiman/uint256 v1.2.4 // indirect + github.com/DataDog/zstd v1.4.5 // indirect + github.com/StackExchange/wmi v1.2.1 // indirect + github.com/VictoriaMetrics/fastcache v1.12.1 // indirect + github.com/beorn7/perks v1.0.1 // indirect + github.com/bits-and-blooms/bitset v1.10.0 // indirect + github.com/btcsuite/btcd/btcec/v2 v2.2.0 // indirect + github.com/cespare/xxhash/v2 v2.3.0 // indirect + github.com/cockroachdb/errors v1.8.1 // indirect + github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f // indirect + github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 // indirect + github.com/cockroachdb/redact v1.0.8 // indirect + github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 // indirect + github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 // indirect + github.com/consensys/bavard v0.1.13 // indirect + github.com/consensys/gnark-crypto v0.12.1 // indirect + github.com/crate-crypto/go-ipa v0.0.0-20231025140028-3c0104f4b233 // indirect + github.com/crate-crypto/go-kzg-4844 v1.0.0 // indirect + github.com/davecgh/go-spew v1.1.1 // indirect + github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 // indirect + github.com/ethereum/c-kzg-4844 v1.0.0 // indirect + github.com/gballet/go-verkle v0.1.1-0.20231031103413-a67434b50f46 // indirect + github.com/go-ole/go-ole v1.3.0 // indirect + github.com/gofrs/flock v0.8.1 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb // indirect + github.com/holiman/bloomfilter/v2 v2.0.3 // indirect + github.com/klauspost/compress v1.18.0 // indirect + github.com/kr/pretty v0.3.1 // indirect + github.com/kr/text v0.2.0 // indirect + github.com/mattn/go-runewidth v0.0.13 // indirect + github.com/mmcloughlin/addchain v0.4.0 // indirect + github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 // indirect + github.com/olekukonko/tablewriter v0.0.5 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/prometheus/client_golang v1.22.0 // indirect + github.com/prometheus/client_model v0.6.1 // indirect + github.com/prometheus/common v0.62.0 // indirect + github.com/prometheus/procfs v0.15.1 // indirect + github.com/rivo/uniseg v0.2.0 // indirect + github.com/rogpeppe/go-internal v1.10.0 // indirect + github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible // indirect + github.com/supranational/blst v0.3.14 // indirect + github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 // indirect + github.com/tklauser/go-sysconf v0.3.12 // indirect + github.com/tklauser/numcpus v0.6.1 // indirect golang.org/x/crypto v0.45.0 // indirect + golang.org/x/exp v0.0.0-20231110203233-9a3e6036ecaa // indirect + golang.org/x/sync v0.17.0 // indirect golang.org/x/sys v0.38.0 // indirect + google.golang.org/protobuf v1.36.5 // indirect + gopkg.in/yaml.v3 v3.0.1 // indirect + rsc.io/tmplfunc v0.0.3 // indirect ) replace github.com/ava-labs/avalanchego => ../../ diff --git a/graft/evm/go.sum b/graft/evm/go.sum index b59abce5d4bf..376ee75c0f30 100644 --- a/graft/evm/go.sum +++ b/graft/evm/go.sum @@ -1,8 +1,426 @@ +cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= +github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/CloudyKit/fastprinter v0.0.0-20170127035650-74b38d55f37a/go.mod h1:EFZQ978U7x8IRnstaskI3IysnWY5Ao3QgZUKOXlsAdw= +github.com/CloudyKit/jet v2.1.3-0.20180809161101-62edd43e4f88+incompatible/go.mod h1:HPYO+50pSWkPoj9Q/eq0aRGByCL6ScRlUmiEX5Zgm+w= +github.com/DataDog/zstd v1.4.5 h1:EndNeuB0l9syBZhut0wns3gV1hL8zX8LIu6ZiVHWLIQ= +github.com/DataDog/zstd v1.4.5/go.mod h1:1jcaCB/ufaK+sKp1NBhlGmpz41jOoPQ35bpF36t7BBo= +github.com/Joker/hpp v1.0.0/go.mod h1:8x5n+M1Hp5hC0g8okX3sR3vFQwynaX/UgSOM9MeBKzY= +github.com/Joker/jade v1.0.1-0.20190614124447-d475f43051e7/go.mod h1:6E6s8o2AE4KhCrqr6GRJjdC/gNfTdxkIXvuGZZda2VM= +github.com/Microsoft/go-winio v0.6.1 h1:9/kr64B9VUZrLm5YYwbGtUJnMgqWVOdUAXu6Migciow= +github.com/Microsoft/go-winio v0.6.1/go.mod h1:LRdKpFKfdobln8UmuiYcKPot9D2v6svN5+sAH+4kjUM= +github.com/Shopify/goreferrer v0.0.0-20181106222321-ec9c9a553398/go.mod h1:a1uqRtAwp2Xwc6WNPJEufxJ7fx3npB4UV/JOLmbu5I0= +github.com/StackExchange/wmi v1.2.1 h1:VIkavFPXSjcnS+O8yTq7NI32k0R5Aj+v39y29VYDOSA= +github.com/StackExchange/wmi v1.2.1/go.mod h1:rcmrprowKIVzvc+NUiLncP2uuArMWLCbu9SBzvHz7e8= +github.com/VictoriaMetrics/fastcache v1.12.1 h1:i0mICQuojGDL3KblA7wUNlY5lOK6a4bwt3uRKnkZU40= +github.com/VictoriaMetrics/fastcache v1.12.1/go.mod h1:tX04vaqcNoQeGLD+ra5pU5sWkuxnzWhEzLwhP9w653o= +github.com/ajg/form v1.5.1/go.mod h1:uL1WgH+h2mgNtvBq0339dVnzXdBETtL2LeUXaIv25UY= +github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156 h1:eMwmnE/GDgah4HI848JfFxHt+iPb26b4zyfspmqY0/8= +github.com/allegro/bigcache v1.2.1-0.20190218064605-e24eb225f156/go.mod h1:Cb/ax3seSYIx7SuZdm2G2xzfwmv3TPSk2ucNfQESPXM= +github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= +github.com/ava-labs/firewood-go-ethhash/ffi v0.0.16 h1:J+evKZFzlS85Y9AL8z6ThNFthdTIF0YzRT6MTm9S+y8= +github.com/ava-labs/firewood-go-ethhash/ffi v0.0.16/go.mod h1:hR/JSGXxST9B9olwu/NpLXHAykfAyNGfyKnYQqiiOeE= github.com/ava-labs/libevm v1.13.15-0.20251016142715-1bccf4f2ddb2 h1:hQ15IJxY7WOKqeJqCXawsiXh0NZTzmoQOemkWHz7rr4= github.com/ava-labs/libevm v1.13.15-0.20251016142715-1bccf4f2ddb2/go.mod h1:DqSotSn4Dx/UJV+d3svfW8raR+cH7+Ohl9BpsQ5HlGU= +github.com/aymerick/raymond v2.0.3-0.20180322193309-b565731e1464+incompatible/go.mod h1:osfaiScAUVup+UC9Nfq76eWqDhXlp+4UYaA8uhTBO6g= +github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= +github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= +github.com/bits-and-blooms/bitset v1.10.0 h1:ePXTeiPEazB5+opbv5fr8umg2R/1NlzgDsyepwsSr88= +github.com/bits-and-blooms/bitset v1.10.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= +github.com/btcsuite/btcd/btcec/v2 v2.2.0 h1:fzn1qaOt32TuLjFlkzYSsBC35Q3KUjT1SwPxiMSCF5k= +github.com/btcsuite/btcd/btcec/v2 v2.2.0/go.mod h1:U7MHm051Al6XmscBQ0BoNydpOTsFAn707034b5nY8zU= +github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1 h1:q0rUy8C/TYNBQS1+CGKw68tLOFYSNEs0TFnxxnS9+4U= +github.com/btcsuite/btcd/chaincfg/chainhash v1.0.1/go.mod h1:7SFka0XMvUgj3hfZtydOrQY2mwhPclbT2snogU7SQQc= +github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= +github.com/cespare/xxhash/v2 v2.2.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/cespare/xxhash/v2 v2.3.0 h1:UL815xU9SqsFlibzuggzjXhog7bL6oX9BbNZnL2UFvs= +github.com/cespare/xxhash/v2 v2.3.0/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= +github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= +github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= +github.com/cockroachdb/datadriven v1.0.0/go.mod h1:5Ib8Meh+jk1RlHIXej6Pzevx/NLlNvQB9pmSBZErGA4= +github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f h1:otljaYPt5hWxV3MUfO5dFPFiOXg9CyG5/kCfayTqsJ4= +github.com/cockroachdb/datadriven v1.0.3-0.20230413201302-be42291fc80f/go.mod h1:a9RdTaap04u637JoCzcUoIcDmvwSUtcUFtT/C3kJlTU= +github.com/cockroachdb/errors v1.6.1/go.mod h1:tm6FTP5G81vwJ5lC0SizQo374JNCOPrHyXGitRJoDqM= +github.com/cockroachdb/errors v1.8.1 h1:A5+txlVZfOqFBDa4mGz2bUWSp0aHElvHX2bKkdbQu+Y= +github.com/cockroachdb/errors v1.8.1/go.mod h1:qGwQn6JmZ+oMjuLwjWzUNqblqk0xl4CVV3SQbGwK7Ac= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY= +github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593 h1:aPEJyR4rPBvDmeyi+l/FS/VtA00IWvjeFvjen1m1l1A= +github.com/cockroachdb/pebble v0.0.0-20230928194634-aa077af62593/go.mod h1:6hk1eMY/u5t+Cf18q5lFMUA1Rc+Sm5I6Ra1QuPyxXCo= +github.com/cockroachdb/redact v1.0.8 h1:8QG/764wK+vmEYoOlfobpe12EQcS81ukx/a4hdVMxNw= +github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2 h1:IKgmqgMQlVJIZj19CdocBeSfSaiCbEBZGKODaixqtHM= +github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2/go.mod h1:8BT+cPK6xvFOcRlk0R8eg+OTkcqI6baNH4xAkpiYVvQ= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06 h1:zuQyyAKVxetITBuuhv3BI9cMrmStnpT18zmgmTxunpo= +github.com/cockroachdb/tokenbucket v0.0.0-20230807174530-cc333fc44b06/go.mod h1:7nc4anLGjupUW/PeY5qiNYsdNXj7zopG+eqsS7To5IQ= +github.com/codegangsta/inject v0.0.0-20150114235600-33e0aa1cb7c0/go.mod h1:4Zcjuz89kmFXt9morQgcfYZAYZ5n8WHjt81YYWIwtTM= +github.com/consensys/bavard v0.1.13 h1:oLhMLOFGTLdlda/kma4VOJazblc7IM5y5QPd2A/YjhQ= +github.com/consensys/bavard v0.1.13/go.mod h1:9ItSMtA/dXMAiL7BG6bqW2m3NdSEObYWoH223nGHukI= +github.com/consensys/gnark-crypto v0.12.1 h1:lHH39WuuFgVHONRl3J0LRBtuYdQTumFSDtJF7HpyG8M= +github.com/consensys/gnark-crypto v0.12.1/go.mod h1:v2Gy7L/4ZRosZ7Ivs+9SfUDr0f5UlG+EM5t7MPHiLuY= +github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= +github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= +github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE= +github.com/crate-crypto/go-ipa v0.0.0-20231025140028-3c0104f4b233 h1:d28BXYi+wUpz1KBmiF9bWrjEMacUEREV6MBi2ODnrfQ= +github.com/crate-crypto/go-ipa v0.0.0-20231025140028-3c0104f4b233/go.mod h1:geZJZH3SzKCqnz5VT0q/DyIG/tvu/dZk+VIfXicupJs= +github.com/crate-crypto/go-kzg-4844 v1.0.0 h1:TsSgHwrkTKecKJ4kadtHi4b3xHW5dCFUDFnUp1TsawI= +github.com/crate-crypto/go-kzg-4844 v1.0.0/go.mod h1:1kMhvPgI0Ky3yIa+9lFySEBUBXkYxeOi8ZF1sYioxhc= +github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E= +github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= +github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= +github.com/deckarep/golang-set/v2 v2.1.0 h1:g47V4Or+DUdzbs8FxCCmgb6VYd+ptPAngjM6dtGktsI= +github.com/deckarep/golang-set/v2 v2.1.0/go.mod h1:VAky9rY/yGXJOLEDv3OMci+7wtDpOF4IN+y82NBOac4= +github.com/decred/dcrd/crypto/blake256 v1.0.0 h1:/8DMNYp9SGi5f0w7uCm6d6M4OU2rGFK09Y2A4Xv7EE0= +github.com/decred/dcrd/crypto/blake256 v1.0.0/go.mod h1:sQl2p6Y26YV+ZOcSTP6thNdn47hh8kt6rqSlvmrXFAc= +github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1 h1:YLtO71vCjJRCBcrPMtQ9nqBsqpA1m5sE92cU+pd5Mcc= +github.com/decred/dcrd/dcrec/secp256k1/v4 v4.0.1/go.mod h1:hyedUtir6IdtD/7lIxGeCxkaw7y45JueMRL4DIyJDKs= +github.com/dgraph-io/badger v1.6.0/go.mod h1:zwt7syl517jmP8s94KqSxTlM6IMsdhYy6psNgSztDR4= +github.com/dgrijalva/jwt-go v3.2.0+incompatible/go.mod h1:E3ru+11k8xSBh+hMPgOLZmtrrCbhqsmaPHjLKYnJCaQ= +github.com/dgryski/go-farm v0.0.0-20190423205320-6a90982ecee2/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= +github.com/eknkc/amber v0.0.0-20171010120322-cdade1c07385/go.mod h1:0vRUJqYpeSZifjYj7uP3BG/gKcuzL9xWVV/Y+cK33KM= +github.com/envoyproxy/go-control-plane v0.9.0/go.mod h1:YTl/9mNaCwkRvm6d1a2C3ymFceY/DCBVvsKhRF0iEA4= +github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1mIlRU8Am5FuJP05cCM98= +github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= +github.com/etcd-io/bbolt v1.3.3/go.mod h1:ZF2nL25h33cCyBtcyWeZ2/I3HQOfTP+0PIEvHjkjCrw= +github.com/ethereum/c-kzg-4844 v1.0.0 h1:0X1LBXxaEtYD9xsyj9B9ctQEZIpnvVDeoBx8aHEwTNA= +github.com/ethereum/c-kzg-4844 v1.0.0/go.mod h1:VewdlzQmpT5QSrVhbBuGoCdFJkpaJlO1aQputP83wc0= +github.com/fasthttp-contrib/websocket v0.0.0-20160511215533-1f3b11f56072/go.mod h1:duJ4Jxv5lDcvg4QuQr0oowTf7dz4/CR8NtyCooz9HL8= +github.com/fatih/structs v1.1.0/go.mod h1:9NiDSp5zOcgEDl+j00MP/WkGVPOlPRLejGD8Ga6PJ7M= +github.com/flosch/pongo2 v0.0.0-20190707114632-bbf5a6c351f4/go.mod h1:T9YF2M40nIgbVgp3rreNmTged+9HrbNTIQf1PsaIiTA= +github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= +github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= +github.com/fsnotify/fsnotify v1.6.0 h1:n+5WquG0fcWoWp6xPWfHdbskMCQaFnG6PfBrh1Ky4HY= +github.com/fsnotify/fsnotify v1.6.0/go.mod h1:sl3t1tCWJFWoRz9R8WJCbQihKKwmorjAbSClcnxKAGw= +github.com/gavv/httpexpect v2.0.0+incompatible/go.mod h1:x+9tiU1YnrOvnB725RkpoLv1M62hOWzwo5OXotisrKc= +github.com/gballet/go-verkle v0.1.1-0.20231031103413-a67434b50f46 h1:BAIP2GihuqhwdILrV+7GJel5lyPV3u1+PgzrWLc0TkE= +github.com/gballet/go-verkle v0.1.1-0.20231031103413-a67434b50f46/go.mod h1:QNpY22eby74jVhqH4WhDLDwxc/vqsern6pW+u2kbkpc= +github.com/gin-contrib/sse v0.0.0-20190301062529-5545eab6dad3/go.mod h1:VJ0WA2NBN22VlZ2dKZQPAPnyWw5XTlK1KymzLKsr59s= +github.com/gin-gonic/gin v1.4.0/go.mod h1:OW2EZn3DO8Ln9oIKOvM++LBO+5UPHJJDH72/q/3rZdM= +github.com/go-check/check v0.0.0-20180628173108-788fd7840127/go.mod h1:9ES+weclKsC9YodN5RgxqK/VD9HM9JsCSh7rNhMZE98= +github.com/go-errors/errors v1.0.1 h1:LUHzmkK3GUKUrL/1gfBUxAHzcev3apQlezX/+O7ma6w= +github.com/go-errors/errors v1.0.1/go.mod h1:f4zRHt4oKfwPJE5k8C9vpYG+aDHdBFUsgrm6/TyX73Q= +github.com/go-martini/martini v0.0.0-20170121215854-22fa46961aab/go.mod h1:/P9AEU963A2AYjv4d1V5eVL1CQbEJq6aCNHDDjibzu8= +github.com/go-ole/go-ole v1.2.5/go.mod h1:pprOEPIfldk/42T2oK7lQ4v4JSDwmV0As9GaiUsvbm0= +github.com/go-ole/go-ole v1.3.0 h1:Dt6ye7+vXGIKZ7Xtk4s6/xVdGDQynvom7xCFEdWr6uE= +github.com/go-ole/go-ole v1.3.0/go.mod h1:5LS6F96DhAwUc7C+1HLexzMXY1xGRSryjyPPKW6zv78= +github.com/gobwas/httphead v0.0.0-20180130184737-2c6c146eadee/go.mod h1:L0fX3K22YWvt/FAX9NnzrNzcI4wNYi9Yku4O0LKYflo= +github.com/gobwas/pool v0.2.0/go.mod h1:q8bcK0KcYlCgd9e7WYLm9LpyS+YeLd8JVDW6WezmKEw= +github.com/gobwas/ws v1.0.2/go.mod h1:szmBTxLgaFppYjEmNtny/v3w89xOydFnnZMcgRRu/EM= +github.com/gofrs/flock v0.8.1 h1:+gYjHKf32LDeiEEFhQaotPbLuUXjY5ZqxKgXy7n59aw= +github.com/gofrs/flock v0.8.1/go.mod h1:F1TvTiK9OcQqauNUHlbJvyl9Qa1QvF/gOUDKA14jxHU= +github.com/gogo/googleapis v0.0.0-20180223154316-0cd9801be74a/go.mod h1:gf4bu3Q80BeJ6H1S1vYPm8/ELATdvryBaNFGgqEef3s= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.3.1/go.mod h1:SlYgWuQ5SjCEi6WLHjHCa1yvBfUnHcTbrrZtXPKa29o= +github.com/gogo/protobuf v1.3.2 h1:Ov1cvc58UF3b5XjBnZv7+opcTcQFZebYjWzi34vdm4Q= +github.com/gogo/protobuf v1.3.2/go.mod h1:P1XiOD3dCwIKUDQYPy72D8LYyHL2YPYrpS2s69NZV8Q= +github.com/gogo/status v1.1.0/go.mod h1:BFv9nrluPLmrS0EmGVvLaPNmRosr9KapBYd5/hpY1WM= +github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b/go.mod h1:SBH7ygxi8pfUlaOkMMuAQtPIUF8ecWP5IEl/CR7VP2Q= +github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= +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= +github.com/golang/protobuf v1.3.2/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= +github.com/golang/protobuf v1.3.3/go.mod h1:vzj43D7+SQXF/4pzW/hwtAqwc6iTitCiVSaWz5lYuqw= +github.com/golang/protobuf v1.4.0-rc.1/go.mod h1:ceaxUfeHdC40wWswd/P6IGgMaK3YpKi5j83Wpe3EHw8= +github.com/golang/protobuf v1.4.0-rc.1.0.20200221234624-67d41d38c208/go.mod h1:xKAWHe0F5eneWXFV3EuXVDTCmh+JuBKY0li0aMyXATA= +github.com/golang/protobuf v1.4.0-rc.2/go.mod h1:LlEzMj4AhA7rCAGe4KMBDvJI+AwstrUpVNzEA03Pprs= +github.com/golang/protobuf v1.4.0-rc.4.0.20200313231945-b860323f09d0/go.mod h1:WU3c8KckQ9AFe+yFwt9sWVRKCVIyN9cPHBJSNnbL67w= +github.com/golang/protobuf v1.4.0/go.mod h1:jodUvKwWbYaEsadDk5Fwe5c77LiNKVO9IDvqG2KuDX0= +github.com/golang/protobuf v1.4.2/go.mod h1:oDoupMAO8OvCJWAcko0GGGIgR6R6ocIYbsSw735rRwI= +github.com/golang/snappy v0.0.4/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb h1:PBC98N2aIaM3XXiurYmW7fx4GZkL8feAMVq7nEjURHk= +github.com/golang/snappy v0.0.5-0.20220116011046-fa5810519dcb/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= +github.com/gomodule/redigo v1.7.1-0.20190724094224-574c33c3df38/go.mod h1:B4C85qUVwatsJoIUNIfCRsp7qO0iAmpGFZ4EELWSbC4= +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= +github.com/google/go-cmp v0.3.1/go.mod h1:8QqcDgzrUqlUb/G2PQTWiueGozuR1884gddMywk6iLU= +github.com/google/go-cmp v0.4.0/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= +github.com/google/go-cmp v0.7.0 h1:wk8382ETsv4JYUZwIsn6YpYiWiBsYLSJiTsyBybVuN8= +github.com/google/go-cmp v0.7.0/go.mod h1:pXiqmnSA92OHEEa9HXL2W4E7lf9JzCmGVUdgjX3N/iU= +github.com/google/go-querystring v1.0.0/go.mod h1:odCYkC5MyYFN7vkCjXpyrEuKhc/BUO6wN/zVPAxq5ck= +github.com/google/subcommands v1.2.0/go.mod h1:ZjhPrFU+Olkh9WazFPsl27BQ4UPiG37m3yTrtFlrHVk= +github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= +github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= +github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= +github.com/hashicorp/go-version v1.2.0/go.mod h1:fltr4n8CU8Ke44wwGCBoEymUuxUHl09ZGVZPK5anwXA= +github.com/hashicorp/hcl v1.0.0/go.mod h1:E5yfLk+7swimpb2L/Alb/PJmXilQ/rhwaUYs4T20WEQ= +github.com/holiman/bloomfilter/v2 v2.0.3 h1:73e0e/V0tCydx14a0SCYS/EWCxgwLZ18CZcZKVu0fao= +github.com/holiman/bloomfilter/v2 v2.0.3/go.mod h1:zpoh+gs7qcpqrHr3dB55AMiJwo0iURXE7ZOP9L9hSkA= github.com/holiman/uint256 v1.2.4 h1:jUc4Nk8fm9jZabQuqr2JzednajVmBpC+oiTiXZJEApU= github.com/holiman/uint256 v1.2.4/go.mod h1:EOMSn4q6Nyt9P6efbI3bueV4e1b3dGlUCXeiRV4ng7E= +github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= +github.com/hydrogen18/memlistener v0.0.0-20141126152155-54553eb933fb/go.mod h1:qEIFzExnS6016fRpRfxrExeVn2gbClQA99gQhnIcdhE= +github.com/imkira/go-interpol v1.1.0/go.mod h1:z0h2/2T3XF8kyEPpRgJ3kmNv+C43p+I/CoI+jC3w2iA= +github.com/inconshreveable/mousetrap v1.0.0/go.mod h1:PxqpIevigyE2G7u3NXJIT2ANytuPF1OarO4DADm73n8= +github.com/iris-contrib/blackfriday v2.0.0+incompatible/go.mod h1:UzZ2bDEoaSGPbkg6SAB4att1aAwTmVIx/5gCVqeyUdI= +github.com/iris-contrib/go.uuid v2.0.0+incompatible/go.mod h1:iz2lgM/1UnEf1kP0L/+fafWORmlnuysV2EMP8MW+qe0= +github.com/iris-contrib/i18n v0.0.0-20171121225848-987a633949d0/go.mod h1:pMCz62A0xJL6I+umB2YTlFRwWXaDFA0jy+5HzGiJjqI= +github.com/iris-contrib/schema v0.0.1/go.mod h1:urYA3uvUNG1TIIjOSCzHr9/LmbQo8LrOcOqfqxa4hXw= +github.com/json-iterator/go v1.1.6/go.mod h1:+SdeFBvtyEkXs7REEP0seUULqWtbJapLOCVDaaPEHmU= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/juju/errors v0.0.0-20181118221551-089d3ea4e4d5/go.mod h1:W54LbzXuIE0boCoNJfwqpmkKJ1O4TCTZMetAt6jGk7Q= +github.com/juju/loggo v0.0.0-20180524022052-584905176618/go.mod h1:vgyd7OREkbtVEN/8IXZe5Ooef3LQePvuBm9UWj6ZL8U= +github.com/juju/testing v0.0.0-20180920084828-472a3e8b2073/go.mod h1:63prj8cnj0tU0S9OHjGJn+b1h0ZghCndfnbQolrYTwA= +github.com/k0kubun/colorstring v0.0.0-20150214042306-9440f1994b88/go.mod h1:3w7q1U84EfirKl04SVQ/s7nPm1ZPhiXd34z40TNz36k= +github.com/kataras/golog v0.0.9/go.mod h1:12HJgwBIZFNGL0EJnMRhmvGA0PQGx8VFwrZtM4CqbAk= +github.com/kataras/iris/v12 v12.0.1/go.mod h1:udK4vLQKkdDqMGJJVd/msuMtN6hpYJhg/lSzuxjhO+U= +github.com/kataras/neffos v0.0.10/go.mod h1:ZYmJC07hQPW67eKuzlfY7SO3bC0mw83A3j6im82hfqw= +github.com/kataras/pio v0.0.0-20190103105442-ea782b38602d/go.mod h1:NV88laa9UiiDuX9AhMbDPkGYSPugBOV6yTZB1l2K9Z0= +github.com/kisielk/errcheck v1.2.0/go.mod h1:/BMXB+zMLi60iA8Vv6Ksmxu/1UDYcXs4uQLJ+jE2L00= +github.com/kisielk/errcheck v1.5.0/go.mod h1:pFxgyoBC7bSaBwPgfKdkLd5X25qrDl4LWUI2bnpBCr8= +github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= +github.com/klauspost/compress v1.8.2/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.9.0/go.mod h1:RyIbtBH6LamlWaDj8nUwkbUhJ87Yi3uG0guNDohfE1A= +github.com/klauspost/compress v1.18.0 h1:c/Cqfb0r+Yi+JtIEq73FWXVkRonBlf0CRNYc8Zttxdo= +github.com/klauspost/compress v1.18.0/go.mod h1:2Pp+KzxcywXVXMr50+X0Q/Lsb43OQHYWRCY2AiWywWQ= +github.com/klauspost/cpuid v1.2.1/go.mod h1:Pj4uuM528wm8OyEC2QMXAi2YiTZ96dNQPGgoMS4s3ek= +github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= +github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= +github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= +github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= +github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= +github.com/kylelemons/godebug v1.1.0 h1:RPNrshWIDI6G2gRW9EHilWtl7Z6Sb1BR0xunSBf0SNc= +github.com/kylelemons/godebug v1.1.0/go.mod h1:9/0rRGxNHcop5bhtWyNeEfOS8JIWk580+fNqagV/RAw= +github.com/labstack/echo/v4 v4.1.11/go.mod h1:i541M3Fj6f76NZtHSj7TXnyM8n2gaodfvfxNnFqi74g= +github.com/labstack/gommon v0.3.0/go.mod h1:MULnywXg0yavhxWKc+lOruYdAhDwPK9wf0OL7NoOu+k= +github.com/leanovate/gopter v0.2.9 h1:fQjYxZaynp97ozCzfOyOuAGOU4aU/z37zf/tOujFk7c= +github.com/leanovate/gopter v0.2.9/go.mod h1:U2L/78B+KVFIx2VmW6onHJQzXtFb+p5y3y2Sh+Jxxv8= +github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= +github.com/mattn/go-colorable v0.1.2/go.mod h1:U0ppj6V5qS13XJ6of8GYAs25YV2eR4EVcfRqFIhoBtE= +github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.8/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.9/go.mod h1:YNRxwqDuOph6SZLI9vUUz6OYw3QyUt7WiY2yME+cCiQ= +github.com/mattn/go-runewidth v0.0.9/go.mod h1:H031xJmbD/WCDINGzjvQ9THkh0rPKHF+m2gUSrubnMI= +github.com/mattn/go-runewidth v0.0.13 h1:lTGmDsbAYt5DmK6OnoV7EuIF1wEIFAcxld6ypU4OSgU= +github.com/mattn/go-runewidth v0.0.13/go.mod h1:Jdepj2loyihRzMpdS35Xk/zdY8IAYHsh153qUoGf23w= +github.com/mattn/goveralls v0.0.2/go.mod h1:8d1ZMHsd7fW6IRPKQh46F2WRpyib5/X4FOpevwGNQEw= +github.com/mediocregopher/mediocre-go-lib v0.0.0-20181029021733-cb65787f37ed/go.mod h1:dSsfyI2zABAdhcbvkXqgxOxrCsbYeHCPgrZkku60dSg= +github.com/mediocregopher/radix/v3 v3.3.0/go.mod h1:EmfVyvspXz1uZEyPBMyGK+kjWiKQGvsUt6O3Pj+LDCQ= +github.com/microcosm-cc/bluemonday v1.0.2/go.mod h1:iVP4YcDBq+n/5fb23BhYFvIMq/leAFZyRl6bYmGDlGc= +github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= +github.com/mitchellh/mapstructure v1.1.2/go.mod h1:FVVH3fgwuzCH5S8UJGiWEs2h04kUh9fWfEaFds41c1Y= +github.com/mmcloughlin/addchain v0.4.0 h1:SobOdjm2xLj1KkXN5/n0xTIWyZA2+s99UCY1iPfkHRY= +github.com/mmcloughlin/addchain v0.4.0/go.mod h1:A86O+tHqZLMNO4w6ZZ4FlVQEadcoqkyU72HC5wJ4RlU= +github.com/mmcloughlin/profile v0.1.1/go.mod h1:IhHD7q1ooxgwTgjxQYkACGA77oFTDdFVejUS1/tS/qU= +github.com/modern-go/concurrent v0.0.0-20180306012644-bacd9c7ef1dd/go.mod h1:6dJC0mAP4ikYIbvyc7fijjWJddQyLn8Ig3JB5CqoB9Q= +github.com/modern-go/reflect2 v1.0.1/go.mod h1:bx2lNnkwVCuqBIxFjflWJWanXIb3RllmbCylyMrvgv0= +github.com/moul/http2curl v1.0.0/go.mod h1:8UbvGypXm98wA/IqH45anm5Y2Z6ep6O31QGOAZ3H0fQ= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822 h1:C3w9PqII01/Oq1c1nUAm88MOHcQC9l5mIlSMApZMrHA= +github.com/munnerz/goautoneg v0.0.0-20191010083416-a7dc8b61c822/go.mod h1:+n7T8mK8HuQTcFwEeznm/DIxMOiR9yIdICNftLE1DvQ= +github.com/nats-io/nats.go v1.8.1/go.mod h1:BrFz9vVn0fU3AcH9Vn4Kd7W0NpJ651tD5omQ3M8LwxM= +github.com/nats-io/nkeys v0.0.2/go.mod h1:dab7URMsZm6Z/jp9Z5UGa87Uutgc2mVpXLC4B7TDb/4= +github.com/nats-io/nuid v1.0.1/go.mod h1:19wcPz3Ph3q0Jbyiqsd0kePYG7A95tJPxeL+1OSON2c= +github.com/nxadm/tail v1.4.4 h1:DQuhQpB1tVlglWS2hLQ5OV6B5r8aGxSrPc5Qo6uTN78= +github.com/nxadm/tail v1.4.4/go.mod h1:kenIhsEOeOJmVchQTgglprH7qJGnHDVpk1VPCcaMI8A= +github.com/olekukonko/tablewriter v0.0.5 h1:P2Ga83D34wi1o9J6Wh1mRuqd4mF/x/lgBS7N7AbDhec= +github.com/olekukonko/tablewriter v0.0.5/go.mod h1:hPp6KlRPjbx+hW8ykQs1w3UBbZlj6HuIJcUGPhkA7kY= +github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.12.1/go.mod h1:zj2OWP4+oCPe1qIXoGWkgMRwljMUYCdkwsT2108oapk= +github.com/onsi/ginkgo v1.13.0/go.mod h1:+REjRxOmWfHCjfv9TTWB1jD1Frx4XydAD3zm1lskyM0= +github.com/onsi/ginkgo v1.14.0 h1:2mOpI4JVVPBN+WQRa0WKH2eXR+Ey+uK4n7Zj0aYpIQA= +github.com/onsi/ginkgo v1.14.0/go.mod h1:iSB4RoI2tjJc9BBv4NKIKWKya62Rps+oPG/Lv9klQyY= +github.com/onsi/gomega v1.7.1/go.mod h1:XdKZgCCFLUoM/7CFJVPcG8C1xQ1AJ0vpAezJrB7JYyY= +github.com/onsi/gomega v1.10.1 h1:o0+MgICZLuZ7xjH7Vx6zS/zcu93/BEp1VwkIW1mEXCE= +github.com/onsi/gomega v1.10.1/go.mod h1:iN09h71vgCQne3DLsj+A5owkum+a2tYe+TOCB1ybHNo= +github.com/pelletier/go-toml v1.2.0/go.mod h1:5z9KED0ma1S8pY6P1sdut58dfprrGBbd/94hg7ilaic= +github.com/pingcap/errors v0.11.4 h1:lFuQV/oaUMGcD2tqt+01ROSmJs75VG1ToEOkZIZ4nE4= +github.com/pingcap/errors v0.11.4/go.mod h1:Oi8TUi2kEtXXLMJk9l1cGmz20kV3TaQ0usTwv5KuLY8= +github.com/pkg/diff v0.0.0-20210226163009-20ebb0f2a09e/go.mod h1:pJLUxLENpZxwdsKMEsNbx1VGcRFpLqf3715MtcvvzbA= +github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pkg/errors v0.9.1 h1:FEBLx1zS214owpjy7qsBeixbURkuhQAwrK5UwLGTwt4= +github.com/pkg/errors v0.9.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= +github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZbAQM= +github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= +github.com/prometheus/client_golang v1.22.0 h1:rb93p9lokFEsctTys46VnV1kLCDpVZ0a/Y92Vm0Zc6Q= +github.com/prometheus/client_golang v1.22.0/go.mod h1:R7ljNsLXhuQXYZYtw6GAE9AZg8Y7vEW5scdCXrWRXC0= +github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= +github.com/prometheus/client_model v0.6.1 h1:ZKSh/rekM+n3CeS952MLRAdFwIKqeY8b62p8ais2e9E= +github.com/prometheus/client_model v0.6.1/go.mod h1:OrxVMOVHjw3lKMa8+x6HeMGkHMQyHDk9E3jmP2AmGiY= +github.com/prometheus/common v0.62.0 h1:xasJaQlnWAeyHdUBeGjXmutelfJHWMRr+Fg4QszZ2Io= +github.com/prometheus/common v0.62.0/go.mod h1:vyBcEuLSvWos9B1+CyL7JZ2up+uFzXhkqml0W5zIY1I= +github.com/prometheus/procfs v0.15.1 h1:YagwOFzUgYfKKHX6Dr+sHT7km/hxC76UB0learggepc= +github.com/prometheus/procfs v0.15.1/go.mod h1:fB45yRUv8NstnjriLhBQLuOUt+WW4BsoGhij/e3PBqk= +github.com/rivo/uniseg v0.2.0 h1:S1pD9weZBuJdFmowNwbpi7BJ8TNftyUImj/0WQi72jY= +github.com/rivo/uniseg v0.2.0/go.mod h1:J6wj4VEh+S6ZtnVlnTBMWIodfgj8LQOQFoIToxlJtxc= +github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs= +github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= +github.com/russross/blackfriday v1.5.2/go.mod h1:JO/DiYxRf+HjHt06OyowR9PTA263kcR/rfWxYHBV53g= +github.com/ryanuber/columnize v2.1.0+incompatible/go.mod h1:sm1tb6uqfes/u+d4ooFouqFdy9/2g9QGwK3SQygK0Ts= +github.com/sclevine/agouti v3.0.0+incompatible/go.mod h1:b4WX9W9L1sfQKXeJf1mUTLZKJ48R1S7H23Ji7oFO5Bw= +github.com/sergi/go-diff v1.1.0/go.mod h1:STckp+ISIX8hZLjrqAeVduY0gWCT9IjLuqbuNXdaHfM= +github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible h1:Bn1aCHHRnjv4Bl16T8rcaFjYSrGrIZvpiGO6P3Q4GpU= +github.com/shirou/gopsutil v3.21.4-0.20210419000835-c7a38de76ee5+incompatible/go.mod h1:5b4v6he4MtMOwMlS0TUMTu2PcXUg8+E1lC7eC3UO/RA= +github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= +github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/goconvey v1.6.4/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= +github.com/spf13/afero v1.1.2/go.mod h1:j4pytiNVoe2o6bmDsKpLACNPDBIoEAkihy7loJ1B0CQ= +github.com/spf13/cast v1.3.0/go.mod h1:Qx5cxh0v+4UWYiBimWS+eyWzqEqokIECu5etghLkUJE= +github.com/spf13/cobra v0.0.5/go.mod h1:3K3wKZymM7VvHMDS9+Akkh4K60UwM26emMESw8tLCHU= +github.com/spf13/jwalterweatherman v1.0.0/go.mod h1:cQK4TGJAtQXfYWX+Ddv3mKDzgVb68N+wFjFa4jdeBTo= +github.com/spf13/pflag v1.0.3/go.mod h1:DYY7MBk1bdzusC3SYhjObp+wFpr4gzcvqqNjLnInEg4= +github.com/spf13/viper v1.3.2/go.mod h1:ZiWeW+zYFKm7srdB9IoDzzZXaJaI5eL9QjNiN/DMA2s= +github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= +github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= +github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= +github.com/stretchr/testify v1.4.0/go.mod h1:j7eGeouHqKxXV5pUuKE4zz7dFj8WfuZ+81PSLYec5m4= +github.com/stretchr/testify v1.10.0 h1:Xv5erBjTwe/5IxqUQTdXv5kgmIvbHo3QQyRwhJsOfJA= +github.com/stretchr/testify v1.10.0/go.mod h1:r2ic/lqez/lEtzL7wO/rwa5dbSLXVDPFyf8C91i36aY= +github.com/supranational/blst v0.3.14 h1:xNMoHRJOTwMn63ip6qoWJ2Ymgvj7E2b9jY2FAwY+qRo= +github.com/supranational/blst v0.3.14/go.mod h1:jZJtfjgudtNl4en1tzwPIV3KjUnQUvG3/j+w+fVonLw= +github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7 h1:epCh84lMvA70Z7CTTCmYQn2CKbY8j86K7/FAIr141uY= +github.com/syndtr/goleveldb v1.0.1-0.20210819022825-2ae1ddf74ef7/go.mod h1:q4W45IWZaF22tdD+VEXcAWRA037jwmWEB5VWYORlTpc= +github.com/tklauser/go-sysconf v0.3.12 h1:0QaGUFOdQaIVdPgfITYzaTegZvdCjmYO52cSFAEVmqU= +github.com/tklauser/go-sysconf v0.3.12/go.mod h1:Ho14jnntGE1fpdOqQEEaiKRpvIavV0hSfmBq8nJbHYI= +github.com/tklauser/numcpus v0.6.1 h1:ng9scYS7az0Bk4OZLvrNXNSAO2Pxr1XXRAPyjhIx+Fk= +github.com/tklauser/numcpus v0.6.1/go.mod h1:1XfjsgE2zo8GVw7POkMbHENHzVg3GzmoZ9fESEdAacY= +github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= +github.com/ugorji/go/codec v0.0.0-20181204163529-d75b2dcb6bc8/go.mod h1:VFNgLljTbGfSG7qAOspJ7OScBnGdDN/yBr0sguwnwf0= +github.com/urfave/negroni v1.0.0/go.mod h1:Meg73S6kFm/4PpbYdq35yYWoCZ9mS/YSx+lKnmiohz4= +github.com/valyala/bytebufferpool v1.0.0/go.mod h1:6bBcMArwyJ5K/AmCkWv1jt77kVWyCJ6HpOuEn7z0Csc= +github.com/valyala/fasthttp v1.6.0/go.mod h1:FstJa9V+Pj9vQ7OJie2qMHdwemEDaDiSdBnvPM1Su9w= +github.com/valyala/fasttemplate v1.0.1/go.mod h1:UQGH1tvbgY+Nz5t2n7tXsz52dQxojPUpymEIMZ47gx8= +github.com/valyala/tcplisten v0.0.0-20161114210144-ceec8f93295a/go.mod h1:v3UYOV9WzVtRmSR+PDvWpU/qWl4Wa5LApYYX4ZtKbio= +github.com/xeipuuv/gojsonpointer v0.0.0-20180127040702-4e3ac2762d5f/go.mod h1:N2zxlSyiKSe5eX1tZViRH5QA0qijqEDrYZiPEAiq3wU= +github.com/xeipuuv/gojsonreference v0.0.0-20180127040603-bd5ef7bd5415/go.mod h1:GwrjFmJcFw6At/Gs6z4yjiIwzuJ1/+UwLxMQDVQXShQ= +github.com/xeipuuv/gojsonschema v1.2.0/go.mod h1:anYRn/JVcOK2ZgGU+IjEV4nwlhoK5sQluxsYJ78Id3Y= +github.com/xordataexchange/crypt v0.0.3-0.20170626215501-b2862e3d0a77/go.mod h1:aYKd//L2LvnjZzWKhF00oedf4jCCReLcmhLdhm1A27Q= +github.com/yalp/jsonpath v0.0.0-20180802001716-5cc68e5049a0/go.mod h1:/LWChgwKmvncFJFHJ7Gvn9wZArjbV5/FppcK2fKk/tI= +github.com/yudai/gojsondiff v1.0.0/go.mod h1:AY32+k2cwILAkW1fbgxQ5mUmMiZFgLIV+FBNExI05xg= +github.com/yudai/golcs v0.0.0-20170316035057-ecda9a501e82/go.mod h1:lgjkn3NuSvDfVJdfcVVdX+jpBxNmX4rDAzaS45IcYoM= +github.com/yudai/pp v2.0.1+incompatible/go.mod h1:PuxR/8QJ7cyCkFp/aUDS+JY727OFEZkTdatxwunjIkc= +github.com/yuin/goldmark v1.1.27/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +github.com/yuin/goldmark v1.2.1/go.mod h1:3hX8gzYuyVAZsxl0MRgGTJEmQBFcNTphYh9decYSb74= +golang.org/x/crypto v0.0.0-20181203042331-505ab145d0a9/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190701094942-4def268fd1a4/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= +golang.org/x/crypto v0.0.0-20200622213623-75b288015ac9/go.mod h1:LzIPMQfyMNhhGPhUkYOs5KpL4U8rLKemX1yGLhDgUto= golang.org/x/crypto v0.45.0 h1:jMBrvKuj23MTlT0bQEOBcAE0mjg8mK9RXFhRH6nyF3Q= golang.org/x/crypto v0.45.0/go.mod h1:XTGrrkGJve7CYK7J8PEww4aY7gM3qMCElcJQ8n8JdX4= +golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= +golang.org/x/exp v0.0.0-20231110203233-9a3e6036ecaa h1:FRnLl4eNAQl8hwxVVC17teOw8kdjVDVAiFMtgUdTSRQ= +golang.org/x/exp v0.0.0-20231110203233-9a3e6036ecaa/go.mod h1:zk2irFbV9DP96SEBUUAy67IdHUaZuSnrz1n472HUCLE= +golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= +golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= +golang.org/x/mod v0.2.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.3.0/go.mod h1:s0Qsj1ACt9ePp/hMypM3fl4fZqREWJwdYDEqhRiZZUA= +golang.org/x/mod v0.29.0 h1:HV8lRxZC4l2cr3Zq1LvtOsi/ThTgWnUk/y64QSs8GwA= +golang.org/x/mod v0.29.0/go.mod h1:NyhrlYXJ2H4eJiRy/WDBO6HMqZQ6q9nk4JzS3NuCK+w= +golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190327091125-710a502c58a2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190404232315-eb5bcb51f2a3/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190503192946-f4e77d36d62c/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190620200207-3b0461eec859/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20190827160401-ba9fcec4b297/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200226121028-0de0cce0169b/go.mod h1:z5CRVTTTmAJ677TzLLGU+0bjPO0LkuOLi4/5GtJWs/s= +golang.org/x/net v0.0.0-20200520004742-59133d7f0dd7/go.mod h1:qpuaurCH72eLCgpAm/N6yyVIVM9cpaDIP3A8BGJEC5A= +golang.org/x/net v0.0.0-20200813134508-3edf25e44fcc/go.mod h1:/O7V0waA8r7cgGh81Ro3o1hOxt32SMVPicZroKQ2sZA= +golang.org/x/net v0.0.0-20201021035429-f5854403a974/go.mod h1:sp8m0HH+o8qH0wwXwYZr8TS3Oi6o0r6Gce1SSxlDquU= +golang.org/x/net v0.47.0 h1:Mx+4dIFzqraBXUugkia1OOvlD6LemFo1ALMHjrXDOhY= +golang.org/x/net v0.47.0/go.mod h1:/jNxtkgq5yWUGYkaZGqo27cfGZ1c5Nen03aYrrKpVRU= +golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190423024810-112230192c58/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.17.0 h1:l60nONMj9l5drqw6jlhIELNv9I0A4OFgRsG9k2oT9Ug= +golang.org/x/sync v0.17.0/go.mod h1:9KTHXmSnoGruLpwFjVSX0lNNA75CykiMECbovNTZqGI= +golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181205085412-a5c9d58dba9a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20190412213103-97732733099d/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190626221950-04f50cda93cb/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190813064441-fde4db37ae7a/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190904154756-749cb33beabd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190916202348-b4ddaad3f8a3/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191005200804-aed5e4c7ecf9/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20191120155948-bd437916bb0e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200323222414-85ca7c5b95cd/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200519105757-fe76b779f299/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200814200057-3d37ad5750ed/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.8.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= +golang.org/x/sys v0.11.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.38.0 h1:3yZWxaJjBmCWXqhN1qh02AkOnCQ1poK6oF+a7xWL6Gc= golang.org/x/sys v0.38.0/go.mod h1:OgkHotnGiDImocRcuBABYBEXf8A9a87e/uXjp9XT3ks= +golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= +golang.org/x/text v0.3.2/go.mod h1:bEr9sfX3Q8Zfm5fL9x+3itogRgK3+ptLWKqgva+5dAk= +golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= +golang.org/x/text v0.31.0 h1:aC8ghyu4JhP8VojJ2lEHBnochRno1sgL6nEi9WGFGMM= +golang.org/x/text v0.31.0/go.mod h1:tKRAlv61yKIjGGHX/4tP1LTbc13YSec1pxVEWXzfoeM= +golang.org/x/tools v0.0.0-20180917221912-90fa682c2a6e/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181030221726-6c7e314b6563/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20181221001348-537d06c36207/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= +golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190311212946-11955173bddd/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190327201419-c70d86f8b7cf/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190524140312-2c0ae7006135/go.mod h1:RgjU9mgBXZiqYHBnxXauZ1Gv1EHHAz9KjViQ78xBX0Q= +golang.org/x/tools v0.0.0-20191119224855-298f0cb1881e/go.mod h1:b+2E5dAYhXwXZwtnZ6UAqBI28+e2cm9otk0dWdXHAEo= +golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= +golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.38.0 h1:Hx2Xv8hISq8Lm16jvBZ2VQf+RLmbd7wVUsALibYI/IQ= +golang.org/x/tools v0.38.0/go.mod h1:yEsQ/d/YK8cjh0L6rZlY8tgtlKiBNTL14pGDJPJpYQs= +golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 h1:go1bK/D/BFZV2I8cIQd1NKEZ+0owSTG1fDTci4IqFcE= +golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= +google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= +google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= +google.golang.org/genproto v0.0.0-20180518175338-11a468237815/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= +google.golang.org/genproto v0.0.0-20190819201941-24fa4b261c55/go.mod h1:DMBHOl98Agz4BDEuKkezgsaosCRResVns1a3J2ZsMNc= +google.golang.org/grpc v1.12.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= +google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.23.0/go.mod h1:Y5yQAOtifL1yxbo5wqy6BxZv8vAUGQwXBOALyacEbxg= +google.golang.org/grpc v1.25.1/go.mod h1:c3i+UQWmh7LiEpx4sFZnkU36qjEYZ0imhYfXVyQciAY= +google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk= +google.golang.org/protobuf v0.0.0-20200109180630-ec00e32a8dfd/go.mod h1:DFci5gLYBciE7Vtevhsrf46CRTquxDuWsQurQQe4oz8= +google.golang.org/protobuf v0.0.0-20200221191635-4d8936d0db64/go.mod h1:kwYJMbMJ01Woi6D6+Kah6886xMZcty6N08ah7+eCXa0= +google.golang.org/protobuf v0.0.0-20200228230310-ab0ca4ff8a60/go.mod h1:cfTl7dwQJ+fmap5saPgwCLgHXTUD7jkjRqWcaiX5VyM= +google.golang.org/protobuf v1.20.1-0.20200309200217-e05f789c0967/go.mod h1:A+miEFZTKqfCUM6K7xSMQL9OKL/b6hQv+e19PK+JZNE= +google.golang.org/protobuf v1.21.0/go.mod h1:47Nbq4nVaFHyn7ilMalzfO3qCViNmqZ2kzikPIcrTAo= +google.golang.org/protobuf v1.23.0/go.mod h1:EGpADcykh3NcUnDUJcl1+ZksZNG86OlYog2l/sGQquU= +google.golang.org/protobuf v1.36.5 h1:tPhr+woSbjfYvY6/GPufUoYizxw1cF/yFoxJ2fmpwlM= +google.golang.org/protobuf v1.36.5/go.mod h1:9fA7Ob0pmnwhb644+1+CVWFRbNajQ6iRojtC/QF5bRE= +gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20190902080502-41f04d3bba15/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= +gopkg.in/fsnotify.v1 v1.4.7/go.mod h1:Tz8NjZHkW78fSQdbUxIjBTcgA1z1m8ZHf0WmKUhAMys= +gopkg.in/go-playground/assert.v1 v1.2.1/go.mod h1:9RXL0bg/zibRAgZUYszZSwO/z8Y/a8bDuhia5mkpMnE= +gopkg.in/go-playground/validator.v8 v8.18.2/go.mod h1:RX2a/7Ha8BgOhfk7j780h4/u/RRjR0eouCJSH80/M2Y= +gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7 h1:uRGJdciOHaEIrze2W8Q3AKkepLTh2hOroT7a+7czfdQ= +gopkg.in/tomb.v1 v1.0.0-20141024135613-dd632973f1e7/go.mod h1:dt/ZhP58zS4L8KSrWDmTeBkI65Dw0HsyUHuEVlX15mw= +gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.2.4/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.3.0/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= +gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= +gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= +gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= +honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= +rsc.io/tmplfunc v0.0.3 h1:53XFQh69AfOa8Tw0Jm7t+GV7KZhOi6jzsCzTtKbMvzU= +rsc.io/tmplfunc v0.0.3/go.mod h1:AG3sTPzElb1Io3Yg4voV9AGZJuleGAwaVRxL9M49PhA= diff --git a/graft/subnet-evm/core/blockchain.go b/graft/subnet-evm/core/blockchain.go index 0d5e33ecc980..4405d0b3c19e 100644 --- a/graft/subnet-evm/core/blockchain.go +++ b/graft/subnet-evm/core/blockchain.go @@ -40,6 +40,7 @@ import ( "sync/atomic" "time" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/commontype" "github.com/ava-labs/avalanchego/graft/subnet-evm/consensus" "github.com/ava-labs/avalanchego/graft/subnet-evm/core/extstate" @@ -48,7 +49,6 @@ import ( "github.com/ava-labs/avalanchego/graft/subnet-evm/params" "github.com/ava-labs/avalanchego/graft/subnet-evm/plugin/evm/customrawdb" "github.com/ava-labs/avalanchego/graft/subnet-evm/plugin/evm/customtypes" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/hashdb" "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/pathdb" "github.com/ava-labs/libevm/common" diff --git a/graft/subnet-evm/core/extstate/database.go b/graft/subnet-evm/core/extstate/database.go index a9946ffe3a68..3d013a16e3d7 100644 --- a/graft/subnet-evm/core/extstate/database.go +++ b/graft/subnet-evm/core/extstate/database.go @@ -8,7 +8,7 @@ import ( "github.com/ava-labs/libevm/ethdb" "github.com/ava-labs/libevm/triedb" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" + "github.com/ava-labs/avalanchego/graft/evm/firewood" ) func NewDatabaseWithConfig(db ethdb.Database, config *triedb.Config) state.Database { @@ -22,12 +22,9 @@ func NewDatabaseWithNodeDB(db ethdb.Database, triedb *triedb.Database) state.Dat } func wrapIfFirewood(db state.Database) state.Database { - fw, ok := db.TrieDB().Backend().(*firewood.Database) + fw, ok := db.TrieDB().Backend().(*firewood.TrieDB) if !ok { return db } - return &firewoodAccessorDB{ - Database: db, - fw: fw, - } + return firewood.NewStateAccessor(db, fw) } diff --git a/graft/subnet-evm/core/extstate/database_test.go b/graft/subnet-evm/core/extstate/database_test.go deleted file mode 100644 index eefa210a903d..000000000000 --- a/graft/subnet-evm/core/extstate/database_test.go +++ /dev/null @@ -1,358 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package extstate - -import ( - "encoding/binary" - "math/rand" - "slices" - "testing" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/rawdb" - "github.com/ava-labs/libevm/core/state" - "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/libevm/crypto" - "github.com/ava-labs/libevm/libevm/stateconf" - "github.com/ava-labs/libevm/trie/trienode" - "github.com/ava-labs/libevm/triedb" - "github.com/holiman/uint256" - "github.com/stretchr/testify/require" - - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/hashdb" -) - -const ( - commit byte = iota - createAccount - updateAccount - deleteAccount - addStorage - updateStorage - deleteStorage - maxStep -) - -var stepMap = map[byte]string{ - commit: "commit", - createAccount: "createAccount", - updateAccount: "updateAccount", - deleteAccount: "deleteAccount", - addStorage: "addStorage", - updateStorage: "updateStorage", - deleteStorage: "deleteStorage", -} - -type fuzzState struct { - require *require.Assertions - - // current state - currentAddrs []common.Address - currentStorageInputIndices map[common.Address]uint64 - inputCounter uint64 - blockNumber uint64 - - // pending changes to be committed - merkleTries []*merkleTrie -} -type merkleTrie struct { - name string - ethDatabase state.Database - accountTrie state.Trie - openStorageTries map[common.Address]state.Trie - lastRoot common.Hash -} - -func newFuzzState(t *testing.T) *fuzzState { - r := require.New(t) - - hashState := NewDatabaseWithConfig( - rawdb.NewMemoryDatabase(), - &triedb.Config{ - DBOverride: hashdb.Defaults.BackendConstructor, - }) - ethRoot := types.EmptyRootHash - hashTr, err := hashState.OpenTrie(ethRoot) - r.NoError(err) - t.Cleanup(func() { - r.NoError(hashState.TrieDB().Close()) - }) - - firewoodMemdb := rawdb.NewMemoryDatabase() - fwCfg := firewood.Defaults // copy the defaults - fwCfg.ChainDataDir = t.TempDir() // Use a temporary directory for the Firewood - firewoodState := NewDatabaseWithConfig( - firewoodMemdb, - &triedb.Config{ - DBOverride: fwCfg.BackendConstructor, - }, - ) - fwTr, err := firewoodState.OpenTrie(ethRoot) - r.NoError(err) - t.Cleanup(func() { - r.NoError(firewoodState.TrieDB().Close()) - }) - - return &fuzzState{ - merkleTries: []*merkleTrie{ - { - name: "hash", - ethDatabase: hashState, - accountTrie: hashTr, - openStorageTries: make(map[common.Address]state.Trie), - lastRoot: ethRoot, - }, - { - name: "firewood", - ethDatabase: firewoodState, - accountTrie: fwTr, - openStorageTries: make(map[common.Address]state.Trie), - lastRoot: ethRoot, - }, - }, - currentStorageInputIndices: make(map[common.Address]uint64), - require: r, - } -} - -// commit writes the pending changes to both tries and clears the pending changes -func (fs *fuzzState) commit() { - for _, tr := range fs.merkleTries { - mergedNodeSet := trienode.NewMergedNodeSet() - for addr, str := range tr.openStorageTries { - accountStateRoot, set, err := str.Commit(false) - fs.require.NoError(err, "failed to commit storage trie for account %s in %s", addr.Hex(), tr.name) - // A no-op change returns a nil set, which will cause merge to panic. - if set != nil { - fs.require.NoError(mergedNodeSet.Merge(set), "failed to merge storage trie nodeset for account %s in %s", addr.Hex(), tr.name) - } - - acc, err := tr.accountTrie.GetAccount(addr) - fs.require.NoError(err, "failed to get account %s in %s", addr.Hex(), tr.name) - // If the account was deleted, we can skip updating the account's - // state root. - fs.require.NotNil(acc, "account %s is nil in %s", addr.Hex(), tr.name) - - acc.Root = accountStateRoot - fs.require.NoError(tr.accountTrie.UpdateAccount(addr, acc), "failed to update account %s in %s", addr.Hex(), tr.name) - } - - updatedRoot, set, err := tr.accountTrie.Commit(true) - fs.require.NoError(err, "failed to commit account trie in %s", tr.name) - - // A no-op change returns a nil set, which will cause merge to panic. - if set != nil { - fs.require.NoError(mergedNodeSet.Merge(set), "failed to merge account trie nodeset in %s", tr.name) - } - - // HashDB/PathDB only allows updating the triedb if there have been changes. - if _, ok := tr.ethDatabase.TrieDB().Backend().(*firewood.Database); ok { - triedbopt := stateconf.WithTrieDBUpdatePayload(common.Hash{byte(int64(fs.blockNumber - 1))}, common.Hash{byte(int64(fs.blockNumber))}) - fs.require.NoError(tr.ethDatabase.TrieDB().Update(updatedRoot, tr.lastRoot, fs.blockNumber, mergedNodeSet, nil, triedbopt), "failed to update triedb in %s", tr.name) - tr.lastRoot = updatedRoot - } else if updatedRoot != tr.lastRoot { - fs.require.NoError(tr.ethDatabase.TrieDB().Update(updatedRoot, tr.lastRoot, fs.blockNumber, mergedNodeSet, nil), "failed to update triedb in %s", tr.name) - tr.lastRoot = updatedRoot - } - tr.openStorageTries = make(map[common.Address]state.Trie) - fs.require.NoError(tr.ethDatabase.TrieDB().Commit(updatedRoot, true), - "failed to commit %s: expected hashdb root %s", tr.name, fs.merkleTries[0].lastRoot.Hex()) - tr.accountTrie, err = tr.ethDatabase.OpenTrie(tr.lastRoot) - fs.require.NoError(err, "failed to reopen account trie for %s", tr.name) - } - fs.blockNumber++ - - // After computing the new root for each trie, we can confirm that the hashing matches - expectedRoot := fs.merkleTries[0].lastRoot - for i, tr := range fs.merkleTries[1:] { - fs.require.Equalf(expectedRoot, tr.lastRoot, - "root mismatch for %s: expected %x, got %x (trie index %d)", - tr.name, expectedRoot.Hex(), tr.lastRoot.Hex(), i, - ) - } -} - -// createAccount generates a new, unique account and adds it to both tries and the tracked -// current state. -func (fs *fuzzState) createAccount() { - fs.inputCounter++ - addr := common.BytesToAddress(crypto.Keccak256Hash(binary.BigEndian.AppendUint64(nil, fs.inputCounter)).Bytes()) - acc := &types.StateAccount{ - Nonce: 1, - Balance: uint256.NewInt(100), - Root: types.EmptyRootHash, - CodeHash: types.EmptyCodeHash[:], - } - fs.currentAddrs = append(fs.currentAddrs, addr) - - for _, tr := range fs.merkleTries { - fs.require.NoError(tr.accountTrie.UpdateAccount(addr, acc), "failed to create account %s in %s", addr.Hex(), tr.name) - } -} - -// selectAccount returns a random account and account hash for the provided index -// assumes: addrIndex < len(tr.currentAddrs) -func (fs *fuzzState) selectAccount(addrIndex int) common.Address { - return fs.currentAddrs[addrIndex] -} - -// updateAccount selects a random account, increments its nonce, and adds the update -// to the pending changes for both tries. -func (fs *fuzzState) updateAccount(addrIndex int) { - addr := fs.selectAccount(addrIndex) - - for _, tr := range fs.merkleTries { - acc, err := tr.accountTrie.GetAccount(addr) - fs.require.NoError(err, "failed to get account %s for update in %s", addr.Hex(), tr.name) - fs.require.NotNil(acc, "account %s is nil for update in %s", addr.Hex(), tr.name) - acc.Nonce++ - acc.CodeHash = crypto.Keccak256Hash(acc.CodeHash).Bytes() - acc.Balance.Add(acc.Balance, uint256.NewInt(3)) - fs.require.NoError(tr.accountTrie.UpdateAccount(addr, acc), "failed to update account %s in %s", addr.Hex(), tr.name) - } -} - -// deleteAccount selects a random account and deletes it from both tries and the tracked -// current state. -func (fs *fuzzState) deleteAccount(accountIndex int) { - deleteAddr := fs.selectAccount(accountIndex) - fs.currentAddrs = slices.DeleteFunc(fs.currentAddrs, func(addr common.Address) bool { - return deleteAddr == addr - }) - for _, tr := range fs.merkleTries { - fs.require.NoError(tr.accountTrie.DeleteAccount(deleteAddr), "failed to delete account %s in %s", deleteAddr.Hex(), tr.name) - delete(tr.openStorageTries, deleteAddr) // remove any open storage trie for the deleted account - } -} - -// openStorageTrie opens the storage trie for the provided account address. -// Uses an already opened trie, if there's a pending update to the ethereum nested -// storage trie. -// -// must maintain a map of currently open storage tries, so we can defer committing them -// until commit as opposed to after each storage update. -// This mimics the actual handling of state commitments in the EVM where storage tries are all committed immediately -// before updating the account trie along with the updated storage trie roots: -// https://github.com/ava-labs/libevm/blob/0bfe4a0380c86d7c9bf19fe84368b9695fcb96c7/core/state/statedb.go#L1155 -// -// If we attempt to commit the storage tries after each operation, then attempting to re-open the storage trie -// with an updated storage trie root from ethDatabase will fail since the storage trie root will not have been -// persisted yet - leading to a missing trie node error. -func (fs *fuzzState) openStorageTrie(addr common.Address, tr *merkleTrie) state.Trie { - storageTrie, ok := tr.openStorageTries[addr] - if ok { - return storageTrie - } - - acc, err := tr.accountTrie.GetAccount(addr) - fs.require.NoError(err, "failed to get account %s for storage trie in %s", addr.Hex(), tr.name) - fs.require.NotNil(acc, "account %s not found in %s", addr.Hex(), tr.name) - storageTrie, err = tr.ethDatabase.OpenStorageTrie(tr.lastRoot, addr, acc.Root, tr.accountTrie) - fs.require.NoError(err, "failed to open storage trie for %s in %s", addr.Hex(), tr.name) - tr.openStorageTries[addr] = storageTrie - return storageTrie -} - -// addStorage selects an account and adds a new storage key-value pair to the account. -func (fs *fuzzState) addStorage(accountIndex int) { - addr := fs.selectAccount(accountIndex) - // Increment storageInputIndices for the account and take the next input to generate - // a new storage key-value pair for the account. - fs.currentStorageInputIndices[addr]++ - storageIndex := fs.currentStorageInputIndices[addr] - key := crypto.Keccak256Hash(binary.BigEndian.AppendUint64(nil, storageIndex)) - keyHash := crypto.Keccak256Hash(key[:]) - val := crypto.Keccak256Hash(keyHash[:]) - - for _, tr := range fs.merkleTries { - str := fs.openStorageTrie(addr, tr) - fs.require.NoError(str.UpdateStorage(addr, key[:], val[:]), "failed to add storage for account %s in %s", addr.Hex(), tr.name) - } - - fs.currentStorageInputIndices[addr]++ -} - -// updateStorage selects an account and updates an existing storage key-value pair -// note: this may "update" a key-value pair that doesn't exist if it was previously deleted. -func (fs *fuzzState) updateStorage(accountIndex int, storageIndexInput uint64) { - addr := fs.selectAccount(accountIndex) - storageIndex := fs.currentStorageInputIndices[addr] - storageIndex %= storageIndexInput - - storageKey := crypto.Keccak256Hash(binary.BigEndian.AppendUint64(nil, storageIndex)) - storageKeyHash := crypto.Keccak256Hash(storageKey[:]) - fs.inputCounter++ - updatedValInput := binary.BigEndian.AppendUint64(storageKeyHash[:], fs.inputCounter) - updatedVal := crypto.Keccak256Hash(updatedValInput) - - for _, tr := range fs.merkleTries { - str := fs.openStorageTrie(addr, tr) - fs.require.NoError(str.UpdateStorage(addr, storageKey[:], updatedVal[:]), "failed to update storage for account %s in %s", addr.Hex(), tr.name) - } -} - -// deleteStorage selects an account and deletes an existing storage key-value pair -// note: this may "delete" a key-value pair that doesn't exist if it was previously deleted. -func (fs *fuzzState) deleteStorage(accountIndex int, storageIndexInput uint64) { - addr := fs.selectAccount(accountIndex) - storageIndex := fs.currentStorageInputIndices[addr] - storageIndex %= storageIndexInput - storageKey := crypto.Keccak256Hash(binary.BigEndian.AppendUint64(nil, storageIndex)) - - for _, tr := range fs.merkleTries { - str := fs.openStorageTrie(addr, tr) - fs.require.NoError(str.DeleteStorage(addr, storageKey[:]), "failed to delete storage for account %s in %s", addr.Hex(), tr.name) - } -} - -func FuzzTree(f *testing.F) { - f.Fuzz(func(t *testing.T, randSeed int64, byteSteps []byte) { - fuzzState := newFuzzState(t) - rand := rand.New(rand.NewSource(randSeed)) // this isn't a good fuzz test, but it is reproducible. - - for range 10 { - fuzzState.createAccount() - } - fuzzState.commit() - - const maxSteps = 1000 - if len(byteSteps) > maxSteps { - byteSteps = byteSteps[:maxSteps] - } - - for _, step := range byteSteps { - step %= maxStep - t.Log(stepMap[step]) - switch step { - case commit: - fuzzState.commit() - case createAccount: - fuzzState.createAccount() - case updateAccount: - if len(fuzzState.currentAddrs) > 0 { - fuzzState.updateAccount(rand.Intn(len(fuzzState.currentAddrs))) - } - case deleteAccount: - if len(fuzzState.currentAddrs) > 0 { - fuzzState.deleteAccount(rand.Intn(len(fuzzState.currentAddrs))) - } - case addStorage: - if len(fuzzState.currentAddrs) > 0 { - fuzzState.addStorage(rand.Intn(len(fuzzState.currentAddrs))) - } - case updateStorage: - if len(fuzzState.currentAddrs) > 0 { - fuzzState.updateStorage(rand.Intn(len(fuzzState.currentAddrs)), rand.Uint64()) - } - case deleteStorage: - if len(fuzzState.currentAddrs) > 0 { - fuzzState.deleteStorage(rand.Intn(len(fuzzState.currentAddrs)), rand.Uint64()) - } - default: - require.Failf(t, "unknown step", "got: %d", step) - } - } - }) -} diff --git a/graft/subnet-evm/core/extstate/firewood_database.go b/graft/subnet-evm/core/extstate/firewood_database.go deleted file mode 100644 index b55145f2ea09..000000000000 --- a/graft/subnet-evm/core/extstate/firewood_database.go +++ /dev/null @@ -1,53 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package extstate - -import ( - "fmt" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/state" - - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" -) - -var ( - _ state.Database = (*firewoodAccessorDB)(nil) - _ state.Trie = (*firewood.AccountTrie)(nil) - _ state.Trie = (*firewood.StorageTrie)(nil) -) - -type firewoodAccessorDB struct { - state.Database - fw *firewood.Database -} - -// OpenTrie opens the main account trie. -func (db *firewoodAccessorDB) OpenTrie(root common.Hash) (state.Trie, error) { - return firewood.NewAccountTrie(root, db.fw) -} - -// OpenStorageTrie opens a wrapped version of the account trie. -// -//nolint:revive // removing names loses context. -func (*firewoodAccessorDB) OpenStorageTrie(stateRoot common.Hash, addr common.Address, accountRoot common.Hash, self state.Trie) (state.Trie, error) { - accountTrie, ok := self.(*firewood.AccountTrie) - if !ok { - return nil, fmt.Errorf("invalid account trie type: %T", self) - } - return firewood.NewStorageTrie(accountTrie) -} - -// CopyTrie returns a deep copy of the given trie. -// It can be altered by the caller. -func (*firewoodAccessorDB) CopyTrie(t state.Trie) state.Trie { - switch t := t.(type) { - case *firewood.AccountTrie: - return t.Copy() - case *firewood.StorageTrie: - return nil // The storage trie just wraps the account trie, so we must re-open it separately. - default: - panic(fmt.Errorf("unknown trie type %T", t)) - } -} diff --git a/graft/subnet-evm/core/genesis_test.go b/graft/subnet-evm/core/genesis_test.go index c17fc91693d3..856ae1e2af6f 100644 --- a/graft/subnet-evm/core/genesis_test.go +++ b/graft/subnet-evm/core/genesis_test.go @@ -35,6 +35,7 @@ import ( "reflect" "testing" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/consensus/dummy" "github.com/ava-labs/avalanchego/graft/subnet-evm/params" "github.com/ava-labs/avalanchego/graft/subnet-evm/params/extras" @@ -42,7 +43,6 @@ import ( "github.com/ava-labs/avalanchego/graft/subnet-evm/plugin/evm/upgrade/legacy" "github.com/ava-labs/avalanchego/graft/subnet-evm/precompile/allowlist" "github.com/ava-labs/avalanchego/graft/subnet-evm/precompile/contracts/deployerallowlist" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/pathdb" "github.com/ava-labs/avalanchego/graft/subnet-evm/utils" "github.com/ava-labs/libevm/common" diff --git a/graft/subnet-evm/internal/ethapi/api.go b/graft/subnet-evm/internal/ethapi/api.go index 0b50fa4132cc..98ac4a217701 100644 --- a/graft/subnet-evm/internal/ethapi/api.go +++ b/graft/subnet-evm/internal/ethapi/api.go @@ -36,13 +36,13 @@ import ( "strings" "time" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/consensus" "github.com/ava-labs/avalanchego/graft/subnet-evm/core" "github.com/ava-labs/avalanchego/graft/subnet-evm/eth/gasestimator" "github.com/ava-labs/avalanchego/graft/subnet-evm/params" "github.com/ava-labs/avalanchego/graft/subnet-evm/plugin/evm/customtypes" "github.com/ava-labs/avalanchego/graft/subnet-evm/rpc" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" "github.com/ava-labs/libevm/accounts" "github.com/ava-labs/libevm/accounts/keystore" "github.com/ava-labs/libevm/accounts/scwallet" @@ -704,7 +704,7 @@ func (s *BlockChainAPI) GetProof(ctx context.Context, address common.Address, st if statedb == nil || err != nil { return nil, err } - if _, ok := statedb.Database().TrieDB().Backend().(*firewood.Database); ok { + if _, ok := statedb.Database().TrieDB().Backend().(*firewood.TrieDB); ok { return nil, errors.New("firewood database does not yet support getProof") } codeHash := statedb.GetCodeHash(address) diff --git a/graft/subnet-evm/tests/state_test_util.go b/graft/subnet-evm/tests/state_test_util.go index 89f78665bf6d..e6b6675c38b9 100644 --- a/graft/subnet-evm/tests/state_test_util.go +++ b/graft/subnet-evm/tests/state_test_util.go @@ -37,12 +37,12 @@ import ( "strconv" "strings" + "github.com/ava-labs/avalanchego/graft/evm/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/core" "github.com/ava-labs/avalanchego/graft/subnet-evm/core/extstate" "github.com/ava-labs/avalanchego/graft/subnet-evm/core/state/snapshot" "github.com/ava-labs/avalanchego/graft/subnet-evm/params" "github.com/ava-labs/avalanchego/graft/subnet-evm/plugin/evm/customrawdb" - "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/firewood" "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/hashdb" "github.com/ava-labs/avalanchego/graft/subnet-evm/triedb/pathdb" "github.com/ava-labs/libevm/common" diff --git a/graft/subnet-evm/triedb/firewood/account_trie.go b/graft/subnet-evm/triedb/firewood/account_trie.go deleted file mode 100644 index de5fa2a0de94..000000000000 --- a/graft/subnet-evm/triedb/firewood/account_trie.go +++ /dev/null @@ -1,288 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package firewood - -import ( - "errors" - - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/core/types" - "github.com/ava-labs/libevm/crypto" - "github.com/ava-labs/libevm/ethdb" - "github.com/ava-labs/libevm/log" - "github.com/ava-labs/libevm/rlp" - "github.com/ava-labs/libevm/trie" - "github.com/ava-labs/libevm/trie/trienode" - "github.com/ava-labs/libevm/triedb/database" -) - -// AccountTrie implements state.Trie for managing account states. -// There are a couple caveats to the current implementation: -// 1. `Commit` is not used as expected in the state package. The `StorageTrie` doesn't return -// values, and we thus rely on the `AccountTrie`. -// 2. The `Hash` method actually creates the proposal, since Firewood cannot calculate -// the hash of the trie without committing it. It is immediately dropped, and this -// can likely be optimized. -// -// Note this is not concurrent safe. -type AccountTrie struct { - fw *Database - parentRoot common.Hash - root common.Hash - reader database.Reader - dirtyKeys map[string][]byte // Store dirty changes - updateKeys [][]byte - updateValues [][]byte - hasChanges bool -} - -func NewAccountTrie(root common.Hash, db *Database) (*AccountTrie, error) { - reader, err := db.Reader(root) - if err != nil { - return nil, err - } - return &AccountTrie{ - fw: db, - parentRoot: root, - reader: reader, - dirtyKeys: make(map[string][]byte), - hasChanges: true, // Start with hasChanges true to allow computing the proposal hash - }, nil -} - -// GetAccount returns the state account associated with an address. -// - If the account has been updated, the new value is returned. -// - If the account has been deleted, (nil, nil) is returned. -// - If the account does not exist, (nil, nil) is returned. -func (a *AccountTrie) GetAccount(addr common.Address) (*types.StateAccount, error) { - key := crypto.Keccak256Hash(addr.Bytes()).Bytes() - - // First check if there's a pending update for this account - if updateValue, exists := a.dirtyKeys[string(key)]; exists { - // If the value is empty, it indicates deletion - // Invariant: All encoded values have length > 0 - if len(updateValue) == 0 { - return nil, nil - } - // Decode and return the updated account - account := new(types.StateAccount) - err := rlp.DecodeBytes(updateValue, account) - return account, err - } - - // No pending update found, read from the underlying reader - accountBytes, err := a.reader.Node(common.Hash{}, key, common.Hash{}) - if err != nil { - return nil, err - } - - if accountBytes == nil { - return nil, nil - } - - // Decode the account node - account := new(types.StateAccount) - err = rlp.DecodeBytes(accountBytes, account) - return account, err -} - -// GetStorage returns the value associated with a storage key for a given account address. -// - If the storage slot has been updated, the new value is returned. -// - If the storage slot has been deleted, (nil, nil) is returned. -// - If the storage slot does not exist, (nil, nil) is returned. -func (a *AccountTrie) GetStorage(addr common.Address, key []byte) ([]byte, error) { - // If the account has been deleted, we should return nil - accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() - if val, exists := a.dirtyKeys[string(accountKey)]; exists && len(val) == 0 { - return nil, nil - } - - var combinedKey [2 * common.HashLength]byte - storageKey := crypto.Keccak256Hash(key).Bytes() - copy(combinedKey[:common.HashLength], accountKey) - copy(combinedKey[common.HashLength:], storageKey) - - // Check if there's a pending update for this storage slot - if updateValue, exists := a.dirtyKeys[string(combinedKey[:])]; exists { - // If the value is empty, it indicates deletion - if len(updateValue) == 0 { - return nil, nil - } - // Decode and return the updated storage value - _, decoded, _, err := rlp.Split(updateValue) - return decoded, err - } - - // No pending update found, read from the underlying reader - storageBytes, err := a.reader.Node(common.Hash{}, combinedKey[:], common.Hash{}) - if err != nil || storageBytes == nil { - return nil, err - } - - // Decode the storage value - _, decoded, _, err := rlp.Split(storageBytes) - return decoded, err -} - -// UpdateAccount replaces or creates the state account associated with an address. -// This new value will be returned for subsequent `GetAccount` calls. -func (a *AccountTrie) UpdateAccount(addr common.Address, account *types.StateAccount) error { - // Queue the keys and values for later commit - key := crypto.Keccak256Hash(addr.Bytes()).Bytes() - data, err := rlp.EncodeToBytes(account) - if err != nil { - return err - } - a.dirtyKeys[string(key)] = data - a.updateKeys = append(a.updateKeys, key) - a.updateValues = append(a.updateValues, data) - a.hasChanges = true // Mark that there are changes to commit - return nil -} - -// UpdateStorage replaces or creates the value associated with a storage key for a given account address. -// This new value will be returned for subsequent `GetStorage` calls. -func (a *AccountTrie) UpdateStorage(addr common.Address, key []byte, value []byte) error { - var combinedKey [2 * common.HashLength]byte - accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() - storageKey := crypto.Keccak256Hash(key).Bytes() - copy(combinedKey[:common.HashLength], accountKey) - copy(combinedKey[common.HashLength:], storageKey) - - data, err := rlp.EncodeToBytes(value) - if err != nil { - return err - } - - // Queue the keys and values for later commit - a.dirtyKeys[string(combinedKey[:])] = data - a.updateKeys = append(a.updateKeys, combinedKey[:]) - a.updateValues = append(a.updateValues, data) - a.hasChanges = true // Mark that there are changes to commit - return nil -} - -// DeleteAccount removes the state account associated with an address. -func (a *AccountTrie) DeleteAccount(addr common.Address) error { - key := crypto.Keccak256Hash(addr.Bytes()).Bytes() - // Queue the key for deletion - a.dirtyKeys[string(key)] = nil - a.updateKeys = append(a.updateKeys, key) - a.updateValues = append(a.updateValues, nil) // Nil value indicates deletion - a.hasChanges = true // Mark that there are changes to commit - return nil -} - -// DeleteStorage removes the value associated with a storage key for a given account address. -func (a *AccountTrie) DeleteStorage(addr common.Address, key []byte) error { - var combinedKey [2 * common.HashLength]byte - accountKey := crypto.Keccak256Hash(addr.Bytes()).Bytes() - storageKey := crypto.Keccak256Hash(key).Bytes() - copy(combinedKey[:common.HashLength], accountKey) - copy(combinedKey[common.HashLength:], storageKey) - - // Queue the key for deletion - a.dirtyKeys[string(combinedKey[:])] = nil - a.updateKeys = append(a.updateKeys, combinedKey[:]) - a.updateValues = append(a.updateValues, nil) // Nil value indicates deletion - a.hasChanges = true // Mark that there are changes to commit - return nil -} - -// Hash returns the current hash of the state trie. -// This will create a proposal and drop it, so it is not efficient to call for each transaction. -// If there are no changes since the last call, the cached root is returned. -func (a *AccountTrie) Hash() common.Hash { - hash, err := a.hash() - if err != nil { - log.Error("Failed to hash account trie", "error", err) - return common.Hash{} - } - return hash -} - -func (a *AccountTrie) hash() (common.Hash, error) { - // If we haven't already hashed, we need to do so. - if a.hasChanges { - root, err := a.fw.getProposalHash(a.parentRoot, a.updateKeys, a.updateValues) - if err != nil { - return common.Hash{}, err - } - a.root = root - a.hasChanges = false // Avoid re-hashing until next update - } - return a.root, nil -} - -// Commit returns the new root hash of the trie and a NodeSet containing all modified accounts and storage slots. -// The format of the NodeSet is different than in go-ethereum's trie implementation due to Firewood's design. -// This boolean is ignored, as it is a relic of the StateTrie implementation. -func (a *AccountTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { - // Get the hash of the trie. - hash, err := a.hash() - if err != nil { - return common.Hash{}, nil, err - } - - // Create the NodeSet. This will be sent to `triedb.Update` later. - nodeset := trienode.NewNodeSet(common.Hash{}) - for i, key := range a.updateKeys { - nodeset.AddNode(key, &trienode.Node{ - Blob: a.updateValues[i], - }) - } - - return hash, nodeset, nil -} - -// UpdateContractCode implements state.Trie. -// Contract code is controlled by rawdb, so we don't need to do anything here. -func (*AccountTrie) UpdateContractCode(_ common.Address, _ common.Hash, _ []byte) error { - return nil -} - -// GetKey implements state.Trie. -// This should not be used, since any user should not be accessing by raw key. -func (*AccountTrie) GetKey([]byte) []byte { - return nil -} - -// NodeIterator implements state.Trie. -// Firewood does not support iterating over internal nodes. -func (*AccountTrie) NodeIterator([]byte) (trie.NodeIterator, error) { - return nil, errors.New("NodeIterator not implemented for Firewood") -} - -// Prove implements state.Trie. -// Firewood does not yet support providing key proofs. -func (*AccountTrie) Prove([]byte, ethdb.KeyValueWriter) error { - return errors.New("Prove not implemented for Firewood") -} - -func (a *AccountTrie) Copy() *AccountTrie { - // Create a new AccountTrie with the same root and reader - newTrie := &AccountTrie{ - fw: a.fw, - parentRoot: a.parentRoot, - root: a.root, - reader: a.reader, // Share the same reader - hasChanges: a.hasChanges, - dirtyKeys: make(map[string][]byte, len(a.dirtyKeys)), - updateKeys: make([][]byte, len(a.updateKeys)), - updateValues: make([][]byte, len(a.updateValues)), - } - - // Deep copy dirtyKeys map - for k, v := range a.dirtyKeys { - newTrie.dirtyKeys[k] = append([]byte{}, v...) - } - - // Deep copy updateKeys and updateValues slices - for i := range a.updateKeys { - newTrie.updateKeys[i] = append([]byte{}, a.updateKeys[i]...) - newTrie.updateValues[i] = append([]byte{}, a.updateValues[i]...) - } - - return newTrie -} diff --git a/graft/subnet-evm/triedb/firewood/storage_trie.go b/graft/subnet-evm/triedb/firewood/storage_trie.go deleted file mode 100644 index 9e6367183b16..000000000000 --- a/graft/subnet-evm/triedb/firewood/storage_trie.go +++ /dev/null @@ -1,41 +0,0 @@ -// Copyright (C) 2019-2025, Ava Labs, Inc. All rights reserved. -// See the file LICENSE for licensing terms. - -package firewood - -import ( - "github.com/ava-labs/libevm/common" - "github.com/ava-labs/libevm/trie/trienode" -) - -type StorageTrie struct { - *AccountTrie -} - -// `NewStorageTrie` returns a wrapper around an `AccountTrie` since Firewood -// does not require a separate storage trie. All changes are managed by the account trie. -func NewStorageTrie(accountTrie *AccountTrie) (*StorageTrie, error) { - return &StorageTrie{ - AccountTrie: accountTrie, - }, nil -} - -// Actual commit is handled by the account trie. -// Return the old storage root as if there was no change since Firewood -// will manage the hash calculations without it. -// All changes are managed by the account trie. -func (*StorageTrie) Commit(bool) (common.Hash, *trienode.NodeSet, error) { - return common.Hash{}, nil, nil -} - -// Firewood doesn't require tracking storage roots inside of an account. -// They will be updated in place when hashing of the proposal takes place. -func (*StorageTrie) Hash() common.Hash { - return common.Hash{} -} - -// Copy should never be called on a storage trie, as it is just a wrapper around the account trie. -// Each storage trie should be re-opened with the account trie separately. -func (*StorageTrie) Copy() *StorageTrie { - return nil -}