Chain ranged export: rework and address current shortcomings

This commit moderately refactors the ranged export code. It addresses several
problems:
  * Code does not finish cleanly and things hang on ctrl-c
  * Same block is read multiple times in a row (artificially increasing cached
    blockstore metrics to 50%)
  * It is unclear whether there are additional races (a single worker quits
    when reaching height 0)
  * CARs produced have duplicated blocks (~400k for an 80M-blocks CAR or
    so). Some blocks appear up to 5 times.
  * Using pointers for tasks where it is not necessary.

The changes:

  * Use a FIFO instead of stack: simpler implementation as its own type. This
has not proven to be much more memory-friendly, but it has not made things
worse either.
  * We avoid a probably not small amount of allocations by not using
    unnecessary pointers.
  * Fix duplicated blocks by atomically checking+adding to CID set.
  * Context-termination now works correctly. Worker lifetime is correctly tracked and all channels
  are closed, avoiding any memory leaks and deadlocks.
  * We ensure all work is finished before finishing, something that might have
  been broken in some edge cases previously. In practice, we would not have
  seen this except perhaps in very early snapshots close to genesis.

Initial testing shows the code is currently about 5% faster. Resulting
snapshots do not have duplicates so they are a bit smaller. We have manually
verified that no CID is lost versus previous results, with both old and recent
snapshots.
This commit is contained in:
Hector Sanjuan 2023-02-02 17:51:52 +01:00
parent 21efd481d8
commit fa93c23813
8 changed files with 366 additions and 262 deletions

View File

@ -173,9 +173,9 @@ type FullNode interface {
// If oldmsgskip is set, messages from before the requested roots are also not included. // If oldmsgskip is set, messages from before the requested roots are also not included.
ChainExport(ctx context.Context, nroots abi.ChainEpoch, oldmsgskip bool, tsk types.TipSetKey) (<-chan []byte, error) //perm:read ChainExport(ctx context.Context, nroots abi.ChainEpoch, oldmsgskip bool, tsk types.TipSetKey) (<-chan []byte, error) //perm:read
ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg *ChainExportConfig) (<-chan []byte, error) //perm:read ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg ChainExportConfig) (<-chan []byte, error) //perm:read
ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg *ChainExportConfig) error //perm:read ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg ChainExportConfig) error //perm:read
// ChainPrune prunes the stored chain state and garbage collects; only supported if you // ChainPrune prunes the stored chain state and garbage collects; only supported if you
// are using the splitstore // are using the splitstore

View File

@ -138,9 +138,9 @@ type FullNodeMethods struct {
ChainExport func(p0 context.Context, p1 abi.ChainEpoch, p2 bool, p3 types.TipSetKey) (<-chan []byte, error) `perm:"read"` ChainExport func(p0 context.Context, p1 abi.ChainEpoch, p2 bool, p3 types.TipSetKey) (<-chan []byte, error) `perm:"read"`
ChainExportRange func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) (<-chan []byte, error) `perm:"read"` ChainExportRange func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) (<-chan []byte, error) `perm:"read"`
ChainExportRangeInternal func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) error `perm:"read"` ChainExportRangeInternal func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) error `perm:"read"`
ChainGetBlock func(p0 context.Context, p1 cid.Cid) (*types.BlockHeader, error) `perm:"read"` ChainGetBlock func(p0 context.Context, p1 cid.Cid) (*types.BlockHeader, error) `perm:"read"`
@ -1437,25 +1437,25 @@ func (s *FullNodeStub) ChainExport(p0 context.Context, p1 abi.ChainEpoch, p2 boo
return nil, ErrNotSupported return nil, ErrNotSupported
} }
func (s *FullNodeStruct) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) (<-chan []byte, error) { func (s *FullNodeStruct) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) (<-chan []byte, error) {
if s.Internal.ChainExportRange == nil { if s.Internal.ChainExportRange == nil {
return nil, ErrNotSupported return nil, ErrNotSupported
} }
return s.Internal.ChainExportRange(p0, p1, p2, p3) return s.Internal.ChainExportRange(p0, p1, p2, p3)
} }
func (s *FullNodeStub) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) (<-chan []byte, error) { func (s *FullNodeStub) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) (<-chan []byte, error) {
return nil, ErrNotSupported return nil, ErrNotSupported
} }
func (s *FullNodeStruct) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) error { func (s *FullNodeStruct) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) error {
if s.Internal.ChainExportRangeInternal == nil { if s.Internal.ChainExportRangeInternal == nil {
return ErrNotSupported return ErrNotSupported
} }
return s.Internal.ChainExportRangeInternal(p0, p1, p2, p3) return s.Internal.ChainExportRangeInternal(p0, p1, p2, p3)
} }
func (s *FullNodeStub) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *ChainExportConfig) error { func (s *FullNodeStub) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 ChainExportConfig) error {
return ErrNotSupported return ErrNotSupported
} }

View File

@ -399,9 +399,10 @@ func (m *MsgUuidMapType) UnmarshalJSON(b []byte) error {
return nil return nil
} }
// ChainExportConfig holds configuration for chain ranged exports.
type ChainExportConfig struct { type ChainExportConfig struct {
WriteBufferSize int WriteBufferSize int
Workers int64 NumWorkers int
CacheSize int CacheSize int
IncludeMessages bool IncludeMessages bool
IncludeReceipts bool IncludeReceipts bool

View File

@ -161,9 +161,9 @@ type FullNode interface {
// If oldmsgskip is set, messages from before the requested roots are also not included. // If oldmsgskip is set, messages from before the requested roots are also not included.
ChainExport(ctx context.Context, nroots abi.ChainEpoch, oldmsgskip bool, tsk types.TipSetKey) (<-chan []byte, error) //perm:read ChainExport(ctx context.Context, nroots abi.ChainEpoch, oldmsgskip bool, tsk types.TipSetKey) (<-chan []byte, error) //perm:read
ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg *api.ChainExportConfig) (<-chan []byte, error) //perm:read ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg api.ChainExportConfig) (<-chan []byte, error) //perm:read
ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg *api.ChainExportConfig) error //perm:read ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg api.ChainExportConfig) error //perm:read
// MethodGroup: Beacon // MethodGroup: Beacon
// The Beacon method group contains methods for interacting with the random beacon (DRAND) // The Beacon method group contains methods for interacting with the random beacon (DRAND)

View File

@ -47,9 +47,9 @@ type FullNodeMethods struct {
ChainExport func(p0 context.Context, p1 abi.ChainEpoch, p2 bool, p3 types.TipSetKey) (<-chan []byte, error) `perm:"read"` ChainExport func(p0 context.Context, p1 abi.ChainEpoch, p2 bool, p3 types.TipSetKey) (<-chan []byte, error) `perm:"read"`
ChainExportRange func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) (<-chan []byte, error) `perm:"read"` ChainExportRange func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) (<-chan []byte, error) `perm:"read"`
ChainExportRangeInternal func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) error `perm:"read"` ChainExportRangeInternal func(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) error `perm:"read"`
ChainGetBlock func(p0 context.Context, p1 cid.Cid) (*types.BlockHeader, error) `perm:"read"` ChainGetBlock func(p0 context.Context, p1 cid.Cid) (*types.BlockHeader, error) `perm:"read"`
@ -534,25 +534,25 @@ func (s *FullNodeStub) ChainExport(p0 context.Context, p1 abi.ChainEpoch, p2 boo
return nil, ErrNotSupported return nil, ErrNotSupported
} }
func (s *FullNodeStruct) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) (<-chan []byte, error) { func (s *FullNodeStruct) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) (<-chan []byte, error) {
if s.Internal.ChainExportRange == nil { if s.Internal.ChainExportRange == nil {
return nil, ErrNotSupported return nil, ErrNotSupported
} }
return s.Internal.ChainExportRange(p0, p1, p2, p3) return s.Internal.ChainExportRange(p0, p1, p2, p3)
} }
func (s *FullNodeStub) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) (<-chan []byte, error) { func (s *FullNodeStub) ChainExportRange(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) (<-chan []byte, error) {
return nil, ErrNotSupported return nil, ErrNotSupported
} }
func (s *FullNodeStruct) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) error { func (s *FullNodeStruct) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) error {
if s.Internal.ChainExportRangeInternal == nil { if s.Internal.ChainExportRangeInternal == nil {
return ErrNotSupported return ErrNotSupported
} }
return s.Internal.ChainExportRangeInternal(p0, p1, p2, p3) return s.Internal.ChainExportRangeInternal(p0, p1, p2, p3)
} }
func (s *FullNodeStub) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 *api.ChainExportConfig) error { func (s *FullNodeStub) ChainExportRangeInternal(p0 context.Context, p1 types.TipSetKey, p2 types.TipSetKey, p3 api.ChainExportConfig) error {
return ErrNotSupported return ErrNotSupported
} }

View File

@ -15,6 +15,7 @@ import (
carv2 "github.com/ipld/go-car/v2" carv2 "github.com/ipld/go-car/v2"
mh "github.com/multiformats/go-multihash" mh "github.com/multiformats/go-multihash"
cbg "github.com/whyrusleeping/cbor-gen" cbg "github.com/whyrusleeping/cbor-gen"
"go.uber.org/atomic"
"golang.org/x/sync/errgroup" "golang.org/x/sync/errgroup"
"golang.org/x/xerrors" "golang.org/x/xerrors"
@ -32,35 +33,6 @@ func (cs *ChainStore) UnionStore() bstore.Blockstore {
return bstore.Union(cs.stateBlockstore, cs.chainBlockstore) return bstore.Union(cs.stateBlockstore, cs.chainBlockstore)
} }
func (cs *ChainStore) ExportRange(ctx context.Context, head, tail *types.TipSet, messages, receipts, stateroots bool, workers int64, cacheSize int, w io.Writer) error {
h := &car.CarHeader{
Roots: head.Cids(),
Version: 1,
}
if err := car.WriteHeader(h, w); err != nil {
return xerrors.Errorf("failed to write car header: %s", err)
}
cacheStore, err := NewCachingBlockstore(cs.UnionStore(), cacheSize)
if err != nil {
return err
}
return cs.WalkSnapshotRange(ctx, cacheStore, head, tail, messages, receipts, stateroots, workers, func(c cid.Cid) error {
blk, err := cacheStore.Get(ctx, c)
if err != nil {
return xerrors.Errorf("writing object to car, bs.Get: %w", err)
}
if err := carutil.LdWrite(w, c.Bytes(), blk.RawData()); err != nil {
return xerrors.Errorf("failed to write block to car output: %w", err)
}
return nil
})
}
func (cs *ChainStore) Export(ctx context.Context, ts *types.TipSet, inclRecentRoots abi.ChainEpoch, skipOldMsgs bool, w io.Writer) error { func (cs *ChainStore) Export(ctx context.Context, ts *types.TipSet, inclRecentRoots abi.ChainEpoch, skipOldMsgs bool, w io.Writer) error {
h := &car.CarHeader{ h := &car.CarHeader{
Roots: ts.Cids(), Roots: ts.Cids(),
@ -165,67 +137,228 @@ func (cs *ChainStore) Import(ctx context.Context, r io.Reader) (*types.TipSet, e
return root, nil return root, nil
} }
type walkTask struct { type walkSchedTaskType int
c cid.Cid
taskType taskType const (
finishTask walkSchedTaskType = -1
blockTask walkSchedTaskType = iota
messageTask
receiptTask
stateTask
dagTask
)
func (t walkSchedTaskType) String() string {
switch t {
case finishTask:
return "finish"
case blockTask:
return "block"
case messageTask:
return "message"
case receiptTask:
return "receipt"
case stateTask:
return "state"
case dagTask:
return "dag"
}
panic(fmt.Sprintf("unknow task %d", t))
} }
type walkResult struct { type walkTask struct {
c cid.Cid
taskType walkSchedTaskType
}
// an ever growing FIFO
type taskFifo struct {
in chan walkTask
out chan walkTask
fifo []walkTask
}
type taskResult struct {
c cid.Cid c cid.Cid
b blocks.Block
}
func newTaskFifo(bufferLen int) *taskFifo {
f := taskFifo{
in: make(chan walkTask, bufferLen),
out: make(chan walkTask, bufferLen),
fifo: make([]walkTask, 0),
}
go f.run()
return &f
}
func (f *taskFifo) Close() error {
close(f.in)
return nil
}
func (f *taskFifo) run() {
for {
if len(f.fifo) > 0 {
// we have items in slice
// try to put next out or read something in.
// blocks if nothing works.
next := f.fifo[0]
select {
case f.out <- next:
f.fifo = f.fifo[1:]
case elem, ok := <-f.in:
if !ok {
// drain and close out.
for _, elem := range f.fifo {
f.out <- elem
}
close(f.out)
return
}
f.fifo = append(f.fifo, elem)
}
} else {
// no elements in fifo to put out.
// Try to read in and block.
// When done, try to put out or add to fifo.
select {
case elem, ok := <-f.in:
if !ok {
close(f.out)
return
}
select {
case f.out <- elem:
default:
f.fifo = append(f.fifo, elem)
}
}
}
}
} }
type walkSchedulerConfig struct { type walkSchedulerConfig struct {
numWorkers int64 numWorkers int
tail *types.TipSet
head *types.TipSet // Tipset to start walking from.
tail *types.TipSet // Tipset to end at.
includeMessages bool includeMessages bool
includeReceipts bool includeReceipts bool
includeState bool includeState bool
} }
func newWalkScheduler(ctx context.Context, store bstore.Blockstore, cfg *walkSchedulerConfig, rootTasks ...*walkTask) (*walkScheduler, context.Context) { type walkScheduler struct {
tailSet := cid.NewSet() ctx context.Context
for i := range cfg.tail.Cids() { cancel context.CancelFunc
tailSet.Add(cfg.tail.Cids()[i])
} store bstore.Blockstore
grp, ctx := errgroup.WithContext(ctx) cfg walkSchedulerConfig
s := &walkScheduler{ writer io.Writer
store: store,
numWorkers: cfg.numWorkers, workerTasks *taskFifo
stack: rootTasks, totalTasks atomic.Int64
in: make(chan *walkTask, cfg.numWorkers*64), results chan taskResult
out: make(chan *walkTask, cfg.numWorkers*64), writeErrorChan chan error
grp: grp,
tail: tailSet, // tracks number of inflight tasks
cfg: cfg, //taskWg sync.WaitGroup
} // launches workers and collects errors if any occur
s.taskWg.Add(len(rootTasks)) workers *errgroup.Group
return s, ctx // set of CIDs already exported
seen sync.Map
} }
type walkScheduler struct { func newWalkScheduler(ctx context.Context, store bstore.Blockstore, cfg walkSchedulerConfig, w io.Writer) (*walkScheduler, error) {
store bstore.Blockstore ctx, cancel := context.WithCancel(ctx)
// number of worker routine to spawn workers, ctx := errgroup.WithContext(ctx)
numWorkers int64 s := &walkScheduler{
// buffer holds tasks until they are processed ctx: ctx,
stack []*walkTask cancel: cancel,
// inbound and outbound tasks store: store,
in, out chan *walkTask cfg: cfg,
// tracks number of inflight tasks writer: w,
taskWg sync.WaitGroup results: make(chan taskResult, cfg.numWorkers*64),
// launches workers and collects errors if any occur workerTasks: newTaskFifo(cfg.numWorkers * 64),
grp *errgroup.Group writeErrorChan: make(chan error, 1),
// set of tasks seen workers: workers,
seen sync.Map }
tail *cid.Set go func() {
cfg *walkSchedulerConfig defer close(s.writeErrorChan)
for r := range s.results {
// Write
if err := carutil.LdWrite(s.writer, r.c.Bytes(), r.b.RawData()); err != nil {
// abort operations
cancel()
s.writeErrorChan <- err
}
}
}()
// workers
for i := 0; i < cfg.numWorkers; i++ {
f := func(n int) func() error {
return func() error {
return s.workerFunc(n)
}
}(i)
s.workers.Go(f)
}
s.totalTasks.Add(int64(len(cfg.head.Blocks())))
for _, b := range cfg.head.Blocks() {
select {
case <-ctx.Done():
log.Errorw("context done while sending root tasks", ctx.Err())
cancel() // kill workers
return nil, ctx.Err()
case s.workerTasks.in <- walkTask{
c: b.Cid(),
taskType: blockTask,
}:
}
}
return s, nil
} }
func (s *walkScheduler) Wait() error { func (s *walkScheduler) Wait() error {
return s.grp.Wait() err := s.workers.Wait()
// all workers done. One would have reached genesis and notified the
// rest to exit. Yet, there might be some pending tasks in the queue,
// so we need to run a "single worker".
if err != nil {
log.Errorw("export workers finished with error", "error", err)
}
for {
if n := s.totalTasks.Load(); n == 0 {
break // finally fully done
}
select {
case task := <-s.workerTasks.out:
s.totalTasks.Add(-1)
if err != nil {
continue // just drain if errors happened.
}
err = s.processTask(task, 0)
}
}
close(s.results)
errWrite := <-s.writeErrorChan
if errWrite != nil {
log.Errorw("error writing to CAR file", "error", err)
return errWrite
}
s.workerTasks.Close() //nolint:errcheck
return err
} }
func (s *walkScheduler) enqueueIfNew(task *walkTask) { func (s *walkScheduler) enqueueIfNew(task walkTask) {
if task.c.Prefix().MhType == mh.IDENTITY { if task.c.Prefix().MhType == mh.IDENTITY {
//log.Infow("ignored", "cid", todo.c.String()) //log.Infow("ignored", "cid", todo.c.String())
return return
@ -234,118 +367,68 @@ func (s *walkScheduler) enqueueIfNew(task *walkTask) {
//log.Infow("ignored", "cid", todo.c.String()) //log.Infow("ignored", "cid", todo.c.String())
return return
} }
if _, ok := s.seen.Load(task.c); ok { if _, loaded := s.seen.LoadOrStore(task.c, struct{}{}); loaded {
// we already had it on the map
return return
} }
log.Debugw("enqueue", "type", task.taskType.String(), "cid", task.c.String()) log.Debugw("enqueue", "type", task.taskType.String(), "cid", task.c.String())
s.taskWg.Add(1) s.totalTasks.Add(1)
s.seen.Store(task.c, struct{}{}) s.workerTasks.in <- task
s.in <- task
} }
func (s *walkScheduler) startScheduler(ctx context.Context) { func (s *walkScheduler) sendFinish(workerN int) error {
s.grp.Go(func() error { log.Infow("worker finished work", "worker", workerN)
defer func() { s.totalTasks.Add(1)
log.Infow("walkScheduler shutting down") s.workerTasks.in <- walkTask{
close(s.out) taskType: finishTask,
}
return nil
}
// Because the workers may have exited early (due to the context being canceled). func (s *walkScheduler) workerFunc(workerN int) error {
for range s.out { log.Infow("starting worker", "worker", workerN)
s.taskWg.Done() for t := range s.workerTasks.out {
} s.totalTasks.Add(-1)
log.Info("closed scheduler out wait group") select {
case <-s.ctx.Done():
// Because the workers may have enqueued additional tasks. return s.ctx.Err()
for range s.in { default:
s.taskWg.Done() // A worker reached genesis, so we wind down and let others do
} // the same. Exit.
log.Info("closed scheduler in wait group") if t.taskType == finishTask {
return s.sendFinish(workerN)
// now, the waitgroup should be at 0, and the goroutine that was _waiting_ on it should have exited.
log.Infow("walkScheduler stopped")
}()
go func() {
s.taskWg.Wait()
close(s.in)
log.Info("closed scheduler in channel")
}()
for {
if n := len(s.stack) - 1; n >= 0 {
select {
case <-ctx.Done():
return ctx.Err()
case newJob, ok := <-s.in:
if !ok {
return nil
}
s.stack = append(s.stack, newJob)
case s.out <- s.stack[n]:
s.stack[n] = nil
s.stack = s.stack[:n]
}
} else {
select {
case <-ctx.Done():
return ctx.Err()
case newJob, ok := <-s.in:
if !ok {
return nil
}
s.stack = append(s.stack, newJob)
}
} }
} }
})
}
func (s *walkScheduler) startWorkers(ctx context.Context, out chan *walkResult) { err := s.processTask(t, workerN)
for i := int64(0); i < s.numWorkers; i++ { if err != nil {
s.grp.Go(func() error { return err
for task := range s.out { }
if err := s.work(ctx, task, out); err != nil { // continue
return err
}
}
return nil
})
} }
return nil
} }
type taskType int func (s *walkScheduler) processTask(t walkTask, workerN int) error {
if t.taskType == finishTask {
func (t taskType) String() string { return nil
switch t {
case Block:
return "block"
case Message:
return "message"
case Receipt:
return "receipt"
case State:
return "state"
case Dag:
return "dag"
} }
panic(fmt.Sprintf("unknow task %d", t))
}
const ( blk, err := s.store.Get(s.ctx, t.c)
Block taskType = iota if err != nil {
Message return xerrors.Errorf("writing object to car, bs.Get: %w", err)
Receipt }
State
Dag
)
func (s *walkScheduler) work(ctx context.Context, todo *walkTask, results chan *walkResult) error { s.results <- taskResult{
defer s.taskWg.Done() c: t.c,
// unseen cid, its a result b: blk,
results <- &walkResult{c: todo.c} }
// extract relevant dags to walk from the block // extract relevant dags to walk from the block
if todo.taskType == Block { if t.taskType == blockTask {
blk := todo.c blk := t.c
data, err := s.store.Get(ctx, blk) data, err := s.store.Get(s.ctx, blk)
if err != nil { if err != nil {
return err return err
} }
@ -359,128 +442,122 @@ func (s *walkScheduler) work(ctx context.Context, todo *walkTask, results chan *
if b.Height == 0 { if b.Height == 0 {
log.Info("exporting genesis block") log.Info("exporting genesis block")
for i := range b.Parents { for i := range b.Parents {
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.Parents[i], c: b.Parents[i],
taskType: Dag, taskType: dagTask,
}) })
} }
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.ParentStateRoot, c: b.ParentStateRoot,
taskType: State, taskType: stateTask,
}) })
return nil
return s.sendFinish(workerN)
} }
// enqueue block parents // enqueue block parents
for i := range b.Parents { for i := range b.Parents {
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.Parents[i], c: b.Parents[i],
taskType: Block, taskType: blockTask,
}) })
} }
if s.cfg.tail.Height() >= b.Height { if s.cfg.tail.Height() >= b.Height {
log.Debugw("tail reached", "cid", blk.String()) log.Debugw("tail reached: only blocks will be exported from now until genesis", "cid", blk.String())
return nil return nil
} }
if s.cfg.includeMessages { if s.cfg.includeMessages {
// enqueue block messages // enqueue block messages
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.Messages, c: b.Messages,
taskType: Message, taskType: messageTask,
}) })
} }
if s.cfg.includeReceipts { if s.cfg.includeReceipts {
// enqueue block receipts // enqueue block receipts
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.ParentMessageReceipts, c: b.ParentMessageReceipts,
taskType: Receipt, taskType: receiptTask,
}) })
} }
if s.cfg.includeState { if s.cfg.includeState {
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: b.ParentStateRoot, c: b.ParentStateRoot,
taskType: State, taskType: stateTask,
}) })
} }
return nil return nil
} }
data, err := s.store.Get(ctx, todo.c)
if err != nil { // Not a chain-block: we scan for CIDs in the raw block-data
return err return cbg.ScanForLinks(bytes.NewReader(blk.RawData()), func(c cid.Cid) {
} if t.c.Prefix().Codec != cid.DagCBOR || t.c.Prefix().MhType == mh.IDENTITY {
return cbg.ScanForLinks(bytes.NewReader(data.RawData()), func(c cid.Cid) {
if todo.c.Prefix().Codec != cid.DagCBOR || todo.c.Prefix().MhType == mh.IDENTITY {
return return
} }
s.enqueueIfNew(&walkTask{ s.enqueueIfNew(walkTask{
c: c, c: c,
taskType: Dag, taskType: dagTask,
}) })
}) })
} }
func (cs *ChainStore) WalkSnapshotRange(ctx context.Context, store bstore.Blockstore, head, tail *types.TipSet, messages, receipts, stateroots bool, workers int64, cb func(cid.Cid) error) error { func (cs *ChainStore) ExportRange(
start := time.Now() ctx context.Context,
log.Infow("walking snapshot range", "head", head.Key(), "tail", tail.Key(), "messages", messages, "receipts", receipts, "stateroots", stateroots, "workers", workers, "start", start) w io.Writer,
var tasks []*walkTask head, tail *types.TipSet,
for i := range head.Blocks() { messages, receipts, stateroots bool,
tasks = append(tasks, &walkTask{ workers int,
c: head.Blocks()[i].Cid(), cacheSize int) error {
taskType: 0,
}) h := &car.CarHeader{
Roots: head.Cids(),
Version: 1,
} }
cfg := &walkSchedulerConfig{ if err := car.WriteHeader(h, w); err != nil {
return xerrors.Errorf("failed to write car header: %s", err)
}
cacheStore, err := NewCachingBlockstore(cs.UnionStore(), cacheSize)
if err != nil {
return err
}
start := time.Now()
log.Infow("walking snapshot range",
"head", head.Key(),
"tail", tail.Key(),
"messages", messages,
"receipts", receipts,
"stateroots",
stateroots,
"workers", workers)
cfg := walkSchedulerConfig{
numWorkers: workers, numWorkers: workers,
head: head,
tail: tail, tail: tail,
includeMessages: messages, includeMessages: messages,
includeState: stateroots, includeState: stateroots,
includeReceipts: receipts, includeReceipts: receipts,
} }
pw, ctx := newWalkScheduler(ctx, store, cfg, tasks...) pw, err := newWalkScheduler(ctx, cacheStore, cfg, w)
// create a buffered channel for exported CID's scaled on the number of workers. if err != nil {
results := make(chan *walkResult, workers*64) return err
}
pw.startScheduler(ctx)
// workers accept channel and write results to it.
pw.startWorkers(ctx, results)
// used to wait until result channel has been drained.
resultsDone := make(chan struct{})
var cbErr error
go func() {
// signal we are done draining results when this method exits.
defer close(resultsDone)
// drain the results channel until is closes.
for res := range results {
if err := cb(res.c); err != nil {
log.Errorw("export range callback error", "error", err)
cbErr = err
return
}
}
}()
// wait until all workers are done. // wait until all workers are done.
err := pw.Wait() err = pw.Wait()
if err != nil { if err != nil {
log.Errorw("walker scheduler", "error", err) log.Errorw("walker scheduler", "error", err)
return err
} }
// workers are done, close the results channel.
close(results)
// wait until all results have been consumed before exiting (its buffered).
<-resultsDone
// if there was a callback error return it.
if cbErr != nil {
return cbErr
}
log.Infow("walking snapshot range complete", "duration", time.Since(start), "success", err == nil) log.Infow("walking snapshot range complete", "duration", time.Since(start), "success", err == nil)
return nil
// return any error encountered by the walker.
return err
} }
func (cs *ChainStore) WalkSnapshot(ctx context.Context, ts *types.TipSet, inclRecentRoots abi.ChainEpoch, skipOldMsgs, skipMsgReceipts bool, cb func(cid.Cid) error) error { func (cs *ChainStore) WalkSnapshot(ctx context.Context, ts *types.TipSet, inclRecentRoots abi.ChainEpoch, skipOldMsgs, skipMsgReceipts bool, cb func(cid.Cid) error) error {

View File

@ -6,6 +6,7 @@ import (
"encoding/base64" "encoding/base64"
"encoding/hex" "encoding/hex"
"encoding/json" "encoding/json"
"errors"
"fmt" "fmt"
"io" "io"
"os" "os"
@ -1153,12 +1154,12 @@ var ChainExportRangeCmd = &cli.Command{
Flags: []cli.Flag{ Flags: []cli.Flag{
&cli.StringFlag{ &cli.StringFlag{
Name: "head", Name: "head",
Usage: "specify tipset to start the export from", Usage: "specify tipset to start the export from (higher epoch)",
Value: "@head", Value: "@head",
}, },
&cli.StringFlag{ &cli.StringFlag{
Name: "tail", Name: "tail",
Usage: "specify tipset to end the export at", Usage: "specify tipset to end the export at (lower epoch)",
Value: "@tail", Value: "@tail",
}, },
&cli.BoolFlag{ &cli.BoolFlag{
@ -1176,7 +1177,7 @@ var ChainExportRangeCmd = &cli.Command{
Usage: "specify if stateroots should be include", Usage: "specify if stateroots should be include",
Value: false, Value: false,
}, },
&cli.Int64Flag{ &cli.IntFlag{
Name: "workers", Name: "workers",
Usage: "specify the number of workers", Usage: "specify the number of workers",
Value: 1, Value: 1,
@ -1234,10 +1235,14 @@ var ChainExportRangeCmd = &cli.Command{
} }
} }
if head.Height() < tail.Height() {
return errors.New("Height of --head tipset must be greater or equal to the height of the --tail tipset")
}
if cctx.Bool("internal") { if cctx.Bool("internal") {
if err := api.ChainExportRangeInternal(ctx, head.Key(), tail.Key(), &lapi.ChainExportConfig{ if err := api.ChainExportRangeInternal(ctx, head.Key(), tail.Key(), lapi.ChainExportConfig{
WriteBufferSize: cctx.Int("write-buffer"), WriteBufferSize: cctx.Int("write-buffer"),
Workers: cctx.Int64("workers"), NumWorkers: cctx.Int("workers"),
CacheSize: cctx.Int("cache-size"), CacheSize: cctx.Int("cache-size"),
IncludeMessages: cctx.Bool("messages"), IncludeMessages: cctx.Bool("messages"),
IncludeReceipts: cctx.Bool("receipts"), IncludeReceipts: cctx.Bool("receipts"),
@ -1248,9 +1253,9 @@ var ChainExportRangeCmd = &cli.Command{
return nil return nil
} }
stream, err := api.ChainExportRange(ctx, head.Key(), tail.Key(), &lapi.ChainExportConfig{ stream, err := api.ChainExportRange(ctx, head.Key(), tail.Key(), lapi.ChainExportConfig{
WriteBufferSize: cctx.Int("write-buffer"), WriteBufferSize: cctx.Int("write-buffer"),
Workers: cctx.Int64("workers"), NumWorkers: cctx.Int("workers"),
CacheSize: cctx.Int("cache-size"), CacheSize: cctx.Int("cache-size"),
IncludeMessages: cctx.Bool("messages"), IncludeMessages: cctx.Bool("messages"),
IncludeReceipts: cctx.Bool("receipts"), IncludeReceipts: cctx.Bool("receipts"),

View File

@ -9,6 +9,7 @@ import (
"io" "io"
"math" "math"
"os" "os"
"path/filepath"
"strconv" "strconv"
"strings" "strings"
"sync" "sync"
@ -592,7 +593,7 @@ func (m *ChainModule) ChainGetMessage(ctx context.Context, mc cid.Cid) (*types.M
return cm.VMMessage(), nil return cm.VMMessage(), nil
} }
func (a ChainAPI) ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg *api.ChainExportConfig) error { func (a ChainAPI) ChainExportRangeInternal(ctx context.Context, head, tail types.TipSetKey, cfg api.ChainExportConfig) error {
headTs, err := a.Chain.GetTipSetFromKey(ctx, head) headTs, err := a.Chain.GetTipSetFromKey(ctx, head)
if err != nil { if err != nil {
return xerrors.Errorf("loading tipset %s: %w", head, err) return xerrors.Errorf("loading tipset %s: %w", head, err)
@ -601,10 +602,18 @@ func (a ChainAPI) ChainExportRangeInternal(ctx context.Context, head, tail types
if err != nil { if err != nil {
return xerrors.Errorf("loading tipset %s: %w", tail, err) return xerrors.Errorf("loading tipset %s: %w", tail, err)
} }
f, err := os.Create(fmt.Sprintf("./snapshot_%d_%d_%d.car", tailTs.Height(), headTs.Height(), time.Now().Unix())) fileName := fmt.Sprintf("./snapshot_%d_%d_%d.car", tailTs.Height(), headTs.Height(), time.Now().Unix())
absFileName, err := filepath.Abs(fileName)
if err != nil { if err != nil {
return err return err
} }
f, err := os.Create(fileName)
if err != nil {
return err
}
log.Infow("Exporting chain range", "path", absFileName)
// buffer writes to the chain export file. // buffer writes to the chain export file.
bw := bufio.NewWriterSize(f, cfg.WriteBufferSize) bw := bufio.NewWriterSize(f, cfg.WriteBufferSize)
@ -617,14 +626,20 @@ func (a ChainAPI) ChainExportRangeInternal(ctx context.Context, head, tail types
} }
}() }()
if err := a.Chain.ExportRange(ctx, headTs, tailTs, cfg.IncludeMessages, cfg.IncludeReceipts, cfg.IncludeStateRoots, cfg.Workers, cfg.CacheSize, bw); err != nil { if err := a.Chain.ExportRange(ctx,
bw,
headTs, tailTs,
cfg.IncludeMessages, cfg.IncludeReceipts, cfg.IncludeStateRoots,
cfg.NumWorkers, cfg.CacheSize,
); err != nil {
return fmt.Errorf("exporting chain range: %w", err) return fmt.Errorf("exporting chain range: %w", err)
} }
// FIXME: return progress.
return nil return nil
} }
func (a ChainAPI) ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg *api.ChainExportConfig) (<-chan []byte, error) { func (a ChainAPI) ChainExportRange(ctx context.Context, head, tail types.TipSetKey, cfg api.ChainExportConfig) (<-chan []byte, error) {
headTs, err := a.Chain.GetTipSetFromKey(ctx, head) headTs, err := a.Chain.GetTipSetFromKey(ctx, head)
if err != nil { if err != nil {
return nil, xerrors.Errorf("loading tipset %s: %w", head, err) return nil, xerrors.Errorf("loading tipset %s: %w", head, err)
@ -637,8 +652,14 @@ func (a ChainAPI) ChainExportRange(ctx context.Context, head, tail types.TipSetK
out := make(chan []byte) out := make(chan []byte)
go func() { go func() {
bw := bufio.NewWriterSize(w, cfg.WriteBufferSize) bw := bufio.NewWriterSize(w, cfg.WriteBufferSize)
err := a.Chain.ExportRange(
err := a.Chain.ExportRange(ctx, headTs, tailTs, cfg.IncludeMessages, cfg.IncludeReceipts, cfg.IncludeStateRoots, cfg.Workers, cfg.CacheSize, bw) ctx,
bw,
headTs,
tailTs,
cfg.IncludeMessages, cfg.IncludeReceipts, cfg.IncludeStateRoots,
cfg.NumWorkers, cfg.CacheSize,
)
bw.Flush() //nolint:errcheck // it is a write to a pipe bw.Flush() //nolint:errcheck // it is a write to a pipe
w.CloseWithError(err) //nolint:errcheck // it is a pipe w.CloseWithError(err) //nolint:errcheck // it is a pipe
}() }()