Merge pull request #9481 from consensus-shipyard/adlrocha/cns-iface-master
IPC: Abstract common consensus functions and consensus interface
This commit is contained in:
commit
9a46682d9d
@ -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: <multisig address>, <start epoch>, <end epoch>
|
||||
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: <required number of senders>, <approving addresses>, <unlock duration>
|
||||
//<initial balance>, <sender address of the create msg>, <gas price>
|
||||
// <initial balance>, <sender address of the create msg>, <gas price>
|
||||
MsigCreate(context.Context, uint64, []address.Address, abi.ChainEpoch, types.BigInt, address.Address, types.BigInt) (*MessagePrototype, error) //perm:sign
|
||||
|
||||
// MsigPropose proposes a multisig message
|
||||
|
@ -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:
|
||||
|
@ -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: <multisig address>, <start epoch>, <end epoch>
|
||||
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: <required number of senders>, <approving addresses>, <unlock duration>
|
||||
//<initial balance>, <sender address of the create msg>, <gas price>
|
||||
// <initial balance>, <sender address of the create msg>, <gas price>
|
||||
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: <multisig address>, <recipient address>, <value to transfer>,
|
||||
|
482
chain/consensus/common.go
Normal file
482
chain/consensus/common.go
Normal file
@ -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
|
||||
}
|
@ -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)
|
||||
}
|
||||
}
|
||||
|
@ -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{}
|
||||
|
@ -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,
|
||||
|
@ -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, ""
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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,
|
||||
|
@ -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,
|
||||
|
@ -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)
|
||||
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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 {
|
||||
|
@ -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]++
|
||||
|
@ -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)
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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),
|
||||
|
@ -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)
|
||||
}
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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)
|
||||
|
@ -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
|
||||
}
|
||||
|
@ -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"
|
||||
|
@ -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),
|
||||
|
Loading…
Reference in New Issue
Block a user