2022-06-14 17:27:04 +00:00
|
|
|
package wdpost
|
2019-11-28 17:44:49 +00:00
|
|
|
|
|
|
|
import (
|
2020-03-11 06:30:48 +00:00
|
|
|
"bytes"
|
2019-11-28 17:44:49 +00:00
|
|
|
"context"
|
2019-11-28 18:08:10 +00:00
|
|
|
"time"
|
|
|
|
|
2022-06-14 15:00:51 +00:00
|
|
|
"github.com/ipfs/go-cid"
|
|
|
|
"go.opencensus.io/trace"
|
|
|
|
"golang.org/x/xerrors"
|
2020-05-28 15:45:34 +00:00
|
|
|
|
2020-04-16 20:11:07 +00:00
|
|
|
"github.com/filecoin-project/go-address"
|
2022-06-14 15:00:51 +00:00
|
|
|
"github.com/filecoin-project/go-bitfield"
|
2020-09-07 03:49:10 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/abi"
|
|
|
|
"github.com/filecoin-project/go-state-types/big"
|
2022-06-14 15:00:51 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/builtin"
|
2022-09-06 15:49:29 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/builtin/v9/miner"
|
2020-09-07 03:49:10 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/crypto"
|
2020-09-12 03:07:52 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/dline"
|
2020-10-23 19:32:07 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/network"
|
2022-06-14 15:00:51 +00:00
|
|
|
"github.com/filecoin-project/go-state-types/proof"
|
2021-12-08 17:11:19 +00:00
|
|
|
proof7 "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof"
|
2020-09-21 22:24:45 +00:00
|
|
|
|
2020-08-17 13:26:18 +00:00
|
|
|
"github.com/filecoin-project/lotus/api"
|
2020-05-26 03:43:17 +00:00
|
|
|
"github.com/filecoin-project/lotus/build"
|
2019-11-28 17:44:49 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/actors"
|
2020-10-08 01:09:33 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/actors/policy"
|
2020-10-07 17:41:07 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/messagepool"
|
2019-11-28 17:44:49 +00:00
|
|
|
"github.com/filecoin-project/lotus/chain/types"
|
2022-06-17 11:31:05 +00:00
|
|
|
"github.com/filecoin-project/lotus/storage/sealer/storiface"
|
2019-11-28 17:44:49 +00:00
|
|
|
)
|
|
|
|
|
2021-05-14 20:00:13 +00:00
|
|
|
// recordPoStFailure records a failure in the journal.
|
|
|
|
func (s *WindowPoStScheduler) recordPoStFailure(err error, ts *types.TipSet, deadline *dline.Info) {
|
2020-10-09 19:52:04 +00:00
|
|
|
s.journal.RecordEvent(s.evtTypes[evtTypeWdPoStScheduler], func() interface{} {
|
2020-09-18 16:03:59 +00:00
|
|
|
c := evtCommon{Error: err}
|
|
|
|
if ts != nil {
|
|
|
|
c.Deadline = deadline
|
|
|
|
c.Height = ts.Height()
|
|
|
|
c.TipSet = ts.Cids()
|
|
|
|
}
|
2020-09-02 18:45:25 +00:00
|
|
|
return WdPoStSchedulerEvt{
|
2020-09-18 16:03:59 +00:00
|
|
|
evtCommon: c,
|
2020-09-02 18:45:25 +00:00
|
|
|
State: SchedulerStateFaulted,
|
|
|
|
}
|
2020-07-20 13:45:17 +00:00
|
|
|
})
|
2019-12-08 19:48:17 +00:00
|
|
|
}
|
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
// recordProofsEvent records a successful proofs_processed event in the
|
|
|
|
// journal, even if it was a noop (no partitions).
|
|
|
|
func (s *WindowPoStScheduler) recordProofsEvent(partitions []miner.PoStPartition, mcid cid.Cid) {
|
2020-10-09 19:52:04 +00:00
|
|
|
s.journal.RecordEvent(s.evtTypes[evtTypeWdPoStProofs], func() interface{} {
|
2020-09-18 16:03:59 +00:00
|
|
|
return &WdPoStProofsProcessedEvt{
|
|
|
|
evtCommon: s.getEvtCommon(nil),
|
|
|
|
Partitions: partitions,
|
|
|
|
MessageCID: mcid,
|
|
|
|
}
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// startGeneratePoST kicks off the process of generating a PoST
|
|
|
|
func (s *WindowPoStScheduler) startGeneratePoST(
|
|
|
|
ctx context.Context,
|
|
|
|
ts *types.TipSet,
|
|
|
|
deadline *dline.Info,
|
|
|
|
completeGeneratePoST CompleteGeneratePoSTCb,
|
|
|
|
) context.CancelFunc {
|
2019-11-28 17:44:49 +00:00
|
|
|
ctx, abort := context.WithCancel(ctx)
|
2020-09-18 16:03:59 +00:00
|
|
|
go func() {
|
|
|
|
defer abort()
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2020-10-09 19:52:04 +00:00
|
|
|
s.journal.RecordEvent(s.evtTypes[evtTypeWdPoStScheduler], func() interface{} {
|
2020-09-18 16:03:59 +00:00
|
|
|
return WdPoStSchedulerEvt{
|
|
|
|
evtCommon: s.getEvtCommon(nil),
|
|
|
|
State: SchedulerStateStarted,
|
|
|
|
}
|
|
|
|
})
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
posts, err := s.runGeneratePoST(ctx, ts, deadline)
|
|
|
|
completeGeneratePoST(posts, err)
|
|
|
|
}()
|
2020-07-20 13:45:17 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
return abort
|
|
|
|
}
|
|
|
|
|
|
|
|
// runGeneratePoST generates the PoST
|
|
|
|
func (s *WindowPoStScheduler) runGeneratePoST(
|
|
|
|
ctx context.Context,
|
|
|
|
ts *types.TipSet,
|
|
|
|
deadline *dline.Info,
|
|
|
|
) ([]miner.SubmitWindowedPoStParams, error) {
|
|
|
|
ctx, span := trace.StartSpan(ctx, "WindowPoStScheduler.generatePoST")
|
|
|
|
defer span.End()
|
|
|
|
|
2022-03-28 20:54:22 +00:00
|
|
|
posts, err := s.runPoStCycle(ctx, false, *deadline, ts)
|
2020-09-18 16:03:59 +00:00
|
|
|
if err != nil {
|
2021-05-14 20:00:13 +00:00
|
|
|
log.Errorf("runPoStCycle failed: %+v", err)
|
2020-09-18 16:03:59 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
if len(posts) == 0 {
|
|
|
|
s.recordProofsEvent(nil, cid.Undef)
|
|
|
|
}
|
|
|
|
|
|
|
|
return posts, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// startSubmitPoST kicks of the process of submitting PoST
|
|
|
|
func (s *WindowPoStScheduler) startSubmitPoST(
|
|
|
|
ctx context.Context,
|
|
|
|
ts *types.TipSet,
|
|
|
|
deadline *dline.Info,
|
|
|
|
posts []miner.SubmitWindowedPoStParams,
|
|
|
|
completeSubmitPoST CompleteSubmitPoSTCb,
|
|
|
|
) context.CancelFunc {
|
|
|
|
|
|
|
|
ctx, abort := context.WithCancel(ctx)
|
2019-11-28 17:44:49 +00:00
|
|
|
go func() {
|
|
|
|
defer abort()
|
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
err := s.runSubmitPoST(ctx, ts, deadline, posts)
|
|
|
|
if err == nil {
|
2020-10-09 19:52:04 +00:00
|
|
|
s.journal.RecordEvent(s.evtTypes[evtTypeWdPoStScheduler], func() interface{} {
|
2020-09-18 16:03:59 +00:00
|
|
|
return WdPoStSchedulerEvt{
|
|
|
|
evtCommon: s.getEvtCommon(nil),
|
|
|
|
State: SchedulerStateSucceeded,
|
2020-09-02 18:45:25 +00:00
|
|
|
}
|
2020-08-11 15:30:54 +00:00
|
|
|
})
|
|
|
|
}
|
2020-09-18 16:03:59 +00:00
|
|
|
completeSubmitPoST(err)
|
|
|
|
}()
|
2020-08-11 15:30:54 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
return abort
|
|
|
|
}
|
2020-09-09 13:01:37 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
// runSubmitPoST submits PoST
|
|
|
|
func (s *WindowPoStScheduler) runSubmitPoST(
|
|
|
|
ctx context.Context,
|
|
|
|
ts *types.TipSet,
|
|
|
|
deadline *dline.Info,
|
|
|
|
posts []miner.SubmitWindowedPoStParams,
|
|
|
|
) error {
|
|
|
|
if len(posts) == 0 {
|
|
|
|
return nil
|
|
|
|
}
|
2020-09-09 13:01:37 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
ctx, span := trace.StartSpan(ctx, "WindowPoStScheduler.submitPoST")
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
// Get randomness from tickets
|
2020-10-23 19:32:07 +00:00
|
|
|
// use the challenge epoch if we've upgraded to network version 4
|
|
|
|
// (actors version 2). We want to go back as far as possible to be safe.
|
2020-09-18 16:03:59 +00:00
|
|
|
commEpoch := deadline.Open
|
2020-10-23 19:32:07 +00:00
|
|
|
if ver, err := s.api.StateNetworkVersion(ctx, types.EmptyTSK); err != nil {
|
|
|
|
log.Errorw("failed to get network version to determine PoSt epoch randomness lookback", "error", err)
|
|
|
|
} else if ver >= network.Version4 {
|
|
|
|
commEpoch = deadline.Challenge
|
|
|
|
}
|
|
|
|
|
2021-09-12 02:24:53 +00:00
|
|
|
commRand, err := s.api.StateGetRandomnessFromTickets(ctx, crypto.DomainSeparationTag_PoStChainCommit, commEpoch, nil, ts.Key())
|
2020-09-18 16:03:59 +00:00
|
|
|
if err != nil {
|
|
|
|
err = xerrors.Errorf("failed to get chain randomness from tickets for windowPost (ts=%d; deadline=%d): %w", ts.Height(), commEpoch, err)
|
2021-05-14 18:48:38 +00:00
|
|
|
log.Errorf("submitPoStMessage failed: %+v", err)
|
2020-09-18 16:03:59 +00:00
|
|
|
|
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
|
|
|
var submitErr error
|
|
|
|
for i := range posts {
|
|
|
|
// Add randomness to PoST
|
|
|
|
post := &posts[i]
|
|
|
|
post.ChainCommitEpoch = commEpoch
|
|
|
|
post.ChainCommitRand = commRand
|
|
|
|
|
|
|
|
// Submit PoST
|
2021-12-29 09:37:37 +00:00
|
|
|
sm, err := s.submitPoStMessage(ctx, post)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("submit window post failed: %+v", err)
|
|
|
|
submitErr = err
|
2020-09-18 16:03:59 +00:00
|
|
|
} else {
|
|
|
|
s.recordProofsEvent(post.Partitions, sm.Cid())
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
2020-09-18 16:03:59 +00:00
|
|
|
}
|
2020-07-20 13:45:17 +00:00
|
|
|
|
2020-09-18 16:03:59 +00:00
|
|
|
return submitErr
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
2020-11-05 05:52:51 +00:00
|
|
|
func (s *WindowPoStScheduler) checkSectors(ctx context.Context, check bitfield.BitField, tsk types.TipSetKey) (bitfield.BitField, error) {
|
2020-05-16 21:50:50 +00:00
|
|
|
mid, err := address.IDFromAddress(s.actor)
|
|
|
|
if err != nil {
|
2020-08-05 07:37:28 +00:00
|
|
|
return bitfield.BitField{}, err
|
2020-05-16 21:50:50 +00:00
|
|
|
}
|
|
|
|
|
2020-11-05 05:52:51 +00:00
|
|
|
sectorInfos, err := s.api.StateMinerSectors(ctx, s.actor, &check, tsk)
|
|
|
|
if err != nil {
|
|
|
|
return bitfield.BitField{}, err
|
|
|
|
}
|
2020-05-16 21:50:50 +00:00
|
|
|
|
2022-01-18 10:25:04 +00:00
|
|
|
type checkSector struct {
|
|
|
|
sealed cid.Cid
|
|
|
|
update bool
|
|
|
|
}
|
|
|
|
|
|
|
|
sectors := make(map[abi.SectorNumber]checkSector)
|
2022-06-17 11:31:05 +00:00
|
|
|
var tocheck []storiface.SectorRef
|
2020-11-05 05:52:51 +00:00
|
|
|
for _, info := range sectorInfos {
|
2022-01-18 10:25:04 +00:00
|
|
|
sectors[info.SectorNumber] = checkSector{
|
|
|
|
sealed: info.SealedCID,
|
|
|
|
update: info.SectorKeyCID != nil,
|
|
|
|
}
|
2022-06-17 11:31:05 +00:00
|
|
|
tocheck = append(tocheck, storiface.SectorRef{
|
2020-11-05 05:52:51 +00:00
|
|
|
ProofType: info.SealProof,
|
|
|
|
ID: abi.SectorID{
|
|
|
|
Miner: abi.ActorID(mid),
|
|
|
|
Number: info.SectorNumber,
|
|
|
|
},
|
|
|
|
})
|
|
|
|
}
|
2020-05-16 21:50:50 +00:00
|
|
|
|
2022-01-18 10:25:04 +00:00
|
|
|
bad, err := s.faultTracker.CheckProvable(ctx, s.proofType, tocheck, func(ctx context.Context, id abi.SectorID) (cid.Cid, bool, error) {
|
|
|
|
s, ok := sectors[id.Number]
|
2022-01-14 13:11:04 +00:00
|
|
|
if !ok {
|
2022-01-18 10:25:04 +00:00
|
|
|
return cid.Undef, false, xerrors.Errorf("sealed CID not found")
|
2021-07-27 03:15:53 +00:00
|
|
|
}
|
2022-01-18 10:25:04 +00:00
|
|
|
return s.sealed, s.update, nil
|
2021-07-27 03:15:53 +00:00
|
|
|
})
|
2020-05-16 21:50:50 +00:00
|
|
|
if err != nil {
|
2020-08-05 07:37:28 +00:00
|
|
|
return bitfield.BitField{}, xerrors.Errorf("checking provable sectors: %w", err)
|
2020-05-16 21:50:50 +00:00
|
|
|
}
|
2020-11-26 07:02:43 +00:00
|
|
|
for id := range bad {
|
2020-11-05 05:52:51 +00:00
|
|
|
delete(sectors, id.Number)
|
2020-05-16 21:50:50 +00:00
|
|
|
}
|
|
|
|
|
2020-05-29 19:36:04 +00:00
|
|
|
log.Warnw("Checked sectors", "checked", len(tocheck), "good", len(sectors))
|
2020-05-16 21:50:50 +00:00
|
|
|
|
|
|
|
sbf := bitfield.New()
|
|
|
|
for s := range sectors {
|
2020-11-05 05:52:51 +00:00
|
|
|
sbf.Set(uint64(s))
|
2020-05-16 21:50:50 +00:00
|
|
|
}
|
|
|
|
|
2020-08-05 07:37:28 +00:00
|
|
|
return sbf, nil
|
2020-05-29 19:36:04 +00:00
|
|
|
}
|
|
|
|
|
2022-03-28 20:54:22 +00:00
|
|
|
// runPoStCycle runs a full cycle of the PoSt process:
|
|
|
|
//
|
|
|
|
// 1. performs recovery declarations for the next deadline.
|
|
|
|
// 2. performs fault declarations for the next deadline.
|
|
|
|
// 3. computes and submits proofs, batching partitions and making sure they
|
|
|
|
// don't exceed message capacity.
|
|
|
|
//
|
|
|
|
// When `manual` is set, no messages (fault/recover) will be automatically sent
|
|
|
|
func (s *WindowPoStScheduler) runPoStCycle(ctx context.Context, manual bool, di dline.Info, ts *types.TipSet) ([]miner.SubmitWindowedPoStParams, error) {
|
|
|
|
ctx, span := trace.StartSpan(ctx, "storage.runPoStCycle")
|
|
|
|
defer span.End()
|
|
|
|
|
|
|
|
if !manual {
|
|
|
|
// TODO: extract from runPoStCycle, run on fault cutoff boundaries
|
|
|
|
s.asyncFaultRecover(di, ts)
|
|
|
|
}
|
2020-05-16 21:50:50 +00:00
|
|
|
|
2020-03-11 06:30:48 +00:00
|
|
|
buf := new(bytes.Buffer)
|
|
|
|
if err := s.actor.MarshalCBOR(buf); err != nil {
|
|
|
|
return nil, xerrors.Errorf("failed to marshal address to cbor: %w", err)
|
|
|
|
}
|
2020-08-12 17:29:35 +00:00
|
|
|
|
2021-03-11 08:59:26 +00:00
|
|
|
headTs, err := s.api.ChainHead(ctx)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting current head: %w", err)
|
|
|
|
}
|
|
|
|
|
2021-09-12 02:24:53 +00:00
|
|
|
rand, err := s.api.StateGetRandomnessFromBeacon(ctx, crypto.DomainSeparationTag_WindowedPoStChallengeSeed, di.Challenge, buf.Bytes(), headTs.Key())
|
2020-08-11 23:58:35 +00:00
|
|
|
if err != nil {
|
2020-09-18 16:03:59 +00:00
|
|
|
return nil, xerrors.Errorf("failed to get chain randomness from beacon for window post (ts=%d; deadline=%d): %w", ts.Height(), di, err)
|
2020-08-11 23:58:35 +00:00
|
|
|
}
|
|
|
|
|
2020-09-09 13:01:37 +00:00
|
|
|
// Get the partitions for the given deadline
|
2020-09-18 22:40:49 +00:00
|
|
|
partitions, err := s.api.StateMinerPartitions(ctx, s.actor, di.Index, ts.Key())
|
2020-04-16 20:11:07 +00:00
|
|
|
if err != nil {
|
2020-09-18 22:40:49 +00:00
|
|
|
return nil, xerrors.Errorf("getting partitions: %w", err)
|
2020-04-16 20:11:07 +00:00
|
|
|
}
|
|
|
|
|
2021-05-31 23:28:49 +00:00
|
|
|
nv, err := s.api.StateNetworkVersion(ctx, ts.Key())
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting network version: %w", err)
|
|
|
|
}
|
|
|
|
|
2020-09-09 13:01:37 +00:00
|
|
|
// Split partitions into batches, so as not to exceed the number of sectors
|
|
|
|
// allowed in a single message
|
2022-10-04 18:33:18 +00:00
|
|
|
partitionBatches, err := s.BatchPartitions(partitions, nv)
|
2020-09-09 13:01:37 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
2020-04-16 20:11:07 +00:00
|
|
|
}
|
|
|
|
|
2022-01-20 10:19:57 +00:00
|
|
|
defer func() {
|
|
|
|
if r := recover(); r != nil {
|
|
|
|
log.Errorf("recover: %s", r)
|
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
2020-09-10 08:21:07 +00:00
|
|
|
// Generate proofs in batches
|
2022-01-14 13:11:04 +00:00
|
|
|
posts := make([]miner.SubmitWindowedPoStParams, 0, len(partitionBatches))
|
2020-09-09 13:01:37 +00:00
|
|
|
for batchIdx, batch := range partitionBatches {
|
|
|
|
batchPartitionStartIdx := 0
|
|
|
|
for _, batch := range partitionBatches[:batchIdx] {
|
|
|
|
batchPartitionStartIdx += len(batch)
|
|
|
|
}
|
2020-09-10 00:59:37 +00:00
|
|
|
|
2020-09-09 13:01:37 +00:00
|
|
|
params := miner.SubmitWindowedPoStParams{
|
|
|
|
Deadline: di.Index,
|
|
|
|
Partitions: make([]miner.PoStPartition, 0, len(batch)),
|
|
|
|
Proofs: nil,
|
|
|
|
}
|
2020-09-10 00:59:37 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
postSkipped := bitfield.New()
|
|
|
|
somethingToProve := false
|
|
|
|
|
|
|
|
// Retry until we run out of sectors to prove.
|
|
|
|
for retries := 0; ; retries++ {
|
|
|
|
skipCount := uint64(0)
|
|
|
|
var partitions []miner.PoStPartition
|
2021-12-08 17:11:19 +00:00
|
|
|
var xsinfos []proof7.ExtendedSectorInfo
|
2022-01-14 13:11:04 +00:00
|
|
|
for partIdx, partition := range batch {
|
|
|
|
// TODO: Can do this in parallel
|
|
|
|
toProve, err := bitfield.SubtractBitField(partition.LiveSectors, partition.FaultySectors)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("removing faults from set of sectors to prove: %w", err)
|
|
|
|
}
|
2022-05-27 11:14:47 +00:00
|
|
|
if manual {
|
|
|
|
// this is a check run, we want to prove faulty sectors, even
|
|
|
|
// if they are not declared as recovering.
|
|
|
|
toProve = partition.LiveSectors
|
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
toProve, err = bitfield.MergeBitFields(toProve, partition.RecoveringSectors)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("adding recoveries to set of sectors to prove: %w", err)
|
|
|
|
}
|
2020-04-16 20:11:07 +00:00
|
|
|
|
2022-07-01 20:20:05 +00:00
|
|
|
good, err := toProve.Copy()
|
2022-01-14 13:11:04 +00:00
|
|
|
if err != nil {
|
2022-07-01 20:20:05 +00:00
|
|
|
return nil, xerrors.Errorf("copy toProve: %w", err)
|
|
|
|
}
|
|
|
|
if !s.disablePreChecks {
|
|
|
|
good, err = s.checkSectors(ctx, toProve, ts.Key())
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("checking sectors to skip: %w", err)
|
|
|
|
}
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2020-04-16 20:11:07 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
good, err = bitfield.SubtractBitField(good, postSkipped)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("toProve - postSkipped: %w", err)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2020-05-26 09:58:20 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
skipped, err := bitfield.SubtractBitField(toProve, good)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("toProve - good: %w", err)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2020-05-29 19:36:04 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
sc, err := skipped.Count()
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting skipped sector count: %w", err)
|
|
|
|
}
|
2020-05-29 19:36:04 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
skipCount += sc
|
2020-09-17 02:34:13 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
ssi, err := s.sectorsForProof(ctx, good, partition.AllSectors, ts)
|
2020-09-09 13:01:37 +00:00
|
|
|
if err != nil {
|
2022-01-14 13:11:04 +00:00
|
|
|
return nil, xerrors.Errorf("getting sorted sector info: %w", err)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
if len(ssi) == 0 {
|
|
|
|
continue
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
xsinfos = append(xsinfos, ssi...)
|
2022-01-14 13:11:04 +00:00
|
|
|
partitions = append(partitions, miner.PoStPartition{
|
|
|
|
Index: uint64(batchPartitionStartIdx + partIdx),
|
|
|
|
Skipped: skipped,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
if len(xsinfos) == 0 {
|
2022-01-14 13:11:04 +00:00
|
|
|
// nothing to prove for this batch
|
|
|
|
break
|
|
|
|
}
|
|
|
|
|
|
|
|
// Generate proof
|
|
|
|
log.Infow("running window post",
|
|
|
|
"chain-random", rand,
|
|
|
|
"deadline", di,
|
|
|
|
"height", ts.Height(),
|
|
|
|
"skipped", skipCount)
|
|
|
|
|
|
|
|
tsStart := build.Clock.Now()
|
|
|
|
|
|
|
|
mid, err := address.IDFromAddress(s.actor)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
postOut, ps, err := s.prover.GenerateWindowPoSt(ctx, abi.ActorID(mid), xsinfos, append(abi.PoStRandomness{}, rand...))
|
2022-01-14 13:11:04 +00:00
|
|
|
elapsed := time.Since(tsStart)
|
2022-03-18 18:53:59 +00:00
|
|
|
log.Infow("computing window post", "batch", batchIdx, "elapsed", elapsed, "skip", len(ps), "err", err)
|
2021-12-08 17:11:19 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Errorf("error generating window post: %s", err)
|
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
if err == nil {
|
2021-12-08 17:11:19 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
// If we proved nothing, something is very wrong.
|
|
|
|
if len(postOut) == 0 {
|
2021-12-08 17:11:19 +00:00
|
|
|
log.Errorf("len(postOut) == 0")
|
2022-01-14 13:11:04 +00:00
|
|
|
return nil, xerrors.Errorf("received no proofs back from generate window post")
|
|
|
|
}
|
|
|
|
|
|
|
|
headTs, err := s.api.ChainHead(ctx)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting current head: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
checkRand, err := s.api.StateGetRandomnessFromBeacon(ctx, crypto.DomainSeparationTag_WindowedPoStChallengeSeed, di.Challenge, buf.Bytes(), headTs.Key())
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("failed to get chain randomness from beacon for window post (ts=%d; deadline=%d): %w", ts.Height(), di, err)
|
2021-03-10 12:13:03 +00:00
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
|
|
|
|
if !bytes.Equal(checkRand, rand) {
|
|
|
|
log.Warnw("windowpost randomness changed", "old", rand, "new", checkRand, "ts-height", ts.Height(), "challenge-height", di.Challenge, "tsk", ts.Key())
|
|
|
|
rand = checkRand
|
|
|
|
continue
|
2021-03-10 08:19:38 +00:00
|
|
|
}
|
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
// If we generated an incorrect proof, try again.
|
2021-12-08 17:11:19 +00:00
|
|
|
sinfos := make([]proof7.SectorInfo, len(xsinfos))
|
|
|
|
for i, xsi := range xsinfos {
|
|
|
|
sinfos[i] = proof7.SectorInfo{
|
|
|
|
SealProof: xsi.SealProof,
|
|
|
|
SectorNumber: xsi.SectorNumber,
|
|
|
|
SealedCID: xsi.SealedCID,
|
|
|
|
}
|
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
if correct, err := s.verifier.VerifyWindowPoSt(ctx, proof.WindowPoStVerifyInfo{
|
|
|
|
Randomness: abi.PoStRandomness(checkRand),
|
|
|
|
Proofs: postOut,
|
|
|
|
ChallengedSectors: sinfos,
|
|
|
|
Prover: abi.ActorID(mid),
|
|
|
|
}); err != nil {
|
|
|
|
log.Errorw("window post verification failed", "post", postOut, "error", err)
|
|
|
|
time.Sleep(5 * time.Second)
|
|
|
|
continue
|
|
|
|
} else if !correct {
|
|
|
|
log.Errorw("generated incorrect window post proof", "post", postOut, "error", err)
|
|
|
|
continue
|
2021-03-10 08:19:38 +00:00
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
|
|
|
|
// Proof generation successful, stop retrying
|
|
|
|
somethingToProve = true
|
|
|
|
params.Partitions = partitions
|
|
|
|
params.Proofs = postOut
|
|
|
|
break
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2020-09-10 00:59:37 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
// Proof generation failed, so retry
|
2021-12-08 17:11:19 +00:00
|
|
|
log.Debugf("Proof generation failed, retry")
|
2022-01-14 13:11:04 +00:00
|
|
|
if len(ps) == 0 {
|
|
|
|
// If we didn't skip any new sectors, we failed
|
|
|
|
// for some other reason and we need to abort.
|
|
|
|
return nil, xerrors.Errorf("running window post failed: %w", err)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2022-01-14 13:11:04 +00:00
|
|
|
// TODO: maybe mark these as faulty somewhere?
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2022-01-14 13:11:04 +00:00
|
|
|
log.Warnw("generate window post skipped sectors", "sectors", ps, "error", err, "try", retries)
|
|
|
|
|
|
|
|
// Explicitly make sure we haven't aborted this PoSt
|
|
|
|
// (GenerateWindowPoSt may or may not check this).
|
|
|
|
// Otherwise, we could try to continue proving a
|
|
|
|
// deadline after the deadline has ended.
|
|
|
|
if ctx.Err() != nil {
|
|
|
|
log.Warnw("aborting PoSt due to context cancellation", "error", ctx.Err(), "deadline", di.Index)
|
|
|
|
return nil, ctx.Err()
|
|
|
|
}
|
|
|
|
|
|
|
|
for _, sector := range ps {
|
|
|
|
postSkipped.Set(uint64(sector.Number))
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Nothing to prove for this batch, try the next batch
|
|
|
|
if !somethingToProve {
|
|
|
|
continue
|
|
|
|
}
|
|
|
|
posts = append(posts, params)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
|
|
|
return posts, nil
|
|
|
|
}
|
|
|
|
|
2022-10-04 18:33:18 +00:00
|
|
|
// Note: Partition order within batches must match original partition order in order
|
|
|
|
// for code following the user code to work
|
|
|
|
func (s *WindowPoStScheduler) BatchPartitions(partitions []api.Partition, nv network.Version) ([][]api.Partition, error) {
|
2020-09-09 13:01:37 +00:00
|
|
|
// We don't want to exceed the number of sectors allowed in a message.
|
|
|
|
// So given the number of sectors in a partition, work out the number of
|
|
|
|
// partitions that can be in a message without exceeding sectors per
|
|
|
|
// message:
|
|
|
|
// floor(number of sectors allowed in a message / sectors per partition)
|
|
|
|
// eg:
|
|
|
|
// max sectors per message 7: ooooooo
|
|
|
|
// sectors per partition 3: ooo
|
|
|
|
// partitions per message 2: oooOOO
|
|
|
|
// <1><2> (3rd doesn't fit)
|
2021-06-16 23:39:23 +00:00
|
|
|
partitionsPerMsg, err := policy.GetMaxPoStPartitions(nv, s.proofType)
|
2020-10-08 01:09:33 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting sectors per partition: %w", err)
|
|
|
|
}
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2021-05-31 23:28:49 +00:00
|
|
|
// Also respect the AddressedPartitionsMax (which is the same as DeclarationsMax (which is all really just MaxPartitionsPerDeadline))
|
2021-08-10 17:07:30 +00:00
|
|
|
declMax, err := policy.GetDeclarationsMax(nv)
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting max declarations: %w", err)
|
|
|
|
}
|
|
|
|
if partitionsPerMsg > declMax {
|
|
|
|
partitionsPerMsg = declMax
|
2021-05-31 23:28:49 +00:00
|
|
|
}
|
|
|
|
|
2022-07-07 10:33:40 +00:00
|
|
|
// respect user config if set
|
2022-07-07 14:52:22 +00:00
|
|
|
if s.maxPartitionsPerPostMessage > 0 {
|
|
|
|
if partitionsPerMsg > s.maxPartitionsPerPostMessage {
|
|
|
|
partitionsPerMsg = s.maxPartitionsPerPostMessage
|
2022-07-07 10:33:40 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2022-10-04 18:33:18 +00:00
|
|
|
batches := [][]api.Partition{}
|
|
|
|
|
|
|
|
currBatch := []api.Partition{}
|
2022-10-04 19:21:55 +00:00
|
|
|
for _, partition := range partitions {
|
|
|
|
recSectors, err := partition.RecoveringSectors.Count()
|
2022-10-04 18:33:18 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2020-09-09 13:01:37 +00:00
|
|
|
|
2022-10-04 18:33:18 +00:00
|
|
|
// Only add single partition to a batch if it contains recovery sectors
|
|
|
|
// and has the below user config set
|
|
|
|
if s.singleRecoveringPartitionPerPostMessage && recSectors > 0 {
|
|
|
|
if len(currBatch) > 0 {
|
|
|
|
batches = append(batches, currBatch)
|
|
|
|
currBatch = []api.Partition{}
|
|
|
|
}
|
2022-10-04 19:21:55 +00:00
|
|
|
batches = append(batches, []api.Partition{partition})
|
2022-10-04 18:33:18 +00:00
|
|
|
} else {
|
|
|
|
if len(currBatch) >= partitionsPerMsg {
|
|
|
|
batches = append(batches, currBatch)
|
|
|
|
currBatch = []api.Partition{}
|
|
|
|
}
|
2022-10-04 19:21:55 +00:00
|
|
|
currBatch = append(currBatch, partition)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2022-10-04 18:33:18 +00:00
|
|
|
}
|
|
|
|
if len(currBatch) > 0 {
|
|
|
|
batches = append(batches, currBatch)
|
2020-09-09 13:01:37 +00:00
|
|
|
}
|
2020-09-18 16:03:59 +00:00
|
|
|
|
2020-09-09 13:01:37 +00:00
|
|
|
return batches, nil
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
func (s *WindowPoStScheduler) sectorsForProof(ctx context.Context, goodSectors, allSectors bitfield.BitField, ts *types.TipSet) ([]proof7.ExtendedSectorInfo, error) {
|
2020-09-21 19:05:01 +00:00
|
|
|
sset, err := s.api.StateMinerSectors(ctx, s.actor, &goodSectors, ts.Key())
|
2019-11-28 17:44:49 +00:00
|
|
|
if err != nil {
|
2020-04-07 19:55:34 +00:00
|
|
|
return nil, err
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
2020-08-26 02:53:21 +00:00
|
|
|
if len(sset) == 0 {
|
|
|
|
return nil, nil
|
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
substitute := proof7.ExtendedSectorInfo{
|
2020-09-21 19:05:01 +00:00
|
|
|
SectorNumber: sset[0].SectorNumber,
|
|
|
|
SealedCID: sset[0].SealedCID,
|
|
|
|
SealProof: sset[0].SealProof,
|
2021-12-08 17:11:19 +00:00
|
|
|
SectorKey: sset[0].SectorKeyCID,
|
2020-08-26 02:53:21 +00:00
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
sectorByID := make(map[uint64]proof7.ExtendedSectorInfo, len(sset))
|
2020-08-26 02:53:21 +00:00
|
|
|
for _, sector := range sset {
|
2021-12-08 17:11:19 +00:00
|
|
|
sectorByID[uint64(sector.SectorNumber)] = proof7.ExtendedSectorInfo{
|
2020-09-21 19:05:01 +00:00
|
|
|
SectorNumber: sector.SectorNumber,
|
|
|
|
SealedCID: sector.SealedCID,
|
|
|
|
SealProof: sector.SealProof,
|
2021-12-08 17:11:19 +00:00
|
|
|
SectorKey: sector.SectorKeyCID,
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-12-08 17:11:19 +00:00
|
|
|
proofSectors := make([]proof7.ExtendedSectorInfo, 0, len(sset))
|
2020-08-26 02:53:21 +00:00
|
|
|
if err := allSectors.ForEach(func(sectorNo uint64) error {
|
|
|
|
if info, found := sectorByID[sectorNo]; found {
|
|
|
|
proofSectors = append(proofSectors, info)
|
|
|
|
} else {
|
|
|
|
proofSectors = append(proofSectors, substitute)
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}); err != nil {
|
|
|
|
return nil, xerrors.Errorf("iterating partition sector bitmap: %w", err)
|
|
|
|
}
|
|
|
|
|
|
|
|
return proofSectors, nil
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
2021-05-14 18:48:38 +00:00
|
|
|
// submitPoStMessage builds a SubmitWindowedPoSt message and submits it to
|
|
|
|
// the mpool. It doesn't synchronously block on confirmations, but it does
|
|
|
|
// monitor in the background simply for the purposes of logging.
|
|
|
|
func (s *WindowPoStScheduler) submitPoStMessage(ctx context.Context, proof *miner.SubmitWindowedPoStParams) (*types.SignedMessage, error) {
|
2019-11-28 17:44:49 +00:00
|
|
|
ctx, span := trace.StartSpan(ctx, "storage.commitPost")
|
|
|
|
defer span.End()
|
|
|
|
|
2020-07-20 13:45:17 +00:00
|
|
|
var sm *types.SignedMessage
|
|
|
|
|
2019-11-28 17:44:49 +00:00
|
|
|
enc, aerr := actors.SerializeParams(proof)
|
|
|
|
if aerr != nil {
|
2020-09-16 01:19:27 +00:00
|
|
|
return nil, xerrors.Errorf("could not serialize submit window post parameters: %w", aerr)
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
msg := &types.Message{
|
2020-08-01 14:23:13 +00:00
|
|
|
To: s.actor,
|
2022-04-20 21:34:28 +00:00
|
|
|
Method: builtin.MethodsMiner.SubmitWindowedPoSt,
|
2020-08-01 14:23:13 +00:00
|
|
|
Params: enc,
|
2020-09-09 18:34:55 +00:00
|
|
|
Value: types.NewInt(0),
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
2020-08-19 21:25:58 +00:00
|
|
|
spec := &api.MessageSendSpec{MaxFee: abi.TokenAmount(s.feeCfg.MaxWindowPoStGasFee)}
|
2021-05-14 18:49:05 +00:00
|
|
|
if err := s.prepareMessage(ctx, msg, spec); err != nil {
|
2020-10-21 03:35:18 +00:00
|
|
|
return nil, err
|
|
|
|
}
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2020-08-19 21:25:58 +00:00
|
|
|
sm, err := s.api.MpoolPushMessage(ctx, msg, spec)
|
2019-11-28 17:44:49 +00:00
|
|
|
if err != nil {
|
2020-08-11 15:30:54 +00:00
|
|
|
return nil, xerrors.Errorf("pushing message to mpool: %w", err)
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
|
|
|
|
2022-01-20 13:59:57 +00:00
|
|
|
log.Infof("Submitted window post: %s (deadline %d)", sm.Cid(), proof.Deadline)
|
2019-11-28 17:44:49 +00:00
|
|
|
|
2020-01-29 22:47:28 +00:00
|
|
|
go func() {
|
2021-04-05 17:56:53 +00:00
|
|
|
rec, err := s.api.StateWaitMsg(context.TODO(), sm.Cid(), build.MessageConfidence, api.LookbackNoLimit, true)
|
2020-01-29 22:47:28 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Error(err)
|
|
|
|
return
|
|
|
|
}
|
|
|
|
|
|
|
|
if rec.Receipt.ExitCode == 0 {
|
2022-01-20 13:59:57 +00:00
|
|
|
log.Infow("Window post submission successful", "cid", sm.Cid(), "deadline", proof.Deadline, "epoch", rec.Height, "ts", rec.TipSet.Cids())
|
2020-01-29 22:47:28 +00:00
|
|
|
return
|
|
|
|
}
|
|
|
|
|
2020-04-21 17:22:53 +00:00
|
|
|
log.Errorf("Submitting window post %s failed: exit %d", sm.Cid(), rec.Receipt.ExitCode)
|
2020-01-29 22:47:28 +00:00
|
|
|
}()
|
|
|
|
|
2020-08-11 15:30:54 +00:00
|
|
|
return sm, nil
|
2019-11-28 17:44:49 +00:00
|
|
|
}
|
2020-08-19 20:08:04 +00:00
|
|
|
|
2021-05-14 18:49:05 +00:00
|
|
|
// prepareMessage prepares a message before sending it, setting:
|
|
|
|
//
|
|
|
|
// * the sender (from the AddressSelector, falling back to the worker address if none set)
|
|
|
|
// * the right gas parameters
|
|
|
|
func (s *WindowPoStScheduler) prepareMessage(ctx context.Context, msg *types.Message, spec *api.MessageSendSpec) error {
|
2020-08-19 20:08:04 +00:00
|
|
|
mi, err := s.api.StateMinerInfo(ctx, s.actor, types.EmptyTSK)
|
|
|
|
if err != nil {
|
2020-10-21 03:35:18 +00:00
|
|
|
return xerrors.Errorf("error getting miner info: %w", err)
|
2020-10-26 06:52:42 +00:00
|
|
|
}
|
2021-05-14 18:49:05 +00:00
|
|
|
// set the worker as a fallback
|
2020-10-21 03:35:18 +00:00
|
|
|
msg.From = mi.Worker
|
2020-08-19 20:08:04 +00:00
|
|
|
|
2021-05-14 18:49:05 +00:00
|
|
|
// (optimal) initial estimation with some overestimation that guarantees
|
|
|
|
// block inclusion within the next 20 tipsets.
|
2020-08-19 21:25:58 +00:00
|
|
|
gm, err := s.api.GasEstimateMessageGas(ctx, msg, spec, types.EmptyTSK)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorw("estimating gas", "error", err)
|
2020-10-21 03:35:18 +00:00
|
|
|
return nil
|
2020-08-19 21:25:58 +00:00
|
|
|
}
|
|
|
|
*msg = *gm
|
|
|
|
|
2021-05-14 18:45:47 +00:00
|
|
|
// calculate a more frugal estimation; premium is estimated to guarantee
|
|
|
|
// inclusion within 5 tipsets, and fee cap is estimated for inclusion
|
|
|
|
// within 4 tipsets.
|
2020-10-07 17:41:07 +00:00
|
|
|
minGasFeeMsg := *msg
|
2020-08-19 20:08:04 +00:00
|
|
|
|
2021-04-29 04:56:16 +00:00
|
|
|
minGasFeeMsg.GasPremium, err = s.api.GasEstimateGasPremium(ctx, 5, msg.From, msg.GasLimit, types.EmptyTSK)
|
2020-10-07 17:41:07 +00:00
|
|
|
if err != nil {
|
|
|
|
log.Errorf("failed to estimate minimum gas premium: %+v", err)
|
|
|
|
minGasFeeMsg.GasPremium = msg.GasPremium
|
|
|
|
}
|
|
|
|
|
|
|
|
minGasFeeMsg.GasFeeCap, err = s.api.GasEstimateFeeCap(ctx, &minGasFeeMsg, 4, types.EmptyTSK)
|
|
|
|
if err != nil {
|
|
|
|
log.Errorf("failed to estimate minimum gas fee cap: %+v", err)
|
|
|
|
minGasFeeMsg.GasFeeCap = msg.GasFeeCap
|
|
|
|
}
|
|
|
|
|
2021-05-14 18:45:47 +00:00
|
|
|
// goodFunds = funds needed for optimal inclusion probability.
|
|
|
|
// minFunds = funds needed for more speculative inclusion probability.
|
2020-10-07 17:41:07 +00:00
|
|
|
goodFunds := big.Add(msg.RequiredFunds(), msg.Value)
|
|
|
|
minFunds := big.Min(big.Add(minGasFeeMsg.RequiredFunds(), minGasFeeMsg.Value), goodFunds)
|
|
|
|
|
2020-12-02 20:47:45 +00:00
|
|
|
pa, avail, err := s.addrSel.AddressFor(ctx, s.api, mi, api.PoStAddr, goodFunds, minFunds)
|
2020-08-19 20:08:04 +00:00
|
|
|
if err != nil {
|
2020-09-16 01:19:27 +00:00
|
|
|
log.Errorw("error selecting address for window post", "error", err)
|
2020-10-21 03:35:18 +00:00
|
|
|
return nil
|
2020-08-19 20:08:04 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
msg.From = pa
|
2020-10-07 17:41:07 +00:00
|
|
|
bestReq := big.Add(msg.RequiredFunds(), msg.Value)
|
|
|
|
if avail.LessThan(bestReq) {
|
2020-11-19 17:30:53 +00:00
|
|
|
mff := func() (abi.TokenAmount, error) {
|
|
|
|
return msg.RequiredFunds(), nil
|
|
|
|
}
|
|
|
|
|
2020-12-08 19:51:27 +00:00
|
|
|
messagepool.CapGasFee(mff, msg, &api.MessageSendSpec{MaxFee: big.Min(big.Sub(avail, msg.Value), msg.RequiredFunds())})
|
2020-10-07 17:41:07 +00:00
|
|
|
}
|
2020-10-21 03:35:18 +00:00
|
|
|
return nil
|
2020-08-19 23:26:13 +00:00
|
|
|
}
|
2022-03-28 20:54:22 +00:00
|
|
|
|
|
|
|
func (s *WindowPoStScheduler) ComputePoSt(ctx context.Context, dlIdx uint64, ts *types.TipSet) ([]miner.SubmitWindowedPoStParams, error) {
|
|
|
|
dl, err := s.api.StateMinerProvingDeadline(ctx, s.actor, ts.Key())
|
|
|
|
if err != nil {
|
|
|
|
return nil, xerrors.Errorf("getting deadline: %w", err)
|
|
|
|
}
|
|
|
|
curIdx := dl.Index
|
|
|
|
dl.Index = dlIdx
|
|
|
|
dlDiff := dl.Index - curIdx
|
|
|
|
if dl.Index > curIdx {
|
|
|
|
dlDiff -= dl.WPoStPeriodDeadlines
|
|
|
|
dl.PeriodStart -= dl.WPoStProvingPeriod
|
|
|
|
}
|
|
|
|
|
|
|
|
epochDiff := (dl.WPoStProvingPeriod / abi.ChainEpoch(dl.WPoStPeriodDeadlines)) * abi.ChainEpoch(dlDiff)
|
|
|
|
|
|
|
|
// runPoStCycle only needs dl.Index and dl.Challenge
|
|
|
|
dl.Challenge += epochDiff
|
|
|
|
|
|
|
|
return s.runPoStCycle(ctx, true, *dl, ts)
|
|
|
|
}
|
2022-09-06 15:47:30 +00:00
|
|
|
|
|
|
|
func (s *WindowPoStScheduler) ManualFaultRecovery(ctx context.Context, maddr address.Address, sectors []abi.SectorNumber) ([]cid.Cid, error) {
|
|
|
|
return s.declareManualRecoveries(ctx, maddr, sectors, types.TipSetKey{})
|
|
|
|
}
|