2020-11-26 14:51:16 +00:00
|
|
|
package splitstore
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2020-11-24 17:41:07 +00:00
|
|
|
"encoding/binary"
|
2020-11-24 14:51:00 +00:00
|
|
|
"errors"
|
2020-11-25 07:10:58 +00:00
|
|
|
"sync"
|
2020-11-29 13:10:30 +00:00
|
|
|
"sync/atomic"
|
2020-11-25 08:11:42 +00:00
|
|
|
"time"
|
2020-11-24 14:51:00 +00:00
|
|
|
|
2021-02-26 13:59:36 +00:00
|
|
|
"golang.org/x/xerrors"
|
|
|
|
|
2021-01-25 19:28:38 +00:00
|
|
|
"github.com/ledgerwatch/lmdb-go/lmdb"
|
2020-11-26 15:49:47 +00:00
|
|
|
|
2020-11-24 14:51:00 +00:00
|
|
|
blocks "github.com/ipfs/go-block-format"
|
|
|
|
cid "github.com/ipfs/go-cid"
|
2020-11-24 17:41:07 +00:00
|
|
|
dstore "github.com/ipfs/go-datastore"
|
2020-11-26 14:51:16 +00:00
|
|
|
logging "github.com/ipfs/go-log/v2"
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
"github.com/filecoin-project/go-state-types/abi"
|
2020-11-24 17:15:38 +00:00
|
|
|
"github.com/filecoin-project/lotus/build"
|
2020-11-26 14:51:16 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/store"
|
2020-11-24 14:51:00 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/types"
|
2020-11-25 07:07:48 +00:00
|
|
|
bstore "github.com/filecoin-project/lotus/lib/blockstore"
|
2020-11-24 14:51:00 +00:00
|
|
|
)
|
|
|
|
|
2021-02-26 14:59:03 +00:00
|
|
|
const (
|
|
|
|
CompactionThreshold = 5 * build.Finality
|
|
|
|
CompactionCold = build.Finality
|
|
|
|
)
|
2020-11-24 17:26:28 +00:00
|
|
|
|
2020-11-24 17:41:07 +00:00
|
|
|
var baseEpochKey = dstore.NewKey("baseEpoch")
|
|
|
|
|
2020-11-26 14:51:16 +00:00
|
|
|
var log = logging.Logger("splitstore")
|
|
|
|
|
2021-01-13 12:21:35 +00:00
|
|
|
func init() {
|
|
|
|
// TODO temporary for debugging purposes; to be removed for merge.
|
|
|
|
logging.SetLogLevel("splitstore", "DEBUG")
|
|
|
|
}
|
|
|
|
|
2021-02-27 13:20:14 +00:00
|
|
|
type Config struct {
|
2021-02-28 07:59:11 +00:00
|
|
|
// TrackingStore type; bolt (default) or lmdb
|
|
|
|
TrackingStoreType string
|
|
|
|
// LiveSet type; bloom (default), bolt, or lmdb
|
|
|
|
LiveSetType string
|
2021-02-27 13:20:14 +00:00
|
|
|
// perform full reachability analysis (expensive) for compaction
|
|
|
|
// You should enable this option if you plan to use the splitstore without a backing coldstore
|
|
|
|
EnableFullCompaction bool
|
|
|
|
// EXPERIMENTAL enable pruning of unreachable objects.
|
|
|
|
// This has not been sufficiently tested yet; only enable if you know what you are doing.
|
|
|
|
// Only applies if you enable full compaction.
|
|
|
|
EnableGC bool
|
|
|
|
// full archival nodes should enable this if EnableFullCompaction is enabled
|
|
|
|
// do NOT enable this if you synced from a snapshot.
|
|
|
|
// Only applies if you enabled full compaction
|
|
|
|
Archival bool
|
|
|
|
}
|
|
|
|
|
2020-11-24 14:51:00 +00:00
|
|
|
type SplitStore struct {
|
2020-11-29 13:10:30 +00:00
|
|
|
compacting int32
|
|
|
|
|
2021-02-27 13:20:14 +00:00
|
|
|
fullCompaction bool
|
|
|
|
enableGC bool
|
|
|
|
skipOldMsgs bool
|
|
|
|
skipMsgReceipts bool
|
2021-02-26 10:47:27 +00:00
|
|
|
|
2020-11-29 13:10:30 +00:00
|
|
|
baseEpoch abi.ChainEpoch
|
|
|
|
|
|
|
|
mx sync.Mutex
|
|
|
|
curTs *types.TipSet
|
2020-11-24 14:51:00 +00:00
|
|
|
|
2020-12-01 15:17:34 +00:00
|
|
|
cs *store.ChainStore
|
|
|
|
ds dstore.Datastore
|
|
|
|
hot bstore.Blockstore
|
|
|
|
cold bstore.Blockstore
|
2020-11-24 14:51:00 +00:00
|
|
|
snoop TrackingStore
|
2020-11-24 17:26:28 +00:00
|
|
|
|
2021-02-27 10:01:55 +00:00
|
|
|
env LiveSetEnv
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
2020-11-25 07:07:48 +00:00
|
|
|
var _ bstore.Blockstore = (*SplitStore)(nil)
|
2020-11-24 14:51:00 +00:00
|
|
|
|
2020-12-01 15:17:34 +00:00
|
|
|
// NewSplitStore creates a new SplitStore instance, given a path for the hotstore dbs and a cold
|
|
|
|
// blockstore. The SplitStore must be attached to the ChainStore with Start in order to trigger
|
|
|
|
// compaction.
|
2021-02-27 13:20:14 +00:00
|
|
|
func NewSplitStore(path string, ds dstore.Datastore, cold, hot bstore.Blockstore, cfg *Config) (*SplitStore, error) {
|
2020-12-01 15:17:34 +00:00
|
|
|
// the tracking store
|
2021-02-28 07:59:11 +00:00
|
|
|
snoop, err := NewTrackingStore(path, cfg.TrackingStoreType)
|
2020-12-01 15:17:34 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
// the liveset env
|
2021-02-28 07:59:11 +00:00
|
|
|
env, err := NewLiveSetEnv(path, cfg.LiveSetType)
|
2021-02-27 16:27:58 +00:00
|
|
|
if err != nil {
|
|
|
|
snoop.Close() //nolint:errcheck
|
|
|
|
return nil, err
|
2020-12-01 15:17:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// and now we can make a SplitStore
|
|
|
|
ss := &SplitStore{
|
2021-02-27 13:20:14 +00:00
|
|
|
ds: ds,
|
|
|
|
hot: hot,
|
|
|
|
cold: cold,
|
|
|
|
snoop: snoop,
|
|
|
|
env: env,
|
|
|
|
|
|
|
|
fullCompaction: cfg.EnableFullCompaction,
|
|
|
|
enableGC: cfg.EnableGC,
|
2021-02-27 19:30:01 +00:00
|
|
|
skipOldMsgs: !(cfg.EnableFullCompaction && cfg.Archival),
|
|
|
|
skipMsgReceipts: !(cfg.EnableFullCompaction && cfg.Archival),
|
2020-12-01 15:17:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return ss, nil
|
|
|
|
}
|
|
|
|
|
2020-11-24 14:51:00 +00:00
|
|
|
// Blockstore interface
|
|
|
|
func (s *SplitStore) DeleteBlock(cid cid.Cid) error {
|
|
|
|
// afaict we don't seem to be using this method, so it's not implemented
|
2020-11-24 22:01:10 +00:00
|
|
|
return errors.New("DeleteBlock not implemented on SplitStore; don't do this Luke!") //nolint
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) Has(cid cid.Cid) (bool, error) {
|
|
|
|
has, err := s.hot.Has(cid)
|
|
|
|
|
2020-11-29 11:00:36 +00:00
|
|
|
if err != nil || has {
|
|
|
|
return has, err
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return s.cold.Has(cid)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) Get(cid cid.Cid) (blocks.Block, error) {
|
|
|
|
blk, err := s.hot.Get(cid)
|
|
|
|
|
|
|
|
switch err {
|
|
|
|
case nil:
|
|
|
|
return blk, nil
|
|
|
|
|
|
|
|
case bstore.ErrNotFound:
|
|
|
|
return s.cold.Get(cid)
|
|
|
|
|
|
|
|
default:
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) GetSize(cid cid.Cid) (int, error) {
|
|
|
|
size, err := s.hot.GetSize(cid)
|
|
|
|
|
|
|
|
switch err {
|
|
|
|
case nil:
|
|
|
|
return size, nil
|
|
|
|
|
|
|
|
case bstore.ErrNotFound:
|
|
|
|
return s.cold.GetSize(cid)
|
|
|
|
|
|
|
|
default:
|
|
|
|
return 0, err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) Put(blk blocks.Block) error {
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Lock()
|
2020-12-01 15:56:22 +00:00
|
|
|
if s.curTs == nil {
|
|
|
|
s.mx.Unlock()
|
|
|
|
return s.cold.Put(blk)
|
|
|
|
}
|
|
|
|
|
2020-11-24 17:15:38 +00:00
|
|
|
epoch := s.curTs.Height()
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Unlock()
|
|
|
|
|
2020-11-24 17:15:38 +00:00
|
|
|
err := s.snoop.Put(blk.Cid(), epoch)
|
2020-12-01 16:44:39 +00:00
|
|
|
if err != nil && !lmdb.IsErrno(err, lmdb.KeyExist) {
|
2020-11-29 10:50:59 +00:00
|
|
|
log.Errorf("error tracking CID in hotstore: %s; falling back to coldstore", err)
|
|
|
|
return s.cold.Put(blk)
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
2020-11-24 17:15:38 +00:00
|
|
|
return s.hot.Put(blk)
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) PutMany(blks []blocks.Block) error {
|
2020-12-01 15:56:22 +00:00
|
|
|
s.mx.Lock()
|
|
|
|
if s.curTs == nil {
|
|
|
|
s.mx.Unlock()
|
2020-11-29 10:50:59 +00:00
|
|
|
return s.cold.PutMany(blks)
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
epoch := s.curTs.Height()
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Unlock()
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
batch := make([]cid.Cid, 0, len(blks))
|
|
|
|
for _, blk := range blks {
|
|
|
|
batch = append(batch, blk.Cid())
|
|
|
|
}
|
|
|
|
|
2020-12-01 16:21:46 +00:00
|
|
|
err := s.snoop.PutBatch(batch, epoch)
|
|
|
|
if err != nil {
|
|
|
|
if lmdb.IsErrno(err, lmdb.KeyExist) {
|
|
|
|
// a write is duplicate, but we don't know which; write each block separately
|
|
|
|
for _, blk := range blks {
|
|
|
|
err = s.Put(blk)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Errorf("error tracking CIDs in hotstore: %s; falling back to coldstore", err)
|
|
|
|
return s.cold.PutMany(blks)
|
|
|
|
}
|
|
|
|
|
|
|
|
return s.hot.PutMany(blks)
|
2020-11-24 14:51:00 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) AllKeysChan(ctx context.Context) (<-chan cid.Cid, error) {
|
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
|
|
|
|
chHot, err := s.hot.AllKeysChan(ctx)
|
|
|
|
if err != nil {
|
2020-11-24 22:01:10 +00:00
|
|
|
cancel()
|
2020-11-24 14:51:00 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
chCold, err := s.cold.AllKeysChan(ctx)
|
|
|
|
if err != nil {
|
|
|
|
cancel()
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
ch := make(chan cid.Cid)
|
|
|
|
go func() {
|
|
|
|
defer cancel()
|
2020-11-25 07:07:06 +00:00
|
|
|
defer close(ch)
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
for _, in := range []<-chan cid.Cid{chHot, chCold} {
|
|
|
|
for cid := range in {
|
|
|
|
select {
|
|
|
|
case ch <- cid:
|
|
|
|
case <-ctx.Done():
|
|
|
|
return
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
return ch, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) HashOnRead(enabled bool) {
|
|
|
|
s.hot.HashOnRead(enabled)
|
|
|
|
s.cold.HashOnRead(enabled)
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) View(cid cid.Cid, cb func([]byte) error) error {
|
|
|
|
err := s.hot.View(cid, cb)
|
|
|
|
switch err {
|
|
|
|
case bstore.ErrNotFound:
|
|
|
|
return s.cold.View(cid, cb)
|
|
|
|
|
|
|
|
default:
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
}
|
2020-11-24 17:15:38 +00:00
|
|
|
|
2020-11-24 17:26:28 +00:00
|
|
|
// State tracking
|
2020-11-26 14:51:16 +00:00
|
|
|
func (s *SplitStore) Start(cs *store.ChainStore) error {
|
2020-11-24 17:26:28 +00:00
|
|
|
s.cs = cs
|
|
|
|
s.curTs = cs.GetHeaviestTipSet()
|
2020-11-24 17:41:07 +00:00
|
|
|
|
|
|
|
// load base epoch from metadata ds
|
|
|
|
// if none, then use current epoch because it's a fresh start
|
|
|
|
bs, err := s.ds.Get(baseEpochKey)
|
|
|
|
switch err {
|
|
|
|
case nil:
|
2020-11-26 18:37:02 +00:00
|
|
|
s.baseEpoch = bytesToEpoch(bs)
|
2020-11-24 17:41:07 +00:00
|
|
|
|
|
|
|
case dstore.ErrNotFound:
|
2020-12-01 15:56:22 +00:00
|
|
|
if s.curTs == nil {
|
|
|
|
// this can happen in some tests
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
2020-11-24 17:41:07 +00:00
|
|
|
err = s.setBaseEpoch(s.curTs.Height())
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
default:
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
// watch the chain
|
2020-11-24 17:26:28 +00:00
|
|
|
cs.SubscribeHeadChanges(s.HeadChange)
|
2020-11-24 17:41:07 +00:00
|
|
|
|
|
|
|
return nil
|
2020-11-24 17:26:28 +00:00
|
|
|
}
|
|
|
|
|
2020-11-26 15:49:47 +00:00
|
|
|
func (s *SplitStore) Close() error {
|
2020-11-29 13:10:30 +00:00
|
|
|
if atomic.LoadInt32(&s.compacting) == 1 {
|
2020-11-26 15:49:47 +00:00
|
|
|
log.Warn("ongoing compaction; waiting for it to finish...")
|
2020-11-29 13:10:30 +00:00
|
|
|
for atomic.LoadInt32(&s.compacting) == 1 {
|
2020-11-26 15:49:47 +00:00
|
|
|
time.Sleep(time.Second)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-02-27 16:27:58 +00:00
|
|
|
return s.env.Close()
|
2020-11-26 15:49:47 +00:00
|
|
|
}
|
|
|
|
|
2020-11-24 17:26:28 +00:00
|
|
|
func (s *SplitStore) HeadChange(revert, apply []*types.TipSet) error {
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Lock()
|
2020-11-24 17:26:28 +00:00
|
|
|
s.curTs = apply[len(apply)-1]
|
|
|
|
epoch := s.curTs.Height()
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Unlock()
|
|
|
|
|
2020-11-29 13:10:30 +00:00
|
|
|
if !atomic.CompareAndSwapInt32(&s.compacting, 0, 1) {
|
|
|
|
// we are currently compacting, do nothing and wait for the next head change
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if epoch-s.baseEpoch > CompactionThreshold {
|
2020-11-24 17:26:28 +00:00
|
|
|
go func() {
|
2020-11-29 13:10:30 +00:00
|
|
|
defer atomic.StoreInt32(&s.compacting, 0)
|
2020-11-25 08:11:42 +00:00
|
|
|
|
|
|
|
log.Info("compacting splitstore")
|
|
|
|
start := time.Now()
|
|
|
|
|
2020-11-24 17:26:28 +00:00
|
|
|
s.compact()
|
2020-11-25 08:11:42 +00:00
|
|
|
|
|
|
|
log.Infow("compaction done", "took", time.Since(start))
|
2020-11-24 17:26:28 +00:00
|
|
|
}()
|
2020-11-29 13:10:30 +00:00
|
|
|
} else {
|
|
|
|
// no compaction necessary
|
|
|
|
atomic.StoreInt32(&s.compacting, 0)
|
2020-11-24 17:26:28 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-11-24 17:15:38 +00:00
|
|
|
// Compaction/GC Algorithm
|
|
|
|
func (s *SplitStore) compact() {
|
2021-02-27 13:20:14 +00:00
|
|
|
if s.fullCompaction {
|
|
|
|
s.compactFull()
|
|
|
|
} else {
|
|
|
|
s.compactSimple()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) compactSimple() {
|
2021-02-27 16:47:13 +00:00
|
|
|
s.mx.Lock()
|
|
|
|
curTs := s.curTs
|
|
|
|
s.mx.Unlock()
|
2021-02-27 13:20:14 +00:00
|
|
|
|
|
|
|
coldEpoch := s.baseEpoch + CompactionCold
|
2021-02-27 16:47:13 +00:00
|
|
|
|
2021-02-27 19:46:40 +00:00
|
|
|
log.Infow("running simple compaction", "currentEpoch", curTs.Height(), "baseEpoch", s.baseEpoch, "coldEpoch", coldEpoch)
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-02-27 16:27:58 +00:00
|
|
|
coldSet, err := s.env.NewLiveSet("cold")
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
defer coldSet.Close() //nolint:errcheck
|
|
|
|
|
|
|
|
// 1. mark reachable cold objects by looking at the objects reachable only from the cold epoch
|
2021-02-27 16:47:13 +00:00
|
|
|
log.Info("marking reachable cold objects")
|
2021-02-27 16:27:58 +00:00
|
|
|
startMark := time.Now()
|
|
|
|
|
|
|
|
coldTs, err := s.cs.GetTipsetByHeight(context.Background(), coldEpoch, curTs, true)
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
err = s.cs.WalkSnapshot(context.Background(), coldTs, 1, s.skipOldMsgs, s.skipMsgReceipts,
|
|
|
|
func(cid cid.Cid) error {
|
|
|
|
return coldSet.Mark(cid)
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Infow("marking done", "took", time.Since(startMark))
|
|
|
|
|
|
|
|
// 2. move cold unreachable objects to the coldstore
|
2021-02-27 13:20:14 +00:00
|
|
|
log.Info("collecting cold objects")
|
|
|
|
startCollect := time.Now()
|
|
|
|
|
2021-02-27 16:47:13 +00:00
|
|
|
cold := make(map[cid.Cid]struct{})
|
|
|
|
|
|
|
|
// some stats for logging
|
|
|
|
var stHot, stCold int
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.1 iterate through the snoop and collect unreachable cold objects
|
2021-02-27 16:27:58 +00:00
|
|
|
err = s.snoop.ForEach(func(cid cid.Cid, wrEpoch abi.ChainEpoch) error {
|
2021-02-27 13:20:14 +00:00
|
|
|
// is the object stil hot?
|
|
|
|
if wrEpoch > coldEpoch {
|
|
|
|
// yes, stay in the hotstore
|
|
|
|
stHot++
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-02-27 16:27:58 +00:00
|
|
|
// check whether it is reachable in the cold boundary
|
|
|
|
mark, err := coldSet.Has(cid)
|
|
|
|
if err != nil {
|
|
|
|
return xerrors.Errorf("error checkiing cold set for %s: %w", cid, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
if mark {
|
|
|
|
stHot++
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// it's cold, mark it for move
|
2021-02-27 13:20:14 +00:00
|
|
|
cold[cid] = struct{}{}
|
|
|
|
stCold++
|
|
|
|
return nil
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Infow("collection done", "took", time.Since(startCollect))
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Infow("compaction stats", "hot", stHot, "cold", stCold)
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.2 copy the cold objects to the coldstore
|
2021-02-27 13:20:14 +00:00
|
|
|
log.Info("moving cold objects to the coldstore")
|
|
|
|
startMove := time.Now()
|
2021-02-28 08:21:48 +00:00
|
|
|
|
|
|
|
const batchSize = 1024
|
|
|
|
batch := make([]blocks.Block, 0, batchSize)
|
|
|
|
|
2021-02-27 13:20:14 +00:00
|
|
|
for cid := range cold {
|
|
|
|
blk, err := s.hot.Get(cid)
|
|
|
|
if err != nil {
|
|
|
|
if err == dstore.ErrNotFound {
|
|
|
|
// this can happen if the node is killed after we have deleted the block from the hotstore
|
|
|
|
// but before we have deleted it from the snoop; just delete the snoop.
|
|
|
|
err = s.snoop.Delete(cid)
|
|
|
|
if err != nil {
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Errorf("error deleting cid %s from snoop: %s", cid, err)
|
2021-02-27 13:20:14 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
} else {
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Errorf("error retrieving tracked block %s from hotstore: %s", cid, err)
|
2021-02-27 13:20:14 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
batch = append(batch, blk)
|
|
|
|
if len(batch) == batchSize {
|
|
|
|
err = s.cold.PutMany(batch)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error putting cold batch to coldstore: %s", err)
|
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
batch = batch[:0]
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(batch) > 0 {
|
|
|
|
err = s.cold.PutMany(batch)
|
2021-02-27 13:20:14 +00:00
|
|
|
if err != nil {
|
2021-02-28 08:21:48 +00:00
|
|
|
log.Errorf("error putting cold batch to coldstore: %s", err)
|
2021-02-27 13:20:14 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
}
|
|
|
|
log.Infow("moving done", "took", time.Since(startMove))
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.3 delete cold objects from the hotstore
|
|
|
|
// TODO we really want batching for this!
|
|
|
|
log.Info("purging cold objects from the hotstore")
|
|
|
|
purgeStart := time.Now()
|
|
|
|
for cid := range cold {
|
2021-02-27 13:20:14 +00:00
|
|
|
// delete the object from the hotstore
|
|
|
|
err = s.hot.DeleteBlock(cid)
|
|
|
|
if err != nil {
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Errorf("error deleting block %s from hotstore: %s", cid, err)
|
2021-02-27 13:20:14 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
log.Infow("purging cold from hotstore done", "took", time.Since(purgeStart))
|
2021-02-27 21:08:23 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.4 remove the snoop tracking for cold objects
|
|
|
|
purgeStart = time.Now()
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Info("purging cold objects from snoop")
|
2021-02-27 21:08:23 +00:00
|
|
|
|
|
|
|
err = s.snoop.DeleteBatch(cold)
|
|
|
|
if err != nil {
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Errorf("error purging cold objects from snoop: %s", err)
|
2021-02-27 21:08:23 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Infow("purging cold from snoop done", "took", time.Since(purgeStart))
|
2021-02-27 16:27:58 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// we are done; do some housekeeping
|
2021-02-27 16:27:58 +00:00
|
|
|
err = s.snoop.Sync()
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
err = s.setBaseEpoch(coldEpoch)
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) compactFull() {
|
2021-02-27 16:47:13 +00:00
|
|
|
s.mx.Lock()
|
|
|
|
curTs := s.curTs
|
|
|
|
s.mx.Unlock()
|
|
|
|
|
|
|
|
epoch := curTs.Height()
|
|
|
|
coldEpoch := s.baseEpoch + CompactionCold
|
|
|
|
|
2021-02-27 19:46:40 +00:00
|
|
|
log.Infow("running full compaction", "currentEpoch", curTs.Height(), "baseEpoch", s.baseEpoch, "coldEpoch", coldEpoch)
|
2021-02-27 16:47:13 +00:00
|
|
|
|
|
|
|
// create two live sets, one for marking the cold finality region
|
2020-11-24 21:55:57 +00:00
|
|
|
// and one for marking the hot region
|
2021-02-27 10:01:55 +00:00
|
|
|
hotSet, err := s.env.NewLiveSet("hot")
|
2020-11-24 21:55:57 +00:00
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
defer hotSet.Close() //nolint:errcheck
|
|
|
|
|
2021-02-27 10:01:55 +00:00
|
|
|
coldSet, err := s.env.NewLiveSet("cold")
|
2020-11-24 21:55:57 +00:00
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
defer coldSet.Close() //nolint:errcheck
|
|
|
|
|
2020-11-25 08:11:42 +00:00
|
|
|
// Phase 1: marking
|
|
|
|
log.Info("marking live objects")
|
|
|
|
startMark := time.Now()
|
|
|
|
|
2020-11-24 21:55:57 +00:00
|
|
|
// Phase 1a: mark all reachable CIDs in the hot range
|
2021-02-26 15:14:10 +00:00
|
|
|
err = s.cs.WalkSnapshot(context.Background(), curTs, epoch-coldEpoch, s.skipOldMsgs, s.skipMsgReceipts,
|
2020-11-24 17:15:38 +00:00
|
|
|
func(cid cid.Cid) error {
|
2020-11-24 21:55:57 +00:00
|
|
|
return hotSet.Mark(cid)
|
2020-11-24 17:15:38 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2020-11-24 21:55:57 +00:00
|
|
|
// Phase 1b: mark all reachable CIDs in the cold range
|
2020-12-01 19:48:08 +00:00
|
|
|
coldTs, err := s.cs.GetTipsetByHeight(context.Background(), coldEpoch, curTs, true)
|
2020-11-24 21:55:57 +00:00
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2021-02-26 15:14:10 +00:00
|
|
|
err = s.cs.WalkSnapshot(context.Background(), coldTs, CompactionCold, s.skipOldMsgs, s.skipMsgReceipts,
|
2020-11-24 21:55:57 +00:00
|
|
|
func(cid cid.Cid) error {
|
|
|
|
return coldSet.Mark(cid)
|
|
|
|
})
|
2020-11-24 17:15:38 +00:00
|
|
|
|
2020-11-24 21:55:57 +00:00
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2020-11-25 08:11:42 +00:00
|
|
|
log.Infow("marking done", "took", time.Since(startMark))
|
|
|
|
|
2020-11-24 21:55:57 +00:00
|
|
|
// Phase 2: sweep cold objects:
|
|
|
|
// - If a cold object is reachable in the hot range, it stays in the hotstore.
|
|
|
|
// - If a cold object is reachable in the cold range, it is moved to the coldstore.
|
2021-02-26 13:59:36 +00:00
|
|
|
// - If a cold object is unreachable, it is deleted if GC is enabled, otherwise moved to the coldstore.
|
2020-11-25 08:11:42 +00:00
|
|
|
startSweep := time.Now()
|
|
|
|
log.Info("sweeping cold objects")
|
|
|
|
|
|
|
|
// some stats for logging
|
|
|
|
var stHot, stCold, stDead int
|
|
|
|
|
2021-02-26 19:28:16 +00:00
|
|
|
cold := make(map[cid.Cid]struct{})
|
|
|
|
dead := make(map[cid.Cid]struct{})
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.1 iterate through the snoop and collect cold and dead objects
|
2021-02-26 13:59:36 +00:00
|
|
|
err = s.snoop.ForEach(func(cid cid.Cid, wrEpoch abi.ChainEpoch) error {
|
2020-11-24 17:15:38 +00:00
|
|
|
// is the object stil hot?
|
2020-12-01 19:48:08 +00:00
|
|
|
if wrEpoch > coldEpoch {
|
2020-11-24 21:55:57 +00:00
|
|
|
// yes, stay in the hotstore
|
2020-11-25 08:11:42 +00:00
|
|
|
stHot++
|
2021-02-26 13:59:36 +00:00
|
|
|
return nil
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|
|
|
|
|
2020-11-24 21:55:57 +00:00
|
|
|
// the object is cold -- check whether it is reachable in the hot range
|
|
|
|
mark, err := hotSet.Has(cid)
|
2020-11-24 17:15:38 +00:00
|
|
|
if err != nil {
|
2021-02-26 13:59:36 +00:00
|
|
|
return xerrors.Errorf("error checking live mark for %s: %w", cid, err)
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
if mark {
|
2020-11-24 21:55:57 +00:00
|
|
|
// the object is reachable in the hot range, stay in the hotstore
|
2020-11-25 08:11:42 +00:00
|
|
|
stHot++
|
2021-02-26 13:59:36 +00:00
|
|
|
return nil
|
2020-11-24 21:55:57 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// check whether it is reachable in the cold range
|
|
|
|
mark, err = coldSet.Has(cid)
|
|
|
|
if err != nil {
|
2021-02-26 13:59:36 +00:00
|
|
|
return xerrors.Errorf("error checkiing cold set for %s: %w", cid, err)
|
2020-11-24 21:55:57 +00:00
|
|
|
}
|
|
|
|
|
2021-02-26 10:47:27 +00:00
|
|
|
if s.enableGC {
|
|
|
|
if mark {
|
|
|
|
// the object is reachable in the cold range, move it to the cold store
|
2021-02-26 19:28:16 +00:00
|
|
|
cold[cid] = struct{}{}
|
2021-02-26 10:47:27 +00:00
|
|
|
stCold++
|
|
|
|
} else {
|
2021-02-26 19:28:16 +00:00
|
|
|
// the object is dead and will be deleted
|
|
|
|
dead[cid] = struct{}{}
|
2021-02-26 10:47:27 +00:00
|
|
|
stDead++
|
|
|
|
}
|
|
|
|
} else {
|
|
|
|
// if GC is disabled, we move both cold and dead objects to the coldstore
|
2021-02-26 19:28:16 +00:00
|
|
|
cold[cid] = struct{}{}
|
2021-02-26 10:47:27 +00:00
|
|
|
if mark {
|
|
|
|
stCold++
|
|
|
|
} else {
|
|
|
|
stDead++
|
|
|
|
}
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|
|
|
|
|
2021-02-26 19:28:16 +00:00
|
|
|
return nil
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Infow("compaction stats", "hot", stHot, "cold", stCold, "dead", stDead)
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.2 copy the cold objects to the coldstore
|
2021-02-26 19:28:16 +00:00
|
|
|
log.Info("moving cold objects to the coldstore")
|
2021-02-28 08:21:48 +00:00
|
|
|
startMove := time.Now()
|
|
|
|
|
|
|
|
const batchSize = 1024
|
|
|
|
batch := make([]blocks.Block, 0, batchSize)
|
|
|
|
|
2021-02-26 19:28:16 +00:00
|
|
|
for cid := range cold {
|
|
|
|
blk, err := s.hot.Get(cid)
|
|
|
|
if err != nil {
|
|
|
|
if err == dstore.ErrNotFound {
|
|
|
|
// this can happen if the node is killed after we have deleted the block from the hotstore
|
|
|
|
// but before we have deleted it from the snoop; just delete the snoop.
|
|
|
|
err = s.snoop.Delete(cid)
|
|
|
|
if err != nil {
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Errorf("error deleting cid %s from snoop: %s", cid, err)
|
2021-02-26 19:28:16 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
} else {
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Errorf("error retrieving tracked block %s from hotstore: %s", cid, err)
|
2021-02-26 19:28:16 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
batch = append(batch, blk)
|
|
|
|
if len(batch) == batchSize {
|
|
|
|
err = s.cold.PutMany(batch)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error putting cold batch to coldstore: %s", err)
|
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
batch = batch[:0]
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(batch) > 0 {
|
|
|
|
err = s.cold.PutMany(batch)
|
2021-02-26 19:28:16 +00:00
|
|
|
if err != nil {
|
2021-02-28 08:21:48 +00:00
|
|
|
log.Errorf("error putting cold batch to coldstore: %s", err)
|
2021-02-26 19:28:16 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
}
|
|
|
|
log.Infow("moving done", "took", time.Since(startMove))
|
2021-02-26 19:28:16 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.3 delete cold objects from the hotstore
|
|
|
|
// TODO we really want batching for this!
|
|
|
|
log.Info("purging cold objects from the hotstore")
|
|
|
|
purgeStart := time.Now()
|
|
|
|
for cid := range cold {
|
2020-11-24 17:15:38 +00:00
|
|
|
// delete the object from the hotstore
|
|
|
|
err = s.hot.DeleteBlock(cid)
|
|
|
|
if err != nil {
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Errorf("error deleting block %s from hotstore: %s", cid, err)
|
2021-02-26 19:28:16 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|
2021-02-27 21:08:23 +00:00
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
log.Infow("purging cold from hotstore done", "took", time.Since(purgeStart))
|
2020-11-24 17:15:38 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.4 remove the snoop tracking for cold objects
|
|
|
|
purgeStart = time.Now()
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Info("purging cold objects from snoop")
|
2021-02-27 21:08:23 +00:00
|
|
|
|
|
|
|
err = s.snoop.DeleteBatch(cold)
|
|
|
|
if err != nil {
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Errorf("error purging cold objects from snoop: %s", err)
|
2021-02-27 21:08:23 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
2021-02-26 19:28:16 +00:00
|
|
|
}
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Infow("purging cold from snoop done", "took", time.Since(purgeStart))
|
2021-02-26 13:59:36 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 3. if we have dead objects, delete them from the hotstore and remove the tracking
|
2021-02-26 19:28:16 +00:00
|
|
|
if len(dead) > 0 {
|
|
|
|
log.Info("deleting dead objects")
|
2021-02-26 13:59:36 +00:00
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
purgeStart = time.Now()
|
|
|
|
log.Info("purging dead objects from the hotstore")
|
|
|
|
// TODO we really want batching for this!
|
2021-02-26 19:28:16 +00:00
|
|
|
for cid := range dead {
|
|
|
|
// delete the object from the hotstore
|
|
|
|
err = s.hot.DeleteBlock(cid)
|
|
|
|
if err != nil {
|
2021-02-27 21:08:23 +00:00
|
|
|
log.Errorf("error deleting block %s from hotstore: %s", cid, err)
|
2021-02-26 19:28:16 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
|
|
|
}
|
2021-02-27 21:08:23 +00:00
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
log.Infow("purging dead from hotstore done", "took", time.Since(purgeStart))
|
2021-02-26 19:28:16 +00:00
|
|
|
|
2021-02-27 21:08:23 +00:00
|
|
|
// remove the snoop tracking
|
|
|
|
purgeStart := time.Now()
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Info("purging dead objects from snoop")
|
2021-02-27 21:08:23 +00:00
|
|
|
|
|
|
|
err = s.snoop.DeleteBatch(dead)
|
|
|
|
if err != nil {
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Errorf("error purging dead objects from snoop: %s", err)
|
2021-02-27 21:08:23 +00:00
|
|
|
// TODO do something better here -- just continue?
|
|
|
|
panic(err)
|
2021-02-26 19:28:16 +00:00
|
|
|
}
|
2021-02-27 21:08:23 +00:00
|
|
|
|
2021-02-28 08:25:35 +00:00
|
|
|
log.Infow("purging dead from snoop done", "took", time.Since(purgeStart))
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|
|
|
|
|
2020-11-25 08:11:42 +00:00
|
|
|
log.Infow("sweeping done", "took", time.Since(startSweep))
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// we are done; do some housekeeping
|
2021-02-27 16:16:09 +00:00
|
|
|
err = s.snoop.Sync()
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
|
2020-11-24 17:41:07 +00:00
|
|
|
err = s.setBaseEpoch(coldEpoch)
|
|
|
|
if err != nil {
|
|
|
|
// TODO do something better here
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) setBaseEpoch(epoch abi.ChainEpoch) error {
|
|
|
|
s.baseEpoch = epoch
|
|
|
|
// write to datastore
|
2020-11-26 18:37:02 +00:00
|
|
|
return s.ds.Put(baseEpochKey, epochToBytes(epoch))
|
|
|
|
}
|
|
|
|
|
|
|
|
func epochToBytes(epoch abi.ChainEpoch) []byte {
|
|
|
|
buf := make([]byte, 16)
|
|
|
|
n := binary.PutUvarint(buf, uint64(epoch))
|
|
|
|
return buf[:n]
|
|
|
|
}
|
|
|
|
|
|
|
|
func bytesToEpoch(buf []byte) abi.ChainEpoch {
|
|
|
|
epoch, n := binary.Uvarint(buf)
|
|
|
|
if n < 0 {
|
|
|
|
panic("bogus base epoch bytes")
|
|
|
|
}
|
|
|
|
return abi.ChainEpoch(epoch)
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|