diff --git a/api/api_full.go b/api/api_full.go index 0c281c12d..ae7609168 100644 --- a/api/api_full.go +++ b/api/api_full.go @@ -151,7 +151,7 @@ type FullNode interface { // ChainGetPath returns a set of revert/apply operations needed to get from // one tipset to another, for example: - //``` + // ``` // to // ^ // from tAA @@ -160,7 +160,7 @@ type FullNode interface { // ^---*--^ // ^ // tRR - //``` + // ``` // Would return `[revert(tBA), apply(tAB), apply(tAA)]` ChainGetPath(ctx context.Context, from types.TipSetKey, to types.TipSetKey) ([]*HeadChange, error) //perm:read @@ -388,12 +388,12 @@ type FullNode interface { ClientCancelRetrievalDeal(ctx context.Context, dealid retrievalmarket.DealID) error //perm:write // ClientUnimport removes references to the specified file from filestore - //ClientUnimport(path string) + // ClientUnimport(path string) // ClientListImports lists imported files and their root CIDs ClientListImports(ctx context.Context) ([]Import, error) //perm:write - //ClientListAsks() []Ask + // ClientListAsks() []Ask // MethodGroup: State // The State methods are used to query, inspect, and interact with chain state. @@ -640,14 +640,14 @@ type FullNode interface { // It takes the following params: , , MsigGetVested(context.Context, address.Address, types.TipSetKey, types.TipSetKey) (types.BigInt, error) //perm:read - //MsigGetPending returns pending transactions for the given multisig - //wallet. Once pending transactions are fully approved, they will no longer - //appear here. + // MsigGetPending returns pending transactions for the given multisig + // wallet. Once pending transactions are fully approved, they will no longer + // appear here. MsigGetPending(context.Context, address.Address, types.TipSetKey) ([]*MsigTransaction, error) //perm:read // MsigCreate creates a multisig wallet // It takes the following params: , , - //, , + // , , MsigCreate(context.Context, uint64, []address.Address, abi.ChainEpoch, types.BigInt, address.Address, types.BigInt) (*MessagePrototype, error) //perm:sign // MsigPropose proposes a multisig message diff --git a/api/docgen/docgen.go b/api/docgen/docgen.go index e8c667770..9478c25ff 100644 --- a/api/docgen/docgen.go +++ b/api/docgen/docgen.go @@ -439,7 +439,7 @@ func ExampleValue(method string, t, parent reflect.Type) interface{} { case reflect.Ptr: if t.Elem().Kind() == reflect.Struct { es := exampleStruct(method, t.Elem(), t) - //ExampleValues[t] = es + // ExampleValues[t] = es return es } case reflect.Interface: diff --git a/api/v0api/full.go b/api/v0api/full.go index ca1371794..87d5cba4e 100644 --- a/api/v0api/full.go +++ b/api/v0api/full.go @@ -141,7 +141,7 @@ type FullNode interface { // ChainGetPath returns a set of revert/apply operations needed to get from // one tipset to another, for example: - //``` + // ``` // to // ^ // from tAA @@ -150,7 +150,7 @@ type FullNode interface { // ^---*--^ // ^ // tRR - //``` + // ``` // Would return `[revert(tBA), apply(tAB), apply(tAA)]` ChainGetPath(ctx context.Context, from types.TipSetKey, to types.TipSetKey) ([]*api.HeadChange, error) //perm:read @@ -367,12 +367,12 @@ type FullNode interface { ClientCancelRetrievalDeal(ctx context.Context, dealid retrievalmarket.DealID) error //perm:write // ClientUnimport removes references to the specified file from filestore - //ClientUnimport(path string) + // ClientUnimport(path string) // ClientListImports lists imported files and their root CIDs ClientListImports(ctx context.Context) ([]api.Import, error) //perm:write - //ClientListAsks() []Ask + // ClientListAsks() []Ask // MethodGroup: State // The State methods are used to query, inspect, and interact with chain state. @@ -641,14 +641,14 @@ type FullNode interface { // It takes the following params: , , MsigGetVested(context.Context, address.Address, types.TipSetKey, types.TipSetKey) (types.BigInt, error) //perm:read - //MsigGetPending returns pending transactions for the given multisig - //wallet. Once pending transactions are fully approved, they will no longer - //appear here. + // MsigGetPending returns pending transactions for the given multisig + // wallet. Once pending transactions are fully approved, they will no longer + // appear here. MsigGetPending(context.Context, address.Address, types.TipSetKey) ([]*api.MsigTransaction, error) //perm:read // MsigCreate creates a multisig wallet // It takes the following params: , , - //, , + // , , MsigCreate(context.Context, uint64, []address.Address, abi.ChainEpoch, types.BigInt, address.Address, types.BigInt) (cid.Cid, error) //perm:sign // MsigPropose proposes a multisig message // It takes the following params: , , , diff --git a/chain/consensus/common.go b/chain/consensus/common.go new file mode 100644 index 000000000..4dc70f0d4 --- /dev/null +++ b/chain/consensus/common.go @@ -0,0 +1,482 @@ +package consensus + +import ( + "context" + "fmt" + "strings" + + "github.com/hashicorp/go-multierror" + "github.com/ipfs/go-cid" + cbor "github.com/ipfs/go-ipld-cbor" + logging "github.com/ipfs/go-log/v2" + pubsub "github.com/libp2p/go-libp2p-pubsub" + cbg "github.com/whyrusleeping/cbor-gen" + "go.opencensus.io/stats" + "golang.org/x/xerrors" + + "github.com/filecoin-project/go-address" + "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" + "github.com/filecoin-project/lotus/chain/actors/builtin" + "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/lib/async" + "github.com/filecoin-project/lotus/lib/sigs" + "github.com/filecoin-project/lotus/metrics" +) + +// Common operations shared by all consensus algorithm implementations. +var log = logging.Logger("consensus-common") + +// RunAsyncChecks accepts a list of checks to perform in parallel. +// +// Each consensus algorithm may choose to perform a set of different +// checks when a new blocks is received. +func RunAsyncChecks(ctx context.Context, await []async.ErrorFuture) error { + var merr error + for _, fut := range await { + if err := fut.AwaitContext(ctx); err != nil { + merr = multierror.Append(merr, err) + } + } + if merr != nil { + mulErr := merr.(*multierror.Error) + mulErr.ErrorFormat = func(es []error) string { + if len(es) == 1 { + return fmt.Sprintf("1 error occurred:\n\t* %+v\n\n", es[0]) + } + + points := make([]string, len(es)) + for i, err := range es { + points[i] = fmt.Sprintf("* %+v", err) + } + + return fmt.Sprintf( + "%d errors occurred:\n\t%s\n\n", + len(es), strings.Join(points, "\n\t")) + } + return mulErr + } + + return nil +} + +// CommonBlkChecks performed by all consensus implementations. +func CommonBlkChecks(ctx context.Context, sm *stmgr.StateManager, cs *store.ChainStore, + b *types.FullBlock, baseTs *types.TipSet) []async.ErrorFuture { + h := b.Header + msgsCheck := async.Err(func() error { + if b.Cid() == build.WhitelistedBlock { + return nil + } + + if err := checkBlockMessages(ctx, sm, cs, b, baseTs); err != nil { + return xerrors.Errorf("block had invalid messages: %w", err) + } + return nil + }) + + baseFeeCheck := async.Err(func() error { + baseFee, err := cs.ComputeBaseFee(ctx, baseTs) + if err != nil { + return xerrors.Errorf("computing base fee: %w", err) + } + if types.BigCmp(baseFee, b.Header.ParentBaseFee) != 0 { + return xerrors.Errorf("base fee doesn't match: %s (header) != %s (computed)", + b.Header.ParentBaseFee, baseFee) + } + return nil + }) + + stateRootCheck := async.Err(func() error { + stateroot, precp, err := sm.TipSetState(ctx, baseTs) + if err != nil { + return xerrors.Errorf("get tipsetstate(%d, %s) failed: %w", h.Height, h.Parents, err) + } + + if stateroot != h.ParentStateRoot { + msgs, err := cs.MessagesForTipset(ctx, baseTs) + if err != nil { + log.Error("failed to load messages for tipset during tipset state mismatch error: ", err) + } else { + log.Warn("Messages for tipset with mismatching state:") + for i, m := range msgs { + mm := m.VMMessage() + log.Warnf("Message[%d]: from=%s to=%s method=%d params=%x", i, mm.From, mm.To, mm.Method, mm.Params) + } + } + + return xerrors.Errorf("parent state root did not match computed state (%s != %s)", h.ParentStateRoot, stateroot) + } + + if precp != h.ParentMessageReceipts { + return xerrors.Errorf("parent receipts root did not match computed value (%s != %s)", precp, h.ParentMessageReceipts) + } + return nil + }) + + return []async.ErrorFuture{ + msgsCheck, + baseFeeCheck, + stateRootCheck, + } +} + +// Check the validity of the messages included in a block. +func checkBlockMessages(ctx context.Context, sm *stmgr.StateManager, cs *store.ChainStore, b *types.FullBlock, baseTs *types.TipSet) error { + { + var sigCids []cid.Cid // this is what we get for people not wanting the marshalcbor method on the cid type + var pubks [][]byte + + for _, m := range b.BlsMessages { + sigCids = append(sigCids, m.Cid()) + + pubk, err := sm.GetBlsPublicKey(ctx, m.From, baseTs) + if err != nil { + return xerrors.Errorf("failed to load bls public to validate block: %w", err) + } + + pubks = append(pubks, pubk) + } + + if err := VerifyBlsAggregate(ctx, b.Header.BLSAggregate, sigCids, pubks); err != nil { + return xerrors.Errorf("bls aggregate signature was invalid: %w", err) + } + } + + nonces := make(map[address.Address]uint64) + + stateroot, _, err := sm.TipSetState(ctx, baseTs) + if err != nil { + return xerrors.Errorf("failed to compute tipsettate for %s: %w", baseTs.Key(), err) + } + + st, err := state.LoadStateTree(cs.ActorStore(ctx), stateroot) + if err != nil { + return xerrors.Errorf("failed to load base state tree: %w", err) + } + + nv := sm.GetNetworkVersion(ctx, b.Header.Height) + pl := vm.PricelistByEpoch(b.Header.Height) + var sumGasLimit int64 + checkMsg := func(msg types.ChainMsg) error { + m := msg.VMMessage() + + // Phase 1: syntactic validation, as defined in the spec + minGas := pl.OnChainMessage(msg.ChainLength()) + if err := m.ValidForBlockInclusion(minGas.Total(), nv); err != nil { + return xerrors.Errorf("msg %s invalid for block inclusion: %w", m.Cid(), err) + } + + // ValidForBlockInclusion checks if any single message does not exceed BlockGasLimit + // So below is overflow safe + sumGasLimit += m.GasLimit + if sumGasLimit > build.BlockGasLimit { + return xerrors.Errorf("block gas limit exceeded") + } + + // Phase 2: (Partial) semantic validation: + // the sender exists and is an account actor, and the nonces make sense + var sender address.Address + if nv >= network.Version13 { + sender, err = st.LookupID(m.From) + if err != nil { + return xerrors.Errorf("failed to lookup sender %s: %w", m.From, err) + } + } else { + sender = m.From + } + + if _, ok := nonces[sender]; !ok { + // `GetActor` does not validate that this is an account actor. + act, err := st.GetActor(sender) + if err != nil { + return xerrors.Errorf("failed to get actor: %w", err) + } + + if !builtin.IsAccountActor(act.Code) { + return xerrors.New("Sender must be an account actor") + } + nonces[sender] = act.Nonce + } + + if nonces[sender] != m.Nonce { + return xerrors.Errorf("wrong nonce (exp: %d, got: %d)", nonces[sender], m.Nonce) + } + nonces[sender]++ + + return nil + } + + // Validate message arrays in a temporary blockstore. + tmpbs := bstore.NewMemory() + tmpstore := blockadt.WrapStore(ctx, cbor.NewCborStore(tmpbs)) + + bmArr := blockadt.MakeEmptyArray(tmpstore) + for i, m := range b.BlsMessages { + if err := checkMsg(m); err != nil { + return xerrors.Errorf("block had invalid bls message at index %d: %w", i, err) + } + + c, err := store.PutMessage(ctx, tmpbs, m) + if err != nil { + return xerrors.Errorf("failed to store message %s: %w", m.Cid(), err) + } + + k := cbg.CborCid(c) + if err := bmArr.Set(uint64(i), &k); err != nil { + return xerrors.Errorf("failed to put bls message at index %d: %w", i, err) + } + } + + smArr := blockadt.MakeEmptyArray(tmpstore) + for i, m := range b.SecpkMessages { + if sm.GetNetworkVersion(ctx, b.Header.Height) >= network.Version14 { + if m.Signature.Type != crypto.SigTypeSecp256k1 { + return xerrors.Errorf("block had invalid secpk message at index %d: %w", i, err) + } + } + + if err := checkMsg(m); err != nil { + return xerrors.Errorf("block had invalid secpk message at index %d: %w", i, err) + } + + // `From` being an account actor is only validated inside the `vm.ResolveToKeyAddr` call + // in `StateManager.ResolveToKeyAddress` here (and not in `checkMsg`). + kaddr, err := sm.ResolveToKeyAddress(ctx, m.Message.From, baseTs) + if err != nil { + return xerrors.Errorf("failed to resolve key addr: %w", err) + } + + if err := sigs.Verify(&m.Signature, kaddr, m.Message.Cid().Bytes()); err != nil { + return xerrors.Errorf("secpk message %s has invalid signature: %w", m.Cid(), err) + } + + c, err := store.PutMessage(ctx, tmpbs, m) + if err != nil { + return xerrors.Errorf("failed to store message %s: %w", m.Cid(), err) + } + k := cbg.CborCid(c) + if err := smArr.Set(uint64(i), &k); err != nil { + return xerrors.Errorf("failed to put secpk message at index %d: %w", i, err) + } + } + + bmroot, err := bmArr.Root() + if err != nil { + return xerrors.Errorf("failed to root bls msgs: %w", err) + + } + + smroot, err := smArr.Root() + if err != nil { + return xerrors.Errorf("failed to root secp msgs: %w", err) + } + + mrcid, err := tmpstore.Put(ctx, &types.MsgMeta{ + BlsMessages: bmroot, + SecpkMessages: smroot, + }) + if err != nil { + return xerrors.Errorf("failed to put msg meta: %w", err) + } + + if b.Header.Messages != mrcid { + return fmt.Errorf("messages didnt match message root in header") + } + + // Finally, flush. + err = vm.Copy(ctx, tmpbs, cs.ChainBlockstore(), mrcid) + if err != nil { + return xerrors.Errorf("failed to flush:%w", err) + } + + return nil +} + +// CreateBlockHeader generates the block header from the block template of +// the block being proposed. +func CreateBlockHeader(ctx context.Context, sm *stmgr.StateManager, pts *types.TipSet, + bt *api.BlockTemplate) (*types.BlockHeader, []*types.Message, []*types.SignedMessage, error) { + + st, recpts, err := sm.TipSetState(ctx, pts) + if err != nil { + return nil, nil, nil, xerrors.Errorf("failed to load tipset state: %w", err) + } + next := &types.BlockHeader{ + Miner: bt.Miner, + Parents: bt.Parents.Cids(), + Ticket: bt.Ticket, + ElectionProof: bt.Eproof, + + BeaconEntries: bt.BeaconValues, + Height: bt.Epoch, + Timestamp: bt.Timestamp, + WinPoStProof: bt.WinningPoStProof, + ParentStateRoot: st, + ParentMessageReceipts: recpts, + } + + var blsMessages []*types.Message + var secpkMessages []*types.SignedMessage + + var blsMsgCids, secpkMsgCids []cid.Cid + var blsSigs []crypto.Signature + for _, msg := range bt.Messages { + if msg.Signature.Type == crypto.SigTypeBLS { + blsSigs = append(blsSigs, msg.Signature) + blsMessages = append(blsMessages, &msg.Message) + + c, err := sm.ChainStore().PutMessage(ctx, &msg.Message) + if err != nil { + return nil, nil, nil, err + } + + blsMsgCids = append(blsMsgCids, c) + } else if msg.Signature.Type == crypto.SigTypeSecp256k1 { + c, err := sm.ChainStore().PutMessage(ctx, msg) + if err != nil { + return nil, nil, nil, err + } + + secpkMsgCids = append(secpkMsgCids, c) + secpkMessages = append(secpkMessages, msg) + + } else { + return nil, nil, nil, xerrors.Errorf("unknown sig type: %d", msg.Signature.Type) + } + } + + store := sm.ChainStore().ActorStore(ctx) + blsmsgroot, err := ToMessagesArray(store, blsMsgCids) + if err != nil { + return nil, nil, nil, xerrors.Errorf("building bls amt: %w", err) + } + secpkmsgroot, err := ToMessagesArray(store, secpkMsgCids) + if err != nil { + return nil, nil, nil, xerrors.Errorf("building secpk amt: %w", err) + } + + mmcid, err := store.Put(store.Context(), &types.MsgMeta{ + BlsMessages: blsmsgroot, + SecpkMessages: secpkmsgroot, + }) + if err != nil { + return nil, nil, nil, err + } + next.Messages = mmcid + + aggSig, err := AggregateSignatures(blsSigs) + if err != nil { + return nil, nil, nil, err + } + + next.BLSAggregate = aggSig + pweight, err := sm.ChainStore().Weight(ctx, pts) + if err != nil { + return nil, nil, nil, err + } + next.ParentWeight = pweight + + baseFee, err := sm.ChainStore().ComputeBaseFee(ctx, pts) + if err != nil { + return nil, nil, nil, xerrors.Errorf("computing base fee: %w", err) + } + next.ParentBaseFee = baseFee + + return next, blsMessages, secpkMessages, err + +} + +// Basic sanity-checks performed when a block is proposed locally. +func validateLocalBlock(ctx context.Context, msg *pubsub.Message) (pubsub.ValidationResult, string) { + stats.Record(ctx, metrics.BlockPublished.M(1)) + + if size := msg.Size(); size > 1<<20-1<<15 { + log.Errorf("ignoring oversize block (%dB)", size) + return pubsub.ValidationIgnore, "oversize_block" + } + + blk, what, err := decodeAndCheckBlock(msg) + if err != nil { + log.Errorf("got invalid local block: %s", err) + return pubsub.ValidationIgnore, what + } + + msg.ValidatorData = blk + stats.Record(ctx, metrics.BlockValidationSuccess.M(1)) + return pubsub.ValidationAccept, "" +} + +func decodeAndCheckBlock(msg *pubsub.Message) (*types.BlockMsg, string, error) { + blk, err := types.DecodeBlockMsg(msg.GetData()) + if err != nil { + return nil, "invalid", xerrors.Errorf("error decoding block: %w", err) + } + + if count := len(blk.BlsMessages) + len(blk.SecpkMessages); count > build.BlockMessageLimit { + return nil, "too_many_messages", fmt.Errorf("block contains too many messages (%d)", count) + } + + // make sure we have a signature + if blk.Header.BlockSig == nil { + return nil, "missing_signature", fmt.Errorf("block without a signature") + } + + return blk, "", nil +} + +func validateMsgMeta(ctx context.Context, msg *types.BlockMsg) error { + // TODO there has to be a simpler way to do this without the blockstore dance + // block headers use adt0 + store := blockadt.WrapStore(ctx, cbor.NewCborStore(bstore.NewMemory())) + bmArr := blockadt.MakeEmptyArray(store) + smArr := blockadt.MakeEmptyArray(store) + + for i, m := range msg.BlsMessages { + c := cbg.CborCid(m) + if err := bmArr.Set(uint64(i), &c); err != nil { + return err + } + } + + for i, m := range msg.SecpkMessages { + c := cbg.CborCid(m) + if err := smArr.Set(uint64(i), &c); err != nil { + return err + } + } + + bmroot, err := bmArr.Root() + if err != nil { + return err + } + + smroot, err := smArr.Root() + if err != nil { + return err + } + + mrcid, err := store.Put(store.Context(), &types.MsgMeta{ + BlsMessages: bmroot, + SecpkMessages: smroot, + }) + + if err != nil { + return err + } + + if msg.Header.Messages != mrcid { + return fmt.Errorf("messages didn't match root cid in header") + } + + return nil +} diff --git a/chain/consensus/filcns/compute_state.go b/chain/consensus/compute_state.go similarity index 88% rename from chain/consensus/filcns/compute_state.go rename to chain/consensus/compute_state.go index ece973798..0785f93f0 100644 --- a/chain/consensus/filcns/compute_state.go +++ b/chain/consensus/compute_state.go @@ -1,4 +1,4 @@ -package filcns +package consensus import ( "context" @@ -24,7 +24,6 @@ import ( "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/cron" "github.com/filecoin-project/lotus/chain/actors/builtin/reward" @@ -53,10 +52,12 @@ func NewActorRegistry() *vm.ActorRegistry { return inv } -type TipSetExecutor struct{} +type TipSetExecutor struct { + reward RewardFunc +} -func NewTipSetExecutor() *TipSetExecutor { - return &TipSetExecutor{} +func NewTipSetExecutor(r RewardFunc) *TipSetExecutor { + return &TipSetExecutor{reward: r} } func (t *TipSetExecutor) NewActorRegistry() *vm.ActorRegistry { @@ -200,39 +201,15 @@ func (t *TipSetExecutor) ApplyBlocks(ctx context.Context, processedMsgs[m.Cid()] = struct{}{} } - params, err := actors.SerializeParams(&reward.AwardBlockRewardParams{ + params := &reward.AwardBlockRewardParams{ Miner: b.Miner, Penalty: penalty, GasReward: gasReward, WinCount: b.WinCount, - }) - if err != nil { - return cid.Undef, cid.Undef, xerrors.Errorf("failed to serialize award params: %w", err) } - - rwMsg := &types.Message{ - From: builtin.SystemActorAddr, - To: reward.Address, - Nonce: uint64(epoch), - Value: types.NewInt(0), - GasFeeCap: types.NewInt(0), - GasPremium: types.NewInt(0), - GasLimit: 1 << 30, - Method: reward.Methods.AwardBlockReward, - Params: params, - } - ret, actErr := vmi.ApplyImplicitMessage(ctx, rwMsg) - if actErr != nil { - return cid.Undef, cid.Undef, xerrors.Errorf("failed to apply reward message for miner %s: %w", b.Miner, actErr) - } - if em != nil { - if err := em.MessageApplied(ctx, ts, rwMsg.Cid(), rwMsg, ret, true); err != nil { - return cid.Undef, cid.Undef, xerrors.Errorf("callback failed on reward message: %w", err) - } - } - - if ret.ExitCode != 0 { - return cid.Undef, cid.Undef, xerrors.Errorf("reward application message failed (exit %d): %s", ret.ExitCode, ret.ActorErr) + rErr := t.reward(ctx, vmi, em, epoch, ts, params) + if rErr != nil { + return cid.Undef, cid.Undef, xerrors.Errorf("error applying reward: %w", err) } } diff --git a/chain/consensus/filcns/filecoin.go b/chain/consensus/filcns/filecoin.go index de3cf7cf7..2e3baa4db 100644 --- a/chain/consensus/filcns/filecoin.go +++ b/chain/consensus/filcns/filecoin.go @@ -4,18 +4,11 @@ import ( "bytes" "context" "errors" - "fmt" "os" - "strings" "time" - "github.com/hashicorp/go-multierror" "github.com/ipfs/go-cid" - cbor "github.com/ipfs/go-ipld-cbor" logging "github.com/ipfs/go-log/v2" - pubsub "github.com/libp2p/go-libp2p-pubsub" - cbg "github.com/whyrusleeping/cbor-gen" - "go.opencensus.io/stats" "go.opencensus.io/trace" "golang.org/x/xerrors" @@ -23,25 +16,24 @@ 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" - blockadt "github.com/filecoin-project/specs-actors/actors/util/adt" "github.com/filecoin-project/specs-actors/v7/actors/runtime/proof" - bstore "github.com/filecoin-project/lotus/blockstore" + "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain" + "github.com/filecoin-project/lotus/chain/actors" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/power" + "github.com/filecoin-project/lotus/chain/actors/builtin/reward" "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/rand" - "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/lib/async" "github.com/filecoin-project/lotus/lib/sigs" - "github.com/filecoin-project/lotus/metrics" "github.com/filecoin-project/lotus/storage/sealer/ffiwrapper" "github.com/filecoin-project/lotus/storage/sealer/storiface" ) @@ -67,6 +59,39 @@ type FilecoinEC struct { // the theoretical max height based on systime are quickly rejected const MaxHeightDrift = 5 +var RewardFunc = func(ctx context.Context, vmi vm.Interface, em stmgr.ExecMonitor, + epoch abi.ChainEpoch, ts *types.TipSet, params *reward.AwardBlockRewardParams) error { + ser, err := actors.SerializeParams(params) + if err != nil { + return xerrors.Errorf("failed to serialize award params: %w", err) + } + rwMsg := &types.Message{ + From: builtin.SystemActorAddr, + To: reward.Address, + Nonce: uint64(epoch), + Value: types.NewInt(0), + GasFeeCap: types.NewInt(0), + GasPremium: types.NewInt(0), + GasLimit: 1 << 30, + Method: reward.Methods.AwardBlockReward, + Params: ser, + } + ret, actErr := vmi.ApplyImplicitMessage(ctx, rwMsg) + if actErr != nil { + return xerrors.Errorf("failed to apply reward message: %w", actErr) + } + if em != nil { + if err := em.MessageApplied(ctx, ts, rwMsg.Cid(), rwMsg, ret, true); err != nil { + return xerrors.Errorf("callback failed on reward message: %w", err) + } + } + + if ret.ExitCode != 0 { + return xerrors.Errorf("reward application message failed (exit %d): %s", ret.ExitCode, ret.ActorErr) + } + return nil +} + func NewFilecoinExpectedConsensus(sm *stmgr.StateManager, beacon beacon.Schedule, verifier storiface.Verifier, genesis chain.Genesis) consensus.Consensus { if build.InsecurePoStValidation { log.Warn("*********************************************************************************************") @@ -125,17 +150,6 @@ func (filec *FilecoinEC) ValidateBlock(ctx context.Context, b *types.FullBlock) log.Warn("Got block from the future, but within threshold", h.Timestamp, build.Clock.Now().Unix()) } - msgsCheck := async.Err(func() error { - if b.Cid() == build.WhitelistedBlock { - return nil - } - - if err := filec.checkBlockMessages(ctx, b, baseTs); err != nil { - return xerrors.Errorf("block had invalid messages: %w", err) - } - return nil - }) - minerCheck := async.Err(func() error { if err := filec.minerIsValid(ctx, h.Miner, baseTs); err != nil { return xerrors.Errorf("minerIsValid failed: %w", err) @@ -143,17 +157,6 @@ func (filec *FilecoinEC) ValidateBlock(ctx context.Context, b *types.FullBlock) return nil }) - baseFeeCheck := async.Err(func() error { - baseFee, err := filec.store.ComputeBaseFee(ctx, baseTs) - if err != nil { - return xerrors.Errorf("computing base fee: %w", err) - } - if types.BigCmp(baseFee, b.Header.ParentBaseFee) != 0 { - return xerrors.Errorf("base fee doesn't match: %s (header) != %s (computed)", - b.Header.ParentBaseFee, baseFee) - } - return nil - }) pweight, err := filec.store.Weight(ctx, baseTs) if err != nil { return xerrors.Errorf("getting parent weight: %w", err) @@ -164,34 +167,6 @@ func (filec *FilecoinEC) ValidateBlock(ctx context.Context, b *types.FullBlock) b.Header.ParentWeight, pweight) } - stateRootCheck := async.Err(func() error { - stateroot, precp, err := filec.sm.TipSetState(ctx, baseTs) - if err != nil { - return xerrors.Errorf("get tipsetstate(%d, %s) failed: %w", h.Height, h.Parents, err) - } - - if stateroot != h.ParentStateRoot { - msgs, err := filec.store.MessagesForTipset(ctx, baseTs) - if err != nil { - log.Error("failed to load messages for tipset during tipset state mismatch error: ", err) - } else { - log.Warn("Messages for tipset with mismatching state:") - for i, m := range msgs { - mm := m.VMMessage() - log.Warnf("Message[%d]: from=%s to=%s method=%d params=%x", i, mm.From, mm.To, mm.Method, mm.Params) - } - } - - return xerrors.Errorf("parent state root did not match computed state (%s != %s)", h.ParentStateRoot, stateroot) - } - - if precp != h.ParentMessageReceipts { - return xerrors.Errorf("parent receipts root did not match computed value (%s != %s)", precp, h.ParentMessageReceipts) - } - - return nil - }) - // Stuff that needs worker address waddr, err := stmgr.GetMinerWorkerRaw(ctx, filec.sm, lbst, h.Miner) if err != nil { @@ -253,10 +228,11 @@ func (filec *FilecoinEC) ValidateBlock(ctx context.Context, b *types.FullBlock) }) blockSigCheck := async.Err(func() error { - if err := sigs.CheckBlockSignature(ctx, h, waddr); err != nil { + if err := verifyBlockSignature(ctx, h, waddr); err != nil { return xerrors.Errorf("check block signature failed: %w", err) } return nil + }) beaconValuesCheck := async.Err(func() error { @@ -305,44 +281,17 @@ func (filec *FilecoinEC) ValidateBlock(ctx context.Context, b *types.FullBlock) return nil }) - await := []async.ErrorFuture{ + commonChecks := consensus.CommonBlkChecks(ctx, filec.sm, filec.store, b, baseTs) + await := append([]async.ErrorFuture{ minerCheck, tktsCheck, blockSigCheck, beaconValuesCheck, wproofCheck, winnerCheck, - msgsCheck, - baseFeeCheck, - stateRootCheck, - } + }, commonChecks...) - var merr error - for _, fut := range await { - if err := fut.AwaitContext(ctx); err != nil { - merr = multierror.Append(merr, err) - } - } - if merr != nil { - mulErr := merr.(*multierror.Error) - mulErr.ErrorFormat = func(es []error) string { - if len(es) == 1 { - return fmt.Sprintf("1 error occurred:\n\t* %+v\n\n", es[0]) - } - - points := make([]string, len(es)) - for i, err := range es { - points[i] = fmt.Sprintf("* %+v", err) - } - - return fmt.Sprintf( - "%d errors occurred:\n\t%s\n\n", - len(es), strings.Join(points, "\n\t")) - } - return mulErr - } - - return nil + return consensus.RunAsyncChecks(ctx, await) } func blockSanityChecks(h *types.BlockHeader) error { @@ -433,178 +382,6 @@ func (filec *FilecoinEC) VerifyWinningPoStProof(ctx context.Context, nv network. return nil } -// TODO: We should extract this somewhere else and make the message pool and miner use the same logic -func (filec *FilecoinEC) checkBlockMessages(ctx context.Context, b *types.FullBlock, baseTs *types.TipSet) error { - { - var sigCids []cid.Cid // this is what we get for people not wanting the marshalcbor method on the cid type - var pubks [][]byte - - for _, m := range b.BlsMessages { - sigCids = append(sigCids, m.Cid()) - - pubk, err := filec.sm.GetBlsPublicKey(ctx, m.From, baseTs) - if err != nil { - return xerrors.Errorf("failed to load bls public to validate block: %w", err) - } - - pubks = append(pubks, pubk) - } - - if err := consensus.VerifyBlsAggregate(ctx, b.Header.BLSAggregate, sigCids, pubks); err != nil { - return xerrors.Errorf("bls aggregate signature was invalid: %w", err) - } - } - - nonces := make(map[address.Address]uint64) - - stateroot, _, err := filec.sm.TipSetState(ctx, baseTs) - if err != nil { - return xerrors.Errorf("failed to compute tipsettate for %s: %w", baseTs.Key(), err) - } - - st, err := state.LoadStateTree(filec.store.ActorStore(ctx), stateroot) - if err != nil { - return xerrors.Errorf("failed to load base state tree: %w", err) - } - - nv := filec.sm.GetNetworkVersion(ctx, b.Header.Height) - pl := vm.PricelistByEpoch(b.Header.Height) - var sumGasLimit int64 - checkMsg := func(msg types.ChainMsg) error { - m := msg.VMMessage() - - // Phase 1: syntactic validation, as defined in the spec - minGas := pl.OnChainMessage(msg.ChainLength()) - if err := m.ValidForBlockInclusion(minGas.Total(), nv); err != nil { - return xerrors.Errorf("msg %s invalid for block inclusion: %w", m.Cid(), err) - } - - // ValidForBlockInclusion checks if any single message does not exceed BlockGasLimit - // So below is overflow safe - sumGasLimit += m.GasLimit - if sumGasLimit > build.BlockGasLimit { - return xerrors.Errorf("block gas limit exceeded") - } - - // Phase 2: (Partial) semantic validation: - // the sender exists and is an account actor, and the nonces make sense - var sender address.Address - if nv >= network.Version13 { - sender, err = st.LookupID(m.From) - if err != nil { - return xerrors.Errorf("failed to lookup sender %s: %w", m.From, err) - } - } else { - sender = m.From - } - - if _, ok := nonces[sender]; !ok { - // `GetActor` does not validate that this is an account actor. - act, err := st.GetActor(sender) - if err != nil { - return xerrors.Errorf("failed to get actor: %w", err) - } - - if !builtin.IsAccountActor(act.Code) { - return xerrors.New("Sender must be an account actor") - } - nonces[sender] = act.Nonce - } - - if nonces[sender] != m.Nonce { - return xerrors.Errorf("wrong nonce (exp: %d, got: %d)", nonces[sender], m.Nonce) - } - nonces[sender]++ - - return nil - } - - // Validate message arrays in a temporary blockstore. - tmpbs := bstore.NewMemory() - tmpstore := blockadt.WrapStore(ctx, cbor.NewCborStore(tmpbs)) - - bmArr := blockadt.MakeEmptyArray(tmpstore) - for i, m := range b.BlsMessages { - if err := checkMsg(m); err != nil { - return xerrors.Errorf("block had invalid bls message at index %d: %w", i, err) - } - - c, err := store.PutMessage(ctx, tmpbs, m) - if err != nil { - return xerrors.Errorf("failed to store message %s: %w", m.Cid(), err) - } - - k := cbg.CborCid(c) - if err := bmArr.Set(uint64(i), &k); err != nil { - return xerrors.Errorf("failed to put bls message at index %d: %w", i, err) - } - } - - smArr := blockadt.MakeEmptyArray(tmpstore) - for i, m := range b.SecpkMessages { - if filec.sm.GetNetworkVersion(ctx, b.Header.Height) >= network.Version14 { - if m.Signature.Type != crypto.SigTypeSecp256k1 { - return xerrors.Errorf("block had invalid secpk message at index %d: %w", i, err) - } - } - - if err := checkMsg(m); err != nil { - return xerrors.Errorf("block had invalid secpk message at index %d: %w", i, err) - } - - // `From` being an account actor is only validated inside the `vm.ResolveToKeyAddr` call - // in `StateManager.ResolveToKeyAddress` here (and not in `checkMsg`). - kaddr, err := filec.sm.ResolveToKeyAddress(ctx, m.Message.From, baseTs) - if err != nil { - return xerrors.Errorf("failed to resolve key addr: %w", err) - } - - if err := sigs.Verify(&m.Signature, kaddr, m.Message.Cid().Bytes()); err != nil { - return xerrors.Errorf("secpk message %s has invalid signature: %w", m.Cid(), err) - } - - c, err := store.PutMessage(ctx, tmpbs, m) - if err != nil { - return xerrors.Errorf("failed to store message %s: %w", m.Cid(), err) - } - k := cbg.CborCid(c) - if err := smArr.Set(uint64(i), &k); err != nil { - return xerrors.Errorf("failed to put secpk message at index %d: %w", i, err) - } - } - - bmroot, err := bmArr.Root() - if err != nil { - return xerrors.Errorf("failed to root bls msgs: %w", err) - - } - - smroot, err := smArr.Root() - if err != nil { - return xerrors.Errorf("failed to root secp msgs: %w", err) - } - - mrcid, err := tmpstore.Put(ctx, &types.MsgMeta{ - BlsMessages: bmroot, - SecpkMessages: smroot, - }) - if err != nil { - return xerrors.Errorf("failed to put msg meta: %w", err) - } - - if b.Header.Messages != mrcid { - return fmt.Errorf("messages didnt match message root in header") - } - - // Finally, flush. - err = vm.Copy(ctx, tmpbs, filec.store.ChainBlockstore(), mrcid) - if err != nil { - return xerrors.Errorf("failed to flush:%w", err) - } - - return nil -} - func (filec *FilecoinEC) IsEpochBeyondCurrMax(epoch abi.ChainEpoch) bool { if filec.genesis == nil { return false @@ -660,140 +437,7 @@ func VerifyVRF(ctx context.Context, worker address.Address, vrfBase, vrfproof [] var ErrSoftFailure = errors.New("soft validation failure") var ErrInsufficientPower = errors.New("incoming block's miner does not have minimum power") -func (filec *FilecoinEC) ValidateBlockPubsub(ctx context.Context, self bool, msg *pubsub.Message) (pubsub.ValidationResult, string) { - if self { - return filec.validateLocalBlock(ctx, msg) - } - - // track validation time - begin := build.Clock.Now() - defer func() { - log.Debugf("block validation time: %s", build.Clock.Since(begin)) - }() - - stats.Record(ctx, metrics.BlockReceived.M(1)) - - recordFailureFlagPeer := func(what string) { - // bv.Validate will flag the peer in that case - panic(what) - } - - blk, what, err := filec.decodeAndCheckBlock(msg) - if err != nil { - log.Error("got invalid block over pubsub: ", err) - recordFailureFlagPeer(what) - return pubsub.ValidationReject, what - } - - // validate the block meta: the Message CID in the header must match the included messages - err = filec.validateMsgMeta(ctx, blk) - if err != nil { - log.Warnf("error validating message metadata: %s", err) - recordFailureFlagPeer("invalid_block_meta") - return pubsub.ValidationReject, "invalid_block_meta" - } - - reject, err := filec.validateBlockHeader(ctx, blk.Header) - if err != nil { - if reject == "" { - log.Warn("ignoring block msg: ", err) - return pubsub.ValidationIgnore, reject - } - recordFailureFlagPeer(reject) - return pubsub.ValidationReject, reject - } - - // all good, accept the block - msg.ValidatorData = blk - stats.Record(ctx, metrics.BlockValidationSuccess.M(1)) - return pubsub.ValidationAccept, "" -} - -func (filec *FilecoinEC) validateLocalBlock(ctx context.Context, msg *pubsub.Message) (pubsub.ValidationResult, string) { - stats.Record(ctx, metrics.BlockPublished.M(1)) - - if size := msg.Size(); size > 1<<20-1<<15 { - log.Errorf("ignoring oversize block (%dB)", size) - return pubsub.ValidationIgnore, "oversize_block" - } - - blk, what, err := filec.decodeAndCheckBlock(msg) - if err != nil { - log.Errorf("got invalid local block: %s", err) - return pubsub.ValidationIgnore, what - } - - msg.ValidatorData = blk - stats.Record(ctx, metrics.BlockValidationSuccess.M(1)) - return pubsub.ValidationAccept, "" -} - -func (filec *FilecoinEC) decodeAndCheckBlock(msg *pubsub.Message) (*types.BlockMsg, string, error) { - blk, err := types.DecodeBlockMsg(msg.GetData()) - if err != nil { - return nil, "invalid", xerrors.Errorf("error decoding block: %w", err) - } - - if count := len(blk.BlsMessages) + len(blk.SecpkMessages); count > build.BlockMessageLimit { - return nil, "too_many_messages", fmt.Errorf("block contains too many messages (%d)", count) - } - - // make sure we have a signature - if blk.Header.BlockSig == nil { - return nil, "missing_signature", fmt.Errorf("block without a signature") - } - - return blk, "", nil -} - -func (filec *FilecoinEC) validateMsgMeta(ctx context.Context, msg *types.BlockMsg) error { - // TODO there has to be a simpler way to do this without the blockstore dance - // block headers use adt0 - store := blockadt.WrapStore(ctx, cbor.NewCborStore(bstore.NewMemory())) - bmArr := blockadt.MakeEmptyArray(store) - smArr := blockadt.MakeEmptyArray(store) - - for i, m := range msg.BlsMessages { - c := cbg.CborCid(m) - if err := bmArr.Set(uint64(i), &c); err != nil { - return err - } - } - - for i, m := range msg.SecpkMessages { - c := cbg.CborCid(m) - if err := smArr.Set(uint64(i), &c); err != nil { - return err - } - } - - bmroot, err := bmArr.Root() - if err != nil { - return err - } - - smroot, err := smArr.Root() - if err != nil { - return err - } - - mrcid, err := store.Put(store.Context(), &types.MsgMeta{ - BlsMessages: bmroot, - SecpkMessages: smroot, - }) - - if err != nil { - return err - } - - if msg.Header.Messages != mrcid { - return fmt.Errorf("messages didn't match root cid in header") - } - - return nil -} - -func (filec *FilecoinEC) validateBlockHeader(ctx context.Context, b *types.BlockHeader) (rejectReason string, err error) { +func (filec *FilecoinEC) ValidateBlockHeader(ctx context.Context, b *types.BlockHeader) (rejectReason string, err error) { // we want to ensure that it is a block from a known miner; we reject blocks from unknown miners // to prevent spam attacks. @@ -868,4 +512,27 @@ func (filec *FilecoinEC) isChainNearSynced() bool { return build.Clock.Since(timestampTime) < 6*time.Hour } +func verifyBlockSignature(ctx context.Context, h *types.BlockHeader, + addr address.Address) error { + return sigs.CheckBlockSignature(ctx, h, addr) +} + +func signBlock(ctx context.Context, w api.Wallet, + addr address.Address, next *types.BlockHeader) error { + + nosigbytes, err := next.SigningBytes() + if err != nil { + return xerrors.Errorf("failed to get signing bytes for block: %w", err) + } + + sig, err := w.WalletSign(ctx, addr, nosigbytes, api.MsgMeta{ + Type: api.MTBlock, + }) + if err != nil { + return xerrors.Errorf("failed to sign new block: %w", err) + } + next.BlockSig = sig + return nil +} + var _ consensus.Consensus = &FilecoinEC{} diff --git a/chain/consensus/filcns/mine.go b/chain/consensus/filcns/mine.go index 35e38883d..956cba252 100644 --- a/chain/consensus/filcns/mine.go +++ b/chain/consensus/filcns/mine.go @@ -3,11 +3,8 @@ package filcns import ( "context" - "github.com/ipfs/go-cid" "golang.org/x/xerrors" - "github.com/filecoin-project/go-state-types/crypto" - "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/stmgr" @@ -20,11 +17,6 @@ func (filec *FilecoinEC) CreateBlock(ctx context.Context, w api.Wallet, bt *api. return nil, xerrors.Errorf("failed to load parent tipset: %w", err) } - st, recpts, err := filec.sm.TipSetState(ctx, pts) - if err != nil { - return nil, xerrors.Errorf("failed to load tipset state: %w", err) - } - _, lbst, err := stmgr.GetLookbackTipSetForRound(ctx, filec.sm, pts, bt.Epoch) if err != nil { return nil, xerrors.Errorf("getting lookback miner actor state: %w", err) @@ -35,101 +27,15 @@ func (filec *FilecoinEC) CreateBlock(ctx context.Context, w api.Wallet, bt *api. return nil, xerrors.Errorf("failed to get miner worker: %w", err) } - next := &types.BlockHeader{ - Miner: bt.Miner, - Parents: bt.Parents.Cids(), - Ticket: bt.Ticket, - ElectionProof: bt.Eproof, - - BeaconEntries: bt.BeaconValues, - Height: bt.Epoch, - Timestamp: bt.Timestamp, - WinPoStProof: bt.WinningPoStProof, - ParentStateRoot: st, - ParentMessageReceipts: recpts, - } - - var blsMessages []*types.Message - var secpkMessages []*types.SignedMessage - - var blsMsgCids, secpkMsgCids []cid.Cid - var blsSigs []crypto.Signature - for _, msg := range bt.Messages { - if msg.Signature.Type == crypto.SigTypeBLS { - blsSigs = append(blsSigs, msg.Signature) - blsMessages = append(blsMessages, &msg.Message) - - c, err := filec.sm.ChainStore().PutMessage(ctx, &msg.Message) - if err != nil { - return nil, err - } - - blsMsgCids = append(blsMsgCids, c) - } else if msg.Signature.Type == crypto.SigTypeSecp256k1 { - c, err := filec.sm.ChainStore().PutMessage(ctx, msg) - if err != nil { - return nil, err - } - - secpkMsgCids = append(secpkMsgCids, c) - secpkMessages = append(secpkMessages, msg) - - } else { - return nil, xerrors.Errorf("unknown sig type: %d", msg.Signature.Type) - } - } - - store := filec.sm.ChainStore().ActorStore(ctx) - blsmsgroot, err := consensus.ToMessagesArray(store, blsMsgCids) + next, blsMessages, secpkMessages, err := consensus.CreateBlockHeader(ctx, filec.sm, pts, bt) if err != nil { - return nil, xerrors.Errorf("building bls amt: %w", err) - } - secpkmsgroot, err := consensus.ToMessagesArray(store, secpkMsgCids) - if err != nil { - return nil, xerrors.Errorf("building secpk amt: %w", err) + return nil, xerrors.Errorf("failed to process messages from block template: %w", err) } - mmcid, err := store.Put(store.Context(), &types.MsgMeta{ - BlsMessages: blsmsgroot, - SecpkMessages: secpkmsgroot, - }) - if err != nil { - return nil, err - } - next.Messages = mmcid - - aggSig, err := consensus.AggregateSignatures(blsSigs) - if err != nil { - return nil, err - } - - next.BLSAggregate = aggSig - pweight, err := filec.sm.ChainStore().Weight(ctx, pts) - if err != nil { - return nil, err - } - next.ParentWeight = pweight - - baseFee, err := filec.sm.ChainStore().ComputeBaseFee(ctx, pts) - if err != nil { - return nil, xerrors.Errorf("computing base fee: %w", err) - } - next.ParentBaseFee = baseFee - - nosigbytes, err := next.SigningBytes() - if err != nil { - return nil, xerrors.Errorf("failed to get signing bytes for block: %w", err) - } - - sig, err := w.WalletSign(ctx, worker, nosigbytes, api.MsgMeta{ - Type: api.MTBlock, - }) - if err != nil { + if err := signBlock(ctx, w, worker, next); err != nil { return nil, xerrors.Errorf("failed to sign new block: %w", err) } - next.BlockSig = sig - fullBlock := &types.FullBlock{ Header: next, BlsMessages: blsMessages, diff --git a/chain/consensus/iface.go b/chain/consensus/iface.go index 06dc0a113..10c3ead74 100644 --- a/chain/consensus/iface.go +++ b/chain/consensus/iface.go @@ -4,17 +4,99 @@ import ( "context" pubsub "github.com/libp2p/go-libp2p-pubsub" + "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" + "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/types" + "github.com/filecoin-project/lotus/chain/vm" + "github.com/filecoin-project/lotus/metrics" ) type Consensus interface { + // ValidateBlockHeader is called by peers when they receive a new block through the network. + // + // This is a fast sanity-check validation performed by the PubSub protocol before delivering + // it to the syncer. It checks that the block has the right format and it performs + // other consensus-specific light verifications like ensuring that the block is signed by + // a valid miner, or that it includes all the data required for a full verification. + ValidateBlockHeader(ctx context.Context, b *types.BlockHeader) (rejectReason string, err error) + + // ValidateBlock is called by the syncer to determine if to accept a block or not. + // + // It performs all the checks needed by the syncer to accept + // the block (signature verifications, VRF checks, message validity, etc.) ValidateBlock(ctx context.Context, b *types.FullBlock) (err error) - ValidateBlockPubsub(ctx context.Context, self bool, msg *pubsub.Message) (pubsub.ValidationResult, string) + + // IsEpochBeyondCurrMax is used to configure the fork rules for longest-chain + // consensus protocols. IsEpochBeyondCurrMax(epoch abi.ChainEpoch) bool + // CreateBlock implements all the logic required to propose and assemble a new Filecoin block. + // + // This function encapsulate all the consensus-specific actions to propose a new block + // such as the ordering of transactions, the inclusion of consensus proofs, the signature + // of the block, etc. CreateBlock(ctx context.Context, w api.Wallet, bt *api.BlockTemplate) (*types.FullBlock, error) } + +// RewardFunc parametrizes the logic for rewards when a message is executed. +// +// Each consensus implementation can set their own reward function. +type RewardFunc func(ctx context.Context, vmi vm.Interface, em stmgr.ExecMonitor, + epoch abi.ChainEpoch, ts *types.TipSet, params *reward.AwardBlockRewardParams) error + +// ValidateBlockPubsub implements the common checks performed by all consensus implementations +// when a block is received through the pubsub channel. +func ValidateBlockPubsub(ctx context.Context, cns Consensus, self bool, msg *pubsub.Message) (pubsub.ValidationResult, string) { + if self { + return validateLocalBlock(ctx, msg) + } + + // track validation time + begin := build.Clock.Now() + defer func() { + log.Debugf("block validation time: %s", build.Clock.Since(begin)) + }() + + stats.Record(ctx, metrics.BlockReceived.M(1)) + + recordFailureFlagPeer := func(what string) { + // bv.Validate will flag the peer in that case + panic(what) + } + + blk, what, err := decodeAndCheckBlock(msg) + if err != nil { + log.Error("got invalid block over pubsub: ", err) + recordFailureFlagPeer(what) + return pubsub.ValidationReject, what + } + + // validate the block meta: the Message CID in the header must match the included messages + err = validateMsgMeta(ctx, blk) + if err != nil { + log.Warnf("error validating message metadata: %s", err) + recordFailureFlagPeer("invalid_block_meta") + return pubsub.ValidationReject, "invalid_block_meta" + } + + reject, err := cns.ValidateBlockHeader(ctx, blk.Header) + if err != nil { + if reject == "" { + log.Warn("ignoring block msg: ", err) + return pubsub.ValidationIgnore, reject + } + recordFailureFlagPeer(reject) + return pubsub.ValidationReject, reject + } + + // all good, accept the block + msg.ValidatorData = blk + stats.Record(ctx, metrics.BlockValidationSuccess.M(1)) + return pubsub.ValidationAccept, "" +} diff --git a/chain/gen/gen.go b/chain/gen/gen.go index fe7480202..674befaef 100644 --- a/chain/gen/gen.go +++ b/chain/gen/gen.go @@ -31,6 +31,7 @@ import ( "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors/policy" "github.com/filecoin-project/lotus/chain/beacon" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" genesis2 "github.com/filecoin-project/lotus/chain/gen/genesis" "github.com/filecoin-project/lotus/chain/rand" @@ -255,7 +256,7 @@ func NewGeneratorWithSectorsAndUpgradeSchedule(numSectors int, us stmgr.UpgradeS //return nil, xerrors.Errorf("creating drand beacon: %w", err) //} - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), sys, us, beac) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), sys, us, beac) if err != nil { return nil, xerrors.Errorf("initing stmgr: %w", err) } diff --git a/chain/gen/genesis/genesis.go b/chain/gen/genesis/genesis.go index 300571f15..241c3501c 100644 --- a/chain/gen/genesis/genesis.go +++ b/chain/gen/genesis/genesis.go @@ -38,7 +38,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/reward" "github.com/filecoin-project/lotus/chain/actors/builtin/system" "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" @@ -494,7 +494,7 @@ func VerifyPreSealedData(ctx context.Context, cs *store.ChainStore, sys vm.Sysca Epoch: 0, Rand: &fakeRand{}, Bstore: cs.StateBlockstore(), - Actors: filcns.NewActorRegistry(), + Actors: consensus.NewActorRegistry(), Syscalls: mkFakedSigSyscalls(sys), CircSupplyCalc: csc, NetworkVersion: nv, diff --git a/chain/gen/genesis/miners.go b/chain/gen/genesis/miners.go index ed237c1a5..5f741fd7c 100644 --- a/chain/gen/genesis/miners.go +++ b/chain/gen/genesis/miners.go @@ -40,7 +40,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/actors/builtin/reward" "github.com/filecoin-project/lotus/chain/actors/policy" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/store" "github.com/filecoin-project/lotus/chain/types" @@ -94,7 +94,7 @@ func SetupStorageMiners(ctx context.Context, cs *store.ChainStore, sys vm.Syscal Epoch: 0, Rand: &fakeRand{}, Bstore: cs.StateBlockstore(), - Actors: filcns.NewActorRegistry(), + Actors: consensus.NewActorRegistry(), Syscalls: mkFakedSigSyscalls(sys), CircSupplyCalc: csc, NetworkVersion: nv, diff --git a/chain/stmgr/forks_test.go b/chain/stmgr/forks_test.go index 98ab647c9..f56fc5ed1 100644 --- a/chain/stmgr/forks_test.go +++ b/chain/stmgr/forks_test.go @@ -31,6 +31,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin" _init "github.com/filecoin-project/lotus/chain/actors/builtin/init" "github.com/filecoin-project/lotus/chain/actors/policy" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" . "github.com/filecoin-project/lotus/chain/stmgr" @@ -128,7 +129,7 @@ func TestForkHeightTriggers(t *testing.T) { } sm, err := NewStateManager( - cg.ChainStore(), filcns.NewTipSetExecutor(), cg.StateManager().VMSys(), UpgradeSchedule{{ + cg.ChainStore(), consensus.NewTipSetExecutor(filcns.RewardFunc), cg.StateManager().VMSys(), UpgradeSchedule{{ Network: network.Version1, Height: testForkHeight, Migration: func(ctx context.Context, sm *StateManager, cache MigrationCache, cb ExecMonitor, @@ -169,7 +170,7 @@ func TestForkHeightTriggers(t *testing.T) { t.Fatal(err) } - inv := filcns.NewActorRegistry() + inv := consensus.NewActorRegistry() registry := builtin.MakeRegistryLegacy([]rtt.VMActor{testActor{}}) inv.Register(actorstypes.Version0, nil, registry) @@ -274,7 +275,7 @@ func testForkRefuseCall(t *testing.T, nullsBefore, nullsAfter int) { var migrationCount int sm, err := NewStateManager( - cg.ChainStore(), filcns.NewTipSetExecutor(), cg.StateManager().VMSys(), UpgradeSchedule{{ + cg.ChainStore(), consensus.NewTipSetExecutor(filcns.RewardFunc), cg.StateManager().VMSys(), UpgradeSchedule{{ Network: network.Version1, Expensive: true, Height: testForkHeight, @@ -287,7 +288,7 @@ func testForkRefuseCall(t *testing.T, nullsBefore, nullsAfter int) { t.Fatal(err) } - inv := filcns.NewActorRegistry() + inv := consensus.NewActorRegistry() registry := builtin.MakeRegistryLegacy([]rtt.VMActor{testActor{}}) inv.Register(actorstypes.Version0, nil, registry) @@ -412,7 +413,7 @@ func TestForkPreMigration(t *testing.T) { counter := make(chan struct{}, 10) sm, err := NewStateManager( - cg.ChainStore(), filcns.NewTipSetExecutor(), cg.StateManager().VMSys(), UpgradeSchedule{{ + cg.ChainStore(), consensus.NewTipSetExecutor(filcns.RewardFunc), cg.StateManager().VMSys(), UpgradeSchedule{{ Network: network.Version1, Height: testForkHeight, Migration: func(ctx context.Context, sm *StateManager, cache MigrationCache, cb ExecMonitor, @@ -509,7 +510,7 @@ func TestForkPreMigration(t *testing.T) { require.NoError(t, sm.Stop(context.Background())) }() - inv := filcns.NewActorRegistry() + inv := consensus.NewActorRegistry() registry := builtin.MakeRegistryLegacy([]rtt.VMActor{testActor{}}) inv.Register(actorstypes.Version0, nil, registry) diff --git a/chain/store/store_test.go b/chain/store/store_test.go index 6dc340737..42a57ab7c 100644 --- a/chain/store/store_test.go +++ b/chain/store/store_test.go @@ -14,6 +14,7 @@ import ( "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/policy" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen" "github.com/filecoin-project/lotus/chain/stmgr" @@ -166,7 +167,7 @@ func TestChainExportImportFull(t *testing.T) { t.Fatal("imported chain differed from exported chain") } - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), nil, filcns.DefaultUpgradeSchedule(), cg.BeaconSchedule()) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), nil, filcns.DefaultUpgradeSchedule(), cg.BeaconSchedule()) if err != nil { t.Fatal(err) } diff --git a/chain/sub/incoming.go b/chain/sub/incoming.go index b8427e036..2eb846d80 100644 --- a/chain/sub/incoming.go +++ b/chain/sub/incoming.go @@ -273,7 +273,7 @@ func (bv *BlockValidator) Validate(ctx context.Context, pid peer.ID, msg *pubsub }() var what string - res, what = bv.consensus.ValidateBlockPubsub(ctx, pid == bv.self, msg) + res, what = consensus.ValidateBlockPubsub(ctx, bv.consensus, pid == bv.self, msg) if res == pubsub.ValidationAccept { // it's a good block! make sure we've only seen it once if count := bv.recvBlocks.add(msg.ValidatorData.(*types.BlockMsg).Cid()); count > 0 { diff --git a/cli/chain.go b/cli/chain.go index 9270c990b..de48afb2b 100644 --- a/cli/chain.go +++ b/cli/chain.go @@ -37,7 +37,7 @@ import ( "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/actors" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" ) @@ -493,7 +493,7 @@ var ChainInspectUsage = &cli.Command{ return err } - mm := filcns.NewActorRegistry().Methods[code][m.Message.Method] // TODO: use remote map + mm := consensus.NewActorRegistry().Methods[code][m.Message.Method] // TODO: use remote map byMethod[mm.Name] += m.Message.GasLimit byMethodC[mm.Name]++ diff --git a/cli/multisig.go b/cli/multisig.go index 38923a04f..1af2a4c9e 100644 --- a/cli/multisig.go +++ b/cli/multisig.go @@ -28,7 +28,7 @@ import ( "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" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" ) @@ -325,7 +325,7 @@ var msigInspectCmd = &cli.Command{ fmt.Fprintf(w, "%d\t%s\t%d\t%s\t%s\t%s(%d)\t%s\n", txid, "pending", len(tx.Approved), target, types.FIL(tx.Value), "new account, unknown method", tx.Method, paramStr) } } else { - method := filcns.NewActorRegistry().Methods[targAct.Code][tx.Method] // TODO: use remote map + method := consensus.NewActorRegistry().Methods[targAct.Code][tx.Method] // TODO: use remote map if decParams && tx.Method != 0 { ptyp := reflect.New(method.Params.Elem()).Interface().(cbg.CBORUnmarshaler) diff --git a/cli/services.go b/cli/services.go index d90da419c..ef257693f 100644 --- a/cli/services.go +++ b/cli/services.go @@ -17,7 +17,7 @@ import ( "github.com/filecoin-project/go-state-types/big" "github.com/filecoin-project/lotus/api" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" ) @@ -88,7 +88,7 @@ func (s *ServicesImpl) DecodeTypedParamsFromJSON(ctx context.Context, to address return nil, err } - methodMeta, found := filcns.NewActorRegistry().Methods[act.Code][method] // TODO: use remote map + methodMeta, found := consensus.NewActorRegistry().Methods[act.Code][method] // TODO: use remote map if !found { return nil, fmt.Errorf("method %d not found on actor %s", method, act.Code) } diff --git a/cli/state.go b/cli/state.go index 434fb1a1c..85d8f26e7 100644 --- a/cli/state.go +++ b/cli/state.go @@ -41,7 +41,7 @@ import ( "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/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -1408,7 +1408,7 @@ func codeStr(c cid.Cid) string { } func getMethod(code cid.Cid, method abi.MethodNum) string { - return filcns.NewActorRegistry().Methods[code][method].Name // todo: use remote + return consensus.NewActorRegistry().Methods[code][method].Name // todo: use remote } func toFil(f types.BigInt) types.FIL { @@ -1424,7 +1424,7 @@ func isVerySlow(t time.Duration) bool { } func JsonParams(code cid.Cid, method abi.MethodNum, params []byte) (string, error) { - p, err := stmgr.GetParamType(filcns.NewActorRegistry(), code, method) // todo use api for correct actor registry + p, err := stmgr.GetParamType(consensus.NewActorRegistry(), code, method) // todo use api for correct actor registry if err != nil { return "", err } @@ -1438,7 +1438,7 @@ func JsonParams(code cid.Cid, method abi.MethodNum, params []byte) (string, erro } func jsonReturn(code cid.Cid, method abi.MethodNum, ret []byte) (string, error) { - methodMeta, found := filcns.NewActorRegistry().Methods[code][method] // TODO: use remote + methodMeta, found := consensus.NewActorRegistry().Methods[code][method] // TODO: use remote if !found { return "", fmt.Errorf("method %d not found on actor %s", method, code) } diff --git a/cmd/lotus-bench/import.go b/cmd/lotus-bench/import.go index c0a3c4a53..537109104 100644 --- a/cmd/lotus-bench/import.go +++ b/cmd/lotus-bench/import.go @@ -34,6 +34,7 @@ import ( "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/blockstore" badgerbs "github.com/filecoin-project/lotus/blockstore/badger" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -227,7 +228,7 @@ var importBenchCmd = &cli.Command{ defer cs.Close() //nolint:errcheck // TODO: We need to supply the actual beacon after v14 - stm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(verifier), filcns.DefaultUpgradeSchedule(), nil) + stm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(verifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } diff --git a/cmd/lotus-shed/balances.go b/cmd/lotus-shed/balances.go index 9ce4faf72..d678c7977 100644 --- a/cmd/lotus-shed/balances.go +++ b/cmd/lotus-shed/balances.go @@ -32,6 +32,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" "github.com/filecoin-project/lotus/chain/actors/builtin/power" "github.com/filecoin-project/lotus/chain/actors/builtin/reward" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/gen/genesis" "github.com/filecoin-project/lotus/chain/state" @@ -512,7 +513,7 @@ var chainBalanceStateCmd = &cli.Command{ cst := cbor.NewCborStore(bs) store := adt.WrapStore(ctx, cst) - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } @@ -736,7 +737,7 @@ var chainPledgeCmd = &cli.Command{ cst := cbor.NewCborStore(bs) store := adt.WrapStore(ctx, cst) - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } diff --git a/cmd/lotus-shed/gas-estimation.go b/cmd/lotus-shed/gas-estimation.go index cf56ea03d..26c4663bf 100644 --- a/cmd/lotus-shed/gas-estimation.go +++ b/cmd/lotus-shed/gas-estimation.go @@ -18,6 +18,7 @@ import ( "github.com/filecoin-project/lotus/build" "github.com/filecoin-project/lotus/chain/beacon" "github.com/filecoin-project/lotus/chain/beacon/drand" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -110,7 +111,7 @@ var gasTraceCmd = &cli.Command{ cs := store.NewChainStore(bs, bs, mds, filcns.Weight, nil) defer cs.Close() //nolint:errcheck - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), shd) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), shd) if err != nil { return err } @@ -211,7 +212,7 @@ var replayOfflineCmd = &cli.Command{ cs := store.NewChainStore(bs, bs, mds, filcns.Weight, nil) defer cs.Close() //nolint:errcheck - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), shd) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), shd) if err != nil { return err } diff --git a/cmd/lotus-shed/invariants.go b/cmd/lotus-shed/invariants.go index e48f301c4..9798d111a 100644 --- a/cmd/lotus-shed/invariants.go +++ b/cmd/lotus-shed/invariants.go @@ -19,6 +19,7 @@ import ( "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -89,7 +90,7 @@ var invariantsCmd = &cli.Command{ cs := store.NewChainStore(bs, bs, mds, filcns.Weight, nil) defer cs.Close() //nolint:errcheck - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } diff --git a/cmd/lotus-shed/migrations.go b/cmd/lotus-shed/migrations.go index ce570c6ac..9cadb50ab 100644 --- a/cmd/lotus-shed/migrations.go +++ b/cmd/lotus-shed/migrations.go @@ -35,6 +35,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/builtin/market" "github.com/filecoin-project/lotus/chain/actors/builtin/miner" "github.com/filecoin-project/lotus/chain/actors/builtin/verifreg" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/stmgr" @@ -112,7 +113,7 @@ var migrationsCmd = &cli.Command{ cs := store.NewChainStore(bs, bs, mds, filcns.Weight, nil) defer cs.Close() //nolint:errcheck - sm, err := stmgr.NewStateManager(cs, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) + sm, err := stmgr.NewStateManager(cs, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } diff --git a/cmd/lotus-shed/msg.go b/cmd/lotus-shed/msg.go index 9096c6a5f..062e077df 100644 --- a/cmd/lotus-shed/msg.go +++ b/cmd/lotus-shed/msg.go @@ -16,7 +16,7 @@ import ( "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/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) @@ -141,7 +141,7 @@ func printMessage(cctx *cli.Context, msg *types.Message) error { return nil } - fmt.Println("Method:", filcns.NewActorRegistry().Methods[toact.Code][msg.Method].Name) // todo use remote + fmt.Println("Method:", consensus.NewActorRegistry().Methods[toact.Code][msg.Method].Name) // todo use remote p, err := lcli.JsonParams(toact.Code, msg.Method, msg.Params) if err != nil { return err diff --git a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go index f984c993d..a4c9fdeaa 100644 --- a/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go +++ b/cmd/lotus-sim/simulation/blockbuilder/blockbuilder.go @@ -16,7 +16,7 @@ import ( "github.com/filecoin-project/lotus/chain/actors/adt" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/account" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" lrand "github.com/filecoin-project/lotus/chain/rand" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/stmgr" @@ -85,7 +85,7 @@ func NewBlockBuilder(ctx context.Context, logger *zap.SugaredLogger, sm *stmgr.S Epoch: parentTs.Height() + 1, Rand: r, Bstore: sm.ChainStore().StateBlockstore(), - Actors: filcns.NewActorRegistry(), + Actors: consensus.NewActorRegistry(), Syscalls: sm.VMSys(), CircSupplyCalc: sm.GetVMCirculatingSupply, NetworkVersion: sm.GetNetworkVersion(ctx, parentTs.Height()+1), diff --git a/cmd/lotus-sim/simulation/node.go b/cmd/lotus-sim/simulation/node.go index f115ffe19..06e837f8f 100644 --- a/cmd/lotus-sim/simulation/node.go +++ b/cmd/lotus-sim/simulation/node.go @@ -10,6 +10,7 @@ import ( "golang.org/x/xerrors" "github.com/filecoin-project/lotus/blockstore" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -105,7 +106,7 @@ func (nd *Node) LoadSim(ctx context.Context, name string) (*Simulation, error) { if err != nil { return nil, xerrors.Errorf("failed to create upgrade schedule for simulation %s: %w", name, err) } - sim.StateManager, err = stmgr.NewStateManager(nd.Chainstore, filcns.NewTipSetExecutor(), vm.Syscalls(mock.Verifier), us, nil) + sim.StateManager, err = stmgr.NewStateManager(nd.Chainstore, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(mock.Verifier), us, nil) if err != nil { return nil, xerrors.Errorf("failed to create state manager for simulation %s: %w", name, err) } @@ -124,7 +125,7 @@ func (nd *Node) CreateSim(ctx context.Context, name string, head *types.TipSet) if err != nil { return nil, err } - sm, err := stmgr.NewStateManager(nd.Chainstore, filcns.NewTipSetExecutor(), vm.Syscalls(mock.Verifier), filcns.DefaultUpgradeSchedule(), nil) + sm, err := stmgr.NewStateManager(nd.Chainstore, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(mock.Verifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return nil, xerrors.Errorf("creating state manager: %w", err) } diff --git a/cmd/lotus-sim/simulation/simulation.go b/cmd/lotus-sim/simulation/simulation.go index dc0b0e263..19ab391dc 100644 --- a/cmd/lotus-sim/simulation/simulation.go +++ b/cmd/lotus-sim/simulation/simulation.go @@ -15,6 +15,7 @@ import ( "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/stmgr" "github.com/filecoin-project/lotus/chain/types" @@ -200,7 +201,7 @@ func (sim *Simulation) SetUpgradeHeight(nv network.Version, epoch abi.ChainEpoch if err != nil { return err } - sm, err := stmgr.NewStateManager(sim.Node.Chainstore, filcns.NewTipSetExecutor(), vm.Syscalls(mock.Verifier), newUpgradeSchedule, nil) + sm, err := stmgr.NewStateManager(sim.Node.Chainstore, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(mock.Verifier), newUpgradeSchedule, nil) if err != nil { return err } diff --git a/cmd/lotus-wallet/interactive.go b/cmd/lotus-wallet/interactive.go index 40c3f8922..96d2ad9f2 100644 --- a/cmd/lotus-wallet/interactive.go +++ b/cmd/lotus-wallet/interactive.go @@ -23,7 +23,7 @@ import ( "github.com/filecoin-project/lotus/api/v0api" "github.com/filecoin-project/lotus/chain/actors/builtin" "github.com/filecoin-project/lotus/chain/actors/builtin/multisig" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/types" lcli "github.com/filecoin-project/lotus/cli" ) @@ -103,7 +103,7 @@ func (c *InteractiveWallet) WalletSign(ctx context.Context, k address.Address, m return xerrors.Errorf("looking up dest actor: %w", err) } - fmt.Println("Method:", filcns.NewActorRegistry().Methods[toact.Code][cmsg.Method].Name) + fmt.Println("Method:", consensus.NewActorRegistry().Methods[toact.Code][cmsg.Method].Name) p, err := lcli.JsonParams(toact.Code, cmsg.Method, cmsg.Params) if err != nil { return err @@ -125,7 +125,7 @@ func (c *InteractiveWallet) WalletSign(ctx context.Context, k address.Address, m return xerrors.Errorf("looking up msig dest actor: %w", err) } - fmt.Println("\tMultiSig Proposal Method:", filcns.NewActorRegistry().Methods[toact.Code][mp.Method].Name) // todo use remote + fmt.Println("\tMultiSig Proposal Method:", consensus.NewActorRegistry().Methods[toact.Code][mp.Method].Name) // todo use remote p, err := lcli.JsonParams(toact.Code, mp.Method, mp.Params) if err != nil { return err diff --git a/cmd/lotus/daemon.go b/cmd/lotus/daemon.go index 59f9d7258..4a4a691ee 100644 --- a/cmd/lotus/daemon.go +++ b/cmd/lotus/daemon.go @@ -33,6 +33,7 @@ import ( lapi "github.com/filecoin-project/lotus/api" "github.com/filecoin-project/lotus/build" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/stmgr" "github.com/filecoin-project/lotus/chain/store" @@ -539,7 +540,7 @@ func ImportChain(ctx context.Context, r repo.Repo, fname string, snapshot bool) } // TODO: We need to supply the actual beacon after v14 - stm, err := stmgr.NewStateManager(cst, filcns.NewTipSetExecutor(), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) + stm, err := stmgr.NewStateManager(cst, consensus.NewTipSetExecutor(filcns.RewardFunc), vm.Syscalls(ffiwrapper.ProofVerifier), filcns.DefaultUpgradeSchedule(), nil) if err != nil { return err } diff --git a/cmd/tvx/extract_many.go b/cmd/tvx/extract_many.go index bc5ad2a30..96ce2473b 100644 --- a/cmd/tvx/extract_many.go +++ b/cmd/tvx/extract_many.go @@ -19,7 +19,7 @@ import ( "github.com/filecoin-project/go-state-types/abi" "github.com/filecoin-project/go-state-types/exitcode" - "github.com/filecoin-project/lotus/chain/consensus/filcns" + "github.com/filecoin-project/lotus/chain/consensus" ) var extractManyFlags struct { @@ -159,7 +159,7 @@ func runExtractMany(c *cli.Context) error { } // Lookup the method in actor method table. - if m, ok := filcns.NewActorRegistry().Methods[codeCid]; !ok { + if m, ok := consensus.NewActorRegistry().Methods[codeCid]; !ok { return fmt.Errorf("unrecognized actor: %s", actorcode) } else if methodnum >= len(m) { return fmt.Errorf("unrecognized method number for actor %s: %d", actorcode, methodnum) diff --git a/conformance/driver.go b/conformance/driver.go index fc03a48d8..ef9165e32 100644 --- a/conformance/driver.go +++ b/conformance/driver.go @@ -20,6 +20,7 @@ import ( "github.com/filecoin-project/lotus/blockstore" "github.com/filecoin-project/lotus/chain/actors/builtin" + "github.com/filecoin-project/lotus/chain/consensus" "github.com/filecoin-project/lotus/chain/consensus/filcns" "github.com/filecoin-project/lotus/chain/state" "github.com/filecoin-project/lotus/chain/stmgr" @@ -105,7 +106,7 @@ func (d *Driver) ExecuteTipset(bs blockstore.Blockstore, ds ds.Batching, params syscalls = vm.Syscalls(ffiwrapper.ProofVerifier) cs = store.NewChainStore(bs, bs, ds, filcns.Weight, nil) - tse = filcns.NewTipSetExecutor() + tse = consensus.NewTipSetExecutor(filcns.RewardFunc) sm, err = stmgr.NewStateManager(cs, tse, syscalls, filcns.DefaultUpgradeSchedule(), nil) ) if err != nil { @@ -122,7 +123,7 @@ func (d *Driver) ExecuteTipset(bs blockstore.Blockstore, ds ds.Batching, params defer cs.Close() //nolint:errcheck - blocks := make([]filcns.FilecoinBlockMessages, 0, len(tipset.Blocks)) + blocks := make([]consensus.FilecoinBlockMessages, 0, len(tipset.Blocks)) for _, b := range tipset.Blocks { sb := store.BlockMessages{ Miner: b.MinerAddr, @@ -144,7 +145,7 @@ func (d *Driver) ExecuteTipset(bs blockstore.Blockstore, ds ds.Batching, params sb.BlsMessages = append(sb.BlsMessages, msg) } } - blocks = append(blocks, filcns.FilecoinBlockMessages{ + blocks = append(blocks, consensus.FilecoinBlockMessages{ BlockMessages: sb, WinCount: b.WinCount, }) @@ -264,7 +265,7 @@ func (d *Driver) ExecuteMessage(bs blockstore.Blockstore, params ExecuteMessageP return nil, cid.Undef, err } - invoker := filcns.NewActorRegistry() + invoker := consensus.NewActorRegistry() av, _ := actorstypes.VersionForNetwork(params.NetworkVersion) registry := builtin.MakeRegistryLegacy([]rtt.VMActor{chaos.Actor{}}) invoker.Register(av, nil, registry) @@ -282,7 +283,7 @@ func (d *Driver) ExecuteMessage(bs blockstore.Blockstore, params ExecuteMessageP if err != nil { return nil, cid.Undef, err } - invoker := filcns.NewActorRegistry() + invoker := consensus.NewActorRegistry() lvm.SetInvoker(invoker) vmi = lvm } diff --git a/lib/sigs/secp/init.go b/lib/sigs/secp/init.go index 05238fdcd..49d5cef17 100644 --- a/lib/sigs/secp/init.go +++ b/lib/sigs/secp/init.go @@ -6,7 +6,7 @@ import ( "github.com/minio/blake2b-simd" "github.com/filecoin-project/go-address" - "github.com/filecoin-project/go-crypto" + 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/node/builder_chain.go b/node/builder_chain.go index 2201be2e6..87e994c4d 100644 --- a/node/builder_chain.go +++ b/node/builder_chain.go @@ -75,7 +75,7 @@ var ChainNode = Options( // Consensus: Chain storage/access Override(new(chain.Genesis), chain.LoadGenesis), Override(new(store.WeightFunc), filcns.Weight), - Override(new(stmgr.Executor), filcns.NewTipSetExecutor()), + Override(new(stmgr.Executor), consensus.NewTipSetExecutor(filcns.RewardFunc)), Override(new(consensus.Consensus), filcns.NewFilecoinExpectedConsensus), Override(new(*store.ChainStore), modules.ChainStore), Override(new(*stmgr.StateManager), modules.StateManager),