From 3cabde9ec69fc00b051780ffd5f977d576bda1ed Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Fri, 12 Jan 2024 13:34:18 +0100 Subject: [PATCH 001/144] worker: close pipeReader at the end of serveContent (#891) --- worker/serve.go | 1 + 1 file changed, 1 insertion(+) diff --git a/worker/serve.go b/worker/serve.go index a7fec1e21..2ea1c5d4e 100644 --- a/worker/serve.go +++ b/worker/serve.go @@ -67,6 +67,7 @@ func serveContent(rw http.ResponseWriter, req *http.Request, obj api.Object, dow // launch the download in a goroutine pr, pw := io.Pipe() + defer pr.Close() go func() { if err := downloadFn(pw, offset, length); err != nil { pw.CloseWithError(err) From 4e55e0f4fbb1638a86bddf73670f64cea094c42e Mon Sep 17 00:00:00 2001 From: Peter-Jan Brone Date: Fri, 12 Jan 2024 13:29:25 +0100 Subject: [PATCH 002/144] Avoid deadlock in uploads in case context gets cancelled (#892) * worker: check context * worker: add custom error --- worker/upload.go | 3 +++ worker/upload_test.go | 8 ++++++++ 2 files changed, 11 insertions(+) diff --git a/worker/upload.go b/worker/upload.go index f3fc3754c..bab0a34b8 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -31,6 +31,7 @@ var ( errNoCandidateUploader = errors.New("no candidate uploader found") errNotEnoughContracts = errors.New("not enough contracts to support requested redundancy") errWorkerShutDown = errors.New("worker was shut down") + errUploadInterrupted = errors.New("upload was interrupted") ) type ( @@ -569,6 +570,8 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a select { case <-mgr.shutdownCtx.Done(): return false, "", errWorkerShutDown + case <-ctx.Done(): + return false, "", errUploadInterrupted case numSlabs = <-numSlabsChan: case res := <-respChan: if res.err != nil { diff --git a/worker/upload_test.go b/worker/upload_test.go index fd4a82907..3534d2edf 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -125,6 +125,14 @@ func TestUploadDownload(t *testing.T) { if !errors.Is(err, errBucketNotFound) { t.Fatal("expected bucket not found error", err) } + + // upload data using a cancelled context - assert we don't hang + ctx, cancel := context.WithCancel(context.Background()) + cancel() + _, _, err = ul.Upload(ctx, bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + if err == nil || !errors.Is(err, errUploadInterrupted) { + t.Fatal(err) + } } func testParameters(bucket, path string) uploadParameters { From 13d25bc89be647511ebafffb88a3085d941003a2 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 16 Jan 2024 13:12:22 +0100 Subject: [PATCH 003/144] worker: try renew contract when max revision is reached --- internal/testing/cluster_test.go | 5 +- worker/mocks_test.go | 108 +++++++++++++++++++++++++------ worker/upload.go | 56 +++++++++------- worker/upload_test.go | 4 +- worker/uploader.go | 34 +++++++--- worker/worker.go | 9 ++- 6 files changed, 163 insertions(+), 53 deletions(-) diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index e811738f6..ef4b4efd7 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -891,7 +891,10 @@ func TestUploadDownloadSpending(t *testing.T) { cluster.MineToRenewWindow() // wait for the contract to be renewed - tt.Retry(100, 100*time.Millisecond, func() error { + tt.Retry(10, time.Second, func() error { + // mine a block + cluster.MineBlocks(1) + // fetch contracts cms, err := cluster.Bus.Contracts(context.Background(), api.ContractsOpts{}) tt.OK(err) diff --git a/worker/mocks_test.go b/worker/mocks_test.go index b2597240c..7335a4962 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -8,10 +8,12 @@ import ( "time" 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/object" + "go.uber.org/zap" "lukechampine.com/frand" ) @@ -23,7 +25,7 @@ type ( sectors map[types.Hash256]*[rhpv2.SectorSize]byte } - mockContractLocker struct { + mockContractStore struct { contracts map[types.FileContractID]*mockContract } @@ -42,18 +44,39 @@ type ( mockMemoryManager struct{} mockObjectStore struct { - mu sync.Mutex - objects map[string]map[string]object.Object + mu sync.Mutex + objects map[string]map[string]object.Object + partials map[string]mockPackedSlab + bufferIDCntr uint // allows marking packed slabs as uploaded + } + + mockPackedSlab struct { + parameterKey string // ([minshards]-[totalshards]-[contractset]) + bufferID uint + slabKey object.EncryptionKey + data []byte + } + + mockWorker struct { + cs *mockContractStore + hm *mockHostManager + mm *mockMemoryManager + os *mockObjectStore + + dl *downloadManager + ul *uploadManager + + contracts map[types.PublicKey]api.ContractMetadata } ) var ( - _ ContractLocker = (*mockContractLocker)(nil) - _ Host = (*mockHost)(nil) - _ HostManager = (*mockHostManager)(nil) - _ Memory = (*mockMemory)(nil) - _ MemoryManager = (*mockMemoryManager)(nil) - _ ObjectStore = (*mockObjectStore)(nil) + _ ContractStore = (*mockContractStore)(nil) + _ Host = (*mockHost)(nil) + _ HostManager = (*mockHostManager)(nil) + _ Memory = (*mockMemory)(nil) + _ MemoryManager = (*mockMemoryManager)(nil) + _ ObjectStore = (*mockObjectStore)(nil) ) var ( @@ -74,6 +97,10 @@ func (mm *mockMemoryManager) AcquireMemory(ctx context.Context, amt uint64) Memo return &mockMemory{} } +func (os *mockContractStore) RenewedContract(ctx context.Context, renewedFrom types.FileContractID) (api.ContractMetadata, error) { + return api.ContractMetadata{}, api.ErrContractNotFound +} + func (os *mockObjectStore) AddMultipartPart(ctx context.Context, bucket, path, contractSet, ETag, uploadID string, partNumber int, slices []object.SlabSlice) (err error) { return nil } @@ -189,8 +216,8 @@ func (hp *mockHostManager) Host(hk types.PublicKey, fcid types.FileContractID, s 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 { +func (cs *mockContractStore) AcquireContract(ctx context.Context, fcid types.FileContractID, priority int, d time.Duration) (lockID uint64, err error) { + if lock, ok := cs.contracts[fcid]; !ok { return 0, errContractNotFound } else { lock.mu.Lock() @@ -199,8 +226,8 @@ func (cl *mockContractLocker) AcquireContract(ctx context.Context, fcid types.Fi return 0, nil } -func (cl *mockContractLocker) ReleaseContract(ctx context.Context, fcid types.FileContractID, lockID uint64) (err error) { - if lock, ok := cl.contracts[fcid]; !ok { +func (cs *mockContractStore) ReleaseContract(ctx context.Context, fcid types.FileContractID, lockID uint64) (err error) { + if lock, ok := cs.contracts[fcid]; !ok { return errContractNotFound } else { lock.mu.Unlock() @@ -208,7 +235,7 @@ func (cl *mockContractLocker) ReleaseContract(ctx context.Context, fcid types.Fi return nil } -func (cl *mockContractLocker) KeepaliveContract(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration) (err error) { +func (cs *mockContractStore) KeepaliveContract(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration) (err error) { return nil } @@ -243,12 +270,12 @@ func newMockContract(fcid types.FileContractID) *mockContract { } } -func newMockContractLocker(contracts []*mockContract) *mockContractLocker { - cl := &mockContractLocker{contracts: make(map[types.FileContractID]*mockContract)} +func newMockContractStore(contracts []*mockContract) *mockContractStore { + cs := &mockContractStore{contracts: make(map[types.FileContractID]*mockContract)} for _, c := range contracts { - cl.contracts[c.rev.ParentID] = c + cs.contracts[c.rev.ParentID] = c } - return cl + return cs } func newMockHostManager(hosts []*mockHost) *mockHostManager { @@ -270,3 +297,48 @@ func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { frand.Read(sector[:]) return §or, rhpv2.SectorRoot(§or) } + +func newMockWorker(numHosts int) *mockWorker { + // create hosts and contracts + hosts := newMockHosts(numHosts) + contracts := newMockContracts(hosts) + + // create dependencies + cs := newMockContractStore(contracts) + hm := newMockHostManager(hosts) + os := newMockObjectStore() + mm := &mockMemoryManager{} + + dl := newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()) + ul := newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()) + + // create contract metadata + metadatas := make(map[types.PublicKey]api.ContractMetadata) + for _, h := range hosts { + metadatas[h.hk] = api.ContractMetadata{ + ID: h.c.rev.ParentID, + HostKey: h.hk, + } + } + + return &mockWorker{ + hm: hm, + mm: mm, + os: os, + + dl: dl, + ul: ul, + + contracts: metadatas, + } +} + +func newTestHostPriceTable(expiry time.Time) hostdb.HostPriceTable { + var uid rhpv3.SettingsID + frand.Read(uid[:]) + + return hostdb.HostPriceTable{ + HostPriceTable: rhpv3.HostPriceTable{UID: uid, Validity: time.Minute}, + Expiry: expiry, + } +} diff --git a/worker/upload.go b/worker/upload.go index bab0a34b8..8ce41b342 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -39,7 +39,7 @@ type ( hm HostManager mm MemoryManager os ObjectStore - cl ContractLocker + cs ContractStore logger *zap.SugaredLogger contractLockDuration time.Duration @@ -110,13 +110,15 @@ type ( } sectorUpload struct { - index int - data *[rhpv2.SectorSize]byte - root types.Hash256 - uploaded object.Sector + index int + root types.Hash256 ctx context.Context cancel context.CancelFunc + + mu sync.Mutex + uploaded object.Sector + data *[rhpv2.SectorSize]byte } sectorUploadReq struct { @@ -313,12 +315,12 @@ func (w *worker) uploadPackedSlab(ctx context.Context, rs api.RedundancySettings return nil } -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 { +func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os ObjectStore, cs ContractStore, maxOverdrive uint64, overdriveTimeout time.Duration, contractLockDuration time.Duration, logger *zap.SugaredLogger) *uploadManager { return &uploadManager{ hm: hm, mm: mm, os: os, - cl: cl, + cs: cs, logger: logger, contractLockDuration: contractLockDuration, @@ -335,10 +337,11 @@ func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os } } -func (mgr *uploadManager) newUploader(os ObjectStore, cl ContractLocker, h Host, c api.ContractMetadata, bh uint64) *uploader { +func (mgr *uploadManager) newUploader(os ObjectStore, cs ContractStore, hm HostManager, c api.ContractMetadata, bh uint64) *uploader { return &uploader{ os: os, - cl: cl, + cs: cs, + hm: hm, logger: mgr.logger, // static @@ -352,7 +355,7 @@ func (mgr *uploadManager) newUploader(os ObjectStore, cl ContractLocker, h Host, statsSectorUploadSpeedBytesPerMS: stats.NoDecay(), // covered by mutex - host: h, + host: hm.Host(c.HostKey, c.ID, c.SiamuxAddr), bh: bh, fcid: c.ID, endHeight: c.WindowEnd, @@ -559,6 +562,7 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a mem.Release() }(up.rs, data, length, slabIndex) } + slabIndex++ } }() @@ -727,10 +731,9 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh var refreshed []*uploader existing := make(map[types.FileContractID]struct{}) for _, uploader := range mgr.uploaders { - // renew uploaders that got renewed + // refresh uploaders that got renewed if renewal, renewed := renewals[uploader.ContractID()]; renewed { - host := mgr.hm.Host(renewal.HostKey, renewal.ID, renewal.SiamuxAddr) - uploader.Renew(host, renewal, bh) + uploader.Refresh(renewal, bh) } // stop uploaders that expired @@ -751,8 +754,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh // add missing uploaders 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, mgr.cl, host, c, bh) + uploader := mgr.newUploader(mgr.os, mgr.cs, mgr.hm, c, bh) refreshed = append(refreshed, uploader) go uploader.Start() } @@ -1000,7 +1002,6 @@ func (s *slabUpload) launch(req *sectorUploadReq) error { s.lastOverdrive = time.Now() s.numOverdriving++ } - // update the state s.numInflight++ s.numLaunched++ @@ -1071,18 +1072,15 @@ func (s *slabUpload) receive(resp sectorUploadResp) (bool, bool) { } // store the sector - sector.uploaded = object.Sector{ + sector.finish(object.Sector{ Contracts: map[types.PublicKey][]types.FileContractID{req.hk: {req.fcid}}, LatestHost: req.hk, Root: resp.root, - } + }) // update uploaded sectors s.numUploaded++ - // cancel the sector's context - sector.cancel() - // release all other candidates for this sector for _, candidate := range s.candidates { if candidate.req != nil && candidate.req != req && candidate.req.sector.index == sector.index { @@ -1091,16 +1089,30 @@ func (s *slabUpload) receive(resp sectorUploadResp) (bool, bool) { } // release memory - sector.data = nil s.mem.ReleaseSome(rhpv2.SectorSize) return true, s.numUploaded == s.numSectors } +func (s *sectorUpload) finish(sector object.Sector) { + s.mu.Lock() + defer s.mu.Unlock() + + s.cancel() + s.uploaded = sector + s.data = nil +} + func (s *sectorUpload) isUploaded() bool { return s.uploaded.Root != (types.Hash256{}) } +func (s *sectorUpload) sectorData() *[rhpv2.SectorSize]byte { + s.mu.Lock() + defer s.mu.Unlock() + return s.data +} + func (req *sectorUploadReq) done() bool { select { case <-req.sector.ctx.Done(): diff --git a/worker/upload_test.go b/worker/upload_test.go index 3534d2edf..aff063e00 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -27,14 +27,14 @@ func TestUploadDownload(t *testing.T) { contracts := newMockContracts(hosts) // mock dependencies - cl := newMockContractLocker(contracts) + cs := newMockContractStore(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()) + ul := newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()) // create test data data := make([]byte, 128) diff --git a/worker/uploader.go b/worker/uploader.go index e85bc260d..bc8f06a5e 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -22,7 +22,8 @@ const ( type ( uploader struct { os ObjectStore - cl ContractLocker + cs ContractStore + hm HostManager logger *zap.SugaredLogger hk types.PublicKey @@ -70,12 +71,12 @@ func (u *uploader) Healthy() bool { return u.consecutiveFailures == 0 } -func (u *uploader) Renew(h Host, c api.ContractMetadata, bh uint64) { +func (u *uploader) Refresh(c api.ContractMetadata, bh uint64) { u.mu.Lock() defer u.mu.Unlock() u.bh = bh - u.host = h + u.host = u.hm.Host(c.HostKey, c.ID, c.SiamuxAddr) u.fcid = c.ID u.siamuxAddr = c.SiamuxAddr u.endHeight = c.WindowEnd @@ -120,8 +121,10 @@ outer: // the uploader's contract got renewed, requeue the request if errors.Is(err, errMaxRevisionReached) { - u.enqueue(req) - continue outer + if u.tryRefresh(req.sector.ctx) { + u.enqueue(req) + continue outer + } } // send the response @@ -196,13 +199,13 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, u.mu.Unlock() // acquire contract lock - lockID, err := u.cl.AcquireContract(req.sector.ctx, fcid, req.contractLockPriority, req.contractLockDuration) + lockID, err := u.cs.AcquireContract(req.sector.ctx, fcid, req.contractLockPriority, req.contractLockDuration) if err != nil { return types.Hash256{}, 0, err } // defer the release - lock := newContractLock(u.shutdownCtx, fcid, lockID, req.contractLockDuration, u.cl, u.logger) + lock := newContractLock(u.shutdownCtx, fcid, lockID, req.contractLockDuration, u.cs, u.logger) defer func() { ctx, cancel := context.WithTimeout(u.shutdownCtx, 10*time.Second) lock.Release(ctx) @@ -228,7 +231,7 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, // upload the sector start := time.Now() - root, err := host.UploadSector(ctx, req.sector.data, rev) + root, err := host.UploadSector(ctx, req.sector.sectorData(), rev) if err != nil { return types.Hash256{}, 0, fmt.Errorf("failed to upload sector to contract %v, err: %v", fcid, err) } @@ -287,3 +290,18 @@ func (u *uploader) tryRecomputeStats() { u.statsSectorUploadEstimateInMS.Recompute() u.statsSectorUploadSpeedBytesPerMS.Recompute() } + +func (u *uploader) tryRefresh(ctx context.Context) bool { + // fetch the renewed contract + renewed, err := u.cs.RenewedContract(ctx, u.ContractID()) + if isError(err, api.ErrContractNotFound) || isError(err, context.Canceled) { + return false + } else if err != nil { + u.logger.Errorf("failed to fetch renewed contract %v, err: %v", u.ContractID(), err) + return false + } + + // renew the uploader with the renewed contract + u.Refresh(renewed, u.BlockHeight()) + return true +} diff --git a/worker/worker.go b/worker/worker.go index 8ba6f03b0..04c947ed9 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -69,7 +69,7 @@ type ( webhooks.Broadcaster AccountStore - ContractLocker + ContractStore ObjectStore BroadcastTransaction(ctx context.Context, txns []types.Transaction) error @@ -79,7 +79,6 @@ type ( 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, opts api.ContractsOpts) ([]api.ContractMetadata, error) - RenewedContract(ctx context.Context, renewedFrom types.FileContractID) (api.ContractMetadata, error) RecordHostScans(ctx context.Context, scans []hostdb.HostScan) error RecordPriceTables(ctx context.Context, priceTableUpdate []hostdb.PriceTableUpdate) error @@ -116,6 +115,12 @@ type ( ScheduleSync(ctx context.Context, id rhpv3.Account, hk types.PublicKey) error } + ContractStore interface { + ContractLocker + + RenewedContract(ctx context.Context, renewedFrom types.FileContractID) (api.ContractMetadata, error) + } + ObjectStore interface { // NOTE: used for download DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) error From 738d1c8d47f0016f5281d797b9ba50d0de20c631 Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 22 Jan 2024 10:19:32 +0100 Subject: [PATCH 004/144] worker: add TestRefreshUploaders --- worker/host_test.go | 4 +- worker/migrations.go | 2 +- worker/mocks_test.go | 118 +++++++++++++++---------- worker/pricetables_test.go | 11 +-- worker/upload.go | 131 ++++++++++++++-------------- worker/upload_test.go | 174 +++++++++++++++++++++++++++---------- worker/uploader.go | 4 +- 7 files changed, 279 insertions(+), 165 deletions(-) diff --git a/worker/host_test.go b/worker/host_test.go index 89c8fc2dc..e9b3ab798 100644 --- a/worker/host_test.go +++ b/worker/host_test.go @@ -5,15 +5,13 @@ import ( "context" "errors" "testing" - "time" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" ) func TestHost(t *testing.T) { - c := newMockContract(types.FileContractID{1}) - h := newMockHost(types.PublicKey{1}, newTestHostPriceTable(time.Now()), c) + h := newMockHost(true) sector, root := newMockSector() // upload the sector diff --git a/worker/migrations.go b/worker/migrations.go index 05bd20193..6c25b789f 100644 --- a/worker/migrations.go +++ b/worker/migrations.go @@ -108,7 +108,7 @@ SHARDS: } // migrate the shards - err = w.uploadManager.MigrateShards(ctx, s, shardIndices, shards, contractSet, allowed, bh, lockingPriorityUpload, mem) + err = w.uploadManager.UploadShards(ctx, s, shardIndices, shards, contractSet, allowed, bh, lockingPriorityUpload, mem) if err != nil { return 0, surchargeApplied, fmt.Errorf("failed to upload slab for migration: %w", err) } diff --git a/worker/mocks_test.go b/worker/mocks_test.go index 0649d8c75..e861edcdc 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -20,10 +20,9 @@ import ( ) type ( - contractsMap map[types.PublicKey]api.ContractMetadata - mockContract struct { - rev types.FileContractRevision + rev types.FileContractRevision + metadata api.ContractMetadata mu sync.Mutex sectors map[types.Hash256]*[rhpv2.SectorSize]byte @@ -43,8 +42,11 @@ type ( hptBlockChan chan struct{} } + mockHosts []*mockHost + mockContracts []*mockContract + mockHostManager struct { - hosts map[types.PublicKey]Host + hosts map[types.PublicKey]*mockHost } mockMemory struct{} @@ -72,8 +74,6 @@ type ( dl *downloadManager ul *uploadManager - - contracts contractsMap } ) @@ -94,14 +94,38 @@ var ( errSectorOutOfBounds = errors.New("sector out of bounds") ) -func (c contractsMap) values() []api.ContractMetadata { - var contracts []api.ContractMetadata - for _, contract := range c { - contracts = append(contracts, contract) +var mockContractCntr = 0 + +func newFileContractID() (fcid types.FileContractID) { + fcid = types.FileContractID{byte(mockContractCntr)} + mockContractCntr++ + return +} + +var mockHostCntr = 0 + +func newHostKey() (hk types.PublicKey) { + hk = types.PublicKey{byte(mockHostCntr)} + mockHostCntr++ + return +} + +func (hosts mockHosts) contracts() mockContracts { + contracts := make([]*mockContract, len(hosts)) + for i, host := range hosts { + contracts[i] = host.c } return contracts } +func (contracts mockContracts) metadata() []api.ContractMetadata { + metadata := make([]api.ContractMetadata, len(contracts)) + for i, contract := range contracts { + metadata[i] = contract.metadata + } + return metadata +} + func (m *mockMemory) Release() {} func (m *mockMemory) ReleaseSome(uint64) {} @@ -340,6 +364,15 @@ func (h *mockHost) FundAccount(ctx context.Context, balance types.Currency, rev } func (h *mockHost) RenewContract(ctx context.Context, rrr api.RHPRenewRequest) (_ rhpv2.ContractRevision, _ []types.Transaction, _ types.Currency, err error) { + h.mu.Lock() + defer h.mu.Unlock() + + curr := h.c.metadata + update := newMockContract(h.hk) + update.metadata.RenewedFrom = curr.ID + update.metadata.WindowStart = curr.WindowEnd + update.metadata.WindowEnd = update.metadata.WindowStart + (curr.WindowEnd - curr.WindowStart) + h.c = update return } @@ -377,34 +410,33 @@ func (cs *mockContractStore) KeepaliveContract(ctx context.Context, fcid types.F return nil } -func newMockHosts(n int) []*mockHost { - hosts := make([]*mockHost, n) - for i := range hosts { - hosts[i] = newMockHost(types.PublicKey{byte(i)}, newTestHostPriceTable(time.Now().Add(time.Minute)), nil) +func newMockHost(contract bool) *mockHost { + host := &mockHost{ + hk: newHostKey(), + hpt: newTestHostPriceTable(time.Now().Add(time.Minute)), } - return hosts -} - -func newMockHost(hk types.PublicKey, hpt hostdb.HostPriceTable, c *mockContract) *mockHost { - return &mockHost{ - hk: hk, - c: c, - - hpt: hpt, + if contract { + host.c = newMockContract(host.hk) } + return host } -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] +func newMockHosts(n int, contract bool) (hosts mockHosts) { + for i := 0; i < n; i++ { + hosts = append(hosts, newMockHost(contract)) } - return contracts + return } -func newMockContract(fcid types.FileContractID) *mockContract { +func newMockContract(hk types.PublicKey) *mockContract { + fcid := newFileContractID() return &mockContract{ + metadata: api.ContractMetadata{ + ID: fcid, + HostKey: hk, + WindowStart: 0, + WindowEnd: 10, + }, rev: types.FileContractRevision{ParentID: fcid}, sectors: make(map[types.Hash256]*[rhpv2.SectorSize]byte), } @@ -419,7 +451,7 @@ func newMockContractStore(contracts []*mockContract) *mockContractStore { } func newMockHostManager(hosts []*mockHost) *mockHostManager { - hm := &mockHostManager{hosts: make(map[types.PublicKey]Host)} + hm := &mockHostManager{hosts: make(map[types.PublicKey]*mockHost)} for _, h := range hosts { hm.hosts[h.hk] = h } @@ -439,12 +471,11 @@ func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { } func newMockWorker(numHosts int) *mockWorker { - // create hosts and contracts - hosts := newMockHosts(numHosts) - contracts := newMockContracts(hosts) + // create hosts + hosts := newMockHosts(numHosts, true) // create dependencies - cs := newMockContractStore(contracts) + cs := newMockContractStore(hosts.contracts()) hm := newMockHostManager(hosts) os := newMockObjectStore() mm := &mockMemoryManager{} @@ -452,25 +483,22 @@ func newMockWorker(numHosts int) *mockWorker { dl := newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()) ul := newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()) - // create contract metadata - metadatas := make(contractsMap) - for _, h := range hosts { - metadatas[h.hk] = api.ContractMetadata{ - ID: h.c.rev.ParentID, - HostKey: h.hk, - } - } - return &mockWorker{ + cs: cs, hm: hm, mm: mm, os: os, dl: dl, ul: ul, + } +} - contracts: metadatas, +func (w *mockWorker) contracts() (metadatas []api.ContractMetadata) { + for _, h := range w.hm.hosts { + metadatas = append(metadatas, h.c.metadata) } + return } func newTestHostPriceTable(expiry time.Time) hostdb.HostPriceTable { diff --git a/worker/pricetables_test.go b/worker/pricetables_test.go index 2a9f84cbc..b90ca4fec 100644 --- a/worker/pricetables_test.go +++ b/worker/pricetables_test.go @@ -44,15 +44,16 @@ func newMockHostStore(hosts []*hostdb.HostInfo) *mockHostStore { } func TestPriceTables(t *testing.T) { - // create two price tables + // create two price tables, a valid one and one that expired expiredPT := newTestHostPriceTable(time.Now()) validPT := newTestHostPriceTable(time.Now().Add(time.Minute)) - // create a mock host with that returns a valid price table - hk1 := types.PublicKey{1} - h1 := newMockHost(hk1, validPT, nil) + // create a mock host that has a valid price table + h1 := newMockHost(false) + h1.hpt = validPT + hk1 := h1.hk - // create a hostdb host with an expired pt + // create a hostdb entry for that host that returns the expired price table hdb1 := &hostdb.HostInfo{ Host: hostdb.Host{ PublicKey: hk1, diff --git a/worker/upload.go b/worker/upload.go index aebd341a4..692e40c70 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -28,6 +28,7 @@ const ( ) var ( + errContractExpired = errors.New("contract expired") errNoCandidateUploader = errors.New("no candidate uploader found") errNotEnoughContracts = errors.New("not enough contracts to support requested redundancy") errWorkerShutDown = errors.New("worker was shut down") @@ -363,67 +364,6 @@ func (mgr *uploadManager) newUploader(os ObjectStore, cs ContractStore, hm HostM } } -func (mgr *uploadManager) MigrateShards(ctx context.Context, s *object.Slab, shardIndices []int, shards [][]byte, contractSet string, contracts []api.ContractMetadata, bh uint64, lockPriority int, mem Memory) (err error) { - // cancel all in-flight requests when the upload is done - ctx, cancel := context.WithCancel(ctx) - defer cancel() - - // create the upload - upload, err := mgr.newUpload(ctx, len(shards), contracts, bh, lockPriority) - if err != nil { - return err - } - - // track the upload in the bus - if err := mgr.os.TrackUpload(ctx, upload.id); err != nil { - return fmt.Errorf("failed to track upload '%v', err: %w", upload.id, err) - } - - // defer a function that finishes the upload - defer func() { - ctx, cancel := context.WithTimeout(mgr.shutdownCtx, time.Minute) - if err := mgr.os.FinishUpload(ctx, upload.id); err != nil { - mgr.logger.Errorf("failed to mark upload %v as finished: %v", upload.id, err) - } - cancel() - }() - - // upload the shards - uploaded, uploadSpeed, overdrivePct, err := upload.uploadShards(ctx, shards, mgr.candidates(upload.allowed), mem, mgr.maxOverdrive, mgr.overdriveTimeout) - if err != nil { - return err - } - - // track stats - mgr.statsOverdrivePct.Track(overdrivePct) - mgr.statsSlabUploadSpeedBytesPerMS.Track(float64(uploadSpeed)) - - // overwrite the shards with the newly uploaded ones - for i, si := range shardIndices { - s.Shards[si].LatestHost = uploaded[i].LatestHost - - knownContracts := make(map[types.FileContractID]struct{}) - for _, fcids := range s.Shards[si].Contracts { - for _, fcid := range fcids { - knownContracts[fcid] = struct{}{} - } - } - for hk, fcids := range uploaded[i].Contracts { - for _, fcid := range fcids { - if _, exists := knownContracts[fcid]; !exists { - if s.Shards[si].Contracts == nil { - s.Shards[si].Contracts = make(map[types.PublicKey][]types.FileContractID) - } - s.Shards[si].Contracts[hk] = append(s.Shards[si].Contracts[hk], fcid) - } - } - } - } - - // update the slab - return mgr.os.UpdateSlab(ctx, *s, contractSet) -} - func (mgr *uploadManager) Stats() uploadManagerStats { mgr.mu.Lock() defer mgr.mu.Unlock() @@ -451,7 +391,7 @@ func (mgr *uploadManager) Stop() { mgr.mu.Lock() defer mgr.mu.Unlock() for _, u := range mgr.uploaders { - u.Stop() + u.Stop(errWorkerShutDown) } } @@ -673,6 +613,67 @@ func (mgr *uploadManager) UploadPackedSlab(ctx context.Context, rs api.Redundanc return nil } +func (mgr *uploadManager) UploadShards(ctx context.Context, s *object.Slab, shardIndices []int, shards [][]byte, contractSet string, contracts []api.ContractMetadata, bh uint64, lockPriority int, mem Memory) (err error) { + // cancel all in-flight requests when the upload is done + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + // create the upload + upload, err := mgr.newUpload(ctx, len(shards), contracts, bh, lockPriority) + if err != nil { + return err + } + + // track the upload in the bus + if err := mgr.os.TrackUpload(ctx, upload.id); err != nil { + return fmt.Errorf("failed to track upload '%v', err: %w", upload.id, err) + } + + // defer a function that finishes the upload + defer func() { + ctx, cancel := context.WithTimeout(mgr.shutdownCtx, time.Minute) + if err := mgr.os.FinishUpload(ctx, upload.id); err != nil { + mgr.logger.Errorf("failed to mark upload %v as finished: %v", upload.id, err) + } + cancel() + }() + + // upload the shards + uploaded, uploadSpeed, overdrivePct, err := upload.uploadShards(ctx, shards, mgr.candidates(upload.allowed), mem, mgr.maxOverdrive, mgr.overdriveTimeout) + if err != nil { + return err + } + + // track stats + mgr.statsOverdrivePct.Track(overdrivePct) + mgr.statsSlabUploadSpeedBytesPerMS.Track(float64(uploadSpeed)) + + // overwrite the shards with the newly uploaded ones + for i, si := range shardIndices { + s.Shards[si].LatestHost = uploaded[i].LatestHost + + knownContracts := make(map[types.FileContractID]struct{}) + for _, fcids := range s.Shards[si].Contracts { + for _, fcid := range fcids { + knownContracts[fcid] = struct{}{} + } + } + for hk, fcids := range uploaded[i].Contracts { + for _, fcid := range fcids { + if _, exists := knownContracts[fcid]; !exists { + if s.Shards[si].Contracts == nil { + s.Shards[si].Contracts = make(map[types.PublicKey][]types.FileContractID) + } + s.Shards[si].Contracts[hk] = append(s.Shards[si].Contracts[hk], fcid) + } + } + } + } + + // update the slab + return mgr.os.UpdateSlab(ctx, *s, contractSet) +} + func (mgr *uploadManager) candidates(allowed map[types.PublicKey]struct{}) (candidates []*uploader) { mgr.mu.Lock() defer mgr.mu.Unlock() @@ -722,7 +723,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh // build map of renewals renewals := make(map[types.FileContractID]api.ContractMetadata) for _, c := range contracts { - if c.RenewedFrom == (types.FileContractID{}) { + if c.RenewedFrom != (types.FileContractID{}) { renewals[c.RenewedFrom] = c } } @@ -738,7 +739,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh // stop uploaders that expired if uploader.Expired(bh) { - uploader.Stop() + uploader.Stop(errContractExpired) continue } @@ -753,7 +754,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh // add missing uploaders for _, c := range contracts { - if _, exists := existing[c.ID]; !exists { + if _, exists := existing[c.ID]; !exists && bh < c.WindowEnd { uploader := mgr.newUploader(mgr.os, mgr.cs, mgr.hm, c, bh) refreshed = append(refreshed, uploader) go uploader.Start() diff --git a/worker/upload_test.go b/worker/upload_test.go index b87d242e9..5b6df4de4 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -11,7 +11,6 @@ import ( "go.sia.tech/core/types" "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" - "go.uber.org/zap" "lukechampine.com/frand" ) @@ -25,22 +24,13 @@ var ( ) func TestUpload(t *testing.T) { - // create upload params - params := testParameters(t.Name()) - - // create test hosts and contracts - hosts := newMockHosts(params.rs.TotalShards * 2) - contracts := newMockContracts(hosts) - - // mock dependencies - cs := newMockContractStore(contracts) - hm := newMockHostManager(hosts) - os := newMockObjectStore() - mm := &mockMemoryManager{} + // mock worker + w := newMockWorker(testRedundancySettings.TotalShards * 2) - // create managers - dl := newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()) - ul := newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()) + // convenience variables + os := w.os + dl := w.dl + ul := w.ul // create test data data := make([]byte, 128) @@ -48,17 +38,11 @@ func TestUpload(t *testing.T) { 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, - } - } + // create upload params + params := testParameters(t.Name()) // upload data - _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), w.contracts(), params, lockingPriorityUpload) if err != nil { t.Fatal(err) } @@ -77,7 +61,7 @@ func TestUpload(t *testing.T) { // 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) + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -87,7 +71,7 @@ func TestUpload(t *testing.T) { // filter contracts to have (at most) min shards used contracts var n int var filtered []api.ContractMetadata - for _, md := range metadatas { + for _, md := range w.contracts() { // add unused contracts if _, used := used[md.HostKey]; !used { filtered = append(filtered, md) @@ -127,7 +111,7 @@ func TestUpload(t *testing.T) { // try and upload into a bucket that does not exist params.bucket = "doesnotexist" - _, _, err = ul.Upload(context.Background(), bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + _, _, err = ul.Upload(context.Background(), bytes.NewReader(data), w.contracts(), params, lockingPriorityUpload) if !errors.Is(err, errBucketNotFound) { t.Fatal("expected bucket not found error", err) } @@ -135,7 +119,7 @@ func TestUpload(t *testing.T) { // upload data using a cancelled context - assert we don't hang ctx, cancel := context.WithCancel(context.Background()) cancel() - _, _, err = ul.Upload(ctx, bytes.NewReader(data), metadatas, params, lockingPriorityUpload) + _, _, err = ul.Upload(ctx, bytes.NewReader(data), w.contracts(), params, lockingPriorityUpload) if err == nil || !errors.Is(err, errUploadInterrupted) { t.Fatal(err) } @@ -162,7 +146,7 @@ func TestUploadPackedSlab(t *testing.T) { params.packing = true // upload data - _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), w.contracts.values(), params, lockingPriorityUpload) + _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), w.contracts(), params, lockingPriorityUpload) if err != nil { t.Fatal(err) } @@ -180,7 +164,7 @@ func TestUploadPackedSlab(t *testing.T) { // 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), w.contracts.values()) + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -198,7 +182,7 @@ func TestUploadPackedSlab(t *testing.T) { mem := mm.AcquireMemory(context.Background(), uint64(params.rs.TotalShards*rhpv2.SectorSize)) // upload the packed slab - err = ul.UploadPackedSlab(context.Background(), params.rs, ps, mem, w.contracts.values(), 0, lockingPriorityUpload) + err = ul.UploadPackedSlab(context.Background(), params.rs, ps, mem, w.contracts(), 0, lockingPriorityUpload) if err != nil { t.Fatal(err) } @@ -216,7 +200,7 @@ func TestUploadPackedSlab(t *testing.T) { // download the data again and assert it matches buf.Reset() - err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts.values()) + err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -224,7 +208,7 @@ func TestUploadPackedSlab(t *testing.T) { } } -func TestMigrateShards(t *testing.T) { +func TestUploadShards(t *testing.T) { // mock worker w := newMockWorker(testRedundancySettings.TotalShards * 2) @@ -244,7 +228,7 @@ func TestMigrateShards(t *testing.T) { params := testParameters(t.Name()) // upload data - _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), w.contracts.values(), params, lockingPriorityUpload) + _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), w.contracts(), params, lockingPriorityUpload) if err != nil { t.Fatal(err) } @@ -275,7 +259,7 @@ func TestMigrateShards(t *testing.T) { } // download the slab - shards, _, err := dl.DownloadSlab(context.Background(), slab.Slab, w.contracts.values()) + shards, _, err := dl.DownloadSlab(context.Background(), slab.Slab, w.contracts()) if err != nil { t.Fatal(err) } @@ -291,17 +275,17 @@ func TestMigrateShards(t *testing.T) { // recreate upload contracts contracts := make([]api.ContractMetadata, 0) - for hk := range w.hm.hosts { - _, used := usedHosts[hk] - _, bad := badHosts[hk] + for _, c := range w.contracts() { + _, used := usedHosts[c.HostKey] + _, bad := badHosts[c.HostKey] if !used && !bad { - contracts = append(contracts, w.contracts[hk]) + contracts = append(contracts, c) } } // migrate those shards away from bad hosts mem := mm.AcquireMemory(context.Background(), uint64(len(badIndices))*rhpv2.SectorSize) - err = ul.MigrateShards(context.Background(), &o.Object.Object.Slabs[0].Slab, badIndices, shards, testContractSet, contracts, 0, lockingPriorityUpload, mem) + err = ul.UploadShards(context.Background(), &o.Object.Object.Slabs[0].Slab, badIndices, shards, testContractSet, contracts, 0, lockingPriorityUpload, mem) if err != nil { t.Fatal(err) } @@ -324,9 +308,9 @@ func TestMigrateShards(t *testing.T) { // create download contracts contracts = contracts[:0] - for hk := range w.hm.hosts { - if _, bad := badHosts[hk]; !bad { - contracts = append(contracts, w.contracts[hk]) + for _, c := range w.contracts() { + if _, bad := badHosts[c.HostKey]; !bad { + contracts = append(contracts, c) } } @@ -340,6 +324,108 @@ func TestMigrateShards(t *testing.T) { } } +func TestRefreshUploaders(t *testing.T) { + // mock worker + w := newMockWorker(testRedundancySettings.TotalShards * 2) + + // convenience variables + ul := w.ul + hm := w.hm + cs := w.cs + + // create test data + data := make([]byte, 128) + if _, err := frand.Read(data); err != nil { + t.Fatal(err) + } + + // create upload params + params := testParameters(t.Name()) + + // upload data + contracts := w.contracts() + _, _, err := ul.Upload(context.Background(), bytes.NewReader(data), contracts, params, lockingPriorityUpload) + if err != nil { + t.Fatal(err) + } + + // assert we have the expected number of uploaders + if len(ul.uploaders) != len(contracts) { + t.Fatalf("unexpected number of uploaders, %v != %v", len(ul.uploaders), len(contracts)) + } + + // renew the first contract + c1 := contracts[0] + hm.hosts[c1.HostKey].RenewContract(context.Background(), api.RHPRenewRequest{}) + c1Renewed := hm.hosts[c1.HostKey].c + + // remove the host from the second contract + c2 := contracts[1] + delete(hm.hosts, c2.HostKey) + delete(cs.contracts, c2.ID) + + // add a new host/contract + hNew := newMockHost(true) + hm.hosts[hNew.hk] = hNew + cs.contracts[hNew.c.metadata.ID] = hNew.c + + // upload data + contracts = w.contracts() + _, _, err = ul.Upload(context.Background(), bytes.NewReader(data), contracts, params, lockingPriorityUpload) + if err != nil { + t.Fatal(err) + } + + // assert we added and renewed exactly one uploader + var added, renewed int + for _, ul := range ul.uploaders { + switch ul.ContractID() { + case hNew.c.metadata.ID: + added++ + case c1Renewed.metadata.ID: + renewed++ + default: + } + } + if added != 1 { + t.Fatalf("expected 1 added uploader, got %v", added) + } else if renewed != 1 { + t.Fatalf("expected 1 renewed uploader, got %v", renewed) + } + + // assert we have one more uploader than we used to + if len(ul.uploaders) != len(contracts)+1 { + t.Fatalf("unexpected number of uploaders, %v != %v", len(ul.uploaders), len(contracts)+1) + } + + // manually add a request to the queue of one of the uploaders we're about to expire + responseChan := make(chan sectorUploadResp, 1) + for _, ul := range ul.uploaders { + if ul.fcid == hNew.c.metadata.ID { + ul.mu.Lock() + ul.queue = append(ul.queue, §orUploadReq{responseChan: responseChan, sector: §orUpload{ctx: context.Background()}}) + ul.mu.Unlock() + break + } + } + + // upload data again but now with a blockheight that should expire most uploaders + params.bh = c1.WindowEnd + ul.Upload(context.Background(), bytes.NewReader(data), contracts, params, lockingPriorityUpload) + + // assert we only have one uploader left + if len(ul.uploaders) != 1 { + t.Fatalf("unexpected number of uploaders, %v != %v", len(ul.uploaders), 1) + } + + // assert all queued requests failed with an error indicating the underlying + // contract expired + res := <-responseChan + if !errors.Is(res.err, errContractExpired) { + t.Fatal("expected contract expired error", res.err) + } +} + func testParameters(path string) uploadParameters { return uploadParameters{ bucket: testBucket, diff --git a/worker/uploader.go b/worker/uploader.go index bc8f06a5e..8601e9601 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -143,14 +143,14 @@ outer: } } -func (u *uploader) Stop() { +func (u *uploader) Stop(err error) { for { upload := u.pop() if upload == nil { break } if !upload.done() { - upload.fail(errors.New("uploader stopped")) + upload.fail(err) } } } From 7b45ecc371daf6bd1e7c75299a75986d4aac09d5 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 23 Jan 2024 10:15:46 +0100 Subject: [PATCH 005/144] worker: update mocks --- worker/host_test.go | 3 +- worker/mocks_test.go | 287 +++++++++++++++++++++++-------------- worker/pricetables_test.go | 11 +- worker/upload_test.go | 29 ++-- 4 files changed, 211 insertions(+), 119 deletions(-) diff --git a/worker/host_test.go b/worker/host_test.go index e9b3ab798..87d35fb36 100644 --- a/worker/host_test.go +++ b/worker/host_test.go @@ -11,7 +11,8 @@ import ( ) func TestHost(t *testing.T) { - h := newMockHost(true) + h := newMockHost(types.PublicKey{1}) + h.c = newMockContract(h.hk, types.FileContractID{1}) sector, root := newMockSector() // upload the sector diff --git a/worker/mocks_test.go b/worker/mocks_test.go index e861edcdc..353c4e8e2 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -29,7 +29,8 @@ type ( } mockContractStore struct { - contracts map[types.FileContractID]*mockContract + mu sync.Mutex + locks map[types.FileContractID]*sync.Mutex } mockHost struct { @@ -42,10 +43,8 @@ type ( hptBlockChan chan struct{} } - mockHosts []*mockHost - mockContracts []*mockContract - mockHostManager struct { + mu sync.Mutex hosts map[types.PublicKey]*mockHost } @@ -74,6 +73,10 @@ type ( dl *downloadManager ul *uploadManager + + mu sync.Mutex + hkCntr uint + fcidCntr uint } ) @@ -94,21 +97,10 @@ var ( errSectorOutOfBounds = errors.New("sector out of bounds") ) -var mockContractCntr = 0 - -func newFileContractID() (fcid types.FileContractID) { - fcid = types.FileContractID{byte(mockContractCntr)} - mockContractCntr++ - return -} - -var mockHostCntr = 0 - -func newHostKey() (hk types.PublicKey) { - hk = types.PublicKey{byte(mockHostCntr)} - mockHostCntr++ - return -} +type ( + mockHosts []*mockHost + mockContracts []*mockContract +) func (hosts mockHosts) contracts() mockContracts { contracts := make([]*mockContract, len(hosts)) @@ -137,10 +129,59 @@ func (mm *mockMemoryManager) AcquireMemory(ctx context.Context, amt uint64) Memo return &mockMemory{} } +func newMockContractStore() *mockContractStore { + return &mockContractStore{ + locks: make(map[types.FileContractID]*sync.Mutex), + } +} + +func (cs *mockContractStore) AcquireContract(ctx context.Context, fcid types.FileContractID, priority int, d time.Duration) (lockID uint64, err error) { + cs.mu.Lock() + defer cs.mu.Unlock() + + if lock, ok := cs.locks[fcid]; !ok { + return 0, errContractNotFound + } else { + lock.Lock() + } + return 0, nil +} + +func (cs *mockContractStore) ReleaseContract(ctx context.Context, fcid types.FileContractID, lockID uint64) (err error) { + cs.mu.Lock() + defer cs.mu.Unlock() + + if lock, ok := cs.locks[fcid]; !ok { + return errContractNotFound + } else { + lock.Unlock() + } + return nil +} + +func (cs *mockContractStore) KeepaliveContract(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration) (err error) { + return nil +} + func (os *mockContractStore) RenewedContract(ctx context.Context, renewedFrom types.FileContractID) (api.ContractMetadata, error) { return api.ContractMetadata{}, api.ErrContractNotFound } +func newMockObjectStore() *mockObjectStore { + os := &mockObjectStore{ + objects: make(map[string]map[string]object.Object), + partials: make(map[string]mockPackedSlab), + } + os.objects[testBucket] = make(map[string]object.Object) + return os +} + +func (cs *mockContractStore) addContract(c *mockContract) { + cs.mu.Lock() + defer cs.mu.Unlock() + cs.locks[c.metadata.ID] = new(sync.Mutex) +} + func (os *mockObjectStore) AddMultipartPart(ctx context.Context, bucket, path, contractSet, ETag, uploadID string, partNumber int, slices []object.SlabSlice) (err error) { return nil } @@ -329,8 +370,15 @@ func (os *mockObjectStore) forEachObject(fn func(bucket, path string, o object.O } } +func newMockHost(hk types.PublicKey) *mockHost { + return &mockHost{ + hk: hk, + hpt: newTestHostPriceTable(time.Now().Add(time.Minute)), + } +} + 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] + sector, exist := h.contract().sector(root) if !exist { return errSectorNotFound } @@ -342,9 +390,7 @@ func (h *mockHost) DownloadSector(ctx context.Context, w io.Writer, root types.H } 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 + return h.contract().addSector(sector), nil } func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (rev types.FileContractRevision, _ error) { @@ -364,72 +410,25 @@ func (h *mockHost) FundAccount(ctx context.Context, balance types.Currency, rev } func (h *mockHost) RenewContract(ctx context.Context, rrr api.RHPRenewRequest) (_ rhpv2.ContractRevision, _ []types.Transaction, _ types.Currency, err error) { - h.mu.Lock() - defer h.mu.Unlock() - - curr := h.c.metadata - update := newMockContract(h.hk) - update.metadata.RenewedFrom = curr.ID - update.metadata.WindowStart = curr.WindowEnd - update.metadata.WindowEnd = update.metadata.WindowStart + (curr.WindowEnd - curr.WindowStart) - h.c = update - return + return rhpv2.ContractRevision{}, nil, types.ZeroCurrency, nil } 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 (cs *mockContractStore) AcquireContract(ctx context.Context, fcid types.FileContractID, priority int, d time.Duration) (lockID uint64, err error) { - if lock, ok := cs.contracts[fcid]; !ok { - return 0, errContractNotFound - } else { - lock.mu.Lock() - } - - return 0, nil -} - -func (cs *mockContractStore) ReleaseContract(ctx context.Context, fcid types.FileContractID, lockID uint64) (err error) { - if lock, ok := cs.contracts[fcid]; !ok { - return errContractNotFound - } else { - lock.mu.Unlock() - } - return nil -} - -func (cs *mockContractStore) KeepaliveContract(ctx context.Context, fcid types.FileContractID, lockID uint64, d time.Duration) (err error) { - return nil -} - -func newMockHost(contract bool) *mockHost { - host := &mockHost{ - hk: newHostKey(), - hpt: newTestHostPriceTable(time.Now().Add(time.Minute)), - } - if contract { - host.c = newMockContract(host.hk) - } - return host -} +func (h *mockHost) contract() (c *mockContract) { + h.mu.Lock() + c = h.c + h.mu.Unlock() -func newMockHosts(n int, contract bool) (hosts mockHosts) { - for i := 0; i < n; i++ { - hosts = append(hosts, newMockHost(contract)) + if c == nil { + panic("host does not have a contract") } return } -func newMockContract(hk types.PublicKey) *mockContract { - fcid := newFileContractID() +func newMockContract(hk types.PublicKey, fcid types.FileContractID) *mockContract { return &mockContract{ metadata: api.ContractMetadata{ ID: fcid, @@ -442,26 +441,52 @@ func newMockContract(hk types.PublicKey) *mockContract { } } -func newMockContractStore(contracts []*mockContract) *mockContractStore { - cs := &mockContractStore{contracts: make(map[types.FileContractID]*mockContract)} - for _, c := range contracts { - cs.contracts[c.rev.ParentID] = c +func (c *mockContract) addSector(sector *[rhpv2.SectorSize]byte) (root types.Hash256) { + root = rhpv2.SectorRoot(sector) + c.mu.Lock() + c.sectors[root] = sector + c.mu.Unlock() + return +} + +func (c *mockContract) sector(root types.Hash256) (sector *[rhpv2.SectorSize]byte, found bool) { + c.mu.Lock() + sector, found = c.sectors[root] + c.mu.Unlock() + return +} + +func newMockHostManager() *mockHostManager { + return &mockHostManager{ + hosts: make(map[types.PublicKey]*mockHost), } - return cs } -func newMockHostManager(hosts []*mockHost) *mockHostManager { - hm := &mockHostManager{hosts: make(map[types.PublicKey]*mockHost)} - for _, h := range hosts { - hm.hosts[h.hk] = h +func (hm *mockHostManager) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr string) Host { + hm.mu.Lock() + defer hm.mu.Unlock() + + if _, ok := hm.hosts[hk]; !ok { + panic("host not found") } - return hm + return hm.hosts[hk] } -func newMockObjectStore() *mockObjectStore { - os := &mockObjectStore{objects: make(map[string]map[string]object.Object), partials: make(map[string]mockPackedSlab)} - os.objects[testBucket] = make(map[string]object.Object) - return os +func (hm *mockHostManager) addHost(h *mockHost) { + hm.mu.Lock() + defer hm.mu.Unlock() + + if _, ok := hm.hosts[h.hk]; ok { + panic("host already exists") + } + + hm.hosts[h.hk] = h +} + +func (hm *mockHostManager) host(hk types.PublicKey) *mockHost { + hm.mu.Lock() + defer hm.mu.Unlock() + return hm.hosts[hk] } func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { @@ -470,30 +495,68 @@ func newMockSector() (*[rhpv2.SectorSize]byte, types.Hash256) { return §or, rhpv2.SectorRoot(§or) } -func newMockWorker(numHosts int) *mockWorker { - // create hosts - hosts := newMockHosts(numHosts, true) - - // create dependencies - cs := newMockContractStore(hosts.contracts()) - hm := newMockHostManager(hosts) +func newMockWorker() *mockWorker { + cs := newMockContractStore() + hm := newMockHostManager() os := newMockObjectStore() mm := &mockMemoryManager{} - dl := newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()) - ul := newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()) - return &mockWorker{ cs: cs, hm: hm, mm: mm, os: os, - dl: dl, - ul: ul, + dl: newDownloadManager(context.Background(), hm, mm, os, 0, 0, zap.NewNop().Sugar()), + ul: newUploadManager(context.Background(), hm, mm, os, cs, 0, 0, time.Minute, zap.NewNop().Sugar()), } } +func (w *mockWorker) addHosts(n int) { + for i := 0; i < n; i++ { + w.addHost() + } +} + +func (w *mockWorker) addHost() *mockHost { + host := newMockHost(w.newHostKey()) + w.hm.addHost(host) + w.formContractWithHost(host.hk) + return host +} + +func (w *mockWorker) formContractWithHost(hk types.PublicKey) *mockContract { + host := w.hm.host(hk) + if host == nil { + panic("host not found") + } else if host.c != nil { + panic("host already has contract, use renew") + } + + host.c = newMockContract(host.hk, w.newFileContractID()) + w.cs.addContract(host.c) + return host.c +} + +func (w *mockWorker) renewContractWithHost(hk types.PublicKey) *mockContract { + host := w.hm.host(hk) + if host == nil { + panic("host not found") + } else if host.c == nil { + panic("host does not have a contract to renew") + } + + curr := host.c.metadata + update := newMockContract(host.hk, w.newFileContractID()) + update.metadata.RenewedFrom = curr.ID + update.metadata.WindowStart = curr.WindowEnd + update.metadata.WindowEnd = update.metadata.WindowStart + (curr.WindowEnd - curr.WindowStart) + host.c = update + + w.cs.addContract(host.c) + return host.c +} + func (w *mockWorker) contracts() (metadatas []api.ContractMetadata) { for _, h := range w.hm.hosts { metadatas = append(metadatas, h.c.metadata) @@ -501,6 +564,22 @@ func (w *mockWorker) contracts() (metadatas []api.ContractMetadata) { return } +func (w *mockWorker) newHostKey() (hk types.PublicKey) { + w.mu.Lock() + defer w.mu.Unlock() + w.hkCntr++ + hk = types.PublicKey{byte(w.hkCntr)} + return +} + +func (w *mockWorker) newFileContractID() (fcid types.FileContractID) { + w.mu.Lock() + defer w.mu.Unlock() + w.fcidCntr++ + fcid = types.FileContractID{byte(w.fcidCntr)} + return +} + func newTestHostPriceTable(expiry time.Time) hostdb.HostPriceTable { var uid rhpv3.SettingsID frand.Read(uid[:]) diff --git a/worker/pricetables_test.go b/worker/pricetables_test.go index b90ca4fec..3aa7b11f3 100644 --- a/worker/pricetables_test.go +++ b/worker/pricetables_test.go @@ -49,9 +49,13 @@ func TestPriceTables(t *testing.T) { validPT := newTestHostPriceTable(time.Now().Add(time.Minute)) // create a mock host that has a valid price table - h1 := newMockHost(false) + hk1 := types.PublicKey{1} + h1 := newMockHost(hk1) h1.hpt = validPT - hk1 := h1.hk + + // create host manager + hm := newMockHostManager() + hm.addHost(h1) // create a hostdb entry for that host that returns the expired price table hdb1 := &hostdb.HostInfo{ @@ -62,8 +66,7 @@ func TestPriceTables(t *testing.T) { }, } - // create host store and manager - hm := newMockHostManager([]*mockHost{h1}) + // create host store hs := newMockHostStore([]*hostdb.HostInfo{hdb1}) // create price tables diff --git a/worker/upload_test.go b/worker/upload_test.go index 5b6df4de4..200aa462a 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -25,7 +25,10 @@ var ( func TestUpload(t *testing.T) { // mock worker - w := newMockWorker(testRedundancySettings.TotalShards * 2) + w := newMockWorker() + + // add hosts to worker + w.addHosts(testRedundancySettings.TotalShards * 2) // convenience variables os := w.os @@ -127,7 +130,10 @@ func TestUpload(t *testing.T) { func TestUploadPackedSlab(t *testing.T) { // mock worker - w := newMockWorker(testRedundancySettings.TotalShards * 2) + w := newMockWorker() + + // add hosts to worker + w.addHosts(testRedundancySettings.TotalShards * 2) // convenience variables os := w.os @@ -210,7 +216,10 @@ func TestUploadPackedSlab(t *testing.T) { func TestUploadShards(t *testing.T) { // mock worker - w := newMockWorker(testRedundancySettings.TotalShards * 2) + w := newMockWorker() + + // add hosts to worker + w.addHosts(testRedundancySettings.TotalShards * 2) // convenience variables os := w.os @@ -326,7 +335,10 @@ func TestUploadShards(t *testing.T) { func TestRefreshUploaders(t *testing.T) { // mock worker - w := newMockWorker(testRedundancySettings.TotalShards * 2) + w := newMockWorker() + + // add hosts to worker + w.addHosts(testRedundancySettings.TotalShards) // convenience variables ul := w.ul @@ -356,18 +368,15 @@ func TestRefreshUploaders(t *testing.T) { // renew the first contract c1 := contracts[0] - hm.hosts[c1.HostKey].RenewContract(context.Background(), api.RHPRenewRequest{}) - c1Renewed := hm.hosts[c1.HostKey].c + c1Renewed := w.renewContractWithHost(c1.HostKey) // remove the host from the second contract c2 := contracts[1] delete(hm.hosts, c2.HostKey) - delete(cs.contracts, c2.ID) + delete(cs.locks, c2.ID) // add a new host/contract - hNew := newMockHost(true) - hm.hosts[hNew.hk] = hNew - cs.contracts[hNew.c.metadata.ID] = hNew.c + hNew := w.addHost() // upload data contracts = w.contracts() From 90bf115e48971a74a690005f894278fe5f446b9e Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 23 Jan 2024 10:43:14 +0100 Subject: [PATCH 006/144] worker: use host blockheight --- worker/host.go | 2 +- worker/mocks_test.go | 2 +- worker/rhpv3.go | 16 +++++++--------- worker/upload.go | 10 ++++------ worker/uploader.go | 21 +++------------------ worker/worker.go | 24 +++++++++--------------- 6 files changed, 25 insertions(+), 50 deletions(-) diff --git a/worker/host.go b/worker/host.go index 2e13af353..320645b8b 100644 --- a/worker/host.go +++ b/worker/host.go @@ -22,7 +22,7 @@ type ( UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) FetchPriceTable(ctx context.Context, rev *types.FileContractRevision) (hpt hostdb.HostPriceTable, err error) - FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (types.FileContractRevision, error) + FetchRevision(ctx context.Context, fetchTimeout time.Duration) (types.FileContractRevision, error) FundAccount(ctx context.Context, balance types.Currency, rev *types.FileContractRevision) error SyncAccount(ctx context.Context, rev *types.FileContractRevision) error diff --git a/worker/mocks_test.go b/worker/mocks_test.go index 353c4e8e2..329242410 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -393,7 +393,7 @@ func (h *mockHost) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]b return h.contract().addSector(sector), nil } -func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (rev types.FileContractRevision, _ error) { +func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration) (rev types.FileContractRevision, _ error) { h.mu.Lock() defer h.mu.Unlock() rev = h.c.rev diff --git a/worker/rhpv3.go b/worker/rhpv3.go index 7f83af4f2..03f67c6f6 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -239,20 +239,19 @@ func (p *transportPoolV3) withTransportV3(ctx context.Context, hostKey types.Pub return err } -// FetchRevision tries to fetch a contract revision from the host. We pass in -// the blockHeight instead of using the blockHeight from the pricetable since we -// might not have a price table. -func (h *host) FetchRevision(ctx context.Context, fetchTimeout time.Duration, blockHeight uint64) (types.FileContractRevision, error) { +// FetchRevision tries to fetch a contract revision from the host. +func (h *host) FetchRevision(ctx context.Context, fetchTimeout time.Duration) (types.FileContractRevision, error) { timeoutCtx := func() (context.Context, context.CancelFunc) { if fetchTimeout > 0 { return context.WithTimeout(ctx, fetchTimeout) } return ctx, func() {} } + // Try to fetch the revision with an account first. ctx, cancel := timeoutCtx() defer cancel() - rev, err := h.fetchRevisionWithAccount(ctx, h.hk, h.siamuxAddr, blockHeight, h.fcid) + rev, err := h.fetchRevisionWithAccount(ctx, h.hk, h.siamuxAddr, h.fcid) if err != nil && !(isBalanceInsufficient(err) || isWithdrawalsInactive(err) || isWithdrawalExpired(err) || isClosedStream(err)) { // TODO: checking for a closed stream here can be removed once the withdrawal timeout on the host side is removed return types.FileContractRevision{}, fmt.Errorf("unable to fetch revision with account: %v", err) } else if err == nil { @@ -279,18 +278,17 @@ func (h *host) FetchRevision(ctx context.Context, fetchTimeout time.Duration, bl return rev, nil } -func (h *host) fetchRevisionWithAccount(ctx context.Context, hostKey types.PublicKey, siamuxAddr string, bh uint64, contractID types.FileContractID) (rev types.FileContractRevision, err error) { +func (h *host) fetchRevisionWithAccount(ctx context.Context, hostKey types.PublicKey, siamuxAddr string, fcid types.FileContractID) (rev types.FileContractRevision, err error) { err = h.acc.WithWithdrawal(ctx, func() (types.Currency, error) { var cost types.Currency return cost, h.transportPool.withTransportV3(ctx, hostKey, siamuxAddr, func(ctx context.Context, t *transportV3) (err error) { - rev, err = RPCLatestRevision(ctx, t, contractID, func(rev *types.FileContractRevision) (rhpv3.HostPriceTable, rhpv3.PaymentMethod, error) { - // Fetch pt. + rev, err = RPCLatestRevision(ctx, t, fcid, func(rev *types.FileContractRevision) (rhpv3.HostPriceTable, rhpv3.PaymentMethod, error) { pt, err := h.priceTable(ctx, nil) if err != nil { return rhpv3.HostPriceTable{}, nil, fmt.Errorf("failed to fetch pricetable, err: %w", err) } cost = pt.LatestRevisionCost.Add(pt.UpdatePriceTableCost) // add cost of fetching the pricetable since we might need a new one and it's better to stay pessimistic - payment := rhpv3.PayByEphemeralAccount(h.acc.id, cost, bh+defaultWithdrawalExpiryBlocks, h.accountKey) + payment := rhpv3.PayByEphemeralAccount(h.acc.id, cost, pt.HostBlockHeight+defaultWithdrawalExpiryBlocks, h.accountKey) return pt, &payment, nil }) if err != nil { diff --git a/worker/upload.go b/worker/upload.go index 692e40c70..c89a3a60c 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -338,7 +338,7 @@ func newUploadManager(ctx context.Context, hm HostManager, mm MemoryManager, os } } -func (mgr *uploadManager) newUploader(os ObjectStore, cs ContractStore, hm HostManager, c api.ContractMetadata, bh uint64) *uploader { +func (mgr *uploadManager) newUploader(os ObjectStore, cs ContractStore, hm HostManager, c api.ContractMetadata) *uploader { return &uploader{ os: os, cs: cs, @@ -357,7 +357,6 @@ func (mgr *uploadManager) newUploader(os ObjectStore, cs ContractStore, hm HostM // covered by mutex host: hm.Host(c.HostKey, c.ID, c.SiamuxAddr), - bh: bh, fcid: c.ID, endHeight: c.WindowEnd, queue: make([]*sectorUploadReq, 0), @@ -734,7 +733,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh for _, uploader := range mgr.uploaders { // refresh uploaders that got renewed if renewal, renewed := renewals[uploader.ContractID()]; renewed { - uploader.Refresh(renewal, bh) + uploader.Refresh(renewal) } // stop uploaders that expired @@ -743,8 +742,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh continue } - // update uploader - uploader.UpdateBlockHeight(bh) + // recompute the stats uploader.tryRecomputeStats() // add to the list @@ -755,7 +753,7 @@ func (mgr *uploadManager) refreshUploaders(contracts []api.ContractMetadata, bh // add missing uploaders for _, c := range contracts { if _, exists := existing[c.ID]; !exists && bh < c.WindowEnd { - uploader := mgr.newUploader(mgr.os, mgr.cs, mgr.hm, c, bh) + uploader := mgr.newUploader(mgr.os, mgr.cs, mgr.hm, c) refreshed = append(refreshed, uploader) go uploader.Start() } diff --git a/worker/uploader.go b/worker/uploader.go index 8601e9601..3ec88c6fa 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -32,7 +32,6 @@ type ( shutdownCtx context.Context mu sync.Mutex - bh uint64 endHeight uint64 fcid types.FileContractID host Host @@ -47,12 +46,6 @@ type ( } ) -func (u *uploader) BlockHeight() uint64 { - u.mu.Lock() - defer u.mu.Unlock() - return u.bh -} - func (u *uploader) ContractID() types.FileContractID { u.mu.Lock() defer u.mu.Unlock() @@ -71,11 +64,10 @@ func (u *uploader) Healthy() bool { return u.consecutiveFailures == 0 } -func (u *uploader) Refresh(c api.ContractMetadata, bh uint64) { +func (u *uploader) Refresh(c api.ContractMetadata) { u.mu.Lock() defer u.mu.Unlock() - u.bh = bh u.host = u.hm.Host(c.HostKey, c.ID, c.SiamuxAddr) u.fcid = c.ID u.siamuxAddr = c.SiamuxAddr @@ -155,12 +147,6 @@ func (u *uploader) Stop(err error) { } } -func (u *uploader) UpdateBlockHeight(bh uint64) { - u.mu.Lock() - defer u.mu.Unlock() - u.bh = bh -} - func (u *uploader) enqueue(req *sectorUploadReq) { // decorate the request req.fcid = u.ContractID() @@ -193,7 +179,6 @@ func (u *uploader) estimate() float64 { 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() @@ -217,7 +202,7 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, defer cancel() // fetch the revision - rev, err := host.FetchRevision(ctx, defaultRevisionFetchTimeout, bh) + rev, err := host.FetchRevision(ctx, defaultRevisionFetchTimeout) if err != nil { return types.Hash256{}, 0, err } else if rev.RevisionNumber == math.MaxUint64 { @@ -302,6 +287,6 @@ func (u *uploader) tryRefresh(ctx context.Context) bool { } // renew the uploader with the renewed contract - u.Refresh(renewed, u.BlockHeight()) + u.Refresh(renewed) return true } diff --git a/worker/worker.go b/worker/worker.go index f8ce65d2b..bd044952a 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -211,10 +211,10 @@ type worker struct { shutdownCtxCancel context.CancelFunc } -func (w *worker) withRevision(ctx context.Context, fetchTimeout time.Duration, fcid types.FileContractID, hk types.PublicKey, siamuxAddr string, lockPriority int, blockHeight uint64, fn func(rev types.FileContractRevision) error) error { +func (w *worker) withRevision(ctx context.Context, fetchTimeout time.Duration, fcid types.FileContractID, hk types.PublicKey, siamuxAddr string, lockPriority int, fn func(rev types.FileContractRevision) error) error { return w.withContractLock(ctx, fcid, lockPriority, func() error { h := w.Host(hk, fcid, siamuxAddr) - rev, err := h.FetchRevision(ctx, fetchTimeout, blockHeight) + rev, err := h.FetchRevision(ctx, fetchTimeout) if err != nil { return err } @@ -271,7 +271,7 @@ func (w *worker) rhpScanHandler(jc jape.Context) { }) } -func (w *worker) fetchContracts(ctx context.Context, metadatas []api.ContractMetadata, timeout time.Duration, blockHeight uint64) (contracts []api.Contract, errs HostErrorSet) { +func (w *worker) fetchContracts(ctx context.Context, metadatas []api.ContractMetadata, timeout time.Duration) (contracts []api.Contract, errs HostErrorSet) { errs = make(HostErrorSet) // create requests channel @@ -282,7 +282,7 @@ func (w *worker) fetchContracts(ctx context.Context, metadatas []api.ContractMet worker := func() { for md := range reqs { var revision types.FileContractRevision - err := w.withRevision(ctx, timeout, md.ID, md.HostKey, md.SiamuxAddr, lockingPriorityActiveContractRevision, blockHeight, func(rev types.FileContractRevision) error { + err := w.withRevision(ctx, timeout, md.ID, md.HostKey, md.SiamuxAddr, lockingPriorityActiveContractRevision, func(rev types.FileContractRevision) error { revision = rev return nil }) @@ -619,17 +619,13 @@ func (w *worker) rhpRenewHandler(jc jape.Context) { if jc.Check("could not get gouging parameters", err) != nil { return } - cs, err := w.bus.ConsensusState(ctx) - if jc.Check("could not get consensus state", err) != nil { - return - } ctx = WithGougingChecker(ctx, w.bus, gp) // renew the contract var renewed rhpv2.ContractRevision var txnSet []types.Transaction var contractPrice types.Currency - if jc.Check("couldn't renew contract", w.withRevision(ctx, defaultRevisionFetchTimeout, rrr.ContractID, rrr.HostKey, rrr.SiamuxAddr, lockingPriorityRenew, cs.BlockHeight, func(_ types.FileContractRevision) (err error) { + if jc.Check("couldn't renew contract", w.withRevision(ctx, defaultRevisionFetchTimeout, rrr.ContractID, rrr.HostKey, rrr.SiamuxAddr, lockingPriorityRenew, func(_ types.FileContractRevision) (err error) { h := w.Host(rrr.HostKey, rrr.ContractID, rrr.SiamuxAddr) renewed, txnSet, contractPrice, err = h.RenewContract(ctx, rrr) return err @@ -669,7 +665,7 @@ func (w *worker) rhpFundHandler(jc jape.Context) { ctx = WithGougingChecker(ctx, w.bus, gp) // fund the account - jc.Check("couldn't fund account", w.withRevision(ctx, defaultRevisionFetchTimeout, rfr.ContractID, rfr.HostKey, rfr.SiamuxAddr, lockingPriorityFunding, gp.ConsensusState.BlockHeight, func(rev types.FileContractRevision) (err error) { + jc.Check("couldn't fund account", w.withRevision(ctx, defaultRevisionFetchTimeout, rfr.ContractID, rfr.HostKey, rfr.SiamuxAddr, lockingPriorityFunding, func(rev types.FileContractRevision) (err error) { h := w.Host(rfr.HostKey, rev.ParentID, rfr.SiamuxAddr) err = h.FundAccount(ctx, rfr.Balance, &rev) if isBalanceMaxExceeded(err) { @@ -699,18 +695,16 @@ func (w *worker) rhpSyncHandler(jc jape.Context) { return } - // fetch gouging params + // attach gouging checker up, err := w.bus.UploadParams(ctx) if jc.Check("couldn't fetch upload parameters from bus", err) != nil { return } - - // attach gouging checker to the context ctx = WithGougingChecker(ctx, w.bus, up.GougingParams) // sync the account h := w.Host(rsr.HostKey, rsr.ContractID, rsr.SiamuxAddr) - jc.Check("couldn't sync account", w.withRevision(ctx, defaultRevisionFetchTimeout, rsr.ContractID, rsr.HostKey, rsr.SiamuxAddr, lockingPrioritySyncing, up.CurrentHeight, func(rev types.FileContractRevision) error { + jc.Check("couldn't sync account", w.withRevision(ctx, defaultRevisionFetchTimeout, rsr.ContractID, rsr.HostKey, rsr.SiamuxAddr, lockingPrioritySyncing, func(rev types.FileContractRevision) error { return h.SyncAccount(ctx, &rev) })) } @@ -1236,7 +1230,7 @@ func (w *worker) rhpContractsHandlerGET(jc jape.Context) { } ctx = WithGougingChecker(ctx, w.bus, gp) - contracts, errs := w.fetchContracts(ctx, busContracts, hosttimeout, gp.ConsensusState.BlockHeight) + contracts, errs := w.fetchContracts(ctx, busContracts, hosttimeout) resp := api.ContractsResponse{Contracts: contracts} if errs != nil { resp.Error = errs.Error() From 71807f1dff2c8b5b9467a94432befac1f569da1f Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 23 Jan 2024 11:29:11 +0100 Subject: [PATCH 007/144] worker: update mocks --- worker/mocks_test.go | 22 +++++++++------------- worker/pricetables_test.go | 9 ++++----- worker/upload_test.go | 2 +- 3 files changed, 14 insertions(+), 19 deletions(-) diff --git a/worker/mocks_test.go b/worker/mocks_test.go index 329242410..260d6b3c3 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -472,15 +472,16 @@ func (hm *mockHostManager) Host(hk types.PublicKey, fcid types.FileContractID, s return hm.hosts[hk] } -func (hm *mockHostManager) addHost(h *mockHost) { +func (hm *mockHostManager) newHost(hk types.PublicKey) *mockHost { hm.mu.Lock() defer hm.mu.Unlock() - if _, ok := hm.hosts[h.hk]; ok { + if _, ok := hm.hosts[hk]; ok { panic("host already exists") } - hm.hosts[h.hk] = h + hm.hosts[hk] = newMockHost(hk) + return hm.hosts[hk] } func (hm *mockHostManager) host(hk types.PublicKey) *mockHost { @@ -519,26 +520,21 @@ func (w *mockWorker) addHosts(n int) { } func (w *mockWorker) addHost() *mockHost { - host := newMockHost(w.newHostKey()) - w.hm.addHost(host) - w.formContractWithHost(host.hk) + host := w.hm.newHost(w.newHostKey()) + w.formContract(host) return host } -func (w *mockWorker) formContractWithHost(hk types.PublicKey) *mockContract { - host := w.hm.host(hk) - if host == nil { - panic("host not found") - } else if host.c != nil { +func (w *mockWorker) formContract(host *mockHost) *mockContract { + if host.c != nil { panic("host already has contract, use renew") } - host.c = newMockContract(host.hk, w.newFileContractID()) w.cs.addContract(host.c) return host.c } -func (w *mockWorker) renewContractWithHost(hk types.PublicKey) *mockContract { +func (w *mockWorker) renewContract(hk types.PublicKey) *mockContract { host := w.hm.host(hk) if host == nil { panic("host not found") diff --git a/worker/pricetables_test.go b/worker/pricetables_test.go index 3aa7b11f3..115abcd31 100644 --- a/worker/pricetables_test.go +++ b/worker/pricetables_test.go @@ -48,15 +48,14 @@ func TestPriceTables(t *testing.T) { expiredPT := newTestHostPriceTable(time.Now()) validPT := newTestHostPriceTable(time.Now().Add(time.Minute)) + // create host manager + hm := newMockHostManager() + // create a mock host that has a valid price table hk1 := types.PublicKey{1} - h1 := newMockHost(hk1) + h1 := hm.newHost(hk1) h1.hpt = validPT - // create host manager - hm := newMockHostManager() - hm.addHost(h1) - // create a hostdb entry for that host that returns the expired price table hdb1 := &hostdb.HostInfo{ Host: hostdb.Host{ diff --git a/worker/upload_test.go b/worker/upload_test.go index 200aa462a..fe050432f 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -368,7 +368,7 @@ func TestRefreshUploaders(t *testing.T) { // renew the first contract c1 := contracts[0] - c1Renewed := w.renewContractWithHost(c1.HostKey) + c1Renewed := w.renewContract(c1.HostKey) // remove the host from the second contract c2 := contracts[1] From 23b1ace798d91137b78907f34d00ca5a349e5a35 Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 22 Jan 2024 10:34:47 +0100 Subject: [PATCH 008/144] worker: improve contract and host interaction recorders --- worker/download.go | 3 +- worker/host.go | 4 +- worker/interactions.go | 144 +++++++++++++++++++++++++++++------------ worker/spending.go | 120 ++++++++++++++++++++-------------- worker/upload.go | 3 +- worker/worker.go | 85 +++++++++++------------- 6 files changed, 218 insertions(+), 141 deletions(-) diff --git a/worker/download.go b/worker/download.go index bfeb272ac..128b7637c 100644 --- a/worker/download.go +++ b/worker/download.go @@ -25,7 +25,6 @@ const ( ) var ( - errDownloadManagerStopped = errors.New("download manager stopped") errDownloadNotEnoughHosts = errors.New("not enough hosts available to download the slab") ) @@ -291,7 +290,7 @@ outer: var resp *slabDownloadResponse select { case <-mgr.shutdownCtx.Done(): - return errDownloadManagerStopped + return ErrShuttingDown case <-ctx.Done(): return errors.New("download timed out") case resp = <-responseChan: diff --git a/worker/host.go b/worker/host.go index 2e13af353..f7ae6cb1f 100644 --- a/worker/host.go +++ b/worker/host.go @@ -49,7 +49,7 @@ type ( acc *account bus Bus - contractSpendingRecorder *contractSpendingRecorder + contractSpendingRecorder ContractSpendingRecorder logger *zap.SugaredLogger transportPool *transportPoolV3 priceTables *priceTables @@ -192,7 +192,7 @@ func (h *host) FetchPriceTable(ctx context.Context, rev *types.FileContractRevis fetchPT := func(paymentFn PriceTablePaymentFunc) (hpt hostdb.HostPriceTable, err error) { err = h.transportPool.withTransportV3(ctx, h.hk, h.siamuxAddr, func(ctx context.Context, t *transportV3) (err error) { hpt, err = RPCPriceTable(ctx, t, paymentFn) - InteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ + HostInteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ HostKey: h.hk, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), diff --git a/worker/interactions.go b/worker/interactions.go index 2321ebb30..70629c1f0 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -4,10 +4,12 @@ import ( "context" "fmt" "net/http" + "sync" "time" "go.sia.tech/jape" "go.sia.tech/renterd/hostdb" + "go.uber.org/zap" ) const ( @@ -15,15 +17,40 @@ const ( ) type ( - InteractionRecorder interface { + HostInteractionRecorder interface { RecordHostScan(...hostdb.HostScan) RecordPriceTableUpdate(...hostdb.PriceTableUpdate) + Stop(context.Context) + } + + hostInteractionRecorder struct { + flushInterval time.Duration + + bus Bus + logger *zap.SugaredLogger + + mu sync.Mutex + hostScans []hostdb.HostScan + priceTableUpdates []hostdb.PriceTableUpdate + + flushCtx context.Context + flushTimer *time.Timer } ) -var _ InteractionRecorder = &worker{} +var ( + _ HostInteractionRecorder = (*hostInteractionRecorder)(nil) +) -func interactionMiddleware(ir InteractionRecorder, routes map[string]jape.Handler) map[string]jape.Handler { +func HostInteractionRecorderFromContext(ctx context.Context) HostInteractionRecorder { + ir, ok := ctx.Value(keyInteractionRecorder).(HostInteractionRecorder) + if !ok { + panic("no interaction recorder attached to the context") // developer error + } + return ir +} + +func interactionMiddleware(ir HostInteractionRecorder, routes map[string]jape.Handler) map[string]jape.Handler { for route, handler := range routes { routes[route] = jape.Adapt(func(h http.Handler) http.Handler { return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { @@ -35,61 +62,94 @@ func interactionMiddleware(ir InteractionRecorder, routes map[string]jape.Handle return routes } -func InteractionRecorderFromContext(ctx context.Context) InteractionRecorder { - ir, ok := ctx.Value(keyInteractionRecorder).(InteractionRecorder) - if !ok { - panic("no interaction recorder attached to the context") // developer error +func (w *worker) initHostInteractionRecorder(flushInterval time.Duration) { + if w.hostInteractionRecorder != nil { + panic("HostInteractionRecorder already initialized") // developer error } - return ir -} + w.hostInteractionRecorder = &hostInteractionRecorder{ + bus: w.bus, + logger: w.logger, -func (w *worker) RecordHostScan(scans ...hostdb.HostScan) { - w.interactionsMu.Lock() - defer w.interactionsMu.Unlock() + flushCtx: w.shutdownCtx, + flushInterval: flushInterval, - w.interactionsScans = append(w.interactionsScans, scans...) - w.tryFlushInteractionsBuffer() + hostScans: make([]hostdb.HostScan, 0), + priceTableUpdates: make([]hostdb.PriceTableUpdate, 0), + } } -func (w *worker) RecordPriceTableUpdate(ptUpdates ...hostdb.PriceTableUpdate) { - w.interactionsMu.Lock() - defer w.interactionsMu.Unlock() +func (r *hostInteractionRecorder) RecordHostScan(scans ...hostdb.HostScan) { + r.mu.Lock() + defer r.mu.Unlock() + r.hostScans = append(r.hostScans, scans...) + r.tryFlushInteractionsBuffer() +} - w.interactionsPriceTableUpdates = append(w.interactionsPriceTableUpdates, ptUpdates...) - w.tryFlushInteractionsBuffer() +func (r *hostInteractionRecorder) RecordPriceTableUpdate(ptUpdates ...hostdb.PriceTableUpdate) { + r.mu.Lock() + defer r.mu.Unlock() + r.priceTableUpdates = append(r.priceTableUpdates, ptUpdates...) + r.tryFlushInteractionsBuffer() } -func (w *worker) tryFlushInteractionsBuffer() { - // If a thread was scheduled to flush the buffer we are done. - if w.interactionsFlushTimer != nil { - return +func (r *hostInteractionRecorder) Stop(ctx context.Context) { + // stop the flush timer + r.mu.Lock() + if r.flushTimer != nil { + r.flushTimer.Stop() } + r.flushCtx = ctx + r.mu.Unlock() - // Otherwise we schedule a flush. - w.interactionsFlushTimer = time.AfterFunc(w.busFlushInterval, func() { - w.interactionsMu.Lock() - w.flushInteractions() - w.interactionsMu.Unlock() - }) + // flush all interactions + r.flush() + + // log if we weren't able to flush them + r.mu.Lock() + if len(r.hostScans) > 0 { + r.logger.Errorw(fmt.Sprintf("failed to record %d host scans on worker shutdown", len(r.hostScans))) + } + if len(r.priceTableUpdates) > 0 { + r.logger.Errorw(fmt.Sprintf("failed to record %d price table updates on worker shutdown", len(r.priceTableUpdates))) + } + r.mu.Unlock() } -// flushInteractions flushes the worker's interaction buffer to the bus. -func (w *worker) flushInteractions() { - if len(w.interactionsScans) > 0 { - 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 +func (r *hostInteractionRecorder) flush() { + r.mu.Lock() + defer r.mu.Unlock() + + // NOTE: don't bother flushing if the context is cancelled, we can safely + // ignore the buffered scans and price tables since we'll flush on shutdown + // and log in case we weren't able to flush all interactions to the bus + select { + case <-r.flushCtx.Done(): + r.flushTimer = nil + return + default: + } + + if len(r.hostScans) > 0 { + if err := r.bus.RecordHostScans(r.flushCtx, r.hostScans); err != nil { + r.logger.Errorw(fmt.Sprintf("failed to record scans: %v", err)) + } else if err == nil { + r.hostScans = nil } } - if len(w.interactionsPriceTableUpdates) > 0 { - 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 + if len(r.priceTableUpdates) > 0 { + if err := r.bus.RecordPriceTables(r.flushCtx, r.priceTableUpdates); err != nil { + r.logger.Errorw(fmt.Sprintf("failed to record price table updates: %v", err)) + } else if err == nil { + r.priceTableUpdates = nil } } - w.interactionsFlushTimer = nil + r.flushTimer = nil +} + +func (r *hostInteractionRecorder) tryFlushInteractionsBuffer() { + if r.flushTimer == nil { + r.flushTimer = time.AfterFunc(r.flushInterval, r.flush) + } } func isSuccessfulInteraction(err error) bool { diff --git a/worker/spending.go b/worker/spending.go index ec09e803a..87d2ec17d 100644 --- a/worker/spending.go +++ b/worker/spending.go @@ -12,43 +12,51 @@ import ( ) type ( - // A ContractSpendingRecorder records the spending of a contract. ContractSpendingRecorder interface { Record(rev types.FileContractRevision, cs api.ContractSpending) + Stop(context.Context) } contractSpendingRecorder struct { - bus Bus flushInterval time.Duration - shutdownCtx context.Context - logger *zap.SugaredLogger - mu sync.Mutex - contractSpendings map[types.FileContractID]api.ContractSpendingRecord - contractSpendingsFlushTimer *time.Timer + bus Bus + logger *zap.SugaredLogger + + mu sync.Mutex + contractSpendings map[types.FileContractID]api.ContractSpendingRecord + + flushCtx context.Context + flushTimer *time.Timer } ) -func (w *worker) initContractSpendingRecorder() { +var ( + _ ContractSpendingRecorder = (*contractSpendingRecorder)(nil) +) + +func (w *worker) initContractSpendingRecorder(flushInterval time.Duration) { if w.contractSpendingRecorder != nil { - panic("contractSpendingRecorder already initialized") // developer error + panic("ContractSpendingRecorder already initialized") // developer error } w.contractSpendingRecorder = &contractSpendingRecorder{ - bus: w.bus, + bus: w.bus, + logger: w.logger, + + flushCtx: w.shutdownCtx, + flushInterval: flushInterval, + contractSpendings: make(map[types.FileContractID]api.ContractSpendingRecord), - flushInterval: w.busFlushInterval, - shutdownCtx: w.shutdownCtx, - logger: w.logger, } } -// Record sends contract spending records to the bus. -func (sr *contractSpendingRecorder) Record(rev types.FileContractRevision, cs api.ContractSpending) { - sr.mu.Lock() - defer sr.mu.Unlock() +// Record stores the given contract spending record until it gets flushed to the bus. +func (r *contractSpendingRecorder) Record(rev types.FileContractRevision, cs api.ContractSpending) { + r.mu.Lock() + defer r.mu.Unlock() - // Update buffer. - csr, found := sr.contractSpendings[rev.ParentID] + // record the spending + csr, found := r.contractSpendings[rev.ParentID] if !found { csr = api.ContractSpendingRecord{ ContractID: rev.ParentID, @@ -61,41 +69,59 @@ func (sr *contractSpendingRecorder) Record(rev types.FileContractRevision, cs ap csr.ValidRenterPayout = rev.ValidRenterPayout() csr.MissedHostPayout = rev.MissedHostPayout() } - sr.contractSpendings[rev.ParentID] = csr + r.contractSpendings[rev.ParentID] = csr - // If a thread was scheduled to flush the buffer we are done. - if sr.contractSpendingsFlushTimer != nil { - return + // schedule flush + if r.flushTimer == nil { + r.flushTimer = time.AfterFunc(r.flushInterval, r.flush) + } +} + +// Stop stops the flush timer and flushes one last time. +func (r *contractSpendingRecorder) Stop(ctx context.Context) { + // stop the flush timer + r.mu.Lock() + if r.flushTimer != nil { + r.flushTimer.Stop() } - // Otherwise we schedule a flush. - sr.contractSpendingsFlushTimer = time.AfterFunc(sr.flushInterval, func() { - sr.mu.Lock() - sr.flush() - sr.mu.Unlock() - }) + r.flushCtx = ctx + r.mu.Unlock() + + // flush all interactions + r.flush() + + // log if we weren't able to flush them + r.mu.Lock() + if len(r.contractSpendings) > 0 { + r.logger.Errorw(fmt.Sprintf("failed to record %d contract spendings on worker shutdown", len(r.contractSpendings))) + } + r.mu.Unlock() } -func (sr *contractSpendingRecorder) flush() { - if len(sr.contractSpendings) > 0 { - records := make([]api.ContractSpendingRecord, 0, len(sr.contractSpendings)) - for _, cs := range sr.contractSpendings { +func (r *contractSpendingRecorder) flush() { + r.mu.Lock() + defer r.mu.Unlock() + + // NOTE: don't bother flushing if the context is cancelled, we can safely + // ignore the buffered records since we'll flush on shutdown and log in case + // we weren't able to flush all spendings o the bus + select { + case <-r.flushCtx.Done(): + r.flushTimer = nil + return + default: + } + + if len(r.contractSpendings) > 0 { + records := make([]api.ContractSpendingRecord, 0, len(r.contractSpendings)) + for _, cs := range r.contractSpendings { records = append(records, cs) } - if err := sr.bus.RecordContractSpending(sr.shutdownCtx, records); err != nil { - sr.logger.Errorw(fmt.Sprintf("failed to record contract spending: %v", err)) + if err := r.bus.RecordContractSpending(r.flushCtx, records); err != nil { + r.logger.Errorw(fmt.Sprintf("failed to record contract spending: %v", err)) } else { - sr.contractSpendings = make(map[types.FileContractID]api.ContractSpendingRecord) + r.contractSpendings = make(map[types.FileContractID]api.ContractSpendingRecord) } } - sr.contractSpendingsFlushTimer = nil -} - -// Stop stops the flush timer. -func (sr *contractSpendingRecorder) Stop() { - sr.mu.Lock() - defer sr.mu.Unlock() - if sr.contractSpendingsFlushTimer != nil { - sr.contractSpendingsFlushTimer.Stop() - sr.flush() - } + r.flushTimer = nil } diff --git a/worker/upload.go b/worker/upload.go index aebd341a4..52b28fc2d 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -30,7 +30,6 @@ const ( var ( errNoCandidateUploader = errors.New("no candidate uploader found") errNotEnoughContracts = errors.New("not enough contracts to support requested redundancy") - errWorkerShutDown = errors.New("worker was shut down") errUploadInterrupted = errors.New("upload was interrupted") ) @@ -573,7 +572,7 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a for len(responses) < numSlabs { select { case <-mgr.shutdownCtx.Done(): - return false, "", errWorkerShutDown + return false, "", ErrShuttingDown case <-ctx.Done(): return false, "", errUploadInterrupted case numSlabs = <-numSlabsChan: diff --git a/worker/worker.go b/worker/worker.go index f8ce65d2b..013df1a3e 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -51,6 +51,10 @@ const ( lockingPriorityBackgroundUpload = 5 ) +var ( + ErrShuttingDown = errors.New("worker is shutting down") +) + // re-export the client type Client struct { *client.Client @@ -189,26 +193,21 @@ type worker struct { downloadManager *downloadManager uploadManager *uploadManager - accounts *accounts - priceTables *priceTables - - busFlushInterval time.Duration + accounts *accounts + priceTables *priceTables + transportPoolV3 *transportPoolV3 uploadsMu sync.Mutex uploadingPackedSlabs map[string]bool - interactionsMu sync.Mutex - interactionsScans []hostdb.HostScan - interactionsPriceTableUpdates []hostdb.PriceTableUpdate - interactionsFlushTimer *time.Timer - - contractSpendingRecorder *contractSpendingRecorder + hostInteractionRecorder HostInteractionRecorder + contractSpendingRecorder ContractSpendingRecorder contractLockingDuration time.Duration - transportPoolV3 *transportPoolV3 - logger *zap.SugaredLogger shutdownCtx context.Context shutdownCtxCancel context.CancelFunc + + logger *zap.SugaredLogger } func (w *worker) withRevision(ctx context.Context, fetchTimeout time.Duration, fcid types.FileContractID, hk types.PublicKey, siamuxAddr string, lockPriority int, blockHeight uint64, fn func(rev types.FileContractRevision) error) error { @@ -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(w.shutdownCtx, time.Minute) if err := w.alerts.RegisterAlert(ctx, a); err != nil { - w.logger.Error("failed to register alert", err) + w.logger.Errorf("failed to register alert, err: %v", err) } cancel() } @@ -343,7 +342,7 @@ func (w *worker) rhpPriceTableHandler(jc jape.Context) { var err error var hpt hostdb.HostPriceTable defer func() { - InteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ + HostInteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ HostKey: rptr.HostKey, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), @@ -927,11 +926,11 @@ func (w *worker) objectsHandlerGET(jc jape.Context) { downloadFn := func(wr io.Writer, offset, length int64) (err error) { ctx = WithGougingChecker(ctx, w.bus, gp) err = w.downloadManager.DownloadObject(ctx, wr, res.Object.Object, uint64(offset), uint64(length), contracts) - if err != nil && !(errors.Is(err, errDownloadManagerStopped) || - errors.Is(err, errNotEnoughContracts) || - errors.Is(err, context.Canceled)) { + if err != nil { w.logger.Error(err) - w.registerAlert(newDownloadFailedAlert(bucket, path, prefix, marker, offset, length, int64(len(contracts)), err)) + if !errors.Is(err, ErrShuttingDown) { + w.registerAlert(newDownloadFailedAlert(bucket, path, prefix, marker, offset, length, int64(len(contracts)), err)) + } } return } @@ -1043,11 +1042,11 @@ func (w *worker) objectsHandlerPUT(jc jape.Context) { params := defaultParameters(bucket, path) eTag, err := w.upload(ctx, jc.Request.Body, contracts, params, opts...) if err := jc.Check("couldn't upload object", err); err != nil { - if err != nil && !(errors.Is(err, errWorkerShutDown) || - errors.Is(err, errNotEnoughContracts) || - errors.Is(err, context.Canceled)) { + if err != nil { w.logger.Error(err) - w.registerAlert(newUploadFailedAlert(bucket, path, up.ContractSet, mimeType, rs.MinShards, rs.TotalShards, len(contracts), up.UploadPacking, false, err)) + if !errors.Is(err, ErrShuttingDown) { + w.registerAlert(newUploadFailedAlert(bucket, path, up.ContractSet, mimeType, rs.MinShards, rs.TotalShards, len(contracts), up.UploadPacking, false, err)) + } } return } @@ -1182,11 +1181,11 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { params := multipartParameters(bucket, path, uploadID, partNumber) eTag, err := w.upload(ctx, jc.Request.Body, contracts, params, opts...) if jc.Check("couldn't upload object", err) != nil { - if err != nil && !(errors.Is(err, errWorkerShutDown) || - errors.Is(err, errNotEnoughContracts) || - errors.Is(err, context.Canceled)) { + if err != nil { w.logger.Error(err) - w.registerAlert(newUploadFailedAlert(bucket, path, up.ContractSet, "", rs.MinShards, rs.TotalShards, len(contracts), up.UploadPacking, true, err)) + if !errors.Is(err, ErrShuttingDown) { + w.registerAlert(newUploadFailedAlert(bucket, path, up.ContractSet, "", rs.MinShards, rs.TotalShards, len(contracts), up.UploadPacking, true, err)) + } } return } @@ -1307,7 +1306,6 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush id: id, bus: b, masterKey: masterKey, - busFlushInterval: busFlushInterval, logger: l.Sugar().Named("worker").Named(id), startTime: time.Now(), uploadingPackedSlabs: make(map[string]bool), @@ -1315,17 +1313,20 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush shutdownCtxCancel: shutdownCtxCancel, } w.initAccounts(b) - w.initContractSpendingRecorder() - w.initDownloadManager(downloadMaxMemory, downloadMaxOverdrive, downloadOverdriveTimeout, l.Sugar().Named("downloadmanager")) w.initPriceTables() w.initTransportPool() + + w.initDownloadManager(downloadMaxMemory, downloadMaxOverdrive, downloadOverdriveTimeout, l.Sugar().Named("downloadmanager")) w.initUploadManager(uploadMaxMemory, uploadMaxOverdrive, uploadOverdriveTimeout, l.Sugar().Named("uploadmanager")) + + w.initContractSpendingRecorder(busFlushInterval) + w.initHostInteractionRecorder(busFlushInterval) return w, nil } // Handler returns an HTTP handler that serves the worker API. func (w *worker) Handler() http.Handler { - return jape.Mux(interactionMiddleware(w, map[string]jape.Handler{ + return jape.Mux(interactionMiddleware(w.hostInteractionRecorder, map[string]jape.Handler{ "GET /account/:hostkey": w.accountHandlerGET, "GET /id": w.idHandlerGET, @@ -1357,32 +1358,24 @@ func (w *worker) Handler() http.Handler { } // Shutdown shuts down the worker. -func (w *worker) Shutdown(_ context.Context) error { - w.interactionsMu.Lock() - if w.interactionsFlushTimer != nil { - w.interactionsFlushTimer.Stop() - w.flushInteractions() - } - w.interactionsMu.Unlock() - - // Cancel shutdown context. +func (w *worker) Shutdown(ctx context.Context) error { + // cancel shutdown context w.shutdownCtxCancel() - // Stop contract spending recorder. - w.contractSpendingRecorder.Stop() - - // Stop the downloader. + // stop uploads and downloads w.downloadManager.Stop() - - // Stop the uploader. w.uploadManager.Stop() + + // stop recorders + w.hostInteractionRecorder.Stop(ctx) + w.contractSpendingRecorder.Stop(ctx) return nil } func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP string) (settings rhpv2.HostSettings, pt rhpv3.HostPriceTable, elapsed time.Duration, err error) { // record host scan defer func() { - InteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ + HostInteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ HostKey: hostKey, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), From ac05a6e6be38f3ea2820da2cedbe836976303b81 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 23 Jan 2024 14:49:33 +0100 Subject: [PATCH 009/144] worker: use host blockheight --- worker/host.go | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/worker/host.go b/worker/host.go index 320645b8b..2fa7fff1e 100644 --- a/worker/host.go +++ b/worker/host.go @@ -209,11 +209,7 @@ func (h *host) FetchPriceTable(ctx context.Context, rev *types.FileContractRevis } // pay by account - cs, err := h.bus.ConsensusState(ctx) - if err != nil { - return hostdb.HostPriceTable{}, err - } - return fetchPT(h.preparePriceTableAccountPayment(cs.BlockHeight)) + return fetchPT(h.preparePriceTableAccountPayment()) } func (h *host) FundAccount(ctx context.Context, balance types.Currency, rev *types.FileContractRevision) error { @@ -296,10 +292,10 @@ func (h *host) SyncAccount(ctx context.Context, rev *types.FileContractRevision) // // NOTE: This is the preferred way of paying for a price table since it is // faster and doesn't require locking a contract. -func (h *host) preparePriceTableAccountPayment(bh uint64) PriceTablePaymentFunc { +func (h *host) preparePriceTableAccountPayment() PriceTablePaymentFunc { return func(pt rhpv3.HostPriceTable) (rhpv3.PaymentMethod, error) { account := rhpv3.Account(h.accountKey.PublicKey()) - payment := rhpv3.PayByEphemeralAccount(account, pt.UpdatePriceTableCost, bh+defaultWithdrawalExpiryBlocks, h.accountKey) + payment := rhpv3.PayByEphemeralAccount(account, pt.UpdatePriceTableCost, pt.HostBlockHeight+defaultWithdrawalExpiryBlocks, h.accountKey) return &payment, nil } } From 7757409fa2e4c709901e71e8f9fba81eaf7d2384 Mon Sep 17 00:00:00 2001 From: Nate Maninger Date: Tue, 23 Jan 2024 11:00:40 -0800 Subject: [PATCH 010/144] cmd: add mysql settings to config prompt --- cmd/renterd/config.go | 50 ++++++++++++++++++++++++++++++++++++++----- 1 file changed, 45 insertions(+), 5 deletions(-) diff --git a/cmd/renterd/config.go b/cmd/renterd/config.go index e57e0457a..391d77ea3 100644 --- a/cmd/renterd/config.go +++ b/cmd/renterd/config.go @@ -104,12 +104,16 @@ func stdoutError(msg string) { } } -func setListenAddress(context string, value *string) { +func setListenAddress(context string, value *string, allowEmpty bool) { // will continue to prompt until a valid value is entered for { input := readInput(fmt.Sprintf("%s (currently %q)", context, *value)) if input == "" { - return + if allowEmpty { + return + } + stdoutError(fmt.Sprintf("Invalid %s %q: must not be empty", context, input)) + continue } host, port, err := net.SplitHostPort(input) @@ -221,13 +225,49 @@ func setAdvancedConfig() { fmt.Println("The HTTP address is used to serve the renter's admin API.") fmt.Println("The admin API is used to configure the renter.") fmt.Println("It should not be exposed to the public internet without setting up a reverse proxy.") - setListenAddress("HTTP Address", &cfg.HTTP.Address) + setListenAddress("HTTP Address", &cfg.HTTP.Address, true) // gateway address fmt.Println("") fmt.Println("The gateway address is used to exchange blocks with other nodes in the Sia network") fmt.Println("It should be exposed publicly to improve the renter's connectivity.") - setListenAddress("Gateway Address", &cfg.Bus.GatewayAddr) + setListenAddress("Gateway Address", &cfg.Bus.GatewayAddr, true) + + // database + fmt.Println("") + fmt.Println("The database is used to store the renter's metadata.") + fmt.Println("The embedded SQLite database is recommended for small (< 50TB), single-user setups. Choose this for the easiest setup.") + fmt.Println("MySQL database is recommended for larger (> 50TB) or multi-user setups. MySQL requires a separate MySQL server to connect to.") + setStoreConfig() +} + +func setStoreConfig() { + store := promptQuestion("Which data store would you like to use?", []string{"mysql", "sqlite"}) + switch store { + case "mysql": + fmt.Println("") + fmt.Println("The MySQL database is used to store the renter metadata.") + fmt.Println("You will need to set up a MySQL server to connect to.") + fmt.Println("") + fmt.Println("You will also need to create two database") + fmt.Println(" - The first database will be used to store the object metadata.") + fmt.Println(" - The second database will be used to store metrics.") + fmt.Println("") + setListenAddress("MySQL address", &cfg.Database.MySQL.URI, false) + + cfg.Database.MySQL.User = readInput("MySQL username") + cfg.Database.MySQL.Password = readPasswordInput("MySQL password") + objectDB := readInput("Object database name (default: renterd)") + if objectDB != "" { + cfg.Database.MySQL.Database = objectDB + } + metricsDB := readInput("Metrics database name (default: renterd_metrics)") + if metricsDB != "" { + cfg.Database.MySQL.MetricsDatabase = metricsDB + } + default: + return + } } func setS3Config() { @@ -244,7 +284,7 @@ func setS3Config() { fmt.Println("The S3 address is used to serve the renter's S3 API.") fmt.Println("The S3 API provides an S3-compatible gateway for uploading data to Sia.") fmt.Println("It should not be exposed to the public internet without setting up a reverse proxy.") - setListenAddress("S3 Address", &cfg.S3.Address) + setListenAddress("S3 Address", &cfg.S3.Address, true) // s3 access key if len(cfg.S3.KeypairsV4) != 0 { From d0b2c3dfb71a71e792ad6ef38f50ab12d8b10248 Mon Sep 17 00:00:00 2001 From: ChrisSchinnerl Date: Thu, 25 Jan 2024 00:09:00 +0000 Subject: [PATCH 011/144] ui: v0.44.0 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index c85df7bca..991c15f4e 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.43.0 + go.sia.tech/web/renterd v0.44.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 6487f7725..850032a69 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.43.0 h1:AvCm+J76UeHxwzdfyTgzHozI6tPz4BWP5D833oJiU1M= -go.sia.tech/web/renterd v0.43.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= +go.sia.tech/web/renterd v0.44.0 h1:yKu1Kq/6ssV9Vbv4oa+sn2Pc2TNyfcrv/mRPNOuYuB0= +go.sia.tech/web/renterd v0.44.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 e9bb01241fb1eaf663f9baf9681a3bd350d8365e Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 30 Jan 2024 09:14:00 -0400 Subject: [PATCH 012/144] ci: add dispatch.yml --- .github/workflows/dispatch.yml | 26 ++++++++++++++++++++++++++ 1 file changed, 26 insertions(+) create mode 100644 .github/workflows/dispatch.yml diff --git a/.github/workflows/dispatch.yml b/.github/workflows/dispatch.yml new file mode 100644 index 000000000..74bf4f7df --- /dev/null +++ b/.github/workflows/dispatch.yml @@ -0,0 +1,26 @@ +name: Trigger Package Build + +on: + push: + tags: + - 'v[0-9]+.[0-9]+.[0-9]+' + - 'v[0-9]+.[0-9]+.[0-9]+-rc[0-9]+' + +jobs: + dispatch: + strategy: + matrix: + repo: ['siafoundation/homebrew-sia', 'siafoundation/linux'] + runs-on: ubuntu-latest + steps: + - name: Extract Tag Name + id: get_tag + run: echo "::set-output name=tag_name::${GITHUB_REF#refs/tags/}" + + - name: Repository Dispatch + uses: peter-evans/repository-dispatch@v2 + with: + token: ${{ secrets.PAT_REPOSITORY_DISPATCH }} + repository: ${{ matrix.repo }} + event-type: release-tagged + client-payload: '{"tag": "${{ steps.get_tag.outputs.tag_name }}", "project": "renterd"}' \ No newline at end of file From 5a51ff0e8da12052ff384f3f68abb809496dd39f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 30 Jan 2024 12:23:29 -0400 Subject: [PATCH 013/144] ci: make dispatch a step in publish.yml --- .github/workflows/dispatch.yml | 26 -------------------------- .github/workflows/publish.yml | 20 +++++++++++++++++++- 2 files changed, 19 insertions(+), 27 deletions(-) delete mode 100644 .github/workflows/dispatch.yml diff --git a/.github/workflows/dispatch.yml b/.github/workflows/dispatch.yml deleted file mode 100644 index 74bf4f7df..000000000 --- a/.github/workflows/dispatch.yml +++ /dev/null @@ -1,26 +0,0 @@ -name: Trigger Package Build - -on: - push: - tags: - - 'v[0-9]+.[0-9]+.[0-9]+' - - 'v[0-9]+.[0-9]+.[0-9]+-rc[0-9]+' - -jobs: - dispatch: - strategy: - matrix: - repo: ['siafoundation/homebrew-sia', 'siafoundation/linux'] - runs-on: ubuntu-latest - steps: - - name: Extract Tag Name - id: get_tag - run: echo "::set-output name=tag_name::${GITHUB_REF#refs/tags/}" - - - name: Repository Dispatch - uses: peter-evans/repository-dispatch@v2 - with: - token: ${{ secrets.PAT_REPOSITORY_DISPATCH }} - repository: ${{ matrix.repo }} - event-type: release-tagged - client-payload: '{"tag": "${{ steps.get_tag.outputs.tag_name }}", "project": "renterd"}' \ No newline at end of file diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index a02becd75..d7d7fa1d2 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -252,4 +252,22 @@ jobs: - uses: actions/upload-artifact@v3 with: name: renterd - path: release/ \ No newline at end of file + path: release/ + dispatch: + if: startsWith(github.ref, 'refs/tags/v') + strategy: + matrix: + repo: ['siafoundation/homebrew-sia', 'siafoundation/linux'] + runs-on: ubuntu-latest + steps: + - name: Extract Tag Name + id: get_tag + run: echo "::set-output name=tag_name::${GITHUB_REF#refs/tags/}" + + - name: Repository Dispatch + uses: peter-evans/repository-dispatch@v3 + with: + token: ${{ secrets.PAT_REPOSITORY_DISPATCH }} + repository: ${{ matrix.repo }} + event-type: release-tagged + client-payload: '{"tag": "${{ steps.get_tag.outputs.tag_name }}", "project": "renterd"}' \ No newline at end of file From 1f2407bbab6bdc8e5522a673cd8291a63080c46c Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 30 Jan 2024 13:44:25 -0400 Subject: [PATCH 014/144] ci: include workflow run id in the dispatch --- .github/workflows/publish.yml | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index d7d7fa1d2..1d6cf0226 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -270,4 +270,9 @@ jobs: token: ${{ secrets.PAT_REPOSITORY_DISPATCH }} repository: ${{ matrix.repo }} event-type: release-tagged - client-payload: '{"tag": "${{ steps.get_tag.outputs.tag_name }}", "project": "renterd"}' \ No newline at end of file + client-payload: > + { + "tag": "${{ steps.get_tag.outputs.tag_name }}", + "project": "renterd", + "workflow_run_id": "${{ github.run_id }}" + } \ No newline at end of file From 54615f7f82eddea84098ddf91f42afa0c07d4232 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 5 Feb 2024 10:29:22 +0100 Subject: [PATCH 015/144] ci: update client-payload --- .github/workflows/publish.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index 1d6cf0226..f02e1134a 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -272,7 +272,8 @@ jobs: event-type: release-tagged client-payload: > { + "description": "Renterd: The Next-Gen Sia Renter", "tag": "${{ steps.get_tag.outputs.tag_name }}", "project": "renterd", - "workflow_run_id": "${{ github.run_id }}" + "workflow_id": "${{ github.run_id }}" } \ No newline at end of file From 1469b6655ec276497865850d1c707372cc8881f9 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 5 Feb 2024 11:14:42 +0100 Subject: [PATCH 016/144] worker: fix build --- worker/upload.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/upload.go b/worker/upload.go index 1f66fd642..bbc0aff5f 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -389,7 +389,7 @@ func (mgr *uploadManager) Stop() { mgr.mu.Lock() defer mgr.mu.Unlock() for _, u := range mgr.uploaders { - u.Stop(errWorkerShutDown) + u.Stop(ErrShuttingDown) } } From c06dea13bfe86360808944f07798cf8147fbad33 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 5 Feb 2024 13:47:31 +0100 Subject: [PATCH 017/144] autopilot: adjust min score if fewer candidates than needed are available --- autopilot/contractor.go | 17 ++++++++++++++++- 1 file changed, 16 insertions(+), 1 deletion(-) diff --git a/autopilot/contractor.go b/autopilot/contractor.go index b4a0c1cc3..2af724f4d 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -1197,11 +1197,17 @@ func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredH return math.SmallestNonzeroFloat64, nil } + // determine the number of random hosts we fetch per iteration when + // calculating the min score - it contains a constant factor in case the + // number of contracts is very low and a linear factor to make sure the + // number is relative to the number of contracts we want to form + randSetSize := 2*int(numContracts) + 50 + // do multiple rounds to select the lowest score var lowestScores []float64 for r := 0; r < 5; r++ { lowestScore := math.MaxFloat64 - for _, host := range scoredHosts(candidates).randSelectByScore(int(numContracts) + 50) { // buffer + for _, host := range scoredHosts(candidates).randSelectByScore(randSetSize) { if host.score < lowestScore { lowestScore = host.score } @@ -1216,8 +1222,17 @@ func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredH } minScore := lowestScore / minAllowedScoreLeeway + // make sure the min score allows for 'numContracts' contracts to be formed + if len(candidates) < int(numContracts) { + return math.SmallestNonzeroFloat64, nil + } else if cutoff := candidates[numContracts-1].score; minScore < cutoff { + minScore = cutoff + } + c.logger.Infow("finished computing minScore", + "candidates", len(candidates), "minScore", minScore, + "numContracts", numContracts, "lowestScore", lowestScore) return minScore, nil } From ab95e07317ad33985225b79498af38aa0ae1f611 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 5 Feb 2024 14:15:09 +0100 Subject: [PATCH 018/144] autopilot: add TestCalculateMinScore --- autopilot/contractor.go | 21 ++++++++++---------- autopilot/contractor_test.go | 38 ++++++++++++++++++++++++++++++++++++ 2 files changed, 49 insertions(+), 10 deletions(-) create mode 100644 autopilot/contractor_test.go diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 2af724f4d..adad5d1b7 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -246,10 +246,7 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( // min score to pass checks var minScore float64 if len(hosts) > 0 { - minScore, err = c.calculateMinScore(ctx, candidates, state.cfg.Contracts.Amount) - if err != nil { - return false, fmt.Errorf("failed to determine min score for contract check: %w", err) - } + minScore = c.calculateMinScore(ctx, candidates, state.cfg.Contracts.Amount) } else { c.logger.Warn("could not calculate min score, no hosts found") } @@ -1190,11 +1187,11 @@ func (c *contractor) renewFundingEstimate(ctx context.Context, ci contractInfo, return cappedEstimatedCost, nil } -func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredHost, numContracts uint64) (float64, error) { +func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredHost, numContracts uint64) float64 { // return early if there's no hosts if len(candidates) == 0 { c.logger.Warn("min host score is set to the smallest non-zero float because there are no candidate hosts") - return math.SmallestNonzeroFloat64, nil + return math.SmallestNonzeroFloat64 } // determine the number of random hosts we fetch per iteration when @@ -1216,16 +1213,20 @@ func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredH } // compute the min score + var lowestScore float64 lowestScore, err := stats.Float64Data(lowestScores).Median() if err != nil { - return 0, err + panic("never fails since len(candidates) > 0 so len(lowestScores) > 0 as well") } minScore := lowestScore / minAllowedScoreLeeway // make sure the min score allows for 'numContracts' contracts to be formed + sort.Slice(candidates, func(i, j int) bool { + return candidates[i].score > candidates[j].score + }) if len(candidates) < int(numContracts) { - return math.SmallestNonzeroFloat64, nil - } else if cutoff := candidates[numContracts-1].score; minScore < cutoff { + return math.SmallestNonzeroFloat64 + } else if cutoff := candidates[numContracts-1].score; minScore > cutoff { minScore = cutoff } @@ -1234,7 +1235,7 @@ func (c *contractor) calculateMinScore(ctx context.Context, candidates []scoredH "minScore", minScore, "numContracts", numContracts, "lowestScore", lowestScore) - return minScore, nil + return minScore } func (c *contractor) candidateHosts(ctx context.Context, hosts []hostdb.Host, usedHosts map[types.PublicKey]struct{}, storedData map[types.PublicKey]uint64, minScore float64) ([]scoredHost, unusableHostResult, error) { diff --git a/autopilot/contractor_test.go b/autopilot/contractor_test.go new file mode 100644 index 000000000..a0f63425b --- /dev/null +++ b/autopilot/contractor_test.go @@ -0,0 +1,38 @@ +package autopilot + +import ( + "context" + "math" + "testing" + + "go.uber.org/zap" +) + +func TestCalculateMinScore(t *testing.T) { + c := &contractor{ + logger: zap.NewNop().Sugar(), + } + + var candidates []scoredHost + for i := 0; i < 250; i++ { + candidates = append(candidates, scoredHost{score: float64(i + 1)}) + } + + // Test with 100 hosts which makes for a random set size of 250 + minScore := c.calculateMinScore(context.Background(), candidates, 100) + if minScore != 0.002 { + t.Fatalf("expected minScore to be 0.002 but was %v", minScore) + } + + // Test with 0 hosts + minScore = c.calculateMinScore(context.Background(), []scoredHost{}, 100) + if minScore != math.SmallestNonzeroFloat64 { + t.Fatalf("expected minScore to be math.SmallestNonzeroFLoat64 but was %v", minScore) + } + + // Test with 300 hosts which is 50 more than we have + minScore = c.calculateMinScore(context.Background(), candidates, 300) + if minScore != math.SmallestNonzeroFloat64 { + t.Fatalf("expected minScore to be math.SmallestNonzeroFLoat64 but was %v", minScore) + } +} From d83e5a0e12eadaf75f9d76da096273e20ec93efe Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Tue, 6 Feb 2024 14:34:44 +0100 Subject: [PATCH 019/144] Create dependabot.yml --- .github/dependabot.yml | 11 +++++++++++ 1 file changed, 11 insertions(+) create mode 100644 .github/dependabot.yml diff --git a/.github/dependabot.yml b/.github/dependabot.yml new file mode 100644 index 000000000..cd885540d --- /dev/null +++ b/.github/dependabot.yml @@ -0,0 +1,11 @@ +# To get started with Dependabot version updates, you'll need to specify which +# package ecosystems to update and where the package manifests are located. +# Please see the documentation for all configuration options: +# https://docs.github.com/code-security/dependabot/dependabot-version-updates/configuration-options-for-the-dependabot.yml-file + +version: 2 +updates: + - package-ecosystem: "gomod" # See documentation for possible values + directory: "/" # Location of package manifests + schedule: + interval: "weekly" From 1005ab77893c6c8de591b8f4c4147f9ec5e051d1 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 13:35:25 +0000 Subject: [PATCH 020/144] build(deps): bump gorm.io/gorm from 1.25.5 to 1.25.7 Bumps [gorm.io/gorm](https://github.com/go-gorm/gorm) from 1.25.5 to 1.25.7. - [Release notes](https://github.com/go-gorm/gorm/releases) - [Commits](https://github.com/go-gorm/gorm/compare/v1.25.5...v1.25.7) --- updated-dependencies: - dependency-name: gorm.io/gorm dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 991c15f4e..bba856c23 100644 --- a/go.mod +++ b/go.mod @@ -24,7 +24,7 @@ require ( gopkg.in/yaml.v3 v3.0.1 gorm.io/driver/mysql v1.5.2 gorm.io/driver/sqlite v1.5.4 - gorm.io/gorm v1.25.5 + gorm.io/gorm v1.25.7 lukechampine.com/frand v1.4.2 ) diff --git a/go.sum b/go.sum index 850032a69..431b697f4 100644 --- a/go.sum +++ b/go.sum @@ -382,8 +382,8 @@ gorm.io/driver/mysql v1.5.2/go.mod h1:pQLhh1Ut/WUAySdTHwBpBv6+JKcj+ua4ZFx1QQTBzb gorm.io/driver/sqlite v1.5.4 h1:IqXwXi8M/ZlPzH/947tn5uik3aYQslP9BVveoax0nV0= gorm.io/driver/sqlite v1.5.4/go.mod h1:qxAuCol+2r6PannQDpOP1FP6ag3mKi4esLnB/jHed+4= gorm.io/gorm v1.25.2-0.20230530020048-26663ab9bf55/go.mod h1:L4uxeKpfBml98NYqVqwAdmV1a2nBtAec/cf3fpucW/k= -gorm.io/gorm v1.25.5 h1:zR9lOiiYf09VNh5Q1gphfyia1JpiClIWG9hQaxB/mls= -gorm.io/gorm v1.25.5/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= +gorm.io/gorm v1.25.7 h1:VsD6acwRjz2zFxGO50gPO6AkNs7KKnvfzUjHQhZDz/A= +gorm.io/gorm v1.25.7/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= lukechampine.com/frand v1.4.2 h1:RzFIpOvkMXuPMBb9maa4ND4wjBn71E1Jpf8BzJHMaVw= lukechampine.com/frand v1.4.2/go.mod h1:4S/TM2ZgrKejMcKMbeLjISpJMO+/eZ1zu3vYX9dtj3s= From c509772ed548ef61fc185ba8b9ff03461020f4cb Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 13:35:30 +0000 Subject: [PATCH 021/144] build(deps): bump golang.org/x/crypto from 0.16.0 to 0.18.0 Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.16.0 to 0.18.0. - [Commits](https://github.com/golang/crypto/compare/v0.16.0...v0.18.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- go.mod | 6 +++--- go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/go.mod b/go.mod index 991c15f4e..00420449a 100644 --- a/go.mod +++ b/go.mod @@ -19,8 +19,8 @@ require ( go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca go.sia.tech/web/renterd v0.44.0 go.uber.org/zap v1.26.0 - golang.org/x/crypto v0.16.0 - golang.org/x/term v0.15.0 + golang.org/x/crypto v0.18.0 + golang.org/x/term v0.16.0 gopkg.in/yaml.v3 v3.0.1 gorm.io/driver/mysql v1.5.2 gorm.io/driver/sqlite v1.5.4 @@ -75,7 +75,7 @@ require ( 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 - golang.org/x/sys v0.15.0 // indirect + golang.org/x/sys v0.16.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.16.1 // indirect diff --git a/go.sum b/go.sum index 850032a69..c3efc2457 100644 --- a/go.sum +++ b/go.sum @@ -266,8 +266,8 @@ golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220507011949-2cf3adece122/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.16.0 h1:mMMrFzRSCF0GvB7Ne27XVtVAaXLrPmgPC7/v0tkwHaY= -golang.org/x/crypto v0.16.0/go.mod h1:gCAAfMLgwOJRpTjQ2zCCt2OcSfYMTeZVSRtQlPC7Nq4= +golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= +golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= @@ -320,16 +320,16 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.15.0 h1:h48lPFYpsTvQJZF4EKyI4aLHaev3CxivZmv7yZig9pc= -golang.org/x/sys v0.15.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= +golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210421210424-b80969c67360/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= -golang.org/x/term v0.15.0 h1:y/Oo/a/q3IXu26lQgl04j/gjuBDOBlx7X6Om1j2CPW4= -golang.org/x/term v0.15.0/go.mod h1:BDl952bC7+uMoWR75FIrCDx79TPU9oHkTZ9yRbYOrX0= +golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= +golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= From 1c113b25255f141c2ea7dbfef7c3335330269003 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 13:35:35 +0000 Subject: [PATCH 022/144] build(deps): bump github.com/klauspost/reedsolomon from 1.12.0 to 1.12.1 Bumps [github.com/klauspost/reedsolomon](https://github.com/klauspost/reedsolomon) from 1.12.0 to 1.12.1. - [Release notes](https://github.com/klauspost/reedsolomon/releases) - [Commits](https://github.com/klauspost/reedsolomon/compare/v1.12.0...v1.12.1) --- updated-dependencies: - dependency-name: github.com/klauspost/reedsolomon dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 991c15f4e..98db1a762 100644 --- a/go.mod +++ b/go.mod @@ -7,7 +7,7 @@ require ( github.com/go-gormigrate/gormigrate/v2 v2.1.1 github.com/google/go-cmp v0.6.0 github.com/gotd/contrib v0.19.0 - github.com/klauspost/reedsolomon v1.12.0 + github.com/klauspost/reedsolomon v1.12.1 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 diff --git a/go.sum b/go.sum index 850032a69..32ef9c0f1 100644 --- a/go.sum +++ b/go.sum @@ -111,8 +111,8 @@ github.com/klauspost/cpuid/v2 v2.0.1/go.mod h1:FInQzS24/EEf25PyTYn52gqo7WaD8xa02 github.com/klauspost/cpuid/v2 v2.2.6 h1:ndNyv040zDGIDh8thGkXYjnFtiN02M1PVVF+JE/48xc= github.com/klauspost/cpuid/v2 v2.2.6/go.mod h1:Lcz8mBdAVJIBVzewtcLocK12l3Y+JytZYpaMropDUws= github.com/klauspost/reedsolomon v1.9.3/go.mod h1:CwCi+NUr9pqSVktrkN+Ondf06rkhYZ/pcNv7fu+8Un4= -github.com/klauspost/reedsolomon v1.12.0 h1:I5FEp3xSwVCcEh3F5A7dofEfhXdF/bWhQWPH+XwBFno= -github.com/klauspost/reedsolomon v1.12.0/go.mod h1:EPLZJeh4l27pUGC3aXOjheaoh1I9yut7xTURiW3LQ9Y= +github.com/klauspost/reedsolomon v1.12.1 h1:NhWgum1efX1x58daOBGCFWcxtEhOhXKKl1HAPQUp03Q= +github.com/klauspost/reedsolomon v1.12.1/go.mod h1:nEi5Kjb6QqtbofI6s+cbG/j1da11c96IBYBSnVGtuBs= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= From 1205044c7f22eef99697c9352ae018ef91af6f80 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Tue, 6 Feb 2024 13:56:53 +0000 Subject: [PATCH 023/144] build(deps): bump github.com/minio/minio-go/v7 from 7.0.65 to 7.0.66 Bumps [github.com/minio/minio-go/v7](https://github.com/minio/minio-go) from 7.0.65 to 7.0.66. - [Release notes](https://github.com/minio/minio-go/releases) - [Commits](https://github.com/minio/minio-go/compare/v7.0.65...v7.0.66) --- updated-dependencies: - dependency-name: github.com/minio/minio-go/v7 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 1f45fa598..319c5a7ff 100644 --- a/go.mod +++ b/go.mod @@ -8,7 +8,7 @@ require ( github.com/google/go-cmp v0.6.0 github.com/gotd/contrib v0.19.0 github.com/klauspost/reedsolomon v1.12.1 - github.com/minio/minio-go/v7 v7.0.65 + github.com/minio/minio-go/v7 v7.0.66 github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b diff --git a/go.sum b/go.sum index 259a1ff3c..8ff9f0ac7 100644 --- a/go.sum +++ b/go.sum @@ -129,8 +129,8 @@ github.com/mattn/go-sqlite3 v1.14.18/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= -github.com/minio/minio-go/v7 v7.0.65 h1:sOlB8T3nQK+TApTpuN3k4WD5KasvZIE3vVFzyyCa0go= -github.com/minio/minio-go/v7 v7.0.65/go.mod h1:R4WVUR6ZTedlCcGwZRauLMIKjgyaWxhs4Mqi/OMPmEc= +github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= +github.com/minio/minio-go/v7 v7.0.66/go.mod h1:DHAgmyQEGdW3Cif0UooKOyrT3Vxs82zNdV6tkKhRtbs= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= From 006cc45f6c5ba2e433fbcfe4eee11e2b3b1415d3 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 6 Feb 2024 15:49:43 +0100 Subject: [PATCH 024/144] all: upgrade go --- .github/workflows/test.yml | 2 +- .github/workflows/ui.yml | 2 +- go.mod | 4 +++- go.sum | 12 ++++++++++++ 4 files changed, 17 insertions(+), 3 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 8398724e2..291b25c10 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -12,7 +12,7 @@ jobs: strategy: matrix: os: [ ubuntu-latest , macos-latest, windows-latest ] - go-version: [ '1.20', '1.21' ] + go-version: [ '1.21' ] steps: - name: Configure Windows if: matrix.os == 'windows-latest' diff --git a/.github/workflows/ui.yml b/.github/workflows/ui.yml index 350b46e28..eeec2a09a 100644 --- a/.github/workflows/ui.yml +++ b/.github/workflows/ui.yml @@ -17,7 +17,7 @@ jobs: - name: Set up Go uses: actions/setup-go@v2 with: - go-version: '1.20.0' + go-version: '1.21.0' - name: Check for new renterd tag in SiaFoundation/web id: check-tag diff --git a/go.mod b/go.mod index 5ff03b8bf..bfa1a7511 100644 --- a/go.mod +++ b/go.mod @@ -1,6 +1,8 @@ module go.sia.tech/renterd -go 1.20 +go 1.21 + +toolchain go1.21.6 require ( github.com/gabriel-vasile/mimetype v1.4.3 diff --git a/go.sum b/go.sum index 6b958550f..54719eabd 100644 --- a/go.sum +++ b/go.sum @@ -33,6 +33,7 @@ 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/fatih/color v1.13.0/go.mod h1:kLAiJbzzSOZDVNGyDpeOxJ47H46qBXwg5ILebYFFOfk= 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= @@ -80,6 +81,7 @@ github.com/hashicorp/go-cleanhttp v0.5.2 h1:035FKYIWjmULyFRBKPs8TBQoi0x6d9G4xc9n github.com/hashicorp/go-cleanhttp v0.5.2/go.mod h1:kO/YDlP8L1346E6Sodw+PrpBSV4/SoxCXGY6BqNFT48= github.com/hashicorp/go-hclog v0.9.2/go.mod h1:5CU+agLiy3J7N7QjHK5d05KxGsuXiQLrjA0H7acj2lQ= github.com/hashicorp/go-hclog v1.2.0 h1:La19f8d7WIlm4ogzNHB0JGqs5AUDAZ2UfCY4sJXcJdM= +github.com/hashicorp/go-hclog v1.2.0/go.mod h1:whpDNt7SSdeAju8AWKIWsul05p54N/39EeqMAyrmvFQ= github.com/hashicorp/go-retryablehttp v0.7.5 h1:bJj+Pj19UZMIweq/iie+1u5YCdGrnxCT9yvm0e+Nd5M= github.com/hashicorp/go-retryablehttp v0.7.5/go.mod h1:Jy/gPYAdjqffZ/yFGCFV2doI5wjtH1ewM9u8iYVjtX8= github.com/hashicorp/golang-lru/v2 v2.0.7 h1:a+bsQ5rvGLjzHuww6tVxozPZFVghXaHOwFs4luLUK2k= @@ -117,13 +119,17 @@ github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxv github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= github.com/kr/pretty v0.3.1 h1:flRD4NNwYAUpkphVc1HcthR4KEIFJ65n8Mw5qdRn3LE= +github.com/kr/pretty v0.3.1/go.mod h1:hoEshYVHaxMs3cyo3Yncou5ZscifuDolrwPKZanG3xk= github.com/kr/pty v1.1.1/go.mod h1:pFQYn66WHrOpPYNljwOMqo10TkYh1fy3cYio2l3bCsQ= github.com/kr/text v0.1.0/go.mod h1:4Jbv+DJW3UT/LiOwJeYQe1efqtUx/iVham/4vfdArNI= github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= +github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v0.0.0-20170820004349-d65d576e9348/go.mod h1:B69LEHPfb2qLo0BaaOLcbitczOKLWTsrBG9LczfCD4k= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= +github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= +github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-sqlite3 v1.14.18 h1:JL0eqdCOq6DJVNPSvArO/bIV9/P7fbGrV00LZHc+5aI= github.com/mattn/go-sqlite3 v1.14.18/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S2DGjv9HUNg= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= @@ -161,6 +167,7 @@ github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7z github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/rogpeppe/fastuuid v0.0.0-20150106093220-6724a57986af/go.mod h1:XWv6SoW27p1b0cqNHllgS5HIMJraePCO15w5zCzIWYg= github.com/rogpeppe/go-internal v1.10.0 h1:TMyTOH3F/DB16zRVcYyreMH6GnZZrwQVAoYjRBZyWFQ= +github.com/rogpeppe/go-internal v1.10.0/go.mod h1:UQnix2H7Ngw/k4C5ijL5+65zddjncjaFoBhdsK/akog= github.com/rs/xid v1.5.0 h1:mKX4bl4iPYJtEIxp6CYiUuLQ/8DYMoz0PUdtGgMFRVc= github.com/rs/xid v1.5.0/go.mod h1:trrq9SKmegXys3aeAKXMUTdJsYXVwGY3RLcfgqegfbg= github.com/russross/blackfriday/v2 v2.0.1/go.mod h1:+Rmxgy9KzJVeS9/2gXHxylqXiyQDYRxCVz55jmeOWTM= @@ -188,6 +195,7 @@ github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXf github.com/stretchr/testify v1.3.0/go.mod h1:M5WIy9Dh21IEIfnGCwXGc5bZfKNJtfHm1UVUgZn+9EI= github.com/stretchr/testify v1.7.0/go.mod h1:6Fq8oRcR53rry900zMqJjRRixrwX3KX962/h/Wwjteg= github.com/stretchr/testify v1.8.4 h1:CcVxjf3Q8PM0mHUKJCdn+eZZtm5yQwehR5yeSVQQcUk= +github.com/stretchr/testify v1.8.4/go.mod h1:sz/lmYIOXD/1dqDmKjjqLyZ2RngseejIcXlSw2iwfAo= github.com/tmc/grpc-websocket-proxy v0.0.0-20190109142713-0ad062ec5ee5/go.mod h1:ncp9v5uamzpCO7NfCPTXjqaC+bZgJeR0sMTm6dMHP7U= github.com/ugorji/go v1.1.4/go.mod h1:uQMGLiO92mf5W77hV/PUCpI3pbzQx3CRekS0kk+RGrc= github.com/vbauerster/mpb/v5 v5.0.3/go.mod h1:h3YxU5CSr8rZP4Q3xZPVB3jJLhWPou63lHEdr9ytH4Y= @@ -249,6 +257,7 @@ go.sia.tech/web/renterd v0.44.0 h1:yKu1Kq/6ssV9Vbv4oa+sn2Pc2TNyfcrv/mRPNOuYuB0= go.sia.tech/web/renterd v0.44.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/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= @@ -277,6 +286,7 @@ golang.org/x/mod v0.6.0-dev.0.20220419223038-86c51ed26bb4/go.mod h1:jJ57K6gSWd91 golang.org/x/mod v0.8.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.10.0/go.mod h1:iBbtSCu2XBx23ZKBPSOrRkjjQPZFPuis4dIYUhu/chs= golang.org/x/mod v0.14.0 h1:dGoOF9QVLYng8IHTm7BAyWqCqSheQ5pYWGhzW00YJr0= +golang.org/x/mod v0.14.0/go.mod h1:hTbmBsO62+eylJbnUtE2MGJUyE7QWk4xUqPFrRgJ+7c= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181220203305-927f97764cc3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -365,6 +375,7 @@ gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLks 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= gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c h1:Hei/4ADfdWqJk1ZMxUNpqntNwaWcugrBjAiHlqqRiVk= +gopkg.in/check.v1 v1.0.0-20201130134442-10cb98267c6c/go.mod h1:JHkPIbrfpd72SG/EVd6muEfDQjcINNoR0C8j2r3qZ4Q= gopkg.in/ini.v1 v1.67.0 h1:Dgnx+6+nfE+IfzjUEISNeydPJh9AXNNsWbGP9KzCsOA= gopkg.in/ini.v1 v1.67.0/go.mod h1:pNLf8WUiyNEtQjuu5G5vTm06TEv9tsIgeAvK8hOrP4k= gopkg.in/mgo.v2 v2.0.0-20180705113604-9856a29383ce/go.mod h1:yeKp02qBN3iKW1OzL3MGk2IdtZzaj7SFntXj72NppTA= @@ -374,6 +385,7 @@ gopkg.in/yaml.v2 v2.2.1/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.2/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.2.8/go.mod h1:hI93XBmqTisBFMUTm0b8Fm+jr3Dg1NNxqwp+5A1VGuI= gopkg.in/yaml.v2 v2.4.0 h1:D8xgwECY7CYvx+Y2n4sBz93Jn9JRvxdiyyo8CTfuKaY= +gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= From 20087f65fc81789a51846b9c646c4ff6441c97c9 Mon Sep 17 00:00:00 2001 From: PJ Date: Wed, 7 Feb 2024 08:11:35 +0100 Subject: [PATCH 025/144] api: use TimeRFC3339 for PruneMetric timestamp --- api/metrics.go | 2 +- autopilot/contract_pruning.go | 2 +- stores/metrics.go | 2 +- stores/metrics_test.go | 4 ++-- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/api/metrics.go b/api/metrics.go index 85684a7b5..412435e34 100644 --- a/api/metrics.go +++ b/api/metrics.go @@ -87,7 +87,7 @@ type ( } ContractPruneMetric struct { - Timestamp time.Time `json:"timestamp"` + Timestamp TimeRFC3339 `json:"timestamp"` ContractID types.FileContractID `json:"contractID"` HostKey types.PublicKey `json:"hostKey"` diff --git a/autopilot/contract_pruning.go b/autopilot/contract_pruning.go index f115da84f..e32cd3fa0 100644 --- a/autopilot/contract_pruning.go +++ b/autopilot/contract_pruning.go @@ -80,7 +80,7 @@ func (pr pruneResult) toAlert() (id types.Hash256, alert *alerts.Alert) { func (pr pruneResult) toMetric() api.ContractPruneMetric { return api.ContractPruneMetric{ - Timestamp: pr.ts, + Timestamp: api.TimeRFC3339(pr.ts), ContractID: pr.fcid, HostKey: pr.hk, Pruned: pr.pruned, diff --git a/stores/metrics.go b/stores/metrics.go index 55af3c518..203ed3b71 100644 --- a/stores/metrics.go +++ b/stores/metrics.go @@ -155,7 +155,7 @@ func (s *SQLStore) ContractPruneMetrics(ctx context.Context, start time.Time, n resp := make([]api.ContractPruneMetric, len(metrics)) for i := range resp { resp[i] = api.ContractPruneMetric{ - Timestamp: time.Time(metrics[i].Timestamp).UTC(), + Timestamp: api.TimeRFC3339(metrics[i].Timestamp), ContractID: types.FileContractID(metrics[i].FCID), HostKey: types.PublicKey(metrics[i].Host), diff --git a/stores/metrics_test.go b/stores/metrics_test.go index 5f59b2866..2b2f572a7 100644 --- a/stores/metrics_test.go +++ b/stores/metrics_test.go @@ -60,7 +60,7 @@ func TestContractPruneMetrics(t *testing.T) { for hi, host := range hosts { for _, recordedTime := range times { metric := api.ContractPruneMetric{ - Timestamp: recordedTime, + Timestamp: api.TimeRFC3339(recordedTime), ContractID: types.FileContractID{i}, HostKey: host, @@ -92,7 +92,7 @@ func TestContractPruneMetrics(t *testing.T) { t.Fatal("expected metrics to be sorted by time") } for _, m := range metrics { - if !cmp.Equal(m, fcid2Metric[m.ContractID]) { + if !cmp.Equal(m, fcid2Metric[m.ContractID], cmp.Comparer(api.CompareTimeRFC3339)) { t.Fatal("unexpected metric", cmp.Diff(m, fcid2Metric[m.ContractID])) } cmpFn(m) From 9922e60098fdbaab9cc2df52301365ccb382d155 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 7 Feb 2024 10:36:30 +0100 Subject: [PATCH 026/144] ci: change condition to ignore rc tags --- .github/workflows/publish.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index f02e1134a..e60966933 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -254,7 +254,7 @@ jobs: name: renterd path: release/ dispatch: - if: startsWith(github.ref, 'refs/tags/v') + if: ${{ !contains(github.ref, 'rc') }} strategy: matrix: repo: ['siafoundation/homebrew-sia', 'siafoundation/linux'] From 5a8d60c1b2cf3b91e2c0baacaf6aca50f7b69e8e Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 7 Feb 2024 13:20:05 +0100 Subject: [PATCH 027/144] test.yml: run tests for go1.22 --- .github/workflows/test.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 291b25c10..8e4c21faf 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -12,7 +12,7 @@ jobs: strategy: matrix: os: [ ubuntu-latest , macos-latest, windows-latest ] - go-version: [ '1.21' ] + go-version: [ '1.21', '1.22' ] steps: - name: Configure Windows if: matrix.os == 'windows-latest' From c0267d3f44a30880a75f093a6e892e88bda103c5 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 7 Feb 2024 13:41:49 +0100 Subject: [PATCH 028/144] ci: add needs --- .github/workflows/publish.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index e60966933..cfaeea747 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -254,7 +254,8 @@ jobs: name: renterd path: release/ dispatch: - if: ${{ !contains(github.ref, 'rc') }} + if: startsWith(github.ref, 'refs/tags/v') && !contains(github.ref, 'rc') + needs: [docker, build-linux, build-mac, build-windows] strategy: matrix: repo: ['siafoundation/homebrew-sia', 'siafoundation/linux'] From 2723561a7fd7e9afaac387f6ccf4aedf12e31c2f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 10:35:15 +0100 Subject: [PATCH 029/144] worker: get rid of 'used' and 'curr' fields in slabDownload --- worker/download.go | 58 ++++++++++++++++++---------------------------- 1 file changed, 23 insertions(+), 35 deletions(-) diff --git a/worker/download.go b/worker/download.go index 128b7637c..44e41a116 100644 --- a/worker/download.go +++ b/worker/download.go @@ -73,9 +73,7 @@ type ( numOverpaid uint64 numRelaunched uint64 - curr types.PublicKey hostToSectors map[types.PublicKey][]sectorInfo - used map[types.PublicKey]struct{} sectors [][]byte errs HostErrorSet @@ -521,7 +519,6 @@ func (mgr *downloadManager) newSlabDownload(ctx context.Context, slice object.Sl overpay: migration && slice.Health <= downloadOverpayHealthThreshold, hostToSectors: hostToSectors, - used: make(map[types.PublicKey]struct{}), sectors: make([][]byte, len(slice.Shards)), errs: make(HostErrorSet), @@ -633,36 +630,34 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, defer s.mu.Unlock() // prepare next sectors to download - if len(s.hostToSectors[s.curr]) == 0 { - // select all possible hosts - var hosts []types.PublicKey - 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) - } - } - - // no more sectors to download - if len(hosts) == 0 { - return nil + // select all possible hosts + var hosts []types.PublicKey + for host, sectors := range s.hostToSectors { + if len(sectors) == 0 { + continue // ignore hosts with no more sectors } + hosts = append(hosts, host) + } - // select the fastest host - fastest := s.mgr.fastest(hosts) - if fastest == (types.PublicKey{}) { - return nil // can happen if downloader got stopped - } + // no more sectors to download + if len(hosts) == 0 { + return nil + } - // make the fastest host the current host - s.curr = fastest - s.used[s.curr] = struct{}{} + // select the fastest host + fastest := s.mgr.fastest(hosts) + if fastest == (types.PublicKey{}) { + return nil // can happen if downloader got stopped } // pop the next sector - sector := s.hostToSectors[s.curr][0] - s.hostToSectors[s.curr] = s.hostToSectors[s.curr][1:] + sector := s.hostToSectors[fastest][0] + s.hostToSectors[fastest] = s.hostToSectors[fastest][1:] + + // if host is out of sectors, remove it + if len(s.hostToSectors[fastest]) == 0 { + delete(s.hostToSectors, fastest) + } // build the request return §orDownloadReq{ @@ -808,14 +803,7 @@ func (s *slabDownload) finish() ([][]byte, bool, error) { s.mu.Lock() defer s.mu.Unlock() if s.numCompleted < s.minShards { - var unused int - for host := range s.hostToSectors { - if _, used := s.used[host]; !used { - unused++ - } - } - - return nil, s.numOverpaid > 0, fmt.Errorf("failed to download slab: completed=%d inflight=%d launched=%d relaunched=%d overpaid=%d downloaders=%d unused=%d errors=%d %v", s.numCompleted, s.numInflight, s.numLaunched, s.numRelaunched, s.numOverpaid, s.mgr.numDownloaders(), unused, len(s.errs), s.errs) + return nil, s.numOverpaid > 0, fmt.Errorf("failed to download slab: completed=%d inflight=%d launched=%d relaunched=%d overpaid=%d downloaders=%d unused=%d errors=%d %v", s.numCompleted, s.numInflight, s.numLaunched, s.numRelaunched, s.numOverpaid, s.mgr.numDownloaders(), len(s.hostToSectors), len(s.errs), s.errs) } return s.sectors, s.numOverpaid > 0, nil } From 6d0037f240a9a758515d846c00bf0ae910de8320 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 11:41:30 +0100 Subject: [PATCH 030/144] worker: move sector index check from launch to nextRequest --- worker/download.go | 39 ++++++++++++++++++++------------------- 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/worker/download.go b/worker/download.go index 44e41a116..d7437b341 100644 --- a/worker/download.go +++ b/worker/download.go @@ -73,7 +73,7 @@ type ( numOverpaid uint64 numRelaunched uint64 - hostToSectors map[types.PublicKey][]sectorInfo + unusedHostSectors map[types.PublicKey][]sectorInfo sectors [][]byte errs HostErrorSet @@ -518,7 +518,7 @@ func (mgr *downloadManager) newSlabDownload(ctx context.Context, slice object.Sl created: time.Now(), overpay: migration && slice.Health <= downloadOverpayHealthThreshold, - hostToSectors: hostToSectors, + unusedHostSectors: hostToSectors, sectors: make([][]byte, len(slice.Shards)), errs: make(HostErrorSet), @@ -632,9 +632,18 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, // prepare next sectors to download // select all possible hosts var hosts []types.PublicKey - for host, sectors := range s.hostToSectors { + for host, sectors := range s.unusedHostSectors { + // remove any sector that has been downloaded already + for i := range sectors { + if len(s.sectors[sectors[i].index]) > 0 { + sectors = append(sectors[:i], sectors[i+1:]...) + s.unusedHostSectors[host] = sectors + } + } + // remove host if no sectors are left if len(sectors) == 0 { - continue // ignore hosts with no more sectors + delete(s.unusedHostSectors, host) + continue } hosts = append(hosts, host) } @@ -647,16 +656,17 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, // select the fastest host fastest := s.mgr.fastest(hosts) if fastest == (types.PublicKey{}) { + s.mgr.logger.Debugw("missing downloaders for hosts", "hosts", len(hosts)) return nil // can happen if downloader got stopped } // pop the next sector - sector := s.hostToSectors[fastest][0] - s.hostToSectors[fastest] = s.hostToSectors[fastest][1:] + sector := s.unusedHostSectors[fastest][0] + s.unusedHostSectors[fastest] = s.unusedHostSectors[fastest][1:] // if host is out of sectors, remove it - if len(s.hostToSectors[fastest]) == 0 { - delete(s.hostToSectors, fastest) + if len(s.unusedHostSectors[fastest]) == 0 { + delete(s.unusedHostSectors, fastest) } // build the request @@ -803,7 +813,7 @@ func (s *slabDownload) finish() ([][]byte, bool, error) { s.mu.Lock() defer s.mu.Unlock() if s.numCompleted < s.minShards { - return nil, s.numOverpaid > 0, fmt.Errorf("failed to download slab: completed=%d inflight=%d launched=%d relaunched=%d overpaid=%d downloaders=%d unused=%d errors=%d %v", s.numCompleted, s.numInflight, s.numLaunched, s.numRelaunched, s.numOverpaid, s.mgr.numDownloaders(), len(s.hostToSectors), len(s.errs), s.errs) + return nil, s.numOverpaid > 0, fmt.Errorf("failed to download slab: completed=%d inflight=%d launched=%d relaunched=%d overpaid=%d downloaders=%d unused=%d errors=%d %v", s.numCompleted, s.numInflight, s.numLaunched, s.numRelaunched, s.numOverpaid, s.mgr.numDownloaders(), len(s.unusedHostSectors), len(s.errs), s.errs) } return s.sectors, s.numOverpaid > 0, nil } @@ -827,19 +837,10 @@ func (s *slabDownload) launch(req *sectorDownloadReq) error { s.mu.Lock() defer s.mu.Unlock() - // check for nil - if req == nil { - return errors.New("no request given") - } - - // check for completed sector - if len(s.sectors[req.sectorIndex]) > 0 { - return errors.New("sector already downloaded") - } - // launch the req err := s.mgr.launch(req) if err != nil { + s.errs[req.hk] = fmt.Errorf("failed to launch request: %w", err) return err } From 36a01e7993d8bdc7a7d8e35b2d1a7885d093295e Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 11:51:17 +0100 Subject: [PATCH 031/144] worker: remove debug logging --- worker/download.go | 1 - 1 file changed, 1 deletion(-) diff --git a/worker/download.go b/worker/download.go index d7437b341..cec10a758 100644 --- a/worker/download.go +++ b/worker/download.go @@ -656,7 +656,6 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, // select the fastest host fastest := s.mgr.fastest(hosts) if fastest == (types.PublicKey{}) { - s.mgr.logger.Debugw("missing downloaders for hosts", "hosts", len(hosts)) return nil // can happen if downloader got stopped } From c3212766eba6973c0a958bec3b8e3c2c0a966883 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 13:09:42 +0100 Subject: [PATCH 032/144] worker: have mgr.fastest return *downloader --- worker/download.go | 66 +++++++++++++++----------------------------- worker/downloader.go | 6 ++++ worker/host.go | 4 +++ worker/mocks_test.go | 2 ++ 4 files changed, 35 insertions(+), 43 deletions(-) diff --git a/worker/download.go b/worker/download.go index cec10a758..7eb519435 100644 --- a/worker/download.go +++ b/worker/download.go @@ -93,7 +93,7 @@ type ( length uint32 offset uint32 root types.Hash256 - hk types.PublicKey + host *downloader overpay bool overdrive bool @@ -610,9 +610,7 @@ func (s *slabDownload) overdrive(ctx context.Context, resps *sectorResponses) (r if canOverdrive(timeout()) { for { if req := s.nextRequest(ctx, resps, true); req != nil { - if err := s.launch(req); err != nil { - continue // try the next request if this fails to launch - } + s.launch(req) } break } @@ -655,17 +653,19 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, // select the fastest host fastest := s.mgr.fastest(hosts) - if fastest == (types.PublicKey{}) { + if fastest == nil { + s.errs[types.PublicKey{}] = errors.New("no more downloaders available") return nil // can happen if downloader got stopped } // pop the next sector - sector := s.unusedHostSectors[fastest][0] - s.unusedHostSectors[fastest] = s.unusedHostSectors[fastest][1:] + hk := fastest.PublicKey() + sector := s.unusedHostSectors[hk][0] + s.unusedHostSectors[hk] = s.unusedHostSectors[hk][1:] // if host is out of sectors, remove it - if len(s.unusedHostSectors[fastest]) == 0 { - delete(s.unusedHostSectors, fastest) + if len(s.unusedHostSectors[hk]) == 0 { + delete(s.unusedHostSectors, hk) } // build the request @@ -675,7 +675,7 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, offset: s.offset, length: s.length, root: sector.Root, - hk: sector.LatestHost, + host: fastest, // overpay is set to 'true' when a request is retried after the slab // download failed and we realise that it might have succeeded if we @@ -709,9 +709,9 @@ func (s *slabDownload) download(ctx context.Context) ([][]byte, bool, error) { req := s.nextRequest(ctx, resps, false) if req == nil { return nil, false, fmt.Errorf("no host available for shard %d", i) - } else if err := s.launch(req); err == nil { - i++ } + s.launch(req) + i++ } // collect requests that failed due to gouging @@ -748,19 +748,17 @@ loop: // launch overdrive requests for { if req := s.nextRequest(ctx, resps, true); req != nil { - if err := s.launch(req); err != nil { - continue - } + s.launch(req) } break } // handle lost sectors if isSectorNotFound(resp.err) { - if err := s.mgr.os.DeleteHostSector(ctx, resp.req.hk, resp.req.root); err != nil { - s.mgr.logger.Errorw("failed to mark sector as lost", "hk", resp.req.hk, "root", resp.req.root, zap.Error(err)) + if err := s.mgr.os.DeleteHostSector(ctx, resp.req.host.PublicKey(), resp.req.root); err != nil { + s.mgr.logger.Errorw("failed to mark sector as lost", "hk", resp.req.host.PublicKey(), "root", resp.req.root, zap.Error(err)) } else { - s.mgr.logger.Infow("successfully marked sector as lost", "hk", resp.req.hk, "root", resp.req.root) + s.mgr.logger.Infow("successfully marked sector as lost", "hk", resp.req.host.PublicKey(), "root", resp.req.root) } } else if isPriceTableGouging(resp.err) && s.overpay && !resp.req.overpay { resp.req.overpay = true // ensures we don't retry the same request over and over again @@ -772,7 +770,7 @@ loop: if !done && len(gouging) >= s.missing() { for _, req := range gouging { - _ = s.launch(req) // ignore error + s.launch(req) } gouging = nil goto loop @@ -832,16 +830,12 @@ func (s *slabDownload) inflight() uint64 { return s.numInflight } -func (s *slabDownload) launch(req *sectorDownloadReq) error { +func (s *slabDownload) launch(req *sectorDownloadReq) { s.mu.Lock() defer s.mu.Unlock() - // launch the req - err := s.mgr.launch(req) - if err != nil { - s.errs[req.hk] = fmt.Errorf("failed to launch request: %w", err) - return err - } + // queue the request + req.host.enqueue(req) // update the state s.numInflight++ @@ -853,7 +847,6 @@ func (s *slabDownload) launch(req *sectorDownloadReq) error { } else { s.numLaunched++ } - return nil } func (s *slabDownload) receive(resp sectorDownloadResp) (finished bool) { @@ -868,7 +861,7 @@ func (s *slabDownload) receive(resp sectorDownloadResp) (finished bool) { // failed reqs can't complete the upload s.numInflight-- if resp.err != nil { - s.errs[resp.req.hk] = resp.err + s.errs[resp.req.host.PublicKey()] = resp.err return false } @@ -884,7 +877,7 @@ func (s *slabDownload) receive(resp sectorDownloadResp) (finished bool) { return s.numCompleted >= s.minShards } -func (mgr *downloadManager) fastest(hosts []types.PublicKey) (fastest types.PublicKey) { +func (mgr *downloadManager) fastest(hosts []types.PublicKey) (fastest *downloader) { // recompute stats mgr.tryRecomputeStats() @@ -897,25 +890,12 @@ func (mgr *downloadManager) fastest(hosts []types.PublicKey) (fastest types.Publ continue } else if estimate := d.estimate(); estimate < lowest { lowest = estimate - fastest = h + fastest = d } } return } -func (mgr *downloadManager) launch(req *sectorDownloadReq) error { - mgr.mu.Lock() - defer mgr.mu.Unlock() - - downloader, exists := mgr.downloaders[req.hk] - if !exists { - return fmt.Errorf("no downloader for host %v", req.hk) - } - - downloader.enqueue(req) - return nil -} - type slabSlice struct { object.SlabSlice PartialSlab bool diff --git a/worker/downloader.go b/worker/downloader.go index 32e5981b6..30c855d80 100644 --- a/worker/downloader.go +++ b/worker/downloader.go @@ -9,6 +9,7 @@ import ( "time" rhpv2 "go.sia.tech/core/rhp/v2" + "go.sia.tech/core/types" "go.sia.tech/renterd/stats" ) @@ -19,6 +20,7 @@ const ( type ( downloader struct { + hk types.PublicKey host Host statsDownloadSpeedBytesPerMS *stats.DataPoints // keep track of this separately for stats (no decay is applied) @@ -48,6 +50,10 @@ func newDownloader(ctx context.Context, host Host) *downloader { } } +func (d *downloader) PublicKey() types.PublicKey { + return d.host.PublicKey() +} + func (d *downloader) Stop() { for { download := d.pop() diff --git a/worker/host.go b/worker/host.go index ba3ec5521..fcaac0cf1 100644 --- a/worker/host.go +++ b/worker/host.go @@ -18,6 +18,8 @@ import ( type ( Host interface { + PublicKey() types.PublicKey + DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) error UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) @@ -77,6 +79,8 @@ func (w *worker) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr } } +func (h *host) PublicKey() types.PublicKey { return h.hk } + func (h *host) DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) (err error) { pt, err := h.priceTables.fetch(ctx, h.hk, nil) if err != nil { diff --git a/worker/mocks_test.go b/worker/mocks_test.go index cff429743..e6fd62d8e 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -382,6 +382,8 @@ func newMockHost(hk types.PublicKey) *mockHost { } } +func (h *mockHost) PublicKey() types.PublicKey { return h.hk } + func (h *mockHost) DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) error { sector, exist := h.contract().sector(root) if !exist { From a9cf515cb9d4f6160a60b2316b974f584472d4d0 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 15:53:40 +0100 Subject: [PATCH 033/144] worker: set error when running out of hosts --- worker/download.go | 15 +++++++++------ 1 file changed, 9 insertions(+), 6 deletions(-) diff --git a/worker/download.go b/worker/download.go index 7eb519435..f23c8e640 100644 --- a/worker/download.go +++ b/worker/download.go @@ -609,10 +609,11 @@ func (s *slabDownload) overdrive(ctx context.Context, resps *sectorResponses) (r case <-timer.C: if canOverdrive(timeout()) { for { - if req := s.nextRequest(ctx, resps, true); req != nil { - s.launch(req) + req := s.nextRequest(ctx, resps, true) + if req == nil { + break } - break + s.launch(req) } } resetTimer() @@ -646,16 +647,18 @@ func (s *slabDownload) nextRequest(ctx context.Context, resps *sectorResponses, hosts = append(hosts, host) } - // no more sectors to download + // no more sectors to download - we don't know if the download failed at + // this point so we register an error that gets propagated in case it did if len(hosts) == 0 { + s.errs[types.PublicKey{}] = fmt.Errorf("%w: no more hosts", errDownloadNotEnoughHosts) return nil } // select the fastest host fastest := s.mgr.fastest(hosts) if fastest == nil { - s.errs[types.PublicKey{}] = errors.New("no more downloaders available") - return nil // can happen if downloader got stopped + s.errs[types.PublicKey{}] = fmt.Errorf("%w: no more downloaders", errDownloadNotEnoughHosts) + return nil } // pop the next sector From 78f9ddd499617cdc75d8a7a8f46a35e06d18ffa1 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 8 Feb 2024 16:30:33 +0100 Subject: [PATCH 034/144] worker: attach interaction recorder to worker shutdownCtx --- worker/upload.go | 3 +-- worker/worker.go | 9 ++++++--- 2 files changed, 7 insertions(+), 5 deletions(-) diff --git a/worker/upload.go b/worker/upload.go index bbc0aff5f..72c65bf07 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -205,9 +205,8 @@ func (w *worker) threadedUploadPackedSlabs(rs api.RedundancySettings, contractSe }() // keep uploading packed slabs until we're done - ctx := context.WithValue(w.shutdownCtx, keyInteractionRecorder, w) for { - uploaded, err := w.uploadPackedSlabs(ctx, defaultPackedSlabsLockDuration, rs, contractSet, lockPriority) + uploaded, err := w.uploadPackedSlabs(w.shutdownCtx, defaultPackedSlabsLockDuration, rs, contractSet, lockPriority) if err != nil { w.logger.Errorf("couldn't upload packed slabs, err: %v", err) return diff --git a/worker/worker.go b/worker/worker.go index 4d3c950e2..92e01c3a2 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1292,7 +1292,6 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush return nil, errors.New("uploadMaxMemory cannot be 0") } - shutdownCtx, shutdownCtxCancel := context.WithCancel(context.Background()) w := &worker{ alerts: alerts.WithOrigin(b, fmt.Sprintf("worker.%s", id)), allowPrivateIPs: allowPrivateIPs, @@ -1303,9 +1302,13 @@ 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: shutdownCtx, - shutdownCtxCancel: shutdownCtxCancel, } + + ctx, cancel := context.WithCancel(context.Background()) + ctx = context.WithValue(ctx, keyInteractionRecorder, w) + w.shutdownCtx = ctx + w.shutdownCtxCancel = cancel + w.initAccounts(b) w.initPriceTables() w.initTransportPool() From d374294c1b66e510a9bdce0b92aea5f1ae9ea25f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 9 Feb 2024 14:17:11 +0100 Subject: [PATCH 035/144] worker: try again if scan fails --- worker/worker.go | 96 ++++++++++++++++++++++++++++-------------------- 1 file changed, 57 insertions(+), 39 deletions(-) diff --git a/worker/worker.go b/worker/worker.go index 92e01c3a2..72f37d38c 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1369,49 +1369,43 @@ func (w *worker) Shutdown(ctx context.Context) error { return nil } -func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP string) (settings rhpv2.HostSettings, pt rhpv3.HostPriceTable, elapsed time.Duration, err error) { - // record host scan - defer func() { - HostInteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ - HostKey: hostKey, - Success: isSuccessfulInteraction(err), - Timestamp: time.Now(), - Settings: settings, - PriceTable: pt, - }) - }() - - // resolve hostIP. We don't want to scan hosts on private networks. - if !w.allowPrivateIPs { - host, _, err := net.SplitHostPort(hostIP) - if err != nil { - return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, err - } - addrs, err := (&net.Resolver{}).LookupIPAddr(ctx, host) - if err != nil { - return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, err - } - for _, addr := range addrs { - if isPrivateIP(addr.IP) { - return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, errors.New("host is on a private network") +func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP string) (rhpv2.HostSettings, rhpv3.HostPriceTable, time.Duration, error) { + // prepare a helper for scanning + scan := func() (rhpv2.HostSettings, rhpv3.HostPriceTable, time.Duration, error) { + // resolve hostIP. We don't want to scan hosts on private networks. + if !w.allowPrivateIPs { + host, _, err := net.SplitHostPort(hostIP) + if err != nil { + return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, err + } + addrs, err := (&net.Resolver{}).LookupIPAddr(ctx, host) + if err != nil { + return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, err + } + for _, addr := range addrs { + if isPrivateIP(addr.IP) { + return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, errors.New("host is on a private network") + } } } - } - // fetch the host settings - start := time.Now() - err = w.withTransportV2(ctx, hostKey, hostIP, func(t *rhpv2.Transport) (err error) { - if settings, err = RPCSettings(ctx, t); err == nil { - // NOTE: we overwrite the NetAddress with the host address here since we - // just used it to dial the host we know it's valid - settings.NetAddress = hostIP + // fetch the host settings + start := time.Now() + var settings rhpv2.HostSettings + err := w.withTransportV2(ctx, hostKey, hostIP, func(t *rhpv2.Transport) (err error) { + if settings, err = RPCSettings(ctx, t); err == nil { + // NOTE: we overwrite the NetAddress with the host address here since we + // just used it to dial the host we know it's valid + settings.NetAddress = hostIP + } + return err + }) + if err != nil { + return settings, rhpv3.HostPriceTable{}, time.Since(start), errors.New("host is on a private network") } - return err - }) - elapsed = time.Since(start) - // fetch the host pricetable - if err == nil { + // fetch the host pricetable + var pt rhpv3.HostPriceTable err = w.transportPoolV3.withTransportV3(ctx, hostKey, settings.SiamuxAddr(), func(ctx context.Context, t *transportV3) error { if hpt, err := RPCPriceTable(ctx, t, func(pt rhpv3.HostPriceTable) (rhpv3.PaymentMethod, error) { return nil, nil }); err != nil { return err @@ -1420,8 +1414,32 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s return nil } }) + return settings, pt, time.Since(start), err } - return + + // scan: first try + settings, pt, duration, err := scan() + if err != nil { + // scan: second try + select { + case <-ctx.Done(): + case <-time.After(time.Second): + } + settings, pt, duration, err = scan() + if err == nil { + w.logger.Infof("successfully scanned host %v after retry", hostKey) + } + } + + // record host scan + HostInteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ + HostKey: hostKey, + Success: isSuccessfulInteraction(err), + Timestamp: time.Now(), + Settings: settings, + PriceTable: pt, + }) + return settings, pt, duration, err } func discardTxnOnErr(ctx context.Context, bus Bus, l *zap.SugaredLogger, txn types.Transaction, errContext string, err *error) { From dc5c0a0cc806faad819c2a648a567d50285021d8 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 9 Feb 2024 14:19:01 +0100 Subject: [PATCH 036/144] publish.yml: don't trigger respository dispatch for releases with a dash --- .github/workflows/publish.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.github/workflows/publish.yml b/.github/workflows/publish.yml index cfaeea747..c0b6f866d 100644 --- a/.github/workflows/publish.yml +++ b/.github/workflows/publish.yml @@ -254,7 +254,7 @@ jobs: name: renterd path: release/ dispatch: - if: startsWith(github.ref, 'refs/tags/v') && !contains(github.ref, 'rc') + if: startsWith(github.ref, 'refs/tags/v') && !contains(github.ref, '-') needs: [docker, build-linux, build-mac, build-windows] strategy: matrix: From 95e86aa66b3eb835634d8ce61da94fb3fd770287 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 9 Feb 2024 14:50:05 +0100 Subject: [PATCH 037/144] worker: check shutdownCtx before recording --- worker/worker.go | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/worker/worker.go b/worker/worker.go index 72f37d38c..ade547727 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1427,10 +1427,19 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s } settings, pt, duration, err = scan() if err == nil { - w.logger.Infof("successfully scanned host %v after retry", hostKey) + w.logger.Debug("successfully scanned host %v after retry", hostKey) } } + // check if the scan failed due to a shutdown - shouldn't be necessary but + // just in case since recording a failed scan might have serious + // repercussions + select { + case <-w.shutdownCtx.Done(): + return rhpv2.HostSettings{}, rhpv3.HostPriceTable{}, 0, w.shutdownCtx.Err() + default: + } + // record host scan HostInteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ HostKey: hostKey, From 0406a8ef4f06e37521c232142f7212db6d95646b Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Sat, 10 Feb 2024 09:38:31 +0100 Subject: [PATCH 038/144] stores: TestUpsertSectors --- stores/metadata.go | 14 ------------ stores/metadata_test.go | 50 +++++++++++++++++++++++++++++++++++++++++ 2 files changed, 50 insertions(+), 14 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 8302c1064..2802bc85f 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -3079,19 +3079,5 @@ func upsertSectors(tx *gorm.DB, sectors []dbSector) ([]dbSector, error) { if err != nil { return nil, err } - // fetch the upserted sectors - roots := make([][]byte, len(sectors)) - for i, sector := range sectors { - roots[i] = sector.Root[:] - } - sectors = sectors[:0] - - var batch []dbSector - if err := tx.Where("root IN (?)", roots).FindInBatches(&batch, sectorQueryBatchSize, func(tx *gorm.DB, _ int) error { - sectors = append(sectors, batch...) - return nil - }).Error; err != nil { - return nil, err - } return sectors, nil } diff --git a/stores/metadata_test.go b/stores/metadata_test.go index a9c41d2a5..60d7358ff 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -3989,3 +3989,53 @@ func TestSlabCleanupTrigger(t *testing.T) { t.Fatalf("expected 1 slabs, got %v", slabCntr) } } + +func TestUpsertSectors(t *testing.T) { + ss := newTestSQLStore(t, defaultTestSQLStoreConfig) + defer ss.Close() + + err := ss.db.Create(&dbSlab{ + DBContractSetID: 1, + Key: []byte{1}, + }).Error + if err != nil { + t.Fatal(err) + } + + err = ss.db.Create(&dbSector{ + DBSlabID: 1, + SlabIndex: 2, + Root: []byte{2}, + }).Error + if err != nil { + t.Fatal(err) + } + + sectors := []dbSector{ + { + DBSlabID: 1, + SlabIndex: 1, + Root: []byte{1}, + }, + { + DBSlabID: 1, + SlabIndex: 2, + Root: []byte{2}, + }, + { + DBSlabID: 1, + SlabIndex: 3, + Root: []byte{3}, + }, + } + result, err := upsertSectors(ss.db, sectors) + if err != nil { + t.Fatal(err) + } + + for i, sector := range result { + if sector.SlabIndex != i+1 { + t.Fatal("unexpected slab index", sector.SlabIndex) + } + } +} From 0bc2afef5a420e4e898521cf38f33055a3ae8937 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Sat, 10 Feb 2024 11:39:59 +0100 Subject: [PATCH 039/144] stores: fetch one sector id at a time in upsert --- stores/metadata.go | 27 +++++++++++++++++++-------- stores/metadata_test.go | 9 ++++++--- 2 files changed, 25 insertions(+), 11 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 2802bc85f..f20f7dbb0 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -1886,7 +1886,7 @@ func (ss *SQLStore) UpdateSlab(ctx context.Context, s object.Slab, contractSet s } // ensure the sectors exists - sectors, err = upsertSectors(tx, sectors) + sectorIDs, err := upsertSectors(tx, sectors) if err != nil { return fmt.Errorf("failed to create sector: %w", err) } @@ -1894,14 +1894,14 @@ func (ss *SQLStore) UpdateSlab(ctx context.Context, s object.Slab, contractSet s // build contract <-> sector links var contractSectors []dbContractSector for i, shard := range s.Shards { - sector := sectors[i] + sectorID := sectorIDs[i] // ensure the associations are updated for _, fcids := range shard.Contracts { for _, fcid := range fcids { if _, ok := contracts[fcid]; ok { contractSectors = append(contractSectors, dbContractSector{ - DBSectorID: sector.ID, + DBSectorID: sectorID, DBContractID: contracts[fcid].ID, }) } @@ -2179,7 +2179,7 @@ func (s *SQLStore) createSlices(tx *gorm.DB, objID, multiPartID *uint, contractS } // create sector that don't exist yet - sectors, err = upsertSectors(tx, sectors) + sectorIDs, err := upsertSectors(tx, sectors) if err != nil { return fmt.Errorf("failed to create sectors: %w", err) } @@ -2189,12 +2189,12 @@ func (s *SQLStore) createSlices(tx *gorm.DB, objID, multiPartID *uint, contractS var contractSectors []dbContractSector for _, ss := range slices { for _, shard := range ss.Shards { - sector := sectors[sectorIdx] + sectorID := sectorIDs[sectorIdx] for _, fcids := range shard.Contracts { for _, fcid := range fcids { if _, ok := contracts[fcid]; ok { contractSectors = append(contractSectors, dbContractSector{ - DBSectorID: sector.ID, + DBSectorID: sectorID, DBContractID: contracts[fcid].ID, }) } @@ -3065,7 +3065,7 @@ func validateSort(sortBy, sortDir string) error { // upsertSectors creates a sector or updates it if it exists already. The // resulting ID is set on the input sector. -func upsertSectors(tx *gorm.DB, sectors []dbSector) ([]dbSector, error) { +func upsertSectors(tx *gorm.DB, sectors []dbSector) ([]uint, error) { if len(sectors) == 0 { return nil, nil // nothing to do } @@ -3079,5 +3079,16 @@ func upsertSectors(tx *gorm.DB, sectors []dbSector) ([]dbSector, error) { if err != nil { return nil, err } - return sectors, nil + + sectorIDs := make([]uint, len(sectors)) + for i := range sectors { + var id uint + if err := tx.Model(dbSector{}). + Where("root", sectors[i].Root). + Select("id").Take(&id).Error; err != nil { + return nil, err + } + sectorIDs[i] = id + } + return sectorIDs, nil } diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 60d7358ff..18f34dee4 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4028,13 +4028,16 @@ func TestUpsertSectors(t *testing.T) { Root: []byte{3}, }, } - result, err := upsertSectors(ss.db, sectors) + sectorIDs, err := upsertSectors(ss.db, sectors) if err != nil { t.Fatal(err) } - for i, sector := range result { - if sector.SlabIndex != i+1 { + for i, id := range sectorIDs { + var sector dbSector + if err := ss.db.Where("id", id).Take(§or).Error; err != nil { + t.Fatal(err) + } else if sector.SlabIndex != i+1 { t.Fatal("unexpected slab index", sector.SlabIndex) } } From 7a0446c7aa6d1b31e6039606b44de842ecb3c4b0 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Feb 2024 01:46:23 +0000 Subject: [PATCH 040/144] build(deps): bump golang.org/x/crypto from 0.18.0 to 0.19.0 Bumps [golang.org/x/crypto](https://github.com/golang/crypto) from 0.18.0 to 0.19.0. - [Commits](https://github.com/golang/crypto/compare/v0.18.0...v0.19.0) --- updated-dependencies: - dependency-name: golang.org/x/crypto dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- go.mod | 6 +++--- go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/go.mod b/go.mod index bfa1a7511..ff700196a 100644 --- a/go.mod +++ b/go.mod @@ -21,8 +21,8 @@ require ( go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca go.sia.tech/web/renterd v0.44.0 go.uber.org/zap v1.26.0 - golang.org/x/crypto v0.18.0 - golang.org/x/term v0.16.0 + golang.org/x/crypto v0.19.0 + golang.org/x/term v0.17.0 gopkg.in/yaml.v3 v3.0.1 gorm.io/driver/mysql v1.5.2 gorm.io/driver/sqlite v1.5.4 @@ -77,7 +77,7 @@ require ( 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 - golang.org/x/sys v0.16.0 // indirect + golang.org/x/sys v0.17.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect golang.org/x/tools v0.16.1 // indirect diff --git a/go.sum b/go.sum index 54719eabd..5e2fa38d9 100644 --- a/go.sum +++ b/go.sum @@ -275,8 +275,8 @@ golang.org/x/crypto v0.0.0-20200510223506-06a226fb4e37/go.mod h1:LzIPMQfyMNhhGPh golang.org/x/crypto v0.0.0-20210322153248-0c34fe9e7dc2/go.mod h1:T9bdIzuCu7OtxOm1hfPfRQxPLYneinmdGuTeoZ9dtd4= golang.org/x/crypto v0.0.0-20210921155107-089bfa567519/go.mod h1:GvvjBRRGRdwPK5ydBHafDWAxML/pGHZbMvKqRZ5+Abc= golang.org/x/crypto v0.0.0-20220507011949-2cf3adece122/go.mod h1:IxCIyHEi3zRg3s0A5j5BB6A9Jmi73HwBIUl50j+osU4= -golang.org/x/crypto v0.18.0 h1:PGVlW0xEltQnzFZ55hkuX5+KLyrMYhHld1YHO4AKcdc= -golang.org/x/crypto v0.18.0/go.mod h1:R0j02AL6hcrfOiy9T4ZYp/rcWeMxM3L6QYxlOuEG1mg= +golang.org/x/crypto v0.19.0 h1:ENy+Az/9Y1vSrlrvBSyna3PITt4tiZLf7sgCjZBX7Wo= +golang.org/x/crypto v0.19.0/go.mod h1:Iy9bg/ha4yyC70EfRS8jz+B6ybOBKMaSxLj6P6oBDfU= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190313153728-d0100b6bd8b3/go.mod h1:6SW0HCj/g11FgYtHlgUYUwCkIfeOF89ocIRzGO/8vkc= golang.org/x/lint v0.0.0-20200302205851-738671d3881b/go.mod h1:3xt1FjdF8hUf6vQPIChWIBhFzV8gjjsPE/fR3IyQdNY= @@ -330,16 +330,16 @@ golang.org/x/sys v0.0.0-20220722155257-8c9f86f7a55f/go.mod h1:oPkhp1MJrh7nUepCBc golang.org/x/sys v0.1.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.5.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= golang.org/x/sys v0.7.0/go.mod h1:oPkhp1MJrh7nUepCBck5+mAzfO9JrbApNNgaTdGDITg= -golang.org/x/sys v0.16.0 h1:xWw16ngr6ZMtmxDyKyIgsE93KNKz5HKmMa3b8ALHidU= -golang.org/x/sys v0.16.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= +golang.org/x/sys v0.17.0 h1:25cE3gD+tdBA7lp7QfhuV+rJiE9YXTcS3VG1SqssI/Y= +golang.org/x/sys v0.17.0/go.mod h1:/VUhepiaJMQUp4+oa/7Zr1D23ma6VTLIYjOOTFZPUcA= golang.org/x/term v0.0.0-20201126162022-7de9c90e9dd1/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210421210424-b80969c67360/go.mod h1:bj7SfCRtBDWHUb9snDiAeCFNEtKQo2Wmx5Cou7ajbmo= golang.org/x/term v0.0.0-20210927222741-03fcf44c2211/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.1.0/go.mod h1:jbD1KX2456YbFQfuXm/mYQcufACuNUgVhRMnK/tPxf8= golang.org/x/term v0.5.0/go.mod h1:jMB1sMXY+tzblOD4FWmEbocvup2/aLOaQEp7JmGp78k= golang.org/x/term v0.7.0/go.mod h1:P32HKFT3hSsZrRxla30E9HqToFYAQPCMs/zFMBUFqPY= -golang.org/x/term v0.16.0 h1:m+B6fahuftsE9qjo0VWp2FW0mB3MTJvR0BaMQrq0pmE= -golang.org/x/term v0.16.0/go.mod h1:yn7UURbUtPyrVJPGPq404EukNFxcm/foM+bV/bfcDsY= +golang.org/x/term v0.17.0 h1:mkTF7LCd6WGJNL3K1Ad7kwxNfYAW6a8a8QqtMblp/4U= +golang.org/x/term v0.17.0/go.mod h1:lLRBjIVuehSbZlaOtGMbcMncT+aqLLLmKrsjNrUguwk= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.3/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= golang.org/x/text v0.3.6/go.mod h1:5Zoc/QRtKVWzQhOtBMvqHzDpF6irO9z98xDceosuGiQ= From 4d5a18a743867bd73bb95d3936c6f5f803f48036 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 08:18:17 +0100 Subject: [PATCH 041/144] worker: fix error message --- worker/worker.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/worker/worker.go b/worker/worker.go index ade547727..b917960d4 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1400,8 +1400,9 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s } return err }) + elapsed := time.Since(start) if err != nil { - return settings, rhpv3.HostPriceTable{}, time.Since(start), errors.New("host is on a private network") + return settings, rhpv3.HostPriceTable{}, elapsed, err } // fetch the host pricetable @@ -1414,7 +1415,7 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s return nil } }) - return settings, pt, time.Since(start), err + return settings, pt, elapsed, err } // scan: first try From 6b53986acd8792ef16a7c2cb350b406bc2641832 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 12 Feb 2024 08:03:00 +0000 Subject: [PATCH 042/144] build(deps): bump gorm.io/driver/sqlite from 1.5.4 to 1.5.5 Bumps [gorm.io/driver/sqlite](https://github.com/go-gorm/sqlite) from 1.5.4 to 1.5.5. - [Commits](https://github.com/go-gorm/sqlite/compare/v1.5.4...v1.5.5) --- updated-dependencies: - dependency-name: gorm.io/driver/sqlite dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index ff700196a..084e6516a 100644 --- a/go.mod +++ b/go.mod @@ -25,7 +25,7 @@ require ( golang.org/x/term v0.17.0 gopkg.in/yaml.v3 v3.0.1 gorm.io/driver/mysql v1.5.2 - gorm.io/driver/sqlite v1.5.4 + gorm.io/driver/sqlite v1.5.5 gorm.io/gorm v1.25.7 lukechampine.com/frand v1.4.2 ) diff --git a/go.sum b/go.sum index 5e2fa38d9..d5a601d4b 100644 --- a/go.sum +++ b/go.sum @@ -391,8 +391,8 @@ gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gorm.io/driver/mysql v1.5.2 h1:QC2HRskSE75wBuOxe0+iCkyJZ+RqpudsQtqkp+IMuXs= gorm.io/driver/mysql v1.5.2/go.mod h1:pQLhh1Ut/WUAySdTHwBpBv6+JKcj+ua4ZFx1QQTBzb8= -gorm.io/driver/sqlite v1.5.4 h1:IqXwXi8M/ZlPzH/947tn5uik3aYQslP9BVveoax0nV0= -gorm.io/driver/sqlite v1.5.4/go.mod h1:qxAuCol+2r6PannQDpOP1FP6ag3mKi4esLnB/jHed+4= +gorm.io/driver/sqlite v1.5.5 h1:7MDMtUZhV065SilG62E0MquljeArQZNfJnjd9i9gx3E= +gorm.io/driver/sqlite v1.5.5/go.mod h1:6NgQ7sQWAIFsPrJJl1lSNSu2TABh0ZZ/zm5fosATavE= gorm.io/gorm v1.25.2-0.20230530020048-26663ab9bf55/go.mod h1:L4uxeKpfBml98NYqVqwAdmV1a2nBtAec/cf3fpucW/k= gorm.io/gorm v1.25.7 h1:VsD6acwRjz2zFxGO50gPO6AkNs7KKnvfzUjHQhZDz/A= gorm.io/gorm v1.25.7/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= From 279f6206de1f49db4146d5dbfb01c72070016e7f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 11:13:50 +0100 Subject: [PATCH 043/144] stores: add TestUploadObject --- stores/metadata_test.go | 220 ++++++++++++++++++++++++++++++++++++++++ 1 file changed, 220 insertions(+) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 18f34dee4..2161e7081 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4042,3 +4042,223 @@ func TestUpsertSectors(t *testing.T) { } } } + +func TestUploadObject(t *testing.T) { + ss := newTestSQLStore(t, defaultTestSQLStoreConfig) + defer ss.Close() + + minShards, totalShards := 10, 30 + + // create 90 hosts, enough for 3 slabs with 30 each + hks, err := ss.addTestHosts(3 * totalShards) + if err != nil { + t.Fatal(err) + } + + // create one contract each + fcids, _, err := ss.addTestContracts(hks) + if err != nil { + t.Fatal(err) + } + + // create an object + obj := object.Object{ + Key: object.GenerateEncryptionKey(), + } + // add 2 slabs + for i := 0; i < 2; i++ { + obj.Slabs = append(obj.Slabs, object.SlabSlice{ + Offset: 0, + Length: uint32(minShards) * rhpv2.SectorSize, + Slab: object.Slab{ + Key: object.GenerateEncryptionKey(), + MinShards: uint8(minShards), + }, + }) + } + // 30 shards each + for i := 0; i < len(obj.Slabs); i++ { + for j := 0; j < totalShards; j++ { + obj.Slabs[i].Shards = append(obj.Slabs[i].Shards, object.Sector{ + Contracts: map[types.PublicKey][]types.FileContractID{ + hks[i*totalShards+j]: []types.FileContractID{ + fcids[i*totalShards+j], + }, + }, + LatestHost: hks[i*totalShards+j], + Root: frand.Entropy256(), + }) + } + } + + // add the object + _, err = ss.addTestObject("1", obj) + if err != nil { + t.Fatal(err) + } + + // fetch the object + var dbObj dbObject + if err := ss.db.Where("db_bucket_id", 1).Take(&dbObj).Error; err != nil { + t.Fatal(err) + } else if dbObj.ID != 1 { + t.Fatal("unexpected id", dbObj.ID) + } else if dbObj.DBBucketID != 1 { + t.Fatal("bucket id mismatch", dbObj.DBBucketID) + } else if dbObj.ObjectID != "1" { + t.Fatal("object id mismatch", dbObj.ObjectID) + } else if dbObj.Health != 1 { + t.Fatal("health mismatch", dbObj.Health) + } else if dbObj.Size != obj.TotalSize() { + t.Fatal("size mismatch", dbObj.Size) + } + + // fetch its slices + var dbSlices []dbSlice + if err := ss.db.Where("db_object_id", dbObj.ID).Find(&dbSlices).Error; err != nil { + t.Fatal(err) + } + for i, dbSlice := range dbSlices { + if dbSlice.ID != uint(i+1) { + t.Fatal("unexpected id", dbSlice.ID) + } else if dbSlice.ObjectIndex != uint(i+1) { + t.Fatal("unexpected object index", dbSlice.ObjectIndex) + } else if dbSlice.Offset != 0 || dbSlice.Length != uint32(minShards)*rhpv2.SectorSize { + t.Fatal("invalid offset/length", dbSlice.Offset, dbSlice.Length) + } + + // fetch the slab + var dbSlab dbSlab + key, _ := obj.Slabs[i].Key.MarshalBinary() + if err := ss.db.Where("id", dbSlice.DBSlabID).Take(&dbSlab).Error; err != nil { + t.Fatal(err) + } else if dbSlab.ID != uint(i+1) { + t.Fatal("unexpected id", dbSlab.ID) + } else if dbSlab.DBContractSetID != 1 { + t.Fatal("invalid contract set id", dbSlab.DBContractSetID) + } else if dbSlab.Health != 1 { + t.Fatal("invalid health", dbSlab.Health) + } else if dbSlab.HealthValidUntil != 0 { + t.Fatal("invalid health validity", dbSlab.HealthValidUntil) + } else if dbSlab.MinShards != uint8(minShards) { + t.Fatal("invalid minShards", dbSlab.MinShards) + } else if dbSlab.TotalShards != uint8(totalShards) { + t.Fatal("invalid totalShards", dbSlab.TotalShards) + } else if !bytes.Equal(dbSlab.Key, key) { + t.Fatal("wrong key") + } + + // fetch the sectors + var dbSectors []dbSector + if err := ss.db.Where("db_slab_id", dbSlab.ID).Find(&dbSectors).Error; err != nil { + t.Fatal(err) + } + for j, dbSector := range dbSectors { + if dbSector.ID != uint(i*totalShards+j+1) { + t.Fatal("invalid id", dbSector.ID) + } else if dbSector.DBSlabID != dbSlab.ID { + t.Fatal("invalid slab id", dbSector.DBSlabID) + } else if dbSector.LatestHost != publicKey(hks[i*totalShards+j]) { + t.Fatal("invalid host") + } else if !bytes.Equal(dbSector.Root, obj.Slabs[i].Shards[j].Root[:]) { + t.Fatal("invalid root") + } + } + } + + obj2 := object.Object{ + Key: object.GenerateEncryptionKey(), + } + // add 1 slab with 30 shards + obj2.Slabs = append(obj2.Slabs, object.SlabSlice{ + Offset: 0, + Length: uint32(minShards) * rhpv2.SectorSize, + Slab: object.Slab{ + Key: object.GenerateEncryptionKey(), + MinShards: uint8(minShards), + }, + }) + // 30 shards each + for i := 0; i < totalShards; i++ { + obj2.Slabs[0].Shards = append(obj2.Slabs[0].Shards, object.Sector{ + Contracts: map[types.PublicKey][]types.FileContractID{ + hks[len(obj.Slabs)*totalShards+i]: { + fcids[len(obj.Slabs)*totalShards+i], + }, + }, + LatestHost: hks[len(obj.Slabs)*totalShards+i], + Root: frand.Entropy256(), + }) + } + // add the second slab of the first object too + obj2.Slabs = append(obj2.Slabs, obj.Slabs[1]) + + // add the object + _, err = ss.addTestObject("2", obj2) + if err != nil { + t.Fatal(err) + } + + // fetch the object + var dbObj2 dbObject + if err := ss.db.Where("db_bucket_id", 1). + Where("object_id", "2"). + Take(&dbObj2).Error; err != nil { + t.Fatal(err) + } else if dbObj2.ID != 2 { + t.Fatal("unexpected id", dbObj2.ID) + } else if dbObj.Size != obj2.TotalSize() { + t.Fatal("size mismatch", dbObj2.Size) + } + + // fetch its slices + var dbSlices2 []dbSlice + if err := ss.db.Where("db_object_id", dbObj2.ID).Find(&dbSlices2).Error; err != nil { + t.Fatal(err) + } + + // check the first one + dbSlice2 := dbSlices2[0] + if dbSlice2.ID != uint(len(dbSlices)+1) { + t.Fatal("unexpected id", dbSlice2.ID) + } else if dbSlice2.ObjectIndex != uint(1) { + t.Fatal("unexpected object index", dbSlice2.ObjectIndex) + } else if dbSlice2.Offset != 0 || dbSlice2.Length != uint32(minShards)*rhpv2.SectorSize { + t.Fatal("invalid offset/length", dbSlice2.Offset, dbSlice2.Length) + } + + // fetch the slab + var dbSlab2 dbSlab + key, _ := obj2.Slabs[0].Key.MarshalBinary() + if err := ss.db.Where("id", dbSlice2.DBSlabID).Take(&dbSlab2).Error; err != nil { + t.Fatal(err) + } else if dbSlab2.ID != uint(len(dbSlices)+1) { + t.Fatal("unexpected id", dbSlab2.ID) + } else if dbSlab2.DBContractSetID != 1 { + t.Fatal("invalid contract set id", dbSlab2.DBContractSetID) + } else if !bytes.Equal(dbSlab2.Key, key) { + t.Fatal("wrong key") + } + + // fetch the sectors + var dbSectors2 []dbSector + if err := ss.db.Where("db_slab_id", dbSlab2.ID).Find(&dbSectors2).Error; err != nil { + t.Fatal(err) + } + for j, dbSector := range dbSectors2 { + if dbSector.ID != uint((len(obj.Slabs))*totalShards+j+1) { + t.Fatal("invalid id", dbSector.ID) + } else if dbSector.DBSlabID != dbSlab2.ID { + t.Fatal("invalid slab id", dbSector.DBSlabID) + } else if dbSector.LatestHost != publicKey(hks[(len(obj.Slabs))*totalShards+j]) { + t.Fatal("invalid host") + } else if !bytes.Equal(dbSector.Root, obj2.Slabs[0].Shards[j].Root[:]) { + t.Fatal("invalid root") + } + } + + // the second slab of obj2 should be the same as the first in obj + if dbSlices2[1].DBSlabID != 2 { + t.Fatal("wrong slab") + } +} From c9d27dd07f9b70b6b33113a5f7c85a7fe4fa0142 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 11:35:36 +0100 Subject: [PATCH 044/144] stores: extend test to cover contract sectors --- stores/metadata_test.go | 22 ++++++++++++++++++++++ 1 file changed, 22 insertions(+) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 2161e7081..886b4e518 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4117,6 +4117,8 @@ func TestUploadObject(t *testing.T) { var dbSlices []dbSlice if err := ss.db.Where("db_object_id", dbObj.ID).Find(&dbSlices).Error; err != nil { t.Fatal(err) + } else if len(dbSlices) != 2 { + t.Fatal("invalid number of slices", len(dbSlices)) } for i, dbSlice := range dbSlices { if dbSlice.ID != uint(i+1) { @@ -4152,6 +4154,8 @@ func TestUploadObject(t *testing.T) { var dbSectors []dbSector if err := ss.db.Where("db_slab_id", dbSlab.ID).Find(&dbSectors).Error; err != nil { t.Fatal(err) + } else if len(dbSectors) != totalShards { + t.Fatal("invalid number of sectors", len(dbSectors)) } for j, dbSector := range dbSectors { if dbSector.ID != uint(i*totalShards+j+1) { @@ -4215,6 +4219,8 @@ func TestUploadObject(t *testing.T) { var dbSlices2 []dbSlice if err := ss.db.Where("db_object_id", dbObj2.ID).Find(&dbSlices2).Error; err != nil { t.Fatal(err) + } else if len(dbSlices2) != 2 { + t.Fatal("invalid number of slices", len(dbSlices)) } // check the first one @@ -4244,6 +4250,8 @@ func TestUploadObject(t *testing.T) { var dbSectors2 []dbSector if err := ss.db.Where("db_slab_id", dbSlab2.ID).Find(&dbSectors2).Error; err != nil { t.Fatal(err) + } else if len(dbSectors2) != totalShards { + t.Fatal("invalid number of sectors", len(dbSectors2)) } for j, dbSector := range dbSectors2 { if dbSector.ID != uint((len(obj.Slabs))*totalShards+j+1) { @@ -4261,4 +4269,18 @@ func TestUploadObject(t *testing.T) { if dbSlices2[1].DBSlabID != 2 { t.Fatal("wrong slab") } + + var contractSectors []dbContractSector + if err := ss.db.Find(&contractSectors).Error; err != nil { + t.Fatal(err) + } else if len(contractSectors) != 3*totalShards { + t.Fatal("invalid number of contract sectors", len(contractSectors)) + } + for i, cs := range contractSectors { + if cs.DBContractID != uint(i+1) { + t.Fatal("invalid contract id") + } else if cs.DBSectorID != uint(i+1) { + t.Fatal("invalid sector id") + } + } } From 7d75df5e2eb36c72ef9c6f5a598ec8392b9440df Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 11:36:01 +0100 Subject: [PATCH 045/144] stores: rename test --- stores/metadata_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 886b4e518..f56598289 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4043,7 +4043,7 @@ func TestUpsertSectors(t *testing.T) { } } -func TestUploadObject(t *testing.T) { +func TestUpdateObjectReuseSlab(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) defer ss.Close() From 53994889a7db540ea7a216b884ce168eaba2dd4b Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 11:37:55 +0100 Subject: [PATCH 046/144] stores: gofmt --- stores/metadata_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index f56598289..96b06c4ec 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4081,7 +4081,7 @@ func TestUpdateObjectReuseSlab(t *testing.T) { for j := 0; j < totalShards; j++ { obj.Slabs[i].Shards = append(obj.Slabs[i].Shards, object.Sector{ Contracts: map[types.PublicKey][]types.FileContractID{ - hks[i*totalShards+j]: []types.FileContractID{ + hks[i*totalShards+j]: { fcids[i*totalShards+j], }, }, From 122715a24421369a3bf5c3e9bd3c90c6bb0c4bc8 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 11:44:25 +0100 Subject: [PATCH 047/144] worker: fix debug message --- worker/worker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/worker.go b/worker/worker.go index b917960d4..adbce821d 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1428,7 +1428,7 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s } settings, pt, duration, err = scan() if err == nil { - w.logger.Debug("successfully scanned host %v after retry", hostKey) + w.logger.Debugf("successfully scanned host %v after retry", hostKey) } } From 3b851fde7f0f3eb47162cd06bde548fd5bcec010 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 12 Feb 2024 16:58:40 +0100 Subject: [PATCH 048/144] autopilot: increase resolver timeout to 10s --- autopilot/ipfilter.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/autopilot/ipfilter.go b/autopilot/ipfilter.go index 1844955f6..6aa244047 100644 --- a/autopilot/ipfilter.go +++ b/autopilot/ipfilter.go @@ -23,7 +23,7 @@ const ( ipCacheEntryValidity = 24 * time.Hour // resolverLookupTimeout is the timeout we apply when resolving a host's IP address - resolverLookupTimeout = 5 * time.Second + resolverLookupTimeout = 10 * time.Second ) var ( From bbb0590237c9f1393d953c4724051c29000578f6 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 13 Feb 2024 11:44:56 +0100 Subject: [PATCH 049/144] worker: fail upload/download request if uploader/downloader was stopped --- worker/downloader.go | 15 +++++++++++++-- worker/uploader.go | 16 ++++++++++++++-- 2 files changed, 27 insertions(+), 4 deletions(-) diff --git a/worker/downloader.go b/worker/downloader.go index 30c855d80..4a50e75f5 100644 --- a/worker/downloader.go +++ b/worker/downloader.go @@ -20,7 +20,6 @@ const ( type ( downloader struct { - hk types.PublicKey host Host statsDownloadSpeedBytesPerMS *stats.DataPoints // keep track of this separately for stats (no decay is applied) @@ -33,6 +32,7 @@ type ( consecutiveFailures uint64 numDownloads uint64 queue []*sectorDownloadReq + stopped bool } ) @@ -55,6 +55,10 @@ func (d *downloader) PublicKey() types.PublicKey { } func (d *downloader) Stop() { + d.mu.Lock() + d.stopped = true + d.mu.Unlock() + for { download := d.pop() if download == nil { @@ -80,8 +84,15 @@ func (d *downloader) fillBatch() (batch []*sectorDownloadReq) { } func (d *downloader) enqueue(download *sectorDownloadReq) { - // enqueue the job d.mu.Lock() + // check for stopped + if d.stopped { + d.mu.Unlock() + go download.fail(errors.New("downloader stopped")) // don't block the caller + return + } + + // enqueue the job d.queue = append(d.queue, download) d.mu.Unlock() diff --git a/worker/uploader.go b/worker/uploader.go index 3ec88c6fa..e20c4dee4 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -36,6 +36,7 @@ type ( fcid types.FileContractID host Host queue []*sectorUploadReq + stopped bool // stats related field consecutiveFailures uint64 @@ -136,6 +137,10 @@ outer: } func (u *uploader) Stop(err error) { + u.mu.Lock() + u.stopped = true + u.mu.Unlock() + for { upload := u.pop() if upload == nil { @@ -148,12 +153,19 @@ func (u *uploader) Stop(err error) { } func (u *uploader) enqueue(req *sectorUploadReq) { + u.mu.Lock() + // check for stopped + if u.stopped { + u.mu.Unlock() + go req.fail(errors.New("uploader stopped")) // don't block the caller + return + } + // decorate the request - req.fcid = u.ContractID() + req.fcid = u.fcid req.hk = u.hk // enqueue the request - u.mu.Lock() u.queue = append(u.queue, req) u.mu.Unlock() From 49475fa38b57ce4d2df702c9731acc07264b79dd Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 13 Feb 2024 12:01:08 +0100 Subject: [PATCH 050/144] worker: add tests --- worker/downloader.go | 8 ++++++-- worker/downloader_test.go | 32 ++++++++++++++++++++++++++++++++ worker/uploader.go | 6 +++++- worker/uploader_test.go | 32 ++++++++++++++++++++++++++++++++ 4 files changed, 75 insertions(+), 3 deletions(-) create mode 100644 worker/downloader_test.go create mode 100644 worker/uploader_test.go diff --git a/worker/downloader.go b/worker/downloader.go index 4a50e75f5..24be245fc 100644 --- a/worker/downloader.go +++ b/worker/downloader.go @@ -18,6 +18,10 @@ const ( maxConcurrentSectorsPerHost = 3 ) +var ( + errDownloaderStopped = errors.New("downloader was stopped") +) + type ( downloader struct { host Host @@ -65,7 +69,7 @@ func (d *downloader) Stop() { break } if !download.done() { - download.fail(errors.New("downloader stopped")) + download.fail(errDownloaderStopped) } } } @@ -88,7 +92,7 @@ func (d *downloader) enqueue(download *sectorDownloadReq) { // check for stopped if d.stopped { d.mu.Unlock() - go download.fail(errors.New("downloader stopped")) // don't block the caller + go download.fail(errDownloaderStopped) // don't block the caller return } diff --git a/worker/downloader_test.go b/worker/downloader_test.go new file mode 100644 index 000000000..0be4bc701 --- /dev/null +++ b/worker/downloader_test.go @@ -0,0 +1,32 @@ +package worker + +import ( + "errors" + "testing" + "time" +) + +func TestDownloaderStopped(t *testing.T) { + w := newMockWorker() + h := w.addHost() + w.dl.refreshDownloaders(w.contracts()) + + dl := w.dl.downloaders[h.PublicKey()] + dl.Stop() + + req := sectorDownloadReq{ + resps: §orResponses{ + c: make(chan struct{}), + }, + } + dl.enqueue(&req) + + select { + case <-req.resps.c: + if err := req.resps.responses[0].err; !errors.Is(err, errDownloaderStopped) { + t.Fatal("unexpected error response", err) + } + case <-time.After(time.Second): + t.Fatal("no response") + } +} diff --git a/worker/uploader.go b/worker/uploader.go index e20c4dee4..dcff27eaf 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -19,6 +19,10 @@ const ( sectorUploadTimeout = 60 * time.Second ) +var ( + errUploaderStopped = errors.New("uploader was stopped") +) + type ( uploader struct { os ObjectStore @@ -157,7 +161,7 @@ func (u *uploader) enqueue(req *sectorUploadReq) { // check for stopped if u.stopped { u.mu.Unlock() - go req.fail(errors.New("uploader stopped")) // don't block the caller + go req.fail(errUploaderStopped) // don't block the caller return } diff --git a/worker/uploader_test.go b/worker/uploader_test.go new file mode 100644 index 000000000..3afba85cb --- /dev/null +++ b/worker/uploader_test.go @@ -0,0 +1,32 @@ +package worker + +import ( + "context" + "errors" + "testing" + "time" +) + +func TestUploaderStopped(t *testing.T) { + w := newMockWorker() + w.addHost() + w.ul.refreshUploaders(w.contracts(), 1) + + ul := w.ul.uploaders[0] + ul.Stop(errors.New("test")) + + req := sectorUploadReq{ + responseChan: make(chan sectorUploadResp), + sector: §orUpload{ctx: context.Background()}, + } + ul.enqueue(&req) + + select { + case res := <-req.responseChan: + if !errors.Is(res.err, errUploaderStopped) { + t.Fatal("expected error response") + } + case <-time.After(time.Second): + t.Fatal("no response") + } +} From 645f93a186645ef8562da5efb1d443b961d49e5b Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 13 Feb 2024 12:11:10 +0100 Subject: [PATCH 051/144] worker: increase timeout in tests for CI --- worker/downloader_test.go | 2 +- worker/uploader_test.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/worker/downloader_test.go b/worker/downloader_test.go index 0be4bc701..357fc2ee8 100644 --- a/worker/downloader_test.go +++ b/worker/downloader_test.go @@ -26,7 +26,7 @@ func TestDownloaderStopped(t *testing.T) { if err := req.resps.responses[0].err; !errors.Is(err, errDownloaderStopped) { t.Fatal("unexpected error response", err) } - case <-time.After(time.Second): + case <-time.After(10 * time.Second): t.Fatal("no response") } } diff --git a/worker/uploader_test.go b/worker/uploader_test.go index 3afba85cb..7217cbaab 100644 --- a/worker/uploader_test.go +++ b/worker/uploader_test.go @@ -26,7 +26,7 @@ func TestUploaderStopped(t *testing.T) { if !errors.Is(res.err, errUploaderStopped) { t.Fatal("expected error response") } - case <-time.After(time.Second): + case <-time.After(10 * time.Second): t.Fatal("no response") } } From afafc0d73b85296549656bc99da6863d7a2bc091 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 13 Feb 2024 16:39:57 +0100 Subject: [PATCH 052/144] alerts: add pagination to alerts endpoint and add another endpoint to dismiss all alerts at once --- alerts/alerts.go | 32 +++++++++++++++++++++++++++- bus/bus.go | 23 +++++++++++++++----- bus/client/alerts.go | 18 ++++++++++++++-- internal/testing/cluster_test.go | 36 +++++++++++++++++++++++++++++++- 4 files changed, 100 insertions(+), 9 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index 4d6463fa2..c76898c8d 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -37,6 +37,7 @@ type ( Alerter interface { RegisterAlert(_ context.Context, a Alert) error DismissAlerts(_ context.Context, ids ...types.Hash256) error + DismissAllAlerts(_ context.Context) error } // Severity indicates the severity of an alert. @@ -63,6 +64,11 @@ type ( alerts map[types.Hash256]Alert webhookBroadcaster webhooks.Broadcaster } + + AlertsOpts struct { + Offset uint64 + Limit uint64 + } ) // String implements the fmt.Stringer interface. @@ -130,6 +136,17 @@ func (m *Manager) RegisterAlert(ctx context.Context, alert Alert) error { }) } +// DismissAllAlerts implements the Alerter interface. +func (m *Manager) DismissAllAlerts(ctx context.Context) error { + m.mu.Lock() + toDismiss := make([]types.Hash256, 0, len(m.alerts)) + for alertID := range m.alerts { + toDismiss = append(toDismiss, alertID) + } + m.mu.Unlock() + return m.DismissAlerts(ctx, toDismiss...) +} + // DismissAlerts implements the Alerter interface. func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { var dismissed []types.Hash256 @@ -159,10 +176,14 @@ func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error } // Active returns the host's active alerts. -func (m *Manager) Active() []Alert { +func (m *Manager) Active(offset, limit uint64) []Alert { m.mu.Lock() defer m.mu.Unlock() + if offset >= uint64(len(m.alerts)) { + return nil + } + alerts := make([]Alert, 0, len(m.alerts)) for _, a := range m.alerts { alerts = append(alerts, a) @@ -170,6 +191,10 @@ func (m *Manager) Active() []Alert { sort.Slice(alerts, func(i, j int) bool { return alerts[i].Timestamp.After(alerts[j].Timestamp) }) + alerts = alerts[offset:] + if limit < uint64(len(alerts)) { + alerts = alerts[:limit] + } return alerts } @@ -213,6 +238,11 @@ func (a *originAlerter) RegisterAlert(ctx context.Context, alert Alert) error { return a.alerter.RegisterAlert(ctx, alert) } +// DismissAllAlerts implements the Alerter interface. +func (a *originAlerter) DismissAllAlerts(ctx context.Context) error { + return a.alerter.DismissAllAlerts(ctx) +} + // DismissAlerts implements the Alerter interface. func (a *originAlerter) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { return a.alerter.DismissAlerts(ctx, ids...) diff --git a/bus/bus.go b/bus/bus.go index d11550595..82a89826f 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -245,9 +245,10 @@ func (b *bus) Handler() http.Handler { "POST /account/:id/requiressync": b.accountsRequiresSyncHandlerPOST, "POST /account/:id/resetdrift": b.accountsResetDriftHandlerPOST, - "GET /alerts": b.handleGETAlerts, - "POST /alerts/dismiss": b.handlePOSTAlertsDismiss, - "POST /alerts/register": b.handlePOSTAlertsRegister, + "GET /alerts": b.handleGETAlerts, + "POST /alerts/dismiss": b.handlePOSTAlertsDismiss, + "POST /alerts/dismissall": b.handlePOSTAlertsDismissAll, + "POST /alerts/register": b.handlePOSTAlertsRegister, "GET /autopilots": b.autopilotsListHandlerGET, "GET /autopilot/:id": b.autopilotsHandlerGET, @@ -1711,8 +1712,16 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { }, nil } -func (b *bus) handleGETAlerts(c jape.Context) { - c.Encode(b.alertMgr.Active()) +func (b *bus) handleGETAlerts(jc jape.Context) { + var offset, limit uint64 + if jc.DecodeForm("offset", &offset) != nil { + return + } else if jc.DecodeForm("limit", &limit) != nil { + return + } else if limit == 0 { + limit = math.MaxUint64 + } + jc.Encode(b.alertMgr.Active(offset, limit)) } func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { @@ -1723,6 +1732,10 @@ func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { jc.Check("failed to dismiss alerts", b.alertMgr.DismissAlerts(jc.Request.Context(), ids...)) } +func (b *bus) handlePOSTAlertsDismissAll(jc jape.Context) { + jc.Check("failed to dismiss alerts", b.alertMgr.DismissAllAlerts(jc.Request.Context())) +} + func (b *bus) handlePOSTAlertsRegister(jc jape.Context) { var alert alerts.Alert if jc.Decode(&alert) != nil { diff --git a/bus/client/alerts.go b/bus/client/alerts.go index 6af68c78d..ab1d7f094 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -2,17 +2,31 @@ package client import ( "context" + "fmt" + "net/url" "go.sia.tech/core/types" "go.sia.tech/renterd/alerts" ) // Alerts fetches the active alerts from the bus. -func (c *Client) Alerts() (alerts []alerts.Alert, err error) { - err = c.c.GET("/alerts", &alerts) +func (c *Client) Alerts(opts alerts.AlertsOpts) (alerts []alerts.Alert, err error) { + values := url.Values{} + if opts.Offset > 0 { + values.Set("offset", fmt.Sprint(opts.Offset)) + } + if opts.Limit != 0 { + values.Set("limit", fmt.Sprint(opts.Limit)) + } + err = c.c.GET("/alerts?"+values.Encode(), &alerts) return } +// DismissAllAlerts dimisses all alerts. +func (c *Client) DismissAllAlerts(ctx context.Context) error { + return c.c.WithContext(ctx).POST("/alerts/dismissall", nil, nil) +} + // DismissAlerts dimisses the alerts with the given IDs. func (c *Client) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { return c.c.WithContext(ctx).POST("/alerts/dismiss", ids, nil) diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index b0de2946e..cd26b6519 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1915,7 +1915,7 @@ func TestAlerts(t *testing.T) { tt.OK(b.RegisterAlert(context.Background(), alert)) findAlert := func(id types.Hash256) *alerts.Alert { t.Helper() - alerts, err := b.Alerts() + alerts, err := b.Alerts(alerts.AlertsOpts{}) tt.OK(err) for _, alert := range alerts { if alert.ID == id { @@ -1938,6 +1938,40 @@ func TestAlerts(t *testing.T) { if foundAlert != nil { t.Fatal("alert found") } + + // register 2 alerts + alert2 := alert + alert2.ID = frand.Entropy256() + alert2.Timestamp = time.Now().Add(time.Second) + tt.OK(b.RegisterAlert(context.Background(), alert)) + tt.OK(b.RegisterAlert(context.Background(), alert2)) + if foundAlert := findAlert(alert.ID); foundAlert == nil { + t.Fatal("alert not found") + } else if foundAlert := findAlert(alert2.ID); foundAlert == nil { + t.Fatal("alert not found") + } + + // try to find with offset = 1 + foundAlerts, err := b.Alerts(alerts.AlertsOpts{Offset: 1}) + tt.OK(err) + if len(foundAlerts) != 1 || foundAlerts[0].ID != alert.ID { + t.Fatal("wrong alert") + } + + // try to find with limit = 1 + foundAlerts, err = b.Alerts(alerts.AlertsOpts{Limit: 1}) + tt.OK(err) + if len(foundAlerts) != 1 || foundAlerts[0].ID != alert2.ID { + t.Fatal("wrong alert") + } + + // dismiss all + tt.OK(b.DismissAllAlerts(context.Background())) + foundAlerts, err = b.Alerts(alerts.AlertsOpts{}) + tt.OK(err) + if len(foundAlerts) != 0 { + t.Fatal("expected 0 alerts", len(foundAlerts)) + } } func TestMultipartUploads(t *testing.T) { From 87de90414b6ef72383d4e0ec3a5b018686e83e19 Mon Sep 17 00:00:00 2001 From: Nate Maninger Date: Tue, 13 Feb 2024 14:05:23 -0800 Subject: [PATCH 053/144] worker: remove context value interaction recorder --- worker/host.go | 4 +++- worker/interactions.go | 8 -------- worker/worker.go | 4 ++-- 3 files changed, 5 insertions(+), 11 deletions(-) diff --git a/worker/host.go b/worker/host.go index fcaac0cf1..2aa97d57a 100644 --- a/worker/host.go +++ b/worker/host.go @@ -55,6 +55,7 @@ type ( logger *zap.SugaredLogger transportPool *transportPoolV3 priceTables *priceTables + interactionRecorder HostInteractionRecorder } ) @@ -76,6 +77,7 @@ func (w *worker) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr accountKey: w.accounts.deriveAccountKey(hk), transportPool: w.transportPoolV3, priceTables: w.priceTables, + interactionRecorder: w.hostInteractionRecorder, } } @@ -196,7 +198,7 @@ func (h *host) FetchPriceTable(ctx context.Context, rev *types.FileContractRevis fetchPT := func(paymentFn PriceTablePaymentFunc) (hpt hostdb.HostPriceTable, err error) { err = h.transportPool.withTransportV3(ctx, h.hk, h.siamuxAddr, func(ctx context.Context, t *transportV3) (err error) { hpt, err = RPCPriceTable(ctx, t, paymentFn) - HostInteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ + h.interactionRecorder.RecordPriceTableUpdate(hostdb.PriceTableUpdate{ HostKey: h.hk, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), diff --git a/worker/interactions.go b/worker/interactions.go index 70629c1f0..a011cc0d4 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -42,14 +42,6 @@ var ( _ HostInteractionRecorder = (*hostInteractionRecorder)(nil) ) -func HostInteractionRecorderFromContext(ctx context.Context) HostInteractionRecorder { - ir, ok := ctx.Value(keyInteractionRecorder).(HostInteractionRecorder) - if !ok { - panic("no interaction recorder attached to the context") // developer error - } - return ir -} - func interactionMiddleware(ir HostInteractionRecorder, routes map[string]jape.Handler) map[string]jape.Handler { for route, handler := range routes { routes[route] = jape.Adapt(func(h http.Handler) http.Handler { diff --git a/worker/worker.go b/worker/worker.go index adbce821d..863c8e27d 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -342,7 +342,7 @@ func (w *worker) rhpPriceTableHandler(jc jape.Context) { var err error var hpt hostdb.HostPriceTable defer func() { - HostInteractionRecorderFromContext(ctx).RecordPriceTableUpdate(hostdb.PriceTableUpdate{ + w.hostInteractionRecorder.RecordPriceTableUpdate(hostdb.PriceTableUpdate{ HostKey: rptr.HostKey, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), @@ -1442,7 +1442,7 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s } // record host scan - HostInteractionRecorderFromContext(ctx).RecordHostScan(hostdb.HostScan{ + w.hostInteractionRecorder.RecordHostScan(hostdb.HostScan{ HostKey: hostKey, Success: isSuccessfulInteraction(err), Timestamp: time.Now(), From fee0b6ca788ce82e8173862ec2463f962bc1396a Mon Sep 17 00:00:00 2001 From: PJ Date: Wed, 14 Feb 2024 09:19:19 +0100 Subject: [PATCH 054/144] worker: ensure applying MigrationSurchargeMultiplier never prevents a download --- api/setting.go | 5 +++++ worker/gouging.go | 9 ++++----- 2 files changed, 9 insertions(+), 5 deletions(-) diff --git a/api/setting.go b/api/setting.go index d11089010..348b3a62d 100644 --- a/api/setting.go +++ b/api/setting.go @@ -112,6 +112,11 @@ func (gs GougingSettings) Validate() error { if gs.MinPriceTableValidity < 10*time.Second { return errors.New("MinPriceTableValidity must be at least 10 seconds") } + _, overflow := gs.MaxDownloadPrice.Mul64WithOverflow(gs.MigrationSurchargeMultiplier) + if overflow { + maxMultiplier := types.MaxCurrency.Div(gs.MaxDownloadPrice).Big().Uint64() + return fmt.Errorf("MigrationSurchargeMultiplier must be less than %v, otherwise applying it to MaxDownloadPrice overflows the currency type", maxMultiplier) + } return nil } diff --git a/worker/gouging.go b/worker/gouging.go index 5e77c3053..36963e24a 100644 --- a/worker/gouging.go +++ b/worker/gouging.go @@ -72,17 +72,16 @@ func WithGougingChecker(ctx context.Context, cs consensusState, gp api.GougingPa // adjust the max download price if we are dealing with a critical // migration that might be failing due to gouging checks + settings := gp.GougingSettings if criticalMigration && gp.GougingSettings.MigrationSurchargeMultiplier > 0 { - if adjustedMaxDownloadPrice, overflow := gp.GougingSettings.MaxDownloadPrice.Mul64WithOverflow(gp.GougingSettings.MigrationSurchargeMultiplier); overflow { - return gougingChecker{}, errors.New("failed to apply the 'MigrationSurchargeMultiplier', overflow detected") - } else { - gp.GougingSettings.MaxDownloadPrice = adjustedMaxDownloadPrice + if adjustedMaxDownloadPrice, overflow := gp.GougingSettings.MaxDownloadPrice.Mul64WithOverflow(gp.GougingSettings.MigrationSurchargeMultiplier); !overflow { + settings.MaxDownloadPrice = adjustedMaxDownloadPrice } } return gougingChecker{ consensusState: consensusState, - settings: gp.GougingSettings, + settings: settings, txFee: gp.TransactionFee, // NOTE: From 3ca11cb858615f443f6c564e1c79dbb7b05c225d Mon Sep 17 00:00:00 2001 From: PJ Date: Wed, 14 Feb 2024 09:33:19 +0100 Subject: [PATCH 055/144] worker: get rid of interaction middleware --- worker/host.go | 4 ++-- worker/interactions.go | 14 -------------- worker/worker.go | 4 ++-- 3 files changed, 4 insertions(+), 18 deletions(-) diff --git a/worker/host.go b/worker/host.go index 2aa97d57a..86e92ce27 100644 --- a/worker/host.go +++ b/worker/host.go @@ -52,10 +52,10 @@ type ( acc *account bus Bus contractSpendingRecorder ContractSpendingRecorder + interactionRecorder HostInteractionRecorder logger *zap.SugaredLogger transportPool *transportPoolV3 priceTables *priceTables - interactionRecorder HostInteractionRecorder } ) @@ -70,6 +70,7 @@ func (w *worker) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr acc: w.accounts.ForHost(hk), bus: w.bus, contractSpendingRecorder: w.contractSpendingRecorder, + interactionRecorder: w.hostInteractionRecorder, logger: w.logger.Named(hk.String()[:4]), fcid: fcid, siamuxAddr: siamuxAddr, @@ -77,7 +78,6 @@ func (w *worker) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr accountKey: w.accounts.deriveAccountKey(hk), transportPool: w.transportPoolV3, priceTables: w.priceTables, - interactionRecorder: w.hostInteractionRecorder, } } diff --git a/worker/interactions.go b/worker/interactions.go index a011cc0d4..dfe8c4017 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -3,11 +3,9 @@ package worker import ( "context" "fmt" - "net/http" "sync" "time" - "go.sia.tech/jape" "go.sia.tech/renterd/hostdb" "go.uber.org/zap" ) @@ -42,18 +40,6 @@ var ( _ HostInteractionRecorder = (*hostInteractionRecorder)(nil) ) -func interactionMiddleware(ir HostInteractionRecorder, routes map[string]jape.Handler) map[string]jape.Handler { - for route, handler := range routes { - routes[route] = jape.Adapt(func(h http.Handler) http.Handler { - return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) { - ctx := context.WithValue(r.Context(), keyInteractionRecorder, ir) - h.ServeHTTP(w, r.WithContext(ctx)) - }) - })(handler) - } - return routes -} - func (w *worker) initHostInteractionRecorder(flushInterval time.Duration) { if w.hostInteractionRecorder != nil { panic("HostInteractionRecorder already initialized") // developer error diff --git a/worker/worker.go b/worker/worker.go index 863c8e27d..8a2a9b1f3 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1323,7 +1323,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(interactionMiddleware(w.hostInteractionRecorder, map[string]jape.Handler{ + return jape.Mux(map[string]jape.Handler{ "GET /account/:hostkey": w.accountHandlerGET, "GET /id": w.idHandlerGET, @@ -1351,7 +1351,7 @@ func (w *worker) Handler() http.Handler { "PUT /multipart/*path": w.multipartUploadHandlerPUT, "GET /state": w.stateHandlerGET, - })) + }) } // Shutdown shuts down the worker. From d90e42f0410ed6c3a1f4e09291e5f87e70c91723 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 14 Feb 2024 11:59:49 +0100 Subject: [PATCH 056/144] stores: allow for filtering object stats by bucket --- api/object.go | 4 +++ bus/bus.go | 8 +++-- bus/client/objects.go | 8 +++-- internal/testing/cluster_test.go | 57 +++++++++++++++++--------------- stores/metadata.go | 29 ++++++++++++++-- stores/metadata_test.go | 48 ++++++++++++++++++--------- 6 files changed, 105 insertions(+), 49 deletions(-) diff --git a/api/object.go b/api/object.go index 73bb9c45c..e4fd4b465 100644 --- a/api/object.go +++ b/api/object.go @@ -119,6 +119,10 @@ type ( Mode string `json:"mode"` } + ObjectsStatsOpts struct { + Bucket string + } + // ObjectsStatsResponse is the response type for the /bus/stats/objects endpoint. ObjectsStatsResponse struct { NumObjects uint64 `json:"numObjects"` // number of objects diff --git a/bus/bus.go b/bus/bus.go index d11550595..10c610296 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -139,7 +139,7 @@ type ( Object(ctx context.Context, bucketName, path string) (api.Object, error) ObjectEntries(ctx context.Context, bucketName, path, prefix, sortBy, sortDir, marker string, offset, limit int) ([]api.ObjectMetadata, bool, error) ObjectsBySlabKey(ctx context.Context, bucketName string, slabKey object.EncryptionKey) ([]api.ObjectMetadata, error) - ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, error) + ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) (api.ObjectsStatsResponse, error) RemoveObject(ctx context.Context, bucketName, path string) error RemoveObjects(ctx context.Context, bucketName, prefix string) error RenameObject(ctx context.Context, bucketName, from, to string, force bool) error @@ -1348,7 +1348,11 @@ func (b *bus) slabbuffersHandlerGET(jc jape.Context) { } func (b *bus) objectsStatshandlerGET(jc jape.Context) { - info, err := b.ms.ObjectsStats(jc.Request.Context()) + opts := api.ObjectsStatsOpts{} + if jc.DecodeForm(("bucket"), &opts.Bucket) != nil { + return + } + info, err := b.ms.ObjectsStats(jc.Request.Context(), opts) if jc.Check("couldn't get objects stats", err) != nil { return } diff --git a/bus/client/objects.go b/bus/client/objects.go index 38a7b14cd..23011a9ba 100644 --- a/bus/client/objects.go +++ b/bus/client/objects.go @@ -82,8 +82,12 @@ func (c *Client) ObjectsBySlabKey(ctx context.Context, bucket string, key object } // ObjectsStats returns information about the number of objects and their size. -func (c *Client) ObjectsStats() (osr api.ObjectsStatsResponse, err error) { - err = c.c.GET("/stats/objects", &osr) +func (c *Client) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) (osr api.ObjectsStatsResponse, err error) { + values := url.Values{} + if opts.Bucket != "" { + values.Set("bucket", opts.Bucket) + } + err = c.c.WithContext(ctx).GET("/stats/objects?"+values.Encode(), &osr) return } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index b0de2946e..13903115d 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -697,24 +697,29 @@ func TestUploadDownloadExtended(t *testing.T) { } // check objects stats. - info, err := cluster.Bus.ObjectsStats() - tt.OK(err) - objectsSize := uint64(len(file1) + len(file2) + len(small) + len(large)) - if info.TotalObjectsSize != objectsSize { - t.Error("wrong size", info.TotalObjectsSize, objectsSize) - } - sectorsSize := 15 * rhpv2.SectorSize - if info.TotalSectorsSize != uint64(sectorsSize) { - t.Error("wrong size", info.TotalSectorsSize, sectorsSize) - } - if info.TotalUploadedSize != uint64(sectorsSize) { - t.Error("wrong size", info.TotalUploadedSize, sectorsSize) - } - if info.NumObjects != 4 { - t.Error("wrong number of objects", info.NumObjects, 4) - } - if info.MinHealth != 1 { - t.Errorf("expected minHealth of 1, got %v", info.MinHealth) + for _, opts := range []api.ObjectsStatsOpts{ + {}, // any bucket + {Bucket: api.DefaultBucketName}, // specific bucket + } { + info, err := cluster.Bus.ObjectsStats(context.Background(), opts) + tt.OK(err) + objectsSize := uint64(len(file1) + len(file2) + len(small) + len(large)) + if info.TotalObjectsSize != objectsSize { + t.Error("wrong size", info.TotalObjectsSize, objectsSize) + } + sectorsSize := 15 * rhpv2.SectorSize + if info.TotalSectorsSize != uint64(sectorsSize) { + t.Error("wrong size", info.TotalSectorsSize, sectorsSize) + } + if info.TotalUploadedSize != uint64(sectorsSize) { + t.Error("wrong size", info.TotalUploadedSize, sectorsSize) + } + if info.NumObjects != 4 { + t.Error("wrong number of objects", info.NumObjects, 4) + } + if info.MinHealth != 1 { + t.Errorf("expected minHealth of 1, got %v", info.MinHealth) + } } // download the data @@ -1633,7 +1638,7 @@ func TestUploadPacking(t *testing.T) { download("file4", data4, 0, int64(len(data4))) // assert number of objects - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.NumObjects != 5 { t.Fatalf("expected 5 objects, got %v", os.NumObjects) @@ -1642,7 +1647,7 @@ func TestUploadPacking(t *testing.T) { // check the object size stats, we use a retry loop since packed slabs are // uploaded in a separate goroutine, so the object stats might lag a bit tt.Retry(60, time.Second, func() error { - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) if err != nil { t.Fatal(err) } @@ -1796,7 +1801,7 @@ func TestSlabBufferStats(t *testing.T) { tt.OKAll(w.UploadObject(context.Background(), bytes.NewReader(data1), api.DefaultBucketName, "1", api.UploadObjectOptions{})) // assert number of objects - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.NumObjects != 1 { t.Fatalf("expected 1 object, got %d", os.NumObjects) @@ -1805,7 +1810,7 @@ func TestSlabBufferStats(t *testing.T) { // check the object size stats, we use a retry loop since packed slabs are // uploaded in a separate goroutine, so the object stats might lag a bit tt.Retry(60, time.Second, func() error { - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) if err != nil { t.Fatal(err) } @@ -1853,7 +1858,7 @@ func TestSlabBufferStats(t *testing.T) { tt.OKAll(w.UploadObject(context.Background(), bytes.NewReader(data2), api.DefaultBucketName, "2", api.UploadObjectOptions{})) // assert number of objects - os, err = b.ObjectsStats() + os, err = b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.NumObjects != 2 { t.Fatalf("expected 1 object, got %d", os.NumObjects) @@ -1862,7 +1867,7 @@ func TestSlabBufferStats(t *testing.T) { // check the object size stats, we use a retry loop since packed slabs are // uploaded in a separate goroutine, so the object stats might lag a bit tt.Retry(60, time.Second, func() error { - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.TotalObjectsSize != uint64(len(data1)+len(data2)) { return fmt.Errorf("expected totalObjectSize of %d, got %d", len(data1)+len(data2), os.TotalObjectsSize) @@ -2006,7 +2011,7 @@ func TestMultipartUploads(t *testing.T) { } // Check objects stats. - os, err := b.ObjectsStats() + os, err := b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.NumObjects != 0 { t.Fatalf("expected 0 object, got %v", os.NumObjects) @@ -2065,7 +2070,7 @@ func TestMultipartUploads(t *testing.T) { } // Check objects stats. - os, err = b.ObjectsStats() + os, err = b.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) tt.OK(err) if os.NumObjects != 1 { t.Fatalf("expected 1 object, got %v", os.NumObjects) diff --git a/stores/metadata.go b/stores/metadata.go index f20f7dbb0..68947ed95 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -582,7 +582,15 @@ func (s *SQLStore) ListBuckets(ctx context.Context) ([]api.Bucket, error) { // ObjectsStats returns some info related to the objects stored in the store. To // 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) { +func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) (api.ObjectsStatsResponse, error) { + // if no bucket is specified, we consider all objects + whereBucket := func(table string) clause.Expr { + if opts.Bucket == "" { + return exprTRUE + } + return sqlWhereBucket(table, opts.Bucket) + } + // number of objects var objInfo struct { NumObjects uint64 @@ -592,6 +600,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, err := s.db. Model(&dbObject{}). Select("COUNT(*) AS NumObjects, COALESCE(MIN(health), 1) as MinHealth, SUM(size) AS TotalObjectsSize"). + Where(whereBucket(dbObject{}.TableName())). Scan(&objInfo). Error if err != nil { @@ -603,6 +612,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, err = s.db. Model(&dbMultipartUpload{}). Select("COUNT(*)"). + Where(whereBucket(dbMultipartUpload{}.TableName())). Scan(&unfinishedObjects). Error if err != nil { @@ -613,13 +623,26 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, var totalUnfinishedObjectsSize uint64 err = s.db. Model(&dbMultipartPart{}). + Joins("INNER JOIN multipart_uploads mu ON multipart_parts.db_multipart_upload_id = mu.id"). Select("COALESCE(SUM(size), 0)"). + Where(whereBucket("mu")). Scan(&totalUnfinishedObjectsSize). Error if err != nil { return api.ObjectsStatsResponse{}, err } + fromContractSectors := gorm.Expr("contract_sectors cs") + if opts.Bucket != "" { + fromContractSectors = gorm.Expr(` + contract_sectors cs + INNER JOIN sectors s ON s.id = cs.db_sector_id + INNER JOIN slabs sla ON sla.id = s.db_slab_id + INNER JOIN slices sli ON sli.db_slab_id = sla.id + INNER JOIN objects o ON o.id = sli.db_object_id AND (?) + `, whereBucket("o")) + } + var totalSectors uint64 batchSize := 500000 @@ -631,7 +654,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, } res := s.db. Model(&dbSector{}). - Raw("SELECT COUNT(*) as Sectors, MAX(sectors.db_sector_id) as Marker FROM (SELECT cs.db_sector_id FROM contract_sectors cs WHERE cs.db_sector_id > ? GROUP BY cs.db_sector_id LIMIT ?) sectors", marker, batchSize). + Raw("SELECT COUNT(*) as Sectors, MAX(sectors.db_sector_id) as Marker FROM (SELECT cs.db_sector_id FROM ? WHERE cs.db_sector_id > ? GROUP BY cs.db_sector_id LIMIT ?) sectors", fromContractSectors, marker, batchSize). Scan(&result) if err := res.Error; err != nil { return api.ObjectsStatsResponse{}, err @@ -644,7 +667,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context) (api.ObjectsStatsResponse, var totalUploaded int64 err = s.db. - Model(&dbContractSector{}). + Table("?", fromContractSectors). Count(&totalUploaded). Error if err != nil { diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 96b06c4ec..a05c0be17 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -2435,7 +2435,7 @@ func TestObjectsStats(t *testing.T) { defer ss.Close() // Fetch stats on clean database. - info, err := ss.ObjectsStats(context.Background()) + info, err := ss.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) if err != nil { t.Fatal(err) } @@ -2499,21 +2499,37 @@ func TestObjectsStats(t *testing.T) { } // Check sizes. - info, err = ss.ObjectsStats(context.Background()) - if err != nil { - t.Fatal(err) - } - if info.TotalObjectsSize != objectsSize { - t.Fatal("wrong size", info.TotalObjectsSize, objectsSize) - } - if info.TotalSectorsSize != sectorsSize { - t.Fatal("wrong size", info.TotalSectorsSize, sectorsSize) - } - if info.TotalUploadedSize != sectorsSize*2 { - t.Fatal("wrong size", info.TotalUploadedSize, sectorsSize*2) + for _, opts := range []api.ObjectsStatsOpts{ + {}, // any bucket + {Bucket: api.DefaultBucketName}, // specific bucket + } { + info, err = ss.ObjectsStats(context.Background(), opts) + if err != nil { + t.Fatal(err) + } else if info.TotalObjectsSize != objectsSize { + t.Fatal("wrong size", info.TotalObjectsSize, objectsSize) + } else if info.TotalSectorsSize != sectorsSize { + t.Fatal("wrong size", info.TotalSectorsSize, sectorsSize) + } else if info.TotalUploadedSize != sectorsSize*2 { + t.Fatal("wrong size", info.TotalUploadedSize, sectorsSize*2) + } else if info.NumObjects != 2 { + t.Fatal("wrong number of objects", info.NumObjects, 2) + } } - if info.NumObjects != 2 { - t.Fatal("wrong number of objects", info.NumObjects, 2) + + // Check other bucket. + if err := ss.CreateBucket(context.Background(), "other", api.BucketPolicy{}); err != nil { + t.Fatal(err) + } else if info, err := ss.ObjectsStats(context.Background(), api.ObjectsStatsOpts{Bucket: "other"}); err != nil { + t.Fatal(err) + } else if info.TotalObjectsSize != 0 { + t.Fatal("wrong size", info.TotalObjectsSize) + } else if info.TotalSectorsSize != 0 { + t.Fatal("wrong size", info.TotalSectorsSize) + } else if info.TotalUploadedSize != 0 { + t.Fatal("wrong size", info.TotalUploadedSize) + } else if info.NumObjects != 0 { + t.Fatal("wrong number of objects", info.NumObjects) } } @@ -2908,7 +2924,7 @@ func TestContractSizes(t *testing.T) { } // assert there's two objects - s, err := ss.ObjectsStats(context.Background()) + s, err := ss.ObjectsStats(context.Background(), api.ObjectsStatsOpts{}) if err != nil { t.Fatal(err) } From b22e96e8987da028ec75cc3a0ccf27d815d50ae4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 14 Feb 2024 13:10:29 +0100 Subject: [PATCH 057/144] bus: address comments --- alerts/alerts.go | 12 +++++++----- bus/bus.go | 7 ++++--- bus/client/alerts.go | 2 +- 3 files changed, 12 insertions(+), 9 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index c76898c8d..424196d4f 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -66,8 +66,8 @@ type ( } AlertsOpts struct { - Offset uint64 - Limit uint64 + Offset int + Limit int } ) @@ -176,12 +176,14 @@ func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error } // Active returns the host's active alerts. -func (m *Manager) Active(offset, limit uint64) []Alert { +func (m *Manager) Active(offset, limit int) []Alert { m.mu.Lock() defer m.mu.Unlock() - if offset >= uint64(len(m.alerts)) { + if offset >= len(m.alerts) { return nil + } else if limit == -1 { + limit = len(m.alerts) } alerts := make([]Alert, 0, len(m.alerts)) @@ -192,7 +194,7 @@ func (m *Manager) Active(offset, limit uint64) []Alert { return alerts[i].Timestamp.After(alerts[j].Timestamp) }) alerts = alerts[offset:] - if limit < uint64(len(alerts)) { + if limit < len(alerts) { alerts = alerts[:limit] } return alerts diff --git a/bus/bus.go b/bus/bus.go index 82a89826f..4342e493b 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1713,13 +1713,14 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { } func (b *bus) handleGETAlerts(jc jape.Context) { - var offset, limit uint64 + offset, limit := 0, -1 if jc.DecodeForm("offset", &offset) != nil { return } else if jc.DecodeForm("limit", &limit) != nil { return - } else if limit == 0 { - limit = math.MaxUint64 + } else if offset < 0 { + jc.Error(errors.New("offset must be non-negative"), http.StatusBadRequest) + return } jc.Encode(b.alertMgr.Active(offset, limit)) } diff --git a/bus/client/alerts.go b/bus/client/alerts.go index ab1d7f094..6151db598 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -12,7 +12,7 @@ import ( // Alerts fetches the active alerts from the bus. func (c *Client) Alerts(opts alerts.AlertsOpts) (alerts []alerts.Alert, err error) { values := url.Values{} - if opts.Offset > 0 { + if opts.Offset != 0 { values.Set("offset", fmt.Sprint(opts.Offset)) } if opts.Limit != 0 { From dff33cd61a457e553b40744ddc644860a5d6f211 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 14 Feb 2024 13:15:41 +0100 Subject: [PATCH 058/144] bus: fix jape --- bus/bus.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bus/bus.go b/bus/bus.go index 10c610296..5c3aea504 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1349,7 +1349,7 @@ func (b *bus) slabbuffersHandlerGET(jc jape.Context) { func (b *bus) objectsStatshandlerGET(jc jape.Context) { opts := api.ObjectsStatsOpts{} - if jc.DecodeForm(("bucket"), &opts.Bucket) != nil { + if jc.DecodeForm("bucket", &opts.Bucket) != nil { return } info, err := b.ms.ObjectsStats(jc.Request.Context(), opts) From ea173eb3cc3beac3a1170260995c5f5d59faf99d Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 14 Feb 2024 16:08:48 +0100 Subject: [PATCH 059/144] stores: binary currency type --- stores/metadata_test.go | 61 +++++++++++++++++++++++++++++++++++++++++ stores/types.go | 28 +++++++++++++++++++ 2 files changed, 89 insertions(+) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 96b06c4ec..0b0a696a5 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -6,8 +6,10 @@ import ( "encoding/hex" "errors" "fmt" + "math" "os" "reflect" + "sort" "strings" "testing" "time" @@ -4284,3 +4286,62 @@ func TestUpdateObjectReuseSlab(t *testing.T) { } } } + +func TestTypeCurrency(t *testing.T) { + ss := newTestSQLStore(t, defaultTestSQLStoreConfig) + defer ss.Close() + + tests := []struct { + a types.Currency + b types.Currency + cmp string + }{ + { + a: types.ZeroCurrency, + b: types.NewCurrency64(1), + cmp: "<", + }, + { + a: types.NewCurrency64(1), + b: types.NewCurrency64(1), + cmp: "=", + }, + { + a: types.NewCurrency(0, math.MaxUint64), + b: types.NewCurrency(math.MaxUint64, 0), + cmp: "<", + }, + { + a: types.NewCurrency(math.MaxUint64, 0), + b: types.NewCurrency(0, math.MaxUint64), + cmp: ">", + }, + } + for _, test := range tests { + var result bool + err := ss.db.Raw("SELECT ? "+test.cmp+" ?", bCurrency(test.a), bCurrency(test.b)).Scan(&result).Error + if err != nil { + t.Fatal(err) + } else if !result { + t.Fatal("unexpected result", result) + } + } + + c := func(c uint64) bCurrency { + return bCurrency(types.NewCurrency64(c)) + } + + var currencies []bCurrency + err := ss.db.Raw(` +WITH input(col) as +(values (?),(?),(?)) +SELECT * FROM input ORDER BY col ASC +`, c(3), c(1), c(2)).Scan(¤cies).Error + if err != nil { + t.Fatal(err) + } else if !sort.SliceIsSorted(currencies, func(i, j int) bool { + return types.Currency(currencies[i]).Cmp(types.Currency(currencies[j])) < 0 + }) { + t.Fatal("currencies not sorted", currencies) + } +} diff --git a/stores/types.go b/stores/types.go index 6b74f7563..9a7c72009 100644 --- a/stores/types.go +++ b/stores/types.go @@ -2,6 +2,7 @@ package stores import ( "database/sql/driver" + "encoding/binary" "encoding/json" "errors" "fmt" @@ -25,6 +26,7 @@ type ( unixTimeMS time.Time datetime time.Time currency types.Currency + bCurrency types.Currency fileContractID types.FileContractID hash256 types.Hash256 publicKey types.PublicKey @@ -338,3 +340,29 @@ func (u *unsigned64) Scan(value interface{}) error { func (u unsigned64) Value() (driver.Value, error) { return int64(u), nil } + +func (bCurrency) GormDataType() string { + return "bytes" +} + +// Scan implements the sql.Scanner interface. +func (sc *bCurrency) Scan(src any) error { + buf, ok := src.([]byte) + if !ok { + return fmt.Errorf("cannot scan %T to Currency", src) + } else if len(buf) != 16 { + return fmt.Errorf("cannot scan %d bytes to Currency", len(buf)) + } + + sc.Lo = binary.LittleEndian.Uint64(buf[:8]) + sc.Hi = binary.LittleEndian.Uint64(buf[8:]) + return nil +} + +// Value implements the driver.Valuer interface. +func (sc bCurrency) Value() (driver.Value, error) { + buf := make([]byte, 16) + binary.LittleEndian.PutUint64(buf[:8], sc.Lo) + binary.LittleEndian.PutUint64(buf[8:], sc.Hi) + return buf, nil +} From fe79e0759a22ffe9a81c8262a35fa70aaeb36455 Mon Sep 17 00:00:00 2001 From: ChrisSchinnerl Date: Thu, 15 Feb 2024 00:08:47 +0000 Subject: [PATCH 060/144] ui: v0.45.0 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 084e6516a..2659a1ceb 100644 --- a/go.mod +++ b/go.mod @@ -19,7 +19,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.44.0 + go.sia.tech/web/renterd v0.45.0 go.uber.org/zap v1.26.0 golang.org/x/crypto v0.19.0 golang.org/x/term v0.17.0 diff --git a/go.sum b/go.sum index d5a601d4b..b7f5f9cc7 100644 --- a/go.sum +++ b/go.sum @@ -253,8 +253,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.44.0 h1:yKu1Kq/6ssV9Vbv4oa+sn2Pc2TNyfcrv/mRPNOuYuB0= -go.sia.tech/web/renterd v0.44.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= +go.sia.tech/web/renterd v0.45.0 h1:5kSiDnHYRacg3JideH9Cl9qHzcZiKnBR0fWRap169hU= +go.sia.tech/web/renterd v0.45.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/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= From 793dd2fa015eb2a104ac07bc23ba65e3228356f7 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 09:55:31 +0100 Subject: [PATCH 061/144] autopilot: dismiss lost sector alerts when there are no lost sectors --- autopilot/alerts.go | 4 ++-- autopilot/contractor.go | 6 ++++++ 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/autopilot/alerts.go b/autopilot/alerts.go index 7b42991e1..c279075ee 100644 --- a/autopilot/alerts.go +++ b/autopilot/alerts.go @@ -48,8 +48,8 @@ func (ap *Autopilot) RegisterAlert(ctx context.Context, a alerts.Alert) { } } -func (ap *Autopilot) DismissAlert(ctx context.Context, id types.Hash256) { - if err := ap.alerts.DismissAlerts(ctx, id); err != nil { +func (ap *Autopilot) DismissAlert(ctx context.Context, ids ...types.Hash256) { + if err := ap.alerts.DismissAlerts(ctx, ids...); err != nil { ap.logger.Errorf("failed to dismiss alert: %v", err) } } diff --git a/autopilot/contractor.go b/autopilot/contractor.go index adad5d1b7..bc947d500 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -231,11 +231,17 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( } // check if any used hosts have lost data to warn the user + var toDismiss []types.Hash256 for _, h := range hosts { if h.Interactions.LostSectors > 0 { c.ap.RegisterAlert(ctx, newLostSectorsAlert(h.PublicKey, h.Interactions.LostSectors)) + } else { + toDismiss = append(toDismiss, alertIDForHost(alertLostSectorsID, h.PublicKey)) } } + if len(toDismiss) > 0 { + c.ap.DismissAlert(ctx, toDismiss...) + } // fetch candidate hosts candidates, unusableHosts, err := c.candidateHosts(ctx, hosts, usedHosts, hostData, math.SmallestNonzeroFloat64) // avoid 0 score hosts From a52b3a769005e25a6ca2a9a5d16770b4339f9009 Mon Sep 17 00:00:00 2001 From: PJ Date: Thu, 15 Feb 2024 11:01:58 +0100 Subject: [PATCH 062/144] stores: fix metric migrations --- stores/migrations.go | 60 ++------------------------- stores/migrations_metrics.go | 29 ++----------- stores/migrations_utils.go | 79 ++++++++++++++++++++++++++++++++++++ 3 files changed, 85 insertions(+), 83 deletions(-) create mode 100644 stores/migrations_utils.go diff --git a/stores/migrations.go b/stores/migrations.go index e79c6c36b..05bda3f69 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -6,7 +6,6 @@ import ( "strings" "github.com/go-gormigrate/gormigrate/v2" - "go.sia.tech/renterd/api" "go.uber.org/zap" "gorm.io/gorm" ) @@ -16,32 +15,6 @@ var ( errMySQLNoSuperPrivilege = errors.New("You do not have the SUPER privilege and binary logging is enabled") ) -// initSchema is executed only on a clean database. Otherwise the individual -// migrations are executed. -func initSchema(tx *gorm.DB) (err error) { - // Pick the right migrations. - var schema []byte - if isSQLite(tx) { - schema, err = migrations.ReadFile("migrations/sqlite/main/schema.sql") - } else { - schema, err = migrations.ReadFile("migrations/mysql/main/schema.sql") - } - if err != nil { - return - } - - // Run it. - err = tx.Exec(string(schema)).Error - if err != nil { - return fmt.Errorf("failed to init schema: %w", err) - } - - // Add default bucket. - return tx.Create(&dbBucket{ - Name: api.DefaultBucketName, - }).Error -} - func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { migrations := []*gormigrate.Migration{ { @@ -51,13 +24,13 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { { ID: "00001_object_metadata", Migrate: func(tx *gorm.DB) error { - return performMigration(tx, "00001_object_metadata", logger) + return performMigration(tx, "00001_object_metadata", false, logger) }, }, { ID: "00002_prune_slabs_trigger", Migrate: func(tx *gorm.DB) error { - err := performMigration(tx, "00002_prune_slabs_trigger", logger) + err := performMigration(tx, "00002_prune_slabs_trigger", false, logger) if err != nil && strings.Contains(err.Error(), errMySQLNoSuperPrivilege.Error()) { logger.Warn("migration 00002_prune_slabs_trigger requires the user to have the SUPER privilege to register triggers") } @@ -70,7 +43,7 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { m := gormigrate.New(db, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initSchema) + m.InitSchema(initSchema(db, false, logger)) // Perform migrations. if err := m.Migrate(); err != nil { @@ -78,30 +51,3 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { } return nil } - -func performMigration(db *gorm.DB, name string, logger *zap.SugaredLogger) error { - logger.Infof("performing migration %s", name) - - // build path - var path string - if isSQLite(db) { - path = fmt.Sprintf("migrations/sqlite/main/migration_" + name + ".sql") - } else { - path = fmt.Sprintf("migrations/mysql/main/migration_" + name + ".sql") - } - - // read migration file - migration, err := migrations.ReadFile(path) - if err != nil { - return fmt.Errorf("migration %s failed: %w", name, err) - } - - // execute it - err = db.Exec(string(migration)).Error - if err != nil { - return fmt.Errorf("migration %s failed: %w", name, err) - } - - logger.Infof("migration %s complete", name) - return nil -} diff --git a/stores/migrations_metrics.go b/stores/migrations_metrics.go index a95d7b914..ddbbe8e4e 100644 --- a/stores/migrations_metrics.go +++ b/stores/migrations_metrics.go @@ -8,30 +8,7 @@ import ( "gorm.io/gorm" ) -// initMetricsSchema is executed only on a clean database. Otherwise the individual -// migrations are executed. -func initMetricsSchema(tx *gorm.DB) error { - // Pick the right migrations. - var schema []byte - var err error - if isSQLite(tx) { - schema, err = migrations.ReadFile("migrations/sqlite/metrics/schema.sql") - } else { - schema, err = migrations.ReadFile("migrations/mysql/metrics/schema.sql") - } - if err != nil { - return err - } - - // Run it. - err = tx.Exec(string(schema)).Error - if err != nil { - return fmt.Errorf("failed to init schema: %w", err) - } - return nil -} - -func performMetricsMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { +func performMetricsMigrations(tx *gorm.DB, logger *zap.SugaredLogger) error { migrations := []*gormigrate.Migration{ { ID: "00001_init", @@ -40,10 +17,10 @@ func performMetricsMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { } // Create migrator. - m := gormigrate.New(db, gormigrate.DefaultOptions, migrations) + m := gormigrate.New(tx, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initMetricsSchema) + m.InitSchema(initSchema(tx, true, logger)) // Perform migrations. if err := m.Migrate(); err != nil { diff --git a/stores/migrations_utils.go b/stores/migrations_utils.go new file mode 100644 index 000000000..4832d96a7 --- /dev/null +++ b/stores/migrations_utils.go @@ -0,0 +1,79 @@ +package stores + +import ( + "fmt" + + gormigrate "github.com/go-gormigrate/gormigrate/v2" + "go.sia.tech/renterd/api" + "go.uber.org/zap" + "gorm.io/gorm" +) + +// initSchema is executed only on a clean database. Otherwise the individual +// migrations are executed. +func initSchema(db *gorm.DB, metrics bool, logger *zap.SugaredLogger) gormigrate.InitSchemaFunc { + return func(tx *gorm.DB) error { + if metrics { + logger.Info("initializing metrics schema") + } else { + logger.Info("initializing schema") + } + + // build filename + filename := "schema" + err := execSQLFile(tx, metrics, filename) + if err != nil { + return fmt.Errorf("failed to init schema: %w", err) + } + + // add default bucket. + if !metrics { + if err := tx.Create(&dbBucket{ + Name: api.DefaultBucketName, + }).Error; err != nil { + return fmt.Errorf("failed to create default bucket: %v", err) + } + } + + logger.Info("initialization complete") + return nil + } +} + +func performMigration(db *gorm.DB, name string, metrics bool, logger *zap.SugaredLogger) error { + logger.Infof("performing migration %s", name) + + // build filename + filename := fmt.Sprintf("migration_%s", name) + + // execute migration + err := execSQLFile(db, metrics, filename) + if err != nil { + return fmt.Errorf("migration %s failed: %w", name, err) + } + + logger.Infof("migration %s complete", name) + return nil +} + +func execSQLFile(db *gorm.DB, metrics bool, filename string) error { + // build path + folder := "main" + if metrics { + folder = "metrics" + } + protocol := "mysql" + if isSQLite(db) { + protocol = "sqlite" + } + path := fmt.Sprintf("migrations/%s/%s/%s.sql", protocol, folder, filename) + + // read file + file, err := migrations.ReadFile(path) + if err != nil { + return err + } + + // execute it + return db.Exec(string(file)).Error +} From cc83fc96cb09b2210c7ec8f5846e617d3c05e396 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 14:26:25 +0100 Subject: [PATCH 063/144] stores: improve performance of objects stats --- stores/metadata.go | 63 ++++++++++++++++++++++++++++++---------------- 1 file changed, 41 insertions(+), 22 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 68947ed95..39a053528 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -584,11 +584,18 @@ func (s *SQLStore) ListBuckets(ctx context.Context) ([]api.Bucket, error) { // within a single transaction. func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) (api.ObjectsStatsResponse, error) { // if no bucket is specified, we consider all objects + var bucketID uint + if opts.Bucket != "" { + err := s.db.Model(&dbBucket{}).Select("id").Where("name = ?", opts.Bucket).Take(&bucketID).Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + } whereBucket := func(table string) clause.Expr { if opts.Bucket == "" { return exprTRUE } - return sqlWhereBucket(table, opts.Bucket) + return gorm.Expr(fmt.Sprintf("%s.db_bucket_id = ?", table), bucketID) } // number of objects @@ -632,37 +639,49 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } - fromContractSectors := gorm.Expr("contract_sectors cs") + fromContractSectors := gorm.Expr("contract_sectors cs WHERE 1=1") if opts.Bucket != "" { fromContractSectors = gorm.Expr(` contract_sectors cs INNER JOIN sectors s ON s.id = cs.db_sector_id INNER JOIN slabs sla ON sla.id = s.db_slab_id - INNER JOIN slices sli ON sli.db_slab_id = sla.id - INNER JOIN objects o ON o.id = sli.db_object_id AND (?) - `, whereBucket("o")) + WHERE EXISTS ( + SELECT 1 FROM slices sli + INNER JOIN objects o ON o.id = sli.db_object_id + WHERE sli.db_slab_id = sla.id AND o.db_bucket_id = ? + ) + `, bucketID) } var totalSectors uint64 - batchSize := 500000 - marker := uint64(0) - for offset := 0; ; offset += batchSize { - var result struct { - Sectors uint64 - Marker uint64 - } - res := s.db. - Model(&dbSector{}). - Raw("SELECT COUNT(*) as Sectors, MAX(sectors.db_sector_id) as Marker FROM (SELECT cs.db_sector_id FROM ? WHERE cs.db_sector_id > ? GROUP BY cs.db_sector_id LIMIT ?) sectors", fromContractSectors, marker, batchSize). - Scan(&result) - if err := res.Error; err != nil { - return api.ObjectsStatsResponse{}, err - } else if result.Sectors == 0 { - break // done + var sectorsInfo struct { + MinID uint + MaxID uint + Total uint + } + err = s.db.Model(&dbContractSector{}). + Raw("SELECT MIN(db_sector_id) as MinID, MAX(db_sector_id) as MaxID, COUNT(*) as Total FROM contract_sectors"). + Scan(§orsInfo).Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + + // compute a good batch size for the ids + batchSize := (sectorsInfo.MaxID - sectorsInfo.MinID) / (sectorsInfo.Total/500000 + 1) + + if sectorsInfo.Total > 0 { + for from, to := sectorsInfo.MinID, sectorsInfo.MinID+batchSize; from <= sectorsInfo.MaxID; from, to = to, to+batchSize { + var nSectors uint64 + err := s.db. + Model(&dbSector{}). + Raw("SELECT COUNT(DISTINCT cs.db_sector_id) FROM ? AND cs.db_sector_id >= ? AND cs.db_sector_id < ?", fromContractSectors, from, to). + Scan(&nSectors).Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + totalSectors += nSectors } - totalSectors += result.Sectors - marker = result.Marker } var totalUploaded int64 From 159276068278a4d3759ae034fe6fb7675da4ad53 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 14:51:50 +0100 Subject: [PATCH 064/144] stores: use select count with subquery --- stores/metadata.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stores/metadata.go b/stores/metadata.go index 39a053528..cfd417dbb 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -675,7 +675,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) var nSectors uint64 err := s.db. Model(&dbSector{}). - Raw("SELECT COUNT(DISTINCT cs.db_sector_id) FROM ? AND cs.db_sector_id >= ? AND cs.db_sector_id < ?", fromContractSectors, from, to). + Raw("SELECT COUNT(*) FROM (SELECT DISTINCT cs.db_sector_id FROM ? AND cs.db_sector_id >= ? AND cs.db_sector_id < ?)", fromContractSectors, from, to). Scan(&nSectors).Error if err != nil { return api.ObjectsStatsResponse{}, err From 47a91acb19203b6ed889bbbfe77e6a95eda22018 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 15:55:17 +0100 Subject: [PATCH 065/144] stores: use sectors table and don't filter sectors stats by bucket --- stores/metadata.go | 49 +++++------------------------------------ stores/metadata_test.go | 8 +++---- 2 files changed, 10 insertions(+), 47 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index cfd417dbb..c2e07ed00 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -639,54 +639,17 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } - fromContractSectors := gorm.Expr("contract_sectors cs WHERE 1=1") - if opts.Bucket != "" { - fromContractSectors = gorm.Expr(` - contract_sectors cs - INNER JOIN sectors s ON s.id = cs.db_sector_id - INNER JOIN slabs sla ON sla.id = s.db_slab_id - WHERE EXISTS ( - SELECT 1 FROM slices sli - INNER JOIN objects o ON o.id = sli.db_object_id - WHERE sli.db_slab_id = sla.id AND o.db_bucket_id = ? - ) - `, bucketID) - } - - var totalSectors uint64 - - var sectorsInfo struct { - MinID uint - MaxID uint - Total uint - } - err = s.db.Model(&dbContractSector{}). - Raw("SELECT MIN(db_sector_id) as MinID, MAX(db_sector_id) as MaxID, COUNT(*) as Total FROM contract_sectors"). - Scan(§orsInfo).Error + var totalSectors int64 + err = s.db. + Model(&dbSector{}). + Count(&totalSectors).Error if err != nil { return api.ObjectsStatsResponse{}, err } - // compute a good batch size for the ids - batchSize := (sectorsInfo.MaxID - sectorsInfo.MinID) / (sectorsInfo.Total/500000 + 1) - - if sectorsInfo.Total > 0 { - for from, to := sectorsInfo.MinID, sectorsInfo.MinID+batchSize; from <= sectorsInfo.MaxID; from, to = to, to+batchSize { - var nSectors uint64 - err := s.db. - Model(&dbSector{}). - Raw("SELECT COUNT(*) FROM (SELECT DISTINCT cs.db_sector_id FROM ? AND cs.db_sector_id >= ? AND cs.db_sector_id < ?)", fromContractSectors, from, to). - Scan(&nSectors).Error - if err != nil { - return api.ObjectsStatsResponse{}, err - } - totalSectors += nSectors - } - } - var totalUploaded int64 err = s.db. - Table("?", fromContractSectors). + Model(&dbContractSector{}). Count(&totalUploaded). Error if err != nil { @@ -699,7 +662,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) NumUnfinishedObjects: unfinishedObjects, TotalUnfinishedObjectsSize: totalUnfinishedObjectsSize, TotalObjectsSize: objInfo.TotalObjectsSize, - TotalSectorsSize: totalSectors * rhpv2.SectorSize, + TotalSectorsSize: uint64(totalSectors) * rhpv2.SectorSize, TotalUploadedSize: uint64(totalUploaded) * rhpv2.SectorSize, }, nil } diff --git a/stores/metadata_test.go b/stores/metadata_test.go index a05c0be17..2d550c1d6 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -2524,10 +2524,10 @@ func TestObjectsStats(t *testing.T) { t.Fatal(err) } else if info.TotalObjectsSize != 0 { t.Fatal("wrong size", info.TotalObjectsSize) - } else if info.TotalSectorsSize != 0 { - t.Fatal("wrong size", info.TotalSectorsSize) - } else if info.TotalUploadedSize != 0 { - t.Fatal("wrong size", info.TotalUploadedSize) + } else if info.TotalSectorsSize != sectorsSize { + t.Fatal("wrong size", info.TotalSectorsSize, sectorsSize) + } else if info.TotalUploadedSize != sectorsSize*2 { + t.Fatal("wrong size", info.TotalUploadedSize, sectorsSize*2) } else if info.NumObjects != 0 { t.Fatal("wrong number of objects", info.NumObjects) } From 3e463cfb4f7103d976ca2d84bb4baeee9438d067 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 16:04:48 +0100 Subject: [PATCH 066/144] stores: use sum over total shards --- stores/metadata.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index c2e07ed00..aae62073d 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -641,8 +641,9 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) var totalSectors int64 err = s.db. - Model(&dbSector{}). - Count(&totalSectors).Error + Model(&dbSlab{}). + Select("COALESCE(SUM(total_shards), 0)"). + Scan(&totalSectors).Error if err != nil { return api.ObjectsStatsResponse{}, err } From 3f6dad71c9ac5add056534d4d8d6da0f18b68837 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 17:06:29 +0100 Subject: [PATCH 067/144] stores: use contract size for total uploaded size --- stores/metadata.go | 24 ++++++++++++++++++++---- stores/metadata_test.go | 19 +++++++++++-------- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index aae62073d..3f13097ad 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -639,9 +639,18 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } + fromSlabs := gorm.Expr("slabs sla") + if opts.Bucket != "" { + fromSlabs = gorm.Expr(` + slabs sla + INNER JOIN slices sli ON sli.db_slab_id = sla.id + INNER JOIN objects o ON o.id = sli.db_object_id AND (?) + `, whereBucket("o")) + } + var totalSectors int64 err = s.db. - Model(&dbSlab{}). + Table("?", fromSlabs). Select("COALESCE(SUM(total_shards), 0)"). Scan(&totalSectors).Error if err != nil { @@ -650,13 +659,20 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) var totalUploaded int64 err = s.db. - Model(&dbContractSector{}). - Count(&totalUploaded). + Model(&dbContract{}). + Select("COALESCE(SUM(size), 0)"). + Scan(&totalUploaded). Error if err != nil { return api.ObjectsStatsResponse{}, err } + var contracts []dbContract + err = s.db.Find(&contracts).Error + if err != nil { + return api.ObjectsStatsResponse{}, err + } + return api.ObjectsStatsResponse{ MinHealth: objInfo.MinHealth, NumObjects: objInfo.NumObjects, @@ -664,7 +680,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) TotalUnfinishedObjectsSize: totalUnfinishedObjectsSize, TotalObjectsSize: objInfo.TotalObjectsSize, TotalSectorsSize: uint64(totalSectors) * rhpv2.SectorSize, - TotalUploadedSize: uint64(totalUploaded) * rhpv2.SectorSize, + TotalUploadedSize: uint64(totalUploaded), }, nil } diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 2d550c1d6..da9ecbbef 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -2446,6 +2446,7 @@ func TestObjectsStats(t *testing.T) { // Create a few objects of different size. var objectsSize uint64 var sectorsSize uint64 + var totalUploadedSize uint64 for i := 0; i < 2; i++ { obj := newTestObject(1) objectsSize += uint64(obj.TotalSize()) @@ -2458,10 +2459,11 @@ func TestObjectsStats(t *testing.T) { t.Fatal(err) } for _, fcid := range fcids { - _, err := ss.addTestContract(fcid, hpk) + c, err := ss.addTestContract(fcid, hpk) if err != nil { t.Fatal(err) } + totalUploadedSize += c.Size } } } @@ -2482,10 +2484,11 @@ func TestObjectsStats(t *testing.T) { } var newContractID types.FileContractID frand.Read(newContractID[:]) - _, err = ss.addTestContract(newContractID, types.PublicKey{}) + c, err := ss.addTestContract(newContractID, types.PublicKey{}) if err != nil { t.Fatal(err) } + totalUploadedSize += c.Size newContract, err := ss.contract(context.Background(), fileContractID(newContractID)) if err != nil { t.Fatal(err) @@ -2510,8 +2513,8 @@ func TestObjectsStats(t *testing.T) { t.Fatal("wrong size", info.TotalObjectsSize, objectsSize) } else if info.TotalSectorsSize != sectorsSize { t.Fatal("wrong size", info.TotalSectorsSize, sectorsSize) - } else if info.TotalUploadedSize != sectorsSize*2 { - t.Fatal("wrong size", info.TotalUploadedSize, sectorsSize*2) + } else if info.TotalUploadedSize != totalUploadedSize { + t.Fatal("wrong size", info.TotalUploadedSize, totalUploadedSize) } else if info.NumObjects != 2 { t.Fatal("wrong number of objects", info.NumObjects, 2) } @@ -2524,10 +2527,10 @@ func TestObjectsStats(t *testing.T) { t.Fatal(err) } else if info.TotalObjectsSize != 0 { t.Fatal("wrong size", info.TotalObjectsSize) - } else if info.TotalSectorsSize != sectorsSize { - t.Fatal("wrong size", info.TotalSectorsSize, sectorsSize) - } else if info.TotalUploadedSize != sectorsSize*2 { - t.Fatal("wrong size", info.TotalUploadedSize, sectorsSize*2) + } else if info.TotalSectorsSize != 0 { + t.Fatal("wrong size", info.TotalSectorsSize, 0) + } else if info.TotalUploadedSize != totalUploadedSize { + t.Fatal("wrong size", info.TotalUploadedSize, totalUploadedSize) } else if info.NumObjects != 0 { t.Fatal("wrong number of objects", info.NumObjects) } From 2c627d950532477b22d9be0668f76a29c240b49c Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 15 Feb 2024 17:17:36 +0100 Subject: [PATCH 068/144] testing: fix TestUploadDownloadExtended --- internal/testing/cluster_test.go | 49 +++++++++++++++++--------------- 1 file changed, 26 insertions(+), 23 deletions(-) diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 13903115d..4262e493b 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -697,30 +697,33 @@ func TestUploadDownloadExtended(t *testing.T) { } // check objects stats. - for _, opts := range []api.ObjectsStatsOpts{ - {}, // any bucket - {Bucket: api.DefaultBucketName}, // specific bucket - } { - info, err := cluster.Bus.ObjectsStats(context.Background(), opts) - tt.OK(err) - objectsSize := uint64(len(file1) + len(file2) + len(small) + len(large)) - if info.TotalObjectsSize != objectsSize { - t.Error("wrong size", info.TotalObjectsSize, objectsSize) - } - sectorsSize := 15 * rhpv2.SectorSize - if info.TotalSectorsSize != uint64(sectorsSize) { - t.Error("wrong size", info.TotalSectorsSize, sectorsSize) - } - if info.TotalUploadedSize != uint64(sectorsSize) { - t.Error("wrong size", info.TotalUploadedSize, sectorsSize) - } - if info.NumObjects != 4 { - t.Error("wrong number of objects", info.NumObjects, 4) - } - if info.MinHealth != 1 { - t.Errorf("expected minHealth of 1, got %v", info.MinHealth) + tt.Retry(100, 100*time.Millisecond, func() error { + for _, opts := range []api.ObjectsStatsOpts{ + {}, // any bucket + {Bucket: api.DefaultBucketName}, // specific bucket + } { + info, err := cluster.Bus.ObjectsStats(context.Background(), opts) + tt.OK(err) + objectsSize := uint64(len(file1) + len(file2) + len(small) + len(large)) + if info.TotalObjectsSize != objectsSize { + return fmt.Errorf("wrong size %v %v", info.TotalObjectsSize, objectsSize) + } + sectorsSize := 15 * rhpv2.SectorSize + if info.TotalSectorsSize != uint64(sectorsSize) { + return fmt.Errorf("wrong size %v %v", info.TotalSectorsSize, sectorsSize) + } + if info.TotalUploadedSize != uint64(sectorsSize) { + return fmt.Errorf("wrong size %v %v", info.TotalUploadedSize, sectorsSize) + } + if info.NumObjects != 4 { + return fmt.Errorf("wrong number of objects %v %v", info.NumObjects, 4) + } + if info.MinHealth != 1 { + return fmt.Errorf("expected minHealth of 1, got %v", info.MinHealth) + } } - } + return nil + }) // download the data for _, data := range [][]byte{small, large} { From c3802d8466d2c7abbf418349a43a1bee41d79239 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 09:03:24 +0100 Subject: [PATCH 069/144] stores: fix TestSlabBufferStats --- stores/metadata.go | 1 + 1 file changed, 1 insertion(+) diff --git a/stores/metadata.go b/stores/metadata.go index 3f13097ad..4a2ab8657 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -652,6 +652,7 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) err = s.db. Table("?", fromSlabs). Select("COALESCE(SUM(total_shards), 0)"). + Where("db_buffered_slab_id IS NULL"). Scan(&totalSectors).Error if err != nil { return api.ObjectsStatsResponse{}, err From 6ed8e754ba2706d419fe2d80f01a6802415b35fd Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 10:20:23 +0100 Subject: [PATCH 070/144] stores: use WHERE EXISTS --- stores/metadata.go | 8 ++++++-- 1 file changed, 6 insertions(+), 2 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 4a2ab8657..d3622502f 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -642,9 +642,13 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) fromSlabs := gorm.Expr("slabs sla") if opts.Bucket != "" { fromSlabs = gorm.Expr(` + (SELECT * FROM slabs sla - INNER JOIN slices sli ON sli.db_slab_id = sla.id - INNER JOIN objects o ON o.id = sli.db_object_id AND (?) + WHERE EXISTS ( + SELECT 1 FROM slices sli + INNER JOIN objects o ON o.id = sli.db_object_id AND ? + WHERE sli.db_slab_id = sla.id + )) sla `, whereBucket("o")) } From 5f8233fb17aa431aafb987654fe879db2f9a35c8 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 10:59:46 +0100 Subject: [PATCH 071/144] stores: remove subquery --- stores/metadata.go | 27 ++++++++++++--------------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index d3622502f..2a879d7ae 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -639,25 +639,22 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } - fromSlabs := gorm.Expr("slabs sla") + var totalSectors int64 + totalSectorsQuery := s.db. + Table("slabs sla"). + Select("COALESCE(SUM(total_shards), 0)"). + Where("db_buffered_slab_id IS NULL") + if opts.Bucket != "" { - fromSlabs = gorm.Expr(` - (SELECT * FROM - slabs sla - WHERE EXISTS ( + totalSectorsQuery = totalSectorsQuery.Where(` + EXISTS ( SELECT 1 FROM slices sli - INNER JOIN objects o ON o.id = sli.db_object_id AND ? + INNER JOIN objects o ON o.id = sli.db_object_id AND o.db_bucket_id = ? WHERE sli.db_slab_id = sla.id - )) sla - `, whereBucket("o")) + ) + `, bucketID) } - - var totalSectors int64 - err = s.db. - Table("?", fromSlabs). - Select("COALESCE(SUM(total_shards), 0)"). - Where("db_buffered_slab_id IS NULL"). - Scan(&totalSectors).Error + err = totalSectorsQuery.Scan(&totalSectors).Error if err != nil { return api.ObjectsStatsResponse{}, err } From f68e6ee779a3aa6a10e1a46475679e227ab6c785 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 11:16:37 +0100 Subject: [PATCH 072/144] stores: remove contract query --- stores/metadata.go | 47 +++++++++++++++++++--------------------------- 1 file changed, 19 insertions(+), 28 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 2a879d7ae..d257bf681 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -583,7 +583,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, opts api.ObjectsStatsOpts) (api.ObjectsStatsResponse, error) { - // if no bucket is specified, we consider all objects + // fetch bucket id if a bucket was specified var bucketID uint if opts.Bucket != "" { err := s.db.Model(&dbBucket{}).Select("id").Where("name = ?", opts.Bucket).Take(&bucketID).Error @@ -591,12 +591,6 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } } - whereBucket := func(table string) clause.Expr { - if opts.Bucket == "" { - return exprTRUE - } - return gorm.Expr(fmt.Sprintf("%s.db_bucket_id = ?", table), bucketID) - } // number of objects var objInfo struct { @@ -604,37 +598,40 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) MinHealth float64 TotalObjectsSize uint64 } - err := s.db. + objInfoQuery := s.db. Model(&dbObject{}). - Select("COUNT(*) AS NumObjects, COALESCE(MIN(health), 1) as MinHealth, SUM(size) AS TotalObjectsSize"). - Where(whereBucket(dbObject{}.TableName())). - Scan(&objInfo). - Error + Select("COUNT(*) AS NumObjects, COALESCE(MIN(health), 1) as MinHealth, SUM(size) AS TotalObjectsSize") + if opts.Bucket != "" { + objInfoQuery = objInfoQuery.Where("db_bucket_id", bucketID) + } + err := objInfoQuery.Scan(&objInfo).Error if err != nil { return api.ObjectsStatsResponse{}, err } // number of unfinished objects var unfinishedObjects uint64 - err = s.db. + unfinishedObjectsQuery := s.db. Model(&dbMultipartUpload{}). - Select("COUNT(*)"). - Where(whereBucket(dbMultipartUpload{}.TableName())). - Scan(&unfinishedObjects). - Error + Select("COUNT(*)") + if opts.Bucket != "" { + unfinishedObjectsQuery = unfinishedObjectsQuery.Where("db_bucket_id", bucketID) + } + err = unfinishedObjectsQuery.Scan(&unfinishedObjects).Error if err != nil { return api.ObjectsStatsResponse{}, err } // size of unfinished objects var totalUnfinishedObjectsSize uint64 - err = s.db. + totalUnfinishedObjectsSizeQuery := s.db. Model(&dbMultipartPart{}). Joins("INNER JOIN multipart_uploads mu ON multipart_parts.db_multipart_upload_id = mu.id"). - Select("COALESCE(SUM(size), 0)"). - Where(whereBucket("mu")). - Scan(&totalUnfinishedObjectsSize). - Error + Select("COALESCE(SUM(size), 0)") + if opts.Bucket != "" { + totalUnfinishedObjectsSizeQuery = totalUnfinishedObjectsSizeQuery.Where("db_bucket_id", bucketID) + } + err = totalUnfinishedObjectsSizeQuery.Scan(&totalUnfinishedObjectsSize).Error if err != nil { return api.ObjectsStatsResponse{}, err } @@ -669,12 +666,6 @@ func (s *SQLStore) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) return api.ObjectsStatsResponse{}, err } - var contracts []dbContract - err = s.db.Find(&contracts).Error - if err != nil { - return api.ObjectsStatsResponse{}, err - } - return api.ObjectsStatsResponse{ MinHealth: objInfo.MinHealth, NumObjects: objInfo.NumObjects, From fe2097a38230bd001326125f8abf89fbdc6ec5cc Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 13:01:55 +0100 Subject: [PATCH 073/144] bus: merge alert dismiss routes --- bus/bus.go | 18 ++++++++++-------- bus/client/alerts.go | 18 +++++++++++++----- 2 files changed, 23 insertions(+), 13 deletions(-) diff --git a/bus/bus.go b/bus/bus.go index b8c21605c..2680c4810 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -245,10 +245,9 @@ func (b *bus) Handler() http.Handler { "POST /account/:id/requiressync": b.accountsRequiresSyncHandlerPOST, "POST /account/:id/resetdrift": b.accountsResetDriftHandlerPOST, - "GET /alerts": b.handleGETAlerts, - "POST /alerts/dismiss": b.handlePOSTAlertsDismiss, - "POST /alerts/dismissall": b.handlePOSTAlertsDismissAll, - "POST /alerts/register": b.handlePOSTAlertsRegister, + "GET /alerts": b.handleGETAlerts, + "POST /alerts/dismiss": b.handlePOSTAlertsDismiss, + "POST /alerts/register": b.handlePOSTAlertsRegister, "GET /autopilots": b.autopilotsListHandlerGET, "GET /autopilot/:id": b.autopilotsHandlerGET, @@ -1730,6 +1729,13 @@ func (b *bus) handleGETAlerts(jc jape.Context) { } func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { + var all bool + if jc.DecodeForm("all", &all) != nil { + return + } else if all { + jc.Check("failed to dismiss all alerts", b.alertMgr.DismissAllAlerts(jc.Request.Context())) + return + } var ids []types.Hash256 if jc.Decode(&ids) != nil { return @@ -1737,10 +1743,6 @@ func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { jc.Check("failed to dismiss alerts", b.alertMgr.DismissAlerts(jc.Request.Context(), ids...)) } -func (b *bus) handlePOSTAlertsDismissAll(jc jape.Context) { - jc.Check("failed to dismiss alerts", b.alertMgr.DismissAllAlerts(jc.Request.Context())) -} - func (b *bus) handlePOSTAlertsRegister(jc jape.Context) { var alert alerts.Alert if jc.Decode(&alert) != nil { diff --git a/bus/client/alerts.go b/bus/client/alerts.go index 6151db598..bff3c13a5 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -22,14 +22,22 @@ func (c *Client) Alerts(opts alerts.AlertsOpts) (alerts []alerts.Alert, err erro return } -// DismissAllAlerts dimisses all alerts. +// DismissAlerts dimisses the alerts with the given IDs. +func (c *Client) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { + return c.dismissAlerts(ctx, false, ids...) +} + +// DismissAllAlerts dimisses all registered alerts. func (c *Client) DismissAllAlerts(ctx context.Context) error { - return c.c.WithContext(ctx).POST("/alerts/dismissall", nil, nil) + return c.dismissAlerts(ctx, true) } -// DismissAlerts dimisses the alerts with the given IDs. -func (c *Client) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { - return c.c.WithContext(ctx).POST("/alerts/dismiss", ids, nil) +func (c *Client) dismissAlerts(ctx context.Context, all bool, ids ...types.Hash256) error { + values := url.Values{} + if all { + values.Set("all", fmt.Sprint(true)) + } + return c.c.WithContext(ctx).POST("/alerts/dismiss?"+values.Encode(), ids, nil) } // RegisterAlert registers the given alert. From a13313bacbd2410782f982a13b06fa774ad2cf66 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 13:49:24 +0100 Subject: [PATCH 074/144] bus: add onlymetadata flag to objects endpoint --- api/object.go | 20 +++++---- bus/bus.go | 16 +++++-- s3/backend.go | 5 ++- stores/metadata.go | 110 ++++++++++++++++++++++++++++++++------------- 4 files changed, 108 insertions(+), 43 deletions(-) diff --git a/api/object.go b/api/object.go index e4fd4b465..09f4b04ab 100644 --- a/api/object.go +++ b/api/object.go @@ -54,7 +54,7 @@ type ( Object struct { Metadata ObjectUserMetadata `json:"metadata,omitempty"` ObjectMetadata - object.Object + object.Object `json:"omitempty"` } // ObjectMetadata contains various metadata about an object. @@ -212,13 +212,14 @@ type ( } GetObjectOptions struct { - Prefix string - Offset int - Limit int - IgnoreDelim bool - Marker string - SortBy string - SortDir string + Prefix string + Offset int + Limit int + IgnoreDelim bool + Marker string + OnlyMetadata bool + SortBy string + SortDir string } ListObjectOptions struct { @@ -324,6 +325,9 @@ func (opts GetObjectOptions) Apply(values url.Values) { if opts.Marker != "" { values.Set("marker", opts.Marker) } + if opts.OnlyMetadata { + values.Set("onlymetadata", "true") + } if opts.SortBy != "" { values.Set("sortBy", opts.SortBy) } diff --git a/bus/bus.go b/bus/bus.go index b8c21605c..66620a684 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -137,6 +137,7 @@ type ( CopyObject(ctx context.Context, srcBucket, dstBucket, srcPath, dstPath, mimeType string, metadata api.ObjectUserMetadata) (api.ObjectMetadata, error) ListObjects(ctx context.Context, bucketName, prefix, sortBy, sortDir, marker string, limit int) (api.ObjectsListResponse, error) Object(ctx context.Context, bucketName, path string) (api.Object, error) + ObjectMetadata(ctx context.Context, bucketName, path string) (api.Object, error) ObjectEntries(ctx context.Context, bucketName, path, prefix, sortBy, sortDir, marker string, offset, limit int) ([]api.ObjectMetadata, bool, error) ObjectsBySlabKey(ctx context.Context, bucketName string, slabKey object.EncryptionKey) ([]api.ObjectMetadata, error) ObjectsStats(ctx context.Context, opts api.ObjectsStatsOpts) (api.ObjectsStatsResponse, error) @@ -1192,13 +1193,22 @@ func (b *bus) objectsHandlerGET(jc jape.Context) { if jc.DecodeForm("bucket", &bucket) != nil { return } + var onlymetadata bool + if jc.DecodeForm("onlymetadata", &onlymetadata) != nil { + return + } - o, err := b.ms.Object(jc.Request.Context(), bucket, path) + var o api.Object + var err error + if onlymetadata { + o, err = b.ms.ObjectMetadata(jc.Request.Context(), bucket, path) + } else { + o, err = b.ms.Object(jc.Request.Context(), bucket, path) + } if errors.Is(err, api.ErrObjectNotFound) { jc.Error(err, http.StatusNotFound) return - } - if jc.Check("couldn't load object", err) != nil { + } else if jc.Check("couldn't load object", err) != nil { return } jc.Encode(api.ObjectsResponse{Object: &o}) diff --git a/s3/backend.go b/s3/backend.go index a481da727..cb53be625 100644 --- a/s3/backend.go +++ b/s3/backend.go @@ -287,7 +287,10 @@ func (s *s3) GetObject(ctx context.Context, bucketName, objectName string, range // HeadObject should return a NotFound() error if the object does not // exist. func (s *s3) HeadObject(ctx context.Context, bucketName, objectName string) (*gofakes3.Object, error) { - res, err := s.b.Object(ctx, bucketName, objectName, api.GetObjectOptions{IgnoreDelim: true}) + res, err := s.b.Object(ctx, bucketName, objectName, api.GetObjectOptions{ + IgnoreDelim: true, + OnlyMetadata: true, + }) if err != nil && strings.Contains(err.Error(), api.ErrObjectNotFound.Error()) { return nil, gofakes3.KeyNotFound(objectName) } else if err != nil { diff --git a/stores/metadata.go b/stores/metadata.go index 68947ed95..15a4690ad 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -410,14 +410,14 @@ func (s dbSlab) convert() (slab object.Slab, err error) { } func (raw rawObjectMetadata) convert() api.ObjectMetadata { - return api.ObjectMetadata{ - ETag: raw.ETag, - Health: raw.Health, - MimeType: raw.MimeType, - ModTime: api.TimeRFC3339(time.Time(raw.ModTime).UTC()), - Name: raw.Name, - Size: raw.Size, - } + return newObjectMetadata( + raw.Name, + raw.ETag, + raw.MimeType, + raw.Health, + time.Time(raw.ModTime), + raw.Size, + ) } func (raw rawObject) toSlabSlice() (slice object.SlabSlice, _ error) { @@ -1556,13 +1556,14 @@ func (s *SQLStore) CopyObject(ctx context.Context, srcBucket, dstBucket, srcPath // No copying is happening. We just update the metadata on the src // object. srcObj.MimeType = mimeType - om = api.ObjectMetadata{ - Health: srcObj.Health, - MimeType: srcObj.MimeType, - ModTime: api.TimeRFC3339(srcObj.CreatedAt.UTC()), - Name: srcObj.ObjectID, - Size: srcObj.Size, - } + om = newObjectMetadata( + srcObj.ObjectID, + srcObj.Etag, + srcObj.MimeType, + srcObj.Health, + srcObj.CreatedAt, + srcObj.Size, + ) if err := s.updateUserMetadata(tx, srcObj.ID, metadata); err != nil { return fmt.Errorf("failed to update user metadata: %w", err) } @@ -1610,14 +1611,14 @@ func (s *SQLStore) CopyObject(ctx context.Context, srcBucket, dstBucket, srcPath return fmt.Errorf("failed to create object metadata: %w", err) } - om = api.ObjectMetadata{ - MimeType: dstObj.MimeType, - ETag: dstObj.Etag, - Health: srcObj.Health, - ModTime: api.TimeRFC3339(dstObj.CreatedAt.UTC()), - Name: dstObj.ObjectID, - Size: dstObj.Size, - } + om = newObjectMetadata( + dstObj.ObjectID, + dstObj.Etag, + dstObj.MimeType, + dstObj.Health, + dstObj.CreatedAt, + dstObj.Size, + ) return nil }) return @@ -2320,14 +2321,14 @@ func (s *SQLStore) objectHydrate(ctx context.Context, tx *gorm.DB, bucket, path // return object return api.Object{ Metadata: metadata, - ObjectMetadata: api.ObjectMetadata{ - ETag: obj[0].ObjectETag, - Health: obj[0].ObjectHealth, - MimeType: obj[0].ObjectMimeType, - ModTime: api.TimeRFC3339(obj[0].ObjectModTime.UTC()), - Name: obj[0].ObjectName, - Size: obj[0].ObjectSize, - }, + ObjectMetadata: newObjectMetadata( + obj[0].ObjectName, + obj[0].ObjectETag, + obj[0].ObjectMimeType, + obj[0].ObjectHealth, + obj[0].ObjectModTime, + obj[0].ObjectSize, + ), Object: object.Object{ Key: key, Slabs: slabs, @@ -2335,6 +2336,42 @@ func (s *SQLStore) objectHydrate(ctx context.Context, tx *gorm.DB, bucket, path }, nil } +// ObjectMetadata returns an object's metadata +func (s *SQLStore) ObjectMetadata(ctx context.Context, bucket, path string) (api.Object, error) { + var resp api.Object + err := s.db.Transaction(func(tx *gorm.DB) error { + var obj dbObject + err := tx.Model(&dbObject{}). + Joins("INNER JOIN buckets b ON objects.db_bucket_id = b.id"). + Where("b.name", bucket). + Where("object_id", path). + Take(&obj). + Error + if errors.Is(err, gorm.ErrRecordNotFound) { + return api.ErrObjectNotFound + } else if err != nil { + return err + } + oum, err := s.objectMetadata(ctx, tx, bucket, path) + if err != nil { + return err + } + resp = api.Object{ + ObjectMetadata: newObjectMetadata( + obj.ObjectID, + obj.Etag, + obj.MimeType, + obj.Health, + obj.CreatedAt, + obj.Size, + ), + Metadata: oum, + } + return nil + }) + return resp, err +} + func (s *SQLStore) objectMetadata(ctx context.Context, tx *gorm.DB, bucket, path string) (api.ObjectUserMetadata, error) { var rows []dbObjectUserMetadata err := tx. @@ -2355,6 +2392,17 @@ func (s *SQLStore) objectMetadata(ctx context.Context, tx *gorm.DB, bucket, path return metadata, nil } +func newObjectMetadata(name, etag, mimeType string, health float64, modTime time.Time, size int64) api.ObjectMetadata { + return api.ObjectMetadata{ + ETag: etag, + Health: health, + ModTime: api.TimeRFC3339(modTime.UTC()), + Name: name, + Size: size, + MimeType: mimeType, + } +} + func (s *SQLStore) objectRaw(ctx context.Context, txn *gorm.DB, bucket string, path string) (rows rawObject, err error) { // NOTE: we LEFT JOIN here because empty objects are valid and need to be // included in the result set, when we convert the rawObject before From 010037927edaae14420a1bedf1972f5ecd5617d4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 14:19:20 +0100 Subject: [PATCH 075/144] testing: fix TestS3Basic --- object/object.go | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/object/object.go b/object/object.go index 49375f3b4..e3c2fe3e7 100644 --- a/object/object.go +++ b/object/object.go @@ -43,6 +43,9 @@ func (k *EncryptionKey) UnmarshalBinary(b []byte) error { // String implements fmt.Stringer. func (k EncryptionKey) String() string { + if k.entropy == nil { + return "" + } return "key:" + hex.EncodeToString(k.entropy[:]) } @@ -53,6 +56,9 @@ func (k EncryptionKey) MarshalText() ([]byte, error) { // UnmarshalText implements the encoding.TextUnmarshaler interface. func (k *EncryptionKey) UnmarshalText(b []byte) error { + if len(b) == 0 { + return nil + } k.entropy = new([32]byte) if n, err := hex.Decode(k.entropy[:], []byte(bytes.TrimPrefix(b, []byte("key:")))); err != nil { return err From 781290d8546147fcf18d1ea0f64f20742749409c Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 14:32:40 +0100 Subject: [PATCH 076/144] worker: fix ndf in TestDownloaderStopped --- worker/downloader_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/downloader_test.go b/worker/downloader_test.go index 357fc2ee8..c1d860c24 100644 --- a/worker/downloader_test.go +++ b/worker/downloader_test.go @@ -16,7 +16,7 @@ func TestDownloaderStopped(t *testing.T) { req := sectorDownloadReq{ resps: §orResponses{ - c: make(chan struct{}), + c: make(chan struct{}, 1), }, } dl.enqueue(&req) From f0ea509bea36674c3cf6d007dca47a51b7fd78f4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 14:33:10 +0100 Subject: [PATCH 077/144] worker: fix ndf in TestDownloaderStopped --- worker/downloader_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/downloader_test.go b/worker/downloader_test.go index 357fc2ee8..c1d860c24 100644 --- a/worker/downloader_test.go +++ b/worker/downloader_test.go @@ -16,7 +16,7 @@ func TestDownloaderStopped(t *testing.T) { req := sectorDownloadReq{ resps: §orResponses{ - c: make(chan struct{}), + c: make(chan struct{}, 1), }, } dl.enqueue(&req) From 814b8e571d6808aa573e33821e7a1fe4d27c77b6 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 15:09:38 +0100 Subject: [PATCH 078/144] bus: add generateKey option to create multipart endpoint --- api/multipart.go | 11 +++++++--- bus/bus.go | 4 +++- bus/client/multipart-upload.go | 11 +++++----- internal/testing/cluster_test.go | 2 +- worker/worker.go | 35 ++++++++++++++++---------------- 5 files changed, 35 insertions(+), 28 deletions(-) diff --git a/api/multipart.go b/api/multipart.go index 955b78849..e786a7c25 100644 --- a/api/multipart.go +++ b/api/multipart.go @@ -46,9 +46,10 @@ type ( } CreateMultipartOptions struct { - Key object.EncryptionKey - MimeType string - Metadata ObjectUserMetadata + GenerateKey bool + Key object.EncryptionKey + MimeType string + Metadata ObjectUserMetadata } ) @@ -86,6 +87,10 @@ type ( Key object.EncryptionKey `json:"key"` MimeType string `json:"mimeType"` Metadata ObjectUserMetadata `json:"metadata"` + + // TODO: The next major version change should invert this to create a + // key by default + GenerateKey bool `json:"generateKey"` } MultipartCreateResponse struct { diff --git a/bus/bus.go b/bus/bus.go index b8c21605c..58739e11a 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -2178,7 +2178,9 @@ func (b *bus) multipartHandlerCreatePOST(jc jape.Context) { } key := req.Key - if key == (object.EncryptionKey{}) { + if req.GenerateKey { + key = object.GenerateEncryptionKey() + } else if key == (object.EncryptionKey{}) { key = object.NoOpKey } diff --git a/bus/client/multipart-upload.go b/bus/client/multipart-upload.go index ffa4d8dc8..281019487 100644 --- a/bus/client/multipart-upload.go +++ b/bus/client/multipart-upload.go @@ -46,11 +46,12 @@ func (c *Client) CompleteMultipartUpload(ctx context.Context, bucket, path, uplo // CreateMultipartUpload creates a new multipart upload. func (c *Client) CreateMultipartUpload(ctx context.Context, bucket, path string, opts api.CreateMultipartOptions) (resp api.MultipartCreateResponse, err error) { err = c.c.WithContext(ctx).POST("/multipart/create", api.MultipartCreateRequest{ - Bucket: bucket, - Path: path, - Key: opts.Key, - MimeType: opts.MimeType, - Metadata: opts.Metadata, + Bucket: bucket, + GenerateKey: opts.GenerateKey, + Path: path, + Key: opts.Key, + MimeType: opts.MimeType, + Metadata: opts.Metadata, }, &resp) return } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index d290faf27..1b57e1e6c 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1996,7 +1996,7 @@ func TestMultipartUploads(t *testing.T) { // Start a new multipart upload. objPath := "/foo" - mpr, err := b.CreateMultipartUpload(context.Background(), api.DefaultBucketName, objPath, api.CreateMultipartOptions{Key: object.GenerateEncryptionKey()}) + mpr, err := b.CreateMultipartUpload(context.Background(), api.DefaultBucketName, objPath, api.CreateMultipartOptions{GenerateKey: true}) tt.OK(err) if mpr.UploadID == "" { t.Fatal("expected non-empty upload ID") diff --git a/worker/worker.go b/worker/worker.go index 8a2a9b1f3..2a2191d2d 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1126,15 +1126,7 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { return } - // make sure only one of the following is set - var disablePreshardingEncryption bool - if jc.DecodeForm("disablepreshardingencryption", &disablePreshardingEncryption) != nil { - return - } - if !disablePreshardingEncryption && jc.Request.FormValue("offset") == "" { - jc.Error(errors.New("if presharding encryption isn't disabled, the offset needs to be set"), http.StatusBadRequest) - return - } + // get the offset var offset int if jc.DecodeForm("offset", &offset) != nil { return @@ -1143,23 +1135,30 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { return } + // fetch upload from bus + upload, err := w.bus.MultipartUpload(ctx, uploadID) + if isError(err, api.ErrMultipartUploadNotFound) { + jc.Error(err, http.StatusNotFound) + return + } else if jc.Check("failed to fetch multipart upload", err) != nil { + return + } + // built options opts := []UploadOption{ WithBlockHeight(up.CurrentHeight), WithContractSet(up.ContractSet), WithPacking(up.UploadPacking), WithRedundancySettings(up.RedundancySettings), + WithCustomKey(upload.Key), } - if disablePreshardingEncryption { - opts = append(opts, WithCustomKey(object.NoOpKey)) - } else { - upload, err := w.bus.MultipartUpload(ctx, uploadID) - if err != nil { - jc.Error(err, http.StatusBadRequest) - return - } + + // make sure only one of the following is set + if encryptionEnabled := upload.Key != object.NoOpKey; encryptionEnabled && jc.Request.FormValue("offset") == "" { + jc.Error(errors.New("if presharding encryption isn't disabled, the offset needs to be set"), http.StatusBadRequest) + return + } else if encryptionEnabled { opts = append(opts, WithCustomEncryptionOffset(uint64(offset))) - opts = append(opts, WithCustomKey(upload.Key)) } // attach gouging checker to the context From a2035a8ffcc18517dcc34a15898ea395bbea6cbf Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 15:15:27 +0100 Subject: [PATCH 079/144] bus: remove DisablePreshardingEncryption --- api/object.go | 30 +++++++++++------------------- internal/testing/cluster_test.go | 11 +++++++---- s3/backend.go | 3 +-- 3 files changed, 19 insertions(+), 25 deletions(-) diff --git a/api/object.go b/api/object.go index e4fd4b465..506f0eed1 100644 --- a/api/object.go +++ b/api/object.go @@ -235,20 +235,18 @@ type ( // UploadObjectOptions is the options type for the worker client. UploadObjectOptions struct { - Offset int - MinShards int - TotalShards int - ContractSet string - DisablePreshardingEncryption bool - ContentLength int64 - MimeType string - Metadata ObjectUserMetadata + Offset int + MinShards int + TotalShards int + ContractSet string + ContentLength int64 + MimeType string + Metadata ObjectUserMetadata } UploadMultipartUploadPartOptions struct { - DisablePreshardingEncryption bool - EncryptionOffset int - ContentLength int64 + EncryptionOffset *int + ContentLength int64 } ) @@ -268,9 +266,6 @@ func (opts UploadObjectOptions) ApplyValues(values url.Values) { if opts.MimeType != "" { values.Set("mimetype", opts.MimeType) } - if opts.DisablePreshardingEncryption { - values.Set("disablepreshardingencryption", "true") - } } func (opts UploadObjectOptions) ApplyHeaders(h http.Header) { @@ -280,11 +275,8 @@ func (opts UploadObjectOptions) ApplyHeaders(h http.Header) { } func (opts UploadMultipartUploadPartOptions) Apply(values url.Values) { - if opts.DisablePreshardingEncryption { - values.Set("disablepreshardingencryption", "true") - } - if !opts.DisablePreshardingEncryption || opts.EncryptionOffset != 0 { - values.Set("offset", fmt.Sprint(opts.EncryptionOffset)) + if opts.EncryptionOffset != nil { + values.Set("offset", fmt.Sprint(*opts.EncryptionOffset)) } } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 1b57e1e6c..61be076b7 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -2015,7 +2015,7 @@ func TestMultipartUploads(t *testing.T) { // correctly. putPart := func(partNum int, offset int, data []byte) string { t.Helper() - res, err := w.UploadMultipartUploadPart(context.Background(), bytes.NewReader(data), api.DefaultBucketName, objPath, mpr.UploadID, partNum, api.UploadMultipartUploadPartOptions{EncryptionOffset: offset}) + res, err := w.UploadMultipartUploadPart(context.Background(), bytes.NewReader(data), api.DefaultBucketName, objPath, mpr.UploadID, partNum, api.UploadMultipartUploadPartOptions{EncryptionOffset: &offset}) tt.OK(err) if res.ETag == "" { t.Fatal("expected non-empty ETag") @@ -2362,22 +2362,25 @@ func TestMultipartUploadWrappedByPartialSlabs(t *testing.T) { // upload a part that is a partial slab part3Data := bytes.Repeat([]byte{3}, int(slabSize)/4) + offset := int(slabSize + slabSize/4) resp3, err := w.UploadMultipartUploadPart(context.Background(), bytes.NewReader(part3Data), api.DefaultBucketName, objPath, mpr.UploadID, 3, api.UploadMultipartUploadPartOptions{ - EncryptionOffset: int(slabSize + slabSize/4), + EncryptionOffset: &offset, }) tt.OK(err) // upload a part that is exactly a full slab part2Data := bytes.Repeat([]byte{2}, int(slabSize)) + offset = int(slabSize / 4) resp2, err := w.UploadMultipartUploadPart(context.Background(), bytes.NewReader(part2Data), api.DefaultBucketName, objPath, mpr.UploadID, 2, api.UploadMultipartUploadPartOptions{ - EncryptionOffset: int(slabSize / 4), + EncryptionOffset: &offset, }) tt.OK(err) // upload another part the same size as the first one part1Data := bytes.Repeat([]byte{1}, int(slabSize)/4) + offset = 0 resp1, err := w.UploadMultipartUploadPart(context.Background(), bytes.NewReader(part1Data), api.DefaultBucketName, objPath, mpr.UploadID, 1, api.UploadMultipartUploadPartOptions{ - EncryptionOffset: 0, + EncryptionOffset: &offset, }) tt.OK(err) diff --git a/s3/backend.go b/s3/backend.go index a481da727..62dc8ddf5 100644 --- a/s3/backend.go +++ b/s3/backend.go @@ -418,8 +418,7 @@ func (s *s3) CreateMultipartUpload(ctx context.Context, bucket, key string, meta func (s *s3) UploadPart(ctx context.Context, bucket, object string, id gofakes3.UploadID, partNumber int, contentLength int64, input io.Reader) (*gofakes3.UploadPartResult, error) { res, err := s.w.UploadMultipartUploadPart(ctx, input, bucket, object, string(id), partNumber, api.UploadMultipartUploadPartOptions{ - DisablePreshardingEncryption: true, - ContentLength: contentLength, + ContentLength: contentLength, }) if err != nil { return nil, gofakes3.ErrorMessage(gofakes3.ErrInternal, err.Error()) From 07bbbb11a7f7aeea85d23df6eb32530d6942e809 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 15:39:13 +0100 Subject: [PATCH 080/144] testing: fix TestMultipartUploads --- api/multipart.go | 12 ++++++------ bus/bus.go | 6 ++++-- internal/testing/cluster_test.go | 3 ++- s3/backend.go | 2 +- 4 files changed, 13 insertions(+), 10 deletions(-) diff --git a/api/multipart.go b/api/multipart.go index e786a7c25..a191b2b13 100644 --- a/api/multipart.go +++ b/api/multipart.go @@ -47,7 +47,7 @@ type ( CreateMultipartOptions struct { GenerateKey bool - Key object.EncryptionKey + Key *object.EncryptionKey MimeType string Metadata ObjectUserMetadata } @@ -82,11 +82,11 @@ type ( } MultipartCreateRequest struct { - Bucket string `json:"bucket"` - Path string `json:"path"` - Key object.EncryptionKey `json:"key"` - MimeType string `json:"mimeType"` - Metadata ObjectUserMetadata `json:"metadata"` + Bucket string `json:"bucket"` + Path string `json:"path"` + Key *object.EncryptionKey `json:"key"` + MimeType string `json:"mimeType"` + Metadata ObjectUserMetadata `json:"metadata"` // TODO: The next major version change should invert this to create a // key by default diff --git a/bus/bus.go b/bus/bus.go index 58739e11a..953da9976 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -2177,11 +2177,13 @@ func (b *bus) multipartHandlerCreatePOST(jc jape.Context) { return } - key := req.Key + var key object.EncryptionKey if req.GenerateKey { key = object.GenerateEncryptionKey() - } else if key == (object.EncryptionKey{}) { + } else if req.Key == nil { key = object.NoOpKey + } else { + key = *req.Key } resp, err := b.ms.CreateMultipartUpload(jc.Request.Context(), req.Bucket, req.Path, key, req.MimeType, req.Metadata) diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 61be076b7..61174a18f 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -2354,7 +2354,8 @@ func TestMultipartUploadWrappedByPartialSlabs(t *testing.T) { // start a new multipart upload. We upload the parts in reverse order objPath := "/foo" - mpr, err := b.CreateMultipartUpload(context.Background(), api.DefaultBucketName, objPath, api.CreateMultipartOptions{Key: object.GenerateEncryptionKey()}) + key := object.GenerateEncryptionKey() + mpr, err := b.CreateMultipartUpload(context.Background(), api.DefaultBucketName, objPath, api.CreateMultipartOptions{Key: &key}) tt.OK(err) if mpr.UploadID == "" { t.Fatal("expected non-empty upload ID") diff --git a/s3/backend.go b/s3/backend.go index 62dc8ddf5..de72bb005 100644 --- a/s3/backend.go +++ b/s3/backend.go @@ -405,7 +405,7 @@ func (s *s3) CopyObject(ctx context.Context, srcBucket, srcKey, dstBucket, dstKe func (s *s3) CreateMultipartUpload(ctx context.Context, bucket, key string, meta map[string]string) (gofakes3.UploadID, error) { convertToSiaMetadataHeaders(meta) resp, err := s.b.CreateMultipartUpload(ctx, bucket, "/"+key, api.CreateMultipartOptions{ - Key: object.NoOpKey, + Key: &object.NoOpKey, MimeType: meta["Content-Type"], Metadata: api.ExtractObjectUserMetadataFrom(meta), }) From 9051e472f06409d5e4613375b47f17f0980933fb Mon Sep 17 00:00:00 2001 From: PJ Date: Fri, 16 Feb 2024 15:46:35 +0100 Subject: [PATCH 081/144] stores: get rid of migrate bool --- stores/migrations.go | 6 ++-- stores/migrations/mysql/main/schema.sql | 5 ++- stores/migrations/sqlite/main/schema.sql | 5 ++- stores/migrations_metrics.go | 2 +- stores/migrations_utils.go | 42 ++++++------------------ 5 files changed, 22 insertions(+), 38 deletions(-) diff --git a/stores/migrations.go b/stores/migrations.go index 05bda3f69..7f225ff17 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -24,13 +24,13 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { { ID: "00001_object_metadata", Migrate: func(tx *gorm.DB) error { - return performMigration(tx, "00001_object_metadata", false, logger) + return performMigration(tx, "main", "00001_object_metadata", logger) }, }, { ID: "00002_prune_slabs_trigger", Migrate: func(tx *gorm.DB) error { - err := performMigration(tx, "00002_prune_slabs_trigger", false, logger) + err := performMigration(tx, "main", "00002_prune_slabs_trigger", logger) if err != nil && strings.Contains(err.Error(), errMySQLNoSuperPrivilege.Error()) { logger.Warn("migration 00002_prune_slabs_trigger requires the user to have the SUPER privilege to register triggers") } @@ -43,7 +43,7 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { m := gormigrate.New(db, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initSchema(db, false, logger)) + m.InitSchema(initSchema(db, "main", logger)) // Perform migrations. if err := m.Migrate(); err != nil { diff --git a/stores/migrations/mysql/main/schema.sql b/stores/migrations/mysql/main/schema.sql index 39bf279f0..d28bdd13f 100644 --- a/stores/migrations/mysql/main/schema.sql +++ b/stores/migrations/mysql/main/schema.sql @@ -452,4 +452,7 @@ AND NOT EXISTS ( SELECT 1 FROM slices WHERE slices.db_slab_id = OLD.db_slab_id -); \ No newline at end of file +); + +-- create default bucket +INSERT INTO buckets (created_at, name) VALUES (CURRENT_TIMESTAMP, 'default'); \ No newline at end of file diff --git a/stores/migrations/sqlite/main/schema.sql b/stores/migrations/sqlite/main/schema.sql index df9fc9a83..daee619b4 100644 --- a/stores/migrations/sqlite/main/schema.sql +++ b/stores/migrations/sqlite/main/schema.sql @@ -183,4 +183,7 @@ BEGIN FROM slices WHERE slices.db_slab_id = OLD.db_slab_id ); -END; \ No newline at end of file +END; + +-- create default bucket +INSERT INTO buckets (created_at, name) VALUES (CURRENT_TIMESTAMP, 'default'); diff --git a/stores/migrations_metrics.go b/stores/migrations_metrics.go index ddbbe8e4e..940917569 100644 --- a/stores/migrations_metrics.go +++ b/stores/migrations_metrics.go @@ -20,7 +20,7 @@ func performMetricsMigrations(tx *gorm.DB, logger *zap.SugaredLogger) error { m := gormigrate.New(tx, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initSchema(tx, true, logger)) + m.InitSchema(initSchema(tx, "metrics", logger)) // Perform migrations. if err := m.Migrate(); err != nil { diff --git a/stores/migrations_utils.go b/stores/migrations_utils.go index 4832d96a7..46d7f3dc4 100644 --- a/stores/migrations_utils.go +++ b/stores/migrations_utils.go @@ -4,64 +4,42 @@ import ( "fmt" gormigrate "github.com/go-gormigrate/gormigrate/v2" - "go.sia.tech/renterd/api" "go.uber.org/zap" "gorm.io/gorm" ) // initSchema is executed only on a clean database. Otherwise the individual // migrations are executed. -func initSchema(db *gorm.DB, metrics bool, logger *zap.SugaredLogger) gormigrate.InitSchemaFunc { +func initSchema(db *gorm.DB, name string, logger *zap.SugaredLogger) gormigrate.InitSchemaFunc { return func(tx *gorm.DB) error { - if metrics { - logger.Info("initializing metrics schema") - } else { - logger.Info("initializing schema") - } + logger.Infof("initializing '%s' schema", name) - // build filename - filename := "schema" - err := execSQLFile(tx, metrics, filename) + // init schema + err := execSQLFile(tx, name, "schema") if err != nil { return fmt.Errorf("failed to init schema: %w", err) } - // add default bucket. - if !metrics { - if err := tx.Create(&dbBucket{ - Name: api.DefaultBucketName, - }).Error; err != nil { - return fmt.Errorf("failed to create default bucket: %v", err) - } - } - logger.Info("initialization complete") return nil } } -func performMigration(db *gorm.DB, name string, metrics bool, logger *zap.SugaredLogger) error { - logger.Infof("performing migration %s", name) - - // build filename - filename := fmt.Sprintf("migration_%s", name) +func performMigration(db *gorm.DB, kind, migration string, logger *zap.SugaredLogger) error { + logger.Infof("performing %s migration '%s'", kind, migration) // execute migration - err := execSQLFile(db, metrics, filename) + err := execSQLFile(db, kind, fmt.Sprintf("migration_%s", migration)) if err != nil { - return fmt.Errorf("migration %s failed: %w", name, err) + return fmt.Errorf("migration '%s' failed: %w", migration, err) } - logger.Infof("migration %s complete", name) + logger.Infof("migration '%s' complete", migration) return nil } -func execSQLFile(db *gorm.DB, metrics bool, filename string) error { +func execSQLFile(db *gorm.DB, folder, filename string) error { // build path - folder := "main" - if metrics { - folder = "metrics" - } protocol := "mysql" if isSQLite(db) { protocol = "sqlite" From e0b3dc03e4069211961f5bad8b88864839547dd1 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 15:46:45 +0100 Subject: [PATCH 082/144] object: fix panic in objects.String --- object/object.go | 3 --- 1 file changed, 3 deletions(-) diff --git a/object/object.go b/object/object.go index e3c2fe3e7..7c74c1c23 100644 --- a/object/object.go +++ b/object/object.go @@ -56,9 +56,6 @@ func (k EncryptionKey) MarshalText() ([]byte, error) { // UnmarshalText implements the encoding.TextUnmarshaler interface. func (k *EncryptionKey) UnmarshalText(b []byte) error { - if len(b) == 0 { - return nil - } k.entropy = new([32]byte) if n, err := hex.Decode(k.entropy[:], []byte(bytes.TrimPrefix(b, []byte("key:")))); err != nil { return err From e7f6bb417c03092cf46aef1903e3e09b1e2bbd82 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 16 Feb 2024 16:23:04 +0100 Subject: [PATCH 083/144] testing: fix TestS3Basic --- api/object.go | 2 +- internal/testing/cluster_test.go | 2 +- stores/metadata.go | 2 +- worker/mocks_test.go | 2 +- worker/upload_test.go | 14 +++++++------- worker/worker.go | 2 +- 6 files changed, 12 insertions(+), 12 deletions(-) diff --git a/api/object.go b/api/object.go index 09f4b04ab..e8ae7fea8 100644 --- a/api/object.go +++ b/api/object.go @@ -54,7 +54,7 @@ type ( Object struct { Metadata ObjectUserMetadata `json:"metadata,omitempty"` ObjectMetadata - object.Object `json:"omitempty"` + *object.Object `json:"omitempty"` } // ObjectMetadata contains various metadata about an object. diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index d290faf27..81779eeac 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1313,7 +1313,7 @@ func TestUploadDownloadSameHost(t *testing.T) { // build a frankenstein object constructed with all sectors on the same host res.Object.Slabs[0].Shards = shards[res.Object.Slabs[0].Shards[0].LatestHost] - tt.OK(b.AddObject(context.Background(), api.DefaultBucketName, "frankenstein", testContractSet, res.Object.Object, api.AddObjectOptions{})) + tt.OK(b.AddObject(context.Background(), api.DefaultBucketName, "frankenstein", testContractSet, *res.Object.Object, api.AddObjectOptions{})) // assert we can download this object tt.OK(w.DownloadObject(context.Background(), io.Discard, api.DefaultBucketName, "frankenstein", api.DownloadObjectOptions{})) diff --git a/stores/metadata.go b/stores/metadata.go index 15a4690ad..0e659a243 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -2329,7 +2329,7 @@ func (s *SQLStore) objectHydrate(ctx context.Context, tx *gorm.DB, bucket, path obj[0].ObjectModTime, obj[0].ObjectSize, ), - Object: object.Object{ + Object: &object.Object{ Key: key, Slabs: slabs, }, diff --git a/worker/mocks_test.go b/worker/mocks_test.go index e6fd62d8e..2490941af 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -274,7 +274,7 @@ func (os *mockObjectStore) Object(ctx context.Context, bucket, path string, opts return api.ObjectsResponse{Object: &api.Object{ ObjectMetadata: api.ObjectMetadata{Name: path, Size: o.TotalSize()}, - Object: o, + Object: &o, }}, nil } diff --git a/worker/upload_test.go b/worker/upload_test.go index 8d32455bd..0b9f6b28b 100644 --- a/worker/upload_test.go +++ b/worker/upload_test.go @@ -64,7 +64,7 @@ func TestUpload(t *testing.T) { // 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), w.contracts()) + err = dl.DownloadObject(context.Background(), &buf, *o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -90,7 +90,7 @@ func TestUpload(t *testing.T) { // 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) + 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()) { @@ -107,7 +107,7 @@ func TestUpload(t *testing.T) { // 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) + 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) } @@ -170,7 +170,7 @@ func TestUploadPackedSlab(t *testing.T) { // 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), w.contracts()) + err = dl.DownloadObject(context.Background(), &buf, *o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -206,7 +206,7 @@ func TestUploadPackedSlab(t *testing.T) { // download the data again and assert it matches buf.Reset() - err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) + err = dl.DownloadObject(context.Background(), &buf, *o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -325,7 +325,7 @@ func TestUploadShards(t *testing.T) { // 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), contracts) + err = dl.DownloadObject(context.Background(), &buf, *o.Object.Object, 0, uint64(o.Object.Size), contracts) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { @@ -485,7 +485,7 @@ func TestUploadRegression(t *testing.T) { // download data for good measure var buf bytes.Buffer - err = dl.DownloadObject(context.Background(), &buf, o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) + err = dl.DownloadObject(context.Background(), &buf, *o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) if err != nil { t.Fatal(err) } else if !bytes.Equal(data, buf.Bytes()) { diff --git a/worker/worker.go b/worker/worker.go index 8a2a9b1f3..95497b383 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -919,7 +919,7 @@ func (w *worker) objectsHandlerGET(jc jape.Context) { // create a download function downloadFn := func(wr io.Writer, offset, length int64) (err error) { ctx = WithGougingChecker(ctx, w.bus, gp) - err = w.downloadManager.DownloadObject(ctx, wr, res.Object.Object, uint64(offset), uint64(length), contracts) + err = w.downloadManager.DownloadObject(ctx, wr, *res.Object.Object, uint64(offset), uint64(length), contracts) if err != nil { w.logger.Error(err) if !errors.Is(err, ErrShuttingDown) { From 93bb567a1158646c6f60cf5a7b0d9544e5cd848b Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 01:42:17 +0000 Subject: [PATCH 084/144] build(deps): bump github.com/minio/minio-go/v7 from 7.0.66 to 7.0.67 Bumps [github.com/minio/minio-go/v7](https://github.com/minio/minio-go) from 7.0.66 to 7.0.67. - [Release notes](https://github.com/minio/minio-go/releases) - [Commits](https://github.com/minio/minio-go/compare/v7.0.66...v7.0.67) --- updated-dependencies: - dependency-name: github.com/minio/minio-go/v7 dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 2659a1ceb..a2e8a45f5 100644 --- a/go.mod +++ b/go.mod @@ -10,7 +10,7 @@ require ( github.com/google/go-cmp v0.6.0 github.com/gotd/contrib v0.19.0 github.com/klauspost/reedsolomon v1.12.1 - github.com/minio/minio-go/v7 v7.0.66 + github.com/minio/minio-go/v7 v7.0.67 github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b diff --git a/go.sum b/go.sum index b7f5f9cc7..57c101799 100644 --- a/go.sum +++ b/go.sum @@ -135,8 +135,8 @@ github.com/mattn/go-sqlite3 v1.14.18/go.mod h1:2eHXhiwb8IkHr+BDWZGa96P6+rkvnG63S github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/minio/md5-simd v1.1.2 h1:Gdi1DZK69+ZVMoNHRXJyNcxrMA4dSxoYHZSQbirFg34= github.com/minio/md5-simd v1.1.2/go.mod h1:MzdKDxYpY2BT9XQFocsiZf/NKVtR7nkE4RoEpN+20RM= -github.com/minio/minio-go/v7 v7.0.66 h1:bnTOXOHjOqv/gcMuiVbN9o2ngRItvqE774dG9nq0Dzw= -github.com/minio/minio-go/v7 v7.0.66/go.mod h1:DHAgmyQEGdW3Cif0UooKOyrT3Vxs82zNdV6tkKhRtbs= +github.com/minio/minio-go/v7 v7.0.67 h1:BeBvZWAS+kRJm1vGTMJYVjKUNoo0FoEt/wUWdUtfmh8= +github.com/minio/minio-go/v7 v7.0.67/go.mod h1:+UXocnUeZ3wHvVh5s95gcrA4YjMIbccT6ubB+1m054A= github.com/minio/sha256-simd v1.0.1 h1:6kaan5IFmwTNynnKKpDHe6FWHohJOHhCPchzK49dzMM= github.com/minio/sha256-simd v1.0.1/go.mod h1:Pz6AKMiUdngCLpeTL/RJY1M9rUuPMYujV5xJjtbRSN8= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= From 9431952288392047e4e30d250586eec413c8c4bf Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 19 Feb 2024 01:42:22 +0000 Subject: [PATCH 085/144] build(deps): bump gorm.io/driver/mysql from 1.5.2 to 1.5.4 Bumps [gorm.io/driver/mysql](https://github.com/go-gorm/mysql) from 1.5.2 to 1.5.4. - [Commits](https://github.com/go-gorm/mysql/compare/v1.5.2...v1.5.4) --- updated-dependencies: - dependency-name: gorm.io/driver/mysql dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index 2659a1ceb..03815a315 100644 --- a/go.mod +++ b/go.mod @@ -24,7 +24,7 @@ require ( golang.org/x/crypto v0.19.0 golang.org/x/term v0.17.0 gopkg.in/yaml.v3 v3.0.1 - gorm.io/driver/mysql v1.5.2 + gorm.io/driver/mysql v1.5.4 gorm.io/driver/sqlite v1.5.5 gorm.io/gorm v1.25.7 lukechampine.com/frand v1.4.2 diff --git a/go.sum b/go.sum index b7f5f9cc7..ee2745f86 100644 --- a/go.sum +++ b/go.sum @@ -389,11 +389,11 @@ gopkg.in/yaml.v2 v2.4.0/go.mod h1:RDklbk79AGWmwhnvt/jBztapEOGDOx6ZbXqjP6csGnQ= gopkg.in/yaml.v3 v3.0.0-20200313102051-9f266ea9e77c/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= gopkg.in/yaml.v3 v3.0.1 h1:fxVm/GzAzEWqLHuvctI91KS9hhNmmWOoWu0XTYJS7CA= gopkg.in/yaml.v3 v3.0.1/go.mod h1:K4uyk7z7BCEPqu6E+C64Yfv1cQ7kz7rIZviUmN+EgEM= -gorm.io/driver/mysql v1.5.2 h1:QC2HRskSE75wBuOxe0+iCkyJZ+RqpudsQtqkp+IMuXs= -gorm.io/driver/mysql v1.5.2/go.mod h1:pQLhh1Ut/WUAySdTHwBpBv6+JKcj+ua4ZFx1QQTBzb8= +gorm.io/driver/mysql v1.5.4 h1:igQmHfKcbaTVyAIHNhhB888vvxh8EdQ2uSUT0LPcBso= +gorm.io/driver/mysql v1.5.4/go.mod h1:9rYxJph/u9SWkWc9yY4XJ1F/+xO0S/ChOmbk3+Z5Tvs= gorm.io/driver/sqlite v1.5.5 h1:7MDMtUZhV065SilG62E0MquljeArQZNfJnjd9i9gx3E= gorm.io/driver/sqlite v1.5.5/go.mod h1:6NgQ7sQWAIFsPrJJl1lSNSu2TABh0ZZ/zm5fosATavE= -gorm.io/gorm v1.25.2-0.20230530020048-26663ab9bf55/go.mod h1:L4uxeKpfBml98NYqVqwAdmV1a2nBtAec/cf3fpucW/k= +gorm.io/gorm v1.25.7-0.20240204074919-46816ad31dde/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= gorm.io/gorm v1.25.7 h1:VsD6acwRjz2zFxGO50gPO6AkNs7KKnvfzUjHQhZDz/A= gorm.io/gorm v1.25.7/go.mod h1:hbnx/Oo0ChWMn1BIhpy1oYozzpM15i4YPuHDmfYtwg8= honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= From 264e4a64f492ffb04fcb3b9ca7476dc45b85fa9b Mon Sep 17 00:00:00 2001 From: Peter-Jan Brone Date: Mon, 19 Feb 2024 10:25:00 +0100 Subject: [PATCH 086/144] Extend contract set change alerts with host info (#966) * contractor: extend contract set alert with host info * autopilot: get rid of set in contract set change alert --- autopilot/alerts.go | 22 +++++-- autopilot/contractor.go | 136 +++++++++++++++++++++++----------------- 2 files changed, 95 insertions(+), 63 deletions(-) diff --git a/autopilot/alerts.go b/autopilot/alerts.go index c279075ee..292670dc5 100644 --- a/autopilot/alerts.go +++ b/autopilot/alerts.go @@ -137,22 +137,32 @@ func newContractPruningFailedAlert(hk types.PublicKey, version string, fcid type } } -func newContractSetChangeAlert(name string, added, removed int, removedReasons map[string]string) alerts.Alert { +func newContractSetChangeAlert(name string, additions map[types.FileContractID]contractSetAddition, removals map[types.FileContractID]contractSetRemoval) alerts.Alert { var hint string - if removed > 0 { + if len(removals) > 0 { hint = "A high churn rate can lead to a lot of unnecessary migrations, it might be necessary to tweak your configuration depending on the reason hosts are being discarded from the set." } + removedReasons := make(map[string]string, len(removals)) + for k, v := range removals { + removedReasons[k.String()] = v.Reason + } + return alerts.Alert{ ID: randomAlertID(), Severity: alerts.SeverityInfo, Message: "Contract set changed", Data: map[string]any{ - "name": name, - "added": added, - "removed": removed, + "name": name, + "set_additions": additions, + "set_removals": removals, + "hint": hint, + + // TODO: these fields can be removed on the next major release, they + // contain redundant information + "added": len(additions), + "removed": len(removals), "removals": removedReasons, - "hint": hint, }, Timestamp: time.Now(), } diff --git a/autopilot/contractor.go b/autopilot/contractor.go index bc947d500..092f2a831 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -122,9 +122,20 @@ type ( recoverable bool } + contractSetAddition struct { + Size uint64 `json:"size"` + HostKey types.PublicKey `json:"hostKey"` + } + + contractSetRemoval struct { + Size uint64 `json:"size"` + HostKey types.PublicKey `json:"hostKey"` + Reason string `json:"reason"` + } + renewal struct { - from types.FileContractID - to types.FileContractID + from api.ContractMetadata + to api.ContractMetadata ci contractInfo } ) @@ -331,17 +342,15 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( // set afterwards var renewed []renewal if limit > 0 { - var toKeep []contractInfo + var toKeep []api.ContractMetadata renewed, toKeep = c.runContractRenewals(ctx, w, toRenew, &remaining, limit) for _, ri := range renewed { if ri.ci.usable || ri.ci.recoverable { updatedSet = append(updatedSet, ri.to) } - contractData[ri.to] = contractData[ri.from] - } - for _, ci := range toKeep { - updatedSet = append(updatedSet, ci.contract.ID) + contractData[ri.to.ID] = contractData[ri.from.ID] } + updatedSet = append(updatedSet, toKeep...) } // run contract refreshes @@ -353,7 +362,7 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( if ri.ci.usable || ri.ci.recoverable { updatedSet = append(updatedSet, ri.to) } - contractData[ri.to] = contractData[ri.from] + contractData[ri.to.ID] = contractData[ri.from.ID] } } @@ -366,7 +375,7 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( } // check if we need to form contracts and add them to the contract set - var formed []types.FileContractID + var formed []api.ContractMetadata if uint64(len(updatedSet)) < threshold { // no need to try and form contracts if wallet is completely empty wallet, err := c.ap.bus.Wallet(ctx) @@ -382,34 +391,40 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( } else { for _, fc := range formed { updatedSet = append(updatedSet, fc) - contractData[fc] = 0 + contractData[fc.ID] = 0 } } } } // cap the amount of contracts we want to keep to the configured amount - for _, fcid := range updatedSet { - if _, exists := contractData[fcid]; !exists { - c.logger.Errorf("contract %v not found in contractData", fcid) + for _, contract := range updatedSet { + if _, exists := contractData[contract.ID]; !exists { + c.logger.Errorf("contract %v not found in contractData", contract.ID) } } if len(updatedSet) > int(state.cfg.Contracts.Amount) { // sort by contract size sort.Slice(updatedSet, func(i, j int) bool { - return contractData[updatedSet[i]] > contractData[updatedSet[j]] + return contractData[updatedSet[i].ID] > contractData[updatedSet[j].ID] }) - for _, c := range updatedSet[state.cfg.Contracts.Amount:] { - toStopUsing[c] = "truncated" + for _, contract := range updatedSet[state.cfg.Contracts.Amount:] { + toStopUsing[contract.ID] = "truncated" } updatedSet = updatedSet[:state.cfg.Contracts.Amount] } + // convert to set of file contract ids + var newSet []types.FileContractID + for _, contract := range updatedSet { + newSet = append(newSet, contract.ID) + } + // update contract set if c.ap.isStopped() { return false, errors.New("autopilot stopped before maintenance could be completed") } - err = c.ap.bus.SetContractSet(ctx, state.cfg.Contracts.Set, updatedSet) + err = c.ap.bus.SetContractSet(ctx, state.cfg.Contracts.Set, newSet) if err != nil { return false, err } @@ -418,54 +433,62 @@ func (c *contractor) performContractMaintenance(ctx context.Context, w Worker) ( return c.computeContractSetChanged(ctx, state.cfg.Contracts.Set, currentSet, updatedSet, formed, refreshed, renewed, toStopUsing, contractData), nil } -func (c *contractor) computeContractSetChanged(ctx context.Context, name string, oldSet []api.ContractMetadata, newSet, formed []types.FileContractID, refreshed, renewed []renewal, toStopUsing map[types.FileContractID]string, contractData map[types.FileContractID]uint64) bool { - // build some maps for easier lookups - previous := make(map[types.FileContractID]struct{}) +func (c *contractor) computeContractSetChanged(ctx context.Context, name string, oldSet, newSet []api.ContractMetadata, formed []api.ContractMetadata, refreshed, renewed []renewal, toStopUsing map[types.FileContractID]string, contractData map[types.FileContractID]uint64) bool { + // build set lookups + inOldSet := make(map[types.FileContractID]struct{}) for _, c := range oldSet { - previous[c.ID] = struct{}{} + inOldSet[c.ID] = struct{}{} } - updated := make(map[types.FileContractID]struct{}) + inNewSet := make(map[types.FileContractID]struct{}) for _, c := range newSet { - updated[c] = struct{}{} + inNewSet[c.ID] = struct{}{} } + + // build renewal lookups renewalsFromTo := make(map[types.FileContractID]types.FileContractID) renewalsToFrom := make(map[types.FileContractID]types.FileContractID) for _, c := range append(refreshed, renewed...) { - renewalsFromTo[c.from] = c.to - renewalsToFrom[c.to] = c.from + renewalsFromTo[c.from.ID] = c.to.ID + renewalsToFrom[c.to.ID] = c.from.ID } // log added and removed contracts - var added []types.FileContractID - var removed []types.FileContractID - removedReasons := make(map[string]string) + setAdditions := make(map[types.FileContractID]contractSetAddition) + setRemovals := make(map[types.FileContractID]contractSetRemoval) for _, contract := range oldSet { - _, exists := updated[contract.ID] - _, renewed := updated[renewalsFromTo[contract.ID]] + _, exists := inNewSet[contract.ID] + _, renewed := inNewSet[renewalsFromTo[contract.ID]] if !exists && !renewed { - removed = append(removed, contract.ID) reason, ok := toStopUsing[contract.ID] if !ok { reason = "unknown" } - removedReasons[contract.ID.String()] = reason + + setRemovals[contract.ID] = contractSetRemoval{ + Size: contractData[contract.ID], + HostKey: contract.HostKey, + Reason: reason, + } c.logger.Debugf("contract %v was removed from the contract set, size: %v, reason: %v", contract.ID, contractData[contract.ID], reason) } } - for _, fcid := range newSet { - _, existed := previous[fcid] - _, renewed := renewalsToFrom[fcid] + for _, contract := range newSet { + _, existed := inOldSet[contract.ID] + _, renewed := renewalsToFrom[contract.ID] if !existed && !renewed { - added = append(added, fcid) - c.logger.Debugf("contract %v was added to the contract set, size: %v", fcid, contractData[fcid]) + setAdditions[contract.ID] = contractSetAddition{ + Size: contractData[contract.ID], + HostKey: contract.HostKey, + } + c.logger.Debugf("contract %v was added to the contract set, size: %v", contract.ID, contractData[contract.ID]) } } // log renewed contracts that did not make it into the contract set for _, fcid := range renewed { - _, exists := updated[fcid.to] + _, exists := inNewSet[fcid.to.ID] if !exists { - c.logger.Debugf("contract %v was renewed but did not make it into the contract set, size: %v", fcid, contractData[fcid.to]) + c.logger.Debugf("contract %v was renewed but did not make it into the contract set, size: %v", fcid, contractData[fcid.to.ID]) } } @@ -478,7 +501,7 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, // record churn metrics now := api.TimeNow() var metrics []api.ContractSetChurnMetric - for _, fcid := range added { + for fcid := range setAdditions { metrics = append(metrics, api.ContractSetChurnMetric{ Name: c.ap.state.cfg.Contracts.Set, ContractID: fcid, @@ -486,12 +509,12 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, Timestamp: now, }) } - for _, fcid := range removed { + for fcid, removal := range setRemovals { metrics = append(metrics, api.ContractSetChurnMetric{ Name: c.ap.state.cfg.Contracts.Set, ContractID: fcid, Direction: api.ChurnDirRemoved, - Reason: removedReasons[fcid.String()], + Reason: removal.Reason, Timestamp: now, }) } @@ -508,12 +531,12 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, "renewed", len(renewed), "refreshed", len(refreshed), "contracts", len(newSet), - "added", len(added), - "removed", len(removed), + "added", len(setAdditions), + "removed", len(setRemovals), ) - hasChanged := len(added)+len(removed) > 0 + hasChanged := len(setAdditions)+len(setRemovals) > 0 if hasChanged { - c.ap.RegisterAlert(ctx, newContractSetChangeAlert(name, len(added), len(removed), removedReasons)) + c.ap.RegisterAlert(ctx, newContractSetChangeAlert(name, setAdditions, setRemovals)) } return hasChanged } @@ -608,7 +631,7 @@ func (c *contractor) performWalletMaintenance(ctx context.Context) error { return nil } -func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts []api.Contract, inCurrentSet map[types.FileContractID]struct{}, minScore float64) (toKeep []types.FileContractID, toArchive, toStopUsing map[types.FileContractID]string, toRefresh, toRenew []contractInfo, _ error) { +func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts []api.Contract, inCurrentSet map[types.FileContractID]struct{}, minScore float64) (toKeep []api.ContractMetadata, toArchive, toStopUsing map[types.FileContractID]string, toRefresh, toRenew []contractInfo, _ error) { if c.ap.isStopped() { return } @@ -740,7 +763,7 @@ func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts } else if !state.cfg.Hosts.AllowRedundantIPs && ipFilter.IsRedundantIP(contract.HostIP, contract.HostKey) { toStopUsing[fcid] = fmt.Sprintf("%v; %v", errHostRedundantIP, errContractNoRevision) } else { - toKeep = append(toKeep, fcid) + toKeep = append(toKeep, contract.ContractMetadata) remainingKeepLeeway-- // we let it slide } continue // can't perform contract checks without revision @@ -783,18 +806,17 @@ func (c *contractor) runContractChecks(ctx context.Context, w Worker, contracts } else if refresh { toRefresh = append(toRefresh, ci) } else if usable { - toKeep = append(toKeep, ci.contract.ID) + toKeep = append(toKeep, ci.contract.ContractMetadata) } } return toKeep, toArchive, toStopUsing, toRefresh, toRenew, nil } -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) { +func (c *contractor) runContractFormations(ctx context.Context, w Worker, candidates scoredHosts, usedHosts map[types.PublicKey]struct{}, unusableHosts unusableHostResult, missing uint64, budget *types.Currency) (formed []api.ContractMetadata, _ error) { if c.ap.isStopped() { return nil, nil } - var formed []types.FileContractID // convenience variables state := c.ap.State() @@ -896,7 +918,7 @@ func (c *contractor) runContractFormations(ctx context.Context, w Worker, candid formedContract, proceed, err := c.formContract(ctx, w, host, minInitialContractFunds, maxInitialContractFunds, budget) if err == nil { // add contract to contract set - formed = append(formed, formedContract.ID) + formed = append(formed, formedContract) missing-- } if !proceed { @@ -976,7 +998,7 @@ 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) { +func (c *contractor) runContractRenewals(ctx context.Context, w Worker, toRenew []contractInfo, budget *types.Currency, limit int) (renewals []renewal, toKeep []api.ContractMetadata) { c.logger.Debugw( "run contracts renewals", "torenew", len(toRenew), @@ -1010,11 +1032,11 @@ func (c *contractor) runContractRenewals(ctx context.Context, w Worker, toRenew if err != nil { c.ap.RegisterAlert(ctx, newContractRenewalFailedAlert(contract, !proceed, err)) if toRenew[i].usable { - toKeep = append(toKeep, toRenew[i]) + toKeep = append(toKeep, toRenew[i].contract.ContractMetadata) } } else { c.ap.DismissAlert(ctx, alertIDForContract(alertRenewalFailedID, contract.ID)) - renewals = append(renewals, renewal{from: contract.ID, to: renewed.ID, ci: toRenew[i]}) + renewals = append(renewals, renewal{from: contract, to: renewed, ci: toRenew[i]}) } // break if we don't want to proceed @@ -1027,7 +1049,7 @@ func (c *contractor) runContractRenewals(ctx context.Context, w Worker, toRenew // they're usable and we have 'limit' left for j := i; j < len(toRenew); j++ { if len(renewals)+len(toKeep) < limit && toRenew[j].usable { - toKeep = append(toKeep, toRenew[j]) + toKeep = append(toKeep, toRenew[j].contract.ContractMetadata) } } @@ -1057,7 +1079,7 @@ func (c *contractor) runContractRefreshes(ctx context.Context, w Worker, toRefre // refresh and add if it succeeds renewed, proceed, err := c.refreshContract(ctx, w, ci, budget) if err == nil { - refreshed = append(refreshed, renewal{from: ci.contract.ID, to: renewed.ID, ci: ci}) + refreshed = append(refreshed, renewal{from: ci.contract.ContractMetadata, to: renewed, ci: ci}) } // break if we don't want to proceed From ac15fc80a73d0854831080dbfbcf3acb577f3891 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 10:44:31 +0100 Subject: [PATCH 087/144] go.mod: uprade hostd dependency --- build/network.go | 2 +- cmd/renterd/config.go | 2 +- go.mod | 17 ++++++++--------- go.sum | 30 ++++++++++++++++-------------- internal/node/transactionpool.go | 2 +- stores/wallet.go | 6 +++--- 6 files changed, 30 insertions(+), 29 deletions(-) diff --git a/build/network.go b/build/network.go index 4183a62bc..a0a452189 100644 --- a/build/network.go +++ b/build/network.go @@ -3,9 +3,9 @@ package build //go:generate go run gen.go import ( - "go.sia.tech/core/chain" "go.sia.tech/core/consensus" "go.sia.tech/core/types" + "go.sia.tech/coreutils/chain" ) // Network returns the Sia network consts and genesis block for the current build. diff --git a/cmd/renterd/config.go b/cmd/renterd/config.go index 391d77ea3..47668ff94 100644 --- a/cmd/renterd/config.go +++ b/cmd/renterd/config.go @@ -11,7 +11,7 @@ import ( "strings" "go.sia.tech/core/types" - "go.sia.tech/core/wallet" + "go.sia.tech/coreutils/wallet" "golang.org/x/term" "gopkg.in/yaml.v3" "lukechampine.com/frand" diff --git a/go.mod b/go.mod index db441a0ec..e727483d1 100644 --- a/go.mod +++ b/go.mod @@ -1,8 +1,6 @@ module go.sia.tech/renterd -go 1.21 - -toolchain go1.21.6 +go 1.21.6 require ( github.com/gabriel-vasile/mimetype v1.4.3 @@ -13,10 +11,11 @@ require ( github.com/minio/minio-go/v7 v7.0.67 github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe - go.sia.tech/core v0.1.12-0.20231211182757-77190f04f90b + go.sia.tech/core v0.2.1 + go.sia.tech/coreutils v0.0.1 go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 - go.sia.tech/hostd v0.3.0-beta.1 - go.sia.tech/jape v0.11.1 + go.sia.tech/hostd v1.0.2 + go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca go.sia.tech/web/renterd v0.45.0 @@ -32,8 +31,8 @@ require ( require ( github.com/aead/chacha20 v0.0.0-20180709150244-8b13a72661da // indirect - github.com/aws/aws-sdk-go v1.49.1 // indirect - github.com/cloudflare/cloudflare-go v0.75.0 // indirect + github.com/aws/aws-sdk-go v1.50.1 // indirect + github.com/cloudflare/cloudflare-go v0.86.0 // indirect github.com/dchest/threefish v0.0.0-20120919164726-3ecf4c494abf // indirect github.com/dustin/go-humanize v1.0.1 // indirect github.com/go-sql-driver/mysql v1.7.1 // indirect @@ -76,7 +75,7 @@ require ( gitlab.com/NebulousLabs/threadgroup v0.0.0-20200608151952-38921fbef213 // 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 + golang.org/x/net v0.20.0 // indirect golang.org/x/sys v0.17.0 // indirect golang.org/x/text v0.14.0 // indirect golang.org/x/time v0.5.0 // indirect diff --git a/go.sum b/go.sum index 3ed88a7d4..006a31ea6 100644 --- a/go.sum +++ b/go.sum @@ -9,14 +9,14 @@ github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuy github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8= github.com/aws/aws-sdk-go v1.44.256/go.mod h1:aVsgQcEevwlmQ7qHE9I3h+dtQgpqhFB+i8Phjh7fkwI= -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/aws/aws-sdk-go v1.50.1 h1:AwnLUM7TcH9vMZqA4TcDKmGfLmDW5VXwT5tPH6kXylo= +github.com/aws/aws-sdk-go v1.50.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/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= -github.com/cloudflare/cloudflare-go v0.75.0/go.mod h1:5ocQT9qQ99QsT1Ii2751490Z5J+W/nv6jOj+lSAe4ug= +github.com/cloudflare/cloudflare-go v0.86.0 h1:jEKN5VHNYNYtfDL2lUFLTRo+nOVNPFxpXTstVx0rqHI= +github.com/cloudflare/cloudflare-go v0.86.0/go.mod h1:wYW/5UP02TUfBToa/yKbQHV+r6h1NnJ1Je7XjuGM4Jw= github.com/coreos/bbolt v1.3.2/go.mod h1:iRUV2dpdMOn7Bo10OQBFzIJO9kkE559Wcmn+qkEiiKk= github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= @@ -126,8 +126,8 @@ github.com/kr/text v0.2.0 h1:5Nx0Ya0ZqY2ygV366QzturHI13Jq95ApcVaJBhpS+AY= github.com/kr/text v0.2.0/go.mod h1:eLer722TekiGuMkidMxC/pM04lWEeraHUUmBw8l2grE= github.com/kylelemons/godebug v0.0.0-20170820004349-d65d576e9348/go.mod h1:B69LEHPfb2qLo0BaaOLcbitczOKLWTsrBG9LczfCD4k= github.com/magiconair/properties v1.8.0/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czPbwD3XqdrwzmxQ= -github.com/mattn/go-colorable v0.1.12 h1:jF+Du6AlPIjs2BiUiQlKOX0rt3SujHxPnksPKZbaA40= -github.com/mattn/go-colorable v0.1.12/go.mod h1:u5H1YNBxpqRaxsYJYSkiCWKzEfiAb1Gb520KVy5xxl4= +github.com/mattn/go-colorable v0.1.13 h1:fFA4WZxdEF4tXPZVKMLwD8oUnCTTo08duU7wxecdEvA= +github.com/mattn/go-colorable v0.1.13/go.mod h1:7S9/ev0klgBDR4GtXTXX8a3vIGJpMovkB8vQcUbaXHg= github.com/mattn/go-isatty v0.0.17 h1:BTarxUcIeDqL27Mc+vyvdWYSL28zpIhv3RoTdsLMPng= github.com/mattn/go-isatty v0.0.17/go.mod h1:kYGgaQfpe5nmfYZH+SKPsOc2e4SrIfOl2e/yFXSvRLM= github.com/mattn/go-sqlite3 v1.14.18 h1:JL0eqdCOq6DJVNPSvArO/bIV9/P7fbGrV00LZHc+5aI= @@ -239,14 +239,16 @@ 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.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/core v0.2.1 h1:CqmMd+T5rAhC+Py3NxfvGtvsj/GgwIqQHHVrdts/LqY= +go.sia.tech/core v0.2.1/go.mod h1:3EoY+rR78w1/uGoXXVqcYdwSjSJKuEMI5bL7WROA27Q= +go.sia.tech/coreutils v0.0.1 h1:Th8iiF9fjkBaxlKRgPJfRtsD3Pb8U4d2m/OahB6wffg= +go.sia.tech/coreutils v0.0.1/go.mod h1:3Mb206QDd3NtRiaHZ2kN87/HKXhcBF6lHVatS7PkViY= go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 h1:ulzfJNjxN5DjXHClkW2pTiDk+eJ+0NQhX87lFDZ03t0= go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2/go.mod h1:PlsiVCn6+wssrR7bsOIlZm0DahsVrDydrlbjY4F14sg= -go.sia.tech/hostd v0.3.0-beta.1 h1:A2RL4wkW18eb28+fJtdyK9OYNiiwpCDO8FO3cyT9r7A= -go.sia.tech/hostd v0.3.0-beta.1/go.mod h1:gVtU631RkbtOEHJKb8qghudhWcYIL8w3phjvV2/bz0A= -go.sia.tech/jape v0.11.1 h1:M7IP+byXL7xOqzxcHUQuXW+q3sYMkYzmMlMw+q8ZZw0= -go.sia.tech/jape v0.11.1/go.mod h1:4QqmBB+t3W7cNplXPj++ZqpoUb2PeiS66RLpXmEGap4= +go.sia.tech/hostd v1.0.2 h1:GjzNIAlwg3/dViF6258Xn5DI3+otQLRqmkoPDugP+9Y= +go.sia.tech/hostd v1.0.2/go.mod h1:zGw+AGVmazAp4ydvo7bZLNKTy1J51RI6Mp/oxRtYT6c= +go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 h1:mSaJ622P7T/M97dAK8iPV+IRIC9M5vV28NHeceoWO3M= +go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640/go.mod h1:4QqmBB+t3W7cNplXPj++ZqpoUb2PeiS66RLpXmEGap4= go.sia.tech/mux v1.2.0 h1:ofa1Us9mdymBbGMY2XH/lSpY8itFsKIo/Aq8zwe+GHU= go.sia.tech/mux v1.2.0/go.mod h1:Yyo6wZelOYTyvrHmJZ6aQfRoer3o4xyKQ4NmQLJrBSo= go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca h1:aZMg2AKevn7jKx+wlusWQfwSM5pNU9aGtRZme29q3O4= @@ -301,8 +303,8 @@ golang.org/x/net v0.0.0-20220722155237-a158d28d115b/go.mod h1:XRhObCWvk6IyKnWLug golang.org/x/net v0.1.0/go.mod h1:Cx3nUiGt4eDBEyega/BKRp+/AlGL8hYe7U9odMt2Cco= golang.org/x/net v0.6.0/go.mod h1:2Tu9+aMcznHK/AK1HMvgo6xiTLG5rD5rZLDS+rp2Bjs= golang.org/x/net v0.9.0/go.mod h1:d48xBJpPfHeWQsugry2m+kC02ZBRGRgulfHnEXEuWns= -golang.org/x/net v0.19.0 h1:zTwKpTd2XuCqf8huc7Fo2iSy+4RHPd10s4KzeTnVr1c= -golang.org/x/net v0.19.0/go.mod h1:CfAk/cbD4CthTvqiEl8NpboMuiuOYsAr/7NOjZJtv1U= +golang.org/x/net v0.20.0 h1:aCL9BSgETF1k+blQaYUBx9hJ9LOGP3gAVemcZlf1Kpo= +golang.org/x/net v0.20.0/go.mod h1:z8BVo6PvndSri0LbOE3hAn0apkU+1YvI6E70E9jsnvY= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= diff --git a/internal/node/transactionpool.go b/internal/node/transactionpool.go index b2226bfb5..c5582a757 100644 --- a/internal/node/transactionpool.go +++ b/internal/node/transactionpool.go @@ -15,7 +15,7 @@ type txpool struct { func (tp txpool) RecommendedFee() (fee types.Currency) { _, max := tp.tp.FeeEstimation() - convertToCore(&max, &fee) + convertToCore(&max, (*types.V1Currency)(&fee)) return } diff --git a/stores/wallet.go b/stores/wallet.go index 679e96074..d9bf51c39 100644 --- a/stores/wallet.go +++ b/stores/wallet.go @@ -130,7 +130,7 @@ func (s *SQLStore) processConsensusChangeWallet(cc modules.ConsensusChange) { // Add/Remove siacoin outputs. for _, diff := range cc.SiacoinOutputDiffs { var sco types.SiacoinOutput - convertToCore(diff.SiacoinOutput, &sco) + convertToCore(diff.SiacoinOutput, (*types.V1SiacoinOutput)(&sco)) if sco.Address != s.walletAddress { continue } @@ -166,7 +166,7 @@ func (s *SQLStore) processConsensusChangeWallet(cc modules.ConsensusChange) { continue } var sco types.SiacoinOutput - convertToCore(dsco.SiacoinOutput, &sco) + convertToCore(dsco.SiacoinOutput, (*types.V1SiacoinOutput)(&sco)) s.unappliedTxnChanges = append(s.unappliedTxnChanges, txnChange{ addition: true, txnID: hash256(dsco.ID), // use output id as txn id @@ -213,7 +213,7 @@ func (s *SQLStore) processConsensusChangeWallet(cc modules.ConsensusChange) { for _, diff := range appliedDiff.SiacoinOutputDiffs { if diff.Direction == modules.DiffRevert { var so types.SiacoinOutput - convertToCore(diff.SiacoinOutput, &so) + convertToCore(diff.SiacoinOutput, (*types.V1SiacoinOutput)(&so)) spentOutputs[types.SiacoinOutputID(diff.ID)] = so } } From 0ed4313bcbb3adf1ad124a8e28d7fcfe0c5e1995 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 10:48:40 +0100 Subject: [PATCH 088/144] stores: fix TestPartialSlab --- stores/metadata_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 4a6102399..c0ef54d0f 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -2661,7 +2661,7 @@ func TestPartialSlab(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(obj, fetched.Object) { + if !reflect.DeepEqual(obj, *fetched.Object) { t.Fatal("mismatch", cmp.Diff(obj, fetched.Object, cmp.AllowUnexported(object.EncryptionKey{}))) } @@ -2697,7 +2697,7 @@ func TestPartialSlab(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(obj2, fetched.Object) { + if !reflect.DeepEqual(obj2, *fetched.Object) { t.Fatal("mismatch", cmp.Diff(obj2, fetched.Object)) } @@ -2745,7 +2745,7 @@ func TestPartialSlab(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(obj3, fetched.Object) { + if !reflect.DeepEqual(obj3, *fetched.Object) { t.Fatal("mismatch", cmp.Diff(obj3, fetched.Object, cmp.AllowUnexported(object.EncryptionKey{}))) } From d7a51d2ba59da9eacd4fb149ec1db31e11fe9ea4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 10:55:19 +0100 Subject: [PATCH 089/144] stores: fix TestSQLMetadataStore --- stores/metadata_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index c0ef54d0f..ee7cd959c 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -1071,7 +1071,7 @@ func TestSQLMetadataStore(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(fullObj.Object, obj1) { + if !reflect.DeepEqual(*fullObj.Object, obj1) { t.Fatal("object mismatch", cmp.Diff(fullObj, obj1)) } @@ -1184,7 +1184,7 @@ func TestSQLMetadataStore(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(fullObj.Object, obj1) { + if !reflect.DeepEqual(*fullObj.Object, obj1) { t.Fatal("object mismatch") } From 3990ece02d492e83945cf150c3d0e0dda0d089f6 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 10:58:01 +0100 Subject: [PATCH 090/144] stores: fix TestObjectBasic --- stores/metadata_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index ee7cd959c..4c7a468b2 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -89,7 +89,7 @@ func TestObjectBasic(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(got.Object, want) { + if !reflect.DeepEqual(*got.Object, want) { t.Fatal("object mismatch", cmp.Diff(got.Object, want)) } @@ -120,7 +120,7 @@ func TestObjectBasic(t *testing.T) { if err != nil { t.Fatal(err) } - if !reflect.DeepEqual(got2.Object, want2) { + if !reflect.DeepEqual(*got2.Object, want2) { t.Fatal("object mismatch", cmp.Diff(got2.Object, want2)) } } From f075c0caa82f7aa46b5bf03e2a7bb2af7ba35b49 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 10:58:42 +0100 Subject: [PATCH 091/144] stores: TestObjectMetadata --- stores/metadata_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 4c7a468b2..d041e590c 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -177,7 +177,7 @@ func TestObjectMetadata(t *testing.T) { } // assert it matches - if !reflect.DeepEqual(got.Object, want) { + if !reflect.DeepEqual(*got.Object, want) { t.Log(got.Object) t.Log(want) t.Fatal("object mismatch", cmp.Diff(got.Object, want, cmp.AllowUnexported(object.EncryptionKey{}))) From 2756576a4012b80fbde57a34bcd5a7675ced39f2 Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 19 Feb 2024 11:54:19 +0100 Subject: [PATCH 092/144] worker: use background context for unlocking accounts --- worker/rhpv3.go | 179 +++++++++++++++++++++++------------------------- 1 file changed, 86 insertions(+), 93 deletions(-) diff --git a/worker/rhpv3.go b/worker/rhpv3.go index 03f67c6f6..5fbcd3ad6 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -337,19 +337,17 @@ type ( // accounts stores the balance and other metrics of accounts that the // worker maintains with a host. accounts struct { - as AccountStore - key types.PrivateKey - shutdownCtx context.Context + as AccountStore + key types.PrivateKey } // account contains information regarding a specific account of the // worker. account struct { - as AccountStore - id rhpv3.Account - key types.PrivateKey - host types.PublicKey - shutdownCtx context.Context + as AccountStore + id rhpv3.Account + key types.PrivateKey + host types.PublicKey } ) @@ -358,9 +356,8 @@ func (w *worker) initAccounts(as AccountStore) { panic("accounts already initialized") // developer error } w.accounts = &accounts{ - as: as, - key: w.deriveSubKey("accountkey"), - shutdownCtx: w.shutdownCtx, + as: as, + key: w.deriveSubKey("accountkey"), } } @@ -376,117 +373,113 @@ func (w *worker) initTransportPool() { func (a *accounts) ForHost(hk types.PublicKey) *account { accountID := rhpv3.Account(a.deriveAccountKey(hk).PublicKey()) return &account{ - as: a.as, - id: accountID, - key: a.key, - host: hk, - shutdownCtx: a.shutdownCtx, + as: a.as, + id: accountID, + key: a.key, + host: hk, } } -// 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.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) +func withAccountLock(ctx context.Context, as AccountStore, id rhpv3.Account, hk types.PublicKey, exclusive bool, fn func(a api.Account) error) error { + acc, lockID, err := as.LockAccount(ctx, id, hk, exclusive, accountLockingDuration) if err != nil { return err } + defer func() { - unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) - a.as.UnlockAccount(unlockCtx, a.id, lockID) - cancel() + select { + case <-ctx.Done(): + var cancel context.CancelFunc + ctx, cancel = context.WithTimeout(context.Background(), time.Minute) + defer cancel() + default: + } + as.UnlockAccount(ctx, acc.ID, lockID) }() - amt, err := amtFn() - if err != nil { - return err - } - return a.as.AddBalance(ctx, a.id, a.host, amt.Big()) + return fn(acc) } -func (a *account) Balance(ctx context.Context) (types.Currency, error) { - account, lockID, err := a.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) - if err != nil { - return types.Currency{}, err - } - defer func() { - unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) - a.as.UnlockAccount(unlockCtx, a.id, lockID) - cancel() - }() +// Balance returns the account balance. +func (a *account) Balance(ctx context.Context) (balance types.Currency, err error) { + err = withAccountLock(ctx, a.as, a.id, a.host, false, func(account api.Account) error { + balance = types.NewCurrency(account.Balance.Uint64(), new(big.Int).Rsh(account.Balance, 64).Uint64()) + return nil + }) + return +} - return types.NewCurrency(account.Balance.Uint64(), new(big.Int).Rsh(account.Balance, 64).Uint64()), nil +// 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 { + return withAccountLock(ctx, a.as, a.id, a.host, false, func(_ api.Account) error { + amt, err := amtFn() + if err != nil { + return err + } + return a.as.AddBalance(ctx, a.id, a.host, amt.Big()) + }) +} + +// 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 { + return withAccountLock(ctx, a.as, a.id, a.host, true, func(_ api.Account) error { + balance, err := balanceFn() + if err != nil { + return err + } + return a.as.SetBalance(ctx, a.id, a.host, balance.Big()) + }) } // WithWithdrawal decreases the balance of an account by the amount returned by // 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.as.LockAccount(ctx, a.id, a.host, false, accountLockingDuration) - if err != nil { - return err - } - defer func() { - unlockCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) - a.as.UnlockAccount(unlockCtx, a.id, lockID) - cancel() - }() + return withAccountLock(ctx, a.as, a.id, a.host, false, func(account api.Account) error { + // return early if the account needs to sync + if account.RequiresSync { + return fmt.Errorf("%w; account requires resync", errBalanceInsufficient) + } - // return early if the account needs to sync - if account.RequiresSync { - return fmt.Errorf("%w; account requires resync", errBalanceInsufficient) - } + // return early if our account is not funded + if account.Balance.Cmp(big.NewInt(0)) <= 0 { + return errBalanceInsufficient + } - // return early if our account is not funded - if account.Balance.Cmp(big.NewInt(0)) <= 0 { - return errBalanceInsufficient - } + // execute amtFn + amt, err := amtFn() + if isBalanceInsufficient(err) { + // in case of an insufficient balance, we schedule a sync + if scheduleErr := a.scheduleSync(); scheduleErr != nil { + err = fmt.Errorf("%w; failed to set requiresSync flag on bus, error: %v", err, scheduleErr) + } + } - // execute amtFn - amt, err := amtFn() - if isBalanceInsufficient(err) { - // in case of an insufficient balance, we schedule a sync - scheduleCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) - defer cancel() - 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) + // if an amount was returned, we withdraw it. + if withdrawErr := a.withdrawFromBalance(amt); withdrawErr != nil { + err = fmt.Errorf("%w; failed to withdraw from account, error: %v", err, withdrawErr) } - } - // if the amount is zero, we are done - if amt.IsZero() { return err + }) +} + +func (a *account) withdrawFromBalance(amt types.Currency) error { + if amt.IsZero() { + return nil } - // if an amount was returned, we withdraw it. - addCtx, cancel := context.WithTimeout(a.shutdownCtx, 10*time.Second) + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) defer cancel() - 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) - } - return err + return a.as.AddBalance(ctx, a.id, a.host, new(big.Int).Neg(amt.Big())) } -// 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.as.LockAccount(ctx, a.id, a.host, true, accountLockingDuration) - if err != nil { - return err - } - 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.as.SetBalance(ctx, a.id, a.host, balance.Big()) +func (a *account) scheduleSync() error { + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + defer cancel() + return a.as.ScheduleSync(ctx, a.id, a.host) } // deriveAccountKey derives an account plus key for a given host and worker. From 2102b91ec6ba0b8ff36f3534dd4452eccb17aa6e Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 19 Feb 2024 13:08:32 +0100 Subject: [PATCH 093/144] ci: add Test Stores - MySQL --- .github/workflows/test.yml | 10 ++++++++++ stores/sql_test.go | 29 ++++++++++++++++++++++++++--- 2 files changed, 36 insertions(+), 3 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index 8e4c21faf..e8a32e5ec 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -47,6 +47,16 @@ jobs: uses: n8maninger/action-golang-test@v1 with: args: "-race;-short" + - name: Test Stores - MySQL + if: matrix.os == 'ubuntu-latest' + uses: n8maninger/action-golang-test@v1 + env: + RENTERD_DB_URI: 127.0.0.1:3800 + RENTERD_DB_USER: root + RENTERD_DB_PASSWORD: test + with: + package: "./stores" + args: "-race;-short" - name: Test Integration uses: n8maninger/action-golang-test@v1 with: diff --git a/stores/sql_test.go b/stores/sql_test.go index 3a51161ae..ae0c54b0c 100644 --- a/stores/sql_test.go +++ b/stores/sql_test.go @@ -48,6 +48,9 @@ type testSQLStore struct { } type testSQLStoreConfig struct { + dbURI string + dbUser string + dbPassword string dbName string dbMetricsName string dir string @@ -65,9 +68,26 @@ func newTestSQLStore(t *testing.T, cfg testSQLStoreConfig) *testSQLStore { if dir == "" { dir = t.TempDir() } - dbName := cfg.dbName + + dbURI, dbUser, dbPassword, dbName := DBConfigFromEnv() + if dbURI == "" { + dbURI = cfg.dbURI + } + if cfg.persistent && dbURI != "" { + t.Fatal("invalid store config, can't use both persistent and dbURI") + } + if dbUser == "" { + dbUser = cfg.dbUser + } + if dbPassword == "" { + dbPassword = cfg.dbPassword + } if dbName == "" { - dbName = hex.EncodeToString(frand.Bytes(32)) // random name for db + if cfg.dbName != "" { + dbName = cfg.dbName + } else { + dbName = hex.EncodeToString(frand.Bytes(32)) // random name for db + } } dbMetricsName := cfg.dbMetricsName if dbMetricsName == "" { @@ -75,7 +95,10 @@ func newTestSQLStore(t *testing.T, cfg testSQLStoreConfig) *testSQLStore { } var conn, connMetrics gorm.Dialector - if cfg.persistent { + if dbURI != "" { + conn = NewMySQLConnection(dbURI, dbUser, dbPassword, dbName) + connMetrics = NewMySQLConnection(dbURI, dbUser, dbPassword, dbMetricsName) + } else if cfg.persistent { conn = NewSQLiteConnection(filepath.Join(cfg.dir, "db.sqlite")) connMetrics = NewSQLiteConnection(filepath.Join(cfg.dir, "metrics.sqlite")) } else { From d7e0cf5751a84c24ca35ebf84191537cf2a17ff4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 14:18:45 +0100 Subject: [PATCH 094/144] testing: fix TestS3MultipartUploads --- worker/worker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/worker.go b/worker/worker.go index 2a2191d2d..b57623b7b 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1154,7 +1154,7 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { } // make sure only one of the following is set - if encryptionEnabled := upload.Key != object.NoOpKey; encryptionEnabled && jc.Request.FormValue("offset") == "" { + if encryptionEnabled := !upload.Key.IsNoopKey(); encryptionEnabled && jc.Request.FormValue("offset") == "" { jc.Error(errors.New("if presharding encryption isn't disabled, the offset needs to be set"), http.StatusBadRequest) return } else if encryptionEnabled { From 361c6cf43f6ffb85b86171bf3deedcd805d5ca88 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Mon, 19 Feb 2024 16:46:41 +0100 Subject: [PATCH 095/144] stores: add index to object size --- stores/migrations.go | 13 ++++++++++--- .../mysql/main/migration_00003_idx_objects_size.sql | 1 + stores/migrations/mysql/main/schema.sql | 1 + .../main/migration_00003_idx_objects_size.sql | 1 + stores/migrations/sqlite/main/schema.sql | 1 + stores/migrations_metrics.go | 3 ++- 6 files changed, 16 insertions(+), 4 deletions(-) create mode 100644 stores/migrations/mysql/main/migration_00003_idx_objects_size.sql create mode 100644 stores/migrations/sqlite/main/migration_00003_idx_objects_size.sql diff --git a/stores/migrations.go b/stores/migrations.go index 7f225ff17..bf3916ca4 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -16,6 +16,7 @@ var ( ) func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { + dbIdentifier := "main" migrations := []*gormigrate.Migration{ { ID: "00001_init", @@ -24,26 +25,32 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { { ID: "00001_object_metadata", Migrate: func(tx *gorm.DB) error { - return performMigration(tx, "main", "00001_object_metadata", logger) + return performMigration(tx, dbIdentifier, "00001_object_metadata", logger) }, }, { ID: "00002_prune_slabs_trigger", Migrate: func(tx *gorm.DB) error { - err := performMigration(tx, "main", "00002_prune_slabs_trigger", logger) + err := performMigration(tx, dbIdentifier, "00002_prune_slabs_trigger", logger) if err != nil && strings.Contains(err.Error(), errMySQLNoSuperPrivilege.Error()) { logger.Warn("migration 00002_prune_slabs_trigger requires the user to have the SUPER privilege to register triggers") } return err }, }, + { + ID: "00003_idx_objects_size", + Migrate: func(tx *gorm.DB) error { + return performMigration(tx, dbIdentifier, "00003_idx_objects_size", logger) + }, + }, } // Create migrator. m := gormigrate.New(db, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initSchema(db, "main", logger)) + m.InitSchema(initSchema(db, dbIdentifier, logger)) // Perform migrations. if err := m.Migrate(); err != nil { diff --git a/stores/migrations/mysql/main/migration_00003_idx_objects_size.sql b/stores/migrations/mysql/main/migration_00003_idx_objects_size.sql new file mode 100644 index 000000000..0df0b5d58 --- /dev/null +++ b/stores/migrations/mysql/main/migration_00003_idx_objects_size.sql @@ -0,0 +1 @@ +CREATE INDEX `idx_objects_size` ON `objects`(`size`); diff --git a/stores/migrations/mysql/main/schema.sql b/stores/migrations/mysql/main/schema.sql index d28bdd13f..1b39e4669 100644 --- a/stores/migrations/mysql/main/schema.sql +++ b/stores/migrations/mysql/main/schema.sql @@ -330,6 +330,7 @@ CREATE TABLE `objects` ( KEY `idx_objects_object_id` (`object_id`), KEY `idx_objects_health` (`health`), KEY `idx_objects_etag` (`etag`), + KEY `idx_objects_size` (`size`), CONSTRAINT `fk_objects_db_bucket` FOREIGN KEY (`db_bucket_id`) REFERENCES `buckets` (`id`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; diff --git a/stores/migrations/sqlite/main/migration_00003_idx_objects_size.sql b/stores/migrations/sqlite/main/migration_00003_idx_objects_size.sql new file mode 100644 index 000000000..0df0b5d58 --- /dev/null +++ b/stores/migrations/sqlite/main/migration_00003_idx_objects_size.sql @@ -0,0 +1 @@ +CREATE INDEX `idx_objects_size` ON `objects`(`size`); diff --git a/stores/migrations/sqlite/main/schema.sql b/stores/migrations/sqlite/main/schema.sql index daee619b4..e6bb2546d 100644 --- a/stores/migrations/sqlite/main/schema.sql +++ b/stores/migrations/sqlite/main/schema.sql @@ -50,6 +50,7 @@ CREATE INDEX `idx_objects_db_bucket_id` ON `objects`(`db_bucket_id`); CREATE INDEX `idx_objects_etag` ON `objects`(`etag`); CREATE INDEX `idx_objects_health` ON `objects`(`health`); CREATE INDEX `idx_objects_object_id` ON `objects`(`object_id`); +CREATE INDEX `idx_objects_size` ON `objects`(`size`); CREATE UNIQUE INDEX `idx_object_bucket` ON `objects`(`db_bucket_id`,`object_id`); -- dbMultipartUpload diff --git a/stores/migrations_metrics.go b/stores/migrations_metrics.go index 940917569..60c62c476 100644 --- a/stores/migrations_metrics.go +++ b/stores/migrations_metrics.go @@ -9,6 +9,7 @@ import ( ) func performMetricsMigrations(tx *gorm.DB, logger *zap.SugaredLogger) error { + dbIdentifier := "metrics" migrations := []*gormigrate.Migration{ { ID: "00001_init", @@ -20,7 +21,7 @@ func performMetricsMigrations(tx *gorm.DB, logger *zap.SugaredLogger) error { m := gormigrate.New(tx, gormigrate.DefaultOptions, migrations) // Set init function. - m.InitSchema(initSchema(tx, "metrics", logger)) + m.InitSchema(initSchema(tx, dbIdentifier, logger)) // Perform migrations. if err := m.Migrate(); err != nil { From 9bb5972586e6fc26ad0c21758b0c265f8bab76a2 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 09:55:13 +0100 Subject: [PATCH 096/144] stores: migration code --- .../migration_00004_prune_slabs_cascade.sql | 24 +++++++++++ stores/migrations/mysql/main/schema.sql | 40 ++---------------- .../migration_00004_prune_slabs_cascade.sql | 28 +++++++++++++ stores/migrations/sqlite/main/schema.sql | 42 +------------------ 4 files changed, 57 insertions(+), 77 deletions(-) create mode 100644 stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql create mode 100644 stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql diff --git a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql new file mode 100644 index 000000000..48f98a40a --- /dev/null +++ b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql @@ -0,0 +1,24 @@ +-- prune manually before creating trigger +DELETE slabs +FROM slabs +LEFT JOIN slices ON slices.db_slab_id = slabs.id +WHERE slices.db_object_id IS NULL +AND slices.db_multipart_part_id IS NULL +AND slabs.db_buffered_slab_id IS NULL; + +-- add ON DELETE CASCADE to slices +ALTER TABLE slices DROP FOREIGN KEY fk_objects_slabs; +ALTER TABLE slices ADD CONSTRAINT fk_objects_slabs FOREIGN KEY (db_object_id) REFERENCES objects (id) ON DELETE CASCADE, + +ALTER TABLE slices DROP FOREIGN KEY fk_multipart_parts_slabs; +ALTER TABLE slices ADD CONSTRAINT fk_multipart_parts_slabs FOREIGN KEY (db_multipart_part_id) REFERENCES multipart_parts (id) ON DELETE CASCADE, + +-- add ON DELETE CASCADE to multipart_parts +ALTER TABLE multipart_parts DROP FOREIGN KEY fk_multipart_uploads_parts; +ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE + +-- drop triggers +DROP TRIGGER before_delete_on_objects_delete_slices +DROP TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts +DROP TRIGGER before_delete_on_multipart_parts_delete_slices +DROP TRIGGER after_delete_on_slices_delete_slabs \ No newline at end of file diff --git a/stores/migrations/mysql/main/schema.sql b/stores/migrations/mysql/main/schema.sql index 1b39e4669..a5ed86807 100644 --- a/stores/migrations/mysql/main/schema.sql +++ b/stores/migrations/mysql/main/schema.sql @@ -310,7 +310,7 @@ CREATE TABLE `multipart_parts` ( KEY `idx_multipart_parts_etag` (`etag`), KEY `idx_multipart_parts_part_number` (`part_number`), KEY `idx_multipart_parts_db_multipart_upload_id` (`db_multipart_upload_id`), - CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads` (`id`) + CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads` (`id`) ON DELETE CASCADE ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; -- dbObject @@ -374,8 +374,8 @@ CREATE TABLE `slices` ( KEY `idx_slices_object_index` (`object_index`), KEY `idx_slices_db_multipart_part_id` (`db_multipart_part_id`), KEY `idx_slices_db_slab_id` (`db_slab_id`), - CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts` (`id`), - CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects` (`id`), + CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts` (`id`) ON DELETE CASCADE, + CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects` (`id`) ON DELETE CASCADE, CONSTRAINT `fk_slabs_slices` FOREIGN KEY (`db_slab_id`) REFERENCES `slabs` (`id`) ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; @@ -421,39 +421,5 @@ CREATE TABLE `object_user_metadata` ( CONSTRAINT `fk_multipart_upload_user_metadata` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads` (`id`) ON DELETE SET NULL ) ENGINE=InnoDB DEFAULT CHARSET=utf8mb4 COLLATE=utf8mb4_0900_ai_ci; --- dbObject trigger to delete from slices -CREATE TRIGGER before_delete_on_objects_delete_slices -BEFORE DELETE -ON objects FOR EACH ROW -DELETE FROM slices -WHERE slices.db_object_id = OLD.id; - --- dbMultipartUpload trigger to delete from dbMultipartPart -CREATE TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts -BEFORE DELETE -ON multipart_uploads FOR EACH ROW -DELETE FROM multipart_parts -WHERE multipart_parts.db_multipart_upload_id = OLD.id; - --- dbMultipartPart trigger to delete from slices -CREATE TRIGGER before_delete_on_multipart_parts_delete_slices -BEFORE DELETE -ON multipart_parts FOR EACH ROW -DELETE FROM slices -WHERE slices.db_multipart_part_id = OLD.id; - --- dbSlices trigger to prune slabs -CREATE TRIGGER after_delete_on_slices_delete_slabs -AFTER DELETE -ON slices FOR EACH ROW -DELETE FROM slabs -WHERE slabs.id = OLD.db_slab_id -AND slabs.db_buffered_slab_id IS NULL -AND NOT EXISTS ( - SELECT 1 - FROM slices - WHERE slices.db_slab_id = OLD.db_slab_id -); - -- create default bucket INSERT INTO buckets (created_at, name) VALUES (CURRENT_TIMESTAMP, 'default'); \ No newline at end of file diff --git a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql new file mode 100644 index 000000000..2cd633c11 --- /dev/null +++ b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql @@ -0,0 +1,28 @@ +PRAGMA foreign_keys=off; +-- update constraints on slices +CREATE TABLE `slices_temp` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`db_object_id` integer,`object_index` integer,`db_multipart_part_id` integer,`db_slab_id` integer,`offset` integer,`length` integer,CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_slabs_slices` FOREIGN KEY (`db_slab_id`) REFERENCES `slabs`(`id`)); +INSERT INTO slices_temp SELECT `id`, `created_at`, `db_object_id`, `object_index`, `db_multipart_part_id`, `db_slab_id`, `offset`, `length` FROM slices; +DROP TABLE slices; +ALTER TABLE slices_temp RENAME TO slices; + +CREATE INDEX `idx_slices_object_index` ON `slices`(`object_index`); +CREATE INDEX `idx_slices_db_object_id` ON `slices`(`db_object_id`); +CREATE INDEX `idx_slices_db_slab_id` ON `slices`(`db_slab_id`); +CREATE INDEX `idx_slices_db_multipart_part_id` ON `slices`(`db_multipart_part_id`); + +-- update constraints multipart_parts +CREATE TABLE `multipart_parts_temp` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`etag` text,`part_number` integer,`size` integer,`db_multipart_upload_id` integer NOT NULL,CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads`(`id`) ON DELETE CASCADE); +INSERT INTO multipart_parts_temp SELECT * FROM multipart_parts; +DROP TABLE multipart_parts; +ALTER TABLE multipart_parts_temp RENAME TO multipart_parts; + +CREATE INDEX `idx_multipart_parts_db_multipart_upload_id` ON `multipart_parts`(`db_multipart_upload_id`); +CREATE INDEX `idx_multipart_parts_part_number` ON `multipart_parts`(`part_number`); +CREATE INDEX `idx_multipart_parts_etag` ON `multipart_parts`(`etag`); +PRAGMA foreign_keys=on; + +-- drop triggers +DROP TRIGGER before_delete_on_objects_delete_slices +DROP TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts +DROP TRIGGER before_delete_on_multipart_parts_delete_slices +DROP TRIGGER after_delete_on_slices_delete_slabs \ No newline at end of file diff --git a/stores/migrations/sqlite/main/schema.sql b/stores/migrations/sqlite/main/schema.sql index e6bb2546d..8d7afeaa1 100644 --- a/stores/migrations/sqlite/main/schema.sql +++ b/stores/migrations/sqlite/main/schema.sql @@ -85,13 +85,13 @@ CREATE INDEX `idx_contract_sectors_db_contract_id` ON `contract_sectors`(`db_con CREATE INDEX `idx_contract_sectors_db_sector_id` ON `contract_sectors`(`db_sector_id`); -- dbMultipartPart -CREATE TABLE `multipart_parts` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`etag` text,`part_number` integer,`size` integer,`db_multipart_upload_id` integer NOT NULL,CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads`(`id`)); +CREATE TABLE `multipart_parts` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`etag` text,`part_number` integer,`size` integer,`db_multipart_upload_id` integer NOT NULL,CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads`(`id`) ON DELETE CASCADE); CREATE INDEX `idx_multipart_parts_db_multipart_upload_id` ON `multipart_parts`(`db_multipart_upload_id`); CREATE INDEX `idx_multipart_parts_part_number` ON `multipart_parts`(`part_number`); CREATE INDEX `idx_multipart_parts_etag` ON `multipart_parts`(`etag`); -- dbSlice -CREATE TABLE `slices` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`db_object_id` integer,`object_index` integer,`db_multipart_part_id` integer,`db_slab_id` integer,`offset` integer,`length` integer,CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects`(`id`),CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts`(`id`),CONSTRAINT `fk_slabs_slices` FOREIGN KEY (`db_slab_id`) REFERENCES `slabs`(`id`)); +CREATE TABLE `slices` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`db_object_id` integer,`object_index` integer,`db_multipart_part_id` integer,`db_slab_id` integer,`offset` integer,`length` integer,CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_slabs_slices` FOREIGN KEY (`db_slab_id`) REFERENCES `slabs`(`id`)); CREATE INDEX `idx_slices_object_index` ON `slices`(`object_index`); CREATE INDEX `idx_slices_db_object_id` ON `slices`(`db_object_id`); CREATE INDEX `idx_slices_db_slab_id` ON `slices`(`db_slab_id`); @@ -148,43 +148,5 @@ CREATE UNIQUE INDEX `idx_module_event_url` ON `webhooks`(`module`,`event`,`url`) CREATE TABLE `object_user_metadata` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`db_object_id` integer DEFAULT NULL,`db_multipart_upload_id` integer DEFAULT NULL,`key` text NOT NULL,`value` text, CONSTRAINT `fk_object_user_metadata` FOREIGN KEY (`db_object_id`) REFERENCES `objects` (`id`) ON DELETE CASCADE, CONSTRAINT `fk_multipart_upload_user_metadata` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads` (`id`) ON DELETE SET NULL); CREATE UNIQUE INDEX `idx_object_user_metadata_key` ON `object_user_metadata`(`db_object_id`,`db_multipart_upload_id`,`key`); --- dbObject trigger to delete from slices -CREATE TRIGGER before_delete_on_objects_delete_slices -BEFORE DELETE ON objects -BEGIN - DELETE FROM slices - WHERE slices.db_object_id = OLD.id; -END; - --- dbMultipartUpload trigger to delete from dbMultipartPart -CREATE TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts -BEFORE DELETE ON multipart_uploads -BEGIN - DELETE FROM multipart_parts - WHERE multipart_parts.db_multipart_upload_id = OLD.id; -END; - --- dbMultipartPart trigger to delete from slices -CREATE TRIGGER before_delete_on_multipart_parts_delete_slices -BEFORE DELETE ON multipart_parts -BEGIN - DELETE FROM slices - WHERE slices.db_multipart_part_id = OLD.id; -END; - --- dbSlices trigger to prune slabs -CREATE TRIGGER after_delete_on_slices_delete_slabs -AFTER DELETE ON slices -BEGIN - DELETE FROM slabs - WHERE slabs.id = OLD.db_slab_id - AND slabs.db_buffered_slab_id IS NULL - AND NOT EXISTS ( - SELECT 1 - FROM slices - WHERE slices.db_slab_id = OLD.db_slab_id - ); -END; - -- create default bucket INSERT INTO buckets (created_at, name) VALUES (CURRENT_TIMESTAMP, 'default'); From 1f004223a11b0c4897438db98584d81460365e7e Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 10:06:21 +0100 Subject: [PATCH 097/144] stores: call pruneSlabs every time an object or multipart object were deleted --- stores/metadata.go | 23 ++++++++++++++++++++--- stores/multipart.go | 9 +++++++++ 2 files changed, 29 insertions(+), 3 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 68947ed95..a9636bf75 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -2685,20 +2685,32 @@ func archiveContracts(ctx context.Context, tx *gorm.DB, contracts []dbContract, return nil } +func pruneSlabs(tx *gorm.DB) error { + // delete slabs without any associated slices or buffers + return tx.Exec(` +DELETE +FROM slabs sla +WHERE NOT EXISTS (SELECT 1 FROM slices sli WHERE sli.db_slab_id = sla.id) +AND sla.db_buffered_slab_id IS NULL +`).Error +} + // deleteObject deletes an object from the store and prunes all slabs which are // without an obect after the deletion. That means in case of packed uploads, // the slab is only deleted when no more objects point to it. -func (s *SQLStore) deleteObject(tx *gorm.DB, bucket string, path string) (numDeleted int64, _ error) { +func (s *SQLStore) deleteObject(tx *gorm.DB, bucket string, path string) (int64, error) { tx = tx.Where("object_id = ? AND ?", path, sqlWhereBucket("objects", bucket)). Delete(&dbObject{}) if tx.Error != nil { return 0, tx.Error } - numDeleted = tx.RowsAffected + numDeleted := tx.RowsAffected if numDeleted == 0 { return 0, nil // nothing to prune if no object was deleted + } else if err := pruneSlabs(tx); err != nil { + return 0, err } - return + return numDeleted, nil } // deleteObjects deletes a batch of objects from the database. The order of @@ -2729,6 +2741,11 @@ func (s *SQLStore) deleteObjects(bucket string, path string) (numDeleted int64, } duration = time.Since(start) rowsAffected = res.RowsAffected + + // prune slabs if we deleted an object + if rowsAffected > 0 { + return pruneSlabs(tx) + } return nil }); err != nil { return 0, fmt.Errorf("failed to delete objects: %w", err) diff --git a/stores/multipart.go b/stores/multipart.go index 18706ed0c..3a5bcd54a 100644 --- a/stores/multipart.go +++ b/stores/multipart.go @@ -295,6 +295,10 @@ func (s *SQLStore) AbortMultipartUpload(ctx context.Context, bucket, path string if err != nil { return fmt.Errorf("failed to delete multipart upload: %w", err) } + // Prune the slabs. + if err := pruneSlabs(tx); err != nil { + return fmt.Errorf("failed to prune slabs: %w", err) + } return nil }) } @@ -435,6 +439,11 @@ func (s *SQLStore) CompleteMultipartUpload(ctx context.Context, bucket, path str if err := tx.Delete(&mu).Error; err != nil { return fmt.Errorf("failed to delete multipart upload: %w", err) } + + // Prune the slabs. + if err := pruneSlabs(tx); err != nil { + return fmt.Errorf("failed to prune slabs: %w", err) + } return nil }) if err != nil { From 671f8da872b0decaf217bd4e4d0119d974b1160d Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 10:19:05 +0100 Subject: [PATCH 098/144] stores: add migration to performMigrations --- stores/metadata.go | 6 +++--- stores/migrations.go | 6 ++++++ 2 files changed, 9 insertions(+), 3 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index a9636bf75..13f274477 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -2689,9 +2689,9 @@ func pruneSlabs(tx *gorm.DB) error { // delete slabs without any associated slices or buffers return tx.Exec(` DELETE -FROM slabs sla -WHERE NOT EXISTS (SELECT 1 FROM slices sli WHERE sli.db_slab_id = sla.id) -AND sla.db_buffered_slab_id IS NULL +FROM slabs +WHERE NOT EXISTS (SELECT 1 FROM slices WHERE slices.db_slab_id = slabs.id) +AND slabs.db_buffered_slab_id IS NULL `).Error } diff --git a/stores/migrations.go b/stores/migrations.go index bf3916ca4..d89be7ab5 100644 --- a/stores/migrations.go +++ b/stores/migrations.go @@ -44,6 +44,12 @@ func performMigrations(db *gorm.DB, logger *zap.SugaredLogger) error { return performMigration(tx, dbIdentifier, "00003_idx_objects_size", logger) }, }, + { + ID: "00004_prune_slabs_cascade", + Migrate: func(tx *gorm.DB) error { + return performMigration(tx, dbIdentifier, "00004_prune_slabs_cascade", logger) + }, + }, } // Create migrator. From 2fbcc26ca387a203e49cf16c6d299b8f0692b9f9 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 10:51:30 +0100 Subject: [PATCH 099/144] worker: use different context in scanHost --- worker/host.go | 14 ++--- worker/interactions.go | 119 ----------------------------------------- worker/worker.go | 46 +++++++++------- 3 files changed, 33 insertions(+), 146 deletions(-) diff --git a/worker/host.go b/worker/host.go index 86e92ce27..43e0891af 100644 --- a/worker/host.go +++ b/worker/host.go @@ -52,7 +52,6 @@ type ( acc *account bus Bus contractSpendingRecorder ContractSpendingRecorder - interactionRecorder HostInteractionRecorder logger *zap.SugaredLogger transportPool *transportPoolV3 priceTables *priceTables @@ -70,7 +69,6 @@ func (w *worker) Host(hk types.PublicKey, fcid types.FileContractID, siamuxAddr acc: w.accounts.ForHost(hk), bus: w.bus, contractSpendingRecorder: w.contractSpendingRecorder, - interactionRecorder: w.hostInteractionRecorder, logger: w.logger.Named(hk.String()[:4]), fcid: fcid, siamuxAddr: siamuxAddr, @@ -198,11 +196,13 @@ func (h *host) FetchPriceTable(ctx context.Context, rev *types.FileContractRevis fetchPT := func(paymentFn PriceTablePaymentFunc) (hpt hostdb.HostPriceTable, err error) { err = h.transportPool.withTransportV3(ctx, h.hk, h.siamuxAddr, func(ctx context.Context, t *transportV3) (err error) { hpt, err = RPCPriceTable(ctx, t, paymentFn) - h.interactionRecorder.RecordPriceTableUpdate(hostdb.PriceTableUpdate{ - HostKey: h.hk, - Success: isSuccessfulInteraction(err), - Timestamp: time.Now(), - PriceTable: hpt, + h.bus.RecordPriceTables(ctx, []hostdb.PriceTableUpdate{ + { + HostKey: h.hk, + Success: isSuccessfulInteraction(err), + Timestamp: time.Now(), + PriceTable: hpt, + }, }) return }) diff --git a/worker/interactions.go b/worker/interactions.go index dfe8c4017..2107ae582 100644 --- a/worker/interactions.go +++ b/worker/interactions.go @@ -1,135 +1,16 @@ package worker import ( - "context" - "fmt" - "sync" - "time" - "go.sia.tech/renterd/hostdb" - "go.uber.org/zap" -) - -const ( - keyInteractionRecorder contextKey = "InteractionRecorder" ) type ( HostInteractionRecorder interface { RecordHostScan(...hostdb.HostScan) RecordPriceTableUpdate(...hostdb.PriceTableUpdate) - Stop(context.Context) - } - - hostInteractionRecorder struct { - flushInterval time.Duration - - bus Bus - logger *zap.SugaredLogger - - mu sync.Mutex - hostScans []hostdb.HostScan - priceTableUpdates []hostdb.PriceTableUpdate - - flushCtx context.Context - flushTimer *time.Timer } ) -var ( - _ HostInteractionRecorder = (*hostInteractionRecorder)(nil) -) - -func (w *worker) initHostInteractionRecorder(flushInterval time.Duration) { - if w.hostInteractionRecorder != nil { - panic("HostInteractionRecorder already initialized") // developer error - } - w.hostInteractionRecorder = &hostInteractionRecorder{ - bus: w.bus, - logger: w.logger, - - flushCtx: w.shutdownCtx, - flushInterval: flushInterval, - - hostScans: make([]hostdb.HostScan, 0), - priceTableUpdates: make([]hostdb.PriceTableUpdate, 0), - } -} - -func (r *hostInteractionRecorder) RecordHostScan(scans ...hostdb.HostScan) { - r.mu.Lock() - defer r.mu.Unlock() - r.hostScans = append(r.hostScans, scans...) - r.tryFlushInteractionsBuffer() -} - -func (r *hostInteractionRecorder) RecordPriceTableUpdate(ptUpdates ...hostdb.PriceTableUpdate) { - r.mu.Lock() - defer r.mu.Unlock() - r.priceTableUpdates = append(r.priceTableUpdates, ptUpdates...) - r.tryFlushInteractionsBuffer() -} - -func (r *hostInteractionRecorder) Stop(ctx context.Context) { - // stop the flush timer - r.mu.Lock() - if r.flushTimer != nil { - r.flushTimer.Stop() - } - r.flushCtx = ctx - r.mu.Unlock() - - // flush all interactions - r.flush() - - // log if we weren't able to flush them - r.mu.Lock() - if len(r.hostScans) > 0 { - r.logger.Errorw(fmt.Sprintf("failed to record %d host scans on worker shutdown", len(r.hostScans))) - } - if len(r.priceTableUpdates) > 0 { - r.logger.Errorw(fmt.Sprintf("failed to record %d price table updates on worker shutdown", len(r.priceTableUpdates))) - } - r.mu.Unlock() -} - -func (r *hostInteractionRecorder) flush() { - r.mu.Lock() - defer r.mu.Unlock() - - // NOTE: don't bother flushing if the context is cancelled, we can safely - // ignore the buffered scans and price tables since we'll flush on shutdown - // and log in case we weren't able to flush all interactions to the bus - select { - case <-r.flushCtx.Done(): - r.flushTimer = nil - return - default: - } - - if len(r.hostScans) > 0 { - if err := r.bus.RecordHostScans(r.flushCtx, r.hostScans); err != nil { - r.logger.Errorw(fmt.Sprintf("failed to record scans: %v", err)) - } else if err == nil { - r.hostScans = nil - } - } - if len(r.priceTableUpdates) > 0 { - if err := r.bus.RecordPriceTables(r.flushCtx, r.priceTableUpdates); err != nil { - r.logger.Errorw(fmt.Sprintf("failed to record price table updates: %v", err)) - } else if err == nil { - r.priceTableUpdates = nil - } - } - r.flushTimer = nil -} - -func (r *hostInteractionRecorder) tryFlushInteractionsBuffer() { - if r.flushTimer == nil { - r.flushTimer = time.AfterFunc(r.flushInterval, r.flush) - } -} - func isSuccessfulInteraction(err error) bool { // No error always means success. if err == nil { diff --git a/worker/worker.go b/worker/worker.go index 8a2a9b1f3..d01d57323 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -200,7 +200,6 @@ type worker struct { uploadsMu sync.Mutex uploadingPackedSlabs map[string]bool - hostInteractionRecorder HostInteractionRecorder contractSpendingRecorder ContractSpendingRecorder contractLockingDuration time.Duration @@ -342,11 +341,13 @@ func (w *worker) rhpPriceTableHandler(jc jape.Context) { var err error var hpt hostdb.HostPriceTable defer func() { - w.hostInteractionRecorder.RecordPriceTableUpdate(hostdb.PriceTableUpdate{ - HostKey: rptr.HostKey, - Success: isSuccessfulInteraction(err), - Timestamp: time.Now(), - PriceTable: hpt, + w.bus.RecordPriceTables(ctx, []hostdb.PriceTableUpdate{ + { + HostKey: rptr.HostKey, + Success: isSuccessfulInteraction(err), + Timestamp: time.Now(), + PriceTable: hpt, + }, }) }() @@ -1292,6 +1293,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()) w := &worker{ alerts: alerts.WithOrigin(b, fmt.Sprintf("worker.%s", id)), allowPrivateIPs: allowPrivateIPs, @@ -1302,13 +1304,10 @@ 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, } - ctx, cancel := context.WithCancel(context.Background()) - ctx = context.WithValue(ctx, keyInteractionRecorder, w) - w.shutdownCtx = ctx - w.shutdownCtxCancel = cancel - w.initAccounts(b) w.initPriceTables() w.initTransportPool() @@ -1317,7 +1316,6 @@ func New(masterKey [32]byte, id string, b Bus, contractLockingDuration, busFlush w.initUploadManager(uploadMaxMemory, uploadMaxOverdrive, uploadOverdriveTimeout, l.Sugar().Named("uploadmanager")) w.initContractSpendingRecorder(busFlushInterval) - w.initHostInteractionRecorder(busFlushInterval) return w, nil } @@ -1364,7 +1362,6 @@ func (w *worker) Shutdown(ctx context.Context) error { w.uploadManager.Stop() // stop recorders - w.hostInteractionRecorder.Stop(ctx) w.contractSpendingRecorder.Stop(ctx) return nil } @@ -1441,14 +1438,23 @@ func (w *worker) scanHost(ctx context.Context, hostKey types.PublicKey, hostIP s default: } - // record host scan - w.hostInteractionRecorder.RecordHostScan(hostdb.HostScan{ - HostKey: hostKey, - Success: isSuccessfulInteraction(err), - Timestamp: time.Now(), - Settings: settings, - PriceTable: pt, + // record host scan - make sure this isn't interrupted by the same context + // used to time out the scan itself because otherwise we won't be able to + // record scans that timed out. + recordCtx, cancel := context.WithTimeout(context.Background(), 30*time.Second) + defer cancel() + scanErr := w.bus.RecordHostScans(recordCtx, []hostdb.HostScan{ + { + HostKey: hostKey, + Success: isSuccessfulInteraction(err), + Timestamp: time.Now(), + Settings: settings, + PriceTable: pt, + }, }) + if scanErr != nil { + w.logger.Errorf("failed to record host scan: %v", scanErr) + } return settings, pt, duration, err } From 0a23dea48598451ab56cb5b836a4f6a3cf4c8240 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 10:57:09 +0100 Subject: [PATCH 100/144] stores: fix TestSlabCleanupTrigger --- stores/metadata.go | 5 ++--- stores/metadata_test.go | 9 ++++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/stores/metadata.go b/stores/metadata.go index 13f274477..2ab2cf5da 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -2739,13 +2739,12 @@ func (s *SQLStore) deleteObjects(bucket string, path string) (numDeleted int64, if err := res.Error; err != nil { return res.Error } - duration = time.Since(start) - rowsAffected = res.RowsAffected - // prune slabs if we deleted an object + rowsAffected = res.RowsAffected if rowsAffected > 0 { return pruneSlabs(tx) } + duration = time.Since(start) return nil }); err != nil { return 0, fmt.Errorf("failed to delete objects: %w", err) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 4a6102399..d5a0f9531 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -3941,7 +3941,8 @@ func TestSlabCleanupTrigger(t *testing.T) { } // delete the object - if err := ss.db.Delete(&obj1).Error; err != nil { + err := ss.RemoveObject(context.Background(), api.DefaultBucketName, obj1.ObjectID) + if err != nil { t.Fatal(err) } @@ -3954,7 +3955,8 @@ func TestSlabCleanupTrigger(t *testing.T) { } // delete second object - if err := ss.db.Delete(&obj2).Error; err != nil { + err = ss.RemoveObject(context.Background(), api.DefaultBucketName, obj2.ObjectID) + if err != nil { t.Fatal(err) } @@ -3998,7 +4000,8 @@ func TestSlabCleanupTrigger(t *testing.T) { } // delete third object - if err := ss.db.Delete(&obj3).Error; err != nil { + err = ss.RemoveObject(context.Background(), api.DefaultBucketName, obj3.ObjectID) + if err != nil { t.Fatal(err) } if err := ss.db.Model(&dbSlab{}).Count(&slabCntr).Error; err != nil { From 50c201aa65066abef4984dce47ee87484945d42c Mon Sep 17 00:00:00 2001 From: PJ Date: Mon, 19 Feb 2024 14:42:38 +0100 Subject: [PATCH 101/144] stores: adapt unit tests to MySQL --- .github/workflows/test.yml | 64 +++++++++--------- stores/hostdb_test.go | 41 ++++++------ stores/metadata.go | 4 ++ stores/metadata_test.go | 129 +++++++++++++++++++++++-------------- stores/metrics.go | 4 +- stores/metrics_test.go | 2 +- stores/sql_test.go | 81 +++++++++++++++++------ 7 files changed, 200 insertions(+), 125 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index e8a32e5ec..b96eddebe 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -30,23 +30,23 @@ jobs: uses: actions/setup-go@v3 with: go-version: ${{ matrix.go-version }} - - name: Lint - uses: golangci/golangci-lint-action@v3 - with: - args: --timeout=30m - - name: Jape Analyzer - uses: SiaFoundation/action-golang-analysis@HEAD - with: - analyzers: | - go.sia.tech/jape.Analyzer@master - directories: | - autopilot - bus bus/client - worker worker/client - - name: Test - uses: n8maninger/action-golang-test@v1 - with: - args: "-race;-short" + # - name: Lint + # uses: golangci/golangci-lint-action@v3 + # with: + # args: --timeout=30m + # - name: Jape Analyzer + # uses: SiaFoundation/action-golang-analysis@HEAD + # with: + # analyzers: | + # go.sia.tech/jape.Analyzer@master + # directories: | + # autopilot + # bus bus/client + # worker worker/client + # - name: Test + # uses: n8maninger/action-golang-test@v1 + # with: + # args: "-race;-short" - name: Test Stores - MySQL if: matrix.os == 'ubuntu-latest' uses: n8maninger/action-golang-test@v1 @@ -57,20 +57,20 @@ jobs: with: package: "./stores" args: "-race;-short" - - name: Test Integration - uses: n8maninger/action-golang-test@v1 - with: - package: "./internal/testing/..." - args: "-failfast;-race;-tags=testing;-timeout=30m" - - name: Test Integration - MySQL - if: matrix.os == 'ubuntu-latest' - uses: n8maninger/action-golang-test@v1 - env: - RENTERD_DB_URI: 127.0.0.1:3800 - RENTERD_DB_USER: root - RENTERD_DB_PASSWORD: test - with: - package: "./internal/testing/..." - args: "-failfast;-race;-tags=testing;-timeout=30m" + # - name: Test Integration + # uses: n8maninger/action-golang-test@v1 + # with: + # package: "./internal/testing/..." + # args: "-failfast;-race;-tags=testing;-timeout=30m" + # - name: Test Integration - MySQL + # if: matrix.os == 'ubuntu-latest' + # uses: n8maninger/action-golang-test@v1 + # env: + # RENTERD_DB_URI: 127.0.0.1:3800 + # RENTERD_DB_USER: root + # RENTERD_DB_PASSWORD: test + # with: + # package: "./internal/testing/..." + # args: "-failfast;-race;-tags=testing;-timeout=30m" - name: Build run: go build -o bin/ ./cmd/renterd diff --git a/stores/hostdb_test.go b/stores/hostdb_test.go index a61f9eea3..35872ea2d 100644 --- a/stores/hostdb_test.go +++ b/stores/hostdb_test.go @@ -63,15 +63,8 @@ func TestSQLHostDB(t *testing.T) { // Insert an announcement for the host and another one for an unknown // host. - a := hostdb.Announcement{ - Index: types.ChainIndex{ - Height: 42, - ID: types.BlockID{1, 2, 3}, - }, - Timestamp: time.Now().UTC().Round(time.Second), - NetAddress: "address", - } - err = ss.insertTestAnnouncement(hk, a) + ann := newTestHostDBAnnouncement("address") + err = ss.insertTestAnnouncement(hk, ann) if err != nil { t.Fatal(err) } @@ -79,7 +72,7 @@ func TestSQLHostDB(t *testing.T) { // Read the host and verify that the announcement related fields were // set. var h dbHost - tx := ss.db.Where("last_announcement = ? AND net_address = ?", a.Timestamp, a.NetAddress).Find(&h) + tx := ss.db.Where("last_announcement = ? AND net_address = ?", ann.Timestamp, ann.NetAddress).Find(&h) if tx.Error != nil { t.Fatal(tx.Error) } @@ -116,7 +109,7 @@ func TestSQLHostDB(t *testing.T) { // Insert another announcement for an unknown host. unknownKey := types.PublicKey{1, 4, 7} - err = ss.insertTestAnnouncement(unknownKey, a) + err = ss.insertTestAnnouncement(unknownKey, ann) if err != nil { t.Fatal(err) } @@ -124,7 +117,7 @@ func TestSQLHostDB(t *testing.T) { if err != nil { t.Fatal(err) } - if h3.NetAddress != a.NetAddress { + if h3.NetAddress != ann.NetAddress { t.Fatal("wrong net address") } if h3.KnownSince.IsZero() { @@ -510,22 +503,18 @@ func TestInsertAnnouncements(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) defer ss.Close() - // Create announcements for 2 hosts. + // Create announcements for 3 hosts. ann1 := announcement{ - hostKey: publicKey(types.GeneratePrivateKey().PublicKey()), - announcement: hostdb.Announcement{ - Index: types.ChainIndex{Height: 1, ID: types.BlockID{1}}, - Timestamp: time.Now(), - NetAddress: "foo.bar:1000", - }, + hostKey: publicKey(types.GeneratePrivateKey().PublicKey()), + announcement: newTestHostDBAnnouncement("foo.bar:1000"), } ann2 := announcement{ hostKey: publicKey(types.GeneratePrivateKey().PublicKey()), - announcement: hostdb.Announcement{}, + announcement: newTestHostDBAnnouncement("bar.baz:1000"), } ann3 := announcement{ hostKey: publicKey(types.GeneratePrivateKey().PublicKey()), - announcement: hostdb.Announcement{}, + announcement: newTestHostDBAnnouncement("quz.qux:1000"), } // Insert the first one and check that all fields are set. @@ -1101,7 +1090,7 @@ func (s *SQLStore) addCustomTestHost(hk types.PublicKey, na string) error { s.unappliedHostKeys[hk] = struct{}{} s.unappliedAnnouncements = append(s.unappliedAnnouncements, []announcement{{ hostKey: publicKey(hk), - announcement: hostdb.Announcement{NetAddress: na}, + announcement: newTestHostDBAnnouncement(na), }}...) s.lastSave = time.Now().Add(s.persistInterval * -2) return s.applyUpdates(false) @@ -1153,6 +1142,14 @@ func newTestHostAnnouncement(na modules.NetAddress) (modules.HostAnnouncement, t }, sk } +func newTestHostDBAnnouncement(addr string) hostdb.Announcement { + return hostdb.Announcement{ + Index: types.ChainIndex{Height: 1, ID: types.BlockID{1}}, + Timestamp: time.Now().UTC().Round(time.Second), + NetAddress: addr, + } +} + func newTestTransaction(ha modules.HostAnnouncement, sk types.PrivateKey) stypes.Transaction { var buf bytes.Buffer buf.Write(encoding.Marshal(ha)) diff --git a/stores/metadata.go b/stores/metadata.go index 68947ed95..997b8b343 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -1507,6 +1507,10 @@ func (s *SQLStore) RenameObjects(ctx context.Context, bucket, prefixOld, prefixN gorm.Expr(sqlConcat(tx, "?", "SUBSTR(object_id, ?)")), prefixNew, utf8.RuneCountInString(prefixOld)+1, prefixOld+"%", utf8.RuneCountInString(prefixOld), prefixOld, sqlWhereBucket("objects", bucket)) + + if !isSQLite(tx) { + inner = tx.Raw("SELECT * FROM (?) as i", inner) + } resp := tx.Model(&dbObject{}). Where("object_id IN (?)", inner). Delete(&dbObject{}) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 4a6102399..ad886e6ef 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -6,7 +6,6 @@ import ( "encoding/hex" "errors" "fmt" - "math" "os" "reflect" "sort" @@ -18,7 +17,6 @@ import ( 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" "gorm.io/gorm" "gorm.io/gorm/schema" @@ -220,7 +218,7 @@ func TestSQLContractStore(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(hk, hostdb.Announcement{NetAddress: "address"}) + err = ss.insertTestAnnouncement(hk, newTestHostDBAnnouncement("address")) if err != nil { t.Fatal(err) } @@ -511,11 +509,11 @@ func TestRenewedContract(t *testing.T) { hk, hk2 := hks[0], hks[1] // Add announcements. - err = ss.insertTestAnnouncement(hk, hostdb.Announcement{NetAddress: "address"}) + err = ss.insertTestAnnouncement(hk, newTestHostDBAnnouncement("address")) if err != nil { t.Fatal(err) } - err = ss.insertTestAnnouncement(hk2, hostdb.Announcement{NetAddress: "address2"}) + err = ss.insertTestAnnouncement(hk2, newTestHostDBAnnouncement("address2")) if err != nil { t.Fatal(err) } @@ -1008,7 +1006,7 @@ func TestSQLMetadataStore(t *testing.T) { one := uint(1) expectedObj := dbObject{ - DBBucketID: 1, + DBBucketID: ss.DefaultBucketID(), Health: 1, ObjectID: objID, Key: obj1Key, @@ -1169,6 +1167,7 @@ func TestSQLMetadataStore(t *testing.T) { slabs[i].Shards[0].Model = Model{} slabs[i].Shards[0].Contracts[0].Model = Model{} slabs[i].Shards[0].Contracts[0].Host.Model = Model{} + slabs[i].Shards[0].Contracts[0].Host.LastAnnouncement = time.Time{} slabs[i].HealthValidUntil = 0 } if !reflect.DeepEqual(slab1, expectedObjSlab1) { @@ -2213,10 +2212,9 @@ func TestUpdateSlab(t *testing.T) { t.Fatal(err) } var s dbSlab - if err := ss.db.Model(&dbSlab{}). + if err := ss.db.Where(&dbSlab{Key: key}). Joins("DBContractSet"). Preload("Shards"). - Where("key = ?", key). Take(&s). Error; err != nil { t.Fatal(err) @@ -2265,7 +2263,7 @@ func TestRecordContractSpending(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(hk, hostdb.Announcement{NetAddress: "address"}) + err = ss.insertTestAnnouncement(hk, newTestHostDBAnnouncement("address")) if err != nil { t.Fatal(err) } @@ -3897,7 +3895,7 @@ func TestSlabCleanupTrigger(t *testing.T) { // create objects obj1 := dbObject{ ObjectID: "1", - DBBucketID: 1, + DBBucketID: ss.DefaultBucketID(), Health: 1, } if err := ss.db.Create(&obj1).Error; err != nil { @@ -3905,7 +3903,7 @@ func TestSlabCleanupTrigger(t *testing.T) { } obj2 := dbObject{ ObjectID: "2", - DBBucketID: 1, + DBBucketID: ss.DefaultBucketID(), Health: 1, } if err := ss.db.Create(&obj2).Error; err != nil { @@ -3978,7 +3976,7 @@ func TestSlabCleanupTrigger(t *testing.T) { } obj3 := dbObject{ ObjectID: "3", - DBBucketID: 1, + DBBucketID: ss.DefaultBucketID(), Health: 1, } if err := ss.db.Create(&obj3).Error; err != nil { @@ -4117,11 +4115,11 @@ func TestUpdateObjectReuseSlab(t *testing.T) { // fetch the object var dbObj dbObject - if err := ss.db.Where("db_bucket_id", 1).Take(&dbObj).Error; err != nil { + if err := ss.db.Where("db_bucket_id", ss.DefaultBucketID()).Take(&dbObj).Error; err != nil { t.Fatal(err) } else if dbObj.ID != 1 { t.Fatal("unexpected id", dbObj.ID) - } else if dbObj.DBBucketID != 1 { + } else if dbObj.DBBucketID != ss.DefaultBucketID() { t.Fatal("bucket id mismatch", dbObj.DBBucketID) } else if dbObj.ObjectID != "1" { t.Fatal("object id mismatch", dbObj.ObjectID) @@ -4223,7 +4221,7 @@ func TestUpdateObjectReuseSlab(t *testing.T) { // fetch the object var dbObj2 dbObject - if err := ss.db.Where("db_bucket_id", 1). + if err := ss.db.Where("db_bucket_id", ss.DefaultBucketID()). Where("object_id", "2"). Take(&dbObj2).Error; err != nil { t.Fatal(err) @@ -4307,57 +4305,94 @@ func TestTypeCurrency(t *testing.T) { ss := newTestSQLStore(t, defaultTestSQLStoreConfig) defer ss.Close() + // prepare the table + if isSQLite(ss.db) { + if err := ss.db.Exec("CREATE TABLE currencies (id INTEGER PRIMARY KEY AUTOINCREMENT,c BLOB);").Error; err != nil { + t.Fatal(err) + } + } else { + if err := ss.db.Exec("CREATE TABLE currencies (id INT AUTO_INCREMENT PRIMARY KEY, c BLOB);").Error; err != nil { + t.Fatal(err) + } + } + + // insert currencies in random order + values := []interface{}{ + bCurrency(types.ZeroCurrency), + bCurrency(types.NewCurrency64(1)), + bCurrency(types.MaxCurrency), + } + frand.Shuffle(len(values), func(i, j int) { values[i], values[j] = values[j], values[i] }) + if err := ss.db.Exec("INSERT INTO currencies (c) VALUES (?),(?),(?);", values...).Error; err != nil { + t.Fatal(err) + } + + // fetch currencies and assert they're sorted + var currencies []bCurrency + if err := ss.db.Raw(`SELECT c FROM currencies ORDER BY c ASC`).Scan(¤cies).Error; err != nil { + t.Fatal(err) + } else if !sort.SliceIsSorted(currencies, func(i, j int) bool { + return types.Currency(currencies[i]).Cmp(types.Currency(currencies[j])) < 0 + }) { + t.Fatal("currencies not sorted", currencies) + } + + // convenience variables + c0 := currencies[0] + c1 := currencies[1] + cM := currencies[2] + tests := []struct { - a types.Currency - b types.Currency + a bCurrency + b bCurrency cmp string }{ { - a: types.ZeroCurrency, - b: types.NewCurrency64(1), + a: c0, + b: c1, cmp: "<", }, { - a: types.NewCurrency64(1), - b: types.NewCurrency64(1), + a: c1, + b: c0, + cmp: ">", + }, + { + a: c0, + b: c1, + cmp: "!=", + }, + { + a: c1, + b: c1, cmp: "=", }, { - a: types.NewCurrency(0, math.MaxUint64), - b: types.NewCurrency(math.MaxUint64, 0), + a: c0, + b: cM, cmp: "<", }, { - a: types.NewCurrency(math.MaxUint64, 0), - b: types.NewCurrency(0, math.MaxUint64), + a: cM, + b: c0, cmp: ">", }, + { + a: cM, + b: cM, + cmp: "=", + }, } - for _, test := range tests { + for i, test := range tests { var result bool - err := ss.db.Raw("SELECT ? "+test.cmp+" ?", bCurrency(test.a), bCurrency(test.b)).Scan(&result).Error - if err != nil { + query := fmt.Sprintf("SELECT ? %s ?", test.cmp) + if !isSQLite(ss.db) { + query = strings.Replace(query, "?", "HEX(?)", -1) + } + if err := ss.db.Raw(query, test.a, test.b).Scan(&result).Error; err != nil { t.Fatal(err) } else if !result { - t.Fatal("unexpected result", result) + t.Errorf("unexpected result in case %d/%d: expected %v %s %v to be true", i+1, len(tests), types.Currency(test.a).String(), test.cmp, types.Currency(test.b).String()) } } - - c := func(c uint64) bCurrency { - return bCurrency(types.NewCurrency64(c)) - } - - var currencies []bCurrency - err := ss.db.Raw(` -WITH input(col) as -(values (?),(?),(?)) -SELECT * FROM input ORDER BY col ASC -`, c(3), c(1), c(2)).Scan(¤cies).Error - if err != nil { - t.Fatal(err) - } else if !sort.SliceIsSorted(currencies, func(i, j int) bool { - return types.Currency(currencies[i]).Cmp(types.Currency(currencies[j])) < 0 - }) { - t.Fatal("currencies not sorted", currencies) - } } diff --git a/stores/metrics.go b/stores/metrics.go index 203ed3b71..8816d1729 100644 --- a/stores/metrics.go +++ b/stores/metrics.go @@ -605,9 +605,7 @@ func (s *SQLStore) findPeriods(table string, dst interface{}, start time.Time, n WHERE ? GROUP BY p.period_start - ORDER BY - p.period_start ASC - ) i ON %s.id = i.id + ) i ON %s.id = i.id ORDER BY Period ASC `, table, table, table, table), unixTimeMS(start), interval.Milliseconds(), diff --git a/stores/metrics_test.go b/stores/metrics_test.go index 2b2f572a7..f71d985bd 100644 --- a/stores/metrics_test.go +++ b/stores/metrics_test.go @@ -517,7 +517,7 @@ func TestWalletMetrics(t *testing.T) { } else if !sort.SliceIsSorted(metrics, func(i, j int) bool { return time.Time(metrics[i].Timestamp).Before(time.Time(metrics[j].Timestamp)) }) { - t.Fatal("expected metrics to be sorted by time") + t.Fatalf("expected metrics to be sorted by time, %+v", metrics) } // Prune metrics diff --git a/stores/sql_test.go b/stores/sql_test.go index ae0c54b0c..776e3e10e 100644 --- a/stores/sql_test.go +++ b/stores/sql_test.go @@ -96,8 +96,16 @@ func newTestSQLStore(t *testing.T, cfg testSQLStoreConfig) *testSQLStore { var conn, connMetrics gorm.Dialector if dbURI != "" { - conn = NewMySQLConnection(dbURI, dbUser, dbPassword, dbName) - connMetrics = NewMySQLConnection(dbURI, dbUser, dbPassword, dbMetricsName) + if tmpDB, err := gorm.Open(NewMySQLConnection(dbUser, dbPassword, dbURI, "")); err != nil { + t.Fatal(err) + } else if err := tmpDB.Exec(fmt.Sprintf("CREATE DATABASE IF NOT EXISTS `%s`", dbName)).Error; err != nil { + t.Fatal(err) + } else if err := tmpDB.Exec(fmt.Sprintf("CREATE DATABASE IF NOT EXISTS `%s`", dbMetricsName)).Error; err != nil { + t.Fatal(err) + } + + conn = NewMySQLConnection(dbUser, dbPassword, dbURI, dbName) + connMetrics = NewMySQLConnection(dbUser, dbPassword, dbURI, dbMetricsName) } else if cfg.persistent { conn = NewSQLiteConnection(filepath.Join(cfg.dir, "db.sqlite")) connMetrics = NewSQLiteConnection(filepath.Join(cfg.dir, "metrics.sqlite")) @@ -148,6 +156,18 @@ func (s *testSQLStore) Close() error { return nil } +func (s *testSQLStore) DefaultBucketID() uint { + var b dbBucket + if err := s.db. + Model(&dbBucket{}). + Where("name = ?", api.DefaultBucketName). + Take(&b). + Error; err != nil { + s.t.Fatal(err) + } + return b.ID +} + func (s *testSQLStore) Reopen() *testSQLStore { s.t.Helper() cfg := defaultTestSQLStoreConfig @@ -240,11 +260,13 @@ func (s *SQLStore) contractsCount() (cnt int64, err error) { func (s *SQLStore) overrideSlabHealth(objectID string, health float64) (err error) { err = s.db.Exec(fmt.Sprintf(` UPDATE slabs SET health = %v WHERE id IN ( - SELECT sla.id - FROM objects o - INNER JOIN slices sli ON o.id = sli.db_object_id - INNER JOIN slabs sla ON sli.db_slab_id = sla.id - WHERE o.object_id = "%s" + SELECT * FROM ( + SELECT sla.id + FROM objects o + INNER JOIN slices sli ON o.id = sli.db_object_id + INNER JOIN slabs sla ON sli.db_slab_id = sla.id + WHERE o.object_id = "%s" + ) AS sub )`, health, objectID)).Error return } @@ -306,11 +328,24 @@ func TestConsensusReset(t *testing.T) { } } -type queryPlanExplain struct { - ID int `json:"id"` - Parent int `json:"parent"` - NotUsed bool `json:"notused"` - Detail string `json:"detail"` +type sqliteQueryPlan struct { + Detail string `json:"detail"` +} + +func (p sqliteQueryPlan) usesIndex() bool { + d := strings.ToLower(p.Detail) + return strings.Contains(d, "using index") || strings.Contains(d, "using covering index") +} + +//nolint:tagliatelle +type mysqlQueryPlan struct { + Extra string `json:"Extra"` + PossibleKeys string `json:"possible_keys"` +} + +func (p mysqlQueryPlan) usesIndex() bool { + d := strings.ToLower(p.Extra) + return strings.Contains(d, "using index") || strings.Contains(p.PossibleKeys, "idx_") } func TestQueryPlan(t *testing.T) { @@ -346,14 +381,20 @@ func TestQueryPlan(t *testing.T) { } for _, query := range queries { - var explain queryPlanExplain - err := ss.db.Raw(fmt.Sprintf("EXPLAIN QUERY PLAN %s;", query)).Scan(&explain).Error - if err != nil { - t.Fatal(err) - } - if !(strings.Contains(explain.Detail, "USING INDEX") || - strings.Contains(explain.Detail, "USING COVERING INDEX")) { - t.Fatalf("query '%s' should use an index, instead the plan was '%s'", query, explain.Detail) + if isSQLite(ss.db) { + var explain sqliteQueryPlan + if err := ss.db.Raw(fmt.Sprintf("EXPLAIN QUERY PLAN %s;", query)).Scan(&explain).Error; err != nil { + t.Fatal(err) + } else if !explain.usesIndex() { + t.Fatalf("query '%s' should use an index, instead the plan was %+v", query, explain) + } + } else { + var explain mysqlQueryPlan + if err := ss.db.Raw(fmt.Sprintf("EXPLAIN %s;", query)).Scan(&explain).Error; err != nil { + t.Fatal(err) + } else if !explain.usesIndex() { + t.Fatalf("query '%s' should use an index, instead the plan was %+v", query, explain) + } } } } From e8c5c92460bfda480ca0222c4cc675bde3588f83 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 20 Feb 2024 11:32:20 +0100 Subject: [PATCH 102/144] ci: enable all tests --- .github/workflows/test.yml | 64 +++++++++++++++++++------------------- 1 file changed, 32 insertions(+), 32 deletions(-) diff --git a/.github/workflows/test.yml b/.github/workflows/test.yml index b96eddebe..e8a32e5ec 100644 --- a/.github/workflows/test.yml +++ b/.github/workflows/test.yml @@ -30,23 +30,23 @@ jobs: uses: actions/setup-go@v3 with: go-version: ${{ matrix.go-version }} - # - name: Lint - # uses: golangci/golangci-lint-action@v3 - # with: - # args: --timeout=30m - # - name: Jape Analyzer - # uses: SiaFoundation/action-golang-analysis@HEAD - # with: - # analyzers: | - # go.sia.tech/jape.Analyzer@master - # directories: | - # autopilot - # bus bus/client - # worker worker/client - # - name: Test - # uses: n8maninger/action-golang-test@v1 - # with: - # args: "-race;-short" + - name: Lint + uses: golangci/golangci-lint-action@v3 + with: + args: --timeout=30m + - name: Jape Analyzer + uses: SiaFoundation/action-golang-analysis@HEAD + with: + analyzers: | + go.sia.tech/jape.Analyzer@master + directories: | + autopilot + bus bus/client + worker worker/client + - name: Test + uses: n8maninger/action-golang-test@v1 + with: + args: "-race;-short" - name: Test Stores - MySQL if: matrix.os == 'ubuntu-latest' uses: n8maninger/action-golang-test@v1 @@ -57,20 +57,20 @@ jobs: with: package: "./stores" args: "-race;-short" - # - name: Test Integration - # uses: n8maninger/action-golang-test@v1 - # with: - # package: "./internal/testing/..." - # args: "-failfast;-race;-tags=testing;-timeout=30m" - # - name: Test Integration - MySQL - # if: matrix.os == 'ubuntu-latest' - # uses: n8maninger/action-golang-test@v1 - # env: - # RENTERD_DB_URI: 127.0.0.1:3800 - # RENTERD_DB_USER: root - # RENTERD_DB_PASSWORD: test - # with: - # package: "./internal/testing/..." - # args: "-failfast;-race;-tags=testing;-timeout=30m" + - name: Test Integration + uses: n8maninger/action-golang-test@v1 + with: + package: "./internal/testing/..." + args: "-failfast;-race;-tags=testing;-timeout=30m" + - name: Test Integration - MySQL + if: matrix.os == 'ubuntu-latest' + uses: n8maninger/action-golang-test@v1 + env: + RENTERD_DB_URI: 127.0.0.1:3800 + RENTERD_DB_USER: root + RENTERD_DB_PASSWORD: test + with: + package: "./internal/testing/..." + args: "-failfast;-race;-tags=testing;-timeout=30m" - name: Build run: go build -o bin/ ./cmd/renterd From 96d4bba92f350ffc4490ff6f53011e9c989f7e47 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 13:37:50 +0100 Subject: [PATCH 103/144] stores: only drop triggers if they exist --- .../mysql/main/migration_00004_prune_slabs_cascade.sql | 8 ++++---- .../sqlite/main/migration_00004_prune_slabs_cascade.sql | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql index 48f98a40a..125da0ecb 100644 --- a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql @@ -18,7 +18,7 @@ ALTER TABLE multipart_parts DROP FOREIGN KEY fk_multipart_uploads_parts; ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE -- drop triggers -DROP TRIGGER before_delete_on_objects_delete_slices -DROP TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts -DROP TRIGGER before_delete_on_multipart_parts_delete_slices -DROP TRIGGER after_delete_on_slices_delete_slabs \ No newline at end of file +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs \ No newline at end of file diff --git a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql index 2cd633c11..b7f5ab128 100644 --- a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql @@ -22,7 +22,7 @@ CREATE INDEX `idx_multipart_parts_etag` ON `multipart_parts`(`etag`); PRAGMA foreign_keys=on; -- drop triggers -DROP TRIGGER before_delete_on_objects_delete_slices -DROP TRIGGER before_delete_on_multipart_uploads_delete_multipart_parts -DROP TRIGGER before_delete_on_multipart_parts_delete_slices -DROP TRIGGER after_delete_on_slices_delete_slabs \ No newline at end of file +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs \ No newline at end of file From 4cb5efaf0d1277d3225de8689421a963009490e2 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 14:42:10 +0100 Subject: [PATCH 104/144] worker: address review comment --- worker/worker.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/worker.go b/worker/worker.go index b57623b7b..094722c00 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -1155,7 +1155,7 @@ func (w *worker) multipartUploadHandlerPUT(jc jape.Context) { // make sure only one of the following is set if encryptionEnabled := !upload.Key.IsNoopKey(); encryptionEnabled && jc.Request.FormValue("offset") == "" { - jc.Error(errors.New("if presharding encryption isn't disabled, the offset needs to be set"), http.StatusBadRequest) + jc.Error(errors.New("if object encryption (pre-erasure coding) wasn't disabled by creating the multipart upload with the no-op key, the offset needs to be set"), http.StatusBadRequest) return } else if encryptionEnabled { opts = append(opts, WithCustomEncryptionOffset(uint64(offset))) From adafec043860b8514c71e535eddec1e3729bb407 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 15:24:05 +0100 Subject: [PATCH 105/144] stores: add missing newline --- .../mysql/main/migration_00004_prune_slabs_cascade.sql | 2 +- .../sqlite/main/migration_00004_prune_slabs_cascade.sql | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql index 125da0ecb..0b1c06994 100644 --- a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql @@ -21,4 +21,4 @@ ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KE DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs \ No newline at end of file +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs diff --git a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql index b7f5ab128..1132dd2f5 100644 --- a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql @@ -25,4 +25,4 @@ PRAGMA foreign_keys=on; DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs \ No newline at end of file +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs From c346017cb83c40a351576a6056e6baecd2686e0f Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 18:03:56 +0100 Subject: [PATCH 106/144] bus: change response type for paginated alerts request --- alerts/alerts.go | 18 +++++++++++++++--- bus/bus.go | 9 +++++++++ bus/client/alerts.go | 8 +++----- internal/testing/cluster_test.go | 13 ++++++++----- 4 files changed, 35 insertions(+), 13 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index 424196d4f..f5e19231a 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -69,6 +69,12 @@ type ( Offset int Limit int } + + AlertsResponse struct { + Alerts []Alert `json:"alerts"` + HasMore bool `json:"hasMore"` + Total int `json:"total"` + } ) // String implements the fmt.Stringer interface. @@ -176,12 +182,16 @@ func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error } // Active returns the host's active alerts. -func (m *Manager) Active(offset, limit int) []Alert { +func (m *Manager) Active(offset, limit int) AlertsResponse { m.mu.Lock() defer m.mu.Unlock() + resp := AlertsResponse{ + Total: len(m.alerts), + } + if offset >= len(m.alerts) { - return nil + return resp } else if limit == -1 { limit = len(m.alerts) } @@ -196,8 +206,10 @@ func (m *Manager) Active(offset, limit int) []Alert { alerts = alerts[offset:] if limit < len(alerts) { alerts = alerts[:limit] + resp.HasMore = true } - return alerts + resp.Alerts = alerts + return resp } func (m *Manager) RegisterWebhookBroadcaster(b webhooks.Broadcaster) { diff --git a/bus/bus.go b/bus/bus.go index 9fb4c1254..6cce5256d 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1716,6 +1716,15 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { } func (b *bus) handleGETAlerts(jc jape.Context) { + if jc.Request.FormValue("offset") != "" || jc.Request.FormValue("limit") != "" { + b.handleGETAlertsPaginated(jc) + return + } + ar := b.alertMgr.Active(0, -1) + jc.Encode(ar.Alerts) +} + +func (b *bus) handleGETAlertsPaginated(jc jape.Context) { offset, limit := 0, -1 if jc.DecodeForm("offset", &offset) != nil { return diff --git a/bus/client/alerts.go b/bus/client/alerts.go index bff3c13a5..1b876e877 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -10,15 +10,13 @@ import ( ) // Alerts fetches the active alerts from the bus. -func (c *Client) Alerts(opts alerts.AlertsOpts) (alerts []alerts.Alert, err error) { +func (c *Client) Alerts(opts alerts.AlertsOpts) (resp alerts.AlertsResponse, err error) { values := url.Values{} - if opts.Offset != 0 { - values.Set("offset", fmt.Sprint(opts.Offset)) - } + values.Set("offset", fmt.Sprint(opts.Offset)) if opts.Limit != 0 { values.Set("limit", fmt.Sprint(opts.Limit)) } - err = c.c.GET("/alerts?"+values.Encode(), &alerts) + err = c.c.GET("/alerts?"+values.Encode(), &resp) return } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 39d6b373a..f0734a0f3 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1923,9 +1923,9 @@ func TestAlerts(t *testing.T) { tt.OK(b.RegisterAlert(context.Background(), alert)) findAlert := func(id types.Hash256) *alerts.Alert { t.Helper() - alerts, err := b.Alerts(alerts.AlertsOpts{}) + ar, err := b.Alerts(alerts.AlertsOpts{}) tt.OK(err) - for _, alert := range alerts { + for _, alert := range ar.Alerts { if alert.ID == id { return &alert } @@ -1960,14 +1960,16 @@ func TestAlerts(t *testing.T) { } // try to find with offset = 1 - foundAlerts, err := b.Alerts(alerts.AlertsOpts{Offset: 1}) + ar, err := b.Alerts(alerts.AlertsOpts{Offset: 1}) + foundAlerts := ar.Alerts tt.OK(err) if len(foundAlerts) != 1 || foundAlerts[0].ID != alert.ID { t.Fatal("wrong alert") } // try to find with limit = 1 - foundAlerts, err = b.Alerts(alerts.AlertsOpts{Limit: 1}) + ar, err = b.Alerts(alerts.AlertsOpts{Limit: 1}) + foundAlerts = ar.Alerts tt.OK(err) if len(foundAlerts) != 1 || foundAlerts[0].ID != alert2.ID { t.Fatal("wrong alert") @@ -1975,7 +1977,8 @@ func TestAlerts(t *testing.T) { // dismiss all tt.OK(b.DismissAllAlerts(context.Background())) - foundAlerts, err = b.Alerts(alerts.AlertsOpts{}) + ar, err = b.Alerts(alerts.AlertsOpts{}) + foundAlerts = ar.Alerts tt.OK(err) if len(foundAlerts) != 0 { t.Fatal("expected 0 alerts", len(foundAlerts)) From ce3e6924e1604c61c9b528749185b451107530c9 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 18:52:02 +0100 Subject: [PATCH 107/144] bus: revert dismissall alerts endpoint --- alerts/alerts.go | 17 ----------------- bus/bus.go | 7 ------- bus/client/alerts.go | 5 ----- internal/testing/cluster_test.go | 9 --------- 4 files changed, 38 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index f5e19231a..b0d4963c6 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -37,7 +37,6 @@ type ( Alerter interface { RegisterAlert(_ context.Context, a Alert) error DismissAlerts(_ context.Context, ids ...types.Hash256) error - DismissAllAlerts(_ context.Context) error } // Severity indicates the severity of an alert. @@ -142,17 +141,6 @@ func (m *Manager) RegisterAlert(ctx context.Context, alert Alert) error { }) } -// DismissAllAlerts implements the Alerter interface. -func (m *Manager) DismissAllAlerts(ctx context.Context) error { - m.mu.Lock() - toDismiss := make([]types.Hash256, 0, len(m.alerts)) - for alertID := range m.alerts { - toDismiss = append(toDismiss, alertID) - } - m.mu.Unlock() - return m.DismissAlerts(ctx, toDismiss...) -} - // DismissAlerts implements the Alerter interface. func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { var dismissed []types.Hash256 @@ -252,11 +240,6 @@ func (a *originAlerter) RegisterAlert(ctx context.Context, alert Alert) error { return a.alerter.RegisterAlert(ctx, alert) } -// DismissAllAlerts implements the Alerter interface. -func (a *originAlerter) DismissAllAlerts(ctx context.Context) error { - return a.alerter.DismissAllAlerts(ctx) -} - // DismissAlerts implements the Alerter interface. func (a *originAlerter) DismissAlerts(ctx context.Context, ids ...types.Hash256) error { return a.alerter.DismissAlerts(ctx, ids...) diff --git a/bus/bus.go b/bus/bus.go index 6cce5256d..6c80f065a 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1738,13 +1738,6 @@ func (b *bus) handleGETAlertsPaginated(jc jape.Context) { } func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { - var all bool - if jc.DecodeForm("all", &all) != nil { - return - } else if all { - jc.Check("failed to dismiss all alerts", b.alertMgr.DismissAllAlerts(jc.Request.Context())) - return - } var ids []types.Hash256 if jc.Decode(&ids) != nil { return diff --git a/bus/client/alerts.go b/bus/client/alerts.go index 1b876e877..7f2bf9aa7 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -25,11 +25,6 @@ func (c *Client) DismissAlerts(ctx context.Context, ids ...types.Hash256) error return c.dismissAlerts(ctx, false, ids...) } -// DismissAllAlerts dimisses all registered alerts. -func (c *Client) DismissAllAlerts(ctx context.Context) error { - return c.dismissAlerts(ctx, true) -} - func (c *Client) dismissAlerts(ctx context.Context, all bool, ids ...types.Hash256) error { values := url.Values{} if all { diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index f0734a0f3..5d439f5b7 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1974,15 +1974,6 @@ func TestAlerts(t *testing.T) { if len(foundAlerts) != 1 || foundAlerts[0].ID != alert2.ID { t.Fatal("wrong alert") } - - // dismiss all - tt.OK(b.DismissAllAlerts(context.Background())) - ar, err = b.Alerts(alerts.AlertsOpts{}) - foundAlerts = ar.Alerts - tt.OK(err) - if len(foundAlerts) != 0 { - t.Fatal("expected 0 alerts", len(foundAlerts)) - } } func TestMultipartUploads(t *testing.T) { From 11980346ed168492d6f70ecf6812238eb3d9e6d1 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 20 Feb 2024 19:03:25 +0100 Subject: [PATCH 108/144] worker: don't register alert for cancelled download --- worker/download.go | 3 ++- worker/worker.go | 2 +- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/worker/download.go b/worker/download.go index f23c8e640..462a2292d 100644 --- a/worker/download.go +++ b/worker/download.go @@ -26,6 +26,7 @@ const ( var ( errDownloadNotEnoughHosts = errors.New("not enough hosts available to download the slab") + errDownloadCancelled = errors.New("download was cancelled") ) type ( @@ -290,7 +291,7 @@ outer: case <-mgr.shutdownCtx.Done(): return ErrShuttingDown case <-ctx.Done(): - return errors.New("download timed out") + return errDownloadCancelled case resp = <-responseChan: } diff --git a/worker/worker.go b/worker/worker.go index 094722c00..15b3ae509 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -922,7 +922,7 @@ func (w *worker) objectsHandlerGET(jc jape.Context) { err = w.downloadManager.DownloadObject(ctx, wr, res.Object.Object, uint64(offset), uint64(length), contracts) if err != nil { w.logger.Error(err) - if !errors.Is(err, ErrShuttingDown) { + if !errors.Is(err, ErrShuttingDown) && !errors.Is(err, errDownloadCancelled) { w.registerAlert(newDownloadFailedAlert(bucket, path, prefix, marker, offset, length, int64(len(contracts)), err)) } } From 69bde0662d0bed67e4a8afed271e1d6165bb599b Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 10:45:28 +0100 Subject: [PATCH 109/144] worker: don't register download error alert for errDownloadCancelled or io.ErrClosedPipe --- worker/worker.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/worker/worker.go b/worker/worker.go index 15b3ae509..e8355686b 100644 --- a/worker/worker.go +++ b/worker/worker.go @@ -922,7 +922,9 @@ func (w *worker) objectsHandlerGET(jc jape.Context) { err = w.downloadManager.DownloadObject(ctx, wr, res.Object.Object, uint64(offset), uint64(length), contracts) if err != nil { w.logger.Error(err) - if !errors.Is(err, ErrShuttingDown) && !errors.Is(err, errDownloadCancelled) { + if !errors.Is(err, ErrShuttingDown) && + !errors.Is(err, errDownloadCancelled) && + !errors.Is(err, io.ErrClosedPipe) { w.registerAlert(newDownloadFailedAlert(bucket, path, prefix, marker, offset, length, int64(len(contracts)), err)) } } From 709acfe4cccd0f9886d6497afd1646135b0bdb0b Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 10:54:25 +0100 Subject: [PATCH 110/144] bus: fix japecheck --- bus/bus.go | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/bus/bus.go b/bus/bus.go index 6c80f065a..70471fa05 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1715,16 +1715,16 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { }, nil } -func (b *bus) handleGETAlerts(jc jape.Context) { - if jc.Request.FormValue("offset") != "" || jc.Request.FormValue("limit") != "" { - b.handleGETAlertsPaginated(jc) - return - } +func (b *bus) handleGETAlertsDeprecated(jc jape.Context) { ar := b.alertMgr.Active(0, -1) jc.Encode(ar.Alerts) } -func (b *bus) handleGETAlertsPaginated(jc jape.Context) { +func (b *bus) handleGETAlerts(jc jape.Context) { + if jc.Request.FormValue("offset") == "" && jc.Request.FormValue("limit") == "" { + b.handleGETAlertsDeprecated(jc) + return + } offset, limit := 0, -1 if jc.DecodeForm("offset", &offset) != nil { return From aacaaa7b4f6aedadb37458c8e51a02243a50a83c Mon Sep 17 00:00:00 2001 From: PJ Date: Thu, 22 Feb 2024 10:35:46 +0100 Subject: [PATCH 111/144] stores: remove frand.Shuffle --- stores/metadata_test.go | 8 +------- 1 file changed, 1 insertion(+), 7 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index ad886e6ef..3c1ca4997 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4317,13 +4317,7 @@ func TestTypeCurrency(t *testing.T) { } // insert currencies in random order - values := []interface{}{ - bCurrency(types.ZeroCurrency), - bCurrency(types.NewCurrency64(1)), - bCurrency(types.MaxCurrency), - } - frand.Shuffle(len(values), func(i, j int) { values[i], values[j] = values[j], values[i] }) - if err := ss.db.Exec("INSERT INTO currencies (c) VALUES (?),(?),(?);", values...).Error; err != nil { + if err := ss.db.Exec("INSERT INTO currencies (c) VALUES (?),(?),(?);", bCurrency(types.MaxCurrency), bCurrency(types.NewCurrency64(1)), bCurrency(types.ZeroCurrency)).Error; err != nil { t.Fatal(err) } From 414090b2e615d77a93d6397640dda984216e837a Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Thu, 22 Feb 2024 10:51:36 +0100 Subject: [PATCH 112/144] stores: change bCurrency type to BigEndian (#991) --- stores/metadata_test.go | 16 +++++++++++----- stores/types.go | 8 ++++---- 2 files changed, 15 insertions(+), 9 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index f4ef64283..07036064e 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -4326,13 +4326,13 @@ func TestTypeCurrency(t *testing.T) { cmp: "=", }, { - a: types.NewCurrency(0, math.MaxUint64), - b: types.NewCurrency(math.MaxUint64, 0), + a: types.NewCurrency(math.MaxUint64, 0), + b: types.NewCurrency(0, math.MaxUint64), cmp: "<", }, { - a: types.NewCurrency(math.MaxUint64, 0), - b: types.NewCurrency(0, math.MaxUint64), + a: types.NewCurrency(0, math.MaxUint64), + b: types.NewCurrency(math.MaxUint64, 0), cmp: ">", }, } @@ -4342,7 +4342,13 @@ func TestTypeCurrency(t *testing.T) { if err != nil { t.Fatal(err) } else if !result { - t.Fatal("unexpected result", result) + t.Fatalf("unexpected result %v for %v %v %v", result, test.a, test.cmp, test.b) + } else if test.cmp == "<" && test.a.Cmp(test.b) >= 0 { + t.Fatal("invalid result") + } else if test.cmp == ">" && test.a.Cmp(test.b) <= 0 { + t.Fatal("invalid result") + } else if test.cmp == "=" && test.a.Cmp(test.b) != 0 { + t.Fatal("invalid result") } } diff --git a/stores/types.go b/stores/types.go index 9a7c72009..42a8d29e4 100644 --- a/stores/types.go +++ b/stores/types.go @@ -354,15 +354,15 @@ func (sc *bCurrency) Scan(src any) error { return fmt.Errorf("cannot scan %d bytes to Currency", len(buf)) } - sc.Lo = binary.LittleEndian.Uint64(buf[:8]) - sc.Hi = binary.LittleEndian.Uint64(buf[8:]) + sc.Hi = binary.BigEndian.Uint64(buf[:8]) + sc.Lo = binary.BigEndian.Uint64(buf[8:]) return nil } // Value implements the driver.Valuer interface. func (sc bCurrency) Value() (driver.Value, error) { buf := make([]byte, 16) - binary.LittleEndian.PutUint64(buf[:8], sc.Lo) - binary.LittleEndian.PutUint64(buf[8:], sc.Hi) + binary.BigEndian.PutUint64(buf[:8], sc.Hi) + binary.BigEndian.PutUint64(buf[8:], sc.Lo) return buf, nil } From 5b968e7e71ccce04f98a3d36f5c17997a2d17907 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 13:49:23 +0100 Subject: [PATCH 113/144] stores: numDeleted --- stores/metadata.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/stores/metadata.go b/stores/metadata.go index 2ab2cf5da..eaad76c17 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -2708,7 +2708,7 @@ func (s *SQLStore) deleteObject(tx *gorm.DB, bucket string, path string) (int64, if numDeleted == 0 { return 0, nil // nothing to prune if no object was deleted } else if err := pruneSlabs(tx); err != nil { - return 0, err + return numDeleted, err } return numDeleted, nil } From fb5bc3519bbd4f3321457aca056d8ad3cd90800a Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 16:12:46 +0100 Subject: [PATCH 114/144] worker: add benchmarks for upload through uploader --- worker/bench_test.go | 88 ++++++++++++++++++++++++++++++++++++++++++++ worker/upload.go | 39 +++++++++++++------- 2 files changed, 113 insertions(+), 14 deletions(-) create mode 100644 worker/bench_test.go diff --git a/worker/bench_test.go b/worker/bench_test.go new file mode 100644 index 000000000..d5db86c0a --- /dev/null +++ b/worker/bench_test.go @@ -0,0 +1,88 @@ +package worker + +import ( + "context" + "io" + "sync" + "testing" + + rhpv2 "go.sia.tech/core/rhp/v2" +) + +// zeroReader is a reader that leaves the buffer unchanged and returns no error. +// It's useful for benchmarks that need to produce data for uploading and should +// be used together with a io.LimitReader. +type zeroReader struct{} + +func (z *zeroReader) Read(p []byte) (n int, err error) { + return len(p), nil +} + +// BenchmarkUploaderPacking benchmarks the Upload function with packing +// disabled. +func BenchmarkUploaderNoPacking(b *testing.B) { + w := newMockWorker() + + minDataPieces := 10 + totalDataPieces := 30 + + w.addHosts(totalDataPieces) + + // create a reader that returns dev/null + data := io.LimitReader(&zeroReader{}, int64(b.N*rhpv2.SectorSize*minDataPieces)) + + up := testParameters(b.TempDir()) + up.rs.MinShards = minDataPieces + up.rs.TotalShards = totalDataPieces + up.packing = false + + b.ResetTimer() + + _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) + if err != nil { + b.Fatal(err) + } + b.SetBytes(int64(rhpv2.SectorSize * minDataPieces)) +} + +// BenchmarkSectorRoot30Goroutines benchmarks the SectorRoot function with 30 +// goroutines processing roots in parallel to simulate sequential uploads of +// slabs. +func BenchmarkSectorRoot30Goroutines(b *testing.B) { + data := make([]byte, rhpv2.SectorSize) + b.SetBytes(int64(rhpv2.SectorSize)) + + // spin up workers + c := make(chan struct{}) + work := func() { + for range c { + rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(data)) + } + } + var wg sync.WaitGroup + for i := 0; i < 30; i++ { + wg.Add(1) + go func() { + work() + wg.Done() + }() + } + b.ResetTimer() + + // run the benchmark + for i := 0; i < b.N; i++ { + c <- struct{}{} + } + close(c) + wg.Wait() +} + +// BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. +func BenchmarkSectorRootSingleGoroutine(b *testing.B) { + data := make([]byte, rhpv2.SectorSize) + b.SetBytes(rhpv2.SectorSize) + b.ResetTimer() + for i := 0; i < b.N; i++ { + rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(data)) + } +} diff --git a/worker/upload.go b/worker/upload.go index 72c65bf07..911593929 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -765,20 +765,26 @@ func (u *upload) newSlabUpload(ctx context.Context, shards [][]byte, uploaders [ responseChan := make(chan sectorUploadResp) // prepare sectors + var wg sync.WaitGroup sectors := make([]*sectorUpload, len(shards)) - for sI, shard := range shards { - // create the ctx - sCtx, sCancel := context.WithCancel(ctx) - - // create the sector - sectors[sI] = §orUpload{ - data: (*[rhpv2.SectorSize]byte)(shard), - index: sI, - root: rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(shard)), - ctx: sCtx, - cancel: sCancel, - } + for sI := range shards { + wg.Add(1) + go func(idx int) { + // create the ctx + sCtx, sCancel := context.WithCancel(ctx) + + // create the sector + sectors[idx] = §orUpload{ + data: (*[rhpv2.SectorSize]byte)(shards[idx]), + index: idx, + root: rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(shards[idx])), + ctx: sCtx, + cancel: sCancel, + } + wg.Done() + }(sI) } + wg.Wait() // prepare candidates candidates := make([]*candidate, len(uploaders)) @@ -833,8 +839,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() - // ensure inflight uploads get cancelled ctx, cancel := context.WithCancel(ctx) defer cancel() @@ -871,6 +875,10 @@ func (u *upload) uploadShards(ctx context.Context, shards [][]byte, candidates [ // create a request buffer var buffer []*sectorUploadReq + // start the timer after the upload has started + // newSlabUpload is quite slow due to computing the sector roots + start := time.Now() + // collect responses var used bool var done bool @@ -930,6 +938,9 @@ loop: // calculate the upload speed bytes := slab.numUploaded * rhpv2.SectorSize ms := time.Since(start).Milliseconds() + if ms == 0 { + ms = 1 + } uploadSpeed = int64(bytes) / ms // calculate overdrive pct From e1d45a46c643bd65dd87c130f51fc049b0df2401 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 16:24:15 +0100 Subject: [PATCH 115/144] worker: add benchmark results --- worker/bench_test.go | 61 ++++++++++++++++++++++++++++++++++---------- 1 file changed, 47 insertions(+), 14 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index d5db86c0a..d1f31d2a0 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -18,36 +18,66 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { return len(p), nil } -// BenchmarkUploaderPacking benchmarks the Upload function with packing -// disabled. -func BenchmarkUploaderNoPacking(b *testing.B) { +// BenchmarkUploaderSingleObjectNoPacking benchmarks uploading a single object +// without packing. +// +// Speed | CPU | Commit +// 201.59 MB/s | M2 Pro | c31245f +func BenchmarkUploaderSingleObjectNoPacking(b *testing.B) { w := newMockWorker() - minDataPieces := 10 - totalDataPieces := 30 + up := testParameters(b.TempDir()) + up.rs.MinShards = 10 + up.rs.TotalShards = 30 + up.packing = false - w.addHosts(totalDataPieces) + w.addHosts(up.rs.TotalShards) // create a reader that returns dev/null - data := io.LimitReader(&zeroReader{}, int64(b.N*rhpv2.SectorSize*minDataPieces)) - - up := testParameters(b.TempDir()) - up.rs.MinShards = minDataPieces - up.rs.TotalShards = totalDataPieces - up.packing = false + data := io.LimitReader(&zeroReader{}, int64(b.N*rhpv2.SectorSize*up.rs.MinShards)) + b.SetBytes(int64(rhpv2.SectorSize * up.rs.MinShards)) b.ResetTimer() - _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) if err != nil { b.Fatal(err) } - b.SetBytes(int64(rhpv2.SectorSize * minDataPieces)) +} + +// BenchmarkUploaderSingleObjectNoPacking benchmarks uploading one object per +// slab without packing. +// +// Speed | CPU | Commit +// 116.40 MB/s | M2 Pro | c31245f +func BenchmarkUploaderMultiObjectNoPacking(b *testing.B) { + w := newMockWorker() + + up := testParameters(b.TempDir()) + up.rs.MinShards = 10 + up.rs.TotalShards = 30 + up.packing = false + + w.addHosts(up.rs.TotalShards) + + // create a reader that returns dev/null + b.SetBytes(int64(rhpv2.SectorSize * up.rs.MinShards)) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + data := io.LimitReader(&zeroReader{}, int64(rhpv2.SectorSize*up.rs.MinShards)) + _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) + if err != nil { + b.Fatal(err) + } + } } // BenchmarkSectorRoot30Goroutines benchmarks the SectorRoot function with 30 // goroutines processing roots in parallel to simulate sequential uploads of // slabs. +// +// Speed | CPU | Commit +// 1671.26 MB/s | M2 Pro | c31245f func BenchmarkSectorRoot30Goroutines(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(int64(rhpv2.SectorSize)) @@ -78,6 +108,9 @@ func BenchmarkSectorRoot30Goroutines(b *testing.B) { } // BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. +// +// Speed | CPU | Commit +// 176.43 MB/s | M2 Pro | c31245f func BenchmarkSectorRootSingleGoroutine(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(rhpv2.SectorSize) From fd166d80f1c791f879a8ed0801e5e2ff4d60a52c Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 17:10:00 +0100 Subject: [PATCH 116/144] worker: docstring update --- worker/bench_test.go | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index d1f31d2a0..84a3aaa8a 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -18,12 +18,11 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { return len(p), nil } -// BenchmarkUploaderSingleObjectNoPacking benchmarks uploading a single object -// without packing. +// BenchmarkUploaderSingleObject benchmarks uploading a single object. // // Speed | CPU | Commit // 201.59 MB/s | M2 Pro | c31245f -func BenchmarkUploaderSingleObjectNoPacking(b *testing.B) { +func BenchmarkUploaderSingleObject(b *testing.B) { w := newMockWorker() up := testParameters(b.TempDir()) @@ -44,12 +43,11 @@ func BenchmarkUploaderSingleObjectNoPacking(b *testing.B) { } } -// BenchmarkUploaderSingleObjectNoPacking benchmarks uploading one object per -// slab without packing. +// BenchmarkUploaderSingleObject benchmarks uploading one object per slab. // // Speed | CPU | Commit // 116.40 MB/s | M2 Pro | c31245f -func BenchmarkUploaderMultiObjectNoPacking(b *testing.B) { +func BenchmarkUploaderMultiObject(b *testing.B) { w := newMockWorker() up := testParameters(b.TempDir()) From 9f27c17d597252de29e74d6c4d248f65c42e62a3 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 17:26:14 +0100 Subject: [PATCH 117/144] worker: remove hash reader --- worker/upload.go | 15 ++++++++++----- worker/upload_utils.go | 27 --------------------------- 2 files changed, 10 insertions(+), 32 deletions(-) diff --git a/worker/upload.go b/worker/upload.go index 911593929..56444cded 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -2,6 +2,8 @@ package worker import ( "context" + "crypto/md5" + "encoding/hex" "errors" "fmt" "io" @@ -400,11 +402,8 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a // create the object o := object.NewObject(up.ec) - // create the hash reader - hr := newHashReader(r) - // create the cipher reader - cr, err := o.Encrypt(hr, up.encryptionOffset) + cr, err := o.Encrypt(r, up.encryptionOffset) if err != nil { return false, "", err } @@ -533,7 +532,13 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a } // calculate the eTag - eTag = hr.Hash() + h := md5.New() + for _, slab := range o.Slabs { + for _, shard := range slab.Shards { + h.Write(shard.Root[:]) + } + } + eTag = string(hex.EncodeToString(h.Sum(nil))) // add partial slabs if len(partialSlab) > 0 { diff --git a/worker/upload_utils.go b/worker/upload_utils.go index 4b5241b4d..306e1774f 100644 --- a/worker/upload_utils.go +++ b/worker/upload_utils.go @@ -2,11 +2,9 @@ package worker import ( "bytes" - "encoding/hex" "io" "github.com/gabriel-vasile/mimetype" - "go.sia.tech/core/types" "go.sia.tech/renterd/object" ) @@ -28,28 +26,3 @@ func newMimeReader(r io.Reader) (mimeType string, recycled io.Reader, err error) recycled = io.MultiReader(buf, r) return mtype.String(), recycled, err } - -type hashReader struct { - r io.Reader - h *types.Hasher -} - -func newHashReader(r io.Reader) *hashReader { - return &hashReader{ - r: r, - h: types.NewHasher(), - } -} - -func (e *hashReader) Read(p []byte) (int, error) { - n, err := e.r.Read(p) - if _, wErr := e.h.E.Write(p[:n]); wErr != nil { - return 0, wErr - } - return n, err -} - -func (e *hashReader) Hash() string { - sum := e.h.Sum() - return hex.EncodeToString(sum[:]) -} From 24fb81a4c249be120d8a59a22f39919f8dbd48e4 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 17:28:32 +0100 Subject: [PATCH 118/144] worker: update benchmark result --- worker/bench_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index 84a3aaa8a..d9264eddc 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -21,7 +21,7 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { // BenchmarkUploaderSingleObject benchmarks uploading a single object. // // Speed | CPU | Commit -// 201.59 MB/s | M2 Pro | c31245f +// 217.35 MB/s | M2 Pro | afee1ac func BenchmarkUploaderSingleObject(b *testing.B) { w := newMockWorker() @@ -46,7 +46,7 @@ func BenchmarkUploaderSingleObject(b *testing.B) { // BenchmarkUploaderSingleObject benchmarks uploading one object per slab. // // Speed | CPU | Commit -// 116.40 MB/s | M2 Pro | c31245f +// 139.74 MB/s | M2 Pro | afee1ac func BenchmarkUploaderMultiObject(b *testing.B) { w := newMockWorker() @@ -75,7 +75,7 @@ func BenchmarkUploaderMultiObject(b *testing.B) { // slabs. // // Speed | CPU | Commit -// 1671.26 MB/s | M2 Pro | c31245f +// 1611.98 MB/s | M2 Pro | afee1ac func BenchmarkSectorRoot30Goroutines(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(int64(rhpv2.SectorSize)) @@ -108,7 +108,7 @@ func BenchmarkSectorRoot30Goroutines(b *testing.B) { // BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. // // Speed | CPU | Commit -// 176.43 MB/s | M2 Pro | c31245f +// 174.71 MB/s | M2 Pro | afee1ac func BenchmarkSectorRootSingleGoroutine(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(rhpv2.SectorSize) From 11d0152ba37b47c3c6c5867e8a1a43bc36366317 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 18:02:49 +0100 Subject: [PATCH 119/144] worker: encrypt using multiple goroutines --- object/slab.go | 15 +++++++++++---- 1 file changed, 11 insertions(+), 4 deletions(-) diff --git a/object/slab.go b/object/slab.go index 9c3afa608..aa8bb7d45 100644 --- a/object/slab.go +++ b/object/slab.go @@ -3,6 +3,7 @@ package object import ( "bytes" "io" + "sync" "github.com/klauspost/reedsolomon" rhpv2 "go.sia.tech/core/rhp/v2" @@ -79,11 +80,17 @@ func (s Slab) Length() int { // Encrypt xors shards with the keystream derived from s.Key, using a // different nonce for each shard. func (s Slab) Encrypt(shards [][]byte) { - for i, shard := range shards { - nonce := [24]byte{1: byte(i)} - c, _ := chacha20.NewUnauthenticatedCipher(s.Key.entropy[:], nonce[:]) - c.XORKeyStream(shard, shard) + var wg sync.WaitGroup + for i := range shards { + wg.Add(1) + go func(i int) { + nonce := [24]byte{1: byte(i)} + c, _ := chacha20.NewUnauthenticatedCipher(s.Key.entropy[:], nonce[:]) + c.XORKeyStream(shards[i], shards[i]) + wg.Done() + }(i) } + wg.Wait() } // Encode encodes slab data into sector-sized shards. The supplied shards should From c91892506bab43f0163e14fa4bf509d0324f25d6 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 18:05:31 +0100 Subject: [PATCH 120/144] worker: update benchmark results --- worker/bench_test.go | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index d9264eddc..f864df9ca 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -21,7 +21,7 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { // BenchmarkUploaderSingleObject benchmarks uploading a single object. // // Speed | CPU | Commit -// 217.35 MB/s | M2 Pro | afee1ac +// 232.97 MB/s | M2 Pro | 26d3119 func BenchmarkUploaderSingleObject(b *testing.B) { w := newMockWorker() @@ -46,7 +46,7 @@ func BenchmarkUploaderSingleObject(b *testing.B) { // BenchmarkUploaderSingleObject benchmarks uploading one object per slab. // // Speed | CPU | Commit -// 139.74 MB/s | M2 Pro | afee1ac +// 185.10 MB/s | M2 Pro | 26d3119 func BenchmarkUploaderMultiObject(b *testing.B) { w := newMockWorker() @@ -75,7 +75,7 @@ func BenchmarkUploaderMultiObject(b *testing.B) { // slabs. // // Speed | CPU | Commit -// 1611.98 MB/s | M2 Pro | afee1ac +// 1668.87 MB/s | M2 Pro | 26d3119 func BenchmarkSectorRoot30Goroutines(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(int64(rhpv2.SectorSize)) @@ -108,7 +108,7 @@ func BenchmarkSectorRoot30Goroutines(b *testing.B) { // BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. // // Speed | CPU | Commit -// 174.71 MB/s | M2 Pro | afee1ac +// 176.91 MB/s | M2 Pro | 26d3119 func BenchmarkSectorRootSingleGoroutine(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(rhpv2.SectorSize) From 5ceccfc411a614b57e7ef4faa995815f4be04f76 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 09:40:54 +0100 Subject: [PATCH 121/144] worker: ComputeEtag method --- object/object.go | 17 +++++++++++++++++ worker/upload.go | 12 ++---------- 2 files changed, 19 insertions(+), 10 deletions(-) diff --git a/object/object.go b/object/object.go index 7c74c1c23..2331f6251 100644 --- a/object/object.go +++ b/object/object.go @@ -3,6 +3,7 @@ package object import ( "bytes" "crypto/cipher" + "crypto/md5" "encoding/binary" "encoding/hex" "fmt" @@ -142,6 +143,22 @@ func (o Object) Contracts() map[types.PublicKey]map[types.FileContractID]struct{ return usedContracts } +func (o *Object) ComputeETag() string { + // calculate the eTag using the precomputed sector roots to avoid having to + // hash the entire object again. + h := md5.New() + b := make([]byte, 8) + for _, slab := range o.Slabs { + binary.LittleEndian.PutUint32(b[:4], slab.Offset) + binary.LittleEndian.PutUint32(b[4:], slab.Length) + h.Write(b) + for _, shard := range slab.Shards { + h.Write(shard.Root[:]) + } + } + return string(hex.EncodeToString(h.Sum(nil))) +} + // TotalSize returns the total size of the object. func (o Object) TotalSize() int64 { var n int64 diff --git a/worker/upload.go b/worker/upload.go index 56444cded..232e05981 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -2,8 +2,6 @@ package worker import ( "context" - "crypto/md5" - "encoding/hex" "errors" "fmt" "io" @@ -531,14 +529,8 @@ func (mgr *uploadManager) Upload(ctx context.Context, r io.Reader, contracts []a o.Slabs = append(o.Slabs, resp.slab) } - // calculate the eTag - h := md5.New() - for _, slab := range o.Slabs { - for _, shard := range slab.Shards { - h.Write(shard.Root[:]) - } - } - eTag = string(hex.EncodeToString(h.Sum(nil))) + // compute etag + eTag = o.ComputeETag() // add partial slabs if len(partialSlab) > 0 { From cfb8cb600273f92209e7df1706707bf3c2c741b8 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 13:28:04 +0100 Subject: [PATCH 122/144] worker: pass expected root to UploadSector --- worker/host.go | 18 +++++++++--------- worker/host_test.go | 4 +--- worker/mocks_test.go | 11 +++++------ worker/rhpv3.go | 21 ++++++++++----------- worker/upload.go | 25 ++++--------------------- worker/uploader.go | 32 +++++++++++++++++--------------- 6 files changed, 46 insertions(+), 65 deletions(-) diff --git a/worker/host.go b/worker/host.go index 43e0891af..e5642efdd 100644 --- a/worker/host.go +++ b/worker/host.go @@ -21,7 +21,7 @@ type ( PublicKey() types.PublicKey DownloadSector(ctx context.Context, w io.Writer, root types.Hash256, offset, length uint32, overpay bool) error - UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) + UploadSector(ctx context.Context, sectorRoot types.Hash256, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) error FetchPriceTable(ctx context.Context, rev *types.FileContractRevision) (hpt hostdb.HostPriceTable, err error) FetchRevision(ctx context.Context, fetchTimeout time.Duration) (types.FileContractRevision, error) @@ -121,11 +121,11 @@ func (h *host) DownloadSector(ctx context.Context, w io.Writer, root types.Hash2 }) } -func (h *host) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (root types.Hash256, err error) { +func (h *host) UploadSector(ctx context.Context, sectorRoot types.Hash256, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (err error) { // fetch price table pt, err := h.priceTable(ctx, nil) if err != nil { - return types.Hash256{}, err + return err } // prepare payment @@ -134,28 +134,28 @@ func (h *host) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, // insufficient balance error expectedCost, _, _, err := uploadSectorCost(pt, rev.WindowEnd) if err != nil { - return types.Hash256{}, err + return err } if rev.RevisionNumber == math.MaxUint64 { - return types.Hash256{}, fmt.Errorf("revision number has reached max, fcid %v", rev.ParentID) + return fmt.Errorf("revision number has reached max, fcid %v", rev.ParentID) } payment, ok := rhpv3.PayByContract(&rev, expectedCost, h.acc.id, h.renterKey) if !ok { - return types.Hash256{}, errors.New("failed to create payment") + return errors.New("failed to create payment") } var cost types.Currency err = h.transportPool.withTransportV3(ctx, h.hk, h.siamuxAddr, func(ctx context.Context, t *transportV3) error { - root, cost, err = RPCAppendSector(ctx, t, h.renterKey, pt, &rev, &payment, sector) + cost, err = RPCAppendSector(ctx, t, h.renterKey, pt, &rev, &payment, sectorRoot, sector) return err }) if err != nil { - return types.Hash256{}, err + return err } // record spending h.contractSpendingRecorder.Record(rev, api.ContractSpending{Uploads: cost}) - return root, nil + return nil } func (h *host) RenewContract(ctx context.Context, rrr api.RHPRenewRequest) (_ rhpv2.ContractRevision, _ []types.Transaction, _ types.Currency, err error) { diff --git a/worker/host_test.go b/worker/host_test.go index 87d35fb36..78ce6b74e 100644 --- a/worker/host_test.go +++ b/worker/host_test.go @@ -16,11 +16,9 @@ func TestHost(t *testing.T) { sector, root := newMockSector() // upload the sector - uploaded, err := h.UploadSector(context.Background(), sector, types.FileContractRevision{}) + err := h.UploadSector(context.Background(), rhpv2.SectorRoot(sector), sector, types.FileContractRevision{}) if err != nil { t.Fatal(err) - } else if uploaded != root { - t.Fatal("root mismatch") } // download entire sector diff --git a/worker/mocks_test.go b/worker/mocks_test.go index 2490941af..a28e9256c 100644 --- a/worker/mocks_test.go +++ b/worker/mocks_test.go @@ -396,8 +396,9 @@ func (h *mockHost) DownloadSector(ctx context.Context, w io.Writer, root types.H return err } -func (h *mockHost) UploadSector(ctx context.Context, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) (types.Hash256, error) { - return h.contract().addSector(sector), nil +func (h *mockHost) UploadSector(ctx context.Context, sectorRoot types.Hash256, sector *[rhpv2.SectorSize]byte, rev types.FileContractRevision) error { + h.contract().addSector(sectorRoot, sector) + return nil } func (h *mockHost) FetchRevision(ctx context.Context, fetchTimeout time.Duration) (rev types.FileContractRevision, _ error) { @@ -448,12 +449,10 @@ func newMockContract(hk types.PublicKey, fcid types.FileContractID) *mockContrac } } -func (c *mockContract) addSector(sector *[rhpv2.SectorSize]byte) (root types.Hash256) { - root = rhpv2.SectorRoot(sector) +func (c *mockContract) addSector(sectorRoot types.Hash256, sector *[rhpv2.SectorSize]byte) { c.mu.Lock() - c.sectors[root] = sector + c.sectors[sectorRoot] = sector c.mu.Unlock() - return } func (c *mockContract) sector(root types.Hash256) (sector *[rhpv2.SectorSize]byte, found bool) { diff --git a/worker/rhpv3.go b/worker/rhpv3.go index 03f67c6f6..b7ccfd69a 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -789,17 +789,17 @@ func RPCReadSector(ctx context.Context, t *transportV3, w io.Writer, pt rhpv3.Ho return } -func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.PrivateKey, pt rhpv3.HostPriceTable, rev *types.FileContractRevision, payment rhpv3.PaymentMethod, sector *[rhpv2.SectorSize]byte) (sectorRoot types.Hash256, cost types.Currency, err error) { +func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.PrivateKey, pt rhpv3.HostPriceTable, rev *types.FileContractRevision, payment rhpv3.PaymentMethod, sectorRoot types.Hash256, sector *[rhpv2.SectorSize]byte) (cost types.Currency, err error) { defer wrapErr(&err, "AppendSector") // sanity check revision first if rev.RevisionNumber == math.MaxUint64 { - return types.Hash256{}, types.ZeroCurrency, errMaxRevisionReached + return types.ZeroCurrency, errMaxRevisionReached } s, err := t.DialStream(ctx) if err != nil { - return types.Hash256{}, types.ZeroCurrency, err + return types.ZeroCurrency, err } defer s.Close() @@ -829,7 +829,7 @@ func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.Privat // compute expected collateral and refund expectedCost, expectedCollateral, expectedRefund, err := uploadSectorCost(pt, rev.WindowEnd) if err != nil { - return types.Hash256{}, types.ZeroCurrency, err + return types.ZeroCurrency, err } // apply leeways. @@ -840,13 +840,13 @@ func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.Privat // check if the cost, collateral and refund match our expectation. if executeResp.TotalCost.Cmp(expectedCost) > 0 { - return types.Hash256{}, types.ZeroCurrency, fmt.Errorf("cost exceeds expectation: %v > %v", executeResp.TotalCost.String(), expectedCost.String()) + return types.ZeroCurrency, fmt.Errorf("cost exceeds expectation: %v > %v", executeResp.TotalCost.String(), expectedCost.String()) } if executeResp.FailureRefund.Cmp(expectedRefund) < 0 { - return types.Hash256{}, types.ZeroCurrency, fmt.Errorf("insufficient refund: %v < %v", executeResp.FailureRefund.String(), expectedRefund.String()) + return types.ZeroCurrency, fmt.Errorf("insufficient refund: %v < %v", executeResp.FailureRefund.String(), expectedRefund.String()) } if executeResp.AdditionalCollateral.Cmp(expectedCollateral) < 0 { - return types.Hash256{}, types.ZeroCurrency, fmt.Errorf("insufficient collateral: %v < %v", executeResp.AdditionalCollateral.String(), expectedCollateral.String()) + return types.ZeroCurrency, fmt.Errorf("insufficient collateral: %v < %v", executeResp.AdditionalCollateral.String(), expectedCollateral.String()) } // set the cost and refund @@ -870,18 +870,17 @@ func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.Privat collateral := executeResp.AdditionalCollateral.Add(executeResp.FailureRefund) // check proof - sectorRoot = rhpv2.SectorRoot(sector) if rev.Filesize == 0 { // For the first upload to a contract we don't get a proof. So we just // assert that the new contract root matches the root of the sector. if rev.Filesize == 0 && executeResp.NewMerkleRoot != sectorRoot { - return types.Hash256{}, types.ZeroCurrency, fmt.Errorf("merkle root doesn't match the sector root upon first upload to contract: %v != %v", executeResp.NewMerkleRoot, sectorRoot) + return types.ZeroCurrency, fmt.Errorf("merkle root doesn't match the sector root upon first upload to contract: %v != %v", executeResp.NewMerkleRoot, sectorRoot) } } else { // Otherwise we make sure the proof was transmitted and verify it. actions := []rhpv2.RPCWriteAction{{Type: rhpv2.RPCWriteActionAppend}} // TODO: change once rhpv3 support is available if !rhpv2.VerifyDiffProof(actions, rev.Filesize/rhpv2.SectorSize, executeResp.Proof, []types.Hash256{}, rev.FileMerkleRoot, executeResp.NewMerkleRoot, []types.Hash256{sectorRoot}) { - return types.Hash256{}, types.ZeroCurrency, errors.New("proof verification failed") + return types.ZeroCurrency, errors.New("proof verification failed") } } @@ -889,7 +888,7 @@ func RPCAppendSector(ctx context.Context, t *transportV3, renterKey types.Privat newRevision := *rev newValid, newMissed, err := updateRevisionOutputs(&newRevision, types.ZeroCurrency, collateral) if err != nil { - return types.Hash256{}, types.ZeroCurrency, err + return types.ZeroCurrency, err } newRevision.Filesize += rhpv2.SectorSize newRevision.RevisionNumber++ diff --git a/worker/upload.go b/worker/upload.go index 232e05981..63be07b2b 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -137,9 +137,8 @@ type ( } sectorUploadResp struct { - req *sectorUploadReq - root types.Hash256 - err error + req *sectorUploadReq + err error } ) @@ -1065,12 +1064,6 @@ func (s *slabUpload) receive(resp sectorUploadResp) (bool, bool) { return false, false } - // sanity check we receive the expected root - if resp.root != req.sector.root { - s.errs[req.hk] = fmt.Errorf("root mismatch, %v != %v", resp.root, req.sector.root) - return false, false - } - // redundant sectors can't complete the upload if sector.uploaded.Root != (types.Hash256{}) { return false, false @@ -1080,7 +1073,7 @@ func (s *slabUpload) receive(resp sectorUploadResp) (bool, bool) { sector.finish(object.Sector{ Contracts: map[types.PublicKey][]types.FileContractID{req.hk: {req.fcid}}, LatestHost: req.hk, - Root: resp.root, + Root: req.sector.root, }) // update uploaded sectors @@ -1127,7 +1120,7 @@ func (req *sectorUploadReq) done() bool { } } -func (req *sectorUploadReq) fail(err error) { +func (req *sectorUploadReq) finish(err error) { select { case <-req.sector.ctx.Done(): case req.responseChan <- sectorUploadResp{ @@ -1136,13 +1129,3 @@ func (req *sectorUploadReq) fail(err error) { }: } } - -func (req *sectorUploadReq) succeed(root types.Hash256) { - select { - case <-req.sector.ctx.Done(): - case req.responseChan <- sectorUploadResp{ - req: req, - root: root, - }: - } -} diff --git a/worker/uploader.go b/worker/uploader.go index dcff27eaf..fa1d04651 100644 --- a/worker/uploader.go +++ b/worker/uploader.go @@ -114,7 +114,7 @@ outer: } // execute it - root, elapsed, err := u.execute(req) + elapsed, err := u.execute(req) // the uploader's contract got renewed, requeue the request if errors.Is(err, errMaxRevisionReached) { @@ -125,10 +125,12 @@ outer: } // send the response - if err != nil { - req.fail(err) - } else { - req.succeed(root) + select { + case <-req.sector.ctx.Done(): + case req.responseChan <- sectorUploadResp{ + req: req, + err: err, + }: } // track the error, ignore gracefully closed streams and canceled overdrives @@ -151,7 +153,7 @@ func (u *uploader) Stop(err error) { break } if !upload.done() { - upload.fail(err) + upload.finish(err) } } } @@ -161,7 +163,7 @@ func (u *uploader) enqueue(req *sectorUploadReq) { // check for stopped if u.stopped { u.mu.Unlock() - go req.fail(errUploaderStopped) // don't block the caller + go req.finish(errUploaderStopped) // don't block the caller return } @@ -192,7 +194,7 @@ func (u *uploader) estimate() float64 { return numSectors * estimateP90 } -func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, error) { +func (u *uploader) execute(req *sectorUploadReq) (time.Duration, error) { // grab fields u.mu.Lock() host := u.host @@ -202,7 +204,7 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, // acquire contract lock lockID, err := u.cs.AcquireContract(req.sector.ctx, fcid, req.contractLockPriority, req.contractLockDuration) if err != nil { - return types.Hash256{}, 0, err + return 0, err } // defer the release @@ -220,26 +222,26 @@ func (u *uploader) execute(req *sectorUploadReq) (types.Hash256, time.Duration, // fetch the revision rev, err := host.FetchRevision(ctx, defaultRevisionFetchTimeout) if err != nil { - return types.Hash256{}, 0, err + return 0, err } else if rev.RevisionNumber == math.MaxUint64 { - return types.Hash256{}, 0, errMaxRevisionReached + return 0, errMaxRevisionReached } // update the bus if err := u.os.AddUploadingSector(ctx, req.uploadID, fcid, req.sector.root); err != nil { - return types.Hash256{}, 0, fmt.Errorf("failed to add uploading sector to contract %v, err: %v", fcid, err) + return 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(ctx, req.sector.sectorData(), rev) + err = host.UploadSector(ctx, req.sector.root, req.sector.sectorData(), rev) if err != nil { - return types.Hash256{}, 0, fmt.Errorf("failed to upload sector to contract %v, err: %v", fcid, err) + return 0, fmt.Errorf("failed to upload sector to contract %v, err: %v", fcid, err) } // calculate elapsed time elapsed := time.Since(start) - return root, elapsed, nil + return elapsed, nil } func (u *uploader) pop() *sectorUploadReq { From 67a6c871389bc422bd72df7a15bc93d10ac0f627 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 13:38:45 +0100 Subject: [PATCH 123/144] worker: update benchmark results --- worker/bench_test.go | 8 ++++---- worker/upload.go | 6 ++++++ 2 files changed, 10 insertions(+), 4 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index f864df9ca..4748f3d85 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -21,7 +21,7 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { // BenchmarkUploaderSingleObject benchmarks uploading a single object. // // Speed | CPU | Commit -// 232.97 MB/s | M2 Pro | 26d3119 +// 433.86 MB/s | M2 Pro | bae6e77 func BenchmarkUploaderSingleObject(b *testing.B) { w := newMockWorker() @@ -46,7 +46,7 @@ func BenchmarkUploaderSingleObject(b *testing.B) { // BenchmarkUploaderSingleObject benchmarks uploading one object per slab. // // Speed | CPU | Commit -// 185.10 MB/s | M2 Pro | 26d3119 +// 282.47 MB/s | M2 Pro | bae6e77 func BenchmarkUploaderMultiObject(b *testing.B) { w := newMockWorker() @@ -75,7 +75,7 @@ func BenchmarkUploaderMultiObject(b *testing.B) { // slabs. // // Speed | CPU | Commit -// 1668.87 MB/s | M2 Pro | 26d3119 +// 1658.49 MB/s | M2 Pro | bae6e77 func BenchmarkSectorRoot30Goroutines(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(int64(rhpv2.SectorSize)) @@ -108,7 +108,7 @@ func BenchmarkSectorRoot30Goroutines(b *testing.B) { // BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. // // Speed | CPU | Commit -// 176.91 MB/s | M2 Pro | 26d3119 +// 177.33 MB/s | M2 Pro | bae6e77 func BenchmarkSectorRootSingleGoroutine(b *testing.B) { data := make([]byte, rhpv2.SectorSize) b.SetBytes(rhpv2.SectorSize) diff --git a/worker/upload.go b/worker/upload.go index 63be07b2b..8bd87f881 100644 --- a/worker/upload.go +++ b/worker/upload.go @@ -770,6 +770,12 @@ func (u *upload) newSlabUpload(ctx context.Context, shards [][]byte, uploaders [ sCtx, sCancel := context.WithCancel(ctx) // create the sector + // NOTE: we are computing the sector root here and pass it all the + // way down to the RPC to avoid having to recompute it for the proof + // verification. This is necessary because we need it ahead of time + // for the call to AddUploadingSector in uploader.go + // Once we upload to temp storage we don't need AddUploadingSector + // anymore and can move it back to the RPC. sectors[idx] = §orUpload{ data: (*[rhpv2.SectorSize]byte)(shards[idx]), index: idx, From 3f11ef698af02f1af58236c297774871401e5546 Mon Sep 17 00:00:00 2001 From: PJ Date: Thu, 22 Feb 2024 14:09:12 +0100 Subject: [PATCH 124/144] worker: implement PR remarks --- worker/rhpv3.go | 50 ++++++++++++++++--------------------------------- 1 file changed, 16 insertions(+), 34 deletions(-) diff --git a/worker/rhpv3.go b/worker/rhpv3.go index 5fbcd3ad6..ee2dfcd85 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -385,19 +385,14 @@ func withAccountLock(ctx context.Context, as AccountStore, id rhpv3.Account, hk if err != nil { return err } + err = fn(acc) - defer func() { - select { - case <-ctx.Done(): - var cancel context.CancelFunc - ctx, cancel = context.WithTimeout(context.Background(), time.Minute) - defer cancel() - default: - } - as.UnlockAccount(ctx, acc.ID, lockID) - }() + // unlock account + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + _ = as.UnlockAccount(ctx, acc.ID, lockID) // ignore error + cancel() - return fn(acc) + return nil } // Balance returns the account balance. @@ -450,38 +445,25 @@ func (a *account) WithWithdrawal(ctx context.Context, amtFn func() (types.Curren // execute amtFn amt, err := amtFn() + + // in case of an insufficient balance, we schedule a sync if isBalanceInsufficient(err) { - // in case of an insufficient balance, we schedule a sync - if scheduleErr := a.scheduleSync(); scheduleErr != nil { - err = fmt.Errorf("%w; failed to set requiresSync flag on bus, error: %v", err, scheduleErr) - } + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + err = errors.Join(err, a.as.ScheduleSync(ctx, a.id, a.host)) + cancel() } - // if an amount was returned, we withdraw it. - if withdrawErr := a.withdrawFromBalance(amt); withdrawErr != nil { - err = fmt.Errorf("%w; failed to withdraw from account, error: %v", err, withdrawErr) + // if an amount was returned, we withdraw it + if !amt.IsZero() { + ctx, cancel := context.WithTimeout(context.Background(), time.Minute) + err = errors.Join(err, a.as.AddBalance(ctx, a.id, a.host, new(big.Int).Neg(amt.Big()))) + cancel() } return err }) } -func (a *account) withdrawFromBalance(amt types.Currency) error { - if amt.IsZero() { - return nil - } - - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() - return a.as.AddBalance(ctx, a.id, a.host, new(big.Int).Neg(amt.Big())) -} - -func (a *account) scheduleSync() error { - ctx, cancel := context.WithTimeout(context.Background(), time.Minute) - defer cancel() - return a.as.ScheduleSync(ctx, a.id, a.host) -} - // deriveAccountKey derives an account plus key for a given host and worker. // Each worker has its own account for a given host. That makes concurrency // around keeping track of an accounts balance and refilling it a lot easier in From bcdeb0de465e8ac3936b314bca6b6ea641f76ce3 Mon Sep 17 00:00:00 2001 From: PJ Date: Thu, 22 Feb 2024 14:41:01 +0100 Subject: [PATCH 125/144] worker: return err --- worker/rhpv3.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/rhpv3.go b/worker/rhpv3.go index ee2dfcd85..25c26e42d 100644 --- a/worker/rhpv3.go +++ b/worker/rhpv3.go @@ -392,7 +392,7 @@ func withAccountLock(ctx context.Context, as AccountStore, id rhpv3.Account, hk _ = as.UnlockAccount(ctx, acc.ID, lockID) // ignore error cancel() - return nil + return err } // Balance returns the account balance. From 9af03a3d007545954baa34b0cbd350ed51ea2081 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 15:53:05 +0100 Subject: [PATCH 126/144] worker: add BenchmarkDownloaderSingleObjecdt --- worker/bench_test.go | 43 +++++++++++++++++++++++++++++++++++++------ 1 file changed, 37 insertions(+), 6 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index 4748f3d85..f896ce993 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -1,12 +1,15 @@ package worker import ( + "bytes" "context" "io" "sync" "testing" rhpv2 "go.sia.tech/core/rhp/v2" + "go.sia.tech/renterd/api" + "lukechampine.com/frand" ) // zeroReader is a reader that leaves the buffer unchanged and returns no error. @@ -18,6 +21,38 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { return len(p), nil } +// BenchmarkDownlaoderSingleObject benchmarks downloading a single, slab-sized +// object. +// 485.48 MB/s | M2 Pro | bae6e77 +func BenchmarkDownloaderSingleObject(b *testing.B) { + w := newMockWorker() + + up := testParameters(b.TempDir()) + up.rs.MinShards = 10 + up.rs.TotalShards = 30 + up.packing = false + w.addHosts(up.rs.TotalShards) + + data := bytes.NewReader(frand.Bytes(int(up.rs.SlabSizeNoRedundancy()))) + _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) + if err != nil { + b.Fatal(err) + } + o, err := w.os.Object(context.Background(), testBucket, up.path, api.GetObjectOptions{}) + if err != nil { + b.Fatal(err) + } + + b.SetBytes(o.Object.Size) + b.ResetTimer() + for i := 0; i < b.N; i++ { + err = w.dl.DownloadObject(context.Background(), io.Discard, *o.Object.Object, 0, uint64(o.Object.Size), w.contracts()) + if err != nil { + b.Fatal(err) + } + } +} + // BenchmarkUploaderSingleObject benchmarks uploading a single object. // // Speed | CPU | Commit @@ -29,14 +64,12 @@ func BenchmarkUploaderSingleObject(b *testing.B) { up.rs.MinShards = 10 up.rs.TotalShards = 30 up.packing = false - w.addHosts(up.rs.TotalShards) - // create a reader that returns dev/null data := io.LimitReader(&zeroReader{}, int64(b.N*rhpv2.SectorSize*up.rs.MinShards)) b.SetBytes(int64(rhpv2.SectorSize * up.rs.MinShards)) - b.ResetTimer() + _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) if err != nil { b.Fatal(err) @@ -54,13 +87,11 @@ func BenchmarkUploaderMultiObject(b *testing.B) { up.rs.MinShards = 10 up.rs.TotalShards = 30 up.packing = false - w.addHosts(up.rs.TotalShards) - // create a reader that returns dev/null b.SetBytes(int64(rhpv2.SectorSize * up.rs.MinShards)) - b.ResetTimer() + for i := 0; i < b.N; i++ { data := io.LimitReader(&zeroReader{}, int64(rhpv2.SectorSize*up.rs.MinShards)) _, _, err := w.ul.Upload(context.Background(), data, w.contracts(), up, lockingPriorityUpload) From c9dc1b6cf2e09e7678db1de4eb2b2b7db1ad01f5 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 16:33:09 +0100 Subject: [PATCH 127/144] worker: decrypt in parallel --- object/slab.go | 16 +++++++++++----- worker/download.go | 13 +++++++------ 2 files changed, 18 insertions(+), 11 deletions(-) diff --git a/object/slab.go b/object/slab.go index aa8bb7d45..f2762abf3 100644 --- a/object/slab.go +++ b/object/slab.go @@ -158,12 +158,18 @@ func (ss SlabSlice) SectorRegion() (offset, length uint32) { // slice offset), using a different nonce for each shard. func (ss SlabSlice) Decrypt(shards [][]byte) { offset := ss.Offset / (rhpv2.LeafSize * uint32(ss.MinShards)) - for i, shard := range shards { - nonce := [24]byte{1: byte(i)} - c, _ := chacha20.NewUnauthenticatedCipher(ss.Key.entropy[:], nonce[:]) - c.SetCounter(offset) - c.XORKeyStream(shard, shard) + var wg sync.WaitGroup + for i := range shards { + wg.Add(1) + go func(i int) { + nonce := [24]byte{1: byte(i)} + c, _ := chacha20.NewUnauthenticatedCipher(ss.Key.entropy[:], nonce[:]) + c.SetCounter(offset) + c.XORKeyStream(shards[i], shards[i]) + wg.Done() + }(i) } + wg.Wait() } // Recover recovers a slice of slab data from the supplied shards. diff --git a/worker/download.go b/worker/download.go index 462a2292d..a1cc8f501 100644 --- a/worker/download.go +++ b/worker/download.go @@ -195,12 +195,13 @@ func (mgr *downloadManager) DownloadObject(ctx context.Context, w io.Writer, o o hosts[c.HostKey] = struct{}{} } - // buffer the writer - bw := bufio.NewWriter(w) - defer bw.Flush() - // create the cipher writer - cw := o.Key.Decrypt(bw, offset) + cw := o.Key.Decrypt(w, offset) + + // buffer the writer we recover to making sure that we don't hammer the + // response writer with tiny writes + bw := bufio.NewWriter(cw) + defer bw.Flush() // create response chan and ensure it's closed properly var wg sync.WaitGroup @@ -322,7 +323,7 @@ outer: } else { // Regular slab. slabs[respIndex].Decrypt(next.shards) - err := slabs[respIndex].Recover(cw, next.shards) + err := slabs[respIndex].Recover(bw, next.shards) if err != nil { mgr.logger.Errorf("failed to recover slab %v: %v", respIndex, err) return err From aa08202054f0b32a0a622cc988094c0cac1a720a Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 16:34:28 +0100 Subject: [PATCH 128/144] worker: update download benchmark results --- worker/bench_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index f896ce993..575e4640f 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -23,7 +23,7 @@ func (z *zeroReader) Read(p []byte) (n int, err error) { // BenchmarkDownlaoderSingleObject benchmarks downloading a single, slab-sized // object. -// 485.48 MB/s | M2 Pro | bae6e77 +// 1036.74 MB/s | M2 Pro | c9dc1b6 func BenchmarkDownloaderSingleObject(b *testing.B) { w := newMockWorker() From 29f4201d2d48802329474607fb3427e8adcebd07 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 16:47:37 +0100 Subject: [PATCH 129/144] worker: remove irrelevant benchmarks --- worker/bench_test.go | 49 -------------------------------------------- 1 file changed, 49 deletions(-) diff --git a/worker/bench_test.go b/worker/bench_test.go index 575e4640f..552eca17c 100644 --- a/worker/bench_test.go +++ b/worker/bench_test.go @@ -4,7 +4,6 @@ import ( "bytes" "context" "io" - "sync" "testing" rhpv2 "go.sia.tech/core/rhp/v2" @@ -100,51 +99,3 @@ func BenchmarkUploaderMultiObject(b *testing.B) { } } } - -// BenchmarkSectorRoot30Goroutines benchmarks the SectorRoot function with 30 -// goroutines processing roots in parallel to simulate sequential uploads of -// slabs. -// -// Speed | CPU | Commit -// 1658.49 MB/s | M2 Pro | bae6e77 -func BenchmarkSectorRoot30Goroutines(b *testing.B) { - data := make([]byte, rhpv2.SectorSize) - b.SetBytes(int64(rhpv2.SectorSize)) - - // spin up workers - c := make(chan struct{}) - work := func() { - for range c { - rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(data)) - } - } - var wg sync.WaitGroup - for i := 0; i < 30; i++ { - wg.Add(1) - go func() { - work() - wg.Done() - }() - } - b.ResetTimer() - - // run the benchmark - for i := 0; i < b.N; i++ { - c <- struct{}{} - } - close(c) - wg.Wait() -} - -// BenchmarkSectorRootSingleGoroutine benchmarks the SectorRoot function. -// -// Speed | CPU | Commit -// 177.33 MB/s | M2 Pro | bae6e77 -func BenchmarkSectorRootSingleGoroutine(b *testing.B) { - data := make([]byte, rhpv2.SectorSize) - b.SetBytes(rhpv2.SectorSize) - b.ResetTimer() - for i := 0; i < b.N; i++ { - rhpv2.SectorRoot((*[rhpv2.SectorSize]byte)(data)) - } -} From 10e88fd4a00196b467a5b3d3d0ac26564ec861c6 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Thu, 22 Feb 2024 18:15:46 +0100 Subject: [PATCH 130/144] stores: fix mysql migration --- .../migration_00004_prune_slabs_cascade.sql | 22 ++++++------------- .../migration_00004_prune_slabs_cascade.sql | 8 +++---- 2 files changed, 11 insertions(+), 19 deletions(-) diff --git a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql index 0b1c06994..9014582e0 100644 --- a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql @@ -1,24 +1,16 @@ --- prune manually before creating trigger -DELETE slabs -FROM slabs -LEFT JOIN slices ON slices.db_slab_id = slabs.id -WHERE slices.db_object_id IS NULL -AND slices.db_multipart_part_id IS NULL -AND slabs.db_buffered_slab_id IS NULL; - -- add ON DELETE CASCADE to slices ALTER TABLE slices DROP FOREIGN KEY fk_objects_slabs; -ALTER TABLE slices ADD CONSTRAINT fk_objects_slabs FOREIGN KEY (db_object_id) REFERENCES objects (id) ON DELETE CASCADE, +ALTER TABLE slices ADD CONSTRAINT fk_objects_slabs FOREIGN KEY (db_object_id) REFERENCES objects (id) ON DELETE CASCADE; ALTER TABLE slices DROP FOREIGN KEY fk_multipart_parts_slabs; -ALTER TABLE slices ADD CONSTRAINT fk_multipart_parts_slabs FOREIGN KEY (db_multipart_part_id) REFERENCES multipart_parts (id) ON DELETE CASCADE, +ALTER TABLE slices ADD CONSTRAINT fk_multipart_parts_slabs FOREIGN KEY (db_multipart_part_id) REFERENCES multipart_parts (id) ON DELETE CASCADE; -- add ON DELETE CASCADE to multipart_parts ALTER TABLE multipart_parts DROP FOREIGN KEY fk_multipart_uploads_parts; -ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE +ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE; -- drop triggers -DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices -DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts -DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; diff --git a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql index 1132dd2f5..38cd40199 100644 --- a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql @@ -22,7 +22,7 @@ CREATE INDEX `idx_multipart_parts_etag` ON `multipart_parts`(`etag`); PRAGMA foreign_keys=on; -- drop triggers -DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices -DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts -DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; From 3e92f08d4fabab1992970797a9200afe3141b568 Mon Sep 17 00:00:00 2001 From: ChrisSchinnerl Date: Fri, 23 Feb 2024 00:08:30 +0000 Subject: [PATCH 131/144] ui: v0.46.0 --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index e727483d1..0ce29b965 100644 --- a/go.mod +++ b/go.mod @@ -18,7 +18,7 @@ require ( go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca - go.sia.tech/web/renterd v0.45.0 + go.sia.tech/web/renterd v0.46.0 go.uber.org/zap v1.26.0 golang.org/x/crypto v0.19.0 golang.org/x/term v0.17.0 diff --git a/go.sum b/go.sum index 006a31ea6..2570e9fc5 100644 --- a/go.sum +++ b/go.sum @@ -255,8 +255,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.45.0 h1:5kSiDnHYRacg3JideH9Cl9qHzcZiKnBR0fWRap169hU= -go.sia.tech/web/renterd v0.45.0/go.mod h1:FgXrdmAnu591a3h96RB/15pMZ74xO9457g902uE06BM= +go.sia.tech/web/renterd v0.46.0 h1:BMVg4i7LxSlc8wZ4T0EG1k3EK4JxVIzCfD3/cjmwH0k= +go.sia.tech/web/renterd v0.46.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/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= From 99014c79fe65c3bffa5c737a768d3905351c7517 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 23 Feb 2024 09:47:48 +0100 Subject: [PATCH 132/144] stores: drop indices first --- .../main/migration_00004_prune_slabs_cascade.sql | 14 +++++++------- .../main/migration_00004_prune_slabs_cascade.sql | 14 ++++++++------ 2 files changed, 15 insertions(+), 13 deletions(-) diff --git a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql index 9014582e0..c2efe3467 100644 --- a/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/mysql/main/migration_00004_prune_slabs_cascade.sql @@ -1,3 +1,9 @@ +-- drop triggers +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; + -- add ON DELETE CASCADE to slices ALTER TABLE slices DROP FOREIGN KEY fk_objects_slabs; ALTER TABLE slices ADD CONSTRAINT fk_objects_slabs FOREIGN KEY (db_object_id) REFERENCES objects (id) ON DELETE CASCADE; @@ -7,10 +13,4 @@ ALTER TABLE slices ADD CONSTRAINT fk_multipart_parts_slabs FOREIGN KEY (db_multi -- add ON DELETE CASCADE to multipart_parts ALTER TABLE multipart_parts DROP FOREIGN KEY fk_multipart_uploads_parts; -ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE; - --- drop triggers -DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; -DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; -DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; +ALTER TABLE multipart_parts ADD CONSTRAINT fk_multipart_uploads_parts FOREIGN KEY (db_multipart_upload_id) REFERENCES multipart_uploads (id) ON DELETE CASCADE; \ No newline at end of file diff --git a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql index 38cd40199..03f006acd 100644 --- a/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql +++ b/stores/migrations/sqlite/main/migration_00004_prune_slabs_cascade.sql @@ -1,5 +1,12 @@ +-- drop triggers +DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; +DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; +DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; +DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; + PRAGMA foreign_keys=off; -- update constraints on slices +DROP TABLE IF EXISTS slices_temp; CREATE TABLE `slices_temp` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`db_object_id` integer,`object_index` integer,`db_multipart_part_id` integer,`db_slab_id` integer,`offset` integer,`length` integer,CONSTRAINT `fk_objects_slabs` FOREIGN KEY (`db_object_id`) REFERENCES `objects`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_multipart_parts_slabs` FOREIGN KEY (`db_multipart_part_id`) REFERENCES `multipart_parts`(`id`) ON DELETE CASCADE,CONSTRAINT `fk_slabs_slices` FOREIGN KEY (`db_slab_id`) REFERENCES `slabs`(`id`)); INSERT INTO slices_temp SELECT `id`, `created_at`, `db_object_id`, `object_index`, `db_multipart_part_id`, `db_slab_id`, `offset`, `length` FROM slices; DROP TABLE slices; @@ -11,6 +18,7 @@ CREATE INDEX `idx_slices_db_slab_id` ON `slices`(`db_slab_id`); CREATE INDEX `idx_slices_db_multipart_part_id` ON `slices`(`db_multipart_part_id`); -- update constraints multipart_parts +DROP TABLE IF EXISTS multipart_parts_temp; CREATE TABLE `multipart_parts_temp` (`id` integer PRIMARY KEY AUTOINCREMENT,`created_at` datetime,`etag` text,`part_number` integer,`size` integer,`db_multipart_upload_id` integer NOT NULL,CONSTRAINT `fk_multipart_uploads_parts` FOREIGN KEY (`db_multipart_upload_id`) REFERENCES `multipart_uploads`(`id`) ON DELETE CASCADE); INSERT INTO multipart_parts_temp SELECT * FROM multipart_parts; DROP TABLE multipart_parts; @@ -20,9 +28,3 @@ CREATE INDEX `idx_multipart_parts_db_multipart_upload_id` ON `multipart_parts`(` CREATE INDEX `idx_multipart_parts_part_number` ON `multipart_parts`(`part_number`); CREATE INDEX `idx_multipart_parts_etag` ON `multipart_parts`(`etag`); PRAGMA foreign_keys=on; - --- drop triggers -DROP TRIGGER IF EXISTS before_delete_on_objects_delete_slices; -DROP TRIGGER IF EXISTS before_delete_on_multipart_uploads_delete_multipart_parts; -DROP TRIGGER IF EXISTS before_delete_on_multipart_parts_delete_slices; -DROP TRIGGER IF EXISTS after_delete_on_slices_delete_slabs; From 89b4729f083145606dedec152c28ebac409c6b3d Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 23 Feb 2024 10:15:34 +0100 Subject: [PATCH 133/144] testing: fix TestUploadPacking --- worker/download.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/worker/download.go b/worker/download.go index a1cc8f501..9048c033b 100644 --- a/worker/download.go +++ b/worker/download.go @@ -315,7 +315,7 @@ outer: s := slabs[respIndex] if s.PartialSlab { // Partial slab. - _, err = cw.Write(s.Data) + _, err = bw.Write(s.Data) if err != nil { mgr.logger.Errorf("failed to send partial slab", respIndex, err) return err From b45b80f70b20c16b8216c99b1d7dd138f055c224 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Wed, 21 Feb 2024 13:31:44 +0100 Subject: [PATCH 134/144] autopilot: accumulate churn information into single alert --- alerts/alerts.go | 15 ++++++-- autopilot/alerts.go | 58 +++++++++++------------------ autopilot/churn.go | 63 ++++++++++++++++++++++++++++++++ autopilot/contractor.go | 10 ++++- bus/bus.go | 11 +++++- bus/client/alerts.go | 4 +- internal/testing/cluster_test.go | 6 +-- 7 files changed, 117 insertions(+), 50 deletions(-) create mode 100644 autopilot/churn.go diff --git a/alerts/alerts.go b/alerts/alerts.go index b0d4963c6..f11004dbe 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -35,6 +35,7 @@ const ( type ( Alerter interface { + Alerts(_ context.Context, opts AlertsOpts) (resp AlertsResponse, err error) RegisterAlert(_ context.Context, a Alert) error DismissAlerts(_ context.Context, ids ...types.Hash256) error } @@ -169,17 +170,18 @@ func (m *Manager) DismissAlerts(ctx context.Context, ids ...types.Hash256) error }) } -// Active returns the host's active alerts. -func (m *Manager) Active(offset, limit int) AlertsResponse { +// Alerts returns the host's active alerts. +func (m *Manager) Alerts(_ context.Context, opts AlertsOpts) (AlertsResponse, error) { m.mu.Lock() defer m.mu.Unlock() + offset, limit := opts.Offset, opts.Limit resp := AlertsResponse{ Total: len(m.alerts), } if offset >= len(m.alerts) { - return resp + return resp, nil } else if limit == -1 { limit = len(m.alerts) } @@ -197,7 +199,7 @@ func (m *Manager) Active(offset, limit int) AlertsResponse { resp.HasMore = true } resp.Alerts = alerts - return resp + return resp, nil } func (m *Manager) RegisterWebhookBroadcaster(b webhooks.Broadcaster) { @@ -231,6 +233,11 @@ func WithOrigin(alerter Alerter, origin string) Alerter { } } +// Alerts implements the Alerter interface. +func (a *originAlerter) Alerts(ctx context.Context, opts AlertsOpts) (resp AlertsResponse, err error) { + return a.alerter.Alerts(ctx, opts) +} + // RegisterAlert implements the Alerter interface. func (a *originAlerter) RegisterAlert(ctx context.Context, alert Alert) error { if alert.Data == nil { diff --git a/autopilot/alerts.go b/autopilot/alerts.go index 292670dc5..f4762c4d4 100644 --- a/autopilot/alerts.go +++ b/autopilot/alerts.go @@ -14,12 +14,13 @@ import ( ) var ( - alertAccountRefillID = frand.Entropy256() // constant until restarted - alertLostSectorsID = frand.Entropy256() // constant until restarted - alertLowBalanceID = frand.Entropy256() // constant until restarted - alertMigrationID = frand.Entropy256() // constant until restarted - alertPruningID = frand.Entropy256() // constant until restarted - alertRenewalFailedID = frand.Entropy256() // constant until restarted + alertAccountRefillID = randomAlertID() // constant until restarted + alertChurnID = randomAlertID() // constant until restarted + alertLostSectorsID = randomAlertID() // constant until restarted + alertLowBalanceID = randomAlertID() // constant until restarted + alertMigrationID = randomAlertID() // constant until restarted + alertPruningID = randomAlertID() // constant until restarted + alertRenewalFailedID = randomAlertID() // constant until restarted ) func alertIDForAccount(alertID [32]byte, id rhpv3.Account) types.Hash256 { @@ -54,6 +55,20 @@ func (ap *Autopilot) DismissAlert(ctx context.Context, ids ...types.Hash256) { } } +func (ap *Autopilot) HasAlert(ctx context.Context, id types.Hash256) bool { + ar, err := ap.alerts.Alerts(ctx, alerts.AlertsOpts{Offset: 0, Limit: -1}) + if err != nil { + ap.logger.Errorf("failed to fetch alerts: %v", err) + return false + } + for _, alert := range ar.Alerts { + if alert.ID == id { + return true + } + } + return false +} + func newAccountLowBalanceAlert(address types.Address, balance, allowance types.Currency, bh, renewWindow, endHeight uint64) alerts.Alert { severity := alerts.SeverityInfo if bh+renewWindow/2 >= endHeight { @@ -137,37 +152,6 @@ func newContractPruningFailedAlert(hk types.PublicKey, version string, fcid type } } -func newContractSetChangeAlert(name string, additions map[types.FileContractID]contractSetAddition, removals map[types.FileContractID]contractSetRemoval) alerts.Alert { - var hint string - if len(removals) > 0 { - hint = "A high churn rate can lead to a lot of unnecessary migrations, it might be necessary to tweak your configuration depending on the reason hosts are being discarded from the set." - } - - removedReasons := make(map[string]string, len(removals)) - for k, v := range removals { - removedReasons[k.String()] = v.Reason - } - - return alerts.Alert{ - ID: randomAlertID(), - Severity: alerts.SeverityInfo, - Message: "Contract set changed", - Data: map[string]any{ - "name": name, - "set_additions": additions, - "set_removals": removals, - "hint": hint, - - // TODO: these fields can be removed on the next major release, they - // contain redundant information - "added": len(additions), - "removed": len(removals), - "removals": removedReasons, - }, - Timestamp: time.Now(), - } -} - func newLostSectorsAlert(hk types.PublicKey, lostSectors uint64) alerts.Alert { return alerts.Alert{ ID: alertIDForHost(alertLostSectorsID, hk), diff --git a/autopilot/churn.go b/autopilot/churn.go new file mode 100644 index 000000000..70c4651c2 --- /dev/null +++ b/autopilot/churn.go @@ -0,0 +1,63 @@ +package autopilot + +import ( + "time" + + "go.sia.tech/core/types" + "go.sia.tech/renterd/alerts" +) + +type ( + accumulatedChurn struct { + additions map[types.FileContractID][]contractSetAddition + removals map[types.FileContractID][]contractSetRemoval + } +) + +func newAccumulatedChurn() *accumulatedChurn { + return &accumulatedChurn{ + additions: make(map[types.FileContractID][]contractSetAddition), + removals: make(map[types.FileContractID][]contractSetRemoval), + } +} + +func (c *accumulatedChurn) Alert(name string) alerts.Alert { + var hint string + if len(c.removals) > 0 { + hint = "A high churn rate can lead to a lot of unnecessary migrations, it might be necessary to tweak your configuration depending on the reason hosts are being discarded from the set." + } + + removedReasons := make(map[string][]string, len(c.removals)) + for fcid, contractRemovals := range c.removals { + for _, removal := range contractRemovals { + removedReasons[fcid.String()] = append(removedReasons[fcid.String()], removal.Reason) + } + } + + return alerts.Alert{ + ID: alertChurnID, + Severity: alerts.SeverityInfo, + Message: "Contract set changed", + Data: map[string]any{ + "name": name, + "set_additions": c.additions, + "set_removals": c.removals, + "hint": hint, + }, + Timestamp: time.Now(), + } +} + +func (c *accumulatedChurn) Apply(additions map[types.FileContractID]contractSetAddition, removals map[types.FileContractID]contractSetRemoval) { + for fcid, addition := range additions { + c.additions[fcid] = append(c.additions[fcid], addition) + } + for fcid, removal := range removals { + c.removals[fcid] = append(c.removals[fcid], removal) + } +} + +func (c *accumulatedChurn) Reset() { + c.additions = make(map[types.FileContractID][]contractSetAddition) + c.removals = make(map[types.FileContractID][]contractSetRemoval) +} diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 092f2a831..7909277f0 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -85,6 +85,7 @@ const ( type ( contractor struct { ap *Autopilot + churn *accumulatedChurn resolver *ipResolver logger *zap.SugaredLogger @@ -130,7 +131,7 @@ type ( contractSetRemoval struct { Size uint64 `json:"size"` HostKey types.PublicKey `json:"hostKey"` - Reason string `json:"reason"` + Reason string `json:"reasons"` } renewal struct { @@ -143,6 +144,7 @@ type ( func newContractor(ap *Autopilot, revisionSubmissionBuffer uint64, revisionBroadcastInterval time.Duration) *contractor { return &contractor{ ap: ap, + churn: newAccumulatedChurn(), logger: ap.logger.Named("contractor"), revisionBroadcastInterval: revisionBroadcastInterval, @@ -536,7 +538,11 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, ) hasChanged := len(setAdditions)+len(setRemovals) > 0 if hasChanged { - c.ap.RegisterAlert(ctx, newContractSetChangeAlert(name, setAdditions, setRemovals)) + if !c.ap.HasAlert(ctx, alertChurnID) { + c.churn.Reset() + } + c.churn.Apply(setAdditions, setRemovals) + c.ap.RegisterAlert(ctx, c.churn.Alert(name)) } return hasChanged } diff --git a/bus/bus.go b/bus/bus.go index 9ee6e1ba2..e7e6ddaac 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1726,7 +1726,10 @@ func (b *bus) gougingParams(ctx context.Context) (api.GougingParams, error) { } func (b *bus) handleGETAlertsDeprecated(jc jape.Context) { - ar := b.alertMgr.Active(0, -1) + ar, err := b.alertMgr.Alerts(jc.Request.Context(), alerts.AlertsOpts{Offset: 0, Limit: -1}) + if jc.Check("failed to fetch alerts", err) != nil { + return + } jc.Encode(ar.Alerts) } @@ -1744,7 +1747,11 @@ func (b *bus) handleGETAlerts(jc jape.Context) { jc.Error(errors.New("offset must be non-negative"), http.StatusBadRequest) return } - jc.Encode(b.alertMgr.Active(offset, limit)) + ar, err := b.alertMgr.Alerts(jc.Request.Context(), alerts.AlertsOpts{Offset: offset, Limit: limit}) + if jc.Check("failed to fetch alerts", err) != nil { + return + } + jc.Encode(ar) } func (b *bus) handlePOSTAlertsDismiss(jc jape.Context) { diff --git a/bus/client/alerts.go b/bus/client/alerts.go index 7f2bf9aa7..7eceaeaed 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -10,13 +10,13 @@ import ( ) // Alerts fetches the active alerts from the bus. -func (c *Client) Alerts(opts alerts.AlertsOpts) (resp alerts.AlertsResponse, err error) { +func (c *Client) Alerts(ctx context.Context, opts alerts.AlertsOpts) (resp alerts.AlertsResponse, err error) { values := url.Values{} values.Set("offset", fmt.Sprint(opts.Offset)) if opts.Limit != 0 { values.Set("limit", fmt.Sprint(opts.Limit)) } - err = c.c.GET("/alerts?"+values.Encode(), &resp) + err = c.c.WithContext(ctx).GET("/alerts?"+values.Encode(), &resp) return } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 4fb62ff31..f30a0906a 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1923,7 +1923,7 @@ func TestAlerts(t *testing.T) { tt.OK(b.RegisterAlert(context.Background(), alert)) findAlert := func(id types.Hash256) *alerts.Alert { t.Helper() - ar, err := b.Alerts(alerts.AlertsOpts{}) + ar, err := b.Alerts(context.Background(), alerts.AlertsOpts{}) tt.OK(err) for _, alert := range ar.Alerts { if alert.ID == id { @@ -1960,7 +1960,7 @@ func TestAlerts(t *testing.T) { } // try to find with offset = 1 - ar, err := b.Alerts(alerts.AlertsOpts{Offset: 1}) + ar, err := b.Alerts(context.Background(), alerts.AlertsOpts{Offset: 1}) foundAlerts := ar.Alerts tt.OK(err) if len(foundAlerts) != 1 || foundAlerts[0].ID != alert.ID { @@ -1968,7 +1968,7 @@ func TestAlerts(t *testing.T) { } // try to find with limit = 1 - ar, err = b.Alerts(alerts.AlertsOpts{Limit: 1}) + ar, err = b.Alerts(context.Background(), alerts.AlertsOpts{Limit: 1}) foundAlerts = ar.Alerts tt.OK(err) if len(foundAlerts) != 1 || foundAlerts[0].ID != alert2.ID { From c17252a8f250cc0184918ec93dab5e7adeb31791 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 23 Feb 2024 13:14:30 +0100 Subject: [PATCH 135/144] autopilot: update churn alert to contain timestamp --- autopilot/churn.go | 41 +++++++++++++++++------------- autopilot/contractor.go | 56 +++++++++++++++++++++++++++++------------ 2 files changed, 63 insertions(+), 34 deletions(-) diff --git a/autopilot/churn.go b/autopilot/churn.go index 70c4651c2..fdc1a0f54 100644 --- a/autopilot/churn.go +++ b/autopilot/churn.go @@ -9,15 +9,15 @@ import ( type ( accumulatedChurn struct { - additions map[types.FileContractID][]contractSetAddition - removals map[types.FileContractID][]contractSetRemoval + additions map[types.FileContractID]contractSetAdditions + removals map[types.FileContractID]contractSetRemovals } ) func newAccumulatedChurn() *accumulatedChurn { return &accumulatedChurn{ - additions: make(map[types.FileContractID][]contractSetAddition), - removals: make(map[types.FileContractID][]contractSetRemoval), + additions: make(map[types.FileContractID]contractSetAdditions), + removals: make(map[types.FileContractID]contractSetRemovals), } } @@ -27,13 +27,6 @@ func (c *accumulatedChurn) Alert(name string) alerts.Alert { hint = "A high churn rate can lead to a lot of unnecessary migrations, it might be necessary to tweak your configuration depending on the reason hosts are being discarded from the set." } - removedReasons := make(map[string][]string, len(c.removals)) - for fcid, contractRemovals := range c.removals { - for _, removal := range contractRemovals { - removedReasons[fcid.String()] = append(removedReasons[fcid.String()], removal.Reason) - } - } - return alerts.Alert{ ID: alertChurnID, Severity: alerts.SeverityInfo, @@ -48,16 +41,28 @@ func (c *accumulatedChurn) Alert(name string) alerts.Alert { } } -func (c *accumulatedChurn) Apply(additions map[types.FileContractID]contractSetAddition, removals map[types.FileContractID]contractSetRemoval) { - for fcid, addition := range additions { - c.additions[fcid] = append(c.additions[fcid], addition) +func (c *accumulatedChurn) Apply(additions map[types.FileContractID]contractSetAdditions, removals map[types.FileContractID]contractSetRemovals) { + for fcid, a := range additions { + if _, exists := c.additions[fcid]; !exists { + c.additions[fcid] = a + } else { + additions := c.additions[fcid] + additions.Additions = append(additions.Additions, a.Additions...) + c.additions[fcid] = additions + } } - for fcid, removal := range removals { - c.removals[fcid] = append(c.removals[fcid], removal) + for fcid, r := range removals { + if _, exists := c.removals[fcid]; !exists { + c.removals[fcid] = r + } else { + removals := c.removals[fcid] + removals.Removals = append(removals.Removals, r.Removals...) + c.removals[fcid] = removals + } } } func (c *accumulatedChurn) Reset() { - c.additions = make(map[types.FileContractID][]contractSetAddition) - c.removals = make(map[types.FileContractID][]contractSetRemoval) + c.additions = make(map[types.FileContractID]contractSetAdditions) + c.removals = make(map[types.FileContractID]contractSetRemovals) } diff --git a/autopilot/contractor.go b/autopilot/contractor.go index 7909277f0..9e2b52cca 100644 --- a/autopilot/contractor.go +++ b/autopilot/contractor.go @@ -123,15 +123,25 @@ type ( recoverable bool } + contractSetAdditions struct { + HostKey types.PublicKey `json:"hostKey"` + Additions []contractSetAddition `json:"additions"` + } + contractSetAddition struct { - Size uint64 `json:"size"` - HostKey types.PublicKey `json:"hostKey"` + Size uint64 `json:"size"` + Time api.TimeRFC3339 `json:"time"` + } + + contractSetRemovals struct { + HostKey types.PublicKey `json:"hostKey"` + Removals []contractSetRemoval `json:"removals"` } contractSetRemoval struct { - Size uint64 `json:"size"` - HostKey types.PublicKey `json:"hostKey"` - Reason string `json:"reasons"` + Size uint64 `json:"size"` + Reason string `json:"reasons"` + Time api.TimeRFC3339 `json:"time"` } renewal struct { @@ -455,8 +465,9 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, } // log added and removed contracts - setAdditions := make(map[types.FileContractID]contractSetAddition) - setRemovals := make(map[types.FileContractID]contractSetRemoval) + setAdditions := make(map[types.FileContractID]contractSetAdditions) + setRemovals := make(map[types.FileContractID]contractSetRemovals) + now := api.TimeNow() for _, contract := range oldSet { _, exists := inNewSet[contract.ID] _, renewed := inNewSet[renewalsFromTo[contract.ID]] @@ -466,11 +477,18 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, reason = "unknown" } - setRemovals[contract.ID] = contractSetRemoval{ - Size: contractData[contract.ID], - HostKey: contract.HostKey, - Reason: reason, + if _, exists := setRemovals[contract.ID]; !exists { + setRemovals[contract.ID] = contractSetRemovals{ + HostKey: contract.HostKey, + } } + removals := setRemovals[contract.ID] + removals.Removals = append(removals.Removals, contractSetRemoval{ + Size: contractData[contract.ID], + Reason: reason, + Time: now, + }) + setRemovals[contract.ID] = removals c.logger.Debugf("contract %v was removed from the contract set, size: %v, reason: %v", contract.ID, contractData[contract.ID], reason) } } @@ -478,10 +496,17 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, _, existed := inOldSet[contract.ID] _, renewed := renewalsToFrom[contract.ID] if !existed && !renewed { - setAdditions[contract.ID] = contractSetAddition{ - Size: contractData[contract.ID], - HostKey: contract.HostKey, + if _, exists := setAdditions[contract.ID]; !exists { + setAdditions[contract.ID] = contractSetAdditions{ + HostKey: contract.HostKey, + } } + additions := setAdditions[contract.ID] + additions.Additions = append(additions.Additions, contractSetAddition{ + Size: contractData[contract.ID], + Time: now, + }) + setAdditions[contract.ID] = additions c.logger.Debugf("contract %v was added to the contract set, size: %v", contract.ID, contractData[contract.ID]) } } @@ -501,7 +526,6 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, } // record churn metrics - now := api.TimeNow() var metrics []api.ContractSetChurnMetric for fcid := range setAdditions { metrics = append(metrics, api.ContractSetChurnMetric{ @@ -516,7 +540,7 @@ func (c *contractor) computeContractSetChanged(ctx context.Context, name string, Name: c.ap.state.cfg.Contracts.Set, ContractID: fcid, Direction: api.ChurnDirRemoved, - Reason: removal.Reason, + Reason: removal.Removals[0].Reason, Timestamp: now, }) } From e9a593db12e9e159000f645b38fb0ab3c98c6e55 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Fri, 23 Feb 2024 13:55:11 +0100 Subject: [PATCH 136/144] bus: only log lost sectors when there was a contract with that host --- bus/bus.go | 6 ++++-- stores/metadata.go | 7 +++++-- stores/metadata_test.go | 4 +++- worker/download.go | 2 -- 4 files changed, 12 insertions(+), 7 deletions(-) diff --git a/bus/bus.go b/bus/bus.go index 9ee6e1ba2..17220d3d4 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -126,7 +126,7 @@ type ( ContractSizes(ctx context.Context) (map[types.FileContractID]api.ContractSize, error) ContractSize(ctx context.Context, id types.FileContractID) (api.ContractSize, error) - DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) error + DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) (int, error) Bucket(_ context.Context, bucketName string) (api.Bucket, error) CreateBucket(_ context.Context, bucketName string, policy api.BucketPolicy) error @@ -1409,9 +1409,11 @@ func (b *bus) sectorsHostRootHandlerDELETE(jc jape.Context) { } else if jc.DecodeParam("root", &root) != nil { return } - err := b.ms.DeleteHostSector(jc.Request.Context(), hk, root) + n, err := b.ms.DeleteHostSector(jc.Request.Context(), hk, root) if jc.Check("failed to mark sector as lost", err) != nil { return + } else if n > 0 { + b.logger.Infow("successfully marked sector as lost", "hk", hk, "root", root) } } diff --git a/stores/metadata.go b/stores/metadata.go index 2c84e6624..c281c9800 100644 --- a/stores/metadata.go +++ b/stores/metadata.go @@ -1620,8 +1620,9 @@ func (s *SQLStore) CopyObject(ctx context.Context, srcBucket, dstBucket, srcPath return } -func (s *SQLStore) DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) error { - return s.retryTransaction(func(tx *gorm.DB) error { +func (s *SQLStore) DeleteHostSector(ctx context.Context, hk types.PublicKey, root types.Hash256) (int, error) { + var deletedSectors int + err := s.retryTransaction(func(tx *gorm.DB) error { // Fetch contract_sectors to delete. var sectors []dbContractSector err := tx.Raw(` @@ -1660,6 +1661,7 @@ func (s *SQLStore) DeleteHostSector(ctx context.Context, hk types.PublicKey, roo } else if res.RowsAffected != int64(len(sectors)) { return fmt.Errorf("expected %v affected rows but got %v", len(sectors), res.RowsAffected) } + deletedSectors = len(sectors) // Increment the host's lostSectors by the number of lost sectors. if err := tx.Exec("UPDATE hosts SET lost_sectors = lost_sectors + ? WHERE public_key = ?", len(sectors), publicKey(hk)).Error; err != nil { @@ -1687,6 +1689,7 @@ func (s *SQLStore) DeleteHostSector(ctx context.Context, hk types.PublicKey, roo } return nil }) + return deletedSectors, err } func (s *SQLStore) UpdateObject(ctx context.Context, bucket, path, contractSet, eTag, mimeType string, metadata api.ObjectUserMetadata, o object.Object) error { diff --git a/stores/metadata_test.go b/stores/metadata_test.go index e36ecbf8e..0b785f6b6 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -3582,8 +3582,10 @@ func TestDeleteHostSector(t *testing.T) { } // Prune the sector from hk1. - if err := ss.DeleteHostSector(context.Background(), hk1, root); err != nil { + if n, err := ss.DeleteHostSector(context.Background(), hk1, root); err != nil { t.Fatal(err) + } else if n == 0 { + t.Fatal("no sectors were pruned") } // Make sure 2 contractSector entries exist. diff --git a/worker/download.go b/worker/download.go index 462a2292d..3f3d63ac4 100644 --- a/worker/download.go +++ b/worker/download.go @@ -761,8 +761,6 @@ loop: if isSectorNotFound(resp.err) { if err := s.mgr.os.DeleteHostSector(ctx, resp.req.host.PublicKey(), resp.req.root); err != nil { s.mgr.logger.Errorw("failed to mark sector as lost", "hk", resp.req.host.PublicKey(), "root", resp.req.root, zap.Error(err)) - } else { - s.mgr.logger.Infow("successfully marked sector as lost", "hk", resp.req.host.PublicKey(), "root", resp.req.root) } } else if isPriceTableGouging(resp.err) && s.overpay && !resp.req.overpay { resp.req.overpay = true // ensures we don't retry the same request over and over again From 9e0250557254fa554b4361b0ffa00ba4ef524d8c Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Feb 2024 01:42:33 +0000 Subject: [PATCH 137/144] build(deps): bump go.sia.tech/coreutils from 0.0.1 to 0.0.3 Bumps [go.sia.tech/coreutils](https://github.com/SiaFoundation/coreutils) from 0.0.1 to 0.0.3. - [Commits](https://github.com/SiaFoundation/coreutils/compare/v0.0.1...v0.0.3) --- updated-dependencies: - dependency-name: go.sia.tech/coreutils dependency-type: direct:production update-type: version-update:semver-patch ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/go.mod b/go.mod index 0ce29b965..7932deecf 100644 --- a/go.mod +++ b/go.mod @@ -12,7 +12,7 @@ require ( github.com/montanaflynn/stats v0.7.1 gitlab.com/NebulousLabs/encoding v0.0.0-20200604091946-456c3dc907fe go.sia.tech/core v0.2.1 - go.sia.tech/coreutils v0.0.1 + go.sia.tech/coreutils v0.0.3 go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 go.sia.tech/hostd v1.0.2 go.sia.tech/jape v0.11.2-0.20240124024603-93559895d640 diff --git a/go.sum b/go.sum index 2570e9fc5..aa84a9afb 100644 --- a/go.sum +++ b/go.sum @@ -241,8 +241,8 @@ 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.sia.tech/core v0.2.1 h1:CqmMd+T5rAhC+Py3NxfvGtvsj/GgwIqQHHVrdts/LqY= go.sia.tech/core v0.2.1/go.mod h1:3EoY+rR78w1/uGoXXVqcYdwSjSJKuEMI5bL7WROA27Q= -go.sia.tech/coreutils v0.0.1 h1:Th8iiF9fjkBaxlKRgPJfRtsD3Pb8U4d2m/OahB6wffg= -go.sia.tech/coreutils v0.0.1/go.mod h1:3Mb206QDd3NtRiaHZ2kN87/HKXhcBF6lHVatS7PkViY= +go.sia.tech/coreutils v0.0.3 h1:ZxuzovRpQMvfy/pCOV4om1cPF6sE15GyJyK36kIrF1Y= +go.sia.tech/coreutils v0.0.3/go.mod h1:UBFc77wXiE//eyilO5HLOncIEj7F69j0Nv2OkFujtP0= go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2 h1:ulzfJNjxN5DjXHClkW2pTiDk+eJ+0NQhX87lFDZ03t0= go.sia.tech/gofakes3 v0.0.0-20231109151325-e0d47c10dce2/go.mod h1:PlsiVCn6+wssrR7bsOIlZm0DahsVrDydrlbjY4F14sg= go.sia.tech/hostd v1.0.2 h1:GjzNIAlwg3/dViF6258Xn5DI3+otQLRqmkoPDugP+9Y= From b1efe2c1d71abe095a76d1e2190b9515576f0284 Mon Sep 17 00:00:00 2001 From: "dependabot[bot]" <49699333+dependabot[bot]@users.noreply.github.com> Date: Mon, 26 Feb 2024 01:42:42 +0000 Subject: [PATCH 138/144] build(deps): bump go.uber.org/zap from 1.26.0 to 1.27.0 Bumps [go.uber.org/zap](https://github.com/uber-go/zap) from 1.26.0 to 1.27.0. - [Release notes](https://github.com/uber-go/zap/releases) - [Changelog](https://github.com/uber-go/zap/blob/master/CHANGELOG.md) - [Commits](https://github.com/uber-go/zap/compare/v1.26.0...v1.27.0) --- updated-dependencies: - dependency-name: go.uber.org/zap dependency-type: direct:production update-type: version-update:semver-minor ... Signed-off-by: dependabot[bot] --- go.mod | 2 +- go.sum | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/go.mod b/go.mod index 0ce29b965..1804eda8a 100644 --- a/go.mod +++ b/go.mod @@ -19,7 +19,7 @@ require ( go.sia.tech/mux v1.2.0 go.sia.tech/siad v1.5.10-0.20230228235644-3059c0b930ca go.sia.tech/web/renterd v0.46.0 - go.uber.org/zap v1.26.0 + go.uber.org/zap v1.27.0 golang.org/x/crypto v0.19.0 golang.org/x/term v0.17.0 gopkg.in/yaml.v3 v3.0.1 diff --git a/go.sum b/go.sum index 2570e9fc5..24e1f476c 100644 --- a/go.sum +++ b/go.sum @@ -258,14 +258,14 @@ go.sia.tech/web v0.0.0-20231213145933-3f175a86abff/go.mod h1:RKODSdOmR3VtObPAcGw go.sia.tech/web/renterd v0.46.0 h1:BMVg4i7LxSlc8wZ4T0EG1k3EK4JxVIzCfD3/cjmwH0k= go.sia.tech/web/renterd v0.46.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/goleak v1.2.0/go.mod h1:XJYK+MuIchqpmGmUSAzotztawfKvYLUIgg7guXrwVUo= +go.uber.org/goleak v1.3.0 h1:2K3zAYmnTNqV73imy9J1T3WC+gmCePx2hEGkimedGto= +go.uber.org/goleak v1.3.0/go.mod h1:CoHD4mav9JJNrW/WLlf7HGZPjdw8EucARQHekz1X6bE= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.11.0 h1:blXXJkSxSSfBVBlC76pxqeO+LN3aDfLQo+309xJstO0= go.uber.org/multierr v1.11.0/go.mod h1:20+QtiLqy0Nd6FdQB9TLXag12DsQkrbs3htMFfDN80Y= go.uber.org/zap v1.10.0/go.mod h1:vwi/ZaCAaUcBkycHslxD9B2zi4UTXhF60s6SWpuDF0Q= -go.uber.org/zap v1.26.0 h1:sI7k6L95XOKS281NhVKOFCUNIvv9e0w4BF8N3u+tCRo= -go.uber.org/zap v1.26.0/go.mod h1:dtElttAiwGvoJ/vj4IwHBS/gXsEu/pZ50mUIRWuG0so= +go.uber.org/zap v1.27.0 h1:aJMhYGrd5QSmlpLMr2MftRKl7t8J8PTZPA732ud/XR8= +go.uber.org/zap v1.27.0/go.mod h1:GB2qFLM7cTU87MWRP2mPIjqfIDnGu+VIO4V/SdhGo2E= golang.org/x/crypto v0.0.0-20180904163835-0709b304e793/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= golang.org/x/crypto v0.0.0-20191011191535-87dc89f01550/go.mod h1:yigFU9vqHzYiE8UmvKecakEJjdnWj3jj499lnFckfCI= From 0ef1757537003c419df3909b925d9f1d76f064a9 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 27 Feb 2024 10:55:58 +0100 Subject: [PATCH 139/144] bus: filter by alert severity --- alerts/alerts.go | 55 +++++++++++++++++++++----------- bus/bus.go | 9 +++++- bus/client/alerts.go | 3 ++ internal/testing/cluster_test.go | 38 ++++++++++++++++++++++ 4 files changed, 86 insertions(+), 19 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index f11004dbe..a14d460b6 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -66,14 +66,19 @@ type ( } AlertsOpts struct { - Offset int - Limit int + Offset int + Limit int + Severity Severity } AlertsResponse struct { - Alerts []Alert `json:"alerts"` - HasMore bool `json:"hasMore"` - Total int `json:"total"` + Alerts []Alert `json:"alerts"` + HasMore bool `json:"hasMore"` + Total int `json:"total"` + TotalInfo int `json:"totalInfo"` + TotalWarning int `json:"totalWarning"` + TotalError int `json:"totalError"` + TotalCritical int `json:"totalCritical"` } ) @@ -93,15 +98,8 @@ func (s Severity) String() string { } } -// MarshalJSON implements the json.Marshaler interface. -func (s Severity) MarshalJSON() ([]byte, error) { - return []byte(fmt.Sprintf(`%q`, s.String())), nil -} - -// UnmarshalJSON implements the json.Unmarshaler interface. -func (s *Severity) UnmarshalJSON(b []byte) error { - status := strings.Trim(string(b), `"`) - switch status { +func (s *Severity) LoadString(str string) error { + switch str { case severityInfoStr: *s = SeverityInfo case severityWarningStr: @@ -111,11 +109,21 @@ func (s *Severity) UnmarshalJSON(b []byte) error { case severityCriticalStr: *s = SeverityCritical default: - return fmt.Errorf("unrecognized severity: %v", status) + return fmt.Errorf("unrecognized severity: %v", str) } return nil } +// MarshalJSON implements the json.Marshaler interface. +func (s Severity) MarshalJSON() ([]byte, error) { + return []byte(fmt.Sprintf(`%q`, s.String())), nil +} + +// UnmarshalJSON implements the json.Unmarshaler interface. +func (s *Severity) UnmarshalJSON(b []byte) error { + return s.LoadString(strings.Trim(string(b), `"`)) +} + // RegisterAlert implements the Alerter interface. func (m *Manager) RegisterAlert(ctx context.Context, alert Alert) error { if alert.ID == (types.Hash256{}) { @@ -176,9 +184,7 @@ func (m *Manager) Alerts(_ context.Context, opts AlertsOpts) (AlertsResponse, er defer m.mu.Unlock() offset, limit := opts.Offset, opts.Limit - resp := AlertsResponse{ - Total: len(m.alerts), - } + resp := AlertsResponse{} if offset >= len(m.alerts) { return resp, nil @@ -188,6 +194,19 @@ func (m *Manager) Alerts(_ context.Context, opts AlertsOpts) (AlertsResponse, er alerts := make([]Alert, 0, len(m.alerts)) for _, a := range m.alerts { + resp.Total++ + if a.Severity == SeverityInfo { + resp.TotalInfo++ + } else if a.Severity == SeverityWarning { + resp.TotalWarning++ + } else if a.Severity == SeverityError { + resp.TotalError++ + } else if a.Severity == SeverityCritical { + resp.TotalCritical++ + } + if opts.Severity != 0 && a.Severity != opts.Severity { + continue // filter by severity + } alerts = append(alerts, a) } sort.Slice(alerts, func(i, j int) bool { diff --git a/bus/bus.go b/bus/bus.go index e7e6ddaac..4106bc231 100644 --- a/bus/bus.go +++ b/bus/bus.go @@ -1739,6 +1739,7 @@ func (b *bus) handleGETAlerts(jc jape.Context) { return } offset, limit := 0, -1 + var severity alerts.Severity if jc.DecodeForm("offset", &offset) != nil { return } else if jc.DecodeForm("limit", &limit) != nil { @@ -1746,8 +1747,14 @@ func (b *bus) handleGETAlerts(jc jape.Context) { } else if offset < 0 { jc.Error(errors.New("offset must be non-negative"), http.StatusBadRequest) return + } else if jc.DecodeForm("severity", &severity) != nil { + return } - ar, err := b.alertMgr.Alerts(jc.Request.Context(), alerts.AlertsOpts{Offset: offset, Limit: limit}) + ar, err := b.alertMgr.Alerts(jc.Request.Context(), alerts.AlertsOpts{ + Offset: offset, + Limit: limit, + Severity: severity, + }) if jc.Check("failed to fetch alerts", err) != nil { return } diff --git a/bus/client/alerts.go b/bus/client/alerts.go index 7eceaeaed..28c3b9a84 100644 --- a/bus/client/alerts.go +++ b/bus/client/alerts.go @@ -16,6 +16,9 @@ func (c *Client) Alerts(ctx context.Context, opts alerts.AlertsOpts) (resp alert if opts.Limit != 0 { values.Set("limit", fmt.Sprint(opts.Limit)) } + if opts.Severity != 0 { + values.Set("severity", opts.Severity.String()) + } err = c.c.WithContext(ctx).GET("/alerts?"+values.Encode(), &resp) return } diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index f30a0906a..69b318f66 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1974,6 +1974,44 @@ func TestAlerts(t *testing.T) { if len(foundAlerts) != 1 || foundAlerts[0].ID != alert2.ID { t.Fatal("wrong alert") } + + // register more alerts + for severity := alerts.SeverityInfo; severity <= alerts.SeverityCritical; severity++ { + for j := 0; j < 3*int(severity); j++ { + tt.OK(b.RegisterAlert(context.Background(), alerts.Alert{ + ID: frand.Entropy256(), + Severity: severity, + Message: "test", + Data: map[string]interface{}{ + "origin": "test", + }, + Timestamp: time.Now(), + })) + } + } + for severity := alerts.SeverityInfo; severity <= alerts.SeverityCritical; severity++ { + ar, err = b.Alerts(context.Background(), alerts.AlertsOpts{Severity: severity}) + tt.OK(err) + if ar.Total != 32 { + t.Fatal("expected 32 alerts", ar.Total) + } else if ar.TotalInfo != 3 { + t.Fatal("expected 3 info alerts", ar.TotalInfo) + } else if ar.TotalWarning != 6 { + t.Fatal("expected 6 warning alerts", ar.TotalWarning) + } else if ar.TotalError != 9 { + t.Fatal("expected 9 error alerts", ar.TotalError) + } else if ar.TotalCritical != 14 { + t.Fatal("expected 14 critical alerts", ar.TotalCritical) + } else if severity == alerts.SeverityInfo && len(ar.Alerts) != ar.TotalInfo { + t.Fatalf("expected %v info alerts, got %v", ar.TotalInfo, len(ar.Alerts)) + } else if severity == alerts.SeverityWarning && len(ar.Alerts) != ar.TotalWarning { + t.Fatalf("expected %v warning alerts, got %v", ar.TotalWarning, len(ar.Alerts)) + } else if severity == alerts.SeverityError && len(ar.Alerts) != ar.TotalError { + t.Fatalf("expected %v error alerts, got %v", ar.TotalError, len(ar.Alerts)) + } else if severity == alerts.SeverityCritical && len(ar.Alerts) != ar.TotalCritical { + t.Fatalf("expected %v critical alerts, got %v", ar.TotalCritical, len(ar.Alerts)) + } + } } func TestMultipartUploads(t *testing.T) { From b7e14e8b3353cc8c4928b9a677c22a69a8cc6015 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 27 Feb 2024 11:00:00 +0100 Subject: [PATCH 140/144] stores: check for exact value in TestDeleteHostSector --- stores/metadata_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 0b785f6b6..16e104695 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -3584,8 +3584,8 @@ func TestDeleteHostSector(t *testing.T) { // Prune the sector from hk1. if n, err := ss.DeleteHostSector(context.Background(), hk1, root); err != nil { t.Fatal(err) - } else if n == 0 { - t.Fatal("no sectors were pruned") + } else if n != 2 { + t.Fatal("no sectors were pruned", n) } // Make sure 2 contractSector entries exist. From 7c5b3c91efab2acf8f4b90588dace1240d00c0f7 Mon Sep 17 00:00:00 2001 From: Chris Schinnerl Date: Tue, 27 Feb 2024 13:56:29 +0100 Subject: [PATCH 141/144] bus: breakdown totals --- alerts/alerts.go | 28 ++++++++++++++----------- internal/testing/cluster_test.go | 36 ++++++++++++++++---------------- 2 files changed, 34 insertions(+), 30 deletions(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index a14d460b6..1ebebbc80 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -72,16 +72,21 @@ type ( } AlertsResponse struct { - Alerts []Alert `json:"alerts"` - HasMore bool `json:"hasMore"` - Total int `json:"total"` - TotalInfo int `json:"totalInfo"` - TotalWarning int `json:"totalWarning"` - TotalError int `json:"totalError"` - TotalCritical int `json:"totalCritical"` + Alerts []Alert `json:"alerts"` + HasMore bool `json:"hasMore"` + Totals struct { + Info int `json:"info"` + Warning int `json:"warning"` + Error int `json:"error"` + Critical int `json:"critical"` + } `json:"total"` } ) +func (ar AlertsResponse) Total() int { + return ar.Totals.Info + ar.Totals.Warning + ar.Totals.Error + ar.Totals.Critical +} + // String implements the fmt.Stringer interface. func (s Severity) String() string { switch s { @@ -194,15 +199,14 @@ func (m *Manager) Alerts(_ context.Context, opts AlertsOpts) (AlertsResponse, er alerts := make([]Alert, 0, len(m.alerts)) for _, a := range m.alerts { - resp.Total++ if a.Severity == SeverityInfo { - resp.TotalInfo++ + resp.Totals.Info++ } else if a.Severity == SeverityWarning { - resp.TotalWarning++ + resp.Totals.Warning++ } else if a.Severity == SeverityError { - resp.TotalError++ + resp.Totals.Error++ } else if a.Severity == SeverityCritical { - resp.TotalCritical++ + resp.Totals.Critical++ } if opts.Severity != 0 && a.Severity != opts.Severity { continue // filter by severity diff --git a/internal/testing/cluster_test.go b/internal/testing/cluster_test.go index 69b318f66..6b5f88769 100644 --- a/internal/testing/cluster_test.go +++ b/internal/testing/cluster_test.go @@ -1992,24 +1992,24 @@ func TestAlerts(t *testing.T) { for severity := alerts.SeverityInfo; severity <= alerts.SeverityCritical; severity++ { ar, err = b.Alerts(context.Background(), alerts.AlertsOpts{Severity: severity}) tt.OK(err) - if ar.Total != 32 { - t.Fatal("expected 32 alerts", ar.Total) - } else if ar.TotalInfo != 3 { - t.Fatal("expected 3 info alerts", ar.TotalInfo) - } else if ar.TotalWarning != 6 { - t.Fatal("expected 6 warning alerts", ar.TotalWarning) - } else if ar.TotalError != 9 { - t.Fatal("expected 9 error alerts", ar.TotalError) - } else if ar.TotalCritical != 14 { - t.Fatal("expected 14 critical alerts", ar.TotalCritical) - } else if severity == alerts.SeverityInfo && len(ar.Alerts) != ar.TotalInfo { - t.Fatalf("expected %v info alerts, got %v", ar.TotalInfo, len(ar.Alerts)) - } else if severity == alerts.SeverityWarning && len(ar.Alerts) != ar.TotalWarning { - t.Fatalf("expected %v warning alerts, got %v", ar.TotalWarning, len(ar.Alerts)) - } else if severity == alerts.SeverityError && len(ar.Alerts) != ar.TotalError { - t.Fatalf("expected %v error alerts, got %v", ar.TotalError, len(ar.Alerts)) - } else if severity == alerts.SeverityCritical && len(ar.Alerts) != ar.TotalCritical { - t.Fatalf("expected %v critical alerts, got %v", ar.TotalCritical, len(ar.Alerts)) + if ar.Total() != 32 { + t.Fatal("expected 32 alerts", ar.Total()) + } else if ar.Totals.Info != 3 { + t.Fatal("expected 3 info alerts", ar.Totals.Info) + } else if ar.Totals.Warning != 6 { + t.Fatal("expected 6 warning alerts", ar.Totals.Warning) + } else if ar.Totals.Error != 9 { + t.Fatal("expected 9 error alerts", ar.Totals.Error) + } else if ar.Totals.Critical != 14 { + t.Fatal("expected 14 critical alerts", ar.Totals.Critical) + } else if severity == alerts.SeverityInfo && len(ar.Alerts) != ar.Totals.Info { + t.Fatalf("expected %v info alerts, got %v", ar.Totals.Info, len(ar.Alerts)) + } else if severity == alerts.SeverityWarning && len(ar.Alerts) != ar.Totals.Warning { + t.Fatalf("expected %v warning alerts, got %v", ar.Totals.Warning, len(ar.Alerts)) + } else if severity == alerts.SeverityError && len(ar.Alerts) != ar.Totals.Error { + t.Fatalf("expected %v error alerts, got %v", ar.Totals.Error, len(ar.Alerts)) + } else if severity == alerts.SeverityCritical && len(ar.Alerts) != ar.Totals.Critical { + t.Fatalf("expected %v critical alerts, got %v", ar.Totals.Critical, len(ar.Alerts)) } } } From 6db1e38cf0f415f330c6ba54a24682af69d1dc16 Mon Sep 17 00:00:00 2001 From: Christopher Schinnerl Date: Tue, 27 Feb 2024 13:58:47 +0100 Subject: [PATCH 142/144] Update alerts/alerts.go Co-authored-by: Peter-Jan Brone --- alerts/alerts.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/alerts/alerts.go b/alerts/alerts.go index 1ebebbc80..6b009360d 100644 --- a/alerts/alerts.go +++ b/alerts/alerts.go @@ -79,7 +79,7 @@ type ( Warning int `json:"warning"` Error int `json:"error"` Critical int `json:"critical"` - } `json:"total"` + } `json:"totals"` } ) From 1beaf32fd9ea43eba33389a70dfd936c972a0a23 Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 27 Feb 2024 14:57:45 +0100 Subject: [PATCH 143/144] stores: fix TestInsertAnnouncements --- stores/hostdb_test.go | 55 ++++++++++++++++--------------------------- 1 file changed, 20 insertions(+), 35 deletions(-) diff --git a/stores/hostdb_test.go b/stores/hostdb_test.go index 3b67c248f..cbbc4428a 100644 --- a/stores/hostdb_test.go +++ b/stores/hostdb_test.go @@ -56,15 +56,7 @@ func TestSQLHostDB(t *testing.T) { // Insert an announcement for the host and another one for an unknown // host. - a := announcement{ - blockHeight: 42, - blockID: types.BlockID{1, 2, 3}, - hk: hk, - timestamp: time.Now().UTC().Round(time.Second), - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "address", - }, - } + a := newTestAnnouncement(hk, "address") err = ss.insertTestAnnouncement(a) if err != nil { t.Fatal(err) @@ -506,23 +498,9 @@ func TestInsertAnnouncements(t *testing.T) { defer ss.Close() // Create announcements for 3 hosts. - ann1 := announcement{ - timestamp: time.Now(), - blockHeight: 1, - blockID: types.BlockID{1}, - hk: types.GeneratePrivateKey().PublicKey(), - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "foo.bar:1000", - }, - } - ann2 := announcement{ - hk: types.GeneratePrivateKey().PublicKey(), - HostAnnouncement: chain.HostAnnouncement{}, - } - ann3 := announcement{ - hk: types.GeneratePrivateKey().PublicKey(), - HostAnnouncement: chain.HostAnnouncement{}, - } + ann1 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "foo.bar:1000") + ann2 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") + ann3 := newTestAnnouncement(types.GeneratePrivateKey().PublicKey(), "") // Insert the first one and check that all fields are set. if err := insertAnnouncements(ss.db, []announcement{ann1}); err != nil { @@ -535,12 +513,12 @@ func TestInsertAnnouncements(t *testing.T) { ann.Model = Model{} // ignore expectedAnn := dbAnnouncement{ HostKey: publicKey(ann1.hk), - BlockHeight: 1, - BlockID: types.BlockID{1}.String(), + BlockHeight: ann1.blockHeight, + BlockID: ann1.blockID.String(), NetAddress: "foo.bar:1000", } if ann != expectedAnn { - t.Fatal("mismatch") + t.Fatal("mismatch", cmp.Diff(ann, expectedAnn)) } // Insert the first and second one. if err := insertAnnouncements(ss.db, []announcement{ann1, ann2}); err != nil { @@ -1095,12 +1073,7 @@ func (s *SQLStore) addTestHost(hk types.PublicKey) error { // addCustomTestHost ensures a host with given hostkey and net address exists. func (s *SQLStore) addCustomTestHost(hk types.PublicKey, na string) error { s.unappliedHostKeys[hk] = struct{}{} - s.unappliedAnnouncements = append(s.unappliedAnnouncements, []announcement{{ - hk: hk, - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: na, - }, - }}...) + s.unappliedAnnouncements = append(s.unappliedAnnouncements, newTestAnnouncement(hk, na)) s.lastSave = time.Now().Add(s.persistInterval * -2) return s.applyUpdates(false) } @@ -1139,6 +1112,18 @@ func newTestPK() (types.PublicKey, types.PrivateKey) { return pk, sk } +func newTestAnnouncement(hk types.PublicKey, na string) announcement { + return announcement{ + blockHeight: 42, + blockID: types.BlockID{1, 2, 3}, + hk: hk, + timestamp: time.Now().UTC().Round(time.Second), + HostAnnouncement: chain.HostAnnouncement{ + NetAddress: na, + }, + } +} + func newTestHostAnnouncement(na string) (chain.HostAnnouncement, types.PrivateKey) { _, sk := newTestPK() a := chain.HostAnnouncement{ From 3279ced0d47f44ba14b1837a0a4dfe852321f5bc Mon Sep 17 00:00:00 2001 From: PJ Date: Tue, 27 Feb 2024 15:03:01 +0100 Subject: [PATCH 144/144] testing: fix TestSQLContractStore, TestRenewedContract, TestRecordContractSpending --- stores/metadata_test.go | 29 ++++------------------------- 1 file changed, 4 insertions(+), 25 deletions(-) diff --git a/stores/metadata_test.go b/stores/metadata_test.go index 07e83338e..205812287 100644 --- a/stores/metadata_test.go +++ b/stores/metadata_test.go @@ -16,7 +16,6 @@ import ( "github.com/google/go-cmp/cmp" rhpv2 "go.sia.tech/core/rhp/v2" "go.sia.tech/core/types" - "go.sia.tech/coreutils/chain" "go.sia.tech/renterd/api" "go.sia.tech/renterd/object" "gorm.io/gorm" @@ -219,12 +218,7 @@ func TestSQLContractStore(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(announcement{ - hk: hk, - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "address", - }, - }) + err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) if err != nil { t.Fatal(err) } @@ -515,21 +509,11 @@ func TestRenewedContract(t *testing.T) { hk, hk2 := hks[0], hks[1] // Add announcements. - err = ss.insertTestAnnouncement(announcement{ - hk: hk, - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "address", - }, - }) + err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) if err != nil { t.Fatal(err) } - err = ss.insertTestAnnouncement(announcement{ - hk: hk2, - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "address2", - }, - }) + err = ss.insertTestAnnouncement(newTestAnnouncement(hk2, "address2")) if err != nil { t.Fatal(err) } @@ -2279,12 +2263,7 @@ func TestRecordContractSpending(t *testing.T) { } // Add an announcement. - err = ss.insertTestAnnouncement(announcement{ - hk: hk, - HostAnnouncement: chain.HostAnnouncement{ - NetAddress: "address", - }, - }) + err = ss.insertTestAnnouncement(newTestAnnouncement(hk, "address")) if err != nil { t.Fatal(err) }