plugeth/core/rawdb/freezer.go
Martin Holst Swende 794c6133ef
core/rawdb: freezer batch write (#23462)
This change is a rewrite of the freezer code.

When writing ancient chain data to the freezer, the previous version first encoded each
individual item to a temporary buffer, then wrote the buffer. For small item sizes (for
example, in the block hash freezer table), this strategy causes a lot of system calls for
writing tiny chunks of data. It also allocated a lot of temporary []byte buffers.

In the new version, we instead encode multiple items into a re-useable batch buffer, which
is then written to the file all at once. This avoids performing a system call for every
inserted item.

To make the internal batching work, the ancient database API had to be changed. While
integrating this new API in BlockChain.InsertReceiptChain, additional optimizations were
also added there.

Co-authored-by: Felix Lange <fjl@twurst.com>
2021-09-07 12:31:17 +02:00

540 lines
16 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 an 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
threshold uint64 // Number of recent blocks not to freeze (params.FullImmutabilityThreshold apart from tests)
// This lock synchronizes writers and the truncate operation.
writeLock sync.Mutex
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)
if err != nil {
for _, table := range freezer.tables {
table.Close()
}
lock.Release()
return nil, err
}
freezer.tables[name] = table
}
// Truncate all tables to common length.
if err := freezer.repair(); 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
}
// ReadAncients 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) ReadAncients(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
}
// 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.Lock()
defer f.writeLock.Unlock()
if table := f.tables[kind]; table != nil {
return table.size()
}
return 0, errUnknownTable
}
// 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.truncate(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
}
// TruncateAncients discards any recent data above the provided threshold number.
func (f *freezer) TruncateAncients(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.truncate(items); err != nil {
return err
}
}
atomic.StoreUint64(&f.frozen, items)
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
}
// repair truncates all data tables to the same length.
func (f *freezer) repair() error {
min := uint64(math.MaxUint64)
for _, table := range f.tables {
items := atomic.LoadUint64(&table.items)
if min > items {
min = items
}
}
for _, table := range f.tables {
if err := table.truncate(min); err != nil {
return err
}
}
atomic.StoreUint64(&f.frozen, min)
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
}