X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/74df5a58360fe6bcb273480d3ddec507a53b6b2b..09cbdc3074b3f1e69c9c537875146f6da0a6ed8f:/services/keep-balance/balance.go diff --git a/services/keep-balance/balance.go b/services/keep-balance/balance.go index 836be2e608..9f581751d9 100644 --- a/services/keep-balance/balance.go +++ b/services/keep-balance/balance.go @@ -2,22 +2,32 @@ // // SPDX-License-Identifier: AGPL-3.0 -package main +package keepbalance import ( "bytes" + "context" "crypto/md5" + "errors" "fmt" + "io" + "io/ioutil" "log" "math" + "os" "runtime" "sort" "strings" "sync" + "sync/atomic" + "syscall" "time" - "git.curoverse.com/arvados.git/sdk/go/arvados" - "git.curoverse.com/arvados.git/sdk/go/keepclient" + "git.arvados.org/arvados.git/lib/controller/dblock" + "git.arvados.org/arvados.git/sdk/go/arvados" + "git.arvados.org/arvados.git/sdk/go/ctxlog" + "git.arvados.org/arvados.git/sdk/go/keepclient" + "github.com/jmoiron/sqlx" "github.com/sirupsen/logrus" ) @@ -31,10 +41,13 @@ import ( // BlobSignatureTTL; and all N existing replicas of a given data block // are in the N best positions in rendezvous probe order. type Balancer struct { - Logger *logrus.Logger - Dumper *logrus.Logger + DB *sqlx.DB + Logger logrus.FieldLogger + Dumper logrus.FieldLogger Metrics *metrics + LostBlocksFile string + *BlockStateMap KeepServices map[string]*KeepService DefaultReplication int @@ -43,11 +56,12 @@ type Balancer struct { classes []string mounts int mountsByClass map[string]map[*KeepMount]bool - collScanned int + collScanned int64 serviceRoots map[string]string errors []error stats balancerStats mutex sync.Mutex + lostBlocks io.Writer } // Run performs a balance operation using the given config and @@ -55,42 +69,76 @@ type Balancer struct { // subsequent balance operation. // // Run should only be called once on a given Balancer object. -// -// Typical usage: -// -// runOptions, err = (&Balancer{}).Run(config, runOptions) -func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions RunOptions, err error) { +func (bal *Balancer) Run(ctx context.Context, client *arvados.Client, cluster *arvados.Cluster, runOptions RunOptions) (nextRunOptions RunOptions, err error) { nextRunOptions = runOptions + ctxlog.FromContext(ctx).Info("acquiring active lock") + if !dblock.KeepBalanceActive.Lock(ctx, func(context.Context) (*sqlx.DB, error) { return bal.DB, nil }) { + // context canceled + return + } + defer dblock.KeepBalanceActive.Unlock() + defer bal.time("sweep", "wall clock time to run one full sweep")() - if len(config.KeepServiceList.Items) > 0 { - err = bal.SetKeepServices(config.KeepServiceList) + ctx, cancel := context.WithDeadline(ctx, time.Now().Add(cluster.Collections.BalanceTimeout.Duration())) + defer cancel() + + var lbFile *os.File + if bal.LostBlocksFile != "" { + tmpfn := bal.LostBlocksFile + ".tmp" + lbFile, err = os.OpenFile(tmpfn, os.O_CREATE|os.O_WRONLY, 0777) + if err != nil { + return + } + defer lbFile.Close() + err = syscall.Flock(int(lbFile.Fd()), syscall.LOCK_EX|syscall.LOCK_NB) + if err != nil { + return + } + defer func() { + // Remove the tempfile only if we didn't get + // as far as successfully renaming it. + if lbFile != nil { + os.Remove(tmpfn) + } + }() + bal.lostBlocks = lbFile } else { - err = bal.DiscoverKeepServices(&config.Client, config.KeepServiceTypes) + bal.lostBlocks = ioutil.Discard } + + err = bal.DiscoverKeepServices(client) if err != nil { return } for _, srv := range bal.KeepServices { - err = srv.discoverMounts(&config.Client) + err = srv.discoverMounts(client) if err != nil { return } } bal.cleanupMounts() - if err = bal.CheckSanityEarly(&config.Client); err != nil { + if err = bal.CheckSanityEarly(client); err != nil { return } + + // On a big site, indexing and sending trash/pull lists can + // take much longer than the usual 5 minute client + // timeout. From here on, we rely on the context deadline + // instead, aborting the entire operation if any part takes + // too long. + client.Timeout = 0 + rs := bal.rendezvousState() if runOptions.CommitTrash && rs != runOptions.SafeRendezvousState { if runOptions.SafeRendezvousState != "" { bal.logf("notice: KeepServices list has changed since last run") } bal.logf("clearing existing trash lists, in case the new rendezvous order differs from previous run") - if err = bal.ClearTrashLists(&config.Client); err != nil { + if err = bal.ClearTrashLists(ctx, client); err != nil { return } // The current rendezvous state becomes "safe" (i.e., @@ -99,7 +147,8 @@ func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions R // succeed in clearing existing trash lists. nextRunOptions.SafeRendezvousState = rs } - if err = bal.GetCurrentState(&config.Client, config.CollectionBatchSize, config.CollectionBuffers); err != nil { + + if err = bal.GetCurrentState(ctx, client, cluster.Collections.BalanceCollectionBatch, cluster.Collections.BalanceCollectionBuffers); err != nil { return } bal.ComputeChangeSets() @@ -107,15 +156,35 @@ func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions R if err = bal.CheckSanityLate(); err != nil { return } + if lbFile != nil { + err = lbFile.Sync() + if err != nil { + return + } + err = os.Rename(bal.LostBlocksFile+".tmp", bal.LostBlocksFile) + if err != nil { + return + } + lbFile = nil + } if runOptions.CommitPulls { - err = bal.CommitPulls(&config.Client) + err = bal.CommitPulls(ctx, client) if err != nil { // Skip trash if we can't pull. (Too cautious?) return } } if runOptions.CommitTrash { - err = bal.CommitTrash(&config.Client) + err = bal.CommitTrash(ctx, client) + if err != nil { + return + } + } + if runOptions.CommitConfirmedFields { + err = bal.updateCollections(ctx, client, cluster) + if err != nil { + return + } } return } @@ -134,15 +203,11 @@ func (bal *Balancer) SetKeepServices(srvList arvados.KeepServiceList) error { // DiscoverKeepServices sets the list of KeepServices by calling the // API to get a list of all services, and selecting the ones whose -// ServiceType is in okTypes. -func (bal *Balancer) DiscoverKeepServices(c *arvados.Client, okTypes []string) error { +// ServiceType is "disk" +func (bal *Balancer) DiscoverKeepServices(c *arvados.Client) error { bal.KeepServices = make(map[string]*KeepService) - ok := make(map[string]bool) - for _, t := range okTypes { - ok[t] = true - } return c.EachKeepService(func(srv arvados.KeepService) error { - if ok[srv.ServiceType] { + if srv.ServiceType == "disk" { bal.KeepServices[srv.UUID] = &KeepService{ KeepService: srv, ChangeSet: &ChangeSet{}, @@ -158,8 +223,8 @@ func (bal *Balancer) cleanupMounts() { rwdev := map[string]*KeepService{} for _, srv := range bal.KeepServices { for _, mnt := range srv.mounts { - if !mnt.ReadOnly && mnt.DeviceID != "" { - rwdev[mnt.DeviceID] = srv + if !mnt.ReadOnly { + rwdev[mnt.UUID] = srv } } } @@ -168,8 +233,8 @@ func (bal *Balancer) cleanupMounts() { for _, srv := range bal.KeepServices { var dedup []*KeepMount for _, mnt := range srv.mounts { - if mnt.ReadOnly && rwdev[mnt.DeviceID] != nil { - bal.logf("skipping srv %s readonly mount %q because same device %q is mounted read-write on srv %s", srv, mnt.UUID, mnt.DeviceID, rwdev[mnt.DeviceID]) + if mnt.ReadOnly && rwdev[mnt.UUID] != nil { + bal.logf("skipping srv %s readonly mount %q because same volume is mounted read-write on srv %s", srv, mnt.UUID, rwdev[mnt.UUID]) } else { dedup = append(dedup, mnt) } @@ -206,6 +271,47 @@ func (bal *Balancer) CheckSanityEarly(c *arvados.Client) error { return fmt.Errorf("config error: %s: proxy servers cannot be balanced", srv) } } + + mountProblem := false + type deviceMount struct { + srv *KeepService + mnt *KeepMount + } + deviceMounted := map[string]deviceMount{} // DeviceID -> mount + for _, srv := range bal.KeepServices { + for _, mnt := range srv.mounts { + if first, dup := deviceMounted[mnt.DeviceID]; dup && first.mnt.UUID != mnt.UUID && mnt.DeviceID != "" { + bal.logf("config error: device %s is mounted with multiple volume UUIDs: %s on %s, and %s on %s", + mnt.DeviceID, + first.mnt.UUID, first.srv, + mnt.UUID, srv) + mountProblem = true + continue + } + deviceMounted[mnt.DeviceID] = deviceMount{srv, mnt} + } + } + if mountProblem { + return errors.New("cannot continue with config errors (see above)") + } + + var checkPage arvados.CollectionList + if err = c.RequestAndDecode(&checkPage, "GET", "arvados/v1/collections", nil, arvados.ResourceListParams{ + Limit: new(int), + Count: "exact", + IncludeTrash: true, + IncludeOldVersions: true, + Filters: []arvados.Filter{{ + Attr: "modified_at", + Operator: "=", + Operand: nil, + }}, + }); err != nil { + return err + } else if n := checkPage.ItemsAvailable; n > 0 { + return fmt.Errorf("%d collections exist with null modified_at; cannot fetch reliably", n) + } + return nil } @@ -234,11 +340,11 @@ func (bal *Balancer) rendezvousState() string { // We avoid this problem if we clear all trash lists before getting // indexes. (We also assume there is only one rebalancing process // running at a time.) -func (bal *Balancer) ClearTrashLists(c *arvados.Client) error { +func (bal *Balancer) ClearTrashLists(ctx context.Context, c *arvados.Client) error { for _, srv := range bal.KeepServices { srv.ChangeSet = &ChangeSet{} } - return bal.CommitTrash(c) + return bal.CommitTrash(ctx, c) } // GetCurrentState determines the current replication state, and the @@ -252,7 +358,10 @@ func (bal *Balancer) ClearTrashLists(c *arvados.Client) error { // collection manifests in the database (API server). // // It encodes the resulting information in BlockStateMap. -func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) error { +func (bal *Balancer) GetCurrentState(ctx context.Context, c *arvados.Client, pageSize, bufs int) error { + ctx, cancel := context.WithCancel(ctx) + defer cancel() + defer bal.time("get_state", "wall clock time to get current state")() bal.BlockStateMap = NewBlockStateMap() @@ -277,12 +386,10 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro deviceMount := map[string]*KeepMount{} for _, srv := range bal.KeepServices { for _, mnt := range srv.mounts { - equiv := deviceMount[mnt.DeviceID] + equiv := deviceMount[mnt.UUID] if equiv == nil { equiv = mnt - if mnt.DeviceID != "" { - deviceMount[mnt.DeviceID] = equiv - } + deviceMount[mnt.UUID] = equiv } equivMount[equiv] = append(equivMount[equiv], mnt) } @@ -296,12 +403,13 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro go func(mounts []*KeepMount) { defer wg.Done() bal.logf("mount %s: retrieve index from %s", mounts[0], mounts[0].KeepService) - idx, err := mounts[0].KeepService.IndexMount(c, mounts[0].UUID, "") + idx, err := mounts[0].KeepService.IndexMount(ctx, c, mounts[0].UUID, "") if err != nil { select { case errs <- fmt.Errorf("%s: retrieve index: %v", mounts[0], err): default: } + cancel() return } if len(errs) > 0 { @@ -319,38 +427,14 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro }(mounts) } - // collQ buffers incoming collections so we can start fetching - // the next page without waiting for the current page to - // finish processing. collQ := make(chan arvados.Collection, bufs) - // Start a goroutine to process collections. (We could use a - // worker pool here, but even with a single worker we already - // process collections much faster than we can retrieve them.) + // Retrieve all collections from the database and send them to + // collQ. wg.Add(1) go func() { defer wg.Done() - for coll := range collQ { - err := bal.addCollection(coll) - if err != nil { - select { - case errs <- err: - default: - } - for range collQ { - } - return - } - bal.collScanned++ - } - }() - - // Start a goroutine to retrieve all collections from the - // Arvados database and send them to collQ for processing. - wg.Add(1) - go func() { - defer wg.Done() - err = EachCollection(c, pageSize, + err = EachCollection(ctx, bal.DB, c, func(coll arvados.Collection) error { collQ <- coll if len(errs) > 0 { @@ -370,9 +454,31 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro case errs <- err: default: } + cancel() } }() + // Parse manifests from collQ and pass the block hashes to + // BlockStateMap to track desired replication. + for i := 0; i < runtime.NumCPU(); i++ { + wg.Add(1) + go func() { + defer wg.Done() + for coll := range collQ { + err := bal.addCollection(coll) + if err != nil || len(errs) > 0 { + select { + case errs <- err: + default: + } + cancel() + continue + } + atomic.AddInt64(&bal.collScanned, 1) + } + }() + } + wg.Wait() if len(errs) > 0 { return <-errs @@ -383,17 +489,20 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro func (bal *Balancer) addCollection(coll arvados.Collection) error { blkids, err := coll.SizedDigests() if err != nil { - bal.mutex.Lock() - bal.errors = append(bal.errors, fmt.Errorf("%v: %v", coll.UUID, err)) - bal.mutex.Unlock() - return nil + return fmt.Errorf("%v: %v", coll.UUID, err) } repl := bal.DefaultReplication if coll.ReplicationDesired != nil { repl = *coll.ReplicationDesired } - debugf("%v: %d block x%d", coll.UUID, len(blkids), repl) - bal.BlockStateMap.IncreaseDesired(coll.StorageClassesDesired, repl, blkids) + bal.Logger.Debugf("%v: %d blocks x%d", coll.UUID, len(blkids), repl) + // Pass pdh to IncreaseDesired only if LostBlocksFile is being + // written -- otherwise it's just a waste of memory. + pdh := "" + if bal.LostBlocksFile != "" { + pdh = coll.PortableDataHash + } + bal.BlockStateMap.IncreaseDesired(pdh, coll.StorageClassesDesired, repl, blkids) return nil } @@ -444,7 +553,7 @@ func (bal *Balancer) ComputeChangeSets() { func (bal *Balancer) setupLookupTables() { bal.serviceRoots = make(map[string]string) - bal.classes = []string{"default"} + bal.classes = defaultClasses bal.mountsByClass = map[string]map[*KeepMount]bool{"default": {}} bal.mounts = 0 for _, srv := range bal.KeepServices { @@ -456,11 +565,7 @@ func (bal *Balancer) setupLookupTables() { // effectively read-only. mnt.ReadOnly = mnt.ReadOnly || srv.ReadOnly - if len(mnt.StorageClasses) == 0 { - bal.mountsByClass["default"][mnt] = true - continue - } - for _, class := range mnt.StorageClasses { + for class := range mnt.StorageClasses { if mbc := bal.mountsByClass[class]; mbc == nil { bal.classes = append(bal.classes, class) bal.mountsByClass[class] = map[*KeepMount]bool{mnt: true} @@ -492,24 +597,31 @@ var changeName = map[int]string{ changeNone: "none", } +type balancedBlockState struct { + needed int + unneeded int + pulling int + unachievable bool +} + type balanceResult struct { blk *BlockState blkid arvados.SizedDigest - have int - want int + lost bool + blockState balancedBlockState classState map[string]balancedBlockState } +type slot struct { + mnt *KeepMount // never nil + repl *Replica // replica already stored here (or nil) + want bool // we should pull/leave a replica here +} + // balanceBlock compares current state to desired state for a single // block, and makes the appropriate ChangeSet calls. func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) balanceResult { - debugf("balanceBlock: %v %+v", blkid, blk) - - type slot struct { - mnt *KeepMount // never nil - repl *Replica // replica already stored here (or nil) - want bool // we should pull/leave a replica here - } + bal.Logger.Debugf("balanceBlock: %v %+v", blkid, blk) // Build a list of all slots (one per mounted volume). slots := make([]slot, 0, bal.mounts) @@ -546,26 +658,9 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // won't want to trash any replicas. underreplicated := false - classState := make(map[string]balancedBlockState, len(bal.classes)) unsafeToDelete := make(map[int64]bool, len(slots)) for _, class := range bal.classes { desired := blk.Desired[class] - - countedDev := map[string]bool{} - have := 0 - for _, slot := range slots { - if slot.repl != nil && bal.mountsByClass[class][slot.mnt] && !countedDev[slot.mnt.DeviceID] { - have += slot.mnt.Replication - if slot.mnt.DeviceID != "" { - countedDev[slot.mnt.DeviceID] = true - } - } - } - classState[class] = balancedBlockState{ - desired: desired, - surplus: have - desired, - } - if desired == 0 { continue } @@ -599,7 +694,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // new/remaining replicas uniformly // across qualifying mounts on a given // server. - return rendezvousLess(si.mnt.DeviceID, sj.mnt.DeviceID, blkid) + return rendezvousLess(si.mnt.UUID, sj.mnt.UUID, blkid) } }) @@ -624,7 +719,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // and returns true if all requirements are met. trySlot := func(i int) bool { slot := slots[i] - if wantMnt[slot.mnt] || wantDev[slot.mnt.DeviceID] { + if wantMnt[slot.mnt] || wantDev[slot.mnt.UUID] { // Already allocated a replica to this // backend device, possibly on a // different server. @@ -639,9 +734,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba slots[i].want = true wantSrv[slot.mnt.KeepService] = true wantMnt[slot.mnt] = true - if slot.mnt.DeviceID != "" { - wantDev[slot.mnt.DeviceID] = true - } + wantDev[slot.mnt.UUID] = true replWant += slot.mnt.Replication } return replProt >= desired && replWant >= desired @@ -678,22 +771,12 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba underreplicated = safe < desired } - // set the unachievable flag if there aren't enough - // slots offering the relevant storage class. (This is - // as easy as checking slots[desired] because we - // already sorted the qualifying slots to the front.) - if desired >= len(slots) || !bal.mountsByClass[class][slots[desired].mnt] { - cs := classState[class] - cs.unachievable = true - classState[class] = cs - } - // Avoid deleting wanted replicas from devices that // are mounted on multiple servers -- even if they // haven't already been added to unsafeToDelete // because the servers report different Mtimes. for _, slot := range slots { - if slot.repl != nil && wantDev[slot.mnt.DeviceID] { + if slot.repl != nil && wantDev[slot.mnt.UUID] { unsafeToDelete[slot.repl.Mtime] = true } } @@ -703,36 +786,38 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // replica that doesn't have a timestamp collision with // others. - countedDev := map[string]bool{} - var have, want int - for _, slot := range slots { - if countedDev[slot.mnt.DeviceID] { - continue - } - if slot.want { - want += slot.mnt.Replication - } - if slot.repl != nil { - have += slot.mnt.Replication - } - if slot.mnt.DeviceID != "" { - countedDev[slot.mnt.DeviceID] = true + for i, slot := range slots { + // Don't trash (1) any replicas of an underreplicated + // block, even if they're in the wrong positions, or + // (2) any replicas whose Mtimes are identical to + // needed replicas (in case we're really seeing the + // same copy via different mounts). + if slot.repl != nil && (underreplicated || unsafeToDelete[slot.repl.Mtime]) { + slots[i].want = true } } + classState := make(map[string]balancedBlockState, len(bal.classes)) + for _, class := range bal.classes { + classState[class] = computeBlockState(slots, bal.mountsByClass[class], len(blk.Replicas), blk.Desired[class]) + } + blockState := computeBlockState(slots, nil, len(blk.Replicas), 0) + + var lost bool var changes []string for _, slot := range slots { // TODO: request a Touch if Mtime is duplicated. var change int switch { - case !underreplicated && !slot.want && slot.repl != nil && slot.repl.Mtime < bal.MinMtime && !unsafeToDelete[slot.repl.Mtime]: + case !slot.want && slot.repl != nil && slot.repl.Mtime < bal.MinMtime: slot.mnt.KeepService.AddTrash(Trash{ SizedDigest: blkid, Mtime: slot.repl.Mtime, From: slot.mnt, }) change = changeTrash - case len(blk.Replicas) == 0: + case slot.repl == nil && slot.want && len(blk.Replicas) == 0: + lost = true change = changeNone case slot.repl == nil && slot.want && !slot.mnt.ReadOnly: slot.mnt.KeepService.AddPull(Pull{ @@ -741,8 +826,10 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba To: slot.mnt, }) change = changePull - default: + case slot.repl != nil: change = changeStay + default: + change = changeNone } if bal.Dumper != nil { var mtime int64 @@ -754,17 +841,46 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba } } if bal.Dumper != nil { - bal.Dumper.Printf("%s have=%d want=%v %s", blkid, have, want, strings.Join(changes, " ")) + bal.Dumper.Printf("%s refs=%d needed=%d unneeded=%d pulling=%v %v %v", blkid, blk.RefCount, blockState.needed, blockState.unneeded, blockState.pulling, blk.Desired, changes) } return balanceResult{ blk: blk, blkid: blkid, - have: have, - want: want, + lost: lost, + blockState: blockState, classState: classState, } } +func computeBlockState(slots []slot, onlyCount map[*KeepMount]bool, have, needRepl int) (bbs balancedBlockState) { + repl := 0 + countedDev := map[string]bool{} + for _, slot := range slots { + if onlyCount != nil && !onlyCount[slot.mnt] { + continue + } + if countedDev[slot.mnt.UUID] { + continue + } + switch { + case slot.repl != nil && slot.want: + bbs.needed++ + repl += slot.mnt.Replication + case slot.repl != nil && !slot.want: + bbs.unneeded++ + repl += slot.mnt.Replication + case slot.repl == nil && slot.want && have > 0: + bbs.pulling++ + repl += slot.mnt.Replication + } + countedDev[slot.mnt.UUID] = true + } + if repl < needRepl { + bbs.unachievable = true + } + return +} + type blocksNBytes struct { replicas int blocks int @@ -775,6 +891,13 @@ func (bb blocksNBytes) String() string { return fmt.Sprintf("%d replicas (%d blocks, %d bytes)", bb.replicas, bb.blocks, bb.bytes) } +type replicationStats struct { + needed blocksNBytes + unneeded blocksNBytes + pulling blocksNBytes + unachievable blocksNBytes +} + type balancerStats struct { lost blocksNBytes overrep blocksNBytes @@ -811,25 +934,11 @@ func (s *balancerStats) dedupBlockRatio() float64 { return float64(s.collectionBlockRefs) / float64(s.collectionBlocks) } -type replicationStats struct { - desired blocksNBytes - surplus blocksNBytes - short blocksNBytes - unachievable blocksNBytes -} - -type balancedBlockState struct { - desired int - surplus int - unachievable bool -} - func (bal *Balancer) collectStatistics(results <-chan balanceResult) { var s balancerStats s.replHistogram = make([]int, 2) s.classStats = make(map[string]replicationStats, len(bal.classes)) for result := range results { - surplus := result.have - result.want bytes := result.blkid.Size() if rc := int64(result.blk.RefCount); rc > 0 { @@ -842,36 +951,51 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { for class, state := range result.classState { cs := s.classStats[class] if state.unachievable { + cs.unachievable.replicas++ cs.unachievable.blocks++ cs.unachievable.bytes += bytes } - if state.desired > 0 { - cs.desired.replicas += state.desired - cs.desired.blocks++ - cs.desired.bytes += bytes * int64(state.desired) + if state.needed > 0 { + cs.needed.replicas += state.needed + cs.needed.blocks++ + cs.needed.bytes += bytes * int64(state.needed) + } + if state.unneeded > 0 { + cs.unneeded.replicas += state.unneeded + cs.unneeded.blocks++ + cs.unneeded.bytes += bytes * int64(state.unneeded) } - if state.surplus > 0 { - cs.surplus.replicas += state.surplus - cs.surplus.blocks++ - cs.surplus.bytes += bytes * int64(state.surplus) - } else if state.surplus < 0 { - cs.short.replicas += -state.surplus - cs.short.blocks++ - cs.short.bytes += bytes * int64(-state.surplus) + if state.pulling > 0 { + cs.pulling.replicas += state.pulling + cs.pulling.blocks++ + cs.pulling.bytes += bytes * int64(state.pulling) } s.classStats[class] = cs } + bs := result.blockState switch { - case result.have == 0 && result.want > 0: - s.lost.replicas -= surplus + case result.lost: + s.lost.replicas++ s.lost.blocks++ - s.lost.bytes += bytes * int64(-surplus) - case surplus < 0: - s.underrep.replicas -= surplus + s.lost.bytes += bytes + fmt.Fprintf(bal.lostBlocks, "%s", strings.SplitN(string(result.blkid), "+", 2)[0]) + for pdh := range result.blk.Refs { + fmt.Fprintf(bal.lostBlocks, " %s", pdh) + } + fmt.Fprint(bal.lostBlocks, "\n") + case bs.pulling > 0: + s.underrep.replicas += bs.pulling + s.underrep.blocks++ + s.underrep.bytes += bytes * int64(bs.pulling) + case bs.unachievable: + s.underrep.replicas++ s.underrep.blocks++ - s.underrep.bytes += bytes * int64(-surplus) - case surplus > 0 && result.want == 0: + s.underrep.bytes += bytes + case bs.unneeded > 0 && bs.needed == 0: + // Count as "garbage" if all replicas are old + // enough to trash, otherwise count as + // "unref". counter := &s.garbage for _, r := range result.blk.Replicas { if r.Mtime >= bal.MinMtime { @@ -879,34 +1003,34 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { break } } - counter.replicas += surplus + counter.replicas += bs.unneeded counter.blocks++ - counter.bytes += bytes * int64(surplus) - case surplus > 0: - s.overrep.replicas += surplus + counter.bytes += bytes * int64(bs.unneeded) + case bs.unneeded > 0: + s.overrep.replicas += bs.unneeded s.overrep.blocks++ - s.overrep.bytes += bytes * int64(result.have-result.want) + s.overrep.bytes += bytes * int64(bs.unneeded) default: - s.justright.replicas += result.want + s.justright.replicas += bs.needed s.justright.blocks++ - s.justright.bytes += bytes * int64(result.want) + s.justright.bytes += bytes * int64(bs.needed) } - if result.want > 0 { - s.desired.replicas += result.want + if bs.needed > 0 { + s.desired.replicas += bs.needed s.desired.blocks++ - s.desired.bytes += bytes * int64(result.want) + s.desired.bytes += bytes * int64(bs.needed) } - if result.have > 0 { - s.current.replicas += result.have + if bs.needed+bs.unneeded > 0 { + s.current.replicas += bs.needed + bs.unneeded s.current.blocks++ - s.current.bytes += bytes * int64(result.have) + s.current.bytes += bytes * int64(bs.needed+bs.unneeded) } - for len(s.replHistogram) <= result.have { + for len(s.replHistogram) <= bs.needed+bs.unneeded { s.replHistogram = append(s.replHistogram, 0) } - s.replHistogram[result.have]++ + s.replHistogram[bs.needed+bs.unneeded]++ } for _, srv := range bal.KeepServices { s.pulls += len(srv.ChangeSet.Pulls) @@ -930,9 +1054,9 @@ func (bal *Balancer) PrintStatistics() { for _, class := range bal.classes { cs := bal.stats.classStats[class] bal.logf("===") - bal.logf("storage class %q: %s desired", class, cs.desired) - bal.logf("storage class %q: %s short", class, cs.short) - bal.logf("storage class %q: %s surplus", class, cs.surplus) + bal.logf("storage class %q: %s needed", class, cs.needed) + bal.logf("storage class %q: %s unneeded", class, cs.unneeded) + bal.logf("storage class %q: %s pulling", class, cs.pulling) bal.logf("storage class %q: %s unachievable", class, cs.unachievable) } bal.logf("===") @@ -948,7 +1072,7 @@ func (bal *Balancer) PrintStatistics() { } func (bal *Balancer) printHistogram(hashColumns int) { - bal.logf("Replication level distribution (counting N replicas on a single server as N):") + bal.logf("Replication level distribution:") maxCount := 0 for _, count := range bal.stats.replHistogram { if maxCount < count { @@ -1006,22 +1130,22 @@ func (bal *Balancer) CheckSanityLate() error { // keepstore servers. This has the effect of increasing replication of // existing blocks that are either underreplicated or poorly // distributed according to rendezvous hashing. -func (bal *Balancer) CommitPulls(c *arvados.Client) error { +func (bal *Balancer) CommitPulls(ctx context.Context, c *arvados.Client) error { defer bal.time("send_pull_lists", "wall clock time to send pull lists")() return bal.commitAsync(c, "send pull list", func(srv *KeepService) error { - return srv.CommitPulls(c) + return srv.CommitPulls(ctx, c) }) } // CommitTrash sends the computed lists of trash requests to the // keepstore servers. This has the effect of deleting blocks that are // overreplicated or unreferenced. -func (bal *Balancer) CommitTrash(c *arvados.Client) error { +func (bal *Balancer) CommitTrash(ctx context.Context, c *arvados.Client) error { defer bal.time("send_trash_lists", "wall clock time to send trash lists")() return bal.commitAsync(c, "send trash list", func(srv *KeepService) error { - return srv.CommitTrash(c) + return srv.CommitTrash(ctx, c) }) }