storj/satellite/repair/checker/checker.go

400 lines
14 KiB
Go
Raw Normal View History

2019-01-24 20:15:10 +00:00
// Copyright (C) 2019 Storj Labs, Inc.
// See LICENSE for copying information.
package checker
import (
"context"
"time"
"github.com/spacemonkeygo/monkit/v3"
2019-01-23 19:58:44 +00:00
"github.com/zeebo/errs"
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"storj.io/common/errs2"
"storj.io/common/pb"
"storj.io/common/sync2"
"storj.io/storj/satellite/metainfo"
"storj.io/storj/satellite/metainfo/metabase"
"storj.io/storj/satellite/overlay"
"storj.io/storj/satellite/repair/irreparable"
"storj.io/storj/satellite/repair/queue"
)
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.
2019-01-23 19:58:44 +00:00
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"`
IrreparableInterval time.Duration `help:"how frequently irrepairable checker should check for lost pieces" releaseDefault:"30m" devDefault:"0h0m5s"`
ReliabilityCacheStaleness time.Duration `help:"how stale reliable node cache can be" releaseDefault:"5m" devDefault:"5m"`
RepairOverride int `help:"override value for repair threshold" default:"0"`
}
// durabilityStats remote segment information.
type durabilityStats struct {
objectsChecked int64
remoteSegmentsChecked int64
remoteSegmentsNeedingRepair int64
newRemoteSegmentsNeedingRepair int64
remoteSegmentsLost int64
remoteSegmentsFailedToCheck int64
remoteSegmentInfo []metabase.ObjectLocation
// remoteSegmentsOverThreshold[0]=# of healthy=rt+1, remoteSegmentsOverThreshold[1]=# of healthy=rt+2, etc...
remoteSegmentsOverThreshold [5]int64
2019-01-23 19:58:44 +00:00
}
// Checker contains the information needed to do checks for missing pieces
2019-09-10 14:24:16 +01:00
//
// architecture: Chore
type Checker struct {
logger *zap.Logger
repairQueue queue.RepairQueue
irrdb irreparable.DB
metainfo *metainfo.Service
metaLoop *metainfo.Loop
nodestate *ReliabilityCache
repairOverride int32
Loop *sync2.Cycle
IrreparableLoop *sync2.Cycle
}
// NewChecker creates a new instance of checker.
func NewChecker(logger *zap.Logger, repairQueue queue.RepairQueue, irrdb irreparable.DB, metainfo *metainfo.Service, metaLoop *metainfo.Loop, overlay *overlay.Service, config Config) *Checker {
return &Checker{
logger: logger,
repairQueue: repairQueue,
irrdb: irrdb,
metainfo: metainfo,
metaLoop: metaLoop,
nodestate: NewReliabilityCache(overlay, config.ReliabilityCacheStaleness),
repairOverride: int32(config.RepairOverride),
Loop: sync2.NewCycle(config.Interval),
IrreparableLoop: sync2.NewCycle(config.IrreparableInterval),
}
}
// Run the checker loop.
func (checker *Checker) Run(ctx context.Context) (err error) {
defer mon.Task()(&ctx)(&err)
group, ctx := errgroup.WithContext(ctx)
group.Go(func() error {
return checker.Loop.Run(ctx, checker.IdentifyInjuredSegments)
})
group.Go(func() error {
return checker.IrreparableLoop.Run(ctx, checker.IrreparableProcess)
})
return group.Wait()
}
// RefreshReliabilityCache forces refreshing node online status cache.
func (checker *Checker) RefreshReliabilityCache(ctx context.Context) error {
return checker.nodestate.Refresh(ctx)
}
// Close halts the Checker loop.
func (checker *Checker) Close() error {
checker.Loop.Close()
return nil
}
Satellite Peer (#1034) * add satellite peer * Add overlay * reorganize kademlia * add RunRefresh * add refresh to storagenode.Peer * add discovery * add agreements and metainfo * rename * add datarepair checker * add repair * add todo notes for audit * add testing interface * add into testplanet * fixes * fix compilation errors * fix compilation errors * make testplanet run * remove audit refrences * ensure that audit tests run * dev * checker tests compilable * fix discovery * fix compilation * fix * fix * dev * fix * disable auth * fixes * revert go.mod/sum * fix linter errors * fix * fix copyright * Add address param for SN dashboard (#1076) * Rename storj-sdk to storj-sim (#1078) * Storagenode logs and config improvements (#1075) * Add more info to SN logs * remove config-dir from user config * add output where config was stored * add message for successful connection * fix linter * remove storage.path from user config * resolve config path * move success message to info * log improvements * Remove captplanet (#1070) * pkg/server: include production cert (#1082) Change-Id: Ie8e6fe78550be83c3bd797db7a1e58d37c684792 * Generate Payments Report (#1079) * memory.Size: autoformat sizes based on value entropy (#1081) * Jj/bytes (#1085) * run tally and rollup * sets dev default tally and rollup intervals * nonessential storj-sim edits (#1086) * Closing context doesn't stop storage node (#1084) * Print when cancelled * Close properly * Don't log nil * Don't print error when closing dashboard * Fix panic in inspector if ping fails (#1088) * Consolidate identity management to identity cli commands (#1083) * Consolidate identity management: Move identity cretaion/signing out of storagenode setup command. * fixes * linters * Consolidate identity management: Move identity cretaion/signing out of storagenode setup command. * fixes * sava backups before saving signed certs * add "-prebuilt-test-cmds" test flag * linters * prepare cli tests for travis * linter fixes * more fixes * linter gods * sp/sdk/sim * remove ca.difficulty * remove unused difficulty * return setup to its rightful place * wip travis * Revert "wip travis" This reverts commit 56834849dcf066d3cc0a4f139033fc3f6d7188ca. * typo in travis.yaml * remove tests * remove more * make it only create one identity at a time for consistency * add config-dir for consitency * add identity creation to storj-sim * add flags * simplify * fix nolint and compile * prevent overwrite and pass difficulty, concurrency, and parent creds * goimports
2019-01-18 13:54:08 +00:00
// IdentifyInjuredSegments checks for missing pieces off of the metainfo and overlay.
func (checker *Checker) IdentifyInjuredSegments(ctx context.Context) (err error) {
defer mon.Task()(&ctx)(&err)
observer := &checkerObserver{
repairQueue: checker.repairQueue,
irrdb: checker.irrdb,
nodestate: checker.nodestate,
monStats: durabilityStats{},
overrideRepair: checker.repairOverride,
log: checker.logger,
}
err = checker.metaLoop.Join(ctx, observer)
if err != nil {
if !errs2.IsCanceled(err) {
checker.logger.Error("IdentifyInjuredSegments error", zap.Error(err))
}
return err
}
mon.IntVal("remote_files_checked").Observe(observer.monStats.objectsChecked) //locked
mon.IntVal("remote_segments_checked").Observe(observer.monStats.remoteSegmentsChecked) //locked
mon.IntVal("remote_segments_failed_to_check").Observe(observer.monStats.remoteSegmentsFailedToCheck) //locked
mon.IntVal("remote_segments_needing_repair").Observe(observer.monStats.remoteSegmentsNeedingRepair) //locked
mon.IntVal("new_remote_segments_needing_repair").Observe(observer.monStats.newRemoteSegmentsNeedingRepair) //locked
mon.IntVal("remote_segments_lost").Observe(observer.monStats.remoteSegmentsLost) //locked
mon.IntVal("remote_files_lost").Observe(int64(len(observer.monStats.remoteSegmentInfo))) //locked
mon.IntVal("remote_segments_over_threshold_1").Observe(observer.monStats.remoteSegmentsOverThreshold[0]) //locked
mon.IntVal("remote_segments_over_threshold_2").Observe(observer.monStats.remoteSegmentsOverThreshold[1]) //locked
mon.IntVal("remote_segments_over_threshold_3").Observe(observer.monStats.remoteSegmentsOverThreshold[2]) //locked
mon.IntVal("remote_segments_over_threshold_4").Observe(observer.monStats.remoteSegmentsOverThreshold[3]) //locked
mon.IntVal("remote_segments_over_threshold_5").Observe(observer.monStats.remoteSegmentsOverThreshold[4]) //locked
allUnhealthy := observer.monStats.remoteSegmentsNeedingRepair + observer.monStats.remoteSegmentsFailedToCheck
allChecked := observer.monStats.remoteSegmentsChecked
allHealthy := allChecked - allUnhealthy
mon.FloatVal("remote_segments_healthy_percentage").Observe(100 * float64(allHealthy) / float64(allChecked)) //locked
return nil
}
// checks for a object location in slice.
func containsObjectLocation(a []metabase.ObjectLocation, x metabase.ObjectLocation) bool {
for _, n := range a {
if x == n {
return true
}
}
return false
}
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
defer mon.Task()(&ctx)(&err)
remote := pointer.GetRemote()
if pointer.GetType() == pb.Pointer_INLINE || remote == nil {
return nil
}
pieces := remote.GetRemotePieces()
if pieces == nil {
checker.logger.Debug("no pieces on remote segment")
return nil
}
missingPieces, err := checker.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))
redundancy := pointer.Remote.Redundancy
repairThreshold := redundancy.RepairThreshold
if checker.repairOverride != 0 {
repairThreshold = checker.repairOverride
}
// 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
// except for the case when the repair and success thresholds are the same (a case usually seen during testing)
//
// If the segment is suddenly entirely healthy again, we don't need to repair and we don't need to
// keep it in the irreparabledb queue either.
if numHealthy >= redundancy.MinReq && numHealthy <= repairThreshold && numHealthy < redundancy.SuccessThreshold {
_, err = checker.repairQueue.Insert(ctx, &pb.InjuredSegment{
Path: []byte(path),
LostPieces: missingPieces,
InsertedTime: time.Now().UTC(),
}, int(numHealthy))
if err != nil {
return errs.Combine(Error.New("error adding injured segment to queue"), err)
}
// 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))
}
} else if numHealthy < redundancy.MinReq && numHealthy < repairThreshold {
// 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)
}
} else if numHealthy > repairThreshold || numHealthy >= redundancy.SuccessThreshold {
err = checker.irrdb.Delete(ctx, []byte(path))
if err != nil {
return Error.New("error removing segment from irreparable queue: %v", err)
}
}
return nil
}
2019-09-10 14:24:16 +01:00
var _ metainfo.Observer = (*checkerObserver)(nil)
// checkerObserver implements the metainfo loop Observer interface
2019-09-10 14:24:16 +01:00
//
// architecture: Observer
type checkerObserver struct {
repairQueue queue.RepairQueue
irrdb irreparable.DB
nodestate *ReliabilityCache
monStats durabilityStats
overrideRepair int32
log *zap.Logger
}
func (obs *checkerObserver) RemoteSegment(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
defer mon.Task()(&ctx)(&err)
// ignore pointer if expired
if !pointer.ExpirationDate.IsZero() && pointer.ExpirationDate.Before(time.Now().UTC()) {
return nil
}
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 {
obs.monStats.remoteSegmentsFailedToCheck++
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))) //locked
mon.IntVal("checker_segment_healthy_count").Observe(int64(numHealthy)) //locked
segmentAge := time.Since(pointer.CreationDate)
mon.IntVal("checker_segment_age").Observe(int64(segmentAge.Seconds())) //locked
redundancy := pointer.Remote.Redundancy
repairThreshold := redundancy.RepairThreshold
if obs.overrideRepair != 0 {
repairThreshold = obs.overrideRepair
}
// 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
// except for the case when the repair and success thresholds are the same (a case usually seen during testing)
if numHealthy >= redundancy.MinReq && numHealthy <= repairThreshold && numHealthy < redundancy.SuccessThreshold {
obs.monStats.remoteSegmentsNeedingRepair++
alreadyInserted, err := obs.repairQueue.Insert(ctx, &pb.InjuredSegment{
Path: []byte(path.Raw),
LostPieces: missingPieces,
InsertedTime: time.Now().UTC(),
}, int(numHealthy))
if err != nil {
obs.log.Error("error adding injured segment to queue", zap.Error(err))
return nil
}
if !alreadyInserted {
obs.monStats.newRemoteSegmentsNeedingRepair++
}
// delete always returns nil when something was deleted and also when element didn't exists
err = obs.irrdb.Delete(ctx, []byte(path.Raw))
if err != nil {
obs.log.Error("error deleting entry from irreparable db", zap.Error(err))
return nil
}
} else if numHealthy < redundancy.MinReq && numHealthy < repairThreshold {
lostSegInfo := path.Object()
if !containsObjectLocation(obs.monStats.remoteSegmentInfo, lostSegInfo) {
obs.monStats.remoteSegmentInfo = append(obs.monStats.remoteSegmentInfo, lostSegInfo)
}
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())) //locked
obs.monStats.remoteSegmentsLost++
// make an entry into the irreparable table
segmentInfo := &pb.IrreparableSegment{
Path: []byte(path.Raw),
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 := obs.irrdb.IncrementRepairAttempts(ctx, segmentInfo)
if err != nil {
obs.log.Error("error handling irreparable segment to queue", zap.Error(err))
return nil
}
} else if numHealthy > repairThreshold && numHealthy <= (repairThreshold+int32(len(obs.monStats.remoteSegmentsOverThreshold))) {
// record metrics for segments right above repair threshold
// numHealthy=repairThreshold+1 through numHealthy=repairThreshold+5
for i := range obs.monStats.remoteSegmentsOverThreshold {
if numHealthy == (repairThreshold + int32(i) + 1) {
obs.monStats.remoteSegmentsOverThreshold[i]++
break
}
}
}
return nil
}
func (obs *checkerObserver) Object(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
defer mon.Task()(&ctx)(&err)
obs.monStats.objectsChecked++
return nil
}
func (obs *checkerObserver) InlineSegment(ctx context.Context, path metainfo.ScopedPath, pointer *pb.Pointer) (err error) {
defer mon.Task()(&ctx)(&err)
return nil
}
// IrreparableProcess iterates over all items in the irreparabledb. If an item can
// now be repaired then it is added to a worker queue.
func (checker *Checker) IrreparableProcess(ctx context.Context) (err error) {
defer mon.Task()(&ctx)(&err)
const limit = 1000
lastSeenSegmentPath := []byte{}
for {
segments, err := checker.irrdb.GetLimited(ctx, limit, lastSeenSegmentPath)
if err != nil {
return errs.Combine(Error.New("error reading segment from the queue"), err)
}
// zero segments returned with nil err
if len(segments) == 0 {
break
}
lastSeenSegmentPath = segments[len(segments)-1].Path
for _, segment := range segments {
err = checker.updateIrreparableSegmentStatus(ctx, segment.GetSegmentDetail(), string(segment.GetPath()))
if err != nil {
checker.logger.Error("irrepair segment checker failed: ", zap.Error(err))
}
}
}
return nil
}