core/rawdb: untie freezer and ancient chain data (#24684)

Previously freezer has only been used for storing ancient chain data, while obviously it can be used more. This PR unties the chain data and freezer, keep the minimal freezer structure and move all other logic (like incrementally freezing block data) into a separate structure called ChainFreezer.

This PR also extends the database interface by adding a new ancient store function AncientDatadir which can return the root directory of ancient store. The ancient root directory can be used when we want to open some other ancient-stores (e.g. reverse diff freezer).
This commit is contained in:
rjl493456442 2022-05-06 19:28:42 +08:00 committed by GitHub
parent cef1a86df2
commit 1941c5e6c9
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
9 changed files with 401 additions and 310 deletions

View File

@ -289,7 +289,7 @@ func inspect(ctx *cli.Context) error {
return rawdb.InspectDatabase(db, prefix, start)
}
func showLeveldbStats(db ethdb.Stater) {
func showLeveldbStats(db ethdb.KeyValueStater) {
if stats, err := db.Stat("leveldb.stats"); err != nil {
log.Warn("Failed to read database stats", "error", err)
} else {

View File

@ -36,7 +36,7 @@ import (
// ReadCanonicalHash retrieves the hash assigned to a canonical block number.
func ReadCanonicalHash(db ethdb.Reader, number uint64) common.Hash {
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
data, _ = reader.Ancient(freezerHashTable, number)
if len(data) == 0 {
// Get it by hash from leveldb
@ -332,7 +332,7 @@ func ReadHeaderRange(db ethdb.Reader, number uint64, count uint64) []rlp.RawValu
// ReadHeaderRLP retrieves a block header in its raw RLP database encoding.
func ReadHeaderRLP(db ethdb.Reader, hash common.Hash, number uint64) rlp.RawValue {
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
// First try to look up the data in ancient database. Extra hash
// comparison is necessary since ancient database only maintains
// the canonical data.
@ -411,7 +411,7 @@ func deleteHeaderWithoutNumber(db ethdb.KeyValueWriter, hash common.Hash, number
// isCanon is an internal utility method, to check whether the given number/hash
// is part of the ancient (canon) set.
func isCanon(reader ethdb.AncientReader, number uint64, hash common.Hash) bool {
func isCanon(reader ethdb.AncientReaderOp, number uint64, hash common.Hash) bool {
h, err := reader.Ancient(freezerHashTable, number)
if err != nil {
return false
@ -425,7 +425,7 @@ func ReadBodyRLP(db ethdb.Reader, hash common.Hash, number uint64) rlp.RawValue
// comparison is necessary since ancient database only maintains
// the canonical data.
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
// Check if the data is in ancients
if isCanon(reader, number, hash) {
data, _ = reader.Ancient(freezerBodiesTable, number)
@ -442,7 +442,7 @@ func ReadBodyRLP(db ethdb.Reader, hash common.Hash, number uint64) rlp.RawValue
// block at number, in RLP encoding.
func ReadCanonicalBodyRLP(db ethdb.Reader, number uint64) rlp.RawValue {
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
data, _ = reader.Ancient(freezerBodiesTable, number)
if len(data) > 0 {
return nil
@ -508,7 +508,7 @@ func DeleteBody(db ethdb.KeyValueWriter, hash common.Hash, number uint64) {
// ReadTdRLP retrieves a block's total difficulty corresponding to the hash in RLP encoding.
func ReadTdRLP(db ethdb.Reader, hash common.Hash, number uint64) rlp.RawValue {
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
// Check if the data is in ancients
if isCanon(reader, number, hash) {
data, _ = reader.Ancient(freezerDifficultyTable, number)
@ -568,7 +568,7 @@ func HasReceipts(db ethdb.Reader, hash common.Hash, number uint64) bool {
// ReadReceiptsRLP retrieves all the transaction receipts belonging to a block in RLP encoding.
func ReadReceiptsRLP(db ethdb.Reader, hash common.Hash, number uint64) rlp.RawValue {
var data []byte
db.ReadAncients(func(reader ethdb.AncientReader) error {
db.ReadAncients(func(reader ethdb.AncientReaderOp) error {
// Check if the data is in ancients
if isCanon(reader, number, hash) {
data, _ = reader.Ancient(freezerReceiptTable, number)

303
core/rawdb/chain_freezer.go Normal file
View File

@ -0,0 +1,303 @@
// Copyright 2022 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 (
"fmt"
"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/params"
)
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
)
// chainFreezer is a wrapper of freezer with additional chain freezing feature.
// The background thread will keep moving ancient chain segments from key-value
// database to flat files for saving space on live database.
type chainFreezer struct {
// WARNING: The `threshold` 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).
threshold uint64 // Number of recent blocks not to freeze (params.FullImmutabilityThreshold apart from tests)
*Freezer
quit chan struct{}
wg sync.WaitGroup
trigger chan chan struct{} // Manual blocking freeze trigger, test determinism
}
// newChainFreezer initializes the freezer for ancient chain data.
func newChainFreezer(datadir string, namespace string, readonly bool, maxTableSize uint32, tables map[string]bool) (*chainFreezer, error) {
freezer, err := NewFreezer(datadir, namespace, readonly, maxTableSize, tables)
if err != nil {
return nil, err
}
return &chainFreezer{
Freezer: freezer,
threshold: params.FullImmutabilityThreshold,
quit: make(chan struct{}),
trigger: make(chan chan struct{}),
}, nil
}
// Close closes the chain freezer instance and terminates the background thread.
func (f *chainFreezer) Close() error {
err := f.Freezer.Close()
select {
case <-f.quit:
default:
close(f.quit)
}
f.wg.Wait()
return err
}
// 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 *chainFreezer) 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)
frozen := atomic.LoadUint64(&f.frozen)
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 <= frozen:
log.Debug("Ancient blocks frozen already", "number", *number, "hash", hash, "frozen", 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
frozen = atomic.LoadUint64(&f.frozen) // Needs reload after during freezeRange
for number := first; number < 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 frozen > 0 {
tip := 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", frozen - first, "elapsed", common.PrettyDuration(time.Since(start)), "number", 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 frozen-first < freezerBatchLimit {
backoff = true
}
}
}
func (f *chainFreezer) 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
}

View File

@ -58,18 +58,18 @@ func (frdb *freezerdb) Close() error {
// a freeze cycle completes, without having to sleep for a minute to trigger the
// automatic background run.
func (frdb *freezerdb) Freeze(threshold uint64) error {
if frdb.AncientStore.(*freezer).readonly {
if frdb.AncientStore.(*chainFreezer).readonly {
return errReadOnly
}
// 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)
atomic.StoreUint64(&frdb.AncientStore.(*chainFreezer).threshold, old)
}(atomic.LoadUint64(&frdb.AncientStore.(*chainFreezer).threshold))
atomic.StoreUint64(&frdb.AncientStore.(*chainFreezer).threshold, threshold)
// Trigger a freeze cycle and block until it's done
trigger := make(chan struct{}, 1)
frdb.AncientStore.(*freezer).trigger <- trigger
frdb.AncientStore.(*chainFreezer).trigger <- trigger
<-trigger
return nil
}
@ -129,7 +129,7 @@ func (db *nofreezedb) Sync() error {
return errNotSupported
}
func (db *nofreezedb) ReadAncients(fn func(reader ethdb.AncientReader) error) (err error) {
func (db *nofreezedb) ReadAncients(fn func(reader ethdb.AncientReaderOp) error) (err error) {
// Unlike other ancient-related methods, this method does not return
// errNotSupported when invoked.
// The reason for this is that the caller might want to do several things:
@ -151,6 +151,11 @@ func (db *nofreezedb) MigrateTable(kind string, convert convertLegacyFn) error {
return errNotSupported
}
// AncientDatadir returns an error as we don't have a backing chain freezer.
func (db *nofreezedb) AncientDatadir() (string, error) {
return "", errNotSupported
}
// NewDatabase creates a high level database on top of a given key-value data
// store without a freezer moving immutable chain segments into cold storage.
func NewDatabase(db ethdb.KeyValueStore) ethdb.Database {
@ -162,7 +167,7 @@ func NewDatabase(db ethdb.KeyValueStore) ethdb.Database {
// storage.
func NewDatabaseWithFreezer(db ethdb.KeyValueStore, freezer string, namespace string, readonly bool) (ethdb.Database, error) {
// Create the idle freezer instance
frdb, err := newFreezer(freezer, namespace, readonly, freezerTableSize, FreezerNoSnappy)
frdb, err := newChainFreezer(freezer, namespace, readonly, freezerTableSize, FreezerNoSnappy)
if err != nil {
return nil, err
}

View File

@ -31,7 +31,6 @@ import (
"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"
)
@ -53,34 +52,24 @@ var (
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
// freezerTableSize defines the maximum size of freezer data files.
const freezerTableSize = 2 * 1000 * 1000 * 1000
// 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:
// Freezer is a memory mapped append-only database to store immutable ordered
// data into flat files:
//
// - The append only nature ensures that disk writes are minimized.
// - 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
type Freezer struct {
// WARNING: The `frozen` and `tail` fields are 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)
frozen uint64 // Number of blocks already frozen
tail uint64 // Number of the first stored item in the freezer
datadir string // Path of root directory of ancient store
// This lock synchronizes writers and the truncate operation, as well as
// the "atomic" (batched) read operations.
@ -90,20 +79,15 @@ type freezer struct {
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
closeOnce sync.Once
}
// newFreezer creates a chain freezer that moves ancient chain data into
// append-only flat file containers.
// NewFreezer creates a freezer instance for maintaining immutable ordered
// data according to the given parameters.
//
// 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) {
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)
@ -124,13 +108,11 @@ func newFreezer(datadir string, namespace string, readonly bool, maxTableSize ui
return nil, err
}
// Open all the supported data tables
freezer := &freezer{
freezer := &Freezer{
readonly: readonly,
threshold: params.FullImmutabilityThreshold,
tables: make(map[string]*freezerTable),
instanceLock: lock,
trigger: make(chan chan struct{}),
quit: make(chan struct{}),
datadir: datadir,
}
// Create the tables.
@ -170,15 +152,12 @@ func newFreezer(datadir string, namespace string, readonly bool, maxTableSize ui
}
// Close terminates the chain freezer, unmapping all the data files.
func (f *freezer) Close() error {
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)
@ -196,7 +175,7 @@ func (f *freezer) Close() error {
// HasAncient returns an indicator whether the specified ancient data exists
// in the freezer.
func (f *freezer) HasAncient(kind string, number uint64) (bool, error) {
func (f *Freezer) HasAncient(kind string, number uint64) (bool, error) {
if table := f.tables[kind]; table != nil {
return table.has(number), nil
}
@ -204,7 +183,7 @@ func (f *freezer) HasAncient(kind string, number uint64) (bool, error) {
}
// Ancient retrieves an ancient binary blob from the append-only immutable files.
func (f *freezer) Ancient(kind string, number uint64) ([]byte, error) {
func (f *Freezer) Ancient(kind string, number uint64) ([]byte, error) {
if table := f.tables[kind]; table != nil {
return table.Retrieve(number)
}
@ -216,7 +195,7 @@ func (f *freezer) Ancient(kind string, number uint64) ([]byte, error) {
// - 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) {
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)
}
@ -224,17 +203,17 @@ func (f *freezer) AncientRange(kind string, start, count, maxBytes uint64) ([][]
}
// Ancients returns the length of the frozen items.
func (f *freezer) Ancients() (uint64, error) {
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) {
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) {
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()
@ -248,14 +227,15 @@ func (f *freezer) AncientSize(kind string) (uint64, error) {
// 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) {
func (f *Freezer) ReadAncients(fn func(ethdb.AncientReaderOp) 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) {
func (f *Freezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (writeSize int64, err error) {
if f.readonly {
return 0, errReadOnly
}
@ -263,7 +243,7 @@ func (f *freezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (writeSize
defer f.writeLock.Unlock()
// Roll back all tables to the starting position in case of error.
prevItem := f.frozen
prevItem := atomic.LoadUint64(&f.frozen)
defer func() {
if err != nil {
// The write operation has failed. Go back to the previous item position.
@ -289,7 +269,7 @@ func (f *freezer) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (writeSize
}
// TruncateHead discards any recent data above the provided threshold number.
func (f *freezer) TruncateHead(items uint64) error {
func (f *Freezer) TruncateHead(items uint64) error {
if f.readonly {
return errReadOnly
}
@ -309,7 +289,7 @@ func (f *freezer) TruncateHead(items uint64) error {
}
// TruncateTail discards any recent data below the provided threshold number.
func (f *freezer) TruncateTail(tail uint64) error {
func (f *Freezer) TruncateTail(tail uint64) error {
if f.readonly {
return errReadOnly
}
@ -329,7 +309,7 @@ func (f *freezer) TruncateTail(tail uint64) error {
}
// Sync flushes all data tables to disk.
func (f *freezer) Sync() error {
func (f *Freezer) Sync() error {
var errs []error
for _, table := range f.tables {
if err := table.Sync(); err != nil {
@ -344,7 +324,7 @@ func (f *freezer) Sync() error {
// validate checks that every table has the same length.
// Used instead of `repair` in readonly mode.
func (f *freezer) validate() error {
func (f *Freezer) validate() error {
if len(f.tables) == 0 {
return nil
}
@ -370,7 +350,7 @@ func (f *freezer) validate() error {
}
// repair truncates all data tables to the same length.
func (f *freezer) repair() error {
func (f *Freezer) repair() error {
var (
head = uint64(math.MaxUint64)
tail = uint64(0)
@ -398,234 +378,13 @@ func (f *freezer) repair() error {
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
}
// convertLegacyFn takes a raw freezer entry in an older format and
// returns it in the new format.
type convertLegacyFn = func([]byte) ([]byte, error)
// MigrateTable processes the entries in a given table in sequence
// converting them to a new format if they're of an old format.
func (f *freezer) MigrateTable(kind string, convert convertLegacyFn) error {
func (f *Freezer) MigrateTable(kind string, convert convertLegacyFn) error {
if f.readonly {
return errReadOnly
}
@ -671,7 +430,7 @@ func (f *freezer) MigrateTable(kind string, convert convertLegacyFn) error {
// Set up new dir for the migrated table, the content of which
// we'll at the end move over to the ancients dir.
migrationPath := filepath.Join(ancientsPath, "migration")
newTable, err := NewFreezerTable(migrationPath, kind, FreezerNoSnappy[kind], false)
newTable, err := NewFreezerTable(migrationPath, kind, table.noCompression, false)
if err != nil {
return err
}
@ -731,3 +490,8 @@ func (f *freezer) MigrateTable(kind string, convert convertLegacyFn) error {
return nil
}
// AncientDatadir returns the root directory path of the ancient store.
func (f *Freezer) AncientDatadir() (string, error) {
return f.datadir, nil
}

View File

@ -34,7 +34,7 @@ type freezerBatch struct {
tables map[string]*freezerTableBatch
}
func newFreezerBatch(f *freezer) *freezerBatch {
func newFreezerBatch(f *Freezer) *freezerBatch {
batch := &freezerBatch{tables: make(map[string]*freezerTableBatch, len(f.tables))}
for kind, table := range f.tables {
batch.tables[kind] = table.newBatch()

View File

@ -113,7 +113,7 @@ func TestFreezerModifyRollback(t *testing.T) {
// Reopen and check that the rolled-back data doesn't reappear.
tables := map[string]bool{"test": true}
f2, err := newFreezer(dir, "", false, 2049, tables)
f2, err := NewFreezer(dir, "", false, 2049, tables)
if err != nil {
t.Fatalf("can't reopen freezer after failed ModifyAncients: %v", err)
}
@ -254,7 +254,7 @@ func TestFreezerReadonlyValidate(t *testing.T) {
dir := t.TempDir()
// Open non-readonly freezer and fill individual tables
// with different amount of data.
f, err := newFreezer(dir, "", false, 2049, tables)
f, err := NewFreezer(dir, "", false, 2049, tables)
if err != nil {
t.Fatal("can't open freezer", err)
}
@ -277,19 +277,19 @@ func TestFreezerReadonlyValidate(t *testing.T) {
// Re-openening as readonly should fail when validating
// table lengths.
f, err = newFreezer(dir, "", true, 2049, tables)
f, err = NewFreezer(dir, "", true, 2049, tables)
if err == nil {
t.Fatal("readonly freezer should fail with differing table lengths")
}
}
func newFreezerForTesting(t *testing.T, tables map[string]bool) (*freezer, string) {
func newFreezerForTesting(t *testing.T, tables map[string]bool) (*Freezer, string) {
t.Helper()
dir := t.TempDir()
// note: using low max table size here to ensure the tests actually
// switch between multiple files.
f, err := newFreezer(dir, "", false, 2049, tables)
f, err := NewFreezer(dir, "", false, 2049, tables)
if err != nil {
t.Fatal("can't open freezer", err)
}
@ -297,7 +297,7 @@ func newFreezerForTesting(t *testing.T, tables map[string]bool) (*freezer, strin
}
// checkAncientCount verifies that the freezer contains n items.
func checkAncientCount(t *testing.T, f *freezer, kind string, n uint64) {
func checkAncientCount(t *testing.T, f *Freezer, kind string, n uint64) {
t.Helper()
if frozen, _ := f.Ancients(); frozen != n {

View File

@ -91,7 +91,7 @@ func (t *table) ModifyAncients(fn func(ethdb.AncientWriteOp) error) (int64, erro
return t.db.ModifyAncients(fn)
}
func (t *table) ReadAncients(fn func(reader ethdb.AncientReader) error) (err error) {
func (t *table) ReadAncients(fn func(reader ethdb.AncientReaderOp) error) (err error) {
return t.db.ReadAncients(fn)
}
@ -119,6 +119,11 @@ func (t *table) MigrateTable(kind string, convert convertLegacyFn) error {
return t.db.MigrateTable(kind, convert)
}
// AncientDatadir returns the ancient datadir of the underlying database.
func (t *table) AncientDatadir() (string, error) {
return t.db.AncientDatadir()
}
// Put inserts the given value into the database at a prefixed version of the
// provided key.
func (t *table) Put(key []byte, value []byte) error {

View File

@ -37,8 +37,8 @@ type KeyValueWriter interface {
Delete(key []byte) error
}
// Stater wraps the Stat method of a backing data store.
type Stater interface {
// KeyValueStater wraps the Stat method of a backing data store.
type KeyValueStater interface {
// Stat returns a particular internal stat of the database.
Stat(property string) (string, error)
}
@ -60,16 +60,16 @@ type Compacter interface {
type KeyValueStore interface {
KeyValueReader
KeyValueWriter
KeyValueStater
Batcher
Iteratee
Stater
Compacter
Snapshotter
io.Closer
}
// AncientReader contains the methods required to read from immutable ancient data.
type AncientReader interface {
// AncientReaderOp contains the methods required to read from immutable ancient data.
type AncientReaderOp interface {
// HasAncient returns an indicator whether the specified data exists in the
// ancient store.
HasAncient(kind string, number uint64) (bool, error)
@ -95,13 +95,13 @@ type AncientReader interface {
AncientSize(kind string) (uint64, error)
}
// AncientBatchReader is the interface for 'batched' or 'atomic' reading.
type AncientBatchReader interface {
AncientReader
// AncientReader is the extended ancient reader interface including 'batched' or 'atomic' reading.
type AncientReader interface {
AncientReaderOp
// ReadAncients runs the given read operation while ensuring that no writes take place
// on the underlying freezer.
ReadAncients(fn func(AncientReader) error) (err error)
ReadAncients(fn func(AncientReaderOp) error) (err error)
}
// AncientWriter contains the methods required to write to immutable ancient data.
@ -140,11 +140,17 @@ type AncientWriteOp interface {
AppendRaw(kind string, number uint64, item []byte) error
}
// AncientStater wraps the Stat method of a backing data store.
type AncientStater interface {
// AncientDatadir returns the root directory path of the ancient store.
AncientDatadir() (string, error)
}
// Reader contains the methods required to read data from both key-value as well as
// immutable ancient data.
type Reader interface {
KeyValueReader
AncientBatchReader
AncientReader
}
// Writer contains the methods required to write data to both key-value as well as
@ -154,11 +160,19 @@ type Writer interface {
AncientWriter
}
// Stater contains the methods required to retrieve states from both key-value as well as
// immutable ancient data.
type Stater interface {
KeyValueStater
AncientStater
}
// AncientStore contains all the methods required to allow handling different
// ancient data stores backing immutable chain data store.
type AncientStore interface {
AncientBatchReader
AncientReader
AncientWriter
AncientStater
io.Closer
}