2019-01-24 20:15:10 +00:00
|
|
|
// Copyright (C) 2019 Storj Labs, Inc.
|
2018-10-02 20:46:29 +01:00
|
|
|
// See LICENSE for copying information.
|
|
|
|
|
|
|
|
package checker
|
2018-10-03 19:35:56 +01:00
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2018-10-30 19:16:40 +00:00
|
|
|
"time"
|
2018-10-04 22:40:34 +01:00
|
|
|
|
2019-01-23 19:58:44 +00:00
|
|
|
"github.com/zeebo/errs"
|
2018-10-04 22:40:34 +01:00
|
|
|
"go.uber.org/zap"
|
2019-05-31 15:12:49 +01:00
|
|
|
"golang.org/x/sync/errgroup"
|
2019-01-23 19:58:44 +00:00
|
|
|
monkit "gopkg.in/spacemonkeygo/monkit.v2"
|
2018-10-04 22:40:34 +01:00
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
"storj.io/storj/internal/errs2"
|
2019-02-11 21:06:39 +00:00
|
|
|
"storj.io/storj/internal/sync2"
|
2018-10-09 17:09:33 +01:00
|
|
|
"storj.io/storj/pkg/pb"
|
2018-11-29 18:39:27 +00:00
|
|
|
"storj.io/storj/pkg/storj"
|
2019-04-25 09:46:32 +01:00
|
|
|
"storj.io/storj/satellite/metainfo"
|
2019-07-28 06:55:36 +01:00
|
|
|
"storj.io/storj/satellite/overlay"
|
|
|
|
"storj.io/storj/satellite/repair/irreparable"
|
|
|
|
"storj.io/storj/satellite/repair/queue"
|
2018-10-03 19:35:56 +01:00
|
|
|
)
|
|
|
|
|
2019-01-23 19:58:44 +00:00
|
|
|
// Error is a standard error class for this package.
|
|
|
|
var (
|
|
|
|
Error = errs.Class("checker error")
|
|
|
|
mon = monkit.Package()
|
|
|
|
)
|
|
|
|
|
|
|
|
// Config contains configurable values for checker
|
|
|
|
type Config struct {
|
2019-06-04 13:13:31 +01:00
|
|
|
Interval time.Duration `help:"how frequently checker should check for bad segments" releaseDefault:"30s" devDefault:"0h0m10s"`
|
2019-06-15 17:19:19 +01:00
|
|
|
IrreparableInterval time.Duration `help:"how frequently irrepairable checker should check for lost pieces" releaseDefault:"30m" devDefault:"0h0m5s"`
|
2019-07-08 23:04:35 +01:00
|
|
|
|
|
|
|
ReliabilityCacheStaleness time.Duration `help:"how stale reliable node cache can be" releaseDefault:"5m" devDefault:"5m"`
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
// durabilityStats remote segment information
|
|
|
|
type durabilityStats struct {
|
2019-09-13 14:51:41 +01:00
|
|
|
objectsChecked int64
|
2019-05-30 16:18:20 +01:00
|
|
|
remoteSegmentsChecked int64
|
|
|
|
remoteSegmentsNeedingRepair int64
|
|
|
|
remoteSegmentsLost int64
|
|
|
|
remoteSegmentInfo []string
|
2019-01-23 19:58:44 +00:00
|
|
|
}
|
|
|
|
|
2018-10-09 17:09:33 +01:00
|
|
|
// Checker contains the information needed to do checks for missing pieces
|
2019-09-10 14:24:16 +01:00
|
|
|
//
|
|
|
|
// architecture: Chore
|
2019-02-11 21:06:39 +00:00
|
|
|
type Checker struct {
|
2019-08-01 19:44:32 +01:00
|
|
|
logger *zap.Logger
|
2019-05-22 22:17:52 +01:00
|
|
|
repairQueue queue.RepairQueue
|
|
|
|
irrdb irreparable.DB
|
2019-08-01 19:44:32 +01:00
|
|
|
metainfo *metainfo.Service
|
|
|
|
metaLoop *metainfo.Loop
|
|
|
|
nodestate *ReliabilityCache
|
2019-05-22 22:17:52 +01:00
|
|
|
Loop sync2.Cycle
|
2019-05-30 16:18:20 +01:00
|
|
|
IrreparableLoop sync2.Cycle
|
2018-10-09 17:09:33 +01:00
|
|
|
}
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
// NewChecker creates a new instance of checker
|
2019-08-06 17:35:59 +01:00
|
|
|
func NewChecker(logger *zap.Logger, repairQueue queue.RepairQueue, irrdb irreparable.DB, metainfo *metainfo.Service, metaLoop *metainfo.Loop, overlay *overlay.Service, config Config) *Checker {
|
2019-07-08 23:04:35 +01:00
|
|
|
return &Checker{
|
2019-08-01 19:44:32 +01:00
|
|
|
logger: logger,
|
|
|
|
|
|
|
|
repairQueue: repairQueue,
|
|
|
|
irrdb: irrdb,
|
|
|
|
metainfo: metainfo,
|
|
|
|
metaLoop: metaLoop,
|
|
|
|
nodestate: NewReliabilityCache(overlay, config.ReliabilityCacheStaleness),
|
|
|
|
|
2019-07-08 23:04:35 +01:00
|
|
|
Loop: *sync2.NewCycle(config.Interval),
|
|
|
|
IrreparableLoop: *sync2.NewCycle(config.IrreparableInterval),
|
2018-10-09 17:09:33 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-11-01 14:03:45 +00:00
|
|
|
// Run the checker loop
|
2019-02-11 21:06:39 +00:00
|
|
|
func (checker *Checker) Run(ctx context.Context) (err error) {
|
2018-11-01 14:03:45 +00:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
|
|
|
|
2019-05-31 15:12:49 +01:00
|
|
|
group, ctx := errgroup.WithContext(ctx)
|
2019-05-30 16:18:20 +01:00
|
|
|
|
2019-05-31 15:12:49 +01:00
|
|
|
group.Go(func() error {
|
|
|
|
return checker.Loop.Run(ctx, checker.IdentifyInjuredSegments)
|
|
|
|
})
|
2019-05-30 16:18:20 +01:00
|
|
|
|
2019-05-31 15:12:49 +01:00
|
|
|
group.Go(func() error {
|
|
|
|
return checker.IrreparableLoop.Run(ctx, checker.IrreparableProcess)
|
|
|
|
})
|
2019-05-30 16:18:20 +01:00
|
|
|
|
2019-05-31 15:12:49 +01:00
|
|
|
return group.Wait()
|
2018-11-01 14:03:45 +00:00
|
|
|
}
|
|
|
|
|
2019-07-08 23:04:35 +01:00
|
|
|
// RefreshReliabilityCache forces refreshing node online status cache.
|
|
|
|
func (checker *Checker) RefreshReliabilityCache(ctx context.Context) error {
|
|
|
|
return checker.nodestate.Refresh(ctx)
|
|
|
|
}
|
|
|
|
|
2019-02-14 12:33:41 +00:00
|
|
|
// Close halts the Checker loop
|
|
|
|
func (checker *Checker) Close() error {
|
|
|
|
checker.Loop.Close()
|
|
|
|
return nil
|
|
|
|
}
|
2019-01-18 13:54:08 +00:00
|
|
|
|
2019-08-06 17:35:59 +01:00
|
|
|
// IdentifyInjuredSegments checks for missing pieces off of the metainfo and overlay.
|
2019-02-11 21:06:39 +00:00
|
|
|
func (checker *Checker) IdentifyInjuredSegments(ctx context.Context) (err error) {
|
2018-10-09 17:09:33 +01:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
observer := &checkerObserver{
|
|
|
|
repairQueue: checker.repairQueue,
|
|
|
|
irrdb: checker.irrdb,
|
|
|
|
nodestate: checker.nodestate,
|
|
|
|
monStats: durabilityStats{},
|
|
|
|
log: checker.logger,
|
|
|
|
}
|
|
|
|
err = checker.metaLoop.Join(ctx, observer)
|
2019-02-26 15:17:51 +00:00
|
|
|
if err != nil {
|
2019-08-01 19:44:32 +01:00
|
|
|
if !errs2.IsCanceled(err) {
|
|
|
|
checker.logger.Error("IdentifyInjuredSegments error", zap.Error(err))
|
|
|
|
}
|
2019-02-26 15:17:51 +00:00
|
|
|
return err
|
|
|
|
}
|
|
|
|
|
2019-09-13 14:51:41 +01:00
|
|
|
mon.IntVal("remote_files_checked").Observe(observer.monStats.objectsChecked)
|
2019-08-01 19:44:32 +01:00
|
|
|
mon.IntVal("remote_segments_checked").Observe(observer.monStats.remoteSegmentsChecked)
|
|
|
|
mon.IntVal("remote_segments_needing_repair").Observe(observer.monStats.remoteSegmentsNeedingRepair)
|
|
|
|
mon.IntVal("remote_segments_lost").Observe(observer.monStats.remoteSegmentsLost)
|
|
|
|
mon.IntVal("remote_files_lost").Observe(int64(len(observer.monStats.remoteSegmentInfo)))
|
|
|
|
|
2019-02-26 15:17:51 +00:00
|
|
|
return nil
|
2018-10-09 17:09:33 +01:00
|
|
|
}
|
|
|
|
|
2019-02-26 15:17:51 +00:00
|
|
|
// checks for a string in slice
|
|
|
|
func contains(a []string, x string) bool {
|
|
|
|
for _, n := range a {
|
|
|
|
if x == n {
|
|
|
|
return true
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return false
|
|
|
|
}
|
2019-05-30 16:18:20 +01:00
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
func (checker *Checker) updateIrreparableSegmentStatus(ctx context.Context, pointer *pb.Pointer, path string) (err error) {
|
|
|
|
// TODO figure out how to reduce duplicate code between here and checkerObs.RemoteSegment
|
2019-06-04 12:36:27 +01:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
2019-05-30 16:18:20 +01:00
|
|
|
remote := pointer.GetRemote()
|
|
|
|
if remote == nil {
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
pieces := remote.GetRemotePieces()
|
|
|
|
if pieces == nil {
|
|
|
|
checker.logger.Debug("no pieces on remote segment")
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-07-08 23:16:50 +01:00
|
|
|
missingPieces, err := checker.nodestate.MissingPieces(ctx, pointer.CreationDate, pieces)
|
2019-05-30 16:18:20 +01:00
|
|
|
if err != nil {
|
2019-08-01 19:44:32 +01:00
|
|
|
return errs.Combine(Error.New("error getting missing pieces"), err)
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
numHealthy := int32(len(pieces) - len(missingPieces))
|
|
|
|
redundancy := pointer.Remote.Redundancy
|
2019-07-10 22:27:46 +01:00
|
|
|
|
2019-09-06 20:20:36 +01:00
|
|
|
// we repair when the number of healthy pieces is less than or equal to the repair threshold and is greater or equal to
|
|
|
|
// minimum required pieces in redundancy
|
2019-05-30 16:18:20 +01:00
|
|
|
// except for the case when the repair and success thresholds are the same (a case usually seen during testing)
|
2019-09-06 20:20:36 +01:00
|
|
|
if numHealthy >= redundancy.MinReq && numHealthy <= redundancy.RepairThreshold && numHealthy < redundancy.SuccessThreshold {
|
2019-05-30 16:18:20 +01:00
|
|
|
if len(missingPieces) == 0 {
|
2019-08-01 19:44:32 +01:00
|
|
|
checker.logger.Error("Missing pieces is zero in checker, but this should be impossible -- bad redundancy scheme:",
|
|
|
|
zap.String("path", path),
|
|
|
|
zap.Int32("min", redundancy.MinReq),
|
|
|
|
zap.Int32("repair", redundancy.RepairThreshold),
|
|
|
|
zap.Int32("success", redundancy.SuccessThreshold),
|
|
|
|
zap.Int32("total", redundancy.Total))
|
2019-05-30 16:18:20 +01:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
err = checker.repairQueue.Insert(ctx, &pb.InjuredSegment{
|
2019-07-11 18:26:07 +01:00
|
|
|
Path: []byte(path),
|
2019-07-10 22:27:46 +01:00
|
|
|
LostPieces: missingPieces,
|
|
|
|
InsertedTime: time.Now().UTC(),
|
2019-05-30 16:18:20 +01:00
|
|
|
})
|
|
|
|
if err != nil {
|
2019-08-01 19:44:32 +01:00
|
|
|
return errs.Combine(Error.New("error adding injured segment to queue"), err)
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
// delete always returns nil when something was deleted and also when element didn't exists
|
|
|
|
err = checker.irrdb.Delete(ctx, []byte(path))
|
|
|
|
if err != nil {
|
|
|
|
checker.logger.Error("error deleting entry from irreparable db: ", zap.Error(err))
|
|
|
|
}
|
2019-09-06 20:20:36 +01:00
|
|
|
} else if numHealthy < redundancy.MinReq && numHealthy < redundancy.RepairThreshold {
|
2019-08-01 19:44:32 +01:00
|
|
|
|
|
|
|
// make an entry into the irreparable table
|
|
|
|
segmentInfo := &pb.IrreparableSegment{
|
|
|
|
Path: []byte(path),
|
|
|
|
SegmentDetail: pointer,
|
|
|
|
LostPieces: int32(len(missingPieces)),
|
|
|
|
LastRepairAttempt: time.Now().Unix(),
|
|
|
|
RepairAttemptCount: int64(1),
|
|
|
|
}
|
|
|
|
|
|
|
|
// add the entry if new or update attempt count if already exists
|
|
|
|
err := checker.irrdb.IncrementRepairAttempts(ctx, segmentInfo)
|
|
|
|
if err != nil {
|
|
|
|
return errs.Combine(Error.New("error handling irreparable segment to queue"), err)
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-09-10 14:24:16 +01:00
|
|
|
var _ metainfo.Observer = (*checkerObserver)(nil)
|
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
// checkerObserver implements the metainfo loop Observer interface
|
2019-09-10 14:24:16 +01:00
|
|
|
//
|
|
|
|
// architecture: Observer
|
2019-08-01 19:44:32 +01:00
|
|
|
type checkerObserver struct {
|
|
|
|
repairQueue queue.RepairQueue
|
|
|
|
irrdb irreparable.DB
|
|
|
|
nodestate *ReliabilityCache
|
|
|
|
monStats durabilityStats
|
|
|
|
log *zap.Logger
|
|
|
|
}
|
|
|
|
|
2019-09-12 11:38:49 +01:00
|
|
|
func (obs *checkerObserver) RemoteSegment(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
|
2019-08-01 19:44:32 +01:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
|
|
|
|
|
|
|
obs.monStats.remoteSegmentsChecked++
|
|
|
|
remote := pointer.GetRemote()
|
|
|
|
|
|
|
|
pieces := remote.GetRemotePieces()
|
|
|
|
if pieces == nil {
|
|
|
|
obs.log.Debug("no pieces on remote segment")
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
missingPieces, err := obs.nodestate.MissingPieces(ctx, pointer.CreationDate, pieces)
|
|
|
|
if err != nil {
|
|
|
|
return errs.Combine(Error.New("error getting missing pieces"), err)
|
|
|
|
}
|
|
|
|
|
|
|
|
numHealthy := int32(len(pieces) - len(missingPieces))
|
|
|
|
mon.IntVal("checker_segment_total_count").Observe(int64(len(pieces)))
|
|
|
|
mon.IntVal("checker_segment_healthy_count").Observe(int64(numHealthy))
|
|
|
|
|
2019-09-17 20:18:48 +01:00
|
|
|
segmentAge := time.Since(pointer.CreationDate)
|
|
|
|
mon.IntVal("checker_segment_age").Observe(int64(segmentAge.Seconds()))
|
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
redundancy := pointer.Remote.Redundancy
|
|
|
|
|
2019-09-06 20:20:36 +01:00
|
|
|
// we repair when the number of healthy pieces is less than or equal to the repair threshold and is greater or equal to
|
|
|
|
// minimum required pieces in redundancy
|
2019-08-01 19:44:32 +01:00
|
|
|
// except for the case when the repair and success thresholds are the same (a case usually seen during testing)
|
2019-09-06 20:20:36 +01:00
|
|
|
if numHealthy >= redundancy.MinReq && numHealthy <= redundancy.RepairThreshold && numHealthy < redundancy.SuccessThreshold {
|
2019-08-01 19:44:32 +01:00
|
|
|
if len(missingPieces) == 0 {
|
|
|
|
obs.log.Error("Missing pieces is zero in checker, but this should be impossible -- bad redundancy scheme:",
|
2019-09-12 11:38:49 +01:00
|
|
|
zap.String("path", path.Raw),
|
2019-08-01 19:44:32 +01:00
|
|
|
zap.Int32("min", redundancy.MinReq),
|
|
|
|
zap.Int32("repair", redundancy.RepairThreshold),
|
|
|
|
zap.Int32("success", redundancy.SuccessThreshold),
|
|
|
|
zap.Int32("total", redundancy.Total))
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
obs.monStats.remoteSegmentsNeedingRepair++
|
|
|
|
err = obs.repairQueue.Insert(ctx, &pb.InjuredSegment{
|
2019-09-12 11:38:49 +01:00
|
|
|
Path: []byte(path.Raw),
|
2019-08-01 19:44:32 +01:00
|
|
|
LostPieces: missingPieces,
|
|
|
|
InsertedTime: time.Now().UTC(),
|
|
|
|
})
|
|
|
|
if err != nil {
|
|
|
|
obs.log.Error("error adding injured segment to queue", zap.Error(err))
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// delete always returns nil when something was deleted and also when element didn't exists
|
2019-09-12 11:38:49 +01:00
|
|
|
err = obs.irrdb.Delete(ctx, []byte(path.Raw))
|
2019-08-01 19:44:32 +01:00
|
|
|
if err != nil {
|
|
|
|
obs.log.Error("error deleting entry from irreparable db", zap.Error(err))
|
|
|
|
return nil
|
|
|
|
}
|
2019-09-06 20:20:36 +01:00
|
|
|
} else if numHealthy < redundancy.MinReq && numHealthy < redundancy.RepairThreshold {
|
2019-09-12 11:38:49 +01:00
|
|
|
// TODO: see whether this can be handled with metainfo.ScopedPath
|
|
|
|
pathElements := storj.SplitPath(path.Raw)
|
2019-08-01 19:44:32 +01:00
|
|
|
|
2019-05-30 16:18:20 +01:00
|
|
|
// check to make sure there are at least *4* path elements. the first three
|
|
|
|
// are project, segment, and bucket name, but we want to make sure we're talking
|
|
|
|
// about an actual object, and that there's an object name specified
|
|
|
|
if len(pathElements) >= 4 {
|
|
|
|
project, bucketName, segmentpath := pathElements[0], pathElements[2], pathElements[3]
|
2019-09-12 11:38:49 +01:00
|
|
|
|
|
|
|
// TODO: is this correct? split splits all path components, but it's only using the third.
|
2019-05-30 16:18:20 +01:00
|
|
|
lostSegInfo := storj.JoinPaths(project, bucketName, segmentpath)
|
2019-08-22 12:40:15 +01:00
|
|
|
if !contains(obs.monStats.remoteSegmentInfo, lostSegInfo) {
|
2019-08-01 19:44:32 +01:00
|
|
|
obs.monStats.remoteSegmentInfo = append(obs.monStats.remoteSegmentInfo, lostSegInfo)
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2019-09-17 20:18:48 +01:00
|
|
|
var segmentAge time.Duration
|
|
|
|
if pointer.CreationDate.Before(pointer.LastRepaired) {
|
|
|
|
segmentAge = time.Since(pointer.LastRepaired)
|
|
|
|
} else {
|
|
|
|
segmentAge = time.Since(pointer.CreationDate)
|
|
|
|
}
|
|
|
|
mon.IntVal("checker_segment_time_until_irreparable").Observe(int64(segmentAge.Seconds()))
|
|
|
|
|
2019-08-01 19:44:32 +01:00
|
|
|
obs.monStats.remoteSegmentsLost++
|
|
|
|
// make an entry into the irreparable table
|
2019-05-30 16:18:20 +01:00
|
|
|
segmentInfo := &pb.IrreparableSegment{
|
2019-09-12 11:38:49 +01:00
|
|
|
Path: []byte(path.Raw),
|
2019-05-30 16:18:20 +01:00
|
|
|
SegmentDetail: pointer,
|
|
|
|
LostPieces: int32(len(missingPieces)),
|
|
|
|
LastRepairAttempt: time.Now().Unix(),
|
|
|
|
RepairAttemptCount: int64(1),
|
|
|
|
}
|
|
|
|
|
|
|
|
// add the entry if new or update attempt count if already exists
|
2019-08-01 19:44:32 +01:00
|
|
|
err := obs.irrdb.IncrementRepairAttempts(ctx, segmentInfo)
|
2019-05-30 16:18:20 +01:00
|
|
|
if err != nil {
|
2019-08-01 19:44:32 +01:00
|
|
|
obs.log.Error("error handling irreparable segment to queue", zap.Error(err))
|
|
|
|
return nil
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
}
|
2019-08-01 19:44:32 +01:00
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-09-13 14:51:41 +01:00
|
|
|
func (obs *checkerObserver) Object(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
|
2019-08-01 19:44:32 +01:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
|
|
|
|
2019-09-13 14:51:41 +01:00
|
|
|
obs.monStats.objectsChecked++
|
2019-08-01 19:44:32 +01:00
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-09-12 11:38:49 +01:00
|
|
|
func (obs *checkerObserver) InlineSegment(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
|
2019-08-01 19:44:32 +01:00
|
|
|
defer mon.Task()(&ctx)(&err)
|
2019-05-30 16:18:20 +01:00
|
|
|
return nil
|
|
|
|
}
|
|
|
|
|
2019-07-18 17:21:21 +01:00
|
|
|
// IrreparableProcess iterates over all items in the irreparabledb. If an item can
|
|
|
|
// now be repaired then it is added to a worker queue.
|
2019-05-30 16:18:20 +01:00
|
|
|
func (checker *Checker) IrreparableProcess(ctx context.Context) (err error) {
|
|
|
|
defer mon.Task()(&ctx)(&err)
|
2019-07-18 17:21:21 +01:00
|
|
|
const limit = 1000
|
|
|
|
lastSeenSegmentPath := []byte{}
|
2019-05-30 16:18:20 +01:00
|
|
|
|
|
|
|
for {
|
2019-07-18 17:21:21 +01:00
|
|
|
segments, err := checker.irrdb.GetLimited(ctx, limit, lastSeenSegmentPath)
|
2019-05-30 16:18:20 +01:00
|
|
|
if err != nil {
|
2019-08-01 19:44:32 +01:00
|
|
|
return errs.Combine(Error.New("error reading segment from the queue"), err)
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
|
|
|
|
// zero segments returned with nil err
|
2019-07-18 17:21:21 +01:00
|
|
|
if len(segments) == 0 {
|
2019-05-30 16:18:20 +01:00
|
|
|
break
|
|
|
|
}
|
|
|
|
|
2019-07-18 17:21:21 +01:00
|
|
|
lastSeenSegmentPath = segments[len(segments)-1].Path
|
|
|
|
|
|
|
|
for _, segment := range segments {
|
2019-08-01 19:44:32 +01:00
|
|
|
err = checker.updateIrreparableSegmentStatus(ctx, segment.GetSegmentDetail(), string(segment.GetPath()))
|
2019-07-18 17:21:21 +01:00
|
|
|
if err != nil {
|
|
|
|
checker.logger.Error("irrepair segment checker failed: ", zap.Error(err))
|
|
|
|
}
|
2019-05-30 16:18:20 +01:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return nil
|
|
|
|
}
|