diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index fa1d690c8..2b0f689c9 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -49,7 +49,7 @@ jobs: host port: 3800 mysql version: '8' mysql root password: test - - name: Test + - name: Test Stores uses: n8maninger/action-golang-test@v1 with: args: "-race;-short" diff --git a/autopilot/contractor/contractor.go b/autopilot/contractor/contractor.go index 86c27f7ac..873394e7b 100644 --- a/autopilot/contractor/contractor.go +++ b/autopilot/contractor/contractor.go @@ -89,6 +89,7 @@ type Bus interface { AncestorContracts(ctx context.Context, id types.FileContractID, minStartHeight uint64) ([]api.ArchivedContract, error) ArchiveContracts(ctx context.Context, toArchive map[types.FileContractID]string) error ConsensusState(ctx context.Context) (api.ConsensusState, error) + Contract(ctx context.Context, id types.FileContractID) (api.ContractMetadata, error) Contracts(ctx context.Context, opts api.ContractsOpts) (contracts []api.ContractMetadata, err error) FileContractTax(ctx context.Context, payout types.Currency) (types.Currency, error) Host(ctx context.Context, hostKey types.PublicKey) (api.Host, error) @@ -615,6 +616,7 @@ func (c *Contractor) runContractChecks(ctx *mCtx, hostChecks map[types.PublicKey "toArchive", len(toArchive), "toRefresh", len(toRefresh), "toRenew", len(toRenew), + "bh", bh, ) }() @@ -650,7 +652,7 @@ LOOP: toArchive[fcid] = errContractMaxRevisionNumber.Error() } else if contract.RevisionNumber == math.MaxUint64 { toArchive[fcid] = errContractMaxRevisionNumber.Error() - } else if contract.State == api.ContractStatePending && bh-contract.StartHeight > contractConfirmationDeadline { + } else if contract.State == api.ContractStatePending && bh-contract.StartHeight > ContractConfirmationDeadline { toArchive[fcid] = errContractNotConfirmed.Error() } if _, archived := toArchive[fcid]; archived { diff --git a/autopilot/contractor/hostfilter.go b/autopilot/contractor/hostfilter.go index eb1a559a4..31d53eb13 100644 --- a/autopilot/contractor/hostfilter.go +++ b/autopilot/contractor/hostfilter.go @@ -14,6 +14,10 @@ import ( ) const ( + // ContractConfirmationDeadline is the number of blocks since its start + // height we wait for a contract to appear on chain. + ContractConfirmationDeadline = 18 + // minContractFundUploadThreshold is the percentage of contract funds // remaining at which the contract gets marked as not good for upload minContractFundUploadThreshold = float64(0.05) // 5% @@ -23,10 +27,6 @@ const ( // acquirable storage below which the contract is considered to be // out-of-collateral. minContractCollateralDenominator = 20 // 5% - - // contractConfirmationDeadline is the number of blocks since its start - // height we wait for a contract to appear on chain. - contractConfirmationDeadline = 18 ) var ( diff --git a/bus/bus.go b/bus/bus.go index 0add147bd..b9ecad580 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -26,12 +26,17 @@ import ( "go.sia.tech/renterd/api" "go.sia.tech/renterd/build" "go.sia.tech/renterd/bus/client" + "go.sia.tech/renterd/chain" "go.sia.tech/renterd/object" "go.sia.tech/renterd/webhooks" "go.sia.tech/siad/modules" "go.uber.org/zap" ) +// maxSyncTime is the maximum time since the last block before we consider +// ourselves unsynced. +const maxSyncTime = time.Hour + // Client re-exports the client from the client package. type Client struct { *client.Client @@ -205,7 +210,7 @@ type ( Redistribute(outputs int, amount, feePerByte types.Currency) (txns []types.Transaction, toSign []types.Hash256, err error) ReleaseInputs(txns ...types.Transaction) SignTransaction(txn *types.Transaction, toSign []types.Hash256, cf types.CoveredFields) - SpendableOutputs() ([]wallet.SiacoinElement, error) + SpendableOutputs() ([]types.SiacoinElement, error) Tip() (types.ChainIndex, error) UnconfirmedTransactions() ([]wallet.Event, error) Events(offset, limit int) ([]wallet.Event, error) @@ -232,6 +237,7 @@ type bus struct { w Wallet as AutopilotStore + cs chain.ChainStore eas EphemeralAccountStore hdb HostDB ms MetadataStore @@ -450,7 +456,11 @@ func (b *bus) syncerConnectHandler(jc jape.Context) { } func (b *bus) consensusStateHandler(jc jape.Context) { - jc.Encode(b.consensusState()) + cs, err := b.consensusState(jc.Request.Context()) + if jc.Check("couldn't fetch consensus state", err) != nil { + return + } + jc.Encode(cs) } func (b *bus) consensusNetworkHandler(jc jape.Context) { @@ -1771,19 +1781,23 @@ func (b *bus) paramsHandlerUploadGET(jc jape.Context) { }) } -func (b *bus) consensusState() api.ConsensusState { - cs := b.cm.TipState() +func (b *bus) consensusState(ctx context.Context) (api.ConsensusState, error) { + index, err := b.cs.ChainIndex(ctx) + if err != nil { + return api.ConsensusState{}, err + } var synced bool - if block, ok := b.cm.Block(cs.Index.ID); ok && time.Since(block.Timestamp) < 2*cs.BlockInterval() { + block, found := b.cm.Block(index.ID) + if found && time.Since(block.Timestamp) <= maxSyncTime { synced = true } return api.ConsensusState{ - BlockHeight: cs.Index.Height, - LastBlockTime: api.TimeRFC3339(cs.PrevTimestamps[0]), + BlockHeight: index.Height, + LastBlockTime: api.TimeRFC3339(block.Timestamp), Synced: synced, - } + }, nil } func (b *bus) paramsHandlerGougingGET(jc jape.Context) { @@ -1809,7 +1823,10 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { b.logger.Panicf("failed to unmarshal redundancy settings '%s': %v", rss, err) } - cs := b.consensusState() + cs, err := b.consensusState(ctx) + if err != nil { + return api.GougingParams{}, err + } return api.GougingParams{ ConsensusState: cs, @@ -2422,12 +2439,13 @@ func (b *bus) multipartHandlerListPartsPOST(jc jape.Context) { } // New returns a new Bus. -func New(am *alerts.Manager, hm WebhookManager, cm ChainManager, s Syncer, w Wallet, hdb HostDB, as AutopilotStore, ms MetadataStore, ss SettingStore, eas EphemeralAccountStore, mtrcs MetricsStore, l *zap.Logger) (*bus, error) { +func New(am *alerts.Manager, hm WebhookManager, cm ChainManager, cs chain.ChainStore, s Syncer, w Wallet, hdb HostDB, as AutopilotStore, ms MetadataStore, ss SettingStore, eas EphemeralAccountStore, mtrcs MetricsStore, l *zap.Logger) (*bus, error) { b := &bus{ alerts: alerts.WithOrigin(am, "bus"), alertMgr: am, webhooks: hm, cm: cm, + cs: cs, s: s, w: w, hdb: hdb, diff --git a/bus/client/client_test.go b/bus/client/client_test.go index 6bed8a0d9..482ae9bd1 100644 --- a/bus/client/client_test.go +++ b/bus/client/client_test.go @@ -70,7 +70,7 @@ func newTestClient(dir string) (*client.Client, func() error, func(context.Conte // create bus network, genesis := build.Network() - b, shutdown, _, err := node.NewBus(node.BusConfig{ + b, shutdown, _, _, err := node.NewBus(node.BusConfig{ Bus: config.Bus{ AnnouncementMaxAgeHours: 24 * 7 * 52, // 1 year Bootstrap: false, diff --git a/chain/chain.go b/chain/chain.go new file mode 100644 index 000000000..40fcb1967 --- /dev/null +++ b/chain/chain.go @@ -0,0 +1,26 @@ +package chain + +import ( + "go.sia.tech/core/consensus" + "go.sia.tech/core/types" + "go.sia.tech/coreutils/chain" +) + +type ( + Manager = chain.Manager + HostAnnouncement = chain.HostAnnouncement + ApplyUpdate = chain.ApplyUpdate + RevertUpdate = chain.RevertUpdate +) + +func TestnetZen() (*consensus.Network, types.Block) { + return chain.TestnetZen() +} + +func NewDBStore(db chain.DB, n *consensus.Network, genesisBlock types.Block) (_ *chain.DBStore, _ consensus.State, err error) { + return chain.NewDBStore(db, n, genesisBlock) +} + +func NewManager(store chain.Store, cs consensus.State) *Manager { + return chain.NewManager(store, cs) +} diff --git a/chain/manager.go b/chain/manager.go deleted file mode 100644 index 5614604da..000000000 --- a/chain/manager.go +++ /dev/null @@ -1,10 +0,0 @@ -package chain - -import ( - "go.sia.tech/coreutils/chain" -) - -type ( - Manager = chain.Manager - HostAnnouncement = chain.HostAnnouncement -) diff --git a/chain/subscriber.go b/chain/subscriber.go index 5b3da7e70..781dda70a 100644 --- a/chain/subscriber.go +++ b/chain/subscriber.go @@ -1,23 +1,43 @@ package chain import ( + "context" + "errors" + "fmt" + "sync" "time" "go.sia.tech/core/types" "go.sia.tech/coreutils/chain" + "go.sia.tech/coreutils/wallet" "go.sia.tech/renterd/api" + "go.sia.tech/renterd/internal/utils" + "go.uber.org/zap" +) + +const ( + // updatesBatchSize is the maximum number of updates to fetch in a single + // call to the chain manager when we request updates since a given index. + updatesBatchSize = 50 +) + +var ( + errClosed = errors.New("subscriber closed") ) type ( + ChainManager interface { + Tip() types.ChainIndex + OnReorg(fn func(types.ChainIndex)) (cancel func()) + UpdatesSince(index types.ChainIndex, max int) (rus []chain.RevertUpdate, aus []chain.ApplyUpdate, err error) + } + ChainStore interface { - BeginChainUpdateTx() (ChainUpdateTx, error) - ChainIndex() (types.ChainIndex, error) + ProcessChainUpdate(ctx context.Context, fn func(ChainUpdateTx) error) error + ChainIndex(ctx context.Context) (types.ChainIndex, error) } ChainUpdateTx interface { - Commit() error - Rollback() error - ContractState(fcid types.FileContractID) (api.ContractState, error) UpdateChainIndex(index types.ChainIndex) error UpdateContract(fcid types.FileContractID, revisionHeight, revisionNumber, size uint64) error @@ -25,5 +45,446 @@ type ( UpdateContractProofHeight(fcid types.FileContractID, proofHeight uint64) error UpdateFailedContracts(blockHeight uint64) error UpdateHost(hk types.PublicKey, ha chain.HostAnnouncement, bh uint64, blockID types.BlockID, ts time.Time) error + + wallet.UpdateTx + } + + Subscriber struct { + cm ChainManager + cs ChainStore + logger *zap.SugaredLogger + + announcementMaxAge time.Duration + walletAddress types.Address + + shutdownCtx context.Context + shutdownCtxCancel context.CancelCauseFunc + syncSig chan struct{} + wg sync.WaitGroup + + mu sync.Mutex + knownContracts map[types.FileContractID]bool + } + + revision struct { + revisionNumber uint64 + fileSize uint64 + } + + contractUpdate struct { + fcid types.FileContractID + prev *revision + curr *revision + resolved bool + valid bool + } + + hostUpdate struct { + hk types.PublicKey + ha chain.HostAnnouncement } ) + +func NewSubscriber(cm ChainManager, cs ChainStore, walletAddress types.Address, announcementMaxAge time.Duration, logger *zap.Logger) (_ *Subscriber, err error) { + if announcementMaxAge == 0 { + return nil, errors.New("announcementMaxAge must be non-zero") + } + + ctx, cancel := context.WithCancelCause(context.Background()) + return &Subscriber{ + cm: cm, + cs: cs, + logger: logger.Sugar(), + + announcementMaxAge: announcementMaxAge, + walletAddress: walletAddress, + + shutdownCtx: ctx, + shutdownCtxCancel: cancel, + syncSig: make(chan struct{}, 1), + + knownContracts: make(map[types.FileContractID]bool), + }, nil +} + +func (s *Subscriber) Close() error { + // cancel shutdown context + s.shutdownCtxCancel(errClosed) + + // wait for sync loop to finish + s.wg.Wait() + return nil +} + +func (s *Subscriber) Run() (func(), error) { + // perform an initial sync + start := time.Now() + if err := s.sync(); err != nil { + return nil, fmt.Errorf("initial sync failed: %w", err) + } + s.logger.Debugw("initial sync completed", "duration", time.Since(start)) + + // start sync loop in separate goroutine + s.wg.Add(1) + go func() { + defer s.wg.Done() + + for { + select { + case <-s.shutdownCtx.Done(): + return + case <-s.syncSig: + } + + if err := s.sync(); errors.Is(err, errClosed) || errors.Is(err, context.Canceled) { + return + } else if err != nil { + s.logger.Panicf("failed to sync: %v", err) + } + } + }() + + // trigger a sync on reorgs + return s.cm.OnReorg(func(ci types.ChainIndex) { + select { + case s.syncSig <- struct{}{}: + s.logger.Debugw("reorg triggered", "height", ci.Height, "block_id", ci.ID) + default: + } + }), nil +} + +func (s *Subscriber) applyChainUpdate(tx ChainUpdateTx, cau chain.ApplyUpdate) error { + // apply host updates + b := cau.Block + if time.Since(b.Timestamp) <= s.announcementMaxAge { + var hus []hostUpdate + chain.ForEachHostAnnouncement(b, func(hk types.PublicKey, ha chain.HostAnnouncement) { + if ha.NetAddress != "" { + hus = append(hus, hostUpdate{hk, ha}) + } + }) + for _, hu := range hus { + if err := tx.UpdateHost(hu.hk, hu.ha, cau.State.Index.Height, b.ID(), b.Timestamp); err != nil { + return fmt.Errorf("failed to update host: %w", err) + } + } + } + + // v1 contracts + var cus []contractUpdate + cau.ForEachFileContractElement(func(fce types.FileContractElement, rev *types.FileContractElement, resolved, valid bool) { + cus = append(cus, v1ContractUpdate(fce, rev, resolved, valid)) + }) + for _, cu := range cus { + if err := s.updateContract(tx, cau.State.Index, cu.fcid, cu.prev, cu.curr, cu.resolved, cu.valid); err != nil { + return fmt.Errorf("failed to apply v1 contract update: %w", err) + } + } + + // v2 contracts + cus = cus[:0] + cau.ForEachV2FileContractElement(func(fce types.V2FileContractElement, rev *types.V2FileContractElement, res types.V2FileContractResolutionType) { + cus = append(cus, v2ContractUpdate(fce, rev, res)) + }) + for _, cu := range cus { + if err := s.updateContract(tx, cau.State.Index, cu.fcid, cu.prev, cu.curr, cu.resolved, cu.valid); err != nil { + return fmt.Errorf("failed to apply v2 contract update: %w", err) + } + } + return nil +} + +func (s *Subscriber) revertChainUpdate(tx ChainUpdateTx, cru chain.RevertUpdate) error { + // NOTE: host updates are not reverted + + // v1 contracts + var cus []contractUpdate + cru.ForEachFileContractElement(func(fce types.FileContractElement, rev *types.FileContractElement, resolved, valid bool) { + cus = append(cus, v1ContractUpdate(fce, rev, resolved, valid)) + }) + for _, cu := range cus { + if err := s.updateContract(tx, cru.State.Index, cu.fcid, cu.prev, cu.curr, cu.resolved, cu.valid); err != nil { + return fmt.Errorf("failed to revert v1 contract update: %w", err) + } + } + + // v2 contracts + cus = cus[:0] + cru.ForEachV2FileContractElement(func(fce types.V2FileContractElement, rev *types.V2FileContractElement, res types.V2FileContractResolutionType) { + cus = append(cus, v2ContractUpdate(fce, rev, res)) + }) + for _, cu := range cus { + if err := s.updateContract(tx, cru.State.Index, cu.fcid, cu.prev, cu.curr, cu.resolved, cu.valid); err != nil { + return fmt.Errorf("failed to revert v2 contract update: %w", err) + } + } + + return nil +} + +func (s *Subscriber) sync() error { + start := time.Now() + + // fetch current chain index + index, err := s.cs.ChainIndex(s.shutdownCtx) + if err != nil { + return fmt.Errorf("failed to get chain index: %w", err) + } + s.logger.Debugw("sync started", "height", index.Height, "block_id", index.ID) + + // fetch updates until we're caught up + var cnt uint64 + for index != s.cm.Tip() && !s.isClosed() { + // fetch updates + istart := time.Now() + crus, caus, err := s.cm.UpdatesSince(index, updatesBatchSize) + if err != nil { + return fmt.Errorf("failed to fetch updates: %w", err) + } + s.logger.Debugw("fetched updates since", "caus", len(caus), "crus", len(crus), "since_height", index.Height, "since_block_id", index.ID, "ms", time.Since(istart).Milliseconds()) + + // process updates + istart = time.Now() + index, err = s.processUpdates(s.shutdownCtx, crus, caus) + if err != nil { + return fmt.Errorf("failed to process updates: %w", err) + } + s.logger.Debugw("processed updates successfully", "new_height", index.Height, "new_block_id", index.ID, "ms", time.Since(istart).Milliseconds()) + cnt++ + } + + s.logger.Debugw("sync completed", "start_height", index.Height, "block_id", index.ID, "ms", time.Since(start).Milliseconds(), "iterations", cnt) + return nil +} + +func (s *Subscriber) processUpdates(ctx context.Context, crus []chain.RevertUpdate, caus []chain.ApplyUpdate) (types.ChainIndex, error) { + var index types.ChainIndex + if err := s.cs.ProcessChainUpdate(ctx, func(tx ChainUpdateTx) error { + // process wallet updates + if err := wallet.UpdateChainState(tx, s.walletAddress, caus, crus); err != nil { + return fmt.Errorf("failed to process wallet updates: %w", err) + } + + // process revert updates + for _, cru := range crus { + if err := s.revertChainUpdate(tx, cru); err != nil { + return fmt.Errorf("failed to revert chain update: %w", err) + } + } + + // process apply updates + for _, cau := range caus { + if err := s.applyChainUpdate(tx, cau); err != nil { + return fmt.Errorf("failed to apply chain updates: %w", err) + } + } + + // update chain index + index = caus[len(caus)-1].State.Index + if err := tx.UpdateChainIndex(index); err != nil { + return fmt.Errorf("failed to update chain index: %w", err) + } + + // update failed contracts + if err := tx.UpdateFailedContracts(index.Height); err != nil { + return fmt.Errorf("failed to update failed contracts: %w", err) + } + + return nil + }); err != nil { + return types.ChainIndex{}, err + } + return index, nil +} + +func (s *Subscriber) updateContract(tx ChainUpdateTx, index types.ChainIndex, fcid types.FileContractID, prev, curr *revision, resolved, valid bool) error { + // sanity check at least one is not nil + if prev == nil && curr == nil { + return errors.New("both prev and curr revisions are nil") // developer error + } + + // ignore unknown contracts + if !s.isKnownContract(fcid) { + return nil + } + + // fetch contract state + state, err := tx.ContractState(fcid) + if err != nil && utils.IsErr(err, api.ErrContractNotFound) { + s.updateKnownContracts(fcid, false) // ignore unknown contracts + return nil + } else if err != nil { + return fmt.Errorf("failed to get contract state: %w", err) + } else { + s.updateKnownContracts(fcid, true) // update known contracts + } + + // handle reverts + if prev != nil { + // update state from 'active' -> 'pending' + if curr == nil { + if err := tx.UpdateContractState(fcid, api.ContractStatePending); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + } + + // reverted renewal: 'complete' -> 'active' + if curr != nil { + if err := tx.UpdateContract(fcid, index.Height, prev.revisionNumber, prev.fileSize); err != nil { + return fmt.Errorf("failed to revert contract: %w", err) + } + if state == api.ContractStateComplete { + if err := tx.UpdateContractState(fcid, api.ContractStateActive); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + s.logger.Infow("contract state changed: complete -> active", + "fcid", fcid, + "reason", "final revision reverted") + } + } + + // reverted storage proof: 'complete/failed' -> 'active' + if resolved { + if err := tx.UpdateContractState(fcid, api.ContractStateActive); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + if valid { + s.logger.Infow("contract state changed: complete -> active", + "fcid", fcid, + "reason", "storage proof reverted") + } else { + s.logger.Infow("contract state changed: failed -> active", + "fcid", fcid, + "reason", "storage proof reverted") + } + } + + return nil + } + + // handle apply + if err := tx.UpdateContract(fcid, index.Height, curr.revisionNumber, curr.fileSize); err != nil { + return fmt.Errorf("failed to update contract: %w", err) + } + + // update state from 'pending' -> 'active' + if state == api.ContractStatePending || state == api.ContractStateUnknown { + if err := tx.UpdateContractState(fcid, api.ContractStateActive); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + s.logger.Infow("contract state changed: pending -> active", + "fcid", fcid, + "reason", "contract confirmed") + } + + // renewed: 'active' -> 'complete' + if curr.revisionNumber == types.MaxRevisionNumber && curr.fileSize == 0 { + if err := tx.UpdateContractState(fcid, api.ContractStateComplete); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + s.logger.Infow("contract state changed: active -> complete", + "fcid", fcid, + "reason", "final revision confirmed") + } + + // storage proof: 'active' -> 'complete/failed' + if resolved { + if err := tx.UpdateContractProofHeight(fcid, index.Height); err != nil { + return fmt.Errorf("failed to update contract proof height: %w", err) + } + if valid { + if err := tx.UpdateContractState(fcid, api.ContractStateComplete); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + s.logger.Infow("contract state changed: active -> complete", + "fcid", fcid, + "reason", "storage proof valid") + } else { + if err := tx.UpdateContractState(fcid, api.ContractStateFailed); err != nil { + return fmt.Errorf("failed to update contract state: %w", err) + } + s.logger.Infow("contract state changed: active -> failed", + "fcid", fcid, + "reason", "storage proof missed") + } + } + return nil +} + +func (s *Subscriber) isClosed() bool { + select { + case <-s.shutdownCtx.Done(): + return true + default: + } + return false +} + +func (s *Subscriber) isKnownContract(fcid types.FileContractID) bool { + s.mu.Lock() + defer s.mu.Unlock() + known, ok := s.knownContracts[fcid] + if !ok { + return true // assume known + } + return known +} + +func (s *Subscriber) updateKnownContracts(fcid types.FileContractID, known bool) { + s.mu.Lock() + defer s.mu.Unlock() + s.knownContracts[fcid] = known +} + +func v1ContractUpdate(fce types.FileContractElement, rev *types.FileContractElement, resolved, valid bool) contractUpdate { + curr := &revision{ + revisionNumber: fce.FileContract.RevisionNumber, + fileSize: fce.FileContract.Filesize, + } + if rev != nil { + curr.revisionNumber = rev.FileContract.RevisionNumber + curr.fileSize = rev.FileContract.Filesize + } + return contractUpdate{ + fcid: types.FileContractID(fce.ID), + prev: nil, + curr: curr, + resolved: resolved, + valid: valid, + } +} + +func v2ContractUpdate(fce types.V2FileContractElement, rev *types.V2FileContractElement, res types.V2FileContractResolutionType) contractUpdate { + curr := &revision{ + revisionNumber: fce.V2FileContract.RevisionNumber, + fileSize: fce.V2FileContract.Filesize, + } + if rev != nil { + curr.revisionNumber = rev.V2FileContract.RevisionNumber + curr.fileSize = rev.V2FileContract.Filesize + } + + var resolved, valid bool + if res != nil { + resolved = true + switch res.(type) { + case *types.V2FileContractFinalization: + valid = true + case *types.V2FileContractRenewal: + valid = true + case *types.V2StorageProof: + valid = true + case *types.V2FileContractExpiration: + valid = fce.V2FileContract.Filesize == 0 + } + } + + return contractUpdate{ + fcid: types.FileContractID(fce.ID), + prev: nil, + curr: curr, + resolved: resolved, + valid: valid, + } +} diff --git a/cmd/renterd/main.go b/cmd/renterd/main.go index 592c7b412..3d4695c71 100644 --- a/cmd/renterd/main.go +++ b/cmd/renterd/main.go @@ -105,7 +105,6 @@ var ( AnnouncementMaxAgeHours: 24 * 7 * 52, // 1 year Bootstrap: true, GatewayAddr: build.DefaultGatewayAddress, - PersistInterval: time.Minute, UsedUTXOExpiry: 24 * time.Hour, SlabBufferCompletionThreshold: 1 << 12, }, @@ -272,7 +271,7 @@ func main() { flag.Uint64Var(&cfg.Bus.AnnouncementMaxAgeHours, "bus.announcementMaxAgeHours", cfg.Bus.AnnouncementMaxAgeHours, "Max age for announcements") flag.BoolVar(&cfg.Bus.Bootstrap, "bus.bootstrap", cfg.Bus.Bootstrap, "Bootstraps gateway and consensus modules") flag.StringVar(&cfg.Bus.GatewayAddr, "bus.gatewayAddr", cfg.Bus.GatewayAddr, "Address for Sia peer connections (overrides with RENTERD_BUS_GATEWAY_ADDR)") - flag.DurationVar(&cfg.Bus.PersistInterval, "bus.persistInterval", cfg.Bus.PersistInterval, "Interval for persisting consensus updates") + flag.DurationVar(&cfg.Bus.PersistInterval, "bus.persistInterval", cfg.Bus.PersistInterval, "(deprecated) Interval for persisting consensus updates") flag.DurationVar(&cfg.Bus.UsedUTXOExpiry, "bus.usedUTXOExpiry", cfg.Bus.UsedUTXOExpiry, "Expiry for used UTXOs in transactions") flag.Int64Var(&cfg.Bus.SlabBufferCompletionThreshold, "bus.slabBufferCompletionThreshold", cfg.Bus.SlabBufferCompletionThreshold, "Threshold for slab buffer upload (overrides with RENTERD_BUS_SLAB_BUFFER_COMPLETION_THRESHOLD)") @@ -447,14 +446,23 @@ func main() { cfg.Log.Database.Level = cfg.Log.Level } - network, _ := build.Network() + network, genesis := build.Network() busCfg := node.BusConfig{ Bus: cfg.Bus, Database: cfg.Database, DatabaseLog: cfg.Log.Database, Logger: logger, Network: network, + Genesis: genesis, SlabPruningInterval: time.Hour, + RetryTxIntervals: []time.Duration{ + 200 * time.Millisecond, + 500 * time.Millisecond, + time.Second, + 3 * time.Second, + 10 * time.Second, + 10 * time.Second, + }, } type shutdownFn struct { @@ -498,7 +506,7 @@ func main() { busAddr, busPassword := cfg.Bus.RemoteAddr, cfg.Bus.RemotePassword if cfg.Bus.RemoteAddr == "" { - b, shutdown, _, err := node.NewBus(busCfg, cfg.Directory, seed, logger) + b, shutdown, _, _, err := node.NewBus(busCfg, cfg.Directory, seed, logger) if err != nil { logger.Fatal("failed to create bus, err: " + err.Error()) } diff --git a/config/config.go b/config/config.go index 2f4220cc1..1602bf75f 100644 --- a/config/config.go +++ b/config/config.go @@ -51,9 +51,9 @@ type ( GatewayAddr string `yaml:"gatewayAddr,omitempty"` RemoteAddr string `yaml:"remoteAddr,omitempty"` RemotePassword string `yaml:"remotePassword,omitempty"` - PersistInterval time.Duration `yaml:"persistInterval,omitempty"` UsedUTXOExpiry time.Duration `yaml:"usedUtxoExpiry,omitempty"` SlabBufferCompletionThreshold int64 `yaml:"slabBufferCompleionThreshold,omitempty"` + PersistInterval time.Duration `yaml:"persistInterval,omitempty"` // deprecated } // LogFile configures the file output of the logger. diff --git a/go.mod b/go.mod index ebf564d4f..f6e6b3457 100644 --- a/go.mod +++ b/go.mod @@ -13,10 +13,10 @@ require ( github.com/minio/minio-go/v7 v7.0.70 github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe - go.sia.tech/core v0.2.2 - go.sia.tech/coreutils v0.0.4-0.20240313143809-01b5d444a630 + go.sia.tech/core v0.2.3 + go.sia.tech/coreutils v0.0.4-0.20240502154058-5df7ad9c0b7c go.sia.tech/gofakes3 v0.0.3 - go.sia.tech/hostd v1.0.4 + go.sia.tech/hostd v1.1.0-beta.1.0.20240507093050-bc2a34c93355 go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca @@ -35,8 +35,8 @@ require ( require ( github.com/aead/chacha20 v0.0.0-20180709150244-8b13a72661da // indirect - github.com/aws/aws-sdk-go v1.51.21 // indirect - github.com/cloudflare/cloudflare-go v0.91.0 // indirect + github.com/aws/aws-sdk-go v1.52.3 // indirect + github.com/cloudflare/cloudflare-go v0.94.0 // indirect github.com/dchest/threefish v0.0.0-20120919164726-3ecf4c494abf // indirect github.com/dustin/go-humanize v1.0.1 // indirect github.com/go-sql-driver/mysql v1.7.1 // indirect @@ -59,7 +59,7 @@ require ( github.com/rs/xid v1.5.0 // indirect github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46 // indirect github.com/shabbyrobe/gocovmerge v0.0.0-20230507112040-c3350d9342df // indirect - github.com/shopspring/decimal v1.3.1 // indirect + github.com/shopspring/decimal v1.4.0 // indirect gitlab.com/NebulousLabs/bolt v1.4.4 // indirect gitlab.com/NebulousLabs/demotemutex v0.0.0-20151003192217-235395f71c40 // indirect gitlab.com/NebulousLabs/entropy-mnemonics v0.0.0-20181018051301-7532f67e3500 // indirect @@ -81,5 +81,5 @@ require ( golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.20.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect - nhooyr.io/websocket v1.8.10 // indirect + nhooyr.io/websocket v1.8.11 // indirect ) diff --git a/go.sum b/go.sum index 626096b4d..6a3648c90 100644 --- a/go.sum +++ b/go.sum @@ -1157,8 +1157,9 @@ github.com/apache/arrow/go/v11 v11.0.0/go.mod h1:Eg5OsL5H+e299f7u5ssuXsuHQVEGC4x github.com/apache/arrow/go/v12 v12.0.0/go.mod h1:d+tV/eHZZ7Dz7RPrFKtPK02tpr+c9/PEd/zm8mDS9Vg= github.com/apache/thrift v0.16.0/go.mod h1:PHK3hniurgQaNMZYaCLEqXKsYK8upmhPbmdP2FXSqgU= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= -github.com/aws/aws-sdk-go v1.51.21 h1:UrT6JC9R9PkYYXDZBV0qDKTualMr+bfK2eboTknMgbs= github.com/aws/aws-sdk-go v1.51.21/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= +github.com/aws/aws-sdk-go v1.52.3 h1:BNPJmHOXNoM/iBWJKrvaQvJOweRcp3KLpzdb65CfQwU= +github.com/aws/aws-sdk-go v1.52.3/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= @@ -1174,8 +1175,8 @@ github.com/chzyer/logex v1.1.10/go.mod h1:+Ywpsq7O8HXn0nuIou7OrIPyXbp3wmkHB+jjWR github.com/chzyer/readline v0.0.0-20180603132655-2972be24d48e/go.mod h1:nSuG5e5PlCu98SY8svDHJxuZscDgtXS6KTTbou5AhLI= github.com/chzyer/test v0.0.0-20180213035817-a1ea475d72b1/go.mod h1:Q3SI9o4m/ZMnBNeIyt5eFwwo7qiLfzFZmjNmxjkiQlU= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= -github.com/cloudflare/cloudflare-go v0.91.0 h1:L7IR+86qrZuEMSjGFg4cwRwtHqC8uCPmMUkP7BD4CPw= -github.com/cloudflare/cloudflare-go v0.91.0/go.mod h1:nUqvBUUDRxNzsDSQjbqUNWHEIYAoUlgRmcAzMKlFdKs= +github.com/cloudflare/cloudflare-go v0.94.0 h1:WADmVhCdnn1A9sm5NU08by49Vbh4Lj/JBgTWTr7q7Qc= +github.com/cloudflare/cloudflare-go v0.94.0/go.mod h1:N1u1cLZ4lG6NeezGOWi7P6aq1DK2iVYg9ze7GZbUmZE= github.com/cncf/udpa/go v0.0.0-20191209042840-269d4d468f6f/go.mod h1:M8M6+tZqaGXZJjfX53e64911xZQV5JYwmTeXPW+k8Sc= github.com/cncf/udpa/go v0.0.0-20200629203442-efcf912fb354/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= github.com/cncf/udpa/go v0.0.0-20201120205902-5459f2c99403/go.mod h1:WmhPx2Nbnhtbo57+VJT5O0JRkEi1Wbu0z5j0R8u5Hbk= @@ -1535,8 +1536,8 @@ github.com/ryszard/goskiplist v0.0.0-20150312221310-2dfbae5fcf46/go.mod h1:uAQ5P github.com/shabbyrobe/gocovmerge v0.0.0-20190829150210-3e036491d500/go.mod h1:+njLrG5wSeoG4Ds61rFgEzKvenR2UHbjMoDHsczxly0= github.com/shabbyrobe/gocovmerge v0.0.0-20230507112040-c3350d9342df h1:S77Pf5fIGMa7oSwp8SQPp7Hb4ZiI38K3RNBKD2LLeEM= github.com/shabbyrobe/gocovmerge v0.0.0-20230507112040-c3350d9342df/go.mod h1:dcuzJZ83w/SqN9k4eQqwKYMgmKWzg/KzJAURBhRL1tc= -github.com/shopspring/decimal v1.3.1 h1:2Usl1nmF/WZucqkFZhnfFYxxxu8LG21F6nPQBE5gKV8= -github.com/shopspring/decimal v1.3.1/go.mod h1:DKyhrW/HYNuLGql+MJL6WCR6knT2jwCFRcu2hWCYk4o= +github.com/shopspring/decimal v1.4.0 h1:bxl37RwXBklmTi0C79JfXCEBD1cqqHt0bbgBAGFp81k= +github.com/shopspring/decimal v1.4.0/go.mod h1:gawqmDU56v4yIKSwfBSFip1HdCCXN8/+DMd9qYNcwME= github.com/shurcooL/sanitized_anchor_name v1.0.0/go.mod h1:1NzhyTcUVG4SuEtjjoZeVRXNmyL/1OwPU0+IJeTBvfc= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/soheilhy/cmux v0.1.4/go.mod h1:IM3LyeVVIOuxMH7sFAkER9+bJ4dT7Ms6E4xg4kGIyLM= @@ -1632,18 +1633,14 @@ go.opencensus.io v0.24.0/go.mod h1:vNK8G9p7aAivkbmorf4v+7Hgx+Zs0yY+0fOtgBfjQKo= go.opentelemetry.io/proto/otlp v0.7.0/go.mod h1:PqfVotwruBrMGOCsRd/89rSnXhoiJIqeYNgFYFoEGnI= go.opentelemetry.io/proto/otlp v0.15.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= go.opentelemetry.io/proto/otlp v0.19.0/go.mod h1:H7XAot3MsfNsj7EXtrA2q5xSNQ10UqI405h3+duxN4U= -go.sia.tech/core v0.2.2 h1:33RJrt08o7KyUOY4tITH6ECmRq1lhtapqc/SncIF/2A= -go.sia.tech/core v0.2.2/go.mod h1:Zk7HaybEPgkPC1p6e6tTQr8PIeZClTgNcLNGYDLQJeE= -go.sia.tech/coreutils v0.0.4-0.20240313143809-01b5d444a630 h1:KpVSI9ijpyyjwXvxV0tSWK9ukFyTupibg9OrlvjiKDk= -go.sia.tech/coreutils v0.0.4-0.20240313143809-01b5d444a630/go.mod h1:QvsXghS4wqhJosQq3AkMjA2mJ6pbDB7PgG+w5b09/z0= -go.sia.tech/gofakes3 v0.0.2 h1:oWnsYjHvSyf4ddtEH6XO76xeXC10N2cRqduI6B0d/EU= -go.sia.tech/gofakes3 v0.0.2/go.mod h1:+NLzpsL6M0WJvdRGL3q7SWo9O1DdBaBrPGm++Ue9WHo= -go.sia.tech/coreutils v0.0.3 h1:ZxuzovRpQMvfy/pCOV4om1cPF6sE15GyJyK36kIrF1Y= -go.sia.tech/coreutils v0.0.3/go.mod h1:UBFc77wXiE//eyilO5HLOncIEj7F69j0Nv2OkFujtP0= +go.sia.tech/core v0.2.3 h1:k+10zeV1V4bYFCGFaUiubbwRxlyV96WXForVlKnc8Rc= +go.sia.tech/core v0.2.3/go.mod h1:24liZWimivGQF+h3d14ly9oEpMIYxHPSgEMKmunxxi0= +go.sia.tech/coreutils v0.0.4-0.20240502154058-5df7ad9c0b7c h1:P6IyJYBciAiEpko6TxaOR3Rnp0Fiv7f66wgMzeM07GM= +go.sia.tech/coreutils v0.0.4-0.20240502154058-5df7ad9c0b7c/go.mod h1:JUwBPvP4716BZoY3XEWLswC2AsiBrQbBfgmC8CZVZQw= go.sia.tech/gofakes3 v0.0.3 h1:yvCIHGKb5EtZMiOe+GcO3VoXCfE7inMRq/LOnHRBw1c= go.sia.tech/gofakes3 v0.0.3/go.mod h1:+NLzpsL6M0WJvdRGL3q7SWo9O1DdBaBrPGm++Ue9WHo= -go.sia.tech/hostd v1.0.4 h1:rFzuNJ7sSFQfdrTHKSNYyMX+wlHyei/vZcVbXmrUl6I= -go.sia.tech/hostd v1.0.4/go.mod h1:s1W4/Okfcs2rGM3sC7xL95HY+I/oJ0Dsix3zTER+hpQ= +go.sia.tech/hostd v1.1.0-beta.1.0.20240507093050-bc2a34c93355 h1:i3Pacf8JlxDztmIV2om7pem1crIxE97P4AHHMF0nFYg= +go.sia.tech/hostd v1.1.0-beta.1.0.20240507093050-bc2a34c93355/go.mod h1:n0la8Syhg8+75c7ukkAUnlCrAOlrGrEPiAQ4lfbLL1Y= go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 h1:mSaJ622P7T/M97dAK8iPV+IRIC9M5vV28NHeceoWO3M= go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640/go.mod h1:4QqmBB+t3W7cNplXPj++ZqpoUb2PeiS66RLpXmEGap4= go.sia.tech/mux v1.2.0 h1:ofa1Us9mdymBbGMY2XH/lSpY8itFsKIo/Aq8zwe+GHU= @@ -2550,8 +2547,8 @@ modernc.org/token v1.1.0/go.mod h1:UGzOrNV1mAFSEB63lOFHIpNRUVMvYTc6yu1SMY/XTDM= modernc.org/z v1.5.1/go.mod h1:eWFB510QWW5Th9YGZT81s+LwvaAs3Q2yr4sP0rmLkv8= moul.io/zapgorm2 v1.3.0 h1:+CzUTMIcnafd0d/BvBce8T4uPn6DQnpIrz64cyixlkk= moul.io/zapgorm2 v1.3.0/go.mod h1:nPVy6U9goFKHR4s+zfSo1xVFaoU7Qgd5DoCdOfzoCqs= -nhooyr.io/websocket v1.8.10 h1:mv4p+MnGrLDcPlBoWsvPP7XCzTYMXP9F9eIGoKbgx7Q= -nhooyr.io/websocket v1.8.10/go.mod h1:rN9OFWIUwuxg4fR5tELlYC04bXYowCP9GX47ivo2l+c= +nhooyr.io/websocket v1.8.11 h1:f/qXNc2/3DpoSZkHt1DQu6rj4zGC8JmkkLkWss0MgN0= +nhooyr.io/websocket v1.8.11/go.mod h1:rN9OFWIUwuxg4fR5tELlYC04bXYowCP9GX47ivo2l+c= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/pdf v0.1.1/go.mod h1:n8OzWcQ6Sp37PL01nO98y4iUCRdTGarVfzxY20ICaU4= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= diff --git a/internal/node/chainmanager.go b/internal/node/chainmanager.go index d0f27bed3..c389191f8 100644 --- a/internal/node/chainmanager.go +++ b/internal/node/chainmanager.go @@ -9,6 +9,7 @@ import ( "go.sia.tech/core/consensus" "go.sia.tech/core/types" + "go.sia.tech/renterd/bus" "go.sia.tech/siad/modules" stypes "go.sia.tech/siad/types" ) @@ -24,6 +25,7 @@ var ( type chainManager struct { cs modules.ConsensusSet + tp bus.TransactionPool network *consensus.Network close chan struct{} @@ -114,6 +116,11 @@ func (m *chainManager) AcceptBlock(b types.Block) error { return m.cs.AcceptBlock(sb) } +// PoolTransactions returns all transactions in the transaction pool +func (m *chainManager) PoolTransactions() []types.Transaction { + return m.tp.Transactions() +} + // Subscribe subscribes to the consensus set. func (m *chainManager) Subscribe(s modules.ConsensusSetSubscriber, ccID modules.ConsensusChangeID, cancel <-chan struct{}) error { if err := m.cs.ConsensusSetSubscribe(s, ccID, cancel); err != nil { @@ -130,7 +137,7 @@ func synced(timestamp stypes.Timestamp) bool { } // NewManager creates a new chain manager. -func NewChainManager(cs modules.ConsensusSet, network *consensus.Network) (*chainManager, error) { +func NewChainManager(cs modules.ConsensusSet, tp bus.TransactionPool, network *consensus.Network) (*chainManager, error) { height := cs.Height() block, ok := cs.BlockAtHeight(height) if !ok { @@ -139,6 +146,7 @@ func NewChainManager(cs modules.ConsensusSet, network *consensus.Network) (*chai m := &chainManager{ cs: cs, + tp: tp, network: network, tip: consensus.State{ Network: network, diff --git a/internal/node/node.go b/internal/node/node.go index f73429d86..839501942 100644 --- a/internal/node/node.go +++ b/internal/node/node.go @@ -16,12 +16,12 @@ import ( "go.sia.tech/core/gateway" "go.sia.tech/core/types" "go.sia.tech/coreutils" - "go.sia.tech/coreutils/chain" "go.sia.tech/coreutils/syncer" "go.sia.tech/coreutils/wallet" "go.sia.tech/renterd/alerts" "go.sia.tech/renterd/autopilot" "go.sia.tech/renterd/bus" + "go.sia.tech/renterd/chain" "go.sia.tech/renterd/config" "go.sia.tech/renterd/stores" "go.sia.tech/renterd/webhooks" @@ -45,12 +45,15 @@ type Bus interface { type BusConfig struct { config.Bus - Database config.Database - DatabaseLog config.DatabaseLog - Genesis types.Block - Logger *zap.Logger - Network *consensus.Network - SlabPruningInterval time.Duration + Database config.Database + DatabaseLog config.DatabaseLog + Genesis types.Block + Logger *zap.Logger + Network *consensus.Network + RetryTxIntervals []time.Duration + SlabPruningInterval time.Duration + SyncerSyncInterval time.Duration + SyncerPeerDiscoveryInterval time.Duration } type AutopilotConfig struct { @@ -63,15 +66,15 @@ type ( ShutdownFn = func(context.Context) error ) -func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger) (http.Handler, ShutdownFn, *chain.Manager, error) { +func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger) (http.Handler, ShutdownFn, *chain.Manager, *chain.Subscriber, error) { // create consensus directory consensusDir := filepath.Join(dir, "consensus") if err := os.MkdirAll(consensusDir, 0700); err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } - bdb, err := coreutils.OpenBoltChainDB(filepath.Join(dir, "chain.db")) + bdb, err := coreutils.OpenBoltChainDB(filepath.Join(consensusDir, "chain.db")) if err != nil { - return nil, nil, nil, fmt.Errorf("failed to open chain database: %w", err) + return nil, nil, nil, nil, fmt.Errorf("failed to open chain database: %w", err) } // create database connections @@ -94,7 +97,7 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger // create database directory dbDir := filepath.Join(dir, "db") if err := os.MkdirAll(dbDir, 0700); err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } // create SQLite connections @@ -113,29 +116,25 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger } alertsMgr := alerts.NewManager() - walletAddr := types.StandardUnlockHash(seed.PublicKey()) sqlStoreDir := filepath.Join(dir, "partial_slabs") - announcementMaxAge := time.Duration(cfg.AnnouncementMaxAgeHours) * time.Hour sqlStore, err := stores.NewSQLStore(stores.Config{ Conn: dbConn, ConnMetrics: dbMetricsConn, Alerts: alerts.WithOrigin(alertsMgr, "bus"), PartialSlabDir: sqlStoreDir, Migrate: true, - AnnouncementMaxAge: announcementMaxAge, - PersistInterval: cfg.PersistInterval, - WalletAddress: walletAddr, SlabBufferCompletionThreshold: cfg.SlabBufferCompletionThreshold, Logger: logger.Sugar(), GormLogger: dbLogger, - RetryTransactionIntervals: []time.Duration{200 * time.Millisecond, 500 * time.Millisecond, time.Second, 3 * time.Second, 10 * time.Second, 10 * time.Second}, + RetryTransactionIntervals: cfg.RetryTxIntervals, + WalletAddress: types.StandardUnlockHash(seed.PublicKey()), }) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } wh, err := webhooks.NewManager(logger.Named("webhooks").Sugar(), sqlStore) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } // Hook up webhooks to alerts. @@ -144,20 +143,20 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger // create chain manager store, state, err := chain.NewDBStore(bdb, cfg.Network, cfg.Genesis) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } cm := chain.NewManager(store, state) // create wallet w, err := wallet.NewSingleAddressWallet(seed, cm, sqlStore, wallet.WithReservationDuration(cfg.UsedUTXOExpiry)) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } // create syncer l, err := net.Listen("tcp", cfg.GatewayAddr) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } syncerAddr := l.Addr().String() @@ -172,17 +171,30 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger UniqueID: gateway.GenerateUniqueID(), NetAddress: syncerAddr, } - s := syncer.New(l, cm, sqlStore, header, syncer.WithSyncInterval(100*time.Millisecond), syncer.WithLogger(logger.Named("syncer"))) - b, err := bus.New(alertsMgr, wh, cm, s, w, sqlStore, sqlStore, sqlStore, sqlStore, sqlStore, sqlStore, logger) + opts := []syncer.Option{syncer.WithLogger(logger.Named("syncer"))} + if cfg.SyncerSyncInterval > 0 { + opts = append(opts, syncer.WithSyncInterval(cfg.SyncerSyncInterval)) + } + if cfg.SyncerPeerDiscoveryInterval > 0 { + opts = append(opts, syncer.WithPeerDiscoveryInterval(cfg.SyncerPeerDiscoveryInterval)) + } + s := syncer.New(l, cm, sqlStore, header, opts...) + + b, err := bus.New(alertsMgr, wh, cm, sqlStore, s, w, sqlStore, sqlStore, sqlStore, sqlStore, sqlStore, sqlStore, logger) if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err + } + + cs, err := chain.NewSubscriber(cm, sqlStore, types.StandardUnlockHash(seed.PublicKey()), time.Duration(cfg.AnnouncementMaxAgeHours)*time.Hour, logger.Named("chainsubscriber")) + if err != nil { + return nil, nil, nil, nil, err } // bootstrap the syncer if cfg.Bootstrap { if cfg.Network == nil { - return nil, nil, nil, errors.New("cannot bootstrap without a network") + return nil, nil, nil, nil, errors.New("cannot bootstrap without a network") } var bootstrapPeers []string @@ -194,12 +206,12 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger case "anagami": bootstrapPeers = syncer.AnagamiBootstrapPeers default: - return nil, nil, nil, fmt.Errorf("no available bootstrap peers for unknown network '%s'", cfg.Network.Name) + return nil, nil, nil, nil, fmt.Errorf("no available bootstrap peers for unknown network '%s'", cfg.Network.Name) } for _, addr := range bootstrapPeers { if err := sqlStore.AddPeer(addr); err != nil { - return nil, nil, nil, fmt.Errorf("%w: failed to add bootstrap peer '%s'", err, addr) + return nil, nil, nil, nil, fmt.Errorf("%w: failed to add bootstrap peer '%s'", err, addr) } } } @@ -207,29 +219,24 @@ func NewBus(cfg BusConfig, dir string, seed types.PrivateKey, logger *zap.Logger // start the syncer go s.Run() - // fetch chain index - ci, err := sqlStore.ChainIndex() + // start the subscriber + unsubscribeFn, err := cs.Run() if err != nil { - return nil, nil, nil, fmt.Errorf("%w: failed to fetch chain index", err) - } - - // subscribe the store to the chain manager - err = cm.AddSubscriber(sqlStore, ci) - if err != nil { - return nil, nil, nil, err + return nil, nil, nil, nil, err } shutdownFn := func(ctx context.Context) error { + unsubscribeFn() return errors.Join( l.Close(), + cs.Close(), w.Close(), b.Shutdown(ctx), sqlStore.Close(), - store.Close(), bdb.Close(), ) } - return b.Handler(), shutdownFn, cm, nil + return b.Handler(), shutdownFn, cm, cs, nil } func NewWorker(cfg config.Worker, s3Opts s3.Opts, b Bus, seed types.PrivateKey, l *zap.Logger) (http.Handler, http.Handler, ShutdownFn, error) { diff --git a/internal/test/e2e/__debug_bin1717411325 b/internal/test/e2e/__debug_bin1717411325 deleted file mode 100755 index 0353a696e..000000000 Binary files a/internal/test/e2e/__debug_bin1717411325 and /dev/null differ diff --git a/internal/test/e2e/blocklist_test.go b/internal/test/e2e/blocklist_test.go index 64acc2fba..dd72acd77 100644 --- a/internal/test/e2e/blocklist_test.go +++ b/internal/test/e2e/blocklist_test.go @@ -27,7 +27,8 @@ func TestBlocklist(t *testing.T) { tt := cluster.tt // fetch contracts - contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: test.AutopilotConfig.Contracts.Set}) + opts := api.ContractsOpts{ContractSet: test.AutopilotConfig.Contracts.Set} + contracts, err := b.Contracts(ctx, opts) tt.OK(err) if len(contracts) != 3 { t.Fatalf("unexpected number of contracts, %v != 3", len(contracts)) @@ -37,14 +38,15 @@ func TestBlocklist(t *testing.T) { hk1 := contracts[0].HostKey hk2 := contracts[1].HostKey hk3 := contracts[2].HostKey - b.UpdateHostAllowlist(ctx, []types.PublicKey{hk1, hk2}, nil, false) + err = b.UpdateHostAllowlist(ctx, []types.PublicKey{hk1, hk2}, nil, false) + tt.OK(err) // assert h3 is no longer in the contract set - tt.Retry(5, time.Second, func() error { - contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: test.AutopilotConfig.Contracts.Set}) + tt.Retry(100, 100*time.Millisecond, func() error { + contracts, err := b.Contracts(ctx, opts) tt.OK(err) if len(contracts) != 2 { - return fmt.Errorf("unexpected number of contracts, %v != 2", len(contracts)) + return fmt.Errorf("unexpected number of contracts in set '%v', %v != 2", opts.ContractSet, len(contracts)) } for _, c := range contracts { if c.HostKey == hk3 { @@ -60,11 +62,11 @@ func TestBlocklist(t *testing.T) { tt.OK(b.UpdateHostBlocklist(ctx, []string{h1.NetAddress}, nil, false)) // assert h1 is no longer in the contract set - tt.Retry(5, time.Second, func() error { + tt.Retry(100, 100*time.Millisecond, func() error { contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: test.AutopilotConfig.Contracts.Set}) tt.OK(err) if len(contracts) != 1 { - return fmt.Errorf("unexpected number of contracts, %v != 1", len(contracts)) + return fmt.Errorf("unexpected number of contracts in set '%v', %v != 1", opts.ContractSet, len(contracts)) } for _, c := range contracts { if c.HostKey == hk1 { @@ -77,11 +79,11 @@ func TestBlocklist(t *testing.T) { // clear the allowlist and blocklist and assert we have 3 contracts again tt.OK(b.UpdateHostAllowlist(ctx, nil, []types.PublicKey{hk1, hk2}, false)) tt.OK(b.UpdateHostBlocklist(ctx, nil, []string{h1.NetAddress}, false)) - tt.Retry(5, time.Second, func() error { - contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: test.AutopilotConfig.Contracts.Set}) + tt.Retry(100, 100*time.Millisecond, func() error { + contracts, err := b.Contracts(ctx, opts) tt.OK(err) if len(contracts) != 3 { - return fmt.Errorf("unexpected number of contracts, %v != 3", len(contracts)) + return fmt.Errorf("unexpected number of contracts in set '%v', %v != 3", opts.ContractSet, len(contracts)) } return nil }) diff --git a/internal/test/e2e/cluster.go b/internal/test/e2e/cluster.go index e43ca6d0b..300f0835c 100644 --- a/internal/test/e2e/cluster.go +++ b/internal/test/e2e/cluster.go @@ -19,12 +19,12 @@ import ( "go.sia.tech/core/consensus" "go.sia.tech/core/types" "go.sia.tech/coreutils" - "go.sia.tech/coreutils/chain" "go.sia.tech/jape" "go.sia.tech/renterd/api" "go.sia.tech/renterd/autopilot" "go.sia.tech/renterd/build" "go.sia.tech/renterd/bus" + "go.sia.tech/renterd/chain" "go.sia.tech/renterd/config" "go.sia.tech/renterd/internal/node" "go.sia.tech/renterd/internal/test" @@ -42,8 +42,7 @@ import ( ) const ( - testBusFlushInterval = 100 * time.Millisecond - testBusPersistInterval = 2 * time.Second + testBusFlushInterval = 100 * time.Millisecond ) var ( @@ -69,6 +68,7 @@ type TestCluster struct { network *consensus.Network cm *chain.Manager + cs *chain.Subscriber apID string dbName string dir string @@ -311,7 +311,7 @@ func newTestCluster(t *testing.T, opts testClusterOptions) *TestCluster { tt.OK(err) // Create bus. - b, bShutdownFn, cm, err := node.NewBus(busCfg, busDir, wk, logger) + b, bShutdownFn, cm, cs, err := node.NewBus(busCfg, busDir, wk, logger) tt.OK(err) busAuth := jape.BasicAuth(busPassword) @@ -371,6 +371,7 @@ func newTestCluster(t *testing.T, opts testClusterOptions) *TestCluster { logger: logger, network: busCfg.Network, cm: cm, + cs: cs, tt: tt, wk: wk, @@ -523,44 +524,9 @@ func (c *TestCluster) MineToRenewWindow() { c.tt.Fatalf("already in renew window: bh: %v, currentPeriod: %v, periodLength: %v, renewWindow: %v", cs.BlockHeight, ap.CurrentPeriod, ap.Config.Contracts.Period, renewWindowStart) } c.MineBlocks(renewWindowStart - cs.BlockHeight) - c.Sync() } -// sync blocks until the cluster is synced. -func (c *TestCluster) sync(hosts []*Host) { - c.tt.Helper() - c.tt.Retry(100, 100*time.Millisecond, func() error { - synced, err := c.synced(hosts) - if err != nil { - return err - } - if !synced { - return errors.New("cluster was unable to sync in time") - } - return nil - }) -} - -// synced returns true if bus and hosts are at the same blockheight. -func (c *TestCluster) synced(hosts []*Host) (bool, error) { - c.tt.Helper() - cs, err := c.Bus.ConsensusState(context.Background()) - if err != nil { - return false, err - } - if !cs.Synced { - return false, nil // can't be synced if bus itself isn't synced - } - for _, h := range hosts { - bh := h.cs.Height() - if cs.BlockHeight != uint64(bh) { - return false, nil - } - } - return true, nil -} - -// MineBlocks uses the bus' miner to mine n blocks. +// MineBlocks mines n blocks func (c *TestCluster) MineBlocks(n uint64) { c.tt.Helper() wallet, err := c.Bus.Wallet(context.Background()) @@ -569,21 +535,43 @@ func (c *TestCluster) MineBlocks(n uint64) { // If we don't have any hosts in the cluster mine all blocks right away. if len(c.hosts) == 0 { c.tt.OK(c.mineBlocks(wallet.Address, n)) - c.Sync() + c.sync() return } - // Otherwise mine blocks in batches of 3 to avoid going out of sync with - // hosts by too many blocks. + // Otherwise mine blocks in batches of 10 blocks to avoid going out of sync + // with hosts by too many blocks. for mined := uint64(0); mined < n; { toMine := n - mined if toMine > 10 { toMine = 10 } c.tt.OK(c.mineBlocks(wallet.Address, toMine)) - c.Sync() mined += toMine + c.sync() } + c.sync() +} + +func (c *TestCluster) sync() { + tip := c.cm.Tip() + c.tt.Retry(300, 100*time.Millisecond, func() error { + cs, err := c.Bus.ConsensusState(context.Background()) + if err != nil { + return err + } else if !cs.Synced { + return errors.New("bus is not synced") + } else if cs.BlockHeight < tip.Height { + return fmt.Errorf("subscriber hasn't caught up, %d < %d", cs.BlockHeight, tip.Height) + } + + for _, h := range c.hosts { + if hh := h.cs.Height(); uint64(hh) < cs.BlockHeight { + return fmt.Errorf("host %v is not synced, %v < %v", h.PublicKey(), hh, cs.BlockHeight) + } + } + return nil + }) } func (c *TestCluster) WaitForAccounts() []api.Account { @@ -723,30 +711,23 @@ func (c *TestCluster) AddHost(h *Host) { // Mine transaction. c.MineBlocks(1) - // Wait for hosts to sync up with consensus. - hosts := []*Host{h} - c.sync(hosts) - // Announce hosts. ctx, cancel := context.WithTimeout(context.Background(), time.Minute) defer cancel() - c.tt.OK(addStorageFolderToHost(ctx, hosts)) - c.tt.OK(announceHosts(hosts)) + c.tt.OK(addStorageFolderToHost(ctx, []*Host{h})) + c.tt.OK(announceHosts([]*Host{h})) // Mine a few blocks. The host should show up eventually. c.tt.Retry(10, time.Second, func() error { c.tt.Helper() - c.MineBlocks(1) _, err := c.Bus.Host(context.Background(), h.PublicKey()) if err != nil { + c.MineBlocks(1) return err } return nil }) - - // Wait for host to be synced. - c.Sync() } // AddHosts adds n hosts to the cluster. These hosts will be funded and announce @@ -793,12 +774,6 @@ func (c *TestCluster) Shutdown() { c.wg.Wait() } -// Sync blocks until the whole cluster has reached the same block height. -func (c *TestCluster) Sync() { - c.tt.Helper() - c.sync(c.hosts) -} - // waitForHostAccounts will fetch the accounts from the worker and wait until // they have money in them func (c *TestCluster) waitForHostAccounts(hosts map[types.PublicKey]struct{}) { @@ -868,8 +843,8 @@ func (c *TestCluster) waitForHostContracts(hosts map[types.PublicKey]struct{}) { func (c *TestCluster) mineBlocks(addr types.Address, n uint64) error { for i := uint64(0); i < n; i++ { - if block, found := coreutils.MineBlock(c.cm, addr, time.Second); !found { - return errors.New("failed to find block") + if block, found := coreutils.MineBlock(c.cm, addr, 5*time.Second); !found { + c.tt.Fatal("failed to mine block") } else if err := c.Bus.AcceptBlock(context.Background(), block); err != nil { return err } @@ -922,12 +897,9 @@ func testBusCfg() node.BusConfig { AnnouncementMaxAgeHours: 24 * 7 * 52, // 1 year Bootstrap: false, GatewayAddr: "127.0.0.1:0", - PersistInterval: testBusPersistInterval, UsedUTXOExpiry: time.Minute, SlabBufferCompletionThreshold: 0, }, - Network: network, - Genesis: genesis, Database: config.Database{ MySQL: config.MySQLConfigFromEnv(), }, @@ -936,7 +908,19 @@ func testBusCfg() node.BusConfig { IgnoreRecordNotFoundError: true, SlowThreshold: 100 * time.Millisecond, }, - SlabPruningInterval: time.Second, + Network: network, + Genesis: genesis, + SlabPruningInterval: time.Second, + SyncerSyncInterval: 100 * time.Millisecond, + SyncerPeerDiscoveryInterval: 100 * time.Millisecond, + RetryTxIntervals: []time.Duration{ + 50 * time.Millisecond, + 100 * time.Millisecond, + 200 * time.Millisecond, + 500 * time.Millisecond, + time.Second, + 5 * time.Second, + }, } } diff --git a/internal/test/e2e/cluster_test.go b/internal/test/e2e/cluster_test.go index d6a0a10e3..fd95595ed 100644 --- a/internal/test/e2e/cluster_test.go +++ b/internal/test/e2e/cluster_test.go @@ -24,6 +24,7 @@ import ( "go.sia.tech/coreutils/wallet" "go.sia.tech/renterd/alerts" "go.sia.tech/renterd/api" + "go.sia.tech/renterd/autopilot/contractor" "go.sia.tech/renterd/internal/test" "go.sia.tech/renterd/object" "go.uber.org/zap" @@ -73,13 +74,14 @@ func TestNewTestCluster(t *testing.T) { cluster.MineToRenewWindow() // Wait for the contract to be renewed. + var renewalID types.FileContractID tt.Retry(100, 100*time.Millisecond, func() error { contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { return err } if len(contracts) != 1 { - return errors.New("no renewed contract") + return fmt.Errorf("unexpected number of contracts %d != 1", len(contracts)) } if contracts[0].RenewedFrom != contract.ID { return fmt.Errorf("contract wasn't renewed %v != %v", contracts[0].RenewedFrom, contract.ID) @@ -93,6 +95,7 @@ func TestNewTestCluster(t *testing.T) { if contracts[0].State != api.ContractStatePending { return fmt.Errorf("contract should be pending but was %v", contracts[0].State) } + renewalID = contracts[0].ID return nil }) @@ -101,7 +104,6 @@ func TestNewTestCluster(t *testing.T) { cs, err := cluster.Bus.ConsensusState(context.Background()) tt.OK(err) cluster.MineBlocks(contract.WindowStart - cs.BlockHeight - 4) - cluster.Sync() if cs.LastBlockTime.IsZero() { t.Fatal("last block time not set") } @@ -109,14 +111,7 @@ func TestNewTestCluster(t *testing.T) { // Now wait for the revision and proof to be caught by the hostdb. var ac api.ArchivedContract tt.Retry(20, time.Second, func() error { - cluster.MineBlocks(1) - - // Fetch renewed contract and make sure we caught the proof and revision. - contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) - if err != nil { - t.Fatal(err) - } - archivedContracts, err := cluster.Bus.AncestorContracts(context.Background(), contracts[0].ID, 0) + archivedContracts, err := cluster.Bus.AncestorContracts(context.Background(), renewalID, 0) if err != nil { t.Fatal(err) } @@ -125,7 +120,7 @@ func TestNewTestCluster(t *testing.T) { } ac = archivedContracts[0] if ac.RevisionHeight == 0 || ac.RevisionNumber != math.MaxUint64 { - return fmt.Errorf("revision information is wrong: %v %v", ac.RevisionHeight, ac.RevisionNumber) + return fmt.Errorf("revision information is wrong: %v %v %v", ac.RevisionHeight, ac.RevisionNumber, ac.ID) } if ac.ProofHeight != 0 { t.Fatal("proof height should be 0 since the contract was renewed and therefore doesn't require a proof") @@ -133,13 +128,6 @@ func TestNewTestCluster(t *testing.T) { if ac.State != api.ContractStateComplete { return fmt.Errorf("contract should be complete but was %v", ac.State) } - archivedContracts, err = cluster.Bus.AncestorContracts(context.Background(), contracts[0].ID, math.MaxUint32) - if err != nil { - t.Fatal(err) - } - if len(archivedContracts) != 0 { - return fmt.Errorf("should have 0 archived contracts but got %v", len(archivedContracts)) - } return nil }) @@ -596,13 +584,16 @@ func TestUploadDownloadBasic(t *testing.T) { } // check that stored data on hosts was updated - hosts, err := cluster.Bus.Hosts(context.Background(), api.GetHostsOptions{}) - tt.OK(err) - for _, host := range hosts { - if host.StoredData != rhpv2.SectorSize { - t.Fatalf("stored data should be %v, got %v", rhpv2.SectorSize, host.StoredData) + tt.Retry(100, 100*time.Millisecond, func() error { + hosts, err := cluster.Bus.Hosts(context.Background(), api.GetHostsOptions{}) + tt.OK(err) + for _, host := range hosts { + if host.StoredData != rhpv2.SectorSize { + return fmt.Errorf("stored data should be %v, got %v", rhpv2.SectorSize, host.StoredData) + } } - } + return nil + }) } // TestUploadDownloadExtended is an integration test that verifies objects can @@ -759,7 +750,8 @@ func TestUploadDownloadSpending(t *testing.T) { // create a test cluster cluster := newTestCluster(t, testClusterOptions{ - hosts: test.RedundancySettings.TotalShards, + hosts: test.RedundancySettings.TotalShards, + logger: zap.NewNop(), }) defer cluster.Shutdown() @@ -771,8 +763,8 @@ func TestUploadDownloadSpending(t *testing.T) { tt.Retry(100, testBusFlushInterval, func() error { cms, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) - if len(cms) == 0 { - t.Fatal("no contracts found") + if len(cms) != test.RedundancySettings.TotalShards { + t.Fatalf("unexpected number of contracts %v", len(cms)) } nFunded := 0 @@ -977,29 +969,16 @@ func TestEphemeralAccounts(t *testing.T) { t.SkipNow() } - cluster := newTestCluster(t, testClusterOptions{}) + // Create cluster + cluster := newTestCluster(t, testClusterOptions{hosts: 1}) defer cluster.Shutdown() tt := cluster.tt - // add host - nodes := cluster.AddHosts(1) - host := nodes[0] - - // make the cost of fetching a revision 0. That allows us to check for exact - // balances when funding the account and avoid NDFs. - settings := host.settings.Settings() - settings.BaseRPCPrice = types.ZeroCurrency - settings.EgressPrice = types.ZeroCurrency - if err := host.settings.UpdateSettings(settings); err != nil { - t.Fatal(err) - } - - // Wait for contracts to form. - var contract api.Contract - contracts := cluster.WaitForContracts() - contract = contracts[0] + // Shut down the autopilot to prevent it from interfering. + cluster.ShutdownAutopilot(context.Background()) - // Wait for account to appear. + // Wait for contract and accounts. + contract := cluster.WaitForContracts()[0] accounts := cluster.WaitForAccounts() // Shut down the autopilot to prevent it from interfering with the test. @@ -1013,7 +992,7 @@ func TestEphemeralAccounts(t *testing.T) { } else if acc.RequiresSync { t.Fatal("new account should not require a sync") } - if err := cluster.Bus.SetBalance(context.Background(), acc.ID, acc.HostKey, acc.Balance); err != nil { + if err := cluster.Bus.SetBalance(context.Background(), acc.ID, acc.HostKey, types.Siacoins(1).Big()); err != nil { t.Fatal(err) } } @@ -1023,13 +1002,13 @@ func TestEphemeralAccounts(t *testing.T) { tt.OK(err) acc := accounts[0] - minExpectedBalance := types.Siacoins(1).Sub(types.NewCurrency64(1)) - if acc.Balance.Cmp(minExpectedBalance.Big()) < 0 { + if acc.Balance.Cmp(types.Siacoins(1).Big()) < 0 { t.Fatalf("wrong balance %v", acc.Balance) } if acc.ID == (rhpv3.Account{}) { t.Fatal("account id not set") } + host := cluster.hosts[0] if acc.HostKey != types.PublicKey(host.PublicKey()) { t.Fatal("wrong host") } @@ -1350,8 +1329,7 @@ func TestContractArchival(t *testing.T) { // create a test cluster cluster := newTestCluster(t, testClusterOptions{ - hosts: 1, - logger: zap.NewNop(), + hosts: 1, }) defer cluster.Shutdown() tt := cluster.tt @@ -1379,7 +1357,14 @@ func TestContractArchival(t *testing.T) { return err } if len(contracts) != 0 { - return fmt.Errorf("expected 0 contracts, got %v", len(contracts)) + // trigger contract maintenance again, there's an NDF where we use + // the keep leeway because we can't fetch the revision preventing + // the contract from being archived + _, err := cluster.Autopilot.Trigger(false) + tt.OK(err) + + cs, _ := cluster.Bus.ConsensusState(context.Background()) + return fmt.Errorf("expected 0 contracts, got %v (bh: %v we: %v)", len(contracts), cs.BlockHeight, contracts[0].WindowEnd) } return nil }) @@ -1391,10 +1376,7 @@ func TestUnconfirmedContractArchival(t *testing.T) { } // create a test cluster - cluster := newTestCluster(t, testClusterOptions{ - logger: zap.NewNop(), - hosts: 1, - }) + cluster := newTestCluster(t, testClusterOptions{hosts: 1}) defer cluster.Shutdown() tt := cluster.tt @@ -1439,9 +1421,8 @@ func TestUnconfirmedContractArchival(t *testing.T) { t.Fatalf("expected 2 contracts, got %v", len(contracts)) } - // mine for 20 blocks to make sure we are beyond the 18 block deadline for - // contract confirmation - cluster.MineBlocks(20) + // mine enough blocks to ensure we're passed the confirmation deadline + cluster.MineBlocks(contractor.ContractConfirmationDeadline + 1) tt.Retry(100, 100*time.Millisecond, func() error { contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) @@ -1488,7 +1469,7 @@ func TestWalletTransactions(t *testing.T) { txns, err := b.WalletTransactions(context.Background(), api.WalletTransactionsWithOffset(2)) tt.OK(err) if !reflect.DeepEqual(txns, allTxns[2:]) { - t.Fatal("transactions don't match") + t.Fatal("transactions don't match", cmp.Diff(txns, allTxns[2:])) } // Find the first index that has a different timestamp than the first. @@ -2303,34 +2284,27 @@ func TestBusRecordedMetrics(t *testing.T) { }) defer cluster.Shutdown() - // Get contract set metrics. - csMetrics, err := cluster.Bus.ContractSetMetrics(context.Background(), startTime, api.MetricMaxIntervals, time.Second, api.ContractSetMetricsQueryOpts{}) - cluster.tt.OK(err) + // fetch contract set metrics + cluster.tt.Retry(100, 100*time.Millisecond, func() error { + csMetrics, err := cluster.Bus.ContractSetMetrics(context.Background(), startTime, api.MetricMaxIntervals, time.Second, api.ContractSetMetricsQueryOpts{}) + cluster.tt.OK(err) - for i := 0; i < len(csMetrics); i++ { - // Remove metrics from before contract was formed. - if csMetrics[i].Contracts > 0 { - csMetrics = csMetrics[i:] - break - } - } - if len(csMetrics) == 0 { - t.Fatal("expected at least 1 metric with contracts") - } - for _, m := range csMetrics { - if m.Contracts != 1 { - t.Fatalf("expected 1 contract, got %v", m.Contracts) + // expect at least 1 metric with contracts + if len(csMetrics) < 1 { + return fmt.Errorf("expected at least 1 metric, got %v", len(csMetrics)) + } else if m := csMetrics[len(csMetrics)-1]; m.Contracts != 1 { + return fmt.Errorf("expected 1 contract, got %v", m.Contracts) } else if m.Name != test.ContractSet { - t.Fatalf("expected contract set %v, got %v", test.ContractSet, m.Name) + return fmt.Errorf("expected contract set %v, got %v", test.ContractSet, m.Name) } else if m.Timestamp.Std().Before(startTime) { - t.Fatalf("expected time to be after start time %v, got %v", startTime, m.Timestamp.Std()) + return fmt.Errorf("expected time to be after start time %v, got %v", startTime, m.Timestamp.Std()) } - } + return nil + }) - // Get churn metrics. Should have 1 for the new contract. + // get churn metrics, should have 1 for the new contract cscMetrics, err := cluster.Bus.ContractSetChurnMetrics(context.Background(), startTime, api.MetricMaxIntervals, time.Second, api.ContractSetChurnMetricsQueryOpts{}) cluster.tt.OK(err) - if len(cscMetrics) != 1 { t.Fatalf("expected 1 metric, got %v", len(cscMetrics)) } else if m := cscMetrics[0]; m.Direction != api.ChurnDirAdded { @@ -2343,7 +2317,7 @@ func TestBusRecordedMetrics(t *testing.T) { t.Fatalf("expected time to be after start time %v, got %v", startTime, m.Timestamp.Std()) } - // Get contract metrics. + // get contract metrics var cMetrics []api.ContractMetric cluster.tt.Retry(100, 100*time.Millisecond, func() error { // Retry fetching metrics since they are buffered. @@ -2381,7 +2355,7 @@ func TestBusRecordedMetrics(t *testing.T) { t.Fatal("expected zero ListSpending") } - // Prune one of the metrics + // prune one of the metrics if err := cluster.Bus.PruneMetrics(context.Background(), api.MetricContract, time.Now()); err != nil { t.Fatal(err) } else if cMetrics, err = cluster.Bus.ContractMetrics(context.Background(), startTime, api.MetricMaxIntervals, time.Second, api.ContractMetricsQueryOpts{}); err != nil { diff --git a/internal/test/e2e/gouging_test.go b/internal/test/e2e/gouging_test.go index 28332ac1c..ae15db7f4 100644 --- a/internal/test/e2e/gouging_test.go +++ b/internal/test/e2e/gouging_test.go @@ -12,7 +12,6 @@ import ( "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/internal/test" - "go.uber.org/zap/zapcore" "lukechampine.com/frand" ) @@ -22,9 +21,7 @@ func TestGouging(t *testing.T) { } // create a new test cluster - cluster := newTestCluster(t, testClusterOptions{ - logger: newTestLoggerCustom(zapcore.ErrorLevel), - }) + cluster := newTestCluster(t, clusterOptsDefault) defer cluster.Shutdown() cfg := test.AutopilotConfig.Contracts diff --git a/internal/test/e2e/host.go b/internal/test/e2e/host.go index 428712032..3ffc633cd 100644 --- a/internal/test/e2e/host.go +++ b/internal/test/e2e/host.go @@ -175,24 +175,22 @@ func NewHost(privKey types.PrivateKey, dir string, network *consensus.Network, d if err := <-errCh; err != nil { return nil, fmt.Errorf("failed to create consensus set: %w", err) } - cm, err := node.NewChainManager(cs, network) - if err != nil { - return nil, err - } - tpool, err := transactionpool.New(cs, g, filepath.Join(dir, "transactionpool")) if err != nil { return nil, fmt.Errorf("failed to create transaction pool: %w", err) } tp := node.NewTransactionPool(tpool) - + cm, err := node.NewChainManager(cs, tp, network) + if err != nil { + return nil, err + } log := zap.NewNop() db, err := sqlite.OpenDatabase(filepath.Join(dir, "hostd.db"), log.Named("sqlite")) if err != nil { return nil, fmt.Errorf("failed to create sql store: %w", err) } - wallet, err := wallet.NewSingleAddressWallet(privKey, cm, tp, db, log.Named("wallet")) + wallet, err := wallet.NewSingleAddressWallet(privKey, cm, db, log.Named("wallet")) if err != nil { return nil, fmt.Errorf("failed to create wallet: %w", err) } diff --git a/internal/test/e2e/metadata_test.go b/internal/test/e2e/metadata_test.go index 4bb1ea2dd..4dd6c1229 100644 --- a/internal/test/e2e/metadata_test.go +++ b/internal/test/e2e/metadata_test.go @@ -10,7 +10,6 @@ import ( "go.sia.tech/renterd/api" "go.sia.tech/renterd/internal/test" - "go.uber.org/zap" ) func TestObjectMetadata(t *testing.T) { @@ -20,8 +19,7 @@ func TestObjectMetadata(t *testing.T) { // create cluster cluster := newTestCluster(t, testClusterOptions{ - hosts: test.RedundancySettings.TotalShards, - logger: zap.NewNop(), + hosts: test.RedundancySettings.TotalShards, }) defer cluster.Shutdown() diff --git a/internal/test/e2e/s3_test.go b/internal/test/e2e/s3_test.go index daaefed5e..3f20e22ad 100644 --- a/internal/test/e2e/s3_test.go +++ b/internal/test/e2e/s3_test.go @@ -18,7 +18,6 @@ import ( "go.sia.tech/gofakes3" "go.sia.tech/renterd/api" "go.sia.tech/renterd/internal/test" - "go.uber.org/zap" "lukechampine.com/frand" ) @@ -195,8 +194,7 @@ func TestS3ObjectMetadata(t *testing.T) { // create cluster opts := testClusterOptions{ - hosts: test.RedundancySettings.TotalShards, - logger: zap.NewNop(), + hosts: test.RedundancySettings.TotalShards, } cluster := newTestCluster(t, opts) defer cluster.Shutdown() diff --git a/stores/chain.go b/stores/chain.go index 2920c8b37..823344841 100644 --- a/stores/chain.go +++ b/stores/chain.go @@ -1,6 +1,8 @@ package stores import ( + "context" + "errors" "fmt" "time" @@ -8,6 +10,7 @@ import ( "go.sia.tech/coreutils/wallet" "go.sia.tech/renterd/api" "go.sia.tech/renterd/chain" + "go.uber.org/zap" "gorm.io/gorm" "gorm.io/gorm/clause" ) @@ -20,24 +23,44 @@ var ( // chainUpdateTx implements the ChainUpdateTx interface. type chainUpdateTx struct { tx *gorm.DB + + logs []logEntry // only logged if tx was successfully committed } -// BeginChainUpdateTx starts a transaction and wraps it in a chainUpdateTx. This -// transaction will be used to process a chain update in the subscriber. -func (s *SQLStore) BeginChainUpdateTx() (chain.ChainUpdateTx, error) { - tx := s.db.Begin() - if tx.Error != nil { - return nil, tx.Error - } - return &chainUpdateTx{tx: tx}, nil +type logEntry struct { + msg string + keysAndValues []interface{} +} + +// ProcessChainUpdate returns a callback function that process a chain update +// inside a transaction. +func (s *SQLStore) ProcessChainUpdate(ctx context.Context, fn func(chain.ChainUpdateTx) error) error { + return s.retryTransaction(ctx, func(tx *gorm.DB) error { + updateTx := &chainUpdateTx{tx: tx} + err := fn(updateTx) + if err == nil { + updateTx.log(s.logger.Named("ProcessChainUpdate")) + } + return err + }) +} + +// UpdateChainState process the given revert and apply updates. +func (s *SQLStore) UpdateChainState(reverted []chain.RevertUpdate, applied []chain.ApplyUpdate) error { + return s.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + return wallet.UpdateChainState(tx, s.walletAddress, applied, reverted) + }) } // ApplyIndex is called with the chain index that is being applied. Any // transactions and siacoin elements that were created by the index should be // added and any siacoin elements that were spent should be removed. func (u *chainUpdateTx) ApplyIndex(index types.ChainIndex, created, spent []types.SiacoinElement, events []wallet.Event) error { + u.debug("applying index", "height", index.Height, "block_id", index.ID) + // remove spent outputs for _, e := range spent { + u.debug(fmt.Sprintf("remove output %v", e.ID), "height", index.Height, "block_id", index.ID) if res := u.tx. Where("output_id", hash256(e.ID)). Delete(&dbWalletOutput{}); res.Error != nil { @@ -49,6 +72,7 @@ func (u *chainUpdateTx) ApplyIndex(index types.ChainIndex, created, spent []type // create outputs for _, e := range created { + u.debug(fmt.Sprintf("create output %v", e.ID), "height", index.Height, "block_id", index.ID) if err := u.tx. Clauses(clause.OnConflict{ DoNothing: true, @@ -61,8 +85,6 @@ func (u *chainUpdateTx) ApplyIndex(index types.ChainIndex, created, spent []type Value: currency(e.SiacoinOutput.Value), Address: hash256(e.SiacoinOutput.Address), MaturityHeight: e.MaturityHeight, - Height: index.Height, - BlockID: hash256(index.ID), }).Error; err != nil { return nil } @@ -70,6 +92,7 @@ func (u *chainUpdateTx) ApplyIndex(index types.ChainIndex, created, spent []type // create events for _, e := range events { + u.debug(fmt.Sprintf("create event %v", e.ID), "height", index.Height, "block_id", index.ID) if err := u.tx. Clauses(clause.OnConflict{ DoNothing: true, @@ -92,59 +115,41 @@ func (u *chainUpdateTx) ApplyIndex(index types.ChainIndex, created, spent []type return nil } -// Commit commits the updates to the database. -func (u *chainUpdateTx) Commit() error { - return u.tx.Commit().Error -} - -// Rollback rolls back the transaction -func (u *chainUpdateTx) Rollback() error { - return u.tx.Rollback().Error -} - // ContractState returns the state of a file contract. func (u *chainUpdateTx) ContractState(fcid types.FileContractID) (api.ContractState, error) { - var state contractState - err := u.tx. - Select("state"). + // try regular contracts + var c dbContract + if err := u.tx. Model(&dbContract{}). Where("fcid", fileContractID(fcid)). - Scan(&state). - Error - - if err == gorm.ErrRecordNotFound { - err = u.tx. - Select("state"). - Model(&dbArchivedContract{}). - Where("fcid", fileContractID(fcid)). - Scan(&state). - Error + Take(&c). + Error; err != nil && !errors.Is(err, gorm.ErrRecordNotFound) { + return "", err + } else if err == nil { + return api.ContractState(c.State.String()), nil } - if err != nil { + // try archived contracts + var ac dbArchivedContract + if err := u.tx. + Model(&dbArchivedContract{}). + Where("fcid", fileContractID(fcid)). + Take(&ac). + Error; err != nil && !errors.Is(err, gorm.ErrRecordNotFound) { return "", err + } else if err == nil { + return api.ContractState(ac.State.String()), nil } - return api.ContractState(state.String()), nil -} -// RemoveSiacoinElements is called with all siacoin elements that were spent in -// the update. -func (u *chainUpdateTx) RemoveSiacoinElements(ids []types.SiacoinOutputID) error { - for _, id := range ids { - if err := u.tx. - Where("output_id", hash256(id)). - Delete(&dbWalletOutput{}). - Error; err != nil { - return err - } - } - return nil + return "", api.ErrContractNotFound } // RevertIndex is called with the chain index that is being reverted. Any // transactions and siacoin elements that were created by the index should be // removed. func (u *chainUpdateTx) RevertIndex(index types.ChainIndex, removed, unspent []types.SiacoinElement) error { + u.debug("reverting index", "height", index.Height, "block_id", index.ID) + // recreate unspent outputs for _, e := range unspent { if err := u.tx. @@ -159,11 +164,10 @@ func (u *chainUpdateTx) RevertIndex(index types.ChainIndex, removed, unspent []t Value: currency(e.SiacoinOutput.Value), Address: hash256(e.SiacoinOutput.Address), MaturityHeight: e.MaturityHeight, - Height: index.Height, - BlockID: hash256(index.ID), }).Error; err != nil { return nil } + u.debug(fmt.Sprintf("recreate unspent output %v", e.ID), "height", index.Height, "block_id", index.ID) } // remove outputs created at the reverted index @@ -174,18 +178,26 @@ func (u *chainUpdateTx) RevertIndex(index types.ChainIndex, removed, unspent []t Error; err != nil { return err } + u.debug(fmt.Sprintf("remove output %v", e.ID), "height", index.Height, "block_id", index.ID) } // remove events created at the reverted index - return u.tx. + res := u.tx. Model(&dbWalletEvent{}). Where("height = ? AND block_id = ?", index.Height, hash256(index.ID)). - Delete(&dbWalletEvent{}). - Error + Delete(&dbWalletEvent{}) + if res.Error != nil { + return res.Error + } + if res.RowsAffected > 0 { + u.debug(fmt.Sprintf("removed %d events", res.RowsAffected), "height", index.Height, "block_id", index.ID) + } + return nil } // UpdateChainIndex updates the chain index in the database. func (u *chainUpdateTx) UpdateChainIndex(index types.ChainIndex) error { + u.debug("updating index", "height", index.Height, "block_id", index.ID) return u.tx. Model(&dbConsensusInfo{}). Where(&dbConsensusInfo{Model: Model{ID: consensusInfoID}}). @@ -207,92 +219,140 @@ func (u *chainUpdateTx) UpdateContract(fcid types.FileContractID, revisionHeight return revisionNumber > currRev } - // update either active or archived contract - var update interface{} + // try regular contract var c dbContract - if err := u.tx. + err := u.tx. Model(&dbContract{}). Where("fcid", fileContractID(fcid)). - Take(&c).Error; err == nil { + Take(&c). + Error + if err != nil && !errors.Is(err, gorm.ErrRecordNotFound) { + return err + } else if err == nil { + // save old valules + oldRevH := c.RevisionHeight + oldRevN := c.RevisionNumber + oldSize := c.Size + c.RevisionHeight = revisionHeight if isUpdatedRevision(c.RevisionNumber) { c.RevisionNumber = fmt.Sprint(revisionNumber) c.Size = size } - update = c - } else if err == gorm.ErrRecordNotFound { - // try archived contracts - var ac dbArchivedContract - if err := u.tx. - Model(&dbArchivedContract{}). - Where("fcid", fileContractID(fcid)). - Take(&ac).Error; err == nil { - ac.RevisionHeight = revisionHeight - if isUpdatedRevision(ac.RevisionNumber) { - ac.RevisionNumber = fmt.Sprint(revisionNumber) - ac.Size = size - } - update = ac - } + + u.debug(fmt.Sprintf("update contract, revision number %s -> %s, revision height %d -> %d, size %d -> %d", oldRevN, c.RevisionNumber, oldRevH, c.RevisionHeight, oldSize, c.Size), "fcid", fcid) + return u.tx.Save(&c).Error } - if update == nil { - return nil + + // try archived contract + var ac dbArchivedContract + err = u.tx. + Model(&dbArchivedContract{}). + Where("fcid", fileContractID(fcid)). + Take(&ac). + Error + if err != nil && !errors.Is(err, gorm.ErrRecordNotFound) { + return err + } else if err == nil { + // save old valules + oldRevH := ac.RevisionHeight + oldRevN := ac.RevisionNumber + oldSize := ac.Size + + ac.RevisionHeight = revisionHeight + if isUpdatedRevision(ac.RevisionNumber) { + ac.RevisionNumber = fmt.Sprint(revisionNumber) + ac.Size = size + } + + u.debug(fmt.Sprintf("update archived contract, revision number %s -> %s, revision height %d -> %d, size %d -> %d", oldRevN, ac.RevisionNumber, oldRevH, ac.RevisionHeight, oldSize, ac.Size), "fcid", fcid) + return u.tx.Save(&ac).Error } - return u.tx.Save(update).Error + return api.ErrContractNotFound } // UpdateContractState updates the state of the contract with given fcid. func (u *chainUpdateTx) UpdateContractState(fcid types.FileContractID, state api.ContractState) error { + u.debug("update contract state", "fcid", fcid, "state", state) + var cs contractState if err := cs.LoadString(string(state)); err != nil { return err } - if err := u.tx. + // try regular contract + if res := u.tx. Model(&dbContract{}). Where("fcid", fileContractID(fcid)). - Update("state", cs). - Error; err != nil { - return err + Update("state", cs); res.Error != nil { + return res.Error + } else if res.RowsAffected > 0 { + return nil } - return u.tx. + + // try archived contract + if res := u.tx. Model(&dbArchivedContract{}). Where("fcid", fileContractID(fcid)). - Update("state", cs). - Error + Update("state", cs); res.Error != nil { + return res.Error + } else if res.RowsAffected > 0 { + return nil + } + + // wrap ErrContractNotFound + return fmt.Errorf("%v %w", fcid, api.ErrContractNotFound) } // UpdateContractProofHeight updates the proof height of the contract with given // fcid. func (u *chainUpdateTx) UpdateContractProofHeight(fcid types.FileContractID, proofHeight uint64) error { - if err := u.tx. + u.debug("update contract proof height", "fcid", fcid, "proof_height", proofHeight) + + // try regular contract + if res := u.tx. Model(&dbContract{}). Where("fcid", fileContractID(fcid)). - Update("proof_height", proofHeight). - Error; err != nil { - return err + Update("proof_height", proofHeight); res.Error != nil { + return res.Error + } else if res.RowsAffected > 0 { + return nil } - return u.tx. + + // try archived contract + if res := u.tx. Model(&dbArchivedContract{}). Where("fcid", fileContractID(fcid)). - Update("proof_height", proofHeight). - Error + Update("proof_height", proofHeight); res.Error != nil { + return res.Error + } else if res.RowsAffected > 0 { + return nil + } + + // wrap api.ErrContractNotFound + return fmt.Errorf("%v %w", fcid, api.ErrContractNotFound) } // UpdateFailedContracts marks active contract as failed if the current // blockheight surposses their window_end. func (u *chainUpdateTx) UpdateFailedContracts(blockHeight uint64) error { - return u.tx. + if res := u.tx. Model(&dbContract{}). Where("window_end <= ?", blockHeight). Where("state", contractStateActive). - Update("state", contractStateFailed). - Error + Update("state", contractStateFailed); res.Error != nil { + return res.Error + } else if res.RowsAffected > 0 { + u.debug(fmt.Sprintf("marked %d active contracts as failed", res.RowsAffected), "window_end", blockHeight) + } + return nil } // UpdateHost creates the announcement and upserts the host in the database. func (u *chainUpdateTx) UpdateHost(hk types.PublicKey, ha chain.HostAnnouncement, bh uint64, blockID types.BlockID, ts time.Time) error { + u.debug("updated host", "hk", hk, "netaddress", ha.NetAddress) + // create the announcement if err := u.tx.Create(&dbAnnouncement{ HostKey: publicKey(hk), @@ -334,18 +394,30 @@ func (u *chainUpdateTx) UpdateHost(hk types.PublicKey, ha chain.HostAnnouncement // UpdateStateElements updates the proofs of all state elements affected by the // update. func (u *chainUpdateTx) UpdateStateElements(elements []types.StateElement) error { - for _, se := range elements { - if err := u.tx. - Model(&dbWalletOutput{}). - Where("output_id", hash256(se.ID)). - Updates(map[string]interface{}{ - "merkle_proof": merkleProof{proof: se.MerkleProof}, - "leaf_index": se.LeafIndex, - }).Error; err != nil { - return err + if len(elements) == 0 { + return nil + } + + utxos, err := u.outputs() + if err != nil { + return err + } + + lookup := make(map[types.Hash256]int) + for i, utxo := range utxos { + lookup[types.Hash256(utxo.OutputID)] = i + } + + for _, el := range elements { + if index, ok := lookup[el.ID]; ok { + update := utxos[index] + update.LeafIndex = el.LeafIndex + update.MerkleProof = merkleProof{proof: el.MerkleProof} + utxos[index] = update } } - return nil + + return u.tx.Save(utxos).Error } // WalletStateElements implements the ChainStore interface and returns all state @@ -374,3 +446,22 @@ func (u *chainUpdateTx) WalletStateElements() ([]types.StateElement, error) { } return elements, nil } + +func (u *chainUpdateTx) outputs() (outputs []dbWalletOutput, err error) { + err = u.tx. + Model(&dbWalletOutput{}). + Find(&outputs). + Error + return +} + +func (u *chainUpdateTx) debug(msg string, keysAndValues ...interface{}) { + u.logs = append(u.logs, logEntry{msg: msg, keysAndValues: keysAndValues}) +} + +func (u *chainUpdateTx) log(l *zap.SugaredLogger) { + for _, log := range u.logs { + l.Debugw(log.msg, log.keysAndValues...) + } + u.logs = nil +} diff --git a/stores/chain_test.go b/stores/chain_test.go index 11bada2c4..addb290ec 100644 --- a/stores/chain_test.go +++ b/stores/chain_test.go @@ -2,6 +2,9 @@ package stores import ( "context" + "errors" + "fmt" + "strings" "testing" "go.sia.tech/core/types" @@ -9,8 +12,8 @@ import ( "go.sia.tech/renterd/chain" ) -// TestChainUpdateTx tests the chain update transaction. -func TestChainUpdateTx(t *testing.T) { +// TestProcessChainUpdate tests the ProcessChainUpdate method on the SQL store. +func TestProcessChainUpdate(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) // add test host and contract @@ -26,68 +29,58 @@ func TestChainUpdateTx(t *testing.T) { } fcid := fcids[0] - // assert commit with no changes is successful - tx, err := ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } - if err := tx.Commit(); err != nil { - t.Fatal("unexpected error", err) - } - - // assert rollback with no changes is successful - tx, err = ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } - if err := tx.Rollback(); err != nil { - t.Fatal("unexpected error", err) - } - // assert contract state returns the correct state - tx, err = ss.BeginChainUpdateTx() - if err != nil { + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + if state, err := tx.ContractState(fcid); err != nil { + return err + } else if state != api.ContractStatePending { + return fmt.Errorf("unexpected state '%v'", state) + } else { + return nil + } + }); err != nil { t.Fatal("unexpected error", err) } - state, err := tx.ContractState(fcid) - if err != nil { - t.Fatal("unexpected error", err) - } else if state != api.ContractStatePending { - t.Fatal("expected pending state", state) + + // check current index + if curr, err := ss.ChainIndex(context.Background()); err != nil { + t.Fatal(err) + } else if curr.Height != 0 { + t.Fatalf("unexpected height %v", curr.Height) } // assert update chain index is successful - if curr, err := ss.ChainIndex(); err != nil { + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + return tx.UpdateChainIndex(types.ChainIndex{Height: 1}) + }); err != nil { t.Fatal("unexpected error", err) - } else if curr.Height != 0 { - t.Fatal("unexpected height", curr.Height) } - index := types.ChainIndex{Height: 1} - if err := tx.UpdateChainIndex(index); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.Commit(); err != nil { - t.Fatal("unexpected error", err) + + // check updated index + if curr, err := ss.ChainIndex(context.Background()); err != nil { + t.Fatal(err) + } else if curr.Height != 1 { + t.Fatalf("unexpected height %v", curr.Height) } - if got, err := ss.ChainIndex(); err != nil { + + // assert update contract is successful + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + if err := tx.UpdateContract(fcid, 1, 2, 3); err != nil { + return err + } else if err := tx.UpdateContractState(fcid, api.ContractStateActive); err != nil { + return err + } else if err := tx.UpdateContractProofHeight(fcid, 4); err != nil { + return err + } else { + return nil + } + }); err != nil { t.Fatal("unexpected error", err) - } else if got.Height != index.Height { - t.Fatal("unexpected height", got.Height) } - // assert update contract is successful + // assert contract was updated successfully var we uint64 - tx, err = ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateContract(fcid, 1, 2, 3); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateContractState(fcid, api.ContractStateActive); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateContractProofHeight(fcid, 4); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.Commit(); err != nil { - t.Fatal("unexpected error", err) - } else if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { + if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { t.Fatal("unexpected error", err) } else if c.RevisionHeight != 1 { t.Fatal("unexpected revision height", c.RevisionHeight) @@ -102,14 +95,12 @@ func TestChainUpdateTx(t *testing.T) { } // assert we only update revision height if the rev number doesn't increase - tx, err = ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateContract(fcid, 2, 2, 4); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.Commit(); err != nil { + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + return tx.UpdateContract(fcid, 2, 2, 4) + }); err != nil { t.Fatal("unexpected error", err) - } else if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { + } + if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { t.Fatal("unexpected error", err) } else if c.RevisionHeight != 2 { t.Fatal("unexpected revision height", c.RevisionHeight) @@ -120,30 +111,53 @@ func TestChainUpdateTx(t *testing.T) { } // assert update failed contracts is successful - tx, err = ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateFailedContracts(we + 1); err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.Commit(); err != nil { + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + return tx.UpdateFailedContracts(we + 1) + }); err != nil { t.Fatal("unexpected error", err) - } else if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { + } + if c, err := ss.contract(context.Background(), fileContractID(fcid)); err != nil { t.Fatal("unexpected error", err) } else if c.State.String() != api.ContractStateFailed { t.Fatal("unexpected state", c.State) } // assert update host is successful - tx, err = ss.BeginChainUpdateTx() - if err != nil { - t.Fatal("unexpected error", err) - } else if err := tx.UpdateHost(hks[0], chain.HostAnnouncement{NetAddress: "foo"}, 1, types.BlockID{}, types.CurrentTimestamp()); err != nil { + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + return tx.UpdateHost(hks[0], chain.HostAnnouncement{NetAddress: "foo"}, 1, types.BlockID{}, types.CurrentTimestamp()) + }); err != nil { t.Fatal("unexpected error", err) - } else if err := tx.Commit(); err != nil { - t.Fatal("unexpected error", err) - } else if h, err := ss.Host(context.Background(), hks[0]); err != nil { + } + if h, err := ss.Host(context.Background(), hks[0]); err != nil { t.Fatal("unexpected error", err) } else if h.NetAddress != "foo" { t.Fatal("unexpected net address", h.NetAddress) } + + // assert passing empty function is successful + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { return nil }); err != nil { + t.Fatal("unexpected error", err) + } + + // assert we rollback on error + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { + if err := tx.UpdateChainIndex(types.ChainIndex{Height: 2}); err != nil { + return err + } + return errors.New("some error") + }); err == nil || !strings.Contains(err.Error(), "some error") { + t.Fatal("unexpected error", err) + } + + // check chain index was rolled back + if curr, err := ss.ChainIndex(context.Background()); err != nil { + t.Fatal(err) + } else if curr.Height != 1 { + t.Fatalf("unexpected height %v", curr.Height) + } + + // assert we recover from panic + if err := ss.ProcessChainUpdate(context.Background(), func(tx chain.ChainUpdateTx) error { return nil }); err != nil { + panic("oh no") + } } diff --git a/stores/hostdb.go b/stores/hostdb.go index ae3b47e47..9c54ebd9d 100644 --- a/stores/hostdb.go +++ b/stores/hostdb.go @@ -12,8 +12,8 @@ import ( rhpv2 "go.sia.tech/core/rhp/v2" rhpv3 "go.sia.tech/core/rhp/v3" "go.sia.tech/core/types" - "go.sia.tech/coreutils/chain" "go.sia.tech/renterd/api" + "go.sia.tech/renterd/chain" "gorm.io/gorm" "gorm.io/gorm/clause" ) @@ -429,7 +429,7 @@ func (ss *SQLStore) Host(ctx context.Context, hostKey types.PublicKey) (api.Host if err != nil { return api.Host{}, err } else if len(hosts) == 0 { - return api.Host{}, api.ErrHostNotFound + return api.Host{}, fmt.Errorf("%w %v", api.ErrHostNotFound, hostKey) } else { return hosts[0], nil } @@ -458,7 +458,7 @@ func (ss *SQLStore) UpdateHostCheck(ctx context.Context, autopilotID string, hk Select("id"). Take(&hID). Error; errors.Is(err, gorm.ErrRecordNotFound) { - return api.ErrHostNotFound + return fmt.Errorf("%w %v", api.ErrHostNotFound, hk) } else if err != nil { return err } @@ -1085,15 +1085,13 @@ func (ss *SQLStore) isBlocked(h dbHost) (blocked bool) { return } -func updateChainIndex(tx *gorm.DB, newTip types.ChainIndex) error { - return tx.Model(&dbConsensusInfo{}).Where(&dbConsensusInfo{ - Model: Model{ - ID: consensusInfoID, - }, - }).Updates(map[string]interface{}{ - "height": newTip.Height, - "block_id": hash256(newTip.ID), - }).Error +func (s *SQLStore) ResetLostSectors(ctx context.Context, hk types.PublicKey) error { + return s.retryTransaction(ctx, func(tx *gorm.DB) error { + return tx.Model(&dbHost{}). + Where("public_key", publicKey(hk)). + Update("lost_sectors", 0). + Error + }) } func insertAnnouncements(tx *gorm.DB, as []announcement) error { @@ -1118,83 +1116,6 @@ func insertAnnouncements(tx *gorm.DB, as []announcement) error { return tx.Create(&hosts).Error } -func applyRevisionUpdate(db *gorm.DB, fcid types.FileContractID, rev revisionUpdate) error { - // isUpdatedRevision indicates whether the given revision number is greater - // than the one currently set on the contract - isUpdatedRevision := func(currRevStr string) bool { - var currRev uint64 - _, _ = fmt.Sscan(currRevStr, &currRev) - return rev.number > currRev - } - - // update either active or archived contract - var update interface{} - var c dbContract - if err := db. - Model(&dbContract{}). - Where("fcid", fileContractID(fcid)). - Take(&c).Error; err == nil { - c.RevisionHeight = rev.height - if isUpdatedRevision(c.RevisionNumber) { - c.RevisionNumber = fmt.Sprint(rev.number) - c.Size = rev.size - } - update = c - } else if err == gorm.ErrRecordNotFound { - // try archived contracts - var ac dbArchivedContract - if err := db. - Model(&dbArchivedContract{}). - Where("fcid", fileContractID(fcid)). - Take(&ac).Error; err == nil { - ac.RevisionHeight = rev.height - if isUpdatedRevision(ac.RevisionNumber) { - ac.RevisionNumber = fmt.Sprint(rev.number) - ac.Size = rev.size - } - update = ac - } - } - if update == nil { - return nil - } - return db.Save(update).Error -} - -func updateContractState(db *gorm.DB, fcid types.FileContractID, cs contractState) error { - return updateActiveAndArchivedContract(db, fcid, map[string]interface{}{ - "state": cs, - }) -} - -func markFailedContracts(db *gorm.DB, height uint64) error { - if err := db.Model(&dbContract{}). - Where("state = ? AND ? > window_end", contractStateActive, height). - Update("state", contractStateFailed).Error; err != nil { - return fmt.Errorf("failed to mark failed contracts: %w", err) - } - return nil -} - -func updateProofHeight(db *gorm.DB, fcid types.FileContractID, blockHeight uint64) error { - return updateActiveAndArchivedContract(db, fcid, map[string]interface{}{ - "proof_height": blockHeight, - }) -} - -func updateActiveAndArchivedContract(tx *gorm.DB, fcid types.FileContractID, updates map[string]interface{}) error { - err1 := tx.Model(&dbContract{}). - Where("fcid", fileContractID(fcid)). - Updates(updates).Error - err2 := tx.Model(&dbArchivedContract{}). - Where("fcid", fileContractID(fcid)). - Updates(updates).Error - if err1 != nil || err2 != nil { - return fmt.Errorf("%s; %s", err1, err2) - } - return nil -} - func getBlocklists(tx *gorm.DB) ([]dbAllowlistEntry, []dbBlocklistEntry, error) { var allowlist []dbAllowlistEntry if err := tx. @@ -1246,12 +1167,3 @@ func updateBlocklist(tx *gorm.DB, hk types.PublicKey, allowlist []dbAllowlistEnt } return tx.Model(&host).Association("Blocklist").Replace(&dbBlocklist) } - -func (s *SQLStore) ResetLostSectors(ctx context.Context, hk types.PublicKey) error { - return s.retryTransaction(ctx, func(tx *gorm.DB) error { - return tx.Model(&dbHost{}). - Where("public_key", publicKey(hk)). - Update("lost_sectors", 0). - Error - }) -} diff --git a/stores/hostdb_test.go b/stores/hostdb_test.go index 039f2325f..9c8166577 100644 --- a/stores/hostdb_test.go +++ b/stores/hostdb_test.go @@ -11,15 +11,11 @@ import ( "github.com/google/go-cmp/cmp" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" - "go.sia.tech/coreutils/chain" "go.sia.tech/renterd/api" + "go.sia.tech/renterd/chain" "gorm.io/gorm" ) -func (s *SQLStore) insertTestAnnouncement(a announcement) error { - return insertAnnouncements(s.db, []announcement{a}) -} - // TestSQLHostDB tests the basic functionality of SQLHostDB using an in-memory // SQLite DB. func TestSQLHostDB(t *testing.T) { @@ -51,21 +47,20 @@ func TestSQLHostDB(t *testing.T) { // Insert an announcement for the host and another one for an unknown // host. - a := newTestAnnouncement(hk, "address") - err = ss.insertTestAnnouncement(a) + _, err = ss.announceHost(hk, "address") if err != nil { t.Fatal(err) } - // Read the host and verify that the announcement related fields were - // set. + // Fetch the host var h dbHost - tx := ss.db.Where("last_announcement = ? AND net_address = ?", a.timestamp, a.NetAddress).Find(&h) + tx := ss.db.Where("net_address = ?", "address").Find(&h) if tx.Error != nil { t.Fatal(tx.Error) - } - if types.PublicKey(h.PublicKey) != hk { + } else if types.PublicKey(h.PublicKey) != hk { t.Fatal("wrong host returned") + } else if h.LastAnnouncement.IsZero() { + t.Fatal("last announcement not set") } // Same thing again but with hosts. @@ -96,17 +91,16 @@ func TestSQLHostDB(t *testing.T) { } // Insert another announcement for an unknown host. - unknownKeyAnn := a - unknownKeyAnn.hk = types.PublicKey{1, 4, 7} - err = ss.insertTestAnnouncement(unknownKeyAnn) + randomHK := types.PublicKey{1, 4, 7} + _, err = ss.announceHost(types.PublicKey{1, 4, 7}, "na") if err != nil { t.Fatal(err) } - h3, err := ss.Host(ctx, unknownKeyAnn.hk) + h3, err := ss.Host(ctx, randomHK) if err != nil { t.Fatal(err) } - if h3.NetAddress != unknownKeyAnn.NetAddress { + if h3.NetAddress != "na" { t.Fatal("wrong net address") } if h3.KnownSince.IsZero() { @@ -548,6 +542,77 @@ func TestRecordScan(t *testing.T) { } } +// TestInsertAnnouncements is a test for insertAnnouncements. +func TestInsertAnnouncements(t *testing.T) { + ss := newTestSQLStore(t, defaultTestSQLStoreConfig) + defer ss.Close() + + // Create announcements for 3 hosts. + ann1 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "foo.bar:1000") + ann2 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") + ann3 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") + + // Insert the first one and check that all fields are set. + if err := insertAnnouncements(ss.db, []announcement{ann1}); err != nil { + t.Fatal(err) + } + var ann dbAnnouncement + if err := ss.db.Find(&ann).Error; err != nil { + t.Fatal(err) + } + ann.Model = Model{} // ignore + expectedAnn := dbAnnouncement{ + HostKey: publicKey(ann1.hk), + BlockHeight: ann1.blockHeight, + BlockID: ann1.blockID.String(), + NetAddress: "foo.bar:1000", + } + if ann != expectedAnn { + t.Fatal("mismatch", cmp.Diff(ann, expectedAnn)) + } + // Insert the first and second one. + if err := insertAnnouncements(ss.db, []announcement{ann1, ann2}); err != nil { + t.Fatal(err) + } + + // Insert the first one twice. The second one again and the third one. + if err := insertAnnouncements(ss.db, []announcement{ann1, ann2, ann1, ann3}); err != nil { + t.Fatal(err) + } + + // There should be 3 hosts in the db. + hosts, err := ss.hosts() + if err != nil { + t.Fatal(err) + } + if len(hosts) != 3 { + t.Fatal("invalid number of hosts") + } + + // There should be 7 announcements total. + var announcements []dbAnnouncement + if err := ss.db.Find(&announcements).Error; err != nil { + t.Fatal(err) + } + if len(announcements) != 7 { + t.Fatal("invalid number of announcements") + } + + // Add an entry to the blocklist to block host 1 + entry1 := "foo.bar" + err = ss.UpdateHostBlocklistEntries(context.Background(), []string{entry1}, nil, false) + if err != nil { + t.Fatal(err) + } + + // Insert multiple announcements for host 1 - this asserts that the UNIQUE + // constraint on the blocklist table isn't triggered when inserting multiple + // announcements for a host that's on the blocklist + if err := insertAnnouncements(ss.db, []announcement{ann1, ann1}); err != nil { + t.Fatal(err) + } +} + func TestRemoveHosts(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) defer ss.Close() @@ -649,77 +714,6 @@ func TestRemoveHosts(t *testing.T) { } } -// TestInsertAnnouncements is a test for insertAnnouncements. -func TestInsertAnnouncements(t *testing.T) { - ss := newTestSQLStore(t, defaultTestSQLStoreConfig) - defer ss.Close() - - // Create announcements for 3 hosts. - ann1 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "foo.bar:1000") - ann2 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") - ann3 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") - - // Insert the first one and check that all fields are set. - if err := insertAnnouncements(ss.db, []announcement{ann1}); err != nil { - t.Fatal(err) - } - var ann dbAnnouncement - if err := ss.db.Find(&ann).Error; err != nil { - t.Fatal(err) - } - ann.Model = Model{} // ignore - expectedAnn := dbAnnouncement{ - HostKey: publicKey(ann1.hk), - BlockHeight: ann1.blockHeight, - BlockID: ann1.blockID.String(), - NetAddress: "foo.bar:1000", - } - if ann != expectedAnn { - t.Fatal("mismatch", cmp.Diff(ann, expectedAnn)) - } - // Insert the first and second one. - if err := insertAnnouncements(ss.db, []announcement{ann1, ann2}); err != nil { - t.Fatal(err) - } - - // Insert the first one twice. The second one again and the third one. - if err := insertAnnouncements(ss.db, []announcement{ann1, ann2, ann1, ann3}); err != nil { - t.Fatal(err) - } - - // There should be 3 hosts in the db. - hosts, err := ss.hosts() - if err != nil { - t.Fatal(err) - } - if len(hosts) != 3 { - t.Fatal("invalid number of hosts") - } - - // There should be 7 announcements total. - var announcements []dbAnnouncement - if err := ss.db.Find(&announcements).Error; err != nil { - t.Fatal(err) - } - if len(announcements) != 7 { - t.Fatal("invalid number of announcements") - } - - // Add an entry to the blocklist to block host 1 - entry1 := "foo.bar" - err = ss.UpdateHostBlocklistEntries(context.Background(), []string{entry1}, nil, false) - if err != nil { - t.Fatal(err) - } - - // Insert multiple announcements for host 1 - this asserts that the UNIQUE - // constraint on the blocklist table isn't triggered when inserting multiple - // announcements for a host that's on the blocklist - if err := insertAnnouncements(ss.db, []announcement{ann1, ann1}); err != nil { - t.Fatal(err) - } -} - func TestSQLHostAllowlist(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) defer ss.Close() @@ -1173,41 +1167,6 @@ func TestSQLHostBlocklistBasic(t *testing.T) { } } -// TestAnnouncementMaxAge verifies old announcements are ignored. -func TestAnnouncementMaxAge(t *testing.T) { - db := newTestSQLStore(t, defaultTestSQLStoreConfig) - defer db.Close() - - // assert we don't have any announcements - if len(db.cs.announcements) != 0 { - t.Fatal("expected 0 announcements") - } - - // fabricate two blocks with announcements, one before the cutoff and one after - b1 := types.Block{ - Transactions: []types.Transaction{newTestTransaction(newTestHostAnnouncement("foo.com:1000"))}, - Timestamp: time.Now().Add(-db.cs.announcementMaxAge).Add(-time.Second), - } - b2 := types.Block{ - Transactions: []types.Transaction{newTestTransaction(newTestHostAnnouncement("foo.com:1001"))}, - Timestamp: time.Now().Add(-db.cs.announcementMaxAge).Add(time.Second), - } - - // process b1, expect no announcements - db.cs.processChainApplyUpdateHostDB(&chain.ApplyUpdate{Block: b1}) - if len(db.cs.announcements) != 0 { - t.Fatal("expected 0 announcements") - } - - // process b2, expect 1 announcement - db.cs.processChainApplyUpdateHostDB(&chain.ApplyUpdate{Block: b2}) - if len(db.cs.announcements) != 1 { - t.Fatal("expected 1 announcement") - } else if db.cs.announcements[0].HostAnnouncement.NetAddress != "foo.com:1001" { - t.Fatal("unexpected announcement") - } -} - // addTestHosts adds 'n' hosts to the db and returns their keys. func (s *SQLStore) addTestHosts(n int) (keys []types.PublicKey, err error) { cnt, err := s.contractsCount() @@ -1231,15 +1190,58 @@ func (s *SQLStore) addTestHost(hk types.PublicKey) error { // addCustomTestHost ensures a host with given hostkey and net address exists. func (s *SQLStore) addCustomTestHost(hk types.PublicKey, na string) error { - // NOTE: insert through subscriber to ensure allowlist/blocklist get updated - s.cs.announcements = append(s.cs.announcements, announcement{ - blockHeight: s.cs.tip.Height, - blockID: s.cs.tip.ID, - hk: hk, - timestamp: time.Now().UTC().Round(time.Second), - HostAnnouncement: chain.HostAnnouncement{NetAddress: na}, + // announce the host + host, err := s.announceHost(hk, na) + if err != nil { + return err + } + + // fetch blocklists + allowlist, blocklist, err := getBlocklists(s.db) + if err != nil { + return err + } + + // update host allowlist + var dbAllowlist []dbAllowlistEntry + for _, entry := range allowlist { + if entry.Entry == host.PublicKey { + dbAllowlist = append(dbAllowlist, entry) + } + } + if err := s.db.Model(&host).Association("Allowlist").Replace(&dbAllowlist); err != nil { + return err + } + + // update host blocklist + var dbBlocklist []dbBlocklistEntry + for _, entry := range blocklist { + if entry.blocks(host) { + dbBlocklist = append(dbBlocklist, entry) + } + } + return s.db.Model(&host).Association("Blocklist").Replace(&dbBlocklist) +} + +// announceHost adds a host announcement to the database. +func (s *SQLStore) announceHost(hk types.PublicKey, na string) (host dbHost, err error) { + err = s.db.Transaction(func(tx *gorm.DB) error { + host = dbHost{ + PublicKey: publicKey(hk), + LastAnnouncement: time.Now().UTC().Round(time.Second), + NetAddress: na, + } + if err := s.db.Create(&host).Error; err != nil { + return err + } + return s.db.Create(&dbAnnouncement{ + HostKey: publicKey(hk), + BlockHeight: 42, + BlockID: types.BlockID{1, 2, 3}.String(), + NetAddress: na, + }).Error }) - return s.cs.commit() + return } // hosts returns all hosts in the db. Only used in testing since preloading all @@ -1270,12 +1272,6 @@ func newTestScan(hk types.PublicKey, scanTime time.Time, settings rhpv2.HostSett } } -func newTestPK() (types.PublicKey, types.PrivateKey) { - sk := types.GeneratePrivateKey() - pk := sk.PublicKey() - return pk, sk -} - func newTestAnnouncement(hk types.PublicKey, na string) announcement { return announcement{ blockHeight: 42, @@ -1288,18 +1284,6 @@ func newTestAnnouncement(hk types.PublicKey, na string) announcement { } } -func newTestHostAnnouncement(na string) (chain.HostAnnouncement, types.PrivateKey) { - _, sk := newTestPK() - a := chain.HostAnnouncement{ - NetAddress: na, - } - return a, sk -} - -func newTestTransaction(ha chain.HostAnnouncement, sk types.PrivateKey) types.Transaction { - return types.Transaction{ArbitraryData: [][]byte{ha.ToArbitraryData(sk)}} -} - func newTestHostCheck() api.HostCheck { return api.HostCheck{ diff --git a/stores/metadata.go b/stores/metadata.go index d74245052..458c7f713 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -45,13 +45,6 @@ const ( refreshHealthMaxHealthValidity = 72 * time.Hour ) -var ( - errInvalidNumberOfShards = errors.New("slab has invalid number of shards") - errShardRootChanged = errors.New("shard root changed") - - objectDeleteBatchSizes = []int64{10, 50, 100, 200, 500, 1000, 5000, 10000, 50000, 100000} -) - const ( contractStateInvalid contractState = iota contractStatePending @@ -60,6 +53,13 @@ const ( contractStateFailed ) +var ( + errInvalidNumberOfShards = errors.New("slab has invalid number of shards") + errShardRootChanged = errors.New("shard root changed") + + objectDeleteBatchSizes = []int64{10, 50, 100, 200, 500, 1000, 5000, 10000, 50000, 100000} +) + type ( contractState uint8 @@ -724,7 +724,6 @@ func (s *SQLStore) AddContract(ctx context.Context, c rhpv2.ContractRevision, co return } - s.cs.addKnownContract(types.FileContractID(added.FCID)) return added.convert(), nil } @@ -846,7 +845,6 @@ func (s *SQLStore) AddRenewedContract(ctx context.Context, c rhpv2.ContractRevis return err } - s.cs.addKnownContract(c.ID()) renewed = newContract return nil }); err != nil { @@ -1435,14 +1433,17 @@ func (s *SQLStore) RecordContractSpending(ctx context.Context, records []api.Con } updates["revision_number"] = latestValues[fcid].revision updates["size"] = latestValues[fcid].size - return tx.Model(&contract).Updates(updates).Error + err = tx.Model(&contract).Updates(updates).Error + return err }) if err != nil { return err } } - if err := s.RecordContractMetric(ctx, metrics...); err != nil { - s.logger.Errorw("failed to record contract metrics", zap.Error(err)) + if len(metrics) > 0 { + if err := s.RecordContractMetric(ctx, metrics...); err != nil { + s.logger.Errorw("failed to record contract metrics", zap.Error(err)) + } } return nil } diff --git a/stores/metadata_test.go b/stores/metadata_test.go index b56a3c4bd..c8dd9b5ad 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -259,7 +259,7 @@ func TestSQLContractStore(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) + _, err = ss.announceHost(hk, "address") if err != nil { t.Fatal(err) } @@ -355,7 +355,7 @@ func TestSQLContractStore(t *testing.T) { Size: c.Revision.Filesize, } if !reflect.DeepEqual(returned, expected) { - t.Fatal("contract mismatch") + t.Fatal("contract mismatch", cmp.Diff(returned, expected)) } // Look it up again. @@ -550,11 +550,11 @@ func TestRenewedContract(t *testing.T) { hk, hk2 := hks[0], hks[1] // Add announcements. - err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) + _, err = ss.announceHost(hk, "address") if err != nil { t.Fatal(err) } - err = ss.insertTestAnnouncement(newTestAnnouncement(hk2, "address2")) + _, err = ss.announceHost(hk2, "address2") if err != nil { t.Fatal(err) } @@ -857,6 +857,14 @@ func TestAncestorsContracts(t *testing.T) { t.Fatal("wrong contract", i, contracts[i]) } } + + // Fetch the ancestors with startHeight >= 3. That should return 0 contracts. + contracts, err = ss.AncestorContracts(context.Background(), fcids[len(fcids)-1], 3) + if err != nil { + t.Fatal(err) + } else if len(contracts) != 0 { + t.Fatalf("should have 0 contracts but got %v", len(contracts)) + } } func TestArchiveContracts(t *testing.T) { @@ -2303,7 +2311,7 @@ func TestRecordContractSpending(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) + _, err = ss.announceHost(hk, "address") if err != nil { t.Fatal(err) } diff --git a/stores/metrics.go b/stores/metrics.go index 8351ef43e..6405b2ade 100644 --- a/stores/metrics.go +++ b/stores/metrics.go @@ -641,7 +641,7 @@ func (s *SQLStore) findPeriods(ctx context.Context, table string, dst interface{ Error } -func (s *SQLStore) walletMetrics(ctx context.Context, start time.Time, n uint64, interval time.Duration, opts api.WalletMetricsQueryOpts) (metrics []dbWalletMetric, err error) { +func (s *SQLStore) walletMetrics(ctx context.Context, start time.Time, n uint64, interval time.Duration, _ api.WalletMetricsQueryOpts) (metrics []dbWalletMetric, err error) { err = s.findPeriods(ctx, dbWalletMetric{}.TableName(), &metrics, start, n, interval, gorm.Expr("TRUE")) if err != nil { return nil, fmt.Errorf("failed to fetch wallet metrics: %w", err) diff --git a/stores/migrations/mysql/main/migration_00009_coreutils_wallet.sql b/stores/migrations/mysql/main/migration_00009_coreutils_wallet.sql index 144e9f738..406c6c1ab 100644 --- a/stores/migrations/mysql/main/migration_00009_coreutils_wallet.sql +++ b/stores/migrations/mysql/main/migration_00009_coreutils_wallet.sql @@ -23,7 +23,7 @@ CREATE TABLE `wallet_events` ( KEY `idx_wallet_events_maturity_height` (`maturity_height`), KEY `idx_wallet_events_source` (`source`(191)), -- 191 is the max length for utf8mb4 KEY `idx_wallet_events_timestamp` (`timestamp`), - KEY `idx_wallet_events_height` (`height`) + KEY `idx_wallet_events_block_id_height` (`block_id`, `height`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; -- dbWalletOutput @@ -36,10 +36,7 @@ CREATE TABLE `wallet_outputs` ( `value` longtext, `address` varbinary(32) DEFAULT NULL, `maturity_height` bigint unsigned DEFAULT NULL, - `height` bigint unsigned DEFAULT NULL, - `block_id` varbinary(32) NOT NULL, PRIMARY KEY (`id`), UNIQUE KEY `output_id` (`output_id`), - KEY `idx_wallet_outputs_maturity_height` (`maturity_height`), - KEY `idx_wallet_outputs_height` (`height`) + KEY `idx_wallet_outputs_maturity_height` (`maturity_height`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; diff --git a/stores/migrations/mysql/main/schema.sql b/stores/migrations/mysql/main/schema.sql index 5f574660e..028113856 100644 --- a/stores/migrations/mysql/main/schema.sql +++ b/stores/migrations/mysql/main/schema.sql @@ -520,7 +520,7 @@ CREATE TABLE `wallet_events` ( KEY `idx_wallet_events_maturity_height` (`maturity_height`), KEY `idx_wallet_events_source` (`source`(191)), -- 191 is the max length for utf8mb4 KEY `idx_wallet_events_timestamp` (`timestamp`), - KEY `idx_wallet_events_height` (`height`) + KEY `idx_wallet_events_block_id_height` (`block_id`, `height`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; -- dbWalletOutput @@ -533,12 +533,9 @@ CREATE TABLE `wallet_outputs` ( `value` longtext, `address` varbinary(32) DEFAULT NULL, `maturity_height` bigint unsigned DEFAULT NULL, - `height` bigint unsigned DEFAULT NULL, - `block_id` varbinary(32) NOT NULL, PRIMARY KEY (`id`), UNIQUE KEY `output_id` (`output_id`), - KEY `idx_wallet_outputs_maturity_height` (`maturity_height`), - KEY `idx_wallet_outputs_height` (`height`) + KEY `idx_wallet_outputs_maturity_height` (`maturity_height`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; -- create default bucket diff --git a/stores/migrations/sqlite/main/migration_00009_coreutils_wallet.sql b/stores/migrations/sqlite/main/migration_00009_coreutils_wallet.sql index 7d67af025..c81810c89 100644 --- a/stores/migrations/sqlite/main/migration_00009_coreutils_wallet.sql +++ b/stores/migrations/sqlite/main/migration_00009_coreutils_wallet.sql @@ -11,10 +11,9 @@ CREATE UNIQUE INDEX `idx_wallet_events_event_id` ON `wallet_events`(`event_id`); CREATE INDEX `idx_wallet_events_maturity_height` ON `wallet_events`(`maturity_height`); CREATE INDEX `idx_wallet_events_source` ON `wallet_events`(`source`); CREATE INDEX `idx_wallet_events_timestamp` ON `wallet_events`(`timestamp`); -CREATE INDEX `idx_wallet_events_height` ON `wallet_events`(`height`); +CREATE INDEX `idx_wallet_events_block_id_height` ON `wallet_events`(`block_id`,`height`); -- dbWalletOutput -CREATE TABLE `wallet_outputs` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`output_id` blob NOT NULL,`leaf_index` integer,`merkle_proof` blob NOT NULL,`value` text,`address` blob,`maturity_height` integer,`height` integer, `block_id` blob); +CREATE TABLE `wallet_outputs` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`output_id` blob NOT NULL,`leaf_index` integer,`merkle_proof` blob NOT NULL,`value` text,`address` blob,`maturity_height` integer); CREATE UNIQUE INDEX `idx_wallet_outputs_output_id` ON `wallet_outputs`(`output_id`); -CREATE INDEX `idx_wallet_outputs_maturity_height` ON `wallet_outputs`(`maturity_height`); -CREATE INDEX `idx_wallet_outputs_height` ON `wallet_outputs`(`height`); \ No newline at end of file +CREATE INDEX `idx_wallet_outputs_maturity_height` ON `wallet_outputs`(`maturity_height`); \ No newline at end of file diff --git a/stores/migrations/sqlite/main/schema.sql b/stores/migrations/sqlite/main/schema.sql index d2b52f69f..10c336ee0 100644 --- a/stores/migrations/sqlite/main/schema.sql +++ b/stores/migrations/sqlite/main/schema.sql @@ -211,13 +211,12 @@ CREATE UNIQUE INDEX `idx_wallet_events_event_id` ON `wallet_events`(`event_id`); CREATE INDEX `idx_wallet_events_maturity_height` ON `wallet_events`(`maturity_height`); CREATE INDEX `idx_wallet_events_source` ON `wallet_events`(`source`); CREATE INDEX `idx_wallet_events_timestamp` ON `wallet_events`(`timestamp`); -CREATE INDEX `idx_wallet_events_height` ON `wallet_events`(`height`); +CREATE INDEX `idx_wallet_events_block_id_height` ON `wallet_events`(`block_id`,`height`); -- dbWalletOutput -CREATE TABLE `wallet_outputs` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`output_id` blob NOT NULL,`leaf_index` integer,`merkle_proof` blob NOT NULL,`value` text,`address` blob,`maturity_height` integer,`height` integer, `block_id` blob); +CREATE TABLE `wallet_outputs` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`output_id` blob NOT NULL,`leaf_index` integer,`merkle_proof` blob NOT NULL,`value` text,`address` blob,`maturity_height` integer); CREATE UNIQUE INDEX `idx_wallet_outputs_output_id` ON `wallet_outputs`(`output_id`); CREATE INDEX `idx_wallet_outputs_maturity_height` ON `wallet_outputs`(`maturity_height`); -CREATE INDEX `idx_wallet_outputs_height` ON `wallet_outputs`(`height`); -- create default bucket INSERT INTO buckets (created_at, name) VALUES (CURRENT_TIMESTAMP, 'default'); diff --git a/stores/sql.go b/stores/sql.go index f6c4a2659..292b0822a 100644 --- a/stores/sql.go +++ b/stores/sql.go @@ -10,9 +10,7 @@ import ( "time" "go.sia.tech/core/types" - "go.sia.tech/coreutils/chain" "go.sia.tech/coreutils/syncer" - "go.sia.tech/coreutils/wallet" "go.sia.tech/renterd/alerts" "go.sia.tech/renterd/api" "go.sia.tech/renterd/internal/utils" @@ -37,10 +35,6 @@ var ( exprTRUE = gorm.Expr("TRUE") ) -var ( - _ wallet.SingleAddressStore = (*SQLStore)(nil) -) - var ( errNoSuchTable = errors.New("no such table") errDuplicateEntry = errors.New("Duplicate entry") @@ -62,7 +56,6 @@ type ( PartialSlabDir string Migrate bool AnnouncementMaxAge time.Duration - PersistInterval time.Duration WalletAddress types.Address SlabBufferCompletionThreshold int64 Logger *zap.SugaredLogger @@ -73,11 +66,12 @@ type ( // SQLStore is a helper type for interacting with a SQL-based backend. SQLStore struct { alerts alerts.Alerter - cs *chainSubscriber db *gorm.DB dbMetrics *gorm.DB logger *zap.SugaredLogger + walletAddress types.Address + // ObjectDB related fields slabBufferMgr *SlabBufferManager @@ -99,12 +93,6 @@ type ( lastPrunedAt time.Time closed bool } - - revisionUpdate struct { - height uint64 - number uint64 - size uint64 - } ) // NewEphemeralSQLiteConnection creates a connection to an in-memory SQLite DB. @@ -148,11 +136,6 @@ func NewMySQLConnection(user, password, addr, dbName string) gorm.Dialector { // pass migrate=true for the first instance of SQLHostDB if you connect via the // same Dialector multiple times. func NewSQLStore(cfg Config) (*SQLStore, error) { - // Sanity check announcement max age. - if cfg.AnnouncementMaxAge == 0 { - return nil, errors.New("announcementMaxAge must be non-zero") - } - if err := os.MkdirAll(cfg.PartialSlabDir, 0700); err != nil { return nil, fmt.Errorf("failed to create partial slab dir '%s': %v", cfg.PartialSlabDir, err) } @@ -209,15 +192,16 @@ func NewSQLStore(cfg Config) (*SQLStore, error) { shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) ss := &SQLStore{ - alerts: cfg.Alerts, - db: db, - dbMetrics: dbMetrics, - logger: l, - hasAllowlist: allowlistCnt > 0, - hasBlocklist: blocklistCnt > 0, - settings: make(map[string]string), - slabPruneSigChan: make(chan struct{}, 1), - + alerts: cfg.Alerts, + db: db, + dbMetrics: dbMetrics, + logger: l, + settings: make(map[string]string), + hasAllowlist: allowlistCnt > 0, + hasBlocklist: blocklistCnt > 0, + walletAddress: cfg.WalletAddress, + + slabPruneSigChan: make(chan struct{}, 1), lastPrunedAt: time.Now(), retryTransactionIntervals: cfg.RetryTransactionIntervals, @@ -225,11 +209,6 @@ func NewSQLStore(cfg Config) (*SQLStore, error) { shutdownCtxCancel: shutdownCtxCancel, } - ss.cs, err = newChainSubscriber(ss, cfg.Logger, cfg.RetryTransactionIntervals, cfg.PersistInterval, cfg.WalletAddress, cfg.AnnouncementMaxAge) - if err != nil { - return nil, err - } - ss.slabBufferMgr, err = newSlabBufferManager(ss, cfg.SlabBufferCompletionThreshold, cfg.PartialSlabDir) if err != nil { return nil, err @@ -304,12 +283,7 @@ func tableCount(db *gorm.DB, model interface{}) (cnt int64, err error) { func (s *SQLStore) Close() error { s.shutdownCtxCancel() - err := s.cs.Close() - if err != nil { - return err - } - - err = s.slabBufferMgr.Close() + err := s.slabBufferMgr.Close() if err != nil { return err } @@ -339,9 +313,10 @@ func (s *SQLStore) Close() error { } // ChainIndex returns the last stored chain index. -func (ss *SQLStore) ChainIndex() (types.ChainIndex, error) { +func (ss *SQLStore) ChainIndex(ctx context.Context) (types.ChainIndex, error) { var ci dbConsensusInfo if err := ss.db. + WithContext(ctx). Where(&dbConsensusInfo{Model: Model{ID: consensusInfoID}}). FirstOrCreate(&ci). Error; err != nil { @@ -353,39 +328,30 @@ func (ss *SQLStore) ChainIndex() (types.ChainIndex, error) { }, nil } -// ProcessChainApplyUpdate implements chain.Subscriber. -func (s *SQLStore) ProcessChainApplyUpdate(cau *chain.ApplyUpdate, mayCommit bool) error { - return s.cs.ProcessChainApplyUpdate(cau, mayCommit) -} - -// ProcessChainRevertUpdate implements chain.Subscriber. -func (s *SQLStore) ProcessChainRevertUpdate(cru *chain.RevertUpdate) error { - return s.cs.ProcessChainRevertUpdate(cru) +func (s *SQLStore) retryTransaction(ctx context.Context, fc func(tx *gorm.DB) error) error { + return retryTransaction(ctx, s.db, s.logger, s.retryTransactionIntervals, fc, s.retryAbortFn) } -func (s *SQLStore) retryTransaction(ctx context.Context, fc func(tx *gorm.DB) error) error { - return retryTransaction(ctx, s.db, s.logger, s.retryTransactionIntervals, fc, func(err error) bool { - return err == nil || - utils.IsErr(err, context.Canceled) || - utils.IsErr(err, context.DeadlineExceeded) || - utils.IsErr(err, gorm.ErrRecordNotFound) || - utils.IsErr(err, errInvalidNumberOfShards) || - utils.IsErr(err, errShardRootChanged) || - utils.IsErr(err, api.ErrContractNotFound) || - utils.IsErr(err, api.ErrObjectNotFound) || - utils.IsErr(err, api.ErrObjectCorrupted) || - utils.IsErr(err, api.ErrBucketExists) || - utils.IsErr(err, api.ErrBucketNotFound) || - utils.IsErr(err, api.ErrBucketNotEmpty) || - utils.IsErr(err, api.ErrContractNotFound) || - utils.IsErr(err, api.ErrMultipartUploadNotFound) || - utils.IsErr(err, api.ErrObjectExists) || - utils.IsErr(err, errNoSuchTable) || - utils.IsErr(err, errDuplicateEntry) || - utils.IsErr(err, api.ErrPartNotFound) || - utils.IsErr(err, api.ErrSlabNotFound) || - utils.IsErr(err, syncer.ErrPeerNotFound) - }) +func (s *SQLStore) retryAbortFn(err error) bool { + return err == nil || + utils.IsErr(err, context.Canceled) || + utils.IsErr(err, context.DeadlineExceeded) || + utils.IsErr(err, gorm.ErrRecordNotFound) || + utils.IsErr(err, errInvalidNumberOfShards) || + utils.IsErr(err, errShardRootChanged) || + utils.IsErr(err, api.ErrContractNotFound) || + utils.IsErr(err, api.ErrObjectNotFound) || + utils.IsErr(err, api.ErrObjectCorrupted) || + utils.IsErr(err, api.ErrBucketExists) || + utils.IsErr(err, api.ErrBucketNotFound) || + utils.IsErr(err, api.ErrBucketNotEmpty) || + utils.IsErr(err, api.ErrMultipartUploadNotFound) || + utils.IsErr(err, api.ErrObjectExists) || + utils.IsErr(err, errNoSuchTable) || + utils.IsErr(err, api.ErrPartNotFound) || + utils.IsErr(err, api.ErrSlabNotFound) || + utils.IsErr(err, syncer.ErrPeerNotFound) || + utils.IsErr(err, errDuplicateEntry) } func retryTransaction(ctx context.Context, db *gorm.DB, logger *zap.SugaredLogger, intervals []time.Duration, fn func(tx *gorm.DB) error, abortFn func(error) bool) error { diff --git a/stores/sql_test.go b/stores/sql_test.go index 447a8d34c..d8a1d2970 100644 --- a/stores/sql_test.go +++ b/stores/sql_test.go @@ -28,10 +28,9 @@ import ( ) const ( - testPersistInterval = time.Second - testContractSet = "test" - testMimeType = "application/octet-stream" - testETag = "d34db33f" + testContractSet = "test" + testMimeType = "application/octet-stream" + testETag = "d34db33f" ) var ( @@ -128,7 +127,6 @@ func newTestSQLStore(t *testing.T, cfg testSQLStoreConfig) *testSQLStore { t.Fatal("failed to create db connections", err) } - walletAddrs := types.Address(frand.Entropy256()) alerts := alerts.WithOrigin(alerts.NewManager(), "test") sqlStore, err := NewSQLStore(Config{ Conn: conn, @@ -136,9 +134,6 @@ func newTestSQLStore(t *testing.T, cfg testSQLStoreConfig) *testSQLStore { Alerts: alerts, PartialSlabDir: cfg.dir, Migrate: !cfg.skipMigrate, - AnnouncementMaxAge: time.Hour, - PersistInterval: time.Second, - WalletAddress: walletAddrs, SlabBufferCompletionThreshold: 0, Logger: zap.NewNop().Sugar(), GormLogger: newTestLogger(), diff --git a/stores/subscriber.go b/stores/subscriber.go deleted file mode 100644 index a298767d5..000000000 --- a/stores/subscriber.go +++ /dev/null @@ -1,637 +0,0 @@ -package stores - -import ( - "context" - "errors" - "fmt" - "math" - "sync" - "time" - - "go.sia.tech/core/types" - "go.sia.tech/coreutils/chain" - "go.sia.tech/coreutils/wallet" - "go.sia.tech/renterd/internal/utils" - "go.uber.org/zap" - "gorm.io/gorm" -) - -var ( - _ chain.Subscriber = (*chainSubscriber)(nil) - _ wallet.ApplyTx = (*chainSubscriber)(nil) - _ wallet.RevertTx = (*chainSubscriber)(nil) -) - -type ( - chainSubscriber struct { - announcementMaxAge time.Duration - db *gorm.DB - logger *zap.SugaredLogger - persistInterval time.Duration - retryIntervals []time.Duration - walletAddress types.Address - - // buffered state - mu sync.Mutex - closed bool - lastSave time.Time - tip types.ChainIndex - knownContracts map[types.FileContractID]struct{} - persistTimer *time.Timer - - announcements []announcement - events []eventChange - - contractState map[types.Hash256]contractState - mayCommit bool - outputs map[types.Hash256]outputChange - proofs map[types.Hash256]uint64 - revisions map[types.Hash256]revisionUpdate - } -) - -func newChainSubscriber(sqlStore *SQLStore, logger *zap.SugaredLogger, intvls []time.Duration, persistInterval time.Duration, walletAddress types.Address, ancmtMaxAge time.Duration) (*chainSubscriber, error) { - // load known contracts - var activeFCIDs []fileContractID - if err := sqlStore.db.Model(&dbContract{}). - Select("fcid"). - Find(&activeFCIDs).Error; err != nil { - return nil, err - } - var archivedFCIDs []fileContractID - if err := sqlStore.db.Model(&dbArchivedContract{}). - Select("fcid"). - Find(&archivedFCIDs).Error; err != nil { - return nil, err - } - knownContracts := make(map[types.FileContractID]struct{}) - for _, fcid := range append(activeFCIDs, archivedFCIDs...) { - knownContracts[types.FileContractID(fcid)] = struct{}{} - } - - return &chainSubscriber{ - announcementMaxAge: ancmtMaxAge, - db: sqlStore.db, - logger: logger, - retryIntervals: intvls, - - walletAddress: walletAddress, - lastSave: time.Now(), - persistInterval: persistInterval, - - contractState: make(map[types.Hash256]contractState), - outputs: make(map[types.Hash256]outputChange), - proofs: make(map[types.Hash256]uint64), - revisions: make(map[types.Hash256]revisionUpdate), - knownContracts: knownContracts, - }, nil -} - -func (cs *chainSubscriber) Close() error { - cs.mu.Lock() - defer cs.mu.Unlock() - - cs.closed = true - if cs.persistTimer != nil { - cs.persistTimer.Stop() - select { - case <-cs.persistTimer.C: - default: - } - } - return nil -} - -func (cs *chainSubscriber) ProcessChainApplyUpdate(cau *chain.ApplyUpdate, mayCommit bool) error { - cs.mu.Lock() - defer cs.mu.Unlock() - - // check for shutdown, ideally this never happens since the subscriber is - // unsubscribed first and then closed - if cs.closed { - return errors.New("shutting down") - } - - cs.processChainApplyUpdateHostDB(cau) - cs.processChainApplyUpdateContracts(cau) - if err := cs.processChainApplyUpdateWallet(cau); err != nil { - return err - } - - cs.tip = cau.State.Index - cs.mayCommit = mayCommit - - return cs.tryCommit() -} - -func (cs *chainSubscriber) ProcessChainRevertUpdate(cru *chain.RevertUpdate) error { - cs.mu.Lock() - defer cs.mu.Unlock() - - // check for shutdown, ideally this never happens since the subscriber is - // unsubscribed first and then closed - if cs.closed { - return errors.New("shutting down") - } - - cs.processChainRevertUpdateHostDB(cru) - cs.processChainRevertUpdateContracts(cru) - if err := cs.processChainRevertUpdateWallet(cru); err != nil { - return err - } - - cs.tip = cru.State.Index - cs.mayCommit = true - - return cs.tryCommit() -} - -func (cs *chainSubscriber) Tip() types.ChainIndex { - cs.mu.Lock() - defer cs.mu.Unlock() - return cs.tip -} - -func (cs *chainSubscriber) addKnownContract(id types.FileContractID) { - cs.mu.Lock() - defer cs.mu.Unlock() - cs.knownContracts[id] = struct{}{} -} - -func (cs *chainSubscriber) isKnownContract(id types.FileContractID) bool { - _, ok := cs.knownContracts[id] - return ok -} - -func (cs *chainSubscriber) commit() error { - // Fetch allowlist - var allowlist []dbAllowlistEntry - if err := cs.db. - Model(&dbAllowlistEntry{}). - Find(&allowlist). - Error; err != nil { - cs.logger.Error(fmt.Sprintf("failed to fetch allowlist, err: %v", err)) - } - - // Fetch blocklist - var blocklist []dbBlocklistEntry - if err := cs.db. - Model(&dbBlocklistEntry{}). - Find(&blocklist). - Error; err != nil { - cs.logger.Error(fmt.Sprintf("failed to fetch blocklist, err: %v", err)) - } - - err := cs.retryTransaction(func(tx *gorm.DB) (err error) { - if len(cs.announcements) > 0 { - if err = insertAnnouncements(tx, cs.announcements); err != nil { - return fmt.Errorf("%w; failed to insert %d announcements", err, len(cs.announcements)) - } - if len(allowlist)+len(blocklist) > 0 { - updated := make(map[types.PublicKey]struct{}) - for _, ann := range cs.announcements { - if _, seen := updated[ann.hk]; !seen { - updated[ann.hk] = struct{}{} - if err := updateBlocklist(tx, ann.hk, allowlist, blocklist); err != nil { - cs.logger.Error(fmt.Sprintf("failed to update blocklist, err: %v", err)) - } - } - } - } - } - for fcid, rev := range cs.revisions { - if err := applyRevisionUpdate(tx, types.FileContractID(fcid), rev); err != nil { - return fmt.Errorf("%w; failed to update revision number and height", err) - } - } - for fcid, proofHeight := range cs.proofs { - if err := updateProofHeight(tx, types.FileContractID(fcid), proofHeight); err != nil { - return fmt.Errorf("%w; failed to update proof height", err) - } - } - for _, oc := range cs.outputs { - if oc.addition { - err = applyUnappliedOutputAdditions(tx, oc.se) - } else { - err = applyUnappliedOutputRemovals(tx, oc.se.OutputID) - } - if err != nil { - return fmt.Errorf("%w; failed to apply unapplied output change", err) - } - } - for _, tc := range cs.events { - if tc.addition { - err = applyUnappliedEventAdditions(tx, tc.event) - } else { - err = applyUnappliedEventRemovals(tx, tc.event.EventID) - } - if err != nil { - return fmt.Errorf("%w; failed to apply unapplied event change", err) - } - } - for fcid, cs := range cs.contractState { - if err := updateContractState(tx, types.FileContractID(fcid), cs); err != nil { - return fmt.Errorf("%w; failed to update chain state", err) - } - } - if err := markFailedContracts(tx, cs.tip.Height); err != nil { - return err - } - return updateChainIndex(tx, cs.tip) - }) - if err != nil { - return fmt.Errorf("%w; failed to apply updates", err) - } - - cs.announcements = nil - cs.contractState = make(map[types.Hash256]contractState) - cs.mayCommit = false - cs.outputs = make(map[types.Hash256]outputChange) - cs.proofs = make(map[types.Hash256]uint64) - cs.revisions = make(map[types.Hash256]revisionUpdate) - cs.events = nil - cs.lastSave = time.Now() - return nil -} - -// shouldCommit returns whether the subscriber should commit its buffered state. -func (cs *chainSubscriber) shouldCommit() bool { - return cs.mayCommit && (time.Since(cs.lastSave) > cs.persistInterval || - len(cs.announcements) > 0 || - len(cs.revisions) > 0 || - len(cs.proofs) > 0 || - len(cs.outputs) > 0 || - len(cs.events) > 0 || - len(cs.contractState) > 0) -} - -func (cs *chainSubscriber) tryCommit() error { - // commit if we can/should - if !cs.shouldCommit() { - return nil - } else if err := cs.commit(); err != nil { - cs.logger.Errorw("failed to commit chain update", zap.Error(err)) - return err - } - - // force a persist if no block has been received for some time - if cs.persistTimer != nil { - cs.persistTimer.Stop() - select { - case <-cs.persistTimer.C: - default: - } - } - cs.persistTimer = time.AfterFunc(10*time.Second, func() { - cs.mu.Lock() - defer cs.mu.Unlock() - if cs.closed { - return - } else if err := cs.commit(); err != nil { - cs.logger.Errorw("failed to commit delayed chain update", zap.Error(err)) - } - }) - return nil -} - -func (cs *chainSubscriber) processChainApplyUpdateHostDB(cau *chain.ApplyUpdate) { - b := cau.Block - if time.Since(b.Timestamp) > cs.announcementMaxAge { - return // ignore old announcements - } - chain.ForEachHostAnnouncement(b, func(hk types.PublicKey, ha chain.HostAnnouncement) { - if ha.NetAddress == "" { - return // ignore - } - cs.announcements = append(cs.announcements, announcement{ - blockHeight: cau.State.Index.Height, - blockID: b.ID(), - hk: hk, - timestamp: b.Timestamp, - HostAnnouncement: ha, - }) - }) -} - -func (cs *chainSubscriber) processChainRevertUpdateHostDB(cru *chain.RevertUpdate) { - // nothing to do, we are not unannouncing hosts -} - -func (cs *chainSubscriber) processChainApplyUpdateContracts(cau *chain.ApplyUpdate) { - type revision struct { - revisionNumber uint64 - fileSize uint64 - } - - // generic helper for processing v1 and v2 contracts - processContract := func(fcid types.Hash256, rev revision, resolved, valid bool) { - // ignore irrelevant contracts - if !cs.isKnownContract(types.FileContractID(fcid)) { - return - } - - // 'pending' -> 'active' - if cs.contractState[fcid] < contractStateActive { - cs.contractState[fcid] = contractStateActive // 'pending' -> 'active' - cs.logger.Infow("contract state changed: pending -> active", - "fcid", fcid, - "reason", "contract confirmed") - } - - // renewed: 'active' -> 'complete' - if rev.revisionNumber == types.MaxRevisionNumber && rev.fileSize == 0 { - cs.contractState[fcid] = contractStateComplete // renewed: 'active' -> 'complete' - cs.logger.Infow("contract state changed: active -> complete", - "fcid", fcid, - "reason", "final revision confirmed") - } - cs.revisions[fcid] = revisionUpdate{ - height: cau.State.Index.Height, - number: rev.revisionNumber, - size: rev.fileSize, - } - - // storage proof: 'active' -> 'complete/failed' - if resolved { - cs.proofs[fcid] = cau.State.Index.Height - if valid { - cs.contractState[fcid] = contractStateComplete - cs.logger.Infow("contract state changed: active -> complete", - "fcid", fcid, - "reason", "storage proof valid") - } else { - cs.contractState[fcid] = contractStateFailed - cs.logger.Infow("contract state changed: active -> failed", - "fcid", fcid, - "reason", "storage proof missed") - } - } - } - - // v1 contracts - cau.ForEachFileContractElement(func(fce types.FileContractElement, rev *types.FileContractElement, resolved, valid bool) { - var r revision - if rev != nil { - r.revisionNumber = rev.FileContract.RevisionNumber - r.fileSize = rev.FileContract.Filesize - } else { - r.revisionNumber = fce.FileContract.RevisionNumber - r.fileSize = fce.FileContract.Filesize - } - processContract(fce.ID, r, resolved, valid) - }) - - // v2 contracts - cau.ForEachV2FileContractElement(func(fce types.V2FileContractElement, rev *types.V2FileContractElement, res types.V2FileContractResolutionType) { - var r revision - if rev != nil { - r.revisionNumber = rev.V2FileContract.RevisionNumber - r.fileSize = rev.V2FileContract.Filesize - } else { - r.revisionNumber = fce.V2FileContract.RevisionNumber - r.fileSize = fce.V2FileContract.Filesize - } - - var valid bool - var resolved bool - if res != nil { - switch res.(type) { - case *types.V2FileContractFinalization: - valid = true - case *types.V2FileContractRenewal: - valid = true - case *types.V2StorageProof: - valid = true - case *types.V2FileContractExpiration: - valid = fce.V2FileContract.Filesize == 0 - } - - resolved = true - } - processContract(fce.ID, r, resolved, valid) - }) -} - -func (cs *chainSubscriber) processChainRevertUpdateContracts(cru *chain.RevertUpdate) { - type revision struct { - revisionNumber uint64 - fileSize uint64 - } - - // generic helper for processing v1 and v2 contracts - processContract := func(fcid types.Hash256, prevRev revision, rev *revision, resolved, valid bool) { - // ignore irrelevant contracts - if !cs.isKnownContract(types.FileContractID(fcid)) { - return - } - - // 'active' -> 'pending' - if rev == nil { - cs.contractState[fcid] = contractStatePending - } - - // reverted renewal: 'complete' -> 'active' - if rev != nil { - cs.revisions[fcid] = revisionUpdate{ - height: cru.State.Index.Height, - number: prevRev.revisionNumber, - size: prevRev.fileSize, - } - if rev.revisionNumber == math.MaxUint64 && rev.fileSize == 0 { - cs.contractState[fcid] = contractStateActive - cs.logger.Infow("contract state changed: complete -> active", - "fcid", fcid, - "reason", "final revision reverted") - } - } - - // reverted storage proof: 'complete/failed' -> 'active' - if resolved { - cs.contractState[fcid] = contractStateActive // revert from 'complete' to 'active' - if valid { - cs.logger.Infow("contract state changed: complete -> active", - "fcid", fcid, - "reason", "storage proof reverted") - } else { - cs.logger.Infow("contract state changed: failed -> active", - "fcid", fcid, - "reason", "storage proof reverted") - } - } - } - - // v1 contracts - cru.ForEachFileContractElement(func(fce types.FileContractElement, rev *types.FileContractElement, resolved, valid bool) { - var r *revision - if rev != nil { - r = &revision{ - revisionNumber: rev.FileContract.RevisionNumber, - fileSize: rev.FileContract.Filesize, - } - } - prevRev := revision{ - revisionNumber: fce.FileContract.RevisionNumber, - fileSize: fce.FileContract.Filesize, - } - processContract(fce.ID, prevRev, r, resolved, valid) - }) - - // v2 contracts - cru.ForEachV2FileContractElement(func(fce types.V2FileContractElement, rev *types.V2FileContractElement, res types.V2FileContractResolutionType) { - var r *revision - if rev != nil { - r = &revision{ - revisionNumber: rev.V2FileContract.RevisionNumber, - fileSize: rev.V2FileContract.Filesize, - } - } - resolved := res != nil - valid := false - if res != nil { - switch res.(type) { - case *types.V2FileContractFinalization: - valid = true - case *types.V2FileContractRenewal: - valid = true - case *types.V2StorageProof: - valid = true - case *types.V2FileContractExpiration: - valid = fce.V2FileContract.Filesize == 0 - } - } - prevRev := revision{ - revisionNumber: fce.V2FileContract.RevisionNumber, - fileSize: fce.V2FileContract.Filesize, - } - processContract(fce.ID, prevRev, r, resolved, valid) - }) -} - -func (cs *chainSubscriber) processChainApplyUpdateWallet(cau *chain.ApplyUpdate) error { - return wallet.ApplyChainUpdates(cs, cs.walletAddress, []*chain.ApplyUpdate{cau}) -} - -func (cs *chainSubscriber) processChainRevertUpdateWallet(cru *chain.RevertUpdate) error { - return wallet.RevertChainUpdate(cs, cs.walletAddress, cru) -} - -func (cs *chainSubscriber) retryTransaction(fc func(tx *gorm.DB) error) error { - return retryTransaction(context.Background(), cs.db, cs.logger, cs.retryIntervals, fc, func(err error) bool { - return err == nil || - utils.IsErr(err, gorm.ErrRecordNotFound) || - utils.IsErr(err, context.Canceled) || - utils.IsErr(err, errNoSuchTable) || - utils.IsErr(err, errDuplicateEntry) - }) -} - -// AddEvents is called with all relevant events added in the update. -func (cs *chainSubscriber) AddEvents(events []wallet.Event) error { - for _, event := range events { - cs.events = append(cs.events, eventChange{ - addition: true, - event: dbWalletEvent{ - EventID: hash256(event.ID), - Inflow: currency(event.Inflow), - Outflow: currency(event.Outflow), - Transaction: event.Transaction, - MaturityHeight: event.MaturityHeight, - Source: string(event.Source), - Timestamp: event.Timestamp.Unix(), - Height: event.Index.Height, - BlockID: hash256(event.Index.ID), - }, - }) - } - return nil -} - -// AddSiacoinElements is called with all new siacoin elements in the -// update. Ephemeral siacoin elements are not included. -func (cs *chainSubscriber) AddSiacoinElements(elements []wallet.SiacoinElement) error { - for _, el := range elements { - if _, ok := cs.outputs[el.ID]; ok { - return fmt.Errorf("output %q already exists", el.ID) - } - cs.outputs[el.ID] = outputChange{ - addition: true, - se: dbWalletOutput{ - OutputID: hash256(el.ID), - LeafIndex: el.StateElement.LeafIndex, - MerkleProof: merkleProof{proof: el.StateElement.MerkleProof}, - Value: currency(el.SiacoinOutput.Value), - Address: hash256(el.SiacoinOutput.Address), - MaturityHeight: el.MaturityHeight, - Height: el.Index.Height, - BlockID: hash256(el.Index.ID), - }, - } - } - - return nil -} - -// RemoveSiacoinElements is called with all siacoin elements that were -// spent in the update. -func (cs *chainSubscriber) RemoveSiacoinElements(ids []types.SiacoinOutputID) error { - for _, id := range ids { - if _, ok := cs.outputs[types.Hash256(id)]; ok { - return fmt.Errorf("output %q not found", id) - } - - cs.outputs[types.Hash256(id)] = outputChange{ - addition: false, - se: dbWalletOutput{ - OutputID: hash256(id), - }, - } - } - return nil -} - -// WalletStateElements returns all state elements in the database. It is used -// to update the proofs of all state elements affected by the update. -func (cs *chainSubscriber) WalletStateElements() (elements []types.StateElement, _ error) { - for id, el := range cs.outputs { - elements = append(elements, types.StateElement{ - ID: id, - LeafIndex: el.se.LeafIndex, - MerkleProof: el.se.MerkleProof.proof, - }) - } - return -} - -// UpdateStateElements updates the proofs of all state elements affected by the -// update. -func (cs *chainSubscriber) UpdateStateElements(elements []types.StateElement) error { - for _, se := range elements { - curr := cs.outputs[se.ID] - curr.se.MerkleProof = merkleProof{proof: se.MerkleProof} - curr.se.LeafIndex = se.LeafIndex - cs.outputs[se.ID] = curr - } - return nil -} - -// RevertIndex is called with the chain index that is being reverted. Any events -// and siacoin elements that were created by the index should be removed. -func (cs *chainSubscriber) RevertIndex(index types.ChainIndex) error { - // remove any events that were added in the reverted block - filtered := cs.events[:0] - for i := range cs.events { - if cs.events[i].event.Index() != index { - filtered = append(filtered, cs.events[i]) - } - } - cs.events = filtered - - // remove any siacoin elements that were added in the reverted block - for id, el := range cs.outputs { - if el.se.Index() == index { - delete(cs.outputs, id) - } - } - - return nil -} diff --git a/stores/wallet.go b/stores/wallet.go index b58d48ba8..d7b25f8f3 100644 --- a/stores/wallet.go +++ b/stores/wallet.go @@ -1,13 +1,17 @@ package stores import ( + "errors" "math" "time" "go.sia.tech/core/types" "go.sia.tech/coreutils/wallet" "gorm.io/gorm" - "gorm.io/gorm/clause" +) + +var ( + _ wallet.SingleAddressStore = (*SQLStore)(nil) ) type ( @@ -38,20 +42,6 @@ type ( Value currency Address hash256 `gorm:"size:32"` MaturityHeight uint64 `gorm:"index:idx_wallet_outputs_maturity_height"` - - // chain index - Height uint64 `gorm:"index:idx_wallet_outputs_height"` - BlockID hash256 `gorm:"size:32"` - } - - outputChange struct { - addition bool - se dbWalletOutput - } - - eventChange struct { - addition bool - event dbWalletEvent } ) @@ -72,46 +62,43 @@ func (e dbWalletEvent) Index() types.ChainIndex { } } -func (se dbWalletOutput) Index() types.ChainIndex { - return types.ChainIndex{ - Height: se.Height, - ID: types.BlockID(se.BlockID), - } -} - // Tip returns the consensus change ID and block height of the last wallet // change. func (s *SQLStore) Tip() (types.ChainIndex, error) { - return s.cs.Tip(), nil + var cs dbConsensusInfo + if err := s.db. + Model(&dbConsensusInfo{}). + First(&cs).Error; errors.Is(err, gorm.ErrRecordNotFound) { + return types.ChainIndex{}, nil + } else if err != nil { + return types.ChainIndex{}, err + } + return types.ChainIndex{ + Height: cs.Height, + ID: types.BlockID(cs.BlockID), + }, nil } // UnspentSiacoinElements returns a list of all unspent siacoin outputs -func (s *SQLStore) UnspentSiacoinElements() ([]wallet.SiacoinElement, error) { +func (s *SQLStore) UnspentSiacoinElements() ([]types.SiacoinElement, error) { var dbElems []dbWalletOutput if err := s.db.Find(&dbElems).Error; err != nil { return nil, err } - elements := make([]wallet.SiacoinElement, len(dbElems)) + elements := make([]types.SiacoinElement, len(dbElems)) for i, el := range dbElems { - elements[i] = wallet.SiacoinElement{ - SiacoinElement: types.SiacoinElement{ - StateElement: types.StateElement{ - ID: types.Hash256(el.OutputID), - LeafIndex: el.LeafIndex, - MerkleProof: el.MerkleProof.proof, - }, - MaturityHeight: el.MaturityHeight, - SiacoinOutput: types.SiacoinOutput{ - Address: types.Address(el.Address), - Value: types.Currency(el.Value), - }, + elements[i] = types.SiacoinElement{ + StateElement: types.StateElement{ + ID: types.Hash256(el.OutputID), + LeafIndex: el.LeafIndex, + MerkleProof: el.MerkleProof.proof, }, - Index: types.ChainIndex{ - Height: el.Height, - ID: types.BlockID(el.BlockID), + MaturityHeight: el.MaturityHeight, + SiacoinOutput: types.SiacoinOutput{ + Address: types.Address(el.Address), + Value: types.Currency(el.Value), }, - // TODO: Index missing } } return elements, nil @@ -159,31 +146,3 @@ func (s *SQLStore) WalletEventCount() (uint64, error) { } return uint64(count), nil } - -func applyUnappliedOutputAdditions(tx *gorm.DB, sco dbWalletOutput) error { - return tx. - Clauses(clause.OnConflict{ - DoNothing: true, - Columns: []clause.Column{{Name: "output_id"}}, - }).Create(&sco).Error -} - -func applyUnappliedOutputRemovals(tx *gorm.DB, oid hash256) error { - return tx.Where("output_id", oid). - Delete(&dbWalletOutput{}). - Error -} - -func applyUnappliedEventAdditions(tx *gorm.DB, event dbWalletEvent) error { - return tx. - Clauses(clause.OnConflict{ - DoNothing: true, - Columns: []clause.Column{{Name: "event_id"}}, - }).Create(&event).Error -} - -func applyUnappliedEventRemovals(tx *gorm.DB, eventID hash256) error { - return tx.Where("event_id", eventID). - Delete(&dbWalletEvent{}). - Error -} diff --git a/wallet/wallet.go b/wallet/wallet.go deleted file mode 100644 index 6c641ed42..000000000 --- a/wallet/wallet.go +++ /dev/null @@ -1,650 +0,0 @@ -package wallet - -import ( - "bytes" - "context" - "errors" - "fmt" - "sort" - "sync" - "time" - - "gitlab.com/NebulousLabs/encoding" - "go.sia.tech/core/consensus" - "go.sia.tech/core/types" - "go.sia.tech/renterd/api" - "go.sia.tech/siad/modules" - "go.uber.org/zap" -) - -const ( - // BytesPerInput is the encoded size of a SiacoinInput and corresponding - // TransactionSignature, assuming standard UnlockConditions. - BytesPerInput = 241 - - // redistributeBatchSize is the number of outputs to redistribute per txn to - // avoid creating a txn that is too large. - redistributeBatchSize = 10 - - // transactionDefragThreshold is the number of utxos at which the wallet - // will attempt to defrag itself by including small utxos in transactions. - transactionDefragThreshold = 30 - // maxInputsForDefrag is the maximum number of inputs a transaction can - // have before the wallet will stop adding inputs - maxInputsForDefrag = 30 - // maxDefragUTXOs is the maximum number of utxos that will be added to a - // transaction when defragging - maxDefragUTXOs = 10 -) - -// ErrInsufficientBalance is returned when there aren't enough unused outputs to -// cover the requested amount. -var ErrInsufficientBalance = errors.New("insufficient balance") - -// StandardUnlockConditions returns the standard unlock conditions for a single -// Ed25519 key. -func StandardUnlockConditions(pk types.PublicKey) types.UnlockConditions { - return types.UnlockConditions{ - PublicKeys: []types.UnlockKey{{ - Algorithm: types.SpecifierEd25519, - Key: pk[:], - }}, - SignaturesRequired: 1, - } -} - -// StandardAddress returns the standard address for an Ed25519 key. -func StandardAddress(pk types.PublicKey) types.Address { - return StandardUnlockConditions(pk).UnlockHash() -} - -// StandardTransactionSignature returns the standard signature object for a -// siacoin or siafund input. -func StandardTransactionSignature(id types.Hash256) types.TransactionSignature { - return types.TransactionSignature{ - ParentID: id, - CoveredFields: types.CoveredFields{WholeTransaction: true}, - PublicKeyIndex: 0, - } -} - -// ExplicitCoveredFields returns a CoveredFields that covers all elements -// present in txn. -func ExplicitCoveredFields(txn types.Transaction) (cf types.CoveredFields) { - for i := range txn.SiacoinInputs { - cf.SiacoinInputs = append(cf.SiacoinInputs, uint64(i)) - } - for i := range txn.SiacoinOutputs { - cf.SiacoinOutputs = append(cf.SiacoinOutputs, uint64(i)) - } - for i := range txn.FileContracts { - cf.FileContracts = append(cf.FileContracts, uint64(i)) - } - for i := range txn.FileContractRevisions { - cf.FileContractRevisions = append(cf.FileContractRevisions, uint64(i)) - } - for i := range txn.StorageProofs { - cf.StorageProofs = append(cf.StorageProofs, uint64(i)) - } - for i := range txn.SiafundInputs { - cf.SiafundInputs = append(cf.SiafundInputs, uint64(i)) - } - for i := range txn.SiafundOutputs { - cf.SiafundOutputs = append(cf.SiafundOutputs, uint64(i)) - } - for i := range txn.MinerFees { - cf.MinerFees = append(cf.MinerFees, uint64(i)) - } - for i := range txn.ArbitraryData { - cf.ArbitraryData = append(cf.ArbitraryData, uint64(i)) - } - for i := range txn.Signatures { - cf.Signatures = append(cf.Signatures, uint64(i)) - } - return -} - -// A SiacoinElement is a SiacoinOutput along with its ID. -type SiacoinElement struct { - types.SiacoinOutput - ID types.Hash256 `json:"id"` - MaturityHeight uint64 `json:"maturityHeight"` -} - -// A Transaction is an on-chain transaction relevant to a particular wallet, -// paired with useful metadata. -type Transaction struct { - Raw types.Transaction `json:"raw,omitempty"` - Index types.ChainIndex `json:"index"` - ID types.TransactionID `json:"id"` - Inflow types.Currency `json:"inflow"` - Outflow types.Currency `json:"outflow"` - Timestamp time.Time `json:"timestamp"` -} - -// A SingleAddressStore stores the state of a single-address wallet. -// Implementations are assumed to be thread safe. -type SingleAddressStore interface { - Height() uint64 - UnspentSiacoinElements(matured bool) ([]SiacoinElement, error) - Transactions(before, since time.Time, offset, limit int) ([]Transaction, error) - RecordWalletMetric(ctx context.Context, metrics ...api.WalletMetric) error -} - -// A TransactionPool contains transactions that have not yet been included in a -// block. -type TransactionPool interface { - ContainsElement(id types.Hash256) bool -} - -// A SingleAddressWallet is a hot wallet that manages the outputs controlled by -// a single address. -type SingleAddressWallet struct { - log *zap.SugaredLogger - priv types.PrivateKey - addr types.Address - store SingleAddressStore - usedUTXOExpiry time.Duration - - // for building transactions - mu sync.Mutex - lastUsed map[types.Hash256]time.Time - // tpoolTxns maps a transaction set ID to the transactions in that set - tpoolTxns map[types.Hash256][]Transaction - // tpoolUtxos maps a siacoin output ID to its corresponding siacoin - // element. It is used to track siacoin outputs that are currently in - // the transaction pool. - tpoolUtxos map[types.SiacoinOutputID]SiacoinElement - // tpoolSpent is a set of siacoin output IDs that are currently in the - // transaction pool. - tpoolSpent map[types.SiacoinOutputID]bool -} - -// PrivateKey returns the private key of the wallet. -func (w *SingleAddressWallet) PrivateKey() types.PrivateKey { - return w.priv -} - -// Address returns the address of the wallet. -func (w *SingleAddressWallet) Address() types.Address { - return w.addr -} - -// Balance returns the balance of the wallet. -func (w *SingleAddressWallet) Balance() (spendable, confirmed, unconfirmed types.Currency, _ error) { - sces, err := w.store.UnspentSiacoinElements(true) - if err != nil { - return types.Currency{}, types.Currency{}, types.Currency{}, err - } - w.mu.Lock() - defer w.mu.Unlock() - for _, sce := range sces { - if !w.isOutputUsed(sce.ID) { - spendable = spendable.Add(sce.Value) - } - confirmed = confirmed.Add(sce.Value) - } - for _, sco := range w.tpoolUtxos { - if !w.isOutputUsed(sco.ID) { - unconfirmed = unconfirmed.Add(sco.Value) - } - } - return -} - -func (w *SingleAddressWallet) Height() uint64 { - return w.store.Height() -} - -// UnspentOutputs returns the set of unspent Siacoin outputs controlled by the -// wallet. -func (w *SingleAddressWallet) UnspentOutputs() ([]SiacoinElement, error) { - sces, err := w.store.UnspentSiacoinElements(false) - if err != nil { - return nil, err - } - w.mu.Lock() - defer w.mu.Unlock() - filtered := sces[:0] - for _, sce := range sces { - if !w.isOutputUsed(sce.ID) { - filtered = append(filtered, sce) - } - } - return filtered, nil -} - -// Transactions returns up to max transactions relevant to the wallet that have -// a timestamp later than since. -func (w *SingleAddressWallet) Transactions(before, since time.Time, offset, limit int) ([]Transaction, error) { - return w.store.Transactions(before, since, offset, limit) -} - -// FundTransaction adds siacoin inputs worth at least the requested amount to -// the provided transaction. A change output is also added, if necessary. The -// inputs will not be available to future calls to FundTransaction unless -// ReleaseInputs is called or enough time has passed. -func (w *SingleAddressWallet) FundTransaction(cs consensus.State, txn *types.Transaction, amount types.Currency, useUnconfirmedTxns bool) ([]types.Hash256, error) { - if amount.IsZero() { - return nil, nil - } - w.mu.Lock() - defer w.mu.Unlock() - - // fetch all unspent siacoin elements - utxos, err := w.store.UnspentSiacoinElements(false) - if err != nil { - return nil, err - } - - // desc sort - sort.Slice(utxos, func(i, j int) bool { - return utxos[i].Value.Cmp(utxos[j].Value) > 0 - }) - - // add all unconfirmed outputs to the end of the slice as a last resort - if useUnconfirmedTxns { - var tpoolUtxos []SiacoinElement - for _, sco := range w.tpoolUtxos { - tpoolUtxos = append(tpoolUtxos, sco) - } - // desc sort - sort.Slice(tpoolUtxos, func(i, j int) bool { - return tpoolUtxos[i].Value.Cmp(tpoolUtxos[j].Value) > 0 - }) - utxos = append(utxos, tpoolUtxos...) - } - - // remove locked and spent outputs - usableUTXOs := utxos[:0] - for _, sce := range utxos { - if w.isOutputUsed(sce.ID) { - continue - } - usableUTXOs = append(usableUTXOs, sce) - } - - // fund the transaction using the largest utxos first - var selected []SiacoinElement - var inputSum types.Currency - for i, sce := range usableUTXOs { - if inputSum.Cmp(amount) >= 0 { - usableUTXOs = usableUTXOs[i:] - break - } - selected = append(selected, sce) - inputSum = inputSum.Add(sce.Value) - } - - // if the transaction can't be funded, return an error - if inputSum.Cmp(amount) < 0 { - return nil, fmt.Errorf("%w: inputSum: %v, amount: %v", ErrInsufficientBalance, inputSum.String(), amount.String()) - } - - // check if remaining utxos should be defragged - txnInputs := len(txn.SiacoinInputs) + len(selected) - if len(usableUTXOs) > transactionDefragThreshold && txnInputs < maxInputsForDefrag { - // add the smallest utxos to the transaction - defraggable := usableUTXOs - if len(defraggable) > maxDefragUTXOs { - defraggable = defraggable[len(defraggable)-maxDefragUTXOs:] - } - for i := len(defraggable) - 1; i >= 0; i-- { - if txnInputs >= maxInputsForDefrag { - break - } - - sce := defraggable[i] - selected = append(selected, sce) - inputSum = inputSum.Add(sce.Value) - txnInputs++ - } - } - - // add a change output if necessary - if inputSum.Cmp(amount) > 0 { - txn.SiacoinOutputs = append(txn.SiacoinOutputs, types.SiacoinOutput{ - Value: inputSum.Sub(amount), - Address: w.addr, - }) - } - - toSign := make([]types.Hash256, len(selected)) - for i, sce := range selected { - txn.SiacoinInputs = append(txn.SiacoinInputs, types.SiacoinInput{ - ParentID: types.SiacoinOutputID(sce.ID), - UnlockConditions: types.StandardUnlockConditions(w.priv.PublicKey()), - }) - toSign[i] = types.Hash256(sce.ID) - w.lastUsed[sce.ID] = time.Now() - } - - return toSign, nil -} - -// ReleaseInputs is a helper function that releases the inputs of txn for use in -// other transactions. It should only be called on transactions that are invalid -// or will never be broadcast. -func (w *SingleAddressWallet) ReleaseInputs(txns ...types.Transaction) { - w.mu.Lock() - defer w.mu.Unlock() - w.releaseInputs(txns...) -} - -func (w *SingleAddressWallet) releaseInputs(txns ...types.Transaction) { - for _, txn := range txns { - for _, in := range txn.SiacoinInputs { - delete(w.lastUsed, types.Hash256(in.ParentID)) - } - } -} - -// SignTransaction adds a signature to each of the specified inputs. -func (w *SingleAddressWallet) SignTransaction(cs consensus.State, txn *types.Transaction, toSign []types.Hash256, cf types.CoveredFields) error { - for _, id := range toSign { - ts := types.TransactionSignature{ - ParentID: id, - CoveredFields: cf, - PublicKeyIndex: 0, - } - var h types.Hash256 - if cf.WholeTransaction { - h = cs.WholeSigHash(*txn, ts.ParentID, ts.PublicKeyIndex, ts.Timelock, cf.Signatures) - } else { - h = cs.PartialSigHash(*txn, cf) - } - sig := w.priv.SignHash(h) - ts.Signature = sig[:] - txn.Signatures = append(txn.Signatures, ts) - } - return nil -} - -// Redistribute returns a transaction that redistributes money in the wallet by -// selecting a minimal set of inputs to cover the creation of the requested -// outputs. It also returns a list of output IDs that need to be signed. -func (w *SingleAddressWallet) Redistribute(cs consensus.State, outputs int, amount, feePerByte types.Currency, pool []types.Transaction) ([]types.Transaction, []types.Hash256, error) { - w.mu.Lock() - defer w.mu.Unlock() - - // build map of inputs currently in the tx pool - inPool := make(map[types.Hash256]bool) - for _, ptxn := range pool { - for _, in := range ptxn.SiacoinInputs { - inPool[types.Hash256(in.ParentID)] = true - } - } - - // fetch unspent transaction outputs - utxos, err := w.store.UnspentSiacoinElements(false) - if err != nil { - return nil, nil, err - } - - // check whether a redistribution is necessary, adjust number of desired - // outputs accordingly - for _, sce := range utxos { - inUse := w.isOutputUsed(sce.ID) || inPool[sce.ID] - matured := cs.Index.Height >= sce.MaturityHeight - sameValue := sce.Value.Equals(amount) - if !inUse && matured && sameValue { - outputs-- - } - } - if outputs <= 0 { - return nil, nil, nil - } - - // desc sort - sort.Slice(utxos, func(i, j int) bool { - return utxos[i].Value.Cmp(utxos[j].Value) > 0 - }) - - // prepare all outputs - var txns []types.Transaction - var toSign []types.Hash256 - - for outputs > 0 { - var txn types.Transaction - for i := 0; i < outputs && i < redistributeBatchSize; i++ { - txn.SiacoinOutputs = append(txn.SiacoinOutputs, types.SiacoinOutput{ - Value: amount, - Address: w.Address(), - }) - } - outputs -= len(txn.SiacoinOutputs) - - // estimate the fees - outputFees := feePerByte.Mul64(uint64(len(encoding.Marshal(txn.SiacoinOutputs)))) - feePerInput := feePerByte.Mul64(BytesPerInput) - - // collect outputs that cover the total amount - var inputs []SiacoinElement - want := amount.Mul64(uint64(len(txn.SiacoinOutputs))) - var amtInUse, amtSameValue, amtNotMatured types.Currency - for _, sce := range utxos { - inUse := w.isOutputUsed(sce.ID) || inPool[sce.ID] - matured := cs.Index.Height >= sce.MaturityHeight - sameValue := sce.Value.Equals(amount) - if inUse { - amtInUse = amtInUse.Add(sce.Value) - continue - } else if sameValue { - amtSameValue = amtSameValue.Add(sce.Value) - continue - } else if !matured { - amtNotMatured = amtNotMatured.Add(sce.Value) - continue - } - - inputs = append(inputs, sce) - fee := feePerInput.Mul64(uint64(len(inputs))).Add(outputFees) - if SumOutputs(inputs).Cmp(want.Add(fee)) > 0 { - break - } - } - - // not enough outputs found - fee := feePerInput.Mul64(uint64(len(inputs))).Add(outputFees) - if sumOut := SumOutputs(inputs); sumOut.Cmp(want.Add(fee)) < 0 { - // in case of an error we need to free all inputs - w.releaseInputs(txns...) - return nil, nil, fmt.Errorf("%w: inputs %v < needed %v + txnFee %v (usable: %v, inUse: %v, sameValue: %v, notMatured: %v)", - ErrInsufficientBalance, sumOut.String(), want.String(), fee.String(), sumOut.String(), amtInUse.String(), amtSameValue.String(), amtNotMatured.String()) - } - - // set the miner fee - txn.MinerFees = []types.Currency{fee} - - // add the change output - change := SumOutputs(inputs).Sub(want.Add(fee)) - if !change.IsZero() { - txn.SiacoinOutputs = append(txn.SiacoinOutputs, types.SiacoinOutput{ - Value: change, - Address: w.addr, - }) - } - - // add the inputs - for _, sce := range inputs { - txn.SiacoinInputs = append(txn.SiacoinInputs, types.SiacoinInput{ - ParentID: types.SiacoinOutputID(sce.ID), - UnlockConditions: StandardUnlockConditions(w.priv.PublicKey()), - }) - toSign = append(toSign, sce.ID) - w.lastUsed[sce.ID] = time.Now() - } - - txns = append(txns, txn) - } - - return txns, toSign, nil -} - -func (w *SingleAddressWallet) isOutputUsed(id types.Hash256) bool { - inPool := w.tpoolSpent[types.SiacoinOutputID(id)] - lastUsed := w.lastUsed[id] - if w.usedUTXOExpiry == 0 { - return !lastUsed.IsZero() || inPool - } - return time.Since(lastUsed) <= w.usedUTXOExpiry || inPool -} - -// ProcessConsensusChange implements modules.ConsensusSetSubscriber. -func (w *SingleAddressWallet) ProcessConsensusChange(cc modules.ConsensusChange) { - // only record when we are synced - if !cc.Synced { - return - } - - // apply sane timeout - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() - - // fetch balance - spendable, confirmed, unconfirmed, err := w.Balance() - if err != nil { - w.log.Errorf("failed to fetch wallet balance, err: %v", err) - return - } - - // record wallet metric - if err := w.store.RecordWalletMetric(ctx, api.WalletMetric{ - Timestamp: api.TimeNow(), - Confirmed: confirmed, - Unconfirmed: unconfirmed, - Spendable: spendable, - }); err != nil { - w.log.Errorf("failed to record wallet metric, err: %v", err) - return - } -} - -// ReceiveUpdatedUnconfirmedTransactions implements modules.TransactionPoolSubscriber. -func (w *SingleAddressWallet) ReceiveUpdatedUnconfirmedTransactions(diff *modules.TransactionPoolDiff) { - siacoinOutputs := make(map[types.SiacoinOutputID]SiacoinElement) - utxos, err := w.store.UnspentSiacoinElements(false) - if err != nil { - return - } - for _, output := range utxos { - siacoinOutputs[types.SiacoinOutputID(output.ID)] = output - } - - w.mu.Lock() - defer w.mu.Unlock() - - for id, output := range w.tpoolUtxos { - siacoinOutputs[id] = output - } - - for _, txnsetID := range diff.RevertedTransactions { - txns, ok := w.tpoolTxns[types.Hash256(txnsetID)] - if !ok { - continue - } - for _, txn := range txns { - for _, sci := range txn.Raw.SiacoinInputs { - delete(w.tpoolSpent, sci.ParentID) - } - for i := range txn.Raw.SiacoinOutputs { - delete(w.tpoolUtxos, txn.Raw.SiacoinOutputID(i)) - } - } - delete(w.tpoolTxns, types.Hash256(txnsetID)) - } - - currentHeight := w.store.Height() - - for _, txnset := range diff.AppliedTransactions { - var relevantTxns []Transaction - - txnLoop: - for _, stxn := range txnset.Transactions { - var relevant bool - var txn types.Transaction - convertToCore(stxn, &txn) - processed := Transaction{ - ID: txn.ID(), - Index: types.ChainIndex{ - Height: currentHeight + 1, - }, - Raw: txn, - Timestamp: time.Now(), - } - for _, sci := range txn.SiacoinInputs { - if sci.UnlockConditions.UnlockHash() != w.addr { - continue - } - relevant = true - w.tpoolSpent[sci.ParentID] = true - - output, ok := siacoinOutputs[sci.ParentID] - if !ok { - // note: happens during deep reorgs. Possibly a race - // condition in siad. Log and skip. - w.log.Info("tpool transaction unknown utxo", zap.Stringer("outputID", sci.ParentID), zap.Stringer("txnID", txn.ID())) - continue txnLoop - } - processed.Outflow = processed.Outflow.Add(output.Value) - } - - for i, sco := range txn.SiacoinOutputs { - if sco.Address != w.addr { - continue - } - relevant = true - outputID := txn.SiacoinOutputID(i) - processed.Inflow = processed.Inflow.Add(sco.Value) - sce := SiacoinElement{ - ID: types.Hash256(outputID), - SiacoinOutput: sco, - } - siacoinOutputs[outputID] = sce - w.tpoolUtxos[outputID] = sce - } - - if relevant { - relevantTxns = append(relevantTxns, processed) - } - } - - if len(relevantTxns) != 0 { - w.tpoolTxns[types.Hash256(txnset.ID)] = relevantTxns - } - } -} - -// SumOutputs returns the total value of the supplied outputs. -func SumOutputs(outputs []SiacoinElement) (sum types.Currency) { - for _, o := range outputs { - sum = sum.Add(o.Value) - } - return -} - -// NewSingleAddressWallet returns a new SingleAddressWallet using the provided private key and store. -func NewSingleAddressWallet(priv types.PrivateKey, store SingleAddressStore, usedUTXOExpiry time.Duration, log *zap.SugaredLogger) *SingleAddressWallet { - return &SingleAddressWallet{ - priv: priv, - addr: StandardAddress(priv.PublicKey()), - store: store, - lastUsed: make(map[types.Hash256]time.Time), - usedUTXOExpiry: usedUTXOExpiry, - tpoolTxns: make(map[types.Hash256][]Transaction), - tpoolUtxos: make(map[types.SiacoinOutputID]SiacoinElement), - tpoolSpent: make(map[types.SiacoinOutputID]bool), - log: log.Named("wallet"), - } -} - -// convertToCore converts a siad type to an equivalent core type. -func convertToCore(siad encoding.SiaMarshaler, core types.DecoderFrom) { - var buf bytes.Buffer - siad.MarshalSia(&buf) - d := types.NewBufDecoder(buf.Bytes()) - core.DecodeFrom(d) - if d.Err() != nil { - panic(d.Err()) - } -} diff --git a/worker/worker.go b/worker/worker.go index bc33ca044..f38483555 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1478,7 +1478,7 @@ func discardTxnOnErr(ctx context.Context, bus Bus, l *zap.SugaredLogger, txn typ return } - ctx, cancel := context.WithTimeout(ctx, 10*time.Second) + ctx, cancel := context.WithTimeout(ctx, 30*time.Second) if dErr := bus.WalletDiscard(ctx, txn); dErr != nil { l.Errorf("%w: %v, failed to discard txn: %v", *err, errContext, dErr) }