From d5b374951828fb27e810606ebaae45f21ff8dad1 Mon Sep 17 00:00:00 2001 From: PJ Date: Fri, 15 Dec 2023 11:11:47 +0100 Subject: [PATCH 01/24] all: get rid of context.Background() --- autopilot/accounts.go | 6 +- autopilot/autopilot.go | 42 ++++----- autopilot/contract_pruning.go | 8 +- autopilot/contractor.go | 2 +- autopilot/ipfilter.go | 23 ++--- autopilot/migrator.go | 4 +- autopilot/scanner.go | 4 +- autopilot/scanner_test.go | 2 +- bus/bus.go | 20 +++-- internal/testing/cluster.go | 26 +++--- stores/slabbuffer.go | 2 +- stores/sql.go | 10 +++ webhooks/webhooks.go | 165 +++++++++++++++++----------------- worker/contract_lock.go | 16 ++-- worker/interactions.go | 4 +- worker/rhpv3.go | 72 +++++++++------ worker/spending.go | 4 +- worker/worker.go | 30 ++++--- 18 files changed, 244 insertions(+), 196 deletions(-) diff --git a/autopilot/accounts.go b/autopilot/accounts.go index d9b6963e8..d3ea83013 100644 --- a/autopilot/accounts.go +++ b/autopilot/accounts.go @@ -95,7 +95,7 @@ func (a *accounts) refillWorkersAccountsLoop(ctx context.Context) { } a.w.withWorker(func(w Worker) { - a.refillWorkerAccounts(w) + a.refillWorkerAccounts(ctx, w) }) } } @@ -105,8 +105,8 @@ func (a *accounts) refillWorkersAccountsLoop(ctx context.Context) { // is used for every host. If a slow host's account is still being refilled by a // goroutine from a previous call, refillWorkerAccounts will skip that account // until the previously launched goroutine returns. -func (a *accounts) refillWorkerAccounts(w Worker) { - ctx, span := tracing.Tracer.Start(context.Background(), "refillWorkerAccounts") +func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { + ctx, span := tracing.Tracer.Start(ctx, "refillWorkerAccounts") defer span.End() // fetch config diff --git a/autopilot/autopilot.go b/autopilot/autopilot.go index 43b54023d..019eed028 100644 --- a/autopilot/autopilot.go +++ b/autopilot/autopilot.go @@ -109,12 +109,12 @@ type Autopilot struct { stateMu sync.Mutex state state - startStopMu sync.Mutex - startTime time.Time - stopCtx context.Context - stopCtxCancel context.CancelFunc - ticker *time.Ticker - triggerChan chan bool + startStopMu sync.Mutex + startTime time.Time + shutdownCtx context.Context + shutdownCtxCancel context.CancelFunc + ticker *time.Ticker + triggerChan chan bool } // state holds a bunch of variables that are used by the autopilot and updated @@ -178,7 +178,7 @@ func (ap *Autopilot) Run() error { return errors.New("already running") } ap.startTime = time.Now() - ap.stopCtx, ap.stopCtxCancel = context.WithCancel(context.Background()) + ap.shutdownCtx, ap.shutdownCtxCancel = context.WithCancel(context.Background()) ap.triggerChan = make(chan bool, 1) ap.ticker = time.NewTicker(ap.tickerDuration) @@ -205,7 +205,7 @@ func (ap *Autopilot) Run() error { tickerFired := make(chan struct{}) ap.workers.withWorker(func(w Worker) { defer ap.logger.Info("autopilot iteration ended") - ctx, span := tracing.Tracer.Start(context.Background(), "Autopilot Iteration") + ctx, span := tracing.Tracer.Start(ap.shutdownCtx, "Autopilot Iteration") defer span.End() // initiate a host scan - no need to be synced or configured for scanning @@ -283,7 +283,7 @@ func (ap *Autopilot) Run() error { if maintenanceSuccess { launchAccountRefillsOnce.Do(func() { ap.logger.Debug("account refills loop launched") - go ap.a.refillWorkersAccountsLoop(ap.stopCtx) + go ap.a.refillWorkersAccountsLoop(ap.shutdownCtx) }) } else { ap.logger.Errorf("contract maintenance failed, err: %v", err) @@ -301,7 +301,7 @@ func (ap *Autopilot) Run() error { }) select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return nil case forceScan = <-ap.triggerChan: ap.logger.Info("autopilot iteration triggered") @@ -319,7 +319,7 @@ func (ap *Autopilot) Shutdown(_ context.Context) error { if ap.isRunning() { ap.ticker.Stop() - ap.stopCtxCancel() + ap.shutdownCtxCancel() close(ap.triggerChan) ap.wg.Wait() ap.startTime = time.Time{} @@ -368,7 +368,7 @@ func (ap *Autopilot) blockUntilConfigured(interrupt <-chan time.Time) (configure for { // try and fetch the config - ctx, cancel := context.WithTimeout(ap.stopCtx, 30*time.Second) + ctx, cancel := context.WithTimeout(ap.shutdownCtx, 30*time.Second) _, err := ap.bus.Autopilot(ctx, ap.id) cancel() @@ -380,7 +380,7 @@ func (ap *Autopilot) blockUntilConfigured(interrupt <-chan time.Time) (configure } if err != nil { select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return false, false case <-interrupt: return false, true @@ -399,7 +399,7 @@ func (ap *Autopilot) blockUntilOnline() (online bool) { var once sync.Once for { - ctx, cancel := context.WithTimeout(ap.stopCtx, 30*time.Second) + ctx, cancel := context.WithTimeout(ap.shutdownCtx, 30*time.Second) peers, err := ap.bus.SyncerPeers(ctx) online = len(peers) > 0 cancel() @@ -412,7 +412,7 @@ func (ap *Autopilot) blockUntilOnline() (online bool) { if err != nil || !online { select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return case <-ticker.C: continue @@ -430,7 +430,7 @@ func (ap *Autopilot) blockUntilSynced(interrupt <-chan time.Time) (synced, block for { // try and fetch consensus - ctx, cancel := context.WithTimeout(ap.stopCtx, 30*time.Second) + ctx, cancel := context.WithTimeout(ap.shutdownCtx, 30*time.Second) cs, err := ap.bus.ConsensusState(ctx) synced = cs.Synced cancel() @@ -445,7 +445,7 @@ func (ap *Autopilot) blockUntilSynced(interrupt <-chan time.Time) (synced, block if err != nil || !synced { blocked = true select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return case <-interrupt: interrupted = true @@ -459,7 +459,7 @@ func (ap *Autopilot) blockUntilSynced(interrupt <-chan time.Time) (synced, block } func (ap *Autopilot) tryScheduleTriggerWhenFunded() error { - ctx, cancel := context.WithTimeout(ap.stopCtx, 30*time.Second) + ctx, cancel := context.WithTimeout(ap.shutdownCtx, 30*time.Second) wallet, err := ap.bus.Wallet(ctx) cancel() @@ -480,13 +480,13 @@ func (ap *Autopilot) tryScheduleTriggerWhenFunded() error { defer ticker.Stop() for { select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return case <-ticker.C: } // fetch wallet info - ctx, cancel := context.WithTimeout(ap.stopCtx, 30*time.Second) + ctx, cancel := context.WithTimeout(ap.shutdownCtx, 30*time.Second) if wallet, err = ap.bus.Wallet(ctx); err != nil { ap.logger.Errorf("failed to get wallet info, err: %v", err) } @@ -583,7 +583,7 @@ func (ap *Autopilot) updateState(ctx context.Context) error { func (ap *Autopilot) isStopped() bool { select { - case <-ap.stopCtx.Done(): + case <-ap.shutdownCtx.Done(): return true default: return false diff --git a/autopilot/contract_pruning.go b/autopilot/contract_pruning.go index 56ff3dd9c..4d0cd5f46 100644 --- a/autopilot/contract_pruning.go +++ b/autopilot/contract_pruning.go @@ -91,7 +91,7 @@ func (pr pruneResult) toMetric() api.ContractPruneMetric { func (c *contractor) fetchPrunableContracts() (prunable []api.ContractPrunableData, _ error) { // use a sane timeout - ctx, cancel := context.WithTimeout(c.ap.stopCtx, time.Minute) + ctx, cancel := context.WithTimeout(c.ap.shutdownCtx, time.Minute) defer cancel() // fetch prunable data @@ -156,7 +156,7 @@ func (c *contractor) performContractPruning(wp *workerPool) { } // handle alert - ctx, cancel := context.WithTimeout(c.ap.stopCtx, time.Minute) + ctx, cancel := context.WithTimeout(c.ap.shutdownCtx, time.Minute) if id, alert := result.toAlert(); alert != nil { c.ap.RegisterAlert(ctx, *alert) } else { @@ -170,7 +170,7 @@ func (c *contractor) performContractPruning(wp *workerPool) { }) // record metrics - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithTimeout(c.ap.shutdownCtx, time.Minute) if err := c.ap.bus.RecordContractPruneMetric(ctx, metrics...); err != nil { c.logger.Error(err) } @@ -182,7 +182,7 @@ func (c *contractor) performContractPruning(wp *workerPool) { func (c *contractor) pruneContract(w Worker, fcid types.FileContractID) pruneResult { // create a sane timeout - ctx, cancel := context.WithTimeout(c.ap.stopCtx, 2*timeoutPruneContract) + ctx, cancel := context.WithTimeout(c.ap.shutdownCtx, 2*timeoutPruneContract) defer cancel() // fetch the host diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 16666d16c..87382d035 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -141,7 +141,7 @@ func newContractor(ap *Autopilot, revisionSubmissionBuffer uint64, revisionBroad revisionLastBroadcast: make(map[types.FileContractID]time.Time), revisionSubmissionBuffer: revisionSubmissionBuffer, - resolver: newIPResolver(resolverLookupTimeout, ap.logger.Named("resolver")), + resolver: newIPResolver(ap.shutdownCtx, resolverLookupTimeout, ap.logger.Named("resolver")), } } diff --git a/autopilot/ipfilter.go b/autopilot/ipfilter.go index 6a3ceb009..1844955f6 100644 --- a/autopilot/ipfilter.go +++ b/autopilot/ipfilter.go @@ -88,11 +88,11 @@ type ( } ipResolver struct { - resolver resolver - cache map[string]ipCacheEntry - timeout time.Duration - - logger *zap.SugaredLogger + resolver resolver + cache map[string]ipCacheEntry + timeout time.Duration + shutdownCtx context.Context + logger *zap.SugaredLogger } ipCacheEntry struct { @@ -101,15 +101,16 @@ type ( } ) -func newIPResolver(timeout time.Duration, logger *zap.SugaredLogger) *ipResolver { +func newIPResolver(ctx context.Context, timeout time.Duration, logger *zap.SugaredLogger) *ipResolver { if timeout == 0 { panic("timeout must be greater than zero") // developer error } return &ipResolver{ - resolver: &net.Resolver{}, - cache: make(map[string]ipCacheEntry), - timeout: resolverLookupTimeout, - logger: logger, + resolver: &net.Resolver{}, + cache: make(map[string]ipCacheEntry), + timeout: resolverLookupTimeout, + shutdownCtx: ctx, + logger: logger, } } @@ -129,7 +130,7 @@ func (r *ipResolver) lookup(hostIP string) ([]string, error) { } // make sure we don't hang - ctx, cancel := context.WithTimeout(context.Background(), r.timeout) + ctx, cancel := context.WithTimeout(r.shutdownCtx, r.timeout) defer cancel() // lookup IP addresses diff --git a/autopilot/migrator.go b/autopilot/migrator.go index f7319f5fb..16b4adda9 100644 --- a/autopilot/migrator.go +++ b/autopilot/migrator.go @@ -122,7 +122,7 @@ func (m *migrator) performMigrations(p *workerPool) { m.logger.Info("performing migrations") b := m.ap.bus - ctx, span := tracing.Tracer.Start(context.Background(), "migrator.performMigrations") + ctx, span := tracing.Tracer.Start(m.ap.shutdownCtx, "migrator.performMigrations") defer span.End() // prepare a channel to push work to the workers @@ -258,7 +258,7 @@ OUTER: for i, slab := range toMigrate { select { - case <-m.ap.stopCtx.Done(): + case <-m.ap.shutdownCtx.Done(): return case <-m.signalMaintenanceFinished: m.logger.Info("migrations interrupted - updating slabs for migration") diff --git a/autopilot/scanner.go b/autopilot/scanner.go index 6d4977240..3d344ca3e 100644 --- a/autopilot/scanner.go +++ b/autopilot/scanner.go @@ -264,7 +264,7 @@ func (s *scanner) launchHostScans() chan scanReq { cutoff := time.Now().Add(-s.scanMinInterval) for !s.ap.isStopped() && !exhausted { // fetch next batch - hosts, err := s.bus.HostsForScanning(s.ap.stopCtx, api.HostsForScanningOptions{ + hosts, err := s.bus.HostsForScanning(s.ap.shutdownCtx, api.HostsForScanningOptions{ MaxLastScan: api.TimeRFC3339(cutoff), Offset: offset, Limit: int(s.scanBatchSize), @@ -286,7 +286,7 @@ func (s *scanner) launchHostScans() chan scanReq { // add batch to scan queue for _, h := range hosts { select { - case <-s.ap.stopCtx.Done(): + case <-s.ap.shutdownCtx.Done(): return case reqChan <- scanReq{ hostKey: h.PublicKey, diff --git a/autopilot/scanner_test.go b/autopilot/scanner_test.go index ac4fb8d80..d5833d1fb 100644 --- a/autopilot/scanner_test.go +++ b/autopilot/scanner_test.go @@ -141,7 +141,7 @@ func (s *scanner) isScanning() bool { func newTestScanner(b *mockBus, w *mockWorker) *scanner { ap := &Autopilot{} - ap.stopCtx, ap.stopCtxCancel = context.WithCancel(context.Background()) + ap.shutdownCtx, ap.shutdownCtxCancel = context.WithCancel(context.Background()) return &scanner{ ap: ap, bus: b, diff --git a/bus/bus.go b/bus/bus.go index 9689f307c..9380fe215 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -2285,10 +2285,16 @@ func New(s Syncer, am *alerts.Manager, hm *webhooks.Manager, cm ChainManager, tp startTime: time.Now(), } - ctx, span := tracing.Tracer.Start(context.Background(), "bus.New") + + // ensure we don't hang indefinitely + ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) + defer cancel() + + // attach tracing span + ctx, span := tracing.Tracer.Start(ctx, "bus.New") defer span.End() - // Load default settings if the setting is not already set. + // load default settings if the setting is not already set for key, value := range map[string]interface{}{ api.SettingGouging: build.DefaultGougingSettings, api.SettingRedundancy: build.DefaultRedundancySettings, @@ -2303,7 +2309,7 @@ func New(s Syncer, am *alerts.Manager, hm *webhooks.Manager, cm ChainManager, tp } } - // Check redundancy settings for validity + // check redundancy settings for validity var rs api.RedundancySettings if rss, err := b.ss.Setting(ctx, api.SettingRedundancy); err != nil { return nil, err @@ -2317,7 +2323,7 @@ func New(s Syncer, am *alerts.Manager, hm *webhooks.Manager, cm ChainManager, tp } } - // Check gouging settings for validity + // check gouging settings for validity var gs api.GougingSettings if gss, err := b.ss.Setting(ctx, api.SettingGouging); err != nil { return nil, err @@ -2353,15 +2359,15 @@ func New(s Syncer, am *alerts.Manager, hm *webhooks.Manager, cm ChainManager, tp } } - // Load the accounts into memory. They're saved when the bus is stopped. + // load the accounts into memory, they're saved when the bus is stopped accounts, err := eas.Accounts(ctx) if err != nil { return nil, err } b.accounts = newAccounts(accounts, b.logger) - // Mark the shutdown as unclean. This will be overwritten when/if the - // accounts are saved on shutdown. + // mark the shutdown as unclean, this will be overwritten when/if the + // accounts are saved on shutdown if err := eas.SetUncleanShutdown(); err != nil { return nil, fmt.Errorf("failed to mark account shutdown as unclean: %w", err) } diff --git a/internal/testing/cluster.go b/internal/testing/cluster.go index 5d4eb005f..2011cdceb 100644 --- a/internal/testing/cluster.go +++ b/internal/testing/cluster.go @@ -243,7 +243,7 @@ func (c *TestCluster) Reboot(ctx context.Context) *TestCluster { // AutopilotConfig returns the autopilot's config and current period. func (c *TestCluster) AutopilotConfig(ctx context.Context) (api.AutopilotConfig, uint64) { c.tt.Helper() - ap, err := c.Bus.Autopilot(context.Background(), c.apID) + ap, err := c.Bus.Autopilot(ctx, c.apID) c.tt.OK(err) return ap.Config, ap.CurrentPeriod } @@ -251,7 +251,7 @@ func (c *TestCluster) AutopilotConfig(ctx context.Context) (api.AutopilotConfig, // UpdateAutopilotConfig updates the cluster's autopilot with given config. func (c *TestCluster) UpdateAutopilotConfig(ctx context.Context, cfg api.AutopilotConfig) { c.tt.Helper() - c.tt.OK(c.Bus.UpdateAutopilot(context.Background(), api.Autopilot{ + c.tt.OK(c.Bus.UpdateAutopilot(ctx, api.Autopilot{ ID: c.apID, Config: cfg, })) @@ -304,6 +304,10 @@ func newTestCluster(t *testing.T, opts testClusterOptions) *TestCluster { } tt := &TT{t} + // Ensure we don't hang + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + // Apply options. dbName := opts.dbName dir := t.TempDir() @@ -508,30 +512,30 @@ func newTestCluster(t *testing.T, opts testClusterOptions) *TestCluster { // Set the test contract set to make sure we can add objects at the // beginning of a test right away. - tt.OK(busClient.SetContractSet(context.Background(), testContractSet, []types.FileContractID{})) + tt.OK(busClient.SetContractSet(ctx, testContractSet, []types.FileContractID{})) // Update the autopilot to use test settings if !opts.skipSettingAutopilot { - tt.OK(busClient.UpdateAutopilot(context.Background(), api.Autopilot{ + tt.OK(busClient.UpdateAutopilot(ctx, api.Autopilot{ ID: apCfg.ID, Config: apSettings, })) } // Update the bus settings. - tt.OK(busClient.UpdateSetting(context.Background(), api.SettingGouging, testGougingSettings)) - tt.OK(busClient.UpdateSetting(context.Background(), api.SettingRedundancy, testRedundancySettings)) - tt.OK(busClient.UpdateSetting(context.Background(), api.SettingContractSet, testContractSetSettings)) - tt.OK(busClient.UpdateSetting(context.Background(), api.SettingS3Authentication, api.S3AuthenticationSettings{ + tt.OK(busClient.UpdateSetting(ctx, api.SettingGouging, testGougingSettings)) + tt.OK(busClient.UpdateSetting(ctx, api.SettingRedundancy, testRedundancySettings)) + tt.OK(busClient.UpdateSetting(ctx, api.SettingContractSet, testContractSetSettings)) + tt.OK(busClient.UpdateSetting(ctx, api.SettingS3Authentication, api.S3AuthenticationSettings{ V4Keypairs: map[string]string{testS3AccessKeyID: testS3SecretAccessKey}, })) - tt.OK(busClient.UpdateSetting(context.Background(), api.SettingUploadPacking, api.UploadPackingSettings{Enabled: enableUploadPacking})) + tt.OK(busClient.UpdateSetting(ctx, api.SettingUploadPacking, api.UploadPackingSettings{Enabled: enableUploadPacking})) // Fund the bus. if funding { cluster.MineBlocks(latestHardforkHeight) tt.Retry(1000, 100*time.Millisecond, func() error { - resp, err := busClient.ConsensusState(context.Background()) + resp, err := busClient.ConsensusState(ctx) if err != nil { return err } @@ -539,7 +543,7 @@ func newTestCluster(t *testing.T, opts testClusterOptions) *TestCluster { if !resp.Synced || resp.BlockHeight < latestHardforkHeight { return fmt.Errorf("chain not synced: %v %v", resp.Synced, resp.BlockHeight < latestHardforkHeight) } - res, err := cluster.Bus.Wallet(context.Background()) + res, err := cluster.Bus.Wallet(ctx) if err != nil { return err } diff --git a/stores/slabbuffer.go b/stores/slabbuffer.go index c16bcb989..c13e66e07 100644 --- a/stores/slabbuffer.go +++ b/stores/slabbuffer.go @@ -88,7 +88,7 @@ func newSlabBufferManager(sqlStore *SQLStore, slabBufferCompletionThreshold int6 // Open the file. file, err := os.OpenFile(filepath.Join(partialSlabDir, buffer.Filename), os.O_RDWR, 0600) if err != nil { - _ = sqlStore.alerts.RegisterAlert(context.Background(), alerts.Alert{ + _ = sqlStore.alerts.RegisterAlert(sqlStore.shutdownCtx, alerts.Alert{ ID: types.HashBytes([]byte(buffer.Filename)), Severity: alerts.SeverityCritical, Message: "failed to read buffer file on startup", diff --git a/stores/sql.go b/stores/sql.go index 26b1e91f5..6a572f0d6 100644 --- a/stores/sql.go +++ b/stores/sql.go @@ -1,6 +1,7 @@ package stores import ( + "context" "database/sql" "errors" "fmt" @@ -75,6 +76,9 @@ type ( ccid modules.ConsensusChangeID chainIndex types.ChainIndex + shutdownCtx context.Context + shutdownCtxCancel context.CancelFunc + mu sync.Mutex hasAllowlist bool hasBlocklist bool @@ -228,6 +232,7 @@ func NewSQLStore(conn, connMetrics gorm.Dialector, alerts alerts.Alerter, partia isOurContract[types.FileContractID(fcid)] = struct{}{} } + shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) ss := &SQLStore{ alerts: alerts, db: db, @@ -251,6 +256,9 @@ func NewSQLStore(conn, connMetrics gorm.Dialector, alerts alerts.Alerter, partia Height: ci.Height, ID: types.BlockID(ci.BlockID), }, + + shutdownCtx: shutdownCtx, + shutdownCtxCancel: shutdownCtxCancel, } ss.slabBufferMgr, err = newSlabBufferManager(ss, slabBufferCompletionThreshold, partialSlabDir) @@ -311,6 +319,8 @@ func tableCount(db *gorm.DB, model interface{}) (cnt int64, err error) { // Close closes the underlying database connection of the store. func (s *SQLStore) Close() error { + s.shutdownCtxCancel() + db, err := s.db.DB() if err != nil { return err diff --git a/webhooks/webhooks.go b/webhooks/webhooks.go index a14c77a21..e3d388de8 100644 --- a/webhooks/webhooks.go +++ b/webhooks/webhooks.go @@ -59,11 +59,12 @@ type ( ) type Manager struct { - ctx context.Context - ctxCancel context.CancelFunc - logger *zap.SugaredLogger - wg sync.WaitGroup - store WebhookStore + logger *zap.SugaredLogger + wg sync.WaitGroup + store WebhookStore + + shutdownCtx context.Context + shutdownCtxCancel context.CancelFunc mu sync.Mutex queues map[string]*eventQueue // URL -> queue @@ -80,55 +81,64 @@ type eventQueue struct { events []Event } -func (w *Manager) Close() error { - w.ctxCancel() - w.wg.Wait() - return nil -} - -func (w Webhook) String() string { - return fmt.Sprintf("%v.%v.%v", w.URL, w.Module, w.Event) -} +func (m *Manager) BroadcastAction(_ context.Context, event Event) error { + m.mu.Lock() + defer m.mu.Unlock() + for _, hook := range m.webhooks { + if !hook.Matches(event) { + continue + } -func (w *Manager) Register(wh Webhook) error { - ctx, cancel := context.WithTimeout(context.Background(), webhookTimeout) - defer cancel() + // Find queue or create one. + queue, exists := m.queues[hook.URL] + if !exists { + queue = &eventQueue{ + ctx: m.shutdownCtx, + logger: m.logger, + url: hook.URL, + } + m.queues[hook.URL] = queue + } - // Test URL. - err := sendEvent(ctx, wh.URL, Event{ - Event: WebhookEventPing, - }) - if err != nil { - return err + // Add event and launch goroutine to start dequeueing if necessary. + queue.mu.Lock() + queue.events = append(queue.events, event) + if !queue.isDequeueing { + queue.isDequeueing = true + m.wg.Add(1) + go func() { + queue.dequeue() + m.wg.Done() + }() + } + queue.mu.Unlock() } + return nil +} - // Add Webhook. - if err := w.store.AddWebhook(wh); err != nil { - return err - } - w.mu.Lock() - defer w.mu.Unlock() - w.webhooks[wh.String()] = wh +func (m *Manager) Close() error { + m.shutdownCtxCancel() + m.wg.Wait() return nil } -func (w *Manager) Delete(wh Webhook) error { - w.mu.Lock() - defer w.mu.Unlock() - if err := w.store.DeleteWebhook(wh); errors.Is(err, gorm.ErrRecordNotFound) { +func (m *Manager) Delete(wh Webhook) error { + m.mu.Lock() + defer m.mu.Unlock() + if err := m.store.DeleteWebhook(wh); errors.Is(err, gorm.ErrRecordNotFound) { return ErrWebhookNotFound } else if err != nil { return err } - delete(w.webhooks, wh.String()) + delete(m.webhooks, wh.String()) return nil } -func (w *Manager) Info() ([]Webhook, []WebhookQueueInfo) { - w.mu.Lock() - defer w.mu.Unlock() +func (m *Manager) Info() ([]Webhook, []WebhookQueueInfo) { + m.mu.Lock() + defer m.mu.Unlock() var hooks []Webhook - for _, hook := range w.webhooks { + for _, hook := range m.webhooks { hooks = append(hooks, Webhook{ Event: hook.Event, Module: hook.Module, @@ -136,7 +146,7 @@ func (w *Manager) Info() ([]Webhook, []WebhookQueueInfo) { }) } var queueInfos []WebhookQueueInfo - for _, queue := range w.queues { + for _, queue := range m.queues { queue.mu.Lock() queueInfos = append(queueInfos, WebhookQueueInfo{ URL: queue.url, @@ -147,45 +157,32 @@ func (w *Manager) Info() ([]Webhook, []WebhookQueueInfo) { return hooks, queueInfos } -func (a Event) String() string { - return a.Module + "." + a.Event -} - -func (w *Manager) BroadcastAction(_ context.Context, event Event) error { - w.mu.Lock() - defer w.mu.Unlock() - for _, hook := range w.webhooks { - if !hook.Matches(event) { - continue - } +func (m *Manager) Register(wh Webhook) error { + ctx, cancel := context.WithTimeout(m.shutdownCtx, webhookTimeout) + defer cancel() - // Find queue or create one. - queue, exists := w.queues[hook.URL] - if !exists { - queue = &eventQueue{ - ctx: w.ctx, - logger: w.logger, - url: hook.URL, - } - w.queues[hook.URL] = queue - } + // Test URL. + err := sendEvent(ctx, wh.URL, Event{ + Event: WebhookEventPing, + }) + if err != nil { + return err + } - // Add event and launch goroutine to start dequeueing if necessary. - queue.mu.Lock() - queue.events = append(queue.events, event) - if !queue.isDequeueing { - queue.isDequeueing = true - w.wg.Add(1) - go func() { - queue.dequeue() - w.wg.Done() - }() - } - queue.mu.Unlock() + // Add Webhook. + if err := m.store.AddWebhook(wh); err != nil { + return err } + m.mu.Lock() + defer m.mu.Unlock() + m.webhooks[wh.String()] = wh return nil } +func (a Event) String() string { + return a.Module + "." + a.Event +} + func (q *eventQueue) dequeue() { for { q.mu.Lock() @@ -212,20 +209,28 @@ func (w Webhook) Matches(action Event) bool { return w.Event == "" || w.Event == action.Event } +func (w Webhook) String() string { + return fmt.Sprintf("%v.%v.%v", w.URL, w.Module, w.Event) +} + func NewManager(logger *zap.SugaredLogger, store WebhookStore) (*Manager, error) { hooks, err := store.Webhooks() if err != nil { return nil, err } - ctx, cancel := context.WithCancel(context.Background()) + + shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) m := &Manager{ - ctx: ctx, - ctxCancel: cancel, - logger: logger.Named("webhooks"), - queues: make(map[string]*eventQueue), - store: store, - webhooks: make(map[string]Webhook), + logger: logger.Named("webhooks"), + store: store, + + shutdownCtx: shutdownCtx, + shutdownCtxCancel: shutdownCtxCancel, + + queues: make(map[string]*eventQueue), + webhooks: make(map[string]Webhook), } + for _, hook := range hooks { m.webhooks[hook.String()] = hook } diff --git a/worker/contract_lock.go b/worker/contract_lock.go index ea4cd0b41..3361b5ad0 100644 --- a/worker/contract_lock.go +++ b/worker/contract_lock.go @@ -26,9 +26,9 @@ type contractLock struct { locker ContractLocker logger *zap.SugaredLogger - stopCtx context.Context - stopCtxCancel context.CancelFunc - stopWG sync.WaitGroup + keepaliveLoopCtx context.Context + keepaliveLoopCtxCancel context.CancelFunc + stopWG sync.WaitGroup } func newContractLock(fcid types.FileContractID, lockID uint64, d time.Duration, locker ContractLocker, logger *zap.SugaredLogger) *contractLock { @@ -40,8 +40,8 @@ func newContractLock(fcid types.FileContractID, lockID uint64, d time.Duration, locker: locker, logger: logger, - stopCtx: ctx, - stopCtxCancel: cancel, + keepaliveLoopCtx: ctx, + keepaliveLoopCtxCancel: cancel, } cl.stopWG.Add(1) go func() { @@ -75,7 +75,7 @@ func (w *worker) withContractLock(ctx context.Context, fcid types.FileContractID func (cl *contractLock) Release(ctx context.Context) error { // Stop background loop. - cl.stopCtxCancel() + cl.keepaliveLoopCtxCancel() cl.stopWG.Wait() // Release the contract. @@ -101,11 +101,11 @@ func (cl *contractLock) keepaliveLoop() { // Loop until stopped. for { select { - case <-cl.stopCtx.Done(): + case <-cl.keepaliveLoopCtx.Done(): return // released case <-t.C: } - if err := cl.locker.KeepaliveContract(cl.stopCtx, cl.fcid, cl.lockID, cl.d); err != nil && !errors.Is(err, context.Canceled) { + if err := cl.locker.KeepaliveContract(cl.keepaliveLoopCtx, cl.fcid, cl.lockID, cl.d); err != nil && !errors.Is(err, context.Canceled) { cl.logger.Errorw(fmt.Sprintf("failed to send keepalive: %v", err), "contract", cl.fcid, "lockID", cl.lockID, diff --git a/worker/interactions.go b/worker/interactions.go index 1e8a62f89..b1e0203d3 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -77,7 +77,7 @@ func (w *worker) tryFlushInteractionsBuffer() { // flushInteractions flushes the worker's interaction buffer to the bus. func (w *worker) flushInteractions() { if len(w.interactionsScans) > 0 { - ctx, span := tracing.Tracer.Start(context.Background(), "worker: recordHostScans") + ctx, span := tracing.Tracer.Start(w.shutdownCtx, "worker: recordHostScans") defer span.End() if err := w.bus.RecordHostScans(ctx, w.interactionsScans); err != nil { w.logger.Errorw(fmt.Sprintf("failed to record scans: %v", err)) @@ -86,7 +86,7 @@ func (w *worker) flushInteractions() { } } if len(w.interactionsPriceTableUpdates) > 0 { - ctx, span := tracing.Tracer.Start(context.Background(), "worker: recordPriceTableUpdates") + ctx, span := tracing.Tracer.Start(w.shutdownCtx, "worker: recordPriceTableUpdates") defer span.End() if err := w.bus.RecordPriceTables(ctx, w.interactionsPriceTableUpdates); err != nil { w.logger.Errorw(fmt.Sprintf("failed to record price table updates: %v", err)) diff --git a/worker/rhpv3.go b/worker/rhpv3.go index effc9a63e..2550a225a 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -336,17 +336,19 @@ type ( // accounts stores the balance and other metrics of accounts that the // worker maintains with a host. accounts struct { - store AccountStore - key types.PrivateKey + as AccountStore + key types.PrivateKey + shutdownCtx context.Context } // account contains information regarding a specific account of the // worker. account struct { - bus AccountStore - id rhpv3.Account - key types.PrivateKey - host types.PublicKey + as AccountStore + id rhpv3.Account + key types.PrivateKey + host types.PublicKey + shutdownCtx context.Context } ) @@ -355,8 +357,9 @@ func (w *worker) initAccounts(as AccountStore) { panic("accounts already initialized") // developer error } w.accounts = &accounts{ - store: as, - key: w.deriveSubKey("accountkey"), + as: as, + key: w.deriveSubKey("accountkey"), + shutdownCtx: w.shutdownCtx, } } @@ -372,35 +375,45 @@ func (w *worker) initTransportPool() { func (a *accounts) ForHost(hk types.PublicKey) *account { accountID := rhpv3.Account(a.deriveAccountKey(hk).PublicKey()) return &account{ - bus: a.store, - id: accountID, - key: a.key, - host: hk, + as: a.as, + id: accountID, + key: a.key, + host: hk, + shutdownCtx: a.shutdownCtx, } } // WithDeposit increases the balance of an account by the amount returned by // amtFn if amtFn doesn't return an error. func (a *account) WithDeposit(ctx context.Context, amtFn func() (types.Currency, error)) error { - _, lockID, err := a.bus.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) + _, lockID, err := a.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) if err != nil { return err } - defer a.bus.UnlockAccount(ctx, a.id, lockID) + defer func() { + unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) + a.as.UnlockAccount(unlockCtx, a.id, lockID) + cancel() + }() amt, err := amtFn() if err != nil { return err } - return a.bus.AddBalance(ctx, a.id, a.host, amt.Big()) + return a.as.AddBalance(ctx, a.id, a.host, amt.Big()) } func (a *account) Balance(ctx context.Context) (types.Currency, error) { - account, lockID, err := a.bus.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) + account, lockID, err := a.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) if err != nil { return types.Currency{}, err } - defer a.bus.UnlockAccount(ctx, a.id, lockID) + defer func() { + unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) + a.as.UnlockAccount(unlockCtx, a.id, lockID) + cancel() + }() + return types.NewCurrency(account.Balance.Uint64(), new(big.Int).Rsh(account.Balance, 64).Uint64()), nil } @@ -408,13 +421,13 @@ func (a *account) Balance(ctx context.Context) (types.Currency, error) { // amtFn. The amount is still withdrawn if amtFn returns an error since some // costs are non-refundable. func (a *account) WithWithdrawal(ctx context.Context, amtFn func() (types.Currency, error)) error { - account, lockID, err := a.bus.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) + account, lockID, err := a.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) if err != nil { return err } defer func() { - unlockCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) - a.bus.UnlockAccount(unlockCtx, a.id, lockID) + unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) + a.as.UnlockAccount(unlockCtx, a.id, lockID) cancel() }() @@ -432,9 +445,9 @@ func (a *account) WithWithdrawal(ctx context.Context, amtFn func() (types.Curren amt, err := amtFn() if isBalanceInsufficient(err) { // in case of an insufficient balance, we schedule a sync - scheduleCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + scheduleCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) defer cancel() - err2 := a.bus.ScheduleSync(scheduleCtx, a.id, a.host) + err2 := a.as.ScheduleSync(scheduleCtx, a.id, a.host) if err2 != nil { err = fmt.Errorf("%w; failed to set requiresSync flag on bus, error: %v", err, err2) } @@ -446,9 +459,9 @@ func (a *account) WithWithdrawal(ctx context.Context, amtFn func() (types.Curren } // if an amount was returned, we withdraw it. - addCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + addCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) defer cancel() - errAdd := a.bus.AddBalance(addCtx, a.id, a.host, new(big.Int).Neg(amt.Big())) + errAdd := a.as.AddBalance(addCtx, a.id, a.host, new(big.Int).Neg(amt.Big())) if errAdd != nil { err = fmt.Errorf("%w; failed to add balance to account, error: %v", err, errAdd) } @@ -458,16 +471,21 @@ func (a *account) WithWithdrawal(ctx context.Context, amtFn func() (types.Curren // WithSync syncs an accounts balance with the bus. To do so, the account is // locked while the balance is fetched through balanceFn. func (a *account) WithSync(ctx context.Context, balanceFn func() (types.Currency, error)) error { - _, lockID, err := a.bus.LockAccount(ctx, a.id, a.host, true, accountLockingDuration) + _, lockID, err := a.as.LockAccount(ctx, a.id, a.host, true, accountLockingDuration) if err != nil { return err } - defer a.bus.UnlockAccount(ctx, a.id, lockID) + defer func() { + unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) + a.as.UnlockAccount(unlockCtx, a.id, lockID) + cancel() + }() + balance, err := balanceFn() if err != nil { return err } - return a.bus.SetBalance(ctx, a.id, a.host, balance.Big()) + return a.as.SetBalance(ctx, a.id, a.host, balance.Big()) } // deriveAccountKey derives an account plus key for a given host and worker. diff --git a/worker/spending.go b/worker/spending.go index 9e9654081..565748b7e 100644 --- a/worker/spending.go +++ b/worker/spending.go @@ -21,6 +21,7 @@ type ( contractSpendingRecorder struct { bus Bus flushInterval time.Duration + shutdownCtx context.Context logger *zap.SugaredLogger mu sync.Mutex @@ -37,6 +38,7 @@ func (w *worker) initContractSpendingRecorder() { bus: w.bus, contractSpendings: make(map[types.FileContractID]api.ContractSpendingRecord), flushInterval: w.busFlushInterval, + shutdownCtx: w.shutdownCtx, logger: w.logger, } } @@ -76,7 +78,7 @@ func (sr *contractSpendingRecorder) Record(rev types.FileContractRevision, cs ap func (sr *contractSpendingRecorder) flush() { if len(sr.contractSpendings) > 0 { - ctx, span := tracing.Tracer.Start(context.Background(), "worker: flushContractSpending") + ctx, span := tracing.Tracer.Start(sr.shutdownCtx, "worker: flushContractSpending") defer span.End() records := make([]api.ContractSpendingRecord, 0, len(sr.contractSpendings)) for _, cs := range sr.contractSpendings { diff --git a/worker/worker.go b/worker/worker.go index 0b46e5b42..5475486ff 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -16,7 +16,6 @@ import ( "time" "github.com/gotd/contrib/http_range" - "go.opentelemetry.io/otel/trace" rhpv2 "go.sia.tech/core/rhp/v2" rhpv3 "go.sia.tech/core/rhp/v3" "go.sia.tech/core/types" @@ -225,7 +224,7 @@ func (w *worker) withRevision(ctx context.Context, fetchTimeout time.Duration, f } func (w *worker) registerAlert(a alerts.Alert) { - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + ctx, cancel := context.WithTimeout(w.shutdownCtx, time.Minute) if err := w.alerts.RegisterAlert(ctx, a); err != nil { w.logger.Error("failed to register alert", err) } @@ -373,8 +372,8 @@ func (w *worker) rhpPriceTableHandler(jc jape.Context) { jc.Encode(hpt) } -func (w *worker) discardTxnOnErr(ctx context.Context, txn types.Transaction, errContext string, err *error) { - discardTxnOnErr(ctx, w.bus, w.logger, txn, errContext, err) +func (w *worker) discardTxnOnErr(txn types.Transaction, errContext string, err *error) { + discardTxnOnErr(w.shutdownCtx, w.bus, w.logger, txn, errContext, err) } func (w *worker) rhpFormHandler(jc jape.Context) { @@ -429,7 +428,7 @@ func (w *worker) rhpFormHandler(jc jape.Context) { if err != nil { return err } - defer w.discardTxnOnErr(ctx, renterTxnSet[len(renterTxnSet)-1], "rhpFormHandler", &err) + defer w.discardTxnOnErr(renterTxnSet[len(renterTxnSet)-1], "rhpFormHandler", &err) contract, txnSet, err = RPCFormContract(ctx, t, renterKey, renterTxnSet) if err != nil { @@ -1286,7 +1285,7 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush return nil, errors.New("uploadMaxMemory cannot be 0") } - ctx, cancel := context.WithCancel(context.Background()) + shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) w := &worker{ alerts: alerts.WithOrigin(b, fmt.Sprintf("worker.%s", id)), allowPrivateIPs: allowPrivateIPs, @@ -1298,8 +1297,8 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush logger: l.Sugar().Named("worker").Named(id), startTime: time.Now(), uploadingPackedSlabs: make(map[string]bool), - shutdownCtx: ctx, - shutdownCtxCancel: cancel, + shutdownCtx: shutdownCtx, + shutdownCtxCancel: shutdownCtxCancel, } w.initAccounts(b) w.initContractSpendingRecorder() @@ -1424,13 +1423,16 @@ func discardTxnOnErr(ctx context.Context, bus Bus, l *zap.SugaredLogger, txn typ if *err == nil { return } - _, span := tracing.Tracer.Start(ctx, "discardTxn") - defer span.End() - // Attach the span to a new context derived from the background context. - timeoutCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + + // add timeout + ctx, cancel := context.WithTimeout(ctx, 10*time.Second) defer cancel() - timeoutCtx = trace.ContextWithSpan(timeoutCtx, span) - if err := bus.WalletDiscard(timeoutCtx, txn); err != nil { + + // start trace + ctx, span := tracing.Tracer.Start(ctx, "discardTxn") + defer span.End() + + if err := bus.WalletDiscard(ctx, txn); err != nil { l.Errorf("%v: failed to discard txn: %v", err) } } From 0810fe1d116ae5335c55d7e6a31a2f17558b7763 Mon Sep 17 00:00:00 2001 From: PJ Date: Fri, 15 Dec 2023 11:16:05 +0100 Subject: [PATCH 02/24] worker: use w.shutdownCtx --- worker/contract_lock.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/contract_lock.go b/worker/contract_lock.go index 3361b5ad0..787e388b2 100644 --- a/worker/contract_lock.go +++ b/worker/contract_lock.go @@ -65,7 +65,7 @@ func (w *worker) withContractLock(ctx context.Context, fcid types.FileContractID return err } defer func() { - releaseCtx, cancel := context.WithTimeout(context.Background(), 10*time.Second) + releaseCtx, cancel := context.WithTimeout(w.shutdownCtx, 10*time.Second) _ = contractLock.Release(releaseCtx) cancel() }() From 0f3c8c3e2f929db1992e993f33d6899674d8e86c Mon Sep 17 00:00:00 2001 From: PJ Date: Fri, 15 Dec 2023 11:39:11 +0100 Subject: [PATCH 03/24] worker: fix shutdown order --- worker/worker.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/worker/worker.go b/worker/worker.go index 5475486ff..932366b96 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1344,8 +1344,6 @@ func (w *worker) Handler() http.Handler { // Shutdown shuts down the worker. func (w *worker) Shutdown(_ context.Context) error { - w.shutdownCtxCancel() - w.interactionsMu.Lock() if w.interactionsFlushTimer != nil { w.interactionsFlushTimer.Stop() @@ -1353,6 +1351,9 @@ func (w *worker) Shutdown(_ context.Context) error { } w.interactionsMu.Unlock() + // Cancel shutdown context. + w.shutdownCtxCancel() + // Stop contract spending recorder. w.contractSpendingRecorder.Stop() From 7cf9a3dd3a0c319054f46a135e09f244c612cf7b Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Fri, 15 Dec 2023 15:15:21 +0100 Subject: [PATCH 04/24] stores: use raw sql instead of DropColumn (#825) --- stores/migrations.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/stores/migrations.go b/stores/migrations.go index a8cd8225c..94708a8d4 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -1401,12 +1401,12 @@ func performMigration00035_bufferedSlabsDropSizeAndComplete(txn *gorm.DB, logger } if txn.Migrator().HasColumn(&dbBufferedSlab{}, "size") { - if err := txn.Migrator().DropColumn(&dbBufferedSlab{}, "size"); err != nil { + if err := txn.Exec("ALTER TABLE buffered_slabs DROP COLUMN size").Error; err != nil { return err } } if txn.Migrator().HasColumn(&dbBufferedSlab{}, "complete") { - if err := txn.Migrator().DropColumn(&dbBufferedSlab{}, "complete"); err != nil { + if err := txn.Exec("ALTER TABLE buffered_slabs DROP COLUMN complete").Error; err != nil { return err } } From 44a93f3cab97719f3a96009443a4c515074aac48 Mon Sep 17 00:00:00 2001 From: PJ Date: Fri, 15 Dec 2023 16:08:16 +0100 Subject: [PATCH 05/24] autopilot: set shutdown ctx in New --- autopilot/autopilot.go | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/autopilot/autopilot.go b/autopilot/autopilot.go index 019eed028..0e240f432 100644 --- a/autopilot/autopilot.go +++ b/autopilot/autopilot.go @@ -130,6 +130,8 @@ type state struct { // New initializes an Autopilot. func New(id string, bus Bus, workers []Worker, logger *zap.Logger, heartbeat time.Duration, scannerScanInterval time.Duration, scannerBatchSize, scannerNumThreads uint64, migrationHealthCutoff float64, accountsRefillInterval time.Duration, revisionSubmissionBuffer, migratorParallelSlabsPerWorker uint64, revisionBroadcastInterval time.Duration) (*Autopilot, error) { + shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) + ap := &Autopilot{ alerts: alerts.WithOrigin(bus, fmt.Sprintf("autopilot.%s", id)), id: id, @@ -137,6 +139,9 @@ func New(id string, bus Bus, workers []Worker, logger *zap.Logger, heartbeat tim logger: logger.Sugar().Named(api.DefaultAutopilotID), workers: newWorkerPool(workers), + shutdownCtx: shutdownCtx, + shutdownCtxCancel: shutdownCtxCancel, + tickerDuration: heartbeat, } scanner, err := newScanner( @@ -178,7 +183,6 @@ func (ap *Autopilot) Run() error { return errors.New("already running") } ap.startTime = time.Now() - ap.shutdownCtx, ap.shutdownCtxCancel = context.WithCancel(context.Background()) ap.triggerChan = make(chan bool, 1) ap.ticker = time.NewTicker(ap.tickerDuration) From 4d62b62552230eba094e05625aa9d9b0f5985a0a Mon Sep 17 00:00:00 2001 From: Peter-Jan Brone Date: Fri, 15 Dec 2023 16:08:51 +0100 Subject: [PATCH 06/24] Remove Tracing (#828) * all: remove tracing * worker: fix break statement --- autopilot/accounts.go | 27 -------------- autopilot/autopilot.go | 15 ++++---- autopilot/contractor.go | 50 +------------------------ autopilot/migrator.go | 16 ++++---- bus/bus.go | 9 +---- cmd/renterd/main.go | 21 ----------- config/config.go | 7 ---- go.mod | 18 --------- go.sum | 42 --------------------- tracing/tracing.go | 83 ----------------------------------------- worker/download.go | 47 +---------------------- worker/interactions.go | 9 +---- worker/migrations.go | 4 -- worker/spending.go | 5 +-- worker/upload.go | 46 +---------------------- worker/uploader.go | 25 +------------ worker/worker.go | 17 +++------ 17 files changed, 30 insertions(+), 411 deletions(-) delete mode 100644 tracing/tracing.go diff --git a/autopilot/accounts.go b/autopilot/accounts.go index d3ea83013..8f7e49c2f 100644 --- a/autopilot/accounts.go +++ b/autopilot/accounts.go @@ -8,12 +8,9 @@ import ( "sync" "time" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/codes" rhpv3 "go.sia.tech/core/rhp/v3" "go.sia.tech/core/types" "go.sia.tech/renterd/api" - "go.sia.tech/renterd/tracing" "go.uber.org/zap" ) @@ -106,17 +103,12 @@ func (a *accounts) refillWorkersAccountsLoop(ctx context.Context) { // goroutine from a previous call, refillWorkerAccounts will skip that account // until the previously launched goroutine returns. func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { - ctx, span := tracing.Tracer.Start(ctx, "refillWorkerAccounts") - defer span.End() - // fetch config state := a.ap.State() // fetch worker id workerID, err := w.ID(ctx) if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, "failed to fetch worker id") a.l.Errorw(fmt.Sprintf("failed to fetch worker id for refill: %v", err)) return } @@ -124,8 +116,6 @@ func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { // fetch all contracts contracts, err := a.c.Contracts(ctx) if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, "failed to fetch contracts") a.l.Errorw(fmt.Sprintf("failed to fetch contracts for refill: %v", err)) return } else if len(contracts) == 0 { @@ -135,8 +125,6 @@ func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { // fetch all contract set contracts contractSetContracts, err := a.c.ContractSetContracts(ctx, state.cfg.Contracts.Set) if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, fmt.Sprintf("failed to fetch contracts for set '%s'", state.cfg.Contracts.Set)) a.l.Errorw(fmt.Sprintf("failed to fetch contract set contracts: %v", err)) return } @@ -212,17 +200,6 @@ func refillWorkerAccount(ctx context.Context, a AccountStore, w Worker, workerID } } - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "refillAccount") - span.SetAttributes(attribute.Stringer("host", contract.HostKey)) - defer func() { - if rerr != nil { - span.RecordError(rerr.err) - span.SetStatus(codes.Error, "failed to refill account") - } - span.End() - }() - // fetch the account accountID, err := w.Account(ctx, contract.HostKey) if err != nil { @@ -236,10 +213,6 @@ func refillWorkerAccount(ctx context.Context, a AccountStore, w Worker, workerID return } - // update span - span.SetAttributes(attribute.Stringer("account", account.ID)) - span.SetAttributes(attribute.Stringer("balance", account.Balance)) - // check if a host is potentially cheating before refilling. // We only check against the max drift if the account's drift is // negative because we don't care if we have more money than diff --git a/autopilot/autopilot.go b/autopilot/autopilot.go index 0e240f432..2555776f8 100644 --- a/autopilot/autopilot.go +++ b/autopilot/autopilot.go @@ -10,7 +10,6 @@ import ( "sync" "time" - "go.opentelemetry.io/otel/attribute" rhpv2 "go.sia.tech/core/rhp/v2" rhpv3 "go.sia.tech/core/rhp/v3" "go.sia.tech/core/types" @@ -20,7 +19,6 @@ import ( "go.sia.tech/renterd/build" "go.sia.tech/renterd/hostdb" "go.sia.tech/renterd/object" - "go.sia.tech/renterd/tracing" "go.sia.tech/renterd/wallet" "go.sia.tech/renterd/webhooks" "go.uber.org/zap" @@ -166,14 +164,14 @@ func New(id string, bus Bus, workers []Worker, logger *zap.Logger, heartbeat tim // Handler returns an HTTP handler that serves the autopilot api. func (ap *Autopilot) Handler() http.Handler { - return jape.Mux(tracing.TracingMiddleware(api.DefaultAutopilotID, map[string]jape.Handler{ + return jape.Mux(map[string]jape.Handler{ "GET /config": ap.configHandlerGET, "PUT /config": ap.configHandlerPUT, "POST /hosts": ap.hostsHandlerPOST, "GET /host/:hostKey": ap.hostHandlerGET, "GET /state": ap.stateHandlerGET, "POST /trigger": ap.triggerHandlerPOST, - })) + }) } func (ap *Autopilot) Run() error { @@ -209,8 +207,10 @@ func (ap *Autopilot) Run() error { tickerFired := make(chan struct{}) ap.workers.withWorker(func(w Worker) { defer ap.logger.Info("autopilot iteration ended") - ctx, span := tracing.Tracer.Start(ap.shutdownCtx, "Autopilot Iteration") - defer span.End() + + // create a new context for this iteration + ctx, cancel := context.WithCancel(ap.shutdownCtx) + defer cancel() // initiate a host scan - no need to be synced or configured for scanning ap.s.tryUpdateTimeout() @@ -243,13 +243,12 @@ func (ap *Autopilot) Run() error { return } - // Trace/Log worker id chosen for this maintenance iteration. + // Log worker id chosen for this maintenance iteration. workerID, err := w.ID(ctx) if err != nil { ap.logger.Errorf("aborting maintenance, failed to fetch worker id, err: %v", err) return } - span.SetAttributes(attribute.String("worker", workerID)) ap.logger.Infof("using worker %s for iteration", workerID) // update the loop state diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 87382d035..80d898046 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -11,14 +11,11 @@ import ( "time" "github.com/montanaflynn/stats" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/codes" rhpv2 "go.sia.tech/core/rhp/v2" rhpv3 "go.sia.tech/core/rhp/v3" "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/hostdb" - "go.sia.tech/renterd/tracing" "go.sia.tech/renterd/wallet" "go.sia.tech/renterd/worker" "go.uber.org/zap" @@ -152,9 +149,6 @@ func (c *contractor) Status() (bool, time.Time) { } func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) (bool, error) { - ctx, span := tracing.Tracer.Start(ctx, "contractor.performContractMaintenance") - defer span.End() - // skip contract maintenance if we're stopped or not synced if c.ap.isStopped() { return false, nil @@ -522,9 +516,6 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, } func (c *contractor) performWalletMaintenance(ctx context.Context) error { - ctx, span := tracing.Tracer.Start(ctx, "contractor.performWalletMaintenance") - defer span.End() - if c.ap.isStopped() { return nil // skip contract maintenance if we're not synced } @@ -805,9 +796,6 @@ func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts } func (c *contractor) runContractFormations(ctx context.Context, w Worker, candidates scoredHosts, usedHosts map[types.PublicKey]struct{}, unusableHosts unusableHostResult, missing uint64, budget *types.Currency) ([]types.FileContractID, error) { - ctx, span := tracing.Tracer.Start(ctx, "runContractFormations") - defer span.End() - if c.ap.isStopped() { return nil, nil } @@ -994,9 +982,6 @@ func (c *contractor) runRevisionBroadcast(ctx context.Context, w Worker, allCont } func (c *contractor) runContractRenewals(ctx context.Context, w Worker, toRenew []contractInfo, budget *types.Currency, limit int) (renewals []renewal, toKeep []contractInfo) { - ctx, span := tracing.Tracer.Start(ctx, "runContractRenewals") - defer span.End() - c.logger.Debugw( "run contracts renewals", "torenew", len(toRenew), @@ -1055,9 +1040,6 @@ func (c *contractor) runContractRenewals(ctx context.Context, w Worker, toRenew } func (c *contractor) runContractRefreshes(ctx context.Context, w Worker, toRefresh []contractInfo, budget *types.Currency) (refreshed []renewal, _ error) { - ctx, span := tracing.Tracer.Start(ctx, "runContractRefreshes") - defer span.End() - c.logger.Debugw( "run contracts refreshes", "torefresh", len(toRefresh), @@ -1323,16 +1305,6 @@ func (c *contractor) renewContract(ctx context.Context, w Worker, ci contractInf if ci.contract.Revision == nil { return api.ContractMetadata{}, true, errors.New("can't renew contract without a revision") } - ctx, span := tracing.Tracer.Start(ctx, "renewContract") - defer span.End() - defer func() { - if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, "failed to renew contract") - } - }() - span.SetAttributes(attribute.Stringer("host", ci.contract.HostKey)) - span.SetAttributes(attribute.Stringer("contract", ci.contract.ID)) // convenience variables state := c.ap.State() @@ -1415,16 +1387,6 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI if ci.contract.Revision == nil { return api.ContractMetadata{}, true, errors.New("can't refresh contract without a revision") } - ctx, span := tracing.Tracer.Start(ctx, "refreshContract") - defer span.End() - defer func() { - if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, "failed to refresh contract") - } - }() - span.SetAttributes(attribute.Stringer("host", ci.contract.HostKey)) - span.SetAttributes(attribute.Stringer("contract", ci.contract.ID)) // convenience variables state := c.ap.State() @@ -1512,19 +1474,9 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI } func (c *contractor) formContract(ctx context.Context, w Worker, host hostdb.Host, minInitialContractFunds, maxInitialContractFunds types.Currency, budget *types.Currency) (cm api.ContractMetadata, proceed bool, err error) { - ctx, span := tracing.Tracer.Start(ctx, "formContract") - defer span.End() - defer func() { - if err != nil { - span.RecordError(err) - span.SetStatus(codes.Error, "failed to form contract") - } - }() - hk := host.PublicKey - span.SetAttributes(attribute.Stringer("host", hk)) - // convenience variables state := c.ap.State() + hk := host.PublicKey // fetch host settings scan, err := w.RHPScan(ctx, hk, host.NetAddress, 0) diff --git a/autopilot/migrator.go b/autopilot/migrator.go index 16b4adda9..cc81c0a73 100644 --- a/autopilot/migrator.go +++ b/autopilot/migrator.go @@ -12,7 +12,6 @@ import ( "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" "go.sia.tech/renterd/stats" - "go.sia.tech/renterd/tracing" "go.uber.org/zap" ) @@ -122,9 +121,6 @@ func (m *migrator) performMigrations(p *workerPool) { m.logger.Info("performing migrations") b := m.ap.bus - ctx, span := tracing.Tracer.Start(m.ap.shutdownCtx, "migrator.performMigrations") - defer span.End() - // prepare a channel to push work to the workers jobs := make(chan job) var wg sync.WaitGroup @@ -141,6 +137,10 @@ func (m *migrator) performMigrations(p *workerPool) { go func(w Worker) { defer wg.Done() + // derive ctx from shutdown ctx + ctx, cancel := context.WithCancel(m.ap.shutdownCtx) + defer cancel() + // fetch worker id once id, err := w.ID(ctx) if err != nil { @@ -196,15 +196,15 @@ OUTER: // recompute health. start := time.Now() - if err := b.RefreshHealth(ctx); err != nil { - m.ap.RegisterAlert(ctx, newRefreshHealthFailedAlert(err)) + if err := b.RefreshHealth(m.ap.shutdownCtx); err != nil { + m.ap.RegisterAlert(m.ap.shutdownCtx, newRefreshHealthFailedAlert(err)) m.logger.Errorf("failed to recompute cached health before migration: %v", err) return } m.logger.Debugf("recomputed slab health in %v", time.Since(start)) // fetch slabs for migration - toMigrateNew, err := b.SlabsForMigration(ctx, m.healthCutoff, set, migratorBatchSize) + toMigrateNew, err := b.SlabsForMigration(m.ap.shutdownCtx, m.healthCutoff, set, migratorBatchSize) if err != nil { m.logger.Errorf("failed to fetch slabs for migration, err: %v", err) return @@ -248,7 +248,7 @@ OUTER: // register an alert to notify users about ongoing migrations. if len(toMigrate) > 0 { - m.ap.RegisterAlert(ctx, newOngoingMigrationsAlert(len(toMigrate), m.slabMigrationEstimate(len(toMigrate)))) + m.ap.RegisterAlert(m.ap.shutdownCtx, newOngoingMigrationsAlert(len(toMigrate), m.slabMigrationEstimate(len(toMigrate)))) } // return if there are no slabs to migrate diff --git a/bus/bus.go b/bus/bus.go index 9380fe215..d929ab271 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -25,7 +25,6 @@ import ( "go.sia.tech/renterd/bus/client" "go.sia.tech/renterd/hostdb" "go.sia.tech/renterd/object" - "go.sia.tech/renterd/tracing" "go.sia.tech/renterd/wallet" "go.sia.tech/renterd/webhooks" "go.sia.tech/siad/modules" @@ -237,7 +236,7 @@ type bus struct { // Handler returns an HTTP handler that serves the bus API. func (b *bus) Handler() http.Handler { - return jape.Mux(tracing.TracingMiddleware("bus", map[string]jape.Handler{ + return jape.Mux(map[string]jape.Handler{ "GET /accounts": b.accountsHandlerGET, "POST /account/:id": b.accountHandlerGET, "POST /account/:id/add": b.accountsAddHandlerPOST, @@ -373,7 +372,7 @@ func (b *bus) Handler() http.Handler { "POST /webhooks": b.webhookHandlerPost, "POST /webhooks/action": b.webhookActionHandlerPost, "POST /webhook/delete": b.webhookHandlerDelete, - })) + }) } // Shutdown shuts down the bus. @@ -2290,10 +2289,6 @@ func New(s Syncer, am *alerts.Manager, hm *webhooks.Manager, cm ChainManager, tp ctx, cancel := context.WithTimeout(context.Background(), 5*time.Minute) defer cancel() - // attach tracing span - ctx, span := tracing.Tracer.Start(ctx, "bus.New") - defer span.End() - // load default settings if the setting is not already set for key, value := range map[string]interface{}{ api.SettingGouging: build.DefaultGougingSettings, diff --git a/cmd/renterd/main.go b/cmd/renterd/main.go index a884a3c42..13fd24e3e 100644 --- a/cmd/renterd/main.go +++ b/cmd/renterd/main.go @@ -25,7 +25,6 @@ import ( "go.sia.tech/renterd/internal/node" "go.sia.tech/renterd/s3" "go.sia.tech/renterd/stores" - "go.sia.tech/renterd/tracing" "go.sia.tech/renterd/wallet" "go.sia.tech/renterd/worker" "go.sia.tech/web/renterd" @@ -60,9 +59,6 @@ var ( Password: os.Getenv("RENTERD_API_PASSWORD"), }, ShutdownTimeout: 5 * time.Minute, - Tracing: config.Tracing{ - InstanceID: "cluster", - }, Database: config.Database{ Log: config.DatabaseLog{ IgnoreRecordNotFoundError: true, @@ -241,8 +237,6 @@ func main() { // node flag.StringVar(&cfg.HTTP.Address, "http", cfg.HTTP.Address, "Address for serving the API") flag.StringVar(&cfg.Directory, "dir", cfg.Directory, "Directory for storing node state") - flag.BoolVar(&cfg.Tracing.Enabled, "tracing-enabled", cfg.Tracing.Enabled, "Enables OpenTelemetry tracing (overrides with RENTERD_TRACING_ENABLED). See OpenTelemetry spec for configuration details.") - flag.StringVar(&cfg.Tracing.InstanceID, "tracing-service-instance-id", cfg.Tracing.InstanceID, "Service instance ID for tracing (overrides with RENTERD_TRACING_SERVICE_INSTANCE_ID)") flag.StringVar(&cfg.Log.Path, "log-path", cfg.Log.Path, "Path for logs (overrides with RENTERD_LOG_PATH)") // db @@ -311,9 +305,6 @@ func main() { // Overwrite flags from environment if set. parseEnvVar("RENTERD_LOG_PATH", &cfg.Log.Path) - parseEnvVar("RENTERD_TRACING_ENABLED", &cfg.Tracing.Enabled) - parseEnvVar("RENTERD_TRACING_SERVICE_INSTANCE_ID", &cfg.Tracing.InstanceID) - parseEnvVar("RENTERD_BUS_REMOTE_ADDR", &cfg.Bus.RemoteAddr) parseEnvVar("RENTERD_BUS_API_PASSWORD", &cfg.Bus.RemotePassword) parseEnvVar("RENTERD_BUS_GATEWAY_ADDR", &cfg.Bus.GatewayAddr) @@ -422,18 +413,6 @@ func main() { } var shutdownFns []shutdownFn - // Init tracing. - if cfg.Tracing.Enabled { - fn, err := tracing.Init(cfg.Tracing.InstanceID) - if err != nil { - logger.Fatal("failed to init tracing: " + err.Error()) - } - shutdownFns = append(shutdownFns, shutdownFn{ - name: "Tracing", - fn: fn, - }) - } - if cfg.Bus.RemoteAddr != "" && len(cfg.Worker.Remotes) != 0 && !cfg.Autopilot.Enabled { logger.Fatal("remote bus, remote worker, and no autopilot -- nothing to do!") } diff --git a/config/config.go b/config/config.go index c841bbfc3..77655ebcf 100644 --- a/config/config.go +++ b/config/config.go @@ -21,7 +21,6 @@ type ( Autopilot Autopilot `yaml:"autopilot"` Database Database `yaml:"database"` - Tracing Tracing `yaml:"tracing"` } // HTTP contains the configuration for the HTTP server. @@ -41,12 +40,6 @@ type ( MySQL MySQL `yaml:"mysql"` } - // Tracing contains the configuration for tracing. - Tracing struct { - Enabled bool `yaml:"enabled"` - InstanceID string `yaml:"instanceID"` - } - // Bus contains the configuration for a bus. Bus struct { AnnouncementMaxAgeHours uint64 `yaml:"announcementMaxAgeHours"` diff --git a/go.mod b/go.mod index 62990c342..168458edf 100644 --- a/go.mod +++ b/go.mod @@ -11,12 +11,6 @@ require ( github.com/minio/minio-go/v7 v7.0.65 github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe - go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 - go.opentelemetry.io/otel v1.21.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 - go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 - go.opentelemetry.io/otel/sdk v1.21.0 - go.opentelemetry.io/otel/trace v1.21.0 go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 go.sia.tech/hostd v0.2.2 @@ -37,20 +31,14 @@ require ( require ( github.com/aead/chacha20 v0.0.0-20180709150244-8b13a72661da // indirect github.com/aws/aws-sdk-go v1.49.1 // indirect - github.com/cenkalti/backoff/v4 v4.2.1 // indirect github.com/cloudflare/cloudflare-go v0.75.0 // indirect github.com/dchest/threefish v0.0.0-20120919164726-3ecf4c494abf // indirect github.com/dustin/go-humanize v1.0.1 // indirect - github.com/felixge/httpsnoop v1.0.4 // indirect - github.com/go-logr/logr v1.3.0 // indirect - github.com/go-logr/stdr v1.2.2 // indirect github.com/go-sql-driver/mysql v1.7.1 // indirect github.com/goccy/go-json v0.10.2 // indirect - github.com/golang/protobuf v1.5.3 // indirect github.com/google/go-querystring v1.1.0 // indirect github.com/google/uuid v1.5.0 // indirect github.com/gorilla/websocket v1.5.1 // indirect - github.com/grpc-ecosystem/grpc-gateway/v2 v2.18.1 // indirect github.com/hashicorp/go-cleanhttp v0.5.2 // indirect github.com/hashicorp/go-retryablehttp v0.7.5 // indirect github.com/hashicorp/golang-lru/v2 v2.0.7 // indirect @@ -84,8 +72,6 @@ require ( gitlab.com/NebulousLabs/ratelimit v0.0.0-20200811080431-99b8f0768b2e // indirect gitlab.com/NebulousLabs/siamux v0.0.2-0.20220630142132-142a1443a259 // indirect gitlab.com/NebulousLabs/threadgroup v0.0.0-20200608151952-38921fbef213 // indirect - go.opentelemetry.io/otel/metric v1.21.0 // indirect - go.opentelemetry.io/proto/otlp v1.0.0 // indirect go.sia.tech/web v0.0.0-20231213145933-3f175a86abff // indirect go.uber.org/multierr v1.11.0 // indirect golang.org/x/net v0.19.0 // indirect @@ -93,10 +79,6 @@ require ( golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.16.1 // indirect - google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0 // indirect - google.golang.org/genproto/googleapis/rpc v0.0.0-20231212172506-995d672761c0 // indirect - google.golang.org/grpc v1.60.0 // indirect - google.golang.org/protobuf v1.31.0 // indirect gopkg.in/ini.v1 v1.67.0 // indirect nhooyr.io/websocket v1.8.10 // indirect ) diff --git a/go.sum b/go.sum index d7d313ed2..f6ad99546 100644 --- a/go.sum +++ b/go.sum @@ -13,8 +13,6 @@ github.com/aws/aws-sdk-go v1.49.1 h1:Dsamcd8d/nNb3A+bZ0ucfGl0vGZsW5wlRW0vhoYGoeQ github.com/aws/aws-sdk-go v1.49.1/go.mod h1:LF8svs817+Nz+DmiMQKTO3ubZ/6IaTpq3TjupRn3Eqk= 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= -github.com/cenkalti/backoff/v4 v4.2.1 h1:y4OZtCnogmCPw98Zjyt5a6+QwPLGkiQsYW5oUqylYbM= -github.com/cenkalti/backoff/v4 v4.2.1/go.mod h1:Y3VNntkOUPxTVeUxJ/G5vcM//AlwfmyYozVcomhLiZE= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/cloudflare/cloudflare-go v0.75.0 h1:03a4EkwwsDo0yAHjQ/l+D36K9wTkvr0afDiI/uHQ0Xw= @@ -35,8 +33,6 @@ github.com/dgryski/go-sip13 v0.0.0-20181026042036-e10d5fee7954/go.mod h1:vAd38F8 github.com/dustin/go-humanize v1.0.1 h1:GzkhY7T5VNhEkwH0PVJgjz+fX1rhBrR7pRT3mDkpeCY= github.com/dustin/go-humanize v1.0.1/go.mod h1:Mu1zIs6XwVuF/gI1OepvI0qD18qycQx+mFykh5fBlto= github.com/fatih/color v1.13.0 h1:8LOYc1KYPPmyKMuN8QV2DNRWNbLo6LZ0iLs8+mlH53w= -github.com/felixge/httpsnoop v1.0.4 h1:NFTV2Zj1bL4mc9sqWACXbQFVBBg2W3GPvqp8/ESS2Wg= -github.com/felixge/httpsnoop v1.0.4/go.mod h1:m8KPJKqk1gH5J9DgRY2ASl2lWCfGKXixSwevea8zH2U= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= github.com/gabriel-vasile/mimetype v1.4.3 h1:in2uUcidCuFcDKtdcBxlR0rJ1+fsokWf+uqxgUFjbI0= github.com/gabriel-vasile/mimetype v1.4.3/go.mod h1:d8uq/6HKRL6CGdk+aubisF/M5GcPfT7nKyLpA0lbSSk= @@ -46,11 +42,6 @@ github.com/go-gormigrate/gormigrate/v2 v2.1.1/go.mod h1:L7nJ620PFDKei9QOhJzqA8kR github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2as= github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-logfmt/logfmt v0.4.0/go.mod h1:3RMwSq7FuexP4Kalkev3ejPJsZTpXXBr9+V4qmtdjCk= -github.com/go-logr/logr v1.2.2/go.mod h1:jdQByPbusPIv2/zmleS9BjJVeZ6kBagPoEUsqbVz/1A= -github.com/go-logr/logr v1.3.0 h1:2y3SDp0ZXuc6/cjLSZ+Q3ir+QB9T/iG5yYRXqsagWSY= -github.com/go-logr/logr v1.3.0/go.mod h1:9T104GzyrTigFIr8wt5mBrctHMim0Nb2HLGrmQ40KvY= -github.com/go-logr/stdr v1.2.2 h1:hSWxHoqTgW2S2qGc0LTAI563KZ5YKYRhT3MFKZMbjag= -github.com/go-logr/stdr v1.2.2/go.mod h1:mMo/vtBO5dYbehREoey6XUKy/eSumjCCveDpRre4VKE= github.com/go-sql-driver/mysql v1.7.0/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= github.com/go-sql-driver/mysql v1.7.1 h1:lUIinVbN1DY0xBg0eMOzmmtGoHwWBbvnWubQUrtU8EI= github.com/go-sql-driver/mysql v1.7.1/go.mod h1:OXbVy3sEdcQ2Doequ6Z5BW6fXNQTmx+9S1MCJN5yJMI= @@ -64,13 +55,9 @@ github.com/golang/groupcache v0.0.0-20190129154638-5b532d6fd5ef/go.mod h1:cIg4er github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= -github.com/golang/protobuf v1.5.0/go.mod h1:FsONVRAS9T7sI+LIUmWTfcYkHO4aIWwzhcaSAoJOfIk= -github.com/golang/protobuf v1.5.3 h1:KhyjKVUg7Usr/dYsdSqoFveMYd5ko72D+zANwlG1mmg= -github.com/golang/protobuf v1.5.3/go.mod h1:XVQd3VNwM+JqD3oG2Ue2ip4fOMUkwXdXDdiuN0vRsmY= github.com/google/btree v1.0.0/go.mod h1:lNA+9X1NB3Zf8V7Ke586lFgjr2dZNuvo3lPJSGZ5JPQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= github.com/google/go-cmp v0.5.2/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= -github.com/google/go-cmp v0.5.5/go.mod h1:v8dTdLbMG2kIc/vJvl+f65V22dbkXbowE6jgT/gNBxE= github.com/google/go-cmp v0.6.0 h1:ofyhxvXcZhMsU5ulbFiLKl/XBFqE1GSq7atu8tAmTRI= github.com/google/go-cmp v0.6.0/go.mod h1:17dUlkBOakJ0+DkrSSNjCkIjxS6bF9zb3elmeNGIjoY= github.com/google/go-querystring v1.1.0 h1:AnCroh3fv4ZBgVIf1Iwtovgjaw/GiKJo8M8yD/fhyJ8= @@ -87,8 +74,6 @@ github.com/gotd/contrib v0.19.0/go.mod h1:LzPxzRF0FvtpBt/WyODWQnPpk0tm/G9z6RHUoP github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= github.com/grpc-ecosystem/go-grpc-prometheus v1.2.0/go.mod h1:8NvIoxWQoOIhqOTXgfV/d3M/q6VIi02HzZEHgUlZvzk= github.com/grpc-ecosystem/grpc-gateway v1.9.0/go.mod h1:vNeuVxBJEsws4ogUvrchl83t/GYV9WGTSLVdBhOQFDY= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.18.1 h1:6UKoz5ujsI55KNpsJH3UwCq3T8kKbZwNZBNPuTTje8U= -github.com/grpc-ecosystem/grpc-gateway/v2 v2.18.1/go.mod h1:YvJ2f6MplWDhfxiUC3KpyTy76kYUZA4W3pTv/wdKQ9Y= github.com/hanwen/go-fuse v1.0.0/go.mod h1:unqXarDXqzAk0rt98O2tVndEPIpUgLD9+rwFisZH3Ok= github.com/hanwen/go-fuse/v2 v2.1.0/go.mod h1:oRyA5eK+pvJyv5otpO/DgccS8y/RvYMaO00GgRLGryc= github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9neXJWAZQ= @@ -246,22 +231,6 @@ gitlab.com/NebulousLabs/threadgroup v0.0.0-20200608151952-38921fbef213/go.mod h1 gitlab.com/NebulousLabs/writeaheadlog v0.0.0-20200618142844-c59a90f49130/go.mod h1:SxigdS5Q1ui+OMgGAXt1E/Fg3RB6PvKXMov2O3gvIzs= go.etcd.io/bbolt v1.3.2/go.mod h1:IbVyRI1SCnLcuJnV2u8VeU0CEYM7e686BmAb1XKL+uU= go.etcd.io/bbolt v1.3.5/go.mod h1:G5EMThwa9y8QZGBClrRx5EY+Yw9kAhnjy3bSjsnlVTQ= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1 h1:aFJWCqJMNjENlcleuuOkGAPH82y0yULBScfXcIEdS24= -go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.46.1/go.mod h1:sEGXWArGqc3tVa+ekntsN65DmVbVeW+7lTKTjZF3/Fo= -go.opentelemetry.io/otel v1.21.0 h1:hzLeKBZEL7Okw2mGzZ0cc4k/A7Fta0uoPgaJCr8fsFc= -go.opentelemetry.io/otel v1.21.0/go.mod h1:QZzNPQPm1zLX4gZK4cMi+71eaorMSGT3A4znnUvNNEo= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0 h1:cl5P5/GIfFh4t6xyruOgJP5QiA1pw4fYYdv6nc6CBWw= -go.opentelemetry.io/otel/exporters/otlp/otlptrace v1.21.0/go.mod h1:zgBdWWAu7oEEMC06MMKc5NLbA/1YDXV1sMpSqEeLQLg= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0 h1:digkEZCJWobwBqMwC0cwCq8/wkkRy/OowZg5OArWZrM= -go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp v1.21.0/go.mod h1:/OpE/y70qVkndM0TrxT4KBoN3RsFZP0QaofcfYrj76I= -go.opentelemetry.io/otel/metric v1.21.0 h1:tlYWfeo+Bocx5kLEloTjbcDwBuELRrIFxwdQ36PlJu4= -go.opentelemetry.io/otel/metric v1.21.0/go.mod h1:o1p3CA8nNHW8j5yuQLdc1eeqEaPfzug24uvsyIEJRWM= -go.opentelemetry.io/otel/sdk v1.21.0 h1:FTt8qirL1EysG6sTQRZ5TokkU8d0ugCj8htOgThZXQ8= -go.opentelemetry.io/otel/sdk v1.21.0/go.mod h1:Nna6Yv7PWTdgJHVRD9hIYywQBRx7pbox6nwBnZIxl/E= -go.opentelemetry.io/otel/trace v1.21.0 h1:WD9i5gzvoUPuXIXH24ZNBudiarZDKuekPqi/E8fpfLc= -go.opentelemetry.io/otel/trace v1.21.0/go.mod h1:LGbsEB0f9LGjN+OZaQQ26sohbOmiMR+BaslueVtS/qQ= -go.opentelemetry.io/proto/otlp v1.0.0 h1:T0TX0tmXU8a3CbNXzEKGeU5mIVOdf0oykP+u2lIVU/I= -go.opentelemetry.io/proto/otlp v1.0.0/go.mod h1:Sy6pihPLfYHkr3NkUbEhGHFhINUSI/v80hjKIs5JXpM= go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b h1:xJSxYN2kZD3NAijHIwjXhG5+7GoPyjDNIJPEoD3b72g= go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b/go.mod h1:3EoY+rR78w1/uGoXXVqcYdwSjSJKuEMI5bL7WROA27Q= go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 h1:ulzfJNjxN5DjXHClkW2pTiDk+eJ+0NQhX87lFDZ03t0= @@ -390,19 +359,8 @@ golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8T golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20231211222908-989df2bf70f3 h1:1hfbdAfFbkmpg41000wDVqr7jUpK/Yo+LPnIxxGzmkg= -google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0 h1:s1w3X6gQxwrLEpxnLd/qXTVLgQE2yXwaOaoa6IlY/+o= -google.golang.org/genproto/googleapis/api v0.0.0-20231212172506-995d672761c0/go.mod h1:CAny0tYF+0/9rmDB9fahA9YLzX3+AEVl1qXbv5hhj6c= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231212172506-995d672761c0 h1:/jFB8jK5R3Sq3i/lmeZO0cATSzFfZaJq1J2Euan3XKU= -google.golang.org/genproto/googleapis/rpc v0.0.0-20231212172506-995d672761c0/go.mod h1:FUoWkonphQm3RhTS+kOEhF8h0iDpm4tdXolVCeZ9KKA= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= google.golang.org/grpc v1.21.0/go.mod h1:oYelfM1adQP15Ek0mdvEgi9Df8B9CZIaU1084ijfRaM= -google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k= -google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM= -google.golang.org/protobuf v1.26.0-rc.1/go.mod h1:jlhhOSvTdKEhbULTjvd4ARK9grFBp09yW+WbY/TyQbw= -google.golang.org/protobuf v1.26.0/go.mod h1:9q0QmTI4eRPtz6boOQmLYwt+qCgq0jsYwAQnmE0givc= -google.golang.org/protobuf v1.31.0 h1:g0LDEJHgrBl9N9r17Ru3sqWhkIx2NB67okBHPwC7hs8= -google.golang.org/protobuf v1.31.0/go.mod h1:HV8QOd/L58Z+nl8r43ehVNZIU/HEI6OcFqwMG9pJV4I= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= diff --git a/tracing/tracing.go b/tracing/tracing.go deleted file mode 100644 index 8dbf75cc2..000000000 --- a/tracing/tracing.go +++ /dev/null @@ -1,83 +0,0 @@ -package tracing - -import ( - "context" - "fmt" - "net/http" - - "go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp" - "go.opentelemetry.io/otel" - "go.opentelemetry.io/otel/exporters/otlp/otlptrace" - "go.opentelemetry.io/otel/exporters/otlp/otlptrace/otlptracehttp" - "go.opentelemetry.io/otel/propagation" - "go.opentelemetry.io/otel/sdk/resource" - sdktrace "go.opentelemetry.io/otel/sdk/trace" - semconv "go.opentelemetry.io/otel/semconv/v1.4.0" - "go.opentelemetry.io/otel/trace/noop" - "go.sia.tech/jape" -) - -const ( - service = "renterd" - serviceVersion = "0.1.0" -) - -var ( - Tracer = noop.NewTracerProvider().Tracer("noop") -) - -// Init initialises a new OpenTelemetry Tracer using information from the -// environment and process. For more information on available environment -// variables for configuration, check out -// https://opentelemetry.io/docs/reference/specification/sdk-environment-variables/. -// https://github.com/open-telemetry/opentelemetry-go/tree/main/exporters/otlp/otlptrace -func Init(serviceInstanceId string) (func(ctx context.Context) error, error) { - // Create resources. - resources := resource.NewWithAttributes( - semconv.SchemaURL, - semconv.ServiceNameKey.String(service), - semconv.ServiceVersionKey.String(serviceVersion), - semconv.ServiceInstanceIDKey.String(serviceInstanceId), - ) - - // Create exporter. - client := otlptracehttp.NewClient() - exporter, err := otlptrace.New(context.Background(), client) - if err != nil { - return nil, err - } - - // Create provider - provider := sdktrace.NewTracerProvider( - sdktrace.WithSampler(sdktrace.AlwaysSample()), - sdktrace.WithResource(resources), - sdktrace.WithBatcher(exporter), - ) - otel.SetTracerProvider(provider) - - // Set TextMapPropagator. That's the component that defines how contexts are - // propagated over http. - otel.SetTextMapPropagator(propagation.TraceContext{}) - - // Set global tracer. - Tracer = otel.Tracer(service) - - // Overwrite the default transport to make sure all requests attach tracing - // headers. - http.DefaultTransport = otelhttp.NewTransport(http.DefaultTransport) - - return provider.Shutdown, nil -} - -// TracingMiddleware attaches a tracing handler to http routes. -func TracingMiddleware(component string, routes map[string]jape.Handler) map[string]jape.Handler { - adapt := func(route string, h jape.Handler) jape.Handler { - return jape.Adapt(func(h http.Handler) http.Handler { - return otelhttp.NewHandler(h, fmt.Sprintf("%s: %s", component, route)) - })(h) - } - for route, handler := range routes { - routes[route] = adapt(route, handler) - } - return routes -} diff --git a/worker/download.go b/worker/download.go index 6092f7d13..816f2113c 100644 --- a/worker/download.go +++ b/worker/download.go @@ -13,14 +13,11 @@ import ( "sync/atomic" "time" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" "go.sia.tech/renterd/stats" - "go.sia.tech/renterd/tracing" "go.uber.org/zap" "lukechampine.com/frand" ) @@ -197,13 +194,6 @@ func newDownloader(ctx context.Context, host Host) *downloader { } func (mgr *downloadManager) DownloadObject(ctx context.Context, w io.Writer, o object.Object, offset, length uint64, contracts []api.ContractMetadata) (err error) { - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "download") - defer func() { - span.RecordError(err) - span.End() - }() - // create identifier id := newID() @@ -582,10 +572,6 @@ func (mgr *downloadManager) newSlabDownload(ctx context.Context, dID id, slice o } func (mgr *downloadManager) downloadSlab(ctx context.Context, dID id, slice object.SlabSlice, migration bool) ([][]byte, bool, error) { - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "downloadSlab") - defer span.End() - // prepare new download slab := mgr.newSlabDownload(ctx, dID, slice, migration) @@ -655,7 +641,7 @@ func (d *downloader) processBatch(batch []*sectorDownloadReq) chan struct{} { // check if we need to abort select { case <-d.shutdownCtx.Done(): - break + return default: } @@ -768,11 +754,6 @@ func (d *downloader) estimate() float64 { } func (d *downloader) enqueue(download *sectorDownloadReq) { - // add tracing - span := trace.SpanFromContext(download.ctx) - span.SetAttributes(attribute.Float64("estimate", d.estimate())) - span.AddEvent("enqueued") - // enqueue the job d.mu.Lock() d.queue = append(d.queue, download) @@ -819,17 +800,6 @@ func (d *downloader) trackFailure(err error) { } func (d *downloader) execute(req *sectorDownloadReq) (err error) { - // add tracing - start := time.Now() - span := trace.SpanFromContext(req.ctx) - span.AddEvent("execute") - defer func() { - elapsed := time.Since(start) - span.SetAttributes(attribute.Int64("duration", elapsed.Milliseconds())) - span.RecordError(err) - span.End() - }() - // download the sector buf := bytes.NewBuffer(make([]byte, 0, req.length)) err = d.host.DownloadSector(req.ctx, buf, req.root, req.offset, req.length, req.overpay) @@ -966,15 +936,9 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, sector := s.hostToSectors[s.curr][0] s.hostToSectors[s.curr] = s.hostToSectors[s.curr][1:] - // create the span - sCtx, span := tracing.Tracer.Start(ctx, "sectorDownloadReq") - span.SetAttributes(attribute.Stringer("hk", sector.LatestHost)) - span.SetAttributes(attribute.Bool("overdrive", overdrive)) - span.SetAttributes(attribute.Int("sector", sector.index)) - // build the request return §orDownloadReq{ - ctx: sCtx, + ctx: ctx, offset: s.offset, length: s.length, @@ -999,10 +963,6 @@ func (s *slabDownload) download(ctx context.Context) ([][]byte, bool, error) { ctx, cancel := context.WithCancel(ctx) defer cancel() - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "download") - defer span.End() - // create the responses queue resps := §orResponses{ c: make(chan struct{}, 1), @@ -1164,9 +1124,6 @@ func (s *slabDownload) launch(req *sectorDownloadReq) error { // launch the req err := s.mgr.launch(req) if err != nil { - span := trace.SpanFromContext(req.ctx) - span.RecordError(err) - span.End() return err } diff --git a/worker/interactions.go b/worker/interactions.go index b1e0203d3..2321ebb30 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -8,7 +8,6 @@ import ( "go.sia.tech/jape" "go.sia.tech/renterd/hostdb" - "go.sia.tech/renterd/tracing" ) const ( @@ -77,18 +76,14 @@ func (w *worker) tryFlushInteractionsBuffer() { // flushInteractions flushes the worker's interaction buffer to the bus. func (w *worker) flushInteractions() { if len(w.interactionsScans) > 0 { - ctx, span := tracing.Tracer.Start(w.shutdownCtx, "worker: recordHostScans") - defer span.End() - if err := w.bus.RecordHostScans(ctx, w.interactionsScans); err != nil { + if err := w.bus.RecordHostScans(w.shutdownCtx, w.interactionsScans); err != nil { w.logger.Errorw(fmt.Sprintf("failed to record scans: %v", err)) } else { w.interactionsScans = nil } } if len(w.interactionsPriceTableUpdates) > 0 { - ctx, span := tracing.Tracer.Start(w.shutdownCtx, "worker: recordPriceTableUpdates") - defer span.End() - if err := w.bus.RecordPriceTables(ctx, w.interactionsPriceTableUpdates); err != nil { + if err := w.bus.RecordPriceTables(w.shutdownCtx, w.interactionsPriceTableUpdates); err != nil { w.logger.Errorw(fmt.Sprintf("failed to record price table updates: %v", err)) } else { w.interactionsPriceTableUpdates = nil diff --git a/worker/migrations.go b/worker/migrations.go index 47af03c08..05bd20193 100644 --- a/worker/migrations.go +++ b/worker/migrations.go @@ -8,13 +8,9 @@ import ( "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" - "go.sia.tech/renterd/tracing" ) func (w *worker) migrate(ctx context.Context, s *object.Slab, contractSet string, dlContracts, ulContracts []api.ContractMetadata, bh uint64) (int, bool, error) { - ctx, span := tracing.Tracer.Start(ctx, "migrateSlab") - defer span.End() - // make a map of good hosts goodHosts := make(map[types.PublicKey]map[types.FileContractID]bool) for _, c := range ulContracts { diff --git a/worker/spending.go b/worker/spending.go index 565748b7e..ec09e803a 100644 --- a/worker/spending.go +++ b/worker/spending.go @@ -8,7 +8,6 @@ import ( "go.sia.tech/core/types" "go.sia.tech/renterd/api" - "go.sia.tech/renterd/tracing" "go.uber.org/zap" ) @@ -78,13 +77,11 @@ func (sr *contractSpendingRecorder) Record(rev types.FileContractRevision, cs ap func (sr *contractSpendingRecorder) flush() { if len(sr.contractSpendings) > 0 { - ctx, span := tracing.Tracer.Start(sr.shutdownCtx, "worker: flushContractSpending") - defer span.End() records := make([]api.ContractSpendingRecord, 0, len(sr.contractSpendings)) for _, cs := range sr.contractSpendings { records = append(records, cs) } - if err := sr.bus.RecordContractSpending(ctx, records); err != nil { + if err := sr.bus.RecordContractSpending(sr.shutdownCtx, records); err != nil { sr.logger.Errorw(fmt.Sprintf("failed to record contract spending: %v", err)) } else { sr.contractSpendings = make(map[types.FileContractID]api.ContractSpendingRecord) diff --git a/worker/upload.go b/worker/upload.go index d54e7f395..b9a3d706d 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -12,14 +12,11 @@ import ( "sync" "time" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" "go.sia.tech/renterd/stats" - "go.sia.tech/renterd/tracing" "go.uber.org/zap" ) @@ -352,13 +349,6 @@ func (mgr *uploadManager) MigrateShards(ctx context.Context, s *object.Slab, sha ctx, cancel := context.WithCancel(ctx) defer cancel() - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "MigrateShards") - defer func() { - span.RecordError(err) - span.End() - }() - // create the upload upload, err := mgr.newUpload(ctx, len(shards), contracts, bh, lockPriority) if err != nil { @@ -451,13 +441,6 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a ctx, cancel := context.WithCancel(ctx) defer cancel() - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "Upload") - defer func() { - span.RecordError(err) - span.End() - }() - // create the object o := object.NewObject(up.ec) @@ -625,13 +608,6 @@ func (mgr *uploadManager) UploadPackedSlab(ctx context.Context, rs api.Redundanc ctx, cancel := context.WithCancel(ctx) defer cancel() - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "UploadPackedSlab") - defer func() { - span.RecordError(err) - span.End() - }() - // build the shards shards := encryptPartialSlab(ps.Data, ps.Key, uint8(rs.MinShards), uint8(rs.TotalShards)) @@ -776,11 +752,6 @@ func (u *upload) newSlabUpload(ctx context.Context, shards [][]byte, uploaders [ // create the ctx sCtx, sCancel := context.WithCancel(ctx) - // attach the upload's span - sCtx, span := tracing.Tracer.Start(sCtx, "uploadSector") - span.SetAttributes(attribute.Bool("overdrive", false)) - span.SetAttributes(attribute.Int("sector", sI)) - // create the sector sectors[sI] = §orUpload{ data: (*[rhpv2.SectorSize]byte)(shard), @@ -813,10 +784,6 @@ func (u *upload) newSlabUpload(ctx context.Context, shards [][]byte, uploaders [ } func (u *upload) uploadSlab(ctx context.Context, rs api.RedundancySettings, data []byte, length, index int, respChan chan slabUploadResponse, candidates []*uploader, mem Memory, maxOverdrive uint64, overdriveTimeout time.Duration) (uploadSpeed int64, overdrivePct float64) { - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "uploadSlab") - defer span.End() - // create the response resp := slabUploadResponse{ slab: object.SlabSlice{ @@ -847,10 +814,6 @@ func (u *upload) uploadSlab(ctx context.Context, rs api.RedundancySettings, data func (u *upload) uploadShards(ctx context.Context, shards [][]byte, candidates []*uploader, mem Memory, maxOverdrive uint64, overdriveTimeout time.Duration) (sectors []object.Sector, uploadSpeed int64, overdrivePct float64, err error) { start := time.Now() - // add tracing - ctx, span := tracing.Tracer.Start(ctx, "uploadShards") - defer span.End() - // ensure inflight uploads get cancelled ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -954,9 +917,6 @@ loop: } overdrivePct = float64(numOverdrive) / float64(slab.numSectors) - // register the amount of overdrive sectors - span.SetAttributes(attribute.Int("overdrive", int(numOverdrive))) - if slab.numUploaded < slab.numSectors { remaining := slab.numSectors - slab.numUploaded err = fmt.Errorf("failed to upload slab: launched=%d uploaded=%d remaining=%d inflight=%d pending=%d uploaders=%d errors=%d %w", slab.numLaunched, slab.numUploaded, remaining, slab.numInflight, len(buffer), len(slab.candidates), len(slab.errs), slab.errs) @@ -1008,11 +968,7 @@ func (s *slabUpload) launch(req *sectorUploadReq) error { // no candidate found if candidate == nil { - err := errNoCandidateUploader - span := trace.SpanFromContext(req.sector.ctx) - span.RecordError(err) - span.End() - return err + return errNoCandidateUploader } // update the candidate diff --git a/worker/uploader.go b/worker/uploader.go index aef8a5900..c37040569 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -8,8 +8,6 @@ import ( "sync" "time" - "go.opentelemetry.io/otel/attribute" - "go.opentelemetry.io/otel/trace" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" "go.sia.tech/renterd/api" @@ -160,11 +158,6 @@ func (u *uploader) UpdateBlockHeight(bh uint64) { } func (u *uploader) enqueue(req *sectorUploadReq) { - // trace the request - span := trace.SpanFromContext(req.sector.ctx) - span.SetAttributes(attribute.Stringer("hk", u.hk)) - span.AddEvent("enqueued") - // decorate the request req.fcid = u.ContractID() req.hk = u.hk @@ -199,29 +192,13 @@ func (u *uploader) execute(req *sectorUploadReq, rev types.FileContractRevision) fcid := u.fcid u.mu.Unlock() - // fetch span from context - span := trace.SpanFromContext(req.sector.ctx) - span.AddEvent("execute") - // update the bus if err := u.os.AddUploadingSector(req.sector.ctx, req.uploadID, fcid, req.sector.root); err != nil { return types.Hash256{}, fmt.Errorf("failed to add uploading sector to contract %v, err: %v", fcid, err) } // upload the sector - start := time.Now() - root, err := host.UploadSector(req.sector.ctx, req.sector.data, rev) - if err != nil { - return types.Hash256{}, err - } - - // update span - elapsed := time.Since(start) - span.SetAttributes(attribute.Int64("duration", elapsed.Milliseconds())) - span.RecordError(err) - span.End() - - return root, nil + return host.UploadSector(req.sector.ctx, req.sector.data, rev) } func (u *uploader) pop() *sectorUploadReq { diff --git a/worker/worker.go b/worker/worker.go index 932366b96..a06bf5617 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -25,7 +25,6 @@ import ( "go.sia.tech/renterd/build" "go.sia.tech/renterd/hostdb" "go.sia.tech/renterd/object" - "go.sia.tech/renterd/tracing" "go.sia.tech/renterd/webhooks" "go.sia.tech/renterd/worker/client" "go.sia.tech/siad/modules" @@ -1311,7 +1310,7 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush // Handler returns an HTTP handler that serves the worker API. func (w *worker) Handler() http.Handler { - return jape.Mux(tracing.TracingMiddleware("worker", interactionMiddleware(w, map[string]jape.Handler{ + return jape.Mux(interactionMiddleware(w, map[string]jape.Handler{ "GET /account/:hostkey": w.accountHandlerGET, "GET /id": w.idHandlerGET, @@ -1339,7 +1338,7 @@ func (w *worker) Handler() http.Handler { "PUT /multipart/*path": w.multipartUploadHandlerPUT, "GET /state": w.stateHandlerGET, - }))) + })) } // Shutdown shuts down the worker. @@ -1425,17 +1424,11 @@ func discardTxnOnErr(ctx context.Context, bus Bus, l *zap.SugaredLogger, txn typ return } - // add timeout ctx, cancel := context.WithTimeout(ctx, 10*time.Second) - defer cancel() - - // start trace - ctx, span := tracing.Tracer.Start(ctx, "discardTxn") - defer span.End() - - if err := bus.WalletDiscard(ctx, txn); err != nil { - l.Errorf("%v: failed to discard txn: %v", err) + if dErr := bus.WalletDiscard(ctx, txn); dErr != nil { + l.Errorf("%w: failed to discard txn: %v", *err, dErr) } + cancel() } func isErrDuplicateTransactionSet(err error) bool { From c5506828dea00cdbed8dd84c4902ffe6c0b93ffb Mon Sep 17 00:00:00 2001 From: Peter-Jan Brone Date: Fri, 15 Dec 2023 16:09:43 +0100 Subject: [PATCH 07/24] Contract Mock (#823) * worker: add upload_test.go * worker: update * worker: add default bucket * worker: add bucket test * worker: extend TestUploadDownload * worker: fix ineffective break statement * worker: implement MR remarks * worker: move host mocks --- worker/download.go | 25 ++-- worker/host_test.go | 76 +----------- worker/mocks_test.go | 272 ++++++++++++++++++++++++++++++++++++++++++ worker/upload.go | 97 +++++++++------ worker/upload_test.go | 140 ++++++++++++++++++++++ worker/uploader.go | 63 +++++++--- worker/worker.go | 8 +- 7 files changed, 533 insertions(+), 148 deletions(-) create mode 100644 worker/mocks_test.go create mode 100644 worker/upload_test.go diff --git a/worker/download.go b/worker/download.go index 6092f7d13..fc9c5d2cd 100644 --- a/worker/download.go +++ b/worker/download.go @@ -34,6 +34,7 @@ const ( var ( errDownloadManagerStopped = errors.New("download manager stopped") + errDownloadNotEnoughHosts = errors.New("not enough hosts available to download the slab") ) type ( @@ -301,7 +302,7 @@ func (mgr *downloadManager) DownloadObject(ctx context.Context, w io.Writer, o o } } if available < next.MinShards { - responseChan <- &slabDownloadResponse{err: fmt.Errorf("not enough hosts available to download the slab: %v/%v", available, next.MinShards)} + responseChan <- &slabDownloadResponse{err: fmt.Errorf("%w: %v/%v", errDownloadNotEnoughHosts, available, next.MinShards)} return } @@ -655,7 +656,7 @@ func (d *downloader) processBatch(batch []*sectorDownloadReq) chan struct{} { // check if we need to abort select { case <-d.shutdownCtx.Done(): - break + return default: } @@ -944,22 +945,24 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, // prepare next sectors to download if len(s.hostToSectors[s.curr]) == 0 { - // grab unused hosts + // select all possible hosts var hosts []types.PublicKey - for host := range s.hostToSectors { - if _, used := s.used[host]; !used { + for host, sectors := range s.hostToSectors { + if len(sectors) == 0 { + continue // ignore hosts with no more sectors + } else if _, used := s.used[host]; !used { hosts = append(hosts, host) } } - // make the fastest host the current host - s.curr = s.mgr.fastest(hosts) - s.used[s.curr] = struct{}{} - // no more sectors to download - if len(s.hostToSectors[s.curr]) == 0 { + if len(hosts) == 0 { return nil } + + // make the fastest host the current host + s.curr = s.mgr.fastest(hosts) + s.used[s.curr] = struct{}{} } // pop the next sector @@ -1016,7 +1019,7 @@ func (s *slabDownload) download(ctx context.Context) ([][]byte, bool, error) { for i := 0; i < int(s.minShards); { req := s.nextRequest(ctx, resps, false) if req == nil { - return nil, false, fmt.Errorf("no hosts available") + return nil, false, fmt.Errorf("no host available for shard %d", i) } else if err := s.launch(req); err == nil { i++ } diff --git a/worker/host_test.go b/worker/host_test.go index 15e272ca6..19ddf0421 100644 --- a/worker/host_test.go +++ b/worker/host_test.go @@ -4,73 +4,15 @@ import ( "bytes" "context" "errors" - "io" "testing" - "time" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" - "go.sia.tech/renterd/api" - "go.sia.tech/renterd/hostdb" - "lukechampine.com/frand" ) -type ( - mockHost struct { - hk types.PublicKey - fcid types.FileContractID - sectors map[types.Hash256]*[rhpv2.SectorSize]byte - } -) - -var ( - _ Host = (*mockHost)(nil) -) - -var ( - errSectorOutOfBounds = errors.New("sector out of bounds") -) - -func (h *mockHost) DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) error { - sector, exist := h.sectors[root] - if !exist { - return errSectorNotFound - } - if offset+length > rhpv2.SectorSize { - return errSectorOutOfBounds - } - _, err := w.Write(sector[offset : offset+length]) - return err -} - -func (h *mockHost) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) { - root := rhpv2.SectorRoot(sector) - h.sectors[root] = sector - return root, nil -} - -func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (rev types.FileContractRevision, _ error) { - return -} - -func (h *mockHost) FetchPriceTable(ctx context.Context, rev *types.FileContractRevision) (hpt hostdb.HostPriceTable, err error) { - return -} - -func (h *mockHost) FundAccount(ctx context.Context, balance types.Currency, rev *types.FileContractRevision) error { - return nil -} - -func (h *mockHost) RenewContract(ctx context.Context, rrr api.RHPRenewRequest) (_ rhpv2.ContractRevision, _ []types.Transaction, _ types.Currency, err error) { - return -} - -func (h *mockHost) SyncAccount(ctx context.Context, rev *types.FileContractRevision) error { - return nil -} - func TestHost(t *testing.T) { - h := newMockHost() + c := newMockContract(types.FileContractID{1}) + h := newMockHost(types.PublicKey{1}, c) sector, root := newMockSector() // upload the sector @@ -105,17 +47,3 @@ func TestHost(t *testing.T) { t.Fatal("expected out of bounds error", err) } } - -func newMockHost() *mockHost { - return &mockHost{ - hk: types.PublicKey{1}, - fcid: types.FileContractID{1}, - sectors: make(map[types.Hash256]*[rhpv2.SectorSize]byte), - } -} - -func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { - var sector [rhpv2.SectorSize]byte - frand.Read(sector[:]) - return §or, rhpv2.SectorRoot(§or) -} diff --git a/worker/mocks_test.go b/worker/mocks_test.go new file mode 100644 index 000000000..b2597240c --- /dev/null +++ b/worker/mocks_test.go @@ -0,0 +1,272 @@ +package worker + +import ( + "context" + "errors" + "io" + "sync" + "time" + + rhpv2 "go.sia.tech/core/rhp/v2" + "go.sia.tech/core/types" + "go.sia.tech/renterd/api" + "go.sia.tech/renterd/hostdb" + "go.sia.tech/renterd/object" + "lukechampine.com/frand" +) + +type ( + mockContract struct { + rev types.FileContractRevision + + mu sync.Mutex + sectors map[types.Hash256]*[rhpv2.SectorSize]byte + } + + mockContractLocker struct { + contracts map[types.FileContractID]*mockContract + } + + mockHost struct { + hk types.PublicKey + + mu sync.Mutex + c *mockContract + } + + mockHostManager struct { + hosts map[types.PublicKey]Host + } + + mockMemory struct{} + mockMemoryManager struct{} + + mockObjectStore struct { + mu sync.Mutex + objects map[string]map[string]object.Object + } +) + +var ( + _ ContractLocker = (*mockContractLocker)(nil) + _ Host = (*mockHost)(nil) + _ HostManager = (*mockHostManager)(nil) + _ Memory = (*mockMemory)(nil) + _ MemoryManager = (*mockMemoryManager)(nil) + _ ObjectStore = (*mockObjectStore)(nil) +) + +var ( + errBucketNotFound = errors.New("bucket not found") + errContractNotFound = errors.New("contract not found") + errObjectNotFound = errors.New("object not found") + errSectorOutOfBounds = errors.New("sector out of bounds") +) + +func (m *mockMemory) Release() {} +func (m *mockMemory) ReleaseSome(uint64) {} + +func (mm *mockMemoryManager) Limit(amt uint64) (MemoryManager, error) { + return &mockMemoryManager{}, nil +} +func (mm *mockMemoryManager) Status() api.MemoryStatus { return api.MemoryStatus{} } +func (mm *mockMemoryManager) AcquireMemory(ctx context.Context, amt uint64) Memory { + return &mockMemory{} +} + +func (os *mockObjectStore) AddMultipartPart(ctx context.Context, bucket, path, contractSet, ETag, uploadID string, partNumber int, slices []object.SlabSlice) (err error) { + return nil +} + +func (os *mockObjectStore) AddObject(ctx context.Context, bucket, path, contractSet string, o object.Object, opts api.AddObjectOptions) error { + os.mu.Lock() + defer os.mu.Unlock() + + // check if the bucket exists + if _, exists := os.objects[bucket]; !exists { + return errBucketNotFound + } + + os.objects[bucket][path] = o + return nil +} + +func (os *mockObjectStore) AddPartialSlab(ctx context.Context, data []byte, minShards, totalShards uint8, contractSet string) (slabs []object.SlabSlice, slabBufferMaxSizeSoftReached bool, err error) { + return nil, false, nil +} + +func (os *mockObjectStore) AddUploadingSector(ctx context.Context, uID api.UploadID, id types.FileContractID, root types.Hash256) error { + return nil +} + +func (os *mockObjectStore) Object(ctx context.Context, bucket, path string, opts api.GetObjectOptions) (api.ObjectsResponse, error) { + os.mu.Lock() + defer os.mu.Unlock() + + // check if the bucket exists + if _, exists := os.objects[bucket]; !exists { + return api.ObjectsResponse{}, errBucketNotFound + } + + // check if the object exists + if _, exists := os.objects[bucket][path]; !exists { + return api.ObjectsResponse{}, errObjectNotFound + } + + object := os.objects[bucket][path] + return api.ObjectsResponse{Object: &api.Object{ + ObjectMetadata: api.ObjectMetadata{Name: path, Size: object.TotalSize()}, + Object: object, + }}, nil +} + +func (os *mockObjectStore) DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) error { + return nil +} +func (os *mockObjectStore) DeleteObject(ctx context.Context, bucket, path string, opts api.DeleteObjectOptions) error { + return nil +} +func (os *mockObjectStore) FetchPartialSlab(ctx context.Context, key object.EncryptionKey, offset, length uint32) ([]byte, error) { + return nil, nil +} +func (os *mockObjectStore) Slab(ctx context.Context, key object.EncryptionKey) (object.Slab, error) { + return object.Slab{}, nil +} +func (os *mockObjectStore) UpdateSlab(ctx context.Context, s object.Slab, contractSet string) error { + return nil +} +func (os *mockObjectStore) MarkPackedSlabsUploaded(ctx context.Context, slabs []api.UploadedPackedSlab) error { + return nil +} +func (os *mockObjectStore) TrackUpload(ctx context.Context, uID api.UploadID) error { return nil } +func (os *mockObjectStore) FinishUpload(ctx context.Context, uID api.UploadID) error { return nil } + +func (h *mockHost) DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) error { + sector, exist := h.c.sectors[root] + if !exist { + return errSectorNotFound + } + if offset+length > rhpv2.SectorSize { + return errSectorOutOfBounds + } + _, err := w.Write(sector[offset : offset+length]) + return err +} + +func (h *mockHost) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) { + root := rhpv2.SectorRoot(sector) + h.c.sectors[root] = sector + return root, nil +} + +func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (rev types.FileContractRevision, _ error) { + h.mu.Lock() + defer h.mu.Unlock() + rev = h.c.rev + return +} + +func (h *mockHost) FetchPriceTable(ctx context.Context, rev *types.FileContractRevision) (hpt hostdb.HostPriceTable, err error) { + return +} + +func (h *mockHost) FundAccount(ctx context.Context, balance types.Currency, rev *types.FileContractRevision) error { + return nil +} + +func (h *mockHost) RenewContract(ctx context.Context, rrr api.RHPRenewRequest) (_ rhpv2.ContractRevision, _ []types.Transaction, _ types.Currency, err error) { + return +} + +func (h *mockHost) SyncAccount(ctx context.Context, rev *types.FileContractRevision) error { + return nil +} + +func (hp *mockHostManager) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr string) Host { + if _, ok := hp.hosts[hk]; !ok { + panic("host not found") + } + return hp.hosts[hk] +} + +func (cl *mockContractLocker) AcquireContract(ctx context.Context, fcid types.FileContractID, priority int, d time.Duration) (lockID uint64, err error) { + if lock, ok := cl.contracts[fcid]; !ok { + return 0, errContractNotFound + } else { + lock.mu.Lock() + } + + return 0, nil +} + +func (cl *mockContractLocker) ReleaseContract(ctx context.Context, fcid types.FileContractID, lockID uint64) (err error) { + if lock, ok := cl.contracts[fcid]; !ok { + return errContractNotFound + } else { + lock.mu.Unlock() + } + return nil +} + +func (cl *mockContractLocker) KeepaliveContract(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration) (err error) { + return nil +} + +func newMockHosts(n int) []*mockHost { + hosts := make([]*mockHost, n) + for i := range hosts { + hosts[i] = newMockHost(types.PublicKey{byte(i)}, nil) + } + return hosts +} + +func newMockHost(hk types.PublicKey, c *mockContract) *mockHost { + return &mockHost{ + hk: hk, + c: c, + } +} + +func newMockContracts(hosts []*mockHost) []*mockContract { + contracts := make([]*mockContract, len(hosts)) + for i := range contracts { + contracts[i] = newMockContract(types.FileContractID{byte(i)}) + hosts[i].c = contracts[i] + } + return contracts +} + +func newMockContract(fcid types.FileContractID) *mockContract { + return &mockContract{ + rev: types.FileContractRevision{ParentID: fcid}, + sectors: make(map[types.Hash256]*[rhpv2.SectorSize]byte), + } +} + +func newMockContractLocker(contracts []*mockContract) *mockContractLocker { + cl := &mockContractLocker{contracts: make(map[types.FileContractID]*mockContract)} + for _, c := range contracts { + cl.contracts[c.rev.ParentID] = c + } + return cl +} + +func newMockHostManager(hosts []*mockHost) *mockHostManager { + hm := &mockHostManager{hosts: make(map[types.PublicKey]Host)} + for _, h := range hosts { + hm.hosts[h.hk] = h + } + return hm +} + +func newMockObjectStore() *mockObjectStore { + os := &mockObjectStore{objects: make(map[string]map[string]object.Object)} + os.objects[testBucket] = make(map[string]object.Object) + return os +} + +func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { + var sector [rhpv2.SectorSize]byte + frand.Read(sector[:]) + return §or, rhpv2.SectorRoot(§or) +} diff --git a/worker/upload.go b/worker/upload.go index d54e7f395..18bd9012d 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -41,9 +41,11 @@ type ( hm HostManager mm MemoryManager os ObjectStore - rl revisionLocker // TODO: host should implement revisionLocker, would allow us to remove it here + cl ContractLocker logger *zap.SugaredLogger + contractLockDuration time.Duration + maxOverdrive uint64 overdriveTimeout time.Duration @@ -66,16 +68,20 @@ type ( } upload struct { - id api.UploadID - allowed map[types.PublicKey]struct{} - lockPriority int - shutdownCtx context.Context + id api.UploadID + + allowed map[types.PublicKey]struct{} + + contractLockPriority int + contractLockDuration time.Duration + + shutdownCtx context.Context } slabUpload struct { - uploadID api.UploadID - mem Memory - lockPriority int + uploadID api.UploadID + contractLockPriority int + contractLockDuration time.Duration maxOverdrive uint64 lastOverdrive time.Time @@ -89,6 +95,8 @@ type ( numUploaded uint64 numSectors uint64 + mem Memory + errs HostErrorSet } @@ -114,9 +122,12 @@ type ( } sectorUploadReq struct { - uploadID api.UploadID + // upload fields + uploadID api.UploadID + contractLockDuration time.Duration + contractLockPriority int + sector *sectorUpload - lockPriority int overdrive bool responseChan chan sectorUploadResp @@ -138,7 +149,7 @@ func (w *worker) initUploadManager(maxMemory, maxOverdrive uint64, overdriveTime } mm := newMemoryManager(logger, maxMemory) - w.uploadManager = newUploadManager(w.shutdownCtx, w, mm, w.bus, w, maxOverdrive, overdriveTimeout, logger) + w.uploadManager = newUploadManager(w.shutdownCtx, w, mm, w.bus, w.bus, maxOverdrive, overdriveTimeout, w.contractLockingDuration, logger) } func (w *worker) upload(ctx context.Context, r io.Reader, contracts []api.ContractMetadata, up uploadParameters, opts ...UploadOption) (_ string, err error) { @@ -255,7 +266,7 @@ func (w *worker) uploadPackedSlabs(ctx context.Context, lockingDuration time.Dur go func(ps api.PackedSlab) { defer mem.Release() defer wg.Done() - err := w.uploadPackedSlab(ctx, ps, rs, contractSet, lockPriority, mem) + err := w.uploadPackedSlab(ctx, rs, ps, mem, contractSet, lockPriority) mu.Lock() if err != nil { errs = errors.Join(errs, err) @@ -275,7 +286,7 @@ func (w *worker) uploadPackedSlabs(ctx context.Context, lockingDuration time.Dur return } -func (w *worker) uploadPackedSlab(ctx context.Context, ps api.PackedSlab, rs api.RedundancySettings, contractSet string, lockPriority int, mem Memory) error { +func (w *worker) uploadPackedSlab(ctx context.Context, rs api.RedundancySettings, ps api.PackedSlab, mem Memory, contractSet string, lockPriority int) error { // create a context with sane timeout ctx, cancel := context.WithTimeout(ctx, defaultPackedSlabsUploadTimeout) defer cancel() @@ -296,7 +307,7 @@ func (w *worker) uploadPackedSlab(ctx context.Context, ps api.PackedSlab, rs api ctx = WithGougingChecker(ctx, w.bus, up.GougingParams) // upload packed slab - err = w.uploadManager.UploadPackedSlab(ctx, rs, ps, contracts, up.CurrentHeight, lockPriority, mem) + err = w.uploadManager.UploadPackedSlab(ctx, rs, ps, mem, contracts, up.CurrentHeight, lockPriority) if err != nil { return fmt.Errorf("couldn't upload packed slab, err: %v", err) } @@ -304,14 +315,16 @@ func (w *worker) uploadPackedSlab(ctx context.Context, ps api.PackedSlab, rs api return nil } -func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os ObjectStore, rl revisionLocker, maxOverdrive uint64, overdriveTimeout time.Duration, logger *zap.SugaredLogger) *uploadManager { +func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os ObjectStore, cl ContractLocker, maxOverdrive uint64, overdriveTimeout time.Duration, contractLockDuration time.Duration, logger *zap.SugaredLogger) *uploadManager { return &uploadManager{ hm: hm, mm: mm, os: os, - rl: rl, + cl: cl, logger: logger, + contractLockDuration: contractLockDuration, + maxOverdrive: maxOverdrive, overdriveTimeout: overdriveTimeout, @@ -324,9 +337,11 @@ func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os } } -func (mgr *uploadManager) newUploader(os ObjectStore, h Host, c api.ContractMetadata, bh uint64) *uploader { +func (mgr *uploadManager) newUploader(os ObjectStore, cl ContractLocker, h Host, c api.ContractMetadata, bh uint64) *uploader { return &uploader{ - os: os, + os: os, + cl: cl, + logger: mgr.logger, // static hk: c.HostKey, @@ -620,7 +635,7 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a return } -func (mgr *uploadManager) UploadPackedSlab(ctx context.Context, rs api.RedundancySettings, ps api.PackedSlab, contracts []api.ContractMetadata, bh uint64, lockPriority int, mem Memory) (err error) { +func (mgr *uploadManager) UploadPackedSlab(ctx context.Context, rs api.RedundancySettings, ps api.PackedSlab, mem Memory, contracts []api.ContractMetadata, bh uint64, lockPriority int) (err error) { // cancel all in-flight requests when the upload is done ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -712,10 +727,11 @@ func (mgr *uploadManager) newUpload(ctx context.Context, totalShards int, contra // create upload return &upload{ - id: api.NewUploadID(), - allowed: allowed, - lockPriority: lockPriority, - shutdownCtx: mgr.shutdownCtx, + id: api.NewUploadID(), + allowed: allowed, + contractLockDuration: mgr.contractLockDuration, + contractLockPriority: lockPriority, + shutdownCtx: mgr.shutdownCtx, }, nil } @@ -757,9 +773,9 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh for _, c := range contracts { if _, exists := existing[c.ID]; !exists { host := mgr.hm.Host(c.HostKey, c.ID, c.SiamuxAddr) - uploader := mgr.newUploader(mgr.os, host, c, bh) + uploader := mgr.newUploader(mgr.os, mgr.cl, host, c, bh) refreshed = append(refreshed, uploader) - go uploader.Start(mgr.hm, mgr.rl) + go uploader.Start() } } @@ -799,8 +815,11 @@ func (u *upload) newSlabUpload(ctx context.Context, shards [][]byte, uploaders [ // create slab upload return &slabUpload{ - lockPriority: u.lockPriority, - uploadID: u.id, + uploadID: u.id, + + contractLockPriority: u.contractLockPriority, + contractLockDuration: u.contractLockDuration, + maxOverdrive: maxOverdrive, mem: mem, @@ -862,11 +881,12 @@ func (u *upload) uploadShards(ctx context.Context, shards [][]byte, candidates [ requests := make([]*sectorUploadReq, len(shards)) for sI := range shards { requests[sI] = §orUploadReq{ - uploadID: slab.uploadID, - sector: slab.sectors[sI], - lockPriority: slab.lockPriority, - overdrive: false, - responseChan: respChan, + uploadID: slab.uploadID, + sector: slab.sectors[sI], + contractLockPriority: slab.contractLockPriority, + contractLockDuration: slab.contractLockDuration, + overdrive: false, + responseChan: respChan, } } @@ -1054,11 +1074,12 @@ func (s *slabUpload) nextRequest(responseChan chan sectorUploadResp) *sectorUplo } return §orUploadReq{ - lockPriority: s.lockPriority, - overdrive: true, - responseChan: responseChan, - sector: nextSector, - uploadID: s.uploadID, + contractLockDuration: s.contractLockDuration, + contractLockPriority: s.contractLockPriority, + overdrive: true, + responseChan: responseChan, + sector: nextSector, + uploadID: s.uploadID, } } @@ -1081,7 +1102,7 @@ func (s *slabUpload) receive(resp sectorUploadResp) (bool, bool) { // sanity check we receive the expected root if resp.root != req.sector.root { - s.errs[req.hk] = errors.New("root mismatch") + s.errs[req.hk] = fmt.Errorf("root mismatch, %v != %v", resp.root, req.sector.root) return false, false } diff --git a/worker/upload_test.go b/worker/upload_test.go new file mode 100644 index 000000000..fd4a82907 --- /dev/null +++ b/worker/upload_test.go @@ -0,0 +1,140 @@ +package worker + +import ( + "bytes" + "context" + "errors" + "testing" + "time" + + "go.sia.tech/core/types" + "go.sia.tech/renterd/api" + "go.sia.tech/renterd/object" + "go.uber.org/zap" + "lukechampine.com/frand" +) + +var ( + testBucket = "testbucket" +) + +func TestUploadDownload(t *testing.T) { + // create upload params + params := testParameters(testBucket, t.Name()) + + // create test hosts and contracts + hosts := newMockHosts(params.rs.TotalShards * 2) + contracts := newMockContracts(hosts) + + // mock dependencies + cl := newMockContractLocker(contracts) + hm := newMockHostManager(hosts) + os := newMockObjectStore() + mm := &mockMemoryManager{} + + // create managers + dl := newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()) + ul := newUploadManager(context.Background(), hm, mm, os, cl, 0, 0, time.Minute, zap.NewNop().Sugar()) + + // create test data + data := make([]byte, 128) + if _, err := frand.Read(data); err != nil { + t.Fatal(err) + } + + // create upload contracts + metadatas := make([]api.ContractMetadata, len(contracts)) + for i, h := range hosts { + metadatas[i] = api.ContractMetadata{ + ID: h.c.rev.ParentID, + HostKey: h.hk, + } + } + + // upload data + _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + if err != nil { + t.Fatal(err) + } + + // grab the object + o, err := os.Object(context.Background(), testBucket, t.Name(), api.GetObjectOptions{}) + if err != nil { + t.Fatal(err) + } + + // build used hosts + used := make(map[types.PublicKey]struct{}) + for _, shard := range o.Object.Object.Slabs[0].Shards { + used[shard.LatestHost] = struct{}{} + } + + // download the data and assert it matches + var buf bytes.Buffer + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), metadatas) + if err != nil { + t.Fatal(err) + } else if !bytes.Equal(data, buf.Bytes()) { + t.Fatal("data mismatch") + } + + // filter contracts to have (at most) min shards used contracts + var n int + var filtered []api.ContractMetadata + for _, md := range metadatas { + // add unused contracts + if _, used := used[md.HostKey]; !used { + filtered = append(filtered, md) + continue + } + + // add min shards used contracts + if n < int(params.rs.MinShards) { + filtered = append(filtered, md) + n++ + } + } + + // download the data again and assert it matches + buf.Reset() + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), filtered) + if err != nil { + t.Fatal(err) + } else if !bytes.Equal(data, buf.Bytes()) { + t.Fatal("data mismatch") + } + + // filter out one contract - expect download to fail + for i, md := range filtered { + if _, used := used[md.HostKey]; used { + filtered = append(filtered[:i], filtered[i+1:]...) + break + } + } + + // download the data again and assert it fails + buf.Reset() + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), filtered) + if !errors.Is(err, errDownloadNotEnoughHosts) { + t.Fatal("expected not enough hosts error", err) + } + + // try and upload into a bucket that does not exist + params.bucket = "doesnotexist" + _, _, err = ul.Upload(context.Background(), bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + if !errors.Is(err, errBucketNotFound) { + t.Fatal("expected bucket not found error", err) + } +} + +func testParameters(bucket, path string) uploadParameters { + return uploadParameters{ + bucket: bucket, + path: path, + + ec: object.GenerateEncryptionKey(), // random key + encryptionOffset: 0, // from the beginning + + rs: api.RedundancySettings{MinShards: 2, TotalShards: 6}, + } +} diff --git a/worker/uploader.go b/worker/uploader.go index aef8a5900..f7a9af5eb 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -14,11 +14,14 @@ import ( "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/stats" + "go.uber.org/zap" ) type ( uploader struct { - os ObjectStore + os ObjectStore + cl ContractLocker + logger *zap.SugaredLogger hk types.PublicKey siamuxAddr string @@ -76,7 +79,7 @@ func (u *uploader) Renew(h Host, c api.ContractMetadata, bh uint64) { u.endHeight = c.WindowEnd } -func (u *uploader) Start(hm HostManager, rl revisionLocker) { +func (u *uploader) Start() { outer: for { // wait for work @@ -105,19 +108,13 @@ outer: continue } + // sanity check lock duration and priority are set + if req.contractLockDuration == 0 || req.contractLockPriority == 0 { + panic("lock duration and priority can't be 0") // developer error + } + // execute it - var root types.Hash256 - start := time.Now() - fcid := u.ContractID() - err := rl.withRevision(req.sector.ctx, defaultRevisionFetchTimeout, fcid, u.hk, u.siamuxAddr, req.lockPriority, u.BlockHeight(), func(rev types.FileContractRevision) error { - if rev.RevisionNumber == math.MaxUint64 { - return errMaxRevisionReached - } - - var err error - root, err = u.execute(req, rev) - return err - }) + root, elapsed, err := u.execute(req) // the uploader's contract got renewed, requeue the request if errors.Is(err, errMaxRevisionReached) { @@ -135,7 +132,7 @@ outer: // track the error, ignore gracefully closed streams and canceled overdrives canceledOverdrive := req.done() && req.overdrive && err != nil if !canceledOverdrive && !isClosedStream(err) { - u.trackSectorUpload(err, time.Since(start)) + u.trackSectorUpload(err, elapsed) } } } @@ -193,8 +190,10 @@ func (u *uploader) estimate() float64 { return numSectors * estimateP90 } -func (u *uploader) execute(req *sectorUploadReq, rev types.FileContractRevision) (types.Hash256, error) { +func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, error) { + // grab fields u.mu.Lock() + bh := u.bh host := u.host fcid := u.fcid u.mu.Unlock() @@ -203,16 +202,38 @@ func (u *uploader) execute(req *sectorUploadReq, rev types.FileContractRevision) span := trace.SpanFromContext(req.sector.ctx) span.AddEvent("execute") + // acquire contract lock + lockID, err := u.cl.AcquireContract(req.sector.ctx, fcid, req.contractLockPriority, req.contractLockDuration) + if err != nil { + return types.Hash256{}, 0, err + } + + // defer the release + lock := newContractLock(fcid, lockID, req.contractLockDuration, u.cl, u.logger) + defer func() { + ctx, cancel := context.WithTimeout(u.shutdownCtx, 10*time.Second) + lock.Release(ctx) + cancel() + }() + + // fetch the revision + rev, err := host.FetchRevision(req.sector.ctx, defaultRevisionFetchTimeout, bh) + if err != nil { + return types.Hash256{}, 0, err + } else if rev.RevisionNumber == math.MaxUint64 { + return types.Hash256{}, 0, errMaxRevisionReached + } + // update the bus if err := u.os.AddUploadingSector(req.sector.ctx, req.uploadID, fcid, req.sector.root); err != nil { - return types.Hash256{}, fmt.Errorf("failed to add uploading sector to contract %v, err: %v", fcid, err) + return types.Hash256{}, 0, fmt.Errorf("failed to add uploading sector to contract %v, err: %v", fcid, err) } // upload the sector start := time.Now() root, err := host.UploadSector(req.sector.ctx, req.sector.data, rev) if err != nil { - return types.Hash256{}, err + return types.Hash256{}, 0, err } // update span @@ -221,7 +242,7 @@ func (u *uploader) execute(req *sectorUploadReq, rev types.FileContractRevision) span.RecordError(err) span.End() - return root, nil + return root, elapsed, nil } func (u *uploader) pop() *sectorUploadReq { @@ -252,6 +273,10 @@ func (u *uploader) trackSectorUpload(err error, d time.Duration) { u.statsSectorUploadEstimateInMS.Track(float64(time.Hour.Milliseconds())) } else { ms := d.Milliseconds() + if ms == 0 { + ms = 1 // avoid division by zero + } + u.consecutiveFailures = 0 u.statsSectorUploadEstimateInMS.Track(float64(ms)) // duration in ms u.statsSectorUploadSpeedBytesPerMS.Track(float64(rhpv2.SectorSize / ms)) // bytes per ms diff --git a/worker/worker.go b/worker/worker.go index 0b46e5b42..b518f3b66 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -120,12 +120,12 @@ type ( } ObjectStore interface { - // NOTE: used for download (TODO: remove) + // NOTE: used for download DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) error FetchPartialSlab(ctx context.Context, key object.EncryptionKey, offset, length uint32) ([]byte, error) Slab(ctx context.Context, key object.EncryptionKey) (object.Slab, error) - // NOTE: used for upload (TODO: remove) + // NOTE: used for upload AddObject(ctx context.Context, bucket, path, contractSet string, o object.Object, opts api.AddObjectOptions) error AddMultipartPart(ctx context.Context, bucket, path, contractSet, ETag, uploadID string, partNumber int, slices []object.SlabSlice) (err error) AddPartialSlab(ctx context.Context, data []byte, minShards, totalShards uint8, contractSet string) (slabs []object.SlabSlice, slabBufferMaxSizeSoftReached bool, err error) @@ -139,10 +139,6 @@ type ( consensusState interface { ConsensusState(ctx context.Context) (api.ConsensusState, error) } - - revisionLocker interface { - withRevision(ctx context.Context, timeout time.Duration, contractID types.FileContractID, hk types.PublicKey, siamuxAddr string, lockPriority int, blockHeight uint64, fn func(rev types.FileContractRevision) error) error - } ) // deriveSubKey can be used to derive a sub-masterkey from the worker's From fadf663a94bf80b1939a7b691418a7973d08bab0 Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Fri, 15 Dec 2023 17:32:09 +0100 Subject: [PATCH 08/24] autopilot: increase scannerTimeoutMinTimeout (#829) --- autopilot/scanner.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/autopilot/scanner.go b/autopilot/scanner.go index 3d344ca3e..e512d1f87 100644 --- a/autopilot/scanner.go +++ b/autopilot/scanner.go @@ -17,7 +17,7 @@ import ( const ( scannerTimeoutInterval = 10 * time.Minute - scannerTimeoutMinTimeout = time.Second * 5 + scannerTimeoutMinTimeout = 10 * time.Second trackerMinDataPoints = 25 trackerNumDataPoints = 1000 From b72a670939e318b367a174a9b6e6f34a015e2df6 Mon Sep 17 00:00:00 2001 From: Alex Freska Date: Fri, 15 Dec 2023 11:43:14 -0500 Subject: [PATCH 09/24] ui: v0.37.0 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 168458edf..8a79f0e2a 100644 --- a/go.mod +++ b/go.mod @@ -17,7 +17,7 @@ require ( go.sia.tech/jape v0.11.1 go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca - go.sia.tech/web/renterd v0.36.0 + go.sia.tech/web/renterd v0.37.0 go.uber.org/zap v1.26.0 golang.org/x/crypto v0.16.0 golang.org/x/term v0.15.0 diff --git a/go.sum b/go.sum index f6ad99546..f3fc7e4ba 100644 --- a/go.sum +++ b/go.sum @@ -245,8 +245,8 @@ go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca h1:aZMg2AKevn7jKx+wlusWQf go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca/go.mod h1:h/1afFwpxzff6/gG5i1XdAgPK7dEY6FaibhK7N5F86Y= go.sia.tech/web v0.0.0-20231213145933-3f175a86abff h1:/nE7nhewDRxzEdtSKT4SkiUwtjPSiy7Xz7CHEW3MaGQ= go.sia.tech/web v0.0.0-20231213145933-3f175a86abff/go.mod h1:RKODSdOmR3VtObPAcGwQqm4qnqntDVFylbvOBbWYYBU= -go.sia.tech/web/renterd v0.36.0 h1:78FXkALF4ZiJw6IrZY1IZ9G0Z+Bp9hZbR8sXxk28Jxg= -go.sia.tech/web/renterd v0.36.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= +go.sia.tech/web/renterd v0.37.0 h1:Igfi0TcrYFufltmbts5TltpMf6hS6veJWyYiWRPPvt4= +go.sia.tech/web/renterd v0.37.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= From c3b8f57458bb46c01bfda31dd43a66c900f93a2c Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 18 Dec 2023 11:47:38 +0100 Subject: [PATCH 10/24] autopilot: update collateral algorithm --- autopilot/contractor.go | 26 +++------ autopilot/hostfilter.go | 101 ++++++++++++++++++----------------- autopilot/hostfilter_test.go | 96 ++++++++++++++++----------------- 3 files changed, 105 insertions(+), 118 deletions(-) diff --git a/autopilot/contractor.go b/autopilot/contractor.go index ea73777c4..76220d6db 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -762,12 +762,7 @@ func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts // decide whether the contract is still good ci := contractInfo{contract: contract, priceTable: host.PriceTable.HostPriceTable, settings: host.Settings} - renterFunds, err := c.renewFundingEstimate(ctx, ci, state.fee, false) - if err != nil { - c.logger.Errorw(fmt.Sprintf("failed to compute renterFunds for contract: %v", err)) - } - - usable, recoverable, refresh, renew, reasons := c.isUsableContract(state.cfg, ci, cs.BlockHeight, renterFunds, ipFilter) + usable, recoverable, refresh, renew, reasons := c.isUsableContract(state.cfg, state, ci, cs.BlockHeight, ipFilter) ci.usable = usable ci.recoverable = recoverable if !usable { @@ -1406,7 +1401,7 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI // calculate the renter funds var renterFunds types.Currency - if isOutOfFunds(state.cfg, ci.settings, ci.contract) { + if isOutOfFunds(state.cfg, ci.priceTable, ci.contract) { renterFunds, err = c.refreshFundingEstimate(ctx, state.cfg, ci, state.fee) if err != nil { c.logger.Errorw(fmt.Sprintf("could not get refresh funding estimate, err: %v", err), "hk", hk, "fcid", fcid) @@ -1425,24 +1420,19 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI expectedStorage := renterFundsToExpectedStorage(renterFunds, contract.EndHeight()-cs.BlockHeight, ci.priceTable) unallocatedCollateral := rev.MissedHostPayout().Sub(contract.ContractPrice) - // calculate the expected new collateral to determine the minNewCollateral. - // If the contract isn't below the min collateral, we don't enforce a - // minimum. - var minNewColl types.Currency - _, _, expectedNewCollateral := rhpv3.RenewalCosts(contract.Revision.FileContract, ci.priceTable, expectedStorage, contract.EndHeight()) - if isBelowCollateralThreshold(expectedNewCollateral, unallocatedCollateral) { - minNewColl = minNewCollateral(unallocatedCollateral) - } + // a refresh should always result in at least double the minimum collateral + // to avoid refreshing again too soon + minNewCollateral := minRemainingCollateral(state.cfg, state.rs, contract, settings, ci.priceTable).Mul64(2) // renew the contract - resp, err := w.RHPRenew(ctx, contract.ID, contract.EndHeight(), hk, contract.SiamuxAddr, settings.Address, state.address, renterFunds, minNewColl, expectedStorage, settings.WindowSize) + resp, err := w.RHPRenew(ctx, contract.ID, contract.EndHeight(), hk, contract.SiamuxAddr, settings.Address, state.address, renterFunds, minNewCollateral, expectedStorage, settings.WindowSize) if err != nil { if strings.Contains(err.Error(), "new collateral is too low") { c.logger.Debugw("refresh failed: contract wouldn't have enough collateral after refresh", "hk", hk, "fcid", fcid, "unallocatedCollateral", unallocatedCollateral.String(), - "minNewCollateral", minNewColl.String(), + "minNewCollateral", minNewCollateral.String(), ) return api.ContractMetadata{}, true, err } @@ -1469,7 +1459,7 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI "fcid", refreshedContract.ID, "renewedFrom", contract.ID, "renterFunds", renterFunds.String(), - "minNewCollateral", minNewColl.String(), + "minNewCollateral", minNewCollateral.String(), "newCollateral", newCollateral.String(), ) return refreshedContract, true, nil diff --git a/autopilot/hostfilter.go b/autopilot/hostfilter.go index a8a674e0a..d43626973 100644 --- a/autopilot/hostfilter.go +++ b/autopilot/hostfilter.go @@ -20,12 +20,11 @@ const ( // remaining at which the contract gets marked as not good for upload minContractFundUploadThreshold = float64(0.05) // 5% - // minContractCollateralThreshold is 10% of the collateral that we would put - // into a contract upon renewing it. That means, we consider a contract - // worth renewing when that results in 10x the collateral of what it - // currently has remaining. - minContractCollateralThresholdNumerator = 1 - minContractCollateralThresholdDenominator = 10 + // minContractCollateralDenominator is used to define the percentage of + // remaining collateral in a contract in relation to its potential + // 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. @@ -227,7 +226,7 @@ func isUsableHost(cfg api.AutopilotConfig, rs api.RedundancySettings, gc worker. // - recoverable -> can be usable in the contract set if it is refreshed/renewed // - refresh -> should be refreshed // - renew -> should be renewed -func (c *contractor) isUsableContract(cfg api.AutopilotConfig, ci contractInfo, bh uint64, renterFunds types.Currency, f *ipFilter) (usable, recoverable, refresh, renew bool, reasons []string) { +func (c *contractor) isUsableContract(cfg api.AutopilotConfig, state state, ci contractInfo, bh uint64, f *ipFilter) (usable, recoverable, refresh, renew bool, reasons []string) { contract, s, pt := ci.contract, ci.settings, ci.priceTable usable = true @@ -244,14 +243,14 @@ func (c *contractor) isUsableContract(cfg api.AutopilotConfig, ci contractInfo, refresh = false renew = false } else { - if isOutOfCollateral(contract, s, pt, renterFunds, cfg.Contracts.Period, bh) { + if isOutOfCollateral(cfg, state.rs, contract, s, pt) { reasons = append(reasons, errContractOutOfCollateral.Error()) usable = false recoverable = true refresh = true renew = false } - if isOutOfFunds(cfg, s, contract) { + if isOutOfFunds(cfg, pt, contract) { reasons = append(reasons, errContractOutOfFunds.Error()) usable = false recoverable = true @@ -278,19 +277,17 @@ func (c *contractor) isUsableContract(cfg api.AutopilotConfig, ci contractInfo, return } -func isOutOfFunds(cfg api.AutopilotConfig, s rhpv2.HostSettings, c api.Contract) bool { +func isOutOfFunds(cfg api.AutopilotConfig, pt rhpv3.HostPriceTable, c api.Contract) bool { // TotalCost should never be zero but for legacy reasons we check and return // true should it be the case if c.TotalCost.IsZero() { return true } - blockBytes := types.NewCurrency64(rhpv2.SectorSize * cfg.Contracts.Period) - sectorStoragePrice := s.StoragePrice.Mul(blockBytes) - sectorUploadBandwidthPrice := s.UploadBandwidthPrice.Mul64(rhpv2.SectorSize) - sectorDownloadBandwidthPrice := s.DownloadBandwidthPrice.Mul64(rhpv2.SectorSize) - sectorBandwidthPrice := sectorUploadBandwidthPrice.Add(sectorDownloadBandwidthPrice) - sectorPrice := sectorStoragePrice.Add(sectorBandwidthPrice) + sectorPrice, _ := pt.BaseCost(). + Add(pt.AppendSectorCost(cfg.Contracts.Period)). + Add(pt.ReadSectorCost(rhpv2.SectorSize)). + Total() percentRemaining, _ := big.NewRat(0, 1).SetFrac(c.RenterFunds().Big(), c.TotalCost.Big()).Float64() return c.RenterFunds().Cmp(sectorPrice.Mul64(3)) < 0 || percentRemaining < minContractFundUploadThreshold @@ -299,50 +296,56 @@ func isOutOfFunds(cfg api.AutopilotConfig, s rhpv2.HostSettings, c api.Contract) // isOutOfCollateral returns 'true' if the remaining/unallocated collateral in // the contract is below a certain threshold of the collateral we would try to // put into a contract upon renew. -func isOutOfCollateral(c api.Contract, s rhpv2.HostSettings, pt rhpv3.HostPriceTable, renterFunds types.Currency, period, blockHeight uint64) bool { - // Compute the expected storage for the contract given the funds we are - // willing to put into it. +func isOutOfCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, c api.Contract, s rhpv2.HostSettings, pt rhpv3.HostPriceTable) bool { + min := minRemainingCollateral(cfg, rs, c, s, pt) + return c.RemainingCollateral().Cmp(min) < 0 +} + +// minNewCollateral returns the minimum amount of unallocated collateral that a +// contract should contain after a refresh given the current amount of +// unallocated collateral. +func minRemainingCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, c api.Contract, s rhpv2.HostSettings, pt rhpv3.HostPriceTable) types.Currency { + // Compute the expected storage for the contract given its remaining funds. // Note: we use the full period here even though we are checking whether to // do a refresh. Otherwise, the 'expectedStorage' would would become // ridiculously large the closer the contract is to its end height. - expectedStorage := renterFundsToExpectedStorage(renterFunds, period, pt) + expectedStorage := renterFundsToExpectedStorage(c.RenterFunds(), cfg.Contracts.Period, pt) + + // Cap the expected storage at twice the ideal amount of data we expect to + // store on a host. Even if we could afford more storage, there is no point + // in locking up more collateral than we expect to require. + idealDataPerHost := float64(cfg.Contracts.Storage) * rs.Redundancy() / float64(cfg.Contracts.Amount) + allocationPerHost := idealDataPerHost * 2 + if expectedStorage > uint64(allocationPerHost) { + expectedStorage = uint64(allocationPerHost) + } + // Cap the expected storage at the remaining storage of the host. If the // host doesn't have any storage left, there is no point in adding // collateral. if expectedStorage > s.RemainingStorage { expectedStorage = s.RemainingStorage } - _, _, newCollateral := rhpv3.RenewalCosts(c.Revision.FileContract, pt, expectedStorage, c.EndHeight()) - return isBelowCollateralThreshold(newCollateral, c.RemainingCollateral()) -} -// isBelowCollateralThreshold returns true if the remainingCollateral is below a -// certain percentage of newCollateral. The newCollateral is the amount of -// unallocated collateral in a contract after refreshing it and the -// remainingCollateral is the current amount of unallocated collateral in the -// contract. -func isBelowCollateralThreshold(newCollateral, remainingCollateral types.Currency) bool { - if newCollateral.IsZero() { - // Protect against division-by-zero. This can happen for 2 reasons. - // 1. the collateral is already at the host's max collateral so a - // refresh wouldn't result in any new unallocated collateral. - // 2. the host has no more remaining storage so a refresh would only - // lead to unallocated collateral that we can't use. - // In both cases we don't gain anything from refreshing the contract. - // NOTE: This causes us to not immediately consider contracts as bad - // even though we can't upload to them anymore. This is fine since the - // collateral score or remaining storage score should filter these - // contracts out eventually. - return false - } - return newCollateral.Cmp(minNewCollateral(remainingCollateral)) >= 0 -} + // Computet the collateral for a single sector. + _, sectorCollateral := pt.BaseCost(). + Add(pt.AppendSectorCost(cfg.Contracts.Period)). + Add(pt.ReadSectorCost(rhpv2.SectorSize)). + Total() -// minNewCollateral returns the minimum amount of unallocated collateral that a -// contract should contain after a refresh given the current amount of -// unallocated collateral. -func minNewCollateral(unallocatedCollateral types.Currency) types.Currency { - return unallocatedCollateral.Mul64(minContractCollateralThresholdDenominator).Div64(minContractCollateralThresholdNumerator) + // The expectedStorageCollateral is 5% of the collateral we'd need to store + // all of the expectedStorage. + minExpectedStorageCollateral := sectorCollateral.Mul64(expectedStorage / rhpv2.SectorSize).Div64(minContractCollateralDenominator) + + // The absolute minimum collateral we want to put into a contract is 3 + // sectors worth of collateral. + minCollateral := sectorCollateral.Mul64(3) + + // Return the larger of the two. + if minExpectedStorageCollateral.Cmp(minCollateral) > 0 { + minCollateral = minExpectedStorageCollateral + } + return minCollateral } func isUpForRenewal(cfg api.AutopilotConfig, r types.FileContractRevision, blockHeight uint64) (shouldRenew, secondHalf bool) { diff --git a/autopilot/hostfilter_test.go b/autopilot/hostfilter_test.go index 9a7275039..1a7a16f56 100644 --- a/autopilot/hostfilter_test.go +++ b/autopilot/hostfilter_test.go @@ -1,53 +1,47 @@ package autopilot -import ( - "testing" - - "go.sia.tech/core/types" -) - -func TestMinNewCollateral(t *testing.T) { - t.Parallel() - - // The collateral threshold is 10% meaning that we expect 10 times the - // remaining collateral to be the minimum to trigger a renewal. - if min := minNewCollateral(types.Siacoins(1)); !min.Equals(types.Siacoins(10)) { - t.Fatalf("expected 10, got %v", min) - } -} - -func TestIsBelowCollateralThreshold(t *testing.T) { - t.Parallel() - - tests := []struct { - newCollateral types.Currency - remainingCollateral types.Currency - isBelow bool - }{ - { - remainingCollateral: types.NewCurrency64(1), - newCollateral: types.NewCurrency64(10), - isBelow: true, - }, - { - remainingCollateral: types.NewCurrency64(1), - newCollateral: types.NewCurrency64(9), - isBelow: false, - }, - { - remainingCollateral: types.NewCurrency64(1), - newCollateral: types.NewCurrency64(11), - isBelow: true, - }, - { - remainingCollateral: types.NewCurrency64(1), - newCollateral: types.ZeroCurrency, - isBelow: false, - }, - } - for i, test := range tests { - if isBelow := isBelowCollateralThreshold(test.newCollateral, test.remainingCollateral); isBelow != test.isBelow { - t.Fatalf("%v: expected %v, got %v", i+1, test.isBelow, isBelow) - } - } -} +//func TestMinNewCollateral(t *testing.T) { +// t.Parallel() +// +// // The collateral threshold is 10% meaning that we expect 10 times the +// // remaining collateral to be the minimum to trigger a renewal. +// if min := minNewCollateral(types.Siacoins(1)); !min.Equals(types.Siacoins(10)) { +// t.Fatalf("expected 10, got %v", min) +// } +//} +// +//func TestIsBelowCollateralThreshold(t *testing.T) { +// t.Parallel() +// +// tests := []struct { +// newCollateral types.Currency +// remainingCollateral types.Currency +// isBelow bool +// }{ +// { +// remainingCollateral: types.NewCurrency64(1), +// newCollateral: types.NewCurrency64(10), +// isBelow: true, +// }, +// { +// remainingCollateral: types.NewCurrency64(1), +// newCollateral: types.NewCurrency64(9), +// isBelow: false, +// }, +// { +// remainingCollateral: types.NewCurrency64(1), +// newCollateral: types.NewCurrency64(11), +// isBelow: true, +// }, +// { +// remainingCollateral: types.NewCurrency64(1), +// newCollateral: types.ZeroCurrency, +// isBelow: false, +// }, +// } +// for i, test := range tests { +// if isBelow := isBelowCollateralThreshold(test.newCollateral, test.remainingCollateral); isBelow != test.isBelow { +// t.Fatalf("%v: expected %v, got %v", i+1, test.isBelow, isBelow) +// } +// } +//} From c14c89b0cedb24f5f954ccd5989439095826a5f8 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 18 Dec 2023 11:55:11 +0100 Subject: [PATCH 11/24] autopilot: pass in renterFunds to minRemainingCollateral --- autopilot/contractor.go | 5 ++--- autopilot/hostfilter.go | 6 +++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 76220d6db..554206654 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -1420,9 +1420,8 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI expectedStorage := renterFundsToExpectedStorage(renterFunds, contract.EndHeight()-cs.BlockHeight, ci.priceTable) unallocatedCollateral := rev.MissedHostPayout().Sub(contract.ContractPrice) - // a refresh should always result in at least double the minimum collateral - // to avoid refreshing again too soon - minNewCollateral := minRemainingCollateral(state.cfg, state.rs, contract, settings, ci.priceTable).Mul64(2) + // a refresh should always result in a contract that has enough collateral + minNewCollateral := minRemainingCollateral(state.cfg, state.rs, renterFunds, settings, ci.priceTable) // renew the contract resp, err := w.RHPRenew(ctx, contract.ID, contract.EndHeight(), hk, contract.SiamuxAddr, settings.Address, state.address, renterFunds, minNewCollateral, expectedStorage, settings.WindowSize) diff --git a/autopilot/hostfilter.go b/autopilot/hostfilter.go index d43626973..a6d484ae3 100644 --- a/autopilot/hostfilter.go +++ b/autopilot/hostfilter.go @@ -297,19 +297,19 @@ func isOutOfFunds(cfg api.AutopilotConfig, pt rhpv3.HostPriceTable, c api.Contra // the contract is below a certain threshold of the collateral we would try to // put into a contract upon renew. func isOutOfCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, c api.Contract, s rhpv2.HostSettings, pt rhpv3.HostPriceTable) bool { - min := minRemainingCollateral(cfg, rs, c, s, pt) + min := minRemainingCollateral(cfg, rs, c.RenterFunds(), s, pt) return c.RemainingCollateral().Cmp(min) < 0 } // minNewCollateral returns the minimum amount of unallocated collateral that a // contract should contain after a refresh given the current amount of // unallocated collateral. -func minRemainingCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, c api.Contract, s rhpv2.HostSettings, pt rhpv3.HostPriceTable) types.Currency { +func minRemainingCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, renterFunds types.Currency, s rhpv2.HostSettings, pt rhpv3.HostPriceTable) types.Currency { // Compute the expected storage for the contract given its remaining funds. // Note: we use the full period here even though we are checking whether to // do a refresh. Otherwise, the 'expectedStorage' would would become // ridiculously large the closer the contract is to its end height. - expectedStorage := renterFundsToExpectedStorage(c.RenterFunds(), cfg.Contracts.Period, pt) + expectedStorage := renterFundsToExpectedStorage(renterFunds, cfg.Contracts.Period, pt) // Cap the expected storage at twice the ideal amount of data we expect to // store on a host. Even if we could afford more storage, there is no point From cf023d4de2caef0d6947e794bfcc2ecf37f1d4ce Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 18 Dec 2023 14:21:15 +0100 Subject: [PATCH 12/24] autopilot: TestMinRemainingCollateral --- autopilot/contractor.go | 2 +- autopilot/hostfilter.go | 5 ++ autopilot/hostfilter_test.go | 131 +++++++++++++++++++++++------------ 3 files changed, 92 insertions(+), 46 deletions(-) diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 554206654..1e06b65ab 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -1421,7 +1421,7 @@ func (c *contractor) refreshContract(ctx context.Context, w Worker, ci contractI unallocatedCollateral := rev.MissedHostPayout().Sub(contract.ContractPrice) // a refresh should always result in a contract that has enough collateral - minNewCollateral := minRemainingCollateral(state.cfg, state.rs, renterFunds, settings, ci.priceTable) + minNewCollateral := minRemainingCollateral(state.cfg, state.rs, renterFunds, settings, ci.priceTable).Mul64(2) // renew the contract resp, err := w.RHPRenew(ctx, contract.ID, contract.EndHeight(), hk, contract.SiamuxAddr, settings.Address, state.address, renterFunds, minNewCollateral, expectedStorage, settings.WindowSize) diff --git a/autopilot/hostfilter.go b/autopilot/hostfilter.go index a6d484ae3..2ebc81f38 100644 --- a/autopilot/hostfilter.go +++ b/autopilot/hostfilter.go @@ -327,6 +327,11 @@ func minRemainingCollateral(cfg api.AutopilotConfig, rs api.RedundancySettings, expectedStorage = s.RemainingStorage } + // If no storage is expected, return zero. + if expectedStorage == 0 { + return types.ZeroCurrency + } + // Computet the collateral for a single sector. _, sectorCollateral := pt.BaseCost(). Add(pt.AppendSectorCost(cfg.Contracts.Period)). diff --git a/autopilot/hostfilter_test.go b/autopilot/hostfilter_test.go index 1a7a16f56..ec9ea3943 100644 --- a/autopilot/hostfilter_test.go +++ b/autopilot/hostfilter_test.go @@ -1,47 +1,88 @@ package autopilot -//func TestMinNewCollateral(t *testing.T) { -// t.Parallel() -// -// // The collateral threshold is 10% meaning that we expect 10 times the -// // remaining collateral to be the minimum to trigger a renewal. -// if min := minNewCollateral(types.Siacoins(1)); !min.Equals(types.Siacoins(10)) { -// t.Fatalf("expected 10, got %v", min) -// } -//} -// -//func TestIsBelowCollateralThreshold(t *testing.T) { -// t.Parallel() -// -// tests := []struct { -// newCollateral types.Currency -// remainingCollateral types.Currency -// isBelow bool -// }{ -// { -// remainingCollateral: types.NewCurrency64(1), -// newCollateral: types.NewCurrency64(10), -// isBelow: true, -// }, -// { -// remainingCollateral: types.NewCurrency64(1), -// newCollateral: types.NewCurrency64(9), -// isBelow: false, -// }, -// { -// remainingCollateral: types.NewCurrency64(1), -// newCollateral: types.NewCurrency64(11), -// isBelow: true, -// }, -// { -// remainingCollateral: types.NewCurrency64(1), -// newCollateral: types.ZeroCurrency, -// isBelow: false, -// }, -// } -// for i, test := range tests { -// if isBelow := isBelowCollateralThreshold(test.newCollateral, test.remainingCollateral); isBelow != test.isBelow { -// t.Fatalf("%v: expected %v, got %v", i+1, test.isBelow, isBelow) -// } -// } -//} +import ( + "math" + "testing" + + rhpv2 "go.sia.tech/core/rhp/v2" + rhpv3 "go.sia.tech/core/rhp/v3" + "go.sia.tech/core/types" + "go.sia.tech/renterd/api" +) + +func TestMinRemainingCollateral(t *testing.T) { + t.Parallel() + + // consts + rs := api.RedundancySettings{MinShards: 1, TotalShards: 2} // 2x redundancy + cfg := api.AutopilotConfig{ + Contracts: api.ContractsConfig{ + Amount: 5, + Period: 100, + }, + } + one := types.NewCurrency64(1) + pt := rhpv3.HostPriceTable{ + CollateralCost: one, + InitBaseCost: one, + WriteBaseCost: one, + ReadBaseCost: one, + WriteLengthCost: one, + WriteStoreCost: one, + ReadLengthCost: one, + UploadBandwidthCost: one, + DownloadBandwidthCost: one, + } + s := rhpv2.HostSettings{} + _, sectorCollateral := pt.BaseCost(). + Add(pt.AppendSectorCost(cfg.Contracts.Period)). + Add(pt.ReadSectorCost(rhpv2.SectorSize)). + Total() + + // testcases + tests := []struct { + expectedStorage uint64 + remainingStorage uint64 + renterFunds types.Currency + expected types.Currency + }{ + { + // lots of funds but no remaining storage + expected: types.ZeroCurrency, + expectedStorage: 100, + remainingStorage: 0, + renterFunds: types.Siacoins(1000), + }, + { + // lots of funds but only 1 byte of remaining storage + expected: sectorCollateral.Mul64(3), + expectedStorage: 100, + remainingStorage: 1, + renterFunds: types.Siacoins(1000), + }, + { + // ideal data is capping the collateral + // 100 sectors * 2 (redundancy) * 2 (buffer) / 5 (hosts) / 20 (denom) = 4 sectors of collateral + expected: sectorCollateral.Mul64(4), // 100 sectors * 2 (redundancy) * 2 (buffer) + expectedStorage: 5 * rhpv2.SectorSize * minContractCollateralDenominator, // 100 sectors + remainingStorage: math.MaxUint64, + renterFunds: types.Siacoins(1000), + }, + { + // nothing is capping the expected storage + expected: types.NewCurrency64(17175674880), // ~13.65 x the previous 'expected' + expectedStorage: math.MaxUint32, + remainingStorage: math.MaxUint64, + renterFunds: types.Siacoins(1000), + }, + } + + for i, test := range tests { + cfg.Contracts.Storage = test.expectedStorage + s.RemainingStorage = test.remainingStorage + min := minRemainingCollateral(cfg, rs, test.renterFunds, s, pt) + if min.Cmp(test.expected) != 0 { + t.Fatalf("%v: expected %v, got %v", i+1, test.expected, min) + } + } +} From b783e0a04a4a4e7b9950cc1b4360b5701c68c485 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 18 Dec 2023 15:04:41 +0100 Subject: [PATCH 13/24] bus: extend objects stats response with unfinished object info --- api/object.go | 12 ++++++----- internal/testing/cluster_test.go | 26 +++++++++++++++++++++++ stores/metadata.go | 36 ++++++++++++++++++++++++++------ 3 files changed, 63 insertions(+), 11 deletions(-) diff --git a/api/object.go b/api/object.go index 80b24d341..4548eae8a 100644 --- a/api/object.go +++ b/api/object.go @@ -113,11 +113,13 @@ type ( // ObjectsStatsResponse is the response type for the /bus/stats/objects endpoint. ObjectsStatsResponse struct { - NumObjects uint64 `json:"numObjects"` // number of objects - MinHealth float64 `json:"minHealth"` // minimum health of all objects - TotalObjectsSize uint64 `json:"totalObjectsSize"` // size of all objects - TotalSectorsSize uint64 `json:"totalSectorsSize"` // uploaded size of all objects - TotalUploadedSize uint64 `json:"totalUploadedSize"` // uploaded size of all objects including redundant sectors + NumObjects uint64 `json:"numObjects"` // number of objects + NumUnfinishedObjects uint64 `json:"numUnfinishedObjects"` // number of unfinished objects + MinHealth float64 `json:"minHealth"` // minimum health of all objects + TotalObjectsSize uint64 `json:"totalObjectsSize"` // size of all objects + TotalUnfinishedObjectsSize uint64 `json:"totalUnfinishedObjectsSize"` // size of all unfinished objects + TotalSectorsSize uint64 `json:"totalSectorsSize"` // uploaded size of all objects + TotalUploadedSize uint64 `json:"totalUploadedSize"` // uploaded size of all objects including redundant sectors } ) diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index e5836a67b..ac599ebda 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1978,6 +1978,19 @@ func TestMultipartUploads(t *testing.T) { t.Fatal("unexpected part:", part3) } + // Check objects stats. + os, err := b.ObjectsStats() + tt.OK(err) + if os.NumObjects != 0 { + t.Fatalf("expected 0 object, got %v", os.NumObjects) + } else if os.TotalObjectsSize != 0 { + t.Fatalf("expected object size of 0, got %v", os.TotalObjectsSize) + } else if os.NumUnfinishedObjects != 1 { + t.Fatalf("expected 1 unfinished object, got %v", os.NumUnfinishedObjects) + } else if os.TotalUnfinishedObjectsSize != uint64(size) { + t.Fatalf("expected unfinished object size of %v, got %v", size, os.TotalUnfinishedObjectsSize) + } + // Complete upload ui, err := b.CompleteMultipartUpload(context.Background(), api.DefaultBucketName, objPath, mpr.UploadID, []api.MultipartCompletedPart{ { @@ -2023,6 +2036,19 @@ func TestMultipartUploads(t *testing.T) { } else if expectedData := data1[:1]; !bytes.Equal(data, expectedData) { t.Fatal("unexpected data:", cmp.Diff(data, expectedData)) } + + // Check objects stats. + os, err = b.ObjectsStats() + tt.OK(err) + if os.NumObjects != 1 { + t.Fatalf("expected 1 object, got %v", os.NumObjects) + } else if os.TotalObjectsSize != uint64(size) { + t.Fatalf("expected object size of %v, got %v", size, os.TotalObjectsSize) + } else if os.NumUnfinishedObjects != 0 { + t.Fatalf("expected 0 unfinished object, got %v", os.NumUnfinishedObjects) + } else if os.TotalUnfinishedObjectsSize != 0 { + t.Fatalf("expected unfinished object size of 0, got %v", os.TotalUnfinishedObjectsSize) + } } func TestWalletSendUnconfirmed(t *testing.T) { diff --git a/stores/metadata.go b/stores/metadata.go index 56f5e8aac..c86b9f81d 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -636,7 +636,7 @@ func (s *SQLStore) ListBuckets(ctx context.Context) ([]api.Bucket, error) { // reduce locking and make sure all results are consistent, everything is done // within a single transaction. func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, error) { - // Number of objects. + // number of objects var objInfo struct { NumObjects uint64 MinHealth float64 @@ -651,6 +651,28 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, return api.ObjectsStatsResponse{}, err } + // number of unfinished objects + var unfinishedObjects uint64 + err = s.db. + Model(&dbMultipartUpload{}). + Select("COUNT(*)"). + Scan(&unfinishedObjects). + Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + + // size of unfinished objects + var totalUnfinishedObjectsSize uint64 + err = s.db. + Model(&dbMultipartPart{}). + Select("COALESCE(SUM(size), 0)"). + Scan(&totalUnfinishedObjectsSize). + Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + var totalSectors uint64 batchSize := 500000 @@ -683,11 +705,13 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, } return api.ObjectsStatsResponse{ - MinHealth: objInfo.MinHealth, - NumObjects: objInfo.NumObjects, - TotalObjectsSize: objInfo.TotalObjectsSize, - TotalSectorsSize: totalSectors * rhpv2.SectorSize, - TotalUploadedSize: uint64(totalUploaded) * rhpv2.SectorSize, + MinHealth: objInfo.MinHealth, + NumObjects: objInfo.NumObjects, + NumUnfinishedObjects: unfinishedObjects, + TotalUnfinishedObjectsSize: totalUnfinishedObjectsSize, + TotalObjectsSize: objInfo.TotalObjectsSize, + TotalSectorsSize: totalSectors * rhpv2.SectorSize, + TotalUploadedSize: uint64(totalUploaded) * rhpv2.SectorSize, }, nil } From bd6f8d56b4756ec476482297e9ba0c25f804b8bf Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 18 Dec 2023 22:25:10 +0100 Subject: [PATCH 14/24] worker: fix panic --- worker/download.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/worker/download.go b/worker/download.go index c75687346..88d554d41 100644 --- a/worker/download.go +++ b/worker/download.go @@ -930,8 +930,14 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, return nil } + // select the fastest host + fastest := s.mgr.fastest(hosts) + if fastest == (types.PublicKey{}) { + return nil // can happen if downloader got stopped + } + // make the fastest host the current host - s.curr = s.mgr.fastest(hosts) + s.curr = fastest s.used[s.curr] = struct{}{} } From 55780ec5483107244b61e7bf63ec63c877a3cd59 Mon Sep 17 00:00:00 2001 From: Peter-Jan Brone Date: Tue, 19 Dec 2023 09:20:24 +0100 Subject: [PATCH 15/24] Contract Lock Ctx (#839) * worker: derive contract lock ctx * worker: pass shutdown ctx * autopilot: improve error handling --- autopilot/autopilot.go | 18 ++++++++++++------ worker/contract_lock.go | 6 +++--- worker/uploader.go | 2 +- 3 files changed, 16 insertions(+), 10 deletions(-) diff --git a/autopilot/autopilot.go b/autopilot/autopilot.go index 3e15bb66b..e72c3c17a 100644 --- a/autopilot/autopilot.go +++ b/autopilot/autopilot.go @@ -271,7 +271,9 @@ func (ap *Autopilot) Run() error { // perform maintenance setChanged, err := ap.c.performContractMaintenance(ctx, w) - if err != nil { + if err != nil && isErr(err, context.Canceled) { + return + } else if err != nil { ap.logger.Errorf("contract maintenance failed, err: %v", err) } maintenanceSuccess := err == nil @@ -288,8 +290,6 @@ func (ap *Autopilot) Run() error { ap.logger.Debug("account refills loop launched") go ap.a.refillWorkersAccountsLoop(ap.shutdownCtx) }) - } else { - ap.logger.Errorf("contract maintenance failed, err: %v", err) } // migration @@ -376,7 +376,9 @@ func (ap *Autopilot) blockUntilConfigured(interrupt <-chan time.Time) (configure cancel() // if the config was not found, or we were unable to fetch it, keep blocking - if err != nil && strings.Contains(err.Error(), api.ErrAutopilotNotFound.Error()) { + if isErr(err, context.Canceled) { + return + } else if isErr(err, api.ErrAutopilotNotFound) { once.Do(func() { ap.logger.Info("autopilot is waiting to be configured...") }) } else if err != nil { ap.logger.Errorf("autopilot is unable to fetch its configuration from the bus, err: %v", err) @@ -407,7 +409,9 @@ func (ap *Autopilot) blockUntilOnline() (online bool) { online = len(peers) > 0 cancel() - if err != nil { + if isErr(err, context.Canceled) { + return + } else if err != nil { ap.logger.Errorf("failed to get peers, err: %v", err) } else if !online { once.Do(func() { ap.logger.Info("autopilot is waiting on the bus to connect to peers...") }) @@ -439,7 +443,9 @@ func (ap *Autopilot) blockUntilSynced(interrupt <-chan time.Time) (synced, block cancel() // if an error occurred, or if we're not synced, we continue - if err != nil { + if isErr(err, context.Canceled) { + return + } else if err != nil { ap.logger.Errorf("failed to get consensus state, err: %v", err) } else if !synced { once.Do(func() { ap.logger.Info("autopilot is waiting for consensus to sync...") }) diff --git a/worker/contract_lock.go b/worker/contract_lock.go index 787e388b2..f5115d37f 100644 --- a/worker/contract_lock.go +++ b/worker/contract_lock.go @@ -31,8 +31,8 @@ type contractLock struct { stopWG sync.WaitGroup } -func newContractLock(fcid types.FileContractID, lockID uint64, d time.Duration, locker ContractLocker, logger *zap.SugaredLogger) *contractLock { - ctx, cancel := context.WithCancel(context.Background()) +func newContractLock(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration, locker ContractLocker, logger *zap.SugaredLogger) *contractLock { + ctx, cancel := context.WithCancel(ctx) cl := &contractLock{ lockID: lockID, fcid: fcid, @@ -56,7 +56,7 @@ func (w *worker) acquireContractLock(ctx context.Context, fcid types.FileContrac if err != nil { return nil, err } - return newContractLock(fcid, lockID, w.contractLockingDuration, w.bus, w.logger), nil + return newContractLock(w.shutdownCtx, fcid, lockID, w.contractLockingDuration, w.bus, w.logger), nil } func (w *worker) withContractLock(ctx context.Context, fcid types.FileContractID, priority int, fn func() error) error { diff --git a/worker/uploader.go b/worker/uploader.go index 7f0c46b0c..6057a52e5 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -198,7 +198,7 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, } // defer the release - lock := newContractLock(fcid, lockID, req.contractLockDuration, u.cl, u.logger) + lock := newContractLock(u.shutdownCtx, fcid, lockID, req.contractLockDuration, u.cl, u.logger) defer func() { ctx, cancel := context.WithTimeout(u.shutdownCtx, 10*time.Second) lock.Release(ctx) From 12ecaa6bfcf46e3798dec4272d657af5e5666bf1 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 19 Dec 2023 14:57:21 +0100 Subject: [PATCH 16/24] bus: add option to filter /contracts endpoint by contract set --- api/contract.go | 6 ++ autopilot/accounts.go | 7 +-- autopilot/autopilot.go | 3 +- autopilot/contract_pruning.go | 2 +- autopilot/contractor.go | 2 +- bus/bus.go | 18 ++++-- bus/client/contracts.go | 8 ++- internal/testing/cluster.go | 2 +- internal/testing/cluster_test.go | 39 ++++++++----- internal/testing/pruning_test.go | 2 +- internal/testing/uploads_test.go | 2 +- stores/metadata.go | 96 ++++++++++++++++++++++++++------ stores/metadata_test.go | 10 ++-- worker/upload.go | 2 +- worker/worker.go | 15 +++-- 15 files changed, 153 insertions(+), 61 deletions(-) diff --git a/api/contract.go b/api/contract.go index 29f75ea28..12d00e456 100644 --- a/api/contract.go +++ b/api/contract.go @@ -66,6 +66,8 @@ type ( RenewedFrom types.FileContractID `json:"renewedFrom"` Spending ContractSpending `json:"spending"` TotalCost types.Currency `json:"totalCost"` + + Sets []string `json:"sets"` } // ContractPrunableData wraps a contract's size information with its id. @@ -176,6 +178,10 @@ type ( TotalPrunable uint64 `json:"totalPrunable"` TotalSize uint64 `json:"totalSize"` } + + ContractsOpts struct { + ContractSet string `json:"contractset"` + } ) // Add returns the sum of the current and given contract spending. diff --git a/autopilot/accounts.go b/autopilot/accounts.go index 8f7e49c2f..416203de5 100644 --- a/autopilot/accounts.go +++ b/autopilot/accounts.go @@ -41,8 +41,7 @@ type AccountStore interface { } type ContractStore interface { - Contracts(ctx context.Context) ([]api.ContractMetadata, error) - ContractSetContracts(ctx context.Context, set string) ([]api.ContractMetadata, error) + Contracts(ctx context.Context, opts api.ContractsOpts) ([]api.ContractMetadata, error) } func newAccounts(ap *Autopilot, a AccountStore, c ContractStore, w *workerPool, l *zap.SugaredLogger, refillInterval time.Duration) *accounts { @@ -114,7 +113,7 @@ func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { } // fetch all contracts - contracts, err := a.c.Contracts(ctx) + contracts, err := a.c.Contracts(ctx, api.ContractsOpts{}) if err != nil { a.l.Errorw(fmt.Sprintf("failed to fetch contracts for refill: %v", err)) return @@ -123,7 +122,7 @@ func (a *accounts) refillWorkerAccounts(ctx context.Context, w Worker) { } // fetch all contract set contracts - contractSetContracts, err := a.c.ContractSetContracts(ctx, state.cfg.Contracts.Set) + contractSetContracts, err := a.c.Contracts(ctx, api.ContractsOpts{ContractSet: state.cfg.Contracts.Set}) if err != nil { a.l.Errorw(fmt.Sprintf("failed to fetch contract set contracts: %v", err)) return diff --git a/autopilot/autopilot.go b/autopilot/autopilot.go index e72c3c17a..344b1fddd 100644 --- a/autopilot/autopilot.go +++ b/autopilot/autopilot.go @@ -45,8 +45,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 Contract(ctx context.Context, id types.FileContractID) (api.ContractMetadata, error) - Contracts(ctx context.Context) (contracts []api.ContractMetadata, err error) - ContractSetContracts(ctx context.Context, set string) ([]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) SetContractSet(ctx context.Context, set string, contracts []types.FileContractID) error PrunableData(ctx context.Context) (prunableData api.ContractsPrunableDataResponse, err error) diff --git a/autopilot/contract_pruning.go b/autopilot/contract_pruning.go index 4d0cd5f46..f115da84f 100644 --- a/autopilot/contract_pruning.go +++ b/autopilot/contract_pruning.go @@ -103,7 +103,7 @@ func (c *contractor) fetchPrunableContracts() (prunable []api.ContractPrunableDa } // fetch contract set contracts - csc, err := c.ap.bus.ContractSetContracts(ctx, c.ap.state.cfg.Contracts.Set) + csc, err := c.ap.bus.Contracts(ctx, api.ContractsOpts{ContractSet: c.ap.state.cfg.Contracts.Set}) if err != nil { return nil, err } diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 1e06b65ab..f0a77f16e 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -180,7 +180,7 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( } // fetch current contract set - currentSet, err := c.ap.bus.ContractSetContracts(ctx, state.cfg.Contracts.Set) + currentSet, err := c.ap.bus.Contracts(ctx, api.ContractsOpts{ContractSet: state.cfg.Contracts.Set}) if err != nil && !strings.Contains(err.Error(), api.ErrContractSetNotFound.Error()) { return false, err } diff --git a/bus/bus.go b/bus/bus.go index eb5467398..25fd0be18 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -115,8 +115,7 @@ type ( ArchiveContracts(ctx context.Context, toArchive map[types.FileContractID]string) error ArchiveAllContracts(ctx context.Context, reason string) error Contract(ctx context.Context, id types.FileContractID) (api.ContractMetadata, error) - Contracts(ctx context.Context) ([]api.ContractMetadata, error) - ContractSetContracts(ctx context.Context, set string) ([]api.ContractMetadata, error) + Contracts(ctx context.Context, opts api.ContractsOpts) ([]api.ContractMetadata, error) ContractSets(ctx context.Context) ([]string, error) RecordContractSpending(ctx context.Context, records []api.ContractSpendingRecord) error RemoveContractSet(ctx context.Context, name string) error @@ -903,9 +902,15 @@ func (b *bus) hostsBlocklistHandlerPUT(jc jape.Context) { } func (b *bus) contractsHandlerGET(jc jape.Context) { - cs, err := b.ms.Contracts(jc.Request.Context()) + var cs string + if jc.DecodeForm("contractset", &cs) != nil { + return + } + contracts, err := b.ms.Contracts(jc.Request.Context(), api.ContractsOpts{ + ContractSet: cs, + }) if jc.Check("couldn't load contracts", err) == nil { - jc.Encode(cs) + jc.Encode(contracts) } } @@ -931,7 +936,10 @@ func (b *bus) contractsArchiveHandlerPOST(jc jape.Context) { } func (b *bus) contractsSetHandlerGET(jc jape.Context) { - cs, err := b.ms.ContractSetContracts(jc.Request.Context(), jc.PathParam("set")) + cs, err := b.ms.Contracts(jc.Request.Context(), + api.ContractsOpts{ + ContractSet: jc.PathParam("set"), + }) if jc.Check("couldn't load contracts", err) == nil { jc.Encode(cs) } diff --git a/bus/client/contracts.go b/bus/client/contracts.go index 0e1927daf..336265aeb 100644 --- a/bus/client/contracts.go +++ b/bus/client/contracts.go @@ -102,8 +102,12 @@ func (c *Client) ContractSetContracts(ctx context.Context, set string) (contract } // Contracts returns all contracts in the metadata store. -func (c *Client) Contracts(ctx context.Context) (contracts []api.ContractMetadata, err error) { - err = c.c.WithContext(ctx).GET("/contracts", &contracts) +func (c *Client) Contracts(ctx context.Context, opts api.ContractsOpts) (contracts []api.ContractMetadata, err error) { + values := url.Values{} + if opts.ContractSet != "" { + values.Set("contractset", opts.ContractSet) + } + err = c.c.WithContext(ctx).GET("/contracts"+values.Encode(), &contracts) return } diff --git a/internal/testing/cluster.go b/internal/testing/cluster.go index 2011cdceb..3632ef799 100644 --- a/internal/testing/cluster.go +++ b/internal/testing/cluster.go @@ -881,7 +881,7 @@ func (c *TestCluster) WaitForContractSet(set string, n int) { func (c *TestCluster) waitForHostContracts(hosts map[types.PublicKey]struct{}) { c.tt.Helper() c.tt.Retry(300, 100*time.Millisecond, func() error { - contracts, err := c.Bus.Contracts(context.Background()) + contracts, err := c.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { return err } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index ac599ebda..80f359218 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -77,12 +77,25 @@ func TestNewTestCluster(t *testing.T) { t.Fatal("TotalCost and ContractPrice shouldn't be zero") } + // Make sure the contracts are part of the set. + busContracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) + if err != nil { + t.Fatal(err) + } + for _, c := range busContracts { + if len(c.Sets) != 1 { + t.Fatal("contract should be part of one set", len(c.Sets)) + } else if c.Sets[0] != sets[0] { + t.Fatalf("contract should be part of set %v but was %v", sets[0], c.Sets[0]) + } + } + // Mine blocks until contracts start renewing. cluster.MineToRenewWindow() // Wait for the contract to be renewed. tt.Retry(100, 100*time.Millisecond, func() error { - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { return err } @@ -120,7 +133,7 @@ func TestNewTestCluster(t *testing.T) { cluster.MineBlocks(1) // Fetch renewed contract and make sure we caught the proof and revision. - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { t.Fatal(err) } @@ -567,7 +580,7 @@ func TestUploadDownloadBasic(t *testing.T) { } // fetch the contracts. - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) // broadcast the revision for each contract and assert the revision height @@ -585,7 +598,7 @@ func TestUploadDownloadBasic(t *testing.T) { // check the revision height was updated. tt.Retry(100, 100*time.Millisecond, func() error { // fetch the contracts. - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { return err } @@ -755,7 +768,7 @@ func TestUploadDownloadSpending(t *testing.T) { // check that the funding was recorded tt.Retry(100, testBusFlushInterval, func() error { - cms, err := cluster.Bus.Contracts(context.Background()) + cms, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(cms) == 0 { t.Fatal("no contracts found") @@ -849,7 +862,7 @@ func TestUploadDownloadSpending(t *testing.T) { // wait for the contract to be renewed tt.Retry(100, 100*time.Millisecond, func() error { // fetch contracts - cms, err := cluster.Bus.Contracts(context.Background()) + cms, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(cms) == 0 { t.Fatal("no contracts found") @@ -880,7 +893,7 @@ func TestUploadDownloadSpending(t *testing.T) { // check that the spending was recorded tt.Retry(100, testBusFlushInterval, func() error { - cms, err := cluster.Bus.Contracts(context.Background()) + cms, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { t.Fatal(err) } @@ -1301,7 +1314,7 @@ func TestContractArchival(t *testing.T) { tt := cluster.tt // check that we have 1 contract - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(contracts) != 1 { t.Fatal("expected 1 contract", len(contracts)) @@ -1318,7 +1331,7 @@ func TestContractArchival(t *testing.T) { // check that we have 0 contracts tt.Retry(100, 100*time.Millisecond, func() error { - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { return err } @@ -1346,7 +1359,7 @@ func TestUnconfirmedContractArchival(t *testing.T) { tt.OK(err) // we should have a contract with the host - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(contracts) != 1 { t.Fatalf("expected 1 contract, got %v", len(contracts)) @@ -1377,7 +1390,7 @@ func TestUnconfirmedContractArchival(t *testing.T) { tt.OK(err) // should have 2 contracts now - contracts, err = cluster.Bus.Contracts(context.Background()) + contracts, err = cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(contracts) != 2 { t.Fatalf("expected 2 contracts, got %v", len(contracts)) @@ -1388,7 +1401,7 @@ func TestUnconfirmedContractArchival(t *testing.T) { cluster.MineBlocks(20) tt.Retry(100, 100*time.Millisecond, func() error { - contracts, err := cluster.Bus.Contracts(context.Background()) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(contracts) != 1 { return fmt.Errorf("expected 1 contract, got %v", len(contracts)) @@ -2158,7 +2171,7 @@ func TestWalletFormUnconfirmed(t *testing.T) { } // There shouldn't be any contracts at this point. - contracts, err := b.Contracts(context.Background()) + contracts, err := b.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) if len(contracts) != 0 { t.Fatal("expected 0 contracts", len(contracts)) diff --git a/internal/testing/pruning_test.go b/internal/testing/pruning_test.go index 973c0b834..d9f9ff26d 100644 --- a/internal/testing/pruning_test.go +++ b/internal/testing/pruning_test.go @@ -160,7 +160,7 @@ func TestSectorPruning(t *testing.T) { cluster.ShutdownAutopilot(context.Background()) // create a contracts dict - contracts, err := b.Contracts(context.Background()) + contracts, err := b.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) // compare database against roots returned by the host diff --git a/internal/testing/uploads_test.go b/internal/testing/uploads_test.go index 0ad328911..e3e938120 100644 --- a/internal/testing/uploads_test.go +++ b/internal/testing/uploads_test.go @@ -78,7 +78,7 @@ func TestUploadingSectorsCache(t *testing.T) { } // fetch contracts - contracts, err := b.Contracts(context.Background()) + contracts, err := b.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) // fetch pending roots for all contracts diff --git a/stores/metadata.go b/stores/metadata.go index c86b9f81d..05ee9fbe2 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -64,6 +64,8 @@ type ( HostID uint `gorm:"index"` Host dbHost + + ContractSets []dbContractSet `gorm:"many2many:contract_set_contracts;constraint:OnDelete:CASCADE"` } ContractCommon struct { @@ -330,6 +332,10 @@ func (c dbArchivedContract) convert() api.ArchivedContract { func (c dbContract) convert() api.ContractMetadata { var revisionNumber uint64 _, _ = fmt.Sscan(c.RevisionNumber, &revisionNumber) + var contractSets []string + for _, cs := range c.ContractSets { + contractSets = append(contractSets, cs.Name) + } return api.ContractMetadata{ ContractPrice: types.Currency(c.ContractPrice), ID: types.FileContractID(c.FCID), @@ -349,6 +355,7 @@ func (c dbContract) convert() api.ContractMetadata { ProofHeight: c.ProofHeight, RevisionHeight: c.RevisionHeight, RevisionNumber: revisionNumber, + Sets: contractSets, Size: c.Size, StartHeight: c.StartHeight, State: c.State.String(), @@ -756,20 +763,82 @@ func (s *SQLStore) AddContract(ctx context.Context, c rhpv2.ContractRevision, co return added.convert(), nil } -func (s *SQLStore) Contracts(ctx context.Context) ([]api.ContractMetadata, error) { - var dbContracts []dbContract +func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api.ContractMetadata, error) { + // helper to check whether a contract set exists + hasContractSet := func() error { + if opts.ContractSet == "" { + return nil + } + err := s.db.Where("name", opts.ContractSet).Take(&dbContractSet{}).Error + if errors.Is(err, gorm.ErrRecordNotFound) { + return api.ErrContractSetNotFound + } + return err + } + + // fetch all contracts, their hosts and the contract set name + var dbContracts []struct { + Contract dbContract `gorm:"embedded"` + Host dbHost `gorm:"embedded,embeddedPrefix:Host__"` + Name string + } err := s.db. Model(&dbContract{}). + Select("contracts.*, Host.*, cs.name as Name"). Joins("Host"). - Find(&dbContracts). + Joins("LEFT JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). + Joins("LEFT JOIN contract_sets cs ON cs.id = csc.db_contract_set_id"). + Scan(&dbContracts). Error if err != nil { return nil, err + } else if len(dbContracts) == 0 { + return nil, hasContractSet() } - contracts := make([]api.ContractMetadata, len(dbContracts)) - for i, c := range dbContracts { - contracts[i] = c.convert() + // merge 'Host' and 'Name' into 'Contract' + for i := range dbContracts { + dbContracts[i].Contract.Host = dbContracts[i].Host + if dbContracts[i].Name != "" { + dbContracts[i].Contract.ContractSets = append(dbContracts[i].Contract.ContractSets, dbContractSet{Name: dbContracts[i].Name}) + } + } + + // filter out contracts that don't contain the required contract set + var contracts []api.ContractMetadata + appendContract := func(c api.ContractMetadata) { + if opts.ContractSet == "" { + contracts = append(contracts, c) + return + } + var found bool + for _, cs := range c.Sets { + if cs == opts.ContractSet { + found = true + break + } + } + if found { + contracts = append(contracts, c) + } + } + + current, dbContracts := dbContracts[0], dbContracts[1:] + for { + if len(dbContracts) == 0 { + appendContract(current.Contract.convert()) + break + } else if current.Contract.ID != dbContracts[0].Contract.ID { + appendContract(current.Contract.convert()) + } else if len(dbContracts[0].Contract.ContractSets) > 0 { + current.Contract.ContractSets = append(current.Contract.ContractSets, dbContracts[0].Contract.ContractSets...) + } + current, dbContracts = dbContracts[0], dbContracts[1:] + } + + // if no contracts are left, check if the set existed in the first place + if len(contracts) == 0 { + return nil, hasContractSet() } return contracts, nil } @@ -915,15 +984,9 @@ WHERE c.fcid = ? } func (s *SQLStore) ContractSetContracts(ctx context.Context, set string) ([]api.ContractMetadata, error) { - dbContracts, err := s.contracts(ctx, set) - if err != nil { - return nil, err - } - contracts := make([]api.ContractMetadata, len(dbContracts)) - for i, c := range dbContracts { - contracts[i] = c.convert() - } - return contracts, nil + return s.Contracts(ctx, api.ContractsOpts{ + ContractSet: set, + }) } func (s *SQLStore) ContractSets(ctx context.Context) ([]string, error) { @@ -2334,7 +2397,8 @@ func contractsForHost(tx *gorm.DB, host dbHost) (contracts []dbContract, err err func newContract(hostID uint, fcid, renewedFrom types.FileContractID, contractPrice, totalCost types.Currency, startHeight, windowStart, windowEnd, size uint64, state contractState) dbContract { return dbContract{ - HostID: hostID, + HostID: hostID, + ContractSets: nil, // new contract isn't in a set yet ContractCommon: ContractCommon{ FCID: fileContractID(fcid), diff --git a/stores/metadata_test.go b/stores/metadata_test.go index ecf6849ee..0e610309e 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -207,7 +207,7 @@ func TestSQLContractStore(t *testing.T) { if !errors.Is(err, api.ErrContractNotFound) { t.Fatal(err) } - contracts, err := ss.Contracts(ctx) + contracts, err := ss.Contracts(ctx, api.ContractsOpts{}) if err != nil { t.Fatal(err) } @@ -253,7 +253,7 @@ func TestSQLContractStore(t *testing.T) { if !reflect.DeepEqual(fetched, expected) { t.Fatal("contract mismatch") } - contracts, err = ss.Contracts(ctx) + contracts, err = ss.Contracts(ctx, api.ContractsOpts{}) if err != nil { t.Fatal(err) } @@ -304,7 +304,7 @@ func TestSQLContractStore(t *testing.T) { if !errors.Is(err, api.ErrContractNotFound) { t.Fatal(err) } - contracts, err = ss.Contracts(ctx) + contracts, err = ss.Contracts(ctx, api.ContractsOpts{}) if err != nil { t.Fatal(err) } @@ -577,7 +577,7 @@ func TestRenewedContract(t *testing.T) { } // make sure the contract set was updated. - setContracts, err := ss.ContractSetContracts(context.Background(), "test") + setContracts, err := ss.Contracts(ctx, api.ContractsOpts{ContractSet: "test"}) if err != nil { t.Fatal(err) } @@ -770,7 +770,7 @@ func TestArchiveContracts(t *testing.T) { } // assert the first one is still active - active, err := ss.Contracts(context.Background()) + active, err := ss.Contracts(context.Background(), api.ContractsOpts{}) if err != nil { t.Fatal(err) } diff --git a/worker/upload.go b/worker/upload.go index 99ca7efc2..f3fc3754c 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -289,7 +289,7 @@ func (w *worker) uploadPackedSlab(ctx context.Context, rs api.RedundancySettings defer cancel() // fetch contracts - contracts, err := w.bus.ContractSetContracts(ctx, contractSet) + contracts, err := w.bus.Contracts(ctx, api.ContractsOpts{ContractSet: contractSet}) if err != nil { return fmt.Errorf("couldn't fetch packed slabs from bus: %v", err) } diff --git a/worker/worker.go b/worker/worker.go index efc1e9e60..8f0717b6f 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -78,8 +78,7 @@ type ( Contract(ctx context.Context, id types.FileContractID) (api.ContractMetadata, error) ContractSize(ctx context.Context, id types.FileContractID) (api.ContractSize, error) ContractRoots(ctx context.Context, id types.FileContractID) ([]types.Hash256, []types.Hash256, error) - Contracts(ctx context.Context) ([]api.ContractMetadata, error) - ContractSetContracts(ctx context.Context, set string) ([]api.ContractMetadata, error) + Contracts(ctx context.Context, opts api.ContractsOpts) ([]api.ContractMetadata, error) RenewedContract(ctx context.Context, renewedFrom types.FileContractID) (api.ContractMetadata, error) RecordHostScans(ctx context.Context, scans []hostdb.HostScan) error @@ -766,13 +765,13 @@ func (w *worker) slabMigrateHandler(jc jape.Context) { ctx = WithGougingChecker(ctx, w.bus, up.GougingParams) // fetch all contracts - dlContracts, err := w.bus.Contracts(ctx) + dlContracts, err := w.bus.Contracts(ctx, api.ContractsOpts{}) if jc.Check("couldn't fetch contracts from bus", err) != nil { return } // fetch upload contracts - ulContracts, err := w.bus.ContractSetContracts(ctx, up.ContractSet) + ulContracts, err := w.bus.Contracts(ctx, api.ContractsOpts{ContractSet: up.ContractSet}) if jc.Check("couldn't fetch contracts from bus", err) != nil { return } @@ -907,7 +906,7 @@ func (w *worker) objectsHandlerGET(jc jape.Context) { } // fetch all contracts - contracts, err := w.bus.Contracts(ctx) + contracts, err := w.bus.Contracts(ctx, api.ContractsOpts{}) if err != nil { jc.Error(err, http.StatusInternalServerError) return @@ -1015,7 +1014,7 @@ func (w *worker) objectsHandlerPUT(jc jape.Context) { ctx = WithGougingChecker(ctx, w.bus, up.GougingParams) // fetch contracts - contracts, err := w.bus.ContractSetContracts(ctx, up.ContractSet) + contracts, err := w.bus.Contracts(ctx, api.ContractsOpts{ContractSet: up.ContractSet}) if jc.Check("couldn't fetch contracts from bus", err) != nil { return } @@ -1154,7 +1153,7 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { ctx = WithGougingChecker(ctx, w.bus, up.GougingParams) // fetch contracts - contracts, err := w.bus.ContractSetContracts(ctx, up.ContractSet) + contracts, err := w.bus.Contracts(ctx, api.ContractsOpts{ContractSet: up.ContractSet}) if jc.Check("couldn't fetch contracts from bus", err) != nil { return } @@ -1197,7 +1196,7 @@ func (w *worker) rhpContractsHandlerGET(jc jape.Context) { ctx := jc.Request.Context() // fetch contracts - busContracts, err := w.bus.Contracts(ctx) + busContracts, err := w.bus.Contracts(ctx, api.ContractsOpts{}) if jc.Check("failed to fetch contracts from bus", err) != nil { return } From f2c16ef1b5052f3a6fc2548bdfbee88ed8fad6f2 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 19 Dec 2023 15:11:26 +0100 Subject: [PATCH 17/24] jape: fix check --- bus/client/contracts.go | 2 +- stores/metadata.go | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/bus/client/contracts.go b/bus/client/contracts.go index 336265aeb..48ffc979f 100644 --- a/bus/client/contracts.go +++ b/bus/client/contracts.go @@ -107,7 +107,7 @@ func (c *Client) Contracts(ctx context.Context, opts api.ContractsOpts) (contrac if opts.ContractSet != "" { values.Set("contractset", opts.ContractSet) } - err = c.c.WithContext(ctx).GET("/contracts"+values.Encode(), &contracts) + err = c.c.WithContext(ctx).GET("/contracts?"+values.Encode(), &contracts) return } diff --git a/stores/metadata.go b/stores/metadata.go index 05ee9fbe2..75d094541 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -788,6 +788,7 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api Joins("Host"). Joins("LEFT JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). Joins("LEFT JOIN contract_sets cs ON cs.id = csc.db_contract_set_id"). + Order("contracts.id ASC"). Scan(&dbContracts). Error if err != nil { From 1917aed3dfcef049f109642fb8269e865b417e05 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 19 Dec 2023 15:53:45 +0100 Subject: [PATCH 18/24] testing: fix TestUnconfirmedContractArchival --- stores/metadata.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 75d094541..a5631d082 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -779,13 +779,13 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api // fetch all contracts, their hosts and the contract set name var dbContracts []struct { Contract dbContract `gorm:"embedded"` - Host dbHost `gorm:"embedded,embeddedPrefix:Host__"` + Host dbHost `gorm:"embedded"` Name string } err := s.db. Model(&dbContract{}). - Select("contracts.*, Host.*, cs.name as Name"). - Joins("Host"). + Select("contracts.*, h.*, cs.name as Name"). + Joins("INNER JOIN hosts h ON h.id = contracts.host_id"). Joins("LEFT JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). Joins("LEFT JOIN contract_sets cs ON cs.id = csc.db_contract_set_id"). Order("contracts.id ASC"). From 14baeef59b8a2b144d68ba3563b59850a0d98692 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 19 Dec 2023 15:59:12 +0100 Subject: [PATCH 19/24] stores: call query result rows --- stores/metadata.go | 30 +++++++++++++++++------------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index a5631d082..8bdb882e8 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -777,7 +777,7 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api } // fetch all contracts, their hosts and the contract set name - var dbContracts []struct { + var rows []struct { Contract dbContract `gorm:"embedded"` Host dbHost `gorm:"embedded"` Name string @@ -789,20 +789,23 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api Joins("LEFT JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). Joins("LEFT JOIN contract_sets cs ON cs.id = csc.db_contract_set_id"). Order("contracts.id ASC"). - Scan(&dbContracts). + Scan(&rows). Error if err != nil { return nil, err - } else if len(dbContracts) == 0 { + } else if len(rows) == 0 { return nil, hasContractSet() } - // merge 'Host' and 'Name' into 'Contract' - for i := range dbContracts { - dbContracts[i].Contract.Host = dbContracts[i].Host - if dbContracts[i].Name != "" { - dbContracts[i].Contract.ContractSets = append(dbContracts[i].Contract.ContractSets, dbContractSet{Name: dbContracts[i].Name}) + // merge 'Host', 'Name' and 'Contract' into dbContracts + var dbContracts []dbContract + for i := range rows { + dbContract := rows[i].Contract + dbContract.Host = rows[i].Host + if rows[i].Name != "" { + dbContract.ContractSets = append(dbContract.ContractSets, dbContractSet{Name: rows[i].Name}) } + dbContracts = append(dbContracts, dbContract) } // filter out contracts that don't contain the required contract set @@ -824,15 +827,16 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api } } + // merge contract sets current, dbContracts := dbContracts[0], dbContracts[1:] for { if len(dbContracts) == 0 { - appendContract(current.Contract.convert()) + appendContract(current.convert()) break - } else if current.Contract.ID != dbContracts[0].Contract.ID { - appendContract(current.Contract.convert()) - } else if len(dbContracts[0].Contract.ContractSets) > 0 { - current.Contract.ContractSets = append(current.Contract.ContractSets, dbContracts[0].Contract.ContractSets...) + } else if current.ID != dbContracts[0].ID { + appendContract(current.convert()) + } else if len(dbContracts[0].ContractSets) > 0 { + current.ContractSets = append(current.ContractSets, dbContracts[0].ContractSets...) } current, dbContracts = dbContracts[0], dbContracts[1:] } From 0b0b6e7df06b230c36333ad5418a97ab8fef34c5 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 19 Dec 2023 16:23:10 +0100 Subject: [PATCH 20/24] stores: filter in query --- stores/metadata.go | 39 ++++++++++++++++----------------------- 1 file changed, 16 insertions(+), 23 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 8bdb882e8..27f974de3 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -782,8 +782,19 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api Host dbHost `gorm:"embedded"` Name string } - err := s.db. - Model(&dbContract{}). + tx := s.db + if opts.ContractSet == "" { + // no filter, use all contracts + tx = tx.Table("contracts") + } else { + // filter contracts by contract set first + tx = tx.Table("(?) contracts", s.db.Model(&dbContract{}). + Select("contracts.*"). + Joins("INNER JOIN hosts h ON h.id = contracts.host_id"). + Joins("INNER JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). + Joins("INNER JOIN contract_sets cs ON cs.id = csc.db_contract_set_id AND cs.name = ?", opts.ContractSet)) + } + err := tx. Select("contracts.*, h.*, cs.name as Name"). Joins("INNER JOIN hosts h ON h.id = contracts.host_id"). Joins("LEFT JOIN contract_set_contracts csc ON csc.db_contract_id = contracts.id"). @@ -808,33 +819,15 @@ func (s *SQLStore) Contracts(ctx context.Context, opts api.ContractsOpts) ([]api dbContracts = append(dbContracts, dbContract) } - // filter out contracts that don't contain the required contract set - var contracts []api.ContractMetadata - appendContract := func(c api.ContractMetadata) { - if opts.ContractSet == "" { - contracts = append(contracts, c) - return - } - var found bool - for _, cs := range c.Sets { - if cs == opts.ContractSet { - found = true - break - } - } - if found { - contracts = append(contracts, c) - } - } - // merge contract sets + var contracts []api.ContractMetadata current, dbContracts := dbContracts[0], dbContracts[1:] for { if len(dbContracts) == 0 { - appendContract(current.convert()) + contracts = append(contracts, current.convert()) break } else if current.ID != dbContracts[0].ID { - appendContract(current.convert()) + contracts = append(contracts, current.convert()) } else if len(dbContracts[0].ContractSets) > 0 { current.ContractSets = append(current.ContractSets, dbContracts[0].ContractSets...) } From 1f6cea82cdf9746a4e9b6b1940a3812d7bff48a7 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 20 Dec 2023 15:08:17 +0100 Subject: [PATCH 21/24] api: address comments --- api/contract.go | 2 +- bus/client/contracts.go | 3 ++- internal/testing/cluster_test.go | 8 ++++---- stores/metadata.go | 2 +- 4 files changed, 8 insertions(+), 7 deletions(-) diff --git a/api/contract.go b/api/contract.go index 12d00e456..c87830c29 100644 --- a/api/contract.go +++ b/api/contract.go @@ -67,7 +67,7 @@ type ( Spending ContractSpending `json:"spending"` TotalCost types.Currency `json:"totalCost"` - Sets []string `json:"sets"` + ContractSets []string `json:"contractsets"` } // ContractPrunableData wraps a contract's size information with its id. diff --git a/bus/client/contracts.go b/bus/client/contracts.go index 48ffc979f..93d137661 100644 --- a/bus/client/contracts.go +++ b/bus/client/contracts.go @@ -101,7 +101,8 @@ func (c *Client) ContractSetContracts(ctx context.Context, set string) (contract return } -// Contracts returns all contracts in the metadata store. +// Contracts retrieves contracts from the metadata store. If no filter is set, +// all contracts are returned. func (c *Client) Contracts(ctx context.Context, opts api.ContractsOpts) (contracts []api.ContractMetadata, err error) { values := url.Values{} if opts.ContractSet != "" { diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 80f359218..ba231f050 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -83,10 +83,10 @@ func TestNewTestCluster(t *testing.T) { t.Fatal(err) } for _, c := range busContracts { - if len(c.Sets) != 1 { - t.Fatal("contract should be part of one set", len(c.Sets)) - } else if c.Sets[0] != sets[0] { - t.Fatalf("contract should be part of set %v but was %v", sets[0], c.Sets[0]) + if len(c.ContractSets) != 1 { + t.Fatal("contract should be part of one set", len(c.ContractSets)) + } else if c.ContractSets[0] != sets[0] { + t.Fatalf("contract should be part of set %v but was %v", sets[0], c.ContractSets[0]) } } diff --git a/stores/metadata.go b/stores/metadata.go index 27f974de3..daa708309 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -355,7 +355,7 @@ func (c dbContract) convert() api.ContractMetadata { ProofHeight: c.ProofHeight, RevisionHeight: c.RevisionHeight, RevisionNumber: revisionNumber, - Sets: contractSets, + ContractSets: contractSets, Size: c.Size, StartHeight: c.StartHeight, State: c.State.String(), From 77187e1d3a1572a4f45765029eac86bec34a1c41 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 20 Dec 2023 15:14:32 +0100 Subject: [PATCH 22/24] stores: add init migration --- stores/migrations.go | 4 ++++ stores/migrations_metrics.go | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/stores/migrations.go b/stores/migrations.go index 94708a8d4..374751a3b 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -317,6 +317,10 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { return performMigration00036_contractPruneCfg(tx, logger) }, }, + { + ID: "00001_init", + Migrate: func(tx *gorm.DB) error { return nil }, + }, } // Create migrator. m := gormigrate.New(db, gormigrate.DefaultOptions, migrations) diff --git a/stores/migrations_metrics.go b/stores/migrations_metrics.go index 08f2c5ce3..f38595c80 100644 --- a/stores/migrations_metrics.go +++ b/stores/migrations_metrics.go @@ -47,6 +47,10 @@ func performMetricsMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { }, Rollback: nil, }, + { + ID: "00001_init", + Migrate: func(tx *gorm.DB) error { return nil }, + }, } // Create migrator. From 035dc1826fc306037f01a467ec1d4f428486bc2f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 20 Dec 2023 16:10:59 +0100 Subject: [PATCH 23/24] bus: remove contract set contracts endpoint --- bus/bus.go | 11 ----------- bus/client/contracts.go | 11 ----------- internal/testing/blocklist_test.go | 8 ++++---- internal/testing/cluster.go | 2 +- internal/testing/cluster_test.go | 4 ++-- internal/testing/gouging_test.go | 2 +- stores/metadata.go | 6 ------ stores/metadata_test.go | 8 ++++---- 8 files changed, 12 insertions(+), 40 deletions(-) diff --git a/bus/bus.go b/bus/bus.go index 25fd0be18..e0f725270 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -270,7 +270,6 @@ func (b *bus) Handler() http.Handler { "GET /contracts/prunable": b.contractsPrunableDataHandlerGET, "GET /contracts/renewed/:id": b.contractsRenewedIDHandlerGET, "GET /contracts/sets": b.contractsSetsHandlerGET, - "GET /contracts/set/:set": b.contractsSetHandlerGET, "PUT /contracts/set/:set": b.contractsSetHandlerPUT, "DELETE /contracts/set/:set": b.contractsSetHandlerDELETE, "POST /contracts/spending": b.contractsSpendingHandlerPOST, @@ -935,16 +934,6 @@ func (b *bus) contractsArchiveHandlerPOST(jc jape.Context) { jc.Check("failed to archive contracts", b.ms.ArchiveContracts(jc.Request.Context(), toArchive)) } -func (b *bus) contractsSetHandlerGET(jc jape.Context) { - cs, err := b.ms.Contracts(jc.Request.Context(), - api.ContractsOpts{ - ContractSet: jc.PathParam("set"), - }) - if jc.Check("couldn't load contracts", err) == nil { - jc.Encode(cs) - } -} - func (b *bus) contractsSetsHandlerGET(jc jape.Context) { sets, err := b.ms.ContractSets(jc.Request.Context()) if jc.Check("couldn't fetch contract sets", err) == nil { diff --git a/bus/client/contracts.go b/bus/client/contracts.go index 93d137661..84cd7dc88 100644 --- a/bus/client/contracts.go +++ b/bus/client/contracts.go @@ -2,7 +2,6 @@ package client import ( "context" - "errors" "fmt" "net/url" "time" @@ -91,16 +90,6 @@ func (c *Client) ContractSize(ctx context.Context, contractID types.FileContract return } -// ContractSetContracts returns the contracts for the given set from the -// metadata store. -func (c *Client) ContractSetContracts(ctx context.Context, set string) (contracts []api.ContractMetadata, err error) { - if set == "" { - return nil, errors.New("set cannot be empty") - } - err = c.c.WithContext(ctx).GET(fmt.Sprintf("/contracts/set/%s", set), &contracts) - return -} - // Contracts retrieves contracts from the metadata store. If no filter is set, // all contracts are returned. func (c *Client) Contracts(ctx context.Context, opts api.ContractsOpts) (contracts []api.ContractMetadata, err error) { diff --git a/internal/testing/blocklist_test.go b/internal/testing/blocklist_test.go index 506f97982..9d9a12605 100644 --- a/internal/testing/blocklist_test.go +++ b/internal/testing/blocklist_test.go @@ -26,7 +26,7 @@ func TestBlocklist(t *testing.T) { tt := cluster.tt // fetch contracts - contracts, err := b.ContractSetContracts(ctx, testAutopilotConfig.Contracts.Set) + contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: testAutopilotConfig.Contracts.Set}) tt.OK(err) if len(contracts) != 3 { t.Fatalf("unexpected number of contracts, %v != 3", len(contracts)) @@ -40,7 +40,7 @@ func TestBlocklist(t *testing.T) { // assert h3 is no longer in the contract set tt.Retry(5, time.Second, func() error { - contracts, err := b.ContractSetContracts(ctx, testAutopilotConfig.Contracts.Set) + contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: testAutopilotConfig.Contracts.Set}) tt.OK(err) if len(contracts) != 2 { return fmt.Errorf("unexpected number of contracts, %v != 2", len(contracts)) @@ -60,7 +60,7 @@ func TestBlocklist(t *testing.T) { // assert h1 is no longer in the contract set tt.Retry(5, time.Second, func() error { - contracts, err := b.ContractSetContracts(ctx, testAutopilotConfig.Contracts.Set) + contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: testAutopilotConfig.Contracts.Set}) tt.OK(err) if len(contracts) != 1 { return fmt.Errorf("unexpected number of contracts, %v != 1", len(contracts)) @@ -77,7 +77,7 @@ func TestBlocklist(t *testing.T) { 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.ContractSetContracts(ctx, testAutopilotConfig.Contracts.Set) + contracts, err := b.Contracts(ctx, api.ContractsOpts{ContractSet: testAutopilotConfig.Contracts.Set}) tt.OK(err) if len(contracts) != 3 { return fmt.Errorf("unexpected number of contracts, %v != 3", len(contracts)) diff --git a/internal/testing/cluster.go b/internal/testing/cluster.go index 3632ef799..aff84a77e 100644 --- a/internal/testing/cluster.go +++ b/internal/testing/cluster.go @@ -865,7 +865,7 @@ func (c *TestCluster) WaitForContractSet(set string, n int) { ctx, cancel := context.WithTimeout(context.Background(), time.Second) defer cancel() - contracts, err := c.Bus.ContractSetContracts(ctx, set) + contracts, err := c.Bus.Contracts(ctx, api.ContractsOpts{ContractSet: set}) if err != nil { return err } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index ba231f050..aac31ae15 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -725,7 +725,7 @@ func TestUploadDownloadExtended(t *testing.T) { tt.OK(b.SetContractSet(context.Background(), t.Name(), nil)) // assert there are no contracts in the set - csc, err := b.ContractSetContracts(context.Background(), t.Name()) + csc, err := b.Contracts(context.Background(), api.ContractsOpts{ContractSet: t.Name()}) tt.OK(err) if len(csc) != 0 { t.Fatalf("expected no contracts, got %v", len(csc)) @@ -869,7 +869,7 @@ func TestUploadDownloadSpending(t *testing.T) { } // fetch contract set contracts - contracts, err := cluster.Bus.ContractSetContracts(context.Background(), testAutopilotConfig.Contracts.Set) + contracts, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{ContractSet: testAutopilotConfig.Contracts.Set}) tt.OK(err) currentSet := make(map[types.FileContractID]struct{}) for _, c := range contracts { diff --git a/internal/testing/gouging_test.go b/internal/testing/gouging_test.go index fe1f0dedb..7a812354f 100644 --- a/internal/testing/gouging_test.go +++ b/internal/testing/gouging_test.go @@ -53,7 +53,7 @@ func TestGouging(t *testing.T) { } // fetch current contract set - contracts, err := b.ContractSetContracts(context.Background(), cfg.Set) + contracts, err := b.Contracts(context.Background(), api.ContractsOpts{ContractSet: cfg.Set}) tt.OK(err) // update the host settings so it's gouging diff --git a/stores/metadata.go b/stores/metadata.go index daa708309..b475f7095 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -981,12 +981,6 @@ WHERE c.fcid = ? return } -func (s *SQLStore) ContractSetContracts(ctx context.Context, set string) ([]api.ContractMetadata, error) { - return s.Contracts(ctx, api.ContractsOpts{ - ContractSet: set, - }) -} - func (s *SQLStore) ContractSets(ctx context.Context) ([]string, error) { var sets []string err := s.db.Raw("SELECT name FROM contract_sets"). diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 0e610309e..e10f6b311 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -268,12 +268,12 @@ func TestSQLContractStore(t *testing.T) { if err := ss.SetContractSet(ctx, "foo", []types.FileContractID{contracts[0].ID}); err != nil { t.Fatal(err) } - if contracts, err := ss.ContractSetContracts(ctx, "foo"); err != nil { + if contracts, err := ss.Contracts(ctx, api.ContractsOpts{ContractSet: "foo"}); err != nil { t.Fatal(err) } else if len(contracts) != 1 { t.Fatalf("should have 1 contracts but got %v", len(contracts)) } - if _, err := ss.ContractSetContracts(ctx, "bar"); !errors.Is(err, api.ErrContractSetNotFound) { + if _, err := ss.Contracts(ctx, api.ContractsOpts{ContractSet: "bar"}); !errors.Is(err, api.ErrContractSetNotFound) { t.Fatal(err) } @@ -3749,7 +3749,7 @@ func TestSlabHealthInvalidation(t *testing.T) { } // assert there are 0 contracts in the contract set - cscs, err := ss.ContractSetContracts(context.Background(), testContractSet) + cscs, err := ss.Contracts(context.Background(), api.ContractsOpts{ContractSet: testContractSet}) if err != nil { t.Fatal(err) } else if len(cscs) != 0 { @@ -3777,7 +3777,7 @@ func TestSlabHealthInvalidation(t *testing.T) { assertHealthValid(s2, false) // assert there are 2 contracts in the contract set - cscs, err = ss.ContractSetContracts(context.Background(), testContractSet) + cscs, err = ss.Contracts(context.Background(), api.ContractsOpts{ContractSet: testContractSet}) if err != nil { t.Fatal(err) } else if len(cscs) != 2 { From 95cb9f27abd62bc1b2ec92ea20aae163a4e6354f Mon Sep 17 00:00:00 2001 From: Alex Freska Date: Wed, 20 Dec 2023 13:02:07 -0500 Subject: [PATCH 24/24] ui: v0.39.0 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 8a79f0e2a..6c64d18b2 100644 --- a/go.mod +++ b/go.mod @@ -17,7 +17,7 @@ require ( go.sia.tech/jape v0.11.1 go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca - go.sia.tech/web/renterd v0.37.0 + go.sia.tech/web/renterd v0.39.0 go.uber.org/zap v1.26.0 golang.org/x/crypto v0.16.0 golang.org/x/term v0.15.0 diff --git a/go.sum b/go.sum index f3fc7e4ba..c0060a9ac 100644 --- a/go.sum +++ b/go.sum @@ -245,8 +245,8 @@ go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca h1:aZMg2AKevn7jKx+wlusWQf go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca/go.mod h1:h/1afFwpxzff6/gG5i1XdAgPK7dEY6FaibhK7N5F86Y= go.sia.tech/web v0.0.0-20231213145933-3f175a86abff h1:/nE7nhewDRxzEdtSKT4SkiUwtjPSiy7Xz7CHEW3MaGQ= go.sia.tech/web v0.0.0-20231213145933-3f175a86abff/go.mod h1:RKODSdOmR3VtObPAcGwQqm4qnqntDVFylbvOBbWYYBU= -go.sia.tech/web/renterd v0.37.0 h1:Igfi0TcrYFufltmbts5TltpMf6hS6veJWyYiWRPPvt4= -go.sia.tech/web/renterd v0.37.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= +go.sia.tech/web/renterd v0.39.0 h1:basqUd6PFS3Zd07aNo4MnTnoy/qPpeyLhXWwcFeOSPM= +go.sia.tech/web/renterd v0.39.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= go.uber.org/atomic v1.4.0/go.mod h1:gD2HeocX3+yG+ygLZcrzQJaqmWj9AIm7n08wl/qW/PE= go.uber.org/goleak v1.2.0 h1:xqgm/S+aQvhWFTtR0XK3Jvg7z8kGV8P4X14IzwN3Eqk= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0=