From 2f90e8a3da91d43af9bb19511d09e7340915d94a Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 14:31:20 +0800 Subject: [PATCH 01/43] core/state: state size metrics Signed-off-by: jsvisa --- core/state/statedb.go | 17 +++ core/state/statesize.go | 305 ++++++++++++++++++++++++++++++++++++++++ 2 files changed, 322 insertions(+) create mode 100644 core/state/statesize.go diff --git a/core/state/statedb.go b/core/state/statedb.go index cdfd6382212..2c41c2081b1 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -139,6 +139,9 @@ type StateDB struct { witness *stateless.Witness witnessStats *stateless.WitnessStats + // State size tracking + stateSizeGen *stateSizeGenerator + // Measurements gathered during execution for debugging purposes AccountReads time.Duration AccountHashes time.Duration @@ -186,6 +189,15 @@ func NewWithReader(root common.Hash, db Database, reader Reader) (*StateDB, erro if db.TrieDB().IsVerkle() { sdb.accessEvents = NewAccessEvents(db.PointCache()) } + + // Initialize state size tracking + if cachingDB, ok := db.(*CachingDB); ok { + sdb.stateSizeGen = newStateSizeGenerator(cachingDB.disk, db, root) + if !db.TrieDB().IsVerkle() { + sdb.stateSizeGen.run() + } + } + return sdb, nil } @@ -1350,6 +1362,11 @@ func (s *StateDB) commitAndFlush(block uint64, deleteEmptyObjects bool, noStorag return nil, err } s.TrieDBCommits += time.Since(start) + + // Update state size metrics + if s.stateSizeGen != nil { + s.stateSizeGen.updateMetrics(ret) + } } } s.reader, _ = s.db.Reader(s.originalRoot) diff --git a/core/state/statesize.go b/core/state/statesize.go new file mode 100644 index 00000000000..d3424d0c9f5 --- /dev/null +++ b/core/state/statesize.go @@ -0,0 +1,305 @@ +// Copyright 2025 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package state + +import ( + "bytes" + "sync" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/rawdb" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + "github.com/ethereum/go-ethereum/rlp" +) + +// State size metrics +var ( + // Baseline state size metrics + stateSizeAccountsCountMeter = metrics.NewRegisteredMeter("statedb/statesize/accounts/count", nil) + stateSizeAccountsBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/accounts/bytes", nil) + stateSizeStorageCountMeter = metrics.NewRegisteredMeter("statedb/statesize/storage/count", nil) + stateSizeStorageBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/storage/bytes", nil) + stateSizeTrieNodesCountMeter = metrics.NewRegisteredMeter("statedb/statesize/trienodes/count", nil) + stateSizeTrieNodesBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/trienodes/bytes", nil) + stateSizeContractsCountMeter = metrics.NewRegisteredMeter("statedb/statesize/contracts/count", nil) + stateSizeContractsBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/contracts/bytes", nil) +) + +// StateSizeMetrics represents the current state size statistics +type StateSizeMetrics struct { + Root common.Hash // Root hash of the state trie + AccountCount uint64 + AccountBytes uint64 + StorageCount uint64 + StorageBytes uint64 + TrieNodeCount uint64 + TrieNodeBytes uint64 + ContractCount uint64 + ContractBytes uint64 +} + +// stateSizeGenerator handles the initialization and tracking of state size metrics +type stateSizeGenerator struct { + db ethdb.KeyValueStore + sdb Database + + // Generator state + running bool + abort chan chan struct{} + done chan struct{} + + // Metrics state + metrics *StateSizeMetrics + metricsLock sync.RWMutex +} + +// newStateSizeGenerator creates a new state size generator +func newStateSizeGenerator(db ethdb.KeyValueStore, sdb Database, root common.Hash) *stateSizeGenerator { + return &stateSizeGenerator{ + db: db, + sdb: sdb, + abort: make(chan chan struct{}), + done: make(chan struct{}), + metrics: &StateSizeMetrics{Root: root}, + } +} + +// run starts the state size initialization in the background +func (g *stateSizeGenerator) run() { + if g.running { + g.stop() + log.Warn("Paused the leftover state size generation cycle") + } + g.running = true + go g.generate() +} + +// stop terminates the background generation +func (g *stateSizeGenerator) stop() { + if !g.running { + return + } + ch := make(chan struct{}) + g.abort <- ch + <-ch + g.running = false +} + +// generate performs the state size initialization +func (g *stateSizeGenerator) generate() { + defer close(g.done) + start := time.Now() + + if g.hasExistingMetrics() { + log.Info("State size metrics already initialized") + return + } + + // Wait for snapshot generator to complete + if db := g.sdb.TrieDB(); db != nil { + for !db.SnapshotCompleted() { + time.Sleep(5 * time.Second) + } + } + + log.Info("Starting state size initialization") + g.initializeMetrics() + log.Info("Completed state size initialization", "elapsed", time.Since(start)) +} + +// hasExistingMetrics checks if state size metrics already exist in the database +func (g *stateSizeGenerator) hasExistingMetrics() bool { + // Check for existing metrics by looking for a marker key + marker := rawdb.ReadStateSizeMetrics(g.db) + // TODO: check if the marker's root is the same as the current root + return marker != nil +} + +// initializeMetrics performs the actual metrics initialization +func (g *stateSizeGenerator) initializeMetrics() { + var ( + accountCount, accountBytes uint64 + storageCount, storageBytes uint64 + trieNodeCount, trieNodeBytes uint64 + contractCount, contractBytes uint64 + ) + + // Process accounts + log.Info("Initializing account metrics") + accountIter := g.db.NewIterator(rawdb.SnapshotAccountPrefix, nil) + defer accountIter.Release() + + for accountIter.Next() { + key := accountIter.Key() + value := accountIter.Value() + + // Count account + accountCount++ + accountBytes += uint64(len(key) + len(value)) + + // Check if account has code (contract) + var account types.StateAccount + if err := rlp.DecodeBytes(value, &account); err == nil { + if !bytes.Equal(account.CodeHash, types.EmptyCodeHash[:]) { + contractCount++ + // Code size will be counted separately + } + } + + // Process storage for this account + storageIter := g.db.NewIterator(append(rawdb.SnapshotStoragePrefix, key[1:]...), nil) + for storageIter.Next() { + storageKey := storageIter.Key() + storageValue := storageIter.Value() + storageCount++ + storageBytes += uint64(len(storageKey) + len(storageValue)) + } + storageIter.Release() + + // Check for abort + select { + case abort := <-g.abort: + close(abort) + return + default: + } + } + + // Process trie nodes + log.Info("Initializing trie node metrics") + trieNodeIter := g.db.NewIterator(rawdb.TrieNodeAccountPrefix, nil) + defer trieNodeIter.Release() + + for trieNodeIter.Next() { + key := trieNodeIter.Key() + value := trieNodeIter.Value() + trieNodeCount++ + trieNodeBytes += uint64(len(key) + len(value)) + } + + // Process storage trie nodes + storageTrieIter := g.db.NewIterator(rawdb.TrieNodeStoragePrefix, nil) + defer storageTrieIter.Release() + + for storageTrieIter.Next() { + key := storageTrieIter.Key() + value := storageTrieIter.Value() + trieNodeCount++ + trieNodeBytes += uint64(len(key) + len(value)) + } + + // Process contract code + log.Info("Initializing contract code metrics") + codeIter := g.db.NewIterator(rawdb.CodePrefix, nil) + defer codeIter.Release() + + for codeIter.Next() { + key := codeIter.Key() + value := codeIter.Value() + contractBytes += uint64(len(key) + len(value)) + } + + // Update metrics + g.metricsLock.Lock() + g.metrics.AccountCount = accountCount + g.metrics.AccountBytes = accountBytes + g.metrics.StorageCount = storageCount + g.metrics.StorageBytes = storageBytes + g.metrics.TrieNodeCount = trieNodeCount + g.metrics.TrieNodeBytes = trieNodeBytes + g.metrics.ContractCount = contractCount + g.metrics.ContractBytes = contractBytes + g.metricsLock.Unlock() + + // Update metrics in database + g.persistMetrics() + + // Update global metrics + stateSizeAccountsCountMeter.Mark(int64(accountCount)) + stateSizeAccountsBytesMeter.Mark(int64(accountBytes)) + stateSizeStorageCountMeter.Mark(int64(storageCount)) + stateSizeStorageBytesMeter.Mark(int64(storageBytes)) + stateSizeTrieNodesCountMeter.Mark(int64(trieNodeCount)) + stateSizeTrieNodesBytesMeter.Mark(int64(trieNodeBytes)) + stateSizeContractsCountMeter.Mark(int64(contractCount)) + stateSizeContractsBytesMeter.Mark(int64(contractBytes)) +} + +// persistMetrics saves the current metrics to the database +func (g *stateSizeGenerator) persistMetrics() { + g.metricsLock.RLock() + metrics := *g.metrics + g.metricsLock.RUnlock() + + data, err := rlp.EncodeToBytes(metrics) + if err != nil { + log.Error("Failed to encode state size metrics", "err", err) + return + } + + batch := g.db.NewBatch() + rawdb.WriteStateSizeMetrics(batch, data) + if err := batch.Write(); err != nil { + log.Error("Failed to persist state size metrics", "err", err) + } +} + +// updateMetrics updates metrics based on state changes +func (g *stateSizeGenerator) updateMetrics(update *stateUpdate) { + var diff StateSizeMetrics + + // Calculate account changes + for _, data := range update.accounts { + if len(data) > 0 { + diff.AccountCount++ + diff.AccountBytes += uint64(common.HashLength + len(data)) + } + } + + // Calculate storage changes + for _, slots := range update.storages { + for _, data := range slots { + if len(data) > 0 { + diff.StorageCount++ + diff.StorageBytes += uint64(2*common.HashLength + len(data)) + } + } + } + + // Calculate trie node changes + for _, nodeSet := range update.nodes.Sets { + for _, node := range nodeSet.Nodes { + diff.TrieNodeCount++ + diff.TrieNodeBytes += uint64(len(node.Blob)) + } + } + + // Update local metrics + g.metricsLock.Lock() + g.metrics.Root = update.root + g.metrics.AccountCount += diff.AccountCount + g.metrics.AccountBytes += diff.AccountBytes + g.metrics.StorageCount += diff.StorageCount + g.metrics.StorageBytes += diff.StorageBytes + g.metrics.TrieNodeCount += diff.TrieNodeCount + g.metrics.TrieNodeBytes += diff.TrieNodeBytes + g.metricsLock.Unlock() +} From 485d1b136f5f7d4da1f0566f2e033dea6683bab1 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 14:32:32 +0800 Subject: [PATCH 02/43] triedb: Snapshot completed Signed-off-by: jsvisa --- triedb/database.go | 9 +++++++++ triedb/pathdb/database.go | 5 +++++ 2 files changed, 14 insertions(+) diff --git a/triedb/database.go b/triedb/database.go index e2f4334d6e0..d2637bd909a 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -375,3 +375,12 @@ func (db *Database) IsVerkle() bool { func (db *Database) Disk() ethdb.Database { return db.disk } + +// SnapshotCompleted returns the indicator if the snapshot is completed. +func (db *Database) SnapshotCompleted() bool { + pdb, ok := db.backend.(*pathdb.Database) + if !ok { + return false + } + return pdb.SnapshotCompleted() +} diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index 423b921d47d..ae607413f26 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -681,3 +681,8 @@ func (db *Database) StorageIterator(root common.Hash, account common.Hash, seek } return newFastStorageIterator(db, root, account, seek) } + +// SnapshotCompleted returns the indicator if the snapshot generation is completed. +func (db *Database) SnapshotCompleted() bool { + return !db.waitSync && db.tree.bottom().genComplete() +} From cd6b23b3a2664adf4ac7fa491312a7e9dbc1bd9a Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 14:56:05 +0800 Subject: [PATCH 03/43] concurrent iterate snapshot Signed-off-by: jsvisa --- core/state/statesize.go | 121 +++++++++++++++------------------------- 1 file changed, 44 insertions(+), 77 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index d3424d0c9f5..88b1bfc45c3 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -17,13 +17,11 @@ package state import ( - "bytes" "sync" "time" "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/rawdb" - "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" @@ -136,86 +134,55 @@ func (g *stateSizeGenerator) hasExistingMetrics() bool { // initializeMetrics performs the actual metrics initialization func (g *stateSizeGenerator) initializeMetrics() { var ( - accountCount, accountBytes uint64 - storageCount, storageBytes uint64 - trieNodeCount, trieNodeBytes uint64 - contractCount, contractBytes uint64 + wg sync.WaitGroup + accountCount, accountBytes uint64 + storageCount, storageBytes uint64 + trieAccountNodeCount, trieAccountNodeBytes uint64 + trieStorageNodeCount, trieStorageNodeBytes uint64 + contractCount, contractBytes uint64 ) - // Process accounts - log.Info("Initializing account metrics") - accountIter := g.db.NewIterator(rawdb.SnapshotAccountPrefix, nil) - defer accountIter.Release() - - for accountIter.Next() { - key := accountIter.Key() - value := accountIter.Value() - - // Count account - accountCount++ - accountBytes += uint64(len(key) + len(value)) - - // Check if account has code (contract) - var account types.StateAccount - if err := rlp.DecodeBytes(value, &account); err == nil { - if !bytes.Equal(account.CodeHash, types.EmptyCodeHash[:]) { - contractCount++ - // Code size will be counted separately + iterate := func(prefix []byte, name string, count, bytes uint64) { + defer wg.Done() + + log.Info("Iterating over state size", "table", name) + defer func(st time.Time) { + log.Info("Finished iterating over state size", "table", name, "count", count, "bytes", bytes, "elapsed", common.PrettyDuration(time.Since(st))) + }(time.Now()) + + iter := g.db.NewIterator(prefix, nil) + defer iter.Release() + for iter.Next() { + count++ + bytes += uint64(len(iter.Key()) + len(iter.Value())) + + // Check for abort + select { + case abort := <-g.abort: + close(abort) + default: } } - - // Process storage for this account - storageIter := g.db.NewIterator(append(rawdb.SnapshotStoragePrefix, key[1:]...), nil) - for storageIter.Next() { - storageKey := storageIter.Key() - storageValue := storageIter.Value() - storageCount++ - storageBytes += uint64(len(storageKey) + len(storageValue)) - } - storageIter.Release() - - // Check for abort - select { - case abort := <-g.abort: - close(abort) - return - default: - } } - // Process trie nodes - log.Info("Initializing trie node metrics") - trieNodeIter := g.db.NewIterator(rawdb.TrieNodeAccountPrefix, nil) - defer trieNodeIter.Release() - - for trieNodeIter.Next() { - key := trieNodeIter.Key() - value := trieNodeIter.Value() - trieNodeCount++ - trieNodeBytes += uint64(len(key) + len(value)) + tables := []struct { + prefix []byte + name string + count *uint64 + bytes *uint64 + }{ + {rawdb.SnapshotAccountPrefix, "account", &accountCount, &accountBytes}, + {rawdb.SnapshotStoragePrefix, "storage", &storageCount, &storageBytes}, + {rawdb.TrieNodeAccountPrefix, "trie account node", &trieAccountNodeCount, &trieAccountNodeBytes}, + {rawdb.TrieNodeStoragePrefix, "trie storage node", &trieStorageNodeCount, &trieStorageNodeBytes}, + {rawdb.CodePrefix, "contract code", &contractCount, &contractBytes}, } - - // Process storage trie nodes - storageTrieIter := g.db.NewIterator(rawdb.TrieNodeStoragePrefix, nil) - defer storageTrieIter.Release() - - for storageTrieIter.Next() { - key := storageTrieIter.Key() - value := storageTrieIter.Value() - trieNodeCount++ - trieNodeBytes += uint64(len(key) + len(value)) + wg.Add(len(tables)) + for _, table := range tables { + go iterate(table.prefix, table.name, *table.count, *table.bytes) } - // Process contract code - log.Info("Initializing contract code metrics") - codeIter := g.db.NewIterator(rawdb.CodePrefix, nil) - defer codeIter.Release() - - for codeIter.Next() { - key := codeIter.Key() - value := codeIter.Value() - contractBytes += uint64(len(key) + len(value)) - } + wg.Wait() // Update metrics g.metricsLock.Lock() @@ -223,8 +190,8 @@ func (g *stateSizeGenerator) initializeMetrics() { g.metrics.AccountBytes = accountBytes g.metrics.StorageCount = storageCount g.metrics.StorageBytes = storageBytes - g.metrics.TrieNodeCount = trieNodeCount - g.metrics.TrieNodeBytes = trieNodeBytes + g.metrics.TrieNodeCount = trieAccountNodeCount + trieStorageNodeCount + g.metrics.TrieNodeBytes = trieAccountNodeBytes + trieStorageNodeBytes g.metrics.ContractCount = contractCount g.metrics.ContractBytes = contractBytes g.metricsLock.Unlock() @@ -237,8 +204,8 @@ func (g *stateSizeGenerator) initializeMetrics() { stateSizeAccountsBytesMeter.Mark(int64(accountBytes)) stateSizeStorageCountMeter.Mark(int64(storageCount)) stateSizeStorageBytesMeter.Mark(int64(storageBytes)) - stateSizeTrieNodesCountMeter.Mark(int64(trieNodeCount)) - stateSizeTrieNodesBytesMeter.Mark(int64(trieNodeBytes)) + stateSizeTrieNodesCountMeter.Mark(int64(trieAccountNodeCount + trieStorageNodeCount)) + stateSizeTrieNodesBytesMeter.Mark(int64(trieStorageNodeBytes + trieStorageNodeBytes)) stateSizeContractsCountMeter.Mark(int64(contractCount)) stateSizeContractsBytesMeter.Mark(int64(contractBytes)) } From b34e366bb4d6cc61adcfa48af61e1e6a898813f1 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 15:04:18 +0800 Subject: [PATCH 04/43] stop and close iterator Signed-off-by: jsvisa --- core/state/statesize.go | 13 +++++-------- 1 file changed, 5 insertions(+), 8 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index 88b1bfc45c3..ab5fb544dbf 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -61,7 +61,7 @@ type stateSizeGenerator struct { // Generator state running bool - abort chan chan struct{} + abort chan struct{} done chan struct{} // Metrics state @@ -74,7 +74,7 @@ func newStateSizeGenerator(db ethdb.KeyValueStore, sdb Database, root common.Has return &stateSizeGenerator{ db: db, sdb: sdb, - abort: make(chan chan struct{}), + abort: make(chan struct{}), done: make(chan struct{}), metrics: &StateSizeMetrics{Root: root}, } @@ -95,9 +95,7 @@ func (g *stateSizeGenerator) stop() { if !g.running { return } - ch := make(chan struct{}) - g.abort <- ch - <-ch + close(g.abort) g.running = false } @@ -156,10 +154,9 @@ func (g *stateSizeGenerator) initializeMetrics() { count++ bytes += uint64(len(iter.Key()) + len(iter.Value())) - // Check for abort select { - case abort := <-g.abort: - close(abort) + case <-g.abort: + return default: } } From d0b42a90e54c2f9b20941e9d659876084c9385f8 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 15:27:08 +0800 Subject: [PATCH 05/43] state update Signed-off-by: jsvisa --- core/state/statesize.go | 97 +++++++++++++++++++++++++++++++---------- 1 file changed, 73 insertions(+), 24 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index ab5fb544dbf..d88be8a59c3 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -22,6 +22,7 @@ import ( "github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/core/rawdb" + "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" @@ -228,42 +229,90 @@ func (g *stateSizeGenerator) persistMetrics() { // updateMetrics updates metrics based on state changes func (g *stateSizeGenerator) updateMetrics(update *stateUpdate) { - var diff StateSizeMetrics + var ( + accountBytes, storageBytes, nodeBytes, codeBytes int + accountCount, storageCount, nodeCount, codeCount int + ) - // Calculate account changes - for _, data := range update.accounts { - if len(data) > 0 { - diff.AccountCount++ - diff.AccountBytes += uint64(common.HashLength + len(data)) + for addr, oldValue := range update.accountsOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + newValue, exists := update.accounts[addrHash] + if !exists { + log.Warn("State update missing account", "address", addr) + continue + } + if len(newValue) == 0 { + accountCount -= 1 + accountBytes -= common.HashLength } + if len(oldValue) == 0 { + accountCount += 1 + accountBytes += common.HashLength + } + accountBytes += len(newValue) - len(oldValue) } - // Calculate storage changes - for _, slots := range update.storages { - for _, data := range slots { - if len(data) > 0 { - diff.StorageCount++ - diff.StorageBytes += uint64(2*common.HashLength + len(data)) + for addr, slots := range update.storagesOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + subset, exists := update.storages[addrHash] + if !exists { + log.Warn("State update missing storage", "address", addr) + continue + } + for key, oldValue := range slots { + var ( + exists bool + newValue []byte + ) + if update.rawStorageKey { + newValue, exists = subset[crypto.Keccak256Hash(key.Bytes())] + } else { + newValue, exists = subset[key] + } + if !exists { + log.Warn("State update missing storage slot", "address", addr, "key", key) + continue } + if len(newValue) == 0 { + storageCount -= 1 + storageBytes -= common.HashLength + } + if len(oldValue) == 0 { + storageCount += 1 + storageBytes += common.HashLength + } + storageBytes += len(newValue) - len(oldValue) } } - - // Calculate trie node changes - for _, nodeSet := range update.nodes.Sets { - for _, node := range nodeSet.Nodes { - diff.TrieNodeCount++ - diff.TrieNodeBytes += uint64(len(node.Blob)) + for _, subset := range update.nodes.Sets { + for path, n := range subset.Nodes { + if len(n.Blob) == 0 { + nodeCount -= 1 + nodeBytes -= len(path) + common.HashLength + } + prev, ok := subset.Origins[path] + if ok { + nodeCount += 1 + nodeBytes += len(path) + common.HashLength + } + nodeBytes += len(n.Blob) - len(prev) } } + for _, code := range update.codes { + codeCount += 1 + codeBytes += len(code.blob) + common.HashLength // no deduplication + } // Update local metrics g.metricsLock.Lock() g.metrics.Root = update.root - g.metrics.AccountCount += diff.AccountCount - g.metrics.AccountBytes += diff.AccountBytes - g.metrics.StorageCount += diff.StorageCount - g.metrics.StorageBytes += diff.StorageBytes - g.metrics.TrieNodeCount += diff.TrieNodeCount - g.metrics.TrieNodeBytes += diff.TrieNodeBytes + g.metrics.AccountCount += uint64(accountCount) + g.metrics.AccountBytes += uint64(accountBytes) + g.metrics.StorageCount += uint64(storageCount) + g.metrics.StorageBytes += uint64(storageBytes) + g.metrics.TrieNodeCount += uint64(nodeCount) + g.metrics.TrieNodeBytes += uint64(nodeBytes) + g.metrics.ContractCount += uint64(codeCount) + g.metrics.ContractBytes += uint64(codeBytes) g.metricsLock.Unlock() } From 325b24125aebbdd8b89c0fee65e29e7b54e799af Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 20:42:11 +0800 Subject: [PATCH 06/43] core: track state size in blockchain.go Signed-off-by: jsvisa --- core/blockchain.go | 20 ++- core/state/database.go | 5 + core/state/statedb.go | 26 ++- core/state/statesize.go | 387 +++++++++++++++++++++++++--------------- 4 files changed, 280 insertions(+), 158 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index dc6e7e90401..790ec1f1525 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -334,6 +334,8 @@ type BlockChain struct { processor Processor // Block transaction processor interface logger *tracing.Hooks + stateSizeGen *state.StateSizeGenerator // State size tracking + lastForkReadyAlert time.Time // Last time there was a fork readiness print out } @@ -526,6 +528,11 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, if bc.cfg.TxLookupLimit >= 0 { bc.txIndexer = newTxIndexer(uint64(bc.cfg.TxLookupLimit), bc) } + + // Start state size tracker + bc.stateSizeGen = state.NewStateSizeGenerator(bc.statedb.DiskDB(), bc.triedb, head.Root) + log.Info("Started state size generator", "root", head.Root) + return bc, nil } @@ -1313,6 +1320,12 @@ func (bc *BlockChain) Stop() { } } } + // Stop state size generator if running + if bc.stateSizeGen != nil { + bc.stateSizeGen.Stop() + log.Info("Stopped state size generator") + } + // Allow tracers to clean-up and release resources. if bc.logger != nil && bc.logger.OnClose != nil { bc.logger.OnClose() @@ -1586,10 +1599,15 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types. log.Crit("Failed to write block into disk", "err", err) } // Commit all cached state changes into underlying memory database. - root, err := statedb.Commit(block.NumberU64(), bc.chainConfig.IsEIP158(block.Number()), bc.chainConfig.IsCancun(block.Number(), block.Time())) + root, stateUpdate, err := statedb.CommitWithUpdate(block.NumberU64(), bc.chainConfig.IsEIP158(block.Number()), bc.chainConfig.IsCancun(block.Number(), block.Time())) if err != nil { return err } + + // Track state size changes if generator is running + if bc.stateSizeGen != nil && stateUpdate != nil { + bc.stateSizeGen.Track(stateUpdate) + } // If node is running in path mode, skip explicit gc operation // which is unnecessary in this mode. if bc.triedb.Scheme() == rawdb.PathScheme { diff --git a/core/state/database.go b/core/state/database.go index 3a0ac422ee4..052e2acab93 100644 --- a/core/state/database.go +++ b/core/state/database.go @@ -285,6 +285,11 @@ func (db *CachingDB) TrieDB() *triedb.Database { return db.triedb } +// DiskDB returns the underlying disk database for direct access. +func (db *CachingDB) DiskDB() ethdb.KeyValueStore { + return db.disk +} + // PointCache returns the cache of evaluated curve points. func (db *CachingDB) PointCache() *utils.PointCache { return db.pointCache diff --git a/core/state/statedb.go b/core/state/statedb.go index 2c41c2081b1..265561df2d0 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -139,9 +139,6 @@ type StateDB struct { witness *stateless.Witness witnessStats *stateless.WitnessStats - // State size tracking - stateSizeGen *stateSizeGenerator - // Measurements gathered during execution for debugging purposes AccountReads time.Duration AccountHashes time.Duration @@ -190,14 +187,6 @@ func NewWithReader(root common.Hash, db Database, reader Reader) (*StateDB, erro sdb.accessEvents = NewAccessEvents(db.PointCache()) } - // Initialize state size tracking - if cachingDB, ok := db.(*CachingDB); ok { - sdb.stateSizeGen = newStateSizeGenerator(cachingDB.disk, db, root) - if !db.TrieDB().IsVerkle() { - sdb.stateSizeGen.run() - } - } - return sdb, nil } @@ -1362,11 +1351,6 @@ func (s *StateDB) commitAndFlush(block uint64, deleteEmptyObjects bool, noStorag return nil, err } s.TrieDBCommits += time.Since(start) - - // Update state size metrics - if s.stateSizeGen != nil { - s.stateSizeGen.updateMetrics(ret) - } } } s.reader, _ = s.db.Reader(s.originalRoot) @@ -1395,6 +1379,16 @@ func (s *StateDB) Commit(block uint64, deleteEmptyObjects bool, noStorageWiping return ret.root, nil } +// CommitWithUpdate writes the state mutations and returns both the root hash and the state update. +// This is useful for tracking state changes at the blockchain level. +func (s *StateDB) CommitWithUpdate(block uint64, deleteEmptyObjects bool, noStorageWiping bool) (common.Hash, *stateUpdate, error) { + ret, err := s.commitAndFlush(block, deleteEmptyObjects, noStorageWiping) + if err != nil { + return common.Hash{}, nil, err + } + return ret.root, ret, nil +} + // Prepare handles the preparatory steps for executing a state transition with. // This method must be invoked before state transition. // diff --git a/core/state/statesize.go b/core/state/statesize.go index d88be8a59c3..f81181ee4d8 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -27,6 +27,7 @@ import ( "github.com/ethereum/go-ethereum/log" "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/rlp" + "github.com/ethereum/go-ethereum/triedb" ) // State size metrics @@ -55,75 +56,272 @@ type StateSizeMetrics struct { ContractBytes uint64 } -// stateSizeGenerator handles the initialization and tracking of state size metrics -type stateSizeGenerator struct { - db ethdb.KeyValueStore - sdb Database +// StateSizeGenerator handles the initialization and tracking of state size metrics +type StateSizeGenerator struct { + db ethdb.KeyValueStore + triedb *triedb.Database // Generator state running bool abort chan struct{} done chan struct{} + // Async message channel for updates + updateChan chan *stateUpdate + // Metrics state - metrics *StateSizeMetrics - metricsLock sync.RWMutex + metrics *StateSizeMetrics + buffered *StateSizeMetrics } -// newStateSizeGenerator creates a new state size generator -func newStateSizeGenerator(db ethdb.KeyValueStore, sdb Database, root common.Hash) *stateSizeGenerator { - return &stateSizeGenerator{ - db: db, - sdb: sdb, - abort: make(chan struct{}), - done: make(chan struct{}), - metrics: &StateSizeMetrics{Root: root}, +// NewStateSizeGenerator creates a new state size generator and starts it automatically +func NewStateSizeGenerator(db ethdb.KeyValueStore, triedb *triedb.Database, root common.Hash) *StateSizeGenerator { + g := &StateSizeGenerator{ + db: db, + triedb: triedb, + abort: make(chan struct{}), + done: make(chan struct{}), + updateChan: make(chan *stateUpdate, 1000), // Buffered channel for updates + metrics: &StateSizeMetrics{Root: root}, + buffered: &StateSizeMetrics{Root: root}, } -} -// run starts the state size initialization in the background -func (g *stateSizeGenerator) run() { - if g.running { - g.stop() - log.Warn("Paused the leftover state size generation cycle") - } + // Start the generator automatically g.running = true go g.generate() + + return g } // stop terminates the background generation -func (g *stateSizeGenerator) stop() { +func (g *StateSizeGenerator) Stop() { if !g.running { return } + + // Signal the goroutine to stop close(g.abort) + + // Wait for the goroutine to actually finish + <-g.done + + // Now it's safe to persist metrics since the goroutine has stopped g.running = false + g.persistMetrics() +} + +// isRunning returns true if the generator is currently running +func (g *StateSizeGenerator) IsRunning() bool { + return g.running +} + +// waitForCompletion waits for the generator to complete (useful for testing or graceful shutdown) +func (g *StateSizeGenerator) WaitForCompletion() { + if g.running { + <-g.done + } } -// generate performs the state size initialization -func (g *stateSizeGenerator) generate() { +// generate performs the state size initialization and handles updates +func (g *StateSizeGenerator) generate() { defer close(g.done) - start := time.Now() + + var inited bool + var initDone chan struct{} if g.hasExistingMetrics() { log.Info("State size metrics already initialized") - return + inited = true } // Wait for snapshot generator to complete - if db := g.sdb.TrieDB(); db != nil { - for !db.SnapshotCompleted() { - time.Sleep(5 * time.Second) + snapDone := make(chan struct{}) + go func() { + defer close(snapDone) + + for !g.triedb.SnapshotCompleted() { + select { + case <-g.abort: + log.Info("State size generation aborted during snapshot") + return + default: + time.Sleep(10 * time.Second) + } + } + }() + + for { + select { + case update := <-g.updateChan: + g.handleUpdate(update, inited) + case <-g.abort: + log.Info("State size generation aborted") + // Wait for initialization goroutine to finish if it's running + if initDone != nil { + select { + case <-initDone: + case <-time.After(5 * time.Second): + log.Warn("Initialization goroutine did not finish in time") + } + } + return + case <-snapDone: + if !inited { + initDone = make(chan struct{}) + go func() { + defer close(initDone) + start := time.Now() + log.Info("Starting state size initialization") + g.initializeMetrics() + log.Info("Completed state size initialization", "elapsed", time.Since(start)) + inited = true + }() + } + case <-initDone: + // Initialization completed, merge buffered metrics if needed + if g.buffered != nil && g.buffered.Root != (common.Hash{}) { + log.Info("Merging buffered metrics into main metrics") + g.metrics.AccountCount += g.buffered.AccountCount + g.metrics.AccountBytes += g.buffered.AccountBytes + g.metrics.StorageCount += g.buffered.StorageCount + g.metrics.StorageBytes += g.buffered.StorageBytes + g.metrics.TrieNodeCount += g.buffered.TrieNodeCount + g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes + g.metrics.ContractCount += g.buffered.ContractCount + g.metrics.ContractBytes += g.buffered.ContractBytes + // Reset buffered metrics + g.buffered = &StateSizeMetrics{Root: g.metrics.Root} + } + initDone = nil // Clear the channel } } +} + +// handleUpdate processes a single update with proper root continuity checking +func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, inited bool) { + // TODO: Check if the update root matches the current metrics root + + // Calculate the diff + diff := g.calculateUpdateDiff(update) - log.Info("Starting state size initialization") - g.initializeMetrics() - log.Info("Completed state size initialization", "elapsed", time.Since(start)) + var m *StateSizeMetrics + if inited { + m = g.metrics + } else { + m = g.buffered + } + + // TODO: When to merge the buffered metrics into the main metrics + m.Root = update.root + m.AccountCount += diff.AccountCount + m.AccountBytes += diff.AccountBytes + m.StorageCount += diff.StorageCount + m.StorageBytes += diff.StorageBytes + m.TrieNodeCount += diff.TrieNodeCount + m.TrieNodeBytes += diff.TrieNodeBytes + m.ContractCount += diff.ContractCount + m.ContractBytes += diff.ContractBytes + + // Fire the metrics only if the initialization is done + if inited { + g.updateMetrics() + g.persistMetrics() + } +} + +// calculateUpdateDiff calculates the diff for a state update +func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeMetrics { + var diff StateSizeMetrics + + // Calculate account changes + for addr, oldValue := range update.accountsOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + newValue, exists := update.accounts[addrHash] + if !exists { + log.Warn("State update missing account", "address", addr) + continue + } + if len(newValue) == 0 { + diff.AccountCount -= 1 + diff.AccountBytes -= common.HashLength + } + if len(oldValue) == 0 { + diff.AccountCount += 1 + diff.AccountBytes += common.HashLength + } + diff.AccountBytes += uint64(len(newValue) - len(oldValue)) + } + + // Calculate storage changes + for addr, slots := range update.storagesOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + subset, exists := update.storages[addrHash] + if !exists { + log.Warn("State update missing storage", "address", addr) + continue + } + for key, oldValue := range slots { + var ( + exists bool + newValue []byte + ) + if update.rawStorageKey { + newValue, exists = subset[crypto.Keccak256Hash(key.Bytes())] + } else { + newValue, exists = subset[key] + } + if !exists { + log.Warn("State update missing storage slot", "address", addr, "key", key) + continue + } + if len(newValue) == 0 { + diff.StorageCount -= 1 + diff.StorageBytes -= common.HashLength + } + if len(oldValue) == 0 { + diff.StorageCount += 1 + diff.StorageBytes += common.HashLength + } + diff.StorageBytes += uint64(len(newValue) - len(oldValue)) + } + } + + // Calculate trie node changes + for _, subset := range update.nodes.Sets { + for path, n := range subset.Nodes { + if len(n.Blob) == 0 { + diff.TrieNodeCount -= 1 + diff.TrieNodeBytes -= uint64(len(path) + common.HashLength) + } + prev, ok := subset.Origins[path] + if ok { + diff.TrieNodeCount += 1 + diff.TrieNodeBytes += uint64(len(path) + common.HashLength) + } + diff.TrieNodeBytes += uint64(len(n.Blob) - len(prev)) + } + } + + // Calculate code changes + for _, code := range update.codes { + diff.ContractCount += 1 + diff.ContractBytes += uint64(len(code.blob) + common.HashLength) + } + + return diff +} + +// Track is an async method used to send the state update to the generator. +// It ignores empty updates (where no state changes occurred). +func (g *StateSizeGenerator) Track(update *stateUpdate) { + if update == nil || update.empty() { + return + } + g.updateChan <- update } // hasExistingMetrics checks if state size metrics already exist in the database -func (g *stateSizeGenerator) hasExistingMetrics() bool { +func (g *StateSizeGenerator) hasExistingMetrics() bool { // Check for existing metrics by looking for a marker key marker := rawdb.ReadStateSizeMetrics(g.db) // TODO: check if the marker's root is the same as the current root @@ -131,7 +329,7 @@ func (g *stateSizeGenerator) hasExistingMetrics() bool { } // initializeMetrics performs the actual metrics initialization -func (g *stateSizeGenerator) initializeMetrics() { +func (g *StateSizeGenerator) initializeMetrics() { var ( wg sync.WaitGroup accountCount, accountBytes uint64 @@ -155,6 +353,7 @@ func (g *stateSizeGenerator) initializeMetrics() { count++ bytes += uint64(len(iter.Key()) + len(iter.Value())) + // Check for abort select { case <-g.abort: return @@ -183,7 +382,6 @@ func (g *stateSizeGenerator) initializeMetrics() { wg.Wait() // Update metrics - g.metricsLock.Lock() g.metrics.AccountCount = accountCount g.metrics.AccountBytes = accountBytes g.metrics.StorageCount = storageCount @@ -192,29 +390,26 @@ func (g *stateSizeGenerator) initializeMetrics() { g.metrics.TrieNodeBytes = trieAccountNodeBytes + trieStorageNodeBytes g.metrics.ContractCount = contractCount g.metrics.ContractBytes = contractBytes - g.metricsLock.Unlock() - // Update metrics in database + g.updateMetrics() g.persistMetrics() +} +func (g *StateSizeGenerator) updateMetrics() { // Update global metrics - stateSizeAccountsCountMeter.Mark(int64(accountCount)) - stateSizeAccountsBytesMeter.Mark(int64(accountBytes)) - stateSizeStorageCountMeter.Mark(int64(storageCount)) - stateSizeStorageBytesMeter.Mark(int64(storageBytes)) - stateSizeTrieNodesCountMeter.Mark(int64(trieAccountNodeCount + trieStorageNodeCount)) - stateSizeTrieNodesBytesMeter.Mark(int64(trieStorageNodeBytes + trieStorageNodeBytes)) - stateSizeContractsCountMeter.Mark(int64(contractCount)) - stateSizeContractsBytesMeter.Mark(int64(contractBytes)) + stateSizeAccountsCountMeter.Mark(int64(g.metrics.AccountCount)) + stateSizeAccountsBytesMeter.Mark(int64(g.metrics.AccountBytes)) + stateSizeStorageCountMeter.Mark(int64(g.metrics.StorageCount)) + stateSizeStorageBytesMeter.Mark(int64(g.metrics.StorageBytes)) + stateSizeTrieNodesCountMeter.Mark(int64(g.metrics.TrieNodeCount)) + stateSizeTrieNodesBytesMeter.Mark(int64(g.metrics.TrieNodeBytes)) + stateSizeContractsCountMeter.Mark(int64(g.metrics.ContractCount)) + stateSizeContractsBytesMeter.Mark(int64(g.metrics.ContractBytes)) } // persistMetrics saves the current metrics to the database -func (g *stateSizeGenerator) persistMetrics() { - g.metricsLock.RLock() - metrics := *g.metrics - g.metricsLock.RUnlock() - - data, err := rlp.EncodeToBytes(metrics) +func (g *StateSizeGenerator) persistMetrics() { + data, err := rlp.EncodeToBytes(*g.metrics) if err != nil { log.Error("Failed to encode state size metrics", "err", err) return @@ -226,93 +421,3 @@ func (g *stateSizeGenerator) persistMetrics() { log.Error("Failed to persist state size metrics", "err", err) } } - -// updateMetrics updates metrics based on state changes -func (g *stateSizeGenerator) updateMetrics(update *stateUpdate) { - var ( - accountBytes, storageBytes, nodeBytes, codeBytes int - accountCount, storageCount, nodeCount, codeCount int - ) - - for addr, oldValue := range update.accountsOrigin { - addrHash := crypto.Keccak256Hash(addr.Bytes()) - newValue, exists := update.accounts[addrHash] - if !exists { - log.Warn("State update missing account", "address", addr) - continue - } - if len(newValue) == 0 { - accountCount -= 1 - accountBytes -= common.HashLength - } - if len(oldValue) == 0 { - accountCount += 1 - accountBytes += common.HashLength - } - accountBytes += len(newValue) - len(oldValue) - } - - for addr, slots := range update.storagesOrigin { - addrHash := crypto.Keccak256Hash(addr.Bytes()) - subset, exists := update.storages[addrHash] - if !exists { - log.Warn("State update missing storage", "address", addr) - continue - } - for key, oldValue := range slots { - var ( - exists bool - newValue []byte - ) - if update.rawStorageKey { - newValue, exists = subset[crypto.Keccak256Hash(key.Bytes())] - } else { - newValue, exists = subset[key] - } - if !exists { - log.Warn("State update missing storage slot", "address", addr, "key", key) - continue - } - if len(newValue) == 0 { - storageCount -= 1 - storageBytes -= common.HashLength - } - if len(oldValue) == 0 { - storageCount += 1 - storageBytes += common.HashLength - } - storageBytes += len(newValue) - len(oldValue) - } - } - for _, subset := range update.nodes.Sets { - for path, n := range subset.Nodes { - if len(n.Blob) == 0 { - nodeCount -= 1 - nodeBytes -= len(path) + common.HashLength - } - prev, ok := subset.Origins[path] - if ok { - nodeCount += 1 - nodeBytes += len(path) + common.HashLength - } - nodeBytes += len(n.Blob) - len(prev) - } - } - for _, code := range update.codes { - codeCount += 1 - codeBytes += len(code.blob) + common.HashLength // no deduplication - } - - // Update local metrics - g.metricsLock.Lock() - g.metrics.Root = update.root - g.metrics.AccountCount += uint64(accountCount) - g.metrics.AccountBytes += uint64(accountBytes) - g.metrics.StorageCount += uint64(storageCount) - g.metrics.StorageBytes += uint64(storageBytes) - g.metrics.TrieNodeCount += uint64(nodeCount) - g.metrics.TrieNodeBytes += uint64(nodeBytes) - g.metrics.ContractCount += uint64(codeCount) - g.metrics.ContractBytes += uint64(codeBytes) - g.metricsLock.Unlock() -} From b1e0361d3259787c5cc37d252057ad60dfb94571 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 22:04:45 +0800 Subject: [PATCH 07/43] set state root Signed-off-by: jsvisa --- core/state/statesize.go | 387 ++++++++++++++++++++++++-------------- triedb/database.go | 4 +- triedb/pathdb/database.go | 13 +- 3 files changed, 254 insertions(+), 150 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index f81181ee4d8..a1f0aba8968 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -17,7 +17,8 @@ package state import ( - "sync" + "context" + "sync/atomic" "time" "github.com/ethereum/go-ethereum/common" @@ -28,6 +29,7 @@ import ( "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/triedb" + "golang.org/x/sync/errgroup" ) // State size metrics @@ -62,16 +64,18 @@ type StateSizeGenerator struct { triedb *triedb.Database // Generator state - running bool - abort chan struct{} - done chan struct{} + abort chan struct{} + done chan struct{} // Async message channel for updates updateChan chan *stateUpdate - // Metrics state + // Metrics state (only modified by generate() goroutine) metrics *StateSizeMetrics buffered *StateSizeMetrics + + // Initialization state + initialized atomic.Bool } // NewStateSizeGenerator creates a new state size generator and starts it automatically @@ -87,143 +91,156 @@ func NewStateSizeGenerator(db ethdb.KeyValueStore, triedb *triedb.Database, root } // Start the generator automatically - g.running = true go g.generate() return g } -// stop terminates the background generation +// Stop terminates the background generation func (g *StateSizeGenerator) Stop() { - if !g.running { - return - } - - // Signal the goroutine to stop close(g.abort) - // Wait for the goroutine to actually finish <-g.done - // Now it's safe to persist metrics since the goroutine has stopped - g.running = false + // Persist metrics after all the goroutines were stopped g.persistMetrics() } -// isRunning returns true if the generator is currently running -func (g *StateSizeGenerator) IsRunning() bool { - return g.running -} - -// waitForCompletion waits for the generator to complete (useful for testing or graceful shutdown) -func (g *StateSizeGenerator) WaitForCompletion() { - if g.running { - <-g.done - } -} - // generate performs the state size initialization and handles updates func (g *StateSizeGenerator) generate() { defer close(g.done) - var inited bool - var initDone chan struct{} + initDone := g.initialize() + + for { + select { + case update := <-g.updateChan: + g.handleUpdate(update, g.initialized.Load()) + + case <-g.abort: + log.Info("State size generation aborted") + + // Wait for initialization to complete with timeout + if initDone != nil { + select { + case <-initDone: + log.Debug("Initialization completed before abort") + case <-time.After(5 * time.Second): + log.Warn("Initialization did not finish in time during abort") + } + } + return + + case <-initDone: + // Initialization completed, merge buffered metrics + g.mergeBufferedMetrics() + initDone = nil // Clear the channel to prevent future selects + } + } +} +// initialize starts the initialization process if not already initialized +func (g *StateSizeGenerator) initialize() chan struct{} { + initDone := make(chan struct{}) + + // Check if we already have existing metrics if g.hasExistingMetrics() { log.Info("State size metrics already initialized") - inited = true + g.initialized.Store(true) + close(initDone) + return initDone } - // Wait for snapshot generator to complete - snapDone := make(chan struct{}) + // Wait for snapshot completion and then initialize go func() { - defer close(snapDone) + defer close(initDone) - for !g.triedb.SnapshotCompleted() { + LOOP: + // Wait for snapshot generator to complete first + for { select { case <-g.abort: - log.Info("State size generation aborted during snapshot") + log.Info("State size initialization aborted during snapshot wait") return default: + root, done := g.triedb.SnapshotCompleted() + if done { + g.metrics.Root = root + break LOOP + } time.Sleep(10 * time.Second) } } - }() - for { - select { - case update := <-g.updateChan: - g.handleUpdate(update, inited) - case <-g.abort: - log.Info("State size generation aborted") - // Wait for initialization goroutine to finish if it's running - if initDone != nil { - select { - case <-initDone: - case <-time.After(5 * time.Second): - log.Warn("Initialization goroutine did not finish in time") - } - } + // Start actual initialization + start := time.Now() + log.Info("Starting state size initialization") + if err := g.initializeMetrics(); err != nil { + log.Error("Failed to initialize state size metrics", "err", err) return - case <-snapDone: - if !inited { - initDone = make(chan struct{}) - go func() { - defer close(initDone) - start := time.Now() - log.Info("Starting state size initialization") - g.initializeMetrics() - log.Info("Completed state size initialization", "elapsed", time.Since(start)) - inited = true - }() - } - case <-initDone: - // Initialization completed, merge buffered metrics if needed - if g.buffered != nil && g.buffered.Root != (common.Hash{}) { - log.Info("Merging buffered metrics into main metrics") - g.metrics.AccountCount += g.buffered.AccountCount - g.metrics.AccountBytes += g.buffered.AccountBytes - g.metrics.StorageCount += g.buffered.StorageCount - g.metrics.StorageBytes += g.buffered.StorageBytes - g.metrics.TrieNodeCount += g.buffered.TrieNodeCount - g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes - g.metrics.ContractCount += g.buffered.ContractCount - g.metrics.ContractBytes += g.buffered.ContractBytes - // Reset buffered metrics - g.buffered = &StateSizeMetrics{Root: g.metrics.Root} - } - initDone = nil // Clear the channel } + + g.initialized.Store(true) + + log.Info("Completed state size initialization", "elapsed", time.Since(start)) + }() + + return initDone +} + +// mergeBufferedMetrics merges buffered metrics into main metrics +func (g *StateSizeGenerator) mergeBufferedMetrics() { + if g.buffered != nil && g.buffered.Root != (common.Hash{}) { + log.Info("Merging buffered metrics into main metrics") + g.metrics.AccountCount += g.buffered.AccountCount + g.metrics.AccountBytes += g.buffered.AccountBytes + g.metrics.StorageCount += g.buffered.StorageCount + g.metrics.StorageBytes += g.buffered.StorageBytes + g.metrics.TrieNodeCount += g.buffered.TrieNodeCount + g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes + g.metrics.ContractCount += g.buffered.ContractCount + g.metrics.ContractBytes += g.buffered.ContractBytes + + // Reset buffered metrics + g.buffered = &StateSizeMetrics{Root: g.metrics.Root} } } // handleUpdate processes a single update with proper root continuity checking -func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, inited bool) { - // TODO: Check if the update root matches the current metrics root - +func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) { // Calculate the diff diff := g.calculateUpdateDiff(update) - var m *StateSizeMetrics - if inited { - m = g.metrics + var targetMetrics *StateSizeMetrics + if initialized { + targetMetrics = g.metrics } else { - m = g.buffered + targetMetrics = g.buffered } - // TODO: When to merge the buffered metrics into the main metrics - m.Root = update.root - m.AccountCount += diff.AccountCount - m.AccountBytes += diff.AccountBytes - m.StorageCount += diff.StorageCount - m.StorageBytes += diff.StorageBytes - m.TrieNodeCount += diff.TrieNodeCount - m.TrieNodeBytes += diff.TrieNodeBytes - m.ContractCount += diff.ContractCount - m.ContractBytes += diff.ContractBytes - - // Fire the metrics only if the initialization is done - if inited { + // Check root continuity - the update should build on our current state + if targetMetrics.Root != (common.Hash{}) && targetMetrics.Root != update.originRoot { + log.Warn("State update root discontinuity detected", + "current", targetMetrics.Root, + "updateOrigin", update.originRoot, + "updateNew", update.root) + // For now, we accept the discontinuity but log it + // In production, you might want to reset metrics or handle differently + } + + // Update to the new state root + targetMetrics.Root = update.root + targetMetrics.AccountCount += diff.AccountCount + targetMetrics.AccountBytes += diff.AccountBytes + targetMetrics.StorageCount += diff.StorageCount + targetMetrics.StorageBytes += diff.StorageBytes + targetMetrics.TrieNodeCount += diff.TrieNodeCount + targetMetrics.TrieNodeBytes += diff.TrieNodeBytes + targetMetrics.ContractCount += diff.ContractCount + targetMetrics.ContractBytes += diff.ContractBytes + + // Fire the metrics and persist only if initialization is done + if initialized { g.updateMetrics() g.persistMetrics() } @@ -313,25 +330,60 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeM // Track is an async method used to send the state update to the generator. // It ignores empty updates (where no state changes occurred). +// If the channel is full, it drops the update to avoid blocking. func (g *StateSizeGenerator) Track(update *stateUpdate) { if update == nil || update.empty() { return } + g.updateChan <- update } // hasExistingMetrics checks if state size metrics already exist in the database +// and if they are continuous with the current root func (g *StateSizeGenerator) hasExistingMetrics() bool { - // Check for existing metrics by looking for a marker key - marker := rawdb.ReadStateSizeMetrics(g.db) - // TODO: check if the marker's root is the same as the current root - return marker != nil + data := rawdb.ReadStateSizeMetrics(g.db) + if data == nil { + return false + } + + var existed StateSizeMetrics + if err := rlp.DecodeBytes(data, &existed); err != nil { + log.Warn("Failed to decode existing state size metrics", "err", err) + return false + } + + // Check if the existing metrics root matches our current root + if (g.metrics.Root != common.Hash{}) && existed.Root != g.metrics.Root { + log.Info("Existing state size metrics found but root mismatch", "existing", existed.Root, "current", g.metrics.Root) + return false + } + + // Root matches - load the existing metrics + log.Info("Loading existing state size metrics", "root", existed.Root) + g.metrics = &existed + return true } -// initializeMetrics performs the actual metrics initialization -func (g *StateSizeGenerator) initializeMetrics() { +// initializeMetrics performs the actual metrics initialization using errgroup +func (g *StateSizeGenerator) initializeMetrics() error { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + select { + case <-g.abort: + cancel() // Cancel context when abort is signaled + case <-ctx.Done(): + // Context already cancelled + } + }() + + // Create errgroup with context + group, ctx := errgroup.WithContext(ctx) + + // Metrics will be directly updated by each goroutine var ( - wg sync.WaitGroup accountCount, accountBytes uint64 storageCount, storageBytes uint64 trieAccountNodeCount, trieAccountNodeBytes uint64 @@ -339,49 +391,58 @@ func (g *StateSizeGenerator) initializeMetrics() { contractCount, contractBytes uint64 ) - iterate := func(prefix []byte, name string, count, bytes uint64) { - defer wg.Done() - - log.Info("Iterating over state size", "table", name) - defer func(st time.Time) { - log.Info("Finished iterating over state size", "table", name, "count", count, "bytes", bytes, "elapsed", common.PrettyDuration(time.Since(st))) - }(time.Now()) - - iter := g.db.NewIterator(prefix, nil) - defer iter.Release() - for iter.Next() { - count++ - bytes += uint64(len(iter.Key()) + len(iter.Value())) - - // Check for abort - select { - case <-g.abort: - return - default: - } + // Start all table iterations concurrently with direct metric updates + group.Go(func() error { + count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotAccountPrefix, "account") + if err != nil { + return err } - } + accountCount, accountBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotStoragePrefix, "storage") + if err != nil { + return err + } + storageCount, storageBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeAccountPrefix, "trie account node") + if err != nil { + return err + } + trieAccountNodeCount, trieAccountNodeBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeStoragePrefix, "trie storage node") + if err != nil { + return err + } + trieStorageNodeCount, trieStorageNodeBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := g.iterateTable(ctx, rawdb.CodePrefix, "contract code") + if err != nil { + return err + } + contractCount, contractBytes = count, bytes + return nil + }) - tables := []struct { - prefix []byte - name string - count *uint64 - bytes *uint64 - }{ - {rawdb.SnapshotAccountPrefix, "account", &accountCount, &accountBytes}, - {rawdb.SnapshotStoragePrefix, "storage", &storageCount, &storageBytes}, - {rawdb.TrieNodeAccountPrefix, "trie account node", &trieAccountNodeCount, &trieAccountNodeBytes}, - {rawdb.TrieNodeStoragePrefix, "trie storage node", &trieStorageNodeCount, &trieStorageNodeBytes}, - {rawdb.CodePrefix, "contract code", &contractCount, &contractBytes}, - } - wg.Add(len(tables)) - for _, table := range tables { - go iterate(table.prefix, table.name, *table.count, *table.bytes) + // Wait for all goroutines to complete + if err := group.Wait(); err != nil { + return err } - wg.Wait() - - // Update metrics + // Update metrics (safe since we're in the single writer goroutine) g.metrics.AccountCount = accountCount g.metrics.AccountBytes = accountBytes g.metrics.StorageCount = storageCount @@ -393,10 +454,46 @@ func (g *StateSizeGenerator) initializeMetrics() { g.updateMetrics() g.persistMetrics() + + return nil +} + +// iterateTable performs iteration over a specific table and returns the results +func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, name string) (uint64, uint64, error) { + log.Info("Iterating over state size", "table", name) + start := time.Now() + + var count, bytes uint64 + iter := g.db.NewIterator(prefix, nil) + defer iter.Release() + + for iter.Next() { + count++ + bytes += uint64(len(iter.Key()) + len(iter.Value())) + + // Check for cancellation periodically for performance + if count%10000 == 0 { + select { + case <-ctx.Done(): + log.Info("State size iteration cancelled", "table", name, "count", count) + return 0, 0, ctx.Err() + default: + } + } + } + + // Check for iterator errors + if err := iter.Error(); err != nil { + log.Error("Iterator error during state size calculation", "table", name, "err", err) + return 0, 0, err + } + + log.Info("Finished iterating over state size", "table", name, "count", count, "bytes", bytes, "elapsed", common.PrettyDuration(time.Since(start))) + + return count, bytes, nil } func (g *StateSizeGenerator) updateMetrics() { - // Update global metrics stateSizeAccountsCountMeter.Mark(int64(g.metrics.AccountCount)) stateSizeAccountsBytesMeter.Mark(int64(g.metrics.AccountBytes)) stateSizeStorageCountMeter.Mark(int64(g.metrics.StorageCount)) diff --git a/triedb/database.go b/triedb/database.go index d2637bd909a..cba35a67e9c 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -377,10 +377,10 @@ func (db *Database) Disk() ethdb.Database { } // SnapshotCompleted returns the indicator if the snapshot is completed. -func (db *Database) SnapshotCompleted() bool { +func (db *Database) SnapshotCompleted() (common.Hash, bool) { pdb, ok := db.backend.(*pathdb.Database) if !ok { - return false + return common.Hash{}, false } return pdb.SnapshotCompleted() } diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index ae607413f26..0156ddce1b8 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -682,7 +682,14 @@ func (db *Database) StorageIterator(root common.Hash, account common.Hash, seek return newFastStorageIterator(db, root, account, seek) } -// SnapshotCompleted returns the indicator if the snapshot generation is completed. -func (db *Database) SnapshotCompleted() bool { - return !db.waitSync && db.tree.bottom().genComplete() +// SnapshotCompleted returns the snapshot root if the snapshot generation is completed. +func (db *Database) SnapshotCompleted() (common.Hash, bool) { + if db.waitSync { + return common.Hash{}, false + } + dl := db.tree.bottom() + if dl.genComplete() { + return dl.rootHash(), true + } + return common.Hash{}, false } From f0c815e4c9ef942474125e18e179c962e5f50671 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 6 Aug 2025 22:17:24 +0800 Subject: [PATCH 08/43] no need chan if init done Signed-off-by: jsvisa --- core/state/statesize.go | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index a1f0aba8968..0a03df5a5d3 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -141,16 +141,15 @@ func (g *StateSizeGenerator) generate() { // initialize starts the initialization process if not already initialized func (g *StateSizeGenerator) initialize() chan struct{} { - initDone := make(chan struct{}) - // Check if we already have existing metrics if g.hasExistingMetrics() { log.Info("State size metrics already initialized") g.initialized.Store(true) - close(initDone) - return initDone + return nil } + initDone := make(chan struct{}) + // Wait for snapshot completion and then initialize go func() { defer close(initDone) @@ -166,6 +165,7 @@ func (g *StateSizeGenerator) initialize() chan struct{} { root, done := g.triedb.SnapshotCompleted() if done { g.metrics.Root = root + g.buffered.Root = root break LOOP } time.Sleep(10 * time.Second) @@ -190,8 +190,9 @@ func (g *StateSizeGenerator) initialize() chan struct{} { // mergeBufferedMetrics merges buffered metrics into main metrics func (g *StateSizeGenerator) mergeBufferedMetrics() { - if g.buffered != nil && g.buffered.Root != (common.Hash{}) { + if g.buffered != nil { log.Info("Merging buffered metrics into main metrics") + g.metrics.Root = g.buffered.Root g.metrics.AccountCount += g.buffered.AccountCount g.metrics.AccountBytes += g.buffered.AccountBytes g.metrics.StorageCount += g.buffered.StorageCount @@ -201,8 +202,7 @@ func (g *StateSizeGenerator) mergeBufferedMetrics() { g.metrics.ContractCount += g.buffered.ContractCount g.metrics.ContractBytes += g.buffered.ContractBytes - // Reset buffered metrics - g.buffered = &StateSizeMetrics{Root: g.metrics.Root} + g.buffered = nil } } From 2d71e2b38296d27525e19988cc9931e1713cacab Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 7 Aug 2025 08:26:31 +0800 Subject: [PATCH 09/43] use gauge to replace meter Signed-off-by: jsvisa --- core/state/metrics.go | 10 ++++++++++ core/state/statesize.go | 30 ++++++++---------------------- 2 files changed, 18 insertions(+), 22 deletions(-) diff --git a/core/state/metrics.go b/core/state/metrics.go index dd4b2e98381..57f6d4a4edd 100644 --- a/core/state/metrics.go +++ b/core/state/metrics.go @@ -29,4 +29,14 @@ var ( storageTriesUpdatedMeter = metrics.NewRegisteredMeter("state/update/storagenodes", nil) accountTrieDeletedMeter = metrics.NewRegisteredMeter("state/delete/accountnodes", nil) storageTriesDeletedMeter = metrics.NewRegisteredMeter("state/delete/storagenodes", nil) + + // State size metrics + accountCountGauge = metrics.NewRegisteredGauge("state/account/count", nil) + accountBytesGauge = metrics.NewRegisteredGauge("state/account/bytes", nil) + storageCountGauge = metrics.NewRegisteredGauge("state/storage/count", nil) + storageBytesGauge = metrics.NewRegisteredGauge("state/storage/bytes", nil) + trienodeCountGauge = metrics.NewRegisteredGauge("state/trienode/count", nil) + trienodeBytesGauge = metrics.NewRegisteredGauge("state/trienode/bytes", nil) + contractCountGauge = metrics.NewRegisteredGauge("state/contract/count", nil) + contractBytesGauge = metrics.NewRegisteredGauge("state/contract/bytes", nil) ) diff --git a/core/state/statesize.go b/core/state/statesize.go index 0a03df5a5d3..0219ad994d0 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -26,25 +26,11 @@ import ( "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/triedb" "golang.org/x/sync/errgroup" ) -// State size metrics -var ( - // Baseline state size metrics - stateSizeAccountsCountMeter = metrics.NewRegisteredMeter("statedb/statesize/accounts/count", nil) - stateSizeAccountsBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/accounts/bytes", nil) - stateSizeStorageCountMeter = metrics.NewRegisteredMeter("statedb/statesize/storage/count", nil) - stateSizeStorageBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/storage/bytes", nil) - stateSizeTrieNodesCountMeter = metrics.NewRegisteredMeter("statedb/statesize/trienodes/count", nil) - stateSizeTrieNodesBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/trienodes/bytes", nil) - stateSizeContractsCountMeter = metrics.NewRegisteredMeter("statedb/statesize/contracts/count", nil) - stateSizeContractsBytesMeter = metrics.NewRegisteredMeter("statedb/statesize/contracts/bytes", nil) -) - // StateSizeMetrics represents the current state size statistics type StateSizeMetrics struct { Root common.Hash // Root hash of the state trie @@ -494,14 +480,14 @@ func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, na } func (g *StateSizeGenerator) updateMetrics() { - stateSizeAccountsCountMeter.Mark(int64(g.metrics.AccountCount)) - stateSizeAccountsBytesMeter.Mark(int64(g.metrics.AccountBytes)) - stateSizeStorageCountMeter.Mark(int64(g.metrics.StorageCount)) - stateSizeStorageBytesMeter.Mark(int64(g.metrics.StorageBytes)) - stateSizeTrieNodesCountMeter.Mark(int64(g.metrics.TrieNodeCount)) - stateSizeTrieNodesBytesMeter.Mark(int64(g.metrics.TrieNodeBytes)) - stateSizeContractsCountMeter.Mark(int64(g.metrics.ContractCount)) - stateSizeContractsBytesMeter.Mark(int64(g.metrics.ContractBytes)) + accountCountGauge.Update(int64(g.metrics.AccountCount)) + accountBytesGauge.Update(int64(g.metrics.AccountBytes)) + storageCountGauge.Update(int64(g.metrics.StorageCount)) + storageBytesGauge.Update(int64(g.metrics.StorageBytes)) + trienodeCountGauge.Update(int64(g.metrics.TrieNodeCount)) + trienodeBytesGauge.Update(int64(g.metrics.TrieNodeBytes)) + contractCountGauge.Update(int64(g.metrics.ContractCount)) + contractBytesGauge.Update(int64(g.metrics.ContractBytes)) } // persistMetrics saves the current metrics to the database From 2ddf19a15a95de3ecf8565b5f9267eb4523fdd04 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 7 Aug 2025 08:42:57 +0800 Subject: [PATCH 10/43] fix: int64 instead of uint64 Signed-off-by: jsvisa --- core/state/statesize.go | 148 ++++++++++++++++++---------------------- 1 file changed, 67 insertions(+), 81 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index 0219ad994d0..94b4dcccbdf 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -18,6 +18,7 @@ package state import ( "context" + "encoding/json" "sync/atomic" "time" @@ -26,42 +27,33 @@ import ( "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/rlp" "github.com/ethereum/go-ethereum/triedb" "golang.org/x/sync/errgroup" ) -// StateSizeMetrics represents the current state size statistics -type StateSizeMetrics struct { +// stateSizeMetrics represents the current state size statistics +type stateSizeMetrics struct { Root common.Hash // Root hash of the state trie - AccountCount uint64 - AccountBytes uint64 - StorageCount uint64 - StorageBytes uint64 - TrieNodeCount uint64 - TrieNodeBytes uint64 - ContractCount uint64 - ContractBytes uint64 + AccountCount int64 + AccountBytes int64 + StorageCount int64 + StorageBytes int64 + TrieNodeCount int64 + TrieNodeBytes int64 + ContractCount int64 + ContractBytes int64 } // StateSizeGenerator handles the initialization and tracking of state size metrics type StateSizeGenerator struct { - db ethdb.KeyValueStore - triedb *triedb.Database - - // Generator state - abort chan struct{} - done chan struct{} - - // Async message channel for updates - updateChan chan *stateUpdate - - // Metrics state (only modified by generate() goroutine) - metrics *StateSizeMetrics - buffered *StateSizeMetrics - - // Initialization state - initialized atomic.Bool + db ethdb.KeyValueStore + triedb *triedb.Database + abort chan struct{} + done chan struct{} + updateChan chan *stateUpdate // Async message channel for updates + metrics *stateSizeMetrics + buffered *stateSizeMetrics + initialized atomic.Bool // Initialization state } // NewStateSizeGenerator creates a new state size generator and starts it automatically @@ -72,8 +64,8 @@ func NewStateSizeGenerator(db ethdb.KeyValueStore, triedb *triedb.Database, root abort: make(chan struct{}), done: make(chan struct{}), updateChan: make(chan *stateUpdate, 1000), // Buffered channel for updates - metrics: &StateSizeMetrics{Root: root}, - buffered: &StateSizeMetrics{Root: root}, + metrics: &stateSizeMetrics{Root: root}, + buffered: &stateSizeMetrics{Root: root}, } // Start the generator automatically @@ -82,13 +74,12 @@ func NewStateSizeGenerator(db ethdb.KeyValueStore, triedb *triedb.Database, root return g } -// Stop terminates the background generation +// Stop terminates the background generation and persists the metrics. func (g *StateSizeGenerator) Stop() { close(g.abort) <-g.done - // Persist metrics after all the goroutines were stopped g.persistMetrics() } @@ -119,7 +110,20 @@ func (g *StateSizeGenerator) generate() { case <-initDone: // Initialization completed, merge buffered metrics - g.mergeBufferedMetrics() + if g.buffered != nil { + log.Info("Merging buffered metrics into main metrics") + g.metrics.Root = g.buffered.Root + g.metrics.AccountCount += g.buffered.AccountCount + g.metrics.AccountBytes += g.buffered.AccountBytes + g.metrics.StorageCount += g.buffered.StorageCount + g.metrics.StorageBytes += g.buffered.StorageBytes + g.metrics.TrieNodeCount += g.buffered.TrieNodeCount + g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes + g.metrics.ContractCount += g.buffered.ContractCount + g.metrics.ContractBytes += g.buffered.ContractBytes + + g.buffered = nil + } initDone = nil // Clear the channel to prevent future selects } } @@ -174,30 +178,12 @@ func (g *StateSizeGenerator) initialize() chan struct{} { return initDone } -// mergeBufferedMetrics merges buffered metrics into main metrics -func (g *StateSizeGenerator) mergeBufferedMetrics() { - if g.buffered != nil { - log.Info("Merging buffered metrics into main metrics") - g.metrics.Root = g.buffered.Root - g.metrics.AccountCount += g.buffered.AccountCount - g.metrics.AccountBytes += g.buffered.AccountBytes - g.metrics.StorageCount += g.buffered.StorageCount - g.metrics.StorageBytes += g.buffered.StorageBytes - g.metrics.TrieNodeCount += g.buffered.TrieNodeCount - g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes - g.metrics.ContractCount += g.buffered.ContractCount - g.metrics.ContractBytes += g.buffered.ContractBytes - - g.buffered = nil - } -} - // handleUpdate processes a single update with proper root continuity checking func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) { // Calculate the diff diff := g.calculateUpdateDiff(update) - var targetMetrics *StateSizeMetrics + var targetMetrics *stateSizeMetrics if initialized { targetMetrics = g.metrics } else { @@ -233,8 +219,8 @@ func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) } // calculateUpdateDiff calculates the diff for a state update -func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeMetrics { - var diff StateSizeMetrics +func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) stateSizeMetrics { + var diff stateSizeMetrics // Calculate account changes for addr, oldValue := range update.accountsOrigin { @@ -252,7 +238,7 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeM diff.AccountCount += 1 diff.AccountBytes += common.HashLength } - diff.AccountBytes += uint64(len(newValue) - len(oldValue)) + diff.AccountBytes += int64(len(newValue) - len(oldValue)) } // Calculate storage changes @@ -285,7 +271,7 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeM diff.StorageCount += 1 diff.StorageBytes += common.HashLength } - diff.StorageBytes += uint64(len(newValue) - len(oldValue)) + diff.StorageBytes += int64(len(newValue) - len(oldValue)) } } @@ -294,21 +280,21 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) StateSizeM for path, n := range subset.Nodes { if len(n.Blob) == 0 { diff.TrieNodeCount -= 1 - diff.TrieNodeBytes -= uint64(len(path) + common.HashLength) + diff.TrieNodeBytes -= int64(len(path) + common.HashLength) } prev, ok := subset.Origins[path] if ok { diff.TrieNodeCount += 1 - diff.TrieNodeBytes += uint64(len(path) + common.HashLength) + diff.TrieNodeBytes += int64(len(path) + common.HashLength) } - diff.TrieNodeBytes += uint64(len(n.Blob) - len(prev)) + diff.TrieNodeBytes += int64(len(n.Blob) - len(prev)) } } // Calculate code changes for _, code := range update.codes { diff.ContractCount += 1 - diff.ContractBytes += uint64(len(code.blob) + common.HashLength) + diff.ContractBytes += int64(len(code.blob) + common.HashLength) } return diff @@ -333,15 +319,15 @@ func (g *StateSizeGenerator) hasExistingMetrics() bool { return false } - var existed StateSizeMetrics - if err := rlp.DecodeBytes(data, &existed); err != nil { - log.Warn("Failed to decode existing state size metrics", "err", err) + var existed stateSizeMetrics + if err := json.Unmarshal(data, &existed); err != nil { + log.Warn("Failed to decode existed state size metrics", "err", err) return false } // Check if the existing metrics root matches our current root if (g.metrics.Root != common.Hash{}) && existed.Root != g.metrics.Root { - log.Info("Existing state size metrics found but root mismatch", "existing", existed.Root, "current", g.metrics.Root) + log.Info("Existing state size metrics found but root mismatch", "existed", existed.Root, "current", g.metrics.Root) return false } @@ -370,11 +356,11 @@ func (g *StateSizeGenerator) initializeMetrics() error { // Metrics will be directly updated by each goroutine var ( - accountCount, accountBytes uint64 - storageCount, storageBytes uint64 - trieAccountNodeCount, trieAccountNodeBytes uint64 - trieStorageNodeCount, trieStorageNodeBytes uint64 - contractCount, contractBytes uint64 + accountCount, accountBytes int64 + storageCount, storageBytes int64 + trieAccountNodeCount, trieAccountNodeBytes int64 + trieStorageNodeCount, trieStorageNodeBytes int64 + contractCount, contractBytes int64 ) // Start all table iterations concurrently with direct metric updates @@ -428,7 +414,6 @@ func (g *StateSizeGenerator) initializeMetrics() error { return err } - // Update metrics (safe since we're in the single writer goroutine) g.metrics.AccountCount = accountCount g.metrics.AccountBytes = accountBytes g.metrics.StorageCount = storageCount @@ -445,17 +430,17 @@ func (g *StateSizeGenerator) initializeMetrics() error { } // iterateTable performs iteration over a specific table and returns the results -func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, name string) (uint64, uint64, error) { +func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, name string) (int64, int64, error) { log.Info("Iterating over state size", "table", name) start := time.Now() - var count, bytes uint64 + var count, bytes int64 iter := g.db.NewIterator(prefix, nil) defer iter.Release() for iter.Next() { count++ - bytes += uint64(len(iter.Key()) + len(iter.Value())) + bytes += int64(len(iter.Key()) + len(iter.Value())) // Check for cancellation periodically for performance if count%10000 == 0 { @@ -480,19 +465,20 @@ func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, na } func (g *StateSizeGenerator) updateMetrics() { - accountCountGauge.Update(int64(g.metrics.AccountCount)) - accountBytesGauge.Update(int64(g.metrics.AccountBytes)) - storageCountGauge.Update(int64(g.metrics.StorageCount)) - storageBytesGauge.Update(int64(g.metrics.StorageBytes)) - trienodeCountGauge.Update(int64(g.metrics.TrieNodeCount)) - trienodeBytesGauge.Update(int64(g.metrics.TrieNodeBytes)) - contractCountGauge.Update(int64(g.metrics.ContractCount)) - contractBytesGauge.Update(int64(g.metrics.ContractBytes)) + accountCountGauge.Update(g.metrics.AccountCount) + accountBytesGauge.Update(g.metrics.AccountBytes) + storageCountGauge.Update(g.metrics.StorageCount) + storageBytesGauge.Update(g.metrics.StorageBytes) + trienodeCountGauge.Update(g.metrics.TrieNodeCount) + trienodeBytesGauge.Update(g.metrics.TrieNodeBytes) + contractCountGauge.Update(g.metrics.ContractCount) + contractBytesGauge.Update(g.metrics.ContractBytes) } // persistMetrics saves the current metrics to the database func (g *StateSizeGenerator) persistMetrics() { - data, err := rlp.EncodeToBytes(*g.metrics) + // RLP doesn't support int64, so we use JSON for simplicity + data, err := json.Marshal(*g.metrics) if err != nil { log.Error("Failed to encode state size metrics", "err", err) return From 116f327c27be7c56d49559d14b331567073ee2c9 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 7 Aug 2025 11:27:30 +0800 Subject: [PATCH 11/43] fix: trienode calculate Signed-off-by: jsvisa --- core/state/statesize.go | 125 ++++++++++++++++++++++++---------------- 1 file changed, 76 insertions(+), 49 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index 94b4dcccbdf..1871770d5d4 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -31,6 +31,14 @@ import ( "golang.org/x/sync/errgroup" ) +var ( + accountSnapKeySize = int64(len(rawdb.SnapshotAccountPrefix) + common.HashLength) + storageSnapKeySize = int64(len(rawdb.SnapshotStoragePrefix) + common.HashLength + common.HashLength) + accountTrieKeyPrefixSize = int64(len(rawdb.TrieNodeAccountPrefix)) + storageTrieKeyPrefixSize = int64(len(rawdb.TrieNodeStoragePrefix) + common.HashLength) + codeKeySize = int64(len(rawdb.CodePrefix) + common.HashLength) +) + // stateSizeMetrics represents the current state size statistics type stateSizeMetrics struct { Root common.Hash // Root hash of the state trie @@ -180,7 +188,6 @@ func (g *StateSizeGenerator) initialize() chan struct{} { // handleUpdate processes a single update with proper root continuity checking func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) { - // Calculate the diff diff := g.calculateUpdateDiff(update) var targetMetrics *stateSizeMetrics @@ -192,12 +199,7 @@ func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) // Check root continuity - the update should build on our current state if targetMetrics.Root != (common.Hash{}) && targetMetrics.Root != update.originRoot { - log.Warn("State update root discontinuity detected", - "current", targetMetrics.Root, - "updateOrigin", update.originRoot, - "updateNew", update.root) - // For now, we accept the discontinuity but log it - // In production, you might want to reset metrics or handle differently + log.Warn("State update root discontinuity detected", "current", targetMetrics.Root, "updateOrigin", update.originRoot, "updateNew", update.root) } // Update to the new state root @@ -230,15 +232,20 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) stateSizeM log.Warn("State update missing account", "address", addr) continue } - if len(newValue) == 0 { + + oldLen, newLen := len(oldValue), len(newValue) + if oldLen > 0 && newLen == 0 { + // Account deletion diff.AccountCount -= 1 - diff.AccountBytes -= common.HashLength - } - if len(oldValue) == 0 { + diff.AccountBytes -= accountSnapKeySize + int64(oldLen) + } else if oldLen == 0 && newLen > 0 { + // Account creation diff.AccountCount += 1 - diff.AccountBytes += common.HashLength + diff.AccountBytes += accountSnapKeySize + int64(newLen) + } else { + // Account update + diff.AccountBytes += int64(newLen - oldLen) } - diff.AccountBytes += int64(len(newValue) - len(oldValue)) } // Calculate storage changes @@ -263,38 +270,58 @@ func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) stateSizeM log.Warn("State update missing storage slot", "address", addr, "key", key) continue } - if len(newValue) == 0 { + + oldLen, newLen := len(oldValue), len(newValue) + if oldLen > 0 && newLen == 0 { + // Storage deletion diff.StorageCount -= 1 - diff.StorageBytes -= common.HashLength - } - if len(oldValue) == 0 { + diff.StorageBytes -= storageSnapKeySize + int64(oldLen) + } else if oldLen == 0 && newLen > 0 { + // Storage creation diff.StorageCount += 1 - diff.StorageBytes += common.HashLength + diff.StorageBytes += storageSnapKeySize + int64(newLen) + } else { + // Storage update + diff.StorageBytes += int64(newLen - oldLen) } - diff.StorageBytes += int64(len(newValue) - len(oldValue)) } } // Calculate trie node changes - for _, subset := range update.nodes.Sets { - for path, n := range subset.Nodes { - if len(n.Blob) == 0 { + for owner, subset := range update.nodes.Sets { + isAccountTrie := owner == (common.Hash{}) + var keyPrefixSize int64 + if isAccountTrie { + keyPrefixSize = accountTrieKeyPrefixSize + } else { + keyPrefixSize = storageTrieKeyPrefixSize + } + + // Iterate over Origins since every modified node has an origin entry + for path, oldNode := range subset.Origins { + newNode, hasNew := subset.Nodes[path] + + keySize := keyPrefixSize + int64(len(path)) + + if len(oldNode) > 0 && (!hasNew || len(newNode.Blob) == 0) { + // Node deletion diff.TrieNodeCount -= 1 - diff.TrieNodeBytes -= int64(len(path) + common.HashLength) - } - prev, ok := subset.Origins[path] - if ok { + diff.TrieNodeBytes -= keySize + int64(len(oldNode)) + } else if len(oldNode) == 0 && hasNew && len(newNode.Blob) > 0 { + // Node creation diff.TrieNodeCount += 1 - diff.TrieNodeBytes += int64(len(path) + common.HashLength) + diff.TrieNodeBytes += keySize + int64(len(newNode.Blob)) + } else if len(oldNode) > 0 && hasNew && len(newNode.Blob) > 0 { + // Node update + diff.TrieNodeBytes += int64(len(newNode.Blob) - len(oldNode)) } - diff.TrieNodeBytes += int64(len(n.Blob) - len(prev)) } } // Calculate code changes for _, code := range update.codes { diff.ContractCount += 1 - diff.ContractBytes += int64(len(code.blob) + common.HashLength) + diff.ContractBytes += codeKeySize + int64(len(code.blob)) } return diff @@ -356,52 +383,52 @@ func (g *StateSizeGenerator) initializeMetrics() error { // Metrics will be directly updated by each goroutine var ( - accountCount, accountBytes int64 - storageCount, storageBytes int64 - trieAccountNodeCount, trieAccountNodeBytes int64 - trieStorageNodeCount, trieStorageNodeBytes int64 - contractCount, contractBytes int64 + accountSnapCount, accountSnapBytes int64 + storageSnapCount, storageSnapBytes int64 + accountTrieCount, accountTrieBytes int64 + storageTrieCount, storageTrieBytes int64 + contractCount, contractBytes int64 ) // Start all table iterations concurrently with direct metric updates group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotAccountPrefix, "account") + count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotAccountPrefix, "accountSnap") if err != nil { return err } - accountCount, accountBytes = count, bytes + accountSnapCount, accountSnapBytes = count, bytes return nil }) group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotStoragePrefix, "storage") + count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotStoragePrefix, "storageSnap") if err != nil { return err } - storageCount, storageBytes = count, bytes + storageSnapCount, storageSnapBytes = count, bytes return nil }) group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeAccountPrefix, "trie account node") + count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeAccountPrefix, "accountTrie") if err != nil { return err } - trieAccountNodeCount, trieAccountNodeBytes = count, bytes + accountTrieCount, accountTrieBytes = count, bytes return nil }) group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeStoragePrefix, "trie storage node") + count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeStoragePrefix, "storageTrie") if err != nil { return err } - trieStorageNodeCount, trieStorageNodeBytes = count, bytes + storageTrieCount, storageTrieBytes = count, bytes return nil }) group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.CodePrefix, "contract code") + count, bytes, err := g.iterateTable(ctx, rawdb.CodePrefix, "contract") if err != nil { return err } @@ -414,12 +441,12 @@ func (g *StateSizeGenerator) initializeMetrics() error { return err } - g.metrics.AccountCount = accountCount - g.metrics.AccountBytes = accountBytes - g.metrics.StorageCount = storageCount - g.metrics.StorageBytes = storageBytes - g.metrics.TrieNodeCount = trieAccountNodeCount + trieStorageNodeCount - g.metrics.TrieNodeBytes = trieAccountNodeBytes + trieStorageNodeBytes + g.metrics.AccountCount = accountSnapCount + g.metrics.AccountBytes = accountSnapBytes + g.metrics.StorageCount = storageSnapCount + g.metrics.StorageBytes = storageSnapBytes + g.metrics.TrieNodeCount = accountTrieCount + storageTrieCount + g.metrics.TrieNodeBytes = accountTrieBytes + storageTrieBytes g.metrics.ContractCount = contractCount g.metrics.ContractBytes = contractBytes From c3204b338d5996c778858a1b6e19a5961dce6599 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 11 Aug 2025 10:04:49 +0800 Subject: [PATCH 12/43] use time.After instead of sleep, incase of goroutine leak Signed-off-by: jsvisa --- core/state/statesize.go | 17 +++++++++-------- 1 file changed, 9 insertions(+), 8 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index 1871770d5d4..d35d32965c4 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -155,18 +155,19 @@ func (g *StateSizeGenerator) initialize() chan struct{} { LOOP: // Wait for snapshot generator to complete first for { + root, done := g.triedb.SnapshotCompleted() + if done { + g.metrics.Root = root + g.buffered.Root = root + break LOOP + } + select { case <-g.abort: log.Info("State size initialization aborted during snapshot wait") return - default: - root, done := g.triedb.SnapshotCompleted() - if done { - g.metrics.Root = root - g.buffered.Root = root - break LOOP - } - time.Sleep(10 * time.Second) + case <-time.After(10 * time.Second): + // Continue checking for snapshot completion } } From e1c599e28631087eed6bc8c8348cf50d2348ed61 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 12 Aug 2025 12:10:13 +0800 Subject: [PATCH 13/43] check iterator error Signed-off-by: jsvisa --- core/state/statesize.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/state/statesize.go b/core/state/statesize.go index d35d32965c4..19d5bc0bc7b 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -466,6 +466,12 @@ func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, na iter := g.db.NewIterator(prefix, nil) defer iter.Release() + // Check for iterator errors before starting iteration + if err := iter.Error(); err != nil { + log.Warn("Iterator error before iteration, database may be closed", "table", name, "err", err) + return 0, 0, err + } + for iter.Next() { count++ bytes += int64(len(iter.Key()) + len(iter.Value())) From 0a14a72eab7343be0bc6bd675aaf983b5426a369 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 12 Aug 2025 15:03:10 +0800 Subject: [PATCH 14/43] stop state size inside stop without saving Signed-off-by: jsvisa --- core/blockchain.go | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 790ec1f1525..963e7bcec5a 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -1259,6 +1259,12 @@ func (bc *BlockChain) stopWithoutSaving() { // Signal shutdown to all goroutines. bc.InterruptInsert(true) + // Stop state size generator if running + if bc.stateSizeGen != nil { + bc.stateSizeGen.Stop() + log.Info("Stopped state size generator") + } + // Now wait for all chain modifications to end and persistent goroutines to exit. // // Note: Close waits for the mutex to become available, i.e. any running chain @@ -1320,11 +1326,6 @@ func (bc *BlockChain) Stop() { } } } - // Stop state size generator if running - if bc.stateSizeGen != nil { - bc.stateSizeGen.Stop() - log.Info("Stopped state size generator") - } // Allow tracers to clean-up and release resources. if bc.logger != nil && bc.logger.OnClose != nil { From 930e027b84831ba1472cf294febc27e1f305b479 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 12 Aug 2025 15:03:20 +0800 Subject: [PATCH 15/43] Revert "check iterator error" This reverts commit 87e1d8260331140184e244b1b2c280263d9b0cec. Signed-off-by: jsvisa --- core/state/statesize.go | 6 ------ 1 file changed, 6 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index 19d5bc0bc7b..d35d32965c4 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -466,12 +466,6 @@ func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, na iter := g.db.NewIterator(prefix, nil) defer iter.Release() - // Check for iterator errors before starting iteration - if err := iter.Error(); err != nil { - log.Warn("Iterator error before iteration, database may be closed", "table", name, "err", err) - return 0, 0, err - } - for iter.Next() { count++ bytes += int64(len(iter.Key()) + len(iter.Value())) From b62ea6074d0ad8a35cf4960d97b51801899de506 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 12 Aug 2025 16:38:20 +0800 Subject: [PATCH 16/43] fix data race Signed-off-by: jsvisa --- core/state/statesize.go | 43 ++++++++++++++++++++--------------------- 1 file changed, 21 insertions(+), 22 deletions(-) diff --git a/core/state/statesize.go b/core/state/statesize.go index d35d32965c4..a696958064f 100644 --- a/core/state/statesize.go +++ b/core/state/statesize.go @@ -19,7 +19,6 @@ package state import ( "context" "encoding/json" - "sync/atomic" "time" "github.com/ethereum/go-ethereum/common" @@ -54,14 +53,13 @@ type stateSizeMetrics struct { // StateSizeGenerator handles the initialization and tracking of state size metrics type StateSizeGenerator struct { - db ethdb.KeyValueStore - triedb *triedb.Database - abort chan struct{} - done chan struct{} - updateChan chan *stateUpdate // Async message channel for updates - metrics *stateSizeMetrics - buffered *stateSizeMetrics - initialized atomic.Bool // Initialization state + db ethdb.KeyValueStore + triedb *triedb.Database + abort chan struct{} + done chan struct{} + updateChan chan *stateUpdate // Async message channel for updates + metrics *stateSizeMetrics + buffered *stateSizeMetrics } // NewStateSizeGenerator creates a new state size generator and starts it automatically @@ -95,12 +93,20 @@ func (g *StateSizeGenerator) Stop() { func (g *StateSizeGenerator) generate() { defer close(g.done) + var inited bool + + // Check if we already have existing metrics + if g.hasExistingMetrics() { + log.Info("State size metrics already initialized") + inited = true + } + initDone := g.initialize() for { select { case update := <-g.updateChan: - g.handleUpdate(update, g.initialized.Load()) + g.handleUpdate(update, inited) case <-g.abort: log.Info("State size generation aborted") @@ -132,6 +138,8 @@ func (g *StateSizeGenerator) generate() { g.buffered = nil } + + inited = true initDone = nil // Clear the channel to prevent future selects } } @@ -139,18 +147,11 @@ func (g *StateSizeGenerator) generate() { // initialize starts the initialization process if not already initialized func (g *StateSizeGenerator) initialize() chan struct{} { - // Check if we already have existing metrics - if g.hasExistingMetrics() { - log.Info("State size metrics already initialized") - g.initialized.Store(true) - return nil - } - - initDone := make(chan struct{}) + done := make(chan struct{}) // Wait for snapshot completion and then initialize go func() { - defer close(initDone) + defer close(done) LOOP: // Wait for snapshot generator to complete first @@ -179,12 +180,10 @@ func (g *StateSizeGenerator) initialize() chan struct{} { return } - g.initialized.Store(true) - log.Info("Completed state size initialization", "elapsed", time.Since(start)) }() - return initDone + return done } // handleUpdate processes a single update with proper root continuity checking From 0861e6dffe4122c522ed62c77d7eb8780f06a0a0 Mon Sep 17 00:00:00 2001 From: Gary Rong Date: Fri, 15 Aug 2025 19:55:51 +0800 Subject: [PATCH 17/43] core/state: rewrite the state size tracker --- core/blockchain.go | 28 +- core/state/metrics.go | 10 - core/state/state_sizer.go | 547 +++++++++++++++++++++++++++++ core/state/statedb.go | 8 +- core/state/statedb_test.go | 2 +- core/state/statesize.go | 519 --------------------------- core/state/stateupdate.go | 9 +- core/state/trie_prefetcher_test.go | 2 +- triedb/database.go | 4 +- triedb/pathdb/database.go | 15 +- 10 files changed, 582 insertions(+), 562 deletions(-) create mode 100644 core/state/state_sizer.go delete mode 100644 core/state/statesize.go diff --git a/core/blockchain.go b/core/blockchain.go index 963e7bcec5a..663dbc31f6c 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -196,6 +196,9 @@ type BlockChainConfig struct { // If the value is zero, all transactions of the entire chain will be indexed. // If the value is -1, indexing is disabled. TxLookupLimit int64 + + // EnableStateSizeTracking indicates whether the state size tracking is enabled. + EnableStateSizeTracking bool } // DefaultConfig returns the default config. @@ -333,8 +336,7 @@ type BlockChain struct { prefetcher Prefetcher processor Processor // Block transaction processor interface logger *tracing.Hooks - - stateSizeGen *state.StateSizeGenerator // State size tracking + stateSizer *state.SizeTracker // State size tracking lastForkReadyAlert time.Time // Last time there was a fork readiness print out } @@ -530,9 +532,10 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, } // Start state size tracker - bc.stateSizeGen = state.NewStateSizeGenerator(bc.statedb.DiskDB(), bc.triedb, head.Root) - log.Info("Started state size generator", "root", head.Root) - + if bc.cfg.EnableStateSizeTracking { + bc.stateSizer = state.NewSizeTracker(bc.db, bc.triedb) + bc.stateSizer.Start() + } return bc, nil } @@ -1259,12 +1262,10 @@ func (bc *BlockChain) stopWithoutSaving() { // Signal shutdown to all goroutines. bc.InterruptInsert(true) - // Stop state size generator if running - if bc.stateSizeGen != nil { - bc.stateSizeGen.Stop() - log.Info("Stopped state size generator") + // Stop state size tracker + if bc.stateSizer != nil { + bc.stateSizer.Stop() } - // Now wait for all chain modifications to end and persistent goroutines to exit. // // Note: Close waits for the mutex to become available, i.e. any running chain @@ -1604,10 +1605,9 @@ func (bc *BlockChain) writeBlockWithState(block *types.Block, receipts []*types. if err != nil { return err } - - // Track state size changes if generator is running - if bc.stateSizeGen != nil && stateUpdate != nil { - bc.stateSizeGen.Track(stateUpdate) + // Emit the state update to the state sizestats if it's active + if bc.stateSizer != nil { + bc.stateSizer.Notify(stateUpdate) } // If node is running in path mode, skip explicit gc operation // which is unnecessary in this mode. diff --git a/core/state/metrics.go b/core/state/metrics.go index 57f6d4a4edd..dd4b2e98381 100644 --- a/core/state/metrics.go +++ b/core/state/metrics.go @@ -29,14 +29,4 @@ var ( storageTriesUpdatedMeter = metrics.NewRegisteredMeter("state/update/storagenodes", nil) accountTrieDeletedMeter = metrics.NewRegisteredMeter("state/delete/accountnodes", nil) storageTriesDeletedMeter = metrics.NewRegisteredMeter("state/delete/storagenodes", nil) - - // State size metrics - accountCountGauge = metrics.NewRegisteredGauge("state/account/count", nil) - accountBytesGauge = metrics.NewRegisteredGauge("state/account/bytes", nil) - storageCountGauge = metrics.NewRegisteredGauge("state/storage/count", nil) - storageBytesGauge = metrics.NewRegisteredGauge("state/storage/bytes", nil) - trienodeCountGauge = metrics.NewRegisteredGauge("state/trienode/count", nil) - trienodeBytesGauge = metrics.NewRegisteredGauge("state/trienode/bytes", nil) - contractCountGauge = metrics.NewRegisteredGauge("state/contract/count", nil) - contractBytesGauge = metrics.NewRegisteredGauge("state/contract/bytes", nil) ) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go new file mode 100644 index 00000000000..687b2d043e8 --- /dev/null +++ b/core/state/state_sizer.go @@ -0,0 +1,547 @@ +// Copyright 2025 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package state + +import ( + "container/heap" + "errors" + "fmt" + "maps" + "slices" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/rawdb" + "github.com/ethereum/go-ethereum/crypto" + "github.com/ethereum/go-ethereum/ethdb" + "github.com/ethereum/go-ethereum/log" + "github.com/ethereum/go-ethereum/metrics" + "github.com/ethereum/go-ethereum/triedb" + "golang.org/x/sync/errgroup" +) + +const ( + statEvictThreshold = 128 // the depth of statistic to be preserved +) + +// Metrics for uploading the state statistics. +var ( + accountsGauge = metrics.NewRegisteredGauge("state/size/account/count", nil) + accountBytesGauge = metrics.NewRegisteredGauge("state/size/account/bytes", nil) + storagesGauge = metrics.NewRegisteredGauge("state/size/storage/count", nil) + storageBytesGauge = metrics.NewRegisteredGauge("state/size/storage/bytes", nil) + accountTrienodesGauge = metrics.NewRegisteredGauge("state/size/trienode/account/count", nil) + accountTrienodeBytesGauge = metrics.NewRegisteredGauge("state/size/trienode/account/bytes", nil) + storageTrienodesGauge = metrics.NewRegisteredGauge("state/size/trienode/storage/count", nil) + storageTrienodeBytesGauge = metrics.NewRegisteredGauge("state/size/trienode/storage/bytes", nil) + contractCodesGauge = metrics.NewRegisteredGauge("state/size/contractcode/count", nil) + contractCodeBytesGauge = metrics.NewRegisteredGauge("state/size/contractcode/bytes", nil) +) + +// Database key scheme for states. +var ( + accountKeySize = int64(len(rawdb.SnapshotAccountPrefix) + common.HashLength) + storageKeySize = int64(len(rawdb.SnapshotStoragePrefix) + common.HashLength + common.HashLength) + accountTrienodePrefixSize = int64(len(rawdb.TrieNodeAccountPrefix)) + storageTrienodePrefixSize = int64(len(rawdb.TrieNodeStoragePrefix) + common.HashLength) + codeKeySize = int64(len(rawdb.CodePrefix) + common.HashLength) +) + +// SizeStats represents either the current state size statistics or the size +// differences resulting from a state transition. +type SizeStats struct { + StateRoot common.Hash // State root hash at the time of measurement + BlockNumber uint64 // Associated block number at the time of measurement + + Accounts int64 // Total number of accounts in the state + AccountBytes int64 // Total storage size used by all account data (in bytes) + Storages int64 // Total number of storage slots across all accounts + StorageBytes int64 // Total storage size used by all storage slot data (in bytes) + AccountTrienodes int64 // Total number of account trie nodes in the state + AccountTrienodeBytes int64 // Total storage size occupied by account trie nodes (in bytes) + StorageTrienodes int64 // Total number of storage trie nodes in the state + StorageTrienodeBytes int64 // Total storage size occupied by storage trie nodes (in bytes) + ContractCodes int64 // Total number of contract codes in the state + ContractCodeBytes int64 // Total size of all contract code (in bytes) +} + +// add applies the given state diffs and produces a new version of the statistics. +func (s SizeStats) add(diff SizeStats) SizeStats { + var combo SizeStats + combo.StateRoot = diff.StateRoot + combo.BlockNumber = diff.BlockNumber + + combo.Accounts += diff.Accounts + combo.AccountBytes += diff.AccountBytes + combo.Storages += diff.Storages + combo.StorageBytes += diff.StorageBytes + combo.AccountTrienodes += diff.AccountTrienodes + combo.AccountTrienodeBytes += diff.AccountTrienodeBytes + combo.StorageTrienodes += diff.StorageTrienodes + combo.StorageTrienodeBytes += diff.StorageTrienodeBytes + combo.ContractCodes += diff.ContractCodes + combo.ContractCodeBytes += diff.ContractCodeBytes + return combo +} + +// calSizeStats measures the state size changes of the provided state update. +func calSizeStats(update *stateUpdate) (SizeStats, error) { + stats := SizeStats{ + BlockNumber: update.blockNumber, + StateRoot: update.root, + } + + // Measure the account changes + for addr, oldValue := range update.accountsOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + newValue, exists := update.accounts[addrHash] + if !exists { + return SizeStats{}, fmt.Errorf("account %x not found", addr) + } + oldLen, newLen := len(oldValue), len(newValue) + + switch { + case oldLen > 0 && newLen == 0: + // Account deletion + stats.Accounts -= 1 + stats.AccountBytes -= accountKeySize + int64(oldLen) + case oldLen == 0 && newLen > 0: + // Account creation + stats.Accounts += 1 + stats.AccountBytes += accountKeySize + int64(newLen) + default: + // Account update + stats.AccountBytes += int64(newLen - oldLen) + } + } + + // Measure storage changes + for addr, slots := range update.storagesOrigin { + addrHash := crypto.Keccak256Hash(addr.Bytes()) + subset, exists := update.storages[addrHash] + if !exists { + return SizeStats{}, fmt.Errorf("storage %x not found", addr) + } + for key, oldValue := range slots { + var ( + exists bool + newValue []byte + ) + if update.rawStorageKey { + newValue, exists = subset[crypto.Keccak256Hash(key.Bytes())] + } else { + newValue, exists = subset[key] + } + if !exists { + return SizeStats{}, fmt.Errorf("storage slot %x-%x not found", addr, key) + } + oldLen, newLen := len(oldValue), len(newValue) + + switch { + case oldLen > 0 && newLen == 0: + // Storage deletion + stats.Storages -= 1 + stats.StorageBytes -= storageKeySize + int64(oldLen) + case oldLen == 0 && newLen > 0: + // Storage creation + stats.Storages += 1 + stats.StorageBytes += storageKeySize + int64(newLen) + default: + // Storage update + stats.StorageBytes += int64(newLen - oldLen) + } + } + } + + // Measure trienode changes + for owner, subset := range update.nodes.Sets { + var ( + keyPrefix int64 + isAccount = owner == (common.Hash{}) + ) + if isAccount { + keyPrefix = accountTrienodePrefixSize + } else { + keyPrefix = storageTrienodePrefixSize + } + + // Iterate over Origins since every modified node has an origin entry + for path, oldNode := range subset.Origins { + newNode, exists := subset.Nodes[path] + if !exists { + return SizeStats{}, fmt.Errorf("node %x-%v not found", owner, path) + } + keySize := keyPrefix + int64(len(path)) + + switch { + case len(oldNode) > 0 && len(newNode.Blob) == 0: + // Node deletion + if isAccount { + stats.AccountTrienodes -= 1 + stats.AccountTrienodeBytes -= keySize + int64(len(oldNode)) + } else { + stats.StorageTrienodes -= 1 + stats.StorageTrienodeBytes -= keySize + int64(len(oldNode)) + } + case len(oldNode) == 0 && len(newNode.Blob) > 0: + // Node creation + if isAccount { + stats.AccountTrienodes += 1 + stats.AccountTrienodeBytes += keySize + int64(len(newNode.Blob)) + } else { + stats.StorageTrienodes += 1 + stats.StorageTrienodeBytes += keySize + int64(len(newNode.Blob)) + } + default: + // Node update + if isAccount { + stats.AccountTrienodeBytes += int64(len(newNode.Blob) - len(oldNode)) + } else { + stats.StorageTrienodeBytes += int64(len(newNode.Blob) - len(oldNode)) + } + } + } + } + + // Measure code changes. Note that the reported contract code size may be slightly + // inaccurate due to database deduplication (code is stored by its hash). However, + // this deviation is negligible and acceptable for measurement purposes. + for _, code := range update.codes { + stats.ContractCodes += 1 + stats.ContractCodeBytes += codeKeySize + int64(len(code.blob)) + } + return stats, nil +} + +// SizeTracker handles the state size initialization and tracks of state size metrics. +type SizeTracker struct { + db ethdb.KeyValueStore + triedb *triedb.Database + abort chan struct{} + aborted chan struct{} + updateCh chan *stateUpdate +} + +// NewSizeTracker creates a new state size tracker and starts it automatically +func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) *SizeTracker { + return &SizeTracker{ + db: db, + triedb: triedb, + abort: make(chan struct{}), + aborted: make(chan struct{}), + updateCh: make(chan *stateUpdate), + } +} + +func (t *SizeTracker) Start() { + go t.run() +} + +func (t *SizeTracker) Stop() { + close(t.abort) + <-t.aborted +} + +// sizeStatsHeap is a heap.Interface implementation over statesize statistics for +// retrieving the oldest statistics for eviction. +type sizeStatsHeap []SizeStats + +func (h sizeStatsHeap) Len() int { return len(h) } +func (h sizeStatsHeap) Less(i, j int) bool { return h[i].BlockNumber < h[j].BlockNumber } +func (h sizeStatsHeap) Swap(i, j int) { h[i], h[j] = h[j], h[i] } + +func (h *sizeStatsHeap) Push(x any) { + *h = append(*h, x.(SizeStats)) +} + +func (h *sizeStatsHeap) Pop() any { + old := *h + n := len(old) + x := old[n-1] + *h = old[0 : n-1] + return x +} + +// run performs the state size initialization and handles updates +func (t *SizeTracker) run() { + defer close(t.aborted) + + stats, err := t.init() // launch background thread for state size init + if err != nil { + return + } + h := sizeStatsHeap(slices.Collect(maps.Values(stats))) + heap.Init(&h) + + for { + select { + case u := <-t.updateCh: + base, found := stats[u.originRoot] + if !found { + continue + } + diff, err := calSizeStats(u) + if err != nil { + continue + } + stat := base.add(diff) + t.upload(stat) + + stats[u.root] = stat + heap.Push(&h, stats[u.root]) + for u.blockNumber-h[0].BlockNumber > statEvictThreshold { + heap.Pop(&h) + } + + case <-t.abort: + return + } + } +} + +type buildResult struct { + stat SizeStats + root common.Hash + blockNumber uint64 + err error +} + +func (t *SizeTracker) init() (map[common.Hash]SizeStats, error) { + // Wait for snapshot completion and then init + ticker := time.NewTicker(10 * time.Second) + defer ticker.Stop() + +wait: + for { + select { + case <-ticker.C: + if t.triedb.SnapshotCompleted() { + break wait + } + case <-t.abort: + return nil, errors.New("size tracker closed") + } + } + + var ( + updates = make(map[common.Hash]*stateUpdate) + children = make(map[common.Hash][]common.Hash) + done = make(chan buildResult) + ) + for { + select { + case u := <-t.updateCh: + updates[u.root] = u + children[u.originRoot] = append(children[u.originRoot], u.root) + + case <-ticker.C: + root := rawdb.ReadSnapshotRoot(t.db) + if root == (common.Hash{}) { + continue + } + entry, exists := updates[root] + if !exists { + continue + } + if done == nil { + done = make(chan buildResult) + go t.build(entry.root, entry.blockNumber, done) + } + + case result := <-done: + if result.err != nil { + return nil, result.err + } + var ( + stats = make(map[common.Hash]SizeStats) + apply func(root common.Hash, stats SizeStats) error + ) + apply = func(root common.Hash, stat SizeStats) error { + for _, child := range children[root] { + entry, ok := updates[child] + if !ok { + return fmt.Errorf("the state update is not found, %x", child) + } + diff, err := calSizeStats(entry) + if err != nil { + return err + } + stats[child] = stat.add(diff) + if err := apply(child, stats[child]); err != nil { + return err + } + } + return nil + } + if err := apply(result.root, result.stat); err != nil { + return nil, err + } + return stats, nil + + case <-t.abort: + return nil, errors.New("size tracker closed") + } + } +} + +func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buildResult) { + // Metrics will be directly updated by each goroutine + var ( + accounts, accountBytes int64 + storages, storageBytes int64 + codes, codeBytes int64 + + accountTrienodes, accountTrienodeBytes int64 + storageTrienodes, storageTrienodeBytes int64 + + group errgroup.Group + ) + + // Start all table iterations concurrently with direct metric updates + group.Go(func() error { + count, bytes, err := t.iterateTable(t.abort, rawdb.SnapshotAccountPrefix, "account") + if err != nil { + return err + } + accounts, accountBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := t.iterateTable(t.abort, rawdb.SnapshotStoragePrefix, "storage") + if err != nil { + return err + } + storages, storageBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := t.iterateTable(t.abort, rawdb.TrieNodeAccountPrefix, "accountnode") + if err != nil { + return err + } + accountTrienodes, accountTrienodeBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := t.iterateTable(t.abort, rawdb.TrieNodeStoragePrefix, "storagenode") + if err != nil { + return err + } + storageTrienodes, storageTrienodeBytes = count, bytes + return nil + }) + + group.Go(func() error { + count, bytes, err := t.iterateTable(t.abort, rawdb.CodePrefix, "contractcode") + if err != nil { + return err + } + codes, codeBytes = count, bytes + return nil + }) + + // Wait for all goroutines to complete + if err := group.Wait(); err != nil { + done <- buildResult{err: err} + } else { + stat := SizeStats{ + StateRoot: root, + BlockNumber: blockNumber, + Accounts: accounts, + AccountBytes: accountBytes, + Storages: storages, + StorageBytes: storageBytes, + AccountTrienodes: accountTrienodes, + AccountTrienodeBytes: accountTrienodeBytes, + StorageTrienodes: storageTrienodes, + StorageTrienodeBytes: storageTrienodeBytes, + ContractCodes: codes, + ContractCodeBytes: codeBytes, + } + done <- buildResult{ + root: root, + blockNumber: blockNumber, + stat: stat, + } + } +} + +// Notify is an async method used to send the state update to the size tracker. +// It ignores empty updates (where no state changes occurred). +// If the channel is full, it drops the update to avoid blocking. +func (t *SizeTracker) Notify(update *stateUpdate) { + if update == nil || update.empty() { + return + } + select { + case t.updateCh <- update: + case <-t.abort: + return + } +} + +// iterateTable performs iteration over a specific table and returns the results. +func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name string) (int64, int64, error) { + var ( + start = time.Now() + logged = time.Now() + count, bytes int64 + ) + iter := t.db.NewIterator(prefix, nil) + defer iter.Release() + + log.Info("Iterating state", "category", name) + for iter.Next() { + count++ + bytes += int64(len(iter.Key()) + len(iter.Value())) + + if time.Since(logged) > time.Second*8 { + logged = time.Now() + + select { + case <-closed: + log.Info("State iteration cancelled", "category", name) + return 0, 0, errors.New("size tracker closed") + default: + log.Info("Iterating state", "category", name, "count", count, "size", common.StorageSize(bytes)) + } + } + } + // Check for iterator errors + if err := iter.Error(); err != nil { + log.Error("Iterator error", "category", name, "err", err) + return 0, 0, err + } + log.Info("Finished state iteration", "category", name, "count", count, "size", common.StorageSize(bytes), "elapsed", common.PrettyDuration(time.Since(start))) + return count, bytes, nil +} + +func (t *SizeTracker) upload(stats SizeStats) { + accountsGauge.Update(stats.Accounts) + accountBytesGauge.Update(stats.AccountBytes) + storagesGauge.Update(stats.Storages) + storageBytesGauge.Update(stats.StorageBytes) + accountTrienodesGauge.Update(stats.AccountTrienodes) + accountTrienodeBytesGauge.Update(stats.AccountTrienodeBytes) + storageTrienodesGauge.Update(stats.StorageTrienodes) + storageTrienodeBytesGauge.Update(stats.StorageTrienodeBytes) + contractCodesGauge.Update(stats.ContractCodes) + contractCodeBytesGauge.Update(stats.ContractCodeBytes) +} diff --git a/core/state/statedb.go b/core/state/statedb.go index 265561df2d0..f74065666bc 100644 --- a/core/state/statedb.go +++ b/core/state/statedb.go @@ -668,7 +668,7 @@ func (s *StateDB) CreateContract(addr common.Address) { // Copy creates a deep, independent copy of the state. // Snapshots of the copied state cannot be applied to the copy. func (s *StateDB) Copy() *StateDB { - // Copy all the basic fields, initialize the memory ones + // Copy all the basic fields, init the memory ones state := &StateDB{ db: s.db, reader: s.reader, @@ -1156,7 +1156,7 @@ func (s *StateDB) GetTrie() Trie { // commit gathers the state mutations accumulated along with the associated // trie changes, resetting all internal flags with the new state as the base. -func (s *StateDB) commit(deleteEmptyObjects bool, noStorageWiping bool) (*stateUpdate, error) { +func (s *StateDB) commit(deleteEmptyObjects bool, noStorageWiping bool, blockNumber uint64) (*stateUpdate, error) { // Short circuit in case any database failure occurred earlier. if s.dbErr != nil { return nil, fmt.Errorf("commit aborted due to earlier error: %v", s.dbErr) @@ -1308,13 +1308,13 @@ func (s *StateDB) commit(deleteEmptyObjects bool, noStorageWiping bool) (*stateU origin := s.originalRoot s.originalRoot = root - return newStateUpdate(noStorageWiping, origin, root, deletes, updates, nodes), nil + return newStateUpdate(noStorageWiping, origin, root, blockNumber, deletes, updates, nodes), nil } // commitAndFlush is a wrapper of commit which also commits the state mutations // to the configured data stores. func (s *StateDB) commitAndFlush(block uint64, deleteEmptyObjects bool, noStorageWiping bool) (*stateUpdate, error) { - ret, err := s.commit(deleteEmptyObjects, noStorageWiping) + ret, err := s.commit(deleteEmptyObjects, noStorageWiping, block) if err != nil { return nil, err } diff --git a/core/state/statedb_test.go b/core/state/statedb_test.go index 147546a3c75..20bcbabdf8b 100644 --- a/core/state/statedb_test.go +++ b/core/state/statedb_test.go @@ -281,7 +281,7 @@ func TestCopyObjectState(t *testing.T) { cpy := orig.Copy() for _, op := range cpy.mutations { if have, want := op.applied, false; have != want { - t.Fatalf("Error in test itself, the 'done' flag should not be set before Commit, have %v want %v", have, want) + t.Fatalf("Error in test itself, the 'aborted' flag should not be set before Commit, have %v want %v", have, want) } } orig.Commit(0, true, false) diff --git a/core/state/statesize.go b/core/state/statesize.go deleted file mode 100644 index a696958064f..00000000000 --- a/core/state/statesize.go +++ /dev/null @@ -1,519 +0,0 @@ -// Copyright 2025 The go-ethereum Authors -// This file is part of the go-ethereum library. -// -// The go-ethereum library is free software: you can redistribute it and/or modify -// it under the terms of the GNU Lesser General Public License as published by -// the Free Software Foundation, either version 3 of the License, or -// (at your option) any later version. -// -// The go-ethereum library is distributed in the hope that it will be useful, -// but WITHOUT ANY WARRANTY; without even the implied warranty of -// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the -// GNU Lesser General Public License for more details. -// -// You should have received a copy of the GNU Lesser General Public License -// along with the go-ethereum library. If not, see . - -package state - -import ( - "context" - "encoding/json" - "time" - - "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/core/rawdb" - "github.com/ethereum/go-ethereum/crypto" - "github.com/ethereum/go-ethereum/ethdb" - "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/triedb" - "golang.org/x/sync/errgroup" -) - -var ( - accountSnapKeySize = int64(len(rawdb.SnapshotAccountPrefix) + common.HashLength) - storageSnapKeySize = int64(len(rawdb.SnapshotStoragePrefix) + common.HashLength + common.HashLength) - accountTrieKeyPrefixSize = int64(len(rawdb.TrieNodeAccountPrefix)) - storageTrieKeyPrefixSize = int64(len(rawdb.TrieNodeStoragePrefix) + common.HashLength) - codeKeySize = int64(len(rawdb.CodePrefix) + common.HashLength) -) - -// stateSizeMetrics represents the current state size statistics -type stateSizeMetrics struct { - Root common.Hash // Root hash of the state trie - AccountCount int64 - AccountBytes int64 - StorageCount int64 - StorageBytes int64 - TrieNodeCount int64 - TrieNodeBytes int64 - ContractCount int64 - ContractBytes int64 -} - -// StateSizeGenerator handles the initialization and tracking of state size metrics -type StateSizeGenerator struct { - db ethdb.KeyValueStore - triedb *triedb.Database - abort chan struct{} - done chan struct{} - updateChan chan *stateUpdate // Async message channel for updates - metrics *stateSizeMetrics - buffered *stateSizeMetrics -} - -// NewStateSizeGenerator creates a new state size generator and starts it automatically -func NewStateSizeGenerator(db ethdb.KeyValueStore, triedb *triedb.Database, root common.Hash) *StateSizeGenerator { - g := &StateSizeGenerator{ - db: db, - triedb: triedb, - abort: make(chan struct{}), - done: make(chan struct{}), - updateChan: make(chan *stateUpdate, 1000), // Buffered channel for updates - metrics: &stateSizeMetrics{Root: root}, - buffered: &stateSizeMetrics{Root: root}, - } - - // Start the generator automatically - go g.generate() - - return g -} - -// Stop terminates the background generation and persists the metrics. -func (g *StateSizeGenerator) Stop() { - close(g.abort) - - <-g.done - - g.persistMetrics() -} - -// generate performs the state size initialization and handles updates -func (g *StateSizeGenerator) generate() { - defer close(g.done) - - var inited bool - - // Check if we already have existing metrics - if g.hasExistingMetrics() { - log.Info("State size metrics already initialized") - inited = true - } - - initDone := g.initialize() - - for { - select { - case update := <-g.updateChan: - g.handleUpdate(update, inited) - - case <-g.abort: - log.Info("State size generation aborted") - - // Wait for initialization to complete with timeout - if initDone != nil { - select { - case <-initDone: - log.Debug("Initialization completed before abort") - case <-time.After(5 * time.Second): - log.Warn("Initialization did not finish in time during abort") - } - } - return - - case <-initDone: - // Initialization completed, merge buffered metrics - if g.buffered != nil { - log.Info("Merging buffered metrics into main metrics") - g.metrics.Root = g.buffered.Root - g.metrics.AccountCount += g.buffered.AccountCount - g.metrics.AccountBytes += g.buffered.AccountBytes - g.metrics.StorageCount += g.buffered.StorageCount - g.metrics.StorageBytes += g.buffered.StorageBytes - g.metrics.TrieNodeCount += g.buffered.TrieNodeCount - g.metrics.TrieNodeBytes += g.buffered.TrieNodeBytes - g.metrics.ContractCount += g.buffered.ContractCount - g.metrics.ContractBytes += g.buffered.ContractBytes - - g.buffered = nil - } - - inited = true - initDone = nil // Clear the channel to prevent future selects - } - } -} - -// initialize starts the initialization process if not already initialized -func (g *StateSizeGenerator) initialize() chan struct{} { - done := make(chan struct{}) - - // Wait for snapshot completion and then initialize - go func() { - defer close(done) - - LOOP: - // Wait for snapshot generator to complete first - for { - root, done := g.triedb.SnapshotCompleted() - if done { - g.metrics.Root = root - g.buffered.Root = root - break LOOP - } - - select { - case <-g.abort: - log.Info("State size initialization aborted during snapshot wait") - return - case <-time.After(10 * time.Second): - // Continue checking for snapshot completion - } - } - - // Start actual initialization - start := time.Now() - log.Info("Starting state size initialization") - if err := g.initializeMetrics(); err != nil { - log.Error("Failed to initialize state size metrics", "err", err) - return - } - - log.Info("Completed state size initialization", "elapsed", time.Since(start)) - }() - - return done -} - -// handleUpdate processes a single update with proper root continuity checking -func (g *StateSizeGenerator) handleUpdate(update *stateUpdate, initialized bool) { - diff := g.calculateUpdateDiff(update) - - var targetMetrics *stateSizeMetrics - if initialized { - targetMetrics = g.metrics - } else { - targetMetrics = g.buffered - } - - // Check root continuity - the update should build on our current state - if targetMetrics.Root != (common.Hash{}) && targetMetrics.Root != update.originRoot { - log.Warn("State update root discontinuity detected", "current", targetMetrics.Root, "updateOrigin", update.originRoot, "updateNew", update.root) - } - - // Update to the new state root - targetMetrics.Root = update.root - targetMetrics.AccountCount += diff.AccountCount - targetMetrics.AccountBytes += diff.AccountBytes - targetMetrics.StorageCount += diff.StorageCount - targetMetrics.StorageBytes += diff.StorageBytes - targetMetrics.TrieNodeCount += diff.TrieNodeCount - targetMetrics.TrieNodeBytes += diff.TrieNodeBytes - targetMetrics.ContractCount += diff.ContractCount - targetMetrics.ContractBytes += diff.ContractBytes - - // Fire the metrics and persist only if initialization is done - if initialized { - g.updateMetrics() - g.persistMetrics() - } -} - -// calculateUpdateDiff calculates the diff for a state update -func (g *StateSizeGenerator) calculateUpdateDiff(update *stateUpdate) stateSizeMetrics { - var diff stateSizeMetrics - - // Calculate account changes - for addr, oldValue := range update.accountsOrigin { - addrHash := crypto.Keccak256Hash(addr.Bytes()) - newValue, exists := update.accounts[addrHash] - if !exists { - log.Warn("State update missing account", "address", addr) - continue - } - - oldLen, newLen := len(oldValue), len(newValue) - if oldLen > 0 && newLen == 0 { - // Account deletion - diff.AccountCount -= 1 - diff.AccountBytes -= accountSnapKeySize + int64(oldLen) - } else if oldLen == 0 && newLen > 0 { - // Account creation - diff.AccountCount += 1 - diff.AccountBytes += accountSnapKeySize + int64(newLen) - } else { - // Account update - diff.AccountBytes += int64(newLen - oldLen) - } - } - - // Calculate storage changes - for addr, slots := range update.storagesOrigin { - addrHash := crypto.Keccak256Hash(addr.Bytes()) - subset, exists := update.storages[addrHash] - if !exists { - log.Warn("State update missing storage", "address", addr) - continue - } - for key, oldValue := range slots { - var ( - exists bool - newValue []byte - ) - if update.rawStorageKey { - newValue, exists = subset[crypto.Keccak256Hash(key.Bytes())] - } else { - newValue, exists = subset[key] - } - if !exists { - log.Warn("State update missing storage slot", "address", addr, "key", key) - continue - } - - oldLen, newLen := len(oldValue), len(newValue) - if oldLen > 0 && newLen == 0 { - // Storage deletion - diff.StorageCount -= 1 - diff.StorageBytes -= storageSnapKeySize + int64(oldLen) - } else if oldLen == 0 && newLen > 0 { - // Storage creation - diff.StorageCount += 1 - diff.StorageBytes += storageSnapKeySize + int64(newLen) - } else { - // Storage update - diff.StorageBytes += int64(newLen - oldLen) - } - } - } - - // Calculate trie node changes - for owner, subset := range update.nodes.Sets { - isAccountTrie := owner == (common.Hash{}) - var keyPrefixSize int64 - if isAccountTrie { - keyPrefixSize = accountTrieKeyPrefixSize - } else { - keyPrefixSize = storageTrieKeyPrefixSize - } - - // Iterate over Origins since every modified node has an origin entry - for path, oldNode := range subset.Origins { - newNode, hasNew := subset.Nodes[path] - - keySize := keyPrefixSize + int64(len(path)) - - if len(oldNode) > 0 && (!hasNew || len(newNode.Blob) == 0) { - // Node deletion - diff.TrieNodeCount -= 1 - diff.TrieNodeBytes -= keySize + int64(len(oldNode)) - } else if len(oldNode) == 0 && hasNew && len(newNode.Blob) > 0 { - // Node creation - diff.TrieNodeCount += 1 - diff.TrieNodeBytes += keySize + int64(len(newNode.Blob)) - } else if len(oldNode) > 0 && hasNew && len(newNode.Blob) > 0 { - // Node update - diff.TrieNodeBytes += int64(len(newNode.Blob) - len(oldNode)) - } - } - } - - // Calculate code changes - for _, code := range update.codes { - diff.ContractCount += 1 - diff.ContractBytes += codeKeySize + int64(len(code.blob)) - } - - return diff -} - -// Track is an async method used to send the state update to the generator. -// It ignores empty updates (where no state changes occurred). -// If the channel is full, it drops the update to avoid blocking. -func (g *StateSizeGenerator) Track(update *stateUpdate) { - if update == nil || update.empty() { - return - } - - g.updateChan <- update -} - -// hasExistingMetrics checks if state size metrics already exist in the database -// and if they are continuous with the current root -func (g *StateSizeGenerator) hasExistingMetrics() bool { - data := rawdb.ReadStateSizeMetrics(g.db) - if data == nil { - return false - } - - var existed stateSizeMetrics - if err := json.Unmarshal(data, &existed); err != nil { - log.Warn("Failed to decode existed state size metrics", "err", err) - return false - } - - // Check if the existing metrics root matches our current root - if (g.metrics.Root != common.Hash{}) && existed.Root != g.metrics.Root { - log.Info("Existing state size metrics found but root mismatch", "existed", existed.Root, "current", g.metrics.Root) - return false - } - - // Root matches - load the existing metrics - log.Info("Loading existing state size metrics", "root", existed.Root) - g.metrics = &existed - return true -} - -// initializeMetrics performs the actual metrics initialization using errgroup -func (g *StateSizeGenerator) initializeMetrics() error { - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - - go func() { - select { - case <-g.abort: - cancel() // Cancel context when abort is signaled - case <-ctx.Done(): - // Context already cancelled - } - }() - - // Create errgroup with context - group, ctx := errgroup.WithContext(ctx) - - // Metrics will be directly updated by each goroutine - var ( - accountSnapCount, accountSnapBytes int64 - storageSnapCount, storageSnapBytes int64 - accountTrieCount, accountTrieBytes int64 - storageTrieCount, storageTrieBytes int64 - contractCount, contractBytes int64 - ) - - // Start all table iterations concurrently with direct metric updates - group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotAccountPrefix, "accountSnap") - if err != nil { - return err - } - accountSnapCount, accountSnapBytes = count, bytes - return nil - }) - - group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.SnapshotStoragePrefix, "storageSnap") - if err != nil { - return err - } - storageSnapCount, storageSnapBytes = count, bytes - return nil - }) - - group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeAccountPrefix, "accountTrie") - if err != nil { - return err - } - accountTrieCount, accountTrieBytes = count, bytes - return nil - }) - - group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.TrieNodeStoragePrefix, "storageTrie") - if err != nil { - return err - } - storageTrieCount, storageTrieBytes = count, bytes - return nil - }) - - group.Go(func() error { - count, bytes, err := g.iterateTable(ctx, rawdb.CodePrefix, "contract") - if err != nil { - return err - } - contractCount, contractBytes = count, bytes - return nil - }) - - // Wait for all goroutines to complete - if err := group.Wait(); err != nil { - return err - } - - g.metrics.AccountCount = accountSnapCount - g.metrics.AccountBytes = accountSnapBytes - g.metrics.StorageCount = storageSnapCount - g.metrics.StorageBytes = storageSnapBytes - g.metrics.TrieNodeCount = accountTrieCount + storageTrieCount - g.metrics.TrieNodeBytes = accountTrieBytes + storageTrieBytes - g.metrics.ContractCount = contractCount - g.metrics.ContractBytes = contractBytes - - g.updateMetrics() - g.persistMetrics() - - return nil -} - -// iterateTable performs iteration over a specific table and returns the results -func (g *StateSizeGenerator) iterateTable(ctx context.Context, prefix []byte, name string) (int64, int64, error) { - log.Info("Iterating over state size", "table", name) - start := time.Now() - - var count, bytes int64 - iter := g.db.NewIterator(prefix, nil) - defer iter.Release() - - for iter.Next() { - count++ - bytes += int64(len(iter.Key()) + len(iter.Value())) - - // Check for cancellation periodically for performance - if count%10000 == 0 { - select { - case <-ctx.Done(): - log.Info("State size iteration cancelled", "table", name, "count", count) - return 0, 0, ctx.Err() - default: - } - } - } - - // Check for iterator errors - if err := iter.Error(); err != nil { - log.Error("Iterator error during state size calculation", "table", name, "err", err) - return 0, 0, err - } - - log.Info("Finished iterating over state size", "table", name, "count", count, "bytes", bytes, "elapsed", common.PrettyDuration(time.Since(start))) - - return count, bytes, nil -} - -func (g *StateSizeGenerator) updateMetrics() { - accountCountGauge.Update(g.metrics.AccountCount) - accountBytesGauge.Update(g.metrics.AccountBytes) - storageCountGauge.Update(g.metrics.StorageCount) - storageBytesGauge.Update(g.metrics.StorageBytes) - trienodeCountGauge.Update(g.metrics.TrieNodeCount) - trienodeBytesGauge.Update(g.metrics.TrieNodeBytes) - contractCountGauge.Update(g.metrics.ContractCount) - contractBytesGauge.Update(g.metrics.ContractBytes) -} - -// persistMetrics saves the current metrics to the database -func (g *StateSizeGenerator) persistMetrics() { - // RLP doesn't support int64, so we use JSON for simplicity - data, err := json.Marshal(*g.metrics) - if err != nil { - log.Error("Failed to encode state size metrics", "err", err) - return - } - - batch := g.db.NewBatch() - rawdb.WriteStateSizeMetrics(batch, data) - if err := batch.Write(); err != nil { - log.Error("Failed to persist state size metrics", "err", err) - } -} diff --git a/core/state/stateupdate.go b/core/state/stateupdate.go index 75c4ca028c4..a62e2b2d2df 100644 --- a/core/state/stateupdate.go +++ b/core/state/stateupdate.go @@ -64,8 +64,10 @@ type accountUpdate struct { // execution. It contains information about mutated contract codes, accounts, // and storage slots, along with their original values. type stateUpdate struct { - originRoot common.Hash // hash of the state before applying mutation - root common.Hash // hash of the state after applying mutation + originRoot common.Hash // hash of the state before applying mutation + root common.Hash // hash of the state after applying mutation + blockNumber uint64 // Associated block number + accounts map[common.Hash][]byte // accounts stores mutated accounts in 'slim RLP' encoding accountsOrigin map[common.Address][]byte // accountsOrigin stores the original values of mutated accounts in 'slim RLP' encoding @@ -95,7 +97,7 @@ func (sc *stateUpdate) empty() bool { // // rawStorageKey is a flag indicating whether to use the raw storage slot key or // the hash of the slot key for constructing state update object. -func newStateUpdate(rawStorageKey bool, originRoot common.Hash, root common.Hash, deletes map[common.Hash]*accountDelete, updates map[common.Hash]*accountUpdate, nodes *trienode.MergedNodeSet) *stateUpdate { +func newStateUpdate(rawStorageKey bool, originRoot common.Hash, root common.Hash, blockNumber uint64, deletes map[common.Hash]*accountDelete, updates map[common.Hash]*accountUpdate, nodes *trienode.MergedNodeSet) *stateUpdate { var ( accounts = make(map[common.Hash][]byte) accountsOrigin = make(map[common.Address][]byte) @@ -164,6 +166,7 @@ func newStateUpdate(rawStorageKey bool, originRoot common.Hash, root common.Hash return &stateUpdate{ originRoot: originRoot, root: root, + blockNumber: blockNumber, accounts: accounts, accountsOrigin: accountsOrigin, storages: storages, diff --git a/core/state/trie_prefetcher_test.go b/core/state/trie_prefetcher_test.go index 41349c0c0e7..37cd2b6826d 100644 --- a/core/state/trie_prefetcher_test.go +++ b/core/state/trie_prefetcher_test.go @@ -73,7 +73,7 @@ func TestVerklePrefetcher(t *testing.T) { state, err := New(types.EmptyRootHash, sdb) if err != nil { - t.Fatalf("failed to initialize state: %v", err) + t.Fatalf("failed to init state: %v", err) } // Create an account and check if the retrieved balance is correct addr := testrand.Address() diff --git a/triedb/database.go b/triedb/database.go index cba35a67e9c..d2637bd909a 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -377,10 +377,10 @@ func (db *Database) Disk() ethdb.Database { } // SnapshotCompleted returns the indicator if the snapshot is completed. -func (db *Database) SnapshotCompleted() (common.Hash, bool) { +func (db *Database) SnapshotCompleted() bool { pdb, ok := db.backend.(*pathdb.Database) if !ok { - return common.Hash{}, false + return false } return pdb.SnapshotCompleted() } diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index 0156ddce1b8..1592f97d017 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -683,13 +683,12 @@ func (db *Database) StorageIterator(root common.Hash, account common.Hash, seek } // SnapshotCompleted returns the snapshot root if the snapshot generation is completed. -func (db *Database) SnapshotCompleted() (common.Hash, bool) { - if db.waitSync { - return common.Hash{}, false - } - dl := db.tree.bottom() - if dl.genComplete() { - return dl.rootHash(), true +func (db *Database) SnapshotCompleted() bool { + db.lock.RLock() + wait := db.waitSync + db.lock.RUnlock() + if wait { + return false } - return common.Hash{}, false + return db.tree.bottom().genComplete() } From fa1970fba707e367cf084465ff10df85cfe2d3ad Mon Sep 17 00:00:00 2001 From: Gary Rong Date: Wed, 20 Aug 2025 11:01:24 +0800 Subject: [PATCH 18/43] core: update --- core/blockchain.go | 8 ++++++-- core/state/state_sizer.go | 33 ++++++++++++++++++++++++--------- 2 files changed, 30 insertions(+), 11 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 663dbc31f6c..83c09336d7f 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -533,8 +533,12 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, // Start state size tracker if bc.cfg.EnableStateSizeTracking { - bc.stateSizer = state.NewSizeTracker(bc.db, bc.triedb) - bc.stateSizer.Start() + stateSizer, err := state.NewSizeTracker(bc.db, bc.triedb) + if err == nil { + bc.stateSizer = stateSizer + } else { + log.Info("Failed to setup size tracker", "err", err) + } } return bc, nil } diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 687b2d043e8..c092e255af7 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -25,6 +25,7 @@ import ( "time" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" @@ -40,6 +41,7 @@ const ( // Metrics for uploading the state statistics. var ( + blockInfoGauge = metrics.NewRegisteredGaugeInfo("state/size/block", nil) accountsGauge = metrics.NewRegisteredGauge("state/size/account/count", nil) accountBytesGauge = metrics.NewRegisteredGauge("state/size/account/bytes", nil) storagesGauge = metrics.NewRegisteredGauge("state/size/storage/count", nil) @@ -237,18 +239,19 @@ type SizeTracker struct { } // NewSizeTracker creates a new state size tracker and starts it automatically -func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) *SizeTracker { - return &SizeTracker{ +func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) (*SizeTracker, error) { + if triedb.Scheme() != rawdb.PathScheme { + return nil, errors.New("state size tracker is not compatible with hash mode") + } + t := &SizeTracker{ db: db, triedb: triedb, abort: make(chan struct{}), aborted: make(chan struct{}), updateCh: make(chan *stateUpdate), } -} - -func (t *SizeTracker) Start() { go t.run() + return t, nil } func (t *SizeTracker) Stop() { @@ -299,11 +302,12 @@ func (t *SizeTracker) run() { continue } stat := base.add(diff) + stats[u.root] = stat t.upload(stat) - stats[u.root] = stat heap.Push(&h, stats[u.root]) for u.blockNumber-h[0].BlockNumber > statEvictThreshold { + delete(stats, h[0].StateRoot) heap.Pop(&h) } @@ -317,6 +321,7 @@ type buildResult struct { stat SizeStats root common.Hash blockNumber uint64 + elapsed time.Duration err error } @@ -332,6 +337,8 @@ wait: if t.triedb.SnapshotCompleted() { break wait } + case <-t.updateCh: + continue case <-t.abort: return nil, errors.New("size tracker closed") } @@ -360,6 +367,7 @@ wait: if done == nil { done = make(chan buildResult) go t.build(entry.root, entry.blockNumber, done) + log.Info("Measuring persistent state size", "root", root, "number", entry.blockNumber) } case result := <-done: @@ -368,9 +376,9 @@ wait: } var ( stats = make(map[common.Hash]SizeStats) - apply func(root common.Hash, stats SizeStats) error + apply func(root common.Hash, stat SizeStats) error ) - apply = func(root common.Hash, stat SizeStats) error { + apply = func(root common.Hash, base SizeStats) error { for _, child := range children[root] { entry, ok := updates[child] if !ok { @@ -380,7 +388,7 @@ wait: if err != nil { return err } - stats[child] = stat.add(diff) + stats[child] = base.add(diff) if err := apply(child, stats[child]); err != nil { return err } @@ -390,6 +398,7 @@ wait: if err := apply(result.root, result.stat); err != nil { return nil, err } + log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "elapsed", common.PrettyDuration(result.elapsed)) return stats, nil case <-t.abort: @@ -409,6 +418,7 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil storageTrienodes, storageTrienodeBytes int64 group errgroup.Group + start = time.Now() ) // Start all table iterations concurrently with direct metric updates @@ -479,6 +489,7 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil root: root, blockNumber: blockNumber, stat: stat, + elapsed: time.Since(start), } } } @@ -534,6 +545,10 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str } func (t *SizeTracker) upload(stats SizeStats) { + blockInfoGauge.Update(metrics.GaugeInfoValue{ + "number": hexutil.Uint64(stats.BlockNumber).String(), + "hash": stats.StateRoot.Hex(), + }) accountsGauge.Update(stats.Accounts) accountBytesGauge.Update(stats.AccountBytes) storagesGauge.Update(stats.Storages) From 64cfce5b5d5b0bc601f49223e9dfcb97058361d0 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 20 Aug 2025 16:26:38 +0800 Subject: [PATCH 19/43] fix: add snapshot root --- core/state/state_sizer.go | 27 +++++++++++++++++++++------ 1 file changed, 21 insertions(+), 6 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index c092e255af7..8ca4edb9b53 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -347,8 +347,21 @@ wait: var ( updates = make(map[common.Hash]*stateUpdate) children = make(map[common.Hash][]common.Hash) - done = make(chan buildResult) + done chan buildResult ) + + // Add snapshot root to updates map when snapshot completes + if root := rawdb.ReadSnapshotRoot(t.db); root != (common.Hash{}) { + var number uint64 + if recoveryNumber := rawdb.ReadSnapshotRecoveryNumber(t.db); recoveryNumber != nil { + number = *recoveryNumber + } + updates[root] = &stateUpdate{ + root: root, + blockNumber: number, + } + } + for { select { case u := <-t.updateCh: @@ -356,6 +369,10 @@ wait: children[u.originRoot] = append(children[u.originRoot], u.root) case <-ticker.C: + // Only check timer if build hasn't started yet + if done != nil { + continue + } root := rawdb.ReadSnapshotRoot(t.db) if root == (common.Hash{}) { continue @@ -364,11 +381,9 @@ wait: if !exists { continue } - if done == nil { - done = make(chan buildResult) - go t.build(entry.root, entry.blockNumber, done) - log.Info("Measuring persistent state size", "root", root, "number", entry.blockNumber) - } + done = make(chan buildResult) + go t.build(entry.root, entry.blockNumber, done) + log.Info("Measuring persistent state size", "root", root, "number", entry.blockNumber) case result := <-done: if result.err != nil { From 6f7c81ff712eca6fa9287ab14dc2b4c16066689d Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 20 Aug 2025 12:38:50 +0000 Subject: [PATCH 20/43] cmd: --metrics.statesize Signed-off-by: jsvisa --- cmd/geth/config.go | 5 ++++- cmd/geth/main.go | 1 + cmd/utils/flags.go | 11 +++++++++++ metrics/config.go | 4 ++++ 4 files changed, 20 insertions(+), 1 deletion(-) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index 96bd715e889..8c06677167a 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -209,7 +209,7 @@ func constructDevModeBanner(ctx *cli.Context, cfg gethConfig) string { 0x%x (10^49 ETH) `, cfg.Eth.Miner.PendingFeeRecipient) if cfg.Eth.Miner.PendingFeeRecipient == utils.DeveloperAddr { - devModeBanner += fmt.Sprintf(` + devModeBanner += fmt.Sprintf(` Private Key ------------------ 0x%x @@ -375,6 +375,9 @@ func applyMetricConfig(ctx *cli.Context, cfg *gethConfig) { if ctx.IsSet(utils.MetricsInfluxDBOrganizationFlag.Name) { cfg.Metrics.InfluxDBOrganization = ctx.String(utils.MetricsInfluxDBOrganizationFlag.Name) } + if ctx.IsSet(utils.MetricsStateSizeFlag.Name) { + cfg.Metrics.EnableStateSizeTracking = ctx.Bool(utils.MetricsStateSizeFlag.Name) + } // Sanity-check the commandline flags. It is fine if some unused fields is part // of the toml-config, but we expect the commandline to only contain relevant // arguments, otherwise it indicates an error. diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 2da5c432161..54e1b4c4c10 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -200,6 +200,7 @@ var ( utils.MetricsInfluxDBTokenFlag, utils.MetricsInfluxDBBucketFlag, utils.MetricsInfluxDBOrganizationFlag, + utils.MetricsStateSizeFlag, } ) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index bfc1ff0983d..b58249a637c 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -966,6 +966,13 @@ Please note that --` + MetricsHTTPFlag.Name + ` must be set to start the server. Value: metrics.DefaultConfig.InfluxDBOrganization, Category: flags.MetricsCategory, } + + MetricsStateSizeFlag = &cli.BoolFlag{ + Name: "metrics.statesize", + Usage: "Enable state size tracking for metrics collection", + Value: metrics.DefaultConfig.EnableStateSizeTracking, + Category: flags.MetricsCategory, + } ) var ( @@ -2218,6 +2225,10 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh } else if ctx.IsSet(CacheFlag.Name) || ctx.IsSet(CacheSnapshotFlag.Name) { options.SnapshotLimit = ctx.Int(CacheFlag.Name) * ctx.Int(CacheSnapshotFlag.Name) / 100 } + if ctx.Bool(MetricsStateSizeFlag.Name) { + log.Info("Enabling state size tracking") + options.EnableStateSizeTracking = true + } // If we're in readonly, do not bother generating snapshot data. if readonly { options.SnapshotNoBuild = true diff --git a/metrics/config.go b/metrics/config.go index 72f94dd194c..ea916b3673d 100644 --- a/metrics/config.go +++ b/metrics/config.go @@ -33,6 +33,8 @@ type Config struct { InfluxDBToken string `toml:",omitempty"` InfluxDBBucket string `toml:",omitempty"` InfluxDBOrganization string `toml:",omitempty"` + + EnableStateSizeTracking bool `toml:",omitempty"` } // DefaultConfig is the default config for metrics used in go-ethereum. @@ -53,4 +55,6 @@ var DefaultConfig = Config{ InfluxDBToken: "test", InfluxDBBucket: "geth", InfluxDBOrganization: "geth", + + EnableStateSizeTracking: false, } From 1472040dc4ce8f7a0d15cfb37b33617d2d49ae04 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 20 Aug 2025 14:51:13 +0000 Subject: [PATCH 21/43] fix: enable state size tracker if metrics and flag are enabled Signed-off-by: jsvisa --- cmd/geth/config.go | 3 --- cmd/utils/flags.go | 11 ++++++----- eth/backend.go | 3 ++- eth/ethconfig/config.go | 3 +++ metrics/config.go | 4 ---- 5 files changed, 11 insertions(+), 13 deletions(-) diff --git a/cmd/geth/config.go b/cmd/geth/config.go index 8c06677167a..33a3eadea84 100644 --- a/cmd/geth/config.go +++ b/cmd/geth/config.go @@ -375,9 +375,6 @@ func applyMetricConfig(ctx *cli.Context, cfg *gethConfig) { if ctx.IsSet(utils.MetricsInfluxDBOrganizationFlag.Name) { cfg.Metrics.InfluxDBOrganization = ctx.String(utils.MetricsInfluxDBOrganizationFlag.Name) } - if ctx.IsSet(utils.MetricsStateSizeFlag.Name) { - cfg.Metrics.EnableStateSizeTracking = ctx.Bool(utils.MetricsStateSizeFlag.Name) - } // Sanity-check the commandline flags. It is fine if some unused fields is part // of the toml-config, but we expect the commandline to only contain relevant // arguments, otherwise it indicates an error. diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index b58249a637c..e39a13db962 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -970,7 +970,7 @@ Please note that --` + MetricsHTTPFlag.Name + ` must be set to start the server. MetricsStateSizeFlag = &cli.BoolFlag{ Name: "metrics.statesize", Usage: "Enable state size tracking for metrics collection", - Value: metrics.DefaultConfig.EnableStateSizeTracking, + Value: ethconfig.Defaults.EnableStateSizeTracking, Category: flags.MetricsCategory, } ) @@ -1733,6 +1733,11 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { cfg.EthDiscoveryURLs = SplitAndTrim(urls) } } + + if ctx.Bool(MetricsEnabledFlag.Name) && ctx.Bool(MetricsStateSizeFlag.Name) { + log.Error("Enabling state size metrics") + cfg.EnableStateSizeTracking = true + } // Override any default configs for hard coded networks. switch { case ctx.Bool(MainnetFlag.Name): @@ -2225,10 +2230,6 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh } else if ctx.IsSet(CacheFlag.Name) || ctx.IsSet(CacheSnapshotFlag.Name) { options.SnapshotLimit = ctx.Int(CacheFlag.Name) * ctx.Int(CacheSnapshotFlag.Name) / 100 } - if ctx.Bool(MetricsStateSizeFlag.Name) { - log.Info("Enabling state size tracking") - options.EnableStateSizeTracking = true - } // If we're in readonly, do not bother generating snapshot data. if readonly { options.SnapshotNoBuild = true diff --git a/eth/backend.go b/eth/backend.go index 7616ec9d313..ef28878a2d9 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -240,7 +240,8 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) { // within the data directory. The corresponding paths will be either: // - DATADIR/triedb/merkle.journal // - DATADIR/triedb/verkle.journal - TrieJournalDirectory: stack.ResolvePath("triedb"), + TrieJournalDirectory: stack.ResolvePath("triedb"), + EnableStateSizeTracking: config.EnableStateSizeTracking, } ) if config.VMTrace != "" { diff --git a/eth/ethconfig/config.go b/eth/ethconfig/config.go index 82c3c500a77..dc771410816 100644 --- a/eth/ethconfig/config.go +++ b/eth/ethconfig/config.go @@ -144,6 +144,9 @@ type Config struct { // Enables tracking of SHA3 preimages in the VM EnablePreimageRecording bool + // Enables tracking of state size + EnableStateSizeTracking bool + // Enables VM tracing VMTrace string VMTraceJsonConfig string diff --git a/metrics/config.go b/metrics/config.go index ea916b3673d..72f94dd194c 100644 --- a/metrics/config.go +++ b/metrics/config.go @@ -33,8 +33,6 @@ type Config struct { InfluxDBToken string `toml:",omitempty"` InfluxDBBucket string `toml:",omitempty"` InfluxDBOrganization string `toml:",omitempty"` - - EnableStateSizeTracking bool `toml:",omitempty"` } // DefaultConfig is the default config for metrics used in go-ethereum. @@ -55,6 +53,4 @@ var DefaultConfig = Config{ InfluxDBToken: "test", InfluxDBBucket: "geth", InfluxDBOrganization: "geth", - - EnableStateSizeTracking: false, } From 533988cd07f8828f8233e902be101caf22829de7 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 08:43:40 +0800 Subject: [PATCH 22/43] core: add state tracker test --- core/state/state_sizer_test.go | 297 +++++++++++++++++++++++++++++++++ 1 file changed, 297 insertions(+) create mode 100644 core/state/state_sizer_test.go diff --git a/core/state/state_sizer_test.go b/core/state/state_sizer_test.go new file mode 100644 index 00000000000..081f89d4ceb --- /dev/null +++ b/core/state/state_sizer_test.go @@ -0,0 +1,297 @@ +// Copyright 2025 The go-ethereum Authors +// This file is part of the go-ethereum library. +// +// The go-ethereum library is free software: you can redistribute it and/or modify +// it under the terms of the GNU Lesser General Public License as published by +// the Free Software Foundation, either version 3 of the License, or +// (at your option) any later version. +// +// The go-ethereum library is distributed in the hope that it will be useful, +// but WITHOUT ANY WARRANTY; without even the implied warranty of +// MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the +// GNU Lesser General Public License for more details. +// +// You should have received a copy of the GNU Lesser General Public License +// along with the go-ethereum library. If not, see . + +package state + +import ( + "testing" + "time" + + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/core/rawdb" + "github.com/ethereum/go-ethereum/core/tracing" + "github.com/ethereum/go-ethereum/core/types" + "github.com/ethereum/go-ethereum/ethdb/pebble" + "github.com/ethereum/go-ethereum/triedb" + "github.com/ethereum/go-ethereum/triedb/pathdb" + "github.com/holiman/uint256" +) + +func TestSizeTracker(t *testing.T) { + tempDir := t.TempDir() + + pdb, err := pebble.New(tempDir, 0, 0, "", false) + if err != nil { + t.Fatalf("Failed to create pebble database: %v", err) + } + defer pdb.Close() + + db := rawdb.NewDatabase(pdb) + + tdb := triedb.NewDatabase(db, &triedb.Config{PathDB: pathdb.Defaults}) + sdb := NewDatabase(tdb, nil) + + state, _ := New(types.EmptyRootHash, sdb) + + testAddr1 := common.HexToAddress("0x1234567890123456789012345678901234567890") + testAddr2 := common.HexToAddress("0x2345678901234567890123456789012345678901") + testAddr3 := common.HexToAddress("0x3456789012345678901234567890123456789012") + + state.AddBalance(testAddr1, uint256.NewInt(1000), tracing.BalanceChangeUnspecified) + state.SetNonce(testAddr1, 1, tracing.NonceChangeUnspecified) + state.SetState(testAddr1, common.HexToHash("0x1111"), common.HexToHash("0xaaaa")) + state.SetState(testAddr1, common.HexToHash("0x2222"), common.HexToHash("0xbbbb")) + + state.AddBalance(testAddr2, uint256.NewInt(2000), tracing.BalanceChangeUnspecified) + state.SetNonce(testAddr2, 2, tracing.NonceChangeUnspecified) + state.SetCode(testAddr2, []byte{0x60, 0x80, 0x60, 0x40, 0x52}) + + state.AddBalance(testAddr3, uint256.NewInt(3000), tracing.BalanceChangeUnspecified) + state.SetNonce(testAddr3, 3, tracing.NonceChangeUnspecified) + + root1, _, err := state.CommitWithUpdate(1, true, false) + if err != nil { + t.Fatalf("Failed to commit initial state: %v", err) + } + if err := tdb.Commit(root1, false); err != nil { + t.Fatalf("Failed to commit trie: %v", err) + } + + // Generate 50 blocks first to establish a baseline + baselineBlockNum := uint64(50) + currentRoot := root1 + + for i := 0; i < 49; i++ { // blocks 2-50 + blockNum := uint64(i + 2) + + // Create new state from the previous committed root + newState, err := New(currentRoot, sdb) + if err != nil { + t.Fatalf("Failed to create new state at block %d: %v", blockNum, err) + } + + testAddr := common.BigToAddress(uint256.NewInt(uint64(i + 100)).ToBig()) + newState.AddBalance(testAddr, uint256.NewInt(uint64((i+1)*1000)), tracing.BalanceChangeUnspecified) + newState.SetNonce(testAddr, uint64(i+10), tracing.NonceChangeUnspecified) + + if i%2 == 0 { + newState.SetState(testAddr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), + common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) + } + + if i%3 == 0 { + newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}) + } + + root, _, err := newState.CommitWithUpdate(blockNum, true, false) + if err != nil { + t.Fatalf("Failed to commit state at block %d: %v", blockNum, err) + } + if err := tdb.Commit(root, false); err != nil { + t.Fatalf("Failed to commit trie at block %d: %v", blockNum, err) + } + + currentRoot = root + } + + baselineRoot := currentRoot + rawdb.WriteSnapshotRoot(db, baselineRoot) + + // Wait for snapshot completion + for !tdb.SnapshotCompleted() { + time.Sleep(100 * time.Millisecond) + } + + // Calculate baseline from the intermediate persisted state + baselineTracker := &SizeTracker{ + db: db, + triedb: tdb, + abort: make(chan struct{}), + } + + done := make(chan buildResult) + go baselineTracker.build(baselineRoot, baselineBlockNum, done) + var baselineResult buildResult + select { + case baselineResult = <-done: + if baselineResult.err != nil { + t.Fatalf("Failed to get baseline stats: %v", baselineResult.err) + } + case <-time.After(30 * time.Second): + t.Fatal("Timeout waiting for baseline stats") + } + baseline := baselineResult.stat + + // Now start the tracker and notify it of updates that happen AFTER the baseline + tracker, err := NewSizeTracker(db, tdb) + if err != nil { + t.Fatalf("Failed to create size tracker: %v", err) + } + defer tracker.Stop() + + // Continue from where we left off (block 51+) and track those updates + var trackedUpdates []SizeStats + currentRoot = baselineRoot + + // Generate additional blocks beyond the baseline and track them + for i := 49; i < 130; i++ { // blocks 51-132 + blockNum := uint64(i + 2) + newState, _ := New(currentRoot, sdb) + + testAddr := common.BigToAddress(uint256.NewInt(uint64(i + 100)).ToBig()) + newState.AddBalance(testAddr, uint256.NewInt(uint64((i+1)*1000)), tracing.BalanceChangeUnspecified) + newState.SetNonce(testAddr, uint64(i+10), tracing.NonceChangeUnspecified) + + if i%2 == 0 { + newState.SetState(testAddr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), + common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) + } + + if i%3 == 0 { + newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}) + } + + root, update, err := newState.CommitWithUpdate(blockNum, true, false) + if err != nil { + t.Fatalf("Failed to commit state at block %d: %v", blockNum, err) + } + if err := tdb.Commit(root, false); err != nil { + t.Fatalf("Failed to commit trie at block %d: %v", blockNum, err) + } + + diff, err := calSizeStats(update) + if err != nil { + t.Fatalf("Failed to calculate size stats for block %d: %v", blockNum, err) + } + trackedUpdates = append(trackedUpdates, diff) + tracker.Notify(update) + currentRoot = root + } + + // Give the StateTracker time to process all the notifications we sent + time.Sleep(100 * time.Millisecond) + + finalRoot := currentRoot + + finalTracker := &SizeTracker{ + db: db, + triedb: tdb, + abort: make(chan struct{}), + } + + finalDone := make(chan buildResult) + go finalTracker.build(finalRoot, uint64(132), finalDone) + var result buildResult + select { + case result = <-finalDone: + if result.err != nil { + t.Fatalf("Failed to build final stats: %v", result.err) + } + case <-time.After(30 * time.Second): + t.Fatal("Timeout waiting for final stats") + } + + actualStats := result.stat + + // Now we have a proper test: + // - Baseline measured at block 50 (with snapshot completion) + // - Final state measured at block 132 + // - Tracked updates from blocks 51-132 (should show growth) + + // Verify that both baseline and final measurements show reasonable data + if baseline.Accounts < 50 { + t.Errorf("Expected baseline to have at least 50 accounts, got %d", baseline.Accounts) + } + if baseline.StorageBytes == 0 { + t.Errorf("Expected baseline to have storage data, got 0 bytes") + } + + if actualStats.Accounts <= baseline.Accounts { + t.Errorf("Expected final state to have more accounts than baseline: baseline=%d, final=%d", baseline.Accounts, actualStats.Accounts) + } + + if actualStats.StorageBytes <= baseline.StorageBytes { + t.Errorf("Expected final state to have more storage than baseline: baseline=%d, final=%d", baseline.StorageBytes, actualStats.StorageBytes) + } + + expectedStats := baseline + for _, diff := range trackedUpdates { + expectedStats = expectedStats.add(diff) + } + + // The final measured stats should match our calculated expected stats exactly + if actualStats.Accounts != expectedStats.Accounts { + t.Errorf("Account count mismatch: baseline(%d) + tracked_changes = %d, but final_measurement = %d", baseline.Accounts, expectedStats.Accounts, actualStats.Accounts) + } + if actualStats.AccountBytes != expectedStats.AccountBytes { + t.Errorf("Account bytes mismatch: expected %d, got %d", expectedStats.AccountBytes, actualStats.AccountBytes) + } + if actualStats.Storages != expectedStats.Storages { + t.Errorf("Storage count mismatch: baseline(%d) + tracked_changes = %d, but final_measurement = %d", baseline.Storages, expectedStats.Storages, actualStats.Storages) + } + if actualStats.StorageBytes != expectedStats.StorageBytes { + t.Errorf("Storage bytes mismatch: expected %d, got %d", expectedStats.StorageBytes, actualStats.StorageBytes) + } + if actualStats.ContractCodes != expectedStats.ContractCodes { + t.Errorf("Contract code count mismatch: baseline(%d) + tracked_changes = %d, but final_measurement = %d", baseline.ContractCodes, expectedStats.ContractCodes, actualStats.ContractCodes) + } + if actualStats.ContractCodeBytes != expectedStats.ContractCodeBytes { + t.Errorf("Contract code bytes mismatch: expected %d, got %d", expectedStats.ContractCodeBytes, actualStats.ContractCodeBytes) + } + if actualStats.AccountTrienodes != expectedStats.AccountTrienodes { + t.Errorf("Account trie nodes mismatch: expected %d, got %d", expectedStats.AccountTrienodes, actualStats.AccountTrienodes) + } + if actualStats.AccountTrienodeBytes != expectedStats.AccountTrienodeBytes { + t.Errorf("Account trie node bytes mismatch: expected %d, got %d", expectedStats.AccountTrienodeBytes, actualStats.AccountTrienodeBytes) + } + if actualStats.StorageTrienodes != expectedStats.StorageTrienodes { + t.Errorf("Storage trie nodes mismatch: expected %d, got %d", expectedStats.StorageTrienodes, actualStats.StorageTrienodes) + } + if actualStats.StorageTrienodeBytes != expectedStats.StorageTrienodeBytes { + t.Errorf("Storage trie node bytes mismatch: expected %d, got %d", expectedStats.StorageTrienodeBytes, actualStats.StorageTrienodeBytes) + } + + // Verify reasonable growth occurred + accountGrowth := actualStats.Accounts - baseline.Accounts + storageGrowth := actualStats.Storages - baseline.Storages + codeGrowth := actualStats.ContractCodes - baseline.ContractCodes + + if accountGrowth <= 0 { + t.Errorf("Expected account growth, got %d", accountGrowth) + } + if storageGrowth <= 0 { + t.Errorf("Expected storage growth, got %d", storageGrowth) + } + if codeGrowth <= 0 { + t.Errorf("Expected contract code growth, got %d", codeGrowth) + } + + // Verify we successfully tracked updates from blocks 51-132 + expectedUpdates := 81 // blocks 51-132 (81 blocks) + if len(trackedUpdates) < 70 || len(trackedUpdates) > expectedUpdates { + t.Errorf("Expected 70-%d tracked updates, got %d", expectedUpdates, len(trackedUpdates)) + } + + t.Logf("Baseline stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", + baseline.Accounts, baseline.AccountBytes, baseline.Storages, baseline.StorageBytes, baseline.ContractCodes) + t.Logf("Expected stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", + expectedStats.Accounts, expectedStats.AccountBytes, expectedStats.Storages, expectedStats.StorageBytes, expectedStats.ContractCodes) + t.Logf("Final stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", + actualStats.Accounts, actualStats.AccountBytes, actualStats.Storages, actualStats.StorageBytes, actualStats.ContractCodes) + t.Logf("Growth: Accounts=+%d, StorageSlots=+%d, ContractCodes=+%d", + accountGrowth, storageGrowth, codeGrowth) + t.Logf("Tracked %d state updates from %d blocks successfully", len(trackedUpdates), 81) +} From db070e667b0c3cfb7b8f09e8e006fee4d2f0ca64 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 08:45:12 +0800 Subject: [PATCH 23/43] fix: use 0 as snapshot block --- core/state/state_sizer.go | 9 +-------- 1 file changed, 1 insertion(+), 8 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 8ca4edb9b53..5f84b226a45 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -352,14 +352,7 @@ wait: // Add snapshot root to updates map when snapshot completes if root := rawdb.ReadSnapshotRoot(t.db); root != (common.Hash{}) { - var number uint64 - if recoveryNumber := rawdb.ReadSnapshotRecoveryNumber(t.db); recoveryNumber != nil { - number = *recoveryNumber - } - updates[root] = &stateUpdate{ - root: root, - blockNumber: number, - } + updates[root] = &stateUpdate{root: root} } for { From 9a3a6c72e282fea7b27090721b62a5ce0c29fbf1 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 08:45:52 +0800 Subject: [PATCH 24/43] fix: add with base --- core/state/state_sizer.go | 29 ++++++++++++++--------------- 1 file changed, 14 insertions(+), 15 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 5f84b226a45..ac9d007f1d1 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -83,21 +83,20 @@ type SizeStats struct { // add applies the given state diffs and produces a new version of the statistics. func (s SizeStats) add(diff SizeStats) SizeStats { - var combo SizeStats - combo.StateRoot = diff.StateRoot - combo.BlockNumber = diff.BlockNumber - - combo.Accounts += diff.Accounts - combo.AccountBytes += diff.AccountBytes - combo.Storages += diff.Storages - combo.StorageBytes += diff.StorageBytes - combo.AccountTrienodes += diff.AccountTrienodes - combo.AccountTrienodeBytes += diff.AccountTrienodeBytes - combo.StorageTrienodes += diff.StorageTrienodes - combo.StorageTrienodeBytes += diff.StorageTrienodeBytes - combo.ContractCodes += diff.ContractCodes - combo.ContractCodeBytes += diff.ContractCodeBytes - return combo + s.StateRoot = diff.StateRoot + s.BlockNumber = diff.BlockNumber + + s.Accounts += diff.Accounts + s.AccountBytes += diff.AccountBytes + s.Storages += diff.Storages + s.StorageBytes += diff.StorageBytes + s.AccountTrienodes += diff.AccountTrienodes + s.AccountTrienodeBytes += diff.AccountTrienodeBytes + s.StorageTrienodes += diff.StorageTrienodes + s.StorageTrienodeBytes += diff.StorageTrienodeBytes + s.ContractCodes += diff.ContractCodes + s.ContractCodeBytes += diff.ContractCodeBytes + return s } // calSizeStats measures the state size changes of the provided state update. From 48c3f91efc231d6d89cc0ef7e83dfdb9285ab942 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 09:37:20 +0800 Subject: [PATCH 25/43] test --- core/state/state_sizer_test.go | 118 ++++++++++++--------------------- 1 file changed, 41 insertions(+), 77 deletions(-) diff --git a/core/state/state_sizer_test.go b/core/state/state_sizer_test.go index 081f89d4ceb..ef0f9bd0761 100644 --- a/core/state/state_sizer_test.go +++ b/core/state/state_sizer_test.go @@ -40,44 +40,44 @@ func TestSizeTracker(t *testing.T) { defer pdb.Close() db := rawdb.NewDatabase(pdb) + defer db.Close() tdb := triedb.NewDatabase(db, &triedb.Config{PathDB: pathdb.Defaults}) sdb := NewDatabase(tdb, nil) - state, _ := New(types.EmptyRootHash, sdb) + // Generate 50 blocks to establish a baseline + baselineBlockNum := uint64(50) + currentRoot := types.EmptyRootHash - testAddr1 := common.HexToAddress("0x1234567890123456789012345678901234567890") - testAddr2 := common.HexToAddress("0x2345678901234567890123456789012345678901") - testAddr3 := common.HexToAddress("0x3456789012345678901234567890123456789012") + addr1 := common.BytesToAddress([]byte{1, 0, 0, 1}) + addr2 := common.BytesToAddress([]byte{1, 0, 0, 2}) + addr3 := common.BytesToAddress([]byte{1, 0, 0, 3}) - state.AddBalance(testAddr1, uint256.NewInt(1000), tracing.BalanceChangeUnspecified) - state.SetNonce(testAddr1, 1, tracing.NonceChangeUnspecified) - state.SetState(testAddr1, common.HexToHash("0x1111"), common.HexToHash("0xaaaa")) - state.SetState(testAddr1, common.HexToHash("0x2222"), common.HexToHash("0xbbbb")) + // Create initial state with fixed accounts + state, _ := New(currentRoot, sdb) + state.AddBalance(addr1, uint256.NewInt(1000), tracing.BalanceChangeUnspecified) + state.SetNonce(addr1, 1, tracing.NonceChangeUnspecified) + state.SetState(addr1, common.HexToHash("0x1111"), common.HexToHash("0xaaaa")) + state.SetState(addr1, common.HexToHash("0x2222"), common.HexToHash("0xbbbb")) - state.AddBalance(testAddr2, uint256.NewInt(2000), tracing.BalanceChangeUnspecified) - state.SetNonce(testAddr2, 2, tracing.NonceChangeUnspecified) - state.SetCode(testAddr2, []byte{0x60, 0x80, 0x60, 0x40, 0x52}) + state.AddBalance(addr2, uint256.NewInt(2000), tracing.BalanceChangeUnspecified) + state.SetNonce(addr2, 2, tracing.NonceChangeUnspecified) + state.SetCode(addr2, []byte{0x60, 0x80, 0x60, 0x40, 0x52}) - state.AddBalance(testAddr3, uint256.NewInt(3000), tracing.BalanceChangeUnspecified) - state.SetNonce(testAddr3, 3, tracing.NonceChangeUnspecified) + state.AddBalance(addr3, uint256.NewInt(3000), tracing.BalanceChangeUnspecified) + state.SetNonce(addr3, 3, tracing.NonceChangeUnspecified) - root1, _, err := state.CommitWithUpdate(1, true, false) + currentRoot, _, err = state.CommitWithUpdate(1, true, false) if err != nil { t.Fatalf("Failed to commit initial state: %v", err) } - if err := tdb.Commit(root1, false); err != nil { - t.Fatalf("Failed to commit trie: %v", err) + if err := tdb.Commit(currentRoot, false); err != nil { + t.Fatalf("Failed to commit initial trie: %v", err) } - // Generate 50 blocks first to establish a baseline - baselineBlockNum := uint64(50) - currentRoot := root1 - - for i := 0; i < 49; i++ { // blocks 2-50 - blockNum := uint64(i + 2) + for i := 1; i < 50; i++ { // blocks 2-50 + blockNum := uint64(i + 1) - // Create new state from the previous committed root newState, err := New(currentRoot, sdb) if err != nil { t.Fatalf("Failed to create new state at block %d: %v", blockNum, err) @@ -88,8 +88,7 @@ func TestSizeTracker(t *testing.T) { newState.SetNonce(testAddr, uint64(i+10), tracing.NonceChangeUnspecified) if i%2 == 0 { - newState.SetState(testAddr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), - common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) + newState.SetState(addr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) } if i%3 == 0 { @@ -108,7 +107,6 @@ func TestSizeTracker(t *testing.T) { } baselineRoot := currentRoot - rawdb.WriteSnapshotRoot(db, baselineRoot) // Wait for snapshot completion for !tdb.SnapshotCompleted() { @@ -142,22 +140,23 @@ func TestSizeTracker(t *testing.T) { } defer tracker.Stop() - // Continue from where we left off (block 51+) and track those updates var trackedUpdates []SizeStats currentRoot = baselineRoot // Generate additional blocks beyond the baseline and track them for i := 49; i < 130; i++ { // blocks 51-132 blockNum := uint64(i + 2) - newState, _ := New(currentRoot, sdb) + newState, err := New(currentRoot, sdb) + if err != nil { + t.Fatalf("Failed to create new state at block %d: %v", blockNum, err) + } testAddr := common.BigToAddress(uint256.NewInt(uint64(i + 100)).ToBig()) newState.AddBalance(testAddr, uint256.NewInt(uint64((i+1)*1000)), tracing.BalanceChangeUnspecified) newState.SetNonce(testAddr, uint64(i+10), tracing.NonceChangeUnspecified) if i%2 == 0 { - newState.SetState(testAddr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), - common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) + newState.SetState(addr1, common.BigToHash(uint256.NewInt(uint64(i+0x1000)).ToBig()), common.BigToHash(uint256.NewInt(uint64(i+0x2000)).ToBig())) } if i%3 == 0 { @@ -181,10 +180,19 @@ func TestSizeTracker(t *testing.T) { currentRoot = root } - // Give the StateTracker time to process all the notifications we sent - time.Sleep(100 * time.Millisecond) + if len(trackedUpdates) != 130-49 { + t.Errorf("Expected %d tracked updates, got %d", 130-49, len(trackedUpdates)) + } + + finalRoot := rawdb.ReadSnapshotRoot(db) - finalRoot := currentRoot + // Ensure all commits are flushed to disk + if err := tdb.Close(); err != nil { + t.Fatalf("Failed to close triedb: %v", err) + } + + // Reopen the database to simulate a restart + tdb = triedb.NewDatabase(db, &triedb.Config{PathDB: pathdb.Defaults}) finalTracker := &SizeTracker{ db: db, @@ -206,27 +214,6 @@ func TestSizeTracker(t *testing.T) { actualStats := result.stat - // Now we have a proper test: - // - Baseline measured at block 50 (with snapshot completion) - // - Final state measured at block 132 - // - Tracked updates from blocks 51-132 (should show growth) - - // Verify that both baseline and final measurements show reasonable data - if baseline.Accounts < 50 { - t.Errorf("Expected baseline to have at least 50 accounts, got %d", baseline.Accounts) - } - if baseline.StorageBytes == 0 { - t.Errorf("Expected baseline to have storage data, got 0 bytes") - } - - if actualStats.Accounts <= baseline.Accounts { - t.Errorf("Expected final state to have more accounts than baseline: baseline=%d, final=%d", baseline.Accounts, actualStats.Accounts) - } - - if actualStats.StorageBytes <= baseline.StorageBytes { - t.Errorf("Expected final state to have more storage than baseline: baseline=%d, final=%d", baseline.StorageBytes, actualStats.StorageBytes) - } - expectedStats := baseline for _, diff := range trackedUpdates { expectedStats = expectedStats.add(diff) @@ -264,34 +251,11 @@ func TestSizeTracker(t *testing.T) { t.Errorf("Storage trie node bytes mismatch: expected %d, got %d", expectedStats.StorageTrienodeBytes, actualStats.StorageTrienodeBytes) } - // Verify reasonable growth occurred - accountGrowth := actualStats.Accounts - baseline.Accounts - storageGrowth := actualStats.Storages - baseline.Storages - codeGrowth := actualStats.ContractCodes - baseline.ContractCodes - - if accountGrowth <= 0 { - t.Errorf("Expected account growth, got %d", accountGrowth) - } - if storageGrowth <= 0 { - t.Errorf("Expected storage growth, got %d", storageGrowth) - } - if codeGrowth <= 0 { - t.Errorf("Expected contract code growth, got %d", codeGrowth) - } - - // Verify we successfully tracked updates from blocks 51-132 - expectedUpdates := 81 // blocks 51-132 (81 blocks) - if len(trackedUpdates) < 70 || len(trackedUpdates) > expectedUpdates { - t.Errorf("Expected 70-%d tracked updates, got %d", expectedUpdates, len(trackedUpdates)) - } - t.Logf("Baseline stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", baseline.Accounts, baseline.AccountBytes, baseline.Storages, baseline.StorageBytes, baseline.ContractCodes) t.Logf("Expected stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", expectedStats.Accounts, expectedStats.AccountBytes, expectedStats.Storages, expectedStats.StorageBytes, expectedStats.ContractCodes) t.Logf("Final stats: Accounts=%d, AccountBytes=%d, Storages=%d, StorageBytes=%d, ContractCodes=%d", actualStats.Accounts, actualStats.AccountBytes, actualStats.Storages, actualStats.StorageBytes, actualStats.ContractCodes) - t.Logf("Growth: Accounts=+%d, StorageSlots=+%d, ContractCodes=+%d", - accountGrowth, storageGrowth, codeGrowth) t.Logf("Tracked %d state updates from %d blocks successfully", len(trackedUpdates), 81) } From d74dd391311fa99714d7178d76f22841fb1f14bf Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 10:04:31 +0800 Subject: [PATCH 26/43] expose stats --- core/state/state_sizer.go | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index ac9d007f1d1..0d2a13bc965 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -235,6 +235,7 @@ type SizeTracker struct { abort chan struct{} aborted chan struct{} updateCh chan *stateUpdate + stats map[common.Hash]SizeStats // internal stats map } // NewSizeTracker creates a new state size tracker and starts it automatically @@ -248,6 +249,7 @@ func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) (*SizeTrack abort: make(chan struct{}), aborted: make(chan struct{}), updateCh: make(chan *stateUpdate), + stats: make(map[common.Hash]SizeStats), } go t.run() return t, nil @@ -286,13 +288,14 @@ func (t *SizeTracker) run() { if err != nil { return } + t.stats = stats h := sizeStatsHeap(slices.Collect(maps.Values(stats))) heap.Init(&h) for { select { case u := <-t.updateCh: - base, found := stats[u.originRoot] + base, found := t.stats[u.originRoot] if !found { continue } @@ -301,15 +304,14 @@ func (t *SizeTracker) run() { continue } stat := base.add(diff) - stats[u.root] = stat + t.stats[u.root] = stat t.upload(stat) - heap.Push(&h, stats[u.root]) - for u.blockNumber-h[0].BlockNumber > statEvictThreshold { - delete(stats, h[0].StateRoot) + heap.Push(&h, t.stats[u.root]) + for len(h) > 0 && u.blockNumber-h[0].BlockNumber > statEvictThreshold { + delete(t.stats, h[0].StateRoot) heap.Pop(&h) } - case <-t.abort: return } @@ -512,6 +514,8 @@ func (t *SizeTracker) Notify(update *stateUpdate) { case t.updateCh <- update: case <-t.abort: return + default: + // Drop update if channel is full } } From 898057af09e756495f2ceda959ef4b42f1a71f0b Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 10:22:39 +0800 Subject: [PATCH 27/43] Revert "expose stats" This reverts commit d0cef767e12fc63d081871fc3913af83719a0034. --- core/state/state_sizer.go | 16 ++++++---------- 1 file changed, 6 insertions(+), 10 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 0d2a13bc965..ac9d007f1d1 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -235,7 +235,6 @@ type SizeTracker struct { abort chan struct{} aborted chan struct{} updateCh chan *stateUpdate - stats map[common.Hash]SizeStats // internal stats map } // NewSizeTracker creates a new state size tracker and starts it automatically @@ -249,7 +248,6 @@ func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) (*SizeTrack abort: make(chan struct{}), aborted: make(chan struct{}), updateCh: make(chan *stateUpdate), - stats: make(map[common.Hash]SizeStats), } go t.run() return t, nil @@ -288,14 +286,13 @@ func (t *SizeTracker) run() { if err != nil { return } - t.stats = stats h := sizeStatsHeap(slices.Collect(maps.Values(stats))) heap.Init(&h) for { select { case u := <-t.updateCh: - base, found := t.stats[u.originRoot] + base, found := stats[u.originRoot] if !found { continue } @@ -304,14 +301,15 @@ func (t *SizeTracker) run() { continue } stat := base.add(diff) - t.stats[u.root] = stat + stats[u.root] = stat t.upload(stat) - heap.Push(&h, t.stats[u.root]) - for len(h) > 0 && u.blockNumber-h[0].BlockNumber > statEvictThreshold { - delete(t.stats, h[0].StateRoot) + heap.Push(&h, stats[u.root]) + for u.blockNumber-h[0].BlockNumber > statEvictThreshold { + delete(stats, h[0].StateRoot) heap.Pop(&h) } + case <-t.abort: return } @@ -514,8 +512,6 @@ func (t *SizeTracker) Notify(update *stateUpdate) { case t.updateCh <- update: case <-t.abort: return - default: - // Drop update if channel is full } } From 706efc597e72938832ae21b009748cd9e1a96e12 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 25 Aug 2025 10:31:34 +0800 Subject: [PATCH 28/43] debug log --- core/state/state_sizer.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index ac9d007f1d1..8670f3ec24e 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -81,6 +81,17 @@ type SizeStats struct { ContractCodeBytes int64 // Total size of all contract code (in bytes) } +func (s SizeStats) String() string { + return fmt.Sprintf("Block: %d Root: %s, Accounts: %d (%s), Storages: %d (%s), AccountTrienodes: %d (%s), StorageTrienodes: %d (%s), ContractCodes: %d (%s)", + s.BlockNumber, s.StateRoot.Hex(), + s.Accounts, common.StorageSize(s.AccountBytes), + s.Storages, common.StorageSize(s.StorageBytes), + s.AccountTrienodes, common.StorageSize(s.AccountTrienodeBytes), + s.StorageTrienodes, common.StorageSize(s.StorageTrienodeBytes), + s.ContractCodes, common.StorageSize(s.ContractCodeBytes), + ) +} + // add applies the given state diffs and produces a new version of the statistics. func (s SizeStats) add(diff SizeStats) SizeStats { s.StateRoot = diff.StateRoot @@ -375,7 +386,7 @@ wait: } done = make(chan buildResult) go t.build(entry.root, entry.blockNumber, done) - log.Info("Measuring persistent state size", "root", root, "number", entry.blockNumber) + log.Info("Measuring persistent state size", "root", root.Hex()) case result := <-done: if result.err != nil { @@ -405,6 +416,7 @@ wait: if err := apply(result.root, result.stat); err != nil { return nil, err } + log.Info("Init state size", "stat", result.stat) log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "elapsed", common.PrettyDuration(result.elapsed)) return stats, nil @@ -552,6 +564,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str } func (t *SizeTracker) upload(stats SizeStats) { + log.Info("Update state size", "stat", stats) blockInfoGauge.Update(metrics.GaugeInfoValue{ "number": hexutil.Uint64(stats.BlockNumber).String(), "hash": stats.StateRoot.Hex(), From f1cb76d0f2fd17c896c87f08dbed0ea07542cc75 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 26 Aug 2025 06:30:06 +0800 Subject: [PATCH 29/43] triedb: force flush --- core/blockchain.go | 1 + triedb/database.go | 9 +++++++++ triedb/pathdb/database.go | 16 +++++++++++++++- triedb/pathdb/disklayer.go | 2 +- 4 files changed, 26 insertions(+), 2 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 83c09336d7f..eea945f30c2 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -536,6 +536,7 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, stateSizer, err := state.NewSizeTracker(bc.db, bc.triedb) if err == nil { bc.stateSizer = stateSizer + triedb.EnableForceFlush() } else { log.Info("Failed to setup size tracker", "err", err) } diff --git a/triedb/database.go b/triedb/database.go index d2637bd909a..61f612d7cc5 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -384,3 +384,12 @@ func (db *Database) SnapshotCompleted() bool { } return pdb.SnapshotCompleted() } + +// EnableForceFlush enables the pathdb to flush any pending changes to disk immediately, +// regardless of the buffer size threshold. This can be used to accelerate +// state sizer initialization by making buffered state changes visible on disk. +func (db *Database) EnableForceFlush() { + if pdb, ok := db.backend.(*pathdb.Database); ok { + pdb.EnableForceFlush() + } +} diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index 1592f97d017..f708e8ef0aa 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -137,7 +137,8 @@ type Database struct { stateFreezer ethdb.ResettableAncientStore // Freezer for storing state histories, nil possible in tests stateIndexer *historyIndexer // History indexer historical state data, nil possible - lock sync.RWMutex // Lock to prevent mutations from happening at the same time + lock sync.RWMutex // Lock to prevent mutations from happening at the same time + forceFlush bool // Flag to force buffer flush regardless of size } // New attempts to load an already existing layer from a persistent key-value @@ -360,6 +361,19 @@ func (db *Database) Commit(root common.Hash, report bool) error { return db.tree.cap(root, 0) } +// EnableForceFlush enables force flushing for the next state update. +// This will cause the next Update() call to flush the disk buffer immediately, +// regardless of the buffer threshold, while preserving the 128 diff layers in memory. +func (db *Database) EnableForceFlush() { + db.lock.Lock() + defer db.lock.Unlock() + + if !db.forceFlush { + log.Info("Enabling force flush for next pathdb update") + db.forceFlush = true + } +} + // Disable deactivates the database and invalidates all available state layers // as stale to prevent access to the persistent state, which is in the syncing // stage. diff --git a/triedb/pathdb/disklayer.go b/triedb/pathdb/disklayer.go index 2042e916110..5a531672039 100644 --- a/triedb/pathdb/disklayer.go +++ b/triedb/pathdb/disklayer.go @@ -417,7 +417,7 @@ func (dl *diskLayer) commit(bottom *diffLayer, force bool) (*diskLayer, error) { // Terminate the background state snapshot generation before mutating the // persistent state. - if combined.full() || force || flush { + if combined.full() || force || flush || dl.db.forceFlush { // Wait until the previous frozen buffer is fully flushed if dl.frozen != nil { if err := dl.frozen.waitFlush(); err != nil { From e1469bc414eb1cc63ccb19cbc82ccaa930fb0d7e Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 26 Aug 2025 06:31:36 +0800 Subject: [PATCH 30/43] fix: wait for snapshot root is ready --- core/state/state_sizer.go | 5 ----- 1 file changed, 5 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 8670f3ec24e..3692be85f80 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -360,11 +360,6 @@ wait: done chan buildResult ) - // Add snapshot root to updates map when snapshot completes - if root := rawdb.ReadSnapshotRoot(t.db); root != (common.Hash{}) { - updates[root] = &stateUpdate{root: root} - } - for { select { case u := <-t.updateCh: From 02474bd0bced74a1d26e37dcb0bf29e2e93fcb35 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 26 Aug 2025 07:06:23 +0800 Subject: [PATCH 31/43] force flush controlled in state_sizer --- core/blockchain.go | 1 - core/state/state_sizer.go | 4 ++++ triedb/database.go | 8 ++++---- triedb/pathdb/database.go | 12 ++++-------- 4 files changed, 12 insertions(+), 13 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index eea945f30c2..83c09336d7f 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -536,7 +536,6 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, stateSizer, err := state.NewSizeTracker(bc.db, bc.triedb) if err == nil { bc.stateSizer = stateSizer - triedb.EnableForceFlush() } else { log.Info("Failed to setup size tracker", "err", err) } diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 3692be85f80..8a0f8c6009d 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -360,6 +360,8 @@ wait: done chan buildResult ) + t.triedb.SetForceFlush(true) + for { select { case u := <-t.updateCh: @@ -384,6 +386,8 @@ wait: log.Info("Measuring persistent state size", "root", root.Hex()) case result := <-done: + t.triedb.SetForceFlush(false) + if result.err != nil { return nil, result.err } diff --git a/triedb/database.go b/triedb/database.go index 61f612d7cc5..37a57ac0220 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -385,11 +385,11 @@ func (db *Database) SnapshotCompleted() bool { return pdb.SnapshotCompleted() } -// EnableForceFlush enables the pathdb to flush any pending changes to disk immediately, -// regardless of the buffer size threshold. This can be used to accelerate +// SetForceFlush enables or disables the pathdb to flush any pending changes to disk +// immediately, regardless of the buffer size threshold. This can be used to accelerate // state sizer initialization by making buffered state changes visible on disk. -func (db *Database) EnableForceFlush() { +func (db *Database) SetForceFlush(enabled bool) { if pdb, ok := db.backend.(*pathdb.Database); ok { - pdb.EnableForceFlush() + pdb.SetForceFlush(enabled) } } diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index f708e8ef0aa..0ca5d8a73aa 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -361,17 +361,13 @@ func (db *Database) Commit(root common.Hash, report bool) error { return db.tree.cap(root, 0) } -// EnableForceFlush enables force flushing for the next state update. -// This will cause the next Update() call to flush the disk buffer immediately, -// regardless of the buffer threshold, while preserving the 128 diff layers in memory. -func (db *Database) EnableForceFlush() { +// SetForceFlush enables or disables force flushing for the next state update. +func (db *Database) SetForceFlush(enabled bool) { db.lock.Lock() defer db.lock.Unlock() - if !db.forceFlush { - log.Info("Enabling force flush for next pathdb update") - db.forceFlush = true - } + db.forceFlush = enabled + log.Info("Set triedb force flush for next pathdb update", "enabled", enabled) } // Disable deactivates the database and invalidates all available state layers From c26dde39db7e5d0b4451c2246122e705c817871c Mon Sep 17 00:00:00 2001 From: jsvisa Date: Tue, 26 Aug 2025 09:46:06 +0800 Subject: [PATCH 32/43] log --- cmd/utils/flags.go | 2 +- core/state/state_sizer.go | 10 ++++------ 2 files changed, 5 insertions(+), 7 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index e39a13db962..8aa7f023bca 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -1735,7 +1735,7 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { } if ctx.Bool(MetricsEnabledFlag.Name) && ctx.Bool(MetricsStateSizeFlag.Name) { - log.Error("Enabling state size metrics") + log.Info("Enabling state size metrics") cfg.EnableStateSizeTracking = true } // Override any default configs for hard coded networks. diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 8a0f8c6009d..e845aef1ac1 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -82,8 +82,7 @@ type SizeStats struct { } func (s SizeStats) String() string { - return fmt.Sprintf("Block: %d Root: %s, Accounts: %d (%s), Storages: %d (%s), AccountTrienodes: %d (%s), StorageTrienodes: %d (%s), ContractCodes: %d (%s)", - s.BlockNumber, s.StateRoot.Hex(), + return fmt.Sprintf("Accounts: %d (%s), Storages: %d (%s), AccountTrienodes: %d (%s), StorageTrienodes: %d (%s), ContractCodes: %d (%s)", s.Accounts, common.StorageSize(s.AccountBytes), s.Storages, common.StorageSize(s.StorageBytes), s.AccountTrienodes, common.StorageSize(s.AccountTrienodeBytes), @@ -383,7 +382,7 @@ wait: } done = make(chan buildResult) go t.build(entry.root, entry.blockNumber, done) - log.Info("Measuring persistent state size", "root", root.Hex()) + log.Info("Measuring persistent state size", "root", root.Hex(), "number", entry.blockNumber) case result := <-done: t.triedb.SetForceFlush(false) @@ -415,8 +414,7 @@ wait: if err := apply(result.root, result.stat); err != nil { return nil, err } - log.Info("Init state size", "stat", result.stat) - log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "elapsed", common.PrettyDuration(result.elapsed)) + log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "stat", result.stat, "elapsed", common.PrettyDuration(result.elapsed)) return stats, nil case <-t.abort: @@ -563,7 +561,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str } func (t *SizeTracker) upload(stats SizeStats) { - log.Info("Update state size", "stat", stats) + log.Debug("Uploading state size", "number", stats.BlockNumber, "root", stats.StateRoot, "stat", stats) blockInfoGauge.Update(metrics.GaugeInfoValue{ "number": hexutil.Uint64(stats.BlockNumber).String(), "hash": stats.StateRoot.Hex(), From 2ddb6ebd304735a668aca1d554babdc5ca3c9b36 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 28 Aug 2025 09:46:40 +0800 Subject: [PATCH 33/43] use parallel iterate of the storage tables --- core/state/state_sizer.go | 46 +++++++++++++++++++++++++++++++++++++-- 1 file changed, 44 insertions(+), 2 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index e845aef1ac1..ca5a6e4ee4e 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -22,6 +22,7 @@ import ( "fmt" "maps" "slices" + "sync" "time" "github.com/ethereum/go-ethereum/common" @@ -447,8 +448,9 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil return nil }) + // Storage table is huge, iterate in parallel group.Go(func() error { - count, bytes, err := t.iterateTable(t.abort, rawdb.SnapshotStoragePrefix, "storage") + count, bytes, err := t.iterateTableParallel(t.abort, rawdb.SnapshotStoragePrefix, "storage") if err != nil { return err } @@ -465,8 +467,9 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil return nil }) + // Storage trienode table is huge, iterate in parallel group.Go(func() error { - count, bytes, err := t.iterateTable(t.abort, rawdb.TrieNodeStoragePrefix, "storagenode") + count, bytes, err := t.iterateTableParallel(t.abort, rawdb.TrieNodeStoragePrefix, "storagenode") if err != nil { return err } @@ -531,6 +534,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str logged = time.Now() count, bytes int64 ) + iter := t.db.NewIterator(prefix, nil) defer iter.Release() @@ -560,6 +564,44 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str return count, bytes, nil } +// iterateTableParallel performs parallel iteration over a table by splitting into hex ranges +// For storage tables, it splits on the first byte of the account hash (after the prefix) +func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, name string) (int64, int64, error) { + var ( + start = time.Now() + workers = 16 + totalCount int64 + totalBytes int64 + group errgroup.Group + mu sync.Mutex + ) + group.SetLimit(workers) + + log.Info("Starting parallel state iteration", "category", name, "workers", workers) + + for i := 0; i < 256; i++ { + h := byte(i) + group.Go(func() error { + count, bytes, err := t.iterateTable(closed, slices.Concat(prefix, []byte{h}), fmt.Sprintf("%s-%x", name, h)) + if err != nil { + return err + } + mu.Lock() + totalCount += count + totalBytes += bytes + mu.Unlock() + return nil + }) + } + + if err := group.Wait(); err != nil { + return 0, 0, err + } + + log.Info("Finished parallel state iteration", "category", name, "count", totalCount, "size", common.StorageSize(totalBytes), "elapsed", common.PrettyDuration(time.Since(start))) + return totalCount, totalBytes, nil +} + func (t *SizeTracker) upload(stats SizeStats) { log.Debug("Uploading state size", "number", stats.BlockNumber, "root", stats.StateRoot, "stat", stats) blockInfoGauge.Update(metrics.GaugeInfoValue{ From b542e8b3c1fee9a7814b21c35aaa1f04fd0b2091 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 28 Aug 2025 14:19:39 +0800 Subject: [PATCH 34/43] parallel run for all states --- core/state/state_sizer.go | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index ca5a6e4ee4e..67ae35f3795 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -440,7 +440,7 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil // Start all table iterations concurrently with direct metric updates group.Go(func() error { - count, bytes, err := t.iterateTable(t.abort, rawdb.SnapshotAccountPrefix, "account") + count, bytes, err := t.iterateTableParallel(t.abort, rawdb.SnapshotAccountPrefix, "account") if err != nil { return err } @@ -448,7 +448,6 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil return nil }) - // Storage table is huge, iterate in parallel group.Go(func() error { count, bytes, err := t.iterateTableParallel(t.abort, rawdb.SnapshotStoragePrefix, "storage") if err != nil { @@ -459,7 +458,7 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil }) group.Go(func() error { - count, bytes, err := t.iterateTable(t.abort, rawdb.TrieNodeAccountPrefix, "accountnode") + count, bytes, err := t.iterateTableParallel(t.abort, rawdb.TrieNodeAccountPrefix, "accountnode") if err != nil { return err } @@ -467,7 +466,6 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil return nil }) - // Storage trienode table is huge, iterate in parallel group.Go(func() error { count, bytes, err := t.iterateTableParallel(t.abort, rawdb.TrieNodeStoragePrefix, "storagenode") if err != nil { @@ -582,7 +580,7 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, for i := 0; i < 256; i++ { h := byte(i) group.Go(func() error { - count, bytes, err := t.iterateTable(closed, slices.Concat(prefix, []byte{h}), fmt.Sprintf("%s-%x", name, h)) + count, bytes, err := t.iterateTable(closed, slices.Concat(prefix, []byte{h}), fmt.Sprintf("%s-%02x", name, h)) if err != nil { return err } From c99f43e41a3f573e9ef5f5ccd83f4011d30320b3 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Mon, 1 Sep 2025 03:38:11 +0000 Subject: [PATCH 35/43] cmd/geth: enable metrics state size in db import Signed-off-by: jsvisa --- cmd/geth/chaincmd.go | 1 + cmd/utils/flags.go | 3 +++ 2 files changed, 4 insertions(+) diff --git a/cmd/geth/chaincmd.go b/cmd/geth/chaincmd.go index 9868142f53d..1efea541be1 100644 --- a/cmd/geth/chaincmd.go +++ b/cmd/geth/chaincmd.go @@ -108,6 +108,7 @@ if one is set. Otherwise it prints the genesis from the datadir.`, utils.MetricsInfluxDBTokenFlag, utils.MetricsInfluxDBBucketFlag, utils.MetricsInfluxDBOrganizationFlag, + utils.MetricsStateSizeFlag, utils.TxLookupLimitFlag, utils.VMTraceFlag, utils.VMTraceJsonConfigFlag, diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 8aa7f023bca..e8e16f869f0 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -2220,6 +2220,9 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh // - DATADIR/triedb/merkle.journal // - DATADIR/triedb/verkle.journal TrieJournalDirectory: stack.ResolvePath("triedb"), + + // Enable state size tracking if metrics and state size metrics are both enabled + EnableStateSizeTracking: ctx.Bool(MetricsEnabledFlag.Name) && ctx.Bool(MetricsStateSizeFlag.Name), } if options.ArchiveMode && !options.Preimages { options.Preimages = true From 70cd55dcd28208caaad9663c2b193c31d54ba4ac Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 3 Sep 2025 06:12:16 +0800 Subject: [PATCH 36/43] state: debug_stateSize to retrieve the latest state size --- cmd/geth/chaincmd.go | 2 +- cmd/geth/main.go | 2 +- cmd/utils/flags.go | 19 ++++++------- core/blockchain.go | 5 ++++ core/state/state_sizer.go | 57 ++++++++++++++----------------------- eth/api_debug.go | 16 +++++++++++ internal/web3ext/web3ext.go | 5 ++++ 7 files changed, 59 insertions(+), 47 deletions(-) diff --git a/cmd/geth/chaincmd.go b/cmd/geth/chaincmd.go index 1efea541be1..71ff821bb9b 100644 --- a/cmd/geth/chaincmd.go +++ b/cmd/geth/chaincmd.go @@ -108,7 +108,7 @@ if one is set. Otherwise it prints the genesis from the datadir.`, utils.MetricsInfluxDBTokenFlag, utils.MetricsInfluxDBBucketFlag, utils.MetricsInfluxDBOrganizationFlag, - utils.MetricsStateSizeFlag, + utils.StateSizeTrackingFlag, utils.TxLookupLimitFlag, utils.VMTraceFlag, utils.VMTraceJsonConfigFlag, diff --git a/cmd/geth/main.go b/cmd/geth/main.go index 54e1b4c4c10..750bf55927c 100644 --- a/cmd/geth/main.go +++ b/cmd/geth/main.go @@ -200,7 +200,7 @@ var ( utils.MetricsInfluxDBTokenFlag, utils.MetricsInfluxDBBucketFlag, utils.MetricsInfluxDBOrganizationFlag, - utils.MetricsStateSizeFlag, + utils.StateSizeTrackingFlag, } ) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index e8e16f869f0..2328d8545c1 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -628,6 +628,12 @@ var ( TakesFile: true, Category: flags.MiscCategory, } + StateSizeTrackingFlag = &cli.BoolFlag{ + Name: "state-size-tracking", + Usage: "Enable state size tracking", + Value: ethconfig.Defaults.EnableStateSizeTracking, + Category: flags.MiscCategory, + } // RPC settings IPCDisabledFlag = &cli.BoolFlag{ @@ -966,13 +972,6 @@ Please note that --` + MetricsHTTPFlag.Name + ` must be set to start the server. Value: metrics.DefaultConfig.InfluxDBOrganization, Category: flags.MetricsCategory, } - - MetricsStateSizeFlag = &cli.BoolFlag{ - Name: "metrics.statesize", - Usage: "Enable state size tracking for metrics collection", - Value: ethconfig.Defaults.EnableStateSizeTracking, - Category: flags.MetricsCategory, - } ) var ( @@ -1734,7 +1733,7 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *ethconfig.Config) { } } - if ctx.Bool(MetricsEnabledFlag.Name) && ctx.Bool(MetricsStateSizeFlag.Name) { + if ctx.Bool(StateSizeTrackingFlag.Name) { log.Info("Enabling state size metrics") cfg.EnableStateSizeTracking = true } @@ -2221,8 +2220,8 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh // - DATADIR/triedb/verkle.journal TrieJournalDirectory: stack.ResolvePath("triedb"), - // Enable state size tracking if metrics and state size metrics are both enabled - EnableStateSizeTracking: ctx.Bool(MetricsEnabledFlag.Name) && ctx.Bool(MetricsStateSizeFlag.Name), + // Enable state size tracking if enabled + EnableStateSizeTracking: ctx.Bool(StateSizeTrackingFlag.Name), } if options.ArchiveMode && !options.Preimages { options.Preimages = true diff --git a/core/blockchain.go b/core/blockchain.go index 83c09336d7f..cb7fb0c4e3e 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -2814,3 +2814,8 @@ func (bc *BlockChain) SetTrieFlushInterval(interval time.Duration) { func (bc *BlockChain) GetTrieFlushInterval() time.Duration { return time.Duration(bc.flushInterval.Load()) } + +// StateSizer returns the state size tracker, or nil if it's not initialized +func (bc *BlockChain) StateSizer() *state.SizeTracker { + return bc.stateSizer +} diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 67ae35f3795..9e623cd7c4d 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -26,12 +26,10 @@ import ( "time" "github.com/ethereum/go-ethereum/common" - "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/core/rawdb" "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/ethdb" "github.com/ethereum/go-ethereum/log" - "github.com/ethereum/go-ethereum/metrics" "github.com/ethereum/go-ethereum/triedb" "golang.org/x/sync/errgroup" ) @@ -40,25 +38,10 @@ const ( statEvictThreshold = 128 // the depth of statistic to be preserved ) -// Metrics for uploading the state statistics. -var ( - blockInfoGauge = metrics.NewRegisteredGaugeInfo("state/size/block", nil) - accountsGauge = metrics.NewRegisteredGauge("state/size/account/count", nil) - accountBytesGauge = metrics.NewRegisteredGauge("state/size/account/bytes", nil) - storagesGauge = metrics.NewRegisteredGauge("state/size/storage/count", nil) - storageBytesGauge = metrics.NewRegisteredGauge("state/size/storage/bytes", nil) - accountTrienodesGauge = metrics.NewRegisteredGauge("state/size/trienode/account/count", nil) - accountTrienodeBytesGauge = metrics.NewRegisteredGauge("state/size/trienode/account/bytes", nil) - storageTrienodesGauge = metrics.NewRegisteredGauge("state/size/trienode/storage/count", nil) - storageTrienodeBytesGauge = metrics.NewRegisteredGauge("state/size/trienode/storage/bytes", nil) - contractCodesGauge = metrics.NewRegisteredGauge("state/size/contractcode/count", nil) - contractCodeBytesGauge = metrics.NewRegisteredGauge("state/size/contractcode/bytes", nil) -) - // Database key scheme for states. var ( accountKeySize = int64(len(rawdb.SnapshotAccountPrefix) + common.HashLength) - storageKeySize = int64(len(rawdb.SnapshotStoragePrefix) + common.HashLength + common.HashLength) + storageKeySize = int64(len(rawdb.SnapshotStoragePrefix) + common.HashLength*2) accountTrienodePrefixSize = int64(len(rawdb.TrieNodeAccountPrefix)) storageTrienodePrefixSize = int64(len(rawdb.TrieNodeStoragePrefix) + common.HashLength) codeKeySize = int64(len(rawdb.CodePrefix) + common.HashLength) @@ -246,6 +229,9 @@ type SizeTracker struct { abort chan struct{} aborted chan struct{} updateCh chan *stateUpdate + + mu sync.RWMutex + latestStats *SizeStats } // NewSizeTracker creates a new state size tracker and starts it automatically @@ -313,7 +299,12 @@ func (t *SizeTracker) run() { } stat := base.add(diff) stats[u.root] = stat - t.upload(stat) + log.Info("Update state size", "number", stat.BlockNumber, "root", stat.StateRoot, "stat", stat) + + // Update latest stats + t.mu.Lock() + t.latestStats = &stat + t.mu.Unlock() heap.Push(&h, stats[u.root]) for u.blockNumber-h[0].BlockNumber > statEvictThreshold { @@ -415,6 +406,13 @@ wait: if err := apply(result.root, result.stat); err != nil { return nil, err } + + // Set initial latest stats + stats[result.root] = result.stat + t.mu.Lock() + t.latestStats = &result.stat + t.mu.Unlock() + log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "stat", result.stat, "elapsed", common.PrettyDuration(result.elapsed)) return stats, nil @@ -600,20 +598,9 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, return totalCount, totalBytes, nil } -func (t *SizeTracker) upload(stats SizeStats) { - log.Debug("Uploading state size", "number", stats.BlockNumber, "root", stats.StateRoot, "stat", stats) - blockInfoGauge.Update(metrics.GaugeInfoValue{ - "number": hexutil.Uint64(stats.BlockNumber).String(), - "hash": stats.StateRoot.Hex(), - }) - accountsGauge.Update(stats.Accounts) - accountBytesGauge.Update(stats.AccountBytes) - storagesGauge.Update(stats.Storages) - storageBytesGauge.Update(stats.StorageBytes) - accountTrienodesGauge.Update(stats.AccountTrienodes) - accountTrienodeBytesGauge.Update(stats.AccountTrienodeBytes) - storageTrienodesGauge.Update(stats.StorageTrienodes) - storageTrienodeBytesGauge.Update(stats.StorageTrienodeBytes) - contractCodesGauge.Update(stats.ContractCodes) - contractCodeBytesGauge.Update(stats.ContractCodeBytes) +// GetLatestStats returns the latest state size statistics, or nil if not available +func (t *SizeTracker) GetLatestStats() *SizeStats { + t.mu.RLock() + defer t.mu.RUnlock() + return t.latestStats } diff --git a/eth/api_debug.go b/eth/api_debug.go index 188dee11aa5..6934e20d9cd 100644 --- a/eth/api_debug.go +++ b/eth/api_debug.go @@ -443,3 +443,19 @@ func (api *DebugAPI) GetTrieFlushInterval() (string, error) { } return api.eth.blockchain.GetTrieFlushInterval().String(), nil } + +// StateSize returns the current state size statistics from the state size tracker. +// Returns an error if the state size tracker is not initialized or if stats are not ready. +func (api *DebugAPI) StateSize() (*state.SizeStats, error) { + sizer := api.eth.blockchain.StateSizer() + if sizer == nil { + return nil, errors.New("state size tracker is not enabled") + } + + stats := sizer.GetLatestStats() + if stats == nil { + return nil, errors.New("state size statistics are not ready yet") + } + + return stats, nil +} diff --git a/internal/web3ext/web3ext.go b/internal/web3ext/web3ext.go index e81e23ef167..81c30a1f971 100644 --- a/internal/web3ext/web3ext.go +++ b/internal/web3ext/web3ext.go @@ -468,6 +468,11 @@ web3._extend({ call: 'debug_sync', params: 1 }), + new web3._extend.Method({ + name: 'stateSize', + call: 'debug_stateSize', + params: 0, + }), ], properties: [] }); From ff2b072416fe6398d7a6d4970339453f4cedaff7 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Wed, 3 Sep 2025 06:32:26 +0800 Subject: [PATCH 37/43] return hexutil --- eth/api_debug.go | 17 +++++++++++++++-- 1 file changed, 15 insertions(+), 2 deletions(-) diff --git a/eth/api_debug.go b/eth/api_debug.go index 6934e20d9cd..9ce29bc8806 100644 --- a/eth/api_debug.go +++ b/eth/api_debug.go @@ -446,7 +446,7 @@ func (api *DebugAPI) GetTrieFlushInterval() (string, error) { // StateSize returns the current state size statistics from the state size tracker. // Returns an error if the state size tracker is not initialized or if stats are not ready. -func (api *DebugAPI) StateSize() (*state.SizeStats, error) { +func (api *DebugAPI) StateSize() (interface{}, error) { sizer := api.eth.blockchain.StateSizer() if sizer == nil { return nil, errors.New("state size tracker is not enabled") @@ -457,5 +457,18 @@ func (api *DebugAPI) StateSize() (*state.SizeStats, error) { return nil, errors.New("state size statistics are not ready yet") } - return stats, nil + return map[string]interface{}{ + "stateRoot": stats.StateRoot, + "blockNumber": hexutil.Uint64(stats.BlockNumber), + "accounts": hexutil.Uint64(stats.Accounts), + "accountBytes": hexutil.Uint64(stats.AccountBytes), + "storages": hexutil.Uint64(stats.Storages), + "storageBytes": hexutil.Uint64(stats.StorageBytes), + "accountTrienodes": hexutil.Uint64(stats.AccountTrienodes), + "accountTrienodeBytes": hexutil.Uint64(stats.AccountTrienodeBytes), + "storageTrienodes": hexutil.Uint64(stats.StorageTrienodes), + "storageTrienodeBytes": hexutil.Uint64(stats.StorageTrienodeBytes), + "contractCodes": hexutil.Uint64(stats.ContractCodes), + "contractCodeBytes": hexutil.Uint64(stats.ContractCodeBytes), + }, nil } From a4fcb4e8b48b8394625768e22bc6968cfc187df5 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 4 Sep 2025 08:55:50 +0800 Subject: [PATCH 38/43] rename cli to state.size-tracking --- cmd/utils/flags.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 2328d8545c1..21118931109 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -270,6 +270,12 @@ var ( Usage: "Scheme to use for storing ethereum state ('hash' or 'path')", Category: flags.StateCategory, } + StateSizeTrackingFlag = &cli.BoolFlag{ + Name: "state.size-tracking", + Usage: "Enable state size tracking, retrieve state size with debug_stateSize.", + Value: ethconfig.Defaults.EnableStateSizeTracking, + Category: flags.StateCategory, + } StateHistoryFlag = &cli.Uint64Flag{ Name: "history.state", Usage: "Number of recent blocks to retain state history for, only relevant in state.scheme=path (default = 90,000 blocks, 0 = entire chain)", @@ -628,12 +634,6 @@ var ( TakesFile: true, Category: flags.MiscCategory, } - StateSizeTrackingFlag = &cli.BoolFlag{ - Name: "state-size-tracking", - Usage: "Enable state size tracking", - Value: ethconfig.Defaults.EnableStateSizeTracking, - Category: flags.MiscCategory, - } // RPC settings IPCDisabledFlag = &cli.BoolFlag{ From 6f0f67bc6ffa213b6b4890b06185c8fa6a70b022 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 4 Sep 2025 08:40:25 +0000 Subject: [PATCH 39/43] downgrand info to debug log Signed-off-by: jsvisa --- core/state/state_sizer.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 9e623cd7c4d..0f864cfacd0 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -299,7 +299,7 @@ func (t *SizeTracker) run() { } stat := base.add(diff) stats[u.root] = stat - log.Info("Update state size", "number", stat.BlockNumber, "root", stat.StateRoot, "stat", stat) + log.Debug("Update state size", "number", stat.BlockNumber, "root", stat.StateRoot, "stat", stat) // Update latest stats t.mu.Lock() From 0b13632ae2dd41823ad6156bf2cb4fcf6cfe9ce8 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 4 Sep 2025 19:24:18 +0800 Subject: [PATCH 40/43] fix: special case for the prefix only key --- core/state/state_sizer.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 0f864cfacd0..a66e4d3a254 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -21,6 +21,7 @@ import ( "errors" "fmt" "maps" + "runtime" "slices" "sync" "time" @@ -565,7 +566,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, name string) (int64, int64, error) { var ( start = time.Now() - workers = 16 + workers = runtime.NumCPU() totalCount int64 totalBytes int64 group errgroup.Group @@ -575,6 +576,14 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, log.Info("Starting parallel state iteration", "category", name, "workers", workers) + if len(prefix) > 0 { + if blob, err := t.db.Get(prefix); err == nil && len(blob) > 0 { + // If there's a direct hit on the prefix, include it in the stats + totalCount = 1 + totalBytes = int64(len(prefix) + len(blob)) + } + } + for i := 0; i < 256; i++ { h := byte(i) group.Go(func() error { From 1abfb07782d480ab565542e4c4ca846bb50bf996 Mon Sep 17 00:00:00 2001 From: jsvisa Date: Thu, 4 Sep 2025 20:25:52 +0800 Subject: [PATCH 41/43] fix --- core/state/state_sizer.go | 4 ++-- core/state/state_sizer_test.go | 6 +++--- eth/api_debug.go | 24 ++++++++++++------------ 3 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index a66e4d3a254..784d8f8e11e 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -535,7 +535,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str iter := t.db.NewIterator(prefix, nil) defer iter.Release() - log.Info("Iterating state", "category", name) + log.Debug("Iterating state", "category", name) for iter.Next() { count++ bytes += int64(len(iter.Key()) + len(iter.Value())) @@ -557,7 +557,7 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str log.Error("Iterator error", "category", name, "err", err) return 0, 0, err } - log.Info("Finished state iteration", "category", name, "count", count, "size", common.StorageSize(bytes), "elapsed", common.PrettyDuration(time.Since(start))) + log.Debug("Finished state iteration", "category", name, "count", count, "size", common.StorageSize(bytes), "elapsed", common.PrettyDuration(time.Since(start))) return count, bytes, nil } diff --git a/core/state/state_sizer_test.go b/core/state/state_sizer_test.go index ef0f9bd0761..756744d1664 100644 --- a/core/state/state_sizer_test.go +++ b/core/state/state_sizer_test.go @@ -62,7 +62,7 @@ func TestSizeTracker(t *testing.T) { state.AddBalance(addr2, uint256.NewInt(2000), tracing.BalanceChangeUnspecified) state.SetNonce(addr2, 2, tracing.NonceChangeUnspecified) - state.SetCode(addr2, []byte{0x60, 0x80, 0x60, 0x40, 0x52}) + state.SetCode(addr2, []byte{0x60, 0x80, 0x60, 0x40, 0x52}, tracing.CodeChangeUnspecified) state.AddBalance(addr3, uint256.NewInt(3000), tracing.BalanceChangeUnspecified) state.SetNonce(addr3, 3, tracing.NonceChangeUnspecified) @@ -92,7 +92,7 @@ func TestSizeTracker(t *testing.T) { } if i%3 == 0 { - newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}) + newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}, tracing.CodeChangeUnspecified) } root, _, err := newState.CommitWithUpdate(blockNum, true, false) @@ -160,7 +160,7 @@ func TestSizeTracker(t *testing.T) { } if i%3 == 0 { - newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}) + newState.SetCode(testAddr, []byte{byte(i), 0x60, 0x80, byte(i + 1), 0x52}, tracing.CodeChangeUnspecified) } root, update, err := newState.CommitWithUpdate(blockNum, true, false) diff --git a/eth/api_debug.go b/eth/api_debug.go index 9ce29bc8806..3ea6aaeb9e2 100644 --- a/eth/api_debug.go +++ b/eth/api_debug.go @@ -458,17 +458,17 @@ func (api *DebugAPI) StateSize() (interface{}, error) { } return map[string]interface{}{ - "stateRoot": stats.StateRoot, - "blockNumber": hexutil.Uint64(stats.BlockNumber), - "accounts": hexutil.Uint64(stats.Accounts), - "accountBytes": hexutil.Uint64(stats.AccountBytes), - "storages": hexutil.Uint64(stats.Storages), - "storageBytes": hexutil.Uint64(stats.StorageBytes), - "accountTrienodes": hexutil.Uint64(stats.AccountTrienodes), - "accountTrienodeBytes": hexutil.Uint64(stats.AccountTrienodeBytes), - "storageTrienodes": hexutil.Uint64(stats.StorageTrienodes), - "storageTrienodeBytes": hexutil.Uint64(stats.StorageTrienodeBytes), - "contractCodes": hexutil.Uint64(stats.ContractCodes), - "contractCodeBytes": hexutil.Uint64(stats.ContractCodeBytes), + "stateRoot": stats.StateRoot, + "blockNumber": hexutil.Uint64(stats.BlockNumber), + "accounts": hexutil.Uint64(stats.Accounts), + "accountBytes": hexutil.Uint64(stats.AccountBytes), + "storages": hexutil.Uint64(stats.Storages), + "storageBytes": hexutil.Uint64(stats.StorageBytes), + "accountTrienodes": hexutil.Uint64(stats.AccountTrienodes), + "accountTrienodeBytes": hexutil.Uint64(stats.AccountTrienodeBytes), + "storageTrienodes": hexutil.Uint64(stats.StorageTrienodes), + "storageTrienodeBytes": hexutil.Uint64(stats.StorageTrienodeBytes), + "contractCodes": hexutil.Uint64(stats.ContractCodes), + "contractCodeBytes": hexutil.Uint64(stats.ContractCodeBytes), }, nil } From f7332e9d6e356e5b3146bc6301e938476cf75758 Mon Sep 17 00:00:00 2001 From: Gary Rong Date: Fri, 5 Sep 2025 12:10:20 +0800 Subject: [PATCH 42/43] all: polish --- cmd/utils/flags.go | 2 +- core/blockchain.go | 6 +++--- core/state/state_sizer.go | 23 +++++++++++------------ eth/api_debug.go | 2 -- eth/backend.go | 4 ++-- eth/ethconfig/gen_config.go | 6 ++++++ triedb/database.go | 9 --------- triedb/pathdb/database.go | 13 +------------ triedb/pathdb/disklayer.go | 2 +- 9 files changed, 25 insertions(+), 42 deletions(-) diff --git a/cmd/utils/flags.go b/cmd/utils/flags.go index 21118931109..aca3ce91961 100644 --- a/cmd/utils/flags.go +++ b/cmd/utils/flags.go @@ -2221,7 +2221,7 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readonly bool) (*core.BlockCh TrieJournalDirectory: stack.ResolvePath("triedb"), // Enable state size tracking if enabled - EnableStateSizeTracking: ctx.Bool(StateSizeTrackingFlag.Name), + StateSizeTracking: ctx.Bool(StateSizeTrackingFlag.Name), } if options.ArchiveMode && !options.Preimages { options.Preimages = true diff --git a/core/blockchain.go b/core/blockchain.go index cb7fb0c4e3e..2d5527a114b 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -197,8 +197,8 @@ type BlockChainConfig struct { // If the value is -1, indexing is disabled. TxLookupLimit int64 - // EnableStateSizeTracking indicates whether the state size tracking is enabled. - EnableStateSizeTracking bool + // StateSizeTracking indicates whether the state size tracking is enabled. + StateSizeTracking bool } // DefaultConfig returns the default config. @@ -532,7 +532,7 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, } // Start state size tracker - if bc.cfg.EnableStateSizeTracking { + if bc.cfg.StateSizeTracking { stateSizer, err := state.NewSizeTracker(bc.db, bc.triedb) if err == nil { bc.stateSizer = stateSizer diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index 784d8f8e11e..ede4b5b384f 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -292,6 +292,7 @@ func (t *SizeTracker) run() { case u := <-t.updateCh: base, found := stats[u.originRoot] if !found { + log.Debug("Ignored the state size without parent", "parent", u.originRoot, "root", u.root, "number", u.blockNumber) continue } diff, err := calSizeStats(u) @@ -352,8 +353,6 @@ wait: done chan buildResult ) - t.triedb.SetForceFlush(true) - for { select { case u := <-t.updateCh: @@ -378,8 +377,6 @@ wait: log.Info("Measuring persistent state size", "root", root.Hex(), "number", entry.blockNumber) case result := <-done: - t.triedb.SetForceFlush(false) - if result.err != nil { return nil, result.err } @@ -410,6 +407,7 @@ wait: // Set initial latest stats stats[result.root] = result.stat + t.mu.Lock() t.latestStats = &result.stat t.mu.Unlock() @@ -561,16 +559,18 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str return count, bytes, nil } -// iterateTableParallel performs parallel iteration over a table by splitting into hex ranges -// For storage tables, it splits on the first byte of the account hash (after the prefix) +// iterateTableParallel performs parallel iteration over a table by splitting into +// hex ranges. For storage tables, it splits on the first byte of the account hash +// (after the prefix). func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, name string) (int64, int64, error) { var ( - start = time.Now() - workers = runtime.NumCPU() totalCount int64 totalBytes int64 - group errgroup.Group - mu sync.Mutex + + start = time.Now() + workers = runtime.NumCPU() + group errgroup.Group + mu sync.Mutex ) group.SetLimit(workers) @@ -598,11 +598,9 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, return nil }) } - if err := group.Wait(); err != nil { return 0, 0, err } - log.Info("Finished parallel state iteration", "category", name, "count", totalCount, "size", common.StorageSize(totalBytes), "elapsed", common.PrettyDuration(time.Since(start))) return totalCount, totalBytes, nil } @@ -611,5 +609,6 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, func (t *SizeTracker) GetLatestStats() *SizeStats { t.mu.RLock() defer t.mu.RUnlock() + return t.latestStats } diff --git a/eth/api_debug.go b/eth/api_debug.go index 3ea6aaeb9e2..703c584fbc9 100644 --- a/eth/api_debug.go +++ b/eth/api_debug.go @@ -451,12 +451,10 @@ func (api *DebugAPI) StateSize() (interface{}, error) { if sizer == nil { return nil, errors.New("state size tracker is not enabled") } - stats := sizer.GetLatestStats() if stats == nil { return nil, errors.New("state size statistics are not ready yet") } - return map[string]interface{}{ "stateRoot": stats.StateRoot, "blockNumber": hexutil.Uint64(stats.BlockNumber), diff --git a/eth/backend.go b/eth/backend.go index ef28878a2d9..43567331891 100644 --- a/eth/backend.go +++ b/eth/backend.go @@ -240,8 +240,8 @@ func New(stack *node.Node, config *ethconfig.Config) (*Ethereum, error) { // within the data directory. The corresponding paths will be either: // - DATADIR/triedb/merkle.journal // - DATADIR/triedb/verkle.journal - TrieJournalDirectory: stack.ResolvePath("triedb"), - EnableStateSizeTracking: config.EnableStateSizeTracking, + TrieJournalDirectory: stack.ResolvePath("triedb"), + StateSizeTracking: config.EnableStateSizeTracking, } ) if config.VMTrace != "" { diff --git a/eth/ethconfig/gen_config.go b/eth/ethconfig/gen_config.go index 0a188ba23c4..2fdd219dee8 100644 --- a/eth/ethconfig/gen_config.go +++ b/eth/ethconfig/gen_config.go @@ -49,6 +49,7 @@ func (c Config) MarshalTOML() (interface{}, error) { BlobPool blobpool.Config GPO gasprice.Config EnablePreimageRecording bool + EnableStateSizeTracking bool VMTrace string VMTraceJsonConfig string RPCGasCap uint64 @@ -90,6 +91,7 @@ func (c Config) MarshalTOML() (interface{}, error) { enc.BlobPool = c.BlobPool enc.GPO = c.GPO enc.EnablePreimageRecording = c.EnablePreimageRecording + enc.EnableStateSizeTracking = c.EnableStateSizeTracking enc.VMTrace = c.VMTrace enc.VMTraceJsonConfig = c.VMTraceJsonConfig enc.RPCGasCap = c.RPCGasCap @@ -135,6 +137,7 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error { BlobPool *blobpool.Config GPO *gasprice.Config EnablePreimageRecording *bool + EnableStateSizeTracking *bool VMTrace *string VMTraceJsonConfig *string RPCGasCap *uint64 @@ -243,6 +246,9 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error { if dec.EnablePreimageRecording != nil { c.EnablePreimageRecording = *dec.EnablePreimageRecording } + if dec.EnableStateSizeTracking != nil { + c.EnableStateSizeTracking = *dec.EnableStateSizeTracking + } if dec.VMTrace != nil { c.VMTrace = *dec.VMTrace } diff --git a/triedb/database.go b/triedb/database.go index 37a57ac0220..d2637bd909a 100644 --- a/triedb/database.go +++ b/triedb/database.go @@ -384,12 +384,3 @@ func (db *Database) SnapshotCompleted() bool { } return pdb.SnapshotCompleted() } - -// SetForceFlush enables or disables the pathdb to flush any pending changes to disk -// immediately, regardless of the buffer size threshold. This can be used to accelerate -// state sizer initialization by making buffered state changes visible on disk. -func (db *Database) SetForceFlush(enabled bool) { - if pdb, ok := db.backend.(*pathdb.Database); ok { - pdb.SetForceFlush(enabled) - } -} diff --git a/triedb/pathdb/database.go b/triedb/pathdb/database.go index 0ca5d8a73aa..0be875af8f3 100644 --- a/triedb/pathdb/database.go +++ b/triedb/pathdb/database.go @@ -136,9 +136,7 @@ type Database struct { stateFreezer ethdb.ResettableAncientStore // Freezer for storing state histories, nil possible in tests stateIndexer *historyIndexer // History indexer historical state data, nil possible - - lock sync.RWMutex // Lock to prevent mutations from happening at the same time - forceFlush bool // Flag to force buffer flush regardless of size + lock sync.RWMutex // Lock to prevent mutations from happening at the same time } // New attempts to load an already existing layer from a persistent key-value @@ -361,15 +359,6 @@ func (db *Database) Commit(root common.Hash, report bool) error { return db.tree.cap(root, 0) } -// SetForceFlush enables or disables force flushing for the next state update. -func (db *Database) SetForceFlush(enabled bool) { - db.lock.Lock() - defer db.lock.Unlock() - - db.forceFlush = enabled - log.Info("Set triedb force flush for next pathdb update", "enabled", enabled) -} - // Disable deactivates the database and invalidates all available state layers // as stale to prevent access to the persistent state, which is in the syncing // stage. diff --git a/triedb/pathdb/disklayer.go b/triedb/pathdb/disklayer.go index 5a531672039..2042e916110 100644 --- a/triedb/pathdb/disklayer.go +++ b/triedb/pathdb/disklayer.go @@ -417,7 +417,7 @@ func (dl *diskLayer) commit(bottom *diffLayer, force bool) (*diskLayer, error) { // Terminate the background state snapshot generation before mutating the // persistent state. - if combined.full() || force || flush || dl.db.forceFlush { + if combined.full() || force || flush { // Wait until the previous frozen buffer is fully flushed if dl.frozen != nil { if err := dl.frozen.waitFlush(); err != nil { From a5048f64de1ac8be142b5a98d8597a32a04f0e25 Mon Sep 17 00:00:00 2001 From: Gary Rong Date: Fri, 5 Sep 2025 14:38:41 +0800 Subject: [PATCH 43/43] core, eth, internal: polish --- core/blockchain.go | 1 + core/state/state_sizer.go | 104 ++++++++++++++++++++++-------------- eth/api_debug.go | 15 ++++-- internal/web3ext/web3ext.go | 3 +- 4 files changed, 79 insertions(+), 44 deletions(-) diff --git a/core/blockchain.go b/core/blockchain.go index 2d5527a114b..7b93a7606e3 100644 --- a/core/blockchain.go +++ b/core/blockchain.go @@ -536,6 +536,7 @@ func NewBlockChain(db ethdb.Database, genesis *Genesis, engine consensus.Engine, stateSizer, err := state.NewSizeTracker(bc.db, bc.triedb) if err == nil { bc.stateSizer = stateSizer + log.Info("Initialized state sizer") } else { log.Info("Failed to setup size tracker", "err", err) } diff --git a/core/state/state_sizer.go b/core/state/state_sizer.go index ede4b5b384f..2066c94845d 100644 --- a/core/state/state_sizer.go +++ b/core/state/state_sizer.go @@ -67,7 +67,7 @@ type SizeStats struct { } func (s SizeStats) String() string { - return fmt.Sprintf("Accounts: %d (%s), Storages: %d (%s), AccountTrienodes: %d (%s), StorageTrienodes: %d (%s), ContractCodes: %d (%s)", + return fmt.Sprintf("Accounts: %d(%s), Storages: %d(%s), AccountTrienodes: %d(%s), StorageTrienodes: %d(%s), Codes: %d(%s)", s.Accounts, common.StorageSize(s.AccountBytes), s.Storages, common.StorageSize(s.StorageBytes), s.AccountTrienodes, common.StorageSize(s.AccountTrienodeBytes), @@ -223,6 +223,12 @@ func calSizeStats(update *stateUpdate) (SizeStats, error) { return stats, nil } +type stateSizeQuery struct { + root *common.Hash // nil means latest + err error // non-nil if the state size is not yet initialized + result chan *SizeStats // nil means the state is unknown +} + // SizeTracker handles the state size initialization and tracks of state size metrics. type SizeTracker struct { db ethdb.KeyValueStore @@ -230,9 +236,7 @@ type SizeTracker struct { abort chan struct{} aborted chan struct{} updateCh chan *stateUpdate - - mu sync.RWMutex - latestStats *SizeStats + queryCh chan *stateSizeQuery } // NewSizeTracker creates a new state size tracker and starts it automatically @@ -246,6 +250,7 @@ func NewSizeTracker(db ethdb.KeyValueStore, triedb *triedb.Database) (*SizeTrack abort: make(chan struct{}), aborted: make(chan struct{}), updateCh: make(chan *stateUpdate), + queryCh: make(chan *stateSizeQuery), } go t.run() return t, nil @@ -280,6 +285,7 @@ func (h *sizeStatsHeap) Pop() any { func (t *SizeTracker) run() { defer close(t.aborted) + var last common.Hash stats, err := t.init() // launch background thread for state size init if err != nil { return @@ -301,18 +307,27 @@ func (t *SizeTracker) run() { } stat := base.add(diff) stats[u.root] = stat - log.Debug("Update state size", "number", stat.BlockNumber, "root", stat.StateRoot, "stat", stat) - - // Update latest stats - t.mu.Lock() - t.latestStats = &stat - t.mu.Unlock() + last = u.root heap.Push(&h, stats[u.root]) for u.blockNumber-h[0].BlockNumber > statEvictThreshold { delete(stats, h[0].StateRoot) heap.Pop(&h) } + log.Debug("Update state size", "number", stat.BlockNumber, "root", stat.StateRoot, "stat", stat) + + case r := <-t.queryCh: + var root common.Hash + if r.root != nil { + root = *r.root + } else { + root = last + } + if s, ok := stats[root]; ok { + r.result <- &s + } else { + r.result <- nil + } case <-t.abort: return @@ -342,6 +357,9 @@ wait: } case <-t.updateCh: continue + case r := <-t.queryCh: + r.err = errors.New("state size is not initialized yet") + r.result <- nil case <-t.abort: return nil, errors.New("size tracker closed") } @@ -358,6 +376,11 @@ wait: case u := <-t.updateCh: updates[u.root] = u children[u.originRoot] = append(children[u.originRoot], u.root) + log.Debug("Received state update", "root", u.root, "blockNumber", u.blockNumber) + + case r := <-t.queryCh: + r.err = errors.New("state size is not initialized yet") + r.result <- nil case <-ticker.C: // Only check timer if build hasn't started yet @@ -407,11 +430,6 @@ wait: // Set initial latest stats stats[result.root] = result.stat - - t.mu.Lock() - t.latestStats = &result.stat - t.mu.Unlock() - log.Info("Measured persistent state size", "root", result.root, "number", result.blockNumber, "stat", result.stat, "elapsed", common.PrettyDuration(result.elapsed)) return stats, nil @@ -508,20 +526,6 @@ func (t *SizeTracker) build(root common.Hash, blockNumber uint64, done chan buil } } -// Notify is an async method used to send the state update to the size tracker. -// It ignores empty updates (where no state changes occurred). -// If the channel is full, it drops the update to avoid blocking. -func (t *SizeTracker) Notify(update *stateUpdate) { - if update == nil || update.empty() { - return - } - select { - case t.updateCh <- update: - case <-t.abort: - return - } -} - // iterateTable performs iteration over a specific table and returns the results. func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name string) (int64, int64, error) { var ( @@ -543,10 +547,10 @@ func (t *SizeTracker) iterateTable(closed chan struct{}, prefix []byte, name str select { case <-closed: - log.Info("State iteration cancelled", "category", name) + log.Debug("State iteration cancelled", "category", name) return 0, 0, errors.New("size tracker closed") default: - log.Info("Iterating state", "category", name, "count", count, "size", common.StorageSize(bytes)) + log.Debug("Iterating state", "category", name, "count", count, "size", common.StorageSize(bytes)) } } } @@ -573,8 +577,7 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, mu sync.Mutex ) group.SetLimit(workers) - - log.Info("Starting parallel state iteration", "category", name, "workers", workers) + log.Debug("Starting parallel state iteration", "category", name, "workers", workers) if len(prefix) > 0 { if blob, err := t.db.Get(prefix); err == nil && len(blob) > 0 { @@ -583,7 +586,6 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, totalBytes = int64(len(prefix) + len(blob)) } } - for i := 0; i < 256; i++ { h := byte(i) group.Go(func() error { @@ -601,14 +603,36 @@ func (t *SizeTracker) iterateTableParallel(closed chan struct{}, prefix []byte, if err := group.Wait(); err != nil { return 0, 0, err } - log.Info("Finished parallel state iteration", "category", name, "count", totalCount, "size", common.StorageSize(totalBytes), "elapsed", common.PrettyDuration(time.Since(start))) + log.Debug("Finished parallel state iteration", "category", name, "count", totalCount, "size", common.StorageSize(totalBytes), "elapsed", common.PrettyDuration(time.Since(start))) return totalCount, totalBytes, nil } -// GetLatestStats returns the latest state size statistics, or nil if not available -func (t *SizeTracker) GetLatestStats() *SizeStats { - t.mu.RLock() - defer t.mu.RUnlock() +// Notify is an async method used to send the state update to the size tracker. +// It ignores empty updates (where no state changes occurred). +// If the channel is full, it drops the update to avoid blocking. +func (t *SizeTracker) Notify(update *stateUpdate) { + if update == nil || update.empty() { + return + } + select { + case t.updateCh <- update: + case <-t.abort: + return + } +} - return t.latestStats +// Query returns the state size specified by the root, or nil if not available. +// If the root is nil, query the size of latest chain head; +// If the root is non-nil, query the size of the specified state; +func (t *SizeTracker) Query(root *common.Hash) (*SizeStats, error) { + r := &stateSizeQuery{ + root: root, + result: make(chan *SizeStats, 1), + } + select { + case <-t.aborted: + return nil, errors.New("state sizer has been closed") + case t.queryCh <- r: + return <-r.result, r.err + } } diff --git a/eth/api_debug.go b/eth/api_debug.go index 703c584fbc9..1d476be9892 100644 --- a/eth/api_debug.go +++ b/eth/api_debug.go @@ -446,14 +446,23 @@ func (api *DebugAPI) GetTrieFlushInterval() (string, error) { // StateSize returns the current state size statistics from the state size tracker. // Returns an error if the state size tracker is not initialized or if stats are not ready. -func (api *DebugAPI) StateSize() (interface{}, error) { +func (api *DebugAPI) StateSize(root *common.Hash) (interface{}, error) { sizer := api.eth.blockchain.StateSizer() if sizer == nil { return nil, errors.New("state size tracker is not enabled") } - stats := sizer.GetLatestStats() + stats, err := sizer.Query(root) + if err != nil { + return nil, err + } if stats == nil { - return nil, errors.New("state size statistics are not ready yet") + var s string + if root == nil { + s = "latest" + } else { + s = root.Hex() + } + return nil, fmt.Errorf("state size %s is not available", s) } return map[string]interface{}{ "stateRoot": stats.StateRoot, diff --git a/internal/web3ext/web3ext.go b/internal/web3ext/web3ext.go index 81c30a1f971..fdbb0a3f38d 100644 --- a/internal/web3ext/web3ext.go +++ b/internal/web3ext/web3ext.go @@ -471,7 +471,8 @@ web3._extend({ new web3._extend.Method({ name: 'stateSize', call: 'debug_stateSize', - params: 0, + params: 1, + inputFormatter:[null], }), ], properties: []