core: improve snapshot journal recovery (#21594)
* core/state/snapshot: introduce snapshot journal version * core: update the disk layer in an atomic way * core: persist the disk layer generator periodically * core/state/snapshot: improve logging * core/state/snapshot: forcibly ensure the legacy snapshot is matched * core/state/snapshot: add debug logs * core, tests: fix tests and special recovery case * core: polish * core: add more blockchain tests for snapshot recovery * core/state: fix comment * core: add recovery flag for snapshot * core: add restart after start-after-crash tests * core/rawdb: fix imports * core: fix tests * core: remove log * core/state/snapshot: fix snapshot * core: avoid callbacks in SetHead * core: fix setHead cornercase where the threshold root has state * core: small docs for the test cases Co-authored-by: Péter Szilágyi <peterke@gmail.com>
This commit is contained in:
@ -28,6 +28,7 @@ import (
|
||||
"github.com/ethereum/go-ethereum/ethdb"
|
||||
"github.com/ethereum/go-ethereum/ethdb/leveldb"
|
||||
"github.com/ethereum/go-ethereum/ethdb/memorydb"
|
||||
"github.com/ethereum/go-ethereum/rlp"
|
||||
)
|
||||
|
||||
// reverse reverses the contents of a byte slice. It's used to update random accs
|
||||
@ -429,6 +430,81 @@ func TestDiskPartialMerge(t *testing.T) {
|
||||
}
|
||||
}
|
||||
|
||||
// Tests that when the bottom-most diff layer is merged into the disk
|
||||
// layer whether the corresponding generator is persisted correctly.
|
||||
func TestDiskGeneratorPersistence(t *testing.T) {
|
||||
var (
|
||||
accOne = randomHash()
|
||||
accTwo = randomHash()
|
||||
accOneSlotOne = randomHash()
|
||||
accOneSlotTwo = randomHash()
|
||||
|
||||
accThree = randomHash()
|
||||
accThreeSlot = randomHash()
|
||||
baseRoot = randomHash()
|
||||
diffRoot = randomHash()
|
||||
diffTwoRoot = randomHash()
|
||||
genMarker = append(randomHash().Bytes(), randomHash().Bytes()...)
|
||||
)
|
||||
// Testing scenario 1, the disk layer is still under the construction.
|
||||
db := rawdb.NewMemoryDatabase()
|
||||
|
||||
rawdb.WriteAccountSnapshot(db, accOne, accOne[:])
|
||||
rawdb.WriteStorageSnapshot(db, accOne, accOneSlotOne, accOneSlotOne[:])
|
||||
rawdb.WriteStorageSnapshot(db, accOne, accOneSlotTwo, accOneSlotTwo[:])
|
||||
rawdb.WriteSnapshotRoot(db, baseRoot)
|
||||
|
||||
// Create a disk layer based on all above updates
|
||||
snaps := &Tree{
|
||||
layers: map[common.Hash]snapshot{
|
||||
baseRoot: &diskLayer{
|
||||
diskdb: db,
|
||||
cache: fastcache.New(500 * 1024),
|
||||
root: baseRoot,
|
||||
genMarker: genMarker,
|
||||
},
|
||||
},
|
||||
}
|
||||
// Modify or delete some accounts, flatten everything onto disk
|
||||
if err := snaps.Update(diffRoot, baseRoot, nil, map[common.Hash][]byte{
|
||||
accTwo: accTwo[:],
|
||||
}, nil); err != nil {
|
||||
t.Fatalf("failed to update snapshot tree: %v", err)
|
||||
}
|
||||
if err := snaps.Cap(diffRoot, 0); err != nil {
|
||||
t.Fatalf("failed to flatten snapshot tree: %v", err)
|
||||
}
|
||||
blob := rawdb.ReadSnapshotGenerator(db)
|
||||
var generator journalGenerator
|
||||
if err := rlp.DecodeBytes(blob, &generator); err != nil {
|
||||
t.Fatalf("Failed to decode snapshot generator %v", err)
|
||||
}
|
||||
if !bytes.Equal(generator.Marker, genMarker) {
|
||||
t.Fatalf("Generator marker is not matched")
|
||||
}
|
||||
// Test senario 2, the disk layer is fully generated
|
||||
// Modify or delete some accounts, flatten everything onto disk
|
||||
if err := snaps.Update(diffTwoRoot, diffRoot, nil, map[common.Hash][]byte{
|
||||
accThree: accThree.Bytes(),
|
||||
}, map[common.Hash]map[common.Hash][]byte{
|
||||
accThree: {accThreeSlot: accThreeSlot.Bytes()},
|
||||
}); err != nil {
|
||||
t.Fatalf("failed to update snapshot tree: %v", err)
|
||||
}
|
||||
diskLayer := snaps.layers[snaps.diskRoot()].(*diskLayer)
|
||||
diskLayer.genMarker = nil // Construction finished
|
||||
if err := snaps.Cap(diffTwoRoot, 0); err != nil {
|
||||
t.Fatalf("failed to flatten snapshot tree: %v", err)
|
||||
}
|
||||
blob = rawdb.ReadSnapshotGenerator(db)
|
||||
if err := rlp.DecodeBytes(blob, &generator); err != nil {
|
||||
t.Fatalf("Failed to decode snapshot generator %v", err)
|
||||
}
|
||||
if len(generator.Marker) != 0 {
|
||||
t.Fatalf("Failed to update snapshot generator")
|
||||
}
|
||||
}
|
||||
|
||||
// Tests that merging something into a disk layer persists it into the database
|
||||
// and invalidates any previously written and cached values, discarding anything
|
||||
// after the in-progress generation marker.
|
||||
|
@ -112,6 +112,7 @@ func generateSnapshot(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache i
|
||||
genAbort: make(chan chan *generatorStats),
|
||||
}
|
||||
go base.generate(&generatorStats{wiping: wiper, start: time.Now()})
|
||||
log.Debug("Start snapshot generation", "root", root)
|
||||
return base
|
||||
}
|
||||
|
||||
|
@ -33,6 +33,8 @@ import (
|
||||
"github.com/ethereum/go-ethereum/trie"
|
||||
)
|
||||
|
||||
const journalVersion uint64 = 0
|
||||
|
||||
// journalGenerator is a disk layer entry containing the generator progress marker.
|
||||
type journalGenerator struct {
|
||||
Wiping bool // Whether the database was in progress of being wiped
|
||||
@ -61,8 +63,87 @@ type journalStorage struct {
|
||||
Vals [][]byte
|
||||
}
|
||||
|
||||
// loadAndParseLegacyJournal tries to parse the snapshot journal in legacy format.
|
||||
func loadAndParseLegacyJournal(db ethdb.KeyValueStore, base *diskLayer) (snapshot, journalGenerator, error) {
|
||||
// Retrieve the journal, for legacy journal it must exist since even for
|
||||
// 0 layer it stores whether we've already generated the snapshot or are
|
||||
// in progress only.
|
||||
journal := rawdb.ReadSnapshotJournal(db)
|
||||
if len(journal) == 0 {
|
||||
return nil, journalGenerator{}, errors.New("missing or corrupted snapshot journal")
|
||||
}
|
||||
r := rlp.NewStream(bytes.NewReader(journal), 0)
|
||||
|
||||
// Read the snapshot generation progress for the disk layer
|
||||
var generator journalGenerator
|
||||
if err := r.Decode(&generator); err != nil {
|
||||
return nil, journalGenerator{}, fmt.Errorf("failed to load snapshot progress marker: %v", err)
|
||||
}
|
||||
// Load all the snapshot diffs from the journal
|
||||
snapshot, err := loadDiffLayer(base, r)
|
||||
if err != nil {
|
||||
return nil, generator, err
|
||||
}
|
||||
return snapshot, generator, nil
|
||||
}
|
||||
|
||||
// loadAndParseJournal tries to parse the snapshot journal in latest format.
|
||||
func loadAndParseJournal(db ethdb.KeyValueStore, base *diskLayer) (snapshot, journalGenerator, error) {
|
||||
// Retrieve the disk layer generator. It must exist, no matter the
|
||||
// snapshot is fully generated or not. Otherwise the entire disk
|
||||
// layer is invalid.
|
||||
generatorBlob := rawdb.ReadSnapshotGenerator(db)
|
||||
if len(generatorBlob) == 0 {
|
||||
return nil, journalGenerator{}, errors.New("missing snapshot generator")
|
||||
}
|
||||
var generator journalGenerator
|
||||
if err := rlp.DecodeBytes(generatorBlob, &generator); err != nil {
|
||||
return nil, journalGenerator{}, fmt.Errorf("failed to decode snapshot generator: %v", err)
|
||||
}
|
||||
// Retrieve the diff layer journal. It's possible that the journal is
|
||||
// not existent, e.g. the disk layer is generating while that the Geth
|
||||
// crashes without persisting the diff journal.
|
||||
// So if there is no journal, or the journal is not matched with disk
|
||||
// layer, we just discard all diffs and try to recover them later.
|
||||
journal := rawdb.ReadSnapshotJournal(db)
|
||||
if len(journal) == 0 {
|
||||
log.Warn("Loaded snapshot journal", "diskroot", base.root, "diffs", "missing")
|
||||
return base, generator, nil
|
||||
}
|
||||
r := rlp.NewStream(bytes.NewReader(journal), 0)
|
||||
|
||||
// Firstly, resolve the first element as the journal version
|
||||
version, err := r.Uint()
|
||||
if err != nil {
|
||||
return nil, journalGenerator{}, err
|
||||
}
|
||||
if version != journalVersion {
|
||||
return nil, journalGenerator{}, fmt.Errorf("journal version mismatch, want %d got %v", journalVersion, version)
|
||||
}
|
||||
// Secondly, resolve the disk layer root, ensure it's continuous
|
||||
// with disk layer.
|
||||
var root common.Hash
|
||||
if err := r.Decode(&root); err != nil {
|
||||
return nil, journalGenerator{}, errors.New("missing disk layer root")
|
||||
}
|
||||
// The diff journal is not matched with disk, discard them.
|
||||
// It can happen that Geth crashes without persisting the latest
|
||||
// diff journal.
|
||||
if !bytes.Equal(root.Bytes(), base.root.Bytes()) {
|
||||
log.Warn("Loaded snapshot journal", "diskroot", base.root, "diffs", "unmatched")
|
||||
return base, generator, nil
|
||||
}
|
||||
// Load all the snapshot diffs from the journal
|
||||
snapshot, err := loadDiffLayer(base, r)
|
||||
if err != nil {
|
||||
return nil, journalGenerator{}, err
|
||||
}
|
||||
log.Debug("Loaded snapshot journal", "diskroot", base.root, "diffhead", snapshot.Root())
|
||||
return snapshot, generator, nil
|
||||
}
|
||||
|
||||
// loadSnapshot loads a pre-existing state snapshot backed by a key-value store.
|
||||
func loadSnapshot(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root common.Hash) (snapshot, error) {
|
||||
func loadSnapshot(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root common.Hash, recovery bool) (snapshot, error) {
|
||||
// Retrieve the block number and hash of the snapshot, failing if no snapshot
|
||||
// is present in the database (or crashed mid-update).
|
||||
baseRoot := rawdb.ReadSnapshotRoot(diskdb)
|
||||
@ -75,28 +156,36 @@ func loadSnapshot(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int,
|
||||
cache: fastcache.New(cache * 1024 * 1024),
|
||||
root: baseRoot,
|
||||
}
|
||||
// Retrieve the journal, it must exist since even for 0 layer it stores whether
|
||||
// we've already generated the snapshot or are in progress only
|
||||
journal := rawdb.ReadSnapshotJournal(diskdb)
|
||||
if len(journal) == 0 {
|
||||
return nil, errors.New("missing or corrupted snapshot journal")
|
||||
var legacy bool
|
||||
snapshot, generator, err := loadAndParseJournal(diskdb, base)
|
||||
if err != nil {
|
||||
log.Debug("Failed to load new-format journal", "error", err)
|
||||
snapshot, generator, err = loadAndParseLegacyJournal(diskdb, base)
|
||||
legacy = true
|
||||
}
|
||||
r := rlp.NewStream(bytes.NewReader(journal), 0)
|
||||
|
||||
// Read the snapshot generation progress for the disk layer
|
||||
var generator journalGenerator
|
||||
if err := r.Decode(&generator); err != nil {
|
||||
return nil, fmt.Errorf("failed to load snapshot progress marker: %v", err)
|
||||
}
|
||||
// Load all the snapshot diffs from the journal
|
||||
snapshot, err := loadDiffLayer(base, r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
// Entire snapshot journal loaded, sanity check the head and return
|
||||
// Journal doesn't exist, don't worry if it's not supposed to
|
||||
// Entire snapshot journal loaded, sanity check the head. If the loaded
|
||||
// snapshot is not matched with current state root, print a warning log
|
||||
// or discard the entire snapshot it's legacy snapshot.
|
||||
//
|
||||
// Possible scenario: Geth was crashed without persisting journal and then
|
||||
// restart, the head is rewound to the point with available state(trie)
|
||||
// which is below the snapshot. In this case the snapshot can be recovered
|
||||
// by re-executing blocks but right now it's unavailable.
|
||||
if head := snapshot.Root(); head != root {
|
||||
return nil, fmt.Errorf("head doesn't match snapshot: have %#x, want %#x", head, root)
|
||||
// If it's legacy snapshot, or it's new-format snapshot but
|
||||
// it's not in recovery mode, returns the error here for
|
||||
// rebuilding the entire snapshot forcibly.
|
||||
if legacy || !recovery {
|
||||
return nil, fmt.Errorf("head doesn't match snapshot: have %#x, want %#x", head, root)
|
||||
}
|
||||
// It's in snapshot recovery, the assumption is held that
|
||||
// the disk layer is always higher than chain head. It can
|
||||
// be eventually recovered when the chain head beyonds the
|
||||
// disk layer.
|
||||
log.Warn("Snapshot is not continuous with chain", "snaproot", head, "chainroot", root)
|
||||
}
|
||||
// Everything loaded correctly, resume any suspended operations
|
||||
if !generator.Done {
|
||||
@ -203,7 +292,9 @@ func (dl *diskLayer) Journal(buffer *bytes.Buffer) (common.Hash, error) {
|
||||
if dl.stale {
|
||||
return common.Hash{}, ErrSnapshotStale
|
||||
}
|
||||
// Write out the generator marker
|
||||
// Write out the generator marker. Note it's a standalone disk layer generator
|
||||
// which is not mixed with journal. It's ok if the generator is persisted while
|
||||
// journal is not.
|
||||
entry := journalGenerator{
|
||||
Done: dl.genMarker == nil,
|
||||
Marker: dl.genMarker,
|
||||
@ -214,9 +305,12 @@ func (dl *diskLayer) Journal(buffer *bytes.Buffer) (common.Hash, error) {
|
||||
entry.Slots = stats.slots
|
||||
entry.Storage = uint64(stats.storage)
|
||||
}
|
||||
if err := rlp.Encode(buffer, entry); err != nil {
|
||||
blob, err := rlp.EncodeToBytes(entry)
|
||||
if err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
log.Debug("Journalled disk layer", "root", dl.root, "complete", dl.genMarker == nil)
|
||||
rawdb.WriteSnapshotGenerator(dl.diskdb, blob)
|
||||
return dl.root, nil
|
||||
}
|
||||
|
||||
@ -266,5 +360,97 @@ func (dl *diffLayer) Journal(buffer *bytes.Buffer) (common.Hash, error) {
|
||||
if err := rlp.Encode(buffer, storage); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
log.Debug("Journalled diff layer", "root", dl.root, "parent", dl.parent.Root())
|
||||
return base, nil
|
||||
}
|
||||
|
||||
// LegacyJournal writes the persistent layer generator stats into a buffer
|
||||
// to be stored in the database as the snapshot journal.
|
||||
//
|
||||
// Note it's the legacy version which is only used in testing right now.
|
||||
func (dl *diskLayer) LegacyJournal(buffer *bytes.Buffer) (common.Hash, error) {
|
||||
// If the snapshot is currently being generated, abort it
|
||||
var stats *generatorStats
|
||||
if dl.genAbort != nil {
|
||||
abort := make(chan *generatorStats)
|
||||
dl.genAbort <- abort
|
||||
|
||||
if stats = <-abort; stats != nil {
|
||||
stats.Log("Journalling in-progress snapshot", dl.root, dl.genMarker)
|
||||
}
|
||||
}
|
||||
// Ensure the layer didn't get stale
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if dl.stale {
|
||||
return common.Hash{}, ErrSnapshotStale
|
||||
}
|
||||
// Write out the generator marker
|
||||
entry := journalGenerator{
|
||||
Done: dl.genMarker == nil,
|
||||
Marker: dl.genMarker,
|
||||
}
|
||||
if stats != nil {
|
||||
entry.Wiping = (stats.wiping != nil)
|
||||
entry.Accounts = stats.accounts
|
||||
entry.Slots = stats.slots
|
||||
entry.Storage = uint64(stats.storage)
|
||||
}
|
||||
if err := rlp.Encode(buffer, entry); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
return dl.root, nil
|
||||
}
|
||||
|
||||
// Journal writes the memory layer contents into a buffer to be stored in the
|
||||
// database as the snapshot journal.
|
||||
//
|
||||
// Note it's the legacy version which is only used in testing right now.
|
||||
func (dl *diffLayer) LegacyJournal(buffer *bytes.Buffer) (common.Hash, error) {
|
||||
// Journal the parent first
|
||||
base, err := dl.parent.LegacyJournal(buffer)
|
||||
if err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
// Ensure the layer didn't get stale
|
||||
dl.lock.RLock()
|
||||
defer dl.lock.RUnlock()
|
||||
|
||||
if dl.Stale() {
|
||||
return common.Hash{}, ErrSnapshotStale
|
||||
}
|
||||
// Everything below was journalled, persist this layer too
|
||||
if err := rlp.Encode(buffer, dl.root); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
destructs := make([]journalDestruct, 0, len(dl.destructSet))
|
||||
for hash := range dl.destructSet {
|
||||
destructs = append(destructs, journalDestruct{Hash: hash})
|
||||
}
|
||||
if err := rlp.Encode(buffer, destructs); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
accounts := make([]journalAccount, 0, len(dl.accountData))
|
||||
for hash, blob := range dl.accountData {
|
||||
accounts = append(accounts, journalAccount{Hash: hash, Blob: blob})
|
||||
}
|
||||
if err := rlp.Encode(buffer, accounts); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
storage := make([]journalStorage, 0, len(dl.storageData))
|
||||
for hash, slots := range dl.storageData {
|
||||
keys := make([]common.Hash, 0, len(slots))
|
||||
vals := make([][]byte, 0, len(slots))
|
||||
for key, val := range slots {
|
||||
keys = append(keys, key)
|
||||
vals = append(vals, val)
|
||||
}
|
||||
storage = append(storage, journalStorage{Hash: hash, Keys: keys, Vals: vals})
|
||||
}
|
||||
if err := rlp.Encode(buffer, storage); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
log.Debug("Journalled diff layer", "root", dl.root, "parent", dl.parent.Root())
|
||||
return base, nil
|
||||
}
|
||||
|
@ -29,6 +29,7 @@ import (
|
||||
"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/trie"
|
||||
)
|
||||
|
||||
@ -136,6 +137,10 @@ type snapshot interface {
|
||||
// flattening everything down (bad for reorgs).
|
||||
Journal(buffer *bytes.Buffer) (common.Hash, error)
|
||||
|
||||
// LegacyJournal is basically identical to Journal. it's the legacy version for
|
||||
// flushing legacy journal. Now the only purpose of this function is for testing.
|
||||
LegacyJournal(buffer *bytes.Buffer) (common.Hash, error)
|
||||
|
||||
// Stale return whether this layer has become stale (was flattened across) or
|
||||
// if it's still live.
|
||||
Stale() bool
|
||||
@ -168,10 +173,12 @@ type Tree struct {
|
||||
// store (with a number of memory layers from a journal), ensuring that the head
|
||||
// of the snapshot matches the expected one.
|
||||
//
|
||||
// If the snapshot is missing or inconsistent, the entirety is deleted and will
|
||||
// be reconstructed from scratch based on the tries in the key-value store, on a
|
||||
// background thread.
|
||||
func New(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root common.Hash, async bool) *Tree {
|
||||
// If the snapshot is missing or the disk layer is broken, the entire is deleted
|
||||
// and will be reconstructed from scratch based on the tries in the key-value
|
||||
// store, on a background thread. If the memory layers from the journal is not
|
||||
// continuous with disk layer or the journal is missing, all diffs will be discarded
|
||||
// iff it's in "recovery" mode, otherwise rebuild is mandatory.
|
||||
func New(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root common.Hash, async bool, recovery bool) *Tree {
|
||||
// Create a new, empty snapshot tree
|
||||
snap := &Tree{
|
||||
diskdb: diskdb,
|
||||
@ -183,7 +190,7 @@ func New(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root comm
|
||||
defer snap.waitBuild()
|
||||
}
|
||||
// Attempt to load a previously persisted snapshot and rebuild one if failed
|
||||
head, err := loadSnapshot(diskdb, triedb, cache, root)
|
||||
head, err := loadSnapshot(diskdb, triedb, cache, root, recovery)
|
||||
if err != nil {
|
||||
log.Warn("Failed to load snapshot, regenerating", "err", err)
|
||||
snap.Rebuild(root)
|
||||
@ -198,7 +205,7 @@ func New(diskdb ethdb.KeyValueStore, triedb *trie.Database, cache int, root comm
|
||||
}
|
||||
|
||||
// waitBuild blocks until the snapshot finishes rebuilding. This method is meant
|
||||
// to be used by tests to ensure we're testing what we believe we are.
|
||||
// to be used by tests to ensure we're testing what we believe we are.
|
||||
func (t *Tree) waitBuild() {
|
||||
// Find the rebuild termination channel
|
||||
var done chan struct{}
|
||||
@ -415,6 +422,9 @@ func (t *Tree) cap(diff *diffLayer, layers int) *diskLayer {
|
||||
|
||||
// diffToDisk merges a bottom-most diff into the persistent disk layer underneath
|
||||
// it. The method will panic if called onto a non-bottom-most diff layer.
|
||||
//
|
||||
// The disk layer persistence should be operated in an atomic way. All updates should
|
||||
// be discarded if the whole transition if not finished.
|
||||
func diffToDisk(bottom *diffLayer) *diskLayer {
|
||||
var (
|
||||
base = bottom.parent.(*diskLayer)
|
||||
@ -427,8 +437,7 @@ func diffToDisk(bottom *diffLayer) *diskLayer {
|
||||
base.genAbort <- abort
|
||||
stats = <-abort
|
||||
}
|
||||
// Start by temporarily deleting the current snapshot block marker. This
|
||||
// ensures that in the case of a crash, the entire snapshot is invalidated.
|
||||
// Put the deletion in the batch writer, flush all updates in the final step.
|
||||
rawdb.DeleteSnapshotRoot(batch)
|
||||
|
||||
// Mark the original base as stale as we're going to create a new wrapper
|
||||
@ -471,12 +480,6 @@ func diffToDisk(bottom *diffLayer) *diskLayer {
|
||||
base.cache.Set(hash[:], data)
|
||||
snapshotCleanAccountWriteMeter.Mark(int64(len(data)))
|
||||
|
||||
if batch.ValueSize() > ethdb.IdealBatchSize {
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Crit("Failed to write account snapshot", "err", err)
|
||||
}
|
||||
batch.Reset()
|
||||
}
|
||||
snapshotFlushAccountItemMeter.Mark(1)
|
||||
snapshotFlushAccountSizeMeter.Mark(int64(len(data)))
|
||||
}
|
||||
@ -505,18 +508,33 @@ func diffToDisk(bottom *diffLayer) *diskLayer {
|
||||
snapshotFlushStorageItemMeter.Mark(1)
|
||||
snapshotFlushStorageSizeMeter.Mark(int64(len(data)))
|
||||
}
|
||||
if batch.ValueSize() > ethdb.IdealBatchSize {
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Crit("Failed to write storage snapshot", "err", err)
|
||||
}
|
||||
batch.Reset()
|
||||
}
|
||||
}
|
||||
// Update the snapshot block marker and write any remainder data
|
||||
rawdb.WriteSnapshotRoot(batch, bottom.root)
|
||||
|
||||
// Write out the generator marker
|
||||
entry := journalGenerator{
|
||||
Done: base.genMarker == nil,
|
||||
Marker: base.genMarker,
|
||||
}
|
||||
if stats != nil {
|
||||
entry.Wiping = (stats.wiping != nil)
|
||||
entry.Accounts = stats.accounts
|
||||
entry.Slots = stats.slots
|
||||
entry.Storage = uint64(stats.storage)
|
||||
}
|
||||
blob, err := rlp.EncodeToBytes(entry)
|
||||
if err != nil {
|
||||
panic(fmt.Sprintf("Failed to RLP encode generator %v", err))
|
||||
}
|
||||
rawdb.WriteSnapshotGenerator(batch, blob)
|
||||
|
||||
// Flush all the updates in the single db operation. Ensure the
|
||||
// disk layer transition is atomic.
|
||||
if err := batch.Write(); err != nil {
|
||||
log.Crit("Failed to write leftover snapshot", "err", err)
|
||||
}
|
||||
log.Debug("Journalled disk layer", "root", bottom.root, "complete", base.genMarker == nil)
|
||||
res := &diskLayer{
|
||||
root: bottom.root,
|
||||
cache: base.cache,
|
||||
@ -554,7 +572,21 @@ func (t *Tree) Journal(root common.Hash) (common.Hash, error) {
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
|
||||
// Firstly write out the metadata of journal
|
||||
journal := new(bytes.Buffer)
|
||||
if err := rlp.Encode(journal, journalVersion); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
diskroot := t.diskRoot()
|
||||
if diskroot == (common.Hash{}) {
|
||||
return common.Hash{}, errors.New("invalid disk root")
|
||||
}
|
||||
// Secondly write out the disk layer root, ensure the
|
||||
// diff journal is continuous with disk.
|
||||
if err := rlp.Encode(journal, diskroot); err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
// Finally write out the journal of each layer in reverse order.
|
||||
base, err := snap.(snapshot).Journal(journal)
|
||||
if err != nil {
|
||||
return common.Hash{}, err
|
||||
@ -564,6 +596,29 @@ func (t *Tree) Journal(root common.Hash) (common.Hash, error) {
|
||||
return base, nil
|
||||
}
|
||||
|
||||
// LegacyJournal is basically identical to Journal. it's the legacy
|
||||
// version for flushing legacy journal. Now the only purpose of this
|
||||
// function is for testing.
|
||||
func (t *Tree) LegacyJournal(root common.Hash) (common.Hash, error) {
|
||||
// Retrieve the head snapshot to journal from var snap snapshot
|
||||
snap := t.Snapshot(root)
|
||||
if snap == nil {
|
||||
return common.Hash{}, fmt.Errorf("snapshot [%#x] missing", root)
|
||||
}
|
||||
// Run the journaling
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
|
||||
journal := new(bytes.Buffer)
|
||||
base, err := snap.(snapshot).LegacyJournal(journal)
|
||||
if err != nil {
|
||||
return common.Hash{}, err
|
||||
}
|
||||
// Store the journal into the database and return
|
||||
rawdb.WriteSnapshotJournal(t.diskdb, journal.Bytes())
|
||||
return base, nil
|
||||
}
|
||||
|
||||
// Rebuild wipes all available snapshot data from the persistent database and
|
||||
// discard all caches and diff layers. Afterwards, it starts a new snapshot
|
||||
// generator with the given root hash.
|
||||
@ -571,6 +626,10 @@ func (t *Tree) Rebuild(root common.Hash) {
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
|
||||
// Firstly delete any recovery flag in the database. Because now we are
|
||||
// building a brand new snapshot.
|
||||
rawdb.DeleteSnapshotRecoveryNumber(t.diskdb)
|
||||
|
||||
// Track whether there's a wipe currently running and keep it alive if so
|
||||
var wiper chan struct{}
|
||||
|
||||
@ -657,6 +716,16 @@ func (t *Tree) disklayer() *diskLayer {
|
||||
}
|
||||
}
|
||||
|
||||
// diskRoot is a internal helper function to return the disk layer root.
|
||||
// The lock of snapTree is assumed to be held already.
|
||||
func (t *Tree) diskRoot() common.Hash {
|
||||
disklayer := t.disklayer()
|
||||
if disklayer == nil {
|
||||
return common.Hash{}
|
||||
}
|
||||
return disklayer.Root()
|
||||
}
|
||||
|
||||
// generating is an internal helper function which reports whether the snapshot
|
||||
// is still under the construction.
|
||||
func (t *Tree) generating() (bool, error) {
|
||||
@ -671,3 +740,11 @@ func (t *Tree) generating() (bool, error) {
|
||||
defer layer.lock.RUnlock()
|
||||
return layer.genMarker != nil, nil
|
||||
}
|
||||
|
||||
// diskRoot is a external helper function to return the disk layer root.
|
||||
func (t *Tree) DiskRoot() common.Hash {
|
||||
t.lock.Lock()
|
||||
defer t.lock.Unlock()
|
||||
|
||||
return t.diskRoot()
|
||||
}
|
||||
|
Reference in New Issue
Block a user