all: disable recording preimage of trie keys (#21402)

* cmd, core, eth, light, trie: disable recording preimage by default

* core, eth: fix unit tests

* core: fix import

* all: change to nopreimage

* cmd, core, eth, trie: use cache.preimages flag

* cmd: enable preimages for archive node

* cmd/utils, trie: simplify preimage tracking a bit

* core: fix linter

Co-authored-by: Péter Szilágyi <peterke@gmail.com>
This commit is contained in:
gary rong 2020-11-18 17:51:33 +08:00 committed by GitHub
parent 6b9858085f
commit 23524f8900
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
15 changed files with 110 additions and 48 deletions

View File

@ -113,6 +113,7 @@ var (
utils.CacheGCFlag,
utils.CacheSnapshotFlag,
utils.CacheNoPrefetchFlag,
utils.CachePreimagesFlag,
utils.ListenPortFlag,
utils.MaxPeersFlag,
utils.MaxPendingPeersFlag,

View File

@ -114,6 +114,7 @@ var AppHelpFlagGroups = []flags.FlagGroup{
utils.CacheGCFlag,
utils.CacheSnapshotFlag,
utils.CacheNoPrefetchFlag,
utils.CachePreimagesFlag,
},
},
{

View File

@ -383,6 +383,10 @@ var (
Name: "cache.noprefetch",
Usage: "Disable heuristic state prefetch during block import (less CPU and disk IO, more time waiting for data)",
}
CachePreimagesFlag = cli.BoolTFlag{
Name: "cache.preimages",
Usage: "Enable recording the SHA3/keccak preimages of trie keys (default: true)",
}
// Miner settings
MiningEnabledFlag = cli.BoolFlag{
Name: "mine",
@ -1526,6 +1530,12 @@ func SetEthConfig(ctx *cli.Context, stack *node.Node, cfg *eth.Config) {
if ctx.GlobalIsSet(CacheNoPrefetchFlag.Name) {
cfg.NoPrefetch = ctx.GlobalBool(CacheNoPrefetchFlag.Name)
}
// Read the value from the flag no matter if it's set or not.
cfg.Preimages = ctx.GlobalBool(CachePreimagesFlag.Name)
if cfg.NoPruning && !cfg.Preimages {
cfg.Preimages = true
log.Info("Enabling recording of key preimages since archive mode is used")
}
if ctx.GlobalIsSet(TxLookupLimitFlag.Name) {
cfg.TxLookupLimit = ctx.GlobalUint64(TxLookupLimitFlag.Name)
}
@ -1835,6 +1845,11 @@ func MakeChain(ctx *cli.Context, stack *node.Node, readOnly bool) (chain *core.B
TrieDirtyDisabled: ctx.GlobalString(GCModeFlag.Name) == "archive",
TrieTimeLimit: eth.DefaultConfig.TrieTimeout,
SnapshotLimit: eth.DefaultConfig.SnapshotCache,
Preimages: ctx.GlobalBool(CachePreimagesFlag.Name),
}
if cache.TrieDirtyDisabled && !cache.Preimages {
cache.Preimages = true
log.Info("Enabling recording of key preimages since archive mode is used")
}
if !ctx.GlobalIsSet(SnapshotFlag.Name) {
cache.SnapshotLimit = 0 // Disabled

View File

@ -129,6 +129,7 @@ type CacheConfig struct {
TrieDirtyDisabled bool // Whether to disable trie write caching and GC altogether (archive node)
TrieTimeLimit time.Duration // Time limit after which to flush the current in-memory trie to disk
SnapshotLimit int // Memory allowance (MB) to use for caching snapshot entries in memory
Preimages bool // Whether to store preimage of trie key to the disk
SnapshotWait bool // Wait for snapshot construction on startup. TODO(karalabe): This is a dirty hack for testing, nuke it
}
@ -229,11 +230,15 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
badBlocks, _ := lru.New(badBlockLimit)
bc := &BlockChain{
chainConfig: chainConfig,
cacheConfig: cacheConfig,
db: db,
triegc: prque.New(nil),
stateCache: state.NewDatabaseWithCache(db, cacheConfig.TrieCleanLimit, cacheConfig.TrieCleanJournal),
chainConfig: chainConfig,
cacheConfig: cacheConfig,
db: db,
triegc: prque.New(nil),
stateCache: state.NewDatabaseWithConfig(db, &trie.Config{
Cache: cacheConfig.TrieCleanLimit,
Journal: cacheConfig.TrieCleanJournal,
Preimages: cacheConfig.Preimages,
}),
quit: make(chan struct{}),
shouldPreserve: shouldPreserve,
bodyCache: bodyCache,

View File

@ -175,7 +175,7 @@ func SetupGenesisBlock(db ethdb.Database, genesis *Genesis) (*params.ChainConfig
// We have the genesis block in database(perhaps in ancient database)
// but the corresponding state is missing.
header := rawdb.ReadHeader(db, stored, 0)
if _, err := state.New(header.Root, state.NewDatabaseWithCache(db, 0, ""), nil); err != nil {
if _, err := state.New(header.Root, state.NewDatabaseWithConfig(db, nil), nil); err != nil {
if genesis == nil {
genesis = DefaultGenesisBlock()
}

View File

@ -104,18 +104,18 @@ type Trie interface {
// NewDatabase creates a backing store for state. The returned database is safe for
// concurrent use, but does not retain any recent trie nodes in memory. To keep some
// historical state in memory, use the NewDatabaseWithCache constructor.
// historical state in memory, use the NewDatabaseWithConfig constructor.
func NewDatabase(db ethdb.Database) Database {
return NewDatabaseWithCache(db, 0, "")
return NewDatabaseWithConfig(db, nil)
}
// NewDatabaseWithCache creates a backing store for state. The returned database
// NewDatabaseWithConfig creates a backing store for state. The returned database
// is safe for concurrent use and retains a lot of collapsed RLP trie nodes in a
// large memory cache.
func NewDatabaseWithCache(db ethdb.Database, cache int, journal string) Database {
func NewDatabaseWithConfig(db ethdb.Database, config *trie.Config) Database {
csc, _ := lru.New(codeSizeCacheSize)
return &cachingDB{
db: trie.NewDatabaseWithCache(db, cache, journal),
db: trie.NewDatabaseWithConfig(db, config),
codeSizeCache: csc,
codeCache: fastcache.New(codeCacheSize),
}

View File

@ -41,7 +41,9 @@ func newStateTest() *stateTest {
}
func TestDump(t *testing.T) {
s := newStateTest()
db := rawdb.NewMemoryDatabase()
sdb, _ := New(common.Hash{}, NewDatabaseWithConfig(db, nil), nil)
s := &stateTest{db: db, state: sdb}
// generate a few entries
obj1 := s.state.GetOrNewStateObject(toAddr([]byte{0x01}))

View File

@ -58,7 +58,7 @@ func (h resultHash) Less(i, j int) bool { return bytes.Compare(h[i].Bytes(), h[j
func TestAccountRange(t *testing.T) {
var (
statedb = state.NewDatabase(rawdb.NewMemoryDatabase())
statedb = state.NewDatabaseWithConfig(rawdb.NewMemoryDatabase(), nil)
state, _ = state.New(common.Hash{}, statedb, nil)
addrs = [AccountRangeMaxResults * 2]common.Address{}
m = map[common.Address]bool{}

View File

@ -148,7 +148,7 @@ func (api *PrivateDebugAPI) traceChain(ctx context.Context, start, end *types.Bl
// Ensure we have a valid starting state before doing any work
origin := start.NumberU64()
database := state.NewDatabaseWithCache(api.eth.ChainDb(), 16, "") // Chain tracing will probably start at genesis
database := state.NewDatabaseWithConfig(api.eth.ChainDb(), &trie.Config{Cache: 16, Preimages: true})
if number := start.NumberU64(); number > 0 {
start = api.eth.blockchain.GetBlock(start.ParentHash(), start.NumberU64()-1)
@ -659,7 +659,7 @@ func (api *PrivateDebugAPI) computeStateDB(block *types.Block, reexec uint64) (*
}
// Otherwise try to reexec blocks until we find a state or reach our limit
origin := block.NumberU64()
database := state.NewDatabaseWithCache(api.eth.ChainDb(), 16, "")
database := state.NewDatabaseWithConfig(api.eth.ChainDb(), &trie.Config{Cache: 16, Preimages: true})
for i := uint64(0); i < reexec; i++ {
block = api.eth.blockchain.GetBlock(block.ParentHash(), block.NumberU64()-1)

View File

@ -169,6 +169,7 @@ func New(stack *node.Node, config *Config) (*Ethereum, error) {
TrieDirtyDisabled: config.NoPruning,
TrieTimeLimit: config.TrieTimeout,
SnapshotLimit: config.SnapshotCache,
Preimages: config.Preimages,
}
)
eth.blockchain, err = core.NewBlockChain(chainDb, cacheConfig, chainConfig, eth.engine, vmConfig, eth.shouldPreserve, &config.TxLookupLimit)

View File

@ -149,6 +149,7 @@ type Config struct {
TrieDirtyCache int
TrieTimeout time.Duration
SnapshotCache int
Preimages bool
// Mining options
Miner miner.Config

View File

@ -43,6 +43,7 @@ func (c Config) MarshalTOML() (interface{}, error) {
TrieDirtyCache int
TrieTimeout time.Duration
SnapshotCache int
Preimages bool
Miner miner.Config
Ethash ethash.Config
TxPool core.TxPoolConfig
@ -83,6 +84,7 @@ func (c Config) MarshalTOML() (interface{}, error) {
enc.TrieDirtyCache = c.TrieDirtyCache
enc.TrieTimeout = c.TrieTimeout
enc.SnapshotCache = c.SnapshotCache
enc.Preimages = c.Preimages
enc.Miner = c.Miner
enc.Ethash = c.Ethash
enc.TxPool = c.TxPool
@ -127,6 +129,7 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
TrieDirtyCache *int
TrieTimeout *time.Duration
SnapshotCache *int
Preimages *bool
Miner *miner.Config
Ethash *ethash.Config
TxPool *core.TxPoolConfig
@ -222,6 +225,9 @@ func (c *Config) UnmarshalTOML(unmarshal func(interface{}) error) error {
if dec.SnapshotCache != nil {
c.SnapshotCache = *dec.SnapshotCache
}
if dec.Preimages != nil {
c.Preimages = *dec.Preimages
}
if dec.Miner != nil {
c.Miner = *dec.Miner
}

View File

@ -147,7 +147,7 @@ func NewChtIndexer(db ethdb.Database, odr OdrBackend, size, confirms uint64, dis
diskdb: db,
odr: odr,
trieTable: trieTable,
triedb: trie.NewDatabaseWithCache(trieTable, 1, ""), // Use a tiny cache only to keep memory down
triedb: trie.NewDatabaseWithConfig(trieTable, &trie.Config{Cache: 1}), // Use a tiny cache only to keep memory down
trieset: mapset.NewSet(),
sectionSize: size,
disablePruning: disablePruning,
@ -340,7 +340,7 @@ func NewBloomTrieIndexer(db ethdb.Database, odr OdrBackend, parentSize, size uin
diskdb: db,
odr: odr,
trieTable: trieTable,
triedb: trie.NewDatabaseWithCache(trieTable, 1, ""), // Use a tiny cache only to keep memory down
triedb: trie.NewDatabaseWithConfig(trieTable, &trie.Config{Cache: 1}), // Use a tiny cache only to keep memory down
trieset: mapset.NewSet(),
parentSize: parentSize,
size: size,

View File

@ -272,33 +272,43 @@ func expandNode(hash hashNode, n node) node {
}
}
// Config defines all necessary options for database.
type Config struct {
Cache int // Memory allowance (MB) to use for caching trie nodes in memory
Journal string // Journal of clean cache to survive node restarts
Preimages bool // Flag whether the preimage of trie key is recorded
}
// NewDatabase creates a new trie database to store ephemeral trie content before
// its written out to disk or garbage collected. No read cache is created, so all
// data retrievals will hit the underlying disk database.
func NewDatabase(diskdb ethdb.KeyValueStore) *Database {
return NewDatabaseWithCache(diskdb, 0, "")
return NewDatabaseWithConfig(diskdb, nil)
}
// NewDatabaseWithCache creates a new trie database to store ephemeral trie content
// NewDatabaseWithConfig creates a new trie database to store ephemeral trie content
// before its written out to disk or garbage collected. It also acts as a read cache
// for nodes loaded from disk.
func NewDatabaseWithCache(diskdb ethdb.KeyValueStore, cache int, journal string) *Database {
func NewDatabaseWithConfig(diskdb ethdb.KeyValueStore, config *Config) *Database {
var cleans *fastcache.Cache
if cache > 0 {
if journal == "" {
cleans = fastcache.New(cache * 1024 * 1024)
if config != nil && config.Cache > 0 {
if config.Journal == "" {
cleans = fastcache.New(config.Cache * 1024 * 1024)
} else {
cleans = fastcache.LoadFromFileOrNew(journal, cache*1024*1024)
cleans = fastcache.LoadFromFileOrNew(config.Journal, config.Cache*1024*1024)
}
}
return &Database{
db := &Database{
diskdb: diskdb,
cleans: cleans,
dirties: map[common.Hash]*cachedNode{{}: {
children: make(map[common.Hash]uint16),
}},
preimages: make(map[common.Hash][]byte),
}
if config == nil || config.Preimages { // TODO(karalabe): Flip to default off in the future
db.preimages = make(map[common.Hash][]byte)
}
return db
}
// DiskDB retrieves the persistent storage backing the trie database.
@ -345,6 +355,11 @@ func (db *Database) insert(hash common.Hash, size int, node node) {
//
// Note, this method assumes that the database's lock is held!
func (db *Database) insertPreimage(hash common.Hash, preimage []byte) {
// Short circuit if preimage collection is disabled
if db.preimages == nil {
return
}
// Track the preimage if a yet unknown one
if _, ok := db.preimages[hash]; ok {
return
}
@ -431,6 +446,10 @@ func (db *Database) Node(hash common.Hash) ([]byte, error) {
// preimage retrieves a cached trie node pre-image from memory. If it cannot be
// found cached, the method queries the persistent database for the content.
func (db *Database) preimage(hash common.Hash) []byte {
// Short circuit if preimage collection is disabled
if db.preimages == nil {
return nil
}
// Retrieve the node from cache if available
db.lock.RLock()
preimage := db.preimages[hash]
@ -588,12 +607,16 @@ func (db *Database) Cap(limit common.StorageSize) error {
// leave for later to deduplicate writes.
flushPreimages := db.preimagesSize > 4*1024*1024
if flushPreimages {
rawdb.WritePreimages(batch, db.preimages)
if batch.ValueSize() > ethdb.IdealBatchSize {
if err := batch.Write(); err != nil {
return err
if db.preimages == nil {
log.Error("Attempted to write preimages whilst disabled")
} else {
rawdb.WritePreimages(batch, db.preimages)
if batch.ValueSize() > ethdb.IdealBatchSize {
if err := batch.Write(); err != nil {
return err
}
batch.Reset()
}
batch.Reset()
}
}
// Keep committing nodes from the flush-list until we're below allowance
@ -630,7 +653,11 @@ func (db *Database) Cap(limit common.StorageSize) error {
defer db.lock.Unlock()
if flushPreimages {
db.preimages, db.preimagesSize = make(map[common.Hash][]byte), 0
if db.preimages == nil {
log.Error("Attempted to reset preimage cache whilst disabled")
} else {
db.preimages, db.preimagesSize = make(map[common.Hash][]byte), 0
}
}
for db.oldest != oldest {
node := db.dirties[db.oldest]
@ -674,20 +701,21 @@ func (db *Database) Commit(node common.Hash, report bool, callback func(common.H
batch := db.diskdb.NewBatch()
// Move all of the accumulated preimages into a write batch
rawdb.WritePreimages(batch, db.preimages)
if batch.ValueSize() > ethdb.IdealBatchSize {
if db.preimages != nil {
rawdb.WritePreimages(batch, db.preimages)
if batch.ValueSize() > ethdb.IdealBatchSize {
if err := batch.Write(); err != nil {
return err
}
batch.Reset()
}
// Since we're going to replay trie node writes into the clean cache, flush out
// any batched pre-images before continuing.
if err := batch.Write(); err != nil {
return err
}
batch.Reset()
}
// Since we're going to replay trie node writes into the clean cache, flush out
// any batched pre-images before continuing.
if err := batch.Write(); err != nil {
return err
}
batch.Reset()
// Move the trie itself into the batch, flushing if enough data is accumulated
nodes, storage := len(db.dirties), db.dirtiesSize
@ -709,8 +737,9 @@ func (db *Database) Commit(node common.Hash, report bool, callback func(common.H
batch.Reset()
// Reset the storage counters and bumpd metrics
db.preimages, db.preimagesSize = make(map[common.Hash][]byte), 0
if db.preimages != nil {
db.preimages, db.preimagesSize = make(map[common.Hash][]byte), 0
}
memcacheCommitTimeTimer.Update(time.Since(start))
memcacheCommitSizeMeter.Mark(int64(storage - db.dirtiesSize))
memcacheCommitNodesMeter.Mark(int64(nodes - len(db.dirties)))

View File

@ -147,12 +147,13 @@ func (t *SecureTrie) GetKey(shaKey []byte) []byte {
func (t *SecureTrie) Commit(onleaf LeafCallback) (root common.Hash, err error) {
// Write all the pre-images to the actual disk database
if len(t.getSecKeyCache()) > 0 {
t.trie.db.lock.Lock()
for hk, key := range t.secKeyCache {
t.trie.db.insertPreimage(common.BytesToHash([]byte(hk)), key)
if t.trie.db.preimages != nil { // Ugly direct check but avoids the below write lock
t.trie.db.lock.Lock()
for hk, key := range t.secKeyCache {
t.trie.db.insertPreimage(common.BytesToHash([]byte(hk)), key)
}
t.trie.db.lock.Unlock()
}
t.trie.db.lock.Unlock()
t.secKeyCache = make(map[string][]byte)
}
// Commit the trie to its intermediate node database