core: define and test chain rewind corner cases (#21409)

* core: define and test chain reparation cornercases

* core: write up a variety of set-head tests

* core, eth: unify chain rollbacks, handle all the cases

* core: make linter smile

* core: remove commented out legacy code

* core, eth/downloader: fix review comments

* core: revert a removed recovery mechanism
This commit is contained in:
Péter Szilágyi
2020-08-20 13:01:24 +03:00
committed by GitHub
parent 0bdd295cc0
commit 8cbdc8638f
13 changed files with 3953 additions and 215 deletions

View File

@ -130,6 +130,16 @@ type CacheConfig struct {
SnapshotWait bool // Wait for snapshot construction on startup. TODO(karalabe): This is a dirty hack for testing, nuke it
}
// defaultCacheConfig are the default caching values if none are specified by the
// user (also used during testing).
var defaultCacheConfig = &CacheConfig{
TrieCleanLimit: 256,
TrieDirtyLimit: 256,
TrieTimeLimit: 5 * time.Minute,
SnapshotLimit: 256,
SnapshotWait: true,
}
// BlockChain represents the canonical chain given a database with a genesis
// block. The Blockchain manages chain imports, reverts, chain reorganisations.
//
@ -204,13 +214,7 @@ type BlockChain struct {
// Processor.
func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *params.ChainConfig, engine consensus.Engine, vmConfig vm.Config, shouldPreserve func(block *types.Block) bool, txLookupLimit *uint64) (*BlockChain, error) {
if cacheConfig == nil {
cacheConfig = &CacheConfig{
TrieCleanLimit: 256,
TrieDirtyLimit: 256,
TrieTimeLimit: 5 * time.Minute,
SnapshotLimit: 256,
SnapshotWait: true,
}
cacheConfig = defaultCacheConfig
}
bodyCache, _ := lru.New(bodyCacheLimit)
bodyRLPCache, _ := lru.New(bodyCacheLimit)
@ -268,15 +272,18 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
txIndexBlock = frozen
}
}
if err := bc.loadLastState(); err != nil {
return nil, err
}
// The first thing the node will do is reconstruct the verification data for
// the head block (ethash cache or clique voting snapshot). Might as well do
// it in advance.
bc.engine.VerifyHeader(bc, bc.CurrentHeader(), true)
// Make sure the state associated with the block is available
head := bc.CurrentBlock()
if _, err := state.New(head.Root(), bc.stateCache, bc.snaps); err != nil {
log.Warn("Head state missing, repairing", "number", head.Number(), "hash", head.Hash())
if err := bc.SetHead(head.NumberU64()); err != nil {
return nil, err
}
}
// Ensure that a previous crash in SetHead doesn't leave extra ancients
if frozen, err := bc.db.Ancients(); err == nil && frozen > 0 {
var (
needRewind bool
@ -286,7 +293,7 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
// blockchain repair. If the head full block is even lower than the ancient
// chain, truncate the ancient store.
fullBlock := bc.CurrentBlock()
if fullBlock != nil && fullBlock != bc.genesisBlock && fullBlock.NumberU64() < frozen-1 {
if fullBlock != nil && fullBlock.Hash() != bc.genesisBlock.Hash() && fullBlock.NumberU64() < frozen-1 {
needRewind = true
low = fullBlock.NumberU64()
}
@ -301,15 +308,17 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
}
}
if needRewind {
var hashes []common.Hash
previous := bc.CurrentHeader().Number.Uint64()
for i := low + 1; i <= bc.CurrentHeader().Number.Uint64(); i++ {
hashes = append(hashes, rawdb.ReadCanonicalHash(bc.db, i))
log.Error("Truncating ancient chain", "from", bc.CurrentHeader().Number.Uint64(), "to", low)
if err := bc.SetHead(low); err != nil {
return nil, err
}
bc.Rollback(hashes)
log.Warn("Truncate ancient chain", "from", previous, "to", low)
}
}
// The first thing the node will do is reconstruct the verification data for
// the head block (ethash cache or clique voting snapshot). Might as well do
// it in advance.
bc.engine.VerifyHeader(bc, bc.CurrentHeader(), true)
// Check the current state of the block hashes and make sure that we do not have any of the bad blocks in our chain
for hash := range BadHashes {
if header := bc.GetHeaderByHash(hash); header != nil {
@ -318,7 +327,9 @@ func NewBlockChain(db ethdb.Database, cacheConfig *CacheConfig, chainConfig *par
// make sure the headerByNumber (if present) is in our current canonical chain
if headerByNumber != nil && headerByNumber.Hash() == header.Hash() {
log.Error("Found bad hash, rewinding chain", "number", header.Number, "hash", header.ParentHash)
bc.SetHead(header.Number.Uint64() - 1)
if err := bc.SetHead(header.Number.Uint64() - 1); err != nil {
return nil, err
}
log.Error("Chain rewind was successful, resuming normal operation")
}
}
@ -385,15 +396,6 @@ func (bc *BlockChain) loadLastState() error {
log.Warn("Head block missing, resetting chain", "hash", head)
return bc.Reset()
}
// Make sure the state associated with the block is available
if _, err := state.New(currentBlock.Root(), bc.stateCache, bc.snaps); err != nil {
// Dangling block without a state associated, init from scratch
log.Warn("Head state missing, repairing chain", "number", currentBlock.Number(), "hash", currentBlock.Hash())
if err := bc.repair(&currentBlock); err != nil {
return err
}
rawdb.WriteHeadBlockHash(bc.db, currentBlock.Hash())
}
// Everything seems to be fine, set as the head block
bc.currentBlock.Store(currentBlock)
headBlockGauge.Update(int64(currentBlock.NumberU64()))
@ -427,30 +429,48 @@ func (bc *BlockChain) loadLastState() error {
log.Info("Loaded most recent local header", "number", currentHeader.Number, "hash", currentHeader.Hash(), "td", headerTd, "age", common.PrettyAge(time.Unix(int64(currentHeader.Time), 0)))
log.Info("Loaded most recent local full block", "number", currentBlock.Number(), "hash", currentBlock.Hash(), "td", blockTd, "age", common.PrettyAge(time.Unix(int64(currentBlock.Time()), 0)))
log.Info("Loaded most recent local fast block", "number", currentFastBlock.Number(), "hash", currentFastBlock.Hash(), "td", fastTd, "age", common.PrettyAge(time.Unix(int64(currentFastBlock.Time()), 0)))
if pivot := rawdb.ReadLastPivotNumber(bc.db); pivot != nil {
log.Info("Loaded last fast-sync pivot marker", "number", *pivot)
}
return nil
}
// SetHead rewinds the local chain to a new head. In the case of headers, everything
// above the new head will be deleted and the new one set. In the case of blocks
// though, the head may be further rewound if block bodies are missing (non-archive
// nodes after a fast sync).
// SetHead rewinds the local chain to a new head. Depending on whether the node
// was fast synced or full synced and in which state, the method will try to
// delete minimal data from disk whilst retaining chain consistency.
func (bc *BlockChain) SetHead(head uint64) error {
log.Warn("Rewinding blockchain", "target", head)
bc.chainmu.Lock()
defer bc.chainmu.Unlock()
updateFn := func(db ethdb.KeyValueWriter, header *types.Header) {
// Rewind the block chain, ensuring we don't end up with a stateless head block
if currentBlock := bc.CurrentBlock(); currentBlock != nil && header.Number.Uint64() < currentBlock.NumberU64() {
// Retrieve the last pivot block to short circuit rollbacks beyond it and the
// current freezer limit to start nuking id underflown
pivot := rawdb.ReadLastPivotNumber(bc.db)
frozen, _ := bc.db.Ancients()
updateFn := func(db ethdb.KeyValueWriter, header *types.Header) (uint64, bool) {
// Rewind the block chain, ensuring we don't end up with a stateless head
// block. Note, depth equality is permitted to allow using SetHead as a
// chain reparation mechanism without deleting any data!
if currentBlock := bc.CurrentBlock(); currentBlock != nil && header.Number.Uint64() <= currentBlock.NumberU64() {
newHeadBlock := bc.GetBlock(header.Hash(), header.Number.Uint64())
if newHeadBlock == nil {
log.Error("Gap in the chain, rewinding to genesis", "number", header.Number, "hash", header.Hash())
newHeadBlock = bc.genesisBlock
} else {
if _, err := state.New(newHeadBlock.Root(), bc.stateCache, bc.snaps); err != nil {
// Rewound state missing, rolled back to before pivot, reset to genesis
newHeadBlock = bc.genesisBlock
// Block exists, keep rewinding until we find one with state
for {
if _, err := state.New(newHeadBlock.Root(), bc.stateCache, bc.snaps); err != nil {
log.Trace("Block state missing, rewinding further", "number", newHeadBlock.NumberU64(), "hash", newHeadBlock.Hash())
if pivot == nil || newHeadBlock.NumberU64() > *pivot {
newHeadBlock = bc.GetBlock(newHeadBlock.ParentHash(), newHeadBlock.NumberU64()-1)
continue
} else {
log.Trace("Rewind passed pivot, aiming genesis", "number", newHeadBlock.NumberU64(), "hash", newHeadBlock.Hash(), "pivot", *pivot)
newHeadBlock = bc.genesisBlock
}
}
log.Debug("Rewound to block with state", "number", newHeadBlock.NumberU64(), "hash", newHeadBlock.Hash())
break
}
}
rawdb.WriteHeadBlockHash(db, newHeadBlock.Hash())
@ -462,7 +482,6 @@ func (bc *BlockChain) SetHead(head uint64) error {
bc.currentBlock.Store(newHeadBlock)
headBlockGauge.Update(int64(newHeadBlock.NumberU64()))
}
// Rewind the fast block in a simpleton way to the target head
if currentFastBlock := bc.CurrentFastBlock(); currentFastBlock != nil && header.Number.Uint64() < currentFastBlock.NumberU64() {
newHeadFastBlock := bc.GetBlock(header.Hash(), header.Number.Uint64())
@ -479,8 +498,17 @@ func (bc *BlockChain) SetHead(head uint64) error {
bc.currentFastBlock.Store(newHeadFastBlock)
headFastBlockGauge.Update(int64(newHeadFastBlock.NumberU64()))
}
}
head := bc.CurrentBlock().NumberU64()
// If setHead underflown the freezer threshold and the block processing
// intent afterwards is full block importing, delete the chain segment
// between the stateful-block and the sethead target.
var wipe bool
if head+1 < frozen {
wipe = pivot == nil || head >= *pivot
}
return head, wipe // Only force wipe if full synced
}
// Rewind the header chain, deleting all block bodies until then
delFn := func(db ethdb.KeyValueWriter, hash common.Hash, num uint64) {
// Ignore the error here since light client won't hit this path
@ -488,10 +516,9 @@ func (bc *BlockChain) SetHead(head uint64) error {
if num+1 <= frozen {
// Truncate all relative data(header, total difficulty, body, receipt
// and canonical hash) from ancient store.
if err := bc.db.TruncateAncients(num + 1); err != nil {
if err := bc.db.TruncateAncients(num); err != nil {
log.Crit("Failed to truncate ancient data", "number", num, "err", err)
}
// Remove the hash <-> number mapping from the active store.
rawdb.DeleteHeaderNumber(db, hash)
} else {
@ -503,8 +530,18 @@ func (bc *BlockChain) SetHead(head uint64) error {
}
// Todo(rjl493456442) txlookup, bloombits, etc
}
bc.hc.SetHead(head, updateFn, delFn)
// If SetHead was only called as a chain reparation method, try to skip
// touching the header chain altogether, unless the freezer is broken
if block := bc.CurrentBlock(); block.NumberU64() == head {
if target, force := updateFn(bc.db, block.Header()); force {
bc.hc.SetHead(target, updateFn, delFn)
}
} else {
// Rewind the chain to the requested head and keep going backwards until a
// block with a state is found or fast sync pivot is passed
log.Warn("Rewinding blockchain", "target", head)
bc.hc.SetHead(head, updateFn, delFn)
}
// Clear out any stale content from the caches
bc.bodyCache.Purge()
bc.bodyRLPCache.Purge()
@ -627,28 +664,6 @@ func (bc *BlockChain) ResetWithGenesisBlock(genesis *types.Block) error {
return nil
}
// repair tries to repair the current blockchain by rolling back the current block
// until one with associated state is found. This is needed to fix incomplete db
// writes caused either by crashes/power outages, or simply non-committed tries.
//
// This method only rolls back the current block. The current header and current
// fast block are left intact.
func (bc *BlockChain) repair(head **types.Block) error {
for {
// Abort if we've rewound to a head block that does have associated state
if _, err := state.New((*head).Root(), bc.stateCache, bc.snaps); err == nil {
log.Info("Rewound blockchain to past state", "number", (*head).Number(), "hash", (*head).Hash())
return nil
}
// Otherwise rewind one block and recheck state availability there
block := bc.GetBlock((*head).ParentHash(), (*head).NumberU64()-1)
if block == nil {
return fmt.Errorf("missing block %d [%x]", (*head).NumberU64()-1, (*head).ParentHash())
}
*head = block
}
}
// Export writes the active chain to the given writer.
func (bc *BlockChain) Export(w io.Writer) error {
return bc.ExportN(w, uint64(0), bc.CurrentBlock().NumberU64())
@ -985,52 +1000,6 @@ const (
SideStatTy
)
// Rollback is designed to remove a chain of links from the database that aren't
// certain enough to be valid.
func (bc *BlockChain) Rollback(chain []common.Hash) {
bc.chainmu.Lock()
defer bc.chainmu.Unlock()
batch := bc.db.NewBatch()
for i := len(chain) - 1; i >= 0; i-- {
hash := chain[i]
// Degrade the chain markers if they are explicitly reverted.
// In theory we should update all in-memory markers in the
// last step, however the direction of rollback is from high
// to low, so it's safe the update in-memory markers directly.
currentHeader := bc.hc.CurrentHeader()
if currentHeader.Hash() == hash {
newHeadHeader := bc.GetHeader(currentHeader.ParentHash, currentHeader.Number.Uint64()-1)
rawdb.WriteHeadHeaderHash(batch, currentHeader.ParentHash)
bc.hc.SetCurrentHeader(newHeadHeader)
}
if currentFastBlock := bc.CurrentFastBlock(); currentFastBlock.Hash() == hash {
newFastBlock := bc.GetBlock(currentFastBlock.ParentHash(), currentFastBlock.NumberU64()-1)
rawdb.WriteHeadFastBlockHash(batch, currentFastBlock.ParentHash())
bc.currentFastBlock.Store(newFastBlock)
headFastBlockGauge.Update(int64(newFastBlock.NumberU64()))
}
if currentBlock := bc.CurrentBlock(); currentBlock.Hash() == hash {
newBlock := bc.GetBlock(currentBlock.ParentHash(), currentBlock.NumberU64()-1)
rawdb.WriteHeadBlockHash(batch, currentBlock.ParentHash())
bc.currentBlock.Store(newBlock)
headBlockGauge.Update(int64(newBlock.NumberU64()))
}
}
if err := batch.Write(); err != nil {
log.Crit("Failed to rollback chain markers", "err", err)
}
// Truncate ancient data which exceeds the current header.
//
// Notably, it can happen that system crashes without truncating the ancient data
// but the head indicator has been updated in the active store. Regarding this issue,
// system will self recovery by truncating the extra data during the setup phase.
if err := bc.truncateAncient(bc.hc.CurrentHeader().Number.Uint64()); err != nil {
log.Crit("Truncate ancient store failed", "err", err)
}
}
// truncateAncient rewinds the blockchain to the specified header and deletes all
// data in the ancient store that exceeds the specified header.
func (bc *BlockChain) truncateAncient(head uint64) error {

File diff suppressed because it is too large Load Diff

File diff suppressed because it is too large Load Diff

View File

@ -731,12 +731,12 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
return db, func() { os.RemoveAll(dir) }
}
// Configure a subchain to roll back
remove := []common.Hash{}
for _, block := range blocks[height/2:] {
remove = append(remove, block.Hash())
}
remove := blocks[height/2].NumberU64()
// Create a small assertion method to check the three heads
assert := func(t *testing.T, kind string, chain *BlockChain, header uint64, fast uint64, block uint64) {
t.Helper()
if num := chain.CurrentBlock().NumberU64(); num != block {
t.Errorf("%s head block mismatch: have #%v, want #%v", kind, num, block)
}
@ -750,14 +750,18 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
// Import the chain as an archive node and ensure all pointers are updated
archiveDb, delfn := makeDb()
defer delfn()
archive, _ := NewBlockChain(archiveDb, nil, gspec.Config, ethash.NewFaker(), vm.Config{}, nil, nil)
archiveCaching := *defaultCacheConfig
archiveCaching.TrieDirtyDisabled = true
archive, _ := NewBlockChain(archiveDb, &archiveCaching, gspec.Config, ethash.NewFaker(), vm.Config{}, nil, nil)
if n, err := archive.InsertChain(blocks); err != nil {
t.Fatalf("failed to process block %d: %v", n, err)
}
defer archive.Stop()
assert(t, "archive", archive, height, height, height)
archive.Rollback(remove)
archive.SetHead(remove - 1)
assert(t, "archive", archive, height/2, height/2, height/2)
// Import the chain as a non-archive node and ensure all pointers are updated
@ -777,7 +781,7 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
t.Fatalf("failed to insert receipt %d: %v", n, err)
}
assert(t, "fast", fast, height, height, 0)
fast.Rollback(remove)
fast.SetHead(remove - 1)
assert(t, "fast", fast, height/2, height/2, 0)
// Import the chain as a ancient-first node and ensure all pointers are updated
@ -793,12 +797,12 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
t.Fatalf("failed to insert receipt %d: %v", n, err)
}
assert(t, "ancient", ancient, height, height, 0)
ancient.Rollback(remove)
assert(t, "ancient", ancient, height/2, height/2, 0)
if frozen, err := ancientDb.Ancients(); err != nil || frozen != height/2+1 {
t.Fatalf("failed to truncate ancient store, want %v, have %v", height/2+1, frozen)
}
ancient.SetHead(remove - 1)
assert(t, "ancient", ancient, 0, 0, 0)
if frozen, err := ancientDb.Ancients(); err != nil || frozen != 1 {
t.Fatalf("failed to truncate ancient store, want %v, have %v", 1, frozen)
}
// Import the chain as a light node and ensure all pointers are updated
lightDb, delfn := makeDb()
defer delfn()
@ -809,7 +813,7 @@ func TestLightVsFastVsFullChainHeads(t *testing.T) {
defer light.Stop()
assert(t, "light", light, height, 0, 0)
light.Rollback(remove)
light.SetHead(remove - 1)
assert(t, "light", light, height/2, 0, 0)
}
@ -1585,6 +1589,7 @@ func TestBlockchainRecovery(t *testing.T) {
t.Fatalf("failed to create temp freezer dir: %v", err)
}
defer os.Remove(frdir)
ancientDb, err := rawdb.NewDatabaseWithFreezer(rawdb.NewMemoryDatabase(), frdir, "")
if err != nil {
t.Fatalf("failed to create temp freezer db: %v", err)
@ -1602,6 +1607,7 @@ func TestBlockchainRecovery(t *testing.T) {
if n, err := ancient.InsertReceiptChain(blocks, receipts, uint64(3*len(blocks)/4)); err != nil {
t.Fatalf("failed to insert receipt %d: %v", n, err)
}
rawdb.WriteLastPivotNumber(ancientDb, blocks[len(blocks)-1].NumberU64()) // Force fast sync behavior
ancient.Stop()
// Destroy head fast block manually
@ -1912,11 +1918,9 @@ func testInsertKnownChainData(t *testing.T, typ string) {
asserter(t, blocks[len(blocks)-1])
// Import a long canonical chain with some known data as prefix.
var rollback []common.Hash
for i := len(blocks) / 2; i < len(blocks); i++ {
rollback = append(rollback, blocks[i].Hash())
}
chain.Rollback(rollback)
rollback := blocks[len(blocks)/2].NumberU64()
chain.SetHead(rollback - 1)
if err := inserter(append(blocks, blocks2...), append(receipts, receipts2...)); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}
@ -1936,11 +1940,7 @@ func testInsertKnownChainData(t *testing.T, typ string) {
asserter(t, blocks3[len(blocks3)-1])
// Rollback the heavier chain and re-insert the longer chain again
for i := 0; i < len(blocks3); i++ {
rollback = append(rollback, blocks3[i].Hash())
}
chain.Rollback(rollback)
chain.SetHead(rollback - 1)
if err := inserter(append(blocks, blocks2...), append(receipts, receipts2...)); err != nil {
t.Fatalf("failed to insert chain data: %v", err)
}

View File

@ -488,8 +488,10 @@ func (hc *HeaderChain) SetCurrentHeader(head *types.Header) {
type (
// UpdateHeadBlocksCallback is a callback function that is called by SetHead
// before head header is updated.
UpdateHeadBlocksCallback func(ethdb.KeyValueWriter, *types.Header)
// before head header is updated. The method will return the actual block it
// updated the head to (missing state) and a flag if setHead should continue
// rewinding till that forcefully (exceeded ancient limits)
UpdateHeadBlocksCallback func(ethdb.KeyValueWriter, *types.Header) (uint64, bool)
// DeleteBlockContentCallback is a callback function that is called by SetHead
// before each header is deleted.
@ -502,9 +504,10 @@ func (hc *HeaderChain) SetHead(head uint64, updateFn UpdateHeadBlocksCallback, d
var (
parentHash common.Hash
batch = hc.chainDb.NewBatch()
origin = true
)
for hdr := hc.CurrentHeader(); hdr != nil && hdr.Number.Uint64() > head; hdr = hc.CurrentHeader() {
hash, num := hdr.Hash(), hdr.Number.Uint64()
num := hdr.Number.Uint64()
// Rewind block chain to new head.
parent := hc.GetHeader(hdr.ParentHash, num-1)
@ -512,16 +515,21 @@ func (hc *HeaderChain) SetHead(head uint64, updateFn UpdateHeadBlocksCallback, d
parent = hc.genesisHeader
}
parentHash = hdr.ParentHash
// Notably, since geth has the possibility for setting the head to a low
// height which is even lower than ancient head.
// In order to ensure that the head is always no higher than the data in
// the database(ancient store or active store), we need to update head
// the database (ancient store or active store), we need to update head
// first then remove the relative data from the database.
//
// Update head first(head fast block, head full block) before deleting the data.
markerBatch := hc.chainDb.NewBatch()
if updateFn != nil {
updateFn(markerBatch, parent)
newHead, force := updateFn(markerBatch, parent)
if force && newHead < head {
log.Warn("Force rewinding till ancient limit", "head", newHead)
head = newHead
}
}
// Update head header then.
rawdb.WriteHeadHeaderHash(markerBatch, parentHash)
@ -532,14 +540,34 @@ func (hc *HeaderChain) SetHead(head uint64, updateFn UpdateHeadBlocksCallback, d
hc.currentHeaderHash = parentHash
headHeaderGauge.Update(parent.Number.Int64())
// Remove the relative data from the database.
if delFn != nil {
delFn(batch, hash, num)
// If this is the first iteration, wipe any leftover data upwards too so
// we don't end up with dangling daps in the database
var nums []uint64
if origin {
for n := num + 1; len(rawdb.ReadAllHashes(hc.chainDb, n)) > 0; n++ {
nums = append([]uint64{n}, nums...) // suboptimal, but we don't really expect this path
}
origin = false
}
nums = append(nums, num)
// Remove the related data from the database on all sidechains
for _, num := range nums {
// Gather all the side fork hashes
hashes := rawdb.ReadAllHashes(hc.chainDb, num)
if len(hashes) == 0 {
// No hashes in the database whatsoever, probably frozen already
hashes = append(hashes, hdr.Hash())
}
for _, hash := range hashes {
if delFn != nil {
delFn(batch, hash, num)
}
rawdb.DeleteHeader(batch, hash, num)
rawdb.DeleteTd(batch, hash, num)
}
rawdb.DeleteCanonicalHash(batch, num)
}
// Rewind header chain to new head.
rawdb.DeleteHeader(batch, hash, num)
rawdb.DeleteTd(batch, hash, num)
rawdb.DeleteCanonicalHash(batch, num)
}
// Flush all accumulated deletions.
if err := batch.Write(); err != nil {

View File

@ -187,6 +187,32 @@ func WriteHeadFastBlockHash(db ethdb.KeyValueWriter, hash common.Hash) {
}
}
// ReadLastPivotNumber retrieves the number of the last pivot block. If the node
// full synced, the last pivot will always be nil.
func ReadLastPivotNumber(db ethdb.KeyValueReader) *uint64 {
data, _ := db.Get(lastPivotKey)
if len(data) == 0 {
return nil
}
var pivot uint64
if err := rlp.DecodeBytes(data, &pivot); err != nil {
log.Error("Invalid pivot block number in database", "err", err)
return nil
}
return &pivot
}
// WriteLastPivotNumber stores the number of the last pivot block.
func WriteLastPivotNumber(db ethdb.KeyValueWriter, pivot uint64) {
enc, err := rlp.EncodeToBytes(pivot)
if err != nil {
log.Crit("Failed to encode pivot block number", "err", err)
}
if err := db.Put(lastPivotKey, enc); err != nil {
log.Crit("Failed to store pivot block number", "err", err)
}
}
// ReadFastTrieProgress retrieves the number of tries nodes fast synced to allow
// reporting correct numbers across restarts.
func ReadFastTrieProgress(db ethdb.KeyValueReader) uint64 {

View File

@ -21,6 +21,7 @@ import (
"errors"
"fmt"
"os"
"sync/atomic"
"time"
"github.com/ethereum/go-ethereum/common"
@ -53,6 +54,22 @@ func (frdb *freezerdb) Close() error {
return nil
}
// Freeze is a helper method used for external testing to trigger and block until
// a freeze cycle completes, without having to sleep for a minute to trigger the
// automatic background run.
func (frdb *freezerdb) Freeze(threshold uint64) {
// Set the freezer threshold to a temporary value
defer func(old uint64) {
atomic.StoreUint64(&frdb.AncientStore.(*freezer).threshold, old)
}(atomic.LoadUint64(&frdb.AncientStore.(*freezer).threshold))
atomic.StoreUint64(&frdb.AncientStore.(*freezer).threshold, threshold)
// Trigger a freeze cycle and block until it's done
trigger := make(chan struct{}, 1)
frdb.AncientStore.(*freezer).trigger <- trigger
<-trigger
}
// nofreezedb is a database wrapper that disables freezer data retrievals.
type nofreezedb struct {
ethdb.KeyValueStore

View File

@ -70,12 +70,16 @@ type freezer struct {
// WARNING: The `frozen` field is accessed atomically. On 32 bit platforms, only
// 64-bit aligned fields can be atomic. The struct is guaranteed to be so aligned,
// so take advantage of that (https://golang.org/pkg/sync/atomic/#pkg-note-BUG).
frozen uint64 // Number of blocks already frozen
frozen uint64 // Number of blocks already frozen
threshold uint64 // Number of recent blocks not to freeze (params.FullImmutabilityThreshold apart from tests)
tables map[string]*freezerTable // Data tables for storing everything
instanceLock fileutil.Releaser // File-system lock to prevent double opens
quit chan struct{}
closeOnce sync.Once
trigger chan chan struct{} // Manual blocking freeze trigger, test determinism
quit chan struct{}
closeOnce sync.Once
}
// newFreezer creates a chain freezer that moves ancient chain data into
@ -102,8 +106,10 @@ func newFreezer(datadir string, namespace string) (*freezer, error) {
}
// Open all the supported data tables
freezer := &freezer{
threshold: params.FullImmutabilityThreshold,
tables: make(map[string]*freezerTable),
instanceLock: lock,
trigger: make(chan chan struct{}),
quit: make(chan struct{}),
}
for name, disableSnappy := range freezerNoSnappy {
@ -261,7 +267,10 @@ func (f *freezer) Sync() error {
func (f *freezer) freeze(db ethdb.KeyValueStore) {
nfdb := &nofreezedb{KeyValueStore: db}
backoff := false
var (
backoff bool
triggered chan struct{} // Used in tests
)
for {
select {
case <-f.quit:
@ -270,9 +279,16 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
default:
}
if backoff {
// If we were doing a manual trigger, notify it
if triggered != nil {
triggered <- struct{}{}
triggered = nil
}
select {
case <-time.NewTimer(freezerRecheckInterval).C:
backoff = false
case triggered = <-f.trigger:
backoff = false
case <-f.quit:
return
}
@ -285,18 +301,20 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
continue
}
number := ReadHeaderNumber(nfdb, hash)
threshold := atomic.LoadUint64(&f.threshold)
switch {
case number == nil:
log.Error("Current full block number unavailable", "hash", hash)
backoff = true
continue
case *number < params.FullImmutabilityThreshold:
log.Debug("Current full block not old enough", "number", *number, "hash", hash, "delay", params.FullImmutabilityThreshold)
case *number < threshold:
log.Debug("Current full block not old enough", "number", *number, "hash", hash, "delay", threshold)
backoff = true
continue
case *number-params.FullImmutabilityThreshold <= f.frozen:
case *number-threshold <= f.frozen:
log.Debug("Ancient blocks frozen already", "number", *number, "hash", hash, "frozen", f.frozen)
backoff = true
continue
@ -308,7 +326,7 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
continue
}
// Seems we have data ready to be frozen, process in usable batches
limit := *number - params.FullImmutabilityThreshold
limit := *number - threshold
if limit-f.frozen > freezerBatchLimit {
limit = f.frozen + freezerBatchLimit
}
@ -317,7 +335,7 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
first = f.frozen
ancients = make([]common.Hash, 0, limit-f.frozen)
)
for f.frozen < limit {
for f.frozen <= limit {
// Retrieves all the components of the canonical block
hash := ReadCanonicalHash(nfdb, f.frozen)
if hash == (common.Hash{}) {
@ -368,11 +386,15 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
log.Crit("Failed to delete frozen canonical blocks", "err", err)
}
batch.Reset()
// Wipe out side chain also.
// Wipe out side chains also and track dangling side chians
var dangling []common.Hash
for number := first; number < f.frozen; number++ {
// Always keep the genesis block in active database
if number != 0 {
for _, hash := range ReadAllHashes(db, number) {
dangling = ReadAllHashes(db, number)
for _, hash := range dangling {
log.Trace("Deleting side chain", "number", number, "hash", hash)
DeleteBlock(batch, hash, number)
}
}
@ -380,6 +402,41 @@ func (f *freezer) freeze(db ethdb.KeyValueStore) {
if err := batch.Write(); err != nil {
log.Crit("Failed to delete frozen side blocks", "err", err)
}
batch.Reset()
// Step into the future and delete and dangling side chains
if f.frozen > 0 {
tip := f.frozen
for len(dangling) > 0 {
drop := make(map[common.Hash]struct{})
for _, hash := range dangling {
log.Debug("Dangling parent from freezer", "number", tip-1, "hash", hash)
drop[hash] = struct{}{}
}
children := ReadAllHashes(db, tip)
for i := 0; i < len(children); i++ {
// Dig up the child and ensure it's dangling
child := ReadHeader(nfdb, children[i], tip)
if child == nil {
log.Error("Missing dangling header", "number", tip, "hash", children[i])
continue
}
if _, ok := drop[child.ParentHash]; !ok {
children = append(children[:i], children[i+1:]...)
i--
continue
}
// Delete all block data associated with the child
log.Debug("Deleting dangling block", "number", tip, "hash", children[i], "parent", child.ParentHash)
DeleteBlock(batch, children[i], tip)
}
dangling = children
tip++
}
if err := batch.Write(); err != nil {
log.Crit("Failed to delete dangling side blocks", "err", err)
}
}
// Log something friendly for the user
context := []interface{}{
"blocks", f.frozen - first, "elapsed", common.PrettyDuration(time.Since(start)), "number", f.frozen - 1,

View File

@ -38,6 +38,9 @@ var (
// headFastBlockKey tracks the latest known incomplete block's hash during fast sync.
headFastBlockKey = []byte("LastFast")
// lastPivotKey tracks the last pivot block used by fast sync (to reenable on sethead).
lastPivotKey = []byte("LastPivot")
// fastTrieProgressKey tracks the number of trie entries imported during fast sync.
fastTrieProgressKey = []byte("TrieSync")