Merge pull request #230 from filecoin-project/feat/incremental-deals

Incremental deal payments
This commit is contained in:
Łukasz Magiera 2019-09-26 03:00:05 +02:00 committed by GitHub
commit 23138d9bd0
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
27 changed files with 377 additions and 221 deletions

View File

@ -122,7 +122,7 @@ type FullNode interface {
PaychStatus(context.Context, address.Address) (*PaychStatus, error)
PaychClose(context.Context, address.Address) (cid.Cid, error)
PaychAllocateLane(ctx context.Context, ch address.Address) (uint64, error)
PaychNewPayment(ctx context.Context, from, to address.Address, amount types.BigInt, extra *types.ModVerifyParams, tl uint64, minClose uint64) (*PaymentInfo, error)
PaychNewPayment(ctx context.Context, from, to address.Address, vouchers []VoucherSpec) (*PaymentInfo, error)
PaychVoucherCheckValid(context.Context, address.Address, *types.SignedVoucher) error
PaychVoucherCheckSpendable(context.Context, address.Address, *types.SignedVoucher, []byte, []byte) (bool, error)
PaychVoucherCreate(context.Context, address.Address, types.BigInt, uint64) (*types.SignedVoucher, error)
@ -229,7 +229,15 @@ type ChannelInfo struct {
type PaymentInfo struct {
Channel address.Address
ChannelMessage *cid.Cid
Voucher *types.SignedVoucher
Vouchers []*types.SignedVoucher
}
type VoucherSpec struct {
Amount types.BigInt
TimeLock uint64
MinClose uint64
Extra *types.ModVerifyParams
}
type MinerPower struct {

View File

@ -3,15 +3,14 @@ package api
import (
"context"
sectorbuilder "github.com/filecoin-project/go-sectorbuilder"
"github.com/ipfs/go-cid"
"github.com/libp2p/go-libp2p-core/network"
"github.com/libp2p/go-libp2p-core/peer"
"github.com/filecoin-project/go-lotus/chain/address"
"github.com/filecoin-project/go-lotus/chain/store"
"github.com/filecoin-project/go-lotus/chain/types"
sectorbuilder "github.com/filecoin-project/go-sectorbuilder"
"github.com/ipfs/go-cid"
"github.com/libp2p/go-libp2p-core/peer"
)
// All permissions are listed in permissioned.go
@ -86,19 +85,19 @@ type FullNodeStruct struct {
StateGetActor func(context.Context, address.Address, *types.TipSet) (*types.Actor, error) `perm:"read"`
StateReadState func(context.Context, *types.Actor, *types.TipSet) (*ActorState, error) `perm:"read"`
PaychGet func(ctx context.Context, from, to address.Address, ensureFunds types.BigInt) (*ChannelInfo, error) `perm:"sign"`
PaychList func(context.Context) ([]address.Address, error) `perm:"read"`
PaychStatus func(context.Context, address.Address) (*PaychStatus, error) `perm:"read"`
PaychClose func(context.Context, address.Address) (cid.Cid, error) `perm:"sign"`
PaychAllocateLane func(context.Context, address.Address) (uint64, error) `perm:"sign"`
PaychNewPayment func(ctx context.Context, from, to address.Address, amount types.BigInt, extra *types.ModVerifyParams, tl uint64, minClose uint64) (*PaymentInfo, error) `perm:"sign"`
PaychVoucherCheck func(context.Context, *types.SignedVoucher) error `perm:"read"`
PaychVoucherCheckValid func(context.Context, address.Address, *types.SignedVoucher) error `perm:"read"`
PaychVoucherCheckSpendable func(context.Context, address.Address, *types.SignedVoucher, []byte, []byte) (bool, error) `perm:"read"`
PaychVoucherAdd func(context.Context, address.Address, *types.SignedVoucher, []byte, types.BigInt) (types.BigInt, error) `perm:"write"`
PaychVoucherCreate func(context.Context, address.Address, types.BigInt, uint64) (*types.SignedVoucher, error) `perm:"sign"`
PaychVoucherList func(context.Context, address.Address) ([]*types.SignedVoucher, error) `perm:"write"`
PaychVoucherSubmit func(context.Context, address.Address, *types.SignedVoucher) (cid.Cid, error) `perm:"sign"`
PaychGet func(ctx context.Context, from, to address.Address, ensureFunds types.BigInt) (*ChannelInfo, error) `perm:"sign"`
PaychList func(context.Context) ([]address.Address, error) `perm:"read"`
PaychStatus func(context.Context, address.Address) (*PaychStatus, error) `perm:"read"`
PaychClose func(context.Context, address.Address) (cid.Cid, error) `perm:"sign"`
PaychAllocateLane func(context.Context, address.Address) (uint64, error) `perm:"sign"`
PaychNewPayment func(ctx context.Context, from, to address.Address, vouchers []VoucherSpec) (*PaymentInfo, error) `perm:"sign"`
PaychVoucherCheck func(context.Context, *types.SignedVoucher) error `perm:"read"`
PaychVoucherCheckValid func(context.Context, address.Address, *types.SignedVoucher) error `perm:"read"`
PaychVoucherCheckSpendable func(context.Context, address.Address, *types.SignedVoucher, []byte, []byte) (bool, error) `perm:"read"`
PaychVoucherAdd func(context.Context, address.Address, *types.SignedVoucher, []byte, types.BigInt) (types.BigInt, error) `perm:"write"`
PaychVoucherCreate func(context.Context, address.Address, types.BigInt, uint64) (*types.SignedVoucher, error) `perm:"sign"`
PaychVoucherList func(context.Context, address.Address) ([]*types.SignedVoucher, error) `perm:"write"`
PaychVoucherSubmit func(context.Context, address.Address, *types.SignedVoucher) (cid.Cid, error) `perm:"sign"`
}
}
@ -363,8 +362,8 @@ func (c *FullNodeStruct) PaychAllocateLane(ctx context.Context, ch address.Addre
return c.Internal.PaychAllocateLane(ctx, ch)
}
func (c *FullNodeStruct) PaychNewPayment(ctx context.Context, from, to address.Address, amount types.BigInt, extra *types.ModVerifyParams, tl uint64, minClose uint64) (*PaymentInfo, error) {
return c.Internal.PaychNewPayment(ctx, from, to, amount, extra, tl, minClose)
func (c *FullNodeStruct) PaychNewPayment(ctx context.Context, from, to address.Address, vouchers []VoucherSpec) (*PaymentInfo, error) {
return c.Internal.PaychNewPayment(ctx, from, to, vouchers)
}
func (c *FullNodeStruct) PaychVoucherSubmit(ctx context.Context, ch address.Address, sv *types.SignedVoucher) (cid.Cid, error) {

View File

@ -1,7 +0,0 @@
package build
// Seconds
const BlockDelay = 3
// Seconds
const AllowableClockDrift = BlockDelay * 2

View File

@ -2,10 +2,16 @@ package build
// Core network constants
// /////
// Storage
const UnixfsChunkSize uint64 = 1 << 20
const UnixfsLinksPerLevel = 1024
const SectorSize = 1024
const SectorSize = 16 << 20
// /////
// Payments
// Blocks
const PaymentChannelClosingDelay = 6 * 60 * 2 // six hours
@ -13,20 +19,42 @@ const PaymentChannelClosingDelay = 6 * 60 * 2 // six hours
// Blocks
const DealVoucherSkewLimit = 10
// Blocks
const MinDealVoucherIncrement = ProvingPeriodDuration
const MaxVouchersPerDeal = 768 // roughly one voucher per 10h over a year
// /////
// Consensus / Network
// Seconds
const BlockDelay = 3
// Seconds
const AllowableClockDrift = BlockDelay * 2
// Blocks
const ForkLengthThreshold = 20
// /////
// Proofs / Mining
// Blocks
const RandomnessLookback = 20
// Blocks
const ProvingPeriodDuration = 10
// Blocks
const PoSTChallangeTime = 5
const PowerCollateralProportion = 20
const PerCapitaCollateralProportion = 5
const CollateralPrecision = 100
// /////
// Devnet settings
const TotalFilecoin = 2000000000
const FilecoinPrecision = 1000000000000000000

View File

@ -37,7 +37,7 @@ type PaymentChannelActorState struct {
ClosingAt uint64
MinCloseHeight uint64
// TODO: needs to be map[uint64]*LaneState
// TODO: needs to be map[uint64]*laneState
// waiting on refmt#35 to be fixed
LaneStates map[string]*LaneState
}

View File

@ -1894,7 +1894,7 @@ func (t *PaymentChannelActorState) MarshalCBOR(w io.Writer) error {
return err
}
// t.t.LaneStates (map[string]*actors.LaneState)
// t.t.LaneStates (map[string]*actors.laneState)
if err := cbg.CborWriteHeader(w, cbg.MajMap, uint64(len(t.LaneStates))); err != nil {
return err
}
@ -1978,7 +1978,7 @@ func (t *PaymentChannelActorState) UnmarshalCBOR(r io.Reader) error {
return fmt.Errorf("wrong type for uint64 field")
}
t.MinCloseHeight = extra
// t.t.LaneStates (map[string]*actors.LaneState)
// t.t.LaneStates (map[string]*actors.laneState)
maj, extra, err = cbg.CborReadHeader(br)
if err != nil {

View File

@ -3,15 +3,15 @@ package deals
import (
"bytes"
"context"
"github.com/filecoin-project/go-lotus/api"
"github.com/filecoin-project/go-lotus/build"
"github.com/filecoin-project/go-sectorbuilder/sealing_state"
"github.com/filecoin-project/go-sectorbuilder/sealing_state"
cbor "github.com/ipfs/go-ipld-cbor"
"github.com/ipfs/go-merkledag"
unixfile "github.com/ipfs/go-unixfs/file"
"golang.org/x/xerrors"
"github.com/filecoin-project/go-lotus/api"
"github.com/filecoin-project/go-lotus/build"
"github.com/filecoin-project/go-lotus/chain/actors"
"github.com/filecoin-project/go-lotus/chain/types"
"github.com/filecoin-project/go-lotus/lib/sectorbuilder"
@ -42,63 +42,101 @@ func (h *Handler) handle(ctx context.Context, deal MinerDeal, cb minerHandlerFun
// ACCEPTED
func (h *Handler) validateVouchers(ctx context.Context, deal MinerDeal) error {
curHead, err := h.full.ChainHead(ctx)
func (h *Handler) checkVoucher(ctx context.Context, deal MinerDeal, voucher *types.SignedVoucher, lane uint64, maxClose uint64, amount types.BigInt) error {
err := h.full.PaychVoucherCheckValid(ctx, deal.Proposal.Payment.PayChActor, voucher)
if err != nil {
return err
}
if voucher.Extra == nil {
return xerrors.New("voucher.Extra not set")
}
if voucher.Extra.Actor != deal.Proposal.MinerAddress {
return xerrors.Errorf("extra params actor didn't match miner address in proposal: '%s' != '%s'", voucher.Extra.Actor, deal.Proposal.MinerAddress)
}
if voucher.Extra.Method != actors.MAMethods.PaymentVerifyInclusion {
return xerrors.Errorf("expected extra method %d, got %d", actors.MAMethods.PaymentVerifyInclusion, voucher.Extra.Method)
}
var inclChallenge actors.PieceInclVoucherData
if err := cbor.DecodeInto(voucher.Extra.Data, &inclChallenge); err != nil {
return xerrors.Errorf("failed to decode storage voucher data for verification: %w", err)
}
if inclChallenge.PieceSize.Uint64() != deal.Proposal.Size {
return xerrors.Errorf("paych challenge piece size didn't match deal proposal size: %d != %d", inclChallenge.PieceSize.Uint64(), deal.Proposal.Size)
}
if !bytes.Equal(inclChallenge.CommP, deal.Proposal.CommP) {
return xerrors.New("paych challenge commP didn't match deal proposal")
}
if voucher.MinCloseHeight > maxClose {
return xerrors.Errorf("MinCloseHeight too high (%d), max expected: %d", voucher.MinCloseHeight, maxClose)
}
if voucher.TimeLock > maxClose {
return xerrors.Errorf("TimeLock too high (%d), max expected: %d", voucher.TimeLock, maxClose)
}
if len(voucher.Merges) > 0 {
return xerrors.New("didn't expect any merges")
}
if voucher.Amount.LessThan(amount) {
return xerrors.Errorf("not enough funds in the voucher: %s < %s; vl=%d", voucher.Amount, amount, len(deal.Proposal.Payment.Vouchers))
}
if voucher.Lane != lane {
return xerrors.Errorf("expected all vouchers on lane %d, found voucher on lane %d", lane, voucher.Lane)
}
return nil
}
func (h *Handler) consumeVouchers(ctx context.Context, deal MinerDeal) error {
curHead, err := h.full.ChainHead(ctx)
if err != nil {
return err
}
if len(deal.Proposal.Payment.Vouchers) == 0 {
return xerrors.Errorf("no payment vouchers for deal")
}
increments := deal.Proposal.Duration / uint64(len(deal.Proposal.Payment.Vouchers))
startH := deal.Proposal.Payment.Vouchers[0].TimeLock - (deal.Proposal.Duration / increments)
if startH > curHead.Height()+build.DealVoucherSkewLimit {
return xerrors.Errorf("deal starts too far into the future")
}
vspec := VoucherSpec(deal.Proposal.Duration, deal.Proposal.TotalPrice, startH, nil)
lane := deal.Proposal.Payment.Vouchers[0].Lane
for i, voucher := range deal.Proposal.Payment.Vouchers {
err := h.full.PaychVoucherCheckValid(ctx, deal.Proposal.Payment.PayChActor, voucher)
if err != nil {
maxClose := curHead.Height() + (increments * uint64(i+1)) + build.DealVoucherSkewLimit
if err := h.checkVoucher(ctx, deal, voucher, lane, maxClose, vspec[i].Amount); err != nil {
return xerrors.Errorf("validating payment voucher %d: %w", i, err)
}
}
if voucher.Extra == nil {
return xerrors.Errorf("validating payment voucher %d: voucher.Extra not set")
}
minPrice := types.BigMul(types.BigMul(h.pricePerByteBlock, types.NewInt(deal.Proposal.Size)), types.NewInt(deal.Proposal.Duration))
if types.BigCmp(minPrice, deal.Proposal.TotalPrice) > 0 {
return xerrors.Errorf("minimum price: %s", minPrice)
}
if voucher.Extra.Actor != deal.Proposal.MinerAddress {
return xerrors.Errorf("validating payment voucher %d: extra params actor didn't match miner address in proposal: '%s' != '%s'", i, voucher.Extra.Actor, deal.Proposal.MinerAddress)
}
if voucher.Extra.Method != actors.MAMethods.PaymentVerifyInclusion {
return xerrors.Errorf("validating payment voucher %d: expected extra method %d, got %d", i, actors.MAMethods.PaymentVerifyInclusion, voucher.Extra.Method)
}
prevAmt := types.NewInt(0)
var inclChallenge actors.PieceInclVoucherData
if err := cbor.DecodeInto(voucher.Extra.Data, &inclChallenge); err != nil {
return xerrors.Errorf("validating payment voucher %d: failed to decode storage voucher data for verification: %w", i, err)
}
if inclChallenge.PieceSize.Uint64() != deal.Proposal.Size {
return xerrors.Errorf("validating payment voucher %d: paych challenge piece size didn't match deal proposal size: %d != %d", i, inclChallenge.PieceSize.Uint64(), deal.Proposal.Size)
}
if !bytes.Equal(inclChallenge.CommP, deal.Proposal.CommP) {
return xerrors.Errorf("validating payment voucher %d: paych challenge commP didn't match deal proposal", i)
}
maxClose := curHead.Height() + deal.Proposal.Duration + build.DealVoucherSkewLimit
if voucher.MinCloseHeight > maxClose {
return xerrors.Errorf("validating payment voucher %d: MinCloseHeight too high (%d), max expected: %d", i, voucher.MinCloseHeight, maxClose)
}
if voucher.TimeLock > maxClose {
return xerrors.Errorf("validating payment voucher %d: TimeLock too high (%d), max expected: %d", i, voucher.TimeLock, maxClose)
}
if len(voucher.Merges) > 0 {
return xerrors.Errorf("validating payment voucher %d: didn't expect any merges", i)
}
// TODO: make sure that current laneStatus.Amount == 0
if voucher.Amount.LessThan(deal.Proposal.TotalPrice) {
return xerrors.Errorf("validating payment voucher %d: not enough funds in the voucher", i)
}
minPrice := types.BigMul(types.BigMul(h.pricePerByteBlock, types.NewInt(deal.Proposal.Size)), types.NewInt(deal.Proposal.Duration))
if types.BigCmp(minPrice, deal.Proposal.TotalPrice) > 0 {
return xerrors.Errorf("validating payment voucher %d: minimum price: %s", i, minPrice)
for i, voucher := range deal.Proposal.Payment.Vouchers {
delta, err := h.full.PaychVoucherAdd(ctx, deal.Proposal.Payment.PayChActor, voucher, nil, types.BigSub(vspec[i].Amount, prevAmt))
if err != nil {
return xerrors.Errorf("consuming payment voucher %d: %w", i, err)
}
prevAmt = types.BigAdd(prevAmt, delta)
}
return nil
@ -120,17 +158,10 @@ func (h *Handler) accept(ctx context.Context, deal MinerDeal) (func(*MinerDeal),
}
}
if err := h.validateVouchers(ctx, deal); err != nil {
if err := h.consumeVouchers(ctx, deal); err != nil {
return nil, err
}
for i, voucher := range deal.Proposal.Payment.Vouchers {
// TODO: Set correct minAmount
if _, err := h.full.PaychVoucherAdd(ctx, deal.Proposal.Payment.PayChActor, voucher, nil, types.NewInt(0)); err != nil {
return nil, xerrors.Errorf("consuming payment voucher %d: %w", i, err)
}
}
log.Info("fetching data for a deal")
err := h.sendSignedResponse(StorageDealResponse{
State: api.DealAccepted,

31
chain/deals/vouchers.go Normal file
View File

@ -0,0 +1,31 @@
package deals
import (
"github.com/filecoin-project/go-lotus/api"
"github.com/filecoin-project/go-lotus/build"
"github.com/filecoin-project/go-lotus/chain/types"
)
func VoucherSpec(blocksDuration uint64, price types.BigInt, start uint64, extra *types.ModVerifyParams) []api.VoucherSpec {
nVouchers := blocksDuration / build.MinDealVoucherIncrement
if nVouchers < 1 {
nVouchers = 1
}
if nVouchers > build.MaxVouchersPerDeal {
nVouchers = build.MaxVouchersPerDeal
}
hIncrements := blocksDuration / nVouchers
vouchers := make([]api.VoucherSpec, nVouchers)
for i := uint64(0); i < nVouchers; i++ {
vouchers[i] = api.VoucherSpec{
Amount: types.BigDiv(types.BigMul(price, types.NewInt(i+1)), types.NewInt(nVouchers)),
TimeLock: start + (hIncrements * (i + 1)),
MinClose: start + (hIncrements * (i + 1)),
Extra: extra,
}
}
return vouchers
}

View File

@ -129,8 +129,8 @@ func NewGenerator() (*ChainGen, error) {
}
genb, err := MakeGenesisBlock(bs, map[address.Address]types.BigInt{
worker: types.NewInt(50000000),
banker: types.NewInt(90000000),
worker: types.NewInt(5000000000),
banker: types.NewInt(9000000000),
}, minercfg, 100000)
if err != nil {
return nil, xerrors.Errorf("make genesis block failed: %w", err)

View File

@ -202,7 +202,7 @@ func SetupStorageMiners(ctx context.Context, cs *store.ChainStore, sroot cid.Cid
// TODO: hardcoding 7000000 here is a little fragile, it changes any
// time anyone changes the initial account allocations
rval, err := doExecValue(ctx, vm, actors.StorageMarketAddress, owner, types.NewInt(7000000), actors.SMAMethods.CreateStorageMiner, params)
rval, err := doExecValue(ctx, vm, actors.StorageMarketAddress, owner, types.NewInt(700000000), actors.SMAMethods.CreateStorageMiner, params)
if err != nil {
return cid.Undef, xerrors.Errorf("failed to create genesis miner: %w", err)
}

View File

@ -3,12 +3,11 @@ package stmgr
import (
"context"
"fmt"
"github.com/filecoin-project/go-lotus/chain/actors"
"github.com/filecoin-project/go-lotus/chain/types"
"github.com/filecoin-project/go-lotus/chain/vm"
cid "github.com/ipfs/go-cid"
"github.com/ipfs/go-cid"
"golang.org/x/xerrors"
)

View File

@ -182,5 +182,4 @@ func (sm *StateManager) LoadActorState(ctx context.Context, a address.Address, o
}
return act, nil
}

View File

@ -194,7 +194,7 @@ func (cs *ChainStore) PutTipSet(ts *FullTipSet) error {
if err != nil {
return xerrors.Errorf("errored while expanding tipset: %w", err)
}
fmt.Printf("expanded %s into %s\n", ts.TipSet().Cids(), expanded.Cids())
log.Debugf("expanded %s into %s\n", ts.TipSet().Cids(), expanded.Cids())
if err := cs.MaybeTakeHeavierTipSet(expanded); err != nil {
return errors.Wrap(err, "MaybeTakeHeavierTipSet failed in PutTipSet")
@ -223,7 +223,7 @@ func (cs *ChainStore) MaybeTakeHeavierTipSet(ts *types.TipSet) error {
log.Warn("no heaviest tipset found, using %s", ts.Cids())
}
log.Infof("New heaviest tipset! %s", ts.Cids())
log.Debugf("New heaviest tipset! %s", ts.Cids())
cs.heaviest = ts
if err := cs.writeHead(ts); err != nil {
@ -345,7 +345,7 @@ func (cs *ChainStore) addToTipSetTracker(b *types.BlockHeader) error {
tss := cs.tipsets[b.Height]
for _, oc := range tss {
if oc == b.Cid() {
log.Warn("tried to add block to tipset tracker that was already there")
log.Debug("tried to add block to tipset tracker that was already there")
return nil
}
}
@ -727,7 +727,6 @@ func (cs *ChainStore) GetRandomness(ctx context.Context, blks []cid.Cid, tickets
lt := int64(len(mtb.Tickets))
if nv < lt {
t := mtb.Tickets[lt-(1+nv)]
log.Infof("Returning randomness: H:%d, t:%d, mtb:%s", nts.Height(), lt-(1+nv), mtb.Cid())
return t.VDFResult, nil
}

View File

@ -560,7 +560,7 @@ loop:
func (syncer *Syncer) syncMessagesAndCheckState(headers []*types.TipSet) error {
return syncer.iterFullTipsets(headers, func(fts *store.FullTipSet) error {
log.Warnf("validating tipset (heigt=%d, size=%d)", fts.TipSet().Height(), len(fts.TipSet().Cids()))
log.Debugf("validating tipset (heigt=%d, size=%d)", fts.TipSet().Height(), len(fts.TipSet().Cids()))
if err := syncer.ValidateTipSet(context.TODO(), fts); err != nil {
log.Errorf("failed to validate tipset: %s", err)
return xerrors.Errorf("message processing failed: %w", err)

View File

@ -84,16 +84,13 @@ class Address extends React.Component {
return info
}
add200k = async () => {
[...Array(10).keys()].map(() => async () => await this.props.add20k(this.props.addr)).reduce(async (p, c) => [await p, await c()], Promise.resolve(null))
}
render() {
let add20k = <span/>
if(this.props.add20k) {
add20k = <span>&nbsp;<a href="#" onClick={() => this.props.add20k(this.props.addr)}>[+20k]</a></span>
if(this.props.addN) {
add20k = <span>&nbsp;<a href="#" onClick={() => this.props.addN(this.props.addr, 200000)}>[+200k]</a></span>
if (this.props.add10k) {
add20k = <span>{add20k}&nbsp;<a href="#" onClick={this.add200k}>[+200k]</a></span>
add20k = <span>{add20k}&nbsp;<a href="#" onClick={() => this.props.addN(this.props.addr, 2000000)}>[+2M]</a></span>
add20k = <span>{add20k}&nbsp;<a href="#" onClick={() => this.props.addN(this.props.addr, 20000000)}>[+20M]</a></span>
}
}
let addr = truncAddr(this.props.addr, this.props.short ? 12 : 17)

View File

@ -15,7 +15,6 @@ class FullNode extends React.Component {
this.newSecpAddr = this.newSecpAddr.bind(this)
this.newBLSAddr = this.newBLSAddr.bind(this)
this.startStorageMiner = this.startStorageMiner.bind(this)
this.add20k = this.add20k.bind(this)
this.explorer = this.explorer.bind(this)
this.client = this.client.bind(this)
this.stop = this.stop.bind(this)
@ -84,10 +83,6 @@ class FullNode extends React.Component {
this.props.spawnStorageNode(this.props.node.Repo, this.props.client)
}
async add20k(to) {
await this.props.give20k(to)
}
explorer() {
this.props.mountWindow((onClose) => <ChainExplorer onClose={onClose} ts={this.state.tipset} client={this.props.client} mountWindow={this.props.mountWindow}/>)
}
@ -123,14 +118,14 @@ class FullNode extends React.Component {
let storageMine = <a href="#" onClick={this.startStorageMiner} hidden={!this.props.spawnStorageNode}>[Spawn Storage Miner]</a>
let addresses = this.state.addrs.map((addr) => {
let line = <Address client={this.props.client} add20k={this.add20k} add10k={true} nonce={true} addr={addr} mountWindow={this.props.mountWindow}/>
let line = <Address client={this.props.client} addN={this.props.giveN} add10k={true} nonce={true} addr={addr} mountWindow={this.props.mountWindow}/>
if (this.state.defaultAddr === addr) {
line = <b>{line}</b>
}
return <div key={addr}>{line}</div>
})
let paychannels = this.state.paychs.map((addr, ak) => {
const line = <Address client={this.props.client} add20k={this.add20k} add10k={true} addr={addr} mountWindow={this.props.mountWindow}/>
const line = <Address client={this.props.client} addN={this.addN} add10k={true} addr={addr} mountWindow={this.props.mountWindow}/>
const vouchers = this.state.vouchers[ak].map(voucher => {
let extra = <span></span>
if(voucher.Extra) {

View File

@ -27,7 +27,7 @@ class NodeList extends React.Component {
this.spawnStorageNode = this.spawnStorageNode.bind(this)
this.connMgr = this.connMgr.bind(this)
this.consensus = this.consensus.bind(this)
this.transfer20kFrom1 = this.transfer20kFrom1.bind(this)
this.transferNFrom1 = this.transferNFrom1.bind(this)
this.getNodes()
}
@ -52,7 +52,7 @@ class NodeList extends React.Component {
node={{...node}}
client={client}
pondClient={this.props.client}
give20k={this.transfer20kFrom1}
giveN={this.transferNFrom1}
mountWindow={this.props.mountWindow}
spawnStorageNode={this.spawnStorageNode}
stop={this.stopNode(node.ID, onClose)}
@ -81,7 +81,7 @@ class NodeList extends React.Component {
this.setState({existingLoaded: true, nodes: nodes})
}
async transfer20kFrom1(to) {
async transferNFrom1(to, n) {
const addrss = await this.state.nodes[1].conn.call('Filecoin.WalletList', [])
const [bestaddr, bal] = await addrss.map(async addr => {
let balance = 0
@ -96,7 +96,7 @@ class NodeList extends React.Component {
await pushMessage(this.state.nodes[1].conn, bestaddr, {
To: to,
From: bestaddr,
Value: "20000",
Value: String(n),
})
}

View File

@ -8,10 +8,11 @@ const stateConnecting = 'connecting'
const stateGettingToken = 'getting-token'
let sealCodes = [
'Sealed',
'Pending',
'Failed',
'Sealing'
"Unknown",
"Pending",
"Failed",
"Sealing",
"Sealed",
]
class StorageNode extends React.Component {
@ -24,7 +25,7 @@ class StorageNode extends React.Component {
mining: false,
statusCounts: [0, 0, 0, 0]
statusCounts: [0, 0, 0, 0, 0]
}
this.loadInfo = this.loadInfo.bind(this)
@ -84,7 +85,7 @@ class StorageNode extends React.Component {
.map(sector => this.state.client.call("Filecoin.SectorsStatus", [sector.SectorID]))
.reduce(async (p, n) => [...await p, await n], Promise.resolve([]))
let statusCounts = staged.reduce((p, n) => p.map((e, i) => e + (i === n.SealStatusCode ? 1 : 0) ), [0, 0, 0, 0])
let statusCounts = staged.reduce((p, n) => p.map((e, i) => e + (i === n.State ? 1 : 0) ), [0, 0, 0, 0, 0])
this.setState({staged, statusCounts})
}
@ -93,6 +94,10 @@ class StorageNode extends React.Component {
await this.state.client.call("Filecoin.StoreGarbageData", [])
}
sealStaged = async () => {
await this.state.client.call("Filecoin.SectorsStagedSeal", [])
}
async stop() {
await this.props.stop()
}
@ -101,13 +106,14 @@ class StorageNode extends React.Component {
let runtime = <div></div>
if (this.state.actor) {
const sealGarbage = <a href="#" onClick={this.sealGarbage}>[Seal Garbage]</a>
const sealStaged = <a href="#" onClick={this.sealStaged}>[Seal Staged]</a>
runtime = (
<div>
<div>v{this.state.version.Version}, <abbr title={this.state.id}>{this.state.id.substr(-8)}</abbr>, {this.state.peers} peers</div>
<div>Repo: LOTUS_STORAGE_PATH={this.props.node.Repo}</div>
<div>
{sealGarbage}
{sealGarbage} {sealStaged}
</div>
<div>
<Address client={this.props.fullConn} addr={this.state.actor} mountWindow={this.props.mountWindow}/>
@ -116,7 +122,7 @@ class StorageNode extends React.Component {
<div>{this.state.statusCounts.map((c, i) => <span key={i}>{sealCodes[i]}: {c} | </span>)}</div>
<div>
{this.state.staged ? this.state.staged.map((s, i) => (
<div key={i}>{s.SectorID} {sealCodes[s.SealStatusCode]}</div>
<div key={i}>{s.SectorID} {sealCodes[s.State]}</div>
)) : <div/>}
</div>

View File

@ -96,8 +96,9 @@ func (a *API) ClientStartDeal(ctx context.Context, data cid.Cid, miner address.A
}
head := a.Chain.GetHeaviestTipSet()
vouchers := deals.VoucherSpec(blocksDuration, total, head.Height(), extra)
payment, err := a.PaychNewPayment(ctx, self, miner, total, extra, head.Height()+blocksDuration, head.Height()+blocksDuration)
payment, err := a.PaychNewPayment(ctx, self, miner, vouchers)
if err != nil {
return nil, err
}
@ -110,7 +111,7 @@ func (a *API) ClientStartDeal(ctx context.Context, data cid.Cid, miner address.A
PayChActor: payment.Channel,
Payer: self,
ChannelMessage: payment.ChannelMessage,
Vouchers: []*types.SignedVoucher{payment.Voucher},
Vouchers: payment.Vouchers,
},
MinerAddress: miner,
ClientAddress: self,

View File

@ -42,8 +42,11 @@ func (a *PaychAPI) PaychAllocateLane(ctx context.Context, ch address.Address) (u
return a.PaychMgr.AllocateLane(ch)
}
func (a *PaychAPI) PaychNewPayment(ctx context.Context, from, to address.Address, amount types.BigInt, extra *types.ModVerifyParams, tl uint64, minClose uint64) (*api.PaymentInfo, error) {
func (a *PaychAPI) PaychNewPayment(ctx context.Context, from, to address.Address, vouchers []api.VoucherSpec) (*api.PaymentInfo, error) {
amount := vouchers[len(vouchers)-1].Amount
// TODO: Fix free fund tracking in PaychGet
// TODO: validate voucher spec before locking funds
ch, err := a.PaychGet(ctx, from, to, amount)
if err != nil {
return nil, err
@ -54,17 +57,24 @@ func (a *PaychAPI) PaychNewPayment(ctx context.Context, from, to address.Address
return nil, err
}
sv, err := a.paychVoucherCreate(ctx, ch.Channel, types.SignedVoucher{
Amount: amount,
Lane: lane,
svs := make([]*types.SignedVoucher, len(vouchers))
Extra: extra,
TimeLock: tl,
MinCloseHeight: minClose,
})
if err != nil {
return nil, err
for i, v := range vouchers {
sv, err := a.paychVoucherCreate(ctx, ch.Channel, types.SignedVoucher{
Amount: v.Amount,
Lane: lane,
Extra: v.Extra,
TimeLock: v.TimeLock,
MinCloseHeight: v.MinClose,
})
if err != nil {
return nil, err
}
svs[i] = sv
}
var pchCid *cid.Cid
if ch.ChannelMessage != cid.Undef {
pchCid = &ch.ChannelMessage
@ -73,7 +83,7 @@ func (a *PaychAPI) PaychNewPayment(ctx context.Context, from, to address.Address
return &api.PaymentInfo{
Channel: ch.Channel,
ChannelMessage: pchCid,
Voucher: sv,
Vouchers: svs,
}, nil
}
@ -137,10 +147,6 @@ func (a *PaychAPI) PaychVoucherCheckSpendable(ctx context.Context, ch address.Ad
func (a *PaychAPI) PaychVoucherAdd(ctx context.Context, ch address.Address, sv *types.SignedVoucher, proof []byte, minDelta types.BigInt) (types.BigInt, error) {
_ = a.PaychMgr.TrackInboundChannel(ctx, ch) // TODO: expose those calls
if err := a.PaychVoucherCheckValid(ctx, ch, sv); err != nil {
return types.NewInt(0), err
}
return a.PaychMgr.AddVoucher(ctx, ch, sv, proof, minDelta)
}

View File

@ -34,7 +34,7 @@ func (sm *StorageMinerAPI) StoreGarbageData(ctx context.Context) (uint64, error)
size := sectorbuilder.UserBytesForSectorSize(build.SectorSize)
name := fmt.Sprintf("fake-file-%d", rand.Intn(100000000))
sectorId, err := sm.Sectors.AddPiece(name, size, io.LimitReader(rand.New(rand.NewSource(42)), 1016))
sectorId, err := sm.Sectors.AddPiece(name, size, io.LimitReader(rand.New(rand.NewSource(42)), int64(size)))
if err != nil {
return 0, err
}

View File

@ -40,7 +40,7 @@ func MakeGenesisMem(out io.Writer) func(bs dtypes.ChainBlockstore, w *wallet.Wal
PeerIDs: []peer.ID{"peerID 1"},
}
alloc := map[address.Address]types.BigInt{
w: types.NewInt(10000000),
w: types.NewInt(1000000000),
}
b, err := gen.MakeGenesisBlock(bs, alloc, gmc, 100000)
@ -76,7 +76,7 @@ func MakeGenesis(outFile string) func(bs dtypes.ChainBlockstore, w *wallet.Walle
}
addrs := map[address.Address]types.BigInt{
minerAddr: types.NewInt(50000000),
minerAddr: types.NewInt(5000000000),
}
b, err := gen.MakeGenesisBlock(bs, addrs, gmc, 100000)

View File

@ -1,8 +1,10 @@
package paych
import (
"bytes"
"context"
"fmt"
"golang.org/x/xerrors"
"math"
"strconv"
@ -221,16 +223,6 @@ func (pm *Manager) CheckVoucherSpendable(ctx context.Context, ch address.Address
return true, nil
}
func (pm *Manager) loadPaychState(ctx context.Context, ch address.Address) (*types.Actor, *actors.PaymentChannelActorState, error) {
var pcast actors.PaymentChannelActorState
act, err := pm.sm.LoadActorState(ctx, ch, &pcast, nil)
if err != nil {
return nil, nil, err
}
return act, &pcast, nil
}
func (pm *Manager) getPaychOwner(ctx context.Context, ch address.Address) (address.Address, error) {
ret, err := pm.sm.Call(ctx, &types.Message{
From: ch,
@ -253,7 +245,65 @@ func (pm *Manager) AddVoucher(ctx context.Context, ch address.Address, sv *types
return types.NewInt(0), err
}
return pm.store.AddVoucher(ch, sv, proof, minDelta)
pm.store.lk.Lock()
defer pm.store.lk.Unlock()
ci, err := pm.store.getChannelInfo(ch)
if err != nil {
return types.NewInt(0), err
}
laneState, err := pm.laneState(ctx, ch, sv.Lane)
if err != nil {
return types.NewInt(0), err
}
if laneState.Closed {
return types.NewInt(0), xerrors.New("lane closed")
}
if minDelta.GreaterThan(types.NewInt(0)) && laneState.Nonce > sv.Nonce {
return types.NewInt(0), xerrors.Errorf("already storing voucher with higher nonce; %d > %d", laneState.Nonce, sv.Nonce)
}
// look for duplicates
for i, v := range ci.Vouchers {
if !sv.Equals(v.Voucher) {
continue
}
if v.Proof != nil {
if !bytes.Equal(v.Proof, proof) {
log.Warnf("AddVoucher: multiple proofs for single voucher, storing both")
break
}
log.Warnf("AddVoucher: voucher re-added with matching proof")
return types.NewInt(0), nil
}
log.Warnf("AddVoucher: adding proof to stored voucher")
ci.Vouchers[i] = &VoucherInfo{
Voucher: v.Voucher,
Proof: proof,
}
return types.NewInt(0), pm.store.putChannelInfo(ci)
}
delta := types.BigSub(sv.Amount, laneState.Redeemed)
if minDelta.GreaterThan(delta) {
return delta, xerrors.Errorf("addVoucher: supplied token amount too low; minD=%s, D=%s; laneAmt=%s; v.Amt=%s", minDelta, delta, laneState.Redeemed, sv.Amount)
}
ci.Vouchers = append(ci.Vouchers, &VoucherInfo{
Voucher: sv,
Proof: proof,
})
if ci.NextLane <= sv.Lane {
ci.NextLane = sv.Lane + 1
}
return delta, pm.store.putChannelInfo(ci)
}
func (pm *Manager) AllocateLane(ch address.Address) (uint64, error) {

72
paych/state.go Normal file
View File

@ -0,0 +1,72 @@
package paych
import (
"context"
"fmt"
"github.com/filecoin-project/go-lotus/chain/actors"
"github.com/filecoin-project/go-lotus/chain/address"
"github.com/filecoin-project/go-lotus/chain/types"
)
func (pm *Manager) loadPaychState(ctx context.Context, ch address.Address) (*types.Actor, *actors.PaymentChannelActorState, error) {
var pcast actors.PaymentChannelActorState
act, err := pm.sm.LoadActorState(ctx, ch, &pcast, nil)
if err != nil {
return nil, nil, err
}
return act, &pcast, nil
}
func (pm *Manager) laneState(ctx context.Context, ch address.Address, lane uint64) (actors.LaneState, error) {
_, state, err := pm.loadPaychState(ctx, ch)
if err != nil {
return actors.LaneState{}, err
}
// TODO: we probably want to call UpdateChannelState with all vouchers to be fully correct
// (but technically dont't need to)
// TODO: make sure this is correct
ls, ok := state.LaneStates[fmt.Sprintf("%d", lane)]
if !ok {
ls = &actors.LaneState{
Closed: false,
Redeemed: types.NewInt(0),
Nonce: 0,
}
}
if ls.Closed {
return *ls, nil
}
vouchers, err := pm.store.VouchersForPaych(ch)
if err != nil {
if err == ErrChannelNotTracked {
return *ls, nil
}
return actors.LaneState{}, err
}
for _, v := range vouchers {
for range v.Voucher.Merges {
panic("merges todo") // TODO: nonce check
}
if v.Voucher.Lane != lane {
continue
}
if v.Voucher.Nonce < ls.Nonce {
log.Warnf("Found outdated voucher: ch=%s, lane=%d, v.nonce=%d lane.nonce=%d", ch, lane, v.Voucher.Nonce, ls.Nonce)
continue
}
ls.Nonce = v.Voucher.Nonce
ls.Redeemed = v.Voucher.Amount
}
return *ls, nil
}

View File

@ -1,10 +1,8 @@
package paych
import (
"bytes"
"errors"
"fmt"
"math"
"strings"
"sync"
@ -182,66 +180,6 @@ func (ps *Store) findChan(filter func(*ChannelInfo) bool) (address.Address, erro
return address.Undef, nil
}
func (ps *Store) AddVoucher(ch address.Address, sv *types.SignedVoucher, proof []byte, minDelta types.BigInt) (types.BigInt, error) {
ps.lk.Lock()
defer ps.lk.Unlock()
ci, err := ps.getChannelInfo(ch)
if err != nil {
return types.NewInt(0), err
}
var bestAmount types.BigInt
var bestNonce uint64 = math.MaxUint64
// look for duplicates
for i, v := range ci.Vouchers {
if v.Voucher.Lane == sv.Lane && v.Voucher.Nonce+1 > bestNonce+1 {
bestNonce = v.Voucher.Nonce
bestAmount = v.Voucher.Amount
}
if !sv.Equals(v.Voucher) {
continue
}
if v.Proof != nil {
if !bytes.Equal(v.Proof, proof) {
log.Warnf("AddVoucher: multiple proofs for single voucher, storing both")
break
}
log.Warnf("AddVoucher: voucher re-added with matching proof")
return types.NewInt(0), nil
}
log.Warnf("AddVoucher: adding proof to stored voucher")
ci.Vouchers[i] = &VoucherInfo{
Voucher: v.Voucher,
Proof: proof,
}
return types.NewInt(0), ps.putChannelInfo(ci)
}
if bestAmount == (types.BigInt{}) {
bestAmount = types.NewInt(0)
}
delta := types.BigSub(sv.Amount, bestAmount)
if minDelta.GreaterThan(delta) {
return delta, xerrors.Errorf("addVoucher: supplied token amount too low; minD=%s, D=%s; bestAmt=%s; v.Amt=%s", minDelta, delta, bestAmount, sv.Amount)
}
ci.Vouchers = append(ci.Vouchers, &VoucherInfo{
Voucher: sv,
Proof: proof,
})
if ci.NextLane <= sv.Lane {
ci.NextLane = sv.Lane + 1
}
return delta, ps.putChannelInfo(ci)
}
func (ps *Store) AllocateLane(ch address.Address) (uint64, error) {
ps.lk.Lock()
defer ps.lk.Unlock()

View File

@ -272,6 +272,6 @@ func (cst *clientStream) setupPayment(ctx context.Context, toSend types.BigInt)
return api.PaymentInfo{
Channel: cst.paych,
ChannelMessage: nil,
Voucher: sv,
Vouchers: []*types.SignedVoucher{sv},
}, nil
}

View File

@ -122,8 +122,12 @@ func (hnd *handlerDeal) handleNext() (bool, error) {
unixfs0 := deal.Params.Unixfs0
if len(deal.Payment.Vouchers) != 1 {
return false, xerrors.Errorf("expected one signed voucher, got %d", len(deal.Payment.Vouchers))
}
expPayment := types.BigMul(hnd.m.pricePerByte, types.NewInt(deal.Params.Unixfs0.Size))
if _, err := hnd.m.full.PaychVoucherAdd(context.TODO(), deal.Payment.Channel, deal.Payment.Voucher, nil, expPayment); err != nil {
if _, err := hnd.m.full.PaychVoucherAdd(context.TODO(), deal.Payment.Channel, deal.Payment.Vouchers[0], nil, expPayment); err != nil {
return false, xerrors.Errorf("processing retrieval payment: %w", err)
}