2020-11-26 14:51:16 +00:00
|
|
|
package splitstore
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
import (
|
2021-03-13 10:00:28 +00:00
|
|
|
"bytes"
|
2020-11-24 14:51:00 +00:00
|
|
|
"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-03-03 08:56:41 +00:00
|
|
|
"go.uber.org/multierr"
|
2021-02-26 13:59:36 +00:00
|
|
|
"golang.org/x/xerrors"
|
|
|
|
|
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"
|
2021-03-13 10:00:28 +00:00
|
|
|
cbg "github.com/whyrusleeping/cbor-gen"
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
"github.com/filecoin-project/go-state-types/abi"
|
2021-03-01 16:15:39 +00:00
|
|
|
|
2021-03-01 07:25:52 +00:00
|
|
|
bstore "github.com/filecoin-project/lotus/blockstore"
|
2020-11-24 17:15:38 +00:00
|
|
|
"github.com/filecoin-project/lotus/build"
|
2020-11-24 14:51:00 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/types"
|
2021-03-05 09:54:06 +00:00
|
|
|
"github.com/filecoin-project/lotus/metrics"
|
|
|
|
|
|
|
|
"go.opencensus.io/stats"
|
2020-11-24 14:51:00 +00:00
|
|
|
)
|
|
|
|
|
2021-02-28 11:51:42 +00:00
|
|
|
var (
|
2021-03-02 00:47:21 +00:00
|
|
|
// CompactionThreshold is the number of epochs that need to have elapsed
|
|
|
|
// from the previously compacted epoch to trigger a new compaction.
|
|
|
|
//
|
|
|
|
// |················· CompactionThreshold ··················|
|
|
|
|
// | |
|
2021-03-03 09:15:26 +00:00
|
|
|
// =======‖≡≡≡≡≡≡≡‖-----------------------|------------------------»
|
|
|
|
// | | | chain --> ↑__ current epoch
|
|
|
|
// |·······| |
|
|
|
|
// ↑________ CompactionCold ↑________ CompactionBoundary
|
2021-03-02 00:47:21 +00:00
|
|
|
//
|
|
|
|
// === :: cold (already archived)
|
|
|
|
// ≡≡≡ :: to be archived in this compaction
|
|
|
|
// --- :: hot
|
2021-03-14 09:06:00 +00:00
|
|
|
CompactionThreshold = 3 * build.Finality
|
2021-03-02 00:47:21 +00:00
|
|
|
|
|
|
|
// CompactionCold is the number of epochs that will be archived to the
|
|
|
|
// cold store on compaction. See diagram on CompactionThreshold for a
|
|
|
|
// better sense.
|
|
|
|
CompactionCold = build.Finality
|
2021-03-03 09:15:26 +00:00
|
|
|
|
|
|
|
// CompactionBoundary is the number of epochs from the current epoch at which
|
2021-03-13 10:00:28 +00:00
|
|
|
// we will walk the chain for live objects.
|
2021-03-03 09:15:26 +00:00
|
|
|
CompactionBoundary = 2 * build.Finality
|
2021-02-26 14:59:03 +00:00
|
|
|
)
|
2020-11-24 17:26:28 +00:00
|
|
|
|
2021-03-01 16:41:51 +00:00
|
|
|
var (
|
2021-03-02 00:47:21 +00:00
|
|
|
// baseEpochKey stores the base epoch (last compaction epoch) in the
|
|
|
|
// metadata store.
|
|
|
|
baseEpochKey = dstore.NewKey("/splitstore/baseEpoch")
|
|
|
|
|
|
|
|
// warmupEpochKey stores whether a hot store warmup has been performed.
|
|
|
|
// On first start, the splitstore will walk the state tree and will copy
|
|
|
|
// all active blocks into the hotstore.
|
2021-03-01 16:41:51 +00:00
|
|
|
warmupEpochKey = dstore.NewKey("/splitstore/warmupEpoch")
|
2021-03-02 08:04:02 +00:00
|
|
|
|
2021-03-05 08:00:17 +00:00
|
|
|
// markSetSizeKey stores the current estimate for the mark set size.
|
|
|
|
// this is first computed at warmup and updated in every compaction
|
|
|
|
markSetSizeKey = dstore.NewKey("/splitstore/markSetSize")
|
|
|
|
|
2021-03-02 08:04:02 +00:00
|
|
|
log = logging.Logger("splitstore")
|
2021-03-01 16:41:51 +00:00
|
|
|
)
|
2020-11-24 17:41:07 +00:00
|
|
|
|
2021-03-02 16:59:00 +00:00
|
|
|
const (
|
|
|
|
batchSize = 16384
|
|
|
|
|
|
|
|
defaultColdPurgeSize = 7_000_000
|
|
|
|
)
|
2021-03-01 18:30:15 +00:00
|
|
|
|
2021-02-27 13:20:14 +00:00
|
|
|
type Config struct {
|
2021-03-02 00:47:21 +00:00
|
|
|
// TrackingStore is the type of tracking store to use.
|
|
|
|
//
|
2021-03-05 17:55:32 +00:00
|
|
|
// Supported values are: "bolt" (default if omitted), "mem" (for tests and readonly access).
|
2021-02-28 07:59:11 +00:00
|
|
|
TrackingStoreType string
|
2021-03-02 00:47:21 +00:00
|
|
|
|
|
|
|
// MarkSetType is the type of mark set to use.
|
|
|
|
//
|
2021-03-02 08:04:02 +00:00
|
|
|
// Supported values are: "bloom" (default if omitted), "bolt".
|
2021-03-02 00:47:21 +00:00
|
|
|
MarkSetType string
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
|
2021-03-01 17:38:02 +00:00
|
|
|
// ChainAccessor allows the Splitstore to access the chain. It will most likely
|
|
|
|
// be a ChainStore at runtime.
|
|
|
|
type ChainAccessor interface {
|
|
|
|
GetTipsetByHeight(context.Context, abi.ChainEpoch, *types.TipSet, bool) (*types.TipSet, error)
|
|
|
|
GetHeaviestTipSet() *types.TipSet
|
|
|
|
SubscribeHeadChanges(change func(revert []*types.TipSet, apply []*types.TipSet) error)
|
|
|
|
}
|
|
|
|
|
2020-11-24 14:51:00 +00:00
|
|
|
type SplitStore struct {
|
2021-03-05 08:11:54 +00:00
|
|
|
compacting int32 // compaction (or warmp up) in progress
|
|
|
|
critsection int32 // compaction critical section
|
|
|
|
closing int32 // the split store is closing
|
2020-11-29 13:10:30 +00:00
|
|
|
|
2021-03-01 16:41:51 +00:00
|
|
|
baseEpoch abi.ChainEpoch
|
|
|
|
warmupEpoch abi.ChainEpoch
|
2021-03-13 10:00:28 +00:00
|
|
|
warm bool
|
2020-11-29 13:10:30 +00:00
|
|
|
|
2021-03-02 16:59:00 +00:00
|
|
|
coldPurgeSize int
|
|
|
|
|
2020-11-29 13:10:30 +00:00
|
|
|
mx sync.Mutex
|
|
|
|
curTs *types.TipSet
|
2020-11-24 14:51:00 +00:00
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
chain ChainAccessor
|
|
|
|
ds dstore.Datastore
|
|
|
|
hot bstore.Blockstore
|
|
|
|
cold bstore.Blockstore
|
|
|
|
tracker TrackingStore
|
2020-11-24 17:26:28 +00:00
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
env MarkSetEnv
|
2021-02-28 19:35:18 +00:00
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
markSetSize int64
|
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
|
|
|
|
2021-03-01 17:39:00 +00:00
|
|
|
// Open opens an existing splistore, or creates a new splitstore. The splitstore
|
|
|
|
// is backed by the provided hot and cold stores. The returned SplitStore MUST be
|
|
|
|
// attached to the ChainStore with Start in order to trigger compaction.
|
|
|
|
func Open(path string, ds dstore.Datastore, hot, cold bstore.Blockstore, cfg *Config) (*SplitStore, error) {
|
2020-12-01 15:17:34 +00:00
|
|
|
// the tracking store
|
2021-03-02 00:47:21 +00:00
|
|
|
tracker, err := OpenTrackingStore(path, cfg.TrackingStoreType)
|
2020-12-01 15:17:34 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
// the markset env
|
|
|
|
env, err := OpenMarkSetEnv(path, cfg.MarkSetType)
|
2021-02-27 16:27:58 +00:00
|
|
|
if err != nil {
|
2021-03-02 00:47:21 +00:00
|
|
|
_ = tracker.Close()
|
2021-02-27 16:27:58 +00:00
|
|
|
return nil, err
|
2020-12-01 15:17:34 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// and now we can make a SplitStore
|
|
|
|
ss := &SplitStore{
|
2021-03-02 00:47:21 +00:00
|
|
|
ds: ds,
|
|
|
|
hot: hot,
|
|
|
|
cold: cold,
|
|
|
|
tracker: tracker,
|
|
|
|
env: env,
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-03-02 16:59:00 +00:00
|
|
|
coldPurgeSize: defaultColdPurgeSize,
|
|
|
|
}
|
|
|
|
|
2020-12-01 15:17:34 +00:00
|
|
|
return ss, nil
|
|
|
|
}
|
|
|
|
|
2020-11-24 14:51:00 +00:00
|
|
|
// Blockstore interface
|
2021-03-01 17:39:00 +00:00
|
|
|
func (s *SplitStore) DeleteBlock(_ cid.Cid) error {
|
2020-11-24 14:51:00 +00:00
|
|
|
// 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
|
|
|
}
|
|
|
|
|
2021-03-02 14:45:45 +00:00
|
|
|
func (s *SplitStore) DeleteMany(_ []cid.Cid) error {
|
|
|
|
// afaict we don't seem to be using this method, so it's not implemented
|
|
|
|
return errors.New("DeleteMany 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:
|
2021-03-05 09:54:06 +00:00
|
|
|
blk, err = s.cold.Get(cid)
|
2021-03-05 10:04:40 +00:00
|
|
|
if err == nil {
|
2021-03-05 09:54:06 +00:00
|
|
|
stats.Record(context.Background(), metrics.SplitstoreMiss.M(1))
|
|
|
|
}
|
|
|
|
return blk, err
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
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:
|
2021-03-05 09:54:06 +00:00
|
|
|
size, err = s.cold.GetSize(cid)
|
2021-03-05 10:04:40 +00:00
|
|
|
if err == nil {
|
2021-03-05 09:54:06 +00:00
|
|
|
stats.Record(context.Background(), metrics.SplitstoreMiss.M(1))
|
|
|
|
}
|
|
|
|
return size, err
|
2020-11-24 14:51:00 +00:00
|
|
|
|
|
|
|
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()
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
err := s.tracker.Put(blk.Cid(), epoch)
|
2021-02-28 11:23:08 +00:00
|
|
|
if err != nil {
|
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())
|
|
|
|
}
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
err := s.tracker.PutBatch(batch, epoch)
|
2020-12-01 16:21:46 +00:00
|
|
|
if err != 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
|
2021-03-01 17:38:02 +00:00
|
|
|
func (s *SplitStore) Start(chain ChainAccessor) error {
|
|
|
|
s.chain = chain
|
|
|
|
s.curTs = chain.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 {
|
2021-03-01 16:41:51 +00:00
|
|
|
return xerrors.Errorf("error saving base epoch: %w", err)
|
2020-11-24 17:41:07 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
default:
|
2021-03-01 16:41:51 +00:00
|
|
|
return xerrors.Errorf("error loading base epoch: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// load warmup epoch from metadata ds
|
|
|
|
// if none, then the splitstore will warm up the hotstore at first head change notif
|
|
|
|
// by walking the current tipset
|
|
|
|
bs, err = s.ds.Get(warmupEpochKey)
|
|
|
|
switch err {
|
|
|
|
case nil:
|
|
|
|
s.warmupEpoch = bytesToEpoch(bs)
|
2021-03-13 10:00:28 +00:00
|
|
|
s.warm = true
|
2021-03-01 16:41:51 +00:00
|
|
|
|
|
|
|
case dstore.ErrNotFound:
|
|
|
|
default:
|
|
|
|
return xerrors.Errorf("error loading warmup epoch: %w", err)
|
2020-11-24 17:41:07 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 08:00:17 +00:00
|
|
|
// load markSetSize from metadata ds
|
|
|
|
// if none, the splitstore will compute it during warmup and update in every compaction
|
|
|
|
bs, err = s.ds.Get(markSetSizeKey)
|
|
|
|
switch err {
|
|
|
|
case nil:
|
|
|
|
s.markSetSize = bytesToInt64(bs)
|
|
|
|
|
|
|
|
case dstore.ErrNotFound:
|
|
|
|
default:
|
|
|
|
return xerrors.Errorf("error loading mark set size: %w", err)
|
|
|
|
}
|
|
|
|
|
2021-03-02 17:28:53 +00:00
|
|
|
log.Infow("starting splitstore", "baseEpoch", s.baseEpoch, "warmupEpoch", s.warmupEpoch)
|
2021-03-02 17:07:54 +00:00
|
|
|
|
2020-11-24 17:41:07 +00:00
|
|
|
// watch the chain
|
2021-03-01 17:38:02 +00:00
|
|
|
chain.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 {
|
2021-03-05 08:11:54 +00:00
|
|
|
atomic.StoreInt32(&s.closing, 1)
|
|
|
|
|
|
|
|
if atomic.LoadInt32(&s.critsection) == 1 {
|
|
|
|
log.Warn("ongoing compaction in critical section; waiting for it to finish...")
|
|
|
|
for atomic.LoadInt32(&s.critsection) == 1 {
|
2020-11-26 15:49:47 +00:00
|
|
|
time.Sleep(time.Second)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-03 08:56:41 +00:00
|
|
|
return multierr.Combine(s.tracker.Close(), s.env.Close())
|
2020-11-26 15:49:47 +00:00
|
|
|
}
|
|
|
|
|
2021-03-01 17:38:02 +00:00
|
|
|
func (s *SplitStore) HeadChange(_, apply []*types.TipSet) error {
|
2021-04-29 02:55:18 +00:00
|
|
|
// Revert only.
|
|
|
|
if len(apply) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-11-29 10:48:52 +00:00
|
|
|
s.mx.Lock()
|
2021-03-01 16:47:47 +00:00
|
|
|
curTs := apply[len(apply)-1]
|
|
|
|
epoch := curTs.Height()
|
|
|
|
s.curTs = curTs
|
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
|
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
if !s.warm {
|
2021-03-01 16:41:51 +00:00
|
|
|
// splitstore needs to warm up
|
|
|
|
go func() {
|
|
|
|
defer atomic.StoreInt32(&s.compacting, 0)
|
|
|
|
|
|
|
|
log.Info("warming up hotstore")
|
|
|
|
start := time.Now()
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
baseTs, err := s.chain.GetTipsetByHeight(context.Background(), s.baseEpoch, curTs, true)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error warming up hotstore: error getting tipset at base epoch: %s", err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
err = s.warmup(baseTs)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error warming up hotstore: %s", err)
|
|
|
|
return
|
|
|
|
}
|
2021-03-01 16:41:51 +00:00
|
|
|
|
|
|
|
log.Infow("warm up done", "took", time.Since(start))
|
|
|
|
}()
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2020-11-29 13:10:30 +00:00
|
|
|
if epoch-s.baseEpoch > CompactionThreshold {
|
2021-03-01 16:41:51 +00:00
|
|
|
// it's time to compact
|
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()
|
|
|
|
|
2021-03-01 16:47:47 +00:00
|
|
|
s.compact(curTs)
|
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
|
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
func (s *SplitStore) warmup(curTs *types.TipSet) error {
|
2021-03-01 16:41:51 +00:00
|
|
|
epoch := curTs.Height()
|
|
|
|
|
2021-03-01 18:11:35 +00:00
|
|
|
batchHot := make([]blocks.Block, 0, batchSize)
|
|
|
|
batchSnoop := make([]cid.Cid, 0, batchSize)
|
|
|
|
|
2021-03-01 16:49:20 +00:00
|
|
|
count := int64(0)
|
2021-03-13 10:00:28 +00:00
|
|
|
xcount := int64(0)
|
|
|
|
missing := int64(0)
|
|
|
|
err := s.walk(curTs, epoch,
|
2021-03-01 16:41:51 +00:00
|
|
|
func(cid cid.Cid) error {
|
2021-03-01 16:49:20 +00:00
|
|
|
count++
|
|
|
|
|
2021-03-01 16:41:51 +00:00
|
|
|
has, err := s.hot.Has(cid)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
if has {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
blk, err := s.cold.Get(cid)
|
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
if err == bstore.ErrNotFound {
|
|
|
|
missing++
|
|
|
|
return nil
|
|
|
|
}
|
2021-03-01 16:41:51 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
xcount++
|
|
|
|
|
2021-03-01 18:11:35 +00:00
|
|
|
batchHot = append(batchHot, blk)
|
|
|
|
batchSnoop = append(batchSnoop, cid)
|
|
|
|
|
|
|
|
if len(batchHot) == batchSize {
|
2021-03-02 00:47:21 +00:00
|
|
|
err = s.tracker.PutBatch(batchSnoop, epoch)
|
2021-03-01 18:11:35 +00:00
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
batchSnoop = batchSnoop[:0]
|
|
|
|
|
|
|
|
err = s.hot.PutMany(batchHot)
|
|
|
|
if err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
batchHot = batchHot[:0]
|
2021-03-01 16:41:51 +00:00
|
|
|
}
|
|
|
|
|
2021-03-01 18:11:35 +00:00
|
|
|
return nil
|
2021-03-01 16:41:51 +00:00
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
return err
|
2021-03-01 16:41:51 +00:00
|
|
|
}
|
|
|
|
|
2021-03-01 18:11:35 +00:00
|
|
|
if len(batchHot) > 0 {
|
2021-03-02 00:47:21 +00:00
|
|
|
err = s.tracker.PutBatch(batchSnoop, epoch)
|
2021-03-01 18:11:35 +00:00
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
return err
|
2021-03-01 18:11:35 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
err = s.hot.PutMany(batchHot)
|
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
return err
|
2021-03-01 18:11:35 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
log.Infow("warmup stats", "visited", count, "cold", xcount, "missing", missing)
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
if count > s.markSetSize {
|
2021-03-05 08:00:17 +00:00
|
|
|
s.markSetSize = count + count>>2 // overestimate a bit
|
2021-03-01 16:49:20 +00:00
|
|
|
}
|
|
|
|
|
2021-03-01 16:41:51 +00:00
|
|
|
// save the warmup epoch
|
2021-03-13 10:00:28 +00:00
|
|
|
s.warm = true
|
2021-03-01 16:41:51 +00:00
|
|
|
s.warmupEpoch = epoch
|
|
|
|
err = s.ds.Put(warmupEpochKey, epochToBytes(epoch))
|
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
log.Warnf("error saving warmup epoch: %s", err)
|
2021-03-01 16:41:51 +00:00
|
|
|
}
|
2021-03-05 08:00:17 +00:00
|
|
|
|
|
|
|
err = s.ds.Put(markSetSizeKey, int64ToBytes(s.markSetSize))
|
|
|
|
if err != nil {
|
2021-03-13 10:00:28 +00:00
|
|
|
log.Warnf("error saving mark set size: %s", err)
|
2021-03-05 08:00:17 +00:00
|
|
|
}
|
2021-03-13 10:00:28 +00:00
|
|
|
|
|
|
|
return nil
|
2021-03-01 16:41:51 +00:00
|
|
|
}
|
|
|
|
|
2020-11-24 17:15:38 +00:00
|
|
|
// Compaction/GC Algorithm
|
2021-03-01 16:47:47 +00:00
|
|
|
func (s *SplitStore) compact(curTs *types.TipSet) {
|
2021-03-05 08:29:49 +00:00
|
|
|
var err error
|
2021-03-02 00:47:21 +00:00
|
|
|
if s.markSetSize == 0 {
|
2021-02-28 19:35:18 +00:00
|
|
|
start := time.Now()
|
2021-03-02 00:47:21 +00:00
|
|
|
log.Info("estimating mark set size")
|
2021-03-05 08:29:49 +00:00
|
|
|
err = s.estimateMarkSetSize(curTs)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error estimating mark set size: %s; aborting compaction", err)
|
|
|
|
return
|
|
|
|
}
|
2021-03-02 00:47:21 +00:00
|
|
|
log.Infow("estimating mark set size done", "took", time.Since(start), "size", s.markSetSize)
|
2021-02-28 19:35:18 +00:00
|
|
|
} else {
|
2021-03-02 00:47:21 +00:00
|
|
|
log.Infow("current mark set size estimate", "size", s.markSetSize)
|
2021-02-28 19:35:18 +00:00
|
|
|
}
|
|
|
|
|
2021-03-05 09:54:06 +00:00
|
|
|
start := time.Now()
|
2021-06-14 17:19:06 +00:00
|
|
|
err = s.doCompact(curTs)
|
2021-03-05 09:54:06 +00:00
|
|
|
took := time.Since(start).Milliseconds()
|
|
|
|
stats.Record(context.Background(), metrics.SplitstoreCompactionTimeSeconds.M(float64(took)/1e3))
|
2021-03-05 08:29:49 +00:00
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("COMPACTION ERROR: %s", err)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-05 08:29:49 +00:00
|
|
|
func (s *SplitStore) estimateMarkSetSize(curTs *types.TipSet) error {
|
2021-03-13 10:00:28 +00:00
|
|
|
epoch := curTs.Height()
|
|
|
|
|
2021-03-03 07:46:12 +00:00
|
|
|
var count int64
|
2021-03-13 10:00:28 +00:00
|
|
|
err := s.walk(curTs, epoch,
|
2021-02-28 19:35:18 +00:00
|
|
|
func(cid cid.Cid) error {
|
2021-03-03 07:46:12 +00:00
|
|
|
count++
|
2021-02-28 19:35:18 +00:00
|
|
|
return nil
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return err
|
2021-02-28 19:35:18 +00:00
|
|
|
}
|
2021-03-03 07:46:12 +00:00
|
|
|
|
2021-03-05 08:00:17 +00:00
|
|
|
s.markSetSize = count + count>>2 // overestimate a bit
|
2021-03-05 08:29:49 +00:00
|
|
|
return nil
|
2021-02-28 19:35:18 +00:00
|
|
|
}
|
|
|
|
|
2021-06-14 17:19:06 +00:00
|
|
|
func (s *SplitStore) doCompact(curTs *types.TipSet) error {
|
2021-02-27 13:20:14 +00:00
|
|
|
coldEpoch := s.baseEpoch + CompactionCold
|
2021-03-03 09:15:26 +00:00
|
|
|
currentEpoch := curTs.Height()
|
|
|
|
boundaryEpoch := currentEpoch - CompactionBoundary
|
2021-02-27 16:47:13 +00:00
|
|
|
|
2021-06-14 17:19:06 +00:00
|
|
|
log.Infow("running compaction", "currentEpoch", currentEpoch, "baseEpoch", s.baseEpoch, "coldEpoch", coldEpoch, "boundaryEpoch", boundaryEpoch)
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
coldSet, err := s.env.Create("cold", s.markSetSize)
|
2021-02-27 16:27:58 +00:00
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error creating mark set: %w", err)
|
2021-02-27 16:27:58 +00:00
|
|
|
}
|
|
|
|
defer coldSet.Close() //nolint:errcheck
|
|
|
|
|
|
|
|
// 1. mark reachable cold objects by looking at the objects reachable only from the cold epoch
|
2021-03-05 08:29:49 +00:00
|
|
|
log.Infow("marking reachable cold blocks", "boundaryEpoch", boundaryEpoch)
|
2021-02-27 16:27:58 +00:00
|
|
|
startMark := time.Now()
|
|
|
|
|
2021-03-03 09:15:26 +00:00
|
|
|
boundaryTs, err := s.chain.GetTipsetByHeight(context.Background(), boundaryEpoch, curTs, true)
|
2021-02-27 16:27:58 +00:00
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error getting tipset at boundary epoch: %w", err)
|
2021-02-27 16:27:58 +00:00
|
|
|
}
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
var count int64
|
2021-03-13 10:00:28 +00:00
|
|
|
err = s.walk(boundaryTs, boundaryEpoch,
|
2021-02-27 16:27:58 +00:00
|
|
|
func(cid cid.Cid) error {
|
2021-02-28 19:35:18 +00:00
|
|
|
count++
|
2021-02-27 16:27:58 +00:00
|
|
|
return coldSet.Mark(cid)
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error marking cold blocks: %w", err)
|
2021-02-27 16:27:58 +00:00
|
|
|
}
|
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
if count > s.markSetSize {
|
2021-03-05 08:00:17 +00:00
|
|
|
s.markSetSize = count + count>>2 // overestimate a bit
|
2021-02-28 19:35:18 +00:00
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
log.Infow("marking done", "took", time.Since(startMark), "marked", count)
|
2021-02-27 16:27:58 +00:00
|
|
|
|
|
|
|
// 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-03-02 16:59:00 +00:00
|
|
|
cold := make([]cid.Cid, 0, s.coldPurgeSize)
|
2021-02-27 16:47:13 +00:00
|
|
|
|
|
|
|
// some stats for logging
|
2021-03-02 00:47:21 +00:00
|
|
|
var hotCnt, coldCnt int
|
2021-02-27 16:47:13 +00:00
|
|
|
|
2021-03-02 00:47:21 +00:00
|
|
|
// 2.1 iterate through the tracking store and collect unreachable cold objects
|
|
|
|
err = s.tracker.ForEach(func(cid cid.Cid, writeEpoch abi.ChainEpoch) error {
|
|
|
|
// is the object still hot?
|
|
|
|
if writeEpoch > coldEpoch {
|
2021-02-27 13:20:14 +00:00
|
|
|
// yes, stay in the hotstore
|
2021-03-02 00:47:21 +00:00
|
|
|
hotCnt++
|
2021-02-27 13:20:14 +00:00
|
|
|
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 {
|
2021-03-02 00:47:21 +00:00
|
|
|
hotCnt++
|
2021-02-27 16:27:58 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// it's cold, mark it for move
|
2021-03-02 16:59:00 +00:00
|
|
|
cold = append(cold, cid)
|
2021-03-02 00:47:21 +00:00
|
|
|
coldCnt++
|
2021-02-27 13:20:14 +00:00
|
|
|
return nil
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error collecting cold objects: %w", err)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
|
2021-03-02 18:20:07 +00:00
|
|
|
if coldCnt > 0 {
|
|
|
|
s.coldPurgeSize = coldCnt + coldCnt>>2 // overestimate a bit
|
|
|
|
}
|
2021-03-02 16:59:00 +00:00
|
|
|
|
2021-02-27 13:20:14 +00:00
|
|
|
log.Infow("collection done", "took", time.Since(startCollect))
|
2021-03-02 00:47:21 +00:00
|
|
|
log.Infow("compaction stats", "hot", hotCnt, "cold", coldCnt)
|
2021-03-05 09:54:06 +00:00
|
|
|
stats.Record(context.Background(), metrics.SplitstoreCompactionHot.M(int64(hotCnt)))
|
|
|
|
stats.Record(context.Background(), metrics.SplitstoreCompactionCold.M(int64(coldCnt)))
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-03-05 08:11:54 +00:00
|
|
|
// Enter critical section
|
|
|
|
atomic.StoreInt32(&s.critsection, 1)
|
|
|
|
defer atomic.StoreInt32(&s.critsection, 0)
|
|
|
|
|
|
|
|
// check to see if we are closing first; if that's the case just return
|
|
|
|
if atomic.LoadInt32(&s.closing) == 1 {
|
|
|
|
log.Info("splitstore is closing; aborting compaction")
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("compaction aborted")
|
2021-03-05 08:11:54 +00:00
|
|
|
}
|
|
|
|
|
2021-02-28 08:21:48 +00:00
|
|
|
// 2.2 copy the cold objects to the coldstore
|
2021-03-05 08:29:49 +00:00
|
|
|
log.Info("moving cold blocks to the coldstore")
|
2021-02-27 13:20:14 +00:00
|
|
|
startMove := time.Now()
|
2021-03-02 09:20:39 +00:00
|
|
|
err = s.moveColdBlocks(cold)
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error moving cold blocks: %w", err)
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
|
|
|
log.Infow("moving done", "took", time.Since(startMove))
|
|
|
|
|
|
|
|
// 2.3 delete cold objects from the hotstore
|
|
|
|
log.Info("purging cold objects from the hotstore")
|
|
|
|
startPurge := time.Now()
|
|
|
|
err = s.purgeBlocks(cold)
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error purging cold blocks: %w", err)
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
|
|
|
log.Infow("purging cold from hotstore done", "took", time.Since(startPurge))
|
|
|
|
|
|
|
|
// 2.4 remove the tracker tracking for cold objects
|
|
|
|
startPurge = time.Now()
|
|
|
|
log.Info("purging cold objects from tracker")
|
|
|
|
err = s.purgeTracking(cold)
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error purging tracking for cold blocks: %w", err)
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
|
|
|
log.Infow("purging cold from tracker done", "took", time.Since(startPurge))
|
|
|
|
|
|
|
|
// we are done; do some housekeeping
|
|
|
|
err = s.tracker.Sync()
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error syncing tracker: %w", err)
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
|
2021-03-08 17:46:21 +00:00
|
|
|
s.gcHotstore()
|
2021-03-08 16:12:09 +00:00
|
|
|
|
2021-03-02 09:20:39 +00:00
|
|
|
err = s.setBaseEpoch(coldEpoch)
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error saving base epoch: %w", err)
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
2021-03-05 08:00:17 +00:00
|
|
|
|
|
|
|
err = s.ds.Put(markSetSizeKey, int64ToBytes(s.markSetSize))
|
|
|
|
if err != nil {
|
2021-03-05 08:29:49 +00:00
|
|
|
return xerrors.Errorf("error saving mark set size: %w", err)
|
2021-03-05 08:00:17 +00:00
|
|
|
}
|
2021-03-05 08:29:49 +00:00
|
|
|
|
|
|
|
return nil
|
2021-03-02 09:20:39 +00:00
|
|
|
}
|
|
|
|
|
2021-03-13 10:00:28 +00:00
|
|
|
func (s *SplitStore) walk(ts *types.TipSet, boundary abi.ChainEpoch, f func(cid.Cid) error) error {
|
|
|
|
walked := cid.NewSet()
|
|
|
|
toWalk := ts.Cids()
|
|
|
|
|
|
|
|
walkBlock := func(c cid.Cid) error {
|
|
|
|
if !walked.Visit(c) {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
blk, err := s.Get(c)
|
|
|
|
if err != nil {
|
|
|
|
return xerrors.Errorf("error retrieving block (cid: %s): %w", c, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
var hdr types.BlockHeader
|
|
|
|
if err := hdr.UnmarshalCBOR(bytes.NewBuffer(blk.RawData())); err != nil {
|
|
|
|
return xerrors.Errorf("error unmarshaling block header (cid: %s): %w", c, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
// don't walk under the boundary
|
|
|
|
if hdr.Height < boundary {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := f(c); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := s.walkLinks(hdr.Messages, walked, f); err != nil {
|
|
|
|
return xerrors.Errorf("error walking messages (cid: %s): %w", hdr.Messages, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := s.walkLinks(hdr.ParentStateRoot, walked, f); err != nil {
|
|
|
|
return xerrors.Errorf("error walking state root (cid: %s): %w", hdr.ParentStateRoot, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
toWalk = append(toWalk, hdr.Parents...)
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
for len(toWalk) > 0 {
|
|
|
|
walking := toWalk
|
|
|
|
toWalk = nil
|
|
|
|
for _, c := range walking {
|
|
|
|
if err := walkBlock(c); err != nil {
|
|
|
|
return xerrors.Errorf("error walking block (cid: %s): %w", c, err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
func (s *SplitStore) walkLinks(c cid.Cid, walked *cid.Set, f func(cid.Cid) error) error {
|
|
|
|
if !walked.Visit(c) {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if c.Prefix().Codec != cid.DagCBOR {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
if err := f(c); err != nil {
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
blk, err := s.Get(c)
|
|
|
|
if err != nil {
|
|
|
|
return xerrors.Errorf("error retrieving linked block (cid: %s): %w", c, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
var rerr error
|
|
|
|
err = cbg.ScanForLinks(bytes.NewReader(blk.RawData()), func(c cid.Cid) {
|
|
|
|
if rerr != nil {
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
rerr = s.walkLinks(c, walked, f)
|
|
|
|
})
|
|
|
|
|
|
|
|
if err != nil {
|
|
|
|
return xerrors.Errorf("error scanning links (cid: %s): %w", c, err)
|
|
|
|
}
|
|
|
|
|
|
|
|
return rerr
|
|
|
|
}
|
|
|
|
|
2021-03-02 16:59:00 +00:00
|
|
|
func (s *SplitStore) moveColdBlocks(cold []cid.Cid) error {
|
2021-02-28 08:21:48 +00:00
|
|
|
batch := make([]blocks.Block, 0, batchSize)
|
|
|
|
|
2021-03-02 16:59:00 +00:00
|
|
|
for _, cid := range cold {
|
2021-02-27 13:20:14 +00:00
|
|
|
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
|
2021-03-02 00:47:21 +00:00
|
|
|
// but before we have deleted it from the tracker; just delete the tracker.
|
|
|
|
err = s.tracker.Delete(cid)
|
2021-02-27 13:20:14 +00:00
|
|
|
if err != nil {
|
2021-03-02 09:20:39 +00:00
|
|
|
return xerrors.Errorf("error deleting unreachable cid %s from tracker: %w", cid, err)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
} else {
|
2021-03-02 09:20:39 +00:00
|
|
|
return xerrors.Errorf("error retrieving tracked block %s from hotstore: %w", cid, err)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
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 {
|
2021-03-02 09:20:39 +00:00
|
|
|
return xerrors.Errorf("error putting batch to coldstore: %w", err)
|
2021-02-28 08:21:48 +00:00
|
|
|
}
|
|
|
|
batch = batch[:0]
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(batch) > 0 {
|
2021-03-02 09:20:39 +00:00
|
|
|
err := s.cold.PutMany(batch)
|
2021-02-27 13:20:14 +00:00
|
|
|
if err != nil {
|
2021-03-02 09:20:39 +00:00
|
|
|
return xerrors.Errorf("error putting cold to coldstore: %w", err)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
2021-02-28 08:21:48 +00:00
|
|
|
}
|
2021-02-27 13:20:14 +00:00
|
|
|
|
2021-03-02 09:20:39 +00:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2021-03-03 18:04:29 +00:00
|
|
|
func (s *SplitStore) purgeBatch(cids []cid.Cid, deleteBatch func([]cid.Cid) error) error {
|
2021-03-02 18:20:07 +00:00
|
|
|
if len(cids) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// don't delete one giant batch of 7M objects, but rather do smaller batches
|
|
|
|
done := false
|
2021-03-03 17:36:13 +00:00
|
|
|
for i := 0; !done; i++ {
|
2021-03-02 18:20:07 +00:00
|
|
|
start := i * batchSize
|
|
|
|
end := start + batchSize
|
|
|
|
if end >= len(cids) {
|
|
|
|
end = len(cids)
|
|
|
|
done = true
|
|
|
|
}
|
|
|
|
|
2021-03-03 18:04:29 +00:00
|
|
|
err := deleteBatch(cids[start:end])
|
2021-03-02 18:20:07 +00:00
|
|
|
if err != nil {
|
2021-03-03 18:04:29 +00:00
|
|
|
return xerrors.Errorf("error deleting batch: %w", err)
|
2021-03-02 18:20:07 +00:00
|
|
|
}
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
2021-02-27 21:08:23 +00:00
|
|
|
|
2021-03-02 09:20:39 +00:00
|
|
|
return nil
|
|
|
|
}
|
2021-02-27 21:08:23 +00:00
|
|
|
|
2021-03-03 18:04:29 +00:00
|
|
|
func (s *SplitStore) purgeBlocks(cids []cid.Cid) error {
|
|
|
|
return s.purgeBatch(cids, s.hot.DeleteMany)
|
|
|
|
}
|
2021-02-27 16:27:58 +00:00
|
|
|
|
2021-03-03 18:04:29 +00:00
|
|
|
func (s *SplitStore) purgeTracking(cids []cid.Cid) error {
|
|
|
|
return s.purgeBatch(cids, s.tracker.DeleteBatch)
|
2021-02-27 13:20:14 +00:00
|
|
|
}
|
|
|
|
|
2021-03-08 17:46:21 +00:00
|
|
|
func (s *SplitStore) gcHotstore() {
|
2021-03-11 09:45:19 +00:00
|
|
|
if compact, ok := s.hot.(interface{ Compact() error }); ok {
|
|
|
|
log.Infof("compacting hotstore")
|
|
|
|
startCompact := time.Now()
|
|
|
|
err := compact.Compact()
|
|
|
|
if err != nil {
|
|
|
|
log.Warnf("error compacting hotstore: %s", err)
|
|
|
|
return
|
|
|
|
}
|
2021-03-11 11:10:44 +00:00
|
|
|
log.Infow("hotstore compaction done", "took", time.Since(startCompact))
|
2021-03-11 09:45:19 +00:00
|
|
|
}
|
|
|
|
|
2021-03-08 17:46:21 +00:00
|
|
|
if gc, ok := s.hot.(interface{ CollectGarbage() error }); ok {
|
|
|
|
log.Infof("garbage collecting hotstore")
|
|
|
|
startGC := time.Now()
|
|
|
|
err := gc.CollectGarbage()
|
|
|
|
if err != nil {
|
|
|
|
log.Warnf("error garbage collecting hotstore: %s", err)
|
2021-03-11 11:10:44 +00:00
|
|
|
return
|
2021-03-08 17:46:21 +00:00
|
|
|
}
|
2021-03-11 11:10:44 +00:00
|
|
|
log.Infow("hotstore garbage collection done", "took", time.Since(startGC))
|
2021-03-08 17:46:21 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-24 17:41:07 +00:00
|
|
|
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 {
|
2021-03-05 08:00:17 +00:00
|
|
|
return uint64ToBytes(uint64(epoch))
|
|
|
|
}
|
|
|
|
|
|
|
|
func bytesToEpoch(buf []byte) abi.ChainEpoch {
|
|
|
|
return abi.ChainEpoch(bytesToUint64(buf))
|
|
|
|
}
|
|
|
|
|
|
|
|
func int64ToBytes(i int64) []byte {
|
|
|
|
return uint64ToBytes(uint64(i))
|
|
|
|
}
|
|
|
|
|
|
|
|
func bytesToInt64(buf []byte) int64 {
|
|
|
|
return int64(bytesToUint64(buf))
|
|
|
|
}
|
|
|
|
|
|
|
|
func uint64ToBytes(i uint64) []byte {
|
2020-11-26 18:37:02 +00:00
|
|
|
buf := make([]byte, 16)
|
2021-03-05 08:00:17 +00:00
|
|
|
n := binary.PutUvarint(buf, i)
|
2020-11-26 18:37:02 +00:00
|
|
|
return buf[:n]
|
|
|
|
}
|
|
|
|
|
2021-03-05 08:00:17 +00:00
|
|
|
func bytesToUint64(buf []byte) uint64 {
|
|
|
|
i, _ := binary.Uvarint(buf)
|
|
|
|
return i
|
2020-11-24 17:15:38 +00:00
|
|
|
}
|