From f27d54b37b40ed3e871ffda1272b52e32b4d5e06 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Thu, 9 Nov 2023 23:28:36 +0100 Subject: [PATCH 01/24] feat: lpwinning: Initial task layout --- miner/miner.go | 39 +++-- miner/testminer.go | 14 +- provider/lpwinning/winning_task.go | 258 +++++++++++++++++++++++++++++ 3 files changed, 294 insertions(+), 17 deletions(-) create mode 100644 provider/lpwinning/winning_task.go diff --git a/miner/miner.go b/miner/miner.go index 329f3417190..d11e9d4aa04 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -71,7 +71,7 @@ func NewMiner(api v1api.FullNode, epp gen.WinningPoStProver, addr address.Addres api: api, epp: epp, address: addr, - waitFunc: func(ctx context.Context, baseTime uint64) (func(bool, abi.ChainEpoch, error), abi.ChainEpoch, error) { + propagationWaitFunc: func(ctx context.Context, baseTime uint64) (func(bool, abi.ChainEpoch, error), abi.ChainEpoch, error) { // wait around for half the block time in case other parents come in // // if we're mining a block in the past via catch-up/rush mining, @@ -114,7 +114,7 @@ type Miner struct { stop chan struct{} stopping chan struct{} - waitFunc waitFunc + propagationWaitFunc waitFunc // lastWork holds the last MiningBase we built upon. lastWork *MiningBase @@ -205,15 +205,21 @@ func (m *Miner) mine(ctx context.Context) { ctx, span := trace.StartSpan(ctx, "/mine") defer span.End() + // Perform the Winning PoSt warmup in a separate goroutine. go m.doWinPoStWarmup(ctx) var lastBase MiningBase + + // Start the main mining loop. minerLoop: for { + // Prepare a context for a single node operation. ctx := cliutil.OnSingleNode(ctx) + // Handle stop signals. select { case <-m.stop: + // If a stop signal is received, clean up and exit the mining loop. stopping := m.stopping m.stop = nil m.stopping = nil @@ -223,10 +229,11 @@ minerLoop: default: } - var base *MiningBase + var base *MiningBase // NOTE: This points to m.lastWork; Incrementing nulls here will increment it there. var onDone func(bool, abi.ChainEpoch, error) var injectNulls abi.ChainEpoch + // Look for the best mining candidate. for { prebase, err := m.GetBestMiningCandidate(ctx) if err != nil { @@ -237,6 +244,7 @@ minerLoop: continue } + // Check if we have a new base or if the current base is still valid. if base != nil && base.TipSet.Height() == prebase.TipSet.Height() && base.NullRounds == prebase.NullRounds { base = prebase break @@ -253,13 +261,13 @@ minerLoop: // best mining candidate at that time. // Wait until propagation delay period after block we plan to mine on - onDone, injectNulls, err = m.waitFunc(ctx, prebase.TipSet.MinTimestamp()) + onDone, injectNulls, err = m.propagationWaitFunc(ctx, prebase.TipSet.MinTimestamp()) if err != nil { log.Error(err) continue } - // just wait for the beacon entry to become available before we select our final mining base + // Ensure the beacon entry is available before finalizing the mining base. _, err = m.api.StateGetBeaconEntry(ctx, prebase.TipSet.Height()+prebase.NullRounds+1) if err != nil { log.Errorf("failed getting beacon entry: %s", err) @@ -272,8 +280,9 @@ minerLoop: base = prebase } - base.NullRounds += injectNulls // testing + base.NullRounds += injectNulls // Adjust for testing purposes. + // Check for repeated mining candidates and handle sleep for the next round. if base.TipSet.Equals(lastBase.TipSet) && lastBase.NullRounds == base.NullRounds { log.Warnf("BestMiningCandidate from the previous round: %s (nulls:%d)", lastBase.TipSet.Cids(), lastBase.NullRounds) if !m.niceSleep(time.Duration(build.BlockDelaySecs) * time.Second) { @@ -282,6 +291,7 @@ minerLoop: continue } + // Attempt to mine a block. b, err := m.mineOne(ctx, base) if err != nil { log.Errorf("mining block failed: %+v", err) @@ -299,9 +309,12 @@ minerLoop: } onDone(b != nil, h, nil) + // Process the mined block. if b != nil { + // Record the event of mining a block. m.journal.RecordEvent(m.evtTypes[evtTypeBlockMined], func() interface{} { return map[string]interface{}{ + // Data about the mined block. "parents": base.TipSet.Cids(), "nulls": base.NullRounds, "epoch": b.Header.Height, @@ -312,19 +325,23 @@ minerLoop: btime := time.Unix(int64(b.Header.Timestamp), 0) now := build.Clock.Now() + // Handle timing for broadcasting the block. switch { case btime == now: // block timestamp is perfectly aligned with time. case btime.After(now): + // Wait until it's time to broadcast the block. if !m.niceSleep(build.Clock.Until(btime)) { log.Warnf("received interrupt while waiting to broadcast block, will shutdown after block is sent out") build.Clock.Sleep(build.Clock.Until(btime)) } default: + // Log if the block was mined in the past. log.Warnw("mined block in the past", "block-time", btime, "time", build.Clock.Now(), "difference", build.Clock.Since(btime)) } + // Check for slash filter conditions. if os.Getenv("LOTUS_MINER_NO_SLASHFILTER") != "_yes_i_know_i_can_and_probably_will_lose_all_my_fil_and_power_" && !build.IsNearUpgrade(base.TipSet.Height(), build.UpgradeWatermelonFixHeight) { witness, fault, err := m.sf.MinedBlock(ctx, b.Header, base.TipSet.Height()+base.NullRounds) if err != nil { @@ -339,25 +356,27 @@ minerLoop: } } + // Check for blocks created at the same height. if _, ok := m.minedBlockHeights.Get(b.Header.Height); ok { log.Warnw("Created a block at the same height as another block we've created", "height", b.Header.Height, "miner", b.Header.Miner, "parents", b.Header.Parents) continue } + // Add the block height to the mined block heights. m.minedBlockHeights.Add(b.Header.Height, true) + // Submit the newly mined block. if err := m.api.SyncSubmitBlock(ctx, b); err != nil { log.Errorf("failed to submit newly mined block: %+v", err) } } else { + // If no block was mined, increase the null rounds and wait for the next epoch. base.NullRounds++ - // Wait until the next epoch, plus the propagation delay, so a new tipset - // has enough time to form. - // - // See: https://github.com/filecoin-project/lotus/issues/1845 + // Calculate the time for the next round. nextRound := time.Unix(int64(base.TipSet.MinTimestamp()+build.BlockDelaySecs*uint64(base.NullRounds))+int64(build.PropagationDelaySecs), 0) + // Wait for the next round or stop signal. select { case <-build.Clock.After(build.Clock.Until(nextRound)): case <-m.stop: diff --git a/miner/testminer.go b/miner/testminer.go index f1d11bae0a6..e23b26ae29f 100644 --- a/miner/testminer.go +++ b/miner/testminer.go @@ -28,13 +28,13 @@ func NewTestMiner(nextCh <-chan MineReq, addr address.Address) func(v1api.FullNo } m := &Miner{ - api: api, - waitFunc: chanWaiter(nextCh), - epp: epp, - minedBlockHeights: arc, - address: addr, - sf: slashfilter.New(ds.NewMapDatastore()), - journal: journal.NilJournal(), + api: api, + propagationWaitFunc: chanWaiter(nextCh), + epp: epp, + minedBlockHeights: arc, + address: addr, + sf: slashfilter.New(ds.NewMapDatastore()), + journal: journal.NilJournal(), } if err := m.Start(context.TODO()); err != nil { diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go new file mode 100644 index 00000000000..d48da8583ec --- /dev/null +++ b/provider/lpwinning/winning_task.go @@ -0,0 +1,258 @@ +package lpwinning + +import ( + "context" + "crypto/rand" + "encoding/binary" + "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/lib/harmony/harmonytask" + "github.com/filecoin-project/lotus/lib/harmony/resources" + logging "github.com/ipfs/go-log/v2" + "os" + "time" +) + +var log = logging.Logger("lpwinning") + +type WinPostTask struct { + max abi.SectorNumber + + // lastWork holds the last MiningBase we built upon. + lastWork *MiningBase + + api WinPostAPI +} + +type WinPostAPI interface { + ChainHead(context.Context) (*types.TipSet, error) + ChainTipSetWeight(context.Context, types.TipSetKey) (types.BigInt, error) + + StateGetBeaconEntry(context.Context, abi.ChainEpoch) (*types.BeaconEntry, error) + SyncSubmitBlock(context.Context, *types.BlockMsg) error +} + +func NewWinPostTask(max abi.SectorNumber) *WinPostTask { + // todo run warmup +} + +func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (done bool, err error) { + // TODO THIS WILL BASICALLY BE A mineOne() function + + //TODO implement me + panic("implement me") +} + +func (t *WinPostTask) CanAccept(ids []harmonytask.TaskID, engine *harmonytask.TaskEngine) (*harmonytask.TaskID, error) { + //TODO implement me + panic("implement me") +} + +func (t *WinPostTask) TypeDetails() harmonytask.TaskTypeDetails { + return harmonytask.TaskTypeDetails{ + Name: "WinPost", + Max: 10, // todo + MaxFailures: 3, + Follows: nil, + Cost: resources.Resources{ + Cpu: 1, + + // todo set to something for 32/64G sector sizes? Technically windowPoSt is happy on a CPU + // but it will use a GPU if available + Gpu: 0, + + Ram: 1 << 30, // todo arbitrary number + }, + } +} + +func (t *WinPostTask) Adder(taskFunc harmonytask.AddTaskFunc) { + //TODO implement me + panic("implement me") +} + +// MiningBase is the tipset on top of which we plan to construct our next block. +// Refer to godocs on GetBestMiningCandidate. +type MiningBase struct { + TipSet *types.TipSet + ComputeTime time.Time + NullRounds abi.ChainEpoch +} + +func (t *WinPostTask) mine(ctx context.Context) { + var lastBase MiningBase + + // Start the main mining loop. + for { + // todo handle stop signals? + + var base *MiningBase + + // Look for the best mining candidate. + for { + prebase, err := t.GetBestMiningCandidate(ctx) + if err != nil { + log.Errorf("failed to get best mining candidate: %s", err) + time.Sleep(5 * time.Second) + continue + } + + // Check if we have a new base or if the current base is still valid. + if base != nil && base.TipSet.Height() == prebase.TipSet.Height() && base.NullRounds == prebase.NullRounds { + // We have a valid base. + base = prebase + break + } + + // TODO: need to change the orchestration here. the problem is that + // we are waiting *after* we enter this loop and selecta mining + // candidate, which is almost certain to change in multiminer + // tests. Instead, we should block before entering the loop, so + // that when the test 'MineOne' function is triggered, we pull our + // best mining candidate at that time. + + // Wait until propagation delay period after block we plan to mine on + { + // if we're mining a block in the past via catch-up/rush mining, + // such as when recovering from a network halt, this sleep will be + // for a negative duration, and therefore **will return + // immediately**. + // + // the result is that we WILL NOT wait, therefore fast-forwarding + // and thus healing the chain by backfilling it with null rounds + // rapidly. + baseTs := prebase.TipSet.MinTimestamp() + build.PropagationDelaySecs + baseT := time.Unix(int64(baseTs), 0) + baseT = baseT.Add(randTimeOffset(time.Second)) + time.Sleep(time.Until(baseT)) + } + + // Ensure the beacon entry is available before finalizing the mining base. + _, err = t.api.StateGetBeaconEntry(ctx, prebase.TipSet.Height()+prebase.NullRounds+1) + if err != nil { + log.Errorf("failed getting beacon entry: %s", err) + time.Sleep(time.Second) + continue + } + + base = prebase + } + + // Check for repeated mining candidates and handle sleep for the next round. + if base.TipSet.Equals(lastBase.TipSet) && lastBase.NullRounds == base.NullRounds { + log.Warnf("BestMiningCandidate from the previous round: %s (nulls:%d)", lastBase.TipSet.Cids(), lastBase.NullRounds) + time.Sleep(time.Duration(build.BlockDelaySecs) * time.Second) + continue + } + + // Attempt to mine a block. + b, err := m.mineOne(ctx, base) + if err != nil { + log.Errorf("mining block failed: %+v", err) + time.Sleep(time.Second) + continue + } + lastBase = *base + + // todo figure out this whole bottom section + // we won't know if we've mined a block here, we just submit a task + // making attempts to mine one + + // Process the mined block. + if b != nil { + btime := time.Unix(int64(b.Header.Timestamp), 0) + now := build.Clock.Now() + // Handle timing for broadcasting the block. + switch { + case btime == now: + // block timestamp is perfectly aligned with time. + case btime.After(now): + // Wait until it's time to broadcast the block. + if !m.niceSleep(build.Clock.Until(btime)) { + log.Warnf("received interrupt while waiting to broadcast block, will shutdown after block is sent out") + build.Clock.Sleep(build.Clock.Until(btime)) + } + default: + // Log if the block was mined in the past. + log.Warnw("mined block in the past", + "block-time", btime, "time", build.Clock.Now(), "difference", build.Clock.Since(btime)) + } + + // Check for slash filter conditions. + if os.Getenv("LOTUS_MINER_NO_SLASHFILTER") != "_yes_i_know_i_can_and_probably_will_lose_all_my_fil_and_power_" && !build.IsNearUpgrade(base.TipSet.Height(), build.UpgradeWatermelonFixHeight) { + witness, fault, err := m.sf.MinedBlock(ctx, b.Header, base.TipSet.Height()+base.NullRounds) + if err != nil { + log.Errorf(" SLASH FILTER ERRORED: %s", err) + // Continue here, because it's _probably_ wiser to not submit this block + continue + } + + if fault { + log.Errorf(" SLASH FILTER DETECTED FAULT due to blocks %s and %s", b.Header.Cid(), witness) + continue + } + } + + // Submit the newly mined block. + if err := t.api.SyncSubmitBlock(ctx, b); err != nil { + log.Errorf("failed to submit newly mined block: %+v", err) + } + } else { + // If no block was mined, increase the null rounds and wait for the next epoch. + base.NullRounds++ + + // Calculate the time for the next round. + nextRound := time.Unix(int64(base.TipSet.MinTimestamp()+build.BlockDelaySecs*uint64(base.NullRounds))+int64(build.PropagationDelaySecs), 0) + + // Wait for the next round. + time.Sleep(time.Until(nextRound)) + } + } +} + +// GetBestMiningCandidate implements the fork choice rule from a miner's +// perspective. +// +// It obtains the current chain head (HEAD), and compares it to the last tipset +// we selected as our mining base (LAST). If HEAD's weight is larger than +// LAST's weight, it selects HEAD to build on. Else, it selects LAST. +func (t *WinPostTask) GetBestMiningCandidate(ctx context.Context) (*MiningBase, error) { + bts, err := t.api.ChainHead(ctx) + if err != nil { + return nil, err + } + + if t.lastWork != nil { + if t.lastWork.TipSet.Equals(bts) { + return t.lastWork, nil + } + + btsw, err := t.api.ChainTipSetWeight(ctx, bts.Key()) + if err != nil { + return nil, err + } + ltsw, err := t.api.ChainTipSetWeight(ctx, t.lastWork.TipSet.Key()) + if err != nil { + t.lastWork = nil + return nil, err + } + + if types.BigCmp(btsw, ltsw) <= 0 { + return t.lastWork, nil + } + } + + t.lastWork = &MiningBase{TipSet: bts, ComputeTime: time.Now()} + return t.lastWork, nil +} + +func randTimeOffset(width time.Duration) time.Duration { + buf := make([]byte, 8) + rand.Reader.Read(buf) //nolint:errcheck + val := time.Duration(binary.BigEndian.Uint64(buf) % uint64(width)) + + return val - (width / 2) +} + +var _ harmonytask.TaskInterface = &WinPostTask{} From f4d86716ab7108dede700f12fd6752c1178e541f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 11:40:56 +0100 Subject: [PATCH 02/24] lpwinning: More scribbling around adder --- provider/lpwinning/winning_task.go | 76 +++++++++++++++++++++++++++++- 1 file changed, 75 insertions(+), 1 deletion(-) diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index d48da8583ec..1f9161e3a63 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -80,7 +80,81 @@ type MiningBase struct { NullRounds abi.ChainEpoch } -func (t *WinPostTask) mine(ctx context.Context) { +func (mb MiningBase) baseTime() time.Time { + tsTime := time.Unix(int64(mb.TipSet.MinTimestamp()), 0) + nullDelay := build.BlockDelaySecs * uint64(mb.NullRounds) + tsTime = tsTime.Add(time.Duration(nullDelay) * time.Second) + return tsTime +} + +func (mb MiningBase) afterPropDelay() time.Time { + base := mb.baseTime() + base.Add(randTimeOffset(time.Second)) + return base +} + +func retry1[R any](f func() (R, error)) R { + for { + r, err := f() + if err == nil { + return r + } + + log.Errorw("error in mining loop, retrying", "error", err) + time.Sleep(time.Second) + } +} + +func (t *WinPostTask) mineBasic(ctx context.Context) { + var workBase MiningBase + + { + head := retry1(func() (*types.TipSet, error) { + return t.api.ChainHead(ctx) + }) + + workBase = MiningBase{ + TipSet: head, + NullRounds: 0, + ComputeTime: time.Now(), + } + } + + for { + // wait for propagation delay + time.Sleep(time.Until(workBase.afterPropDelay())) + + // check current best candidate + maybeBase := retry1(func() (*types.TipSet, error) { + return t.api.ChainHead(ctx) + }) + + if workBase.TipSet.Equals(maybeBase) { + workBase.NullRounds++ + } else { + btsw := retry1(func() (types.BigInt, error) { + return t.api.ChainTipSetWeight(ctx, maybeBase.Key()) + }) + + ltsw := retry1(func() (types.BigInt, error) { + return t.api.ChainTipSetWeight(ctx, workBase.TipSet.Key()) + }) + + if types.BigCmp(btsw, ltsw) <= 0 { + workBase.NullRounds++ + } else { + workBase = MiningBase{ + TipSet: maybeBase, + NullRounds: 0, + ComputeTime: time.Now(), + } + } + } + + } +} + +func (t *WinPostTask) mine2(ctx context.Context) { var lastBase MiningBase // Start the main mining loop. From f30a7db5dfc9aab535cb72b14b67e2d834b43ed7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 18:00:21 +0100 Subject: [PATCH 03/24] lpwinning: adder done --- lib/harmony/harmonydb/sql/20231110.sql | 24 ++ provider/lpwinning/winning_task.go | 348 +++++++++++++++---------- 2 files changed, 229 insertions(+), 143 deletions(-) create mode 100644 lib/harmony/harmonydb/sql/20231110.sql diff --git a/lib/harmony/harmonydb/sql/20231110.sql b/lib/harmony/harmonydb/sql/20231110.sql new file mode 100644 index 00000000000..910ff05fa51 --- /dev/null +++ b/lib/harmony/harmonydb/sql/20231110.sql @@ -0,0 +1,24 @@ +create table mining_tasks +( + task_id bigint not null + constraint mining_tasks_pk + primary key, + sp_id bigint, + epoch bigint, + constraint mining_tasks_sp_epoch + unique (sp_id, epoch) +); + +create table mining_base_block +( + id bigserial not null + constraint mining_base_block_pk + primary key, + task_id bigint not null + constraint mining_base_block_mining_tasks_task_id_fk + references mining_tasks + on delete cascade, + block_cid text not null + constraint mining_base_block_cid_k + unique +); diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index 1f9161e3a63..d4d556548ce 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -4,13 +4,17 @@ import ( "context" "crypto/rand" "encoding/binary" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/lib/harmony/harmonytask" "github.com/filecoin-project/lotus/lib/harmony/resources" + "github.com/filecoin-project/lotus/lib/promise" + "github.com/filecoin-project/lotus/node/modules/dtypes" logging "github.com/ipfs/go-log/v2" - "os" + "golang.org/x/xerrors" "time" ) @@ -22,7 +26,10 @@ type WinPostTask struct { // lastWork holds the last MiningBase we built upon. lastWork *MiningBase - api WinPostAPI + api WinPostAPI + actors []dtypes.MinerAddress + + mineTF promise.Promise[harmonytask.AddTaskFunc] } type WinPostAPI interface { @@ -68,8 +75,7 @@ func (t *WinPostTask) TypeDetails() harmonytask.TaskTypeDetails { } func (t *WinPostTask) Adder(taskFunc harmonytask.AddTaskFunc) { - //TODO implement me - panic("implement me") + t.mineTF.Set(taskFunc) } // MiningBase is the tipset on top of which we plan to construct our next block. @@ -77,13 +83,18 @@ func (t *WinPostTask) Adder(taskFunc harmonytask.AddTaskFunc) { type MiningBase struct { TipSet *types.TipSet ComputeTime time.Time - NullRounds abi.ChainEpoch + AddRounds abi.ChainEpoch +} + +func (mb MiningBase) epoch() abi.ChainEpoch { + // return the epoch that will result from mining on this base + return mb.TipSet.Height() + mb.AddRounds + 1 } func (mb MiningBase) baseTime() time.Time { tsTime := time.Unix(int64(mb.TipSet.MinTimestamp()), 0) - nullDelay := build.BlockDelaySecs * uint64(mb.NullRounds) - tsTime = tsTime.Add(time.Duration(nullDelay) * time.Second) + roundDelay := build.BlockDelaySecs * uint64(mb.AddRounds+1) + tsTime = tsTime.Add(time.Duration(roundDelay) * time.Second) return tsTime } @@ -108,6 +119,8 @@ func retry1[R any](f func() (R, error)) R { func (t *WinPostTask) mineBasic(ctx context.Context) { var workBase MiningBase + taskFn := t.mineTF.Val(ctx) + { head := retry1(func() (*types.TipSet, error) { return t.api.ChainHead(ctx) @@ -115,13 +128,26 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { workBase = MiningBase{ TipSet: head, - NullRounds: 0, + AddRounds: 0, ComputeTime: time.Now(), } } + /* + + /- T+0 == workBase.baseTime + | + >--------*------*--------[wait until next round]-----> + | + |- T+PD == workBase.afterPropDelay+(~1s) + |- Here we acquire the new workBase, and start a new round task + \- Then we loop around, and wait for the next head + + time --> + */ + for { - // wait for propagation delay + // wait for *NEXT* propagation delay time.Sleep(time.Until(workBase.afterPropDelay())) // check current best candidate @@ -130,7 +156,9 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { }) if workBase.TipSet.Equals(maybeBase) { - workBase.NullRounds++ + // workbase didn't change in the new round so we have a null round here + workBase.AddRounds++ + log.Debugw("workbase update", "tipset", workBase.TipSet.Cids(), "nulls", workBase.AddRounds, "lastUpdate", time.Since(workBase.ComputeTime), "type", "same-tipset") } else { btsw := retry1(func() (types.BigInt, error) { return t.api.ChainTipSetWeight(ctx, maybeBase.Key()) @@ -141,149 +169,182 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { }) if types.BigCmp(btsw, ltsw) <= 0 { - workBase.NullRounds++ + // new tipset for some reason has less weight than the old one, assume null round here + // NOTE: the backing node may have reorged, or manually changed head + workBase.AddRounds++ + log.Debugw("workbase update", "tipset", workBase.TipSet.Cids(), "nulls", workBase.AddRounds, "lastUpdate", time.Since(workBase.ComputeTime), "type", "prefer-local-weight") } else { + // new tipset has more weight, so we should mine on it, no null round here + log.Debugw("workbase update", "tipset", workBase.TipSet.Cids(), "nulls", workBase.AddRounds, "lastUpdate", time.Since(workBase.ComputeTime), "type", "prefer-new-tipset") + workBase = MiningBase{ TipSet: maybeBase, - NullRounds: 0, + AddRounds: 0, ComputeTime: time.Now(), } } } + // dispatch mining task + // (note equivocation prevention is handled by the mining code) + + for _, act := range t.actors { + spID, err := address.IDFromAddress(address.Address(act)) + if err != nil { + log.Errorf("failed to get spID from address %s: %s", act, err) + continue + } + + taskFn(func(id harmonytask.TaskID, tx *harmonydb.Tx) (shouldCommit bool, seriousError error) { + _, err := tx.Exec(`INSERT INTO mining_tasks (task_id, sp_id, epoch) VALUES ($1, $2, $3)`, id, spID, workBase.epoch()) + if err != nil { + return false, xerrors.Errorf("inserting mining_tasks: %w", err) + } + + for _, c := range workBase.TipSet.Cids() { + _, err = tx.Exec(`INSERT INTO mining_base_block (task_id, block_cid) VALUES ($1, $2)`, id, c) + if err != nil { + return false, xerrors.Errorf("inserting mining base blocks: %w", err) + } + } + + return true, nil // no errors, commit the transaction + }) + } } } -func (t *WinPostTask) mine2(ctx context.Context) { - var lastBase MiningBase +/* + func (t *WinPostTask) mine2(ctx context.Context) { + var lastBase MiningBase - // Start the main mining loop. - for { - // todo handle stop signals? + // Start the main mining loop. + for { + // todo handle stop signals? - var base *MiningBase + var base *MiningBase - // Look for the best mining candidate. - for { - prebase, err := t.GetBestMiningCandidate(ctx) - if err != nil { - log.Errorf("failed to get best mining candidate: %s", err) - time.Sleep(5 * time.Second) - continue - } + // Look for the best mining candidate. + for { + prebase, err := t.GetBestMiningCandidate(ctx) + if err != nil { + log.Errorf("failed to get best mining candidate: %s", err) + time.Sleep(5 * time.Second) + continue + } + + // Check if we have a new base or if the current base is still valid. + if base != nil && base.TipSet.Height() == prebase.TipSet.Height() && base.AddRounds == prebase.AddRounds { + // We have a valid base. + base = prebase + break + } + + // TODO: need to change the orchestration here. the problem is that + // we are waiting *after* we enter this loop and selecta mining + // candidate, which is almost certain to change in multiminer + // tests. Instead, we should block before entering the loop, so + // that when the test 'MineOne' function is triggered, we pull our + // best mining candidate at that time. + + // Wait until propagation delay period after block we plan to mine on + { + // if we're mining a block in the past via catch-up/rush mining, + // such as when recovering from a network halt, this sleep will be + // for a negative duration, and therefore **will return + // immediately**. + // + // the result is that we WILL NOT wait, therefore fast-forwarding + // and thus healing the chain by backfilling it with null rounds + // rapidly. + baseTs := prebase.TipSet.MinTimestamp() + build.PropagationDelaySecs + baseT := time.Unix(int64(baseTs), 0) + baseT = baseT.Add(randTimeOffset(time.Second)) + time.Sleep(time.Until(baseT)) + } + + // Ensure the beacon entry is available before finalizing the mining base. + _, err = t.api.StateGetBeaconEntry(ctx, prebase.TipSet.Height()+prebase.AddRounds+1) + if err != nil { + log.Errorf("failed getting beacon entry: %s", err) + time.Sleep(time.Second) + continue + } - // Check if we have a new base or if the current base is still valid. - if base != nil && base.TipSet.Height() == prebase.TipSet.Height() && base.NullRounds == prebase.NullRounds { - // We have a valid base. base = prebase - break } - // TODO: need to change the orchestration here. the problem is that - // we are waiting *after* we enter this loop and selecta mining - // candidate, which is almost certain to change in multiminer - // tests. Instead, we should block before entering the loop, so - // that when the test 'MineOne' function is triggered, we pull our - // best mining candidate at that time. - - // Wait until propagation delay period after block we plan to mine on - { - // if we're mining a block in the past via catch-up/rush mining, - // such as when recovering from a network halt, this sleep will be - // for a negative duration, and therefore **will return - // immediately**. - // - // the result is that we WILL NOT wait, therefore fast-forwarding - // and thus healing the chain by backfilling it with null rounds - // rapidly. - baseTs := prebase.TipSet.MinTimestamp() + build.PropagationDelaySecs - baseT := time.Unix(int64(baseTs), 0) - baseT = baseT.Add(randTimeOffset(time.Second)) - time.Sleep(time.Until(baseT)) + // Check for repeated mining candidates and handle sleep for the next round. + if base.TipSet.Equals(lastBase.TipSet) && lastBase.AddRounds == base.AddRounds { + log.Warnf("BestMiningCandidate from the previous round: %s (nulls:%d)", lastBase.TipSet.Cids(), lastBase.AddRounds) + time.Sleep(time.Duration(build.BlockDelaySecs) * time.Second) + continue } - // Ensure the beacon entry is available before finalizing the mining base. - _, err = t.api.StateGetBeaconEntry(ctx, prebase.TipSet.Height()+prebase.NullRounds+1) + // Attempt to mine a block. + b, err := m.mineOne(ctx, base) if err != nil { - log.Errorf("failed getting beacon entry: %s", err) + log.Errorf("mining block failed: %+v", err) time.Sleep(time.Second) continue } - - base = prebase - } - - // Check for repeated mining candidates and handle sleep for the next round. - if base.TipSet.Equals(lastBase.TipSet) && lastBase.NullRounds == base.NullRounds { - log.Warnf("BestMiningCandidate from the previous round: %s (nulls:%d)", lastBase.TipSet.Cids(), lastBase.NullRounds) - time.Sleep(time.Duration(build.BlockDelaySecs) * time.Second) - continue - } - - // Attempt to mine a block. - b, err := m.mineOne(ctx, base) - if err != nil { - log.Errorf("mining block failed: %+v", err) - time.Sleep(time.Second) - continue - } - lastBase = *base - - // todo figure out this whole bottom section - // we won't know if we've mined a block here, we just submit a task - // making attempts to mine one - - // Process the mined block. - if b != nil { - btime := time.Unix(int64(b.Header.Timestamp), 0) - now := build.Clock.Now() - // Handle timing for broadcasting the block. - switch { - case btime == now: - // block timestamp is perfectly aligned with time. - case btime.After(now): - // Wait until it's time to broadcast the block. - if !m.niceSleep(build.Clock.Until(btime)) { - log.Warnf("received interrupt while waiting to broadcast block, will shutdown after block is sent out") - build.Clock.Sleep(build.Clock.Until(btime)) + lastBase = *base + + // todo figure out this whole bottom section + // we won't know if we've mined a block here, we just submit a task + // making attempts to mine one + + // Process the mined block. + if b != nil { + btime := time.Unix(int64(b.Header.Timestamp), 0) + now := build.Clock.Now() + // Handle timing for broadcasting the block. + switch { + case btime == now: + // block timestamp is perfectly aligned with time. + case btime.After(now): + // Wait until it's time to broadcast the block. + if !m.niceSleep(build.Clock.Until(btime)) { + log.Warnf("received interrupt while waiting to broadcast block, will shutdown after block is sent out") + build.Clock.Sleep(build.Clock.Until(btime)) + } + default: + // Log if the block was mined in the past. + log.Warnw("mined block in the past", + "block-time", btime, "time", build.Clock.Now(), "difference", build.Clock.Since(btime)) } - default: - // Log if the block was mined in the past. - log.Warnw("mined block in the past", - "block-time", btime, "time", build.Clock.Now(), "difference", build.Clock.Since(btime)) - } - // Check for slash filter conditions. - if os.Getenv("LOTUS_MINER_NO_SLASHFILTER") != "_yes_i_know_i_can_and_probably_will_lose_all_my_fil_and_power_" && !build.IsNearUpgrade(base.TipSet.Height(), build.UpgradeWatermelonFixHeight) { - witness, fault, err := m.sf.MinedBlock(ctx, b.Header, base.TipSet.Height()+base.NullRounds) - if err != nil { - log.Errorf(" SLASH FILTER ERRORED: %s", err) - // Continue here, because it's _probably_ wiser to not submit this block - continue + // Check for slash filter conditions. + if os.Getenv("LOTUS_MINER_NO_SLASHFILTER") != "_yes_i_know_i_can_and_probably_will_lose_all_my_fil_and_power_" && !build.IsNearUpgrade(base.TipSet.Height(), build.UpgradeWatermelonFixHeight) { + witness, fault, err := m.sf.MinedBlock(ctx, b.Header, base.TipSet.Height()+base.AddRounds) + if err != nil { + log.Errorf(" SLASH FILTER ERRORED: %s", err) + // Continue here, because it's _probably_ wiser to not submit this block + continue + } + + if fault { + log.Errorf(" SLASH FILTER DETECTED FAULT due to blocks %s and %s", b.Header.Cid(), witness) + continue + } } - if fault { - log.Errorf(" SLASH FILTER DETECTED FAULT due to blocks %s and %s", b.Header.Cid(), witness) - continue + // Submit the newly mined block. + if err := t.api.SyncSubmitBlock(ctx, b); err != nil { + log.Errorf("failed to submit newly mined block: %+v", err) } - } - - // Submit the newly mined block. - if err := t.api.SyncSubmitBlock(ctx, b); err != nil { - log.Errorf("failed to submit newly mined block: %+v", err) - } - } else { - // If no block was mined, increase the null rounds and wait for the next epoch. - base.NullRounds++ + } else { + // If no block was mined, increase the null rounds and wait for the next epoch. + base.AddRounds++ - // Calculate the time for the next round. - nextRound := time.Unix(int64(base.TipSet.MinTimestamp()+build.BlockDelaySecs*uint64(base.NullRounds))+int64(build.PropagationDelaySecs), 0) + // Calculate the time for the next round. + nextRound := time.Unix(int64(base.TipSet.MinTimestamp()+build.BlockDelaySecs*uint64(base.AddRounds))+int64(build.PropagationDelaySecs), 0) - // Wait for the next round. - time.Sleep(time.Until(nextRound)) + // Wait for the next round. + time.Sleep(time.Until(nextRound)) + } } } -} // GetBestMiningCandidate implements the fork choice rule from a miner's // perspective. @@ -291,36 +352,37 @@ func (t *WinPostTask) mine2(ctx context.Context) { // It obtains the current chain head (HEAD), and compares it to the last tipset // we selected as our mining base (LAST). If HEAD's weight is larger than // LAST's weight, it selects HEAD to build on. Else, it selects LAST. -func (t *WinPostTask) GetBestMiningCandidate(ctx context.Context) (*MiningBase, error) { - bts, err := t.api.ChainHead(ctx) - if err != nil { - return nil, err - } - - if t.lastWork != nil { - if t.lastWork.TipSet.Equals(bts) { - return t.lastWork, nil - } - btsw, err := t.api.ChainTipSetWeight(ctx, bts.Key()) - if err != nil { - return nil, err - } - ltsw, err := t.api.ChainTipSetWeight(ctx, t.lastWork.TipSet.Key()) + func (t *WinPostTask) GetBestMiningCandidate(ctx context.Context) (*MiningBase, error) { + bts, err := t.api.ChainHead(ctx) if err != nil { - t.lastWork = nil return nil, err } - if types.BigCmp(btsw, ltsw) <= 0 { - return t.lastWork, nil - } - } + if t.lastWork != nil { + if t.lastWork.TipSet.Equals(bts) { + return t.lastWork, nil + } - t.lastWork = &MiningBase{TipSet: bts, ComputeTime: time.Now()} - return t.lastWork, nil -} + btsw, err := t.api.ChainTipSetWeight(ctx, bts.Key()) + if err != nil { + return nil, err + } + ltsw, err := t.api.ChainTipSetWeight(ctx, t.lastWork.TipSet.Key()) + if err != nil { + t.lastWork = nil + return nil, err + } + if types.BigCmp(btsw, ltsw) <= 0 { + return t.lastWork, nil + } + } + + t.lastWork = &MiningBase{TipSet: bts, ComputeTime: time.Now()} + return t.lastWork, nil + } +*/ func randTimeOffset(width time.Duration) time.Duration { buf := make([]byte, 8) rand.Reader.Read(buf) //nolint:errcheck From 49c56e4d1c190d4d38d7de302fa0405e0103fed8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 18:04:06 +0100 Subject: [PATCH 04/24] lpwinning: basic CanAccept --- provider/lpwinning/winning_task.go | 37 +++++++++++++++++------------- 1 file changed, 21 insertions(+), 16 deletions(-) diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index d4d556548ce..989b090ee27 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -21,7 +21,7 @@ import ( var log = logging.Logger("lpwinning") type WinPostTask struct { - max abi.SectorNumber + max int // lastWork holds the last MiningBase we built upon. lastWork *MiningBase @@ -52,14 +52,18 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don } func (t *WinPostTask) CanAccept(ids []harmonytask.TaskID, engine *harmonytask.TaskEngine) (*harmonytask.TaskID, error) { - //TODO implement me - panic("implement me") + if len(ids) == 0 { + // probably can't happen, but panicking is bad + return nil, nil + } + + return &ids[0], nil } func (t *WinPostTask) TypeDetails() harmonytask.TaskTypeDetails { return harmonytask.TaskTypeDetails{ Name: "WinPost", - Max: 10, // todo + Max: t.max, MaxFailures: 3, Follows: nil, Cost: resources.Resources{ @@ -104,18 +108,6 @@ func (mb MiningBase) afterPropDelay() time.Time { return base } -func retry1[R any](f func() (R, error)) R { - for { - r, err := f() - if err == nil { - return r - } - - log.Errorw("error in mining loop, retrying", "error", err) - time.Sleep(time.Second) - } -} - func (t *WinPostTask) mineBasic(ctx context.Context) { var workBase MiningBase @@ -383,6 +375,7 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { return t.lastWork, nil } */ + func randTimeOffset(width time.Duration) time.Duration { buf := make([]byte, 8) rand.Reader.Read(buf) //nolint:errcheck @@ -391,4 +384,16 @@ func randTimeOffset(width time.Duration) time.Duration { return val - (width / 2) } +func retry1[R any](f func() (R, error)) R { + for { + r, err := f() + if err == nil { + return r + } + + log.Errorw("error in mining loop, retrying", "error", err) + time.Sleep(time.Second) + } +} + var _ harmonytask.TaskInterface = &WinPostTask{} From b9625daf2e659e8781821c58dc0aa99be7116c08 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 20:01:17 +0100 Subject: [PATCH 05/24] lpwinning: implement WinPostTask.Do --- lib/harmony/harmonydb/sql/20231110.sql | 13 +- provider/lpwinning/winning_task.go | 345 ++++++++++++++++++++++++- 2 files changed, 347 insertions(+), 11 deletions(-) diff --git a/lib/harmony/harmonydb/sql/20231110.sql b/lib/harmony/harmonydb/sql/20231110.sql index 910ff05fa51..fad51a28e1e 100644 --- a/lib/harmony/harmonydb/sql/20231110.sql +++ b/lib/harmony/harmonydb/sql/20231110.sql @@ -3,8 +3,17 @@ create table mining_tasks task_id bigint not null constraint mining_tasks_pk primary key, - sp_id bigint, - epoch bigint, + sp_id bigint not null, + epoch bigint not null, + base_compute_time timestamp not null, + + won bool not null default false, + mined_cid text, + mined_header jsonb, + mined_at timestamp, + + submitted_at timestamp, + constraint mining_tasks_sp_epoch unique (sp_id, epoch) ); diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index 989b090ee27..1ebe9c88193 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -1,18 +1,27 @@ package lpwinning import ( + "bytes" "context" "crypto/rand" "encoding/binary" + "encoding/json" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/go-state-types/network" + prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/gen" + lrand "github.com/filecoin-project/lotus/chain/rand" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/lib/harmony/harmonytask" "github.com/filecoin-project/lotus/lib/harmony/resources" "github.com/filecoin-project/lotus/lib/promise" "github.com/filecoin-project/lotus/node/modules/dtypes" + "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" "time" @@ -22,9 +31,8 @@ var log = logging.Logger("lpwinning") type WinPostTask struct { max int - - // lastWork holds the last MiningBase we built upon. - lastWork *MiningBase + db *harmonydb.DB + epp gen.WinningPoStProver api WinPostAPI actors []dtypes.MinerAddress @@ -35,20 +43,314 @@ type WinPostTask struct { type WinPostAPI interface { ChainHead(context.Context) (*types.TipSet, error) ChainTipSetWeight(context.Context, types.TipSetKey) (types.BigInt, error) + ChainGetTipSet(context.Context, types.TipSetKey) (*types.TipSet, error) StateGetBeaconEntry(context.Context, abi.ChainEpoch) (*types.BeaconEntry, error) SyncSubmitBlock(context.Context, *types.BlockMsg) error + StateGetRandomnessFromBeacon(ctx context.Context, personalization crypto.DomainSeparationTag, randEpoch abi.ChainEpoch, entropy []byte, tsk types.TipSetKey) (abi.Randomness, error) + StateGetRandomnessFromTickets(ctx context.Context, personalization crypto.DomainSeparationTag, randEpoch abi.ChainEpoch, entropy []byte, tsk types.TipSetKey) (abi.Randomness, error) + StateNetworkVersion(context.Context, types.TipSetKey) (network.Version, error) + + MinerGetBaseInfo(context.Context, address.Address, abi.ChainEpoch, types.TipSetKey) (*api.MiningBaseInfo, error) + MinerCreateBlock(context.Context, *api.BlockTemplate) (*types.BlockMsg, error) + MpoolSelect(context.Context, types.TipSetKey, float64) ([]*types.SignedMessage, error) + + WalletSign(context.Context, address.Address, []byte) (*crypto.Signature, error) } -func NewWinPostTask(max abi.SectorNumber) *WinPostTask { - // todo run warmup +func NewWinPostTask(max int, db *harmonydb.DB, epp gen.WinningPoStProver, api WinPostAPI, actors []dtypes.MinerAddress) *WinPostTask { + return &WinPostTask{ + max: max, + db: db, + epp: epp, + api: api, + actors: actors, + } + // TODO: run warmup } func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (done bool, err error) { - // TODO THIS WILL BASICALLY BE A mineOne() function + ctx := context.TODO() + + type BlockCID struct { + CID string + } + + type MiningTaskDetails struct { + SpID uint64 + Epoch uint64 + BlockCIDs []BlockCID + CompTime time.Time + } + + var details MiningTaskDetails - //TODO implement me - panic("implement me") + // First query to fetch from mining_tasks + err = t.db.QueryRow(ctx, `SELECT sp_id, epoch, base_compute_time FROM mining_tasks WHERE task_id = $1`, taskID).Scan(&details.SpID, &details.Epoch, &details.CompTime) + if err != nil { + return false, err + } + + // Second query to fetch from mining_base_block + rows, err := t.db.Query(ctx, `SELECT block_cid FROM mining_base_block WHERE task_id = $1`, taskID) + if err != nil { + return false, err + } + defer rows.Close() + + for rows.Next() { + var cid BlockCID + if err := rows.Scan(&cid.CID); err != nil { + return false, err + } + details.BlockCIDs = append(details.BlockCIDs, cid) + } + + if err := rows.Err(); err != nil { + return false, err + } + + // construct base + maddr, err := address.NewIDAddress(details.SpID) + if err != nil { + return false, err + } + + var bcids []cid.Cid + for _, c := range details.BlockCIDs { + bcid, err := cid.Parse(c.CID) + if err != nil { + return false, err + } + bcids = append(bcids, bcid) + } + + tsk := types.NewTipSetKey(bcids...) + baseTs, err := t.api.ChainGetTipSet(ctx, tsk) + if err != nil { + return false, xerrors.Errorf("loading base tipset: %w", err) + } + + base := MiningBase{ + TipSet: baseTs, + AddRounds: abi.ChainEpoch(details.Epoch) - baseTs.Height() - 1, + ComputeTime: details.CompTime, + } + + // ensure we have a beacon entry for the epoch we're mining on + round := base.epoch() + + _ = retry1(func() (*types.BeaconEntry, error) { + return t.api.StateGetBeaconEntry(ctx, round) + }) + + // MAKE A MINING ATTEMPT!! + log.Debugw("attempting to mine a block", "tipset", types.LogCids(base.TipSet.Cids())) + + mbi, err := t.api.MinerGetBaseInfo(ctx, maddr, round, base.TipSet.Key()) + if err != nil { + err = xerrors.Errorf("failed to get mining base info: %w", err) + return false, err + } + if mbi == nil { + // not elloigible to mine on this base, we're done here + return true, nil + } + + if !mbi.EligibleForMining { + // slashed or just have no power yet, we're done here + return true, nil + } + + var rbase types.BeaconEntry + var bvals []types.BeaconEntry + var eproof *types.ElectionProof + + // winner check + { + bvals = mbi.BeaconEntries + rbase = mbi.PrevBeaconEntry + if len(bvals) > 0 { + rbase = bvals[len(bvals)-1] + } + + eproof, err = gen.IsRoundWinner(ctx, round, maddr, rbase, mbi, t.api) + if err != nil { + return false, xerrors.Errorf("failed to check if we win next round: %w", err) + } + + if eproof == nil { + // not a winner, we're done here + return true, nil + } + } + + // winning PoSt + var wpostProof []prooftypes.PoStProof + { + buf := new(bytes.Buffer) + if err := maddr.MarshalCBOR(buf); err != nil { + err = xerrors.Errorf("failed to marshal miner address: %w", err) + return false, err + } + + rand, err := lrand.DrawRandomnessFromBase(rbase.Data, crypto.DomainSeparationTag_WinningPoStChallengeSeed, round, buf.Bytes()) + if err != nil { + err = xerrors.Errorf("failed to get randomness for winning post: %w", err) + return false, err + } + + prand := abi.PoStRandomness(rand) + + nv, err := t.api.StateNetworkVersion(ctx, base.TipSet.Key()) + if err != nil { + return false, err + } + + wpostProof, err = t.epp.ComputeProof(ctx, mbi.Sectors, prand, round, nv) + if err != nil { + err = xerrors.Errorf("failed to compute winning post proof: %w", err) + return false, err + } + } + + ticket, err := t.computeTicket(ctx, maddr, &rbase, round, base.TipSet.MinTicket(), mbi) + if err != nil { + return false, xerrors.Errorf("scratching ticket failed: %w", err) + } + + // get pending messages early, + msgs, err := t.api.MpoolSelect(ctx, base.TipSet.Key(), ticket.Quality()) + if err != nil { + return false, xerrors.Errorf("failed to select messages for block: %w", err) + } + + // equivocation handling + { + // This next block exists to "catch" equivocating miners, + // who submit 2 blocks at the same height at different times in order to split the network. + // To safeguard against this, we make sure it's been EquivocationDelaySecs since our base was calculated, + // then re-calculate it. + // If the daemon detected equivocated blocks, those blocks will no longer be in the new base. + time.Sleep(time.Until(base.ComputeTime.Add(time.Duration(build.EquivocationDelaySecs) * time.Second))) + + bestTs, err := t.api.ChainHead(ctx) + if err != nil { + return false, xerrors.Errorf("failed to get chain head: %w", err) + } + + headWeight, err := t.api.ChainTipSetWeight(ctx, bestTs.Key()) + if err != nil { + return false, xerrors.Errorf("failed to get chain head weight: %w", err) + } + + baseWeight, err := t.api.ChainTipSetWeight(ctx, base.TipSet.Key()) + if err != nil { + return false, xerrors.Errorf("failed to get base weight: %w", err) + } + if types.BigCmp(headWeight, baseWeight) <= 0 { + bestTs = base.TipSet + } + + // If the base has changed, we take the _intersection_ of our old base and new base, + // thus ejecting blocks from any equivocating miners, without taking any new blocks. + if bestTs.Height() == base.TipSet.Height() && !bestTs.Equals(base.TipSet) { + log.Warnf("base changed from %s to %s, taking intersection", base.TipSet.Key(), bestTs.Key()) + newBaseMap := map[cid.Cid]struct{}{} + for _, newBaseBlk := range bestTs.Cids() { + newBaseMap[newBaseBlk] = struct{}{} + } + + refreshedBaseBlocks := make([]*types.BlockHeader, 0, len(base.TipSet.Cids())) + for _, baseBlk := range base.TipSet.Blocks() { + if _, ok := newBaseMap[baseBlk.Cid()]; ok { + refreshedBaseBlocks = append(refreshedBaseBlocks, baseBlk) + } + } + + if len(refreshedBaseBlocks) != 0 && len(refreshedBaseBlocks) != len(base.TipSet.Blocks()) { + refreshedBase, err := types.NewTipSet(refreshedBaseBlocks) + if err != nil { + return false, xerrors.Errorf("failed to create new tipset when refreshing: %w", err) + } + + if !base.TipSet.MinTicket().Equals(refreshedBase.MinTicket()) { + log.Warn("recomputing ticket due to base refresh") + + ticket, err = t.computeTicket(ctx, maddr, &rbase, round, refreshedBase.MinTicket(), mbi) + if err != nil { + return false, xerrors.Errorf("failed to refresh ticket: %w", err) + } + } + + log.Warn("re-selecting messages due to base refresh") + // refresh messages, as the selected messages may no longer be valid + msgs, err = t.api.MpoolSelect(ctx, refreshedBase.Key(), ticket.Quality()) + if err != nil { + return false, xerrors.Errorf("failed to re-select messages for block: %w", err) + } + + base.TipSet = refreshedBase + } + } + } + + // block construction + var blockMsg *types.BlockMsg + { + uts := base.TipSet.MinTimestamp() + build.BlockDelaySecs*(uint64(base.AddRounds)+1) + + blockMsg, err = t.api.MinerCreateBlock(context.TODO(), &api.BlockTemplate{ + Miner: maddr, + Parents: base.TipSet.Key(), + Ticket: ticket, + Eproof: eproof, + BeaconValues: bvals, + Messages: msgs, + Epoch: round, + Timestamp: uts, + WinningPoStProof: wpostProof, + }) + if err != nil { + return false, xerrors.Errorf("failed to create block: %w", err) + } + } + + // persist in db + { + bhjson, err := json.Marshal(blockMsg.Header) + if err != nil { + return false, xerrors.Errorf("failed to marshal block header: %w", err) + } + + _, err = t.db.Exec(ctx, `UPDATE mining_tasks + SET won = true, mined_cid = $2, mined_header = $3, mined_at = $4 + WHERE task_id = $1`, taskID, blockMsg.Header.Cid(), string(bhjson), time.Now().UTC()) + if err != nil { + return false, xerrors.Errorf("failed to update mining task: %w", err) + } + } + + // submit block!! + { + if err := t.api.SyncSubmitBlock(ctx, blockMsg); err != nil { + return false, xerrors.Errorf("failed to submit block: %w", err) + } + } + + log.Infow("mined a block", "tipset", types.LogCids(blockMsg.Header.Parents), "height", blockMsg.Header.Height, "miner", maddr, "cid", blockMsg.Header.Cid()) + + // persist that we've submitted the block + { + _, err = t.db.Exec(ctx, `UPDATE mining_tasks + SET submitted_at = $2 + WHERE task_id = $1`, taskID, time.Now().UTC()) + if err != nil { + return false, xerrors.Errorf("failed to update mining task: %w", err) + } + } + + return true, nil } func (t *WinPostTask) CanAccept(ids []harmonytask.TaskID, engine *harmonytask.TaskEngine) (*harmonytask.TaskID, error) { @@ -188,7 +490,7 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { } taskFn(func(id harmonytask.TaskID, tx *harmonydb.Tx) (shouldCommit bool, seriousError error) { - _, err := tx.Exec(`INSERT INTO mining_tasks (task_id, sp_id, epoch) VALUES ($1, $2, $3)`, id, spID, workBase.epoch()) + _, err := tx.Exec(`INSERT INTO mining_tasks (task_id, sp_id, epoch, base_compute_time) VALUES ($1, $2, $3, $4)`, id, spID, workBase.epoch(), workBase.ComputeTime.UTC()) if err != nil { return false, xerrors.Errorf("inserting mining_tasks: %w", err) } @@ -376,6 +678,31 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { } */ +func (t *WinPostTask) computeTicket(ctx context.Context, maddr address.Address, brand *types.BeaconEntry, round abi.ChainEpoch, chainRand *types.Ticket, mbi *api.MiningBaseInfo) (*types.Ticket, error) { + buf := new(bytes.Buffer) + if err := maddr.MarshalCBOR(buf); err != nil { + return nil, xerrors.Errorf("failed to marshal address to cbor: %w", err) + } + + if round > build.UpgradeSmokeHeight { + buf.Write(chainRand.VRFProof) + } + + input, err := lrand.DrawRandomnessFromBase(brand.Data, crypto.DomainSeparationTag_TicketProduction, round-build.TicketRandomnessLookback, buf.Bytes()) + if err != nil { + return nil, err + } + + vrfOut, err := gen.ComputeVRF(ctx, t.api.WalletSign, mbi.WorkerKey, input) + if err != nil { + return nil, err + } + + return &types.Ticket{ + VRFProof: vrfOut, + }, nil +} + func randTimeOffset(width time.Duration) time.Duration { buf := make([]byte, 8) rand.Reader.Read(buf) //nolint:errcheck From 2a4ce7d358aaa746308bc5c819c347c2d9c8eb30 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 20:17:05 +0100 Subject: [PATCH 06/24] lpwinning: wire up winningPoSt --- cmd/lotus-provider/run.go | 6 + node/config/types.go | 7 +- provider/lpwinning/winning_task.go | 215 ++++------------------------- 3 files changed, 40 insertions(+), 188 deletions(-) diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 005cb9ebf2d..1fcdf37b627 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -3,6 +3,7 @@ package main import ( "encoding/base64" "fmt" + "github.com/filecoin-project/lotus/provider/lpwinning" "net" "net/http" "os" @@ -251,6 +252,11 @@ var runCmd = &cli.Command{ } activeTasks = append(activeTasks, wdPostTask, wdPoStSubmitTask, derlareRecoverTask) } + + if cfg.Subsystems.EnableWinningPost { + winPoStTask := lpwinning.NewWinPostTask(cfg.Subsystems.WinningPostMaxTasks, db, lw, verif, full, maddrs) + activeTasks = append(activeTasks, winPoStTask) + } } taskEngine, err := harmonytask.New(db, activeTasks, listenAddr) if err != nil { diff --git a/node/config/types.go b/node/config/types.go index 120e5024826..172de1d7d02 100644 --- a/node/config/types.go +++ b/node/config/types.go @@ -92,9 +92,10 @@ type JournalConfig struct { } type ProviderSubsystemsConfig struct { - EnableWindowPost bool - WindowPostMaxTasks int - EnableWinningPost bool + EnableWindowPost bool + WindowPostMaxTasks int + EnableWinningPost bool + WinningPostMaxTasks int } type DAGStoreConfig struct { diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index 1ebe9c88193..163aeff841e 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/lib/harmony/resources" "github.com/filecoin-project/lotus/lib/promise" "github.com/filecoin-project/lotus/node/modules/dtypes" + "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" @@ -32,7 +33,9 @@ var log = logging.Logger("lpwinning") type WinPostTask struct { max int db *harmonydb.DB - epp gen.WinningPoStProver + + prover ProverWinningPoSt + verifier storiface.Verifier api WinPostAPI actors []dtypes.MinerAddress @@ -50,6 +53,7 @@ type WinPostAPI interface { StateGetRandomnessFromBeacon(ctx context.Context, personalization crypto.DomainSeparationTag, randEpoch abi.ChainEpoch, entropy []byte, tsk types.TipSetKey) (abi.Randomness, error) StateGetRandomnessFromTickets(ctx context.Context, personalization crypto.DomainSeparationTag, randEpoch abi.ChainEpoch, entropy []byte, tsk types.TipSetKey) (abi.Randomness, error) StateNetworkVersion(context.Context, types.TipSetKey) (network.Version, error) + StateMinerInfo(context.Context, address.Address, types.TipSetKey) (api.MinerInfo, error) MinerGetBaseInfo(context.Context, address.Address, abi.ChainEpoch, types.TipSetKey) (*api.MiningBaseInfo, error) MinerCreateBlock(context.Context, *api.BlockTemplate) (*types.BlockMsg, error) @@ -58,13 +62,18 @@ type WinPostAPI interface { WalletSign(context.Context, address.Address, []byte) (*crypto.Signature, error) } -func NewWinPostTask(max int, db *harmonydb.DB, epp gen.WinningPoStProver, api WinPostAPI, actors []dtypes.MinerAddress) *WinPostTask { +type ProverWinningPoSt interface { + GenerateWinningPoSt(ctx context.Context, ppt abi.RegisteredPoStProof, minerID abi.ActorID, sectorInfo []prooftypes.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]prooftypes.PoStProof, error) +} + +func NewWinPostTask(max int, db *harmonydb.DB, prover ProverWinningPoSt, verifier storiface.Verifier, api WinPostAPI, actors []dtypes.MinerAddress) *WinPostTask { return &WinPostTask{ - max: max, - db: db, - epp: epp, - api: api, - actors: actors, + max: max, + db: db, + prover: prover, + verifier: verifier, + api: api, + actors: actors, } // TODO: run warmup } @@ -116,6 +125,11 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don return false, err } + mi, err := t.api.StateMinerInfo(ctx, maddr, types.EmptyTSK) + if err != nil { + return false, xerrors.Errorf("getting sector size: %w", err) + } + var bcids []cid.Cid for _, c := range details.BlockCIDs { bcid, err := cid.Parse(c.CID) @@ -194,20 +208,15 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don return false, err } - rand, err := lrand.DrawRandomnessFromBase(rbase.Data, crypto.DomainSeparationTag_WinningPoStChallengeSeed, round, buf.Bytes()) + brand, err := lrand.DrawRandomnessFromBase(rbase.Data, crypto.DomainSeparationTag_WinningPoStChallengeSeed, round, buf.Bytes()) if err != nil { err = xerrors.Errorf("failed to get randomness for winning post: %w", err) return false, err } - prand := abi.PoStRandomness(rand) + prand := abi.PoStRandomness(brand) - nv, err := t.api.StateNetworkVersion(ctx, base.TipSet.Key()) - if err != nil { - return false, err - } - - wpostProof, err = t.epp.ComputeProof(ctx, mbi.Sectors, prand, round, nv) + wpostProof, err = t.prover.GenerateWinningPoSt(ctx, mi.WindowPoStProofType, abi.ActorID(details.SpID), mbi.Sectors, prand) if err != nil { err = xerrors.Errorf("failed to compute winning post proof: %w", err) return false, err @@ -331,6 +340,11 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don } } + // wait until block timestamp + { + time.Sleep(time.Until(time.Unix(int64(blockMsg.Header.Timestamp), 0))) + } + // submit block!! { if err := t.api.SyncSubmitBlock(ctx, blockMsg); err != nil { @@ -415,6 +429,7 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { taskFn := t.mineTF.Val(ctx) + // initialize workbase { head := retry1(func() (*types.TipSet, error) { return t.api.ChainHead(ctx) @@ -508,176 +523,6 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { } } -/* - func (t *WinPostTask) mine2(ctx context.Context) { - var lastBase MiningBase - - // Start the main mining loop. - for { - // todo handle stop signals? - - var base *MiningBase - - // Look for the best mining candidate. - for { - prebase, err := t.GetBestMiningCandidate(ctx) - if err != nil { - log.Errorf("failed to get best mining candidate: %s", err) - time.Sleep(5 * time.Second) - continue - } - - // Check if we have a new base or if the current base is still valid. - if base != nil && base.TipSet.Height() == prebase.TipSet.Height() && base.AddRounds == prebase.AddRounds { - // We have a valid base. - base = prebase - break - } - - // TODO: need to change the orchestration here. the problem is that - // we are waiting *after* we enter this loop and selecta mining - // candidate, which is almost certain to change in multiminer - // tests. Instead, we should block before entering the loop, so - // that when the test 'MineOne' function is triggered, we pull our - // best mining candidate at that time. - - // Wait until propagation delay period after block we plan to mine on - { - // if we're mining a block in the past via catch-up/rush mining, - // such as when recovering from a network halt, this sleep will be - // for a negative duration, and therefore **will return - // immediately**. - // - // the result is that we WILL NOT wait, therefore fast-forwarding - // and thus healing the chain by backfilling it with null rounds - // rapidly. - baseTs := prebase.TipSet.MinTimestamp() + build.PropagationDelaySecs - baseT := time.Unix(int64(baseTs), 0) - baseT = baseT.Add(randTimeOffset(time.Second)) - time.Sleep(time.Until(baseT)) - } - - // Ensure the beacon entry is available before finalizing the mining base. - _, err = t.api.StateGetBeaconEntry(ctx, prebase.TipSet.Height()+prebase.AddRounds+1) - if err != nil { - log.Errorf("failed getting beacon entry: %s", err) - time.Sleep(time.Second) - continue - } - - base = prebase - } - - // Check for repeated mining candidates and handle sleep for the next round. - if base.TipSet.Equals(lastBase.TipSet) && lastBase.AddRounds == base.AddRounds { - log.Warnf("BestMiningCandidate from the previous round: %s (nulls:%d)", lastBase.TipSet.Cids(), lastBase.AddRounds) - time.Sleep(time.Duration(build.BlockDelaySecs) * time.Second) - continue - } - - // Attempt to mine a block. - b, err := m.mineOne(ctx, base) - if err != nil { - log.Errorf("mining block failed: %+v", err) - time.Sleep(time.Second) - continue - } - lastBase = *base - - // todo figure out this whole bottom section - // we won't know if we've mined a block here, we just submit a task - // making attempts to mine one - - // Process the mined block. - if b != nil { - btime := time.Unix(int64(b.Header.Timestamp), 0) - now := build.Clock.Now() - // Handle timing for broadcasting the block. - switch { - case btime == now: - // block timestamp is perfectly aligned with time. - case btime.After(now): - // Wait until it's time to broadcast the block. - if !m.niceSleep(build.Clock.Until(btime)) { - log.Warnf("received interrupt while waiting to broadcast block, will shutdown after block is sent out") - build.Clock.Sleep(build.Clock.Until(btime)) - } - default: - // Log if the block was mined in the past. - log.Warnw("mined block in the past", - "block-time", btime, "time", build.Clock.Now(), "difference", build.Clock.Since(btime)) - } - - // Check for slash filter conditions. - if os.Getenv("LOTUS_MINER_NO_SLASHFILTER") != "_yes_i_know_i_can_and_probably_will_lose_all_my_fil_and_power_" && !build.IsNearUpgrade(base.TipSet.Height(), build.UpgradeWatermelonFixHeight) { - witness, fault, err := m.sf.MinedBlock(ctx, b.Header, base.TipSet.Height()+base.AddRounds) - if err != nil { - log.Errorf(" SLASH FILTER ERRORED: %s", err) - // Continue here, because it's _probably_ wiser to not submit this block - continue - } - - if fault { - log.Errorf(" SLASH FILTER DETECTED FAULT due to blocks %s and %s", b.Header.Cid(), witness) - continue - } - } - - // Submit the newly mined block. - if err := t.api.SyncSubmitBlock(ctx, b); err != nil { - log.Errorf("failed to submit newly mined block: %+v", err) - } - } else { - // If no block was mined, increase the null rounds and wait for the next epoch. - base.AddRounds++ - - // Calculate the time for the next round. - nextRound := time.Unix(int64(base.TipSet.MinTimestamp()+build.BlockDelaySecs*uint64(base.AddRounds))+int64(build.PropagationDelaySecs), 0) - - // Wait for the next round. - time.Sleep(time.Until(nextRound)) - } - } - } - -// GetBestMiningCandidate implements the fork choice rule from a miner's -// perspective. -// -// It obtains the current chain head (HEAD), and compares it to the last tipset -// we selected as our mining base (LAST). If HEAD's weight is larger than -// LAST's weight, it selects HEAD to build on. Else, it selects LAST. - - func (t *WinPostTask) GetBestMiningCandidate(ctx context.Context) (*MiningBase, error) { - bts, err := t.api.ChainHead(ctx) - if err != nil { - return nil, err - } - - if t.lastWork != nil { - if t.lastWork.TipSet.Equals(bts) { - return t.lastWork, nil - } - - btsw, err := t.api.ChainTipSetWeight(ctx, bts.Key()) - if err != nil { - return nil, err - } - ltsw, err := t.api.ChainTipSetWeight(ctx, t.lastWork.TipSet.Key()) - if err != nil { - t.lastWork = nil - return nil, err - } - - if types.BigCmp(btsw, ltsw) <= 0 { - return t.lastWork, nil - } - } - - t.lastWork = &MiningBase{TipSet: bts, ComputeTime: time.Now()} - return t.lastWork, nil - } -*/ - func (t *WinPostTask) computeTicket(ctx context.Context, maddr address.Address, brand *types.BeaconEntry, round abi.ChainEpoch, chainRand *types.Ticket, mbi *api.MiningBaseInfo) (*types.Ticket, error) { buf := new(bytes.Buffer) if err := maddr.MarshalCBOR(buf); err != nil { From d719db3f2c1feb61a4d32d6a460b25da4eac09e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Fri, 10 Nov 2023 20:36:41 +0100 Subject: [PATCH 07/24] lpwinning: fix PoSt prover gen --- cmd/lotus-provider/run.go | 2 +- .../en/default-lotus-provider-config.toml | 3 ++ node/config/doc_gen.go | 6 +++ provider/lpwinning/winning_task.go | 37 ++++++++++++++++--- 4 files changed, 41 insertions(+), 7 deletions(-) diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 1fcdf37b627..4484ff6ac39 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -3,7 +3,6 @@ package main import ( "encoding/base64" "fmt" - "github.com/filecoin-project/lotus/provider/lpwinning" "net" "net/http" "os" @@ -41,6 +40,7 @@ import ( "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/provider" + "github.com/filecoin-project/lotus/provider/lpwinning" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" diff --git a/documentation/en/default-lotus-provider-config.toml b/documentation/en/default-lotus-provider-config.toml index cb42c7f5fb5..91606e503fd 100644 --- a/documentation/en/default-lotus-provider-config.toml +++ b/documentation/en/default-lotus-provider-config.toml @@ -8,6 +8,9 @@ # type: bool #EnableWinningPost = false + # type: int + #WinningPostMaxTasks = 0 + [Fees] # type: types.FIL diff --git a/node/config/doc_gen.go b/node/config/doc_gen.go index a65bd874665..9e4b6c7b3b1 100644 --- a/node/config/doc_gen.go +++ b/node/config/doc_gen.go @@ -995,6 +995,12 @@ sectors.`, Name: "EnableWinningPost", Type: "bool", + Comment: ``, + }, + { + Name: "WinningPostMaxTasks", + Type: "int", + Comment: ``, }, }, diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index 163aeff841e..e6582392140 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -6,11 +6,19 @@ import ( "crypto/rand" "encoding/binary" "encoding/json" + "time" + + "github.com/ipfs/go-cid" + logging "github.com/ipfs/go-log/v2" + "golang.org/x/xerrors" + + ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" @@ -22,10 +30,6 @@ import ( "github.com/filecoin-project/lotus/lib/promise" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/storage/sealer/storiface" - "github.com/ipfs/go-cid" - logging "github.com/ipfs/go-log/v2" - "golang.org/x/xerrors" - "time" ) var log = logging.Logger("lpwinning") @@ -63,7 +67,7 @@ type WinPostAPI interface { } type ProverWinningPoSt interface { - GenerateWinningPoSt(ctx context.Context, ppt abi.RegisteredPoStProof, minerID abi.ActorID, sectorInfo []prooftypes.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]prooftypes.PoStProof, error) + GenerateWinningPoSt(ctx context.Context, ppt abi.RegisteredPoStProof, minerID abi.ActorID, sectorInfo []storiface.PostSectorChallenge, randomness abi.PoStRandomness) ([]prooftypes.PoStProof, error) } func NewWinPostTask(max int, db *harmonydb.DB, prover ProverWinningPoSt, verifier storiface.Verifier, api WinPostAPI, actors []dtypes.MinerAddress) *WinPostTask { @@ -216,7 +220,28 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don prand := abi.PoStRandomness(brand) - wpostProof, err = t.prover.GenerateWinningPoSt(ctx, mi.WindowPoStProofType, abi.ActorID(details.SpID), mbi.Sectors, prand) + sectorNums := make([]abi.SectorNumber, len(mbi.Sectors)) + for i, s := range mbi.Sectors { + sectorNums[i] = s.SectorNumber + } + + postChallenges, err := ffi.GeneratePoStFallbackSectorChallenges(mi.WindowPoStProofType, abi.ActorID(details.SpID), prand, sectorNums) + if err != nil { + return false, xerrors.Errorf("generating fallback challenges: %v", err) + } + + sectorChallenges := make([]storiface.PostSectorChallenge, len(mbi.Sectors)) + for i, s := range mbi.Sectors { + sectorChallenges[i] = storiface.PostSectorChallenge{ + SealProof: s.SealProof, + SectorNumber: s.SectorNumber, + SealedCID: s.SealedCID, + Challenge: postChallenges.Challenges[s.SectorNumber], + Update: s.SectorKey != nil, + } + } + + wpostProof, err = t.prover.GenerateWinningPoSt(ctx, mi.WindowPoStProofType, abi.ActorID(details.SpID), sectorChallenges, prand) if err != nil { err = xerrors.Errorf("failed to compute winning post proof: %w", err) return false, err From 8b4101360afee2bf4a5e7dcc5819b6501c2cfe75 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:19:38 +0100 Subject: [PATCH 08/24] lpwindow: Fix recover schema --- lib/harmony/harmonydb/sql/20230823.sql | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/lib/harmony/harmonydb/sql/20230823.sql b/lib/harmony/harmonydb/sql/20230823.sql index 1865f6afe0f..3b001a4c1b5 100644 --- a/lib/harmony/harmonydb/sql/20230823.sql +++ b/lib/harmony/harmonydb/sql/20230823.sql @@ -37,12 +37,12 @@ create table wdpost_proofs create table wdpost_recovery_tasks ( task_id bigint not null - constraint wdpost_partition_tasks_pk + constraint wdpost_recovery_partition_tasks_pk primary key, sp_id bigint not null, proving_period_start bigint not null, deadline_index bigint not null, partition_index bigint not null, - constraint wdpost_partition_tasks_identity_key + constraint wdpost_recovery_partition_tasks_identity_key unique (sp_id, proving_period_start, deadline_index, partition_index) ); \ No newline at end of file From f2ef006abd582a8c0c6ed21e7b5795f5faf9fc09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:19:58 +0100 Subject: [PATCH 09/24] lotus-miner: add config for disabling winning post --- node/builder_miner.go | 8 ++++++-- node/config/types.go | 7 +++++++ 2 files changed, 13 insertions(+), 2 deletions(-) diff --git a/node/builder_miner.go b/node/builder_miner.go index 4ba5f05b57f..dd35c6becf9 100644 --- a/node/builder_miner.go +++ b/node/builder_miner.go @@ -121,8 +121,12 @@ func ConfigStorageMiner(c interface{}) Option { // Mining / proving Override(new(*slashfilter.SlashFilter), modules.NewSlashFilter), - Override(new(*miner.Miner), modules.SetupBlockProducer), - Override(new(gen.WinningPoStProver), storage.NewWinningPoStProver), + + If(!cfg.Subsystems.DisableWinningPoSt, + Override(new(*miner.Miner), modules.SetupBlockProducer), + Override(new(gen.WinningPoStProver), storage.NewWinningPoStProver), + ), + Override(PreflightChecksKey, modules.PreflightChecks), Override(new(*sealing.Sealing), modules.SealingPipeline(cfg.Fees)), diff --git a/node/config/types.go b/node/config/types.go index 172de1d7d02..2152e079569 100644 --- a/node/config/types.go +++ b/node/config/types.go @@ -162,6 +162,13 @@ type MinerSubsystemConfig struct { // to window post, including scheduling, submitting proofs, and recovering // sectors. DisableWindowPoSt bool + + // When winning post is disabled, the miner process will NOT attempt to mine + // blocks. This should only be set when there's an external process mining + // blocks on behalf of the miner. + // When disabled and no external block producers are configured, all potential + // block rewards will be missed! + DisableWinningPoSt bool } type DealmakingConfig struct { From 87443124e6ff6b4bf0c415d3f193e2e6ade6a497 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:24:10 +0100 Subject: [PATCH 10/24] lotus-provider: Make from-miner work offline --- cmd/lotus-provider/migrate.go | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/cmd/lotus-provider/migrate.go b/cmd/lotus-provider/migrate.go index 47095c398af..7d250c4e556 100644 --- a/cmd/lotus-provider/migrate.go +++ b/cmd/lotus-provider/migrate.go @@ -6,6 +6,8 @@ import ( "encoding/base64" "errors" "fmt" + "github.com/filecoin-project/go-address" + "github.com/ipfs/go-datastore" "os" "path" "strings" @@ -117,14 +119,20 @@ func fromMiner(cctx *cli.Context) (err error) { } // Populate Miner Address - sm, cc, err := cliutil.GetStorageMinerAPI(cctx) + mmeta, err := lr.Datastore(ctx, "/metadata") if err != nil { - return fmt.Errorf("could not get storageMiner API: %w", err) + return xerrors.Errorf("opening miner metadata datastore: %w", err) } - defer cc() - addr, err := sm.ActorAddress(ctx) + defer mmeta.Close() + + maddrBytes, err := mmeta.Get(ctx, datastore.NewKey("miner-address")) + if err != nil { + return xerrors.Errorf("getting miner address datastore entry: %w", err) + } + + addr, err := address.NewFromBytes(maddrBytes) if err != nil { - return fmt.Errorf("could not read actor address: %w", err) + return xerrors.Errorf("parsing miner actor address: %w", err) } lpCfg.Addresses.MinerAddresses = []string{addr.String()} From 7efc4e4bf61e96ff93360f09f747de5e25bbc4c8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:37:57 +0100 Subject: [PATCH 11/24] lotus-provider: Somewhat less broken from-miner --- cmd/lotus-provider/migrate.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/cmd/lotus-provider/migrate.go b/cmd/lotus-provider/migrate.go index 7d250c4e556..3b2ea440d4b 100644 --- a/cmd/lotus-provider/migrate.go +++ b/cmd/lotus-provider/migrate.go @@ -35,6 +35,12 @@ var configMigrateCmd = &cli.Command{ Value: "~/.lotusminer", Usage: fmt.Sprintf("Specify miner repo path. flag(%s) and env(LOTUS_STORAGE_PATH) are DEPRECATION, will REMOVE SOON", FlagMinerRepoDeprecation), }, + &cli.StringFlag{ + Name: "repo", + EnvVars: []string{"LOTUS_PATH"}, + Hidden: true, + Value: "~/.lotus", + }, &cli.StringFlag{ Name: "to-layer", Aliases: []string{"t"}, @@ -169,7 +175,7 @@ environment variable LOTUS_WORKER_WINDOWPOST. } if !lo.Contains(titles, "base") { - _, err = db.Exec(ctx, "INSERT INTO harmony_config (title, config) VALUES ('base', '')", "base") + _, err = db.Exec(ctx, "INSERT INTO harmony_config (title, config) VALUES ('base', '')") if err != nil { return err } From 339e06297593c96a488e42fcce45c83327dd504d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:40:42 +0100 Subject: [PATCH 12/24] lotus-provider: Encode storage rpc secret correctly in from-miner --- cmd/lotus-provider/migrate.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/lotus-provider/migrate.go b/cmd/lotus-provider/migrate.go index 3b2ea440d4b..7e885813f5e 100644 --- a/cmd/lotus-provider/migrate.go +++ b/cmd/lotus-provider/migrate.go @@ -151,7 +151,7 @@ func fromMiner(cctx *cli.Context) (err error) { if err != nil { return xerrors.Errorf("error getting JWTSecretName: %w", err) } - lpCfg.Apis.StorageRPCSecret = base64.RawStdEncoding.EncodeToString(js.PrivateKey) + lpCfg.Apis.StorageRPCSecret = base64.StdEncoding.EncodeToString(js.PrivateKey) // Populate API Key _, header, err := cliutil.GetRawAPI(cctx, repo.FullNode, "v0") From 30cf227ef52a78a037282e406e85f799def8520c Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 12:44:36 +0100 Subject: [PATCH 13/24] lpwinning: select lowest epoch task id for mining task --- provider/lpwinning/winning_task.go | 18 +++++++++++++++++- 1 file changed, 17 insertions(+), 1 deletion(-) diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index e6582392140..06bff5255aa 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -398,7 +398,23 @@ func (t *WinPostTask) CanAccept(ids []harmonytask.TaskID, engine *harmonytask.Ta return nil, nil } - return &ids[0], nil + // select lowest epoch + var lowestEpoch abi.ChainEpoch + var lowestEpochID = ids[0] + for _, id := range ids { + var epoch uint64 + err := t.db.QueryRow(context.Background(), `SELECT epoch FROM mining_tasks WHERE task_id = $1`, id).Scan(&epoch) + if err != nil { + return nil, err + } + + if lowestEpoch == 0 || abi.ChainEpoch(epoch) < lowestEpoch { + lowestEpoch = abi.ChainEpoch(epoch) + lowestEpochID = id + } + } + + return &lowestEpochID, nil } func (t *WinPostTask) TypeDetails() harmonytask.TaskTypeDetails { From aa87f3ccf508ec96541cbc4f61b0234f143c084e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 14:16:26 +0100 Subject: [PATCH 14/24] lpwinning: Make block production work --- provider/lpwinning/winning_task.go | 27 ++++++++++++++++++--------- 1 file changed, 18 insertions(+), 9 deletions(-) diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index 06bff5255aa..ad992b10b48 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -71,7 +71,7 @@ type ProverWinningPoSt interface { } func NewWinPostTask(max int, db *harmonydb.DB, prover ProverWinningPoSt, verifier storiface.Verifier, api WinPostAPI, actors []dtypes.MinerAddress) *WinPostTask { - return &WinPostTask{ + t := &WinPostTask{ max: max, db: db, prover: prover, @@ -80,6 +80,10 @@ func NewWinPostTask(max int, db *harmonydb.DB, prover ProverWinningPoSt, verifie actors: actors, } // TODO: run warmup + + go t.mineBasic(context.TODO()) + + return t } func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (done bool, err error) { @@ -129,11 +133,6 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don return false, err } - mi, err := t.api.StateMinerInfo(ctx, maddr, types.EmptyTSK) - if err != nil { - return false, xerrors.Errorf("getting sector size: %w", err) - } - var bcids []cid.Cid for _, c := range details.BlockCIDs { bcid, err := cid.Parse(c.CID) @@ -180,6 +179,10 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don return true, nil } + if len(mbi.Sectors) == 0 { + return false, xerrors.Errorf("no sectors selected for winning PoSt") + } + var rbase types.BeaconEntry var bvals []types.BeaconEntry var eproof *types.ElectionProof @@ -219,15 +222,21 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don } prand := abi.PoStRandomness(brand) + prand[31] &= 0x3f // make into fr sectorNums := make([]abi.SectorNumber, len(mbi.Sectors)) for i, s := range mbi.Sectors { sectorNums[i] = s.SectorNumber } - postChallenges, err := ffi.GeneratePoStFallbackSectorChallenges(mi.WindowPoStProofType, abi.ActorID(details.SpID), prand, sectorNums) + ppt, err := mbi.Sectors[0].SealProof.RegisteredWinningPoStProof() + if err != nil { + return false, xerrors.Errorf("mapping sector seal proof type to post proof type: %w", err) + } + + postChallenges, err := ffi.GeneratePoStFallbackSectorChallenges(ppt, abi.ActorID(details.SpID), prand, sectorNums) if err != nil { - return false, xerrors.Errorf("generating fallback challenges: %v", err) + return false, xerrors.Errorf("generating election challenges: %v", err) } sectorChallenges := make([]storiface.PostSectorChallenge, len(mbi.Sectors)) @@ -241,7 +250,7 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don } } - wpostProof, err = t.prover.GenerateWinningPoSt(ctx, mi.WindowPoStProofType, abi.ActorID(details.SpID), sectorChallenges, prand) + wpostProof, err = t.prover.GenerateWinningPoSt(ctx, ppt, abi.ActorID(details.SpID), sectorChallenges, prand) if err != nil { err = xerrors.Errorf("failed to compute winning post proof: %w", err) return false, err From 10032a1ff6b84041cae391a3fff4748b5b1af2da Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Sat, 11 Nov 2023 14:45:58 +0100 Subject: [PATCH 15/24] lpwinning: Handle null-block mining more correctly --- lib/harmony/harmonydb/sql/20231110.sql | 12 +++++++++--- provider/lpwinning/winning_task.go | 22 +++++++++++++++------- 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/lib/harmony/harmonydb/sql/20231110.sql b/lib/harmony/harmonydb/sql/20231110.sql index fad51a28e1e..15b478f4dd1 100644 --- a/lib/harmony/harmonydb/sql/20231110.sql +++ b/lib/harmony/harmonydb/sql/20231110.sql @@ -27,7 +27,13 @@ create table mining_base_block constraint mining_base_block_mining_tasks_task_id_fk references mining_tasks on delete cascade, - block_cid text not null - constraint mining_base_block_cid_k - unique + sp_id bigint, + block_cid text not null, + + no_win bool not null default false, + + constraint mining_base_block_pk2 + unique (sp_id, task_id, block_cid) ); + +CREATE UNIQUE INDEX mining_base_block_cid_k ON mining_base_block (sp_id, block_cid) WHERE no_win = false; diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index ad992b10b48..f8270a7fc98 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -154,6 +154,15 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don ComputeTime: details.CompTime, } + persistNoWin := func() error { + _, err := t.db.Exec(ctx, `UPDATE mining_base_block SET no_win = true WHERE task_id = $1`, taskID) + if err != nil { + return xerrors.Errorf("marking base as not-won: %w", err) + } + + return nil + } + // ensure we have a beacon entry for the epoch we're mining on round := base.epoch() @@ -166,17 +175,16 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don mbi, err := t.api.MinerGetBaseInfo(ctx, maddr, round, base.TipSet.Key()) if err != nil { - err = xerrors.Errorf("failed to get mining base info: %w", err) - return false, err + return false, xerrors.Errorf("failed to get mining base info: %w", err) } if mbi == nil { - // not elloigible to mine on this base, we're done here - return true, nil + // not eligible to mine on this base, we're done here + return true, persistNoWin() } if !mbi.EligibleForMining { // slashed or just have no power yet, we're done here - return true, nil + return true, persistNoWin() } if len(mbi.Sectors) == 0 { @@ -202,7 +210,7 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don if eproof == nil { // not a winner, we're done here - return true, nil + return true, persistNoWin() } } @@ -561,7 +569,7 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { } for _, c := range workBase.TipSet.Cids() { - _, err = tx.Exec(`INSERT INTO mining_base_block (task_id, block_cid) VALUES ($1, $2)`, id, c) + _, err = tx.Exec(`INSERT INTO mining_base_block (task_id, sp_id, block_cid) VALUES ($1, $2, $3)`, id, spID, c) if err != nil { return false, xerrors.Errorf("inserting mining base blocks: %w", err) } From 63b130aa5e8922043803cc8c3fb9486a0efb1531 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Tue, 14 Nov 2023 14:03:36 +0100 Subject: [PATCH 16/24] lpwinning: limit new base rate --- provider/lpwinning/winning_task.go | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/provider/lpwinning/winning_task.go b/provider/lpwinning/winning_task.go index f8270a7fc98..56d66692158 100644 --- a/provider/lpwinning/winning_task.go +++ b/provider/lpwinning/winning_task.go @@ -87,6 +87,8 @@ func NewWinPostTask(max int, db *harmonydb.DB, prover ProverWinningPoSt, verifie } func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (done bool, err error) { + log.Debugw("WinPostTask.Do()", "taskID", taskID) + ctx := context.TODO() type BlockCID struct { @@ -179,15 +181,18 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don } if mbi == nil { // not eligible to mine on this base, we're done here + log.Debugw("WinPoSt not eligible to mine on this base", "tipset", types.LogCids(base.TipSet.Cids())) return true, persistNoWin() } if !mbi.EligibleForMining { // slashed or just have no power yet, we're done here + log.Debugw("WinPoSt not eligible for mining", "tipset", types.LogCids(base.TipSet.Cids())) return true, persistNoWin() } if len(mbi.Sectors) == 0 { + log.Warnw("WinPoSt no sectors to mine", "tipset", types.LogCids(base.TipSet.Cids())) return false, xerrors.Errorf("no sectors selected for winning PoSt") } @@ -205,11 +210,13 @@ func (t *WinPostTask) Do(taskID harmonytask.TaskID, stillOwned func() bool) (don eproof, err = gen.IsRoundWinner(ctx, round, maddr, rbase, mbi, t.api) if err != nil { + log.Warnw("WinPoSt failed to check if we win next round", "error", err) return false, xerrors.Errorf("failed to check if we win next round: %w", err) } if eproof == nil { // not a winner, we're done here + log.Debugw("WinPoSt not a winner", "tipset", types.LogCids(base.TipSet.Cids())) return true, persistNoWin() } } @@ -514,6 +521,11 @@ func (t *WinPostTask) mineBasic(ctx context.Context) { */ for { + // limit the rate at which we mine blocks to at least EquivocationDelaySecs + // this is to prevent races on devnets in catch up mode. Acts as a minimum + // delay for the sleep below. + time.Sleep(time.Duration(build.EquivocationDelaySecs)*time.Second + time.Second) + // wait for *NEXT* propagation delay time.Sleep(time.Until(workBase.afterPropDelay())) From 68cef72fb711f8fe11681784ef1c5f1528b5487d Mon Sep 17 00:00:00 2001 From: "Andrew Jackson (Ajax)" Date: Tue, 14 Nov 2023 15:33:08 -0600 Subject: [PATCH 17/24] migrate-fix --- cmd/lotus-provider/migrate.go | 12 +++++++++--- 1 file changed, 9 insertions(+), 3 deletions(-) diff --git a/cmd/lotus-provider/migrate.go b/cmd/lotus-provider/migrate.go index 7e885813f5e..d91c63d889e 100644 --- a/cmd/lotus-provider/migrate.go +++ b/cmd/lotus-provider/migrate.go @@ -6,12 +6,13 @@ import ( "encoding/base64" "errors" "fmt" - "github.com/filecoin-project/go-address" - "github.com/ipfs/go-datastore" "os" "path" "strings" + "github.com/filecoin-project/go-address" + "github.com/ipfs/go-datastore" + "github.com/BurntSushi/toml" "github.com/samber/lo" "github.com/urfave/cli/v2" @@ -151,6 +152,7 @@ func fromMiner(cctx *cli.Context) (err error) { if err != nil { return xerrors.Errorf("error getting JWTSecretName: %w", err) } + lpCfg.Apis.StorageRPCSecret = base64.StdEncoding.EncodeToString(js.PrivateKey) // Populate API Key @@ -159,7 +161,11 @@ func fromMiner(cctx *cli.Context) (err error) { return fmt.Errorf("cannot read API: %w", err) } - lpCfg.Apis.ChainApiInfo = []string{header.Get("Authorization")[7:]} + ainfo, err := cliutil.GetAPIInfo(&cli.Context{}, repo.FullNode) + if err != nil { + return xerrors.Errorf("could not get API info for FullNode: %w", err) + } + lpCfg.Apis.ChainApiInfo = []string{header.Get("Authorization")[7:] + ":" + ainfo.Addr} // Enable WindowPoSt lpCfg.Subsystems.EnableWindowPost = true From 98099cc3f8f7f30e5bbdda274c9003ef5ce3de71 Mon Sep 17 00:00:00 2001 From: "Andrew Jackson (Ajax)" Date: Mon, 13 Nov 2023 11:53:36 -0600 Subject: [PATCH 18/24] harmonytask: remember machine --- lib/harmony/harmonydb/sql/20231113.sql | 1 + lib/harmony/harmonytask/harmonytask.go | 16 +++++++++------- lib/harmony/harmonytask/task_type_handler.go | 4 ++-- 3 files changed, 12 insertions(+), 9 deletions(-) create mode 100644 lib/harmony/harmonydb/sql/20231113.sql diff --git a/lib/harmony/harmonydb/sql/20231113.sql b/lib/harmony/harmonydb/sql/20231113.sql new file mode 100644 index 00000000000..7a71d98aead --- /dev/null +++ b/lib/harmony/harmonydb/sql/20231113.sql @@ -0,0 +1 @@ +ALTER TABLE harmony_task_history ADD COLUMN completed_by_host_and_port varchar(300) NOT NULL; \ No newline at end of file diff --git a/lib/harmony/harmonytask/harmonytask.go b/lib/harmony/harmonytask/harmonytask.go index bb25be54962..595e5b63a61 100644 --- a/lib/harmony/harmonytask/harmonytask.go +++ b/lib/harmony/harmonytask/harmonytask.go @@ -106,6 +106,7 @@ type TaskEngine struct { follows map[string][]followStruct lastFollowTime time.Time lastCleanup atomic.Value + hostAndPort string } type followStruct struct { f func(TaskID, AddTaskFunc) (bool, error) @@ -129,13 +130,14 @@ func New( } ctx, grace := context.WithCancel(context.Background()) e := &TaskEngine{ - ctx: ctx, - grace: grace, - db: db, - reg: reg, - ownerID: reg.Resources.MachineID, // The current number representing "hostAndPort" - taskMap: make(map[string]*taskTypeHandler, len(impls)), - follows: make(map[string][]followStruct), + ctx: ctx, + grace: grace, + db: db, + reg: reg, + ownerID: reg.Resources.MachineID, // The current number representing "hostAndPort" + taskMap: make(map[string]*taskTypeHandler, len(impls)), + follows: make(map[string][]followStruct), + hostAndPort: hostnameAndPort, } e.lastCleanup.Store(time.Now()) for _, c := range impls { diff --git a/lib/harmony/harmonytask/task_type_handler.go b/lib/harmony/harmonytask/task_type_handler.go index 09540913d6b..7ec47d32a7c 100644 --- a/lib/harmony/harmonytask/task_type_handler.go +++ b/lib/harmony/harmonytask/task_type_handler.go @@ -198,8 +198,8 @@ func (h *taskTypeHandler) recordCompletion(tID TaskID, workStart time.Time, done } } _, err = tx.Exec(`INSERT INTO harmony_task_history - (task_id, name, posted, work_start, work_end, result, err) - VALUES ($1, $2, $3, $4, $5, $6, $7)`, tID, h.Name, postedTime, workStart, workEnd, done, result) + (task_id, name, posted, work_start, work_end, result, by_host_and_port, err) +VALUES ($1, $2, $3, $4, $5, $6, $7, $8)`, tID, h.Name, postedTime, workStart, workEnd, done, h.TaskEngine.hostAndPort, result) if err != nil { return false, fmt.Errorf("could not write history: %w", err) } From aed9a6dd814d131caf57658db1e7a96788e4ed79 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Wed, 15 Nov 2023 13:06:42 +0100 Subject: [PATCH 19/24] sql startup fix, better errors --- cli/util/api.go | 2 +- cmd/lotus-provider/run.go | 5 ++++- lib/harmony/harmonydb/sql/20230823.sql | 4 ++-- 3 files changed, 7 insertions(+), 4 deletions(-) diff --git a/cli/util/api.go b/cli/util/api.go index 7b06efdf324..b76544d575d 100644 --- a/cli/util/api.go +++ b/cli/util/api.go @@ -168,7 +168,7 @@ func GetRawAPIMultiV2(ctx *cli.Context, ainfoCfg []string, version string) ([]Ht var httpHeads []HttpHead if len(ainfoCfg) == 0 { - return httpHeads, xerrors.Errorf("could not get API info: none configured. \nConsider getting base.toml with './lotus-provider config get base >base.toml' \nthen adding \n[APIs] \n FULLNODE_API_INFO = [\" result_from lotus auth api-info --perm=admin \"]\n and updating it with './lotus-provider config set base.toml'") + return httpHeads, xerrors.Errorf("could not get API info: none configured. \nConsider getting base.toml with './lotus-provider config get base >/tmp/base.toml' \nthen adding \n[APIs] \n ChainApiInfo = [\" result_from lotus auth api-info --perm=admin \"]\n and updating it with './lotus-provider config set /tmp/base.toml'") } for _, i := range ainfoCfg { ainfo := ParseApiInfo(i) diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 4484ff6ac39..6248742cc83 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -205,7 +205,10 @@ var runCmd = &cli.Command{ sa, err := StorageAuth(cfg.Apis.StorageRPCSecret) if err != nil { - return xerrors.Errorf("parsing Apis.StorageRPCSecret config: %w", err) + return xerrors.Errorf(`'%w' while parsing the config toml's + [Apis] + StorageRPCSecret=%v +Get it from the JSON documents in ~/.lotus-miner/keystore called .PrivateKey`, err, cfg.Apis.StorageRPCSecret) } al := alerting.NewAlertingSystem(j) diff --git a/lib/harmony/harmonydb/sql/20230823.sql b/lib/harmony/harmonydb/sql/20230823.sql index 3b001a4c1b5..c6f993d7664 100644 --- a/lib/harmony/harmonydb/sql/20230823.sql +++ b/lib/harmony/harmonydb/sql/20230823.sql @@ -37,12 +37,12 @@ create table wdpost_proofs create table wdpost_recovery_tasks ( task_id bigint not null - constraint wdpost_recovery_partition_tasks_pk + constraint wdpost_recovery_tasks_pk primary key, sp_id bigint not null, proving_period_start bigint not null, deadline_index bigint not null, partition_index bigint not null, - constraint wdpost_recovery_partition_tasks_identity_key + constraint wdpost_recovery_tasks_identity_key unique (sp_id, proving_period_start, deadline_index, partition_index) ); \ No newline at end of file From c1edae62ad92b849ac41fdfacaac61bf5c9fea8a Mon Sep 17 00:00:00 2001 From: "Andrew Jackson (Ajax)" Date: Mon, 13 Nov 2023 17:59:34 -0600 Subject: [PATCH 20/24] gen fix --- .circleci/config.yml | 4 --- .circleci/template.yml | 2 -- api/api_common.go | 3 --- api/api_errors.go | 1 - api/api_full.go | 3 --- api/api_gateway.go | 3 --- api/api_net.go | 1 - api/api_storage.go | 3 --- api/api_test.go | 2 -- api/api_wallet.go | 2 -- api/api_worker.go | 3 --- api/cbor_gen.go | 2 -- api/client/client.go | 2 -- api/docgen-openrpc/cmd/docgen_openrpc.go | 1 - api/docgen-openrpc/openrpc.go | 2 -- api/docgen/cmd/docgen.go | 1 - api/docgen/docgen.go | 3 --- api/mocks/mock_full.go | 1 - api/proxy_gen.go | 3 --- api/proxy_util_test.go | 1 - api/types.go | 3 --- api/utils.go | 1 - api/v0api/full.go | 3 --- api/v0api/gateway.go | 3 --- api/v0api/permissioned.go | 1 - api/v0api/proxy_gen.go | 3 --- api/v0api/v0mocks/mock_full.go | 3 --- api/v0api/v1_wrapper.go | 3 --- blockstore/splitstore/splitstore.go | 1 - blockstore/splitstore/splitstore_compact.go | 1 - blockstore/splitstore/splitstore_test.go | 1 - blockstore/splitstore/splitstore_warmup.go | 1 - build/bootstrap.go | 2 -- build/builtin_actors.go | 3 --- build/builtin_actors_test.go | 3 --- build/drand.go | 1 - build/genesis.go | 1 - build/openrpc.go | 1 - build/openrpc_test.go | 1 - build/panic_reporter.go | 1 - build/params_2k.go | 3 --- build/params_butterfly.go | 2 -- build/params_calibnet.go | 3 --- build/params_interop.go | 3 --- build/params_mainnet.go | 1 - build/params_shared_funcs.go | 2 -- build/params_shared_vals.go | 2 -- build/params_testground.go | 3 --- chain/actors/adt/diff_adt_test.go | 3 +-- chain/actors/aerrors/error_test.go | 1 - chain/actors/builtin/account/account.go | 7 +++--- chain/actors/builtin/account/v0.go | 3 +-- chain/actors/builtin/account/v10.go | 1 - chain/actors/builtin/account/v11.go | 1 - chain/actors/builtin/account/v12.go | 1 - chain/actors/builtin/account/v2.go | 3 +-- chain/actors/builtin/account/v3.go | 3 +-- chain/actors/builtin/account/v4.go | 3 +-- chain/actors/builtin/account/v5.go | 3 +-- chain/actors/builtin/account/v6.go | 3 +-- chain/actors/builtin/account/v7.go | 3 +-- chain/actors/builtin/account/v8.go | 1 - chain/actors/builtin/account/v9.go | 1 - chain/actors/builtin/builtin.go | 3 +-- chain/actors/builtin/cron/cron.go | 7 +++--- chain/actors/builtin/cron/v0.go | 3 +-- chain/actors/builtin/cron/v10.go | 1 - chain/actors/builtin/cron/v11.go | 1 - chain/actors/builtin/cron/v12.go | 1 - chain/actors/builtin/cron/v2.go | 3 +-- chain/actors/builtin/cron/v3.go | 3 +-- chain/actors/builtin/cron/v4.go | 3 +-- chain/actors/builtin/cron/v5.go | 3 +-- chain/actors/builtin/cron/v6.go | 3 +-- chain/actors/builtin/cron/v7.go | 3 +-- chain/actors/builtin/cron/v8.go | 1 - chain/actors/builtin/cron/v9.go | 1 - chain/actors/builtin/datacap/datacap.go | 1 - chain/actors/builtin/datacap/util.go | 1 - chain/actors/builtin/datacap/v10.go | 1 - chain/actors/builtin/datacap/v11.go | 1 - chain/actors/builtin/datacap/v12.go | 1 - chain/actors/builtin/datacap/v9.go | 1 - chain/actors/builtin/evm/evm.go | 1 - chain/actors/builtin/evm/v10.go | 1 - chain/actors/builtin/evm/v11.go | 1 - chain/actors/builtin/evm/v12.go | 1 - chain/actors/builtin/init/diff.go | 1 - chain/actors/builtin/init/init.go | 9 +++---- chain/actors/builtin/init/v0.go | 5 ++-- chain/actors/builtin/init/v10.go | 1 - chain/actors/builtin/init/v11.go | 1 - chain/actors/builtin/init/v12.go | 1 - chain/actors/builtin/init/v2.go | 5 ++-- chain/actors/builtin/init/v3.go | 7 +++--- chain/actors/builtin/init/v4.go | 7 +++--- chain/actors/builtin/init/v5.go | 7 +++--- chain/actors/builtin/init/v6.go | 7 +++--- chain/actors/builtin/init/v7.go | 7 +++--- chain/actors/builtin/init/v8.go | 1 - chain/actors/builtin/init/v9.go | 1 - chain/actors/builtin/market/diff.go | 1 - chain/actors/builtin/market/market.go | 7 +++--- chain/actors/builtin/market/v0.go | 5 ++-- chain/actors/builtin/market/v10.go | 1 - chain/actors/builtin/market/v11.go | 1 - chain/actors/builtin/market/v12.go | 1 - chain/actors/builtin/market/v2.go | 5 ++-- chain/actors/builtin/market/v3.go | 5 ++-- chain/actors/builtin/market/v4.go | 5 ++-- chain/actors/builtin/market/v5.go | 5 ++-- chain/actors/builtin/market/v6.go | 5 ++-- chain/actors/builtin/market/v7.go | 5 ++-- chain/actors/builtin/market/v8.go | 1 - chain/actors/builtin/market/v9.go | 1 - chain/actors/builtin/miner/diff.go | 1 - chain/actors/builtin/miner/miner.go | 7 +++--- chain/actors/builtin/miner/v0.go | 5 ++-- chain/actors/builtin/miner/v10.go | 1 - chain/actors/builtin/miner/v11.go | 1 - chain/actors/builtin/miner/v12.go | 1 - chain/actors/builtin/miner/v2.go | 5 ++-- chain/actors/builtin/miner/v3.go | 5 ++-- chain/actors/builtin/miner/v4.go | 5 ++-- chain/actors/builtin/miner/v5.go | 5 ++-- chain/actors/builtin/miner/v6.go | 5 ++-- chain/actors/builtin/miner/v7.go | 5 ++-- chain/actors/builtin/miner/v8.go | 1 - chain/actors/builtin/miner/v9.go | 1 - chain/actors/builtin/multisig/diff.go | 1 - chain/actors/builtin/multisig/message0.go | 7 +++--- chain/actors/builtin/multisig/message10.go | 1 - chain/actors/builtin/multisig/message11.go | 1 - chain/actors/builtin/multisig/message12.go | 1 - chain/actors/builtin/multisig/message2.go | 7 +++--- chain/actors/builtin/multisig/message3.go | 7 +++--- chain/actors/builtin/multisig/message4.go | 7 +++--- chain/actors/builtin/multisig/message5.go | 7 +++--- chain/actors/builtin/multisig/message6.go | 7 +++--- chain/actors/builtin/multisig/message7.go | 7 +++--- chain/actors/builtin/multisig/message8.go | 1 - chain/actors/builtin/multisig/message9.go | 1 - chain/actors/builtin/multisig/multisig.go | 7 +++--- chain/actors/builtin/multisig/v0.go | 5 ++-- chain/actors/builtin/multisig/v10.go | 1 - chain/actors/builtin/multisig/v11.go | 1 - chain/actors/builtin/multisig/v12.go | 1 - chain/actors/builtin/multisig/v2.go | 5 ++-- chain/actors/builtin/multisig/v3.go | 5 ++-- chain/actors/builtin/multisig/v4.go | 5 ++-- chain/actors/builtin/multisig/v5.go | 5 ++-- chain/actors/builtin/multisig/v6.go | 5 ++-- chain/actors/builtin/multisig/v7.go | 5 ++-- chain/actors/builtin/multisig/v8.go | 1 - chain/actors/builtin/multisig/v9.go | 1 - chain/actors/builtin/paych/message0.go | 7 +++--- chain/actors/builtin/paych/message10.go | 1 - chain/actors/builtin/paych/message11.go | 1 - chain/actors/builtin/paych/message12.go | 1 - chain/actors/builtin/paych/message2.go | 7 +++--- chain/actors/builtin/paych/message3.go | 7 +++--- chain/actors/builtin/paych/message4.go | 7 +++--- chain/actors/builtin/paych/message5.go | 7 +++--- chain/actors/builtin/paych/message6.go | 7 +++--- chain/actors/builtin/paych/message7.go | 7 +++--- chain/actors/builtin/paych/message8.go | 1 - chain/actors/builtin/paych/message9.go | 1 - chain/actors/builtin/paych/mock/mock.go | 1 - chain/actors/builtin/paych/paych.go | 7 +++--- chain/actors/builtin/paych/v0.go | 5 ++-- chain/actors/builtin/paych/v10.go | 1 - chain/actors/builtin/paych/v11.go | 1 - chain/actors/builtin/paych/v12.go | 1 - chain/actors/builtin/paych/v2.go | 5 ++-- chain/actors/builtin/paych/v3.go | 5 ++-- chain/actors/builtin/paych/v4.go | 5 ++-- chain/actors/builtin/paych/v5.go | 5 ++-- chain/actors/builtin/paych/v6.go | 5 ++-- chain/actors/builtin/paych/v7.go | 5 ++-- chain/actors/builtin/paych/v8.go | 1 - chain/actors/builtin/paych/v9.go | 1 - chain/actors/builtin/power/diff.go | 1 - chain/actors/builtin/power/power.go | 9 +++---- chain/actors/builtin/power/v0.go | 5 ++-- chain/actors/builtin/power/v10.go | 1 - chain/actors/builtin/power/v11.go | 1 - chain/actors/builtin/power/v12.go | 1 - chain/actors/builtin/power/v2.go | 5 ++-- chain/actors/builtin/power/v3.go | 7 +++--- chain/actors/builtin/power/v4.go | 7 +++--- chain/actors/builtin/power/v5.go | 7 +++--- chain/actors/builtin/power/v6.go | 7 +++--- chain/actors/builtin/power/v7.go | 7 +++--- chain/actors/builtin/power/v8.go | 1 - chain/actors/builtin/power/v9.go | 1 - chain/actors/builtin/registry.go | 1 - chain/actors/builtin/reward/reward.go | 9 +++---- chain/actors/builtin/reward/v0.go | 7 +++--- chain/actors/builtin/reward/v10.go | 1 - chain/actors/builtin/reward/v11.go | 1 - chain/actors/builtin/reward/v12.go | 1 - chain/actors/builtin/reward/v2.go | 7 +++--- chain/actors/builtin/reward/v3.go | 7 +++--- chain/actors/builtin/reward/v4.go | 7 +++--- chain/actors/builtin/reward/v5.go | 7 +++--- chain/actors/builtin/reward/v6.go | 7 +++--- chain/actors/builtin/reward/v7.go | 7 +++--- chain/actors/builtin/reward/v8.go | 1 - chain/actors/builtin/reward/v9.go | 1 - chain/actors/builtin/system/system.go | 7 +++--- chain/actors/builtin/system/v0.go | 3 +-- chain/actors/builtin/system/v10.go | 1 - chain/actors/builtin/system/v11.go | 1 - chain/actors/builtin/system/v12.go | 1 - chain/actors/builtin/system/v2.go | 3 +-- chain/actors/builtin/system/v3.go | 3 +-- chain/actors/builtin/system/v4.go | 3 +-- chain/actors/builtin/system/v5.go | 3 +-- chain/actors/builtin/system/v6.go | 3 +-- chain/actors/builtin/system/v7.go | 3 +-- chain/actors/builtin/system/v8.go | 1 - chain/actors/builtin/system/v9.go | 1 - chain/actors/builtin/verifreg/util.go | 3 +-- chain/actors/builtin/verifreg/v0.go | 5 ++-- chain/actors/builtin/verifreg/v10.go | 1 - chain/actors/builtin/verifreg/v11.go | 1 - chain/actors/builtin/verifreg/v12.go | 1 - chain/actors/builtin/verifreg/v2.go | 5 ++-- chain/actors/builtin/verifreg/v3.go | 5 ++-- chain/actors/builtin/verifreg/v4.go | 5 ++-- chain/actors/builtin/verifreg/v5.go | 5 ++-- chain/actors/builtin/verifreg/v6.go | 5 ++-- chain/actors/builtin/verifreg/v7.go | 5 ++-- chain/actors/builtin/verifreg/v8.go | 1 - chain/actors/builtin/verifreg/v9.go | 1 - chain/actors/builtin/verifreg/verifreg.go | 7 +++--- chain/actors/manifest.go | 1 - chain/actors/params.go | 1 - chain/beacon/beacon.go | 1 - chain/beacon/drand/drand.go | 1 - chain/beacon/drand/drand_test.go | 1 - chain/beacon/mock.go | 1 - chain/consensus/common.go | 3 +-- chain/consensus/compute_state.go | 17 ++++++------- chain/consensus/filcns/filecoin.go | 3 +-- chain/consensus/filcns/upgrades.go | 25 +++++++++---------- chain/consensus/filcns/weight.go | 1 - chain/consensus/iface.go | 1 - chain/consensus/signatures.go | 1 - chain/events/cache.go | 1 - chain/events/events.go | 1 - chain/events/events_called.go | 1 - chain/events/events_height.go | 1 - chain/events/events_test.go | 1 - chain/events/filter/event.go | 3 +-- chain/events/filter/event_test.go | 3 +-- chain/events/filter/index.go | 1 - chain/events/filter/index_test.go | 1 - chain/events/observer.go | 1 - chain/events/state/fastapi.go | 1 - chain/events/state/mock/api.go | 1 - chain/events/state/mock/tipset.go | 1 - chain/events/state/predicates.go | 1 - chain/events/state/predicates_test.go | 11 ++++---- chain/events/tscache.go | 1 - chain/events/tscache_test.go | 1 - chain/exchange/client.go | 1 - chain/exchange/server.go | 1 - chain/gen/gen.go | 3 +-- chain/gen/gen_test.go | 1 - chain/gen/genesis/f00_system.go | 1 - chain/gen/genesis/f01_init.go | 3 +-- chain/gen/genesis/f02_reward.go | 1 - chain/gen/genesis/f03_cron.go | 1 - chain/gen/genesis/f04_power.go | 3 +-- chain/gen/genesis/f05_market.go | 1 - chain/gen/genesis/f06_vreg.go | 3 +-- chain/gen/genesis/f07_dcap.go | 3 +-- chain/gen/genesis/genesis.go | 7 +++--- chain/gen/genesis/genesis_eth.go | 1 - chain/gen/genesis/miners.go | 21 ++++++++-------- chain/gen/genesis/util.go | 1 - chain/gen/slashfilter/slashfilter.go | 1 - .../gen/slashfilter/slashsvc/slashservice.go | 3 +-- chain/index/msgindex.go | 1 - chain/index/msgindex_test.go | 1 - chain/market/fundmanager.go | 1 - chain/market/fundmanager_test.go | 3 +-- chain/market/store.go | 1 - chain/messagepool/check.go | 1 - chain/messagepool/gasguess/guessgas.go | 5 ++-- chain/messagepool/messagepool.go | 3 +-- chain/messagepool/messagepool_test.go | 3 +-- chain/messagepool/provider.go | 1 - chain/messagepool/pruning.go | 1 - chain/messagepool/repub.go | 1 - chain/messagepool/repub_test.go | 3 +-- chain/messagepool/selection.go | 1 - chain/messagepool/selection_test.go | 3 +-- chain/messagesigner/messagesigner.go | 1 - chain/messagesigner/messagesigner_test.go | 1 - chain/rand/rand.go | 1 - chain/rand/rand_test.go | 1 - chain/state/statetree.go | 7 +++--- chain/state/statetree_test.go | 3 +-- chain/stmgr/actors.go | 1 - chain/stmgr/call.go | 1 - chain/stmgr/forks.go | 3 +-- chain/stmgr/forks_test.go | 7 +++--- chain/stmgr/read.go | 1 - chain/stmgr/rpc/rpcstatemanager.go | 1 - chain/stmgr/searchwait.go | 1 - chain/stmgr/searchwait_test.go | 1 - chain/stmgr/stmgr.go | 3 +-- chain/stmgr/supply.go | 3 +-- chain/stmgr/utils.go | 1 - chain/store/basefee.go | 1 - chain/store/index.go | 1 - chain/store/index_test.go | 1 - chain/store/messages.go | 3 +-- chain/store/snapshot.go | 1 - chain/store/store.go | 3 +-- chain/store/store_test.go | 1 - chain/sub/incoming.go | 1 - chain/sub/incoming_test.go | 1 - chain/sync.go | 6 ++--- chain/sync_manager.go | 1 - chain/sync_test.go | 1 - chain/syncstate.go | 1 - chain/types/bigint.go | 1 - chain/types/ethtypes/eth_transactions.go | 1 - chain/types/ethtypes/eth_transactions_test.go | 1 - chain/types/ethtypes/eth_types.go | 1 - chain/types/message.go | 1 - chain/types/mock/chain.go | 1 - chain/types/vmcontext.go | 1 - chain/types_test.go | 1 - chain/vectors/gen/main.go | 1 - chain/vectors/vector_types.go | 1 - chain/vm/fvm.go | 1 - chain/vm/gas.go | 3 +-- chain/vm/gas_v0.go | 3 +-- chain/vm/invoker.go | 3 +-- chain/vm/invoker_test.go | 3 +-- chain/vm/mkactor.go | 13 +++++----- chain/vm/runtime.go | 15 ++++++----- chain/vm/runtime_test.go | 1 - chain/vm/syscalls.go | 5 ++-- chain/vm/vm.go | 1 - chain/vm/vmi.go | 1 - chain/wallet/key/key.go | 1 - chain/wallet/ledger/ledger.go | 1 - chain/wallet/multi.go | 1 - chain/wallet/wallet.go | 1 - cli/auth.go | 1 - cli/backup.go | 1 - cli/chain.go | 13 +++++----- cli/chain_test.go | 3 +-- cli/client.go | 1 - cli/client_retr.go | 1 - cli/disputer.go | 5 ++-- cli/evm.go | 1 - cli/filplus.go | 1 - cli/info.go | 1 - cli/mpool.go | 1 - cli/mpool_manage.go | 1 - cli/mpool_test.go | 1 - cli/multisig.go | 5 ++-- cli/net.go | 1 - cli/params.go | 1 - cli/paych.go | 1 - cli/send.go | 1 - cli/send_test.go | 1 - cli/sending_ui.go | 1 - cli/services.go | 1 - cli/services_send_test.go | 1 - cli/servicesmock_test.go | 1 - cli/state.go | 1 - cli/sync.go | 1 - cli/sync_test.go | 1 - cli/util/api.go | 3 +-- cli/util/epoch.go | 1 - cli/wallet.go | 1 - cli/wallet_test.go | 1 - cmd/chain-noise/main.go | 1 - cmd/lotus-bench/caching_verifier.go | 1 - cmd/lotus-bench/import.go | 1 - cmd/lotus-bench/main.go | 3 +-- cmd/lotus-bench/simple.go | 3 +-- cmd/lotus-fountain/main.go | 1 - cmd/lotus-gateway/main.go | 1 - cmd/lotus-health/main.go | 1 - cmd/lotus-miner/actor.go | 1 - cmd/lotus-miner/actor_test.go | 1 - cmd/lotus-miner/backup.go | 1 - cmd/lotus-miner/info.go | 3 +-- cmd/lotus-miner/init.go | 7 +++--- cmd/lotus-miner/init_restore.go | 1 - cmd/lotus-miner/init_service.go | 1 - cmd/lotus-miner/main.go | 1 - cmd/lotus-miner/market.go | 1 - cmd/lotus-miner/precommits-info.go | 3 +-- cmd/lotus-miner/proving.go | 1 - cmd/lotus-miner/retrieval-deals.go | 1 - cmd/lotus-miner/sealing.go | 1 - cmd/lotus-miner/sectors.go | 1 - cmd/lotus-miner/storage.go | 1 - cmd/lotus-pcr/main.go | 3 +-- cmd/lotus-provider/run.go | 1 - cmd/lotus-seed/genesis.go | 1 - cmd/lotus-seed/main.go | 1 - cmd/lotus-seed/seed/seed.go | 1 - cmd/lotus-shed/actor.go | 3 +-- cmd/lotus-shed/balancer.go | 1 - cmd/lotus-shed/balances.go | 1 - cmd/lotus-shed/cid.go | 1 - cmd/lotus-shed/consensus.go | 1 - cmd/lotus-shed/cron-count.go | 1 - cmd/lotus-shed/deal-label.go | 3 +-- cmd/lotus-shed/diff.go | 1 - cmd/lotus-shed/election.go | 3 +-- cmd/lotus-shed/eth.go | 1 - cmd/lotus-shed/export.go | 1 - cmd/lotus-shed/fevmanalytics.go | 1 - cmd/lotus-shed/fip-0036.go | 1 - cmd/lotus-shed/fr32.go | 1 - cmd/lotus-shed/frozen-miners.go | 3 +-- cmd/lotus-shed/gas-estimation.go | 1 - cmd/lotus-shed/genesis-verify.go | 1 - cmd/lotus-shed/hello.go | 1 - cmd/lotus-shed/indexes.go | 1 - cmd/lotus-shed/invariants.go | 1 - cmd/lotus-shed/jwt.go | 1 - cmd/lotus-shed/ledger.go | 1 - cmd/lotus-shed/market.go | 1 - cmd/lotus-shed/math.go | 3 +-- cmd/lotus-shed/mempool-stats.go | 1 - cmd/lotus-shed/migrations.go | 3 +-- cmd/lotus-shed/miner-multisig.go | 1 - cmd/lotus-shed/miner-peerid.go | 5 ++-- cmd/lotus-shed/miner-types.go | 5 ++-- cmd/lotus-shed/miner.go | 5 ++-- cmd/lotus-shed/msg.go | 3 +-- cmd/lotus-shed/msig.go | 1 - cmd/lotus-shed/nonce-fix.go | 1 - cmd/lotus-shed/params.go | 1 - cmd/lotus-shed/postfind.go | 3 +-- cmd/lotus-shed/proofs.go | 1 - cmd/lotus-shed/pruning.go | 1 - cmd/lotus-shed/sectors.go | 3 +-- cmd/lotus-shed/send-csv.go | 1 - cmd/lotus-shed/signatures.go | 1 - cmd/lotus-shed/state-stats.go | 1 - cmd/lotus-shed/stateroot-stats.go | 1 - cmd/lotus-shed/sync.go | 1 - cmd/lotus-shed/terminations.go | 3 +-- cmd/lotus-shed/verifreg.go | 3 +-- cmd/lotus-sim/info.go | 1 - cmd/lotus-sim/info_capacity.go | 1 - cmd/lotus-sim/info_commit.go | 1 - cmd/lotus-sim/info_state.go | 1 - cmd/lotus-sim/info_wdpost.go | 1 - cmd/lotus-sim/simulation/block.go | 1 - .../simulation/blockbuilder/blockbuilder.go | 1 - cmd/lotus-sim/simulation/messages.go | 3 +-- cmd/lotus-sim/simulation/mock/mock.go | 3 +-- cmd/lotus-sim/simulation/simulation.go | 3 +-- .../simulation/stages/commit_queue.go | 1 - .../simulation/stages/commit_queue_test.go | 1 - .../simulation/stages/funding_stage.go | 1 - cmd/lotus-sim/simulation/stages/interface.go | 1 - .../simulation/stages/precommit_stage.go | 3 +-- .../simulation/stages/provecommit_stage.go | 5 ++-- cmd/lotus-sim/simulation/stages/util.go | 1 - .../simulation/stages/windowpost_stage.go | 1 - cmd/lotus-stats/main.go | 1 - cmd/lotus-wallet/interactive.go | 1 - cmd/lotus-wallet/logged.go | 1 - cmd/lotus-wallet/main.go | 1 - cmd/lotus-worker/main.go | 1 - cmd/lotus-worker/sealworker/rpc.go | 1 - cmd/lotus/backup.go | 1 - cmd/lotus/daemon.go | 1 - cmd/lotus/debug_advance.go | 1 - cmd/tvx/codenames.go | 1 - cmd/tvx/codenames_test.go | 1 - cmd/tvx/exec.go | 3 +-- cmd/tvx/extract_many.go | 1 - cmd/tvx/extract_message.go | 3 +-- cmd/tvx/extract_tipset.go | 3 +-- cmd/tvx/main.go | 1 - cmd/tvx/simulate.go | 3 +-- cmd/tvx/state.go | 1 - conformance/chaos/actor.go | 2 -- conformance/chaos/actor_test.go | 2 -- conformance/chaos/cbor_gen.go | 2 -- conformance/chaos/gen/gen.go | 1 - conformance/chaos/ids.go | 1 - conformance/corpus_test.go | 1 - conformance/driver.go | 3 --- conformance/rand_fixed.go | 2 -- conformance/rand_record.go | 2 -- conformance/rand_replay.go | 2 -- conformance/reporter.go | 1 - conformance/runner.go | 3 --- gateway/eth_sub.go | 2 -- gateway/handler.go | 3 --- gateway/node.go | 3 --- gateway/node_test.go | 3 --- gateway/proxy_eth.go | 3 --- gateway/proxy_fil.go | 3 --- genesis/types.go | 3 --- itests/api_test.go | 1 - itests/batch_deal_test.go | 1 - itests/deadlines_test.go | 3 +-- itests/deals_512mb_test.go | 1 - itests/deals_anycid_test.go | 1 - itests/deals_concurrent_test.go | 1 - itests/deals_invalid_utf8_label_test.go | 1 - itests/deals_max_staging_deals_test.go | 1 - itests/deals_offline_test.go | 1 - itests/deals_padding_test.go | 1 - .../deals_partial_retrieval_dm-level_test.go | 1 - itests/deals_partial_retrieval_test.go | 1 - itests/deals_pricing_test.go | 1 - itests/deals_publish_test.go | 3 +-- itests/deals_remote_retrieval_test.go | 1 - itests/deals_retry_deal_no_funds_test.go | 1 - itests/decode_params_test.go | 1 - itests/dup_mpool_messages_test.go | 1 - itests/eth_account_abstraction_test.go | 1 - itests/eth_api_test.go | 1 - itests/eth_balance_test.go | 1 - itests/eth_block_hash_test.go | 1 - itests/eth_conformance_test.go | 1 - itests/eth_deploy_test.go | 1 - itests/eth_fee_history_test.go | 1 - itests/eth_filter_test.go | 1 - itests/eth_hash_lookup_test.go | 1 - itests/eth_transactions_test.go | 1 - itests/fevm_address_test.go | 1 - itests/fevm_events_test.go | 1 - itests/fevm_test.go | 1 - itests/gas_estimation_test.go | 1 - itests/gateway_test.go | 5 ++-- itests/get_messages_in_ts_test.go | 1 - itests/kit/blockminer.go | 1 - itests/kit/client.go | 3 +-- itests/kit/control.go | 3 +-- itests/kit/deals.go | 1 - itests/kit/ensemble.go | 5 ++-- itests/kit/ensemble_opts.go | 1 - itests/kit/ensemble_opts_nv.go | 1 - itests/kit/evm.go | 1 - itests/kit/funds.go | 1 - itests/kit/init.go | 1 - itests/kit/node_full.go | 1 - itests/kit/node_miner.go | 1 - itests/kit/node_opts.go | 1 - itests/kit/rpc.go | 1 - itests/kit/state.go | 1 - itests/lite_migration_test.go | 3 +-- itests/lookup_robust_address_test.go | 1 - itests/mempool_test.go | 1 - itests/migration_test.go | 1 - itests/mpool_msg_uuid_test.go | 1 - itests/mpool_push_with_uuid_test.go | 1 - itests/msgindex_test.go | 1 - itests/multisig/suite.go | 1 - itests/multisig_test.go | 1 - itests/nonce_test.go | 1 - itests/path_detach_redeclare_test.go | 1 - itests/paych_api_test.go | 1 - itests/paych_cli_test.go | 1 - itests/pending_deal_allocation_test.go | 1 - itests/raft_messagesigner_test.go | 1 - itests/remove_verifreg_datacap_test.go | 1 - itests/sector_import_full_test.go | 1 - itests/sector_import_simple_test.go | 1 - itests/sector_miner_collateral_test.go | 1 - itests/sector_numassign_test.go | 1 - itests/sector_pledge_test.go | 3 +-- itests/sector_terminate_test.go | 1 - itests/sector_unseal_test.go | 1 - itests/self_sent_txn_test.go | 1 - itests/splitstore_test.go | 5 ++-- itests/verifreg_test.go | 1 - itests/wdpost_config_test.go | 1 - itests/wdpost_dispute_test.go | 1 - itests/wdpost_test.go | 1 - itests/wdpost_worker_config_test.go | 1 - itests/worker_test.go | 1 - lib/consensus/raft/consensus.go | 1 - lib/rpcenc/reader.go | 1 - lib/rpcenc/reader_test.go | 1 - lib/sigs/bls/bls_test.go | 1 - lib/sigs/bls/init.go | 1 - lib/sigs/delegated/init.go | 1 - lib/sigs/secp/init.go | 1 - lib/sigs/sigs.go | 1 - lib/unixfs/filestore.go | 1 - markets/dagstore/mount_test.go | 1 - markets/dagstore/wrapper.go | 1 - markets/dagstore/wrapper_migration_test.go | 1 - markets/dagstore/wrapper_test.go | 1 - markets/dealfilter/cli.go | 1 - markets/journal.go | 1 - markets/pricing/cli.go | 1 - markets/retrievaladapter/client.go | 1 - markets/retrievaladapter/client_blockstore.go | 1 - markets/retrievaladapter/provider.go | 1 - markets/retrievaladapter/provider_test.go | 1 - markets/sectoraccessor/sectoraccessor.go | 1 - markets/storageadapter/api.go | 1 - markets/storageadapter/client.go | 3 +-- markets/storageadapter/client_blockstore.go | 1 - markets/storageadapter/dealpublisher.go | 1 - markets/storageadapter/dealpublisher_test.go | 3 +-- markets/storageadapter/dealstatematcher.go | 1 - .../storageadapter/dealstatematcher_test.go | 7 +++--- .../storageadapter/ondealsectorcommitted.go | 1 - .../ondealsectorcommitted_test.go | 3 +-- markets/storageadapter/provider.go | 1 - markets/utils/converters.go | 1 - metrics/metrics.go | 1 - miner/miner.go | 3 --- miner/testminer.go | 3 --- miner/warmup.go | 3 --- node/builder.go | 2 -- node/builder_chain.go | 3 --- node/builder_miner.go | 3 --- node/bundle/bundle.go | 3 --- node/config/def.go | 3 --- node/config/def_test.go | 1 - node/config/doc_gen.go | 4 +-- node/config/load.go | 1 - node/config/load_test.go | 1 - node/config/storage.go | 2 -- node/config/types.go | 1 - node/health.go | 2 -- node/hello/cbor_gen.go | 2 -- node/hello/hello.go | 3 --- node/impl/backup.go | 2 -- node/impl/client/car_helpers.go | 1 - node/impl/client/client.go | 3 --- node/impl/client/client_test.go | 2 -- node/impl/common/common.go | 3 --- node/impl/full.go | 2 -- node/impl/full/chain.go | 3 --- node/impl/full/dummy.go | 3 --- node/impl/full/eth.go | 3 --- node/impl/full/eth_event.go | 3 --- node/impl/full/eth_test.go | 3 --- node/impl/full/eth_trace.go | 3 --- node/impl/full/eth_utils.go | 3 --- node/impl/full/gas.go | 3 --- node/impl/full/gas_test.go | 3 --- node/impl/full/mpool.go | 3 --- node/impl/full/multisig.go | 3 --- node/impl/full/raft.go | 2 -- node/impl/full/state.go | 3 --- node/impl/full/sync.go | 2 -- node/impl/full/txhashmanager.go | 2 -- node/impl/full/wallet.go | 3 --- node/impl/market/market.go | 3 --- node/impl/net/conngater.go | 2 -- node/impl/net/net.go | 2 -- node/impl/net/protect.go | 1 - node/impl/net/rcmgr.go | 2 -- node/impl/paych/paych.go | 3 --- node/impl/remoteworker.go | 3 --- node/impl/storminer.go | 3 --- node/modules/actorevent.go | 3 --- node/modules/alerts.go | 1 - node/modules/blockstore.go | 2 -- node/modules/chain.go | 2 -- node/modules/client.go | 3 --- node/modules/core.go | 3 --- node/modules/dtypes/api.go | 1 - node/modules/dtypes/miner.go | 3 --- node/modules/dtypes/mpool.go | 1 - node/modules/dtypes/scorekeeper.go | 1 - node/modules/dtypes/storage.go | 2 -- node/modules/ethmodule.go | 3 --- node/modules/faultreport.go | 1 - node/modules/genesis.go | 2 -- node/modules/graphsync.go | 2 -- node/modules/helpers/helpers.go | 1 - node/modules/ipfs.go | 2 -- node/modules/lp2p/addrs.go | 1 - node/modules/lp2p/conngater.go | 1 - node/modules/lp2p/discovery.go | 2 -- node/modules/lp2p/host.go | 2 -- node/modules/lp2p/libp2p.go | 2 -- node/modules/lp2p/pubsub.go | 2 -- node/modules/lp2p/rcmgr.go | 2 -- node/modules/lp2p/relay.go | 1 - node/modules/lp2p/routing.go | 1 - node/modules/lp2p/smux.go | 1 - node/modules/mpoolnonceapi.go | 3 --- node/modules/msgindex.go | 2 -- node/modules/paych.go | 2 -- node/modules/rpc.go | 2 -- node/modules/services.go | 3 --- node/modules/stmgr.go | 1 - node/modules/storage.go | 2 -- node/modules/storageminer.go | 3 --- node/modules/storageminer_dagstore.go | 3 --- node/modules/storageminer_idxprov.go | 3 --- node/modules/storageminer_idxprov_test.go | 3 --- node/modules/storageminer_svc.go | 2 -- node/modules/testing/beacon.go | 1 - node/modules/testing/genesis.go | 2 -- .../modules/tracer/elasticsearch_transport.go | 1 - node/modules/tracer/tracer.go | 1 - node/modules/tracer/tracer_test.go | 1 - node/options.go | 1 - node/repo/blockstore_opts.go | 1 - node/repo/fsrepo.go | 4 +-- node/repo/fsrepo_ds.go | 1 - node/repo/imports/manager.go | 2 -- node/repo/interface.go | 2 -- node/repo/memrepo.go | 2 -- node/repo/repo_test.go | 2 -- node/rpc.go | 3 --- node/shutdown_test.go | 1 - node/testopts.go | 2 -- paychmgr/accessorcache.go | 1 - paychmgr/cbor_gen.go | 2 -- paychmgr/manager.go | 3 --- paychmgr/mock_test.go | 3 --- paychmgr/msglistener_test.go | 1 - paychmgr/paych.go | 3 --- paychmgr/paych_test.go | 3 --- paychmgr/paychget_test.go | 3 --- paychmgr/paychvoucherfunds_test.go | 3 --- paychmgr/settle_test.go | 2 -- paychmgr/settler/settler.go | 3 --- paychmgr/simple.go | 3 --- paychmgr/state.go | 2 -- paychmgr/store.go | 3 --- paychmgr/store_test.go | 2 -- paychmgr/util.go | 1 - provider/address.go | 1 - provider/lpmessage/sender.go | 1 - provider/lpwindow/compute_do.go | 7 +++--- provider/lpwindow/compute_task.go | 1 - provider/lpwindow/compute_task_test.go | 1 - provider/lpwindow/faults_simple.go | 1 - provider/lpwindow/recover_task.go | 4 ++- provider/lpwindow/submit_task.go | 1 - storage/ctladdr/addresses.go | 1 - storage/paths/db_index.go | 1 - storage/paths/http_handler.go | 1 - storage/paths/http_handler_test.go | 1 - storage/paths/index.go | 1 - storage/paths/index_locks.go | 1 - storage/paths/index_locks_test.go | 1 - storage/paths/index_proxy.go | 1 - storage/paths/index_test.go | 1 - storage/paths/interface.go | 1 - storage/paths/local.go | 1 - storage/paths/mocks/index.go | 1 - storage/paths/mocks/pf.go | 1 - storage/paths/mocks/store.go | 1 - storage/paths/remote.go | 1 - storage/paths/remote_test.go | 1 - storage/pipeline/cbor_gen.go | 1 - storage/pipeline/checks.go | 1 - storage/pipeline/commit_batch.go | 1 - storage/pipeline/commit_batch_test.go | 3 +-- storage/pipeline/currentdealinfo.go | 1 - storage/pipeline/currentdealinfo_test.go | 5 ++-- storage/pipeline/fsm.go | 1 - storage/pipeline/fsm_events.go | 1 - storage/pipeline/fsm_test.go | 1 - storage/pipeline/input.go | 1 - storage/pipeline/mocks/api.go | 1 - storage/pipeline/mocks/mock_commit_batcher.go | 1 - .../pipeline/mocks/mock_precommit_batcher.go | 1 - storage/pipeline/numassign.go | 1 - storage/pipeline/precommit_batch.go | 1 - storage/pipeline/precommit_batch_test.go | 3 +-- storage/pipeline/precommit_policy.go | 1 - storage/pipeline/precommit_policy_test.go | 1 - storage/pipeline/receive.go | 1 - storage/pipeline/sealing.go | 1 - storage/pipeline/states_failed.go | 1 - storage/pipeline/states_failed_test.go | 3 +-- storage/pipeline/states_proving.go | 1 - storage/pipeline/states_replica_update.go | 1 - storage/pipeline/states_sealing.go | 1 - storage/pipeline/stats.go | 1 - storage/pipeline/terminate_batch.go | 1 - storage/pipeline/types.go | 1 - storage/pipeline/types_test.go | 3 +-- storage/pipeline/upgrade_queue.go | 3 +-- storage/pipeline/utils.go | 1 - storage/sealer/faults.go | 1 - storage/sealer/ffiwrapper/basicfs/fs.go | 1 - storage/sealer/ffiwrapper/prover_cgo.go | 1 - storage/sealer/ffiwrapper/sealer_cgo.go | 1 - storage/sealer/ffiwrapper/sealer_test.go | 1 - storage/sealer/ffiwrapper/unseal_ranges.go | 1 - storage/sealer/ffiwrapper/verifier_cgo.go | 3 +-- storage/sealer/fr32/fr32_ffi_cmp_test.go | 1 - storage/sealer/fr32/fr32_test.go | 1 - storage/sealer/fr32/readers_test.go | 1 - storage/sealer/manager.go | 1 - storage/sealer/manager_post.go | 3 +-- storage/sealer/manager_test.go | 3 +-- storage/sealer/mock/mock.go | 1 - storage/sealer/mock/util.go | 1 - storage/sealer/partialfile/partialfile.go | 1 - storage/sealer/piece_provider.go | 1 - storage/sealer/piece_provider_test.go | 1 - storage/sealer/piece_reader.go | 1 - storage/sealer/sched.go | 1 - storage/sealer/sched_post.go | 1 - storage/sealer/sched_test.go | 1 - storage/sealer/selector_alloc.go | 1 - storage/sealer/selector_existing.go | 1 - storage/sealer/selector_move.go | 1 - storage/sealer/selector_task.go | 1 - storage/sealer/storiface/ffi.go | 1 - storage/sealer/storiface/index.go | 1 - storage/sealer/storiface/resources.go | 1 - storage/sealer/storiface/resources_test.go | 1 - storage/sealer/storiface/worker.go | 1 - storage/sealer/teststorage_test.go | 3 +-- storage/sealer/testworker_test.go | 1 - storage/sealer/worker_calltracker.go | 1 - storage/sealer/worker_local.go | 1 - storage/sealer/worker_local_test.go | 1 - storage/sealer/worker_tracked.go | 1 - storage/sectorblocks/blocks.go | 1 - storage/wdpost/wdpost_changehandler.go | 1 - storage/wdpost/wdpost_changehandler_test.go | 3 +-- storage/wdpost/wdpost_journal.go | 1 - storage/wdpost/wdpost_run.go | 3 +-- storage/wdpost/wdpost_run_faults.go | 1 - storage/wdpost/wdpost_run_test.go | 3 +-- storage/wdpost/wdpost_sched.go | 1 - storage/winning_prover.go | 1 - tools/stats/points/collect.go | 1 - tools/stats/sync/sync.go | 1 - 846 files changed, 407 insertions(+), 1460 deletions(-) diff --git a/.circleci/config.yml b/.circleci/config.yml index eae5e07c8ff..85bcd045cdd 100644 --- a/.circleci/config.yml +++ b/.circleci/config.yml @@ -1018,7 +1018,6 @@ workflows: suite: utest-unit-cli target: "./cli/... ./cmd/... ./api/..." get-params: true - executor: golang-2xl - test: name: test-unit-node requires: @@ -1026,7 +1025,6 @@ workflows: suite: utest-unit-node target: "./node/..." - - test: name: test-unit-rest requires: @@ -1034,7 +1032,6 @@ workflows: suite: utest-unit-rest target: "./blockstore/... ./build/... ./chain/... ./conformance/... ./gateway/... ./journal/... ./lib/... ./markets/... ./paychmgr/... ./provider/... ./tools/..." - executor: golang-2xl - test: name: test-unit-storage requires: @@ -1042,7 +1039,6 @@ workflows: suite: utest-unit-storage target: "./storage/... ./extern/..." - - test: go-test-flags: "-run=TestMulticoreSDR" requires: diff --git a/.circleci/template.yml b/.circleci/template.yml index 7bc84c2187f..2bfba956483 100644 --- a/.circleci/template.yml +++ b/.circleci/template.yml @@ -567,8 +567,6 @@ workflows: suite: utest-[[ $suite ]] target: "[[ $pkgs ]]" [[if eq $suite "unit-cli"]]get-params: true[[end]] - [[if eq $suite "unit-cli"]]executor: golang-2xl[[end]] - [[- if eq $suite "unit-rest"]]executor: golang-2xl[[end]] [[- end]] - test: go-test-flags: "-run=TestMulticoreSDR" diff --git a/api/api_common.go b/api/api_common.go index 2a887a26a75..c3746d1f48a 100644 --- a/api/api_common.go +++ b/api/api_common.go @@ -4,11 +4,8 @@ import ( "context" "fmt" "time" - "github.com/google/uuid" - "github.com/filecoin-project/go-jsonrpc/auth" - apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/journal/alerting" ) diff --git a/api/api_errors.go b/api/api_errors.go index fd157be5fd1..1fa1bbc227a 100644 --- a/api/api_errors.go +++ b/api/api_errors.go @@ -3,7 +3,6 @@ package api import ( "errors" "reflect" - "github.com/filecoin-project/go-jsonrpc" ) diff --git a/api/api_full.go b/api/api_full.go index f919bc13bb4..ed8103002be 100644 --- a/api/api_full.go +++ b/api/api_full.go @@ -5,12 +5,10 @@ import ( "encoding/json" "fmt" "time" - "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -25,7 +23,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" - apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/api_gateway.go b/api/api_gateway.go index 27e7254573c..18a2a82f18e 100644 --- a/api/api_gateway.go +++ b/api/api_gateway.go @@ -2,16 +2,13 @@ package api import ( "context" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" - apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/api_net.go b/api/api_net.go index cfcd8d87e06..49417943c6d 100644 --- a/api/api_net.go +++ b/api/api_net.go @@ -3,7 +3,6 @@ package api import ( "context" "time" - "github.com/libp2p/go-libp2p/core/metrics" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" diff --git a/api/api_storage.go b/api/api_storage.go index d5b3d5c1d67..722c6bd1617 100644 --- a/api/api_storage.go +++ b/api/api_storage.go @@ -4,11 +4,9 @@ import ( "bytes" "context" "time" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -20,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/market" abinetwork "github.com/filecoin-project/go-state-types/network" - builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/api_test.go b/api/api_test.go index 1316d9fa4ca..8397bba2727 100644 --- a/api/api_test.go +++ b/api/api_test.go @@ -10,10 +10,8 @@ import ( "runtime" "strings" "testing" - "github.com/stretchr/testify/require" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc" ) diff --git a/api/api_wallet.go b/api/api_wallet.go index 973aaaf6d85..dd533562197 100644 --- a/api/api_wallet.go +++ b/api/api_wallet.go @@ -2,10 +2,8 @@ package api import ( "context" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/api/api_worker.go b/api/api_worker.go index 197ca898d43..5fefee0647b 100644 --- a/api/api_worker.go +++ b/api/api_worker.go @@ -2,13 +2,10 @@ package api import ( "context" - "github.com/google/uuid" "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/api/cbor_gen.go b/api/cbor_gen.go index fd2cb30b496..8b5049ffd71 100644 --- a/api/cbor_gen.go +++ b/api/cbor_gen.go @@ -7,11 +7,9 @@ import ( "io" "math" "sort" - cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" - abi "github.com/filecoin-project/go-state-types/abi" paych "github.com/filecoin-project/go-state-types/builtin/v8/paych" market "github.com/filecoin-project/go-state-types/builtin/v9/market" diff --git a/api/client/client.go b/api/client/client.go index 8b159c5b1c9..72204302301 100644 --- a/api/client/client.go +++ b/api/client/client.go @@ -6,9 +6,7 @@ import ( "net/url" "path" "time" - "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/api/docgen-openrpc/cmd/docgen_openrpc.go b/api/docgen-openrpc/cmd/docgen_openrpc.go index 5f6bc566f2b..6bc80155673 100644 --- a/api/docgen-openrpc/cmd/docgen_openrpc.go +++ b/api/docgen-openrpc/cmd/docgen_openrpc.go @@ -6,7 +6,6 @@ import ( "io" "log" "os" - "github.com/filecoin-project/lotus/api/docgen" docgen_openrpc "github.com/filecoin-project/lotus/api/docgen-openrpc" ) diff --git a/api/docgen-openrpc/openrpc.go b/api/docgen-openrpc/openrpc.go index c9504ba89c1..404ce537d75 100644 --- a/api/docgen-openrpc/openrpc.go +++ b/api/docgen-openrpc/openrpc.go @@ -5,12 +5,10 @@ import ( "go/ast" "net" "reflect" - "github.com/alecthomas/jsonschema" go_openrpc_reflect "github.com/etclabscore/go-openrpc-reflect" "github.com/ipfs/go-cid" meta_schema "github.com/open-rpc/meta-schema" - "github.com/filecoin-project/lotus/api/docgen" "github.com/filecoin-project/lotus/build" ) diff --git a/api/docgen/cmd/docgen.go b/api/docgen/cmd/docgen.go index 9ae2df2e707..3b1868028cf 100644 --- a/api/docgen/cmd/docgen.go +++ b/api/docgen/cmd/docgen.go @@ -7,7 +7,6 @@ import ( "reflect" "sort" "strings" - "github.com/filecoin-project/lotus/api/docgen" ) diff --git a/api/docgen/docgen.go b/api/docgen/docgen.go index 01862960076..25ece344246 100644 --- a/api/docgen/docgen.go +++ b/api/docgen/docgen.go @@ -12,7 +12,6 @@ import ( "strings" "time" "unicode" - "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" @@ -24,7 +23,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "github.com/multiformats/go-multiaddr" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -35,7 +33,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/api/v0api" diff --git a/api/mocks/mock_full.go b/api/mocks/mock_full.go index 856d83813af..3cd85d969d0 100644 --- a/api/mocks/mock_full.go +++ b/api/mocks/mock_full.go @@ -33,7 +33,6 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" - api "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" miner0 "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/proxy_gen.go b/api/proxy_gen.go index 2d1333495b3..05a29dc8424 100644 --- a/api/proxy_gen.go +++ b/api/proxy_gen.go @@ -6,7 +6,6 @@ import ( "context" "encoding/json" "time" - "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" @@ -15,7 +14,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -31,7 +29,6 @@ import ( "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" - apitypes "github.com/filecoin-project/lotus/api/types" builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/proxy_util_test.go b/api/proxy_util_test.go index adc78a7d15c..aa4f3b55bef 100644 --- a/api/proxy_util_test.go +++ b/api/proxy_util_test.go @@ -3,7 +3,6 @@ package api import ( "testing" - "github.com/stretchr/testify/require" ) diff --git a/api/types.go b/api/types.go index 93ed4083fb8..a5d1f913da6 100644 --- a/api/types.go +++ b/api/types.go @@ -4,7 +4,6 @@ import ( "encoding/json" "fmt" "time" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-graphsync" @@ -14,12 +13,10 @@ import ( "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" ma "github.com/multiformats/go-multiaddr" - "github.com/filecoin-project/go-address" datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/api/utils.go b/api/utils.go index a9d02c31bb2..eda1fdeccdf 100644 --- a/api/utils.go +++ b/api/utils.go @@ -2,7 +2,6 @@ package api import ( "context" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" ) diff --git a/api/v0api/full.go b/api/v0api/full.go index d92d5a95c8e..b72b6efc048 100644 --- a/api/v0api/full.go +++ b/api/v0api/full.go @@ -2,12 +2,10 @@ package v0api import ( "context" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" textselector "github.com/ipld/go-ipld-selector-text-lite" "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -19,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/v0api/gateway.go b/api/v0api/gateway.go index 1a7f7d3ac92..fc2990b28b9 100644 --- a/api/v0api/gateway.go +++ b/api/v0api/gateway.go @@ -2,16 +2,13 @@ package v0api import ( "context" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/v0api/permissioned.go b/api/v0api/permissioned.go index 95fccdfbb6c..ad64bc29ede 100644 --- a/api/v0api/permissioned.go +++ b/api/v0api/permissioned.go @@ -2,7 +2,6 @@ package v0api import ( "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" ) diff --git a/api/v0api/proxy_gen.go b/api/v0api/proxy_gen.go index bd37f64298d..b61f0f76ebb 100644 --- a/api/v0api/proxy_gen.go +++ b/api/v0api/proxy_gen.go @@ -4,12 +4,10 @@ package v0api import ( "context" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -21,7 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/v0api/v0mocks/mock_full.go b/api/v0api/v0mocks/mock_full.go index 7a722ed257c..c5b30868fa7 100644 --- a/api/v0api/v0mocks/mock_full.go +++ b/api/v0api/v0mocks/mock_full.go @@ -8,7 +8,6 @@ import ( context "context" reflect "reflect" time "time" - gomock "github.com/golang/mock/gomock" uuid "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" @@ -17,7 +16,6 @@ import ( network0 "github.com/libp2p/go-libp2p/core/network" peer "github.com/libp2p/go-libp2p/core/peer" protocol "github.com/libp2p/go-libp2p/core/protocol" - address "github.com/filecoin-project/go-address" bitfield "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -32,7 +30,6 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" - api "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" v0api "github.com/filecoin-project/lotus/api/v0api" diff --git a/api/v0api/v1_wrapper.go b/api/v0api/v1_wrapper.go index 265674e718f..382ed038134 100644 --- a/api/v0api/v1_wrapper.go +++ b/api/v0api/v1_wrapper.go @@ -2,18 +2,15 @@ package v0api import ( "context" - "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/blockstore/splitstore/splitstore.go b/blockstore/splitstore/splitstore.go index c1a95c8b060..0ba2eb1ed32 100644 --- a/blockstore/splitstore/splitstore.go +++ b/blockstore/splitstore/splitstore.go @@ -18,7 +18,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/blockstore/splitstore/splitstore_compact.go b/blockstore/splitstore/splitstore_compact.go index 47caca886da..144e0146f62 100644 --- a/blockstore/splitstore/splitstore_compact.go +++ b/blockstore/splitstore/splitstore_compact.go @@ -19,7 +19,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/metrics" diff --git a/blockstore/splitstore/splitstore_test.go b/blockstore/splitstore/splitstore_test.go index 1b821654d08..19a6b82b683 100644 --- a/blockstore/splitstore/splitstore_test.go +++ b/blockstore/splitstore/splitstore_test.go @@ -20,7 +20,6 @@ import ( mh "github.com/multiformats/go-multihash" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" diff --git a/blockstore/splitstore/splitstore_warmup.go b/blockstore/splitstore/splitstore_warmup.go index 7fb6f3b9d08..9b9ec11e0a8 100644 --- a/blockstore/splitstore/splitstore_warmup.go +++ b/blockstore/splitstore/splitstore_warmup.go @@ -11,7 +11,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/build/bootstrap.go b/build/bootstrap.go index d86115398af..43f629472a5 100644 --- a/build/bootstrap.go +++ b/build/bootstrap.go @@ -5,9 +5,7 @@ import ( "embed" "path" "strings" - "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/lotus/lib/addrutil" ) diff --git a/build/builtin_actors.go b/build/builtin_actors.go index 2cf40a754d6..6296411aed9 100644 --- a/build/builtin_actors.go +++ b/build/builtin_actors.go @@ -11,15 +11,12 @@ import ( "sort" "strconv" "strings" - "github.com/DataDog/zstd" "github.com/ipfs/go-cid" cbor "github.com/ipfs/go-ipld-cbor" "github.com/ipld/go-car" "golang.org/x/xerrors" - actorstypes "github.com/filecoin-project/go-state-types/actors" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/build/builtin_actors_test.go b/build/builtin_actors_test.go index bb133bdabd3..6549859a87f 100644 --- a/build/builtin_actors_test.go +++ b/build/builtin_actors_test.go @@ -2,12 +2,9 @@ package build_test import ( "testing" - "github.com/stretchr/testify/require" - actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" ) diff --git a/build/drand.go b/build/drand.go index 3b976ac9254..21cda3bb57c 100644 --- a/build/drand.go +++ b/build/drand.go @@ -2,7 +2,6 @@ package build import ( "sort" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/build/genesis.go b/build/genesis.go index 6d94b38cf68..cd276fabead 100644 --- a/build/genesis.go +++ b/build/genesis.go @@ -3,7 +3,6 @@ package build import ( "embed" "path" - logging "github.com/ipfs/go-log/v2" ) diff --git a/build/openrpc.go b/build/openrpc.go index a50d6f51e78..10d309e22eb 100644 --- a/build/openrpc.go +++ b/build/openrpc.go @@ -5,7 +5,6 @@ import ( "compress/gzip" "embed" "encoding/json" - apitypes "github.com/filecoin-project/lotus/api/types" ) diff --git a/build/openrpc_test.go b/build/openrpc_test.go index 05119fd5d70..1818bacb9fe 100644 --- a/build/openrpc_test.go +++ b/build/openrpc_test.go @@ -3,7 +3,6 @@ package build import ( "testing" - apitypes "github.com/filecoin-project/lotus/api/types" ) diff --git a/build/panic_reporter.go b/build/panic_reporter.go index 617d619eb46..5257faa845e 100644 --- a/build/panic_reporter.go +++ b/build/panic_reporter.go @@ -10,7 +10,6 @@ import ( "strconv" "strings" "time" - "github.com/icza/backscanner" logging "github.com/ipfs/go-log/v2" ) diff --git a/build/params_2k.go b/build/params_2k.go index df4f3dececd..4ec52d6131b 100644 --- a/build/params_2k.go +++ b/build/params_2k.go @@ -6,13 +6,10 @@ package build import ( "os" "strconv" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_butterfly.go b/build/params_butterfly.go index 371c99d5f59..1a40c0c0a17 100644 --- a/build/params_butterfly.go +++ b/build/params_butterfly.go @@ -5,13 +5,11 @@ package build import ( "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_calibnet.go b/build/params_calibnet.go index 2cc53925cd1..7ee29417270 100644 --- a/build/params_calibnet.go +++ b/build/params_calibnet.go @@ -6,15 +6,12 @@ package build import ( "os" "strconv" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_interop.go b/build/params_interop.go index 104ae83c81c..a1981b8e4d9 100644 --- a/build/params_interop.go +++ b/build/params_interop.go @@ -6,15 +6,12 @@ package build import ( "os" "strconv" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_mainnet.go b/build/params_mainnet.go index c607bcdd0d7..9a07dd03d8c 100644 --- a/build/params_mainnet.go +++ b/build/params_mainnet.go @@ -7,7 +7,6 @@ import ( "math" "os" "strconv" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" diff --git a/build/params_shared_funcs.go b/build/params_shared_funcs.go index d117264ab6f..76530a053a7 100644 --- a/build/params_shared_funcs.go +++ b/build/params_shared_funcs.go @@ -3,9 +3,7 @@ package build import ( "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/protocol" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/build/params_shared_vals.go b/build/params_shared_vals.go index 1d15c2fe8b6..bf5819463e4 100644 --- a/build/params_shared_vals.go +++ b/build/params_shared_vals.go @@ -6,12 +6,10 @@ package build import ( "math/big" "os" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_testground.go b/build/params_testground.go index 8073b0c6e7d..eae08a83a5e 100644 --- a/build/params_testground.go +++ b/build/params_testground.go @@ -9,14 +9,11 @@ package build import ( "math/big" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/chain/actors/adt/diff_adt_test.go b/chain/actors/adt/diff_adt_test.go index 7ea3c53e556..740ee3a1f25 100644 --- a/chain/actors/adt/diff_adt_test.go +++ b/chain/actors/adt/diff_adt_test.go @@ -12,10 +12,9 @@ import ( typegen "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" + bstore "github.com/filecoin-project/lotus/blockstore" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - - bstore "github.com/filecoin-project/lotus/blockstore" ) func TestDiffAdtArray(t *testing.T) { diff --git a/chain/actors/aerrors/error_test.go b/chain/actors/aerrors/error_test.go index 8c3738c8816..7458908645e 100644 --- a/chain/actors/aerrors/error_test.go +++ b/chain/actors/aerrors/error_test.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/exitcode" - . "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/actors/builtin/account/account.go b/chain/actors/builtin/account/account.go index dcb60f80186..3fdb1a72831 100644 --- a/chain/actors/builtin/account/account.go +++ b/chain/actors/builtin/account/account.go @@ -9,6 +9,9 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -16,10 +19,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) var Methods = builtin12.MethodsAccount diff --git a/chain/actors/builtin/account/v0.go b/chain/actors/builtin/account/v0.go index a41ee3879ed..987a386fd20 100644 --- a/chain/actors/builtin/account/v0.go +++ b/chain/actors/builtin/account/v0.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account0 "github.com/filecoin-project/specs-actors/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account0 "github.com/filecoin-project/specs-actors/actors/builtin/account" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/account/v10.go b/chain/actors/builtin/account/v10.go index ff87c421220..4dfe82e4fda 100644 --- a/chain/actors/builtin/account/v10.go +++ b/chain/actors/builtin/account/v10.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account10 "github.com/filecoin-project/go-state-types/builtin/v10/account" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v11.go b/chain/actors/builtin/account/v11.go index 7a0c5f556b4..03a7ddabf1f 100644 --- a/chain/actors/builtin/account/v11.go +++ b/chain/actors/builtin/account/v11.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account11 "github.com/filecoin-project/go-state-types/builtin/v11/account" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v12.go b/chain/actors/builtin/account/v12.go index af2c4186fbd..43a235751be 100644 --- a/chain/actors/builtin/account/v12.go +++ b/chain/actors/builtin/account/v12.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account12 "github.com/filecoin-project/go-state-types/builtin/v12/account" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v2.go b/chain/actors/builtin/account/v2.go index db0af77e2a7..3f64e213343 100644 --- a/chain/actors/builtin/account/v2.go +++ b/chain/actors/builtin/account/v2.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/account" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/account/v3.go b/chain/actors/builtin/account/v3.go index 9e6c71ad0f6..4687dba6c0d 100644 --- a/chain/actors/builtin/account/v3.go +++ b/chain/actors/builtin/account/v3.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/account" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/account/v4.go b/chain/actors/builtin/account/v4.go index 907896312ee..9077f88199c 100644 --- a/chain/actors/builtin/account/v4.go +++ b/chain/actors/builtin/account/v4.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/account" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/account/v5.go b/chain/actors/builtin/account/v5.go index 8514ab325e1..6389316f22b 100644 --- a/chain/actors/builtin/account/v5.go +++ b/chain/actors/builtin/account/v5.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/account" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/account/v6.go b/chain/actors/builtin/account/v6.go index 16369f38c26..3dddb10ecb2 100644 --- a/chain/actors/builtin/account/v6.go +++ b/chain/actors/builtin/account/v6.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/account" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/account/v7.go b/chain/actors/builtin/account/v7.go index cd420da92ae..49de3d54aed 100644 --- a/chain/actors/builtin/account/v7.go +++ b/chain/actors/builtin/account/v7.go @@ -8,10 +8,9 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - account7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + account7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/account" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/account/v8.go b/chain/actors/builtin/account/v8.go index 13b478de890..68301269113 100644 --- a/chain/actors/builtin/account/v8.go +++ b/chain/actors/builtin/account/v8.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account8 "github.com/filecoin-project/go-state-types/builtin/v8/account" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v9.go b/chain/actors/builtin/account/v9.go index fc1fc4d14bf..dc98494bf64 100644 --- a/chain/actors/builtin/account/v9.go +++ b/chain/actors/builtin/account/v9.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account9 "github.com/filecoin-project/go-state-types/builtin/v9/account" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/builtin.go b/chain/actors/builtin/builtin.go index 4516683a85c..12320f4b96d 100644 --- a/chain/actors/builtin/builtin.go +++ b/chain/actors/builtin/builtin.go @@ -12,6 +12,7 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/chain/actors" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -19,8 +20,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" ) var InitActorAddr = builtin.InitActorAddr diff --git a/chain/actors/builtin/cron/cron.go b/chain/actors/builtin/cron/cron.go index 17b29178816..3134922e6e3 100644 --- a/chain/actors/builtin/cron/cron.go +++ b/chain/actors/builtin/cron/cron.go @@ -7,6 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -14,10 +17,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/cron/v0.go b/chain/actors/builtin/cron/v0.go index 6dce524f643..9e0c4613eb4 100644 --- a/chain/actors/builtin/cron/v0.go +++ b/chain/actors/builtin/cron/v0.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron0 "github.com/filecoin-project/specs-actors/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron0 "github.com/filecoin-project/specs-actors/actors/builtin/cron" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/cron/v10.go b/chain/actors/builtin/cron/v10.go index 2d20e2401c3..d0d96a8ddbe 100644 --- a/chain/actors/builtin/cron/v10.go +++ b/chain/actors/builtin/cron/v10.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron10 "github.com/filecoin-project/go-state-types/builtin/v10/cron" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v11.go b/chain/actors/builtin/cron/v11.go index 5c489cede81..aee675393de 100644 --- a/chain/actors/builtin/cron/v11.go +++ b/chain/actors/builtin/cron/v11.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron11 "github.com/filecoin-project/go-state-types/builtin/v11/cron" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v12.go b/chain/actors/builtin/cron/v12.go index 44f018d68af..72a04e98499 100644 --- a/chain/actors/builtin/cron/v12.go +++ b/chain/actors/builtin/cron/v12.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron12 "github.com/filecoin-project/go-state-types/builtin/v12/cron" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v2.go b/chain/actors/builtin/cron/v2.go index 97b3ffbe08b..236882f0804 100644 --- a/chain/actors/builtin/cron/v2.go +++ b/chain/actors/builtin/cron/v2.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/cron" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/cron/v3.go b/chain/actors/builtin/cron/v3.go index 4c0d4f1d938..732301273c3 100644 --- a/chain/actors/builtin/cron/v3.go +++ b/chain/actors/builtin/cron/v3.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/cron" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/cron/v4.go b/chain/actors/builtin/cron/v4.go index a222f0d93dc..928153730e9 100644 --- a/chain/actors/builtin/cron/v4.go +++ b/chain/actors/builtin/cron/v4.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/cron" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/cron/v5.go b/chain/actors/builtin/cron/v5.go index 2487cbbc687..e70941ec5be 100644 --- a/chain/actors/builtin/cron/v5.go +++ b/chain/actors/builtin/cron/v5.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/cron" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/cron/v6.go b/chain/actors/builtin/cron/v6.go index 673e7588a9d..21b57933ac8 100644 --- a/chain/actors/builtin/cron/v6.go +++ b/chain/actors/builtin/cron/v6.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/cron" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/cron/v7.go b/chain/actors/builtin/cron/v7.go index cd71bd41833..46c0d4b95a0 100644 --- a/chain/actors/builtin/cron/v7.go +++ b/chain/actors/builtin/cron/v7.go @@ -7,10 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - cron7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/cron" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + cron7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/cron" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/cron/v8.go b/chain/actors/builtin/cron/v8.go index 904de5496a6..71e9f9903e5 100644 --- a/chain/actors/builtin/cron/v8.go +++ b/chain/actors/builtin/cron/v8.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron8 "github.com/filecoin-project/go-state-types/builtin/v8/cron" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v9.go b/chain/actors/builtin/cron/v9.go index 201348b6cd8..f7160c193d4 100644 --- a/chain/actors/builtin/cron/v9.go +++ b/chain/actors/builtin/cron/v9.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron9 "github.com/filecoin-project/go-state-types/builtin/v9/cron" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/datacap.go b/chain/actors/builtin/datacap/datacap.go index 0c8f04bbf40..798c88c661f 100644 --- a/chain/actors/builtin/datacap/datacap.go +++ b/chain/actors/builtin/datacap/datacap.go @@ -10,7 +10,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/datacap/util.go b/chain/actors/builtin/datacap/util.go index 03e941d6e04..ce59688f9fa 100644 --- a/chain/actors/builtin/datacap/util.go +++ b/chain/actors/builtin/datacap/util.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v10.go b/chain/actors/builtin/datacap/v10.go index 25eec4ea819..e260cf8aab4 100644 --- a/chain/actors/builtin/datacap/v10.go +++ b/chain/actors/builtin/datacap/v10.go @@ -11,7 +11,6 @@ import ( datacap10 "github.com/filecoin-project/go-state-types/builtin/v10/datacap" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v11.go b/chain/actors/builtin/datacap/v11.go index 0c302b5e187..02d1ba04568 100644 --- a/chain/actors/builtin/datacap/v11.go +++ b/chain/actors/builtin/datacap/v11.go @@ -11,7 +11,6 @@ import ( datacap11 "github.com/filecoin-project/go-state-types/builtin/v11/datacap" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v12.go b/chain/actors/builtin/datacap/v12.go index 91563a2b69d..afba772a0bd 100644 --- a/chain/actors/builtin/datacap/v12.go +++ b/chain/actors/builtin/datacap/v12.go @@ -11,7 +11,6 @@ import ( datacap12 "github.com/filecoin-project/go-state-types/builtin/v12/datacap" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v9.go b/chain/actors/builtin/datacap/v9.go index 1d239fb95be..f6ae057e245 100644 --- a/chain/actors/builtin/datacap/v9.go +++ b/chain/actors/builtin/datacap/v9.go @@ -11,7 +11,6 @@ import ( datacap9 "github.com/filecoin-project/go-state-types/builtin/v9/datacap" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/evm.go b/chain/actors/builtin/evm/evm.go index 98f860cac4a..3169ac4a955 100644 --- a/chain/actors/builtin/evm/evm.go +++ b/chain/actors/builtin/evm/evm.go @@ -8,7 +8,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/evm/v10.go b/chain/actors/builtin/evm/v10.go index d467aa187d6..7633cb6defc 100644 --- a/chain/actors/builtin/evm/v10.go +++ b/chain/actors/builtin/evm/v10.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm10 "github.com/filecoin-project/go-state-types/builtin/v10/evm" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/v11.go b/chain/actors/builtin/evm/v11.go index 9e2e984e46c..04ca7794957 100644 --- a/chain/actors/builtin/evm/v11.go +++ b/chain/actors/builtin/evm/v11.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm11 "github.com/filecoin-project/go-state-types/builtin/v11/evm" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/v12.go b/chain/actors/builtin/evm/v12.go index a107368faa4..e6965f3d466 100644 --- a/chain/actors/builtin/evm/v12.go +++ b/chain/actors/builtin/evm/v12.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm12 "github.com/filecoin-project/go-state-types/builtin/v12/evm" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/init/diff.go b/chain/actors/builtin/init/diff.go index 60b5a5bff15..be6825bfc8b 100644 --- a/chain/actors/builtin/init/diff.go +++ b/chain/actors/builtin/init/diff.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/init/init.go b/chain/actors/builtin/init/init.go index 41a763ecf61..b94fdb38830 100644 --- a/chain/actors/builtin/init/init.go +++ b/chain/actors/builtin/init/init.go @@ -10,6 +10,10 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/node/modules/dtypes" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -17,11 +21,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) var ( diff --git a/chain/actors/builtin/init/v0.go b/chain/actors/builtin/init/v0.go index 7e48dda9e13..7cc1af3e28f 100644 --- a/chain/actors/builtin/init/v0.go +++ b/chain/actors/builtin/init/v0.go @@ -12,12 +12,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/init/v10.go b/chain/actors/builtin/init/v10.go index dd8c778dd4a..66a6f2cc592 100644 --- a/chain/actors/builtin/init/v10.go +++ b/chain/actors/builtin/init/v10.go @@ -15,7 +15,6 @@ import ( init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v11.go b/chain/actors/builtin/init/v11.go index 3d8d72e499f..8d89988181e 100644 --- a/chain/actors/builtin/init/v11.go +++ b/chain/actors/builtin/init/v11.go @@ -15,7 +15,6 @@ import ( init11 "github.com/filecoin-project/go-state-types/builtin/v11/init" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v12.go b/chain/actors/builtin/init/v12.go index 3eab7a74050..2ab877a153f 100644 --- a/chain/actors/builtin/init/v12.go +++ b/chain/actors/builtin/init/v12.go @@ -15,7 +15,6 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v2.go b/chain/actors/builtin/init/v2.go index c107bd52d59..39d8f7693c7 100644 --- a/chain/actors/builtin/init/v2.go +++ b/chain/actors/builtin/init/v2.go @@ -12,12 +12,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/init/v3.go b/chain/actors/builtin/init/v3.go index 0be11f9761a..55354208512 100644 --- a/chain/actors/builtin/init/v3.go +++ b/chain/actors/builtin/init/v3.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" - adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" + adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/init/v4.go b/chain/actors/builtin/init/v4.go index 5ca6bc1c81b..366cb5383c0 100644 --- a/chain/actors/builtin/init/v4.go +++ b/chain/actors/builtin/init/v4.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" - adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" + adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/init/v5.go b/chain/actors/builtin/init/v5.go index f6450789d71..16f20af7088 100644 --- a/chain/actors/builtin/init/v5.go +++ b/chain/actors/builtin/init/v5.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" - init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" - adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" + init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" + adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/init/v6.go b/chain/actors/builtin/init/v6.go index 4d2267aa1dc..49dc402dbea 100644 --- a/chain/actors/builtin/init/v6.go +++ b/chain/actors/builtin/init/v6.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" - adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" + adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/init/v7.go b/chain/actors/builtin/init/v7.go index 052faf98502..1f8287220a5 100644 --- a/chain/actors/builtin/init/v7.go +++ b/chain/actors/builtin/init/v7.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" - adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" + builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" + adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/init/v8.go b/chain/actors/builtin/init/v8.go index c7c7860d399..061a5377f58 100644 --- a/chain/actors/builtin/init/v8.go +++ b/chain/actors/builtin/init/v8.go @@ -15,7 +15,6 @@ import ( init8 "github.com/filecoin-project/go-state-types/builtin/v8/init" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v9.go b/chain/actors/builtin/init/v9.go index a221a4a7c4b..b246392260c 100644 --- a/chain/actors/builtin/init/v9.go +++ b/chain/actors/builtin/init/v9.go @@ -15,7 +15,6 @@ import ( init9 "github.com/filecoin-project/go-state-types/builtin/v9/init" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/market/diff.go b/chain/actors/builtin/market/diff.go index ef3c2c28d7c..26b47c3f3b4 100644 --- a/chain/actors/builtin/market/diff.go +++ b/chain/actors/builtin/market/diff.go @@ -6,7 +6,6 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/market/market.go b/chain/actors/builtin/market/market.go index 39473d56062..52238f3247a 100644 --- a/chain/actors/builtin/market/market.go +++ b/chain/actors/builtin/market/market.go @@ -17,6 +17,9 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -24,10 +27,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/market/v0.go b/chain/actors/builtin/market/v0.go index ca6970dfaa1..e597fe979f9 100644 --- a/chain/actors/builtin/market/v0.go +++ b/chain/actors/builtin/market/v0.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/market/v10.go b/chain/actors/builtin/market/v10.go index 878f0d46584..1f454557cd4 100644 --- a/chain/actors/builtin/market/v10.go +++ b/chain/actors/builtin/market/v10.go @@ -18,7 +18,6 @@ import ( adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v11.go b/chain/actors/builtin/market/v11.go index a6427220972..ec328c586f5 100644 --- a/chain/actors/builtin/market/v11.go +++ b/chain/actors/builtin/market/v11.go @@ -18,7 +18,6 @@ import ( adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v12.go b/chain/actors/builtin/market/v12.go index 56e651a9be5..ef87b2cc794 100644 --- a/chain/actors/builtin/market/v12.go +++ b/chain/actors/builtin/market/v12.go @@ -18,7 +18,6 @@ import ( adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v2.go b/chain/actors/builtin/market/v2.go index ba84e3b03a0..48287cb1f78 100644 --- a/chain/actors/builtin/market/v2.go +++ b/chain/actors/builtin/market/v2.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/market/v3.go b/chain/actors/builtin/market/v3.go index f6a0891e730..c6f007a0846 100644 --- a/chain/actors/builtin/market/v3.go +++ b/chain/actors/builtin/market/v3.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/market" - adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/market" + adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/market/v4.go b/chain/actors/builtin/market/v4.go index 629e833b67b..9d7cf0736a5 100644 --- a/chain/actors/builtin/market/v4.go +++ b/chain/actors/builtin/market/v4.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/market" - adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/market" + adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/market/v5.go b/chain/actors/builtin/market/v5.go index 8925889791f..f12c24279e7 100644 --- a/chain/actors/builtin/market/v5.go +++ b/chain/actors/builtin/market/v5.go @@ -12,13 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" - adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" + adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/market/v6.go b/chain/actors/builtin/market/v6.go index b57d49f9117..a4abd4d40c9 100644 --- a/chain/actors/builtin/market/v6.go +++ b/chain/actors/builtin/market/v6.go @@ -14,13 +14,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/market" - adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/market" + adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/market/v7.go b/chain/actors/builtin/market/v7.go index 56a1db328f9..597ca34f1c1 100644 --- a/chain/actors/builtin/market/v7.go +++ b/chain/actors/builtin/market/v7.go @@ -14,13 +14,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" - adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" + adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/market/v8.go b/chain/actors/builtin/market/v8.go index 9c68ee1fd86..067883a565e 100644 --- a/chain/actors/builtin/market/v8.go +++ b/chain/actors/builtin/market/v8.go @@ -17,7 +17,6 @@ import ( adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v9.go b/chain/actors/builtin/market/v9.go index d692c15ccb7..0e6634d1606 100644 --- a/chain/actors/builtin/market/v9.go +++ b/chain/actors/builtin/market/v9.go @@ -18,7 +18,6 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/miner/diff.go b/chain/actors/builtin/miner/diff.go index d20a4374623..c1536043774 100644 --- a/chain/actors/builtin/miner/diff.go +++ b/chain/actors/builtin/miner/diff.go @@ -4,7 +4,6 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/miner.go b/chain/actors/builtin/miner/miner.go index 1637cdadeda..cbc06a63687 100644 --- a/chain/actors/builtin/miner/miner.go +++ b/chain/actors/builtin/miner/miner.go @@ -15,6 +15,9 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -22,10 +25,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/miner/v0.go b/chain/actors/builtin/miner/v0.go index 5301ed1cbca..4970e248b10 100644 --- a/chain/actors/builtin/miner/v0.go +++ b/chain/actors/builtin/miner/v0.go @@ -16,11 +16,10 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/miner/v10.go b/chain/actors/builtin/miner/v10.go index 53dc90b4560..121cac90d8e 100644 --- a/chain/actors/builtin/miner/v10.go +++ b/chain/actors/builtin/miner/v10.go @@ -18,7 +18,6 @@ import ( adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v11.go b/chain/actors/builtin/miner/v11.go index 11a91c26b23..b9497c2193d 100644 --- a/chain/actors/builtin/miner/v11.go +++ b/chain/actors/builtin/miner/v11.go @@ -18,7 +18,6 @@ import ( adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v12.go b/chain/actors/builtin/miner/v12.go index 90ecc97fd3d..0691674f1bd 100644 --- a/chain/actors/builtin/miner/v12.go +++ b/chain/actors/builtin/miner/v12.go @@ -18,7 +18,6 @@ import ( adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v2.go b/chain/actors/builtin/miner/v2.go index 5a81ad31f95..5c9e9d22858 100644 --- a/chain/actors/builtin/miner/v2.go +++ b/chain/actors/builtin/miner/v2.go @@ -15,11 +15,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/miner/v3.go b/chain/actors/builtin/miner/v3.go index aa1574cf47d..ed278c08ed7 100644 --- a/chain/actors/builtin/miner/v3.go +++ b/chain/actors/builtin/miner/v3.go @@ -15,12 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/miner/v4.go b/chain/actors/builtin/miner/v4.go index 1faf30c0982..33dadc032cc 100644 --- a/chain/actors/builtin/miner/v4.go +++ b/chain/actors/builtin/miner/v4.go @@ -15,12 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" miner4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/miner" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/miner/v5.go b/chain/actors/builtin/miner/v5.go index be4b5e0b2e9..771ee57243c 100644 --- a/chain/actors/builtin/miner/v5.go +++ b/chain/actors/builtin/miner/v5.go @@ -15,12 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/miner/v6.go b/chain/actors/builtin/miner/v6.go index fa8c30e4070..38dfdd8565b 100644 --- a/chain/actors/builtin/miner/v6.go +++ b/chain/actors/builtin/miner/v6.go @@ -15,12 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/miner/v7.go b/chain/actors/builtin/miner/v7.go index d6bb0e16eae..c5096507e63 100644 --- a/chain/actors/builtin/miner/v7.go +++ b/chain/actors/builtin/miner/v7.go @@ -15,12 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" miner7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/miner" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/miner/v8.go b/chain/actors/builtin/miner/v8.go index 06a205e7626..a19c4e2aabb 100644 --- a/chain/actors/builtin/miner/v8.go +++ b/chain/actors/builtin/miner/v8.go @@ -18,7 +18,6 @@ import ( adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v9.go b/chain/actors/builtin/miner/v9.go index 6cbbd509ec3..dc13b646e39 100644 --- a/chain/actors/builtin/miner/v9.go +++ b/chain/actors/builtin/miner/v9.go @@ -18,7 +18,6 @@ import ( adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/diff.go b/chain/actors/builtin/multisig/diff.go index f24931fb823..7ffa1f2706f 100644 --- a/chain/actors/builtin/multisig/diff.go +++ b/chain/actors/builtin/multisig/diff.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/message0.go b/chain/actors/builtin/multisig/message0.go index 7dbdf444ce2..699cadd0720 100644 --- a/chain/actors/builtin/multisig/message0.go +++ b/chain/actors/builtin/multisig/message0.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" - multisig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" + multisig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" ) type message0 struct{ from address.Address } diff --git a/chain/actors/builtin/multisig/message10.go b/chain/actors/builtin/multisig/message10.go index 8f7bb5a6f2e..cc94f079248 100644 --- a/chain/actors/builtin/multisig/message10.go +++ b/chain/actors/builtin/multisig/message10.go @@ -10,7 +10,6 @@ import ( multisig10 "github.com/filecoin-project/go-state-types/builtin/v10/multisig" init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message11.go b/chain/actors/builtin/multisig/message11.go index 4c7520d5dea..d6606b512b1 100644 --- a/chain/actors/builtin/multisig/message11.go +++ b/chain/actors/builtin/multisig/message11.go @@ -10,7 +10,6 @@ import ( multisig11 "github.com/filecoin-project/go-state-types/builtin/v11/multisig" init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message12.go b/chain/actors/builtin/multisig/message12.go index 43658c04b03..8a0dad844e9 100644 --- a/chain/actors/builtin/multisig/message12.go +++ b/chain/actors/builtin/multisig/message12.go @@ -10,7 +10,6 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message2.go b/chain/actors/builtin/multisig/message2.go index 91ad17b84ab..e98f7057621 100644 --- a/chain/actors/builtin/multisig/message2.go +++ b/chain/actors/builtin/multisig/message2.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) type message2 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message3.go b/chain/actors/builtin/multisig/message3.go index 4124e00c86b..b397ec79099 100644 --- a/chain/actors/builtin/multisig/message3.go +++ b/chain/actors/builtin/multisig/message3.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" - multisig3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" + multisig3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/multisig" ) type message3 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message4.go b/chain/actors/builtin/multisig/message4.go index 33449df75f3..f56447c3bc6 100644 --- a/chain/actors/builtin/multisig/message4.go +++ b/chain/actors/builtin/multisig/message4.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" - multisig4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" + multisig4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/multisig" ) type message4 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message5.go b/chain/actors/builtin/multisig/message5.go index 46c35dabce8..3b020560a0e 100644 --- a/chain/actors/builtin/multisig/message5.go +++ b/chain/actors/builtin/multisig/message5.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" - init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" - multisig5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" + init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" + multisig5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/multisig" ) type message5 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message6.go b/chain/actors/builtin/multisig/message6.go index f528cfbb5da..779c0009016 100644 --- a/chain/actors/builtin/multisig/message6.go +++ b/chain/actors/builtin/multisig/message6.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" - multisig6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" + multisig6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/multisig" ) type message6 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message7.go b/chain/actors/builtin/multisig/message7.go index 6e62dad13a0..8cb90d70d07 100644 --- a/chain/actors/builtin/multisig/message7.go +++ b/chain/actors/builtin/multisig/message7.go @@ -6,13 +6,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" - multisig7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" + multisig7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/multisig" ) type message7 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message8.go b/chain/actors/builtin/multisig/message8.go index 390c94691e4..7f6fa8de223 100644 --- a/chain/actors/builtin/multisig/message8.go +++ b/chain/actors/builtin/multisig/message8.go @@ -10,7 +10,6 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig8 "github.com/filecoin-project/go-state-types/builtin/v8/multisig" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message9.go b/chain/actors/builtin/multisig/message9.go index 907bec7d556..4a1aa46f532 100644 --- a/chain/actors/builtin/multisig/message9.go +++ b/chain/actors/builtin/multisig/message9.go @@ -10,7 +10,6 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig9 "github.com/filecoin-project/go-state-types/builtin/v9/multisig" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/multisig.go b/chain/actors/builtin/multisig/multisig.go index 71a3b7b2237..bea27523c7e 100644 --- a/chain/actors/builtin/multisig/multisig.go +++ b/chain/actors/builtin/multisig/multisig.go @@ -15,6 +15,9 @@ import ( msig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -22,10 +25,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/multisig/v0.go b/chain/actors/builtin/multisig/v0.go index 86bfdaaf37a..a36efb456f6 100644 --- a/chain/actors/builtin/multisig/v0.go +++ b/chain/actors/builtin/multisig/v0.go @@ -13,11 +13,10 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/multisig/v10.go b/chain/actors/builtin/multisig/v10.go index d87fc5807ab..6d6aa636567 100644 --- a/chain/actors/builtin/multisig/v10.go +++ b/chain/actors/builtin/multisig/v10.go @@ -16,7 +16,6 @@ import ( msig10 "github.com/filecoin-project/go-state-types/builtin/v10/multisig" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v11.go b/chain/actors/builtin/multisig/v11.go index 3627dc959cb..08724d9f01a 100644 --- a/chain/actors/builtin/multisig/v11.go +++ b/chain/actors/builtin/multisig/v11.go @@ -16,7 +16,6 @@ import ( msig11 "github.com/filecoin-project/go-state-types/builtin/v11/multisig" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v12.go b/chain/actors/builtin/multisig/v12.go index d3d2f3809a9..c3200251213 100644 --- a/chain/actors/builtin/multisig/v12.go +++ b/chain/actors/builtin/multisig/v12.go @@ -16,7 +16,6 @@ import ( msig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v2.go b/chain/actors/builtin/multisig/v2.go index 77330d5139d..79b83b8fe61 100644 --- a/chain/actors/builtin/multisig/v2.go +++ b/chain/actors/builtin/multisig/v2.go @@ -13,11 +13,10 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/multisig/v3.go b/chain/actors/builtin/multisig/v3.go index e8659093fc7..7c086b52fb4 100644 --- a/chain/actors/builtin/multisig/v3.go +++ b/chain/actors/builtin/multisig/v3.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" msig3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/multisig" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/multisig/v4.go b/chain/actors/builtin/multisig/v4.go index ddaac547056..6fe2f0ce353 100644 --- a/chain/actors/builtin/multisig/v4.go +++ b/chain/actors/builtin/multisig/v4.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" msig4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/multisig" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/multisig/v5.go b/chain/actors/builtin/multisig/v5.go index 50474d5fd4a..2ff91ebfbd9 100644 --- a/chain/actors/builtin/multisig/v5.go +++ b/chain/actors/builtin/multisig/v5.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" msig5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/multisig" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/multisig/v6.go b/chain/actors/builtin/multisig/v6.go index c51404dc49b..05ca85fa6f1 100644 --- a/chain/actors/builtin/multisig/v6.go +++ b/chain/actors/builtin/multisig/v6.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" msig6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/multisig" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/multisig/v7.go b/chain/actors/builtin/multisig/v7.go index 9ddce167af6..0eae904c23c 100644 --- a/chain/actors/builtin/multisig/v7.go +++ b/chain/actors/builtin/multisig/v7.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" msig7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/multisig" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/multisig/v8.go b/chain/actors/builtin/multisig/v8.go index b28ec5684d0..41055e77bdb 100644 --- a/chain/actors/builtin/multisig/v8.go +++ b/chain/actors/builtin/multisig/v8.go @@ -16,7 +16,6 @@ import ( msig8 "github.com/filecoin-project/go-state-types/builtin/v8/multisig" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v9.go b/chain/actors/builtin/multisig/v9.go index faa3b7d3731..4cba947860a 100644 --- a/chain/actors/builtin/multisig/v9.go +++ b/chain/actors/builtin/multisig/v9.go @@ -16,7 +16,6 @@ import ( msig9 "github.com/filecoin-project/go-state-types/builtin/v9/multisig" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/message0.go b/chain/actors/builtin/paych/message0.go index d5a112d43b2..06402750250 100644 --- a/chain/actors/builtin/paych/message0.go +++ b/chain/actors/builtin/paych/message0.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" - paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" + paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" ) type message0 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message10.go b/chain/actors/builtin/paych/message10.go index 03daa7bcf2f..996c9e6ffce 100644 --- a/chain/actors/builtin/paych/message10.go +++ b/chain/actors/builtin/paych/message10.go @@ -10,7 +10,6 @@ import ( init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" paych10 "github.com/filecoin-project/go-state-types/builtin/v10/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message11.go b/chain/actors/builtin/paych/message11.go index 7a9ec1c8e5f..4f5ce6e7f49 100644 --- a/chain/actors/builtin/paych/message11.go +++ b/chain/actors/builtin/paych/message11.go @@ -10,7 +10,6 @@ import ( init11 "github.com/filecoin-project/go-state-types/builtin/v11/init" paych11 "github.com/filecoin-project/go-state-types/builtin/v11/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message12.go b/chain/actors/builtin/paych/message12.go index bd821641ae4..047de49768d 100644 --- a/chain/actors/builtin/paych/message12.go +++ b/chain/actors/builtin/paych/message12.go @@ -10,7 +10,6 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" paych12 "github.com/filecoin-project/go-state-types/builtin/v12/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message2.go b/chain/actors/builtin/paych/message2.go index 89a6e74ce9b..ee4a50fb950 100644 --- a/chain/actors/builtin/paych/message2.go +++ b/chain/actors/builtin/paych/message2.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" ) type message2 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message3.go b/chain/actors/builtin/paych/message3.go index 47f8e7e9e25..2534daf4130 100644 --- a/chain/actors/builtin/paych/message3.go +++ b/chain/actors/builtin/paych/message3.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" - paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" + paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" ) type message3 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message4.go b/chain/actors/builtin/paych/message4.go index 65e63149d9d..315dec5bb6a 100644 --- a/chain/actors/builtin/paych/message4.go +++ b/chain/actors/builtin/paych/message4.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" - paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" + paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" ) type message4 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message5.go b/chain/actors/builtin/paych/message5.go index ede1c8ceb7f..98d2ceaad28 100644 --- a/chain/actors/builtin/paych/message5.go +++ b/chain/actors/builtin/paych/message5.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" - init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" - paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" + init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" + paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" ) type message5 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message6.go b/chain/actors/builtin/paych/message6.go index 009e8641fe7..5b6f6586bfe 100644 --- a/chain/actors/builtin/paych/message6.go +++ b/chain/actors/builtin/paych/message6.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" - paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" + paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" ) type message6 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message7.go b/chain/actors/builtin/paych/message7.go index 430fb93415b..67ac2e174c2 100644 --- a/chain/actors/builtin/paych/message7.go +++ b/chain/actors/builtin/paych/message7.go @@ -4,13 +4,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" - paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" + builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" + paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" ) type message7 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message8.go b/chain/actors/builtin/paych/message8.go index a72c5fbb383..68104bb566e 100644 --- a/chain/actors/builtin/paych/message8.go +++ b/chain/actors/builtin/paych/message8.go @@ -10,7 +10,6 @@ import ( init8 "github.com/filecoin-project/go-state-types/builtin/v8/init" paych8 "github.com/filecoin-project/go-state-types/builtin/v8/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message9.go b/chain/actors/builtin/paych/message9.go index 350c760eafb..56ba2d0fad2 100644 --- a/chain/actors/builtin/paych/message9.go +++ b/chain/actors/builtin/paych/message9.go @@ -10,7 +10,6 @@ import ( paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" init9 "github.com/filecoin-project/go-state-types/builtin/v9/init" paych9 "github.com/filecoin-project/go-state-types/builtin/v9/paych" - "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/mock/mock.go b/chain/actors/builtin/paych/mock/mock.go index 3b888131966..627be126768 100644 --- a/chain/actors/builtin/paych/mock/mock.go +++ b/chain/actors/builtin/paych/mock/mock.go @@ -10,7 +10,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors/builtin/paych" ) diff --git a/chain/actors/builtin/paych/paych.go b/chain/actors/builtin/paych/paych.go index fc8908cb4e1..75f6a039609 100644 --- a/chain/actors/builtin/paych/paych.go +++ b/chain/actors/builtin/paych/paych.go @@ -15,6 +15,9 @@ import ( paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" @@ -23,10 +26,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) // Load returns an abstract copy of payment channel state, regardless of actor version diff --git a/chain/actors/builtin/paych/v0.go b/chain/actors/builtin/paych/v0.go index facc7f65671..ae2acbb9d23 100644 --- a/chain/actors/builtin/paych/v0.go +++ b/chain/actors/builtin/paych/v0.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/paych/v10.go b/chain/actors/builtin/paych/v10.go index edc6c96b6cc..f74c0791b4f 100644 --- a/chain/actors/builtin/paych/v10.go +++ b/chain/actors/builtin/paych/v10.go @@ -12,7 +12,6 @@ import ( paych10 "github.com/filecoin-project/go-state-types/builtin/v10/paych" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v11.go b/chain/actors/builtin/paych/v11.go index 977a013f1f8..477741335be 100644 --- a/chain/actors/builtin/paych/v11.go +++ b/chain/actors/builtin/paych/v11.go @@ -12,7 +12,6 @@ import ( paych11 "github.com/filecoin-project/go-state-types/builtin/v11/paych" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v12.go b/chain/actors/builtin/paych/v12.go index 5c1330d76bc..bfa9476cd93 100644 --- a/chain/actors/builtin/paych/v12.go +++ b/chain/actors/builtin/paych/v12.go @@ -12,7 +12,6 @@ import ( paych12 "github.com/filecoin-project/go-state-types/builtin/v12/paych" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v2.go b/chain/actors/builtin/paych/v2.go index 63a3cc75eb2..b5c6646f6aa 100644 --- a/chain/actors/builtin/paych/v2.go +++ b/chain/actors/builtin/paych/v2.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/paych/v3.go b/chain/actors/builtin/paych/v3.go index c672c0027e6..65be4413e52 100644 --- a/chain/actors/builtin/paych/v3.go +++ b/chain/actors/builtin/paych/v3.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" - adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" + adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/paych/v4.go b/chain/actors/builtin/paych/v4.go index 842e5209389..1a9d5a6b972 100644 --- a/chain/actors/builtin/paych/v4.go +++ b/chain/actors/builtin/paych/v4.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" - adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" + adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/paych/v5.go b/chain/actors/builtin/paych/v5.go index 8f53fe43fab..779bfd2b0a9 100644 --- a/chain/actors/builtin/paych/v5.go +++ b/chain/actors/builtin/paych/v5.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" - adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" + adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/paych/v6.go b/chain/actors/builtin/paych/v6.go index 0a8a9389600..3d7483f480b 100644 --- a/chain/actors/builtin/paych/v6.go +++ b/chain/actors/builtin/paych/v6.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" - adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" + adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/paych/v7.go b/chain/actors/builtin/paych/v7.go index ce0dcba0aaf..2696934374c 100644 --- a/chain/actors/builtin/paych/v7.go +++ b/chain/actors/builtin/paych/v7.go @@ -10,11 +10,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" - adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" + adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/paych/v8.go b/chain/actors/builtin/paych/v8.go index 51067e87b00..34cfa8ecf7c 100644 --- a/chain/actors/builtin/paych/v8.go +++ b/chain/actors/builtin/paych/v8.go @@ -12,7 +12,6 @@ import ( paych8 "github.com/filecoin-project/go-state-types/builtin/v8/paych" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v9.go b/chain/actors/builtin/paych/v9.go index 8a69e16d4f0..f0de23fccd8 100644 --- a/chain/actors/builtin/paych/v9.go +++ b/chain/actors/builtin/paych/v9.go @@ -12,7 +12,6 @@ import ( paych9 "github.com/filecoin-project/go-state-types/builtin/v9/paych" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/power/diff.go b/chain/actors/builtin/power/diff.go index bb2c354ac0c..7baff32186f 100644 --- a/chain/actors/builtin/power/diff.go +++ b/chain/actors/builtin/power/diff.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/power/power.go b/chain/actors/builtin/power/power.go index 9b64ded3877..6ad7d1fb55b 100644 --- a/chain/actors/builtin/power/power.go +++ b/chain/actors/builtin/power/power.go @@ -12,6 +12,10 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -19,11 +23,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/power/v0.go b/chain/actors/builtin/power/v0.go index 4cf550616b2..a01eec679f0 100644 --- a/chain/actors/builtin/power/v0.go +++ b/chain/actors/builtin/power/v0.go @@ -11,12 +11,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/power/v10.go b/chain/actors/builtin/power/v10.go index dd7a9decf0f..7c71914807b 100644 --- a/chain/actors/builtin/power/v10.go +++ b/chain/actors/builtin/power/v10.go @@ -14,7 +14,6 @@ import ( power10 "github.com/filecoin-project/go-state-types/builtin/v10/power" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v11.go b/chain/actors/builtin/power/v11.go index 0ec1e2bdc5f..5a87900832e 100644 --- a/chain/actors/builtin/power/v11.go +++ b/chain/actors/builtin/power/v11.go @@ -14,7 +14,6 @@ import ( power11 "github.com/filecoin-project/go-state-types/builtin/v11/power" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v12.go b/chain/actors/builtin/power/v12.go index 2e910902269..676113113f4 100644 --- a/chain/actors/builtin/power/v12.go +++ b/chain/actors/builtin/power/v12.go @@ -14,7 +14,6 @@ import ( power12 "github.com/filecoin-project/go-state-types/builtin/v12/power" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v2.go b/chain/actors/builtin/power/v2.go index bac0fa1792d..f5ec1100684 100644 --- a/chain/actors/builtin/power/v2.go +++ b/chain/actors/builtin/power/v2.go @@ -11,12 +11,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/power/v3.go b/chain/actors/builtin/power/v3.go index bdb66e3842a..51ba563cc98 100644 --- a/chain/actors/builtin/power/v3.go +++ b/chain/actors/builtin/power/v3.go @@ -11,13 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" - adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" + adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/power/v4.go b/chain/actors/builtin/power/v4.go index b2dc953475f..45c44e9c13a 100644 --- a/chain/actors/builtin/power/v4.go +++ b/chain/actors/builtin/power/v4.go @@ -11,13 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" - adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" + adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/power/v5.go b/chain/actors/builtin/power/v5.go index 3a3adaf32da..40a2281cb06 100644 --- a/chain/actors/builtin/power/v5.go +++ b/chain/actors/builtin/power/v5.go @@ -11,13 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" - power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" - adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" + power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" + adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/power/v6.go b/chain/actors/builtin/power/v6.go index 7e8eb36546c..98ee3ddea85 100644 --- a/chain/actors/builtin/power/v6.go +++ b/chain/actors/builtin/power/v6.go @@ -11,13 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" - adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" + adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/power/v7.go b/chain/actors/builtin/power/v7.go index 893c58667d5..e40677ee7e1 100644 --- a/chain/actors/builtin/power/v7.go +++ b/chain/actors/builtin/power/v7.go @@ -11,13 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" - adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" + adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/power/v8.go b/chain/actors/builtin/power/v8.go index b15fe435590..7f00054004b 100644 --- a/chain/actors/builtin/power/v8.go +++ b/chain/actors/builtin/power/v8.go @@ -14,7 +14,6 @@ import ( power8 "github.com/filecoin-project/go-state-types/builtin/v8/power" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v9.go b/chain/actors/builtin/power/v9.go index 126fbbfbf66..856c59fb9da 100644 --- a/chain/actors/builtin/power/v9.go +++ b/chain/actors/builtin/power/v9.go @@ -14,7 +14,6 @@ import ( power9 "github.com/filecoin-project/go-state-types/builtin/v9/power" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/registry.go b/chain/actors/builtin/registry.go index 6ba5fef03b2..e8a7c0bdece 100644 --- a/chain/actors/builtin/registry.go +++ b/chain/actors/builtin/registry.go @@ -84,7 +84,6 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" rtt "github.com/filecoin-project/go-state-types/rt" - "github.com/filecoin-project/lotus/chain/actors" ) diff --git a/chain/actors/builtin/reward/reward.go b/chain/actors/builtin/reward/reward.go index 3c646364585..e89433500dc 100644 --- a/chain/actors/builtin/reward/reward.go +++ b/chain/actors/builtin/reward/reward.go @@ -9,6 +9,10 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" @@ -17,11 +21,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/reward/v0.go b/chain/actors/builtin/reward/v0.go index 66736f9e009..203bfea2a8e 100644 --- a/chain/actors/builtin/reward/v0.go +++ b/chain/actors/builtin/reward/v0.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" - reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" - smoothing0 "github.com/filecoin-project/specs-actors/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" + reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" + smoothing0 "github.com/filecoin-project/specs-actors/actors/util/smoothing" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/reward/v10.go b/chain/actors/builtin/reward/v10.go index 3ffe9a26749..9ddc696b050 100644 --- a/chain/actors/builtin/reward/v10.go +++ b/chain/actors/builtin/reward/v10.go @@ -11,7 +11,6 @@ import ( reward10 "github.com/filecoin-project/go-state-types/builtin/v10/reward" smoothing10 "github.com/filecoin-project/go-state-types/builtin/v10/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v11.go b/chain/actors/builtin/reward/v11.go index 1d6a5fdf479..35102d8b751 100644 --- a/chain/actors/builtin/reward/v11.go +++ b/chain/actors/builtin/reward/v11.go @@ -11,7 +11,6 @@ import ( reward11 "github.com/filecoin-project/go-state-types/builtin/v11/reward" smoothing11 "github.com/filecoin-project/go-state-types/builtin/v11/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v12.go b/chain/actors/builtin/reward/v12.go index ecc8ff5a0c4..9df853158c3 100644 --- a/chain/actors/builtin/reward/v12.go +++ b/chain/actors/builtin/reward/v12.go @@ -11,7 +11,6 @@ import ( reward12 "github.com/filecoin-project/go-state-types/builtin/v12/reward" smoothing12 "github.com/filecoin-project/go-state-types/builtin/v12/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v2.go b/chain/actors/builtin/reward/v2.go index 6d640f2e970..b4d4d2fed9d 100644 --- a/chain/actors/builtin/reward/v2.go +++ b/chain/actors/builtin/reward/v2.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" - smoothing2 "github.com/filecoin-project/specs-actors/v2/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" + smoothing2 "github.com/filecoin-project/specs-actors/v2/actors/util/smoothing" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/reward/v3.go b/chain/actors/builtin/reward/v3.go index fe8e555c914..c7201596d0a 100644 --- a/chain/actors/builtin/reward/v3.go +++ b/chain/actors/builtin/reward/v3.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" - reward3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/reward" - smoothing3 "github.com/filecoin-project/specs-actors/v3/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" + reward3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/reward" + smoothing3 "github.com/filecoin-project/specs-actors/v3/actors/util/smoothing" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/reward/v4.go b/chain/actors/builtin/reward/v4.go index f0decd39217..295d0b9e568 100644 --- a/chain/actors/builtin/reward/v4.go +++ b/chain/actors/builtin/reward/v4.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/miner" - reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" - smoothing4 "github.com/filecoin-project/specs-actors/v4/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/miner" + reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" + smoothing4 "github.com/filecoin-project/specs-actors/v4/actors/util/smoothing" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/reward/v5.go b/chain/actors/builtin/reward/v5.go index 82be12c0aea..1af85f58f00 100644 --- a/chain/actors/builtin/reward/v5.go +++ b/chain/actors/builtin/reward/v5.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - reward5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/reward" - smoothing5 "github.com/filecoin-project/specs-actors/v5/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + reward5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/reward" + smoothing5 "github.com/filecoin-project/specs-actors/v5/actors/util/smoothing" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/reward/v6.go b/chain/actors/builtin/reward/v6.go index 71884dadaaf..63723ffba55 100644 --- a/chain/actors/builtin/reward/v6.go +++ b/chain/actors/builtin/reward/v6.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" - reward6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/reward" - smoothing6 "github.com/filecoin-project/specs-actors/v6/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" + reward6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/reward" + smoothing6 "github.com/filecoin-project/specs-actors/v6/actors/util/smoothing" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/reward/v7.go b/chain/actors/builtin/reward/v7.go index bc39f366619..21e938fea0c 100644 --- a/chain/actors/builtin/reward/v7.go +++ b/chain/actors/builtin/reward/v7.go @@ -8,13 +8,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - miner7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/miner" - reward7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/reward" - smoothing7 "github.com/filecoin-project/specs-actors/v7/actors/util/smoothing" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" + miner7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/miner" + reward7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/reward" + smoothing7 "github.com/filecoin-project/specs-actors/v7/actors/util/smoothing" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/reward/v8.go b/chain/actors/builtin/reward/v8.go index 5f6b96d29bf..5f46afbd726 100644 --- a/chain/actors/builtin/reward/v8.go +++ b/chain/actors/builtin/reward/v8.go @@ -11,7 +11,6 @@ import ( reward8 "github.com/filecoin-project/go-state-types/builtin/v8/reward" smoothing8 "github.com/filecoin-project/go-state-types/builtin/v8/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v9.go b/chain/actors/builtin/reward/v9.go index 6118e2b3029..1d13374b9c2 100644 --- a/chain/actors/builtin/reward/v9.go +++ b/chain/actors/builtin/reward/v9.go @@ -11,7 +11,6 @@ import ( reward9 "github.com/filecoin-project/go-state-types/builtin/v9/reward" smoothing9 "github.com/filecoin-project/go-state-types/builtin/v9/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/system/system.go b/chain/actors/builtin/system/system.go index 2a2b703bb1c..b3078ee4aa9 100644 --- a/chain/actors/builtin/system/system.go +++ b/chain/actors/builtin/system/system.go @@ -7,6 +7,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -14,10 +17,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/system/v0.go b/chain/actors/builtin/system/v0.go index d5f0f079e27..7a75e77856e 100644 --- a/chain/actors/builtin/system/v0.go +++ b/chain/actors/builtin/system/v0.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system0 "github.com/filecoin-project/specs-actors/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system0 "github.com/filecoin-project/specs-actors/actors/builtin/system" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/system/v10.go b/chain/actors/builtin/system/v10.go index 2cdb3968268..8fa361811eb 100644 --- a/chain/actors/builtin/system/v10.go +++ b/chain/actors/builtin/system/v10.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system10 "github.com/filecoin-project/go-state-types/builtin/v10/system" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v11.go b/chain/actors/builtin/system/v11.go index 9b92afaf095..7f093c6bdf7 100644 --- a/chain/actors/builtin/system/v11.go +++ b/chain/actors/builtin/system/v11.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system11 "github.com/filecoin-project/go-state-types/builtin/v11/system" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v12.go b/chain/actors/builtin/system/v12.go index 71938e799f6..8a47b8a2f78 100644 --- a/chain/actors/builtin/system/v12.go +++ b/chain/actors/builtin/system/v12.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system12 "github.com/filecoin-project/go-state-types/builtin/v12/system" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v2.go b/chain/actors/builtin/system/v2.go index b0c64205418..3f63cdf111c 100644 --- a/chain/actors/builtin/system/v2.go +++ b/chain/actors/builtin/system/v2.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/system" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/system/v3.go b/chain/actors/builtin/system/v3.go index 8334f780e34..ec2a7402923 100644 --- a/chain/actors/builtin/system/v3.go +++ b/chain/actors/builtin/system/v3.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/system" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/system/v4.go b/chain/actors/builtin/system/v4.go index 227104f3784..78744add84d 100644 --- a/chain/actors/builtin/system/v4.go +++ b/chain/actors/builtin/system/v4.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/system" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/system/v5.go b/chain/actors/builtin/system/v5.go index bbfb70b51bd..7ba43440f37 100644 --- a/chain/actors/builtin/system/v5.go +++ b/chain/actors/builtin/system/v5.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/system" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/system/v6.go b/chain/actors/builtin/system/v6.go index 2319b5929a9..a2b0d80face 100644 --- a/chain/actors/builtin/system/v6.go +++ b/chain/actors/builtin/system/v6.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/system" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/system/v7.go b/chain/actors/builtin/system/v7.go index 0b10129e859..c1dc297a219 100644 --- a/chain/actors/builtin/system/v7.go +++ b/chain/actors/builtin/system/v7.go @@ -8,10 +8,9 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - system7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + system7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/system" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/system/v8.go b/chain/actors/builtin/system/v8.go index eca3b0c044d..18c21746d88 100644 --- a/chain/actors/builtin/system/v8.go +++ b/chain/actors/builtin/system/v8.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system8 "github.com/filecoin-project/go-state-types/builtin/v8/system" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v9.go b/chain/actors/builtin/system/v9.go index 55e073efe94..9eee5410862 100644 --- a/chain/actors/builtin/system/v9.go +++ b/chain/actors/builtin/system/v9.go @@ -8,7 +8,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system9 "github.com/filecoin-project/go-state-types/builtin/v9/system" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/util.go b/chain/actors/builtin/verifreg/util.go index 09a7a132cc9..8e34c9f953f 100644 --- a/chain/actors/builtin/verifreg/util.go +++ b/chain/actors/builtin/verifreg/util.go @@ -6,10 +6,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" ) // taking this as a function instead of asking the caller to call it helps reduce some of the error diff --git a/chain/actors/builtin/verifreg/v0.go b/chain/actors/builtin/verifreg/v0.go index 9913c42c0c6..1da1c997db9 100644 --- a/chain/actors/builtin/verifreg/v0.go +++ b/chain/actors/builtin/verifreg/v0.go @@ -11,11 +11,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/verifreg/v10.go b/chain/actors/builtin/verifreg/v10.go index 256f4d2f888..9124b635ed5 100644 --- a/chain/actors/builtin/verifreg/v10.go +++ b/chain/actors/builtin/verifreg/v10.go @@ -15,7 +15,6 @@ import ( verifreg10 "github.com/filecoin-project/go-state-types/builtin/v10/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v11.go b/chain/actors/builtin/verifreg/v11.go index 7b7b9e4c0a8..55d3b014e8c 100644 --- a/chain/actors/builtin/verifreg/v11.go +++ b/chain/actors/builtin/verifreg/v11.go @@ -15,7 +15,6 @@ import ( verifreg11 "github.com/filecoin-project/go-state-types/builtin/v11/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v12.go b/chain/actors/builtin/verifreg/v12.go index 77a113fbe86..8811a5036ef 100644 --- a/chain/actors/builtin/verifreg/v12.go +++ b/chain/actors/builtin/verifreg/v12.go @@ -15,7 +15,6 @@ import ( verifreg12 "github.com/filecoin-project/go-state-types/builtin/v12/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v2.go b/chain/actors/builtin/verifreg/v2.go index 31f7f775df2..122da853fb5 100644 --- a/chain/actors/builtin/verifreg/v2.go +++ b/chain/actors/builtin/verifreg/v2.go @@ -11,11 +11,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" + verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/verifreg/v3.go b/chain/actors/builtin/verifreg/v3.go index 3ea016fd5c3..1626f5b9748 100644 --- a/chain/actors/builtin/verifreg/v3.go +++ b/chain/actors/builtin/verifreg/v3.go @@ -11,12 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" verifreg3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/verifreg" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/verifreg/v4.go b/chain/actors/builtin/verifreg/v4.go index 464cc9fdc20..42a7bc78a84 100644 --- a/chain/actors/builtin/verifreg/v4.go +++ b/chain/actors/builtin/verifreg/v4.go @@ -11,12 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" verifreg4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/verifreg" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/verifreg/v5.go b/chain/actors/builtin/verifreg/v5.go index 17901dd23a9..95f96168ba3 100644 --- a/chain/actors/builtin/verifreg/v5.go +++ b/chain/actors/builtin/verifreg/v5.go @@ -11,12 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" verifreg5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/verifreg" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/verifreg/v6.go b/chain/actors/builtin/verifreg/v6.go index 68fac64cb4d..13c237b62f5 100644 --- a/chain/actors/builtin/verifreg/v6.go +++ b/chain/actors/builtin/verifreg/v6.go @@ -11,12 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" verifreg6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/verifreg" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/verifreg/v7.go b/chain/actors/builtin/verifreg/v7.go index e8f3ac73984..0e4ca55de8f 100644 --- a/chain/actors/builtin/verifreg/v7.go +++ b/chain/actors/builtin/verifreg/v7.go @@ -11,12 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" verifreg7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/verifreg/v8.go b/chain/actors/builtin/verifreg/v8.go index 89393c4d9cf..c55a5873932 100644 --- a/chain/actors/builtin/verifreg/v8.go +++ b/chain/actors/builtin/verifreg/v8.go @@ -14,7 +14,6 @@ import ( verifreg8 "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v9.go b/chain/actors/builtin/verifreg/v9.go index ce63c7f94b4..6c345223095 100644 --- a/chain/actors/builtin/verifreg/v9.go +++ b/chain/actors/builtin/verifreg/v9.go @@ -14,7 +14,6 @@ import ( adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/verifreg.go b/chain/actors/builtin/verifreg/verifreg.go index de906f52127..537894c24b6 100644 --- a/chain/actors/builtin/verifreg/verifreg.go +++ b/chain/actors/builtin/verifreg/verifreg.go @@ -11,6 +11,9 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -18,10 +21,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/manifest.go b/chain/actors/manifest.go index 62c17193a57..f4949c4e07d 100644 --- a/chain/actors/manifest.go +++ b/chain/actors/manifest.go @@ -11,7 +11,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/params.go b/chain/actors/params.go index 866c72b99df..29428b10f66 100644 --- a/chain/actors/params.go +++ b/chain/actors/params.go @@ -6,7 +6,6 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/beacon/beacon.go b/chain/beacon/beacon.go index aa76bcffeb6..33b18576f60 100644 --- a/chain/beacon/beacon.go +++ b/chain/beacon/beacon.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/beacon/drand/drand.go b/chain/beacon/drand/drand.go index 5825fa69109..8c4506eed8c 100644 --- a/chain/beacon/drand/drand.go +++ b/chain/beacon/drand/drand.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/beacon/drand/drand_test.go b/chain/beacon/drand/drand_test.go index 7434241a53a..a5efdd771f0 100644 --- a/chain/beacon/drand/drand_test.go +++ b/chain/beacon/drand/drand_test.go @@ -12,7 +12,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" ) diff --git a/chain/beacon/mock.go b/chain/beacon/mock.go index 3f26da1097d..b3ce766f4b5 100644 --- a/chain/beacon/mock.go +++ b/chain/beacon/mock.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/consensus/common.go b/chain/consensus/common.go index a7e5c40d2b9..a1550d43529 100644 --- a/chain/consensus/common.go +++ b/chain/consensus/common.go @@ -19,8 +19,6 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -32,6 +30,7 @@ import ( "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/lib/async" "github.com/filecoin-project/lotus/metrics" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) // Common operations shared by all consensus algorithm implementations. diff --git a/chain/consensus/compute_state.go b/chain/consensus/compute_state.go index 4b993b3e72d..b2f971bd351 100644 --- a/chain/consensus/compute_state.go +++ b/chain/consensus/compute_state.go @@ -16,15 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" - exported0 "github.com/filecoin-project/specs-actors/actors/builtin/exported" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - exported2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/exported" - exported3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/exported" - exported4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/exported" - exported5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/exported" - exported6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/exported" - exported7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/exported" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -36,6 +27,14 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/metrics" + exported0 "github.com/filecoin-project/specs-actors/actors/builtin/exported" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + exported2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/exported" + exported3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/exported" + exported4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/exported" + exported5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/exported" + exported6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/exported" + exported7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/exported" ) func NewActorRegistry() *vm.ActorRegistry { diff --git a/chain/consensus/filcns/filecoin.go b/chain/consensus/filcns/filecoin.go index bb70d5d11c1..8dc06b2a463 100644 --- a/chain/consensus/filcns/filecoin.go +++ b/chain/consensus/filcns/filecoin.go @@ -16,8 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" @@ -36,6 +34,7 @@ import ( "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) var log = logging.Logger("fil-consensus") diff --git a/chain/consensus/filcns/upgrades.go b/chain/consensus/filcns/upgrades.go index 91b4bc98ba5..994296ec0f2 100644 --- a/chain/consensus/filcns/upgrades.go +++ b/chain/consensus/filcns/upgrades.go @@ -30,6 +30,18 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/rt" gstStore "github.com/filecoin-project/go-state-types/store" + "github.com/filecoin-project/lotus/blockstore" + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" + "github.com/filecoin-project/lotus/chain/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/state" + "github.com/filecoin-project/lotus/chain/stmgr" + "github.com/filecoin-project/lotus/chain/store" + "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/chain/vm" + "github.com/filecoin-project/lotus/node/bundle" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" multisig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" @@ -44,19 +56,6 @@ import ( "github.com/filecoin-project/specs-actors/v6/actors/migration/nv14" "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" - - "github.com/filecoin-project/lotus/blockstore" - "github.com/filecoin-project/lotus/build" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/state" - "github.com/filecoin-project/lotus/chain/stmgr" - "github.com/filecoin-project/lotus/chain/store" - "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/lotus/chain/vm" - "github.com/filecoin-project/lotus/node/bundle" ) //go:embed FVMLiftoff.txt diff --git a/chain/consensus/filcns/weight.go b/chain/consensus/filcns/weight.go index ab90840c5f8..734ed098127 100644 --- a/chain/consensus/filcns/weight.go +++ b/chain/consensus/filcns/weight.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" big2 "github.com/filecoin-project/go-state-types/big" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/power" diff --git a/chain/consensus/iface.go b/chain/consensus/iface.go index 9449cb5a47a..7db50539f95 100644 --- a/chain/consensus/iface.go +++ b/chain/consensus/iface.go @@ -7,7 +7,6 @@ import ( "go.opencensus.io/stats" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/reward" diff --git a/chain/consensus/signatures.go b/chain/consensus/signatures.go index cb0e229a85b..bb2f518aca7 100644 --- a/chain/consensus/signatures.go +++ b/chain/consensus/signatures.go @@ -6,7 +6,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/chain/events/cache.go b/chain/events/cache.go index 2eba1f085b7..8fd868bd15d 100644 --- a/chain/events/cache.go +++ b/chain/events/cache.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events.go b/chain/events/events.go index c68b62a64e0..103e5749787 100644 --- a/chain/events/events.go +++ b/chain/events/events.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/events/events_called.go b/chain/events/events_called.go index 3ac02b2f7fd..58199b9b032 100644 --- a/chain/events/events_called.go +++ b/chain/events/events_called.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events_height.go b/chain/events/events_height.go index 5789be753a9..84bc6e3499e 100644 --- a/chain/events/events_height.go +++ b/chain/events/events_height.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events_test.go b/chain/events/events_test.go index f16434355ee..b8c57a60c1a 100644 --- a/chain/events/events_test.go +++ b/chain/events/events_test.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/chain/events/filter/event.go b/chain/events/filter/event.go index 24192a53e72..a5657eb402c 100644 --- a/chain/events/filter/event.go +++ b/chain/events/filter/event.go @@ -14,10 +14,9 @@ import ( "github.com/filecoin-project/go-address" amt4 "github.com/filecoin-project/go-amt-ipld/v4" "github.com/filecoin-project/go-state-types/abi" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - cstore "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) func isIndexedValue(b uint8) bool { diff --git a/chain/events/filter/event_test.go b/chain/events/filter/event_test.go index 329573bc13d..833d1b1a951 100644 --- a/chain/events/filter/event_test.go +++ b/chain/events/filter/event_test.go @@ -15,11 +15,10 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) func TestEventFilterCollectEvents(t *testing.T) { diff --git a/chain/events/filter/index.go b/chain/events/filter/index.go index 2b1890c739e..81c629303a4 100644 --- a/chain/events/filter/index.go +++ b/chain/events/filter/index.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/filter/index_test.go b/chain/events/filter/index_test.go index f9b1b14adae..251d1d9428b 100644 --- a/chain/events/filter/index_test.go +++ b/chain/events/filter/index_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/observer.go b/chain/events/observer.go index 4462185858f..66654571375 100644 --- a/chain/events/observer.go +++ b/chain/events/observer.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/chain/events/state/fastapi.go b/chain/events/state/fastapi.go index 9375d9d7846..df2b7c9daab 100644 --- a/chain/events/state/fastapi.go +++ b/chain/events/state/fastapi.go @@ -4,7 +4,6 @@ import ( "context" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/mock/api.go b/chain/events/state/mock/api.go index cdec4265922..1cbb800c8a7 100644 --- a/chain/events/state/mock/api.go +++ b/chain/events/state/mock/api.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/mock/tipset.go b/chain/events/state/mock/tipset.go index 0d25b8790eb..a57480a5199 100644 --- a/chain/events/state/mock/tipset.go +++ b/chain/events/state/mock/tipset.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/predicates.go b/chain/events/state/predicates.go index ff05156a6f9..c186a746474 100644 --- a/chain/events/state/predicates.go +++ b/chain/events/state/predicates.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/events/state/predicates_test.go b/chain/events/state/predicates_test.go index 52fc2668a2b..d123a9a743b 100644 --- a/chain/events/state/predicates_test.go +++ b/chain/events/state/predicates_test.go @@ -13,17 +13,16 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - tutils "github.com/filecoin-project/specs-actors/v5/support/testing" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" test "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + tutils "github.com/filecoin-project/specs-actors/v5/support/testing" ) var dummyCid cid.Cid diff --git a/chain/events/tscache.go b/chain/events/tscache.go index ed19a5f4175..c5558426bf4 100644 --- a/chain/events/tscache.go +++ b/chain/events/tscache.go @@ -7,7 +7,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/tscache_test.go b/chain/events/tscache_test.go index ec312740b7e..65b58794260 100644 --- a/chain/events/tscache_test.go +++ b/chain/events/tscache_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/exchange/client.go b/chain/exchange/client.go index 120b554a11f..581861e9dc3 100644 --- a/chain/exchange/client.go +++ b/chain/exchange/client.go @@ -15,7 +15,6 @@ import ( "golang.org/x/xerrors" cborutil "github.com/filecoin-project/go-cbor-util" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/exchange/server.go b/chain/exchange/server.go index 03dcf0ed79f..d9c4d40d8dd 100644 --- a/chain/exchange/server.go +++ b/chain/exchange/server.go @@ -12,7 +12,6 @@ import ( "golang.org/x/xerrors" cborutil "github.com/filecoin-project/go-cbor-util" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/gen/gen.go b/chain/gen/gen.go index 9f8d0834d6e..52ebc90f121 100644 --- a/chain/gen/gen.go +++ b/chain/gen/gen.go @@ -24,8 +24,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -47,6 +45,7 @@ import ( "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) const msgsPerBlock = 20 diff --git a/chain/gen/gen_test.go b/chain/gen/gen_test.go index d04e55a265a..f2f1fec0cf5 100644 --- a/chain/gen/gen_test.go +++ b/chain/gen/gen_test.go @@ -5,7 +5,6 @@ import ( "testing" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/policy" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" diff --git a/chain/gen/genesis/f00_system.go b/chain/gen/genesis/f00_system.go index 5c6ecacbf92..1bedb662784 100644 --- a/chain/gen/genesis/f00_system.go +++ b/chain/gen/genesis/f00_system.go @@ -10,7 +10,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f01_init.go b/chain/gen/genesis/f01_init.go index 706328d2194..bab6dc75e67 100644 --- a/chain/gen/genesis/f01_init.go +++ b/chain/gen/genesis/f01_init.go @@ -14,13 +14,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/genesis" + "github.com/filecoin-project/specs-actors/actors/util/adt" ) func SetupInitActor(ctx context.Context, bs bstore.Blockstore, netname string, initialActors []genesis.Actor, rootVerifier genesis.Actor, remainder genesis.Actor, av actorstypes.Version) (int64, *types.Actor, map[address.Address]address.Address, error) { diff --git a/chain/gen/genesis/f02_reward.go b/chain/gen/genesis/f02_reward.go index db32517f9cf..6c9f70f8f29 100644 --- a/chain/gen/genesis/f02_reward.go +++ b/chain/gen/genesis/f02_reward.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/gen/genesis/f03_cron.go b/chain/gen/genesis/f03_cron.go index 4c377b19156..a4c4f5fb1b4 100644 --- a/chain/gen/genesis/f03_cron.go +++ b/chain/gen/genesis/f03_cron.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f04_power.go b/chain/gen/genesis/f04_power.go index 385cc97d25e..791743d677f 100644 --- a/chain/gen/genesis/f04_power.go +++ b/chain/gen/genesis/f04_power.go @@ -9,12 +9,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/actors/util/adt" ) func SetupStoragePowerActor(ctx context.Context, bs bstore.Blockstore, av actorstypes.Version) (*types.Actor, error) { diff --git a/chain/gen/genesis/f05_market.go b/chain/gen/genesis/f05_market.go index 59c61a3ae1d..3f623082949 100644 --- a/chain/gen/genesis/f05_market.go +++ b/chain/gen/genesis/f05_market.go @@ -9,7 +9,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f06_vreg.go b/chain/gen/genesis/f06_vreg.go index ffddc814f1c..5b456674fea 100644 --- a/chain/gen/genesis/f06_vreg.go +++ b/chain/gen/genesis/f06_vreg.go @@ -10,12 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/actors/util/adt" ) var RootVerifierID address.Address diff --git a/chain/gen/genesis/f07_dcap.go b/chain/gen/genesis/f07_dcap.go index 6d8e3258ebd..dcfb76c8552 100644 --- a/chain/gen/genesis/f07_dcap.go +++ b/chain/gen/genesis/f07_dcap.go @@ -11,12 +11,11 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/datacap" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/actors/util/adt" ) var GovernorId address.Address diff --git a/chain/gen/genesis/genesis.go b/chain/gen/genesis/genesis.go index 8ec657479f7..d4f464221d9 100644 --- a/chain/gen/genesis/genesis.go +++ b/chain/gen/genesis/genesis.go @@ -19,10 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -47,6 +43,9 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/node/bundle" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) const AccountStart = 100 diff --git a/chain/gen/genesis/genesis_eth.go b/chain/gen/genesis/genesis_eth.go index d5aa2f0b51b..347caa4fffc 100644 --- a/chain/gen/genesis/genesis_eth.go +++ b/chain/gen/genesis/genesis_eth.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" diff --git a/chain/gen/genesis/miners.go b/chain/gen/genesis/miners.go index df8900cab8c..bad96f3bc77 100644 --- a/chain/gen/genesis/miners.go +++ b/chain/gen/genesis/miners.go @@ -24,17 +24,6 @@ import ( smoothing9 "github.com/filecoin-project/go-state-types/builtin/v9/util/smoothing" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" - power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" - reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" - power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" - reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" - "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -51,6 +40,16 @@ import ( "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" "github.com/filecoin-project/lotus/lib/sigs" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" + power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" + reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" + power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" + reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" ) func MinerAddress(genesisIndex uint64) address.Address { diff --git a/chain/gen/genesis/util.go b/chain/gen/genesis/util.go index beca1183c02..a34a43d9aa7 100644 --- a/chain/gen/genesis/util.go +++ b/chain/gen/genesis/util.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" diff --git a/chain/gen/slashfilter/slashfilter.go b/chain/gen/slashfilter/slashfilter.go index 71b5dad9ad9..ade6442b9ae 100644 --- a/chain/gen/slashfilter/slashfilter.go +++ b/chain/gen/slashfilter/slashfilter.go @@ -10,7 +10,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/gen/slashfilter/slashsvc/slashservice.go b/chain/gen/slashfilter/slashsvc/slashservice.go index 7a662288098..02567de9a1a 100644 --- a/chain/gen/slashfilter/slashsvc/slashservice.go +++ b/chain/gen/slashfilter/slashsvc/slashservice.go @@ -13,12 +13,11 @@ import ( "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/specs-actors/actors/builtin/miner" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/gen/slashfilter" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/actors/builtin/miner" ) var log = logging.Logger("slashsvc") diff --git a/chain/index/msgindex.go b/chain/index/msgindex.go index e9e81ae2cf5..9dd6735a89d 100644 --- a/chain/index/msgindex.go +++ b/chain/index/msgindex.go @@ -16,7 +16,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/index/msgindex_test.go b/chain/index/msgindex_test.go index bf4bc6190e8..fa8844e4a90 100644 --- a/chain/index/msgindex_test.go +++ b/chain/index/msgindex_test.go @@ -13,7 +13,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" diff --git a/chain/market/fundmanager.go b/chain/market/fundmanager.go index fab71dfefe1..47e8f0bb64e 100644 --- a/chain/market/fundmanager.go +++ b/chain/market/fundmanager.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/market/fundmanager_test.go b/chain/market/fundmanager_test.go index d79afbc513b..6511574ad03 100644 --- a/chain/market/fundmanager_test.go +++ b/chain/market/fundmanager_test.go @@ -16,12 +16,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) // TestFundManagerBasic verifies that the basic fund manager operations work diff --git a/chain/market/store.go b/chain/market/store.go index 10ab2abe1a3..1950b0239a0 100644 --- a/chain/market/store.go +++ b/chain/market/store.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/chain/messagepool/check.go b/chain/messagepool/check.go index fdec910c4ea..c5208bc5755 100644 --- a/chain/messagepool/check.go +++ b/chain/messagepool/check.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" diff --git a/chain/messagepool/gasguess/guessgas.go b/chain/messagepool/gasguess/guessgas.go index f502e84a6e4..af0a352364c 100644 --- a/chain/messagepool/gasguess/guessgas.go +++ b/chain/messagepool/gasguess/guessgas.go @@ -8,11 +8,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) type ActorLookup func(context.Context, address.Address, types.TipSetKey) (*types.Actor, error) diff --git a/chain/messagepool/messagepool.go b/chain/messagepool/messagepool.go index 7d55b0b16f5..36f949220a3 100644 --- a/chain/messagepool/messagepool.go +++ b/chain/messagepool/messagepool.go @@ -30,8 +30,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - lps "github.com/filecoin-project/pubsub" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus" @@ -42,6 +40,7 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/modules/dtypes" + lps "github.com/filecoin-project/pubsub" ) var log = logging.Logger("messagepool") diff --git a/chain/messagepool/messagepool_test.go b/chain/messagepool/messagepool_test.go index 3c6800d7b2b..35798563a9b 100644 --- a/chain/messagepool/messagepool_test.go +++ b/chain/messagepool/messagepool_test.go @@ -18,8 +18,6 @@ import ( big2 "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -29,6 +27,7 @@ import ( "github.com/filecoin-project/lotus/chain/wallet" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func init() { diff --git a/chain/messagepool/provider.go b/chain/messagepool/provider.go index 764e6c13a92..b7dd845a690 100644 --- a/chain/messagepool/provider.go +++ b/chain/messagepool/provider.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/pruning.go b/chain/messagepool/pruning.go index 24d7fee566d..67629f4c3a6 100644 --- a/chain/messagepool/pruning.go +++ b/chain/messagepool/pruning.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/messagepool/repub.go b/chain/messagepool/repub.go index a87d5e08a84..8d30d3a22db 100644 --- a/chain/messagepool/repub.go +++ b/chain/messagepool/repub.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/repub_test.go b/chain/messagepool/repub_test.go index 9cdabc02fac..d2d34da5a14 100644 --- a/chain/messagepool/repub_test.go +++ b/chain/messagepool/repub_test.go @@ -8,12 +8,11 @@ import ( "github.com/ipfs/go-datastore" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func TestRepubMessages(t *testing.T) { diff --git a/chain/messagepool/selection.go b/chain/messagepool/selection.go index 163bd76f985..9e9e58df1fd 100644 --- a/chain/messagepool/selection.go +++ b/chain/messagepool/selection.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" tbig "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/selection_test.go b/chain/messagepool/selection_test.go index 17e0f34f4e0..836e23ef2b5 100644 --- a/chain/messagepool/selection_test.go +++ b/chain/messagepool/selection_test.go @@ -22,8 +22,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -34,6 +32,7 @@ import ( _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/delegated" _ "github.com/filecoin-project/lotus/lib/sigs/secp" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func init() { diff --git a/chain/messagesigner/messagesigner.go b/chain/messagesigner/messagesigner.go index cd31a3b739e..860aeb6e35c 100644 --- a/chain/messagesigner/messagesigner.go +++ b/chain/messagesigner/messagesigner.go @@ -13,7 +13,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagesigner/messagesigner_test.go b/chain/messagesigner/messagesigner_test.go index 637f17b46cd..4979b327f19 100644 --- a/chain/messagesigner/messagesigner_test.go +++ b/chain/messagesigner/messagesigner_test.go @@ -12,7 +12,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" diff --git a/chain/rand/rand.go b/chain/rand/rand.go index 40f9f593a03..3a4b454c45b 100644 --- a/chain/rand/rand.go +++ b/chain/rand/rand.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/rand/rand_test.go b/chain/rand/rand_test.go index e2e7221658b..e96269632a4 100644 --- a/chain/rand/rand_test.go +++ b/chain/rand/rand_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" diff --git a/chain/state/statetree.go b/chain/state/statetree.go index 61d7d500a87..1fa573abb30 100644 --- a/chain/state/statetree.go +++ b/chain/state/statetree.go @@ -16,15 +16,14 @@ import ( "github.com/filecoin-project/go-state-types/abi" builtin_types "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/actors/adt" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" states0 "github.com/filecoin-project/specs-actors/actors/states" states2 "github.com/filecoin-project/specs-actors/v2/actors/states" states3 "github.com/filecoin-project/specs-actors/v3/actors/states" states4 "github.com/filecoin-project/specs-actors/v4/actors/states" states5 "github.com/filecoin-project/specs-actors/v5/actors/states" - - "github.com/filecoin-project/lotus/chain/actors/adt" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" ) var log = logging.Logger("statetree") diff --git a/chain/state/statetree_test.go b/chain/state/statetree_test.go index 9a221751a75..06a47d8ea67 100644 --- a/chain/state/statetree_test.go +++ b/chain/state/statetree_test.go @@ -11,10 +11,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/network" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func BenchmarkStateTreeSet(b *testing.B) { diff --git a/chain/stmgr/actors.go b/chain/stmgr/actors.go index 56744fa7489..53e2eba5a5f 100644 --- a/chain/stmgr/actors.go +++ b/chain/stmgr/actors.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/chain/stmgr/call.go b/chain/stmgr/call.go index 61056528f11..4c4a6e6bcc3 100644 --- a/chain/stmgr/call.go +++ b/chain/stmgr/call.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/chain/stmgr/forks.go b/chain/stmgr/forks.go index 9dd66ee8bcc..b3a6bd718f0 100644 --- a/chain/stmgr/forks.go +++ b/chain/stmgr/forks.go @@ -19,8 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -28,6 +26,7 @@ import ( "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" + "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" ) // EnvDisablePreMigrations when set to '1' stops pre-migrations from running diff --git a/chain/stmgr/forks_test.go b/chain/stmgr/forks_test.go index bf8793488b6..e55497f902d 100644 --- a/chain/stmgr/forks_test.go +++ b/chain/stmgr/forks_test.go @@ -23,10 +23,6 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" @@ -43,6 +39,9 @@ import ( "github.com/filecoin-project/lotus/chain/vm" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" ) func init() { diff --git a/chain/stmgr/read.go b/chain/stmgr/read.go index 4543f63b34b..1a66129da9d 100644 --- a/chain/stmgr/read.go +++ b/chain/stmgr/read.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/stmgr/rpc/rpcstatemanager.go b/chain/stmgr/rpc/rpcstatemanager.go index 9186501eab9..be17cfd9e0b 100644 --- a/chain/stmgr/rpc/rpcstatemanager.go +++ b/chain/stmgr/rpc/rpcstatemanager.go @@ -7,7 +7,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/stmgr/searchwait.go b/chain/stmgr/searchwait.go index 356ace23c45..03e767167a8 100644 --- a/chain/stmgr/searchwait.go +++ b/chain/stmgr/searchwait.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/stmgr/searchwait_test.go b/chain/stmgr/searchwait_test.go index b23b22376c6..24876a76290 100644 --- a/chain/stmgr/searchwait_test.go +++ b/chain/stmgr/searchwait_test.go @@ -6,7 +6,6 @@ import ( "testing" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/gen" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" diff --git a/chain/stmgr/stmgr.go b/chain/stmgr/stmgr.go index 49913e442d1..c3129bba94f 100644 --- a/chain/stmgr/stmgr.go +++ b/chain/stmgr/stmgr.go @@ -19,8 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" @@ -34,6 +32,7 @@ import ( "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" + "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" // Used for genesis. msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" diff --git a/chain/stmgr/supply.go b/chain/stmgr/supply.go index 9486cb93622..2ce5d4e37a0 100644 --- a/chain/stmgr/supply.go +++ b/chain/stmgr/supply.go @@ -10,8 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" @@ -25,6 +23,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" + msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" ) // sets up information about the vesting schedule diff --git a/chain/stmgr/utils.go b/chain/stmgr/utils.go index 5e3bbd2788b..db84d0825c7 100644 --- a/chain/stmgr/utils.go +++ b/chain/stmgr/utils.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" gstStore "github.com/filecoin-project/go-state-types/store" - "github.com/filecoin-project/lotus/api" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/actors/builtin/system" diff --git a/chain/store/basefee.go b/chain/store/basefee.go index 3b6af5c0716..cccfc04fcd5 100644 --- a/chain/store/basefee.go +++ b/chain/store/basefee.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/store/index.go b/chain/store/index.go index 8361f4db9eb..072e74ef8ed 100644 --- a/chain/store/index.go +++ b/chain/store/index.go @@ -10,7 +10,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/shardedmutex" ) diff --git a/chain/store/index_test.go b/chain/store/index_test.go index a3a4ad6ce7e..78822334a3c 100644 --- a/chain/store/index_test.go +++ b/chain/store/index_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" diff --git a/chain/store/messages.go b/chain/store/messages.go index c23f900d7cb..1074448cebb 100644 --- a/chain/store/messages.go +++ b/chain/store/messages.go @@ -11,12 +11,11 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) type storable interface { diff --git a/chain/store/snapshot.go b/chain/store/snapshot.go index 301a5f87bfd..d79fd0d197b 100644 --- a/chain/store/snapshot.go +++ b/chain/store/snapshot.go @@ -22,7 +22,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/store/store.go b/chain/store/store.go index f2826fc2ff0..f5bea1549f9 100644 --- a/chain/store/store.go +++ b/chain/store/store.go @@ -26,8 +26,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/pubsub" - "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -35,6 +33,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/metrics" + "github.com/filecoin-project/pubsub" ) var log = logging.Logger("chainstore") diff --git a/chain/store/store_test.go b/chain/store/store_test.go index 9c717fdbef4..a98c97afb3d 100644 --- a/chain/store/store_test.go +++ b/chain/store/store_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/chain/sub/incoming.go b/chain/sub/incoming.go index b50ddc46779..43bed7f00e0 100644 --- a/chain/sub/incoming.go +++ b/chain/sub/incoming.go @@ -21,7 +21,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/chain/sub/incoming_test.go b/chain/sub/incoming_test.go index d8ee99b7f12..26a424a48ab 100644 --- a/chain/sub/incoming_test.go +++ b/chain/sub/incoming_test.go @@ -16,7 +16,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/sync.go b/chain/sync.go index 1b9a302f75d..04fb8605c04 100644 --- a/chain/sync.go +++ b/chain/sync.go @@ -24,15 +24,12 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/pubsub" - "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/node/modules/dtypes" + "github.com/filecoin-project/pubsub" // named msgarray here to make it clear that these are the types used by // messages, regardless of specs-actors version. - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -43,6 +40,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/metrics" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) var ( diff --git a/chain/sync_manager.go b/chain/sync_manager.go index 3369c3b5a0d..621c8c2025a 100644 --- a/chain/sync_manager.go +++ b/chain/sync_manager.go @@ -12,7 +12,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/sync_test.go b/chain/sync_test.go index be775960319..e8abb413e5c 100644 --- a/chain/sync_test.go +++ b/chain/sync_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/chain/syncstate.go b/chain/syncstate.go index 527d6be4832..a18ef4ca128 100644 --- a/chain/syncstate.go +++ b/chain/syncstate.go @@ -5,7 +5,6 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/types/bigint.go b/chain/types/bigint.go index 72ef5212862..e7850fa9262 100644 --- a/chain/types/bigint.go +++ b/chain/types/bigint.go @@ -5,7 +5,6 @@ import ( "math/big" big2 "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" ) diff --git a/chain/types/ethtypes/eth_transactions.go b/chain/types/ethtypes/eth_transactions.go index 6c13c5bf6a2..24c8397708d 100644 --- a/chain/types/ethtypes/eth_transactions.go +++ b/chain/types/ethtypes/eth_transactions.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" typescrypto "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/types/ethtypes/eth_transactions_test.go b/chain/types/ethtypes/eth_transactions_test.go index 68abc55dd49..7b4be0b28c0 100644 --- a/chain/types/ethtypes/eth_transactions_test.go +++ b/chain/types/ethtypes/eth_transactions_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/evm" init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" crypto1 "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/lib/sigs" _ "github.com/filecoin-project/lotus/lib/sigs/delegated" diff --git a/chain/types/ethtypes/eth_types.go b/chain/types/ethtypes/eth_types.go index b796e6f56f2..96271435b56 100644 --- a/chain/types/ethtypes/eth_types.go +++ b/chain/types/ethtypes/eth_types.go @@ -21,7 +21,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/lib/must" ) diff --git a/chain/types/message.go b/chain/types/message.go index 473289ead45..8a00ecf318a 100644 --- a/chain/types/message.go +++ b/chain/types/message.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" ) diff --git a/chain/types/mock/chain.go b/chain/types/mock/chain.go index dcbcd85362c..e6d14d9d958 100644 --- a/chain/types/mock/chain.go +++ b/chain/types/mock/chain.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/types/vmcontext.go b/chain/types/vmcontext.go index bab9c213fa6..4b4e8892429 100644 --- a/chain/types/vmcontext.go +++ b/chain/types/vmcontext.go @@ -5,7 +5,6 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/types_test.go b/chain/types_test.go index 0fb3992146e..e7264f18f7d 100644 --- a/chain/types_test.go +++ b/chain/types_test.go @@ -7,7 +7,6 @@ import ( "testing" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/vectors/gen/main.go b/chain/vectors/gen/main.go index f4b7c82da02..308bed39c1e 100644 --- a/chain/vectors/gen/main.go +++ b/chain/vectors/gen/main.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/vectors/vector_types.go b/chain/vectors/vector_types.go index 3251fde38e8..7e014fb77cc 100644 --- a/chain/vectors/vector_types.go +++ b/chain/vectors/vector_types.go @@ -2,7 +2,6 @@ package vectors import ( "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/vm/fvm.go b/chain/vm/fvm.go index c8c02dddd40..c864d00536f 100644 --- a/chain/vm/fvm.go +++ b/chain/vm/fvm.go @@ -25,7 +25,6 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/vm/gas.go b/chain/vm/gas.go index cb0c5def94d..d6d7df45c67 100644 --- a/chain/vm/gas.go +++ b/chain/vm/gas.go @@ -9,10 +9,9 @@ import ( addr "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/build" vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - - "github.com/filecoin-project/lotus/build" ) type GasCharge struct { diff --git a/chain/vm/gas_v0.go b/chain/vm/gas_v0.go index 7a144fc261e..96dd8ee4478 100644 --- a/chain/vm/gas_v0.go +++ b/chain/vm/gas_v0.go @@ -6,9 +6,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/chain/actors/builtin" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) type scalingCost struct { diff --git a/chain/vm/invoker.go b/chain/vm/invoker.go index cea17f61dba..2baaab4c6fd 100644 --- a/chain/vm/invoker.go +++ b/chain/vm/invoker.go @@ -15,12 +15,11 @@ import ( builtinst "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" + vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" ) type MethodMeta struct { diff --git a/chain/vm/invoker_test.go b/chain/vm/invoker_test.go index d3e6dcd7fd4..2ee7b384b67 100644 --- a/chain/vm/invoker_test.go +++ b/chain/vm/invoker_test.go @@ -18,11 +18,10 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/rt" - runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" + runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" ) type basicContract struct{} diff --git a/chain/vm/mkactor.go b/chain/vm/mkactor.go index b33085c0594..e91d91ca011 100644 --- a/chain/vm/mkactor.go +++ b/chain/vm/mkactor.go @@ -11,6 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/aerrors" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -18,13 +24,6 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" - - "github.com/filecoin-project/lotus/build" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/aerrors" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/types" ) func init() { diff --git a/chain/vm/runtime.go b/chain/vm/runtime.go index 355fcea2b09..c18944473a7 100644 --- a/chain/vm/runtime.go +++ b/chain/vm/runtime.go @@ -21,14 +21,6 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" - rt0 "github.com/filecoin-project/specs-actors/actors/runtime" - rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" - rt3 "github.com/filecoin-project/specs-actors/v3/actors/runtime" - rt4 "github.com/filecoin-project/specs-actors/v4/actors/runtime" - rt5 "github.com/filecoin-project/specs-actors/v5/actors/runtime" - rt6 "github.com/filecoin-project/specs-actors/v6/actors/runtime" - rt7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" @@ -36,6 +28,13 @@ import ( "github.com/filecoin-project/lotus/chain/rand" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" + rt0 "github.com/filecoin-project/specs-actors/actors/runtime" + rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" + rt3 "github.com/filecoin-project/specs-actors/v3/actors/runtime" + rt4 "github.com/filecoin-project/specs-actors/v4/actors/runtime" + rt5 "github.com/filecoin-project/specs-actors/v5/actors/runtime" + rt6 "github.com/filecoin-project/specs-actors/v6/actors/runtime" + rt7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" ) type Message struct { diff --git a/chain/vm/runtime_test.go b/chain/vm/runtime_test.go index 88b7366dea0..563cedc26c6 100644 --- a/chain/vm/runtime_test.go +++ b/chain/vm/runtime_test.go @@ -10,7 +10,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/vm/syscalls.go b/chain/vm/syscalls.go index 83a07ca2dae..7a819ad1806 100644 --- a/chain/vm/syscalls.go +++ b/chain/vm/syscalls.go @@ -17,9 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -29,6 +26,8 @@ import ( "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" + runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func init() { diff --git a/chain/vm/vm.go b/chain/vm/vm.go index ba404ab1fa3..9e1799a7fc5 100644 --- a/chain/vm/vm.go +++ b/chain/vm/vm.go @@ -23,7 +23,6 @@ import ( builtin_types "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/vm/vmi.go b/chain/vm/vmi.go index 042621ca2d4..e604a0895dd 100644 --- a/chain/vm/vmi.go +++ b/chain/vm/vmi.go @@ -8,7 +8,6 @@ import ( cid "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/wallet/key/key.go b/chain/wallet/key/key.go index 4220666108e..19c8c2027bc 100644 --- a/chain/wallet/key/key.go +++ b/chain/wallet/key/key.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/chain/wallet/ledger/ledger.go b/chain/wallet/ledger/ledger.go index 5279389de81..31de0f882fb 100644 --- a/chain/wallet/ledger/ledger.go +++ b/chain/wallet/ledger/ledger.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/wallet/multi.go b/chain/wallet/multi.go index 91d2714772f..cb6fd730e2e 100644 --- a/chain/wallet/multi.go +++ b/chain/wallet/multi.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ledgerwallet "github.com/filecoin-project/lotus/chain/wallet/ledger" diff --git a/chain/wallet/wallet.go b/chain/wallet/wallet.go index 76af663c780..ea20e860329 100644 --- a/chain/wallet/wallet.go +++ b/chain/wallet/wallet.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/cli/auth.go b/cli/auth.go index caea4cb424b..bac1e0d1efd 100644 --- a/cli/auth.go +++ b/cli/auth.go @@ -7,7 +7,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cli/backup.go b/cli/backup.go index d2d8f25ff89..6ddb03c2434 100644 --- a/cli/backup.go +++ b/cli/backup.go @@ -11,7 +11,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cli/chain.go b/cli/chain.go index c0d54fd6382..0653516c5ad 100644 --- a/cli/chain.go +++ b/cli/chain.go @@ -26,13 +26,6 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/actors/builtin" - "github.com/filecoin-project/specs-actors/actors/builtin/account" - "github.com/filecoin-project/specs-actors/actors/builtin/market" - "github.com/filecoin-project/specs-actors/actors/builtin/miner" - "github.com/filecoin-project/specs-actors/actors/builtin/power" - "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" @@ -40,6 +33,12 @@ import ( "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/actors/builtin" + "github.com/filecoin-project/specs-actors/actors/builtin/account" + "github.com/filecoin-project/specs-actors/actors/builtin/market" + "github.com/filecoin-project/specs-actors/actors/builtin/miner" + "github.com/filecoin-project/specs-actors/actors/builtin/power" + "github.com/filecoin-project/specs-actors/actors/util/adt" ) var ChainCmd = &cli.Command{ diff --git a/cli/chain_test.go b/cli/chain_test.go index 9fd46724e6b..9c9a65f9957 100644 --- a/cli/chain_test.go +++ b/cli/chain_test.go @@ -16,11 +16,10 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/v7/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" + "github.com/filecoin-project/specs-actors/v7/actors/builtin" ) func TestChainHead(t *testing.T) { diff --git a/cli/client.go b/cli/client.go index 88f7ed2087b..6a7dc90745e 100644 --- a/cli/client.go +++ b/cli/client.go @@ -36,7 +36,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cli/client_retr.go b/cli/client_retr.go index fa8164ab5ef..c713a121c9d 100644 --- a/cli/client_retr.go +++ b/cli/client_retr.go @@ -30,7 +30,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/big" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" cliutil "github.com/filecoin-project/lotus/cli/util" diff --git a/cli/disputer.go b/cli/disputer.go index de3f5032468..0668f634efd 100644 --- a/cli/disputer.go +++ b/cli/disputer.go @@ -13,14 +13,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" ) var disputeLog = logging.Logger("disputer") diff --git a/cli/evm.go b/cli/evm.go index 7eb36f8953b..8eb9edd9749 100644 --- a/cli/evm.go +++ b/cli/evm.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cli/filplus.go b/cli/filplus.go index 9fbd2a489ad..072b49b29b6 100644 --- a/cli/filplus.go +++ b/cli/filplus.go @@ -19,7 +19,6 @@ import ( verifregtypes8 "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" verifregtypes9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/cli/info.go b/cli/info.go index 8b36be4889b..80abae3b4cb 100644 --- a/cli/info.go +++ b/cli/info.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/mpool.go b/cli/mpool.go index c83fb4b614a..c0e068440b6 100644 --- a/cli/mpool.go +++ b/cli/mpool.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/cli/mpool_manage.go b/cli/mpool_manage.go index 67724c77332..c3c49cd956c 100644 --- a/cli/mpool_manage.go +++ b/cli/mpool_manage.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/cli/mpool_test.go b/cli/mpool_test.go index 0aa055ba31f..4f4aa831ce3 100644 --- a/cli/mpool_test.go +++ b/cli/mpool_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/multisig.go b/cli/multisig.go index 1af2a4c9e4e..7cfb2c9c6c8 100644 --- a/cli/multisig.go +++ b/cli/multisig.go @@ -19,9 +19,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -30,6 +27,8 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) var multisigCmd = &cli.Command{ diff --git a/cli/net.go b/cli/net.go index f25799e9503..031c681e915 100644 --- a/cli/net.go +++ b/cli/net.go @@ -21,7 +21,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - atypes "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/addrutil" diff --git a/cli/params.go b/cli/params.go index e79eb8e3071..3491441396b 100644 --- a/cli/params.go +++ b/cli/params.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-paramfetch" - "github.com/filecoin-project/lotus/build" ) diff --git a/cli/paych.go b/cli/paych.go index 1067d091376..9000eab6667 100644 --- a/cli/paych.go +++ b/cli/paych.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin/v8/paych" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lpaych "github.com/filecoin-project/lotus/chain/actors/builtin/paych" diff --git a/cli/send.go b/cli/send.go index cfa2515c07b..bf4843f0ca8 100644 --- a/cli/send.go +++ b/cli/send.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cli/send_test.go b/cli/send_test.go index 2c59a9641f6..dfe87de7cbd 100644 --- a/cli/send_test.go +++ b/cli/send_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cli/sending_ui.go b/cli/sending_ui.go index d2d2ed3c110..88a30cf1064 100644 --- a/cli/sending_ui.go +++ b/cli/sending_ui.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/services.go b/cli/services.go index ef257693f89..a4ea794c314 100644 --- a/cli/services.go +++ b/cli/services.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/services_send_test.go b/cli/services_send_test.go index a524abe8cc1..0640300d5e1 100644 --- a/cli/services_send_test.go +++ b/cli/services_send_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/servicesmock_test.go b/cli/servicesmock_test.go index 11260ab08be..e9a792ce800 100644 --- a/cli/servicesmock_test.go +++ b/cli/servicesmock_test.go @@ -13,7 +13,6 @@ import ( go_address "github.com/filecoin-project/go-address" abi "github.com/filecoin-project/go-state-types/abi" big "github.com/filecoin-project/go-state-types/big" - api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cli/state.go b/cli/state.go index 31666a21c08..d30347ca150 100644 --- a/cli/state.go +++ b/cli/state.go @@ -31,7 +31,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cli/sync.go b/cli/sync.go index 18ff24bc26e..659b270bf1a 100644 --- a/cli/sync.go +++ b/cli/sync.go @@ -9,7 +9,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" diff --git a/cli/sync_test.go b/cli/sync_test.go index 456d165f293..87613c73dba 100644 --- a/cli/sync_test.go +++ b/cli/sync_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" diff --git a/cli/util/api.go b/cli/util/api.go index b76544d575d..5181e876167 100644 --- a/cli/util/api.go +++ b/cli/util/api.go @@ -18,7 +18,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/api/v0api" @@ -119,7 +118,7 @@ func GetAPIInfoMulti(ctx *cli.Context, t repo.RepoType) ([]APIInfo, error) { } } - return []APIInfo{}, fmt.Errorf("could not determine API endpoint for node type: %v", t.Type()) + return []APIInfo{}, fmt.Errorf("could not determine API endpoint for node type: %v. Try setting environment variable: %s", t.Type(), primaryEnv) } func GetAPIInfo(ctx *cli.Context, t repo.RepoType) (APIInfo, error) { diff --git a/cli/util/epoch.go b/cli/util/epoch.go index 81c92a7e3ed..019f49806d3 100644 --- a/cli/util/epoch.go +++ b/cli/util/epoch.go @@ -7,7 +7,6 @@ import ( "github.com/hako/durafmt" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cli/wallet.go b/cli/wallet.go index faf7bc23955..79d432a2390 100644 --- a/cli/wallet.go +++ b/cli/wallet.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/wallet_test.go b/cli/wallet_test.go index eb2c544f0a6..d628fc215c1 100644 --- a/cli/wallet_test.go +++ b/cli/wallet_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/chain-noise/main.go b/cmd/chain-noise/main.go index 6f124fad7ed..6e73779e2de 100644 --- a/cmd/chain-noise/main.go +++ b/cmd/chain-noise/main.go @@ -10,7 +10,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-bench/caching_verifier.go b/cmd/lotus-bench/caching_verifier.go index 4a4e8879ff7..487ea969246 100644 --- a/cmd/lotus-bench/caching_verifier.go +++ b/cmd/lotus-bench/caching_verifier.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/cmd/lotus-bench/import.go b/cmd/lotus-bench/import.go index 95b91054a75..c240897061c 100644 --- a/cmd/lotus-bench/import.go +++ b/cmd/lotus-bench/import.go @@ -29,7 +29,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" diff --git a/cmd/lotus-bench/main.go b/cmd/lotus-bench/main.go index 7d3c0cde084..543b2d48a90 100644 --- a/cmd/lotus-bench/main.go +++ b/cmd/lotus-bench/main.go @@ -30,8 +30,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" prooftypes "github.com/filecoin-project/go-state-types/proof" - adt "github.com/filecoin-project/specs-actors/v6/actors/util/adt" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -43,6 +41,7 @@ import ( "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper/basicfs" "github.com/filecoin-project/lotus/storage/sealer/storiface" + adt "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var log = logging.Logger("lotus-bench") diff --git a/cmd/lotus-bench/simple.go b/cmd/lotus-bench/simple.go index 8ae5713ad07..3c4e3b92eb1 100644 --- a/cmd/lotus-bench/simple.go +++ b/cmd/lotus-bench/simple.go @@ -18,12 +18,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/abi" - prf "github.com/filecoin-project/specs-actors/actors/runtime/proof" - "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" + prf "github.com/filecoin-project/specs-actors/actors/runtime/proof" ) var simpleCmd = &cli.Command{ diff --git a/cmd/lotus-fountain/main.go b/cmd/lotus-fountain/main.go index f6d503c2f3d..1f4ce8d8395 100644 --- a/cmd/lotus-fountain/main.go +++ b/cmd/lotus-fountain/main.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-address" verifregtypes9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-gateway/main.go b/cmd/lotus-gateway/main.go index 35a43e18b1e..0d709cb286e 100644 --- a/cmd/lotus-gateway/main.go +++ b/cmd/lotus-gateway/main.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-health/main.go b/cmd/lotus-health/main.go index a7052f21480..ea21b200ccb 100644 --- a/cmd/lotus-health/main.go +++ b/cmd/lotus-health/main.go @@ -13,7 +13,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/actor.go b/cmd/lotus-miner/actor.go index 320c4e6de81..ce9e77cace2 100644 --- a/cmd/lotus-miner/actor.go +++ b/cmd/lotus-miner/actor.go @@ -24,7 +24,6 @@ import ( minerV12 "github.com/filecoin-project/go-state-types/builtin/v12/miner" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-miner/actor_test.go b/cmd/lotus-miner/actor_test.go index dfb4522137c..19106927601 100644 --- a/cmd/lotus-miner/actor_test.go +++ b/cmd/lotus-miner/actor_test.go @@ -15,7 +15,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/backup.go b/cmd/lotus-miner/backup.go index cf8c9f9125c..7443a101b04 100644 --- a/cmd/lotus-miner/backup.go +++ b/cmd/lotus-miner/backup.go @@ -4,7 +4,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" - lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cmd/lotus-miner/info.go b/cmd/lotus-miner/info.go index 6d8ade340ef..532cef80ed4 100644 --- a/cmd/lotus-miner/info.go +++ b/cmd/lotus-miner/info.go @@ -18,8 +18,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/blockstore" @@ -33,6 +31,7 @@ import ( "github.com/filecoin-project/lotus/journal/alerting" sealing "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" + "github.com/filecoin-project/specs-actors/actors/builtin" ) var infoCmd = &cli.Command{ diff --git a/cmd/lotus-miner/init.go b/cmd/lotus-miner/init.go index c109e85b980..67e710d06a0 100644 --- a/cmd/lotus-miner/init.go +++ b/cmd/lotus-miner/init.go @@ -30,10 +30,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-statestore" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" @@ -58,6 +54,9 @@ import ( "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" ) var initCmd = &cli.Command{ diff --git a/cmd/lotus-miner/init_restore.go b/cmd/lotus-miner/init_restore.go index 7e28729bbeb..ab62fd0a34a 100644 --- a/cmd/lotus-miner/init_restore.go +++ b/cmd/lotus-miner/init_restore.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cmd/lotus-miner/init_service.go b/cmd/lotus-miner/init_service.go index 235e4e4c8cc..45027cc3f8b 100644 --- a/cmd/lotus-miner/init_service.go +++ b/cmd/lotus-miner/init_service.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" diff --git a/cmd/lotus-miner/main.go b/cmd/lotus-miner/main.go index 911e98e260a..0902ec187ae 100644 --- a/cmd/lotus-miner/main.go +++ b/cmd/lotus-miner/main.go @@ -10,7 +10,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-miner/market.go b/cmd/lotus-miner/market.go index 29eb662a78d..69a5442b512 100644 --- a/cmd/lotus-miner/market.go +++ b/cmd/lotus-miner/market.go @@ -27,7 +27,6 @@ import ( datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/precommits-info.go b/cmd/lotus-miner/precommits-info.go index 3f9e8c92742..16146e736b9 100644 --- a/cmd/lotus-miner/precommits-info.go +++ b/cmd/lotus-miner/precommits-info.go @@ -7,12 +7,11 @@ import ( cbor "github.com/ipfs/go-ipld-cbor" "github.com/urfave/cli/v2" - "github.com/filecoin-project/specs-actors/v7/actors/util/adt" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var sectorPreCommitsCmd = &cli.Command{ diff --git a/cmd/lotus-miner/proving.go b/cmd/lotus-miner/proving.go index 3ecc58ba7af..c64072ee1ad 100644 --- a/cmd/lotus-miner/proving.go +++ b/cmd/lotus-miner/proving.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/cmd/lotus-miner/retrieval-deals.go b/cmd/lotus-miner/retrieval-deals.go index 42b0fa1f6f8..a7b4a62eeaf 100644 --- a/cmd/lotus-miner/retrieval-deals.go +++ b/cmd/lotus-miner/retrieval-deals.go @@ -9,7 +9,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-miner/sealing.go b/cmd/lotus-miner/sealing.go index b2f4dcab911..7d732044f0e 100644 --- a/cmd/lotus-miner/sealing.go +++ b/cmd/lotus-miner/sealing.go @@ -21,7 +21,6 @@ import ( "github.com/filecoin-project/go-padreader" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/httpreader" diff --git a/cmd/lotus-miner/sectors.go b/cmd/lotus-miner/sectors.go index 07cc2e795d1..936535bf625 100644 --- a/cmd/lotus-miner/sectors.go +++ b/cmd/lotus-miner/sectors.go @@ -25,7 +25,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-miner/storage.go b/cmd/lotus-miner/storage.go index fdd5b569656..d3edc8e52b7 100644 --- a/cmd/lotus-miner/storage.go +++ b/cmd/lotus-miner/storage.go @@ -22,7 +22,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-pcr/main.go b/cmd/lotus-pcr/main.go index 199810e03cd..b1fcd2079a9 100644 --- a/cmd/lotus-pcr/main.go +++ b/cmd/lotus-pcr/main.go @@ -29,8 +29,6 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -39,6 +37,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/tools/stats/sync" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var log = logging.Logger("main") diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 6248742cc83..71e2acb64d3 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -23,7 +23,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-seed/genesis.go b/cmd/lotus-seed/genesis.go index 9fdce456bf1..44665057f02 100644 --- a/cmd/lotus-seed/genesis.go +++ b/cmd/lotus-seed/genesis.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" diff --git a/cmd/lotus-seed/main.go b/cmd/lotus-seed/main.go index d362804c95d..0a1decc2274 100644 --- a/cmd/lotus-seed/main.go +++ b/cmd/lotus-seed/main.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-seed/seed/seed.go b/cmd/lotus-seed/seed/seed.go index 48f00f8a638..45a292094f7 100644 --- a/cmd/lotus-seed/seed/seed.go +++ b/cmd/lotus-seed/seed/seed.go @@ -22,7 +22,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" diff --git a/cmd/lotus-shed/actor.go b/cmd/lotus-shed/actor.go index 8562b63c32d..da466476e10 100644 --- a/cmd/lotus-shed/actor.go +++ b/cmd/lotus-shed/actor.go @@ -17,14 +17,13 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/tablewriter" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var actorCmd = &cli.Command{ diff --git a/cmd/lotus-shed/balancer.go b/cmd/lotus-shed/balancer.go index edc484ab644..115839b68e8 100644 --- a/cmd/lotus-shed/balancer.go +++ b/cmd/lotus-shed/balancer.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/balances.go b/cmd/lotus-shed/balances.go index 28569cd1212..d0cf8da0f32 100644 --- a/cmd/lotus-shed/balances.go +++ b/cmd/lotus-shed/balances.go @@ -23,7 +23,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-shed/cid.go b/cmd/lotus-shed/cid.go index f6c4a472171..b26c15a7579 100644 --- a/cmd/lotus-shed/cid.go +++ b/cmd/lotus-shed/cid.go @@ -17,7 +17,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/cmd/lotus-shed/consensus.go b/cmd/lotus-shed/consensus.go index 197de56f92b..9939202dd7f 100644 --- a/cmd/lotus-shed/consensus.go +++ b/cmd/lotus-shed/consensus.go @@ -14,7 +14,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-shed/cron-count.go b/cmd/lotus-shed/cron-count.go index 9741792ecf1..b72d0d3552a 100644 --- a/cmd/lotus-shed/cron-count.go +++ b/cmd/lotus-shed/cron-count.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/deal-label.go b/cmd/lotus-shed/deal-label.go index 417d1370193..3407b104bcf 100644 --- a/cmd/lotus-shed/deal-label.go +++ b/cmd/lotus-shed/deal-label.go @@ -10,13 +10,12 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/node/repo" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var dealLabelCmd = &cli.Command{ diff --git a/cmd/lotus-shed/diff.go b/cmd/lotus-shed/diff.go index 981dc850c62..439a40619da 100644 --- a/cmd/lotus-shed/diff.go +++ b/cmd/lotus-shed/diff.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner9 "github.com/filecoin-project/go-state-types/builtin/v9/miner" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/election.go b/cmd/lotus-shed/election.go index 23c533c315f..4737437fb32 100644 --- a/cmd/lotus-shed/election.go +++ b/cmd/lotus-shed/election.go @@ -11,12 +11,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) var electionCmd = &cli.Command{ diff --git a/cmd/lotus-shed/eth.go b/cmd/lotus-shed/eth.go index fde4f96f68f..8df17e6e9ed 100644 --- a/cmd/lotus-shed/eth.go +++ b/cmd/lotus-shed/eth.go @@ -7,7 +7,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/export.go b/cmd/lotus-shed/export.go index e6d0c4e056f..d0dfe231e65 100644 --- a/cmd/lotus-shed/export.go +++ b/cmd/lotus-shed/export.go @@ -29,7 +29,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/store" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/fevmanalytics.go b/cmd/lotus-shed/fevmanalytics.go index 19416b77e12..73306bf1f8f 100644 --- a/cmd/lotus-shed/fevmanalytics.go +++ b/cmd/lotus-shed/fevmanalytics.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-shed/fip-0036.go b/cmd/lotus-shed/fip-0036.go index 4c8456c04ce..22110be0557 100644 --- a/cmd/lotus-shed/fip-0036.go +++ b/cmd/lotus-shed/fip-0036.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/cmd/lotus-shed/fr32.go b/cmd/lotus-shed/fr32.go index 1bdca198cf0..e9dff0029be 100644 --- a/cmd/lotus-shed/fr32.go +++ b/cmd/lotus-shed/fr32.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/cmd/lotus-shed/frozen-miners.go b/cmd/lotus-shed/frozen-miners.go index 035777bd609..e98017f3619 100644 --- a/cmd/lotus-shed/frozen-miners.go +++ b/cmd/lotus-shed/frozen-miners.go @@ -7,9 +7,8 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - lcli "github.com/filecoin-project/lotus/cli" + "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var frozenMinersCmd = &cli.Command{ diff --git a/cmd/lotus-shed/gas-estimation.go b/cmd/lotus-shed/gas-estimation.go index 5dc048f562c..e5f8c6bb25d 100644 --- a/cmd/lotus-shed/gas-estimation.go +++ b/cmd/lotus-shed/gas-estimation.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon/drand" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/cmd/lotus-shed/genesis-verify.go b/cmd/lotus-shed/genesis-verify.go index 6795f1528a1..7e55c7f488d 100644 --- a/cmd/lotus-shed/genesis-verify.go +++ b/cmd/lotus-shed/genesis-verify.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/cmd/lotus-shed/hello.go b/cmd/lotus-shed/hello.go index d16f9373563..8510d66e92d 100644 --- a/cmd/lotus-shed/hello.go +++ b/cmd/lotus-shed/hello.go @@ -11,7 +11,6 @@ import ( "github.com/urfave/cli/v2" cborutil "github.com/filecoin-project/go-cbor-util" - lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/hello" ) diff --git a/cmd/lotus-shed/indexes.go b/cmd/lotus-shed/indexes.go index be7d43e0513..e6b91439021 100644 --- a/cmd/lotus-shed/indexes.go +++ b/cmd/lotus-shed/indexes.go @@ -18,7 +18,6 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cmd/lotus-shed/invariants.go b/cmd/lotus-shed/invariants.go index e74a0dd24c2..2764e2a1020 100644 --- a/cmd/lotus-shed/invariants.go +++ b/cmd/lotus-shed/invariants.go @@ -19,7 +19,6 @@ import ( v12 "github.com/filecoin-project/go-state-types/builtin/v12" v8 "github.com/filecoin-project/go-state-types/builtin/v8" v9 "github.com/filecoin-project/go-state-types/builtin/v9" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/cmd/lotus-shed/jwt.go b/cmd/lotus-shed/jwt.go index 2a24c256933..aaa78a6fd52 100644 --- a/cmd/lotus-shed/jwt.go +++ b/cmd/lotus-shed/jwt.go @@ -14,7 +14,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules" diff --git a/cmd/lotus-shed/ledger.go b/cmd/lotus-shed/ledger.go index d9a888d2061..efce54a5e8c 100644 --- a/cmd/lotus-shed/ledger.go +++ b/cmd/lotus-shed/ledger.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/types" ledgerwallet "github.com/filecoin-project/lotus/chain/wallet/ledger" diff --git a/cmd/lotus-shed/market.go b/cmd/lotus-shed/market.go index 4436e3c404a..ddb82a2f0fb 100644 --- a/cmd/lotus-shed/market.go +++ b/cmd/lotus-shed/market.go @@ -23,7 +23,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" market11 "github.com/filecoin-project/go-state-types/builtin/v11/market" "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/math.go b/cmd/lotus-shed/math.go index 09776d4e76f..fc01ce5561d 100644 --- a/cmd/lotus-shed/math.go +++ b/cmd/lotus-shed/math.go @@ -10,9 +10,8 @@ import ( "github.com/urfave/cli/v2" "golang.org/x/xerrors" - miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" - "github.com/filecoin-project/lotus/chain/types" + miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" ) var mathCmd = &cli.Command{ diff --git a/cmd/lotus-shed/mempool-stats.go b/cmd/lotus-shed/mempool-stats.go index 08aceb02025..5af877bdce5 100644 --- a/cmd/lotus-shed/mempool-stats.go +++ b/cmd/lotus-shed/mempool-stats.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin" - lapi "github.com/filecoin-project/lotus/api" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/migrations.go b/cmd/lotus-shed/migrations.go index 96e4747b7ef..12939913f13 100644 --- a/cmd/lotus-shed/migrations.go +++ b/cmd/lotus-shed/migrations.go @@ -32,8 +32,6 @@ import ( "github.com/filecoin-project/go-state-types/manifest" mutil "github.com/filecoin-project/go-state-types/migration" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" - "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/blockstore/splitstore" @@ -55,6 +53,7 @@ import ( lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" + "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" ) var migrationsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner-multisig.go b/cmd/lotus-shed/miner-multisig.go index e8394b17a60..19804dfbd26 100644 --- a/cmd/lotus-shed/miner-multisig.go +++ b/cmd/lotus-shed/miner-multisig.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/builtin/v9/multisig" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/miner-peerid.go b/cmd/lotus-shed/miner-peerid.go index e430637976c..4bd92de4b43 100644 --- a/cmd/lotus-shed/miner-peerid.go +++ b/cmd/lotus-shed/miner-peerid.go @@ -12,9 +12,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" @@ -22,6 +19,8 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" ) var minerPeeridCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner-types.go b/cmd/lotus-shed/miner-types.go index 822d037aa26..9b7c690b935 100644 --- a/cmd/lotus-shed/miner-types.go +++ b/cmd/lotus-shed/miner-types.go @@ -14,9 +14,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" big2 "github.com/filecoin-project/go-state-types/big" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -24,6 +21,8 @@ import ( "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/repo" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var minerTypesCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner.go b/cmd/lotus-shed/miner.go index a8bb9374422..82026590a22 100644 --- a/cmd/lotus-shed/miner.go +++ b/cmd/lotus-shed/miner.go @@ -28,9 +28,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" miner8 "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/crypto" - power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -38,6 +35,8 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) var minerCmd = &cli.Command{ diff --git a/cmd/lotus-shed/msg.go b/cmd/lotus-shed/msg.go index 35f8eed35c1..618d5195ce9 100644 --- a/cmd/lotus-shed/msg.go +++ b/cmd/lotus-shed/msg.go @@ -14,11 +14,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) var msgCmd = &cli.Command{ diff --git a/cmd/lotus-shed/msig.go b/cmd/lotus-shed/msig.go index ccc932c93ff..e877b4ff733 100644 --- a/cmd/lotus-shed/msig.go +++ b/cmd/lotus-shed/msig.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/cmd/lotus-shed/nonce-fix.go b/cmd/lotus-shed/nonce-fix.go index d69c8a48dc6..c55314b9a78 100644 --- a/cmd/lotus-shed/nonce-fix.go +++ b/cmd/lotus-shed/nonce-fix.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/params.go b/cmd/lotus-shed/params.go index 08ce5ba83f1..88000299b7e 100644 --- a/cmd/lotus-shed/params.go +++ b/cmd/lotus-shed/params.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-paramfetch" - "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/postfind.go b/cmd/lotus-shed/postfind.go index 23eb4ceb717..ec397b56bf9 100644 --- a/cmd/lotus-shed/postfind.go +++ b/cmd/lotus-shed/postfind.go @@ -8,11 +8,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/specs-actors/v2/actors/builtin" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) var postFindCmd = &cli.Command{ diff --git a/cmd/lotus-shed/proofs.go b/cmd/lotus-shed/proofs.go index 1a16e2fdc3b..55cb715b8d6 100644 --- a/cmd/lotus-shed/proofs.go +++ b/cmd/lotus-shed/proofs.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/pruning.go b/cmd/lotus-shed/pruning.go index c0bd453b145..ac96e2ea4f6 100644 --- a/cmd/lotus-shed/pruning.go +++ b/cmd/lotus-shed/pruning.go @@ -11,7 +11,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/store" diff --git a/cmd/lotus-shed/sectors.go b/cmd/lotus-shed/sectors.go index 899e0f290b4..21d14a26d81 100644 --- a/cmd/lotus-shed/sectors.go +++ b/cmd/lotus-shed/sectors.go @@ -25,8 +25,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" @@ -37,6 +35,7 @@ import ( "github.com/filecoin-project/lotus/storage/sealer/fr32" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var sectorsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/send-csv.go b/cmd/lotus-shed/send-csv.go index 17b62150fc5..a697802edf4 100644 --- a/cmd/lotus-shed/send-csv.go +++ b/cmd/lotus-shed/send-csv.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/signatures.go b/cmd/lotus-shed/signatures.go index 536f8e82d6a..ca761a2c0f5 100644 --- a/cmd/lotus-shed/signatures.go +++ b/cmd/lotus-shed/signatures.go @@ -12,7 +12,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/cmd/lotus-shed/state-stats.go b/cmd/lotus-shed/state-stats.go index 4eb00f98183..ca2b8b10e9e 100644 --- a/cmd/lotus-shed/state-stats.go +++ b/cmd/lotus-shed/state-stats.go @@ -26,7 +26,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" gstactors "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-shed/stateroot-stats.go b/cmd/lotus-shed/stateroot-stats.go index 16dfc593578..7605366ed94 100644 --- a/cmd/lotus-shed/stateroot-stats.go +++ b/cmd/lotus-shed/stateroot-stats.go @@ -9,7 +9,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/sync.go b/cmd/lotus-shed/sync.go index eb11dea27eb..f6b0e5c7e2d 100644 --- a/cmd/lotus-shed/sync.go +++ b/cmd/lotus-shed/sync.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/terminations.go b/cmd/lotus-shed/terminations.go index c5f35995a4f..a16fafb6e12 100644 --- a/cmd/lotus-shed/terminations.go +++ b/cmd/lotus-shed/terminations.go @@ -13,8 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" - miner2 "github.com/filecoin-project/specs-actors/actors/builtin/miner" - "github.com/filecoin-project/lotus/chain/actors/adt" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -25,6 +23,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" + miner2 "github.com/filecoin-project/specs-actors/actors/builtin/miner" ) var terminationsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/verifreg.go b/cmd/lotus-shed/verifreg.go index 4e2894442a3..3bf81d541ba 100644 --- a/cmd/lotus-shed/verifreg.go +++ b/cmd/lotus-shed/verifreg.go @@ -13,8 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/big" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" "github.com/filecoin-project/go-state-types/crypto" - verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -23,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" + verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" ) var verifRegCmd = &cli.Command{ diff --git a/cmd/lotus-sim/info.go b/cmd/lotus-sim/info.go index b92fa4b2f4d..7eadde6d8ab 100644 --- a/cmd/lotus-sim/info.go +++ b/cmd/lotus-sim/info.go @@ -10,7 +10,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/cmd/lotus-sim/info_capacity.go b/cmd/lotus-sim/info_capacity.go index a92d2cde494..476d2716da9 100644 --- a/cmd/lotus-sim/info_capacity.go +++ b/cmd/lotus-sim/info_capacity.go @@ -6,7 +6,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/info_commit.go b/cmd/lotus-sim/info_commit.go index e9bf8aa973d..98613748a06 100644 --- a/cmd/lotus-sim/info_commit.go +++ b/cmd/lotus-sim/info_commit.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation" diff --git a/cmd/lotus-sim/info_state.go b/cmd/lotus-sim/info_state.go index 125dae81d96..9050f34eb77 100644 --- a/cmd/lotus-sim/info_state.go +++ b/cmd/lotus-sim/info_state.go @@ -13,7 +13,6 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/info_wdpost.go b/cmd/lotus-sim/info_wdpost.go index 426e85ca831..2f53d4f47b8 100644 --- a/cmd/lotus-sim/info_wdpost.go +++ b/cmd/lotus-sim/info_wdpost.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation" diff --git a/cmd/lotus-sim/simulation/block.go b/cmd/lotus-sim/simulation/block.go index 7cf5a6be66b..846082f42fa 100644 --- a/cmd/lotus-sim/simulation/block.go +++ b/cmd/lotus-sim/simulation/block.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go index a4c9fdeaaa0..e7cb6c996a3 100644 --- a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go +++ b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-sim/simulation/messages.go b/cmd/lotus-sim/simulation/messages.go index d6dd98d4382..44ffc6880b2 100644 --- a/cmd/lotus-sim/simulation/messages.go +++ b/cmd/lotus-sim/simulation/messages.go @@ -6,9 +6,8 @@ import ( "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/types" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) // toArray converts the given set of CIDs to an AMT. This is usually used to pack messages into blocks. diff --git a/cmd/lotus-sim/simulation/mock/mock.go b/cmd/lotus-sim/simulation/mock/mock.go index 4699b2aa665..bbf2cdbcb39 100644 --- a/cmd/lotus-sim/simulation/mock/mock.go +++ b/cmd/lotus-sim/simulation/mock/mock.go @@ -12,10 +12,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/storiface" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" tutils "github.com/filecoin-project/specs-actors/v5/support/testing" - - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) // Ideally, we'd use extern/sealer/mock. Unfortunately, those mocks are a bit _too_ accurate diff --git a/cmd/lotus-sim/simulation/simulation.go b/cmd/lotus-sim/simulation/simulation.go index 47d06aeda91..af793dc1742 100644 --- a/cmd/lotus-sim/simulation/simulation.go +++ b/cmd/lotus-sim/simulation/simulation.go @@ -13,8 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/index" @@ -23,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/stages" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) var log = logging.Logger("simulation") diff --git a/cmd/lotus-sim/simulation/stages/commit_queue.go b/cmd/lotus-sim/simulation/stages/commit_queue.go index 60cbfa4badf..1824f49eba1 100644 --- a/cmd/lotus-sim/simulation/stages/commit_queue.go +++ b/cmd/lotus-sim/simulation/stages/commit_queue.go @@ -6,7 +6,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/cmd/lotus-sim/simulation/stages/commit_queue_test.go b/cmd/lotus-sim/simulation/stages/commit_queue_test.go index 0b9a2ebdbc4..8db4d681e84 100644 --- a/cmd/lotus-sim/simulation/stages/commit_queue_test.go +++ b/cmd/lotus-sim/simulation/stages/commit_queue_test.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" - "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/cmd/lotus-sim/simulation/stages/funding_stage.go b/cmd/lotus-sim/simulation/stages/funding_stage.go index 4ce4afae103..712ea1bc910 100644 --- a/cmd/lotus-sim/simulation/stages/funding_stage.go +++ b/cmd/lotus-sim/simulation/stages/funding_stage.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/cmd/lotus-sim/simulation/stages/interface.go b/cmd/lotus-sim/simulation/stages/interface.go index fffdbec6b54..718f7a9f52a 100644 --- a/cmd/lotus-sim/simulation/stages/interface.go +++ b/cmd/lotus-sim/simulation/stages/interface.go @@ -6,7 +6,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" ) diff --git a/cmd/lotus-sim/simulation/stages/precommit_stage.go b/cmd/lotus-sim/simulation/stages/precommit_stage.go index 1a89413d72c..a67c5ba42f1 100644 --- a/cmd/lotus-sim/simulation/stages/precommit_stage.go +++ b/cmd/lotus-sim/simulation/stages/precommit_stage.go @@ -13,8 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -23,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) const ( diff --git a/cmd/lotus-sim/simulation/stages/provecommit_stage.go b/cmd/lotus-sim/simulation/stages/provecommit_stage.go index d15ea60f0a9..71c483b3e1a 100644 --- a/cmd/lotus-sim/simulation/stages/provecommit_stage.go +++ b/cmd/lotus-sim/simulation/stages/provecommit_stage.go @@ -11,9 +11,6 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -22,6 +19,8 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" ) const ( diff --git a/cmd/lotus-sim/simulation/stages/util.go b/cmd/lotus-sim/simulation/stages/util.go index 0c17823c060..ed820068ec8 100644 --- a/cmd/lotus-sim/simulation/stages/util.go +++ b/cmd/lotus-sim/simulation/stages/util.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/builtin/power" diff --git a/cmd/lotus-sim/simulation/stages/windowpost_stage.go b/cmd/lotus-sim/simulation/stages/windowpost_stage.go index 0b4109d08cb..3e7e5fe1e9f 100644 --- a/cmd/lotus-sim/simulation/stages/windowpost_stage.go +++ b/cmd/lotus-sim/simulation/stages/windowpost_stage.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/cmd/lotus-stats/main.go b/cmd/lotus-stats/main.go index 20971c1f3ef..0318d52f518 100644 --- a/cmd/lotus-stats/main.go +++ b/cmd/lotus-stats/main.go @@ -15,7 +15,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/tools/stats/influx" diff --git a/cmd/lotus-wallet/interactive.go b/cmd/lotus-wallet/interactive.go index 96d2ad9f2e0..a72b090d63c 100644 --- a/cmd/lotus-wallet/interactive.go +++ b/cmd/lotus-wallet/interactive.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-wallet/logged.go b/cmd/lotus-wallet/logged.go index 4f07d6ae46e..cba281f0200 100644 --- a/cmd/lotus-wallet/logged.go +++ b/cmd/lotus-wallet/logged.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-wallet/main.go b/cmd/lotus-wallet/main.go index 8360dae15d0..b40a56cddb0 100644 --- a/cmd/lotus-wallet/main.go +++ b/cmd/lotus-wallet/main.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-worker/main.go b/cmd/lotus-worker/main.go index 257dac800c2..4a4d20574cd 100644 --- a/cmd/lotus-worker/main.go +++ b/cmd/lotus-worker/main.go @@ -26,7 +26,6 @@ import ( "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-worker/sealworker/rpc.go b/cmd/lotus-worker/sealworker/rpc.go index 97f78942e59..9de1303fdd5 100644 --- a/cmd/lotus-worker/sealworker/rpc.go +++ b/cmd/lotus-worker/sealworker/rpc.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus/backup.go b/cmd/lotus/backup.go index efbac3e2bb0..9393e504441 100644 --- a/cmd/lotus/backup.go +++ b/cmd/lotus/backup.go @@ -10,7 +10,6 @@ import ( "gopkg.in/cheggaaa/pb.v1" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/chain/store" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/backupds" diff --git a/cmd/lotus/daemon.go b/cmd/lotus/daemon.go index 042e89f216a..35d30d5f550 100644 --- a/cmd/lotus/daemon.go +++ b/cmd/lotus/daemon.go @@ -30,7 +30,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-paramfetch" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon/drand" diff --git a/cmd/lotus/debug_advance.go b/cmd/lotus/debug_advance.go index dee2db97e2b..de8140623a9 100644 --- a/cmd/lotus/debug_advance.go +++ b/cmd/lotus/debug_advance.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" diff --git a/cmd/tvx/codenames.go b/cmd/tvx/codenames.go index 81143c85ce0..00291c43c06 100644 --- a/cmd/tvx/codenames.go +++ b/cmd/tvx/codenames.go @@ -2,7 +2,6 @@ package main import ( "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" ) diff --git a/cmd/tvx/codenames_test.go b/cmd/tvx/codenames_test.go index 46d8466ecb2..5cf531887ff 100644 --- a/cmd/tvx/codenames_test.go +++ b/cmd/tvx/codenames_test.go @@ -6,7 +6,6 @@ import ( "testing" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" ) diff --git a/cmd/tvx/exec.go b/cmd/tvx/exec.go index 95a16e6d13c..dd65facf820 100644 --- a/cmd/tvx/exec.go +++ b/cmd/tvx/exec.go @@ -16,12 +16,11 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/conformance" + "github.com/filecoin-project/test-vectors/schema" ) var execFlags struct { diff --git a/cmd/tvx/extract_many.go b/cmd/tvx/extract_many.go index 7c8d306d8c0..3d8db2f0d22 100644 --- a/cmd/tvx/extract_many.go +++ b/cmd/tvx/extract_many.go @@ -19,7 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" ) diff --git a/cmd/tvx/extract_message.go b/cmd/tvx/extract_message.go index 95711414bcc..49cddb9cb71 100644 --- a/cmd/tvx/extract_message.go +++ b/cmd/tvx/extract_message.go @@ -12,8 +12,6 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" @@ -23,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/conformance" + "github.com/filecoin-project/test-vectors/schema" ) func doExtractMessage(opts extractOpts) error { diff --git a/cmd/tvx/extract_tipset.go b/cmd/tvx/extract_tipset.go index 553961f4491..64e3b4ff3f9 100644 --- a/cmd/tvx/extract_tipset.go +++ b/cmd/tvx/extract_tipset.go @@ -10,11 +10,10 @@ import ( "github.com/ipfs/go-cid" - "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/conformance" + "github.com/filecoin-project/test-vectors/schema" ) func doExtractTipset(opts extractOpts) error { diff --git a/cmd/tvx/main.go b/cmd/tvx/main.go index 5021dd64b25..dd284983897 100644 --- a/cmd/tvx/main.go +++ b/cmd/tvx/main.go @@ -9,7 +9,6 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api/v1api" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/tvx/simulate.go b/cmd/tvx/simulate.go index ea7b67a662d..b6ee85b27e4 100644 --- a/cmd/tvx/simulate.go +++ b/cmd/tvx/simulate.go @@ -14,11 +14,10 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/conformance" + "github.com/filecoin-project/test-vectors/schema" ) var simulateFlags struct { diff --git a/cmd/tvx/state.go b/cmd/tvx/state.go index 9674bf17ed6..6ee06bdac5e 100644 --- a/cmd/tvx/state.go +++ b/cmd/tvx/state.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" diff --git a/conformance/chaos/actor.go b/conformance/chaos/actor.go index 3a8b2b50af5..cb3ec2147fa 100644 --- a/conformance/chaos/actor.go +++ b/conformance/chaos/actor.go @@ -2,7 +2,6 @@ package chaos import ( "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/cbor" @@ -10,7 +9,6 @@ import ( "github.com/filecoin-project/go-state-types/rt" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" - "github.com/filecoin-project/lotus/chain/actors/builtin" ) diff --git a/conformance/chaos/actor_test.go b/conformance/chaos/actor_test.go index fd45504d27b..a618ef23081 100644 --- a/conformance/chaos/actor_test.go +++ b/conformance/chaos/actor_test.go @@ -4,9 +4,7 @@ package chaos import ( "context" "testing" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" diff --git a/conformance/chaos/cbor_gen.go b/conformance/chaos/cbor_gen.go index d74ae0946a1..aefb37438f6 100644 --- a/conformance/chaos/cbor_gen.go +++ b/conformance/chaos/cbor_gen.go @@ -7,11 +7,9 @@ import ( "io" "math" "sort" - cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" - address "github.com/filecoin-project/go-address" abi "github.com/filecoin-project/go-state-types/abi" exitcode "github.com/filecoin-project/go-state-types/exitcode" diff --git a/conformance/chaos/gen/gen.go b/conformance/chaos/gen/gen.go index 20a1be64fb0..cda7204f6a3 100644 --- a/conformance/chaos/gen/gen.go +++ b/conformance/chaos/gen/gen.go @@ -2,7 +2,6 @@ package main import ( gen "github.com/whyrusleeping/cbor-gen" - "github.com/filecoin-project/lotus/conformance/chaos" ) diff --git a/conformance/chaos/ids.go b/conformance/chaos/ids.go index 478169db5f4..cb3db8b9d7f 100644 --- a/conformance/chaos/ids.go +++ b/conformance/chaos/ids.go @@ -3,7 +3,6 @@ package chaos import ( "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" - "github.com/filecoin-project/go-address" ) diff --git a/conformance/corpus_test.go b/conformance/corpus_test.go index ec9f9d51640..fd7d598141a 100644 --- a/conformance/corpus_test.go +++ b/conformance/corpus_test.go @@ -9,7 +9,6 @@ import ( "path/filepath" "strings" "testing" - "github.com/filecoin-project/test-vectors/schema" ) diff --git a/conformance/driver.go b/conformance/driver.go index 3c62ca7b9ef..78fec4a1dd3 100644 --- a/conformance/driver.go +++ b/conformance/driver.go @@ -4,11 +4,9 @@ import ( "context" gobig "math/big" "os" - "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" cbor "github.com/ipfs/go-ipld-cbor" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" @@ -17,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/conformance/rand_fixed.go b/conformance/rand_fixed.go index f35f05cd4ff..c35ed8553eb 100644 --- a/conformance/rand_fixed.go +++ b/conformance/rand_fixed.go @@ -2,9 +2,7 @@ package conformance import ( "context" - "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/rand" ) diff --git a/conformance/rand_record.go b/conformance/rand_record.go index 4dc30b28ebf..1b2f32e3466 100644 --- a/conformance/rand_record.go +++ b/conformance/rand_record.go @@ -4,10 +4,8 @@ import ( "context" "fmt" "sync" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/rand" "github.com/filecoin-project/lotus/chain/types" diff --git a/conformance/rand_replay.go b/conformance/rand_replay.go index 6d78d813b8a..0a8256d7f6a 100644 --- a/conformance/rand_replay.go +++ b/conformance/rand_replay.go @@ -2,10 +2,8 @@ package conformance import ( "context" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/chain/rand" ) diff --git a/conformance/reporter.go b/conformance/reporter.go index 1cd2d389db7..8dba374767b 100644 --- a/conformance/reporter.go +++ b/conformance/reporter.go @@ -5,7 +5,6 @@ import ( "os" "sync/atomic" "testing" - "github.com/fatih/color" ) diff --git a/conformance/runner.go b/conformance/runner.go index b470eed4e56..433a410d0b2 100644 --- a/conformance/runner.go +++ b/conformance/runner.go @@ -10,7 +10,6 @@ import ( "os" "os/exec" "strconv" - "github.com/fatih/color" "github.com/hashicorp/go-multierror" "github.com/ipfs/boxo/blockservice" @@ -21,12 +20,10 @@ import ( ds "github.com/ipfs/go-datastore" format "github.com/ipfs/go-ipld-format" "github.com/ipld/go-car" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/test-vectors/schema" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/eth_sub.go b/gateway/eth_sub.go index 76d9139835c..be13cc0319d 100644 --- a/gateway/eth_sub.go +++ b/gateway/eth_sub.go @@ -3,9 +3,7 @@ package gateway import ( "context" "sync" - "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/gateway/handler.go b/gateway/handler.go index 54ab2467f91..6405a8c7195 100644 --- a/gateway/handler.go +++ b/gateway/handler.go @@ -6,14 +6,11 @@ import ( "net/http" "sync" "time" - "contrib.go.opencensus.io/exporter/prometheus" "github.com/gorilla/mux" promclient "github.com/prometheus/client_golang/prometheus" "golang.org/x/time/rate" - "github.com/filecoin-project/go-jsonrpc" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/gateway/node.go b/gateway/node.go index 367e645c1f6..6c91b61bea0 100644 --- a/gateway/node.go +++ b/gateway/node.go @@ -4,12 +4,10 @@ import ( "context" "fmt" "time" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "go.opencensus.io/stats" "golang.org/x/time/rate" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-jsonrpc" @@ -17,7 +15,6 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/gateway/node_test.go b/gateway/node_test.go index aa33e8bfb20..55d205877fd 100644 --- a/gateway/node_test.go +++ b/gateway/node_test.go @@ -6,14 +6,11 @@ import ( "sync" "testing" "time" - "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/proxy_eth.go b/gateway/proxy_eth.go index e6d433a1744..34f1c4e3ea7 100644 --- a/gateway/proxy_eth.go +++ b/gateway/proxy_eth.go @@ -7,14 +7,11 @@ import ( "fmt" "sync" "time" - "github.com/ipfs/go-cid" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/proxy_fil.go b/gateway/proxy_fil.go index eb8a354edc5..079dc1991a8 100644 --- a/gateway/proxy_fil.go +++ b/gateway/proxy_fil.go @@ -2,11 +2,9 @@ package gateway import ( "context" - blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" @@ -14,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/genesis/types.go b/genesis/types.go index 815a2f0083a..62da56c42f5 100644 --- a/genesis/types.go +++ b/genesis/types.go @@ -2,15 +2,12 @@ package genesis import ( "encoding/json" - "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/api_test.go b/itests/api_test.go index ff43bd5c02e..583edc39837 100644 --- a/itests/api_test.go +++ b/itests/api_test.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/batch_deal_test.go b/itests/batch_deal_test.go index 21db9f08d0e..9785941c9ef 100644 --- a/itests/batch_deal_test.go +++ b/itests/batch_deal_test.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/markets/storageadapter" "github.com/filecoin-project/lotus/node" diff --git a/itests/deadlines_test.go b/itests/deadlines_test.go index fb28f450974..2a990d0ec45 100644 --- a/itests/deadlines_test.go +++ b/itests/deadlines_test.go @@ -18,8 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -30,6 +28,7 @@ import ( "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node/impl" "github.com/filecoin-project/lotus/storage/sealer/mock" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) // TestDeadlineToggling: diff --git a/itests/deals_512mb_test.go b/itests/deals_512mb_test.go index 7b55204d910..6f6d088b8ac 100644 --- a/itests/deals_512mb_test.go +++ b/itests/deals_512mb_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_anycid_test.go b/itests/deals_anycid_test.go index c17441090b1..22ef7add415 100644 --- a/itests/deals_anycid_test.go +++ b/itests/deals_anycid_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/deals_concurrent_test.go b/itests/deals_concurrent_test.go index a106836bdd1..fbf3cdb7a7d 100644 --- a/itests/deals_concurrent_test.go +++ b/itests/deals_concurrent_test.go @@ -14,7 +14,6 @@ import ( datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/deals_invalid_utf8_label_test.go b/itests/deals_invalid_utf8_label_test.go index cec6fb4c3f5..2ab84ea8f41 100644 --- a/itests/deals_invalid_utf8_label_test.go +++ b/itests/deals_invalid_utf8_label_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_max_staging_deals_test.go b/itests/deals_max_staging_deals_test.go index 738a1e2fed3..8e539c5338c 100644 --- a/itests/deals_max_staging_deals_test.go +++ b/itests/deals_max_staging_deals_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_offline_test.go b/itests/deals_offline_test.go index 997d7723aa6..021a5695c50 100644 --- a/itests/deals_offline_test.go +++ b/itests/deals_offline_test.go @@ -13,7 +13,6 @@ import ( commp "github.com/filecoin-project/go-fil-commp-hashhash" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_padding_test.go b/itests/deals_padding_test.go index aaca4536069..f9f4a9a9c3c 100644 --- a/itests/deals_padding_test.go +++ b/itests/deals_padding_test.go @@ -11,7 +11,6 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" commp "github.com/filecoin-project/go-fil-commp-hashhash" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_partial_retrieval_dm-level_test.go b/itests/deals_partial_retrieval_dm-level_test.go index c03d07aac53..a57a8773b44 100644 --- a/itests/deals_partial_retrieval_dm-level_test.go +++ b/itests/deals_partial_retrieval_dm-level_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" api0 "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/deals_partial_retrieval_test.go b/itests/deals_partial_retrieval_test.go index 0bbf23da054..ce66aba6800 100644 --- a/itests/deals_partial_retrieval_test.go +++ b/itests/deals_partial_retrieval_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_pricing_test.go b/itests/deals_pricing_test.go index f2301eee8dc..22f6973c99b 100644 --- a/itests/deals_pricing_test.go +++ b/itests/deals_pricing_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/itests/deals_publish_test.go b/itests/deals_publish_test.go index 43f4eeb0500..2c11af13448 100644 --- a/itests/deals_publish_test.go +++ b/itests/deals_publish_test.go @@ -11,8 +11,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" @@ -23,6 +21,7 @@ import ( "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules" "github.com/filecoin-project/lotus/storage/ctladdr" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" ) func TestPublishDealsBatching(t *testing.T) { diff --git a/itests/deals_remote_retrieval_test.go b/itests/deals_remote_retrieval_test.go index c0a37e69e33..1a9b4f76747 100644 --- a/itests/deals_remote_retrieval_test.go +++ b/itests/deals_remote_retrieval_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/deals_retry_deal_no_funds_test.go b/itests/deals_retry_deal_no_funds_test.go index 650b2436e0e..b75f961f2dd 100644 --- a/itests/deals_retry_deal_no_funds_test.go +++ b/itests/deals_retry_deal_no_funds_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/decode_params_test.go b/itests/decode_params_test.go index 6a4a8c681ed..812b744b077 100644 --- a/itests/decode_params_test.go +++ b/itests/decode_params_test.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/cli" diff --git a/itests/dup_mpool_messages_test.go b/itests/dup_mpool_messages_test.go index afafec3eeb3..99c692dc09b 100644 --- a/itests/dup_mpool_messages_test.go +++ b/itests/dup_mpool_messages_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_account_abstraction_test.go b/itests/eth_account_abstraction_test.go index 8d92d0a049d..c61ece49c24 100644 --- a/itests/eth_account_abstraction_test.go +++ b/itests/eth_account_abstraction_test.go @@ -14,7 +14,6 @@ import ( builtin2 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/eth_api_test.go b/itests/eth_api_test.go index 43b4b526674..b9575a36a88 100644 --- a/itests/eth_api_test.go +++ b/itests/eth_api_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_balance_test.go b/itests/eth_balance_test.go index d133ff6d90e..a75cdce6e88 100644 --- a/itests/eth_balance_test.go +++ b/itests/eth_balance_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_block_hash_test.go b/itests/eth_block_hash_test.go index b582c84e346..ef7bb13e9d2 100644 --- a/itests/eth_block_hash_test.go +++ b/itests/eth_block_hash_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/eth_conformance_test.go b/itests/eth_conformance_test.go index 15b24997719..4ae562704d7 100644 --- a/itests/eth_conformance_test.go +++ b/itests/eth_conformance_test.go @@ -21,7 +21,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_deploy_test.go b/itests/eth_deploy_test.go index ce4c94a28b8..d07d84875cb 100644 --- a/itests/eth_deploy_test.go +++ b/itests/eth_deploy_test.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" gstStore "github.com/filecoin-project/go-state-types/store" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/eth_fee_history_test.go b/itests/eth_fee_history_test.go index b611efeb18e..29381f849ca 100644 --- a/itests/eth_fee_history_test.go +++ b/itests/eth_fee_history_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/eth_filter_test.go b/itests/eth_filter_test.go index 9212e60fc01..d599fd99e84 100644 --- a/itests/eth_filter_test.go +++ b/itests/eth_filter_test.go @@ -24,7 +24,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_hash_lookup_test.go b/itests/eth_hash_lookup_test.go index 37d0697962c..b2d9d82015e 100644 --- a/itests/eth_hash_lookup_test.go +++ b/itests/eth_hash_lookup_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_transactions_test.go b/itests/eth_transactions_test.go index 6d60f678636..2f8990abace 100644 --- a/itests/eth_transactions_test.go +++ b/itests/eth_transactions_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/itests/fevm_address_test.go b/itests/fevm_address_test.go index 9eaac464773..70ca730f50b 100644 --- a/itests/fevm_address_test.go +++ b/itests/fevm_address_test.go @@ -16,7 +16,6 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/fevm_events_test.go b/itests/fevm_events_test.go index 458ac3470ab..743ef1a775d 100644 --- a/itests/fevm_events_test.go +++ b/itests/fevm_events_test.go @@ -11,7 +11,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/fevm_test.go b/itests/fevm_test.go index cb69c036cad..33f637854a6 100644 --- a/itests/fevm_test.go +++ b/itests/fevm_test.go @@ -17,7 +17,6 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/gas_estimation_test.go b/itests/gas_estimation_test.go index 24013c8855b..7bd09324404 100644 --- a/itests/gas_estimation_test.go +++ b/itests/gas_estimation_test.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/gateway_test.go b/itests/gateway_test.go index d20b3bd1a09..ef172ef6cbf 100644 --- a/itests/gateway_test.go +++ b/itests/gateway_test.go @@ -17,9 +17,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/chain/stmgr" @@ -29,6 +26,8 @@ import ( "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/itests/multisig" "github.com/filecoin-project/lotus/node" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) const ( diff --git a/itests/get_messages_in_ts_test.go b/itests/get_messages_in_ts_test.go index ecc13e3aaa7..e2ff432eb2c 100644 --- a/itests/get_messages_in_ts_test.go +++ b/itests/get_messages_in_ts_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/blockminer.go b/itests/kit/blockminer.go index bd527910d79..e387ef52118 100644 --- a/itests/kit/blockminer.go +++ b/itests/kit/blockminer.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/miner" diff --git a/itests/kit/client.go b/itests/kit/client.go index f7e4657603e..d23f43a0987 100644 --- a/itests/kit/client.go +++ b/itests/kit/client.go @@ -14,11 +14,10 @@ import ( "github.com/stretchr/testify/require" lcli "github.com/urfave/cli/v2" - "github.com/filecoin-project/specs-actors/v2/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) // RunClientTest exercises some of the Client CLI commands diff --git a/itests/kit/control.go b/itests/kit/control.go index de0cab48cb1..b8c8611d3a5 100644 --- a/itests/kit/control.go +++ b/itests/kit/control.go @@ -8,10 +8,9 @@ import ( addr "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) func (tm *TestMiner) SetControlAddresses(addrs ...addr.Address) { diff --git a/itests/kit/deals.go b/itests/kit/deals.go index eb6b58667dc..bf59297ae22 100644 --- a/itests/kit/deals.go +++ b/itests/kit/deals.go @@ -26,7 +26,6 @@ import ( "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/ensemble.go b/itests/kit/ensemble.go index daca6d2d4e5..a75eb25ab55 100644 --- a/itests/kit/ensemble.go +++ b/itests/kit/ensemble.go @@ -29,9 +29,6 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statestore" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" @@ -65,6 +62,8 @@ import ( sectorstorage "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/mock" "github.com/filecoin-project/lotus/storage/sealer/storiface" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" ) func init() { diff --git a/itests/kit/ensemble_opts.go b/itests/kit/ensemble_opts.go index d264da2bb25..0d14a9b7950 100644 --- a/itests/kit/ensemble_opts.go +++ b/itests/kit/ensemble_opts.go @@ -4,7 +4,6 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/ensemble_opts_nv.go b/itests/kit/ensemble_opts_nv.go index d5bb1930ef0..62b4725edc8 100644 --- a/itests/kit/ensemble_opts_nv.go +++ b/itests/kit/ensemble_opts_nv.go @@ -3,7 +3,6 @@ package kit import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" ) diff --git a/itests/kit/evm.go b/itests/kit/evm.go index 99844ca3097..734404f695e 100644 --- a/itests/kit/evm.go +++ b/itests/kit/evm.go @@ -26,7 +26,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/kit/funds.go b/itests/kit/funds.go index 1918d9125ff..e554bbf27bb 100644 --- a/itests/kit/funds.go +++ b/itests/kit/funds.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/kit/init.go b/itests/kit/init.go index 9397c53a218..109234769f3 100644 --- a/itests/kit/init.go +++ b/itests/kit/init.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/itests/kit/node_full.go b/itests/kit/node_full.go index 3e80ed68869..21348f17131 100644 --- a/itests/kit/node_full.go +++ b/itests/kit/node_full.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/node_miner.go b/itests/kit/node_miner.go index 4b81c9df0bd..94802312547 100644 --- a/itests/kit/node_miner.go +++ b/itests/kit/node_miner.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/node_opts.go b/itests/kit/node_opts.go index 6469c0a3076..40c32aa8519 100644 --- a/itests/kit/node_opts.go +++ b/itests/kit/node_opts.go @@ -3,7 +3,6 @@ package kit import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/rpc.go b/itests/kit/rpc.go index 6b63eb1ebf0..cd1cdd0fd35 100644 --- a/itests/kit/rpc.go +++ b/itests/kit/rpc.go @@ -14,7 +14,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/cmd/lotus-worker/sealworker" "github.com/filecoin-project/lotus/node" diff --git a/itests/kit/state.go b/itests/kit/state.go index e66576be393..195ca9f5a00 100644 --- a/itests/kit/state.go +++ b/itests/kit/state.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" - "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/lite_migration_test.go b/itests/lite_migration_test.go index 0f846e6fa10..6c78bc2766c 100644 --- a/itests/lite_migration_test.go +++ b/itests/lite_migration_test.go @@ -16,8 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" gstStore "github.com/filecoin-project/go-state-types/store" - "github.com/filecoin-project/specs-actors/v8/actors/util/adt" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin/system" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -25,6 +23,7 @@ import ( "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" + "github.com/filecoin-project/specs-actors/v8/actors/util/adt" ) func TestLiteMigration(t *testing.T) { diff --git a/itests/lookup_robust_address_test.go b/itests/lookup_robust_address_test.go index 536d8feef36..2ad60602b16 100644 --- a/itests/lookup_robust_address_test.go +++ b/itests/lookup_robust_address_test.go @@ -8,7 +8,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/mempool_test.go b/itests/mempool_test.go index f07b46a737c..d95319e5b06 100644 --- a/itests/mempool_test.go +++ b/itests/mempool_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/migration_test.go b/itests/migration_test.go index 68991a579a9..c6dbf2d6a4c 100644 --- a/itests/migration_test.go +++ b/itests/migration_test.go @@ -25,7 +25,6 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" gstStore "github.com/filecoin-project/go-state-types/store" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/mpool_msg_uuid_test.go b/itests/mpool_msg_uuid_test.go index 3eb30a6f6eb..34d34bc20f9 100644 --- a/itests/mpool_msg_uuid_test.go +++ b/itests/mpool_msg_uuid_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/mpool_push_with_uuid_test.go b/itests/mpool_push_with_uuid_test.go index 6b94dbad191..2edefb40050 100644 --- a/itests/mpool_push_with_uuid_test.go +++ b/itests/mpool_push_with_uuid_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/msgindex_test.go b/itests/msgindex_test.go index 807ab3c03f0..afebfe56b12 100644 --- a/itests/msgindex_test.go +++ b/itests/msgindex_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" diff --git a/itests/multisig/suite.go b/itests/multisig/suite.go index 9a81d0bf99d..1e77604bb66 100644 --- a/itests/multisig/suite.go +++ b/itests/multisig/suite.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cli" diff --git a/itests/multisig_test.go b/itests/multisig_test.go index 92d9afca7dd..9ebc33f8fac 100644 --- a/itests/multisig_test.go +++ b/itests/multisig_test.go @@ -16,7 +16,6 @@ import ( inittypes "github.com/filecoin-project/go-state-types/builtin/v8/init" multisigtypes "github.com/filecoin-project/go-state-types/builtin/v8/multisig" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/itests/nonce_test.go b/itests/nonce_test.go index cac2413f73c..0ebff1e44c6 100644 --- a/itests/nonce_test.go +++ b/itests/nonce_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/path_detach_redeclare_test.go b/itests/path_detach_redeclare_test.go index 10774d01218..101b7617942 100644 --- a/itests/path_detach_redeclare_test.go +++ b/itests/path_detach_redeclare_test.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/itests/paych_api_test.go b/itests/paych_api_test.go index f8ecb5377da..daa8fd54310 100644 --- a/itests/paych_api_test.go +++ b/itests/paych_api_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/paych_cli_test.go b/itests/paych_cli_test.go index f86f5d8deca..fa68e7c96ac 100644 --- a/itests/paych_cli_test.go +++ b/itests/paych_cli_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/pending_deal_allocation_test.go b/itests/pending_deal_allocation_test.go index c1e0531cfeb..7c600296e9f 100644 --- a/itests/pending_deal_allocation_test.go +++ b/itests/pending_deal_allocation_test.go @@ -18,7 +18,6 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/raft_messagesigner_test.go b/itests/raft_messagesigner_test.go index 220da96996b..b822b785646 100644 --- a/itests/raft_messagesigner_test.go +++ b/itests/raft_messagesigner_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/remove_verifreg_datacap_test.go b/itests/remove_verifreg_datacap_test.go index ac88574d2b6..82664a0b26b 100644 --- a/itests/remove_verifreg_datacap_test.go +++ b/itests/remove_verifreg_datacap_test.go @@ -16,7 +16,6 @@ import ( migration "github.com/filecoin-project/go-state-types/builtin/v9/migration/test" verifregst "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/sector_import_full_test.go b/itests/sector_import_full_test.go index c9bd96afde1..a2143d316e7 100644 --- a/itests/sector_import_full_test.go +++ b/itests/sector_import_full_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/itests/sector_import_simple_test.go b/itests/sector_import_simple_test.go index fb1a77a1410..173d436d6c4 100644 --- a/itests/sector_import_simple_test.go +++ b/itests/sector_import_simple_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/itests/sector_miner_collateral_test.go b/itests/sector_miner_collateral_test.go index 579b4e535a1..1b86840083e 100644 --- a/itests/sector_miner_collateral_test.go +++ b/itests/sector_miner_collateral_test.go @@ -10,7 +10,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/sector_numassign_test.go b/itests/sector_numassign_test.go index af667aa9213..436d070a9ed 100644 --- a/itests/sector_numassign_test.go +++ b/itests/sector_numassign_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-bitfield" rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/lib/strle" ) diff --git a/itests/sector_pledge_test.go b/itests/sector_pledge_test.go index 1e045c79d7d..6156c858cbc 100644 --- a/itests/sector_pledge_test.go +++ b/itests/sector_pledge_test.go @@ -12,8 +12,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" @@ -21,6 +19,7 @@ import ( "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/impl" sealing "github.com/filecoin-project/lotus/storage/pipeline" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) func TestPledgeSectors(t *testing.T) { diff --git a/itests/sector_terminate_test.go b/itests/sector_terminate_test.go index 34b325f2ad1..226e60f1ec7 100644 --- a/itests/sector_terminate_test.go +++ b/itests/sector_terminate_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" sealing "github.com/filecoin-project/lotus/storage/pipeline" diff --git a/itests/sector_unseal_test.go b/itests/sector_unseal_test.go index 5d05cb0e33c..f049e64f166 100644 --- a/itests/sector_unseal_test.go +++ b/itests/sector_unseal_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/itests/self_sent_txn_test.go b/itests/self_sent_txn_test.go index 00ca0e5aeeb..81aafbacd51 100644 --- a/itests/self_sent_txn_test.go +++ b/itests/self_sent_txn_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/splitstore_test.go b/itests/splitstore_test.go index ea59faf2a9f..aba52b59de7 100644 --- a/itests/splitstore_test.go +++ b/itests/splitstore_test.go @@ -19,9 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" miner8 "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" - "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore/splitstore" @@ -30,6 +27,8 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" ) // Startup a node with hotstore and discard coldstore. Compact once and return diff --git a/itests/verifreg_test.go b/itests/verifreg_test.go index ffe50c72b19..2fe7144cb9d 100644 --- a/itests/verifreg_test.go +++ b/itests/verifreg_test.go @@ -19,7 +19,6 @@ import ( datacap2 "github.com/filecoin-project/go-state-types/builtin/v9/datacap" verifregst "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" - lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/datacap" diff --git a/itests/wdpost_config_test.go b/itests/wdpost_config_test.go index 984650ae6a6..4261c41702e 100644 --- a/itests/wdpost_config_test.go +++ b/itests/wdpost_config_test.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/wdpost_dispute_test.go b/itests/wdpost_dispute_test.go index c4512874aaa..c090aa0fc45 100644 --- a/itests/wdpost_dispute_test.go +++ b/itests/wdpost_dispute_test.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/wdpost_test.go b/itests/wdpost_test.go index 2a6fc866ea8..f4676ca9ee8 100644 --- a/itests/wdpost_test.go +++ b/itests/wdpost_test.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/wdpost_worker_config_test.go b/itests/wdpost_worker_config_test.go index d1672c20f08..35528528f1c 100644 --- a/itests/wdpost_worker_config_test.go +++ b/itests/wdpost_worker_config_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/worker_test.go b/itests/worker_test.go index c4f885fb08f..51d21f52e0a 100644 --- a/itests/worker_test.go +++ b/itests/worker_test.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/lib/consensus/raft/consensus.go b/lib/consensus/raft/consensus.go index d74f200fab1..4dcdf6b1bae 100644 --- a/lib/consensus/raft/consensus.go +++ b/lib/consensus/raft/consensus.go @@ -14,7 +14,6 @@ import ( "golang.org/x/exp/slices" addr "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/lib/rpcenc/reader.go b/lib/rpcenc/reader.go index 2dd64473e7e..2776b6c7b99 100644 --- a/lib/rpcenc/reader.go +++ b/lib/rpcenc/reader.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/lib/httpreader" "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" ) diff --git a/lib/rpcenc/reader_test.go b/lib/rpcenc/reader_test.go index 3a554a0ca1e..48e1a0b37ae 100644 --- a/lib/rpcenc/reader_test.go +++ b/lib/rpcenc/reader_test.go @@ -16,7 +16,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" ) diff --git a/lib/sigs/bls/bls_test.go b/lib/sigs/bls/bls_test.go index 7a290b8b79f..204b972a8e9 100644 --- a/lib/sigs/bls/bls_test.go +++ b/lib/sigs/bls/bls_test.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/lib/sigs/bls/init.go b/lib/sigs/bls/init.go index f64912fa8ee..d784053f5e4 100644 --- a/lib/sigs/bls/init.go +++ b/lib/sigs/bls/init.go @@ -7,7 +7,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/delegated/init.go b/lib/sigs/delegated/init.go index 81886ceaad1..f883cdd5ad1 100644 --- a/lib/sigs/delegated/init.go +++ b/lib/sigs/delegated/init.go @@ -9,7 +9,6 @@ import ( gocrypto "github.com/filecoin-project/go-crypto" "github.com/filecoin-project/go-state-types/builtin" crypto1 "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/secp/init.go b/lib/sigs/secp/init.go index 49d5cef172a..63a8f716b99 100644 --- a/lib/sigs/secp/init.go +++ b/lib/sigs/secp/init.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-address" crypto "github.com/filecoin-project/go-crypto" crypto2 "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/sigs.go b/lib/sigs/sigs.go index 7792fc02364..29c83cf6314 100644 --- a/lib/sigs/sigs.go +++ b/lib/sigs/sigs.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/lib/unixfs/filestore.go b/lib/unixfs/filestore.go index f50e933b68f..f3d3817bad2 100644 --- a/lib/unixfs/filestore.go +++ b/lib/unixfs/filestore.go @@ -21,7 +21,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/stores" - "github.com/filecoin-project/lotus/build" ) diff --git a/markets/dagstore/mount_test.go b/markets/dagstore/mount_test.go index d415f8d8856..7929d1a89be 100644 --- a/markets/dagstore/mount_test.go +++ b/markets/dagstore/mount_test.go @@ -13,7 +13,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/dagstore/mount" - mock_dagstore "github.com/filecoin-project/lotus/markets/dagstore/mocks" ) diff --git a/markets/dagstore/wrapper.go b/markets/dagstore/wrapper.go index a929ad1fc93..037edd1f8ca 100644 --- a/markets/dagstore/wrapper.go +++ b/markets/dagstore/wrapper.go @@ -28,7 +28,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket/impl/providerstates" "github.com/filecoin-project/go-fil-markets/stores" "github.com/filecoin-project/go-statemachine/fsm" - "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dagstore/wrapper_migration_test.go b/markets/dagstore/wrapper_migration_test.go index db2c9768b1a..854f7087201 100644 --- a/markets/dagstore/wrapper_migration_test.go +++ b/markets/dagstore/wrapper_migration_test.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dagstore/wrapper_test.go b/markets/dagstore/wrapper_test.go index f3b5e1b52c0..772d89fff32 100644 --- a/markets/dagstore/wrapper_test.go +++ b/markets/dagstore/wrapper_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/dagstore" "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/dagstore/shard" - "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dealfilter/cli.go b/markets/dealfilter/cli.go index af832bfa08a..17f0034a700 100644 --- a/markets/dealfilter/cli.go +++ b/markets/dealfilter/cli.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/markets/journal.go b/markets/journal.go index 9c9c5be9c42..f74d89e7bb4 100644 --- a/markets/journal.go +++ b/markets/journal.go @@ -3,7 +3,6 @@ package markets import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" - "github.com/filecoin-project/lotus/journal" ) diff --git a/markets/pricing/cli.go b/markets/pricing/cli.go index 48f56628fae..a58396186a1 100644 --- a/markets/pricing/cli.go +++ b/markets/pricing/cli.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/retrievalmarket" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/markets/retrievaladapter/client.go b/markets/retrievaladapter/client.go index 34bc2489624..451b75cad16 100644 --- a/markets/retrievaladapter/client.go +++ b/markets/retrievaladapter/client.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-fil-markets/shared" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/markets/retrievaladapter/client_blockstore.go b/markets/retrievaladapter/client_blockstore.go index 30fc5c73a5b..5bc743de8cc 100644 --- a/markets/retrievaladapter/client_blockstore.go +++ b/markets/retrievaladapter/client_blockstore.go @@ -11,7 +11,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/retrievalmarket" - "github.com/filecoin-project/lotus/api" lbstore "github.com/filecoin-project/lotus/blockstore" ) diff --git a/markets/retrievaladapter/provider.go b/markets/retrievaladapter/provider.go index 453474d4e6e..584089356b1 100644 --- a/markets/retrievaladapter/provider.go +++ b/markets/retrievaladapter/provider.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-fil-markets/shared" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/markets/retrievaladapter/provider_test.go b/markets/retrievaladapter/provider_test.go index b7b5039d674..249af09459d 100644 --- a/markets/retrievaladapter/provider_test.go +++ b/markets/retrievaladapter/provider_test.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" testnet "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/markets/sectoraccessor/sectoraccessor.go b/markets/sectoraccessor/sectoraccessor.go index 9b709d3b5ff..9f94bf85938 100644 --- a/markets/sectoraccessor/sectoraccessor.go +++ b/markets/sectoraccessor/sectoraccessor.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" diff --git a/markets/storageadapter/api.go b/markets/storageadapter/api.go index b93ffdfbb16..75ad7e1fa11 100644 --- a/markets/storageadapter/api.go +++ b/markets/storageadapter/api.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/markets/storageadapter/client.go b/markets/storageadapter/client.go index eaff4e166a3..f84ee8ef96f 100644 --- a/markets/storageadapter/client.go +++ b/markets/storageadapter/client.go @@ -19,8 +19,6 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" marketactor "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -32,6 +30,7 @@ import ( "github.com/filecoin-project/lotus/markets/utils" "github.com/filecoin-project/lotus/node/impl/full" "github.com/filecoin-project/lotus/node/modules/helpers" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" ) type ClientNodeAdapter struct { diff --git a/markets/storageadapter/client_blockstore.go b/markets/storageadapter/client_blockstore.go index dc7e3f82a62..3ab8cecbb63 100644 --- a/markets/storageadapter/client_blockstore.go +++ b/markets/storageadapter/client_blockstore.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-fil-markets/stores" - "github.com/filecoin-project/lotus/node/repo/imports" ) diff --git a/markets/storageadapter/dealpublisher.go b/markets/storageadapter/dealpublisher.go index 6a274e593f4..188964f82bb 100644 --- a/markets/storageadapter/dealpublisher.go +++ b/markets/storageadapter/dealpublisher.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/markets/storageadapter/dealpublisher_test.go b/markets/storageadapter/dealpublisher_test.go index 35169bf41b9..7a6d2a7d638 100644 --- a/markets/storageadapter/dealpublisher_test.go +++ b/markets/storageadapter/dealpublisher_test.go @@ -17,12 +17,11 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) func TestDealPublisher(t *testing.T) { diff --git a/markets/storageadapter/dealstatematcher.go b/markets/storageadapter/dealstatematcher.go index 8d5598eae01..b8b47ef8e5a 100644 --- a/markets/storageadapter/dealstatematcher.go +++ b/markets/storageadapter/dealstatematcher.go @@ -5,7 +5,6 @@ import ( "sync" "github.com/filecoin-project/go-state-types/abi" - actorsmarket "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/state" diff --git a/markets/storageadapter/dealstatematcher_test.go b/markets/storageadapter/dealstatematcher_test.go index 9a46e4af917..9409f3dc782 100644 --- a/markets/storageadapter/dealstatematcher_test.go +++ b/markets/storageadapter/dealstatematcher_test.go @@ -12,15 +12,14 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" - bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/state" test "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) func TestDealStateMatcher(t *testing.T) { diff --git a/markets/storageadapter/ondealsectorcommitted.go b/markets/storageadapter/ondealsectorcommitted.go index 54ddb73b334..23ef371b51b 100644 --- a/markets/storageadapter/ondealsectorcommitted.go +++ b/markets/storageadapter/ondealsectorcommitted.go @@ -16,7 +16,6 @@ import ( miner2 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/build" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/events" diff --git a/markets/storageadapter/ondealsectorcommitted_test.go b/markets/storageadapter/ondealsectorcommitted_test.go index 1d7519ff9f9..f7965c721fc 100644 --- a/markets/storageadapter/ondealsectorcommitted_test.go +++ b/markets/storageadapter/ondealsectorcommitted_test.go @@ -21,8 +21,6 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/cbor" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -30,6 +28,7 @@ import ( test "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" pipeline "github.com/filecoin-project/lotus/storage/pipeline" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) func TestOnDealSectorPreCommitted(t *testing.T) { diff --git a/markets/storageadapter/provider.go b/markets/storageadapter/provider.go index bdfce6f55af..baa55f28d7e 100644 --- a/markets/storageadapter/provider.go +++ b/markets/storageadapter/provider.go @@ -18,7 +18,6 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/markets/utils/converters.go b/markets/utils/converters.go index 9562de695fc..ca360e44c12 100644 --- a/markets/utils/converters.go +++ b/markets/utils/converters.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" ) diff --git a/metrics/metrics.go b/metrics/metrics.go index 0ea2c841df2..31e119811fc 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -9,7 +9,6 @@ import ( "go.opencensus.io/tag" rpcmetrics "github.com/filecoin-project/go-jsonrpc/metrics" - "github.com/filecoin-project/lotus/blockstore" ) diff --git a/miner/miner.go b/miner/miner.go index d11e9d4aa04..1fbcc41ab91 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -9,19 +9,16 @@ import ( "os" "sync" "time" - "github.com/hashicorp/golang-lru/arc/v2" "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "go.opencensus.io/trace" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/miner/testminer.go b/miner/testminer.go index e23b26ae29f..931816e2f7d 100644 --- a/miner/testminer.go +++ b/miner/testminer.go @@ -2,13 +2,10 @@ package miner import ( "context" - "github.com/hashicorp/golang-lru/arc/v2" ds "github.com/ipfs/go-datastore" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/gen/slashfilter" diff --git a/miner/warmup.go b/miner/warmup.go index 46236784a72..05fb7479034 100644 --- a/miner/warmup.go +++ b/miner/warmup.go @@ -5,13 +5,10 @@ import ( "crypto/rand" "math" "time" - "golang.org/x/xerrors" - "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/builder.go b/node/builder.go index 128a99f8714..866d6c3faf7 100644 --- a/node/builder.go +++ b/node/builder.go @@ -4,7 +4,6 @@ import ( "context" "errors" "time" - logging "github.com/ipfs/go-log/v2" metricsi "github.com/ipfs/go-metrics-interface" dht "github.com/libp2p/go-libp2p-kad-dht" @@ -20,7 +19,6 @@ import ( "github.com/multiformats/go-multiaddr" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/builder_chain.go b/node/builder_chain.go index 267659f0091..960a65b3831 100644 --- a/node/builder_chain.go +++ b/node/builder_chain.go @@ -2,16 +2,13 @@ package node import ( "os" - gorpc "github.com/libp2p/go-libp2p-gorpc" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-fil-markets/discovery" discoveryimpl "github.com/filecoin-project/go-fil-markets/discovery/impl" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/builder_miner.go b/node/builder_miner.go index dd35c6becf9..47855ea1fd1 100644 --- a/node/builder_miner.go +++ b/node/builder_miner.go @@ -3,17 +3,14 @@ package node import ( "errors" "time" - provider "github.com/ipni/index-provider" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-fil-markets/retrievalmarket" rmnet "github.com/filecoin-project/go-fil-markets/retrievalmarket/network" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-fil-markets/storagemarket/impl/storedask" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/node/bundle/bundle.go b/node/bundle/bundle.go index 716c9043bb6..1d1306e490d 100644 --- a/node/bundle/bundle.go +++ b/node/bundle/bundle.go @@ -6,13 +6,10 @@ import ( "fmt" "io" "os" - "github.com/ipfs/go-cid" "github.com/ipld/go-car" "golang.org/x/xerrors" - actorstypes "github.com/filecoin-project/go-state-types/actors" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/config/def.go b/node/config/def.go index 746c963fd07..619adb6021e 100644 --- a/node/config/def.go +++ b/node/config/def.go @@ -5,14 +5,11 @@ import ( "os" "strconv" "time" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/config/def_test.go b/node/config/def_test.go index 627b65a5631..5bc4eef83c6 100644 --- a/node/config/def_test.go +++ b/node/config/def_test.go @@ -7,7 +7,6 @@ import ( "reflect" "strings" "testing" - "github.com/BurntSushi/toml" "github.com/stretchr/testify/require" ) diff --git a/node/config/doc_gen.go b/node/config/doc_gen.go index 9e4b6c7b3b1..fb6c9956c96 100644 --- a/node/config/doc_gen.go +++ b/node/config/doc_gen.go @@ -31,10 +31,10 @@ var Doc = map[string][]DocField{ }, "ApisConfig": { { - Name: "FULLNODE_API_INFO", + Name: "ChainApiInfo", Type: "[]string", - Comment: `FULLNODE_API_INFO is the API endpoint for the Lotus daemon.`, + Comment: `ChainApiInfo is the API endpoint for the Lotus daemon.`, }, { Name: "StorageRPCSecret", diff --git a/node/config/load.go b/node/config/load.go index fd015d53328..05f75158651 100644 --- a/node/config/load.go +++ b/node/config/load.go @@ -9,7 +9,6 @@ import ( "regexp" "strings" "unicode" - "github.com/BurntSushi/toml" "github.com/kelseyhightower/envconfig" "golang.org/x/xerrors" diff --git a/node/config/load_test.go b/node/config/load_test.go index e17660c19f9..d7cfa863ca4 100644 --- a/node/config/load_test.go +++ b/node/config/load_test.go @@ -6,7 +6,6 @@ import ( "os" "testing" "time" - "github.com/stretchr/testify/assert" ) diff --git a/node/config/storage.go b/node/config/storage.go index ac5d57de8b4..d46c30f0c9d 100644 --- a/node/config/storage.go +++ b/node/config/storage.go @@ -7,9 +7,7 @@ import ( "io/fs" "os" "path" - "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/node/config/types.go b/node/config/types.go index 2152e079569..a81651f7101 100644 --- a/node/config/types.go +++ b/node/config/types.go @@ -2,7 +2,6 @@ package config import ( "github.com/ipfs/go-cid" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/health.go b/node/health.go index 1be11921c0c..80c556c44fe 100644 --- a/node/health.go +++ b/node/health.go @@ -5,10 +5,8 @@ import ( "net/http" "sync/atomic" "time" - logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/network" - lapi "github.com/filecoin-project/lotus/api" ) diff --git a/node/hello/cbor_gen.go b/node/hello/cbor_gen.go index 78e950f6f6e..cd3d7bb171d 100644 --- a/node/hello/cbor_gen.go +++ b/node/hello/cbor_gen.go @@ -7,11 +7,9 @@ import ( "io" "math" "sort" - cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" - abi "github.com/filecoin-project/go-state-types/abi" ) diff --git a/node/hello/hello.go b/node/hello/hello.go index e05b8a48287..1f6f3cfd8d2 100644 --- a/node/hello/hello.go +++ b/node/hello/hello.go @@ -3,7 +3,6 @@ package hello import ( "context" "time" - "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/host" @@ -11,11 +10,9 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "golang.org/x/xerrors" - cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/node/impl/backup.go b/node/impl/backup.go index 7acc7e01877..091cdb4cf38 100644 --- a/node/impl/backup.go +++ b/node/impl/backup.go @@ -5,10 +5,8 @@ import ( "os" "path/filepath" "strings" - "github.com/mitchellh/go-homedir" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/node/impl/client/car_helpers.go b/node/impl/client/car_helpers.go index c638b4bef81..26d08dfcb3a 100644 --- a/node/impl/client/car_helpers.go +++ b/node/impl/client/car_helpers.go @@ -3,7 +3,6 @@ package client import ( "fmt" "io" - "github.com/ipfs/go-cid" cbor "github.com/ipfs/go-ipld-cbor" "github.com/ipld/go-car/util" diff --git a/node/impl/client/client.go b/node/impl/client/client.go index c7bb252a10a..8d3b37e446c 100644 --- a/node/impl/client/client.go +++ b/node/impl/client/client.go @@ -12,7 +12,6 @@ import ( "strings" "sync" "time" - "github.com/ipfs/boxo/blockservice" bstore "github.com/ipfs/boxo/blockstore" offline "github.com/ipfs/boxo/exchange/offline" @@ -40,7 +39,6 @@ import ( "github.com/multiformats/go-multibase" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-commp-utils/writer" @@ -55,7 +53,6 @@ import ( "github.com/filecoin-project/go-state-types/big" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/node/impl/client/client_test.go b/node/impl/client/client_test.go index 67a35013166..f19d6705308 100644 --- a/node/impl/client/client_test.go +++ b/node/impl/client/client_test.go @@ -9,7 +9,6 @@ import ( "path/filepath" "strings" "testing" - "github.com/ipfs/boxo/blockservice" blockstore "github.com/ipfs/boxo/blockstore" offline "github.com/ipfs/boxo/exchange/offline" @@ -22,7 +21,6 @@ import ( "github.com/ipld/go-car" carv2 "github.com/ipld/go-car/v2" "github.com/stretchr/testify/require" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/markets/storageadapter" "github.com/filecoin-project/lotus/node/repo/imports" diff --git a/node/impl/common/common.go b/node/impl/common/common.go index eff6b58b8f1..83692787442 100644 --- a/node/impl/common/common.go +++ b/node/impl/common/common.go @@ -3,15 +3,12 @@ package common import ( "context" "time" - "github.com/gbrlsnchs/jwt/v3" "github.com/google/uuid" logging "github.com/ipfs/go-log/v2" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/node/impl/full.go b/node/impl/full.go index affcc960e09..6f1163e00de 100644 --- a/node/impl/full.go +++ b/node/impl/full.go @@ -3,10 +3,8 @@ package impl import ( "context" "time" - logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/peer" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/impl/client" diff --git a/node/impl/full/chain.go b/node/impl/full/chain.go index 1d6b8e566fa..5bbdd95744c 100644 --- a/node/impl/full/chain.go +++ b/node/impl/full/chain.go @@ -14,7 +14,6 @@ import ( "strings" "sync" "time" - "github.com/ipfs/boxo/blockservice" offline "github.com/ipfs/boxo/exchange/offline" "github.com/ipfs/boxo/ipld/merkledag" @@ -27,12 +26,10 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" amt4 "github.com/filecoin-project/go-amt-ipld/v4" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/specs-actors/actors/util/adt" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/impl/full/dummy.go b/node/impl/full/dummy.go index 743eadf34dd..e07088b43d0 100644 --- a/node/impl/full/dummy.go +++ b/node/impl/full/dummy.go @@ -3,13 +3,10 @@ package full import ( "context" "errors" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/node/impl/full/eth.go b/node/impl/full/eth.go index 6b8b0e0aaef..0f3df3b1841 100644 --- a/node/impl/full/eth.go +++ b/node/impl/full/eth.go @@ -10,12 +10,10 @@ import ( "strconv" "strings" "time" - "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" @@ -24,7 +22,6 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/evm" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/eth_event.go b/node/impl/full/eth_event.go index 69021e08aed..3cfc6312e0c 100644 --- a/node/impl/full/eth_event.go +++ b/node/impl/full/eth_event.go @@ -4,14 +4,11 @@ import ( "context" "encoding/json" "sync" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/zyedidia/generic/queue" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc" - "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/impl/full/eth_test.go b/node/impl/full/eth_test.go index c364a4873c4..3a8db08aa1b 100644 --- a/node/impl/full/eth_test.go +++ b/node/impl/full/eth_test.go @@ -3,12 +3,9 @@ package full import ( "encoding/hex" "testing" - "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/node/impl/full/eth_trace.go b/node/impl/full/eth_trace.go index fd5c25566cb..906287d861a 100644 --- a/node/impl/full/eth_trace.go +++ b/node/impl/full/eth_trace.go @@ -3,14 +3,11 @@ package full import ( "bytes" "context" - "github.com/multiformats/go-multicodec" cbg "github.com/whyrusleeping/cbor-gen" "golang.org/x/xerrors" - "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/evm" - builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/node/impl/full/eth_utils.go b/node/impl/full/eth_utils.go index 2799638ddfb..88d6f5a0fbe 100644 --- a/node/impl/full/eth_utils.go +++ b/node/impl/full/eth_utils.go @@ -6,10 +6,8 @@ import ( "encoding/binary" "errors" "fmt" - "github.com/ipfs/go-cid" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" @@ -17,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/gas.go b/node/impl/full/gas.go index c5b22354a52..1bb0ac50fa0 100644 --- a/node/impl/full/gas.go +++ b/node/impl/full/gas.go @@ -6,17 +6,14 @@ import ( "math/rand" "os" "sort" - lru "github.com/hashicorp/golang-lru/v2" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/node/impl/full/gas_test.go b/node/impl/full/gas_test.go index 8fc585bd544..b5218a6b664 100644 --- a/node/impl/full/gas_test.go +++ b/node/impl/full/gas_test.go @@ -3,11 +3,8 @@ package full import ( "testing" - "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/impl/full/mpool.go b/node/impl/full/mpool.go index addcc41be43..a656c3a9ec2 100644 --- a/node/impl/full/mpool.go +++ b/node/impl/full/mpool.go @@ -3,15 +3,12 @@ package full import ( "context" "encoding/json" - "github.com/google/uuid" "github.com/ipfs/go-cid" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/messagesigner" diff --git a/node/impl/full/multisig.go b/node/impl/full/multisig.go index 886d66d5e4a..e41147186ae 100644 --- a/node/impl/full/multisig.go +++ b/node/impl/full/multisig.go @@ -2,16 +2,13 @@ package full import ( "context" - "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/node/impl/full/raft.go b/node/impl/full/raft.go index 8d665ddd502..6f71414f8fe 100644 --- a/node/impl/full/raft.go +++ b/node/impl/full/raft.go @@ -2,11 +2,9 @@ package full import ( "context" - "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" ) diff --git a/node/impl/full/state.go b/node/impl/full/state.go index 0e92c8e5b6f..e632d755e06 100644 --- a/node/impl/full/state.go +++ b/node/impl/full/state.go @@ -8,13 +8,11 @@ import ( "fmt" "math" "strconv" - "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" @@ -27,7 +25,6 @@ import ( "github.com/filecoin-project/go-state-types/network" market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/sync.go b/node/impl/full/sync.go index 4bf44363ce0..4f25ad03a7a 100644 --- a/node/impl/full/sync.go +++ b/node/impl/full/sync.go @@ -4,12 +4,10 @@ import ( "context" "os" "sync/atomic" - "github.com/ipfs/go-cid" pubsub "github.com/libp2p/go-libp2p-pubsub" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" diff --git a/node/impl/full/txhashmanager.go b/node/impl/full/txhashmanager.go index 6757cc6dd92..4c150bea43c 100644 --- a/node/impl/full/txhashmanager.go +++ b/node/impl/full/txhashmanager.go @@ -3,10 +3,8 @@ package full import ( "context" "time" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/ethhashlookup" diff --git a/node/impl/full/wallet.go b/node/impl/full/wallet.go index fdf00e0864f..d217e58e422 100644 --- a/node/impl/full/wallet.go +++ b/node/impl/full/wallet.go @@ -2,14 +2,11 @@ package full import ( "context" - "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/impl/market/market.go b/node/impl/market/market.go index 26ea9c14ca7..0a62cafeece 100644 --- a/node/impl/market/market.go +++ b/node/impl/market/market.go @@ -2,12 +2,9 @@ package market import ( "context" - "github.com/ipfs/go-cid" "go.uber.org/fx" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/actors" marketactor "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/market" diff --git a/node/impl/net/conngater.go b/node/impl/net/conngater.go index f95b60b48ac..13b82b78c7a 100644 --- a/node/impl/net/conngater.go +++ b/node/impl/net/conngater.go @@ -3,11 +3,9 @@ package net import ( "context" "net" - logging "github.com/ipfs/go-log/v2" manet "github.com/multiformats/go-multiaddr/net" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" ) diff --git a/node/impl/net/net.go b/node/impl/net/net.go index 5341092ce28..db9caafb3fc 100644 --- a/node/impl/net/net.go +++ b/node/impl/net/net.go @@ -5,7 +5,6 @@ import ( "sort" "strings" "time" - "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/metrics" "github.com/libp2p/go-libp2p/core/network" @@ -19,7 +18,6 @@ import ( manet "github.com/multiformats/go-multiaddr/net" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/node/modules/lp2p" diff --git a/node/impl/net/protect.go b/node/impl/net/protect.go index c6c864a687c..4f5be997342 100644 --- a/node/impl/net/protect.go +++ b/node/impl/net/protect.go @@ -2,7 +2,6 @@ package net import ( "context" - "github.com/libp2p/go-libp2p/core/peer" ) diff --git a/node/impl/net/rcmgr.go b/node/impl/net/rcmgr.go index c606aabc65b..2157dafd5c6 100644 --- a/node/impl/net/rcmgr.go +++ b/node/impl/net/rcmgr.go @@ -3,13 +3,11 @@ package net import ( "context" "strings" - "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" ) diff --git a/node/impl/paych/paych.go b/node/impl/paych/paych.go index b7d40c1e959..ede872fe850 100644 --- a/node/impl/paych/paych.go +++ b/node/impl/paych/paych.go @@ -2,14 +2,11 @@ package paych import ( "context" - "github.com/ipfs/go-cid" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/paychmgr" diff --git a/node/impl/remoteworker.go b/node/impl/remoteworker.go index b3e68869d94..4ba7c9b66ab 100644 --- a/node/impl/remoteworker.go +++ b/node/impl/remoteworker.go @@ -3,13 +3,10 @@ package impl import ( "context" "net/http" - "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/storage/sealer" diff --git a/node/impl/storminer.go b/node/impl/storminer.go index a087e084f50..a9ff73d57e1 100644 --- a/node/impl/storminer.go +++ b/node/impl/storminer.go @@ -10,7 +10,6 @@ import ( "sort" "strconv" "time" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-graphsync" @@ -20,7 +19,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/dagstore" "github.com/filecoin-project/dagstore/shard" "github.com/filecoin-project/go-address" @@ -36,7 +34,6 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/node/modules/actorevent.go b/node/modules/actorevent.go index 4ce04cefd07..11ae4609e97 100644 --- a/node/modules/actorevent.go +++ b/node/modules/actorevent.go @@ -4,14 +4,11 @@ import ( "context" "path/filepath" "time" - "github.com/multiformats/go-varint" "go.uber.org/fx" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/node/modules/alerts.go b/node/modules/alerts.go index 9976c6d0e42..7c2324a0677 100644 --- a/node/modules/alerts.go +++ b/node/modules/alerts.go @@ -5,7 +5,6 @@ import ( "os" "strconv" "syscall" - "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/ulimit" ) diff --git a/node/modules/blockstore.go b/node/modules/blockstore.go index 9c54d51e60f..50369d23d0a 100644 --- a/node/modules/blockstore.go +++ b/node/modules/blockstore.go @@ -5,11 +5,9 @@ import ( "io" "os" "path/filepath" - bstore "github.com/ipfs/boxo/blockstore" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/blockstore/splitstore" diff --git a/node/modules/chain.go b/node/modules/chain.go index d08b7dbec5e..e3f9e1bc31a 100644 --- a/node/modules/chain.go +++ b/node/modules/chain.go @@ -3,7 +3,6 @@ package modules import ( "context" "time" - "github.com/ipfs/boxo/bitswap" "github.com/ipfs/boxo/bitswap/network" "github.com/ipfs/boxo/blockservice" @@ -11,7 +10,6 @@ import ( "github.com/libp2p/go-libp2p/core/routing" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/blockstore/splitstore" "github.com/filecoin-project/lotus/build" diff --git a/node/modules/client.go b/node/modules/client.go index 9d8eef4217b..ef86f43c9c7 100644 --- a/node/modules/client.go +++ b/node/modules/client.go @@ -6,13 +6,11 @@ import ( "os" "path/filepath" "time" - "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-data-transfer/v2/channelmonitor" dtimpl "github.com/filecoin-project/go-data-transfer/v2/impl" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" @@ -26,7 +24,6 @@ import ( storageimpl "github.com/filecoin-project/go-fil-markets/storagemarket/impl" smnet "github.com/filecoin-project/go-fil-markets/storagemarket/network" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/market" "github.com/filecoin-project/lotus/journal" diff --git a/node/modules/core.go b/node/modules/core.go index a0d52c291bc..1c0c359903d 100644 --- a/node/modules/core.go +++ b/node/modules/core.go @@ -8,7 +8,6 @@ import ( "os" "path/filepath" "time" - "github.com/gbrlsnchs/jwt/v3" logging "github.com/ipfs/go-log/v2" record "github.com/libp2p/go-libp2p-record" @@ -17,10 +16,8 @@ import ( "github.com/raulk/go-watchdog" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/modules/dtypes/api.go b/node/modules/dtypes/api.go index 6de511b9d6f..bd0e6bcab2c 100644 --- a/node/modules/dtypes/api.go +++ b/node/modules/dtypes/api.go @@ -2,7 +2,6 @@ package dtypes import ( "time" - "github.com/gbrlsnchs/jwt/v3" "github.com/multiformats/go-multiaddr" ) diff --git a/node/modules/dtypes/miner.go b/node/modules/dtypes/miner.go index 24bcc714c17..330dd1d079f 100644 --- a/node/modules/dtypes/miner.go +++ b/node/modules/dtypes/miner.go @@ -3,14 +3,11 @@ package dtypes import ( "context" "time" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/pipeline/sealiface" ) diff --git a/node/modules/dtypes/mpool.go b/node/modules/dtypes/mpool.go index df96b8d0ebd..4c32b69f602 100644 --- a/node/modules/dtypes/mpool.go +++ b/node/modules/dtypes/mpool.go @@ -3,7 +3,6 @@ package dtypes import ( "context" "sync" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" ) diff --git a/node/modules/dtypes/scorekeeper.go b/node/modules/dtypes/scorekeeper.go index 343b787b733..903768fdf46 100644 --- a/node/modules/dtypes/scorekeeper.go +++ b/node/modules/dtypes/scorekeeper.go @@ -2,7 +2,6 @@ package dtypes import ( "sync" - pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/peer" ) diff --git a/node/modules/dtypes/storage.go b/node/modules/dtypes/storage.go index 7f0466f1f3d..49f6f35cade 100644 --- a/node/modules/dtypes/storage.go +++ b/node/modules/dtypes/storage.go @@ -5,13 +5,11 @@ import ( exchange "github.com/ipfs/boxo/exchange" "github.com/ipfs/go-datastore" "github.com/ipfs/go-graphsync" - datatransfer "github.com/filecoin-project/go-data-transfer/v2" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" "github.com/filecoin-project/go-fil-markets/piecestore" "github.com/filecoin-project/go-fil-markets/storagemarket/impl/requestvalidation" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/node/repo/imports" ) diff --git a/node/modules/ethmodule.go b/node/modules/ethmodule.go index 0255b61983f..31772baa98b 100644 --- a/node/modules/ethmodule.go +++ b/node/modules/ethmodule.go @@ -5,11 +5,8 @@ import ( "os" "path/filepath" "time" - "go.uber.org/fx" - "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/ethhashlookup" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/node/modules/faultreport.go b/node/modules/faultreport.go index c42602d7e61..c4f5c04582d 100644 --- a/node/modules/faultreport.go +++ b/node/modules/faultreport.go @@ -2,7 +2,6 @@ package modules import ( "go.uber.org/fx" - "github.com/filecoin-project/lotus/chain/gen/slashfilter/slashsvc" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/node/modules/genesis.go b/node/modules/genesis.go index 7f0a46afc50..8a5fb48a0bb 100644 --- a/node/modules/genesis.go +++ b/node/modules/genesis.go @@ -3,12 +3,10 @@ package modules import ( "bytes" "os" - "github.com/ipfs/go-datastore" "github.com/ipld/go-car" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/graphsync.go b/node/modules/graphsync.go index ca69cd2d202..e49508a8ead 100644 --- a/node/modules/graphsync.go +++ b/node/modules/graphsync.go @@ -3,7 +3,6 @@ package modules import ( "context" "time" - "github.com/ipfs/go-graphsync" graphsyncimpl "github.com/ipfs/go-graphsync/impl" gsnet "github.com/ipfs/go-graphsync/network" @@ -12,7 +11,6 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "go.opencensus.io/stats" "go.uber.org/fx" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/helpers/helpers.go b/node/modules/helpers/helpers.go index f63b7eedd3e..a0d30a9bc4c 100644 --- a/node/modules/helpers/helpers.go +++ b/node/modules/helpers/helpers.go @@ -2,7 +2,6 @@ package helpers import ( "context" - "go.uber.org/fx" ) diff --git a/node/modules/ipfs.go b/node/modules/ipfs.go index cb9deb6fde1..e0b108d5d30 100644 --- a/node/modules/ipfs.go +++ b/node/modules/ipfs.go @@ -5,10 +5,8 @@ import ( "github.com/multiformats/go-multiaddr" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/markets/retrievaladapter" "github.com/filecoin-project/lotus/markets/storageadapter" diff --git a/node/modules/lp2p/addrs.go b/node/modules/lp2p/addrs.go index 52ee8c852f6..73a8d79605e 100644 --- a/node/modules/lp2p/addrs.go +++ b/node/modules/lp2p/addrs.go @@ -2,7 +2,6 @@ package lp2p import ( "fmt" - "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/host" p2pbhost "github.com/libp2p/go-libp2p/p2p/host/basic" diff --git a/node/modules/lp2p/conngater.go b/node/modules/lp2p/conngater.go index 29087a1bc28..3c85bc4ad31 100644 --- a/node/modules/lp2p/conngater.go +++ b/node/modules/lp2p/conngater.go @@ -3,7 +3,6 @@ package lp2p import ( "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/p2p/net/conngater" - "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/node/modules/lp2p/discovery.go b/node/modules/lp2p/discovery.go index ca68a7b9cf5..ac51c03829a 100644 --- a/node/modules/lp2p/discovery.go +++ b/node/modules/lp2p/discovery.go @@ -3,11 +3,9 @@ package lp2p import ( "context" "time" - "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" - "github.com/filecoin-project/lotus/node/modules/helpers" ) diff --git a/node/modules/lp2p/host.go b/node/modules/lp2p/host.go index 9c140b41ee6..6313d3333eb 100644 --- a/node/modules/lp2p/host.go +++ b/node/modules/lp2p/host.go @@ -3,7 +3,6 @@ package lp2p import ( "context" "fmt" - nilrouting "github.com/ipfs/boxo/routing/none" "github.com/libp2p/go-libp2p" dht "github.com/libp2p/go-libp2p-kad-dht" @@ -15,7 +14,6 @@ import ( routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" "go.uber.org/fx" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/node/modules/helpers" diff --git a/node/modules/lp2p/libp2p.go b/node/modules/lp2p/libp2p.go index 9d42b3996bc..bd8ddee591a 100644 --- a/node/modules/lp2p/libp2p.go +++ b/node/modules/lp2p/libp2p.go @@ -3,7 +3,6 @@ package lp2p import ( "crypto/rand" "time" - logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/crypto" @@ -12,7 +11,6 @@ import ( "github.com/libp2p/go-libp2p/p2p/net/connmgr" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/modules/lp2p/pubsub.go b/node/modules/lp2p/pubsub.go index 2b3efce6c44..1208e58f307 100644 --- a/node/modules/lp2p/pubsub.go +++ b/node/modules/lp2p/pubsub.go @@ -5,7 +5,6 @@ import ( "encoding/json" "net" "time" - pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/host" @@ -15,7 +14,6 @@ import ( "go.opencensus.io/stats" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/config" diff --git a/node/modules/lp2p/rcmgr.go b/node/modules/lp2p/rcmgr.go index f2b2849863e..0aa00301c05 100644 --- a/node/modules/lp2p/rcmgr.go +++ b/node/modules/lp2p/rcmgr.go @@ -8,7 +8,6 @@ import ( "os" "path/filepath" "sync" - logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/network" @@ -19,7 +18,6 @@ import ( "go.opencensus.io/stats" "go.opencensus.io/tag" "go.uber.org/fx" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/node/modules/lp2p/relay.go b/node/modules/lp2p/relay.go index 87a4983f6b6..21edffed224 100644 --- a/node/modules/lp2p/relay.go +++ b/node/modules/lp2p/relay.go @@ -2,7 +2,6 @@ package lp2p import ( "fmt" - "github.com/libp2p/go-libp2p" coredisc "github.com/libp2p/go-libp2p/core/discovery" "github.com/libp2p/go-libp2p/core/routing" diff --git a/node/modules/lp2p/routing.go b/node/modules/lp2p/routing.go index 816cbe2eab6..5e22b3bb783 100644 --- a/node/modules/lp2p/routing.go +++ b/node/modules/lp2p/routing.go @@ -3,7 +3,6 @@ package lp2p import ( "context" "sort" - dht "github.com/libp2p/go-libp2p-kad-dht" record "github.com/libp2p/go-libp2p-record" routinghelpers "github.com/libp2p/go-libp2p-routing-helpers" diff --git a/node/modules/lp2p/smux.go b/node/modules/lp2p/smux.go index cab7da68251..e45da04ac52 100644 --- a/node/modules/lp2p/smux.go +++ b/node/modules/lp2p/smux.go @@ -2,7 +2,6 @@ package lp2p import ( "os" - "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/p2p/muxer/yamux" ) diff --git a/node/modules/mpoolnonceapi.go b/node/modules/mpoolnonceapi.go index 393bee32f24..0bf17a7a6e0 100644 --- a/node/modules/mpoolnonceapi.go +++ b/node/modules/mpoolnonceapi.go @@ -3,12 +3,9 @@ package modules import ( "context" "strings" - "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/node/modules/msgindex.go b/node/modules/msgindex.go index 72e9840ba33..21e3ed9277a 100644 --- a/node/modules/msgindex.go +++ b/node/modules/msgindex.go @@ -2,9 +2,7 @@ package modules import ( "context" - "go.uber.org/fx" - "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/node/modules/helpers" diff --git a/node/modules/paych.go b/node/modules/paych.go index 4f93bbd6c55..c28d993c097 100644 --- a/node/modules/paych.go +++ b/node/modules/paych.go @@ -2,11 +2,9 @@ package modules import ( "context" - "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "go.uber.org/fx" - "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/node/impl/full" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/rpc.go b/node/modules/rpc.go index d76949737e2..a57939f72ae 100644 --- a/node/modules/rpc.go +++ b/node/modules/rpc.go @@ -2,12 +2,10 @@ package modules import ( "context" - rpc "github.com/libp2p/go-libp2p-gorpc" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" consensus "github.com/filecoin-project/lotus/lib/consensus/raft" diff --git a/node/modules/services.go b/node/modules/services.go index f3dd443d94d..f92cf3ffe2c 100644 --- a/node/modules/services.go +++ b/node/modules/services.go @@ -5,7 +5,6 @@ import ( "os" "strconv" "time" - "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -16,10 +15,8 @@ import ( "github.com/libp2p/go-libp2p/p2p/host/eventbus" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-fil-markets/discovery" discoveryimpl "github.com/filecoin-project/go-fil-markets/discovery/impl" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/modules/stmgr.go b/node/modules/stmgr.go index f3eaee219c5..8af5ea8bfae 100644 --- a/node/modules/stmgr.go +++ b/node/modules/stmgr.go @@ -2,7 +2,6 @@ package modules import ( "go.uber.org/fx" - "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/modules/storage.go b/node/modules/storage.go index cb30eb8c29d..ca70dc90d05 100644 --- a/node/modules/storage.go +++ b/node/modules/storage.go @@ -3,10 +3,8 @@ package modules import ( "context" "path/filepath" - "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer.go b/node/modules/storageminer.go index 0680029bfc0..5cbf6901577 100644 --- a/node/modules/storageminer.go +++ b/node/modules/storageminer.go @@ -10,7 +10,6 @@ import ( "path/filepath" "strings" "time" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" @@ -23,7 +22,6 @@ import ( "go.uber.org/fx" "go.uber.org/multierr" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" dtimpl "github.com/filecoin-project/go-data-transfer/v2/impl" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" @@ -43,7 +41,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/node/modules/storageminer_dagstore.go b/node/modules/storageminer_dagstore.go index 620e690901c..323393b6e27 100644 --- a/node/modules/storageminer_dagstore.go +++ b/node/modules/storageminer_dagstore.go @@ -6,13 +6,10 @@ import ( "os" "path/filepath" "strconv" - "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/dagstore" - mdagstore "github.com/filecoin-project/lotus/markets/dagstore" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_idxprov.go b/node/modules/storageminer_idxprov.go index 777c59386b5..b4e18e0f4d0 100644 --- a/node/modules/storageminer_idxprov.go +++ b/node/modules/storageminer_idxprov.go @@ -2,7 +2,6 @@ package modules import ( "context" - "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" provider "github.com/ipni/index-provider" @@ -11,9 +10,7 @@ import ( "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_idxprov_test.go b/node/modules/storageminer_idxprov_test.go index 434577bab64..48c522afc13 100644 --- a/node/modules/storageminer_idxprov_test.go +++ b/node/modules/storageminer_idxprov_test.go @@ -5,7 +5,6 @@ import ( "strings" "testing" "time" - "github.com/ipfs/go-datastore" provider "github.com/ipni/index-provider" "github.com/libp2p/go-libp2p" @@ -13,9 +12,7 @@ import ( "github.com/libp2p/go-libp2p/core/host" "github.com/stretchr/testify/require" "go.uber.org/fx" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_svc.go b/node/modules/storageminer_svc.go index dec0453095a..3064861f900 100644 --- a/node/modules/storageminer_svc.go +++ b/node/modules/storageminer_svc.go @@ -2,10 +2,8 @@ package modules import ( "context" - "go.uber.org/fx" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" cliutil "github.com/filecoin-project/lotus/cli/util" diff --git a/node/modules/testing/beacon.go b/node/modules/testing/beacon.go index 7876e1d0528..4263d50fddd 100644 --- a/node/modules/testing/beacon.go +++ b/node/modules/testing/beacon.go @@ -2,7 +2,6 @@ package testing import ( "time" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" ) diff --git a/node/modules/testing/genesis.go b/node/modules/testing/genesis.go index cb8d60b5e6b..1e9d85f2330 100644 --- a/node/modules/testing/genesis.go +++ b/node/modules/testing/genesis.go @@ -6,7 +6,6 @@ import ( "fmt" "io" "os" - "github.com/ipfs/boxo/blockservice" offline "github.com/ipfs/boxo/exchange/offline" "github.com/ipfs/boxo/ipld/merkledag" @@ -15,7 +14,6 @@ import ( "github.com/ipld/go-car" "github.com/mitchellh/go-homedir" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" genesis2 "github.com/filecoin-project/lotus/chain/gen/genesis" diff --git a/node/modules/tracer/elasticsearch_transport.go b/node/modules/tracer/elasticsearch_transport.go index e54e0eba241..453ef34bf60 100644 --- a/node/modules/tracer/elasticsearch_transport.go +++ b/node/modules/tracer/elasticsearch_transport.go @@ -8,7 +8,6 @@ import ( "net/http" "net/url" "time" - "github.com/elastic/go-elasticsearch/v7" "github.com/elastic/go-elasticsearch/v7/esutil" ) diff --git a/node/modules/tracer/tracer.go b/node/modules/tracer/tracer.go index b9b56e80c86..e2b79fe63e7 100644 --- a/node/modules/tracer/tracer.go +++ b/node/modules/tracer/tracer.go @@ -2,7 +2,6 @@ package tracer import ( "time" - logging "github.com/ipfs/go-log/v2" pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" diff --git a/node/modules/tracer/tracer_test.go b/node/modules/tracer/tracer_test.go index f0d7b2c0b83..f65fdf8bfed 100644 --- a/node/modules/tracer/tracer_test.go +++ b/node/modules/tracer/tracer_test.go @@ -3,7 +3,6 @@ package tracer import ( "testing" "time" - pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/peer" diff --git a/node/options.go b/node/options.go index 26c2c247f9a..887adfdbe1e 100644 --- a/node/options.go +++ b/node/options.go @@ -2,7 +2,6 @@ package node import ( "reflect" - "go.uber.org/fx" ) diff --git a/node/repo/blockstore_opts.go b/node/repo/blockstore_opts.go index 81f8b9ff416..7f1c71d5c40 100644 --- a/node/repo/blockstore_opts.go +++ b/node/repo/blockstore_opts.go @@ -3,7 +3,6 @@ package repo import ( "os" "strconv" - badgerbs "github.com/filecoin-project/lotus/blockstore/badger" ) diff --git a/node/repo/fsrepo.go b/node/repo/fsrepo.go index bd3e2fe2787..77284b654d6 100644 --- a/node/repo/fsrepo.go +++ b/node/repo/fsrepo.go @@ -10,7 +10,6 @@ import ( "path/filepath" "strings" "sync" - "github.com/BurntSushi/toml" "github.com/ipfs/go-datastore" fslock "github.com/ipfs/go-fs-lock" @@ -19,7 +18,6 @@ import ( "github.com/multiformats/go-base32" "github.com/multiformats/go-multiaddr" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/chain/types" @@ -346,7 +344,7 @@ func (fsr *FsRepo) APIEndpoint() (multiaddr.Multiaddr, error) { f, err := os.Open(p) if os.IsNotExist(err) { - return nil, ErrNoAPIEndpoint + return nil, xerrors.Errorf("No file (%s): %w", p, ErrNoAPIEndpoint) } else if err != nil { return nil, err } diff --git a/node/repo/fsrepo_ds.go b/node/repo/fsrepo_ds.go index a4415692aaf..df7f6ee9225 100644 --- a/node/repo/fsrepo_ds.go +++ b/node/repo/fsrepo_ds.go @@ -4,7 +4,6 @@ import ( "context" "os" "path/filepath" - dgbadger "github.com/dgraph-io/badger/v2" "github.com/ipfs/go-datastore" badger "github.com/ipfs/go-ds-badger2" diff --git a/node/repo/imports/manager.go b/node/repo/imports/manager.go index a3648b6b02a..d1ac3637164 100644 --- a/node/repo/imports/manager.go +++ b/node/repo/imports/manager.go @@ -7,14 +7,12 @@ import ( "os" "path/filepath" "strconv" - "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "github.com/ipfs/go-datastore/query" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" - "github.com/filecoin-project/go-fil-markets/shared" ) diff --git a/node/repo/interface.go b/node/repo/interface.go index 328862b9214..a5178578955 100644 --- a/node/repo/interface.go +++ b/node/repo/interface.go @@ -3,10 +3,8 @@ package repo import ( "context" "errors" - "github.com/ipfs/go-datastore" "github.com/multiformats/go-multiaddr" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/node/repo/memrepo.go b/node/repo/memrepo.go index 6a4b416e204..149dd3f55de 100644 --- a/node/repo/memrepo.go +++ b/node/repo/memrepo.go @@ -6,14 +6,12 @@ import ( "os" "path/filepath" "sync" - "github.com/google/uuid" "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" dssync "github.com/ipfs/go-datastore/sync" "github.com/multiformats/go-multiaddr" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" diff --git a/node/repo/repo_test.go b/node/repo/repo_test.go index 16c101d44b9..4fb155c6400 100644 --- a/node/repo/repo_test.go +++ b/node/repo/repo_test.go @@ -3,12 +3,10 @@ package repo import ( "testing" - "github.com/multiformats/go-multiaddr" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" ) diff --git a/node/rpc.go b/node/rpc.go index cacd33526c8..259b917a258 100644 --- a/node/rpc.go +++ b/node/rpc.go @@ -10,7 +10,6 @@ import ( "runtime" "strconv" "time" - "github.com/google/uuid" "github.com/gorilla/mux" "github.com/gorilla/websocket" @@ -20,10 +19,8 @@ import ( manet "github.com/multiformats/go-multiaddr/net" "go.opencensus.io/tag" "golang.org/x/xerrors" - "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/node/shutdown_test.go b/node/shutdown_test.go index 6a5088efd78..e663cc0e7e3 100644 --- a/node/shutdown_test.go +++ b/node/shutdown_test.go @@ -6,7 +6,6 @@ import ( "sync" "testing" "time" - "github.com/stretchr/testify/require" ) diff --git a/node/testopts.go b/node/testopts.go index ca1e8112759..cf19a46407e 100644 --- a/node/testopts.go +++ b/node/testopts.go @@ -2,9 +2,7 @@ package node import ( "errors" - mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" - "github.com/filecoin-project/lotus/node/modules/lp2p" ) diff --git a/paychmgr/accessorcache.go b/paychmgr/accessorcache.go index 358cf790061..ee6e4b57aba 100644 --- a/paychmgr/accessorcache.go +++ b/paychmgr/accessorcache.go @@ -2,7 +2,6 @@ package paychmgr import ( "context" - "github.com/filecoin-project/go-address" ) diff --git a/paychmgr/cbor_gen.go b/paychmgr/cbor_gen.go index f97c176a304..be3fdb3a9d7 100644 --- a/paychmgr/cbor_gen.go +++ b/paychmgr/cbor_gen.go @@ -7,11 +7,9 @@ import ( "io" "math" "sort" - cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" - address "github.com/filecoin-project/go-address" paych "github.com/filecoin-project/go-state-types/builtin/v8/paych" ) diff --git a/paychmgr/manager.go b/paychmgr/manager.go index b1b6a7517a6..339927ce06b 100644 --- a/paychmgr/manager.go +++ b/paychmgr/manager.go @@ -4,18 +4,15 @@ import ( "context" "errors" "sync" - "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/paychmgr/mock_test.go b/paychmgr/mock_test.go index 5d36e60f074..f4b744e4676 100644 --- a/paychmgr/mock_test.go +++ b/paychmgr/mock_test.go @@ -4,14 +4,11 @@ import ( "context" "errors" "sync" - "github.com/ipfs/go-cid" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" diff --git a/paychmgr/msglistener_test.go b/paychmgr/msglistener_test.go index 38f2351ffa3..8d2e793de1a 100644 --- a/paychmgr/msglistener_test.go +++ b/paychmgr/msglistener_test.go @@ -2,7 +2,6 @@ package paychmgr import ( "testing" - "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" "golang.org/x/xerrors" diff --git a/paychmgr/paych.go b/paychmgr/paych.go index 2729513ce88..1f6b7cf917f 100644 --- a/paychmgr/paych.go +++ b/paychmgr/paych.go @@ -3,16 +3,13 @@ package paychmgr import ( "context" "fmt" - "github.com/ipfs/go-cid" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api" lpaych "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/paych_test.go b/paychmgr/paych_test.go index e72a992a37f..fe6f640f2b2 100644 --- a/paychmgr/paych_test.go +++ b/paychmgr/paych_test.go @@ -5,12 +5,10 @@ import ( "bytes" "context" "testing" - "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" @@ -18,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/specs-actors/v2/actors/builtin" tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" diff --git a/paychmgr/paychget_test.go b/paychmgr/paychget_test.go index fca0022c6b4..e3f39de757b 100644 --- a/paychmgr/paychget_test.go +++ b/paychmgr/paychget_test.go @@ -6,12 +6,10 @@ import ( "sync" "testing" "time" - "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" @@ -19,7 +17,6 @@ import ( "github.com/filecoin-project/specs-actors/v2/actors/builtin" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - lotusinit "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" diff --git a/paychmgr/paychvoucherfunds_test.go b/paychmgr/paychvoucherfunds_test.go index 9cff7fc9d55..600ff30838f 100644 --- a/paychmgr/paychvoucherfunds_test.go +++ b/paychmgr/paychvoucherfunds_test.go @@ -3,18 +3,15 @@ package paychmgr import ( "context" "testing" - "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" tutils2 "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/settle_test.go b/paychmgr/settle_test.go index db1d35da35a..71dd39421db 100644 --- a/paychmgr/settle_test.go +++ b/paychmgr/settle_test.go @@ -4,12 +4,10 @@ package paychmgr import ( "context" "testing" - "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-state-types/big" tutils "github.com/filecoin-project/specs-actors/support/testing" ) diff --git a/paychmgr/settler/settler.go b/paychmgr/settler/settler.go index 55431f0a561..dc04944f9b0 100644 --- a/paychmgr/settler/settler.go +++ b/paychmgr/settler/settler.go @@ -3,16 +3,13 @@ package settler import ( "context" "sync" - "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "go.uber.org/fx" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/events" diff --git a/paychmgr/simple.go b/paychmgr/simple.go index 6d6b78a713e..07fd140e947 100644 --- a/paychmgr/simple.go +++ b/paychmgr/simple.go @@ -7,15 +7,12 @@ import ( "fmt" "sort" "sync" - "github.com/ipfs/go-cid" "golang.org/x/sync/errgroup" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/state.go b/paychmgr/state.go index 0466d2d3651..5943488f60c 100644 --- a/paychmgr/state.go +++ b/paychmgr/state.go @@ -2,9 +2,7 @@ package paychmgr import ( "context" - "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/paychmgr/store.go b/paychmgr/store.go index 72a95d0d442..22f6415e53b 100644 --- a/paychmgr/store.go +++ b/paychmgr/store.go @@ -5,18 +5,15 @@ import ( "context" "errors" "fmt" - "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" dsq "github.com/ipfs/go-datastore/query" "golang.org/x/xerrors" - "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/types" ) diff --git a/paychmgr/store_test.go b/paychmgr/store_test.go index f600b6537a9..c815421120a 100644 --- a/paychmgr/store_test.go +++ b/paychmgr/store_test.go @@ -4,11 +4,9 @@ package paychmgr import ( "context" "testing" - ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" - "github.com/filecoin-project/go-address" tutils "github.com/filecoin-project/specs-actors/support/testing" ) diff --git a/paychmgr/util.go b/paychmgr/util.go index 57853ac1441..f1f627037aa 100644 --- a/paychmgr/util.go +++ b/paychmgr/util.go @@ -2,7 +2,6 @@ package paychmgr import ( "context" - "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin/v8/paych" ) diff --git a/provider/address.go b/provider/address.go index f69ca3fac60..66e53ae8945 100644 --- a/provider/address.go +++ b/provider/address.go @@ -4,7 +4,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/ctladdr" ) diff --git a/provider/lpmessage/sender.go b/provider/lpmessage/sender.go index 5123410c957..aa0826b6216 100644 --- a/provider/lpmessage/sender.go +++ b/provider/lpmessage/sender.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" diff --git a/provider/lpwindow/compute_do.go b/provider/lpwindow/compute_do.go index d36541e8e1d..2de2f26f505 100644 --- a/provider/lpwindow/compute_do.go +++ b/provider/lpwindow/compute_do.go @@ -3,8 +3,6 @@ package lpwindow import ( "bytes" "context" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" - "github.com/filecoin-project/lotus/storage/sealer" "sort" "sync" "time" @@ -22,11 +20,12 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/proof" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" types "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/storiface" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) const disablePreChecks = false // todo config diff --git a/provider/lpwindow/compute_task.go b/provider/lpwindow/compute_task.go index 83bf67a3104..32b3ebf3a60 100644 --- a/provider/lpwindow/compute_task.go +++ b/provider/lpwindow/compute_task.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/provider/lpwindow/compute_task_test.go b/provider/lpwindow/compute_task_test.go index 0f7a1335bce..d543847cb92 100644 --- a/provider/lpwindow/compute_task_test.go +++ b/provider/lpwindow/compute_task_test.go @@ -6,7 +6,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/lib/harmony/harmonytask" diff --git a/provider/lpwindow/faults_simple.go b/provider/lpwindow/faults_simple.go index d43e8ee196a..e7599e2d3c2 100644 --- a/provider/lpwindow/faults_simple.go +++ b/provider/lpwindow/faults_simple.go @@ -11,7 +11,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/provider/lpwindow/recover_task.go b/provider/lpwindow/recover_task.go index d50fddc0e31..13b1ed77822 100644 --- a/provider/lpwindow/recover_task.go +++ b/provider/lpwindow/recover_task.go @@ -2,6 +2,9 @@ package lpwindow import ( "context" + + "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" @@ -21,7 +24,6 @@ import ( "github.com/filecoin-project/lotus/storage/ctladdr" "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/wdpost" - "golang.org/x/xerrors" ) type WdPostRecoverDeclareTask struct { diff --git a/provider/lpwindow/submit_task.go b/provider/lpwindow/submit_task.go index ba0f63f6d83..d4f9643065c 100644 --- a/provider/lpwindow/submit_task.go +++ b/provider/lpwindow/submit_task.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" diff --git a/storage/ctladdr/addresses.go b/storage/ctladdr/addresses.go index ee778cb38e6..5146cb89f16 100644 --- a/storage/ctladdr/addresses.go +++ b/storage/ctladdr/addresses.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/paths/db_index.go b/storage/paths/db_index.go index 7f8bc6ebce1..6a8646b7a24 100644 --- a/storage/paths/db_index.go +++ b/storage/paths/db_index.go @@ -16,7 +16,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/metrics" diff --git a/storage/paths/http_handler.go b/storage/paths/http_handler.go index 4d0539079b2..b36dc3257aa 100644 --- a/storage/paths/http_handler.go +++ b/storage/paths/http_handler.go @@ -14,7 +14,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/filecoin-project/lotus/storage/sealer/tarutil" diff --git a/storage/paths/http_handler_test.go b/storage/paths/http_handler_test.go index cf6d71c3775..525d954a064 100644 --- a/storage/paths/http_handler_test.go +++ b/storage/paths/http_handler_test.go @@ -14,7 +14,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/paths/mocks" "github.com/filecoin-project/lotus/storage/sealer/partialfile" diff --git a/storage/paths/index.go b/storage/paths/index.go index bc26bddb420..e28ad7781da 100644 --- a/storage/paths/index.go +++ b/storage/paths/index.go @@ -16,7 +16,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/paths/index_locks.go b/storage/paths/index_locks.go index ab95cb4a7ab..de34fe856fb 100644 --- a/storage/paths/index_locks.go +++ b/storage/paths/index_locks.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/index_locks_test.go b/storage/paths/index_locks_test.go index d3134055bcb..864c24b4f81 100644 --- a/storage/paths/index_locks_test.go +++ b/storage/paths/index_locks_test.go @@ -8,7 +8,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/index_proxy.go b/storage/paths/index_proxy.go index 06097b665ce..1b1b51ac740 100644 --- a/storage/paths/index_proxy.go +++ b/storage/paths/index_proxy.go @@ -4,7 +4,6 @@ import ( "context" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/paths/index_test.go b/storage/paths/index_test.go index 9a241da23e0..efde3b1b727 100644 --- a/storage/paths/index_test.go +++ b/storage/paths/index_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/interface.go b/storage/paths/interface.go index d96135de8f2..d52e40b777a 100644 --- a/storage/paths/interface.go +++ b/storage/paths/interface.go @@ -5,7 +5,6 @@ import ( "io" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/local.go b/storage/paths/local.go index 577d4dbe008..7a653152b53 100644 --- a/storage/paths/local.go +++ b/storage/paths/local.go @@ -15,7 +15,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/lib/result" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/mocks/index.go b/storage/paths/mocks/index.go index 6fdcb03b9ec..d4d6fd7c686 100644 --- a/storage/paths/mocks/index.go +++ b/storage/paths/mocks/index.go @@ -11,7 +11,6 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" - fsutil "github.com/filecoin-project/lotus/storage/sealer/fsutil" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/mocks/pf.go b/storage/paths/mocks/pf.go index 43b3bc4896b..0ae1eaacb53 100644 --- a/storage/paths/mocks/pf.go +++ b/storage/paths/mocks/pf.go @@ -11,7 +11,6 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" - partialfile "github.com/filecoin-project/lotus/storage/sealer/partialfile" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/mocks/store.go b/storage/paths/mocks/store.go index 72be4832378..69dbec4eac4 100644 --- a/storage/paths/mocks/store.go +++ b/storage/paths/mocks/store.go @@ -11,7 +11,6 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" - fsutil "github.com/filecoin-project/lotus/storage/sealer/fsutil" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/remote.go b/storage/paths/remote.go index 0b7563bb251..dbbc17a106d 100644 --- a/storage/paths/remote.go +++ b/storage/paths/remote.go @@ -20,7 +20,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/remote_test.go b/storage/paths/remote_test.go index e3376e6fa10..231929bc61c 100644 --- a/storage/paths/remote_test.go +++ b/storage/paths/remote_test.go @@ -20,7 +20,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/paths/mocks" diff --git a/storage/pipeline/cbor_gen.go b/storage/pipeline/cbor_gen.go index c832f8a144a..76626a7769c 100644 --- a/storage/pipeline/cbor_gen.go +++ b/storage/pipeline/cbor_gen.go @@ -13,7 +13,6 @@ import ( xerrors "golang.org/x/xerrors" abi "github.com/filecoin-project/go-state-types/abi" - api "github.com/filecoin-project/lotus/api" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/checks.go b/storage/pipeline/checks.go index ecd160231c1..f9ee0547a98 100644 --- a/storage/pipeline/checks.go +++ b/storage/pipeline/checks.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/commit_batch.go b/storage/pipeline/commit_batch.go index 754f317630b..1f9fdf29ea7 100644 --- a/storage/pipeline/commit_batch.go +++ b/storage/pipeline/commit_batch.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/commit_batch_test.go b/storage/pipeline/commit_batch_test.go index 5ae2f171ae2..8899bbfab25 100644 --- a/storage/pipeline/commit_batch_test.go +++ b/storage/pipeline/commit_batch_test.go @@ -20,8 +20,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/ctladdr" @@ -29,6 +27,7 @@ import ( "github.com/filecoin-project/lotus/storage/pipeline/mocks" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" "github.com/filecoin-project/lotus/storage/sealer/storiface" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) func TestCommitBatcher(t *testing.T) { diff --git a/storage/pipeline/currentdealinfo.go b/storage/pipeline/currentdealinfo.go index cd3aa47be27..f54fb535edd 100644 --- a/storage/pipeline/currentdealinfo.go +++ b/storage/pipeline/currentdealinfo.go @@ -13,7 +13,6 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/currentdealinfo_test.go b/storage/pipeline/currentdealinfo_test.go index 21141a35d57..d3d8c4c49c4 100644 --- a/storage/pipeline/currentdealinfo_test.go +++ b/storage/pipeline/currentdealinfo_test.go @@ -20,13 +20,12 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" evtmock "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) var errNotFound = errors.New("could not find") diff --git a/storage/pipeline/fsm.go b/storage/pipeline/fsm.go index ac3dafa86ec..75d094f2b01 100644 --- a/storage/pipeline/fsm.go +++ b/storage/pipeline/fsm.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" ) diff --git a/storage/pipeline/fsm_events.go b/storage/pipeline/fsm_events.go index a798a884b48..c778a1da7d3 100644 --- a/storage/pipeline/fsm_events.go +++ b/storage/pipeline/fsm_events.go @@ -9,7 +9,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/pipeline/fsm_test.go b/storage/pipeline/fsm_test.go index 7d7201953e8..565161b8287 100644 --- a/storage/pipeline/fsm_test.go +++ b/storage/pipeline/fsm_test.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/input.go b/storage/pipeline/input.go index 6fdae03d926..0d0539eef7a 100644 --- a/storage/pipeline/input.go +++ b/storage/pipeline/input.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/mocks/api.go b/storage/pipeline/mocks/api.go index 5c67a1c42e2..9b63cf58c3d 100644 --- a/storage/pipeline/mocks/api.go +++ b/storage/pipeline/mocks/api.go @@ -20,7 +20,6 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" - api "github.com/filecoin-project/lotus/api" miner0 "github.com/filecoin-project/lotus/chain/actors/builtin/miner" types "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/mocks/mock_commit_batcher.go b/storage/pipeline/mocks/mock_commit_batcher.go index 431a47c73d9..a6d30ae2506 100644 --- a/storage/pipeline/mocks/mock_commit_batcher.go +++ b/storage/pipeline/mocks/mock_commit_batcher.go @@ -15,7 +15,6 @@ import ( big "github.com/filecoin-project/go-state-types/big" miner "github.com/filecoin-project/go-state-types/builtin/v9/miner" network "github.com/filecoin-project/go-state-types/network" - api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/mocks/mock_precommit_batcher.go b/storage/pipeline/mocks/mock_precommit_batcher.go index 68cce7fb0f3..518a9c00ae2 100644 --- a/storage/pipeline/mocks/mock_precommit_batcher.go +++ b/storage/pipeline/mocks/mock_precommit_batcher.go @@ -15,7 +15,6 @@ import ( big "github.com/filecoin-project/go-state-types/big" verifreg "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" network "github.com/filecoin-project/go-state-types/network" - api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/numassign.go b/storage/pipeline/numassign.go index 60dddec98ea..2b5f07778ad 100644 --- a/storage/pipeline/numassign.go +++ b/storage/pipeline/numassign.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-bitfield" rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/precommit_batch.go b/storage/pipeline/precommit_batch.go index 3a86c8628e0..083643dfb5f 100644 --- a/storage/pipeline/precommit_batch.go +++ b/storage/pipeline/precommit_batch.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/precommit_batch_test.go b/storage/pipeline/precommit_batch_test.go index 1f3aaf24472..bd9d6285854 100644 --- a/storage/pipeline/precommit_batch_test.go +++ b/storage/pipeline/precommit_batch_test.go @@ -17,14 +17,13 @@ import ( "github.com/filecoin-project/go-state-types/big" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" - miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" pipeline "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/pipeline/mocks" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" + miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" ) var fc = config.MinerFeeConfig{ diff --git a/storage/pipeline/precommit_policy.go b/storage/pipeline/precommit_policy.go index 6e234f93094..45955f8e438 100644 --- a/storage/pipeline/precommit_policy.go +++ b/storage/pipeline/precommit_policy.go @@ -8,7 +8,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/precommit_policy_test.go b/storage/pipeline/precommit_policy_test.go index 7865560dec1..3aeac055824 100644 --- a/storage/pipeline/precommit_policy_test.go +++ b/storage/pipeline/precommit_policy_test.go @@ -12,7 +12,6 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/storage/pipeline/receive.go b/storage/pipeline/receive.go index 8427eba54f5..429f99d909b 100644 --- a/storage/pipeline/receive.go +++ b/storage/pipeline/receive.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/sealing.go b/storage/pipeline/sealing.go index 65d3fb14b1e..4cd99280a3c 100644 --- a/storage/pipeline/sealing.go +++ b/storage/pipeline/sealing.go @@ -21,7 +21,6 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" "github.com/filecoin-project/go-storedcounter" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_failed.go b/storage/pipeline/states_failed.go index 3323c4c9bc4..d7065e3ff7c 100644 --- a/storage/pipeline/states_failed.go +++ b/storage/pipeline/states_failed.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_failed_test.go b/storage/pipeline/states_failed_test.go index f6846c8f5f1..5e94a9f1f55 100644 --- a/storage/pipeline/states_failed_test.go +++ b/storage/pipeline/states_failed_test.go @@ -16,13 +16,12 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" - api2 "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" pipeline "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/pipeline/mocks" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" ) func TestStateRecoverDealIDs(t *testing.T) { diff --git a/storage/pipeline/states_proving.go b/storage/pipeline/states_proving.go index 500dcab48e6..63f20a2793b 100644 --- a/storage/pipeline/states_proving.go +++ b/storage/pipeline/states_proving.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/states_replica_update.go b/storage/pipeline/states_replica_update.go index 6717f49a6a6..c39763a6799 100644 --- a/storage/pipeline/states_replica_update.go +++ b/storage/pipeline/states_replica_update.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_sealing.go b/storage/pipeline/states_sealing.go index 5c91161efb1..2fc929f7374 100644 --- a/storage/pipeline/states_sealing.go +++ b/storage/pipeline/states_sealing.go @@ -22,7 +22,6 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statemachine" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/stats.go b/storage/pipeline/stats.go index 696cd56a560..b257644b56c 100644 --- a/storage/pipeline/stats.go +++ b/storage/pipeline/stats.go @@ -8,7 +8,6 @@ import ( "go.opencensus.io/tag" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" ) diff --git a/storage/pipeline/terminate_batch.go b/storage/pipeline/terminate_batch.go index 4842a4e1bb7..58eaa176e3c 100644 --- a/storage/pipeline/terminate_batch.go +++ b/storage/pipeline/terminate_batch.go @@ -17,7 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/types.go b/storage/pipeline/types.go index e752eb2b93b..c82b024ca5c 100644 --- a/storage/pipeline/types.go +++ b/storage/pipeline/types.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer" diff --git a/storage/pipeline/types_test.go b/storage/pipeline/types_test.go index b8fbb113af1..002294d57a2 100644 --- a/storage/pipeline/types_test.go +++ b/storage/pipeline/types_test.go @@ -10,9 +10,8 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) func TestSectorInfoSerialization(t *testing.T) { diff --git a/storage/pipeline/upgrade_queue.go b/storage/pipeline/upgrade_queue.go index 9d9e1ca46c6..d739ca9fdd0 100644 --- a/storage/pipeline/upgrade_queue.go +++ b/storage/pipeline/upgrade_queue.go @@ -6,9 +6,8 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" - "github.com/filecoin-project/lotus/chain/types" + market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" ) func (m *Sealing) MarkForUpgrade(ctx context.Context, id abi.SectorNumber) error { diff --git a/storage/pipeline/utils.go b/storage/pipeline/utils.go index 4b99a5beadd..997fcf87aa9 100644 --- a/storage/pipeline/utils.go +++ b/storage/pipeline/utils.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" diff --git a/storage/sealer/faults.go b/storage/sealer/faults.go index add2acf9666..b2dc4181bf4 100644 --- a/storage/sealer/faults.go +++ b/storage/sealer/faults.go @@ -10,7 +10,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/basicfs/fs.go b/storage/sealer/ffiwrapper/basicfs/fs.go index 4fd8e271fdf..8b9e889cc81 100644 --- a/storage/sealer/ffiwrapper/basicfs/fs.go +++ b/storage/sealer/ffiwrapper/basicfs/fs.go @@ -7,7 +7,6 @@ import ( "sync" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/prover_cgo.go b/storage/sealer/ffiwrapper/prover_cgo.go index abbd20b6e06..8aa7339e367 100644 --- a/storage/sealer/ffiwrapper/prover_cgo.go +++ b/storage/sealer/ffiwrapper/prover_cgo.go @@ -6,7 +6,6 @@ package ffiwrapper import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/sealer_cgo.go b/storage/sealer/ffiwrapper/sealer_cgo.go index 812a69fa7a6..d9966fd8466 100644 --- a/storage/sealer/ffiwrapper/sealer_cgo.go +++ b/storage/sealer/ffiwrapper/sealer_cgo.go @@ -29,7 +29,6 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/lib/nullreader" spaths "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fr32" diff --git a/storage/sealer/ffiwrapper/sealer_test.go b/storage/sealer/ffiwrapper/sealer_test.go index da1b98429c7..5776b26e745 100644 --- a/storage/sealer/ffiwrapper/sealer_test.go +++ b/storage/sealer/ffiwrapper/sealer_test.go @@ -28,7 +28,6 @@ import ( "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper/basicfs" diff --git a/storage/sealer/ffiwrapper/unseal_ranges.go b/storage/sealer/ffiwrapper/unseal_ranges.go index 37107aeb09f..82f06efab09 100644 --- a/storage/sealer/ffiwrapper/unseal_ranges.go +++ b/storage/sealer/ffiwrapper/unseal_ranges.go @@ -5,7 +5,6 @@ import ( rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/verifier_cgo.go b/storage/sealer/ffiwrapper/verifier_cgo.go index 7576d957d3b..ee4281fc223 100644 --- a/storage/sealer/ffiwrapper/verifier_cgo.go +++ b/storage/sealer/ffiwrapper/verifier_cgo.go @@ -11,9 +11,8 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func (sb *Sealer) GenerateWinningPoSt(ctx context.Context, minerID abi.ActorID, sectorInfo []proof.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]proof.PoStProof, error) { diff --git a/storage/sealer/fr32/fr32_ffi_cmp_test.go b/storage/sealer/fr32/fr32_ffi_cmp_test.go index 32afa470eba..9438ed942ae 100644 --- a/storage/sealer/fr32/fr32_ffi_cmp_test.go +++ b/storage/sealer/fr32/fr32_ffi_cmp_test.go @@ -11,7 +11,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" commpffi "github.com/filecoin-project/go-commp-utils/ffiwrapper" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/fr32/fr32_test.go b/storage/sealer/fr32/fr32_test.go index 437fa4e43cb..ba5a0d673b3 100644 --- a/storage/sealer/fr32/fr32_test.go +++ b/storage/sealer/fr32/fr32_test.go @@ -12,7 +12,6 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" commpffi "github.com/filecoin-project/go-commp-utils/ffiwrapper" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/fr32/readers_test.go b/storage/sealer/fr32/readers_test.go index f84b9d67a2a..c568cbfe23d 100644 --- a/storage/sealer/fr32/readers_test.go +++ b/storage/sealer/fr32/readers_test.go @@ -9,7 +9,6 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/manager.go b/storage/sealer/manager.go index 41b3a1b397a..fab85e83a4e 100644 --- a/storage/sealer/manager.go +++ b/storage/sealer/manager.go @@ -19,7 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" diff --git a/storage/sealer/manager_post.go b/storage/sealer/manager_post.go index 27a71ef8caf..7c9bf595ba8 100644 --- a/storage/sealer/manager_post.go +++ b/storage/sealer/manager_post.go @@ -12,9 +12,8 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func (m *Manager) GenerateWinningPoSt(ctx context.Context, minerID abi.ActorID, sectorInfo []proof.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]proof.PoStProof, error) { diff --git a/storage/sealer/manager_test.go b/storage/sealer/manager_test.go index 7c3e1a1f2b1..5cbe4bc7860 100644 --- a/storage/sealer/manager_test.go +++ b/storage/sealer/manager_test.go @@ -28,14 +28,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statestore" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func init() { diff --git a/storage/sealer/mock/mock.go b/storage/sealer/mock/mock.go index cb498492f49..a67aee091c8 100644 --- a/storage/sealer/mock/mock.go +++ b/storage/sealer/mock/mock.go @@ -18,7 +18,6 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/mock/util.go b/storage/sealer/mock/util.go index 48055b4d083..d78f6ce2b65 100644 --- a/storage/sealer/mock/util.go +++ b/storage/sealer/mock/util.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" diff --git a/storage/sealer/partialfile/partialfile.go b/storage/sealer/partialfile/partialfile.go index 4357f796d6d..9ad98342125 100644 --- a/storage/sealer/partialfile/partialfile.go +++ b/storage/sealer/partialfile/partialfile.go @@ -12,7 +12,6 @@ import ( rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/lib/readerutil" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/piece_provider.go b/storage/sealer/piece_provider.go index 0e992b67918..2383cd6511f 100644 --- a/storage/sealer/piece_provider.go +++ b/storage/sealer/piece_provider.go @@ -12,7 +12,6 @@ import ( "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fr32" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/piece_provider_test.go b/storage/sealer/piece_provider_test.go index de1e07a7849..7466c3ed7e6 100644 --- a/storage/sealer/piece_provider_test.go +++ b/storage/sealer/piece_provider_test.go @@ -20,7 +20,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/piece_reader.go b/storage/sealer/piece_reader.go index 7a7cd184110..3d963de80ed 100644 --- a/storage/sealer/piece_reader.go +++ b/storage/sealer/piece_reader.go @@ -14,7 +14,6 @@ import ( "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/metrics" ) diff --git a/storage/sealer/sched.go b/storage/sealer/sched.go index c0ac11bcf9d..74b0b2a6aba 100644 --- a/storage/sealer/sched.go +++ b/storage/sealer/sched.go @@ -9,7 +9,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/sched_post.go b/storage/sealer/sched_post.go index c6bd8182976..de609a770a3 100644 --- a/storage/sealer/sched_post.go +++ b/storage/sealer/sched_post.go @@ -13,7 +13,6 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/sched_test.go b/storage/sealer/sched_test.go index 2e2b05ab2c3..495347f3761 100644 --- a/storage/sealer/sched_test.go +++ b/storage/sealer/sched_test.go @@ -18,7 +18,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/sealer/selector_alloc.go b/storage/sealer/selector_alloc.go index 130f74461f5..9356902b149 100644 --- a/storage/sealer/selector_alloc.go +++ b/storage/sealer/selector_alloc.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_existing.go b/storage/sealer/selector_existing.go index c1e082db8a5..b63d7a1c792 100644 --- a/storage/sealer/selector_existing.go +++ b/storage/sealer/selector_existing.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_move.go b/storage/sealer/selector_move.go index fde4b3c59cd..1eee3d82153 100644 --- a/storage/sealer/selector_move.go +++ b/storage/sealer/selector_move.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_task.go b/storage/sealer/selector_task.go index 805fcbbd03b..fc2907a1936 100644 --- a/storage/sealer/selector_task.go +++ b/storage/sealer/selector_task.go @@ -6,7 +6,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/storiface/ffi.go b/storage/sealer/storiface/ffi.go index 4a9f832b843..d315832837c 100644 --- a/storage/sealer/storiface/ffi.go +++ b/storage/sealer/storiface/ffi.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/storiface/index.go b/storage/sealer/storiface/index.go index 653bd2fbae5..f7436bce822 100644 --- a/storage/sealer/storiface/index.go +++ b/storage/sealer/storiface/index.go @@ -4,7 +4,6 @@ import ( "strings" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/fsutil" ) diff --git a/storage/sealer/storiface/resources.go b/storage/sealer/storiface/resources.go index 6f8d83265a6..1b09681d4a5 100644 --- a/storage/sealer/storiface/resources.go +++ b/storage/sealer/storiface/resources.go @@ -10,7 +10,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/storiface/resources_test.go b/storage/sealer/storiface/resources_test.go index e360b8d1430..6e877fddc6a 100644 --- a/storage/sealer/storiface/resources_test.go +++ b/storage/sealer/storiface/resources_test.go @@ -7,7 +7,6 @@ import ( "github.com/stretchr/testify/require" stabi "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/storiface/worker.go b/storage/sealer/storiface/worker.go index e84fd8aa9a8..e11eb07cc48 100644 --- a/storage/sealer/storiface/worker.go +++ b/storage/sealer/storiface/worker.go @@ -11,7 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/teststorage_test.go b/storage/sealer/teststorage_test.go index 6e50044b3eb..5c54bbbad20 100644 --- a/storage/sealer/teststorage_test.go +++ b/storage/sealer/teststorage_test.go @@ -7,9 +7,8 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) type apres struct { diff --git a/storage/sealer/testworker_test.go b/storage/sealer/testworker_test.go index 6ea57b78d1f..868bb26eb46 100644 --- a/storage/sealer/testworker_test.go +++ b/storage/sealer/testworker_test.go @@ -8,7 +8,6 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/mock" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/worker_calltracker.go b/storage/sealer/worker_calltracker.go index eb5bab145ce..443530f58a2 100644 --- a/storage/sealer/worker_calltracker.go +++ b/storage/sealer/worker_calltracker.go @@ -8,7 +8,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/worker_local.go b/storage/sealer/worker_local.go index 67510566c2c..783976cae15 100644 --- a/storage/sealer/worker_local.go +++ b/storage/sealer/worker_local.go @@ -21,7 +21,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/worker_local_test.go b/storage/sealer/worker_local_test.go index 8624f2d918b..0b287d05042 100644 --- a/storage/sealer/worker_local_test.go +++ b/storage/sealer/worker_local_test.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" - "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/worker_tracked.go b/storage/sealer/worker_tracked.go index 7fce400a050..5f17c9b8e94 100644 --- a/storage/sealer/worker_tracked.go +++ b/storage/sealer/worker_tracked.go @@ -11,7 +11,6 @@ import ( "go.opencensus.io/tag" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sectorblocks/blocks.go b/storage/sectorblocks/blocks.go index 1593174bd8f..5343c4eb3d1 100644 --- a/storage/sectorblocks/blocks.go +++ b/storage/sectorblocks/blocks.go @@ -16,7 +16,6 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/wdpost/wdpost_changehandler.go b/storage/wdpost/wdpost_changehandler.go index ce58f148920..6e86ccd30a8 100644 --- a/storage/wdpost/wdpost_changehandler.go +++ b/storage/wdpost/wdpost_changehandler.go @@ -7,7 +7,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/wdpost/wdpost_changehandler_test.go b/storage/wdpost/wdpost_changehandler_test.go index 44d0dfe6d02..43c6accf764 100644 --- a/storage/wdpost/wdpost_changehandler_test.go +++ b/storage/wdpost/wdpost_changehandler_test.go @@ -16,9 +16,8 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" - tutils "github.com/filecoin-project/specs-actors/support/testing" - "github.com/filecoin-project/lotus/chain/types" + tutils "github.com/filecoin-project/specs-actors/support/testing" ) var dummyCid cid.Cid diff --git a/storage/wdpost/wdpost_journal.go b/storage/wdpost/wdpost_journal.go index 68cad3fcf3f..039475fc173 100644 --- a/storage/wdpost/wdpost_journal.go +++ b/storage/wdpost/wdpost_journal.go @@ -5,7 +5,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" ) diff --git a/storage/wdpost/wdpost_run.go b/storage/wdpost/wdpost_run.go index edc088ccf64..a34c17d850e 100644 --- a/storage/wdpost/wdpost_run.go +++ b/storage/wdpost/wdpost_run.go @@ -19,8 +19,6 @@ import ( "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -29,6 +27,7 @@ import ( "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/storiface" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) // recordPoStFailure records a failure in the journal. diff --git a/storage/wdpost/wdpost_run_faults.go b/storage/wdpost/wdpost_run_faults.go index 3a41cc4cce4..010086abdfa 100644 --- a/storage/wdpost/wdpost_run_faults.go +++ b/storage/wdpost/wdpost_run_faults.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/storage/wdpost/wdpost_run_test.go b/storage/wdpost/wdpost_run_test.go index a3847a6f6bd..0a843b2adad 100644 --- a/storage/wdpost/wdpost_run_test.go +++ b/storage/wdpost/wdpost_run_test.go @@ -22,8 +22,6 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -32,6 +30,7 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/storage/ctladdr" "github.com/filecoin-project/lotus/storage/sealer/storiface" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) type mockStorageMinerAPI struct { diff --git a/storage/wdpost/wdpost_sched.go b/storage/wdpost/wdpost_sched.go index bbf4596fe30..239f50ff67d 100644 --- a/storage/wdpost/wdpost_sched.go +++ b/storage/wdpost/wdpost_sched.go @@ -15,7 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/winning_prover.go b/storage/winning_prover.go index d61cd213c50..fc5338de248 100644 --- a/storage/winning_prover.go +++ b/storage/winning_prover.go @@ -10,7 +10,6 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/tools/stats/points/collect.go b/tools/stats/points/collect.go index 8b86695742e..fa9700d680f 100644 --- a/tools/stats/points/collect.go +++ b/tools/stats/points/collect.go @@ -15,7 +15,6 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/tools/stats/sync/sync.go b/tools/stats/sync/sync.go index c8db1c543be..7bad431c00c 100644 --- a/tools/stats/sync/sync.go +++ b/tools/stats/sync/sync.go @@ -5,7 +5,6 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" From 9b4a3ba23dd4e8c8255754e48ab3f134ec8860ef Mon Sep 17 00:00:00 2001 From: "Andrew Jackson (Ajax)" Date: Mon, 13 Nov 2023 18:06:11 -0600 Subject: [PATCH 21/24] fix circleci and fiximports --- api/api_common.go | 3 +++ api/api_errors.go | 1 + api/api_full.go | 3 +++ api/api_gateway.go | 3 +++ api/api_net.go | 1 + api/api_storage.go | 3 +++ api/api_test.go | 2 ++ api/api_wallet.go | 2 ++ api/api_worker.go | 3 +++ api/cbor_gen.go | 2 ++ api/client/client.go | 2 ++ api/docgen-openrpc/cmd/docgen_openrpc.go | 1 + api/docgen-openrpc/openrpc.go | 2 ++ api/docgen/cmd/docgen.go | 1 + api/docgen/docgen.go | 3 +++ api/mocks/mock_full.go | 1 + api/proxy_gen.go | 3 +++ api/proxy_util_test.go | 1 + api/types.go | 3 +++ api/utils.go | 1 + api/v0api/full.go | 3 +++ api/v0api/gateway.go | 3 +++ api/v0api/permissioned.go | 1 + api/v0api/proxy_gen.go | 3 +++ api/v0api/v0mocks/mock_full.go | 3 +++ api/v0api/v1_wrapper.go | 3 +++ blockstore/splitstore/splitstore.go | 1 + blockstore/splitstore/splitstore_compact.go | 1 + blockstore/splitstore/splitstore_test.go | 1 + blockstore/splitstore/splitstore_warmup.go | 1 + build/bootstrap.go | 2 ++ build/builtin_actors.go | 3 +++ build/builtin_actors_test.go | 3 +++ build/drand.go | 1 + build/genesis.go | 1 + build/openrpc.go | 1 + build/openrpc_test.go | 1 + build/panic_reporter.go | 1 + build/params_2k.go | 3 +++ build/params_butterfly.go | 2 ++ build/params_calibnet.go | 3 +++ build/params_interop.go | 3 +++ build/params_mainnet.go | 1 + build/params_shared_funcs.go | 2 ++ build/params_shared_vals.go | 2 ++ build/params_testground.go | 3 +++ chain/actors/adt/diff_adt_test.go | 3 ++- chain/actors/aerrors/error_test.go | 1 + chain/actors/builtin/account/account.go | 7 +++--- chain/actors/builtin/account/v0.go | 3 ++- chain/actors/builtin/account/v10.go | 1 + chain/actors/builtin/account/v11.go | 1 + chain/actors/builtin/account/v12.go | 1 + chain/actors/builtin/account/v2.go | 3 ++- chain/actors/builtin/account/v3.go | 3 ++- chain/actors/builtin/account/v4.go | 3 ++- chain/actors/builtin/account/v5.go | 3 ++- chain/actors/builtin/account/v6.go | 3 ++- chain/actors/builtin/account/v7.go | 3 ++- chain/actors/builtin/account/v8.go | 1 + chain/actors/builtin/account/v9.go | 1 + chain/actors/builtin/builtin.go | 3 ++- chain/actors/builtin/cron/cron.go | 7 +++--- chain/actors/builtin/cron/v0.go | 3 ++- chain/actors/builtin/cron/v10.go | 1 + chain/actors/builtin/cron/v11.go | 1 + chain/actors/builtin/cron/v12.go | 1 + chain/actors/builtin/cron/v2.go | 3 ++- chain/actors/builtin/cron/v3.go | 3 ++- chain/actors/builtin/cron/v4.go | 3 ++- chain/actors/builtin/cron/v5.go | 3 ++- chain/actors/builtin/cron/v6.go | 3 ++- chain/actors/builtin/cron/v7.go | 3 ++- chain/actors/builtin/cron/v8.go | 1 + chain/actors/builtin/cron/v9.go | 1 + chain/actors/builtin/datacap/datacap.go | 1 + chain/actors/builtin/datacap/util.go | 1 + chain/actors/builtin/datacap/v10.go | 1 + chain/actors/builtin/datacap/v11.go | 1 + chain/actors/builtin/datacap/v12.go | 1 + chain/actors/builtin/datacap/v9.go | 1 + chain/actors/builtin/evm/evm.go | 1 + chain/actors/builtin/evm/v10.go | 1 + chain/actors/builtin/evm/v11.go | 1 + chain/actors/builtin/evm/v12.go | 1 + chain/actors/builtin/init/diff.go | 1 + chain/actors/builtin/init/init.go | 9 ++++--- chain/actors/builtin/init/v0.go | 5 ++-- chain/actors/builtin/init/v10.go | 1 + chain/actors/builtin/init/v11.go | 1 + chain/actors/builtin/init/v12.go | 1 + chain/actors/builtin/init/v2.go | 5 ++-- chain/actors/builtin/init/v3.go | 7 +++--- chain/actors/builtin/init/v4.go | 7 +++--- chain/actors/builtin/init/v5.go | 7 +++--- chain/actors/builtin/init/v6.go | 7 +++--- chain/actors/builtin/init/v7.go | 7 +++--- chain/actors/builtin/init/v8.go | 1 + chain/actors/builtin/init/v9.go | 1 + chain/actors/builtin/market/diff.go | 1 + chain/actors/builtin/market/market.go | 7 +++--- chain/actors/builtin/market/v0.go | 5 ++-- chain/actors/builtin/market/v10.go | 1 + chain/actors/builtin/market/v11.go | 1 + chain/actors/builtin/market/v12.go | 1 + chain/actors/builtin/market/v2.go | 5 ++-- chain/actors/builtin/market/v3.go | 5 ++-- chain/actors/builtin/market/v4.go | 5 ++-- chain/actors/builtin/market/v5.go | 5 ++-- chain/actors/builtin/market/v6.go | 5 ++-- chain/actors/builtin/market/v7.go | 5 ++-- chain/actors/builtin/market/v8.go | 1 + chain/actors/builtin/market/v9.go | 1 + chain/actors/builtin/miner/diff.go | 1 + chain/actors/builtin/miner/miner.go | 7 +++--- chain/actors/builtin/miner/v0.go | 5 ++-- chain/actors/builtin/miner/v10.go | 1 + chain/actors/builtin/miner/v11.go | 1 + chain/actors/builtin/miner/v12.go | 1 + chain/actors/builtin/miner/v2.go | 5 ++-- chain/actors/builtin/miner/v3.go | 5 ++-- chain/actors/builtin/miner/v4.go | 5 ++-- chain/actors/builtin/miner/v5.go | 5 ++-- chain/actors/builtin/miner/v6.go | 5 ++-- chain/actors/builtin/miner/v7.go | 5 ++-- chain/actors/builtin/miner/v8.go | 1 + chain/actors/builtin/miner/v9.go | 1 + chain/actors/builtin/multisig/diff.go | 1 + chain/actors/builtin/multisig/message0.go | 7 +++--- chain/actors/builtin/multisig/message10.go | 1 + chain/actors/builtin/multisig/message11.go | 1 + chain/actors/builtin/multisig/message12.go | 1 + chain/actors/builtin/multisig/message2.go | 7 +++--- chain/actors/builtin/multisig/message3.go | 7 +++--- chain/actors/builtin/multisig/message4.go | 7 +++--- chain/actors/builtin/multisig/message5.go | 7 +++--- chain/actors/builtin/multisig/message6.go | 7 +++--- chain/actors/builtin/multisig/message7.go | 7 +++--- chain/actors/builtin/multisig/message8.go | 1 + chain/actors/builtin/multisig/message9.go | 1 + chain/actors/builtin/multisig/multisig.go | 7 +++--- chain/actors/builtin/multisig/v0.go | 5 ++-- chain/actors/builtin/multisig/v10.go | 1 + chain/actors/builtin/multisig/v11.go | 1 + chain/actors/builtin/multisig/v12.go | 1 + chain/actors/builtin/multisig/v2.go | 5 ++-- chain/actors/builtin/multisig/v3.go | 5 ++-- chain/actors/builtin/multisig/v4.go | 5 ++-- chain/actors/builtin/multisig/v5.go | 5 ++-- chain/actors/builtin/multisig/v6.go | 5 ++-- chain/actors/builtin/multisig/v7.go | 5 ++-- chain/actors/builtin/multisig/v8.go | 1 + chain/actors/builtin/multisig/v9.go | 1 + chain/actors/builtin/paych/message0.go | 7 +++--- chain/actors/builtin/paych/message10.go | 1 + chain/actors/builtin/paych/message11.go | 1 + chain/actors/builtin/paych/message12.go | 1 + chain/actors/builtin/paych/message2.go | 7 +++--- chain/actors/builtin/paych/message3.go | 7 +++--- chain/actors/builtin/paych/message4.go | 7 +++--- chain/actors/builtin/paych/message5.go | 7 +++--- chain/actors/builtin/paych/message6.go | 7 +++--- chain/actors/builtin/paych/message7.go | 7 +++--- chain/actors/builtin/paych/message8.go | 1 + chain/actors/builtin/paych/message9.go | 1 + chain/actors/builtin/paych/mock/mock.go | 1 + chain/actors/builtin/paych/paych.go | 7 +++--- chain/actors/builtin/paych/v0.go | 5 ++-- chain/actors/builtin/paych/v10.go | 1 + chain/actors/builtin/paych/v11.go | 1 + chain/actors/builtin/paych/v12.go | 1 + chain/actors/builtin/paych/v2.go | 5 ++-- chain/actors/builtin/paych/v3.go | 5 ++-- chain/actors/builtin/paych/v4.go | 5 ++-- chain/actors/builtin/paych/v5.go | 5 ++-- chain/actors/builtin/paych/v6.go | 5 ++-- chain/actors/builtin/paych/v7.go | 5 ++-- chain/actors/builtin/paych/v8.go | 1 + chain/actors/builtin/paych/v9.go | 1 + chain/actors/builtin/power/diff.go | 1 + chain/actors/builtin/power/power.go | 9 ++++--- chain/actors/builtin/power/v0.go | 5 ++-- chain/actors/builtin/power/v10.go | 1 + chain/actors/builtin/power/v11.go | 1 + chain/actors/builtin/power/v12.go | 1 + chain/actors/builtin/power/v2.go | 5 ++-- chain/actors/builtin/power/v3.go | 7 +++--- chain/actors/builtin/power/v4.go | 7 +++--- chain/actors/builtin/power/v5.go | 7 +++--- chain/actors/builtin/power/v6.go | 7 +++--- chain/actors/builtin/power/v7.go | 7 +++--- chain/actors/builtin/power/v8.go | 1 + chain/actors/builtin/power/v9.go | 1 + chain/actors/builtin/registry.go | 1 + chain/actors/builtin/reward/reward.go | 9 ++++--- chain/actors/builtin/reward/v0.go | 7 +++--- chain/actors/builtin/reward/v10.go | 1 + chain/actors/builtin/reward/v11.go | 1 + chain/actors/builtin/reward/v12.go | 1 + chain/actors/builtin/reward/v2.go | 7 +++--- chain/actors/builtin/reward/v3.go | 7 +++--- chain/actors/builtin/reward/v4.go | 7 +++--- chain/actors/builtin/reward/v5.go | 7 +++--- chain/actors/builtin/reward/v6.go | 7 +++--- chain/actors/builtin/reward/v7.go | 7 +++--- chain/actors/builtin/reward/v8.go | 1 + chain/actors/builtin/reward/v9.go | 1 + chain/actors/builtin/system/system.go | 7 +++--- chain/actors/builtin/system/v0.go | 3 ++- chain/actors/builtin/system/v10.go | 1 + chain/actors/builtin/system/v11.go | 1 + chain/actors/builtin/system/v12.go | 1 + chain/actors/builtin/system/v2.go | 3 ++- chain/actors/builtin/system/v3.go | 3 ++- chain/actors/builtin/system/v4.go | 3 ++- chain/actors/builtin/system/v5.go | 3 ++- chain/actors/builtin/system/v6.go | 3 ++- chain/actors/builtin/system/v7.go | 3 ++- chain/actors/builtin/system/v8.go | 1 + chain/actors/builtin/system/v9.go | 1 + chain/actors/builtin/verifreg/util.go | 3 ++- chain/actors/builtin/verifreg/v0.go | 5 ++-- chain/actors/builtin/verifreg/v10.go | 1 + chain/actors/builtin/verifreg/v11.go | 1 + chain/actors/builtin/verifreg/v12.go | 1 + chain/actors/builtin/verifreg/v2.go | 5 ++-- chain/actors/builtin/verifreg/v3.go | 5 ++-- chain/actors/builtin/verifreg/v4.go | 5 ++-- chain/actors/builtin/verifreg/v5.go | 5 ++-- chain/actors/builtin/verifreg/v6.go | 5 ++-- chain/actors/builtin/verifreg/v7.go | 5 ++-- chain/actors/builtin/verifreg/v8.go | 1 + chain/actors/builtin/verifreg/v9.go | 1 + chain/actors/builtin/verifreg/verifreg.go | 7 +++--- chain/actors/manifest.go | 1 + chain/actors/params.go | 1 + chain/beacon/beacon.go | 1 + chain/beacon/drand/drand.go | 1 + chain/beacon/drand/drand_test.go | 1 + chain/beacon/mock.go | 1 + chain/consensus/common.go | 3 ++- chain/consensus/compute_state.go | 17 +++++++------ chain/consensus/filcns/filecoin.go | 3 ++- chain/consensus/filcns/upgrades.go | 25 ++++++++++--------- chain/consensus/filcns/weight.go | 1 + chain/consensus/iface.go | 1 + chain/consensus/signatures.go | 1 + chain/events/cache.go | 1 + chain/events/events.go | 1 + chain/events/events_called.go | 1 + chain/events/events_height.go | 1 + chain/events/events_test.go | 1 + chain/events/filter/event.go | 3 ++- chain/events/filter/event_test.go | 3 ++- chain/events/filter/index.go | 1 + chain/events/filter/index_test.go | 1 + chain/events/observer.go | 1 + chain/events/state/fastapi.go | 1 + chain/events/state/mock/api.go | 1 + chain/events/state/mock/tipset.go | 1 + chain/events/state/predicates.go | 1 + chain/events/state/predicates_test.go | 11 ++++---- chain/events/tscache.go | 1 + chain/events/tscache_test.go | 1 + chain/exchange/client.go | 1 + chain/exchange/server.go | 1 + chain/gen/gen.go | 3 ++- chain/gen/gen_test.go | 1 + chain/gen/genesis/f00_system.go | 1 + chain/gen/genesis/f01_init.go | 3 ++- chain/gen/genesis/f02_reward.go | 1 + chain/gen/genesis/f03_cron.go | 1 + chain/gen/genesis/f04_power.go | 3 ++- chain/gen/genesis/f05_market.go | 1 + chain/gen/genesis/f06_vreg.go | 3 ++- chain/gen/genesis/f07_dcap.go | 3 ++- chain/gen/genesis/genesis.go | 7 +++--- chain/gen/genesis/genesis_eth.go | 1 + chain/gen/genesis/miners.go | 21 ++++++++-------- chain/gen/genesis/util.go | 1 + chain/gen/slashfilter/slashfilter.go | 1 + .../gen/slashfilter/slashsvc/slashservice.go | 3 ++- chain/index/msgindex.go | 1 + chain/index/msgindex_test.go | 1 + chain/market/fundmanager.go | 1 + chain/market/fundmanager_test.go | 3 ++- chain/market/store.go | 1 + chain/messagepool/check.go | 1 + chain/messagepool/gasguess/guessgas.go | 5 ++-- chain/messagepool/messagepool.go | 3 ++- chain/messagepool/messagepool_test.go | 3 ++- chain/messagepool/provider.go | 1 + chain/messagepool/pruning.go | 1 + chain/messagepool/repub.go | 1 + chain/messagepool/repub_test.go | 3 ++- chain/messagepool/selection.go | 1 + chain/messagepool/selection_test.go | 3 ++- chain/messagesigner/messagesigner.go | 1 + chain/messagesigner/messagesigner_test.go | 1 + chain/rand/rand.go | 1 + chain/rand/rand_test.go | 1 + chain/state/statetree.go | 7 +++--- chain/state/statetree_test.go | 3 ++- chain/stmgr/actors.go | 1 + chain/stmgr/call.go | 1 + chain/stmgr/forks.go | 3 ++- chain/stmgr/forks_test.go | 7 +++--- chain/stmgr/read.go | 1 + chain/stmgr/rpc/rpcstatemanager.go | 1 + chain/stmgr/searchwait.go | 1 + chain/stmgr/searchwait_test.go | 1 + chain/stmgr/stmgr.go | 3 ++- chain/stmgr/supply.go | 3 ++- chain/stmgr/utils.go | 1 + chain/store/basefee.go | 1 + chain/store/index.go | 1 + chain/store/index_test.go | 1 + chain/store/messages.go | 3 ++- chain/store/snapshot.go | 1 + chain/store/store.go | 3 ++- chain/store/store_test.go | 1 + chain/sub/incoming.go | 1 + chain/sub/incoming_test.go | 1 + chain/sync.go | 6 +++-- chain/sync_manager.go | 1 + chain/sync_test.go | 1 + chain/syncstate.go | 1 + chain/types/bigint.go | 1 + chain/types/ethtypes/eth_transactions.go | 1 + chain/types/ethtypes/eth_transactions_test.go | 1 + chain/types/ethtypes/eth_types.go | 1 + chain/types/message.go | 1 + chain/types/mock/chain.go | 1 + chain/types/vmcontext.go | 1 + chain/types_test.go | 1 + chain/vectors/gen/main.go | 1 + chain/vectors/vector_types.go | 1 + chain/vm/fvm.go | 1 + chain/vm/gas.go | 3 ++- chain/vm/gas_v0.go | 3 ++- chain/vm/invoker.go | 3 ++- chain/vm/invoker_test.go | 3 ++- chain/vm/mkactor.go | 13 +++++----- chain/vm/runtime.go | 15 +++++------ chain/vm/runtime_test.go | 1 + chain/vm/syscalls.go | 5 ++-- chain/vm/vm.go | 1 + chain/vm/vmi.go | 1 + chain/wallet/key/key.go | 1 + chain/wallet/ledger/ledger.go | 1 + chain/wallet/multi.go | 1 + chain/wallet/wallet.go | 1 + cli/auth.go | 1 + cli/backup.go | 1 + cli/chain.go | 13 +++++----- cli/chain_test.go | 3 ++- cli/client.go | 1 + cli/client_retr.go | 1 + cli/disputer.go | 5 ++-- cli/evm.go | 1 + cli/filplus.go | 1 + cli/info.go | 1 + cli/mpool.go | 1 + cli/mpool_manage.go | 1 + cli/mpool_test.go | 1 + cli/multisig.go | 5 ++-- cli/net.go | 1 + cli/params.go | 1 + cli/paych.go | 1 + cli/send.go | 1 + cli/send_test.go | 1 + cli/sending_ui.go | 1 + cli/services.go | 1 + cli/services_send_test.go | 1 + cli/servicesmock_test.go | 1 + cli/state.go | 1 + cli/sync.go | 1 + cli/sync_test.go | 1 + cli/util/api.go | 1 + cli/util/epoch.go | 1 + cli/wallet.go | 1 + cli/wallet_test.go | 1 + cmd/chain-noise/main.go | 1 + cmd/lotus-bench/caching_verifier.go | 1 + cmd/lotus-bench/import.go | 1 + cmd/lotus-bench/main.go | 3 ++- cmd/lotus-bench/simple.go | 3 ++- cmd/lotus-fountain/main.go | 1 + cmd/lotus-gateway/main.go | 1 + cmd/lotus-health/main.go | 1 + cmd/lotus-miner/actor.go | 1 + cmd/lotus-miner/actor_test.go | 1 + cmd/lotus-miner/backup.go | 1 + cmd/lotus-miner/info.go | 3 ++- cmd/lotus-miner/init.go | 7 +++--- cmd/lotus-miner/init_restore.go | 1 + cmd/lotus-miner/init_service.go | 1 + cmd/lotus-miner/main.go | 1 + cmd/lotus-miner/market.go | 1 + cmd/lotus-miner/precommits-info.go | 3 ++- cmd/lotus-miner/proving.go | 1 + cmd/lotus-miner/retrieval-deals.go | 1 + cmd/lotus-miner/sealing.go | 1 + cmd/lotus-miner/sectors.go | 1 + cmd/lotus-miner/storage.go | 1 + cmd/lotus-pcr/main.go | 3 ++- cmd/lotus-provider/run.go | 1 + cmd/lotus-seed/genesis.go | 1 + cmd/lotus-seed/main.go | 1 + cmd/lotus-seed/seed/seed.go | 1 + cmd/lotus-shed/actor.go | 3 ++- cmd/lotus-shed/balancer.go | 1 + cmd/lotus-shed/balances.go | 1 + cmd/lotus-shed/cid.go | 1 + cmd/lotus-shed/consensus.go | 1 + cmd/lotus-shed/cron-count.go | 1 + cmd/lotus-shed/deal-label.go | 3 ++- cmd/lotus-shed/diff.go | 1 + cmd/lotus-shed/election.go | 3 ++- cmd/lotus-shed/eth.go | 1 + cmd/lotus-shed/export.go | 1 + cmd/lotus-shed/fevmanalytics.go | 1 + cmd/lotus-shed/fip-0036.go | 1 + cmd/lotus-shed/fr32.go | 1 + cmd/lotus-shed/frozen-miners.go | 3 ++- cmd/lotus-shed/gas-estimation.go | 1 + cmd/lotus-shed/genesis-verify.go | 1 + cmd/lotus-shed/hello.go | 1 + cmd/lotus-shed/indexes.go | 1 + cmd/lotus-shed/invariants.go | 1 + cmd/lotus-shed/jwt.go | 1 + cmd/lotus-shed/ledger.go | 1 + cmd/lotus-shed/market.go | 1 + cmd/lotus-shed/math.go | 3 ++- cmd/lotus-shed/mempool-stats.go | 1 + cmd/lotus-shed/migrations.go | 3 ++- cmd/lotus-shed/miner-multisig.go | 1 + cmd/lotus-shed/miner-peerid.go | 5 ++-- cmd/lotus-shed/miner-types.go | 5 ++-- cmd/lotus-shed/miner.go | 5 ++-- cmd/lotus-shed/msg.go | 3 ++- cmd/lotus-shed/msig.go | 1 + cmd/lotus-shed/nonce-fix.go | 1 + cmd/lotus-shed/params.go | 1 + cmd/lotus-shed/postfind.go | 3 ++- cmd/lotus-shed/proofs.go | 1 + cmd/lotus-shed/pruning.go | 1 + cmd/lotus-shed/sectors.go | 3 ++- cmd/lotus-shed/send-csv.go | 1 + cmd/lotus-shed/signatures.go | 1 + cmd/lotus-shed/state-stats.go | 1 + cmd/lotus-shed/stateroot-stats.go | 1 + cmd/lotus-shed/sync.go | 1 + cmd/lotus-shed/terminations.go | 3 ++- cmd/lotus-shed/verifreg.go | 3 ++- cmd/lotus-sim/info.go | 1 + cmd/lotus-sim/info_capacity.go | 1 + cmd/lotus-sim/info_commit.go | 1 + cmd/lotus-sim/info_state.go | 1 + cmd/lotus-sim/info_wdpost.go | 1 + cmd/lotus-sim/simulation/block.go | 1 + .../simulation/blockbuilder/blockbuilder.go | 1 + cmd/lotus-sim/simulation/messages.go | 3 ++- cmd/lotus-sim/simulation/mock/mock.go | 3 ++- cmd/lotus-sim/simulation/simulation.go | 3 ++- .../simulation/stages/commit_queue.go | 1 + .../simulation/stages/commit_queue_test.go | 1 + .../simulation/stages/funding_stage.go | 1 + cmd/lotus-sim/simulation/stages/interface.go | 1 + .../simulation/stages/precommit_stage.go | 3 ++- .../simulation/stages/provecommit_stage.go | 5 ++-- cmd/lotus-sim/simulation/stages/util.go | 1 + .../simulation/stages/windowpost_stage.go | 1 + cmd/lotus-stats/main.go | 1 + cmd/lotus-wallet/interactive.go | 1 + cmd/lotus-wallet/logged.go | 1 + cmd/lotus-wallet/main.go | 1 + cmd/lotus-worker/main.go | 1 + cmd/lotus-worker/sealworker/rpc.go | 1 + cmd/lotus/backup.go | 1 + cmd/lotus/daemon.go | 1 + cmd/lotus/debug_advance.go | 1 + cmd/tvx/codenames.go | 1 + cmd/tvx/codenames_test.go | 1 + cmd/tvx/exec.go | 3 ++- cmd/tvx/extract_many.go | 1 + cmd/tvx/extract_message.go | 3 ++- cmd/tvx/extract_tipset.go | 3 ++- cmd/tvx/main.go | 1 + cmd/tvx/simulate.go | 3 ++- cmd/tvx/state.go | 1 + conformance/chaos/actor.go | 2 ++ conformance/chaos/actor_test.go | 2 ++ conformance/chaos/cbor_gen.go | 2 ++ conformance/chaos/gen/gen.go | 1 + conformance/chaos/ids.go | 1 + conformance/corpus_test.go | 1 + conformance/driver.go | 3 +++ conformance/rand_fixed.go | 2 ++ conformance/rand_record.go | 2 ++ conformance/rand_replay.go | 2 ++ conformance/reporter.go | 1 + conformance/runner.go | 3 +++ gateway/eth_sub.go | 2 ++ gateway/handler.go | 3 +++ gateway/node.go | 3 +++ gateway/node_test.go | 3 +++ gateway/proxy_eth.go | 3 +++ gateway/proxy_fil.go | 3 +++ genesis/types.go | 3 +++ itests/api_test.go | 1 + itests/batch_deal_test.go | 1 + itests/deadlines_test.go | 3 ++- itests/deals_512mb_test.go | 1 + itests/deals_anycid_test.go | 1 + itests/deals_concurrent_test.go | 1 + itests/deals_invalid_utf8_label_test.go | 1 + itests/deals_max_staging_deals_test.go | 1 + itests/deals_offline_test.go | 1 + itests/deals_padding_test.go | 1 + .../deals_partial_retrieval_dm-level_test.go | 1 + itests/deals_partial_retrieval_test.go | 1 + itests/deals_pricing_test.go | 1 + itests/deals_publish_test.go | 3 ++- itests/deals_remote_retrieval_test.go | 1 + itests/deals_retry_deal_no_funds_test.go | 1 + itests/decode_params_test.go | 1 + itests/dup_mpool_messages_test.go | 1 + itests/eth_account_abstraction_test.go | 1 + itests/eth_api_test.go | 1 + itests/eth_balance_test.go | 1 + itests/eth_block_hash_test.go | 1 + itests/eth_conformance_test.go | 1 + itests/eth_deploy_test.go | 1 + itests/eth_fee_history_test.go | 1 + itests/eth_filter_test.go | 1 + itests/eth_hash_lookup_test.go | 1 + itests/eth_transactions_test.go | 1 + itests/fevm_address_test.go | 1 + itests/fevm_events_test.go | 1 + itests/fevm_test.go | 1 + itests/gas_estimation_test.go | 1 + itests/gateway_test.go | 5 ++-- itests/get_messages_in_ts_test.go | 1 + itests/kit/blockminer.go | 1 + itests/kit/client.go | 3 ++- itests/kit/control.go | 3 ++- itests/kit/deals.go | 1 + itests/kit/ensemble.go | 5 ++-- itests/kit/ensemble_opts.go | 1 + itests/kit/ensemble_opts_nv.go | 1 + itests/kit/evm.go | 1 + itests/kit/funds.go | 1 + itests/kit/init.go | 1 + itests/kit/node_full.go | 1 + itests/kit/node_miner.go | 1 + itests/kit/node_opts.go | 1 + itests/kit/rpc.go | 1 + itests/kit/state.go | 1 + itests/lite_migration_test.go | 3 ++- itests/lookup_robust_address_test.go | 1 + itests/mempool_test.go | 1 + itests/migration_test.go | 1 + itests/mpool_msg_uuid_test.go | 1 + itests/mpool_push_with_uuid_test.go | 1 + itests/msgindex_test.go | 1 + itests/multisig/suite.go | 1 + itests/multisig_test.go | 1 + itests/nonce_test.go | 1 + itests/path_detach_redeclare_test.go | 1 + itests/paych_api_test.go | 1 + itests/paych_cli_test.go | 1 + itests/pending_deal_allocation_test.go | 1 + itests/raft_messagesigner_test.go | 1 + itests/remove_verifreg_datacap_test.go | 1 + itests/sector_import_full_test.go | 1 + itests/sector_import_simple_test.go | 1 + itests/sector_miner_collateral_test.go | 1 + itests/sector_numassign_test.go | 1 + itests/sector_pledge_test.go | 3 ++- itests/sector_terminate_test.go | 1 + itests/sector_unseal_test.go | 1 + itests/self_sent_txn_test.go | 1 + itests/splitstore_test.go | 5 ++-- itests/verifreg_test.go | 1 + itests/wdpost_config_test.go | 1 + itests/wdpost_dispute_test.go | 1 + itests/wdpost_test.go | 1 + itests/wdpost_worker_config_test.go | 1 + itests/worker_test.go | 1 + lib/consensus/raft/consensus.go | 1 + lib/rpcenc/reader.go | 1 + lib/rpcenc/reader_test.go | 1 + lib/sigs/bls/bls_test.go | 1 + lib/sigs/bls/init.go | 1 + lib/sigs/delegated/init.go | 1 + lib/sigs/secp/init.go | 1 + lib/sigs/sigs.go | 1 + lib/unixfs/filestore.go | 1 + markets/dagstore/mount_test.go | 1 + markets/dagstore/wrapper.go | 1 + markets/dagstore/wrapper_migration_test.go | 1 + markets/dagstore/wrapper_test.go | 1 + markets/dealfilter/cli.go | 1 + markets/journal.go | 1 + markets/pricing/cli.go | 1 + markets/retrievaladapter/client.go | 1 + markets/retrievaladapter/client_blockstore.go | 1 + markets/retrievaladapter/provider.go | 1 + markets/retrievaladapter/provider_test.go | 1 + markets/sectoraccessor/sectoraccessor.go | 1 + markets/storageadapter/api.go | 1 + markets/storageadapter/client.go | 3 ++- markets/storageadapter/client_blockstore.go | 1 + markets/storageadapter/dealpublisher.go | 1 + markets/storageadapter/dealpublisher_test.go | 3 ++- markets/storageadapter/dealstatematcher.go | 1 + .../storageadapter/dealstatematcher_test.go | 7 +++--- .../storageadapter/ondealsectorcommitted.go | 1 + .../ondealsectorcommitted_test.go | 3 ++- markets/storageadapter/provider.go | 1 + markets/utils/converters.go | 1 + metrics/metrics.go | 1 + miner/miner.go | 3 +++ miner/testminer.go | 3 +++ miner/warmup.go | 3 +++ node/builder.go | 2 ++ node/builder_chain.go | 3 +++ node/builder_miner.go | 3 +++ node/bundle/bundle.go | 3 +++ node/config/def.go | 3 +++ node/config/def_test.go | 1 + node/config/load.go | 1 + node/config/load_test.go | 1 + node/config/storage.go | 2 ++ node/config/types.go | 1 + node/health.go | 2 ++ node/hello/cbor_gen.go | 2 ++ node/hello/hello.go | 3 +++ node/impl/backup.go | 2 ++ node/impl/client/car_helpers.go | 1 + node/impl/client/client.go | 3 +++ node/impl/client/client_test.go | 2 ++ node/impl/common/common.go | 3 +++ node/impl/full.go | 2 ++ node/impl/full/chain.go | 3 +++ node/impl/full/dummy.go | 3 +++ node/impl/full/eth.go | 3 +++ node/impl/full/eth_event.go | 3 +++ node/impl/full/eth_test.go | 3 +++ node/impl/full/eth_trace.go | 3 +++ node/impl/full/eth_utils.go | 3 +++ node/impl/full/gas.go | 3 +++ node/impl/full/gas_test.go | 3 +++ node/impl/full/mpool.go | 3 +++ node/impl/full/multisig.go | 3 +++ node/impl/full/raft.go | 2 ++ node/impl/full/state.go | 3 +++ node/impl/full/sync.go | 2 ++ node/impl/full/txhashmanager.go | 2 ++ node/impl/full/wallet.go | 3 +++ node/impl/market/market.go | 3 +++ node/impl/net/conngater.go | 2 ++ node/impl/net/net.go | 2 ++ node/impl/net/protect.go | 1 + node/impl/net/rcmgr.go | 2 ++ node/impl/paych/paych.go | 3 +++ node/impl/remoteworker.go | 3 +++ node/impl/storminer.go | 3 +++ node/modules/actorevent.go | 3 +++ node/modules/alerts.go | 1 + node/modules/blockstore.go | 2 ++ node/modules/chain.go | 2 ++ node/modules/client.go | 3 +++ node/modules/core.go | 3 +++ node/modules/dtypes/api.go | 1 + node/modules/dtypes/miner.go | 3 +++ node/modules/dtypes/mpool.go | 1 + node/modules/dtypes/scorekeeper.go | 1 + node/modules/dtypes/storage.go | 2 ++ node/modules/ethmodule.go | 3 +++ node/modules/faultreport.go | 1 + node/modules/genesis.go | 2 ++ node/modules/graphsync.go | 2 ++ node/modules/helpers/helpers.go | 1 + node/modules/ipfs.go | 2 ++ node/modules/lp2p/addrs.go | 1 + node/modules/lp2p/conngater.go | 1 + node/modules/lp2p/discovery.go | 2 ++ node/modules/lp2p/host.go | 2 ++ node/modules/lp2p/libp2p.go | 2 ++ node/modules/lp2p/pubsub.go | 2 ++ node/modules/lp2p/rcmgr.go | 2 ++ node/modules/lp2p/relay.go | 1 + node/modules/lp2p/routing.go | 1 + node/modules/lp2p/smux.go | 1 + node/modules/mpoolnonceapi.go | 3 +++ node/modules/msgindex.go | 2 ++ node/modules/paych.go | 2 ++ node/modules/rpc.go | 2 ++ node/modules/services.go | 3 +++ node/modules/stmgr.go | 1 + node/modules/storage.go | 2 ++ node/modules/storageminer.go | 3 +++ node/modules/storageminer_dagstore.go | 3 +++ node/modules/storageminer_idxprov.go | 3 +++ node/modules/storageminer_idxprov_test.go | 3 +++ node/modules/storageminer_svc.go | 2 ++ node/modules/testing/beacon.go | 1 + node/modules/testing/genesis.go | 2 ++ .../modules/tracer/elasticsearch_transport.go | 1 + node/modules/tracer/tracer.go | 1 + node/modules/tracer/tracer_test.go | 1 + node/options.go | 1 + node/repo/blockstore_opts.go | 1 + node/repo/fsrepo.go | 2 ++ node/repo/fsrepo_ds.go | 1 + node/repo/imports/manager.go | 2 ++ node/repo/interface.go | 2 ++ node/repo/memrepo.go | 2 ++ node/repo/repo_test.go | 2 ++ node/rpc.go | 3 +++ node/shutdown_test.go | 1 + node/testopts.go | 2 ++ paychmgr/accessorcache.go | 1 + paychmgr/cbor_gen.go | 2 ++ paychmgr/manager.go | 3 +++ paychmgr/mock_test.go | 3 +++ paychmgr/msglistener_test.go | 1 + paychmgr/paych.go | 3 +++ paychmgr/paych_test.go | 3 +++ paychmgr/paychget_test.go | 3 +++ paychmgr/paychvoucherfunds_test.go | 3 +++ paychmgr/settle_test.go | 2 ++ paychmgr/settler/settler.go | 3 +++ paychmgr/simple.go | 3 +++ paychmgr/state.go | 2 ++ paychmgr/store.go | 3 +++ paychmgr/store_test.go | 2 ++ paychmgr/util.go | 1 + provider/address.go | 1 + provider/lpmessage/sender.go | 1 + provider/lpwindow/compute_do.go | 3 ++- provider/lpwindow/compute_task.go | 1 + provider/lpwindow/compute_task_test.go | 1 + provider/lpwindow/faults_simple.go | 1 + provider/lpwindow/recover_task.go | 1 + provider/lpwindow/submit_task.go | 1 + storage/ctladdr/addresses.go | 1 + storage/paths/db_index.go | 1 + storage/paths/http_handler.go | 1 + storage/paths/http_handler_test.go | 1 + storage/paths/index.go | 1 + storage/paths/index_locks.go | 1 + storage/paths/index_locks_test.go | 1 + storage/paths/index_proxy.go | 1 + storage/paths/index_test.go | 1 + storage/paths/interface.go | 1 + storage/paths/local.go | 1 + storage/paths/mocks/index.go | 1 + storage/paths/mocks/pf.go | 1 + storage/paths/mocks/store.go | 1 + storage/paths/remote.go | 1 + storage/paths/remote_test.go | 1 + storage/pipeline/cbor_gen.go | 1 + storage/pipeline/checks.go | 1 + storage/pipeline/commit_batch.go | 1 + storage/pipeline/commit_batch_test.go | 3 ++- storage/pipeline/currentdealinfo.go | 1 + storage/pipeline/currentdealinfo_test.go | 5 ++-- storage/pipeline/fsm.go | 1 + storage/pipeline/fsm_events.go | 1 + storage/pipeline/fsm_test.go | 1 + storage/pipeline/input.go | 1 + storage/pipeline/mocks/api.go | 1 + storage/pipeline/mocks/mock_commit_batcher.go | 1 + .../pipeline/mocks/mock_precommit_batcher.go | 1 + storage/pipeline/numassign.go | 1 + storage/pipeline/precommit_batch.go | 1 + storage/pipeline/precommit_batch_test.go | 3 ++- storage/pipeline/precommit_policy.go | 1 + storage/pipeline/precommit_policy_test.go | 1 + storage/pipeline/receive.go | 1 + storage/pipeline/sealing.go | 1 + storage/pipeline/states_failed.go | 1 + storage/pipeline/states_failed_test.go | 3 ++- storage/pipeline/states_proving.go | 1 + storage/pipeline/states_replica_update.go | 1 + storage/pipeline/states_sealing.go | 1 + storage/pipeline/stats.go | 1 + storage/pipeline/terminate_batch.go | 1 + storage/pipeline/types.go | 1 + storage/pipeline/types_test.go | 3 ++- storage/pipeline/upgrade_queue.go | 3 ++- storage/pipeline/utils.go | 1 + storage/sealer/faults.go | 1 + storage/sealer/ffiwrapper/basicfs/fs.go | 1 + storage/sealer/ffiwrapper/prover_cgo.go | 1 + storage/sealer/ffiwrapper/sealer_cgo.go | 1 + storage/sealer/ffiwrapper/sealer_test.go | 1 + storage/sealer/ffiwrapper/unseal_ranges.go | 1 + storage/sealer/ffiwrapper/verifier_cgo.go | 3 ++- storage/sealer/fr32/fr32_ffi_cmp_test.go | 1 + storage/sealer/fr32/fr32_test.go | 1 + storage/sealer/fr32/readers_test.go | 1 + storage/sealer/manager.go | 1 + storage/sealer/manager_post.go | 3 ++- storage/sealer/manager_test.go | 3 ++- storage/sealer/mock/mock.go | 1 + storage/sealer/mock/util.go | 1 + storage/sealer/partialfile/partialfile.go | 1 + storage/sealer/piece_provider.go | 1 + storage/sealer/piece_provider_test.go | 1 + storage/sealer/piece_reader.go | 1 + storage/sealer/sched.go | 1 + storage/sealer/sched_post.go | 1 + storage/sealer/sched_test.go | 1 + storage/sealer/selector_alloc.go | 1 + storage/sealer/selector_existing.go | 1 + storage/sealer/selector_move.go | 1 + storage/sealer/selector_task.go | 1 + storage/sealer/storiface/ffi.go | 1 + storage/sealer/storiface/index.go | 1 + storage/sealer/storiface/resources.go | 1 + storage/sealer/storiface/resources_test.go | 1 + storage/sealer/storiface/worker.go | 1 + storage/sealer/teststorage_test.go | 3 ++- storage/sealer/testworker_test.go | 1 + storage/sealer/worker_calltracker.go | 1 + storage/sealer/worker_local.go | 1 + storage/sealer/worker_local_test.go | 1 + storage/sealer/worker_tracked.go | 1 + storage/sectorblocks/blocks.go | 1 + storage/wdpost/wdpost_changehandler.go | 1 + storage/wdpost/wdpost_changehandler_test.go | 3 ++- storage/wdpost/wdpost_journal.go | 1 + storage/wdpost/wdpost_run.go | 3 ++- storage/wdpost/wdpost_run_faults.go | 1 + storage/wdpost/wdpost_run_test.go | 3 ++- storage/wdpost/wdpost_sched.go | 1 + storage/winning_prover.go | 1 + tools/stats/points/collect.go | 1 + tools/stats/sync/sync.go | 1 + 843 files changed, 1448 insertions(+), 398 deletions(-) diff --git a/api/api_common.go b/api/api_common.go index c3746d1f48a..2a887a26a75 100644 --- a/api/api_common.go +++ b/api/api_common.go @@ -4,8 +4,11 @@ import ( "context" "fmt" "time" + "github.com/google/uuid" + "github.com/filecoin-project/go-jsonrpc/auth" + apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/journal/alerting" ) diff --git a/api/api_errors.go b/api/api_errors.go index 1fa1bbc227a..fd157be5fd1 100644 --- a/api/api_errors.go +++ b/api/api_errors.go @@ -3,6 +3,7 @@ package api import ( "errors" "reflect" + "github.com/filecoin-project/go-jsonrpc" ) diff --git a/api/api_full.go b/api/api_full.go index ed8103002be..f919bc13bb4 100644 --- a/api/api_full.go +++ b/api/api_full.go @@ -5,10 +5,12 @@ import ( "encoding/json" "fmt" "time" + "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -23,6 +25,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" + apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/api_gateway.go b/api/api_gateway.go index 18a2a82f18e..27e7254573c 100644 --- a/api/api_gateway.go +++ b/api/api_gateway.go @@ -2,13 +2,16 @@ package api import ( "context" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" + apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/api_net.go b/api/api_net.go index 49417943c6d..cfcd8d87e06 100644 --- a/api/api_net.go +++ b/api/api_net.go @@ -3,6 +3,7 @@ package api import ( "context" "time" + "github.com/libp2p/go-libp2p/core/metrics" "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" diff --git a/api/api_storage.go b/api/api_storage.go index 722c6bd1617..d5b3d5c1d67 100644 --- a/api/api_storage.go +++ b/api/api_storage.go @@ -4,9 +4,11 @@ import ( "bytes" "context" "time" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -18,6 +20,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/market" abinetwork "github.com/filecoin-project/go-state-types/network" + builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/api_test.go b/api/api_test.go index 8397bba2727..1316d9fa4ca 100644 --- a/api/api_test.go +++ b/api/api_test.go @@ -10,8 +10,10 @@ import ( "runtime" "strings" "testing" + "github.com/stretchr/testify/require" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc" ) diff --git a/api/api_wallet.go b/api/api_wallet.go index dd533562197..973aaaf6d85 100644 --- a/api/api_wallet.go +++ b/api/api_wallet.go @@ -2,8 +2,10 @@ package api import ( "context" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/api/api_worker.go b/api/api_worker.go index 5fefee0647b..197ca898d43 100644 --- a/api/api_worker.go +++ b/api/api_worker.go @@ -2,10 +2,13 @@ package api import ( "context" + "github.com/google/uuid" "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/api/cbor_gen.go b/api/cbor_gen.go index 8b5049ffd71..fd2cb30b496 100644 --- a/api/cbor_gen.go +++ b/api/cbor_gen.go @@ -7,9 +7,11 @@ import ( "io" "math" "sort" + cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" + abi "github.com/filecoin-project/go-state-types/abi" paych "github.com/filecoin-project/go-state-types/builtin/v8/paych" market "github.com/filecoin-project/go-state-types/builtin/v9/market" diff --git a/api/client/client.go b/api/client/client.go index 72204302301..8b159c5b1c9 100644 --- a/api/client/client.go +++ b/api/client/client.go @@ -6,7 +6,9 @@ import ( "net/url" "path" "time" + "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/api/docgen-openrpc/cmd/docgen_openrpc.go b/api/docgen-openrpc/cmd/docgen_openrpc.go index 6bc80155673..5f6bc566f2b 100644 --- a/api/docgen-openrpc/cmd/docgen_openrpc.go +++ b/api/docgen-openrpc/cmd/docgen_openrpc.go @@ -6,6 +6,7 @@ import ( "io" "log" "os" + "github.com/filecoin-project/lotus/api/docgen" docgen_openrpc "github.com/filecoin-project/lotus/api/docgen-openrpc" ) diff --git a/api/docgen-openrpc/openrpc.go b/api/docgen-openrpc/openrpc.go index 404ce537d75..c9504ba89c1 100644 --- a/api/docgen-openrpc/openrpc.go +++ b/api/docgen-openrpc/openrpc.go @@ -5,10 +5,12 @@ import ( "go/ast" "net" "reflect" + "github.com/alecthomas/jsonschema" go_openrpc_reflect "github.com/etclabscore/go-openrpc-reflect" "github.com/ipfs/go-cid" meta_schema "github.com/open-rpc/meta-schema" + "github.com/filecoin-project/lotus/api/docgen" "github.com/filecoin-project/lotus/build" ) diff --git a/api/docgen/cmd/docgen.go b/api/docgen/cmd/docgen.go index 3b1868028cf..9ae2df2e707 100644 --- a/api/docgen/cmd/docgen.go +++ b/api/docgen/cmd/docgen.go @@ -7,6 +7,7 @@ import ( "reflect" "sort" "strings" + "github.com/filecoin-project/lotus/api/docgen" ) diff --git a/api/docgen/docgen.go b/api/docgen/docgen.go index 25ece344246..01862960076 100644 --- a/api/docgen/docgen.go +++ b/api/docgen/docgen.go @@ -12,6 +12,7 @@ import ( "strings" "time" "unicode" + "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" @@ -23,6 +24,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "github.com/multiformats/go-multiaddr" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -33,6 +35,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/api/v0api" diff --git a/api/mocks/mock_full.go b/api/mocks/mock_full.go index 3cd85d969d0..856d83813af 100644 --- a/api/mocks/mock_full.go +++ b/api/mocks/mock_full.go @@ -33,6 +33,7 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" + api "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" miner0 "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/proxy_gen.go b/api/proxy_gen.go index 05a29dc8424..2d1333495b3 100644 --- a/api/proxy_gen.go +++ b/api/proxy_gen.go @@ -6,6 +6,7 @@ import ( "context" "encoding/json" "time" + "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" @@ -14,6 +15,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -29,6 +31,7 @@ import ( "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" + apitypes "github.com/filecoin-project/lotus/api/types" builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/proxy_util_test.go b/api/proxy_util_test.go index aa4f3b55bef..adc78a7d15c 100644 --- a/api/proxy_util_test.go +++ b/api/proxy_util_test.go @@ -3,6 +3,7 @@ package api import ( "testing" + "github.com/stretchr/testify/require" ) diff --git a/api/types.go b/api/types.go index a5d1f913da6..93ed4083fb8 100644 --- a/api/types.go +++ b/api/types.go @@ -4,6 +4,7 @@ import ( "encoding/json" "fmt" "time" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-graphsync" @@ -13,10 +14,12 @@ import ( "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" ma "github.com/multiformats/go-multiaddr" + "github.com/filecoin-project/go-address" datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/api/utils.go b/api/utils.go index eda1fdeccdf..a9d02c31bb2 100644 --- a/api/utils.go +++ b/api/utils.go @@ -2,6 +2,7 @@ package api import ( "context" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" ) diff --git a/api/v0api/full.go b/api/v0api/full.go index b72b6efc048..d92d5a95c8e 100644 --- a/api/v0api/full.go +++ b/api/v0api/full.go @@ -2,10 +2,12 @@ package v0api import ( "context" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" textselector "github.com/ipld/go-ipld-selector-text-lite" "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -17,6 +19,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/v0api/gateway.go b/api/v0api/gateway.go index fc2990b28b9..1a7f7d3ac92 100644 --- a/api/v0api/gateway.go +++ b/api/v0api/gateway.go @@ -2,13 +2,16 @@ package v0api import ( "context" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/api/v0api/permissioned.go b/api/v0api/permissioned.go index ad64bc29ede..95fccdfbb6c 100644 --- a/api/v0api/permissioned.go +++ b/api/v0api/permissioned.go @@ -2,6 +2,7 @@ package v0api import ( "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" ) diff --git a/api/v0api/proxy_gen.go b/api/v0api/proxy_gen.go index b61f0f76ebb..bd37f64298d 100644 --- a/api/v0api/proxy_gen.go +++ b/api/v0api/proxy_gen.go @@ -4,10 +4,12 @@ package v0api import ( "context" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -19,6 +21,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" abinetwork "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/api/v0api/v0mocks/mock_full.go b/api/v0api/v0mocks/mock_full.go index c5b30868fa7..7a722ed257c 100644 --- a/api/v0api/v0mocks/mock_full.go +++ b/api/v0api/v0mocks/mock_full.go @@ -8,6 +8,7 @@ import ( context "context" reflect "reflect" time "time" + gomock "github.com/golang/mock/gomock" uuid "github.com/google/uuid" blocks "github.com/ipfs/go-block-format" @@ -16,6 +17,7 @@ import ( network0 "github.com/libp2p/go-libp2p/core/network" peer "github.com/libp2p/go-libp2p/core/peer" protocol "github.com/libp2p/go-libp2p/core/protocol" + address "github.com/filecoin-project/go-address" bitfield "github.com/filecoin-project/go-bitfield" datatransfer "github.com/filecoin-project/go-data-transfer/v2" @@ -30,6 +32,7 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" + api "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" v0api "github.com/filecoin-project/lotus/api/v0api" diff --git a/api/v0api/v1_wrapper.go b/api/v0api/v1_wrapper.go index 382ed038134..265674e718f 100644 --- a/api/v0api/v1_wrapper.go +++ b/api/v0api/v1_wrapper.go @@ -2,15 +2,18 @@ package v0api import ( "context" + "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/blockstore/splitstore/splitstore.go b/blockstore/splitstore/splitstore.go index 0ba2eb1ed32..c1a95c8b060 100644 --- a/blockstore/splitstore/splitstore.go +++ b/blockstore/splitstore/splitstore.go @@ -18,6 +18,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/blockstore/splitstore/splitstore_compact.go b/blockstore/splitstore/splitstore_compact.go index 144e0146f62..47caca886da 100644 --- a/blockstore/splitstore/splitstore_compact.go +++ b/blockstore/splitstore/splitstore_compact.go @@ -19,6 +19,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/metrics" diff --git a/blockstore/splitstore/splitstore_test.go b/blockstore/splitstore/splitstore_test.go index 19a6b82b683..1b821654d08 100644 --- a/blockstore/splitstore/splitstore_test.go +++ b/blockstore/splitstore/splitstore_test.go @@ -20,6 +20,7 @@ import ( mh "github.com/multiformats/go-multihash" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" diff --git a/blockstore/splitstore/splitstore_warmup.go b/blockstore/splitstore/splitstore_warmup.go index 9b9ec11e0a8..7fb6f3b9d08 100644 --- a/blockstore/splitstore/splitstore_warmup.go +++ b/blockstore/splitstore/splitstore_warmup.go @@ -11,6 +11,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/build/bootstrap.go b/build/bootstrap.go index 43f629472a5..d86115398af 100644 --- a/build/bootstrap.go +++ b/build/bootstrap.go @@ -5,7 +5,9 @@ import ( "embed" "path" "strings" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/lotus/lib/addrutil" ) diff --git a/build/builtin_actors.go b/build/builtin_actors.go index 6296411aed9..2cf40a754d6 100644 --- a/build/builtin_actors.go +++ b/build/builtin_actors.go @@ -11,12 +11,15 @@ import ( "sort" "strconv" "strings" + "github.com/DataDog/zstd" "github.com/ipfs/go-cid" cbor "github.com/ipfs/go-ipld-cbor" "github.com/ipld/go-car" "golang.org/x/xerrors" + actorstypes "github.com/filecoin-project/go-state-types/actors" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/build/builtin_actors_test.go b/build/builtin_actors_test.go index 6549859a87f..bb133bdabd3 100644 --- a/build/builtin_actors_test.go +++ b/build/builtin_actors_test.go @@ -2,9 +2,12 @@ package build_test import ( "testing" + "github.com/stretchr/testify/require" + actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" ) diff --git a/build/drand.go b/build/drand.go index 21cda3bb57c..3b976ac9254 100644 --- a/build/drand.go +++ b/build/drand.go @@ -2,6 +2,7 @@ package build import ( "sort" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/build/genesis.go b/build/genesis.go index cd276fabead..6d94b38cf68 100644 --- a/build/genesis.go +++ b/build/genesis.go @@ -3,6 +3,7 @@ package build import ( "embed" "path" + logging "github.com/ipfs/go-log/v2" ) diff --git a/build/openrpc.go b/build/openrpc.go index 10d309e22eb..a50d6f51e78 100644 --- a/build/openrpc.go +++ b/build/openrpc.go @@ -5,6 +5,7 @@ import ( "compress/gzip" "embed" "encoding/json" + apitypes "github.com/filecoin-project/lotus/api/types" ) diff --git a/build/openrpc_test.go b/build/openrpc_test.go index 1818bacb9fe..05119fd5d70 100644 --- a/build/openrpc_test.go +++ b/build/openrpc_test.go @@ -3,6 +3,7 @@ package build import ( "testing" + apitypes "github.com/filecoin-project/lotus/api/types" ) diff --git a/build/panic_reporter.go b/build/panic_reporter.go index 5257faa845e..617d619eb46 100644 --- a/build/panic_reporter.go +++ b/build/panic_reporter.go @@ -10,6 +10,7 @@ import ( "strconv" "strings" "time" + "github.com/icza/backscanner" logging "github.com/ipfs/go-log/v2" ) diff --git a/build/params_2k.go b/build/params_2k.go index 4ec52d6131b..df4f3dececd 100644 --- a/build/params_2k.go +++ b/build/params_2k.go @@ -6,10 +6,13 @@ package build import ( "os" "strconv" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_butterfly.go b/build/params_butterfly.go index 1a40c0c0a17..371c99d5f59 100644 --- a/build/params_butterfly.go +++ b/build/params_butterfly.go @@ -5,11 +5,13 @@ package build import ( "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_calibnet.go b/build/params_calibnet.go index 7ee29417270..2cc53925cd1 100644 --- a/build/params_calibnet.go +++ b/build/params_calibnet.go @@ -6,12 +6,15 @@ package build import ( "os" "strconv" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_interop.go b/build/params_interop.go index a1981b8e4d9..104ae83c81c 100644 --- a/build/params_interop.go +++ b/build/params_interop.go @@ -6,12 +6,15 @@ package build import ( "os" "strconv" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_mainnet.go b/build/params_mainnet.go index 9a07dd03d8c..c607bcdd0d7 100644 --- a/build/params_mainnet.go +++ b/build/params_mainnet.go @@ -7,6 +7,7 @@ import ( "math" "os" "strconv" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" diff --git a/build/params_shared_funcs.go b/build/params_shared_funcs.go index 76530a053a7..d117264ab6f 100644 --- a/build/params_shared_funcs.go +++ b/build/params_shared_funcs.go @@ -3,7 +3,9 @@ package build import ( "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/protocol" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/build/params_shared_vals.go b/build/params_shared_vals.go index bf5819463e4..1d15c2fe8b6 100644 --- a/build/params_shared_vals.go +++ b/build/params_shared_vals.go @@ -6,10 +6,12 @@ package build import ( "math/big" "os" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/build/params_testground.go b/build/params_testground.go index eae08a83a5e..8073b0c6e7d 100644 --- a/build/params_testground.go +++ b/build/params_testground.go @@ -9,11 +9,14 @@ package build import ( "math/big" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/chain/actors/adt/diff_adt_test.go b/chain/actors/adt/diff_adt_test.go index 740ee3a1f25..7ea3c53e556 100644 --- a/chain/actors/adt/diff_adt_test.go +++ b/chain/actors/adt/diff_adt_test.go @@ -12,9 +12,10 @@ import ( typegen "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" - bstore "github.com/filecoin-project/lotus/blockstore" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + + bstore "github.com/filecoin-project/lotus/blockstore" ) func TestDiffAdtArray(t *testing.T) { diff --git a/chain/actors/aerrors/error_test.go b/chain/actors/aerrors/error_test.go index 7458908645e..8c3738c8816 100644 --- a/chain/actors/aerrors/error_test.go +++ b/chain/actors/aerrors/error_test.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/exitcode" + . "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/actors/builtin/account/account.go b/chain/actors/builtin/account/account.go index 3fdb1a72831..dcb60f80186 100644 --- a/chain/actors/builtin/account/account.go +++ b/chain/actors/builtin/account/account.go @@ -9,9 +9,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -19,6 +16,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) var Methods = builtin12.MethodsAccount diff --git a/chain/actors/builtin/account/v0.go b/chain/actors/builtin/account/v0.go index 987a386fd20..a41ee3879ed 100644 --- a/chain/actors/builtin/account/v0.go +++ b/chain/actors/builtin/account/v0.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account0 "github.com/filecoin-project/specs-actors/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account0 "github.com/filecoin-project/specs-actors/actors/builtin/account" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/account/v10.go b/chain/actors/builtin/account/v10.go index 4dfe82e4fda..ff87c421220 100644 --- a/chain/actors/builtin/account/v10.go +++ b/chain/actors/builtin/account/v10.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account10 "github.com/filecoin-project/go-state-types/builtin/v10/account" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v11.go b/chain/actors/builtin/account/v11.go index 03a7ddabf1f..7a0c5f556b4 100644 --- a/chain/actors/builtin/account/v11.go +++ b/chain/actors/builtin/account/v11.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account11 "github.com/filecoin-project/go-state-types/builtin/v11/account" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v12.go b/chain/actors/builtin/account/v12.go index 43a235751be..af2c4186fbd 100644 --- a/chain/actors/builtin/account/v12.go +++ b/chain/actors/builtin/account/v12.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account12 "github.com/filecoin-project/go-state-types/builtin/v12/account" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v2.go b/chain/actors/builtin/account/v2.go index 3f64e213343..db0af77e2a7 100644 --- a/chain/actors/builtin/account/v2.go +++ b/chain/actors/builtin/account/v2.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/account" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/account/v3.go b/chain/actors/builtin/account/v3.go index 4687dba6c0d..9e6c71ad0f6 100644 --- a/chain/actors/builtin/account/v3.go +++ b/chain/actors/builtin/account/v3.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/account" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/account/v4.go b/chain/actors/builtin/account/v4.go index 9077f88199c..907896312ee 100644 --- a/chain/actors/builtin/account/v4.go +++ b/chain/actors/builtin/account/v4.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/account" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/account/v5.go b/chain/actors/builtin/account/v5.go index 6389316f22b..8514ab325e1 100644 --- a/chain/actors/builtin/account/v5.go +++ b/chain/actors/builtin/account/v5.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/account" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/account/v6.go b/chain/actors/builtin/account/v6.go index 3dddb10ecb2..16369f38c26 100644 --- a/chain/actors/builtin/account/v6.go +++ b/chain/actors/builtin/account/v6.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/account" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/account/v7.go b/chain/actors/builtin/account/v7.go index 49de3d54aed..cd420da92ae 100644 --- a/chain/actors/builtin/account/v7.go +++ b/chain/actors/builtin/account/v7.go @@ -8,9 +8,10 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + account7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - account7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/account" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/account/v8.go b/chain/actors/builtin/account/v8.go index 68301269113..13b478de890 100644 --- a/chain/actors/builtin/account/v8.go +++ b/chain/actors/builtin/account/v8.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account8 "github.com/filecoin-project/go-state-types/builtin/v8/account" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/account/v9.go b/chain/actors/builtin/account/v9.go index dc98494bf64..fc1fc4d14bf 100644 --- a/chain/actors/builtin/account/v9.go +++ b/chain/actors/builtin/account/v9.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" account9 "github.com/filecoin-project/go-state-types/builtin/v9/account" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/builtin.go b/chain/actors/builtin/builtin.go index 12320f4b96d..4516683a85c 100644 --- a/chain/actors/builtin/builtin.go +++ b/chain/actors/builtin/builtin.go @@ -12,7 +12,6 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/chain/actors" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -20,6 +19,8 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" ) var InitActorAddr = builtin.InitActorAddr diff --git a/chain/actors/builtin/cron/cron.go b/chain/actors/builtin/cron/cron.go index 3134922e6e3..17b29178816 100644 --- a/chain/actors/builtin/cron/cron.go +++ b/chain/actors/builtin/cron/cron.go @@ -7,9 +7,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -17,6 +14,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/cron/v0.go b/chain/actors/builtin/cron/v0.go index 9e0c4613eb4..6dce524f643 100644 --- a/chain/actors/builtin/cron/v0.go +++ b/chain/actors/builtin/cron/v0.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron0 "github.com/filecoin-project/specs-actors/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron0 "github.com/filecoin-project/specs-actors/actors/builtin/cron" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/cron/v10.go b/chain/actors/builtin/cron/v10.go index d0d96a8ddbe..2d20e2401c3 100644 --- a/chain/actors/builtin/cron/v10.go +++ b/chain/actors/builtin/cron/v10.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron10 "github.com/filecoin-project/go-state-types/builtin/v10/cron" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v11.go b/chain/actors/builtin/cron/v11.go index aee675393de..5c489cede81 100644 --- a/chain/actors/builtin/cron/v11.go +++ b/chain/actors/builtin/cron/v11.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron11 "github.com/filecoin-project/go-state-types/builtin/v11/cron" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v12.go b/chain/actors/builtin/cron/v12.go index 72a04e98499..44f018d68af 100644 --- a/chain/actors/builtin/cron/v12.go +++ b/chain/actors/builtin/cron/v12.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron12 "github.com/filecoin-project/go-state-types/builtin/v12/cron" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v2.go b/chain/actors/builtin/cron/v2.go index 236882f0804..97b3ffbe08b 100644 --- a/chain/actors/builtin/cron/v2.go +++ b/chain/actors/builtin/cron/v2.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/cron" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/cron/v3.go b/chain/actors/builtin/cron/v3.go index 732301273c3..4c0d4f1d938 100644 --- a/chain/actors/builtin/cron/v3.go +++ b/chain/actors/builtin/cron/v3.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/cron" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/cron/v4.go b/chain/actors/builtin/cron/v4.go index 928153730e9..a222f0d93dc 100644 --- a/chain/actors/builtin/cron/v4.go +++ b/chain/actors/builtin/cron/v4.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/cron" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/cron/v5.go b/chain/actors/builtin/cron/v5.go index e70941ec5be..2487cbbc687 100644 --- a/chain/actors/builtin/cron/v5.go +++ b/chain/actors/builtin/cron/v5.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/cron" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/cron/v6.go b/chain/actors/builtin/cron/v6.go index 21b57933ac8..673e7588a9d 100644 --- a/chain/actors/builtin/cron/v6.go +++ b/chain/actors/builtin/cron/v6.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/cron" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/cron/v7.go b/chain/actors/builtin/cron/v7.go index 46c0d4b95a0..cd71bd41833 100644 --- a/chain/actors/builtin/cron/v7.go +++ b/chain/actors/builtin/cron/v7.go @@ -7,9 +7,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + cron7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/cron" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - cron7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/cron" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/cron/v8.go b/chain/actors/builtin/cron/v8.go index 71e9f9903e5..904de5496a6 100644 --- a/chain/actors/builtin/cron/v8.go +++ b/chain/actors/builtin/cron/v8.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron8 "github.com/filecoin-project/go-state-types/builtin/v8/cron" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/cron/v9.go b/chain/actors/builtin/cron/v9.go index f7160c193d4..201348b6cd8 100644 --- a/chain/actors/builtin/cron/v9.go +++ b/chain/actors/builtin/cron/v9.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" cron9 "github.com/filecoin-project/go-state-types/builtin/v9/cron" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/datacap.go b/chain/actors/builtin/datacap/datacap.go index 798c88c661f..0c8f04bbf40 100644 --- a/chain/actors/builtin/datacap/datacap.go +++ b/chain/actors/builtin/datacap/datacap.go @@ -10,6 +10,7 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/datacap/util.go b/chain/actors/builtin/datacap/util.go index ce59688f9fa..03e941d6e04 100644 --- a/chain/actors/builtin/datacap/util.go +++ b/chain/actors/builtin/datacap/util.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v10.go b/chain/actors/builtin/datacap/v10.go index e260cf8aab4..25eec4ea819 100644 --- a/chain/actors/builtin/datacap/v10.go +++ b/chain/actors/builtin/datacap/v10.go @@ -11,6 +11,7 @@ import ( datacap10 "github.com/filecoin-project/go-state-types/builtin/v10/datacap" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v11.go b/chain/actors/builtin/datacap/v11.go index 02d1ba04568..0c302b5e187 100644 --- a/chain/actors/builtin/datacap/v11.go +++ b/chain/actors/builtin/datacap/v11.go @@ -11,6 +11,7 @@ import ( datacap11 "github.com/filecoin-project/go-state-types/builtin/v11/datacap" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v12.go b/chain/actors/builtin/datacap/v12.go index afba772a0bd..91563a2b69d 100644 --- a/chain/actors/builtin/datacap/v12.go +++ b/chain/actors/builtin/datacap/v12.go @@ -11,6 +11,7 @@ import ( datacap12 "github.com/filecoin-project/go-state-types/builtin/v12/datacap" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/datacap/v9.go b/chain/actors/builtin/datacap/v9.go index f6ae057e245..1d239fb95be 100644 --- a/chain/actors/builtin/datacap/v9.go +++ b/chain/actors/builtin/datacap/v9.go @@ -11,6 +11,7 @@ import ( datacap9 "github.com/filecoin-project/go-state-types/builtin/v9/datacap" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/evm.go b/chain/actors/builtin/evm/evm.go index 3169ac4a955..98f860cac4a 100644 --- a/chain/actors/builtin/evm/evm.go +++ b/chain/actors/builtin/evm/evm.go @@ -8,6 +8,7 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/evm/v10.go b/chain/actors/builtin/evm/v10.go index 7633cb6defc..d467aa187d6 100644 --- a/chain/actors/builtin/evm/v10.go +++ b/chain/actors/builtin/evm/v10.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm10 "github.com/filecoin-project/go-state-types/builtin/v10/evm" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/v11.go b/chain/actors/builtin/evm/v11.go index 04ca7794957..9e2e984e46c 100644 --- a/chain/actors/builtin/evm/v11.go +++ b/chain/actors/builtin/evm/v11.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm11 "github.com/filecoin-project/go-state-types/builtin/v11/evm" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/evm/v12.go b/chain/actors/builtin/evm/v12.go index e6965f3d466..a107368faa4 100644 --- a/chain/actors/builtin/evm/v12.go +++ b/chain/actors/builtin/evm/v12.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" evm12 "github.com/filecoin-project/go-state-types/builtin/v12/evm" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/init/diff.go b/chain/actors/builtin/init/diff.go index be6825bfc8b..60b5a5bff15 100644 --- a/chain/actors/builtin/init/diff.go +++ b/chain/actors/builtin/init/diff.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/init/init.go b/chain/actors/builtin/init/init.go index b94fdb38830..41a763ecf61 100644 --- a/chain/actors/builtin/init/init.go +++ b/chain/actors/builtin/init/init.go @@ -10,10 +10,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -21,6 +17,11 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var ( diff --git a/chain/actors/builtin/init/v0.go b/chain/actors/builtin/init/v0.go index 7cc1af3e28f..7e48dda9e13 100644 --- a/chain/actors/builtin/init/v0.go +++ b/chain/actors/builtin/init/v0.go @@ -12,11 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" - init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/init/v10.go b/chain/actors/builtin/init/v10.go index 66a6f2cc592..dd8c778dd4a 100644 --- a/chain/actors/builtin/init/v10.go +++ b/chain/actors/builtin/init/v10.go @@ -15,6 +15,7 @@ import ( init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v11.go b/chain/actors/builtin/init/v11.go index 8d89988181e..3d8d72e499f 100644 --- a/chain/actors/builtin/init/v11.go +++ b/chain/actors/builtin/init/v11.go @@ -15,6 +15,7 @@ import ( init11 "github.com/filecoin-project/go-state-types/builtin/v11/init" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v12.go b/chain/actors/builtin/init/v12.go index 2ab877a153f..3eab7a74050 100644 --- a/chain/actors/builtin/init/v12.go +++ b/chain/actors/builtin/init/v12.go @@ -15,6 +15,7 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v2.go b/chain/actors/builtin/init/v2.go index 39d8f7693c7..c107bd52d59 100644 --- a/chain/actors/builtin/init/v2.go +++ b/chain/actors/builtin/init/v2.go @@ -12,11 +12,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/init/v3.go b/chain/actors/builtin/init/v3.go index 55354208512..0be11f9761a 100644 --- a/chain/actors/builtin/init/v3.go +++ b/chain/actors/builtin/init/v3.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/init/v4.go b/chain/actors/builtin/init/v4.go index 366cb5383c0..5ca6bc1c81b 100644 --- a/chain/actors/builtin/init/v4.go +++ b/chain/actors/builtin/init/v4.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/init/v5.go b/chain/actors/builtin/init/v5.go index 16f20af7088..f6450789d71 100644 --- a/chain/actors/builtin/init/v5.go +++ b/chain/actors/builtin/init/v5.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/init/v6.go b/chain/actors/builtin/init/v6.go index 49dc402dbea..4d2267aa1dc 100644 --- a/chain/actors/builtin/init/v6.go +++ b/chain/actors/builtin/init/v6.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/init/v7.go b/chain/actors/builtin/init/v7.go index 1f8287220a5..052faf98502 100644 --- a/chain/actors/builtin/init/v7.go +++ b/chain/actors/builtin/init/v7.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/node/modules/dtypes" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/init/v8.go b/chain/actors/builtin/init/v8.go index 061a5377f58..c7c7860d399 100644 --- a/chain/actors/builtin/init/v8.go +++ b/chain/actors/builtin/init/v8.go @@ -15,6 +15,7 @@ import ( init8 "github.com/filecoin-project/go-state-types/builtin/v8/init" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/init/v9.go b/chain/actors/builtin/init/v9.go index b246392260c..a221a4a7c4b 100644 --- a/chain/actors/builtin/init/v9.go +++ b/chain/actors/builtin/init/v9.go @@ -15,6 +15,7 @@ import ( init9 "github.com/filecoin-project/go-state-types/builtin/v9/init" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/actors/builtin/market/diff.go b/chain/actors/builtin/market/diff.go index 26b47c3f3b4..ef3c2c28d7c 100644 --- a/chain/actors/builtin/market/diff.go +++ b/chain/actors/builtin/market/diff.go @@ -6,6 +6,7 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/market/market.go b/chain/actors/builtin/market/market.go index 52238f3247a..39473d56062 100644 --- a/chain/actors/builtin/market/market.go +++ b/chain/actors/builtin/market/market.go @@ -17,9 +17,6 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -27,6 +24,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/market/v0.go b/chain/actors/builtin/market/v0.go index e597fe979f9..ca6970dfaa1 100644 --- a/chain/actors/builtin/market/v0.go +++ b/chain/actors/builtin/market/v0.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/market/v10.go b/chain/actors/builtin/market/v10.go index 1f454557cd4..878f0d46584 100644 --- a/chain/actors/builtin/market/v10.go +++ b/chain/actors/builtin/market/v10.go @@ -18,6 +18,7 @@ import ( adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v11.go b/chain/actors/builtin/market/v11.go index ec328c586f5..a6427220972 100644 --- a/chain/actors/builtin/market/v11.go +++ b/chain/actors/builtin/market/v11.go @@ -18,6 +18,7 @@ import ( adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v12.go b/chain/actors/builtin/market/v12.go index ef87b2cc794..56e651a9be5 100644 --- a/chain/actors/builtin/market/v12.go +++ b/chain/actors/builtin/market/v12.go @@ -18,6 +18,7 @@ import ( adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v2.go b/chain/actors/builtin/market/v2.go index 48287cb1f78..ba84e3b03a0 100644 --- a/chain/actors/builtin/market/v2.go +++ b/chain/actors/builtin/market/v2.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/market/v3.go b/chain/actors/builtin/market/v3.go index c6f007a0846..f6a0891e730 100644 --- a/chain/actors/builtin/market/v3.go +++ b/chain/actors/builtin/market/v3.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/market" + adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/market" - adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/market/v4.go b/chain/actors/builtin/market/v4.go index 9d7cf0736a5..629e833b67b 100644 --- a/chain/actors/builtin/market/v4.go +++ b/chain/actors/builtin/market/v4.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/market" + adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/market" - adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/market/v5.go b/chain/actors/builtin/market/v5.go index f12c24279e7..8925889791f 100644 --- a/chain/actors/builtin/market/v5.go +++ b/chain/actors/builtin/market/v5.go @@ -12,12 +12,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" + adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" - adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/market/v6.go b/chain/actors/builtin/market/v6.go index a4abd4d40c9..b57d49f9117 100644 --- a/chain/actors/builtin/market/v6.go +++ b/chain/actors/builtin/market/v6.go @@ -14,12 +14,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/market" + adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/market" - adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/market/v7.go b/chain/actors/builtin/market/v7.go index 597ca34f1c1..56a1db328f9 100644 --- a/chain/actors/builtin/market/v7.go +++ b/chain/actors/builtin/market/v7.go @@ -14,12 +14,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" + adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" - adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/market/v8.go b/chain/actors/builtin/market/v8.go index 067883a565e..9c68ee1fd86 100644 --- a/chain/actors/builtin/market/v8.go +++ b/chain/actors/builtin/market/v8.go @@ -17,6 +17,7 @@ import ( adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/market/v9.go b/chain/actors/builtin/market/v9.go index 0e6634d1606..d692c15ccb7 100644 --- a/chain/actors/builtin/market/v9.go +++ b/chain/actors/builtin/market/v9.go @@ -18,6 +18,7 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" verifregtypes "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" diff --git a/chain/actors/builtin/miner/diff.go b/chain/actors/builtin/miner/diff.go index c1536043774..d20a4374623 100644 --- a/chain/actors/builtin/miner/diff.go +++ b/chain/actors/builtin/miner/diff.go @@ -4,6 +4,7 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/miner.go b/chain/actors/builtin/miner/miner.go index cbc06a63687..1637cdadeda 100644 --- a/chain/actors/builtin/miner/miner.go +++ b/chain/actors/builtin/miner/miner.go @@ -15,9 +15,6 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -25,6 +22,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/miner/v0.go b/chain/actors/builtin/miner/v0.go index 4970e248b10..5301ed1cbca 100644 --- a/chain/actors/builtin/miner/v0.go +++ b/chain/actors/builtin/miner/v0.go @@ -16,10 +16,11 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/miner/v10.go b/chain/actors/builtin/miner/v10.go index 121cac90d8e..53dc90b4560 100644 --- a/chain/actors/builtin/miner/v10.go +++ b/chain/actors/builtin/miner/v10.go @@ -18,6 +18,7 @@ import ( adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v11.go b/chain/actors/builtin/miner/v11.go index b9497c2193d..11a91c26b23 100644 --- a/chain/actors/builtin/miner/v11.go +++ b/chain/actors/builtin/miner/v11.go @@ -18,6 +18,7 @@ import ( adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v12.go b/chain/actors/builtin/miner/v12.go index 0691674f1bd..90ecc97fd3d 100644 --- a/chain/actors/builtin/miner/v12.go +++ b/chain/actors/builtin/miner/v12.go @@ -18,6 +18,7 @@ import ( adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v2.go b/chain/actors/builtin/miner/v2.go index 5c9e9d22858..5a81ad31f95 100644 --- a/chain/actors/builtin/miner/v2.go +++ b/chain/actors/builtin/miner/v2.go @@ -15,10 +15,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/miner/v3.go b/chain/actors/builtin/miner/v3.go index ed278c08ed7..aa1574cf47d 100644 --- a/chain/actors/builtin/miner/v3.go +++ b/chain/actors/builtin/miner/v3.go @@ -15,11 +15,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/miner/v4.go b/chain/actors/builtin/miner/v4.go index 33dadc032cc..1faf30c0982 100644 --- a/chain/actors/builtin/miner/v4.go +++ b/chain/actors/builtin/miner/v4.go @@ -15,11 +15,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" miner4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/miner" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/miner/v5.go b/chain/actors/builtin/miner/v5.go index 771ee57243c..be4b5e0b2e9 100644 --- a/chain/actors/builtin/miner/v5.go +++ b/chain/actors/builtin/miner/v5.go @@ -15,11 +15,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/miner/v6.go b/chain/actors/builtin/miner/v6.go index 38dfdd8565b..fa8c30e4070 100644 --- a/chain/actors/builtin/miner/v6.go +++ b/chain/actors/builtin/miner/v6.go @@ -15,11 +15,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/miner/v7.go b/chain/actors/builtin/miner/v7.go index c5096507e63..d6bb0e16eae 100644 --- a/chain/actors/builtin/miner/v7.go +++ b/chain/actors/builtin/miner/v7.go @@ -15,11 +15,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" miner7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/miner" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/miner/v8.go b/chain/actors/builtin/miner/v8.go index a19c4e2aabb..06a205e7626 100644 --- a/chain/actors/builtin/miner/v8.go +++ b/chain/actors/builtin/miner/v8.go @@ -18,6 +18,7 @@ import ( adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/miner/v9.go b/chain/actors/builtin/miner/v9.go index dc13b646e39..6cbbd509ec3 100644 --- a/chain/actors/builtin/miner/v9.go +++ b/chain/actors/builtin/miner/v9.go @@ -18,6 +18,7 @@ import ( adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/diff.go b/chain/actors/builtin/multisig/diff.go index 7ffa1f2706f..f24931fb823 100644 --- a/chain/actors/builtin/multisig/diff.go +++ b/chain/actors/builtin/multisig/diff.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/message0.go b/chain/actors/builtin/multisig/message0.go index 699cadd0720..7dbdf444ce2 100644 --- a/chain/actors/builtin/multisig/message0.go +++ b/chain/actors/builtin/multisig/message0.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" multisig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message0 struct{ from address.Address } diff --git a/chain/actors/builtin/multisig/message10.go b/chain/actors/builtin/multisig/message10.go index cc94f079248..8f7bb5a6f2e 100644 --- a/chain/actors/builtin/multisig/message10.go +++ b/chain/actors/builtin/multisig/message10.go @@ -10,6 +10,7 @@ import ( multisig10 "github.com/filecoin-project/go-state-types/builtin/v10/multisig" init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message11.go b/chain/actors/builtin/multisig/message11.go index d6606b512b1..4c7520d5dea 100644 --- a/chain/actors/builtin/multisig/message11.go +++ b/chain/actors/builtin/multisig/message11.go @@ -10,6 +10,7 @@ import ( multisig11 "github.com/filecoin-project/go-state-types/builtin/v11/multisig" init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message12.go b/chain/actors/builtin/multisig/message12.go index 8a0dad844e9..43658c04b03 100644 --- a/chain/actors/builtin/multisig/message12.go +++ b/chain/actors/builtin/multisig/message12.go @@ -10,6 +10,7 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message2.go b/chain/actors/builtin/multisig/message2.go index e98f7057621..91ad17b84ab 100644 --- a/chain/actors/builtin/multisig/message2.go +++ b/chain/actors/builtin/multisig/message2.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message2 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message3.go b/chain/actors/builtin/multisig/message3.go index b397ec79099..4124e00c86b 100644 --- a/chain/actors/builtin/multisig/message3.go +++ b/chain/actors/builtin/multisig/message3.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" multisig3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message3 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message4.go b/chain/actors/builtin/multisig/message4.go index f56447c3bc6..33449df75f3 100644 --- a/chain/actors/builtin/multisig/message4.go +++ b/chain/actors/builtin/multisig/message4.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" multisig4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message4 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message5.go b/chain/actors/builtin/multisig/message5.go index 3b020560a0e..46c35dabce8 100644 --- a/chain/actors/builtin/multisig/message5.go +++ b/chain/actors/builtin/multisig/message5.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" multisig5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message5 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message6.go b/chain/actors/builtin/multisig/message6.go index 779c0009016..f528cfbb5da 100644 --- a/chain/actors/builtin/multisig/message6.go +++ b/chain/actors/builtin/multisig/message6.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" multisig6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message6 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message7.go b/chain/actors/builtin/multisig/message7.go index 8cb90d70d07..6e62dad13a0 100644 --- a/chain/actors/builtin/multisig/message7.go +++ b/chain/actors/builtin/multisig/message7.go @@ -6,12 +6,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" multisig7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/multisig" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message7 struct{ message0 } diff --git a/chain/actors/builtin/multisig/message8.go b/chain/actors/builtin/multisig/message8.go index 7f6fa8de223..390c94691e4 100644 --- a/chain/actors/builtin/multisig/message8.go +++ b/chain/actors/builtin/multisig/message8.go @@ -10,6 +10,7 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig8 "github.com/filecoin-project/go-state-types/builtin/v8/multisig" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/message9.go b/chain/actors/builtin/multisig/message9.go index 4a1aa46f532..907bec7d556 100644 --- a/chain/actors/builtin/multisig/message9.go +++ b/chain/actors/builtin/multisig/message9.go @@ -10,6 +10,7 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" multisig9 "github.com/filecoin-project/go-state-types/builtin/v9/multisig" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/multisig/multisig.go b/chain/actors/builtin/multisig/multisig.go index bea27523c7e..71a3b7b2237 100644 --- a/chain/actors/builtin/multisig/multisig.go +++ b/chain/actors/builtin/multisig/multisig.go @@ -15,9 +15,6 @@ import ( msig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -25,6 +22,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) func Load(store adt.Store, act *types.Actor) (State, error) { diff --git a/chain/actors/builtin/multisig/v0.go b/chain/actors/builtin/multisig/v0.go index a36efb456f6..86bfdaaf37a 100644 --- a/chain/actors/builtin/multisig/v0.go +++ b/chain/actors/builtin/multisig/v0.go @@ -13,10 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/multisig/v10.go b/chain/actors/builtin/multisig/v10.go index 6d6aa636567..d87fc5807ab 100644 --- a/chain/actors/builtin/multisig/v10.go +++ b/chain/actors/builtin/multisig/v10.go @@ -16,6 +16,7 @@ import ( msig10 "github.com/filecoin-project/go-state-types/builtin/v10/multisig" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v11.go b/chain/actors/builtin/multisig/v11.go index 08724d9f01a..3627dc959cb 100644 --- a/chain/actors/builtin/multisig/v11.go +++ b/chain/actors/builtin/multisig/v11.go @@ -16,6 +16,7 @@ import ( msig11 "github.com/filecoin-project/go-state-types/builtin/v11/multisig" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v12.go b/chain/actors/builtin/multisig/v12.go index c3200251213..d3d2f3809a9 100644 --- a/chain/actors/builtin/multisig/v12.go +++ b/chain/actors/builtin/multisig/v12.go @@ -16,6 +16,7 @@ import ( msig12 "github.com/filecoin-project/go-state-types/builtin/v12/multisig" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v2.go b/chain/actors/builtin/multisig/v2.go index 79b83b8fe61..77330d5139d 100644 --- a/chain/actors/builtin/multisig/v2.go +++ b/chain/actors/builtin/multisig/v2.go @@ -13,10 +13,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/multisig/v3.go b/chain/actors/builtin/multisig/v3.go index 7c086b52fb4..e8659093fc7 100644 --- a/chain/actors/builtin/multisig/v3.go +++ b/chain/actors/builtin/multisig/v3.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" msig3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/multisig" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/multisig/v4.go b/chain/actors/builtin/multisig/v4.go index 6fe2f0ce353..ddaac547056 100644 --- a/chain/actors/builtin/multisig/v4.go +++ b/chain/actors/builtin/multisig/v4.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" msig4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/multisig" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/multisig/v5.go b/chain/actors/builtin/multisig/v5.go index 2ff91ebfbd9..50474d5fd4a 100644 --- a/chain/actors/builtin/multisig/v5.go +++ b/chain/actors/builtin/multisig/v5.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" msig5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/multisig" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/multisig/v6.go b/chain/actors/builtin/multisig/v6.go index 05ca85fa6f1..c51404dc49b 100644 --- a/chain/actors/builtin/multisig/v6.go +++ b/chain/actors/builtin/multisig/v6.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" msig6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/multisig" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/multisig/v7.go b/chain/actors/builtin/multisig/v7.go index 0eae904c23c..9ddce167af6 100644 --- a/chain/actors/builtin/multisig/v7.go +++ b/chain/actors/builtin/multisig/v7.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" msig7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/multisig" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/multisig/v8.go b/chain/actors/builtin/multisig/v8.go index 41055e77bdb..b28ec5684d0 100644 --- a/chain/actors/builtin/multisig/v8.go +++ b/chain/actors/builtin/multisig/v8.go @@ -16,6 +16,7 @@ import ( msig8 "github.com/filecoin-project/go-state-types/builtin/v8/multisig" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/multisig/v9.go b/chain/actors/builtin/multisig/v9.go index 4cba947860a..faa3b7d3731 100644 --- a/chain/actors/builtin/multisig/v9.go +++ b/chain/actors/builtin/multisig/v9.go @@ -16,6 +16,7 @@ import ( msig9 "github.com/filecoin-project/go-state-types/builtin/v9/multisig" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/message0.go b/chain/actors/builtin/paych/message0.go index 06402750250..d5a112d43b2 100644 --- a/chain/actors/builtin/paych/message0.go +++ b/chain/actors/builtin/paych/message0.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" init0 "github.com/filecoin-project/specs-actors/actors/builtin/init" paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message0 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message10.go b/chain/actors/builtin/paych/message10.go index 996c9e6ffce..03daa7bcf2f 100644 --- a/chain/actors/builtin/paych/message10.go +++ b/chain/actors/builtin/paych/message10.go @@ -10,6 +10,7 @@ import ( init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" paych10 "github.com/filecoin-project/go-state-types/builtin/v10/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message11.go b/chain/actors/builtin/paych/message11.go index 4f5ce6e7f49..7a9ec1c8e5f 100644 --- a/chain/actors/builtin/paych/message11.go +++ b/chain/actors/builtin/paych/message11.go @@ -10,6 +10,7 @@ import ( init11 "github.com/filecoin-project/go-state-types/builtin/v11/init" paych11 "github.com/filecoin-project/go-state-types/builtin/v11/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message12.go b/chain/actors/builtin/paych/message12.go index 047de49768d..bd821641ae4 100644 --- a/chain/actors/builtin/paych/message12.go +++ b/chain/actors/builtin/paych/message12.go @@ -10,6 +10,7 @@ import ( init12 "github.com/filecoin-project/go-state-types/builtin/v12/init" paych12 "github.com/filecoin-project/go-state-types/builtin/v12/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message2.go b/chain/actors/builtin/paych/message2.go index ee4a50fb950..89a6e74ce9b 100644 --- a/chain/actors/builtin/paych/message2.go +++ b/chain/actors/builtin/paych/message2.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message2 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message3.go b/chain/actors/builtin/paych/message3.go index 2534daf4130..47f8e7e9e25 100644 --- a/chain/actors/builtin/paych/message3.go +++ b/chain/actors/builtin/paych/message3.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" init3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/init" paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message3 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message4.go b/chain/actors/builtin/paych/message4.go index 315dec5bb6a..65e63149d9d 100644 --- a/chain/actors/builtin/paych/message4.go +++ b/chain/actors/builtin/paych/message4.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" init4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/init" paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message4 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message5.go b/chain/actors/builtin/paych/message5.go index 98d2ceaad28..ede1c8ceb7f 100644 --- a/chain/actors/builtin/paych/message5.go +++ b/chain/actors/builtin/paych/message5.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" init5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/init" paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message5 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message6.go b/chain/actors/builtin/paych/message6.go index 5b6f6586bfe..009e8641fe7 100644 --- a/chain/actors/builtin/paych/message6.go +++ b/chain/actors/builtin/paych/message6.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" init6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/init" paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message6 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message7.go b/chain/actors/builtin/paych/message7.go index 67ac2e174c2..430fb93415b 100644 --- a/chain/actors/builtin/paych/message7.go +++ b/chain/actors/builtin/paych/message7.go @@ -4,12 +4,13 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" - "github.com/filecoin-project/lotus/chain/actors" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" init7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/init" paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" + + "github.com/filecoin-project/lotus/chain/actors" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) type message7 struct{ from address.Address } diff --git a/chain/actors/builtin/paych/message8.go b/chain/actors/builtin/paych/message8.go index 68104bb566e..a72c5fbb383 100644 --- a/chain/actors/builtin/paych/message8.go +++ b/chain/actors/builtin/paych/message8.go @@ -10,6 +10,7 @@ import ( init8 "github.com/filecoin-project/go-state-types/builtin/v8/init" paych8 "github.com/filecoin-project/go-state-types/builtin/v8/paych" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/message9.go b/chain/actors/builtin/paych/message9.go index 56ba2d0fad2..350c760eafb 100644 --- a/chain/actors/builtin/paych/message9.go +++ b/chain/actors/builtin/paych/message9.go @@ -10,6 +10,7 @@ import ( paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" init9 "github.com/filecoin-project/go-state-types/builtin/v9/init" paych9 "github.com/filecoin-project/go-state-types/builtin/v9/paych" + "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/actors/builtin/paych/mock/mock.go b/chain/actors/builtin/paych/mock/mock.go index 627be126768..3b888131966 100644 --- a/chain/actors/builtin/paych/mock/mock.go +++ b/chain/actors/builtin/paych/mock/mock.go @@ -10,6 +10,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors/builtin/paych" ) diff --git a/chain/actors/builtin/paych/paych.go b/chain/actors/builtin/paych/paych.go index 75f6a039609..fc8908cb4e1 100644 --- a/chain/actors/builtin/paych/paych.go +++ b/chain/actors/builtin/paych/paych.go @@ -15,9 +15,6 @@ import ( paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" @@ -26,6 +23,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) // Load returns an abstract copy of payment channel state, regardless of actor version diff --git a/chain/actors/builtin/paych/v0.go b/chain/actors/builtin/paych/v0.go index ae2acbb9d23..facc7f65671 100644 --- a/chain/actors/builtin/paych/v0.go +++ b/chain/actors/builtin/paych/v0.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych0 "github.com/filecoin-project/specs-actors/actors/builtin/paych" adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/paych/v10.go b/chain/actors/builtin/paych/v10.go index f74c0791b4f..edc6c96b6cc 100644 --- a/chain/actors/builtin/paych/v10.go +++ b/chain/actors/builtin/paych/v10.go @@ -12,6 +12,7 @@ import ( paych10 "github.com/filecoin-project/go-state-types/builtin/v10/paych" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v11.go b/chain/actors/builtin/paych/v11.go index 477741335be..977a013f1f8 100644 --- a/chain/actors/builtin/paych/v11.go +++ b/chain/actors/builtin/paych/v11.go @@ -12,6 +12,7 @@ import ( paych11 "github.com/filecoin-project/go-state-types/builtin/v11/paych" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v12.go b/chain/actors/builtin/paych/v12.go index bfa9476cd93..5c1330d76bc 100644 --- a/chain/actors/builtin/paych/v12.go +++ b/chain/actors/builtin/paych/v12.go @@ -12,6 +12,7 @@ import ( paych12 "github.com/filecoin-project/go-state-types/builtin/v12/paych" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v2.go b/chain/actors/builtin/paych/v2.go index b5c6646f6aa..63a3cc75eb2 100644 --- a/chain/actors/builtin/paych/v2.go +++ b/chain/actors/builtin/paych/v2.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/paych" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/paych/v3.go b/chain/actors/builtin/paych/v3.go index 65be4413e52..c672c0027e6 100644 --- a/chain/actors/builtin/paych/v3.go +++ b/chain/actors/builtin/paych/v3.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/paych" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/paych/v4.go b/chain/actors/builtin/paych/v4.go index 1a9d5a6b972..842e5209389 100644 --- a/chain/actors/builtin/paych/v4.go +++ b/chain/actors/builtin/paych/v4.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/paych" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/paych/v5.go b/chain/actors/builtin/paych/v5.go index 779bfd2b0a9..8f53fe43fab 100644 --- a/chain/actors/builtin/paych/v5.go +++ b/chain/actors/builtin/paych/v5.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/paych" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/paych/v6.go b/chain/actors/builtin/paych/v6.go index 3d7483f480b..0a8a9389600 100644 --- a/chain/actors/builtin/paych/v6.go +++ b/chain/actors/builtin/paych/v6.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/paych" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/paych/v7.go b/chain/actors/builtin/paych/v7.go index 2696934374c..ce0dcba0aaf 100644 --- a/chain/actors/builtin/paych/v7.go +++ b/chain/actors/builtin/paych/v7.go @@ -10,10 +10,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" paych7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/paych" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/paych/v8.go b/chain/actors/builtin/paych/v8.go index 34cfa8ecf7c..51067e87b00 100644 --- a/chain/actors/builtin/paych/v8.go +++ b/chain/actors/builtin/paych/v8.go @@ -12,6 +12,7 @@ import ( paych8 "github.com/filecoin-project/go-state-types/builtin/v8/paych" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/paych/v9.go b/chain/actors/builtin/paych/v9.go index f0de23fccd8..8a69e16d4f0 100644 --- a/chain/actors/builtin/paych/v9.go +++ b/chain/actors/builtin/paych/v9.go @@ -12,6 +12,7 @@ import ( paych9 "github.com/filecoin-project/go-state-types/builtin/v9/paych" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/power/diff.go b/chain/actors/builtin/power/diff.go index 7baff32186f..bb2c354ac0c 100644 --- a/chain/actors/builtin/power/diff.go +++ b/chain/actors/builtin/power/diff.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/power/power.go b/chain/actors/builtin/power/power.go index 6ad7d1fb55b..9b64ded3877 100644 --- a/chain/actors/builtin/power/power.go +++ b/chain/actors/builtin/power/power.go @@ -12,10 +12,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -23,6 +19,11 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/power/v0.go b/chain/actors/builtin/power/v0.go index a01eec679f0..4cf550616b2 100644 --- a/chain/actors/builtin/power/v0.go +++ b/chain/actors/builtin/power/v0.go @@ -11,11 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" - power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/power/v10.go b/chain/actors/builtin/power/v10.go index 7c71914807b..dd7a9decf0f 100644 --- a/chain/actors/builtin/power/v10.go +++ b/chain/actors/builtin/power/v10.go @@ -14,6 +14,7 @@ import ( power10 "github.com/filecoin-project/go-state-types/builtin/v10/power" adt10 "github.com/filecoin-project/go-state-types/builtin/v10/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v11.go b/chain/actors/builtin/power/v11.go index 5a87900832e..0ec1e2bdc5f 100644 --- a/chain/actors/builtin/power/v11.go +++ b/chain/actors/builtin/power/v11.go @@ -14,6 +14,7 @@ import ( power11 "github.com/filecoin-project/go-state-types/builtin/v11/power" adt11 "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v12.go b/chain/actors/builtin/power/v12.go index 676113113f4..2e910902269 100644 --- a/chain/actors/builtin/power/v12.go +++ b/chain/actors/builtin/power/v12.go @@ -14,6 +14,7 @@ import ( power12 "github.com/filecoin-project/go-state-types/builtin/v12/power" adt12 "github.com/filecoin-project/go-state-types/builtin/v12/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v2.go b/chain/actors/builtin/power/v2.go index f5ec1100684..bac0fa1792d 100644 --- a/chain/actors/builtin/power/v2.go +++ b/chain/actors/builtin/power/v2.go @@ -11,11 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/power/v3.go b/chain/actors/builtin/power/v3.go index 51ba563cc98..bdb66e3842a 100644 --- a/chain/actors/builtin/power/v3.go +++ b/chain/actors/builtin/power/v3.go @@ -11,12 +11,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/power/v4.go b/chain/actors/builtin/power/v4.go index 45c44e9c13a..b2dc953475f 100644 --- a/chain/actors/builtin/power/v4.go +++ b/chain/actors/builtin/power/v4.go @@ -11,12 +11,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/power/v5.go b/chain/actors/builtin/power/v5.go index 40a2281cb06..3a3adaf32da 100644 --- a/chain/actors/builtin/power/v5.go +++ b/chain/actors/builtin/power/v5.go @@ -11,12 +11,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/power/v6.go b/chain/actors/builtin/power/v6.go index 98ee3ddea85..7e8eb36546c 100644 --- a/chain/actors/builtin/power/v6.go +++ b/chain/actors/builtin/power/v6.go @@ -11,12 +11,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/power/v7.go b/chain/actors/builtin/power/v7.go index e40677ee7e1..893c58667d5 100644 --- a/chain/actors/builtin/power/v7.go +++ b/chain/actors/builtin/power/v7.go @@ -11,12 +11,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/power/v8.go b/chain/actors/builtin/power/v8.go index 7f00054004b..b15fe435590 100644 --- a/chain/actors/builtin/power/v8.go +++ b/chain/actors/builtin/power/v8.go @@ -14,6 +14,7 @@ import ( power8 "github.com/filecoin-project/go-state-types/builtin/v8/power" adt8 "github.com/filecoin-project/go-state-types/builtin/v8/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/power/v9.go b/chain/actors/builtin/power/v9.go index 856c59fb9da..126fbbfbf66 100644 --- a/chain/actors/builtin/power/v9.go +++ b/chain/actors/builtin/power/v9.go @@ -14,6 +14,7 @@ import ( power9 "github.com/filecoin-project/go-state-types/builtin/v9/power" adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/registry.go b/chain/actors/builtin/registry.go index e8a7c0bdece..6ba5fef03b2 100644 --- a/chain/actors/builtin/registry.go +++ b/chain/actors/builtin/registry.go @@ -84,6 +84,7 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" rtt "github.com/filecoin-project/go-state-types/rt" + "github.com/filecoin-project/lotus/chain/actors" ) diff --git a/chain/actors/builtin/reward/reward.go b/chain/actors/builtin/reward/reward.go index e89433500dc..3c646364585 100644 --- a/chain/actors/builtin/reward/reward.go +++ b/chain/actors/builtin/reward/reward.go @@ -9,10 +9,6 @@ import ( builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" @@ -21,6 +17,11 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/reward/v0.go b/chain/actors/builtin/reward/v0.go index 203bfea2a8e..66736f9e009 100644 --- a/chain/actors/builtin/reward/v0.go +++ b/chain/actors/builtin/reward/v0.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" smoothing0 "github.com/filecoin-project/specs-actors/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/reward/v10.go b/chain/actors/builtin/reward/v10.go index 9ddc696b050..3ffe9a26749 100644 --- a/chain/actors/builtin/reward/v10.go +++ b/chain/actors/builtin/reward/v10.go @@ -11,6 +11,7 @@ import ( reward10 "github.com/filecoin-project/go-state-types/builtin/v10/reward" smoothing10 "github.com/filecoin-project/go-state-types/builtin/v10/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v11.go b/chain/actors/builtin/reward/v11.go index 35102d8b751..1d6a5fdf479 100644 --- a/chain/actors/builtin/reward/v11.go +++ b/chain/actors/builtin/reward/v11.go @@ -11,6 +11,7 @@ import ( reward11 "github.com/filecoin-project/go-state-types/builtin/v11/reward" smoothing11 "github.com/filecoin-project/go-state-types/builtin/v11/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v12.go b/chain/actors/builtin/reward/v12.go index 9df853158c3..ecc8ff5a0c4 100644 --- a/chain/actors/builtin/reward/v12.go +++ b/chain/actors/builtin/reward/v12.go @@ -11,6 +11,7 @@ import ( reward12 "github.com/filecoin-project/go-state-types/builtin/v12/reward" smoothing12 "github.com/filecoin-project/go-state-types/builtin/v12/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v2.go b/chain/actors/builtin/reward/v2.go index b4d4d2fed9d..6d640f2e970 100644 --- a/chain/actors/builtin/reward/v2.go +++ b/chain/actors/builtin/reward/v2.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" smoothing2 "github.com/filecoin-project/specs-actors/v2/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/reward/v3.go b/chain/actors/builtin/reward/v3.go index c7201596d0a..fe8e555c914 100644 --- a/chain/actors/builtin/reward/v3.go +++ b/chain/actors/builtin/reward/v3.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" reward3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/reward" smoothing3 "github.com/filecoin-project/specs-actors/v3/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/reward/v4.go b/chain/actors/builtin/reward/v4.go index 295d0b9e568..f0decd39217 100644 --- a/chain/actors/builtin/reward/v4.go +++ b/chain/actors/builtin/reward/v4.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/miner" reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" smoothing4 "github.com/filecoin-project/specs-actors/v4/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/reward/v5.go b/chain/actors/builtin/reward/v5.go index 1af85f58f00..82be12c0aea 100644 --- a/chain/actors/builtin/reward/v5.go +++ b/chain/actors/builtin/reward/v5.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" reward5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/reward" smoothing5 "github.com/filecoin-project/specs-actors/v5/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/reward/v6.go b/chain/actors/builtin/reward/v6.go index 63723ffba55..71884dadaaf 100644 --- a/chain/actors/builtin/reward/v6.go +++ b/chain/actors/builtin/reward/v6.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" reward6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/reward" smoothing6 "github.com/filecoin-project/specs-actors/v6/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/reward/v7.go b/chain/actors/builtin/reward/v7.go index 21e938fea0c..bc39f366619 100644 --- a/chain/actors/builtin/reward/v7.go +++ b/chain/actors/builtin/reward/v7.go @@ -8,12 +8,13 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/actors/builtin" miner7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/miner" reward7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/reward" smoothing7 "github.com/filecoin-project/specs-actors/v7/actors/util/smoothing" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/reward/v8.go b/chain/actors/builtin/reward/v8.go index 5f46afbd726..5f6b96d29bf 100644 --- a/chain/actors/builtin/reward/v8.go +++ b/chain/actors/builtin/reward/v8.go @@ -11,6 +11,7 @@ import ( reward8 "github.com/filecoin-project/go-state-types/builtin/v8/reward" smoothing8 "github.com/filecoin-project/go-state-types/builtin/v8/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/reward/v9.go b/chain/actors/builtin/reward/v9.go index 1d13374b9c2..6118e2b3029 100644 --- a/chain/actors/builtin/reward/v9.go +++ b/chain/actors/builtin/reward/v9.go @@ -11,6 +11,7 @@ import ( reward9 "github.com/filecoin-project/go-state-types/builtin/v9/reward" smoothing9 "github.com/filecoin-project/go-state-types/builtin/v9/util/smoothing" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/actors/builtin/system/system.go b/chain/actors/builtin/system/system.go index b3078ee4aa9..2a2b703bb1c 100644 --- a/chain/actors/builtin/system/system.go +++ b/chain/actors/builtin/system/system.go @@ -7,9 +7,6 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" builtin12 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -17,6 +14,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/builtin/system/v0.go b/chain/actors/builtin/system/v0.go index 7a75e77856e..d5f0f079e27 100644 --- a/chain/actors/builtin/system/v0.go +++ b/chain/actors/builtin/system/v0.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system0 "github.com/filecoin-project/specs-actors/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system0 "github.com/filecoin-project/specs-actors/actors/builtin/system" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/system/v10.go b/chain/actors/builtin/system/v10.go index 8fa361811eb..2cdb3968268 100644 --- a/chain/actors/builtin/system/v10.go +++ b/chain/actors/builtin/system/v10.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system10 "github.com/filecoin-project/go-state-types/builtin/v10/system" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v11.go b/chain/actors/builtin/system/v11.go index 7f093c6bdf7..9b92afaf095 100644 --- a/chain/actors/builtin/system/v11.go +++ b/chain/actors/builtin/system/v11.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system11 "github.com/filecoin-project/go-state-types/builtin/v11/system" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v12.go b/chain/actors/builtin/system/v12.go index 8a47b8a2f78..71938e799f6 100644 --- a/chain/actors/builtin/system/v12.go +++ b/chain/actors/builtin/system/v12.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system12 "github.com/filecoin-project/go-state-types/builtin/v12/system" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v2.go b/chain/actors/builtin/system/v2.go index 3f63cdf111c..b0c64205418 100644 --- a/chain/actors/builtin/system/v2.go +++ b/chain/actors/builtin/system/v2.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/system" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/system/v3.go b/chain/actors/builtin/system/v3.go index ec2a7402923..8334f780e34 100644 --- a/chain/actors/builtin/system/v3.go +++ b/chain/actors/builtin/system/v3.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/system" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/system/v4.go b/chain/actors/builtin/system/v4.go index 78744add84d..227104f3784 100644 --- a/chain/actors/builtin/system/v4.go +++ b/chain/actors/builtin/system/v4.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/system" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/system/v5.go b/chain/actors/builtin/system/v5.go index 7ba43440f37..bbfb70b51bd 100644 --- a/chain/actors/builtin/system/v5.go +++ b/chain/actors/builtin/system/v5.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/system" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/system/v6.go b/chain/actors/builtin/system/v6.go index a2b0d80face..2319b5929a9 100644 --- a/chain/actors/builtin/system/v6.go +++ b/chain/actors/builtin/system/v6.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/system" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/system/v7.go b/chain/actors/builtin/system/v7.go index c1dc297a219..0b10129e859 100644 --- a/chain/actors/builtin/system/v7.go +++ b/chain/actors/builtin/system/v7.go @@ -8,9 +8,10 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + system7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - system7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/system" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/system/v8.go b/chain/actors/builtin/system/v8.go index 18c21746d88..eca3b0c044d 100644 --- a/chain/actors/builtin/system/v8.go +++ b/chain/actors/builtin/system/v8.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system8 "github.com/filecoin-project/go-state-types/builtin/v8/system" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/system/v9.go b/chain/actors/builtin/system/v9.go index 9eee5410862..55e073efe94 100644 --- a/chain/actors/builtin/system/v9.go +++ b/chain/actors/builtin/system/v9.go @@ -8,6 +8,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" system9 "github.com/filecoin-project/go-state-types/builtin/v9/system" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/util.go b/chain/actors/builtin/verifreg/util.go index 8e34c9f953f..09a7a132cc9 100644 --- a/chain/actors/builtin/verifreg/util.go +++ b/chain/actors/builtin/verifreg/util.go @@ -6,9 +6,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" ) // taking this as a function instead of asking the caller to call it helps reduce some of the error diff --git a/chain/actors/builtin/verifreg/v0.go b/chain/actors/builtin/verifreg/v0.go index 1da1c997db9..9913c42c0c6 100644 --- a/chain/actors/builtin/verifreg/v0.go +++ b/chain/actors/builtin/verifreg/v0.go @@ -11,10 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state0)(nil) diff --git a/chain/actors/builtin/verifreg/v10.go b/chain/actors/builtin/verifreg/v10.go index 9124b635ed5..256f4d2f888 100644 --- a/chain/actors/builtin/verifreg/v10.go +++ b/chain/actors/builtin/verifreg/v10.go @@ -15,6 +15,7 @@ import ( verifreg10 "github.com/filecoin-project/go-state-types/builtin/v10/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v11.go b/chain/actors/builtin/verifreg/v11.go index 55d3b014e8c..7b7b9e4c0a8 100644 --- a/chain/actors/builtin/verifreg/v11.go +++ b/chain/actors/builtin/verifreg/v11.go @@ -15,6 +15,7 @@ import ( verifreg11 "github.com/filecoin-project/go-state-types/builtin/v11/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v12.go b/chain/actors/builtin/verifreg/v12.go index 8811a5036ef..77a113fbe86 100644 --- a/chain/actors/builtin/verifreg/v12.go +++ b/chain/actors/builtin/verifreg/v12.go @@ -15,6 +15,7 @@ import ( verifreg12 "github.com/filecoin-project/go-state-types/builtin/v12/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v2.go b/chain/actors/builtin/verifreg/v2.go index 122da853fb5..31f7f775df2 100644 --- a/chain/actors/builtin/verifreg/v2.go +++ b/chain/actors/builtin/verifreg/v2.go @@ -11,10 +11,11 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state2)(nil) diff --git a/chain/actors/builtin/verifreg/v3.go b/chain/actors/builtin/verifreg/v3.go index 1626f5b9748..3ea016fd5c3 100644 --- a/chain/actors/builtin/verifreg/v3.go +++ b/chain/actors/builtin/verifreg/v3.go @@ -11,11 +11,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" verifreg3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/verifreg" adt3 "github.com/filecoin-project/specs-actors/v3/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state3)(nil) diff --git a/chain/actors/builtin/verifreg/v4.go b/chain/actors/builtin/verifreg/v4.go index 42a7bc78a84..464cc9fdc20 100644 --- a/chain/actors/builtin/verifreg/v4.go +++ b/chain/actors/builtin/verifreg/v4.go @@ -11,11 +11,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" verifreg4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/verifreg" adt4 "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state4)(nil) diff --git a/chain/actors/builtin/verifreg/v5.go b/chain/actors/builtin/verifreg/v5.go index 95f96168ba3..17901dd23a9 100644 --- a/chain/actors/builtin/verifreg/v5.go +++ b/chain/actors/builtin/verifreg/v5.go @@ -11,11 +11,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" verifreg5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/verifreg" adt5 "github.com/filecoin-project/specs-actors/v5/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state5)(nil) diff --git a/chain/actors/builtin/verifreg/v6.go b/chain/actors/builtin/verifreg/v6.go index 13c237b62f5..68fac64cb4d 100644 --- a/chain/actors/builtin/verifreg/v6.go +++ b/chain/actors/builtin/verifreg/v6.go @@ -11,11 +11,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" verifreg6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/verifreg" adt6 "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state6)(nil) diff --git a/chain/actors/builtin/verifreg/v7.go b/chain/actors/builtin/verifreg/v7.go index 0e4ca55de8f..e8f3ac73984 100644 --- a/chain/actors/builtin/verifreg/v7.go +++ b/chain/actors/builtin/verifreg/v7.go @@ -11,11 +11,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" verifreg7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/verifreg" adt7 "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" ) var _ State = (*state7)(nil) diff --git a/chain/actors/builtin/verifreg/v8.go b/chain/actors/builtin/verifreg/v8.go index c55a5873932..89393c4d9cf 100644 --- a/chain/actors/builtin/verifreg/v8.go +++ b/chain/actors/builtin/verifreg/v8.go @@ -14,6 +14,7 @@ import ( verifreg8 "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/v9.go b/chain/actors/builtin/verifreg/v9.go index 6c345223095..ce63c7f94b4 100644 --- a/chain/actors/builtin/verifreg/v9.go +++ b/chain/actors/builtin/verifreg/v9.go @@ -14,6 +14,7 @@ import ( adt9 "github.com/filecoin-project/go-state-types/builtin/v9/util/adt" verifreg9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/builtin/verifreg/verifreg.go b/chain/actors/builtin/verifreg/verifreg.go index 537894c24b6..de906f52127 100644 --- a/chain/actors/builtin/verifreg/verifreg.go +++ b/chain/actors/builtin/verifreg/verifreg.go @@ -11,9 +11,6 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/adt" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -21,6 +18,10 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/adt" + "github.com/filecoin-project/lotus/chain/types" ) var ( diff --git a/chain/actors/manifest.go b/chain/actors/manifest.go index f4949c4e07d..62c17193a57 100644 --- a/chain/actors/manifest.go +++ b/chain/actors/manifest.go @@ -11,6 +11,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/chain/actors/adt" ) diff --git a/chain/actors/params.go b/chain/actors/params.go index 29428b10f66..866c72b99df 100644 --- a/chain/actors/params.go +++ b/chain/actors/params.go @@ -6,6 +6,7 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/beacon/beacon.go b/chain/beacon/beacon.go index 33b18576f60..aa76bcffeb6 100644 --- a/chain/beacon/beacon.go +++ b/chain/beacon/beacon.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/beacon/drand/drand.go b/chain/beacon/drand/drand.go index 8c4506eed8c..5825fa69109 100644 --- a/chain/beacon/drand/drand.go +++ b/chain/beacon/drand/drand.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/beacon/drand/drand_test.go b/chain/beacon/drand/drand_test.go index a5efdd771f0..7434241a53a 100644 --- a/chain/beacon/drand/drand_test.go +++ b/chain/beacon/drand/drand_test.go @@ -12,6 +12,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" ) diff --git a/chain/beacon/mock.go b/chain/beacon/mock.go index b3ce766f4b5..3f26da1097d 100644 --- a/chain/beacon/mock.go +++ b/chain/beacon/mock.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/consensus/common.go b/chain/consensus/common.go index a1550d43529..a7e5c40d2b9 100644 --- a/chain/consensus/common.go +++ b/chain/consensus/common.go @@ -19,6 +19,8 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -30,7 +32,6 @@ import ( "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/lib/async" "github.com/filecoin-project/lotus/metrics" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) // Common operations shared by all consensus algorithm implementations. diff --git a/chain/consensus/compute_state.go b/chain/consensus/compute_state.go index b2f971bd351..4b993b3e72d 100644 --- a/chain/consensus/compute_state.go +++ b/chain/consensus/compute_state.go @@ -16,6 +16,15 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" + exported0 "github.com/filecoin-project/specs-actors/actors/builtin/exported" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + exported2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/exported" + exported3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/exported" + exported4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/exported" + exported5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/exported" + exported6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/exported" + exported7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/exported" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -27,14 +36,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/metrics" - exported0 "github.com/filecoin-project/specs-actors/actors/builtin/exported" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" - exported2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/exported" - exported3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/exported" - exported4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/exported" - exported5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/exported" - exported6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/exported" - exported7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/exported" ) func NewActorRegistry() *vm.ActorRegistry { diff --git a/chain/consensus/filcns/filecoin.go b/chain/consensus/filcns/filecoin.go index 8dc06b2a463..bb70d5d11c1 100644 --- a/chain/consensus/filcns/filecoin.go +++ b/chain/consensus/filcns/filecoin.go @@ -16,6 +16,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" @@ -34,7 +36,6 @@ import ( "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) var log = logging.Logger("fil-consensus") diff --git a/chain/consensus/filcns/upgrades.go b/chain/consensus/filcns/upgrades.go index 994296ec0f2..91b4bc98ba5 100644 --- a/chain/consensus/filcns/upgrades.go +++ b/chain/consensus/filcns/upgrades.go @@ -30,18 +30,6 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/rt" gstStore "github.com/filecoin-project/go-state-types/store" - "github.com/filecoin-project/lotus/blockstore" - "github.com/filecoin-project/lotus/build" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/actors/builtin/system" - "github.com/filecoin-project/lotus/chain/state" - "github.com/filecoin-project/lotus/chain/stmgr" - "github.com/filecoin-project/lotus/chain/store" - "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/lotus/chain/vm" - "github.com/filecoin-project/lotus/node/bundle" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" multisig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" @@ -56,6 +44,19 @@ import ( "github.com/filecoin-project/specs-actors/v6/actors/migration/nv14" "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" + + "github.com/filecoin-project/lotus/blockstore" + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" + "github.com/filecoin-project/lotus/chain/actors/builtin/system" + "github.com/filecoin-project/lotus/chain/state" + "github.com/filecoin-project/lotus/chain/stmgr" + "github.com/filecoin-project/lotus/chain/store" + "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/chain/vm" + "github.com/filecoin-project/lotus/node/bundle" ) //go:embed FVMLiftoff.txt diff --git a/chain/consensus/filcns/weight.go b/chain/consensus/filcns/weight.go index 734ed098127..ab90840c5f8 100644 --- a/chain/consensus/filcns/weight.go +++ b/chain/consensus/filcns/weight.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" big2 "github.com/filecoin-project/go-state-types/big" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/power" diff --git a/chain/consensus/iface.go b/chain/consensus/iface.go index 7db50539f95..9449cb5a47a 100644 --- a/chain/consensus/iface.go +++ b/chain/consensus/iface.go @@ -7,6 +7,7 @@ import ( "go.opencensus.io/stats" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/reward" diff --git a/chain/consensus/signatures.go b/chain/consensus/signatures.go index bb2f518aca7..cb0e229a85b 100644 --- a/chain/consensus/signatures.go +++ b/chain/consensus/signatures.go @@ -6,6 +6,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/chain/events/cache.go b/chain/events/cache.go index 8fd868bd15d..2eba1f085b7 100644 --- a/chain/events/cache.go +++ b/chain/events/cache.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events.go b/chain/events/events.go index 103e5749787..c68b62a64e0 100644 --- a/chain/events/events.go +++ b/chain/events/events.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/events/events_called.go b/chain/events/events_called.go index 58199b9b032..3ac02b2f7fd 100644 --- a/chain/events/events_called.go +++ b/chain/events/events_called.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events_height.go b/chain/events/events_height.go index 84bc6e3499e..5789be753a9 100644 --- a/chain/events/events_height.go +++ b/chain/events/events_height.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/events_test.go b/chain/events/events_test.go index b8c57a60c1a..f16434355ee 100644 --- a/chain/events/events_test.go +++ b/chain/events/events_test.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/chain/events/filter/event.go b/chain/events/filter/event.go index a5657eb402c..24192a53e72 100644 --- a/chain/events/filter/event.go +++ b/chain/events/filter/event.go @@ -14,9 +14,10 @@ import ( "github.com/filecoin-project/go-address" amt4 "github.com/filecoin-project/go-amt-ipld/v4" "github.com/filecoin-project/go-state-types/abi" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + cstore "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) func isIndexedValue(b uint8) bool { diff --git a/chain/events/filter/event_test.go b/chain/events/filter/event_test.go index 833d1b1a951..329573bc13d 100644 --- a/chain/events/filter/event_test.go +++ b/chain/events/filter/event_test.go @@ -15,10 +15,11 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/types" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) func TestEventFilterCollectEvents(t *testing.T) { diff --git a/chain/events/filter/index.go b/chain/events/filter/index.go index 81c629303a4..2b1890c739e 100644 --- a/chain/events/filter/index.go +++ b/chain/events/filter/index.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/filter/index_test.go b/chain/events/filter/index_test.go index 251d1d9428b..f9b1b14adae 100644 --- a/chain/events/filter/index_test.go +++ b/chain/events/filter/index_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/observer.go b/chain/events/observer.go index 66654571375..4462185858f 100644 --- a/chain/events/observer.go +++ b/chain/events/observer.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/chain/events/state/fastapi.go b/chain/events/state/fastapi.go index df2b7c9daab..9375d9d7846 100644 --- a/chain/events/state/fastapi.go +++ b/chain/events/state/fastapi.go @@ -4,6 +4,7 @@ import ( "context" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/mock/api.go b/chain/events/state/mock/api.go index 1cbb800c8a7..cdec4265922 100644 --- a/chain/events/state/mock/api.go +++ b/chain/events/state/mock/api.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/mock/tipset.go b/chain/events/state/mock/tipset.go index a57480a5199..0d25b8790eb 100644 --- a/chain/events/state/mock/tipset.go +++ b/chain/events/state/mock/tipset.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/state/predicates.go b/chain/events/state/predicates.go index c186a746474..ff05156a6f9 100644 --- a/chain/events/state/predicates.go +++ b/chain/events/state/predicates.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/events/state/predicates_test.go b/chain/events/state/predicates_test.go index d123a9a743b..52fc2668a2b 100644 --- a/chain/events/state/predicates_test.go +++ b/chain/events/state/predicates_test.go @@ -13,16 +13,17 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" - bstore "github.com/filecoin-project/lotus/blockstore" - "github.com/filecoin-project/lotus/chain/actors/builtin/market" - "github.com/filecoin-project/lotus/chain/actors/builtin/miner" - test "github.com/filecoin-project/lotus/chain/events/state/mock" - "github.com/filecoin-project/lotus/chain/types" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" tutils "github.com/filecoin-project/specs-actors/v5/support/testing" + + bstore "github.com/filecoin-project/lotus/blockstore" + "github.com/filecoin-project/lotus/chain/actors/builtin/market" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" + test "github.com/filecoin-project/lotus/chain/events/state/mock" + "github.com/filecoin-project/lotus/chain/types" ) var dummyCid cid.Cid diff --git a/chain/events/tscache.go b/chain/events/tscache.go index c5558426bf4..ed19a5f4175 100644 --- a/chain/events/tscache.go +++ b/chain/events/tscache.go @@ -7,6 +7,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/events/tscache_test.go b/chain/events/tscache_test.go index 65b58794260..ec312740b7e 100644 --- a/chain/events/tscache_test.go +++ b/chain/events/tscache_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/exchange/client.go b/chain/exchange/client.go index 581861e9dc3..120b554a11f 100644 --- a/chain/exchange/client.go +++ b/chain/exchange/client.go @@ -15,6 +15,7 @@ import ( "golang.org/x/xerrors" cborutil "github.com/filecoin-project/go-cbor-util" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/exchange/server.go b/chain/exchange/server.go index d9c4d40d8dd..03dcf0ed79f 100644 --- a/chain/exchange/server.go +++ b/chain/exchange/server.go @@ -12,6 +12,7 @@ import ( "golang.org/x/xerrors" cborutil "github.com/filecoin-project/go-cbor-util" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/gen/gen.go b/chain/gen/gen.go index 52ebc90f121..9f8d0834d6e 100644 --- a/chain/gen/gen.go +++ b/chain/gen/gen.go @@ -24,6 +24,8 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -45,7 +47,6 @@ import ( "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) const msgsPerBlock = 20 diff --git a/chain/gen/gen_test.go b/chain/gen/gen_test.go index f2f1fec0cf5..d04e55a265a 100644 --- a/chain/gen/gen_test.go +++ b/chain/gen/gen_test.go @@ -5,6 +5,7 @@ import ( "testing" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/policy" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" diff --git a/chain/gen/genesis/f00_system.go b/chain/gen/genesis/f00_system.go index 1bedb662784..5c6ecacbf92 100644 --- a/chain/gen/genesis/f00_system.go +++ b/chain/gen/genesis/f00_system.go @@ -10,6 +10,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f01_init.go b/chain/gen/genesis/f01_init.go index bab6dc75e67..706328d2194 100644 --- a/chain/gen/genesis/f01_init.go +++ b/chain/gen/genesis/f01_init.go @@ -14,12 +14,13 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/genesis" - "github.com/filecoin-project/specs-actors/actors/util/adt" ) func SetupInitActor(ctx context.Context, bs bstore.Blockstore, netname string, initialActors []genesis.Actor, rootVerifier genesis.Actor, remainder genesis.Actor, av actorstypes.Version) (int64, *types.Actor, map[address.Address]address.Address, error) { diff --git a/chain/gen/genesis/f02_reward.go b/chain/gen/genesis/f02_reward.go index 6c9f70f8f29..db32517f9cf 100644 --- a/chain/gen/genesis/f02_reward.go +++ b/chain/gen/genesis/f02_reward.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/gen/genesis/f03_cron.go b/chain/gen/genesis/f03_cron.go index a4c4f5fb1b4..4c377b19156 100644 --- a/chain/gen/genesis/f03_cron.go +++ b/chain/gen/genesis/f03_cron.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f04_power.go b/chain/gen/genesis/f04_power.go index 791743d677f..385cc97d25e 100644 --- a/chain/gen/genesis/f04_power.go +++ b/chain/gen/genesis/f04_power.go @@ -9,11 +9,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/actors/util/adt" ) func SetupStoragePowerActor(ctx context.Context, bs bstore.Blockstore, av actorstypes.Version) (*types.Actor, error) { diff --git a/chain/gen/genesis/f05_market.go b/chain/gen/genesis/f05_market.go index 3f623082949..59c61a3ae1d 100644 --- a/chain/gen/genesis/f05_market.go +++ b/chain/gen/genesis/f05_market.go @@ -9,6 +9,7 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/gen/genesis/f06_vreg.go b/chain/gen/genesis/f06_vreg.go index 5b456674fea..ffddc814f1c 100644 --- a/chain/gen/genesis/f06_vreg.go +++ b/chain/gen/genesis/f06_vreg.go @@ -10,11 +10,12 @@ import ( actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/actors/util/adt" ) var RootVerifierID address.Address diff --git a/chain/gen/genesis/f07_dcap.go b/chain/gen/genesis/f07_dcap.go index dcfb76c8552..6d8e3258ebd 100644 --- a/chain/gen/genesis/f07_dcap.go +++ b/chain/gen/genesis/f07_dcap.go @@ -11,11 +11,12 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/datacap" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/actors/util/adt" ) var GovernorId address.Address diff --git a/chain/gen/genesis/genesis.go b/chain/gen/genesis/genesis.go index d4f464221d9..8ec657479f7 100644 --- a/chain/gen/genesis/genesis.go +++ b/chain/gen/genesis/genesis.go @@ -19,6 +19,10 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" + adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -43,9 +47,6 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/node/bundle" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - verifreg0 "github.com/filecoin-project/specs-actors/actors/builtin/verifreg" - adt0 "github.com/filecoin-project/specs-actors/actors/util/adt" ) const AccountStart = 100 diff --git a/chain/gen/genesis/genesis_eth.go b/chain/gen/genesis/genesis_eth.go index 347caa4fffc..d5aa2f0b51b 100644 --- a/chain/gen/genesis/genesis_eth.go +++ b/chain/gen/genesis/genesis_eth.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" diff --git a/chain/gen/genesis/miners.go b/chain/gen/genesis/miners.go index bad96f3bc77..df8900cab8c 100644 --- a/chain/gen/genesis/miners.go +++ b/chain/gen/genesis/miners.go @@ -24,6 +24,17 @@ import ( smoothing9 "github.com/filecoin-project/go-state-types/builtin/v9/util/smoothing" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" + power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" + reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" + power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" + reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" + "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -40,16 +51,6 @@ import ( "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" "github.com/filecoin-project/lotus/lib/sigs" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - miner0 "github.com/filecoin-project/specs-actors/actors/builtin/miner" - power0 "github.com/filecoin-project/specs-actors/actors/builtin/power" - reward0 "github.com/filecoin-project/specs-actors/actors/builtin/reward" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - reward2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/reward" - power4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/power" - reward4 "github.com/filecoin-project/specs-actors/v4/actors/builtin/reward" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" - runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" ) func MinerAddress(genesisIndex uint64) address.Address { diff --git a/chain/gen/genesis/util.go b/chain/gen/genesis/util.go index a34a43d9aa7..beca1183c02 100644 --- a/chain/gen/genesis/util.go +++ b/chain/gen/genesis/util.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" diff --git a/chain/gen/slashfilter/slashfilter.go b/chain/gen/slashfilter/slashfilter.go index ade6442b9ae..71b5dad9ad9 100644 --- a/chain/gen/slashfilter/slashfilter.go +++ b/chain/gen/slashfilter/slashfilter.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/gen/slashfilter/slashsvc/slashservice.go b/chain/gen/slashfilter/slashsvc/slashservice.go index 02567de9a1a..7a662288098 100644 --- a/chain/gen/slashfilter/slashsvc/slashservice.go +++ b/chain/gen/slashfilter/slashsvc/slashservice.go @@ -13,11 +13,12 @@ import ( "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/specs-actors/actors/builtin/miner" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/gen/slashfilter" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/actors/builtin/miner" ) var log = logging.Logger("slashsvc") diff --git a/chain/index/msgindex.go b/chain/index/msgindex.go index 9dd6735a89d..e9e81ae2cf5 100644 --- a/chain/index/msgindex.go +++ b/chain/index/msgindex.go @@ -16,6 +16,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/index/msgindex_test.go b/chain/index/msgindex_test.go index fa8844e4a90..bf4bc6190e8 100644 --- a/chain/index/msgindex_test.go +++ b/chain/index/msgindex_test.go @@ -13,6 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" diff --git a/chain/market/fundmanager.go b/chain/market/fundmanager.go index 47e8f0bb64e..fab71dfefe1 100644 --- a/chain/market/fundmanager.go +++ b/chain/market/fundmanager.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/market" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/market/fundmanager_test.go b/chain/market/fundmanager_test.go index 6511574ad03..d79afbc513b 100644 --- a/chain/market/fundmanager_test.go +++ b/chain/market/fundmanager_test.go @@ -16,11 +16,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) // TestFundManagerBasic verifies that the basic fund manager operations work diff --git a/chain/market/store.go b/chain/market/store.go index 1950b0239a0..10ab2abe1a3 100644 --- a/chain/market/store.go +++ b/chain/market/store.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/chain/messagepool/check.go b/chain/messagepool/check.go index c5208bc5755..fdec910c4ea 100644 --- a/chain/messagepool/check.go +++ b/chain/messagepool/check.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" diff --git a/chain/messagepool/gasguess/guessgas.go b/chain/messagepool/gasguess/guessgas.go index af0a352364c..f502e84a6e4 100644 --- a/chain/messagepool/gasguess/guessgas.go +++ b/chain/messagepool/gasguess/guessgas.go @@ -8,10 +8,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/types" ) type ActorLookup func(context.Context, address.Address, types.TipSetKey) (*types.Actor, error) diff --git a/chain/messagepool/messagepool.go b/chain/messagepool/messagepool.go index 36f949220a3..7d55b0b16f5 100644 --- a/chain/messagepool/messagepool.go +++ b/chain/messagepool/messagepool.go @@ -30,6 +30,8 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + lps "github.com/filecoin-project/pubsub" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus" @@ -40,7 +42,6 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/modules/dtypes" - lps "github.com/filecoin-project/pubsub" ) var log = logging.Logger("messagepool") diff --git a/chain/messagepool/messagepool_test.go b/chain/messagepool/messagepool_test.go index 35798563a9b..3c6800d7b2b 100644 --- a/chain/messagepool/messagepool_test.go +++ b/chain/messagepool/messagepool_test.go @@ -18,6 +18,8 @@ import ( big2 "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -27,7 +29,6 @@ import ( "github.com/filecoin-project/lotus/chain/wallet" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func init() { diff --git a/chain/messagepool/provider.go b/chain/messagepool/provider.go index b7dd845a690..764e6c13a92 100644 --- a/chain/messagepool/provider.go +++ b/chain/messagepool/provider.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/pruning.go b/chain/messagepool/pruning.go index 67629f4c3a6..24d7fee566d 100644 --- a/chain/messagepool/pruning.go +++ b/chain/messagepool/pruning.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/messagepool/repub.go b/chain/messagepool/repub.go index 8d30d3a22db..a87d5e08a84 100644 --- a/chain/messagepool/repub.go +++ b/chain/messagepool/repub.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/repub_test.go b/chain/messagepool/repub_test.go index d2d34da5a14..9cdabc02fac 100644 --- a/chain/messagepool/repub_test.go +++ b/chain/messagepool/repub_test.go @@ -8,11 +8,12 @@ import ( "github.com/ipfs/go-datastore" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func TestRepubMessages(t *testing.T) { diff --git a/chain/messagepool/selection.go b/chain/messagepool/selection.go index 9e9e58df1fd..163bd76f985 100644 --- a/chain/messagepool/selection.go +++ b/chain/messagepool/selection.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" tbig "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool/gasguess" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagepool/selection_test.go b/chain/messagepool/selection_test.go index 836e23ef2b5..17e0f34f4e0 100644 --- a/chain/messagepool/selection_test.go +++ b/chain/messagepool/selection_test.go @@ -22,6 +22,8 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -32,7 +34,6 @@ import ( _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/delegated" _ "github.com/filecoin-project/lotus/lib/sigs/secp" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func init() { diff --git a/chain/messagesigner/messagesigner.go b/chain/messagesigner/messagesigner.go index 860aeb6e35c..cd31a3b739e 100644 --- a/chain/messagesigner/messagesigner.go +++ b/chain/messagesigner/messagesigner.go @@ -13,6 +13,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/messagesigner/messagesigner_test.go b/chain/messagesigner/messagesigner_test.go index 4979b327f19..637f17b46cd 100644 --- a/chain/messagesigner/messagesigner_test.go +++ b/chain/messagesigner/messagesigner_test.go @@ -12,6 +12,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet" diff --git a/chain/rand/rand.go b/chain/rand/rand.go index 3a4b454c45b..40f9f593a03 100644 --- a/chain/rand/rand.go +++ b/chain/rand/rand.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/rand/rand_test.go b/chain/rand/rand_test.go index e96269632a4..e2e7221658b 100644 --- a/chain/rand/rand_test.go +++ b/chain/rand/rand_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" diff --git a/chain/state/statetree.go b/chain/state/statetree.go index 1fa573abb30..61d7d500a87 100644 --- a/chain/state/statetree.go +++ b/chain/state/statetree.go @@ -16,14 +16,15 @@ import ( "github.com/filecoin-project/go-state-types/abi" builtin_types "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/chain/actors/adt" - init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" - "github.com/filecoin-project/lotus/chain/types" states0 "github.com/filecoin-project/specs-actors/actors/states" states2 "github.com/filecoin-project/specs-actors/v2/actors/states" states3 "github.com/filecoin-project/specs-actors/v3/actors/states" states4 "github.com/filecoin-project/specs-actors/v4/actors/states" states5 "github.com/filecoin-project/specs-actors/v5/actors/states" + + "github.com/filecoin-project/lotus/chain/actors/adt" + init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" + "github.com/filecoin-project/lotus/chain/types" ) var log = logging.Logger("statetree") diff --git a/chain/state/statetree_test.go b/chain/state/statetree_test.go index 06a47d8ea67..9a221751a75 100644 --- a/chain/state/statetree_test.go +++ b/chain/state/statetree_test.go @@ -11,9 +11,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/network" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) func BenchmarkStateTreeSet(b *testing.B) { diff --git a/chain/stmgr/actors.go b/chain/stmgr/actors.go index 53e2eba5a5f..56744fa7489 100644 --- a/chain/stmgr/actors.go +++ b/chain/stmgr/actors.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/chain/stmgr/call.go b/chain/stmgr/call.go index 4c4a6e6bcc3..61056528f11 100644 --- a/chain/stmgr/call.go +++ b/chain/stmgr/call.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/chain/stmgr/forks.go b/chain/stmgr/forks.go index b3a6bd718f0..9dd66ee8bcc 100644 --- a/chain/stmgr/forks.go +++ b/chain/stmgr/forks.go @@ -19,6 +19,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -26,7 +28,6 @@ import ( "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" - "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" ) // EnvDisablePreMigrations when set to '1' stops pre-migrations from running diff --git a/chain/stmgr/forks_test.go b/chain/stmgr/forks_test.go index e55497f902d..bf8793488b6 100644 --- a/chain/stmgr/forks_test.go +++ b/chain/stmgr/forks_test.go @@ -23,6 +23,10 @@ import ( "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" + builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" @@ -39,9 +43,6 @@ import ( "github.com/filecoin-project/lotus/chain/vm" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" - builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" ) func init() { diff --git a/chain/stmgr/read.go b/chain/stmgr/read.go index 1a66129da9d..4543f63b34b 100644 --- a/chain/stmgr/read.go +++ b/chain/stmgr/read.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/stmgr/rpc/rpcstatemanager.go b/chain/stmgr/rpc/rpcstatemanager.go index be17cfd9e0b..9186501eab9 100644 --- a/chain/stmgr/rpc/rpcstatemanager.go +++ b/chain/stmgr/rpc/rpcstatemanager.go @@ -7,6 +7,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/stmgr/searchwait.go b/chain/stmgr/searchwait.go index 03e767167a8..356ace23c45 100644 --- a/chain/stmgr/searchwait.go +++ b/chain/stmgr/searchwait.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/stmgr/searchwait_test.go b/chain/stmgr/searchwait_test.go index 24876a76290..b23b22376c6 100644 --- a/chain/stmgr/searchwait_test.go +++ b/chain/stmgr/searchwait_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/gen" _ "github.com/filecoin-project/lotus/lib/sigs/bls" _ "github.com/filecoin-project/lotus/lib/sigs/secp" diff --git a/chain/stmgr/stmgr.go b/chain/stmgr/stmgr.go index c3129bba94f..49913e442d1 100644 --- a/chain/stmgr/stmgr.go +++ b/chain/stmgr/stmgr.go @@ -19,6 +19,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" @@ -32,7 +34,6 @@ import ( "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" - "github.com/filecoin-project/specs-actors/v8/actors/migration/nv16" // Used for genesis. msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" diff --git a/chain/stmgr/supply.go b/chain/stmgr/supply.go index 2ce5d4e37a0..9486cb93622 100644 --- a/chain/stmgr/supply.go +++ b/chain/stmgr/supply.go @@ -10,6 +10,8 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" @@ -23,7 +25,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" - msig0 "github.com/filecoin-project/specs-actors/actors/builtin/multisig" ) // sets up information about the vesting schedule diff --git a/chain/stmgr/utils.go b/chain/stmgr/utils.go index db84d0825c7..5e3bbd2788b 100644 --- a/chain/stmgr/utils.go +++ b/chain/stmgr/utils.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" gstStore "github.com/filecoin-project/go-state-types/store" + "github.com/filecoin-project/lotus/api" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/actors/builtin/system" diff --git a/chain/store/basefee.go b/chain/store/basefee.go index cccfc04fcd5..3b6af5c0716 100644 --- a/chain/store/basefee.go +++ b/chain/store/basefee.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/store/index.go b/chain/store/index.go index 072e74ef8ed..8361f4db9eb 100644 --- a/chain/store/index.go +++ b/chain/store/index.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/shardedmutex" ) diff --git a/chain/store/index_test.go b/chain/store/index_test.go index 78822334a3c..a3a4ad6ce7e 100644 --- a/chain/store/index_test.go +++ b/chain/store/index_test.go @@ -11,6 +11,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" diff --git a/chain/store/messages.go b/chain/store/messages.go index 1074448cebb..c23f900d7cb 100644 --- a/chain/store/messages.go +++ b/chain/store/messages.go @@ -11,11 +11,12 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) type storable interface { diff --git a/chain/store/snapshot.go b/chain/store/snapshot.go index d79fd0d197b..301a5f87bfd 100644 --- a/chain/store/snapshot.go +++ b/chain/store/snapshot.go @@ -22,6 +22,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/chain/store/store.go b/chain/store/store.go index f5bea1549f9..f2826fc2ff0 100644 --- a/chain/store/store.go +++ b/chain/store/store.go @@ -26,6 +26,8 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/pubsub" + "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -33,7 +35,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/metrics" - "github.com/filecoin-project/pubsub" ) var log = logging.Logger("chainstore") diff --git a/chain/store/store_test.go b/chain/store/store_test.go index a98c97afb3d..9c717fdbef4 100644 --- a/chain/store/store_test.go +++ b/chain/store/store_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/chain/sub/incoming.go b/chain/sub/incoming.go index 43bed7f00e0..b50ddc46779 100644 --- a/chain/sub/incoming.go +++ b/chain/sub/incoming.go @@ -21,6 +21,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/chain/sub/incoming_test.go b/chain/sub/incoming_test.go index 26a424a48ab..d8ee99b7f12 100644 --- a/chain/sub/incoming_test.go +++ b/chain/sub/incoming_test.go @@ -16,6 +16,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/sync.go b/chain/sync.go index 04fb8605c04..1b9a302f75d 100644 --- a/chain/sync.go +++ b/chain/sync.go @@ -24,12 +24,15 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/pubsub" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/node/modules/dtypes" - "github.com/filecoin-project/pubsub" // named msgarray here to make it clear that these are the types used by // messages, regardless of specs-actors version. + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -40,7 +43,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/metrics" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) var ( diff --git a/chain/sync_manager.go b/chain/sync_manager.go index 621c8c2025a..3369c3b5a0d 100644 --- a/chain/sync_manager.go +++ b/chain/sync_manager.go @@ -12,6 +12,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/sync_test.go b/chain/sync_test.go index e8abb413e5c..be775960319 100644 --- a/chain/sync_test.go +++ b/chain/sync_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/chain/syncstate.go b/chain/syncstate.go index a18ef4ca128..527d6be4832 100644 --- a/chain/syncstate.go +++ b/chain/syncstate.go @@ -5,6 +5,7 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/types/bigint.go b/chain/types/bigint.go index e7850fa9262..72ef5212862 100644 --- a/chain/types/bigint.go +++ b/chain/types/bigint.go @@ -5,6 +5,7 @@ import ( "math/big" big2 "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" ) diff --git a/chain/types/ethtypes/eth_transactions.go b/chain/types/ethtypes/eth_transactions.go index 24c8397708d..6c13c5bf6a2 100644 --- a/chain/types/ethtypes/eth_transactions.go +++ b/chain/types/ethtypes/eth_transactions.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" typescrypto "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/types/ethtypes/eth_transactions_test.go b/chain/types/ethtypes/eth_transactions_test.go index 7b4be0b28c0..68abc55dd49 100644 --- a/chain/types/ethtypes/eth_transactions_test.go +++ b/chain/types/ethtypes/eth_transactions_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/evm" init10 "github.com/filecoin-project/go-state-types/builtin/v10/init" crypto1 "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/lib/sigs" _ "github.com/filecoin-project/lotus/lib/sigs/delegated" diff --git a/chain/types/ethtypes/eth_types.go b/chain/types/ethtypes/eth_types.go index 96271435b56..b796e6f56f2 100644 --- a/chain/types/ethtypes/eth_types.go +++ b/chain/types/ethtypes/eth_types.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/lib/must" ) diff --git a/chain/types/message.go b/chain/types/message.go index 8a00ecf318a..473289ead45 100644 --- a/chain/types/message.go +++ b/chain/types/message.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" ) diff --git a/chain/types/mock/chain.go b/chain/types/mock/chain.go index e6d14d9d958..dcbcd85362c 100644 --- a/chain/types/mock/chain.go +++ b/chain/types/mock/chain.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/types/vmcontext.go b/chain/types/vmcontext.go index 4b4e8892429..bab9c213fa6 100644 --- a/chain/types/vmcontext.go +++ b/chain/types/vmcontext.go @@ -5,6 +5,7 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/types_test.go b/chain/types_test.go index e7264f18f7d..0fb3992146e 100644 --- a/chain/types_test.go +++ b/chain/types_test.go @@ -7,6 +7,7 @@ import ( "testing" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/vectors/gen/main.go b/chain/vectors/gen/main.go index 308bed39c1e..f4b7c82da02 100644 --- a/chain/vectors/gen/main.go +++ b/chain/vectors/gen/main.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/types" diff --git a/chain/vectors/vector_types.go b/chain/vectors/vector_types.go index 7e014fb77cc..3251fde38e8 100644 --- a/chain/vectors/vector_types.go +++ b/chain/vectors/vector_types.go @@ -2,6 +2,7 @@ package vectors import ( "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/vm/fvm.go b/chain/vm/fvm.go index c864d00536f..c8c02dddd40 100644 --- a/chain/vm/fvm.go +++ b/chain/vm/fvm.go @@ -25,6 +25,7 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/chain/vm/gas.go b/chain/vm/gas.go index d6d7df45c67..cb0c5def94d 100644 --- a/chain/vm/gas.go +++ b/chain/vm/gas.go @@ -9,9 +9,10 @@ import ( addr "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/build" vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + + "github.com/filecoin-project/lotus/build" ) type GasCharge struct { diff --git a/chain/vm/gas_v0.go b/chain/vm/gas_v0.go index 96dd8ee4478..7a144fc261e 100644 --- a/chain/vm/gas_v0.go +++ b/chain/vm/gas_v0.go @@ -6,8 +6,9 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/chain/actors/builtin" proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + + "github.com/filecoin-project/lotus/chain/actors/builtin" ) type scalingCost struct { diff --git a/chain/vm/invoker.go b/chain/vm/invoker.go index 2baaab4c6fd..cea17f61dba 100644 --- a/chain/vm/invoker.go +++ b/chain/vm/invoker.go @@ -15,11 +15,12 @@ import ( builtinst "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" - vmr "github.com/filecoin-project/specs-actors/v7/actors/runtime" ) type MethodMeta struct { diff --git a/chain/vm/invoker_test.go b/chain/vm/invoker_test.go index 2ee7b384b67..d3e6dcd7fd4 100644 --- a/chain/vm/invoker_test.go +++ b/chain/vm/invoker_test.go @@ -18,10 +18,11 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/rt" + runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" - runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" ) type basicContract struct{} diff --git a/chain/vm/mkactor.go b/chain/vm/mkactor.go index e91d91ca011..b33085c0594 100644 --- a/chain/vm/mkactor.go +++ b/chain/vm/mkactor.go @@ -11,12 +11,6 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" - "github.com/filecoin-project/lotus/build" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/aerrors" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/types" builtin0 "github.com/filecoin-project/specs-actors/actors/builtin" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" @@ -24,6 +18,13 @@ import ( builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" builtin7 "github.com/filecoin-project/specs-actors/v7/actors/builtin" + + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/aerrors" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin/account" + "github.com/filecoin-project/lotus/chain/types" ) func init() { diff --git a/chain/vm/runtime.go b/chain/vm/runtime.go index c18944473a7..355fcea2b09 100644 --- a/chain/vm/runtime.go +++ b/chain/vm/runtime.go @@ -21,13 +21,6 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" - "github.com/filecoin-project/lotus/build" - "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/actors/aerrors" - "github.com/filecoin-project/lotus/chain/actors/builtin" - "github.com/filecoin-project/lotus/chain/rand" - "github.com/filecoin-project/lotus/chain/state" - "github.com/filecoin-project/lotus/chain/types" rt0 "github.com/filecoin-project/specs-actors/actors/runtime" rt2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" rt3 "github.com/filecoin-project/specs-actors/v3/actors/runtime" @@ -35,6 +28,14 @@ import ( rt5 "github.com/filecoin-project/specs-actors/v5/actors/runtime" rt6 "github.com/filecoin-project/specs-actors/v6/actors/runtime" rt7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" + + "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/actors/aerrors" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/rand" + "github.com/filecoin-project/lotus/chain/state" + "github.com/filecoin-project/lotus/chain/types" ) type Message struct { diff --git a/chain/vm/runtime_test.go b/chain/vm/runtime_test.go index 563cedc26c6..88b7366dea0 100644 --- a/chain/vm/runtime_test.go +++ b/chain/vm/runtime_test.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/actors/aerrors" ) diff --git a/chain/vm/syscalls.go b/chain/vm/syscalls.go index 7a819ad1806..83a07ca2dae 100644 --- a/chain/vm/syscalls.go +++ b/chain/vm/syscalls.go @@ -17,6 +17,9 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -26,8 +29,6 @@ import ( "github.com/filecoin-project/lotus/lib/sigs" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" - runtime7 "github.com/filecoin-project/specs-actors/v7/actors/runtime" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func init() { diff --git a/chain/vm/vm.go b/chain/vm/vm.go index 9e1799a7fc5..ba404ab1fa3 100644 --- a/chain/vm/vm.go +++ b/chain/vm/vm.go @@ -23,6 +23,7 @@ import ( builtin_types "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/chain/vm/vmi.go b/chain/vm/vmi.go index e604a0895dd..042621ca2d4 100644 --- a/chain/vm/vmi.go +++ b/chain/vm/vmi.go @@ -8,6 +8,7 @@ import ( cid "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/chain/wallet/key/key.go b/chain/wallet/key/key.go index 19c8c2027bc..4220666108e 100644 --- a/chain/wallet/key/key.go +++ b/chain/wallet/key/key.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/chain/wallet/ledger/ledger.go b/chain/wallet/ledger/ledger.go index 31de0f882fb..5279389de81 100644 --- a/chain/wallet/ledger/ledger.go +++ b/chain/wallet/ledger/ledger.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/chain/wallet/multi.go b/chain/wallet/multi.go index cb6fd730e2e..91d2714772f 100644 --- a/chain/wallet/multi.go +++ b/chain/wallet/multi.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ledgerwallet "github.com/filecoin-project/lotus/chain/wallet/ledger" diff --git a/chain/wallet/wallet.go b/chain/wallet/wallet.go index ea20e860329..76af663c780 100644 --- a/chain/wallet/wallet.go +++ b/chain/wallet/wallet.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/cli/auth.go b/cli/auth.go index bac1e0d1efd..caea4cb424b 100644 --- a/cli/auth.go +++ b/cli/auth.go @@ -7,6 +7,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cli/backup.go b/cli/backup.go index 6ddb03c2434..d2d8f25ff89 100644 --- a/cli/backup.go +++ b/cli/backup.go @@ -11,6 +11,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cli/chain.go b/cli/chain.go index 0653516c5ad..c0d54fd6382 100644 --- a/cli/chain.go +++ b/cli/chain.go @@ -26,6 +26,13 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/actors/builtin" + "github.com/filecoin-project/specs-actors/actors/builtin/account" + "github.com/filecoin-project/specs-actors/actors/builtin/market" + "github.com/filecoin-project/specs-actors/actors/builtin/miner" + "github.com/filecoin-project/specs-actors/actors/builtin/power" + "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" @@ -33,12 +40,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/actors/builtin" - "github.com/filecoin-project/specs-actors/actors/builtin/account" - "github.com/filecoin-project/specs-actors/actors/builtin/market" - "github.com/filecoin-project/specs-actors/actors/builtin/miner" - "github.com/filecoin-project/specs-actors/actors/builtin/power" - "github.com/filecoin-project/specs-actors/actors/util/adt" ) var ChainCmd = &cli.Command{ diff --git a/cli/chain_test.go b/cli/chain_test.go index 9c9a65f9957..9fd46724e6b 100644 --- a/cli/chain_test.go +++ b/cli/chain_test.go @@ -16,10 +16,11 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/v7/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" - "github.com/filecoin-project/specs-actors/v7/actors/builtin" ) func TestChainHead(t *testing.T) { diff --git a/cli/client.go b/cli/client.go index 6a7dc90745e..88f7ed2087b 100644 --- a/cli/client.go +++ b/cli/client.go @@ -36,6 +36,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cli/client_retr.go b/cli/client_retr.go index c713a121c9d..fa8164ab5ef 100644 --- a/cli/client_retr.go +++ b/cli/client_retr.go @@ -30,6 +30,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/big" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" cliutil "github.com/filecoin-project/lotus/cli/util" diff --git a/cli/disputer.go b/cli/disputer.go index 0668f634efd..de3f5032468 100644 --- a/cli/disputer.go +++ b/cli/disputer.go @@ -13,13 +13,14 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" + miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" - builtin3 "github.com/filecoin-project/specs-actors/v3/actors/builtin" - miner3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/miner" ) var disputeLog = logging.Logger("disputer") diff --git a/cli/evm.go b/cli/evm.go index 8eb9edd9749..7eb36f8953b 100644 --- a/cli/evm.go +++ b/cli/evm.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cli/filplus.go b/cli/filplus.go index 072b49b29b6..9fbd2a489ad 100644 --- a/cli/filplus.go +++ b/cli/filplus.go @@ -19,6 +19,7 @@ import ( verifregtypes8 "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" verifregtypes9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/cli/info.go b/cli/info.go index 80abae3b4cb..8b36be4889b 100644 --- a/cli/info.go +++ b/cli/info.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/mpool.go b/cli/mpool.go index c0e068440b6..c83fb4b614a 100644 --- a/cli/mpool.go +++ b/cli/mpool.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/cli/mpool_manage.go b/cli/mpool_manage.go index c3c49cd956c..67724c77332 100644 --- a/cli/mpool_manage.go +++ b/cli/mpool_manage.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/cli/mpool_test.go b/cli/mpool_test.go index 4f4aa831ce3..0aa055ba31f 100644 --- a/cli/mpool_test.go +++ b/cli/mpool_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/multisig.go b/cli/multisig.go index 7cfb2c9c6c8..1af2a4c9e4e 100644 --- a/cli/multisig.go +++ b/cli/multisig.go @@ -19,6 +19,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -27,8 +30,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - msig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) var multisigCmd = &cli.Command{ diff --git a/cli/net.go b/cli/net.go index 031c681e915..f25799e9503 100644 --- a/cli/net.go +++ b/cli/net.go @@ -21,6 +21,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + atypes "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/addrutil" diff --git a/cli/params.go b/cli/params.go index 3491441396b..e79eb8e3071 100644 --- a/cli/params.go +++ b/cli/params.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-paramfetch" + "github.com/filecoin-project/lotus/build" ) diff --git a/cli/paych.go b/cli/paych.go index 9000eab6667..1067d091376 100644 --- a/cli/paych.go +++ b/cli/paych.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin/v8/paych" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lpaych "github.com/filecoin-project/lotus/chain/actors/builtin/paych" diff --git a/cli/send.go b/cli/send.go index bf4843f0ca8..cfa2515c07b 100644 --- a/cli/send.go +++ b/cli/send.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cli/send_test.go b/cli/send_test.go index dfe87de7cbd..2c59a9641f6 100644 --- a/cli/send_test.go +++ b/cli/send_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cli/sending_ui.go b/cli/sending_ui.go index 88a30cf1064..d2d2ed3c110 100644 --- a/cli/sending_ui.go +++ b/cli/sending_ui.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/services.go b/cli/services.go index a4ea794c314..ef257693f89 100644 --- a/cli/services.go +++ b/cli/services.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/services_send_test.go b/cli/services_send_test.go index 0640300d5e1..a524abe8cc1 100644 --- a/cli/services_send_test.go +++ b/cli/services_send_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/servicesmock_test.go b/cli/servicesmock_test.go index e9a792ce800..11260ab08be 100644 --- a/cli/servicesmock_test.go +++ b/cli/servicesmock_test.go @@ -13,6 +13,7 @@ import ( go_address "github.com/filecoin-project/go-address" abi "github.com/filecoin-project/go-state-types/abi" big "github.com/filecoin-project/go-state-types/big" + api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cli/state.go b/cli/state.go index d30347ca150..31666a21c08 100644 --- a/cli/state.go +++ b/cli/state.go @@ -31,6 +31,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cli/sync.go b/cli/sync.go index 659b270bf1a..18ff24bc26e 100644 --- a/cli/sync.go +++ b/cli/sync.go @@ -9,6 +9,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" diff --git a/cli/sync_test.go b/cli/sync_test.go index 87613c73dba..456d165f293 100644 --- a/cli/sync_test.go +++ b/cli/sync_test.go @@ -11,6 +11,7 @@ import ( "github.com/stretchr/testify/assert" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/mock" diff --git a/cli/util/api.go b/cli/util/api.go index 5181e876167..3602b752de2 100644 --- a/cli/util/api.go +++ b/cli/util/api.go @@ -18,6 +18,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cli/util/epoch.go b/cli/util/epoch.go index 019f49806d3..81c92a7e3ed 100644 --- a/cli/util/epoch.go +++ b/cli/util/epoch.go @@ -7,6 +7,7 @@ import ( "github.com/hako/durafmt" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cli/wallet.go b/cli/wallet.go index 79d432a2390..faf7bc23955 100644 --- a/cli/wallet.go +++ b/cli/wallet.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/cli/wallet_test.go b/cli/wallet_test.go index d628fc215c1..eb2c544f0a6 100644 --- a/cli/wallet_test.go +++ b/cli/wallet_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/chain-noise/main.go b/cmd/chain-noise/main.go index 6e73779e2de..6f124fad7ed 100644 --- a/cmd/chain-noise/main.go +++ b/cmd/chain-noise/main.go @@ -10,6 +10,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-bench/caching_verifier.go b/cmd/lotus-bench/caching_verifier.go index 487ea969246..4a4e8879ff7 100644 --- a/cmd/lotus-bench/caching_verifier.go +++ b/cmd/lotus-bench/caching_verifier.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/cmd/lotus-bench/import.go b/cmd/lotus-bench/import.go index c240897061c..95b91054a75 100644 --- a/cmd/lotus-bench/import.go +++ b/cmd/lotus-bench/import.go @@ -29,6 +29,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" diff --git a/cmd/lotus-bench/main.go b/cmd/lotus-bench/main.go index 543b2d48a90..7d3c0cde084 100644 --- a/cmd/lotus-bench/main.go +++ b/cmd/lotus-bench/main.go @@ -30,6 +30,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" prooftypes "github.com/filecoin-project/go-state-types/proof" + adt "github.com/filecoin-project/specs-actors/v6/actors/util/adt" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -41,7 +43,6 @@ import ( "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper/basicfs" "github.com/filecoin-project/lotus/storage/sealer/storiface" - adt "github.com/filecoin-project/specs-actors/v6/actors/util/adt" ) var log = logging.Logger("lotus-bench") diff --git a/cmd/lotus-bench/simple.go b/cmd/lotus-bench/simple.go index 3c4e3b92eb1..8ae5713ad07 100644 --- a/cmd/lotus-bench/simple.go +++ b/cmd/lotus-bench/simple.go @@ -18,11 +18,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/abi" + prf "github.com/filecoin-project/specs-actors/actors/runtime/proof" + "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" - prf "github.com/filecoin-project/specs-actors/actors/runtime/proof" ) var simpleCmd = &cli.Command{ diff --git a/cmd/lotus-fountain/main.go b/cmd/lotus-fountain/main.go index 1f4ce8d8395..f6d503c2f3d 100644 --- a/cmd/lotus-fountain/main.go +++ b/cmd/lotus-fountain/main.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-address" verifregtypes9 "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-gateway/main.go b/cmd/lotus-gateway/main.go index 0d709cb286e..35a43e18b1e 100644 --- a/cmd/lotus-gateway/main.go +++ b/cmd/lotus-gateway/main.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-health/main.go b/cmd/lotus-health/main.go index ea21b200ccb..a7052f21480 100644 --- a/cmd/lotus-health/main.go +++ b/cmd/lotus-health/main.go @@ -13,6 +13,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/actor.go b/cmd/lotus-miner/actor.go index ce9e77cace2..320c4e6de81 100644 --- a/cmd/lotus-miner/actor.go +++ b/cmd/lotus-miner/actor.go @@ -24,6 +24,7 @@ import ( minerV12 "github.com/filecoin-project/go-state-types/builtin/v12/miner" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-miner/actor_test.go b/cmd/lotus-miner/actor_test.go index 19106927601..dfb4522137c 100644 --- a/cmd/lotus-miner/actor_test.go +++ b/cmd/lotus-miner/actor_test.go @@ -15,6 +15,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/backup.go b/cmd/lotus-miner/backup.go index 7443a101b04..cf8c9f9125c 100644 --- a/cmd/lotus-miner/backup.go +++ b/cmd/lotus-miner/backup.go @@ -4,6 +4,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" + lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/cmd/lotus-miner/info.go b/cmd/lotus-miner/info.go index 532cef80ed4..6d8ade340ef 100644 --- a/cmd/lotus-miner/info.go +++ b/cmd/lotus-miner/info.go @@ -18,6 +18,8 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/blockstore" @@ -31,7 +33,6 @@ import ( "github.com/filecoin-project/lotus/journal/alerting" sealing "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" - "github.com/filecoin-project/specs-actors/actors/builtin" ) var infoCmd = &cli.Command{ diff --git a/cmd/lotus-miner/init.go b/cmd/lotus-miner/init.go index 67e710d06a0..c109e85b980 100644 --- a/cmd/lotus-miner/init.go +++ b/cmd/lotus-miner/init.go @@ -30,6 +30,10 @@ import ( "github.com/filecoin-project/go-state-types/builtin" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-statestore" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" + power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" @@ -54,9 +58,6 @@ import ( "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/power" - power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" ) var initCmd = &cli.Command{ diff --git a/cmd/lotus-miner/init_restore.go b/cmd/lotus-miner/init_restore.go index ab62fd0a34a..7e28729bbeb 100644 --- a/cmd/lotus-miner/init_restore.go +++ b/cmd/lotus-miner/init_restore.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" diff --git a/cmd/lotus-miner/init_service.go b/cmd/lotus-miner/init_service.go index 45027cc3f8b..235e4e4c8cc 100644 --- a/cmd/lotus-miner/init_service.go +++ b/cmd/lotus-miner/init_service.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" diff --git a/cmd/lotus-miner/main.go b/cmd/lotus-miner/main.go index 0902ec187ae..911e98e260a 100644 --- a/cmd/lotus-miner/main.go +++ b/cmd/lotus-miner/main.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-miner/market.go b/cmd/lotus-miner/market.go index 69a5442b512..29eb662a78d 100644 --- a/cmd/lotus-miner/market.go +++ b/cmd/lotus-miner/market.go @@ -27,6 +27,7 @@ import ( datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-miner/precommits-info.go b/cmd/lotus-miner/precommits-info.go index 16146e736b9..3f9e8c92742 100644 --- a/cmd/lotus-miner/precommits-info.go +++ b/cmd/lotus-miner/precommits-info.go @@ -7,11 +7,12 @@ import ( cbor "github.com/ipfs/go-ipld-cbor" "github.com/urfave/cli/v2" + "github.com/filecoin-project/specs-actors/v7/actors/util/adt" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - "github.com/filecoin-project/specs-actors/v7/actors/util/adt" ) var sectorPreCommitsCmd = &cli.Command{ diff --git a/cmd/lotus-miner/proving.go b/cmd/lotus-miner/proving.go index c64072ee1ad..3ecc58ba7af 100644 --- a/cmd/lotus-miner/proving.go +++ b/cmd/lotus-miner/proving.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/cmd/lotus-miner/retrieval-deals.go b/cmd/lotus-miner/retrieval-deals.go index a7b4a62eeaf..42b0fa1f6f8 100644 --- a/cmd/lotus-miner/retrieval-deals.go +++ b/cmd/lotus-miner/retrieval-deals.go @@ -9,6 +9,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-miner/sealing.go b/cmd/lotus-miner/sealing.go index 7d732044f0e..b2f4dcab911 100644 --- a/cmd/lotus-miner/sealing.go +++ b/cmd/lotus-miner/sealing.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/go-padreader" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/httpreader" diff --git a/cmd/lotus-miner/sectors.go b/cmd/lotus-miner/sectors.go index 936535bf625..07cc2e795d1 100644 --- a/cmd/lotus-miner/sectors.go +++ b/cmd/lotus-miner/sectors.go @@ -25,6 +25,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-miner/storage.go b/cmd/lotus-miner/storage.go index d3edc8e52b7..fdd5b569656 100644 --- a/cmd/lotus-miner/storage.go +++ b/cmd/lotus-miner/storage.go @@ -22,6 +22,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-pcr/main.go b/cmd/lotus-pcr/main.go index b1fcd2079a9..199810e03cd 100644 --- a/cmd/lotus-pcr/main.go +++ b/cmd/lotus-pcr/main.go @@ -29,6 +29,8 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" @@ -37,7 +39,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/tools/stats/sync" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var log = logging.Logger("main") diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 71e2acb64d3..6248742cc83 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -23,6 +23,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-seed/genesis.go b/cmd/lotus-seed/genesis.go index 44665057f02..9fdce456bf1 100644 --- a/cmd/lotus-seed/genesis.go +++ b/cmd/lotus-seed/genesis.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" diff --git a/cmd/lotus-seed/main.go b/cmd/lotus-seed/main.go index 0a1decc2274..d362804c95d 100644 --- a/cmd/lotus-seed/main.go +++ b/cmd/lotus-seed/main.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-seed/seed/seed.go b/cmd/lotus-seed/seed/seed.go index 45a292094f7..48f00f8a638 100644 --- a/cmd/lotus-seed/seed/seed.go +++ b/cmd/lotus-seed/seed/seed.go @@ -22,6 +22,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" diff --git a/cmd/lotus-shed/actor.go b/cmd/lotus-shed/actor.go index da466476e10..8562b63c32d 100644 --- a/cmd/lotus-shed/actor.go +++ b/cmd/lotus-shed/actor.go @@ -17,13 +17,14 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/tablewriter" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var actorCmd = &cli.Command{ diff --git a/cmd/lotus-shed/balancer.go b/cmd/lotus-shed/balancer.go index 115839b68e8..edc484ab644 100644 --- a/cmd/lotus-shed/balancer.go +++ b/cmd/lotus-shed/balancer.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/balances.go b/cmd/lotus-shed/balances.go index d0cf8da0f32..28569cd1212 100644 --- a/cmd/lotus-shed/balances.go +++ b/cmd/lotus-shed/balances.go @@ -23,6 +23,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-shed/cid.go b/cmd/lotus-shed/cid.go index b26c15a7579..f6c4a472171 100644 --- a/cmd/lotus-shed/cid.go +++ b/cmd/lotus-shed/cid.go @@ -17,6 +17,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/cmd/lotus-shed/consensus.go b/cmd/lotus-shed/consensus.go index 9939202dd7f..197de56f92b 100644 --- a/cmd/lotus-shed/consensus.go +++ b/cmd/lotus-shed/consensus.go @@ -14,6 +14,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-shed/cron-count.go b/cmd/lotus-shed/cron-count.go index b72d0d3552a..9741792ecf1 100644 --- a/cmd/lotus-shed/cron-count.go +++ b/cmd/lotus-shed/cron-count.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/deal-label.go b/cmd/lotus-shed/deal-label.go index 3407b104bcf..417d1370193 100644 --- a/cmd/lotus-shed/deal-label.go +++ b/cmd/lotus-shed/deal-label.go @@ -10,12 +10,13 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/node/repo" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var dealLabelCmd = &cli.Command{ diff --git a/cmd/lotus-shed/diff.go b/cmd/lotus-shed/diff.go index 439a40619da..981dc850c62 100644 --- a/cmd/lotus-shed/diff.go +++ b/cmd/lotus-shed/diff.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner9 "github.com/filecoin-project/go-state-types/builtin/v9/miner" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/election.go b/cmd/lotus-shed/election.go index 4737437fb32..23c533c315f 100644 --- a/cmd/lotus-shed/election.go +++ b/cmd/lotus-shed/election.go @@ -11,11 +11,12 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) var electionCmd = &cli.Command{ diff --git a/cmd/lotus-shed/eth.go b/cmd/lotus-shed/eth.go index 8df17e6e9ed..fde4f96f68f 100644 --- a/cmd/lotus-shed/eth.go +++ b/cmd/lotus-shed/eth.go @@ -7,6 +7,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/export.go b/cmd/lotus-shed/export.go index d0dfe231e65..e6d0c4e056f 100644 --- a/cmd/lotus-shed/export.go +++ b/cmd/lotus-shed/export.go @@ -29,6 +29,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/store" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/fevmanalytics.go b/cmd/lotus-shed/fevmanalytics.go index 73306bf1f8f..19416b77e12 100644 --- a/cmd/lotus-shed/fevmanalytics.go +++ b/cmd/lotus-shed/fevmanalytics.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-shed/fip-0036.go b/cmd/lotus-shed/fip-0036.go index 22110be0557..4c8456c04ce 100644 --- a/cmd/lotus-shed/fip-0036.go +++ b/cmd/lotus-shed/fip-0036.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/cmd/lotus-shed/fr32.go b/cmd/lotus-shed/fr32.go index e9dff0029be..1bdca198cf0 100644 --- a/cmd/lotus-shed/fr32.go +++ b/cmd/lotus-shed/fr32.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/cmd/lotus-shed/frozen-miners.go b/cmd/lotus-shed/frozen-miners.go index e98017f3619..035777bd609 100644 --- a/cmd/lotus-shed/frozen-miners.go +++ b/cmd/lotus-shed/frozen-miners.go @@ -7,8 +7,9 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + + lcli "github.com/filecoin-project/lotus/cli" ) var frozenMinersCmd = &cli.Command{ diff --git a/cmd/lotus-shed/gas-estimation.go b/cmd/lotus-shed/gas-estimation.go index e5f8c6bb25d..5dc048f562c 100644 --- a/cmd/lotus-shed/gas-estimation.go +++ b/cmd/lotus-shed/gas-estimation.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon/drand" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/cmd/lotus-shed/genesis-verify.go b/cmd/lotus-shed/genesis-verify.go index 7e55c7f488d..6795f1528a1 100644 --- a/cmd/lotus-shed/genesis-verify.go +++ b/cmd/lotus-shed/genesis-verify.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/cmd/lotus-shed/hello.go b/cmd/lotus-shed/hello.go index 8510d66e92d..d16f9373563 100644 --- a/cmd/lotus-shed/hello.go +++ b/cmd/lotus-shed/hello.go @@ -11,6 +11,7 @@ import ( "github.com/urfave/cli/v2" cborutil "github.com/filecoin-project/go-cbor-util" + lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/hello" ) diff --git a/cmd/lotus-shed/indexes.go b/cmd/lotus-shed/indexes.go index e6b91439021..be7d43e0513 100644 --- a/cmd/lotus-shed/indexes.go +++ b/cmd/lotus-shed/indexes.go @@ -18,6 +18,7 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/cmd/lotus-shed/invariants.go b/cmd/lotus-shed/invariants.go index 2764e2a1020..e74a0dd24c2 100644 --- a/cmd/lotus-shed/invariants.go +++ b/cmd/lotus-shed/invariants.go @@ -19,6 +19,7 @@ import ( v12 "github.com/filecoin-project/go-state-types/builtin/v12" v8 "github.com/filecoin-project/go-state-types/builtin/v8" v9 "github.com/filecoin-project/go-state-types/builtin/v9" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/cmd/lotus-shed/jwt.go b/cmd/lotus-shed/jwt.go index aaa78a6fd52..2a24c256933 100644 --- a/cmd/lotus-shed/jwt.go +++ b/cmd/lotus-shed/jwt.go @@ -14,6 +14,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules" diff --git a/cmd/lotus-shed/ledger.go b/cmd/lotus-shed/ledger.go index efce54a5e8c..d9a888d2061 100644 --- a/cmd/lotus-shed/ledger.go +++ b/cmd/lotus-shed/ledger.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/types" ledgerwallet "github.com/filecoin-project/lotus/chain/wallet/ledger" diff --git a/cmd/lotus-shed/market.go b/cmd/lotus-shed/market.go index ddb82a2f0fb..4436e3c404a 100644 --- a/cmd/lotus-shed/market.go +++ b/cmd/lotus-shed/market.go @@ -23,6 +23,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" market11 "github.com/filecoin-project/go-state-types/builtin/v11/market" "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/math.go b/cmd/lotus-shed/math.go index fc01ce5561d..09776d4e76f 100644 --- a/cmd/lotus-shed/math.go +++ b/cmd/lotus-shed/math.go @@ -10,8 +10,9 @@ import ( "github.com/urfave/cli/v2" "golang.org/x/xerrors" - "github.com/filecoin-project/lotus/chain/types" miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" + + "github.com/filecoin-project/lotus/chain/types" ) var mathCmd = &cli.Command{ diff --git a/cmd/lotus-shed/mempool-stats.go b/cmd/lotus-shed/mempool-stats.go index 5af877bdce5..08aceb02025 100644 --- a/cmd/lotus-shed/mempool-stats.go +++ b/cmd/lotus-shed/mempool-stats.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin" + lapi "github.com/filecoin-project/lotus/api" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/migrations.go b/cmd/lotus-shed/migrations.go index 12939913f13..96e4747b7ef 100644 --- a/cmd/lotus-shed/migrations.go +++ b/cmd/lotus-shed/migrations.go @@ -32,6 +32,8 @@ import ( "github.com/filecoin-project/go-state-types/manifest" mutil "github.com/filecoin-project/go-state-types/migration" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" + "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/blockstore/splitstore" @@ -53,7 +55,6 @@ import ( lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" - "github.com/filecoin-project/specs-actors/v7/actors/migration/nv15" ) var migrationsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner-multisig.go b/cmd/lotus-shed/miner-multisig.go index 19804dfbd26..e8394b17a60 100644 --- a/cmd/lotus-shed/miner-multisig.go +++ b/cmd/lotus-shed/miner-multisig.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/builtin/v9/multisig" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" diff --git a/cmd/lotus-shed/miner-peerid.go b/cmd/lotus-shed/miner-peerid.go index 4bd92de4b43..e430637976c 100644 --- a/cmd/lotus-shed/miner-peerid.go +++ b/cmd/lotus-shed/miner-peerid.go @@ -12,6 +12,9 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" @@ -19,8 +22,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" - builtin5 "github.com/filecoin-project/specs-actors/v5/actors/builtin" ) var minerPeeridCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner-types.go b/cmd/lotus-shed/miner-types.go index 9b7c690b935..822d037aa26 100644 --- a/cmd/lotus-shed/miner-types.go +++ b/cmd/lotus-shed/miner-types.go @@ -14,6 +14,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" big2 "github.com/filecoin-project/go-state-types/big" + builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" + "github.com/filecoin-project/specs-actors/v4/actors/util/adt" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -21,8 +24,6 @@ import ( "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/repo" - builtin4 "github.com/filecoin-project/specs-actors/v4/actors/builtin" - "github.com/filecoin-project/specs-actors/v4/actors/util/adt" ) var minerTypesCmd = &cli.Command{ diff --git a/cmd/lotus-shed/miner.go b/cmd/lotus-shed/miner.go index 82026590a22..a8bb9374422 100644 --- a/cmd/lotus-shed/miner.go +++ b/cmd/lotus-shed/miner.go @@ -28,6 +28,9 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v11/util/adt" miner8 "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/crypto" + power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" + "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -35,8 +38,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - power7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/power" - "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) var minerCmd = &cli.Command{ diff --git a/cmd/lotus-shed/msg.go b/cmd/lotus-shed/msg.go index 618d5195ce9..35f8eed35c1 100644 --- a/cmd/lotus-shed/msg.go +++ b/cmd/lotus-shed/msg.go @@ -14,10 +14,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) var msgCmd = &cli.Command{ diff --git a/cmd/lotus-shed/msig.go b/cmd/lotus-shed/msig.go index e877b4ff733..ccc932c93ff 100644 --- a/cmd/lotus-shed/msig.go +++ b/cmd/lotus-shed/msig.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/cmd/lotus-shed/nonce-fix.go b/cmd/lotus-shed/nonce-fix.go index c55314b9a78..d69c8a48dc6 100644 --- a/cmd/lotus-shed/nonce-fix.go +++ b/cmd/lotus-shed/nonce-fix.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/params.go b/cmd/lotus-shed/params.go index 88000299b7e..08ce5ba83f1 100644 --- a/cmd/lotus-shed/params.go +++ b/cmd/lotus-shed/params.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-paramfetch" + "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/postfind.go b/cmd/lotus-shed/postfind.go index ec397b56bf9..23eb4ceb717 100644 --- a/cmd/lotus-shed/postfind.go +++ b/cmd/lotus-shed/postfind.go @@ -8,10 +8,11 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/specs-actors/v2/actors/builtin" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) var postFindCmd = &cli.Command{ diff --git a/cmd/lotus-shed/proofs.go b/cmd/lotus-shed/proofs.go index 55cb715b8d6..1a16e2fdc3b 100644 --- a/cmd/lotus-shed/proofs.go +++ b/cmd/lotus-shed/proofs.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/lotus-shed/pruning.go b/cmd/lotus-shed/pruning.go index ac96e2ea4f6..c0bd453b145 100644 --- a/cmd/lotus-shed/pruning.go +++ b/cmd/lotus-shed/pruning.go @@ -11,6 +11,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/store" diff --git a/cmd/lotus-shed/sectors.go b/cmd/lotus-shed/sectors.go index 21d14a26d81..899e0f290b4 100644 --- a/cmd/lotus-shed/sectors.go +++ b/cmd/lotus-shed/sectors.go @@ -25,6 +25,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" @@ -35,7 +37,6 @@ import ( "github.com/filecoin-project/lotus/storage/sealer/fr32" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) var sectorsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/send-csv.go b/cmd/lotus-shed/send-csv.go index a697802edf4..17b62150fc5 100644 --- a/cmd/lotus-shed/send-csv.go +++ b/cmd/lotus-shed/send-csv.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/signatures.go b/cmd/lotus-shed/signatures.go index ca761a2c0f5..536f8e82d6a 100644 --- a/cmd/lotus-shed/signatures.go +++ b/cmd/lotus-shed/signatures.go @@ -12,6 +12,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/cmd/lotus-shed/state-stats.go b/cmd/lotus-shed/state-stats.go index ca2b8b10e9e..4eb00f98183 100644 --- a/cmd/lotus-shed/state-stats.go +++ b/cmd/lotus-shed/state-stats.go @@ -26,6 +26,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" gstactors "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" diff --git a/cmd/lotus-shed/stateroot-stats.go b/cmd/lotus-shed/stateroot-stats.go index 7605366ed94..16dfc593578 100644 --- a/cmd/lotus-shed/stateroot-stats.go +++ b/cmd/lotus-shed/stateroot-stats.go @@ -9,6 +9,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/sync.go b/cmd/lotus-shed/sync.go index f6b0e5c7e2d..eb11dea27eb 100644 --- a/cmd/lotus-shed/sync.go +++ b/cmd/lotus-shed/sync.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-shed/terminations.go b/cmd/lotus-shed/terminations.go index a16fafb6e12..c5f35995a4f 100644 --- a/cmd/lotus-shed/terminations.go +++ b/cmd/lotus-shed/terminations.go @@ -13,6 +13,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" + miner2 "github.com/filecoin-project/specs-actors/actors/builtin/miner" + "github.com/filecoin-project/lotus/chain/actors/adt" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -23,7 +25,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/node/repo" - miner2 "github.com/filecoin-project/specs-actors/actors/builtin/miner" ) var terminationsCmd = &cli.Command{ diff --git a/cmd/lotus-shed/verifreg.go b/cmd/lotus-shed/verifreg.go index 3bf81d541ba..4e2894442a3 100644 --- a/cmd/lotus-shed/verifreg.go +++ b/cmd/lotus-shed/verifreg.go @@ -13,6 +13,8 @@ import ( "github.com/filecoin-project/go-state-types/big" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v8/verifreg" "github.com/filecoin-project/go-state-types/crypto" + verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -21,7 +23,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" - verifreg2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/verifreg" ) var verifRegCmd = &cli.Command{ diff --git a/cmd/lotus-sim/info.go b/cmd/lotus-sim/info.go index 7eadde6d8ab..b92fa4b2f4d 100644 --- a/cmd/lotus-sim/info.go +++ b/cmd/lotus-sim/info.go @@ -10,6 +10,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/cmd/lotus-sim/info_capacity.go b/cmd/lotus-sim/info_capacity.go index 476d2716da9..a92d2cde494 100644 --- a/cmd/lotus-sim/info_capacity.go +++ b/cmd/lotus-sim/info_capacity.go @@ -6,6 +6,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/info_commit.go b/cmd/lotus-sim/info_commit.go index 98613748a06..e9bf8aa973d 100644 --- a/cmd/lotus-sim/info_commit.go +++ b/cmd/lotus-sim/info_commit.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation" diff --git a/cmd/lotus-sim/info_state.go b/cmd/lotus-sim/info_state.go index 9050f34eb77..125dae81d96 100644 --- a/cmd/lotus-sim/info_state.go +++ b/cmd/lotus-sim/info_state.go @@ -13,6 +13,7 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/info_wdpost.go b/cmd/lotus-sim/info_wdpost.go index 2f53d4f47b8..426e85ca831 100644 --- a/cmd/lotus-sim/info_wdpost.go +++ b/cmd/lotus-sim/info_wdpost.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation" diff --git a/cmd/lotus-sim/simulation/block.go b/cmd/lotus-sim/simulation/block.go index 846082f42fa..7cf5a6be66b 100644 --- a/cmd/lotus-sim/simulation/block.go +++ b/cmd/lotus-sim/simulation/block.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go index e7cb6c996a3..a4c9fdeaaa0 100644 --- a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go +++ b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-sim/simulation/messages.go b/cmd/lotus-sim/simulation/messages.go index 44ffc6880b2..d6dd98d4382 100644 --- a/cmd/lotus-sim/simulation/messages.go +++ b/cmd/lotus-sim/simulation/messages.go @@ -6,8 +6,9 @@ import ( "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" - "github.com/filecoin-project/lotus/chain/types" blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + + "github.com/filecoin-project/lotus/chain/types" ) // toArray converts the given set of CIDs to an AMT. This is usually used to pack messages into blocks. diff --git a/cmd/lotus-sim/simulation/mock/mock.go b/cmd/lotus-sim/simulation/mock/mock.go index bbf2cdbcb39..4699b2aa665 100644 --- a/cmd/lotus-sim/simulation/mock/mock.go +++ b/cmd/lotus-sim/simulation/mock/mock.go @@ -12,9 +12,10 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" - "github.com/filecoin-project/lotus/storage/sealer/storiface" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" tutils "github.com/filecoin-project/specs-actors/v5/support/testing" + + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) // Ideally, we'd use extern/sealer/mock. Unfortunately, those mocks are a bit _too_ accurate diff --git a/cmd/lotus-sim/simulation/simulation.go b/cmd/lotus-sim/simulation/simulation.go index af793dc1742..47d06aeda91 100644 --- a/cmd/lotus-sim/simulation/simulation.go +++ b/cmd/lotus-sim/simulation/simulation.go @@ -13,6 +13,8 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/index" @@ -21,7 +23,6 @@ import ( "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/stages" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" ) var log = logging.Logger("simulation") diff --git a/cmd/lotus-sim/simulation/stages/commit_queue.go b/cmd/lotus-sim/simulation/stages/commit_queue.go index 1824f49eba1..60cbfa4badf 100644 --- a/cmd/lotus-sim/simulation/stages/commit_queue.go +++ b/cmd/lotus-sim/simulation/stages/commit_queue.go @@ -6,6 +6,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/cmd/lotus-sim/simulation/stages/commit_queue_test.go b/cmd/lotus-sim/simulation/stages/commit_queue_test.go index 8db4d681e84..0b9a2ebdbc4 100644 --- a/cmd/lotus-sim/simulation/stages/commit_queue_test.go +++ b/cmd/lotus-sim/simulation/stages/commit_queue_test.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" + "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/cmd/lotus-sim/simulation/stages/funding_stage.go b/cmd/lotus-sim/simulation/stages/funding_stage.go index 712ea1bc910..4ce4afae103 100644 --- a/cmd/lotus-sim/simulation/stages/funding_stage.go +++ b/cmd/lotus-sim/simulation/stages/funding_stage.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/cmd/lotus-sim/simulation/stages/interface.go b/cmd/lotus-sim/simulation/stages/interface.go index 718f7a9f52a..fffdbec6b54 100644 --- a/cmd/lotus-sim/simulation/stages/interface.go +++ b/cmd/lotus-sim/simulation/stages/interface.go @@ -6,6 +6,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" ) diff --git a/cmd/lotus-sim/simulation/stages/precommit_stage.go b/cmd/lotus-sim/simulation/stages/precommit_stage.go index a67c5ba42f1..1a89413d72c 100644 --- a/cmd/lotus-sim/simulation/stages/precommit_stage.go +++ b/cmd/lotus-sim/simulation/stages/precommit_stage.go @@ -13,6 +13,8 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -21,7 +23,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) const ( diff --git a/cmd/lotus-sim/simulation/stages/provecommit_stage.go b/cmd/lotus-sim/simulation/stages/provecommit_stage.go index 71c483b3e1a..d15ea60f0a9 100644 --- a/cmd/lotus-sim/simulation/stages/provecommit_stage.go +++ b/cmd/lotus-sim/simulation/stages/provecommit_stage.go @@ -11,6 +11,9 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -19,8 +22,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/blockbuilder" "github.com/filecoin-project/lotus/cmd/lotus-sim/simulation/mock" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" - power5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/power" ) const ( diff --git a/cmd/lotus-sim/simulation/stages/util.go b/cmd/lotus-sim/simulation/stages/util.go index ed820068ec8..0c17823c060 100644 --- a/cmd/lotus-sim/simulation/stages/util.go +++ b/cmd/lotus-sim/simulation/stages/util.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/builtin/power" diff --git a/cmd/lotus-sim/simulation/stages/windowpost_stage.go b/cmd/lotus-sim/simulation/stages/windowpost_stage.go index 3e7e5fe1e9f..0b4109d08cb 100644 --- a/cmd/lotus-sim/simulation/stages/windowpost_stage.go +++ b/cmd/lotus-sim/simulation/stages/windowpost_stage.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/aerrors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/cmd/lotus-stats/main.go b/cmd/lotus-stats/main.go index 0318d52f518..20971c1f3ef 100644 --- a/cmd/lotus-stats/main.go +++ b/cmd/lotus-stats/main.go @@ -15,6 +15,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/tools/stats/influx" diff --git a/cmd/lotus-wallet/interactive.go b/cmd/lotus-wallet/interactive.go index a72b090d63c..96d2ad9f2e0 100644 --- a/cmd/lotus-wallet/interactive.go +++ b/cmd/lotus-wallet/interactive.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/cmd/lotus-wallet/logged.go b/cmd/lotus-wallet/logged.go index cba281f0200..4f07d6ae46e 100644 --- a/cmd/lotus-wallet/logged.go +++ b/cmd/lotus-wallet/logged.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/cmd/lotus-wallet/main.go b/cmd/lotus-wallet/main.go index b40a56cddb0..8360dae15d0 100644 --- a/cmd/lotus-wallet/main.go +++ b/cmd/lotus-wallet/main.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus-worker/main.go b/cmd/lotus-worker/main.go index 4a4d20574cd..257dac800c2 100644 --- a/cmd/lotus-worker/main.go +++ b/cmd/lotus-worker/main.go @@ -26,6 +26,7 @@ import ( "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lcli "github.com/filecoin-project/lotus/cli" diff --git a/cmd/lotus-worker/sealworker/rpc.go b/cmd/lotus-worker/sealworker/rpc.go index 9de1303fdd5..97f78942e59 100644 --- a/cmd/lotus-worker/sealworker/rpc.go +++ b/cmd/lotus-worker/sealworker/rpc.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/cmd/lotus/backup.go b/cmd/lotus/backup.go index 9393e504441..efbac3e2bb0 100644 --- a/cmd/lotus/backup.go +++ b/cmd/lotus/backup.go @@ -10,6 +10,7 @@ import ( "gopkg.in/cheggaaa/pb.v1" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/chain/store" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/lib/backupds" diff --git a/cmd/lotus/daemon.go b/cmd/lotus/daemon.go index 35d30d5f550..042e89f216a 100644 --- a/cmd/lotus/daemon.go +++ b/cmd/lotus/daemon.go @@ -30,6 +30,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-paramfetch" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon/drand" diff --git a/cmd/lotus/debug_advance.go b/cmd/lotus/debug_advance.go index de8140623a9..dee2db97e2b 100644 --- a/cmd/lotus/debug_advance.go +++ b/cmd/lotus/debug_advance.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" diff --git a/cmd/tvx/codenames.go b/cmd/tvx/codenames.go index 00291c43c06..81143c85ce0 100644 --- a/cmd/tvx/codenames.go +++ b/cmd/tvx/codenames.go @@ -2,6 +2,7 @@ package main import ( "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" ) diff --git a/cmd/tvx/codenames_test.go b/cmd/tvx/codenames_test.go index 5cf531887ff..46d8466ecb2 100644 --- a/cmd/tvx/codenames_test.go +++ b/cmd/tvx/codenames_test.go @@ -6,6 +6,7 @@ import ( "testing" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" ) diff --git a/cmd/tvx/exec.go b/cmd/tvx/exec.go index dd65facf820..95a16e6d13c 100644 --- a/cmd/tvx/exec.go +++ b/cmd/tvx/exec.go @@ -16,11 +16,12 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/conformance" - "github.com/filecoin-project/test-vectors/schema" ) var execFlags struct { diff --git a/cmd/tvx/extract_many.go b/cmd/tvx/extract_many.go index 3d8db2f0d22..7c8d306d8c0 100644 --- a/cmd/tvx/extract_many.go +++ b/cmd/tvx/extract_many.go @@ -19,6 +19,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/consensus" ) diff --git a/cmd/tvx/extract_message.go b/cmd/tvx/extract_message.go index 49cddb9cb71..95711414bcc 100644 --- a/cmd/tvx/extract_message.go +++ b/cmd/tvx/extract_message.go @@ -12,6 +12,8 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" @@ -21,7 +23,6 @@ import ( "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/vm" "github.com/filecoin-project/lotus/conformance" - "github.com/filecoin-project/test-vectors/schema" ) func doExtractMessage(opts extractOpts) error { diff --git a/cmd/tvx/extract_tipset.go b/cmd/tvx/extract_tipset.go index 64e3b4ff3f9..553961f4491 100644 --- a/cmd/tvx/extract_tipset.go +++ b/cmd/tvx/extract_tipset.go @@ -10,10 +10,11 @@ import ( "github.com/ipfs/go-cid" + "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" "github.com/filecoin-project/lotus/conformance" - "github.com/filecoin-project/test-vectors/schema" ) func doExtractTipset(opts extractOpts) error { diff --git a/cmd/tvx/main.go b/cmd/tvx/main.go index dd284983897..5021dd64b25 100644 --- a/cmd/tvx/main.go +++ b/cmd/tvx/main.go @@ -9,6 +9,7 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api/v1api" lcli "github.com/filecoin-project/lotus/cli" ) diff --git a/cmd/tvx/simulate.go b/cmd/tvx/simulate.go index b6ee85b27e4..ea7b67a662d 100644 --- a/cmd/tvx/simulate.go +++ b/cmd/tvx/simulate.go @@ -14,10 +14,11 @@ import ( "github.com/urfave/cli/v2" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/conformance" - "github.com/filecoin-project/test-vectors/schema" ) var simulateFlags struct { diff --git a/cmd/tvx/state.go b/cmd/tvx/state.go index 6ee06bdac5e..9674bf17ed6 100644 --- a/cmd/tvx/state.go +++ b/cmd/tvx/state.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" init_ "github.com/filecoin-project/lotus/chain/actors/builtin/init" diff --git a/conformance/chaos/actor.go b/conformance/chaos/actor.go index cb3ec2147fa..3a8b2b50af5 100644 --- a/conformance/chaos/actor.go +++ b/conformance/chaos/actor.go @@ -2,6 +2,7 @@ package chaos import ( "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/cbor" @@ -9,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/rt" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" runtime2 "github.com/filecoin-project/specs-actors/v2/actors/runtime" + "github.com/filecoin-project/lotus/chain/actors/builtin" ) diff --git a/conformance/chaos/actor_test.go b/conformance/chaos/actor_test.go index a618ef23081..fd45504d27b 100644 --- a/conformance/chaos/actor_test.go +++ b/conformance/chaos/actor_test.go @@ -4,7 +4,9 @@ package chaos import ( "context" "testing" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" diff --git a/conformance/chaos/cbor_gen.go b/conformance/chaos/cbor_gen.go index aefb37438f6..d74ae0946a1 100644 --- a/conformance/chaos/cbor_gen.go +++ b/conformance/chaos/cbor_gen.go @@ -7,9 +7,11 @@ import ( "io" "math" "sort" + cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" + address "github.com/filecoin-project/go-address" abi "github.com/filecoin-project/go-state-types/abi" exitcode "github.com/filecoin-project/go-state-types/exitcode" diff --git a/conformance/chaos/gen/gen.go b/conformance/chaos/gen/gen.go index cda7204f6a3..20a1be64fb0 100644 --- a/conformance/chaos/gen/gen.go +++ b/conformance/chaos/gen/gen.go @@ -2,6 +2,7 @@ package main import ( gen "github.com/whyrusleeping/cbor-gen" + "github.com/filecoin-project/lotus/conformance/chaos" ) diff --git a/conformance/chaos/ids.go b/conformance/chaos/ids.go index cb3db8b9d7f..478169db5f4 100644 --- a/conformance/chaos/ids.go +++ b/conformance/chaos/ids.go @@ -3,6 +3,7 @@ package chaos import ( "github.com/ipfs/go-cid" "github.com/multiformats/go-multihash" + "github.com/filecoin-project/go-address" ) diff --git a/conformance/corpus_test.go b/conformance/corpus_test.go index fd7d598141a..ec9f9d51640 100644 --- a/conformance/corpus_test.go +++ b/conformance/corpus_test.go @@ -9,6 +9,7 @@ import ( "path/filepath" "strings" "testing" + "github.com/filecoin-project/test-vectors/schema" ) diff --git a/conformance/driver.go b/conformance/driver.go index 78fec4a1dd3..3c62ca7b9ef 100644 --- a/conformance/driver.go +++ b/conformance/driver.go @@ -4,9 +4,11 @@ import ( "context" gobig "math/big" "os" + "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" cbor "github.com/ipfs/go-ipld-cbor" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" @@ -15,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/network" rtt "github.com/filecoin-project/go-state-types/rt" "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/conformance/rand_fixed.go b/conformance/rand_fixed.go index c35ed8553eb..f35f05cd4ff 100644 --- a/conformance/rand_fixed.go +++ b/conformance/rand_fixed.go @@ -2,7 +2,9 @@ package conformance import ( "context" + "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/rand" ) diff --git a/conformance/rand_record.go b/conformance/rand_record.go index 1b2f32e3466..4dc30b28ebf 100644 --- a/conformance/rand_record.go +++ b/conformance/rand_record.go @@ -4,8 +4,10 @@ import ( "context" "fmt" "sync" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/rand" "github.com/filecoin-project/lotus/chain/types" diff --git a/conformance/rand_replay.go b/conformance/rand_replay.go index 0a8256d7f6a..6d78d813b8a 100644 --- a/conformance/rand_replay.go +++ b/conformance/rand_replay.go @@ -2,8 +2,10 @@ package conformance import ( "context" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/chain/rand" ) diff --git a/conformance/reporter.go b/conformance/reporter.go index 8dba374767b..1cd2d389db7 100644 --- a/conformance/reporter.go +++ b/conformance/reporter.go @@ -5,6 +5,7 @@ import ( "os" "sync/atomic" "testing" + "github.com/fatih/color" ) diff --git a/conformance/runner.go b/conformance/runner.go index 433a410d0b2..b470eed4e56 100644 --- a/conformance/runner.go +++ b/conformance/runner.go @@ -10,6 +10,7 @@ import ( "os" "os/exec" "strconv" + "github.com/fatih/color" "github.com/hashicorp/go-multierror" "github.com/ipfs/boxo/blockservice" @@ -20,10 +21,12 @@ import ( ds "github.com/ipfs/go-datastore" format "github.com/ipfs/go-ipld-format" "github.com/ipld/go-car" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/test-vectors/schema" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/eth_sub.go b/gateway/eth_sub.go index be13cc0319d..76d9139835c 100644 --- a/gateway/eth_sub.go +++ b/gateway/eth_sub.go @@ -3,7 +3,9 @@ package gateway import ( "context" "sync" + "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/gateway/handler.go b/gateway/handler.go index 6405a8c7195..54ab2467f91 100644 --- a/gateway/handler.go +++ b/gateway/handler.go @@ -6,11 +6,14 @@ import ( "net/http" "sync" "time" + "contrib.go.opencensus.io/exporter/prometheus" "github.com/gorilla/mux" promclient "github.com/prometheus/client_golang/prometheus" "golang.org/x/time/rate" + "github.com/filecoin-project/go-jsonrpc" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/gateway/node.go b/gateway/node.go index 6c91b61bea0..367e645c1f6 100644 --- a/gateway/node.go +++ b/gateway/node.go @@ -4,10 +4,12 @@ import ( "context" "fmt" "time" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "go.opencensus.io/stats" "golang.org/x/time/rate" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-jsonrpc" @@ -15,6 +17,7 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/gateway/node_test.go b/gateway/node_test.go index 55d205877fd..aa33e8bfb20 100644 --- a/gateway/node_test.go +++ b/gateway/node_test.go @@ -6,11 +6,14 @@ import ( "sync" "testing" "time" + "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/proxy_eth.go b/gateway/proxy_eth.go index 34f1c4e3ea7..e6d433a1744 100644 --- a/gateway/proxy_eth.go +++ b/gateway/proxy_eth.go @@ -7,11 +7,14 @@ import ( "fmt" "sync" "time" + "github.com/ipfs/go-cid" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/types" diff --git a/gateway/proxy_fil.go b/gateway/proxy_fil.go index 079dc1991a8..eb8a354edc5 100644 --- a/gateway/proxy_fil.go +++ b/gateway/proxy_fil.go @@ -2,9 +2,11 @@ package gateway import ( "context" + blocks "github.com/ipfs/go-block-format" "github.com/ipfs/go-cid" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" @@ -12,6 +14,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/genesis/types.go b/genesis/types.go index 62da56c42f5..815a2f0083a 100644 --- a/genesis/types.go +++ b/genesis/types.go @@ -2,12 +2,15 @@ package genesis import ( "encoding/json" + "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/api_test.go b/itests/api_test.go index 583edc39837..ff43bd5c02e 100644 --- a/itests/api_test.go +++ b/itests/api_test.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/batch_deal_test.go b/itests/batch_deal_test.go index 9785941c9ef..21db9f08d0e 100644 --- a/itests/batch_deal_test.go +++ b/itests/batch_deal_test.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/markets/storageadapter" "github.com/filecoin-project/lotus/node" diff --git a/itests/deadlines_test.go b/itests/deadlines_test.go index 2a990d0ec45..fb28f450974 100644 --- a/itests/deadlines_test.go +++ b/itests/deadlines_test.go @@ -18,6 +18,8 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" @@ -28,7 +30,6 @@ import ( "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node/impl" "github.com/filecoin-project/lotus/storage/sealer/mock" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) // TestDeadlineToggling: diff --git a/itests/deals_512mb_test.go b/itests/deals_512mb_test.go index 6f6d088b8ac..7b55204d910 100644 --- a/itests/deals_512mb_test.go +++ b/itests/deals_512mb_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_anycid_test.go b/itests/deals_anycid_test.go index 22ef7add415..c17441090b1 100644 --- a/itests/deals_anycid_test.go +++ b/itests/deals_anycid_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/deals_concurrent_test.go b/itests/deals_concurrent_test.go index fbf3cdb7a7d..a106836bdd1 100644 --- a/itests/deals_concurrent_test.go +++ b/itests/deals_concurrent_test.go @@ -14,6 +14,7 @@ import ( datatransfer "github.com/filecoin-project/go-data-transfer/v2" "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/deals_invalid_utf8_label_test.go b/itests/deals_invalid_utf8_label_test.go index 2ab84ea8f41..cec6fb4c3f5 100644 --- a/itests/deals_invalid_utf8_label_test.go +++ b/itests/deals_invalid_utf8_label_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_max_staging_deals_test.go b/itests/deals_max_staging_deals_test.go index 8e539c5338c..738a1e2fed3 100644 --- a/itests/deals_max_staging_deals_test.go +++ b/itests/deals_max_staging_deals_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_offline_test.go b/itests/deals_offline_test.go index 021a5695c50..997d7723aa6 100644 --- a/itests/deals_offline_test.go +++ b/itests/deals_offline_test.go @@ -13,6 +13,7 @@ import ( commp "github.com/filecoin-project/go-fil-commp-hashhash" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_padding_test.go b/itests/deals_padding_test.go index f9f4a9a9c3c..aaca4536069 100644 --- a/itests/deals_padding_test.go +++ b/itests/deals_padding_test.go @@ -11,6 +11,7 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" commp "github.com/filecoin-project/go-fil-commp-hashhash" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_partial_retrieval_dm-level_test.go b/itests/deals_partial_retrieval_dm-level_test.go index a57a8773b44..c03d07aac53 100644 --- a/itests/deals_partial_retrieval_dm-level_test.go +++ b/itests/deals_partial_retrieval_dm-level_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" api0 "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/deals_partial_retrieval_test.go b/itests/deals_partial_retrieval_test.go index ce66aba6800..0bbf23da054 100644 --- a/itests/deals_partial_retrieval_test.go +++ b/itests/deals_partial_retrieval_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/deals_pricing_test.go b/itests/deals_pricing_test.go index 22f6973c99b..f2301eee8dc 100644 --- a/itests/deals_pricing_test.go +++ b/itests/deals_pricing_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/itests/deals_publish_test.go b/itests/deals_publish_test.go index 2c11af13448..43f4eeb0500 100644 --- a/itests/deals_publish_test.go +++ b/itests/deals_publish_test.go @@ -11,6 +11,8 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" @@ -21,7 +23,6 @@ import ( "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules" "github.com/filecoin-project/lotus/storage/ctladdr" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" ) func TestPublishDealsBatching(t *testing.T) { diff --git a/itests/deals_remote_retrieval_test.go b/itests/deals_remote_retrieval_test.go index 1a9b4f76747..c0a37e69e33 100644 --- a/itests/deals_remote_retrieval_test.go +++ b/itests/deals_remote_retrieval_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/deals_retry_deal_no_funds_test.go b/itests/deals_retry_deal_no_funds_test.go index b75f961f2dd..650b2436e0e 100644 --- a/itests/deals_retry_deal_no_funds_test.go +++ b/itests/deals_retry_deal_no_funds_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/decode_params_test.go b/itests/decode_params_test.go index 812b744b077..6a4a8c681ed 100644 --- a/itests/decode_params_test.go +++ b/itests/decode_params_test.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/cbor" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/cli" diff --git a/itests/dup_mpool_messages_test.go b/itests/dup_mpool_messages_test.go index 99c692dc09b..afafec3eeb3 100644 --- a/itests/dup_mpool_messages_test.go +++ b/itests/dup_mpool_messages_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_account_abstraction_test.go b/itests/eth_account_abstraction_test.go index c61ece49c24..8d92d0a049d 100644 --- a/itests/eth_account_abstraction_test.go +++ b/itests/eth_account_abstraction_test.go @@ -14,6 +14,7 @@ import ( builtin2 "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/eth_api_test.go b/itests/eth_api_test.go index b9575a36a88..43b4b526674 100644 --- a/itests/eth_api_test.go +++ b/itests/eth_api_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_balance_test.go b/itests/eth_balance_test.go index a75cdce6e88..d133ff6d90e 100644 --- a/itests/eth_balance_test.go +++ b/itests/eth_balance_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_block_hash_test.go b/itests/eth_block_hash_test.go index ef7bb13e9d2..b582c84e346 100644 --- a/itests/eth_block_hash_test.go +++ b/itests/eth_block_hash_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/eth_conformance_test.go b/itests/eth_conformance_test.go index 4ae562704d7..15b24997719 100644 --- a/itests/eth_conformance_test.go +++ b/itests/eth_conformance_test.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/itests/eth_deploy_test.go b/itests/eth_deploy_test.go index d07d84875cb..ce4c94a28b8 100644 --- a/itests/eth_deploy_test.go +++ b/itests/eth_deploy_test.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" gstStore "github.com/filecoin-project/go-state-types/store" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/eth_fee_history_test.go b/itests/eth_fee_history_test.go index 29381f849ca..b611efeb18e 100644 --- a/itests/eth_fee_history_test.go +++ b/itests/eth_fee_history_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/eth_filter_test.go b/itests/eth_filter_test.go index d599fd99e84..9212e60fc01 100644 --- a/itests/eth_filter_test.go +++ b/itests/eth_filter_test.go @@ -24,6 +24,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_hash_lookup_test.go b/itests/eth_hash_lookup_test.go index b2d9d82015e..37d0697962c 100644 --- a/itests/eth_hash_lookup_test.go +++ b/itests/eth_hash_lookup_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/eth_transactions_test.go b/itests/eth_transactions_test.go index 2f8990abace..6d60f678636 100644 --- a/itests/eth_transactions_test.go +++ b/itests/eth_transactions_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" diff --git a/itests/fevm_address_test.go b/itests/fevm_address_test.go index 70ca730f50b..9eaac464773 100644 --- a/itests/fevm_address_test.go +++ b/itests/fevm_address_test.go @@ -16,6 +16,7 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/fevm_events_test.go b/itests/fevm_events_test.go index 743ef1a775d..458ac3470ab 100644 --- a/itests/fevm_events_test.go +++ b/itests/fevm_events_test.go @@ -11,6 +11,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/types/ethtypes" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/fevm_test.go b/itests/fevm_test.go index 33f637854a6..cb69c036cad 100644 --- a/itests/fevm_test.go +++ b/itests/fevm_test.go @@ -17,6 +17,7 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/manifest" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/gas_estimation_test.go b/itests/gas_estimation_test.go index 7bd09324404..24013c8855b 100644 --- a/itests/gas_estimation_test.go +++ b/itests/gas_estimation_test.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/itests/gateway_test.go b/itests/gateway_test.go index ef172ef6cbf..d20b3bd1a09 100644 --- a/itests/gateway_test.go +++ b/itests/gateway_test.go @@ -17,6 +17,9 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/chain/stmgr" @@ -26,8 +29,6 @@ import ( "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/itests/multisig" "github.com/filecoin-project/lotus/node" - init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" - multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" ) const ( diff --git a/itests/get_messages_in_ts_test.go b/itests/get_messages_in_ts_test.go index e2ff432eb2c..ecc13e3aaa7 100644 --- a/itests/get_messages_in_ts_test.go +++ b/itests/get_messages_in_ts_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/blockminer.go b/itests/kit/blockminer.go index e387ef52118..bd527910d79 100644 --- a/itests/kit/blockminer.go +++ b/itests/kit/blockminer.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" minertypes "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/miner" diff --git a/itests/kit/client.go b/itests/kit/client.go index d23f43a0987..f7e4657603e 100644 --- a/itests/kit/client.go +++ b/itests/kit/client.go @@ -14,10 +14,11 @@ import ( "github.com/stretchr/testify/require" lcli "github.com/urfave/cli/v2" + "github.com/filecoin-project/specs-actors/v2/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" - "github.com/filecoin-project/specs-actors/v2/actors/builtin" ) // RunClientTest exercises some of the Client CLI commands diff --git a/itests/kit/control.go b/itests/kit/control.go index b8c8611d3a5..de0cab48cb1 100644 --- a/itests/kit/control.go +++ b/itests/kit/control.go @@ -8,9 +8,10 @@ import ( addr "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" ) func (tm *TestMiner) SetControlAddresses(addrs ...addr.Address) { diff --git a/itests/kit/deals.go b/itests/kit/deals.go index bf59297ae22..eb6b58667dc 100644 --- a/itests/kit/deals.go +++ b/itests/kit/deals.go @@ -26,6 +26,7 @@ import ( "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/ensemble.go b/itests/kit/ensemble.go index a75eb25ab55..daca6d2d4e5 100644 --- a/itests/kit/ensemble.go +++ b/itests/kit/ensemble.go @@ -29,6 +29,9 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statestore" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" @@ -62,8 +65,6 @@ import ( sectorstorage "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/mock" "github.com/filecoin-project/lotus/storage/sealer/storiface" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power3 "github.com/filecoin-project/specs-actors/v3/actors/builtin/power" ) func init() { diff --git a/itests/kit/ensemble_opts.go b/itests/kit/ensemble_opts.go index 0d14a9b7950..d264da2bb25 100644 --- a/itests/kit/ensemble_opts.go +++ b/itests/kit/ensemble_opts.go @@ -4,6 +4,7 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/ensemble_opts_nv.go b/itests/kit/ensemble_opts_nv.go index 62b4725edc8..d5bb1930ef0 100644 --- a/itests/kit/ensemble_opts_nv.go +++ b/itests/kit/ensemble_opts_nv.go @@ -3,6 +3,7 @@ package kit import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" ) diff --git a/itests/kit/evm.go b/itests/kit/evm.go index 734404f695e..99844ca3097 100644 --- a/itests/kit/evm.go +++ b/itests/kit/evm.go @@ -26,6 +26,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/kit/funds.go b/itests/kit/funds.go index e554bbf27bb..1918d9125ff 100644 --- a/itests/kit/funds.go +++ b/itests/kit/funds.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/kit/init.go b/itests/kit/init.go index 109234769f3..9397c53a218 100644 --- a/itests/kit/init.go +++ b/itests/kit/init.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" ) diff --git a/itests/kit/node_full.go b/itests/kit/node_full.go index 21348f17131..3e80ed68869 100644 --- a/itests/kit/node_full.go +++ b/itests/kit/node_full.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/kit/node_miner.go b/itests/kit/node_miner.go index 94802312547..4b81c9df0bd 100644 --- a/itests/kit/node_miner.go +++ b/itests/kit/node_miner.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/node_opts.go b/itests/kit/node_opts.go index 40c32aa8519..6469c0a3076 100644 --- a/itests/kit/node_opts.go +++ b/itests/kit/node_opts.go @@ -3,6 +3,7 @@ package kit import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" diff --git a/itests/kit/rpc.go b/itests/kit/rpc.go index cd1cdd0fd35..6b63eb1ebf0 100644 --- a/itests/kit/rpc.go +++ b/itests/kit/rpc.go @@ -14,6 +14,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/cmd/lotus-worker/sealworker" "github.com/filecoin-project/lotus/node" diff --git a/itests/kit/state.go b/itests/kit/state.go index 195ca9f5a00..e66576be393 100644 --- a/itests/kit/state.go +++ b/itests/kit/state.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-address" actorstypes "github.com/filecoin-project/go-state-types/actors" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/itests/lite_migration_test.go b/itests/lite_migration_test.go index 6c78bc2766c..0f846e6fa10 100644 --- a/itests/lite_migration_test.go +++ b/itests/lite_migration_test.go @@ -16,6 +16,8 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" gstStore "github.com/filecoin-project/go-state-types/store" + "github.com/filecoin-project/specs-actors/v8/actors/util/adt" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin/system" "github.com/filecoin-project/lotus/chain/consensus/filcns" @@ -23,7 +25,6 @@ import ( "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" - "github.com/filecoin-project/specs-actors/v8/actors/util/adt" ) func TestLiteMigration(t *testing.T) { diff --git a/itests/lookup_robust_address_test.go b/itests/lookup_robust_address_test.go index 2ad60602b16..536d8feef36 100644 --- a/itests/lookup_robust_address_test.go +++ b/itests/lookup_robust_address_test.go @@ -8,6 +8,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" ) diff --git a/itests/mempool_test.go b/itests/mempool_test.go index d95319e5b06..f07b46a737c 100644 --- a/itests/mempool_test.go +++ b/itests/mempool_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/migration_test.go b/itests/migration_test.go index c6dbf2d6a4c..68991a579a9 100644 --- a/itests/migration_test.go +++ b/itests/migration_test.go @@ -25,6 +25,7 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" gstStore "github.com/filecoin-project/go-state-types/store" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/mpool_msg_uuid_test.go b/itests/mpool_msg_uuid_test.go index 34d34bc20f9..3eb30a6f6eb 100644 --- a/itests/mpool_msg_uuid_test.go +++ b/itests/mpool_msg_uuid_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/mpool_push_with_uuid_test.go b/itests/mpool_push_with_uuid_test.go index 2edefb40050..6b94dbad191 100644 --- a/itests/mpool_push_with_uuid_test.go +++ b/itests/mpool_push_with_uuid_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/msgindex_test.go b/itests/msgindex_test.go index afebfe56b12..807ab3c03f0 100644 --- a/itests/msgindex_test.go +++ b/itests/msgindex_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" diff --git a/itests/multisig/suite.go b/itests/multisig/suite.go index 1e77604bb66..9a81d0bf99d 100644 --- a/itests/multisig/suite.go +++ b/itests/multisig/suite.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/cli" diff --git a/itests/multisig_test.go b/itests/multisig_test.go index 9ebc33f8fac..92d9afca7dd 100644 --- a/itests/multisig_test.go +++ b/itests/multisig_test.go @@ -16,6 +16,7 @@ import ( inittypes "github.com/filecoin-project/go-state-types/builtin/v8/init" multisigtypes "github.com/filecoin-project/go-state-types/builtin/v8/multisig" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/itests/nonce_test.go b/itests/nonce_test.go index 0ebff1e44c6..cac2413f73c 100644 --- a/itests/nonce_test.go +++ b/itests/nonce_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/path_detach_redeclare_test.go b/itests/path_detach_redeclare_test.go index 101b7617942..10774d01218 100644 --- a/itests/path_detach_redeclare_test.go +++ b/itests/path_detach_redeclare_test.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/itests/paych_api_test.go b/itests/paych_api_test.go index daa8fd54310..f8ecb5377da 100644 --- a/itests/paych_api_test.go +++ b/itests/paych_api_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/paych_cli_test.go b/itests/paych_cli_test.go index fa68e7c96ac..f86f5d8deca 100644 --- a/itests/paych_cli_test.go +++ b/itests/paych_cli_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" diff --git a/itests/pending_deal_allocation_test.go b/itests/pending_deal_allocation_test.go index 7c600296e9f..c1e0531cfeb 100644 --- a/itests/pending_deal_allocation_test.go +++ b/itests/pending_deal_allocation_test.go @@ -18,6 +18,7 @@ import ( verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/raft_messagesigner_test.go b/itests/raft_messagesigner_test.go index b822b785646..220da96996b 100644 --- a/itests/raft_messagesigner_test.go +++ b/itests/raft_messagesigner_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" "github.com/filecoin-project/lotus/chain/types" diff --git a/itests/remove_verifreg_datacap_test.go b/itests/remove_verifreg_datacap_test.go index 82664a0b26b..ac88574d2b6 100644 --- a/itests/remove_verifreg_datacap_test.go +++ b/itests/remove_verifreg_datacap_test.go @@ -16,6 +16,7 @@ import ( migration "github.com/filecoin-project/go-state-types/builtin/v9/migration/test" verifregst "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/sector_import_full_test.go b/itests/sector_import_full_test.go index a2143d316e7..c9bd96afde1 100644 --- a/itests/sector_import_full_test.go +++ b/itests/sector_import_full_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/itests/sector_import_simple_test.go b/itests/sector_import_simple_test.go index 173d436d6c4..fb1a77a1410 100644 --- a/itests/sector_import_simple_test.go +++ b/itests/sector_import_simple_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/itests/sector_miner_collateral_test.go b/itests/sector_miner_collateral_test.go index 1b86840083e..579b4e535a1 100644 --- a/itests/sector_miner_collateral_test.go +++ b/itests/sector_miner_collateral_test.go @@ -10,6 +10,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/sector_numassign_test.go b/itests/sector_numassign_test.go index 436d070a9ed..af667aa9213 100644 --- a/itests/sector_numassign_test.go +++ b/itests/sector_numassign_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-bitfield" rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/lib/strle" ) diff --git a/itests/sector_pledge_test.go b/itests/sector_pledge_test.go index 6156c858cbc..1e045c79d7d 100644 --- a/itests/sector_pledge_test.go +++ b/itests/sector_pledge_test.go @@ -12,6 +12,8 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" @@ -19,7 +21,6 @@ import ( "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/impl" sealing "github.com/filecoin-project/lotus/storage/pipeline" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) func TestPledgeSectors(t *testing.T) { diff --git a/itests/sector_terminate_test.go b/itests/sector_terminate_test.go index 226e60f1ec7..34b325f2ad1 100644 --- a/itests/sector_terminate_test.go +++ b/itests/sector_terminate_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" sealing "github.com/filecoin-project/lotus/storage/pipeline" diff --git a/itests/sector_unseal_test.go b/itests/sector_unseal_test.go index f049e64f166..5d05cb0e33c 100644 --- a/itests/sector_unseal_test.go +++ b/itests/sector_unseal_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/itests/self_sent_txn_test.go b/itests/self_sent_txn_test.go index 81aafbacd51..00ca0e5aeeb 100644 --- a/itests/self_sent_txn_test.go +++ b/itests/self_sent_txn_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" diff --git a/itests/splitstore_test.go b/itests/splitstore_test.go index aba52b59de7..ea59faf2a9f 100644 --- a/itests/splitstore_test.go +++ b/itests/splitstore_test.go @@ -19,6 +19,9 @@ import ( "github.com/filecoin-project/go-state-types/builtin" miner8 "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/exitcode" + miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" + power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" + "github.com/filecoin-project/lotus/api" lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore/splitstore" @@ -27,8 +30,6 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" - miner2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/miner" - power6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/power" ) // Startup a node with hotstore and discard coldstore. Compact once and return diff --git a/itests/verifreg_test.go b/itests/verifreg_test.go index 2fe7144cb9d..ffe50c72b19 100644 --- a/itests/verifreg_test.go +++ b/itests/verifreg_test.go @@ -19,6 +19,7 @@ import ( datacap2 "github.com/filecoin-project/go-state-types/builtin/v9/datacap" verifregst "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" + lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/datacap" diff --git a/itests/wdpost_config_test.go b/itests/wdpost_config_test.go index 4261c41702e..984650ae6a6 100644 --- a/itests/wdpost_config_test.go +++ b/itests/wdpost_config_test.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/wdpost_dispute_test.go b/itests/wdpost_dispute_test.go index c090aa0fc45..c4512874aaa 100644 --- a/itests/wdpost_dispute_test.go +++ b/itests/wdpost_dispute_test.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/wdpost_test.go b/itests/wdpost_test.go index f4676ca9ee8..2a6fc866ea8 100644 --- a/itests/wdpost_test.go +++ b/itests/wdpost_test.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/itests/wdpost_worker_config_test.go b/itests/wdpost_worker_config_test.go index 35528528f1c..d1672c20f08 100644 --- a/itests/wdpost_worker_config_test.go +++ b/itests/wdpost_worker_config_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/itests/kit" "github.com/filecoin-project/lotus/node" diff --git a/itests/worker_test.go b/itests/worker_test.go index 51d21f52e0a..c4f885fb08f 100644 --- a/itests/worker_test.go +++ b/itests/worker_test.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" miner11 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/lib/consensus/raft/consensus.go b/lib/consensus/raft/consensus.go index 4dcdf6b1bae..d74f200fab1 100644 --- a/lib/consensus/raft/consensus.go +++ b/lib/consensus/raft/consensus.go @@ -14,6 +14,7 @@ import ( "golang.org/x/exp/slices" addr "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" diff --git a/lib/rpcenc/reader.go b/lib/rpcenc/reader.go index 2776b6c7b99..2dd64473e7e 100644 --- a/lib/rpcenc/reader.go +++ b/lib/rpcenc/reader.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/lib/httpreader" "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" ) diff --git a/lib/rpcenc/reader_test.go b/lib/rpcenc/reader_test.go index 48e1a0b37ae..3a554a0ca1e 100644 --- a/lib/rpcenc/reader_test.go +++ b/lib/rpcenc/reader_test.go @@ -16,6 +16,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" ) diff --git a/lib/sigs/bls/bls_test.go b/lib/sigs/bls/bls_test.go index 204b972a8e9..7a290b8b79f 100644 --- a/lib/sigs/bls/bls_test.go +++ b/lib/sigs/bls/bls_test.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/lib/sigs" diff --git a/lib/sigs/bls/init.go b/lib/sigs/bls/init.go index d784053f5e4..f64912fa8ee 100644 --- a/lib/sigs/bls/init.go +++ b/lib/sigs/bls/init.go @@ -7,6 +7,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/delegated/init.go b/lib/sigs/delegated/init.go index f883cdd5ad1..81886ceaad1 100644 --- a/lib/sigs/delegated/init.go +++ b/lib/sigs/delegated/init.go @@ -9,6 +9,7 @@ import ( gocrypto "github.com/filecoin-project/go-crypto" "github.com/filecoin-project/go-state-types/builtin" crypto1 "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/secp/init.go b/lib/sigs/secp/init.go index 63a8f716b99..49d5cef172a 100644 --- a/lib/sigs/secp/init.go +++ b/lib/sigs/secp/init.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-address" crypto "github.com/filecoin-project/go-crypto" crypto2 "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/lib/sigs" ) diff --git a/lib/sigs/sigs.go b/lib/sigs/sigs.go index 29c83cf6314..7792fc02364 100644 --- a/lib/sigs/sigs.go +++ b/lib/sigs/sigs.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/lib/unixfs/filestore.go b/lib/unixfs/filestore.go index f3d3817bad2..f50e933b68f 100644 --- a/lib/unixfs/filestore.go +++ b/lib/unixfs/filestore.go @@ -21,6 +21,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/stores" + "github.com/filecoin-project/lotus/build" ) diff --git a/markets/dagstore/mount_test.go b/markets/dagstore/mount_test.go index 7929d1a89be..d415f8d8856 100644 --- a/markets/dagstore/mount_test.go +++ b/markets/dagstore/mount_test.go @@ -13,6 +13,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/dagstore/mount" + mock_dagstore "github.com/filecoin-project/lotus/markets/dagstore/mocks" ) diff --git a/markets/dagstore/wrapper.go b/markets/dagstore/wrapper.go index 037edd1f8ca..a929ad1fc93 100644 --- a/markets/dagstore/wrapper.go +++ b/markets/dagstore/wrapper.go @@ -28,6 +28,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket/impl/providerstates" "github.com/filecoin-project/go-fil-markets/stores" "github.com/filecoin-project/go-statemachine/fsm" + "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dagstore/wrapper_migration_test.go b/markets/dagstore/wrapper_migration_test.go index 854f7087201..db2c9768b1a 100644 --- a/markets/dagstore/wrapper_migration_test.go +++ b/markets/dagstore/wrapper_migration_test.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" + "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dagstore/wrapper_test.go b/markets/dagstore/wrapper_test.go index 772d89fff32..f3b5e1b52c0 100644 --- a/markets/dagstore/wrapper_test.go +++ b/markets/dagstore/wrapper_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/dagstore" "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/dagstore/shard" + "github.com/filecoin-project/lotus/node/config" ) diff --git a/markets/dealfilter/cli.go b/markets/dealfilter/cli.go index 17f0034a700..af832bfa08a 100644 --- a/markets/dealfilter/cli.go +++ b/markets/dealfilter/cli.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/markets/journal.go b/markets/journal.go index f74d89e7bb4..9c9c5be9c42 100644 --- a/markets/journal.go +++ b/markets/journal.go @@ -3,6 +3,7 @@ package markets import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" + "github.com/filecoin-project/lotus/journal" ) diff --git a/markets/pricing/cli.go b/markets/pricing/cli.go index a58396186a1..48f56628fae 100644 --- a/markets/pricing/cli.go +++ b/markets/pricing/cli.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/retrievalmarket" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/markets/retrievaladapter/client.go b/markets/retrievaladapter/client.go index 451b75cad16..34bc2489624 100644 --- a/markets/retrievaladapter/client.go +++ b/markets/retrievaladapter/client.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-fil-markets/shared" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/markets/retrievaladapter/client_blockstore.go b/markets/retrievaladapter/client_blockstore.go index 5bc743de8cc..30fc5c73a5b 100644 --- a/markets/retrievaladapter/client_blockstore.go +++ b/markets/retrievaladapter/client_blockstore.go @@ -11,6 +11,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-fil-markets/retrievalmarket" + "github.com/filecoin-project/lotus/api" lbstore "github.com/filecoin-project/lotus/blockstore" ) diff --git a/markets/retrievaladapter/provider.go b/markets/retrievaladapter/provider.go index 584089356b1..453474d4e6e 100644 --- a/markets/retrievaladapter/provider.go +++ b/markets/retrievaladapter/provider.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-fil-markets/shared" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/markets/retrievaladapter/provider_test.go b/markets/retrievaladapter/provider_test.go index 249af09459d..b7b5039d674 100644 --- a/markets/retrievaladapter/provider_test.go +++ b/markets/retrievaladapter/provider_test.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-fil-markets/retrievalmarket" testnet "github.com/filecoin-project/go-fil-markets/shared_testutil" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/mocks" "github.com/filecoin-project/lotus/chain/actors/builtin/market" diff --git a/markets/sectoraccessor/sectoraccessor.go b/markets/sectoraccessor/sectoraccessor.go index 9f94bf85938..9b709d3b5ff 100644 --- a/markets/sectoraccessor/sectoraccessor.go +++ b/markets/sectoraccessor/sectoraccessor.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/types" diff --git a/markets/storageadapter/api.go b/markets/storageadapter/api.go index 75ad7e1fa11..b93ffdfbb16 100644 --- a/markets/storageadapter/api.go +++ b/markets/storageadapter/api.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/markets/storageadapter/client.go b/markets/storageadapter/client.go index f84ee8ef96f..eaff4e166a3 100644 --- a/markets/storageadapter/client.go +++ b/markets/storageadapter/client.go @@ -19,6 +19,8 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" marketactor "github.com/filecoin-project/lotus/chain/actors/builtin/market" @@ -30,7 +32,6 @@ import ( "github.com/filecoin-project/lotus/markets/utils" "github.com/filecoin-project/lotus/node/impl/full" "github.com/filecoin-project/lotus/node/modules/helpers" - builtin6 "github.com/filecoin-project/specs-actors/v6/actors/builtin" ) type ClientNodeAdapter struct { diff --git a/markets/storageadapter/client_blockstore.go b/markets/storageadapter/client_blockstore.go index 3ab8cecbb63..dc7e3f82a62 100644 --- a/markets/storageadapter/client_blockstore.go +++ b/markets/storageadapter/client_blockstore.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-fil-markets/stores" + "github.com/filecoin-project/lotus/node/repo/imports" ) diff --git a/markets/storageadapter/dealpublisher.go b/markets/storageadapter/dealpublisher.go index 188964f82bb..6a274e593f4 100644 --- a/markets/storageadapter/dealpublisher.go +++ b/markets/storageadapter/dealpublisher.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/markets/storageadapter/dealpublisher_test.go b/markets/storageadapter/dealpublisher_test.go index 7a6d2a7d638..35169bf41b9 100644 --- a/markets/storageadapter/dealpublisher_test.go +++ b/markets/storageadapter/dealpublisher_test.go @@ -17,11 +17,12 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) func TestDealPublisher(t *testing.T) { diff --git a/markets/storageadapter/dealstatematcher.go b/markets/storageadapter/dealstatematcher.go index b8b47ef8e5a..8d5598eae01 100644 --- a/markets/storageadapter/dealstatematcher.go +++ b/markets/storageadapter/dealstatematcher.go @@ -5,6 +5,7 @@ import ( "sync" "github.com/filecoin-project/go-state-types/abi" + actorsmarket "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/state" diff --git a/markets/storageadapter/dealstatematcher_test.go b/markets/storageadapter/dealstatematcher_test.go index 9409f3dc782..9a46e4af917 100644 --- a/markets/storageadapter/dealstatematcher_test.go +++ b/markets/storageadapter/dealstatematcher_test.go @@ -12,14 +12,15 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" + market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" + adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" + bstore "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/state" test "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" - builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" - market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" - adt2 "github.com/filecoin-project/specs-actors/v2/actors/util/adt" ) func TestDealStateMatcher(t *testing.T) { diff --git a/markets/storageadapter/ondealsectorcommitted.go b/markets/storageadapter/ondealsectorcommitted.go index 23ef371b51b..54ddb73b334 100644 --- a/markets/storageadapter/ondealsectorcommitted.go +++ b/markets/storageadapter/ondealsectorcommitted.go @@ -16,6 +16,7 @@ import ( miner2 "github.com/filecoin-project/go-state-types/builtin/v11/miner" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/builtin/v9/market" + "github.com/filecoin-project/lotus/build" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/events" diff --git a/markets/storageadapter/ondealsectorcommitted_test.go b/markets/storageadapter/ondealsectorcommitted_test.go index f7965c721fc..1d7519ff9f9 100644 --- a/markets/storageadapter/ondealsectorcommitted_test.go +++ b/markets/storageadapter/ondealsectorcommitted_test.go @@ -21,6 +21,8 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/cbor" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" @@ -28,7 +30,6 @@ import ( test "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" pipeline "github.com/filecoin-project/lotus/storage/pipeline" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) func TestOnDealSectorPreCommitted(t *testing.T) { diff --git a/markets/storageadapter/provider.go b/markets/storageadapter/provider.go index baa55f28d7e..bdfce6f55af 100644 --- a/markets/storageadapter/provider.go +++ b/markets/storageadapter/provider.go @@ -18,6 +18,7 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/markets/utils/converters.go b/markets/utils/converters.go index ca360e44c12..9562de695fc 100644 --- a/markets/utils/converters.go +++ b/markets/utils/converters.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" ) diff --git a/metrics/metrics.go b/metrics/metrics.go index 31e119811fc..0ea2c841df2 100644 --- a/metrics/metrics.go +++ b/metrics/metrics.go @@ -9,6 +9,7 @@ import ( "go.opencensus.io/tag" rpcmetrics "github.com/filecoin-project/go-jsonrpc/metrics" + "github.com/filecoin-project/lotus/blockstore" ) diff --git a/miner/miner.go b/miner/miner.go index 1fbcc41ab91..d11e9d4aa04 100644 --- a/miner/miner.go +++ b/miner/miner.go @@ -9,16 +9,19 @@ import ( "os" "sync" "time" + "github.com/hashicorp/golang-lru/arc/v2" "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "go.opencensus.io/trace" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/miner/testminer.go b/miner/testminer.go index 931816e2f7d..e23b26ae29f 100644 --- a/miner/testminer.go +++ b/miner/testminer.go @@ -2,10 +2,13 @@ package miner import ( "context" + "github.com/hashicorp/golang-lru/arc/v2" ds "github.com/ipfs/go-datastore" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/gen/slashfilter" diff --git a/miner/warmup.go b/miner/warmup.go index 05fb7479034..46236784a72 100644 --- a/miner/warmup.go +++ b/miner/warmup.go @@ -5,10 +5,13 @@ import ( "crypto/rand" "math" "time" + "golang.org/x/xerrors" + "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/builder.go b/node/builder.go index 866d6c3faf7..128a99f8714 100644 --- a/node/builder.go +++ b/node/builder.go @@ -4,6 +4,7 @@ import ( "context" "errors" "time" + logging "github.com/ipfs/go-log/v2" metricsi "github.com/ipfs/go-metrics-interface" dht "github.com/libp2p/go-libp2p-kad-dht" @@ -19,6 +20,7 @@ import ( "github.com/multiformats/go-multiaddr" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/builder_chain.go b/node/builder_chain.go index 960a65b3831..267659f0091 100644 --- a/node/builder_chain.go +++ b/node/builder_chain.go @@ -2,13 +2,16 @@ package node import ( "os" + gorpc "github.com/libp2p/go-libp2p-gorpc" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-fil-markets/discovery" discoveryimpl "github.com/filecoin-project/go-fil-markets/discovery/impl" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/builder_miner.go b/node/builder_miner.go index 47855ea1fd1..dd35c6becf9 100644 --- a/node/builder_miner.go +++ b/node/builder_miner.go @@ -3,14 +3,17 @@ package node import ( "errors" "time" + provider "github.com/ipni/index-provider" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-fil-markets/retrievalmarket" rmnet "github.com/filecoin-project/go-fil-markets/retrievalmarket/network" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-fil-markets/storagemarket/impl/storedask" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" diff --git a/node/bundle/bundle.go b/node/bundle/bundle.go index 1d1306e490d..716c9043bb6 100644 --- a/node/bundle/bundle.go +++ b/node/bundle/bundle.go @@ -6,10 +6,13 @@ import ( "fmt" "io" "os" + "github.com/ipfs/go-cid" "github.com/ipld/go-car" "golang.org/x/xerrors" + actorstypes "github.com/filecoin-project/go-state-types/actors" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/config/def.go b/node/config/def.go index 619adb6021e..746c963fd07 100644 --- a/node/config/def.go +++ b/node/config/def.go @@ -5,11 +5,14 @@ import ( "os" "strconv" "time" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/config/def_test.go b/node/config/def_test.go index 5bc4eef83c6..627b65a5631 100644 --- a/node/config/def_test.go +++ b/node/config/def_test.go @@ -7,6 +7,7 @@ import ( "reflect" "strings" "testing" + "github.com/BurntSushi/toml" "github.com/stretchr/testify/require" ) diff --git a/node/config/load.go b/node/config/load.go index 05f75158651..fd015d53328 100644 --- a/node/config/load.go +++ b/node/config/load.go @@ -9,6 +9,7 @@ import ( "regexp" "strings" "unicode" + "github.com/BurntSushi/toml" "github.com/kelseyhightower/envconfig" "golang.org/x/xerrors" diff --git a/node/config/load_test.go b/node/config/load_test.go index d7cfa863ca4..e17660c19f9 100644 --- a/node/config/load_test.go +++ b/node/config/load_test.go @@ -6,6 +6,7 @@ import ( "os" "testing" "time" + "github.com/stretchr/testify/assert" ) diff --git a/node/config/storage.go b/node/config/storage.go index d46c30f0c9d..ac5d57de8b4 100644 --- a/node/config/storage.go +++ b/node/config/storage.go @@ -7,7 +7,9 @@ import ( "io/fs" "os" "path" + "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/node/config/types.go b/node/config/types.go index a81651f7101..2152e079569 100644 --- a/node/config/types.go +++ b/node/config/types.go @@ -2,6 +2,7 @@ package config import ( "github.com/ipfs/go-cid" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/health.go b/node/health.go index 80c556c44fe..1be11921c0c 100644 --- a/node/health.go +++ b/node/health.go @@ -5,8 +5,10 @@ import ( "net/http" "sync/atomic" "time" + logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/network" + lapi "github.com/filecoin-project/lotus/api" ) diff --git a/node/hello/cbor_gen.go b/node/hello/cbor_gen.go index cd3d7bb171d..78e950f6f6e 100644 --- a/node/hello/cbor_gen.go +++ b/node/hello/cbor_gen.go @@ -7,9 +7,11 @@ import ( "io" "math" "sort" + cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" + abi "github.com/filecoin-project/go-state-types/abi" ) diff --git a/node/hello/hello.go b/node/hello/hello.go index 1f6f3cfd8d2..e05b8a48287 100644 --- a/node/hello/hello.go +++ b/node/hello/hello.go @@ -3,6 +3,7 @@ package hello import ( "context" "time" + "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/host" @@ -10,9 +11,11 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" "golang.org/x/xerrors" + cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/consensus" diff --git a/node/impl/backup.go b/node/impl/backup.go index 091cdb4cf38..7acc7e01877 100644 --- a/node/impl/backup.go +++ b/node/impl/backup.go @@ -5,8 +5,10 @@ import ( "os" "path/filepath" "strings" + "github.com/mitchellh/go-homedir" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/node/impl/client/car_helpers.go b/node/impl/client/car_helpers.go index 26d08dfcb3a..c638b4bef81 100644 --- a/node/impl/client/car_helpers.go +++ b/node/impl/client/car_helpers.go @@ -3,6 +3,7 @@ package client import ( "fmt" "io" + "github.com/ipfs/go-cid" cbor "github.com/ipfs/go-ipld-cbor" "github.com/ipld/go-car/util" diff --git a/node/impl/client/client.go b/node/impl/client/client.go index 8d3b37e446c..c7bb252a10a 100644 --- a/node/impl/client/client.go +++ b/node/impl/client/client.go @@ -12,6 +12,7 @@ import ( "strings" "sync" "time" + "github.com/ipfs/boxo/blockservice" bstore "github.com/ipfs/boxo/blockstore" offline "github.com/ipfs/boxo/exchange/offline" @@ -39,6 +40,7 @@ import ( "github.com/multiformats/go-multibase" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-commp-utils/writer" @@ -53,6 +55,7 @@ import ( "github.com/filecoin-project/go-state-types/big" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/node/impl/client/client_test.go b/node/impl/client/client_test.go index f19d6705308..67a35013166 100644 --- a/node/impl/client/client_test.go +++ b/node/impl/client/client_test.go @@ -9,6 +9,7 @@ import ( "path/filepath" "strings" "testing" + "github.com/ipfs/boxo/blockservice" blockstore "github.com/ipfs/boxo/blockstore" offline "github.com/ipfs/boxo/exchange/offline" @@ -21,6 +22,7 @@ import ( "github.com/ipld/go-car" carv2 "github.com/ipld/go-car/v2" "github.com/stretchr/testify/require" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/markets/storageadapter" "github.com/filecoin-project/lotus/node/repo/imports" diff --git a/node/impl/common/common.go b/node/impl/common/common.go index 83692787442..eff6b58b8f1 100644 --- a/node/impl/common/common.go +++ b/node/impl/common/common.go @@ -3,12 +3,15 @@ package common import ( "context" "time" + "github.com/gbrlsnchs/jwt/v3" "github.com/google/uuid" logging "github.com/ipfs/go-log/v2" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/node/impl/full.go b/node/impl/full.go index 6f1163e00de..affcc960e09 100644 --- a/node/impl/full.go +++ b/node/impl/full.go @@ -3,8 +3,10 @@ package impl import ( "context" "time" + logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p/core/peer" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/impl/client" diff --git a/node/impl/full/chain.go b/node/impl/full/chain.go index 5bbdd95744c..1d6b8e566fa 100644 --- a/node/impl/full/chain.go +++ b/node/impl/full/chain.go @@ -14,6 +14,7 @@ import ( "strings" "sync" "time" + "github.com/ipfs/boxo/blockservice" offline "github.com/ipfs/boxo/exchange/offline" "github.com/ipfs/boxo/ipld/merkledag" @@ -26,10 +27,12 @@ import ( cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" amt4 "github.com/filecoin-project/go-amt-ipld/v4" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/specs-actors/actors/util/adt" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/impl/full/dummy.go b/node/impl/full/dummy.go index e07088b43d0..743eadf34dd 100644 --- a/node/impl/full/dummy.go +++ b/node/impl/full/dummy.go @@ -3,10 +3,13 @@ package full import ( "context" "errors" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/node/impl/full/eth.go b/node/impl/full/eth.go index 0f3df3b1841..6b8b0e0aaef 100644 --- a/node/impl/full/eth.go +++ b/node/impl/full/eth.go @@ -10,10 +10,12 @@ import ( "strconv" "strings" "time" + "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" @@ -22,6 +24,7 @@ import ( builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/evm" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/eth_event.go b/node/impl/full/eth_event.go index 3cfc6312e0c..69021e08aed 100644 --- a/node/impl/full/eth_event.go +++ b/node/impl/full/eth_event.go @@ -4,11 +4,14 @@ import ( "context" "encoding/json" "sync" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/zyedidia/generic/queue" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc" + "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/impl/full/eth_test.go b/node/impl/full/eth_test.go index 3a8db08aa1b..c364a4873c4 100644 --- a/node/impl/full/eth_test.go +++ b/node/impl/full/eth_test.go @@ -3,9 +3,12 @@ package full import ( "encoding/hex" "testing" + "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" ) diff --git a/node/impl/full/eth_trace.go b/node/impl/full/eth_trace.go index 906287d861a..fd5c25566cb 100644 --- a/node/impl/full/eth_trace.go +++ b/node/impl/full/eth_trace.go @@ -3,11 +3,14 @@ package full import ( "bytes" "context" + "github.com/multiformats/go-multicodec" cbg "github.com/whyrusleeping/cbor-gen" "golang.org/x/xerrors" + "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v10/evm" + builtinactors "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/types/ethtypes" diff --git a/node/impl/full/eth_utils.go b/node/impl/full/eth_utils.go index 88d6f5a0fbe..2799638ddfb 100644 --- a/node/impl/full/eth_utils.go +++ b/node/impl/full/eth_utils.go @@ -6,8 +6,10 @@ import ( "encoding/binary" "errors" "fmt" + "github.com/ipfs/go-cid" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" @@ -15,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin/v10/eam" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/gas.go b/node/impl/full/gas.go index 1bb0ac50fa0..c5b22354a52 100644 --- a/node/impl/full/gas.go +++ b/node/impl/full/gas.go @@ -6,14 +6,17 @@ import ( "math/rand" "os" "sort" + lru "github.com/hashicorp/golang-lru/v2" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" lbuiltin "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/node/impl/full/gas_test.go b/node/impl/full/gas_test.go index b5218a6b664..8fc585bd544 100644 --- a/node/impl/full/gas_test.go +++ b/node/impl/full/gas_test.go @@ -3,8 +3,11 @@ package full import ( "testing" + "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/impl/full/mpool.go b/node/impl/full/mpool.go index a656c3a9ec2..addcc41be43 100644 --- a/node/impl/full/mpool.go +++ b/node/impl/full/mpool.go @@ -3,12 +3,15 @@ package full import ( "context" "encoding/json" + "github.com/google/uuid" "github.com/ipfs/go-cid" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/messagesigner" diff --git a/node/impl/full/multisig.go b/node/impl/full/multisig.go index e41147186ae..886d66d5e4a 100644 --- a/node/impl/full/multisig.go +++ b/node/impl/full/multisig.go @@ -2,13 +2,16 @@ package full import ( "context" + "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" multisig2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/multisig" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" diff --git a/node/impl/full/raft.go b/node/impl/full/raft.go index 6f71414f8fe..8d665ddd502 100644 --- a/node/impl/full/raft.go +++ b/node/impl/full/raft.go @@ -2,9 +2,11 @@ package full import ( "context" + "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" ) diff --git a/node/impl/full/state.go b/node/impl/full/state.go index e632d755e06..0e92c8e5b6f 100644 --- a/node/impl/full/state.go +++ b/node/impl/full/state.go @@ -8,11 +8,13 @@ import ( "fmt" "math" "strconv" + "github.com/ipfs/go-cid" "github.com/libp2p/go-libp2p/core/peer" cbg "github.com/whyrusleeping/cbor-gen" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-bitfield" "github.com/filecoin-project/go-state-types/abi" @@ -25,6 +27,7 @@ import ( "github.com/filecoin-project/go-state-types/network" market2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/market" market5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/market" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/node/impl/full/sync.go b/node/impl/full/sync.go index 4f25ad03a7a..4bf44363ce0 100644 --- a/node/impl/full/sync.go +++ b/node/impl/full/sync.go @@ -4,10 +4,12 @@ import ( "context" "os" "sync/atomic" + "github.com/ipfs/go-cid" pubsub "github.com/libp2p/go-libp2p-pubsub" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" diff --git a/node/impl/full/txhashmanager.go b/node/impl/full/txhashmanager.go index 4c150bea43c..6757cc6dd92 100644 --- a/node/impl/full/txhashmanager.go +++ b/node/impl/full/txhashmanager.go @@ -3,8 +3,10 @@ package full import ( "context" "time" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/ethhashlookup" diff --git a/node/impl/full/wallet.go b/node/impl/full/wallet.go index d217e58e422..fdf00e0864f 100644 --- a/node/impl/full/wallet.go +++ b/node/impl/full/wallet.go @@ -2,11 +2,14 @@ package full import ( "context" + "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/messagesigner" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/impl/market/market.go b/node/impl/market/market.go index 0a62cafeece..26ea9c14ca7 100644 --- a/node/impl/market/market.go +++ b/node/impl/market/market.go @@ -2,9 +2,12 @@ package market import ( "context" + "github.com/ipfs/go-cid" "go.uber.org/fx" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/actors" marketactor "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/market" diff --git a/node/impl/net/conngater.go b/node/impl/net/conngater.go index 13b82b78c7a..f95b60b48ac 100644 --- a/node/impl/net/conngater.go +++ b/node/impl/net/conngater.go @@ -3,9 +3,11 @@ package net import ( "context" "net" + logging "github.com/ipfs/go-log/v2" manet "github.com/multiformats/go-multiaddr/net" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" ) diff --git a/node/impl/net/net.go b/node/impl/net/net.go index db9caafb3fc..5341092ce28 100644 --- a/node/impl/net/net.go +++ b/node/impl/net/net.go @@ -5,6 +5,7 @@ import ( "sort" "strings" "time" + "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/metrics" "github.com/libp2p/go-libp2p/core/network" @@ -18,6 +19,7 @@ import ( manet "github.com/multiformats/go-multiaddr/net" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/node/modules/lp2p" diff --git a/node/impl/net/protect.go b/node/impl/net/protect.go index 4f5be997342..c6c864a687c 100644 --- a/node/impl/net/protect.go +++ b/node/impl/net/protect.go @@ -2,6 +2,7 @@ package net import ( "context" + "github.com/libp2p/go-libp2p/core/peer" ) diff --git a/node/impl/net/rcmgr.go b/node/impl/net/rcmgr.go index 2157dafd5c6..c606aabc65b 100644 --- a/node/impl/net/rcmgr.go +++ b/node/impl/net/rcmgr.go @@ -3,11 +3,13 @@ package net import ( "context" "strings" + "github.com/libp2p/go-libp2p/core/network" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" rcmgr "github.com/libp2p/go-libp2p/p2p/host/resource-manager" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" ) diff --git a/node/impl/paych/paych.go b/node/impl/paych/paych.go index ede872fe850..b7d40c1e959 100644 --- a/node/impl/paych/paych.go +++ b/node/impl/paych/paych.go @@ -2,11 +2,14 @@ package paych import ( "context" + "github.com/ipfs/go-cid" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/paychmgr" diff --git a/node/impl/remoteworker.go b/node/impl/remoteworker.go index 4ba7c9b66ab..b3e68869d94 100644 --- a/node/impl/remoteworker.go +++ b/node/impl/remoteworker.go @@ -3,10 +3,13 @@ package impl import ( "context" "net/http" + "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" "github.com/filecoin-project/lotus/storage/sealer" diff --git a/node/impl/storminer.go b/node/impl/storminer.go index a9ff73d57e1..a087e084f50 100644 --- a/node/impl/storminer.go +++ b/node/impl/storminer.go @@ -10,6 +10,7 @@ import ( "sort" "strconv" "time" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-graphsync" @@ -19,6 +20,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/dagstore" "github.com/filecoin-project/dagstore/shard" "github.com/filecoin-project/go-address" @@ -34,6 +36,7 @@ import ( "github.com/filecoin-project/go-state-types/big" builtintypes "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" apitypes "github.com/filecoin-project/lotus/api/types" "github.com/filecoin-project/lotus/build" diff --git a/node/modules/actorevent.go b/node/modules/actorevent.go index 11ae4609e97..4ce04cefd07 100644 --- a/node/modules/actorevent.go +++ b/node/modules/actorevent.go @@ -4,11 +4,14 @@ import ( "context" "path/filepath" "time" + "github.com/multiformats/go-varint" "go.uber.org/fx" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" builtintypes "github.com/filecoin-project/go-state-types/builtin" + "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/events/filter" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/node/modules/alerts.go b/node/modules/alerts.go index 7c2324a0677..9976c6d0e42 100644 --- a/node/modules/alerts.go +++ b/node/modules/alerts.go @@ -5,6 +5,7 @@ import ( "os" "strconv" "syscall" + "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/ulimit" ) diff --git a/node/modules/blockstore.go b/node/modules/blockstore.go index 50369d23d0a..9c54d51e60f 100644 --- a/node/modules/blockstore.go +++ b/node/modules/blockstore.go @@ -5,9 +5,11 @@ import ( "io" "os" "path/filepath" + bstore "github.com/ipfs/boxo/blockstore" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/blockstore/splitstore" diff --git a/node/modules/chain.go b/node/modules/chain.go index e3f9e1bc31a..d08b7dbec5e 100644 --- a/node/modules/chain.go +++ b/node/modules/chain.go @@ -3,6 +3,7 @@ package modules import ( "context" "time" + "github.com/ipfs/boxo/bitswap" "github.com/ipfs/boxo/bitswap/network" "github.com/ipfs/boxo/blockservice" @@ -10,6 +11,7 @@ import ( "github.com/libp2p/go-libp2p/core/routing" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/blockstore/splitstore" "github.com/filecoin-project/lotus/build" diff --git a/node/modules/client.go b/node/modules/client.go index ef86f43c9c7..9d8eef4217b 100644 --- a/node/modules/client.go +++ b/node/modules/client.go @@ -6,11 +6,13 @@ import ( "os" "path/filepath" "time" + "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-data-transfer/v2/channelmonitor" dtimpl "github.com/filecoin-project/go-data-transfer/v2/impl" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" @@ -24,6 +26,7 @@ import ( storageimpl "github.com/filecoin-project/go-fil-markets/storagemarket/impl" smnet "github.com/filecoin-project/go-fil-markets/storagemarket/network" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/market" "github.com/filecoin-project/lotus/journal" diff --git a/node/modules/core.go b/node/modules/core.go index 1c0c359903d..a0d52c291bc 100644 --- a/node/modules/core.go +++ b/node/modules/core.go @@ -8,6 +8,7 @@ import ( "os" "path/filepath" "time" + "github.com/gbrlsnchs/jwt/v3" logging "github.com/ipfs/go-log/v2" record "github.com/libp2p/go-libp2p-record" @@ -16,8 +17,10 @@ import ( "github.com/raulk/go-watchdog" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc/auth" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/modules/dtypes/api.go b/node/modules/dtypes/api.go index bd0e6bcab2c..6de511b9d6f 100644 --- a/node/modules/dtypes/api.go +++ b/node/modules/dtypes/api.go @@ -2,6 +2,7 @@ package dtypes import ( "time" + "github.com/gbrlsnchs/jwt/v3" "github.com/multiformats/go-multiaddr" ) diff --git a/node/modules/dtypes/miner.go b/node/modules/dtypes/miner.go index 330dd1d079f..24bcc714c17 100644 --- a/node/modules/dtypes/miner.go +++ b/node/modules/dtypes/miner.go @@ -3,11 +3,14 @@ package dtypes import ( "context" "time" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/pipeline/sealiface" ) diff --git a/node/modules/dtypes/mpool.go b/node/modules/dtypes/mpool.go index 4c32b69f602..df96b8d0ebd 100644 --- a/node/modules/dtypes/mpool.go +++ b/node/modules/dtypes/mpool.go @@ -3,6 +3,7 @@ package dtypes import ( "context" "sync" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" ) diff --git a/node/modules/dtypes/scorekeeper.go b/node/modules/dtypes/scorekeeper.go index 903768fdf46..343b787b733 100644 --- a/node/modules/dtypes/scorekeeper.go +++ b/node/modules/dtypes/scorekeeper.go @@ -2,6 +2,7 @@ package dtypes import ( "sync" + pubsub "github.com/libp2p/go-libp2p-pubsub" "github.com/libp2p/go-libp2p/core/peer" ) diff --git a/node/modules/dtypes/storage.go b/node/modules/dtypes/storage.go index 49f6f35cade..7f0466f1f3d 100644 --- a/node/modules/dtypes/storage.go +++ b/node/modules/dtypes/storage.go @@ -5,11 +5,13 @@ import ( exchange "github.com/ipfs/boxo/exchange" "github.com/ipfs/go-datastore" "github.com/ipfs/go-graphsync" + datatransfer "github.com/filecoin-project/go-data-transfer/v2" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" "github.com/filecoin-project/go-fil-markets/piecestore" "github.com/filecoin-project/go-fil-markets/storagemarket/impl/requestvalidation" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/node/repo/imports" ) diff --git a/node/modules/ethmodule.go b/node/modules/ethmodule.go index 31772baa98b..0255b61983f 100644 --- a/node/modules/ethmodule.go +++ b/node/modules/ethmodule.go @@ -5,8 +5,11 @@ import ( "os" "path/filepath" "time" + "go.uber.org/fx" + "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/chain/ethhashlookup" "github.com/filecoin-project/lotus/chain/events" "github.com/filecoin-project/lotus/chain/messagepool" diff --git a/node/modules/faultreport.go b/node/modules/faultreport.go index c4f5c04582d..c42602d7e61 100644 --- a/node/modules/faultreport.go +++ b/node/modules/faultreport.go @@ -2,6 +2,7 @@ package modules import ( "go.uber.org/fx" + "github.com/filecoin-project/lotus/chain/gen/slashfilter/slashsvc" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/node/modules/genesis.go b/node/modules/genesis.go index 8a5fb48a0bb..7f0a46afc50 100644 --- a/node/modules/genesis.go +++ b/node/modules/genesis.go @@ -3,10 +3,12 @@ package modules import ( "bytes" "os" + "github.com/ipfs/go-datastore" "github.com/ipld/go-car" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/graphsync.go b/node/modules/graphsync.go index e49508a8ead..ca69cd2d202 100644 --- a/node/modules/graphsync.go +++ b/node/modules/graphsync.go @@ -3,6 +3,7 @@ package modules import ( "context" "time" + "github.com/ipfs/go-graphsync" graphsyncimpl "github.com/ipfs/go-graphsync/impl" gsnet "github.com/ipfs/go-graphsync/network" @@ -11,6 +12,7 @@ import ( "github.com/libp2p/go-libp2p/core/peer" "go.opencensus.io/stats" "go.uber.org/fx" + "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/helpers/helpers.go b/node/modules/helpers/helpers.go index a0d30a9bc4c..f63b7eedd3e 100644 --- a/node/modules/helpers/helpers.go +++ b/node/modules/helpers/helpers.go @@ -2,6 +2,7 @@ package helpers import ( "context" + "go.uber.org/fx" ) diff --git a/node/modules/ipfs.go b/node/modules/ipfs.go index e0b108d5d30..cb9deb6fde1 100644 --- a/node/modules/ipfs.go +++ b/node/modules/ipfs.go @@ -5,8 +5,10 @@ import ( "github.com/multiformats/go-multiaddr" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-fil-markets/retrievalmarket" "github.com/filecoin-project/go-fil-markets/storagemarket" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/markets/retrievaladapter" "github.com/filecoin-project/lotus/markets/storageadapter" diff --git a/node/modules/lp2p/addrs.go b/node/modules/lp2p/addrs.go index 73a8d79605e..52ee8c852f6 100644 --- a/node/modules/lp2p/addrs.go +++ b/node/modules/lp2p/addrs.go @@ -2,6 +2,7 @@ package lp2p import ( "fmt" + "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/host" p2pbhost "github.com/libp2p/go-libp2p/p2p/host/basic" diff --git a/node/modules/lp2p/conngater.go b/node/modules/lp2p/conngater.go index 3c85bc4ad31..29087a1bc28 100644 --- a/node/modules/lp2p/conngater.go +++ b/node/modules/lp2p/conngater.go @@ -3,6 +3,7 @@ package lp2p import ( "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/p2p/net/conngater" + "github.com/filecoin-project/lotus/node/modules/dtypes" ) diff --git a/node/modules/lp2p/discovery.go b/node/modules/lp2p/discovery.go index ac51c03829a..ca68a7b9cf5 100644 --- a/node/modules/lp2p/discovery.go +++ b/node/modules/lp2p/discovery.go @@ -3,9 +3,11 @@ package lp2p import ( "context" "time" + "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" "go.uber.org/fx" + "github.com/filecoin-project/lotus/node/modules/helpers" ) diff --git a/node/modules/lp2p/host.go b/node/modules/lp2p/host.go index 6313d3333eb..9c140b41ee6 100644 --- a/node/modules/lp2p/host.go +++ b/node/modules/lp2p/host.go @@ -3,6 +3,7 @@ package lp2p import ( "context" "fmt" + nilrouting "github.com/ipfs/boxo/routing/none" "github.com/libp2p/go-libp2p" dht "github.com/libp2p/go-libp2p-kad-dht" @@ -14,6 +15,7 @@ import ( routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" "go.uber.org/fx" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/node/modules/helpers" diff --git a/node/modules/lp2p/libp2p.go b/node/modules/lp2p/libp2p.go index bd8ddee591a..9d42b3996bc 100644 --- a/node/modules/lp2p/libp2p.go +++ b/node/modules/lp2p/libp2p.go @@ -3,6 +3,7 @@ package lp2p import ( "crypto/rand" "time" + logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/crypto" @@ -11,6 +12,7 @@ import ( "github.com/libp2p/go-libp2p/p2p/net/connmgr" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/node/modules/lp2p/pubsub.go b/node/modules/lp2p/pubsub.go index 1208e58f307..2b3efce6c44 100644 --- a/node/modules/lp2p/pubsub.go +++ b/node/modules/lp2p/pubsub.go @@ -5,6 +5,7 @@ import ( "encoding/json" "net" "time" + pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/host" @@ -14,6 +15,7 @@ import ( "go.opencensus.io/stats" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/config" diff --git a/node/modules/lp2p/rcmgr.go b/node/modules/lp2p/rcmgr.go index 0aa00301c05..f2b2849863e 100644 --- a/node/modules/lp2p/rcmgr.go +++ b/node/modules/lp2p/rcmgr.go @@ -8,6 +8,7 @@ import ( "os" "path/filepath" "sync" + logging "github.com/ipfs/go-log/v2" "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/core/network" @@ -18,6 +19,7 @@ import ( "go.opencensus.io/stats" "go.opencensus.io/tag" "go.uber.org/fx" + "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/node/repo" ) diff --git a/node/modules/lp2p/relay.go b/node/modules/lp2p/relay.go index 21edffed224..87a4983f6b6 100644 --- a/node/modules/lp2p/relay.go +++ b/node/modules/lp2p/relay.go @@ -2,6 +2,7 @@ package lp2p import ( "fmt" + "github.com/libp2p/go-libp2p" coredisc "github.com/libp2p/go-libp2p/core/discovery" "github.com/libp2p/go-libp2p/core/routing" diff --git a/node/modules/lp2p/routing.go b/node/modules/lp2p/routing.go index 5e22b3bb783..816cbe2eab6 100644 --- a/node/modules/lp2p/routing.go +++ b/node/modules/lp2p/routing.go @@ -3,6 +3,7 @@ package lp2p import ( "context" "sort" + dht "github.com/libp2p/go-libp2p-kad-dht" record "github.com/libp2p/go-libp2p-record" routinghelpers "github.com/libp2p/go-libp2p-routing-helpers" diff --git a/node/modules/lp2p/smux.go b/node/modules/lp2p/smux.go index e45da04ac52..cab7da68251 100644 --- a/node/modules/lp2p/smux.go +++ b/node/modules/lp2p/smux.go @@ -2,6 +2,7 @@ package lp2p import ( "os" + "github.com/libp2p/go-libp2p" "github.com/libp2p/go-libp2p/p2p/muxer/yamux" ) diff --git a/node/modules/mpoolnonceapi.go b/node/modules/mpoolnonceapi.go index 0bf17a7a6e0..393bee32f24 100644 --- a/node/modules/mpoolnonceapi.go +++ b/node/modules/mpoolnonceapi.go @@ -3,9 +3,12 @@ package modules import ( "context" "strings" + "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/impl/full" diff --git a/node/modules/msgindex.go b/node/modules/msgindex.go index 21e3ed9277a..72e9840ba33 100644 --- a/node/modules/msgindex.go +++ b/node/modules/msgindex.go @@ -2,7 +2,9 @@ package modules import ( "context" + "go.uber.org/fx" + "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/node/modules/helpers" diff --git a/node/modules/paych.go b/node/modules/paych.go index c28d993c097..4f93bbd6c55 100644 --- a/node/modules/paych.go +++ b/node/modules/paych.go @@ -2,9 +2,11 @@ package modules import ( "context" + "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "go.uber.org/fx" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/node/impl/full" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/rpc.go b/node/modules/rpc.go index a57939f72ae..d76949737e2 100644 --- a/node/modules/rpc.go +++ b/node/modules/rpc.go @@ -2,10 +2,12 @@ package modules import ( "context" + rpc "github.com/libp2p/go-libp2p-gorpc" "github.com/libp2p/go-libp2p/core/host" "github.com/libp2p/go-libp2p/core/peer" "github.com/libp2p/go-libp2p/core/protocol" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" consensus "github.com/filecoin-project/lotus/lib/consensus/raft" diff --git a/node/modules/services.go b/node/modules/services.go index f92cf3ffe2c..f3dd443d94d 100644 --- a/node/modules/services.go +++ b/node/modules/services.go @@ -5,6 +5,7 @@ import ( "os" "strconv" "time" + "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" pubsub "github.com/libp2p/go-libp2p-pubsub" @@ -15,8 +16,10 @@ import ( "github.com/libp2p/go-libp2p/p2p/host/eventbus" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-fil-markets/discovery" discoveryimpl "github.com/filecoin-project/go-fil-markets/discovery/impl" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" "github.com/filecoin-project/lotus/chain/beacon" diff --git a/node/modules/stmgr.go b/node/modules/stmgr.go index 8af5ea8bfae..f3eaee219c5 100644 --- a/node/modules/stmgr.go +++ b/node/modules/stmgr.go @@ -2,6 +2,7 @@ package modules import ( "go.uber.org/fx" + "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/index" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/node/modules/storage.go b/node/modules/storage.go index ca70dc90d05..cb30eb8c29d 100644 --- a/node/modules/storage.go +++ b/node/modules/storage.go @@ -3,8 +3,10 @@ package modules import ( "context" "path/filepath" + "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/backupds" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer.go b/node/modules/storageminer.go index 5cbf6901577..0680029bfc0 100644 --- a/node/modules/storageminer.go +++ b/node/modules/storageminer.go @@ -10,6 +10,7 @@ import ( "path/filepath" "strings" "time" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" @@ -22,6 +23,7 @@ import ( "go.uber.org/fx" "go.uber.org/multierr" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" dtimpl "github.com/filecoin-project/go-data-transfer/v2/impl" dtnet "github.com/filecoin-project/go-data-transfer/v2/network" @@ -41,6 +43,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/node/modules/storageminer_dagstore.go b/node/modules/storageminer_dagstore.go index 323393b6e27..620e690901c 100644 --- a/node/modules/storageminer_dagstore.go +++ b/node/modules/storageminer_dagstore.go @@ -6,10 +6,13 @@ import ( "os" "path/filepath" "strconv" + "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/dagstore" + mdagstore "github.com/filecoin-project/lotus/markets/dagstore" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_idxprov.go b/node/modules/storageminer_idxprov.go index b4e18e0f4d0..777c59386b5 100644 --- a/node/modules/storageminer_idxprov.go +++ b/node/modules/storageminer_idxprov.go @@ -2,6 +2,7 @@ package modules import ( "context" + "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" provider "github.com/ipni/index-provider" @@ -10,7 +11,9 @@ import ( "github.com/libp2p/go-libp2p/core/host" "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_idxprov_test.go b/node/modules/storageminer_idxprov_test.go index 48c522afc13..434577bab64 100644 --- a/node/modules/storageminer_idxprov_test.go +++ b/node/modules/storageminer_idxprov_test.go @@ -5,6 +5,7 @@ import ( "strings" "testing" "time" + "github.com/ipfs/go-datastore" provider "github.com/ipni/index-provider" "github.com/libp2p/go-libp2p" @@ -12,7 +13,9 @@ import ( "github.com/libp2p/go-libp2p/core/host" "github.com/stretchr/testify/require" "go.uber.org/fx" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/node/modules" "github.com/filecoin-project/lotus/node/modules/dtypes" diff --git a/node/modules/storageminer_svc.go b/node/modules/storageminer_svc.go index 3064861f900..dec0453095a 100644 --- a/node/modules/storageminer_svc.go +++ b/node/modules/storageminer_svc.go @@ -2,8 +2,10 @@ package modules import ( "context" + "go.uber.org/fx" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/client" cliutil "github.com/filecoin-project/lotus/cli/util" diff --git a/node/modules/testing/beacon.go b/node/modules/testing/beacon.go index 4263d50fddd..7876e1d0528 100644 --- a/node/modules/testing/beacon.go +++ b/node/modules/testing/beacon.go @@ -2,6 +2,7 @@ package testing import ( "time" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" ) diff --git a/node/modules/testing/genesis.go b/node/modules/testing/genesis.go index 1e9d85f2330..cb8d60b5e6b 100644 --- a/node/modules/testing/genesis.go +++ b/node/modules/testing/genesis.go @@ -6,6 +6,7 @@ import ( "fmt" "io" "os" + "github.com/ipfs/boxo/blockservice" offline "github.com/ipfs/boxo/exchange/offline" "github.com/ipfs/boxo/ipld/merkledag" @@ -14,6 +15,7 @@ import ( "github.com/ipld/go-car" "github.com/mitchellh/go-homedir" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/gen" genesis2 "github.com/filecoin-project/lotus/chain/gen/genesis" diff --git a/node/modules/tracer/elasticsearch_transport.go b/node/modules/tracer/elasticsearch_transport.go index 453ef34bf60..e54e0eba241 100644 --- a/node/modules/tracer/elasticsearch_transport.go +++ b/node/modules/tracer/elasticsearch_transport.go @@ -8,6 +8,7 @@ import ( "net/http" "net/url" "time" + "github.com/elastic/go-elasticsearch/v7" "github.com/elastic/go-elasticsearch/v7/esutil" ) diff --git a/node/modules/tracer/tracer.go b/node/modules/tracer/tracer.go index e2b79fe63e7..b9b56e80c86 100644 --- a/node/modules/tracer/tracer.go +++ b/node/modules/tracer/tracer.go @@ -2,6 +2,7 @@ package tracer import ( "time" + logging "github.com/ipfs/go-log/v2" pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" diff --git a/node/modules/tracer/tracer_test.go b/node/modules/tracer/tracer_test.go index f65fdf8bfed..f0d7b2c0b83 100644 --- a/node/modules/tracer/tracer_test.go +++ b/node/modules/tracer/tracer_test.go @@ -3,6 +3,7 @@ package tracer import ( "testing" "time" + pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub_pb "github.com/libp2p/go-libp2p-pubsub/pb" "github.com/libp2p/go-libp2p/core/peer" diff --git a/node/options.go b/node/options.go index 887adfdbe1e..26c2c247f9a 100644 --- a/node/options.go +++ b/node/options.go @@ -2,6 +2,7 @@ package node import ( "reflect" + "go.uber.org/fx" ) diff --git a/node/repo/blockstore_opts.go b/node/repo/blockstore_opts.go index 7f1c71d5c40..81f8b9ff416 100644 --- a/node/repo/blockstore_opts.go +++ b/node/repo/blockstore_opts.go @@ -3,6 +3,7 @@ package repo import ( "os" "strconv" + badgerbs "github.com/filecoin-project/lotus/blockstore/badger" ) diff --git a/node/repo/fsrepo.go b/node/repo/fsrepo.go index 77284b654d6..d8e41fb2bdf 100644 --- a/node/repo/fsrepo.go +++ b/node/repo/fsrepo.go @@ -10,6 +10,7 @@ import ( "path/filepath" "strings" "sync" + "github.com/BurntSushi/toml" "github.com/ipfs/go-datastore" fslock "github.com/ipfs/go-fs-lock" @@ -18,6 +19,7 @@ import ( "github.com/multiformats/go-base32" "github.com/multiformats/go-multiaddr" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" "github.com/filecoin-project/lotus/chain/types" diff --git a/node/repo/fsrepo_ds.go b/node/repo/fsrepo_ds.go index df7f6ee9225..a4415692aaf 100644 --- a/node/repo/fsrepo_ds.go +++ b/node/repo/fsrepo_ds.go @@ -4,6 +4,7 @@ import ( "context" "os" "path/filepath" + dgbadger "github.com/dgraph-io/badger/v2" "github.com/ipfs/go-datastore" badger "github.com/ipfs/go-ds-badger2" diff --git a/node/repo/imports/manager.go b/node/repo/imports/manager.go index d1ac3637164..a3648b6b02a 100644 --- a/node/repo/imports/manager.go +++ b/node/repo/imports/manager.go @@ -7,12 +7,14 @@ import ( "os" "path/filepath" "strconv" + "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" "github.com/ipfs/go-datastore/query" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" + "github.com/filecoin-project/go-fil-markets/shared" ) diff --git a/node/repo/interface.go b/node/repo/interface.go index a5178578955..328862b9214 100644 --- a/node/repo/interface.go +++ b/node/repo/interface.go @@ -3,8 +3,10 @@ package repo import ( "context" "errors" + "github.com/ipfs/go-datastore" "github.com/multiformats/go-multiaddr" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/node/repo/memrepo.go b/node/repo/memrepo.go index 149dd3f55de..6a4b416e204 100644 --- a/node/repo/memrepo.go +++ b/node/repo/memrepo.go @@ -6,12 +6,14 @@ import ( "os" "path/filepath" "sync" + "github.com/google/uuid" "github.com/ipfs/go-datastore" "github.com/ipfs/go-datastore/namespace" dssync "github.com/ipfs/go-datastore/sync" "github.com/multiformats/go-multiaddr" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" diff --git a/node/repo/repo_test.go b/node/repo/repo_test.go index 4fb155c6400..16c101d44b9 100644 --- a/node/repo/repo_test.go +++ b/node/repo/repo_test.go @@ -3,10 +3,12 @@ package repo import ( "testing" + "github.com/multiformats/go-multiaddr" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" "golang.org/x/xerrors" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" ) diff --git a/node/rpc.go b/node/rpc.go index 259b917a258..cacd33526c8 100644 --- a/node/rpc.go +++ b/node/rpc.go @@ -10,6 +10,7 @@ import ( "runtime" "strconv" "time" + "github.com/google/uuid" "github.com/gorilla/mux" "github.com/gorilla/websocket" @@ -19,8 +20,10 @@ import ( manet "github.com/multiformats/go-multiaddr/net" "go.opencensus.io/tag" "golang.org/x/xerrors" + "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-jsonrpc/auth" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/api/v1api" diff --git a/node/shutdown_test.go b/node/shutdown_test.go index e663cc0e7e3..6a5088efd78 100644 --- a/node/shutdown_test.go +++ b/node/shutdown_test.go @@ -6,6 +6,7 @@ import ( "sync" "testing" "time" + "github.com/stretchr/testify/require" ) diff --git a/node/testopts.go b/node/testopts.go index cf19a46407e..ca1e8112759 100644 --- a/node/testopts.go +++ b/node/testopts.go @@ -2,7 +2,9 @@ package node import ( "errors" + mocknet "github.com/libp2p/go-libp2p/p2p/net/mock" + "github.com/filecoin-project/lotus/node/modules/lp2p" ) diff --git a/paychmgr/accessorcache.go b/paychmgr/accessorcache.go index ee6e4b57aba..358cf790061 100644 --- a/paychmgr/accessorcache.go +++ b/paychmgr/accessorcache.go @@ -2,6 +2,7 @@ package paychmgr import ( "context" + "github.com/filecoin-project/go-address" ) diff --git a/paychmgr/cbor_gen.go b/paychmgr/cbor_gen.go index be3fdb3a9d7..f97c176a304 100644 --- a/paychmgr/cbor_gen.go +++ b/paychmgr/cbor_gen.go @@ -7,9 +7,11 @@ import ( "io" "math" "sort" + cid "github.com/ipfs/go-cid" cbg "github.com/whyrusleeping/cbor-gen" xerrors "golang.org/x/xerrors" + address "github.com/filecoin-project/go-address" paych "github.com/filecoin-project/go-state-types/builtin/v8/paych" ) diff --git a/paychmgr/manager.go b/paychmgr/manager.go index 339927ce06b..b1b6a7517a6 100644 --- a/paychmgr/manager.go +++ b/paychmgr/manager.go @@ -4,15 +4,18 @@ import ( "context" "errors" "sync" + "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" logging "github.com/ipfs/go-log/v2" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/stmgr" diff --git a/paychmgr/mock_test.go b/paychmgr/mock_test.go index f4b744e4676..5d36e60f074 100644 --- a/paychmgr/mock_test.go +++ b/paychmgr/mock_test.go @@ -4,11 +4,14 @@ import ( "context" "errors" "sync" + "github.com/ipfs/go-cid" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" diff --git a/paychmgr/msglistener_test.go b/paychmgr/msglistener_test.go index 8d2e793de1a..38f2351ffa3 100644 --- a/paychmgr/msglistener_test.go +++ b/paychmgr/msglistener_test.go @@ -2,6 +2,7 @@ package paychmgr import ( "testing" + "github.com/ipfs/go-cid" "github.com/stretchr/testify/require" "golang.org/x/xerrors" diff --git a/paychmgr/paych.go b/paychmgr/paych.go index 1f6b7cf917f..2729513ce88 100644 --- a/paychmgr/paych.go +++ b/paychmgr/paych.go @@ -3,13 +3,16 @@ package paychmgr import ( "context" "fmt" + "github.com/ipfs/go-cid" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" cborutil "github.com/filecoin-project/go-cbor-util" actorstypes "github.com/filecoin-project/go-state-types/actors" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api" lpaych "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/paych_test.go b/paychmgr/paych_test.go index fe6f640f2b2..e72a992a37f 100644 --- a/paychmgr/paych_test.go +++ b/paychmgr/paych_test.go @@ -5,10 +5,12 @@ import ( "bytes" "context" "testing" + "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" @@ -16,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/specs-actors/v2/actors/builtin" tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" diff --git a/paychmgr/paychget_test.go b/paychmgr/paychget_test.go index e3f39de757b..fca0022c6b4 100644 --- a/paychmgr/paychget_test.go +++ b/paychmgr/paychget_test.go @@ -6,10 +6,12 @@ import ( "sync" "testing" "time" + "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" @@ -17,6 +19,7 @@ import ( "github.com/filecoin-project/specs-actors/v2/actors/builtin" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + lotusinit "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" diff --git a/paychmgr/paychvoucherfunds_test.go b/paychmgr/paychvoucherfunds_test.go index 600ff30838f..9cff7fc9d55 100644 --- a/paychmgr/paychvoucherfunds_test.go +++ b/paychmgr/paychvoucherfunds_test.go @@ -3,15 +3,18 @@ package paychmgr import ( "context" "testing" + "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" builtin2 "github.com/filecoin-project/specs-actors/v2/actors/builtin" tutils2 "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/chain/actors/builtin/paych" paychmock "github.com/filecoin-project/lotus/chain/actors/builtin/paych/mock" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/settle_test.go b/paychmgr/settle_test.go index 71dd39421db..db1d35da35a 100644 --- a/paychmgr/settle_test.go +++ b/paychmgr/settle_test.go @@ -4,10 +4,12 @@ package paychmgr import ( "context" "testing" + "github.com/ipfs/go-cid" ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-state-types/big" tutils "github.com/filecoin-project/specs-actors/support/testing" ) diff --git a/paychmgr/settler/settler.go b/paychmgr/settler/settler.go index dc04944f9b0..55431f0a561 100644 --- a/paychmgr/settler/settler.go +++ b/paychmgr/settler/settler.go @@ -3,13 +3,16 @@ package settler import ( "context" "sync" + "github.com/ipfs/go-cid" logging "github.com/ipfs/go-log/v2" "go.uber.org/fx" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" paychtypes "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/events" diff --git a/paychmgr/simple.go b/paychmgr/simple.go index 07fd140e947..6d6b78a713e 100644 --- a/paychmgr/simple.go +++ b/paychmgr/simple.go @@ -7,12 +7,15 @@ import ( "fmt" "sort" "sync" + "github.com/ipfs/go-cid" "golang.org/x/sync/errgroup" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" init2 "github.com/filecoin-project/specs-actors/v2/actors/builtin/init" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/types" diff --git a/paychmgr/state.go b/paychmgr/state.go index 5943488f60c..0466d2d3651 100644 --- a/paychmgr/state.go +++ b/paychmgr/state.go @@ -2,7 +2,9 @@ package paychmgr import ( "context" + "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/chain/actors/builtin/paych" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/paychmgr/store.go b/paychmgr/store.go index 22f6415e53b..72a95d0d442 100644 --- a/paychmgr/store.go +++ b/paychmgr/store.go @@ -5,15 +5,18 @@ import ( "context" "errors" "fmt" + "github.com/google/uuid" "github.com/ipfs/go-cid" "github.com/ipfs/go-datastore" dsq "github.com/ipfs/go-datastore/query" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" cborrpc "github.com/filecoin-project/go-cbor-util" cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/builtin/v8/paych" + "github.com/filecoin-project/lotus/chain/types" ) diff --git a/paychmgr/store_test.go b/paychmgr/store_test.go index c815421120a..f600b6537a9 100644 --- a/paychmgr/store_test.go +++ b/paychmgr/store_test.go @@ -4,9 +4,11 @@ package paychmgr import ( "context" "testing" + ds "github.com/ipfs/go-datastore" ds_sync "github.com/ipfs/go-datastore/sync" "github.com/stretchr/testify/require" + "github.com/filecoin-project/go-address" tutils "github.com/filecoin-project/specs-actors/support/testing" ) diff --git a/paychmgr/util.go b/paychmgr/util.go index f1f627037aa..57853ac1441 100644 --- a/paychmgr/util.go +++ b/paychmgr/util.go @@ -2,6 +2,7 @@ package paychmgr import ( "context" + "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/builtin/v8/paych" ) diff --git a/provider/address.go b/provider/address.go index 66e53ae8945..f69ca3fac60 100644 --- a/provider/address.go +++ b/provider/address.go @@ -4,6 +4,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/ctladdr" ) diff --git a/provider/lpmessage/sender.go b/provider/lpmessage/sender.go index aa0826b6216..5123410c957 100644 --- a/provider/lpmessage/sender.go +++ b/provider/lpmessage/sender.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" diff --git a/provider/lpwindow/compute_do.go b/provider/lpwindow/compute_do.go index 2de2f26f505..ba13cdc2b53 100644 --- a/provider/lpwindow/compute_do.go +++ b/provider/lpwindow/compute_do.go @@ -20,12 +20,13 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/proof" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" types "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer" "github.com/filecoin-project/lotus/storage/sealer/storiface" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) const disablePreChecks = false // todo config diff --git a/provider/lpwindow/compute_task.go b/provider/lpwindow/compute_task.go index 32b3ebf3a60..83bf67a3104 100644 --- a/provider/lpwindow/compute_task.go +++ b/provider/lpwindow/compute_task.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/provider/lpwindow/compute_task_test.go b/provider/lpwindow/compute_task_test.go index d543847cb92..0f7a1335bce 100644 --- a/provider/lpwindow/compute_task_test.go +++ b/provider/lpwindow/compute_task_test.go @@ -6,6 +6,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/lib/harmony/harmonytask" diff --git a/provider/lpwindow/faults_simple.go b/provider/lpwindow/faults_simple.go index e7599e2d3c2..d43e8ee196a 100644 --- a/provider/lpwindow/faults_simple.go +++ b/provider/lpwindow/faults_simple.go @@ -11,6 +11,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/provider/lpwindow/recover_task.go b/provider/lpwindow/recover_task.go index 13b1ed77822..fafe76e569b 100644 --- a/provider/lpwindow/recover_task.go +++ b/provider/lpwindow/recover_task.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/provider/lpwindow/submit_task.go b/provider/lpwindow/submit_task.go index d4f9643065c..ba0f63f6d83 100644 --- a/provider/lpwindow/submit_task.go +++ b/provider/lpwindow/submit_task.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/crypto" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" diff --git a/storage/ctladdr/addresses.go b/storage/ctladdr/addresses.go index 5146cb89f16..ee778cb38e6 100644 --- a/storage/ctladdr/addresses.go +++ b/storage/ctladdr/addresses.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/paths/db_index.go b/storage/paths/db_index.go index 6a8646b7a24..7f8bc6ebce1 100644 --- a/storage/paths/db_index.go +++ b/storage/paths/db_index.go @@ -16,6 +16,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/metrics" diff --git a/storage/paths/http_handler.go b/storage/paths/http_handler.go index b36dc3257aa..4d0539079b2 100644 --- a/storage/paths/http_handler.go +++ b/storage/paths/http_handler.go @@ -14,6 +14,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/filecoin-project/lotus/storage/sealer/tarutil" diff --git a/storage/paths/http_handler_test.go b/storage/paths/http_handler_test.go index 525d954a064..cf6d71c3775 100644 --- a/storage/paths/http_handler_test.go +++ b/storage/paths/http_handler_test.go @@ -14,6 +14,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/paths/mocks" "github.com/filecoin-project/lotus/storage/sealer/partialfile" diff --git a/storage/paths/index.go b/storage/paths/index.go index e28ad7781da..bc26bddb420 100644 --- a/storage/paths/index.go +++ b/storage/paths/index.go @@ -16,6 +16,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/paths/index_locks.go b/storage/paths/index_locks.go index de34fe856fb..ab95cb4a7ab 100644 --- a/storage/paths/index_locks.go +++ b/storage/paths/index_locks.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/index_locks_test.go b/storage/paths/index_locks_test.go index 864c24b4f81..d3134055bcb 100644 --- a/storage/paths/index_locks_test.go +++ b/storage/paths/index_locks_test.go @@ -8,6 +8,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/index_proxy.go b/storage/paths/index_proxy.go index 1b1b51ac740..06097b665ce 100644 --- a/storage/paths/index_proxy.go +++ b/storage/paths/index_proxy.go @@ -4,6 +4,7 @@ import ( "context" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/journal/alerting" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/paths/index_test.go b/storage/paths/index_test.go index efde3b1b727..9a241da23e0 100644 --- a/storage/paths/index_test.go +++ b/storage/paths/index_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/interface.go b/storage/paths/interface.go index d52e40b777a..d96135de8f2 100644 --- a/storage/paths/interface.go +++ b/storage/paths/interface.go @@ -5,6 +5,7 @@ import ( "io" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/local.go b/storage/paths/local.go index 7a653152b53..577d4dbe008 100644 --- a/storage/paths/local.go +++ b/storage/paths/local.go @@ -15,6 +15,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/lib/result" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/mocks/index.go b/storage/paths/mocks/index.go index d4d6fd7c686..6fdcb03b9ec 100644 --- a/storage/paths/mocks/index.go +++ b/storage/paths/mocks/index.go @@ -11,6 +11,7 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" + fsutil "github.com/filecoin-project/lotus/storage/sealer/fsutil" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/mocks/pf.go b/storage/paths/mocks/pf.go index 0ae1eaacb53..43b3bc4896b 100644 --- a/storage/paths/mocks/pf.go +++ b/storage/paths/mocks/pf.go @@ -11,6 +11,7 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" + partialfile "github.com/filecoin-project/lotus/storage/sealer/partialfile" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/mocks/store.go b/storage/paths/mocks/store.go index 69dbec4eac4..72be4832378 100644 --- a/storage/paths/mocks/store.go +++ b/storage/paths/mocks/store.go @@ -11,6 +11,7 @@ import ( gomock "github.com/golang/mock/gomock" abi "github.com/filecoin-project/go-state-types/abi" + fsutil "github.com/filecoin-project/lotus/storage/sealer/fsutil" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/paths/remote.go b/storage/paths/remote.go index dbbc17a106d..0b7563bb251 100644 --- a/storage/paths/remote.go +++ b/storage/paths/remote.go @@ -20,6 +20,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/paths/remote_test.go b/storage/paths/remote_test.go index 231929bc61c..e3376e6fa10 100644 --- a/storage/paths/remote_test.go +++ b/storage/paths/remote_test.go @@ -20,6 +20,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/node/repo" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/paths/mocks" diff --git a/storage/pipeline/cbor_gen.go b/storage/pipeline/cbor_gen.go index 76626a7769c..c832f8a144a 100644 --- a/storage/pipeline/cbor_gen.go +++ b/storage/pipeline/cbor_gen.go @@ -13,6 +13,7 @@ import ( xerrors "golang.org/x/xerrors" abi "github.com/filecoin-project/go-state-types/abi" + api "github.com/filecoin-project/lotus/api" storiface "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/checks.go b/storage/pipeline/checks.go index f9ee0547a98..ecd160231c1 100644 --- a/storage/pipeline/checks.go +++ b/storage/pipeline/checks.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/crypto" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/commit_batch.go b/storage/pipeline/commit_batch.go index 1f9fdf29ea7..754f317630b 100644 --- a/storage/pipeline/commit_batch.go +++ b/storage/pipeline/commit_batch.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/commit_batch_test.go b/storage/pipeline/commit_batch_test.go index 8899bbfab25..5ae2f171ae2 100644 --- a/storage/pipeline/commit_batch_test.go +++ b/storage/pipeline/commit_batch_test.go @@ -20,6 +20,8 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" + miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/ctladdr" @@ -27,7 +29,6 @@ import ( "github.com/filecoin-project/lotus/storage/pipeline/mocks" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" "github.com/filecoin-project/lotus/storage/sealer/storiface" - miner5 "github.com/filecoin-project/specs-actors/v5/actors/builtin/miner" ) func TestCommitBatcher(t *testing.T) { diff --git a/storage/pipeline/currentdealinfo.go b/storage/pipeline/currentdealinfo.go index f54fb535edd..cd3aa47be27 100644 --- a/storage/pipeline/currentdealinfo.go +++ b/storage/pipeline/currentdealinfo.go @@ -13,6 +13,7 @@ import ( markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/currentdealinfo_test.go b/storage/pipeline/currentdealinfo_test.go index d3d8c4c49c4..21141a35d57 100644 --- a/storage/pipeline/currentdealinfo_test.go +++ b/storage/pipeline/currentdealinfo_test.go @@ -20,12 +20,13 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" evtmock "github.com/filecoin-project/lotus/chain/events/state/mock" "github.com/filecoin-project/lotus/chain/types" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) var errNotFound = errors.New("could not find") diff --git a/storage/pipeline/fsm.go b/storage/pipeline/fsm.go index 75d094f2b01..ac3dafa86ec 100644 --- a/storage/pipeline/fsm.go +++ b/storage/pipeline/fsm.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" ) diff --git a/storage/pipeline/fsm_events.go b/storage/pipeline/fsm_events.go index c778a1da7d3..a798a884b48 100644 --- a/storage/pipeline/fsm_events.go +++ b/storage/pipeline/fsm_events.go @@ -9,6 +9,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/pipeline/fsm_test.go b/storage/pipeline/fsm_test.go index 565161b8287..7d7201953e8 100644 --- a/storage/pipeline/fsm_test.go +++ b/storage/pipeline/fsm_test.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/input.go b/storage/pipeline/input.go index 0d0539eef7a..6fdae03d926 100644 --- a/storage/pipeline/input.go +++ b/storage/pipeline/input.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/mocks/api.go b/storage/pipeline/mocks/api.go index 9b63cf58c3d..5c67a1c42e2 100644 --- a/storage/pipeline/mocks/api.go +++ b/storage/pipeline/mocks/api.go @@ -20,6 +20,7 @@ import ( crypto "github.com/filecoin-project/go-state-types/crypto" dline "github.com/filecoin-project/go-state-types/dline" network "github.com/filecoin-project/go-state-types/network" + api "github.com/filecoin-project/lotus/api" miner0 "github.com/filecoin-project/lotus/chain/actors/builtin/miner" types "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/mocks/mock_commit_batcher.go b/storage/pipeline/mocks/mock_commit_batcher.go index a6d30ae2506..431a47c73d9 100644 --- a/storage/pipeline/mocks/mock_commit_batcher.go +++ b/storage/pipeline/mocks/mock_commit_batcher.go @@ -15,6 +15,7 @@ import ( big "github.com/filecoin-project/go-state-types/big" miner "github.com/filecoin-project/go-state-types/builtin/v9/miner" network "github.com/filecoin-project/go-state-types/network" + api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/mocks/mock_precommit_batcher.go b/storage/pipeline/mocks/mock_precommit_batcher.go index 518a9c00ae2..68cce7fb0f3 100644 --- a/storage/pipeline/mocks/mock_precommit_batcher.go +++ b/storage/pipeline/mocks/mock_precommit_batcher.go @@ -15,6 +15,7 @@ import ( big "github.com/filecoin-project/go-state-types/big" verifreg "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" network "github.com/filecoin-project/go-state-types/network" + api "github.com/filecoin-project/lotus/api" types "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/numassign.go b/storage/pipeline/numassign.go index 2b5f07778ad..60dddec98ea 100644 --- a/storage/pipeline/numassign.go +++ b/storage/pipeline/numassign.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-bitfield" rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/pipeline/precommit_batch.go b/storage/pipeline/precommit_batch.go index 083643dfb5f..3a86c8628e0 100644 --- a/storage/pipeline/precommit_batch.go +++ b/storage/pipeline/precommit_batch.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" verifregtypes "github.com/filecoin-project/go-state-types/builtin/v9/verifreg" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/precommit_batch_test.go b/storage/pipeline/precommit_batch_test.go index bd9d6285854..1f3aaf24472 100644 --- a/storage/pipeline/precommit_batch_test.go +++ b/storage/pipeline/precommit_batch_test.go @@ -17,13 +17,14 @@ import ( "github.com/filecoin-project/go-state-types/big" minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/network" + miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/node/config" pipeline "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/pipeline/mocks" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" - miner6 "github.com/filecoin-project/specs-actors/v6/actors/builtin/miner" ) var fc = config.MinerFeeConfig{ diff --git a/storage/pipeline/precommit_policy.go b/storage/pipeline/precommit_policy.go index 45955f8e438..6e234f93094 100644 --- a/storage/pipeline/precommit_policy.go +++ b/storage/pipeline/precommit_policy.go @@ -8,6 +8,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/precommit_policy_test.go b/storage/pipeline/precommit_policy_test.go index 3aeac055824..7865560dec1 100644 --- a/storage/pipeline/precommit_policy_test.go +++ b/storage/pipeline/precommit_policy_test.go @@ -12,6 +12,7 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/storage/pipeline/receive.go b/storage/pipeline/receive.go index 429f99d909b..8427eba54f5 100644 --- a/storage/pipeline/receive.go +++ b/storage/pipeline/receive.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/pipeline/sealing.go b/storage/pipeline/sealing.go index 4cd99280a3c..65d3fb14b1e 100644 --- a/storage/pipeline/sealing.go +++ b/storage/pipeline/sealing.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" "github.com/filecoin-project/go-storedcounter" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_failed.go b/storage/pipeline/states_failed.go index d7065e3ff7c..3323c4c9bc4 100644 --- a/storage/pipeline/states_failed.go +++ b/storage/pipeline/states_failed.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_failed_test.go b/storage/pipeline/states_failed_test.go index 5e94a9f1f55..f6846c8f5f1 100644 --- a/storage/pipeline/states_failed_test.go +++ b/storage/pipeline/states_failed_test.go @@ -16,12 +16,13 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-statemachine" + market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" + api2 "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/types" pipeline "github.com/filecoin-project/lotus/storage/pipeline" "github.com/filecoin-project/lotus/storage/pipeline/mocks" - market0 "github.com/filecoin-project/specs-actors/actors/builtin/market" ) func TestStateRecoverDealIDs(t *testing.T) { diff --git a/storage/pipeline/states_proving.go b/storage/pipeline/states_proving.go index 63f20a2793b..500dcab48e6 100644 --- a/storage/pipeline/states_proving.go +++ b/storage/pipeline/states_proving.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" diff --git a/storage/pipeline/states_replica_update.go b/storage/pipeline/states_replica_update.go index c39763a6799..6717f49a6a6 100644 --- a/storage/pipeline/states_replica_update.go +++ b/storage/pipeline/states_replica_update.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/exitcode" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/states_sealing.go b/storage/pipeline/states_sealing.go index 2fc929f7374..5c91161efb1 100644 --- a/storage/pipeline/states_sealing.go +++ b/storage/pipeline/states_sealing.go @@ -22,6 +22,7 @@ import ( "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statemachine" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/pipeline/stats.go b/storage/pipeline/stats.go index b257644b56c..696cd56a560 100644 --- a/storage/pipeline/stats.go +++ b/storage/pipeline/stats.go @@ -8,6 +8,7 @@ import ( "go.opencensus.io/tag" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" ) diff --git a/storage/pipeline/terminate_batch.go b/storage/pipeline/terminate_batch.go index 58eaa176e3c..4842a4e1bb7 100644 --- a/storage/pipeline/terminate_batch.go +++ b/storage/pipeline/terminate_batch.go @@ -17,6 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/builtin/v8/miner" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/api" lminer "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" diff --git a/storage/pipeline/types.go b/storage/pipeline/types.go index c82b024ca5c..e752eb2b93b 100644 --- a/storage/pipeline/types.go +++ b/storage/pipeline/types.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer" diff --git a/storage/pipeline/types_test.go b/storage/pipeline/types_test.go index 002294d57a2..b8fbb113af1 100644 --- a/storage/pipeline/types_test.go +++ b/storage/pipeline/types_test.go @@ -10,8 +10,9 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" markettypes "github.com/filecoin-project/go-state-types/builtin/v9/market" - "github.com/filecoin-project/lotus/api" tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + + "github.com/filecoin-project/lotus/api" ) func TestSectorInfoSerialization(t *testing.T) { diff --git a/storage/pipeline/upgrade_queue.go b/storage/pipeline/upgrade_queue.go index d739ca9fdd0..9d9e1ca46c6 100644 --- a/storage/pipeline/upgrade_queue.go +++ b/storage/pipeline/upgrade_queue.go @@ -6,8 +6,9 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/chain/types" market7 "github.com/filecoin-project/specs-actors/v7/actors/builtin/market" + + "github.com/filecoin-project/lotus/chain/types" ) func (m *Sealing) MarkForUpgrade(ctx context.Context, id abi.SectorNumber) error { diff --git a/storage/pipeline/utils.go b/storage/pipeline/utils.go index 997fcf87aa9..4b99a5beadd 100644 --- a/storage/pipeline/utils.go +++ b/storage/pipeline/utils.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/pipeline/sealiface" diff --git a/storage/sealer/faults.go b/storage/sealer/faults.go index b2dc4181bf4..add2acf9666 100644 --- a/storage/sealer/faults.go +++ b/storage/sealer/faults.go @@ -10,6 +10,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/basicfs/fs.go b/storage/sealer/ffiwrapper/basicfs/fs.go index 8b9e889cc81..4fd8e271fdf 100644 --- a/storage/sealer/ffiwrapper/basicfs/fs.go +++ b/storage/sealer/ffiwrapper/basicfs/fs.go @@ -7,6 +7,7 @@ import ( "sync" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/prover_cgo.go b/storage/sealer/ffiwrapper/prover_cgo.go index 8aa7339e367..abbd20b6e06 100644 --- a/storage/sealer/ffiwrapper/prover_cgo.go +++ b/storage/sealer/ffiwrapper/prover_cgo.go @@ -6,6 +6,7 @@ package ffiwrapper import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/sealer_cgo.go b/storage/sealer/ffiwrapper/sealer_cgo.go index d9966fd8466..812a69fa7a6 100644 --- a/storage/sealer/ffiwrapper/sealer_cgo.go +++ b/storage/sealer/ffiwrapper/sealer_cgo.go @@ -29,6 +29,7 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/lib/nullreader" spaths "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fr32" diff --git a/storage/sealer/ffiwrapper/sealer_test.go b/storage/sealer/ffiwrapper/sealer_test.go index 5776b26e745..da1b98429c7 100644 --- a/storage/sealer/ffiwrapper/sealer_test.go +++ b/storage/sealer/ffiwrapper/sealer_test.go @@ -28,6 +28,7 @@ import ( "github.com/filecoin-project/go-paramfetch" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/storage/pipeline/lib/nullreader" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper/basicfs" diff --git a/storage/sealer/ffiwrapper/unseal_ranges.go b/storage/sealer/ffiwrapper/unseal_ranges.go index 82f06efab09..37107aeb09f 100644 --- a/storage/sealer/ffiwrapper/unseal_ranges.go +++ b/storage/sealer/ffiwrapper/unseal_ranges.go @@ -5,6 +5,7 @@ import ( rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/partialfile" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/ffiwrapper/verifier_cgo.go b/storage/sealer/ffiwrapper/verifier_cgo.go index ee4281fc223..7576d957d3b 100644 --- a/storage/sealer/ffiwrapper/verifier_cgo.go +++ b/storage/sealer/ffiwrapper/verifier_cgo.go @@ -11,8 +11,9 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) func (sb *Sealer) GenerateWinningPoSt(ctx context.Context, minerID abi.ActorID, sectorInfo []proof.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]proof.PoStProof, error) { diff --git a/storage/sealer/fr32/fr32_ffi_cmp_test.go b/storage/sealer/fr32/fr32_ffi_cmp_test.go index 9438ed942ae..32afa470eba 100644 --- a/storage/sealer/fr32/fr32_ffi_cmp_test.go +++ b/storage/sealer/fr32/fr32_ffi_cmp_test.go @@ -11,6 +11,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" commpffi "github.com/filecoin-project/go-commp-utils/ffiwrapper" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/fr32/fr32_test.go b/storage/sealer/fr32/fr32_test.go index ba5a0d673b3..437fa4e43cb 100644 --- a/storage/sealer/fr32/fr32_test.go +++ b/storage/sealer/fr32/fr32_test.go @@ -12,6 +12,7 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" commpffi "github.com/filecoin-project/go-commp-utils/ffiwrapper" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/fr32/readers_test.go b/storage/sealer/fr32/readers_test.go index c568cbfe23d..f84b9d67a2a 100644 --- a/storage/sealer/fr32/readers_test.go +++ b/storage/sealer/fr32/readers_test.go @@ -9,6 +9,7 @@ import ( "github.com/stretchr/testify/require" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/manager.go b/storage/sealer/manager.go index fab85e83a4e..41b3a1b397a 100644 --- a/storage/sealer/manager.go +++ b/storage/sealer/manager.go @@ -19,6 +19,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" diff --git a/storage/sealer/manager_post.go b/storage/sealer/manager_post.go index 7c9bf595ba8..27a71ef8caf 100644 --- a/storage/sealer/manager_post.go +++ b/storage/sealer/manager_post.go @@ -12,8 +12,9 @@ import ( ffi "github.com/filecoin-project/filecoin-ffi" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" - "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) func (m *Manager) GenerateWinningPoSt(ctx context.Context, minerID abi.ActorID, sectorInfo []proof.ExtendedSectorInfo, randomness abi.PoStRandomness) ([]proof.PoStProof, error) { diff --git a/storage/sealer/manager_test.go b/storage/sealer/manager_test.go index 5cbe4bc7860..7c3e1a1f2b1 100644 --- a/storage/sealer/manager_test.go +++ b/storage/sealer/manager_test.go @@ -28,13 +28,14 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statestore" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) func init() { diff --git a/storage/sealer/mock/mock.go b/storage/sealer/mock/mock.go index a67aee091c8..cb498492f49 100644 --- a/storage/sealer/mock/mock.go +++ b/storage/sealer/mock/mock.go @@ -18,6 +18,7 @@ import ( commcid "github.com/filecoin-project/go-fil-commcid" "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/mock/util.go b/storage/sealer/mock/util.go index d78f6ce2b65..48055b4d083 100644 --- a/storage/sealer/mock/util.go +++ b/storage/sealer/mock/util.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/go-state-types/builtin/v9/market" + "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/chain/wallet/key" "github.com/filecoin-project/lotus/genesis" diff --git a/storage/sealer/partialfile/partialfile.go b/storage/sealer/partialfile/partialfile.go index 9ad98342125..4357f796d6d 100644 --- a/storage/sealer/partialfile/partialfile.go +++ b/storage/sealer/partialfile/partialfile.go @@ -12,6 +12,7 @@ import ( rlepluslazy "github.com/filecoin-project/go-bitfield/rle" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/lib/readerutil" "github.com/filecoin-project/lotus/storage/sealer/fsutil" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/piece_provider.go b/storage/sealer/piece_provider.go index 2383cd6511f..0e992b67918 100644 --- a/storage/sealer/piece_provider.go +++ b/storage/sealer/piece_provider.go @@ -12,6 +12,7 @@ import ( "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fr32" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/piece_provider_test.go b/storage/sealer/piece_provider_test.go index 7466c3ed7e6..de1e07a7849 100644 --- a/storage/sealer/piece_provider_test.go +++ b/storage/sealer/piece_provider_test.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/node/config" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/piece_reader.go b/storage/sealer/piece_reader.go index 3d963de80ed..7a7cd184110 100644 --- a/storage/sealer/piece_reader.go +++ b/storage/sealer/piece_reader.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/dagstore/mount" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/metrics" ) diff --git a/storage/sealer/sched.go b/storage/sealer/sched.go index 74b0b2a6aba..c0ac11bcf9d 100644 --- a/storage/sealer/sched.go +++ b/storage/sealer/sched.go @@ -9,6 +9,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/sched_post.go b/storage/sealer/sched_post.go index de609a770a3..c6bd8182976 100644 --- a/storage/sealer/sched_post.go +++ b/storage/sealer/sched_post.go @@ -13,6 +13,7 @@ import ( "github.com/filecoin-project/go-jsonrpc" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/sched_test.go b/storage/sealer/sched_test.go index 495347f3761..2e2b05ab2c3 100644 --- a/storage/sealer/sched_test.go +++ b/storage/sealer/sched_test.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" prooftypes "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/fsutil" diff --git a/storage/sealer/selector_alloc.go b/storage/sealer/selector_alloc.go index 9356902b149..130f74461f5 100644 --- a/storage/sealer/selector_alloc.go +++ b/storage/sealer/selector_alloc.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_existing.go b/storage/sealer/selector_existing.go index b63d7a1c792..c1e082db8a5 100644 --- a/storage/sealer/selector_existing.go +++ b/storage/sealer/selector_existing.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_move.go b/storage/sealer/selector_move.go index 1eee3d82153..fde4b3c59cd 100644 --- a/storage/sealer/selector_move.go +++ b/storage/sealer/selector_move.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sealer/selector_task.go b/storage/sealer/selector_task.go index fc2907a1936..805fcbbd03b 100644 --- a/storage/sealer/selector_task.go +++ b/storage/sealer/selector_task.go @@ -6,6 +6,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/storiface/ffi.go b/storage/sealer/storiface/ffi.go index d315832837c..4a9f832b843 100644 --- a/storage/sealer/storiface/ffi.go +++ b/storage/sealer/storiface/ffi.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fr32" ) diff --git a/storage/sealer/storiface/index.go b/storage/sealer/storiface/index.go index f7436bce822..653bd2fbae5 100644 --- a/storage/sealer/storiface/index.go +++ b/storage/sealer/storiface/index.go @@ -4,6 +4,7 @@ import ( "strings" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/fsutil" ) diff --git a/storage/sealer/storiface/resources.go b/storage/sealer/storiface/resources.go index 1b09681d4a5..6f8d83265a6 100644 --- a/storage/sealer/storiface/resources.go +++ b/storage/sealer/storiface/resources.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/storiface/resources_test.go b/storage/sealer/storiface/resources_test.go index 6e877fddc6a..e360b8d1430 100644 --- a/storage/sealer/storiface/resources_test.go +++ b/storage/sealer/storiface/resources_test.go @@ -7,6 +7,7 @@ import ( "github.com/stretchr/testify/require" stabi "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/storiface/worker.go b/storage/sealer/storiface/worker.go index e11eb07cc48..e84fd8aa9a8 100644 --- a/storage/sealer/storiface/worker.go +++ b/storage/sealer/storiface/worker.go @@ -11,6 +11,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" + "github.com/filecoin-project/lotus/storage/sealer/sealtasks" ) diff --git a/storage/sealer/teststorage_test.go b/storage/sealer/teststorage_test.go index 5c54bbbad20..6e50044b3eb 100644 --- a/storage/sealer/teststorage_test.go +++ b/storage/sealer/teststorage_test.go @@ -7,8 +7,9 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/abi" - "github.com/filecoin-project/lotus/storage/sealer/storiface" "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) type apres struct { diff --git a/storage/sealer/testworker_test.go b/storage/sealer/testworker_test.go index 868bb26eb46..6ea57b78d1f 100644 --- a/storage/sealer/testworker_test.go +++ b/storage/sealer/testworker_test.go @@ -8,6 +8,7 @@ import ( "github.com/ipfs/go-cid" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/mock" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/worker_calltracker.go b/storage/sealer/worker_calltracker.go index 443530f58a2..eb5bab145ce 100644 --- a/storage/sealer/worker_calltracker.go +++ b/storage/sealer/worker_calltracker.go @@ -8,6 +8,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/worker_local.go b/storage/sealer/worker_local.go index 783976cae15..67510566c2c 100644 --- a/storage/sealer/worker_local.go +++ b/storage/sealer/worker_local.go @@ -21,6 +21,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/proof" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" diff --git a/storage/sealer/worker_local_test.go b/storage/sealer/worker_local_test.go index 0b287d05042..8624f2d918b 100644 --- a/storage/sealer/worker_local_test.go +++ b/storage/sealer/worker_local_test.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-statestore" + "github.com/filecoin-project/lotus/storage/paths" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) diff --git a/storage/sealer/worker_tracked.go b/storage/sealer/worker_tracked.go index 5f17c9b8e94..7fce400a050 100644 --- a/storage/sealer/worker_tracked.go +++ b/storage/sealer/worker_tracked.go @@ -11,6 +11,7 @@ import ( "go.opencensus.io/tag" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/sealtasks" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/sectorblocks/blocks.go b/storage/sectorblocks/blocks.go index 5343c4eb3d1..1593174bd8f 100644 --- a/storage/sectorblocks/blocks.go +++ b/storage/sectorblocks/blocks.go @@ -16,6 +16,7 @@ import ( cborutil "github.com/filecoin-project/go-cbor-util" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/node/modules/dtypes" "github.com/filecoin-project/lotus/storage/sealer/storiface" diff --git a/storage/wdpost/wdpost_changehandler.go b/storage/wdpost/wdpost_changehandler.go index 6e86ccd30a8..ce58f148920 100644 --- a/storage/wdpost/wdpost_changehandler.go +++ b/storage/wdpost/wdpost_changehandler.go @@ -7,6 +7,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/types" ) diff --git a/storage/wdpost/wdpost_changehandler_test.go b/storage/wdpost/wdpost_changehandler_test.go index 43c6accf764..44d0dfe6d02 100644 --- a/storage/wdpost/wdpost_changehandler_test.go +++ b/storage/wdpost/wdpost_changehandler_test.go @@ -16,8 +16,9 @@ import ( minertypes "github.com/filecoin-project/go-state-types/builtin/v9/miner" "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" - "github.com/filecoin-project/lotus/chain/types" tutils "github.com/filecoin-project/specs-actors/support/testing" + + "github.com/filecoin-project/lotus/chain/types" ) var dummyCid cid.Cid diff --git a/storage/wdpost/wdpost_journal.go b/storage/wdpost/wdpost_journal.go index 039475fc173..68cad3fcf3f 100644 --- a/storage/wdpost/wdpost_journal.go +++ b/storage/wdpost/wdpost_journal.go @@ -5,6 +5,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/chain/actors/builtin/miner" ) diff --git a/storage/wdpost/wdpost_run.go b/storage/wdpost/wdpost_run.go index a34c17d850e..edc088ccf64 100644 --- a/storage/wdpost/wdpost_run.go +++ b/storage/wdpost/wdpost_run.go @@ -19,6 +19,8 @@ import ( "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" "github.com/filecoin-project/go-state-types/proof" + proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -27,7 +29,6 @@ import ( "github.com/filecoin-project/lotus/chain/messagepool" "github.com/filecoin-project/lotus/chain/types" "github.com/filecoin-project/lotus/storage/sealer/storiface" - proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" ) // recordPoStFailure records a failure in the journal. diff --git a/storage/wdpost/wdpost_run_faults.go b/storage/wdpost/wdpost_run_faults.go index 010086abdfa..3a41cc4cce4 100644 --- a/storage/wdpost/wdpost_run_faults.go +++ b/storage/wdpost/wdpost_run_faults.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/builtin" "github.com/filecoin-project/go-state-types/dline" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" diff --git a/storage/wdpost/wdpost_run_test.go b/storage/wdpost/wdpost_run_test.go index 0a843b2adad..a3847a6f6bd 100644 --- a/storage/wdpost/wdpost_run_test.go +++ b/storage/wdpost/wdpost_run_test.go @@ -22,6 +22,8 @@ import ( "github.com/filecoin-project/go-state-types/manifest" "github.com/filecoin-project/go-state-types/network" prooftypes "github.com/filecoin-project/go-state-types/proof" + tutils "github.com/filecoin-project/specs-actors/v2/support/testing" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" @@ -30,7 +32,6 @@ import ( "github.com/filecoin-project/lotus/journal" "github.com/filecoin-project/lotus/storage/ctladdr" "github.com/filecoin-project/lotus/storage/sealer/storiface" - tutils "github.com/filecoin-project/specs-actors/v2/support/testing" ) type mockStorageMinerAPI struct { diff --git a/storage/wdpost/wdpost_sched.go b/storage/wdpost/wdpost_sched.go index 239f50ff67d..bbf4596fe30 100644 --- a/storage/wdpost/wdpost_sched.go +++ b/storage/wdpost/wdpost_sched.go @@ -15,6 +15,7 @@ import ( "github.com/filecoin-project/go-state-types/crypto" "github.com/filecoin-project/go-state-types/dline" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" diff --git a/storage/winning_prover.go b/storage/winning_prover.go index fc5338de248..d61cd213c50 100644 --- a/storage/winning_prover.go +++ b/storage/winning_prover.go @@ -10,6 +10,7 @@ import ( "github.com/filecoin-project/go-address" "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/network" + "github.com/filecoin-project/lotus/api/v1api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/builtin" diff --git a/tools/stats/points/collect.go b/tools/stats/points/collect.go index fa9700d680f..8b86695742e 100644 --- a/tools/stats/points/collect.go +++ b/tools/stats/points/collect.go @@ -15,6 +15,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/go-address" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/adt" diff --git a/tools/stats/sync/sync.go b/tools/stats/sync/sync.go index 7bad431c00c..c8db1c543be 100644 --- a/tools/stats/sync/sync.go +++ b/tools/stats/sync/sync.go @@ -5,6 +5,7 @@ import ( "time" "github.com/filecoin-project/go-state-types/abi" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/store" From 0d90430cf0bbc30fe74e703f89ed2e7b6d910dea Mon Sep 17 00:00:00 2001 From: "Andrew Jackson (Ajax)" Date: Tue, 14 Nov 2023 14:51:48 -0600 Subject: [PATCH 22/24] better storageRpcSecret errormsg --- cmd/lotus-provider/run.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/lotus-provider/run.go b/cmd/lotus-provider/run.go index 6248742cc83..2d1bc99cd21 100644 --- a/cmd/lotus-provider/run.go +++ b/cmd/lotus-provider/run.go @@ -208,7 +208,7 @@ var runCmd = &cli.Command{ return xerrors.Errorf(`'%w' while parsing the config toml's [Apis] StorageRPCSecret=%v -Get it from the JSON documents in ~/.lotus-miner/keystore called .PrivateKey`, err, cfg.Apis.StorageRPCSecret) +Get it with: jq .PrivateKey ~/.lotus-miner/keystore/MF2XI2BNNJ3XILLQOJUXMYLUMU`, err, cfg.Apis.StorageRPCSecret) } al := alerting.NewAlertingSystem(j) From 39473271bc02d7b3b67f90f47d92be3907c67b58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Wed, 15 Nov 2023 13:12:07 +0100 Subject: [PATCH 23/24] mod tidy --- go.sum | 1 - 1 file changed, 1 deletion(-) diff --git a/go.sum b/go.sum index aee413de4ff..fff87909239 100644 --- a/go.sum +++ b/go.sum @@ -344,7 +344,6 @@ github.com/filecoin-project/go-state-types v0.0.0-20201102161440-c8033295a1fc/go github.com/filecoin-project/go-state-types v0.1.0/go.mod h1:ezYnPf0bNkTsDibL/psSz5dy4B5awOJ/E7P2Saeep8g= github.com/filecoin-project/go-state-types v0.1.6/go.mod h1:UwGVoMsULoCK+bWjEdd/xLCvLAQFBC7EDT477SKml+Q= github.com/filecoin-project/go-state-types v0.1.10/go.mod h1:UwGVoMsULoCK+bWjEdd/xLCvLAQFBC7EDT477SKml+Q= -github.com/filecoin-project/go-state-types v0.11.1/go.mod h1:SyNPwTsU7I22gL2r0OAPcImvLoTVfgRwdK/Y5rR1zz8= github.com/filecoin-project/go-state-types v0.11.2-0.20230712101859-8f37624fa540/go.mod h1:SyNPwTsU7I22gL2r0OAPcImvLoTVfgRwdK/Y5rR1zz8= github.com/filecoin-project/go-state-types v0.12.5 h1:VQ2N2T3JeUDdIHEo/xhjnT7Q218Wl0UYIyglqT7Z9Ck= github.com/filecoin-project/go-state-types v0.12.5/go.mod h1:iJTqGdWDvzXhuVf64Lw0hzt4TIoitMo0VgHdxdjNDZI= From fceabdbd5ca275bcee014266772f14572e473143 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=C5=81ukasz=20Magiera?= Date: Wed, 15 Nov 2023 13:50:31 +0100 Subject: [PATCH 24/24] make gen --- cmd/lotus-provider/migrate.go | 6 +++--- documentation/en/default-lotus-miner-config.toml | 10 ++++++++++ node/config/doc_gen.go | 10 ++++++++++ 3 files changed, 23 insertions(+), 3 deletions(-) diff --git a/cmd/lotus-provider/migrate.go b/cmd/lotus-provider/migrate.go index d91c63d889e..674a537dfd3 100644 --- a/cmd/lotus-provider/migrate.go +++ b/cmd/lotus-provider/migrate.go @@ -10,14 +10,14 @@ import ( "path" "strings" - "github.com/filecoin-project/go-address" - "github.com/ipfs/go-datastore" - "github.com/BurntSushi/toml" + "github.com/ipfs/go-datastore" "github.com/samber/lo" "github.com/urfave/cli/v2" "golang.org/x/xerrors" + "github.com/filecoin-project/go-address" + cliutil "github.com/filecoin-project/lotus/cli/util" "github.com/filecoin-project/lotus/lib/harmony/harmonydb" "github.com/filecoin-project/lotus/node/config" diff --git a/documentation/en/default-lotus-miner-config.toml b/documentation/en/default-lotus-miner-config.toml index f7699e635e1..a65e82e9504 100644 --- a/documentation/en/default-lotus-miner-config.toml +++ b/documentation/en/default-lotus-miner-config.toml @@ -176,6 +176,16 @@ # env var: LOTUS_SUBSYSTEMS_DISABLEWINDOWPOST #DisableWindowPoSt = false + # When winning post is disabled, the miner process will NOT attempt to mine + # blocks. This should only be set when there's an external process mining + # blocks on behalf of the miner. + # When disabled and no external block producers are configured, all potential + # block rewards will be missed! + # + # type: bool + # env var: LOTUS_SUBSYSTEMS_DISABLEWINNINGPOST + #DisableWinningPoSt = false + [Dealmaking] # When enabled, the miner can accept online deals diff --git a/node/config/doc_gen.go b/node/config/doc_gen.go index fb6c9956c96..13ecb2706e3 100644 --- a/node/config/doc_gen.go +++ b/node/config/doc_gen.go @@ -977,6 +977,16 @@ This option will stop lotus-miner from performing any actions related to window post, including scheduling, submitting proofs, and recovering sectors.`, }, + { + Name: "DisableWinningPoSt", + Type: "bool", + + Comment: `When winning post is disabled, the miner process will NOT attempt to mine +blocks. This should only be set when there's an external process mining +blocks on behalf of the miner. +When disabled and no external block producers are configured, all potential +block rewards will be missed!`, + }, }, "ProviderSubsystemsConfig": { {