plugeth/core/rawdb/freezer.go
rjl493456442 538a868384
core/rawdb, cmd, ethdb, eth: implement freezer tail deletion (#23954)
* core/rawdb, cmd, ethdb, eth: implement freezer tail deletion

* core/rawdb: address comments from martin and sina

* core/rawdb: fixes cornercase in tail deletion

* core/rawdb: separate metadata into a standalone file

* core/rawdb: remove unused code

* core/rawdb: add random test

* core/rawdb: polish code

* core/rawdb: fsync meta file before manipulating the index

* core/rawdb: fix typo

* core/rawdb: address comments
2022-03-10 09:37:23 +01:00

620 lines
18 KiB
Go

// Copyright 2019 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 <http://www.gnu.org/licenses/>.
package rawdb
import (
"errors"
"fmt"
"math"
"os"
"path/filepath"
"sync"
"sync/atomic"
"time"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/ethereum/go-ethereum/log"
"github.com/ethereum/go-ethereum/metrics"
"github.com/ethereum/go-ethereum/params"
"github.com/prometheus/tsdb/fileutil"
)
var (
// errReadOnly is returned if the freezer is opened in read only mode. All the
// mutations are disallowed.
errReadOnly = errors.New("read only")
// errUnknownTable is returned if the user attempts to read from a table that is
// not tracked by the freezer.
errUnknownTable = errors.New("unknown table")
// errOutOrderInsertion is returned if the user attempts to inject out-of-order
// binary blobs into the freezer.
errOutOrderInsertion = errors.New("the append operation is out-order")
// errSymlinkDatadir is returned if the ancient directory specified by user
// is a symbolic link.
errSymlinkDatadir = errors.New("symbolic link datadir is not supported")
)
const (
// freezerRecheckInterval is the frequency to check the key-value database for
// chain progression that might permit new blocks to be frozen into immutable
// storage.
freezerRecheckInterval = time.Minute
// freezerBatchLimit is the maximum number of blocks to freeze in one batch
// before doing an fsync and deleting it from the key-value store.
freezerBatchLimit = 30000
// freezerTableSize defines the maximum size of freezer data files.
freezerTableSize = 2 * 1000 * 1000 * 1000
)
// freezer is a memory mapped append-only database to store immutable chain data
// into flat files:
//
// - The append only nature ensures that disk writes are minimized.
// - The memory mapping ensures we can max out system memory for caching without
// reserving it for go-ethereum. This would also reduce the memory requirements
// of Geth, and thus also GC overhead.
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
tail uint64 // Number of the first stored item in the freezer
threshold uint64 // Number of recent blocks not to freeze (params.FullImmutabilityThreshold apart from tests)
// This lock synchronizes writers and the truncate operation, as well as
// the "atomic" (batched) read operations.
writeLock sync.RWMutex
writeBatch *freezerBatch
readonly bool
tables map[string]*freezerTable // Data tables for storing everything
instanceLock fileutil.Releaser // File-system lock to prevent double opens
trigger chan chan struct{} // Manual blocking freeze trigger, test determinism
quit chan struct{}
wg sync.WaitGroup
closeOnce sync.Once
}
// newFreezer creates a chain freezer that moves ancient chain data into
// append-only flat file containers.
//
// The 'tables' argument defines the data tables. If the value of a map
// entry is true, snappy compression is disabled for the table.
func newFreezer(datadir string, namespace string, readonly bool, maxTableSize uint32, tables map[string]bool) (*freezer, error) {
// Create the initial freezer object
var (
readMeter = metrics.NewRegisteredMeter(namespace+"ancient/read", nil)
writeMeter = metrics.NewRegisteredMeter(namespace+"ancient/write", nil)
sizeGauge = metrics.NewRegisteredGauge(namespace+"ancient/size", nil)
)
// Ensure the datadir is not a symbolic link if it exists.
if info, err := os.Lstat(datadir); !os.IsNotExist(err) {
if info.Mode()&os.ModeSymlink != 0 {
log.Warn("Symbolic link ancient database is not supported", "path", datadir)
return nil, errSymlinkDatadir
}
}
// Leveldb uses LOCK as the filelock filename. To prevent the
// name collision, we use FLOCK as the lock name.
lock, _, err := fileutil.Flock(filepath.Join(datadir, "FLOCK"))
if err != nil {
return nil, err
}
// Open all the supported data tables
freezer := &freezer{
readonly: readonly,
threshold: params.FullImmutabilityThreshold,
tables: make(map[string]*freezerTable),
instanceLock: lock,
trigger: make(chan chan struct{}),
quit: make(chan struct{}),
}
// Create the tables.
for name, disableSnappy := range tables {
table, err := newTable(datadir, name, readMeter, writeMeter, sizeGauge, maxTableSize, disableSnappy, readonly)
if err != nil {
for _, table := range freezer.tables {
table.Close()
}
lock.Release()
return nil, err
}
freezer.tables[name] = table
}
if freezer.readonly {
// In readonly mode only validate, don't truncate.
// validate also sets `freezer.frozen`.
err = freezer.validate()
} else {
// Truncate all tables to common length.
err = freezer.repair()
}
if err != nil {
for _, table := range freezer.tables {
table.Close()
}
lock.Release()
return nil, err
}
// Create the write batch.
freezer.writeBatch = newFreezerBatch(freezer)
log.Info("Opened ancient database", "database", datadir, "readonly", readonly)
return freezer, nil
}
// Close terminates the chain freezer, unmapping all the data files.
func (f *freezer) Close() error {
f.writeLock.Lock()
defer f.writeLock.Unlock()
var errs []error
f.closeOnce.Do(func() {
close(f.quit)
// Wait for any background freezing to stop
f.wg.Wait()
for _, table := range f.tables {
if err := table.Close(); err != nil {
errs = append(errs, err)
}
}
if err := f.instanceLock.Release(); err != nil {
errs = append(errs, err)
}
})
if errs != nil {
return fmt.Errorf("%v", errs)
}
return nil
}
// HasAncient returns an indicator whether the specified ancient data exists
// in the freezer.
func (f *freezer) HasAncient(kind string, number uint64) (bool, error) {
if table := f.tables[kind]; table != nil {
return table.has(number), nil
}
return false, nil
}
// Ancient retrieves an ancient binary blob from the append-only immutable files.
func (f *freezer) Ancient(kind string, number uint64) ([]byte, error) {
if table := f.tables[kind]; table != nil {
return table.Retrieve(number)
}
return nil, errUnknownTable
}
// AncientRange retrieves multiple items in sequence, starting from the index 'start'.
// It will return
// - at most 'max' items,
// - at least 1 item (even if exceeding the maxByteSize), but will otherwise
// return as many items as fit into maxByteSize.
func (f *freezer) AncientRange(kind string, start, count, maxBytes uint64) ([][]byte, error) {
if table := f.tables[kind]; table != nil {
return table.RetrieveItems(start, count, maxBytes)
}
return nil, errUnknownTable
}
// Ancients returns the length of the frozen items.
func (f *freezer) Ancients() (uint64, error) {
return atomic.LoadUint64(&f.frozen), nil
}
// Tail returns the number of first stored item in the freezer.
func (f *freezer) Tail() (uint64, error) {
return atomic.LoadUint64(&f.tail), nil
}
// AncientSize returns the ancient size of the specified category.
func (f *freezer) AncientSize(kind string) (uint64, error) {
// This needs the write lock to avoid data races on table fields.
// Speed doesn't matter here, AncientSize is for debugging.
f.writeLock.RLock()
defer f.writeLock.RUnlock()
if table := f.tables[kind]; table != nil {
return table.size()
}
return 0, errUnknownTable
}
// ReadAncients runs the given read operation while ensuring that no writes take place
// on the underlying freezer.
func (f *freezer) ReadAncients(fn func(ethdb.AncientReader) error) (err error) {
f.writeLock.RLock()
defer f.writeLock.RUnlock()
return fn(f)
}
// ModifyAncients runs the given write operation.
func (f *freezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (writeSize int64, err error) {
if f.readonly {
return 0, errReadOnly
}
f.writeLock.Lock()
defer f.writeLock.Unlock()
// Roll back all tables to the starting position in case of error.
prevItem := f.frozen
defer func() {
if err != nil {
// The write operation has failed. Go back to the previous item position.
for name, table := range f.tables {
err := table.truncateHead(prevItem)
if err != nil {
log.Error("Freezer table roll-back failed", "table", name, "index", prevItem, "err", err)
}
}
}
}()
f.writeBatch.reset()
if err := fn(f.writeBatch); err != nil {
return 0, err
}
item, writeSize, err := f.writeBatch.commit()
if err != nil {
return 0, err
}
atomic.StoreUint64(&f.frozen, item)
return writeSize, nil
}
// TruncateHead discards any recent data above the provided threshold number.
func (f *freezer) TruncateHead(items uint64) error {
if f.readonly {
return errReadOnly
}
f.writeLock.Lock()
defer f.writeLock.Unlock()
if atomic.LoadUint64(&f.frozen) <= items {
return nil
}
for _, table := range f.tables {
if err := table.truncateHead(items); err != nil {
return err
}
}
atomic.StoreUint64(&f.frozen, items)
return nil
}
// TruncateTail discards any recent data below the provided threshold number.
func (f *freezer) TruncateTail(tail uint64) error {
if f.readonly {
return errReadOnly
}
f.writeLock.Lock()
defer f.writeLock.Unlock()
if atomic.LoadUint64(&f.tail) >= tail {
return nil
}
for _, table := range f.tables {
if err := table.truncateTail(tail); err != nil {
return err
}
}
atomic.StoreUint64(&f.tail, tail)
return nil
}
// Sync flushes all data tables to disk.
func (f *freezer) Sync() error {
var errs []error
for _, table := range f.tables {
if err := table.Sync(); err != nil {
errs = append(errs, err)
}
}
if errs != nil {
return fmt.Errorf("%v", errs)
}
return nil
}
// validate checks that every table has the same length.
// Used instead of `repair` in readonly mode.
func (f *freezer) validate() error {
if len(f.tables) == 0 {
return nil
}
var (
length uint64
name string
)
// Hack to get length of any table
for kind, table := range f.tables {
length = atomic.LoadUint64(&table.items)
name = kind
break
}
// Now check every table against that length
for kind, table := range f.tables {
items := atomic.LoadUint64(&table.items)
if length != items {
return fmt.Errorf("freezer tables %s and %s have differing lengths: %d != %d", kind, name, items, length)
}
}
atomic.StoreUint64(&f.frozen, length)
return nil
}
// repair truncates all data tables to the same length.
func (f *freezer) repair() error {
var (
head = uint64(math.MaxUint64)
tail = uint64(0)
)
for _, table := range f.tables {
items := atomic.LoadUint64(&table.items)
if head > items {
head = items
}
hidden := atomic.LoadUint64(&table.itemHidden)
if hidden > tail {
tail = hidden
}
}
for _, table := range f.tables {
if err := table.truncateHead(head); err != nil {
return err
}
if err := table.truncateTail(tail); err != nil {
return err
}
}
atomic.StoreUint64(&f.frozen, head)
atomic.StoreUint64(&f.tail, tail)
return nil
}
// freeze is a background thread that periodically checks the blockchain for any
// import progress and moves ancient data from the fast database into the freezer.
//
// This functionality is deliberately broken off from block importing to avoid
// incurring additional data shuffling delays on block propagation.
func (f *freezer) freeze(db ethdb.KeyValueStore) {
nfdb := &nofreezedb{KeyValueStore: db}
var (
backoff bool
triggered chan struct{} // Used in tests
)
for {
select {
case <-f.quit:
log.Info("Freezer shutting down")
return
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
}
}
// Retrieve the freezing threshold.
hash := ReadHeadBlockHash(nfdb)
if hash == (common.Hash{}) {
log.Debug("Current full block hash unavailable") // new chain, empty database
backoff = true
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 < threshold:
log.Debug("Current full block not old enough", "number", *number, "hash", hash, "delay", threshold)
backoff = true
continue
case *number-threshold <= f.frozen:
log.Debug("Ancient blocks frozen already", "number", *number, "hash", hash, "frozen", f.frozen)
backoff = true
continue
}
head := ReadHeader(nfdb, hash, *number)
if head == nil {
log.Error("Current full block unavailable", "number", *number, "hash", hash)
backoff = true
continue
}
// Seems we have data ready to be frozen, process in usable batches
var (
start = time.Now()
first, _ = f.Ancients()
limit = *number - threshold
)
if limit-first > freezerBatchLimit {
limit = first + freezerBatchLimit
}
ancients, err := f.freezeRange(nfdb, first, limit)
if err != nil {
log.Error("Error in block freeze operation", "err", err)
backoff = true
continue
}
// Batch of blocks have been frozen, flush them before wiping from leveldb
if err := f.Sync(); err != nil {
log.Crit("Failed to flush frozen tables", "err", err)
}
// Wipe out all data from the active database
batch := db.NewBatch()
for i := 0; i < len(ancients); i++ {
// Always keep the genesis block in active database
if first+uint64(i) != 0 {
DeleteBlockWithoutNumber(batch, ancients[i], first+uint64(i))
DeleteCanonicalHash(batch, first+uint64(i))
}
}
if err := batch.Write(); err != nil {
log.Crit("Failed to delete frozen canonical blocks", "err", err)
}
batch.Reset()
// Wipe out side chains also and track dangling side chains
var dangling []common.Hash
for number := first; number < f.frozen; number++ {
// Always keep the genesis block in active database
if number != 0 {
dangling = ReadAllHashes(db, number)
for _, hash := range dangling {
log.Trace("Deleting side chain", "number", number, "hash", hash)
DeleteBlock(batch, hash, number)
}
}
}
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,
}
if n := len(ancients); n > 0 {
context = append(context, []interface{}{"hash", ancients[n-1]}...)
}
log.Info("Deep froze chain segment", context...)
// Avoid database thrashing with tiny writes
if f.frozen-first < freezerBatchLimit {
backoff = true
}
}
}
func (f *freezer) freezeRange(nfdb *nofreezedb, number, limit uint64) (hashes []common.Hash, err error) {
hashes = make([]common.Hash, 0, limit-number)
_, err = f.ModifyAncients(func(op ethdb.AncientWriteOp) error {
for ; number <= limit; number++ {
// Retrieve all the components of the canonical block.
hash := ReadCanonicalHash(nfdb, number)
if hash == (common.Hash{}) {
return fmt.Errorf("canonical hash missing, can't freeze block %d", number)
}
header := ReadHeaderRLP(nfdb, hash, number)
if len(header) == 0 {
return fmt.Errorf("block header missing, can't freeze block %d", number)
}
body := ReadBodyRLP(nfdb, hash, number)
if len(body) == 0 {
return fmt.Errorf("block body missing, can't freeze block %d", number)
}
receipts := ReadReceiptsRLP(nfdb, hash, number)
if len(receipts) == 0 {
return fmt.Errorf("block receipts missing, can't freeze block %d", number)
}
td := ReadTdRLP(nfdb, hash, number)
if len(td) == 0 {
return fmt.Errorf("total difficulty missing, can't freeze block %d", number)
}
// Write to the batch.
if err := op.AppendRaw(freezerHashTable, number, hash[:]); err != nil {
return fmt.Errorf("can't write hash to freezer: %v", err)
}
if err := op.AppendRaw(freezerHeaderTable, number, header); err != nil {
return fmt.Errorf("can't write header to freezer: %v", err)
}
if err := op.AppendRaw(freezerBodiesTable, number, body); err != nil {
return fmt.Errorf("can't write body to freezer: %v", err)
}
if err := op.AppendRaw(freezerReceiptTable, number, receipts); err != nil {
return fmt.Errorf("can't write receipts to freezer: %v", err)
}
if err := op.AppendRaw(freezerDifficultyTable, number, td); err != nil {
return fmt.Errorf("can't write td to freezer: %v", err)
}
hashes = append(hashes, hash)
}
return nil
})
return hashes, err
}