X-Git-Url: https://git.arvados.org/arvados.git/blobdiff_plain/606c29cdbb9012ee99043a2da9f28b2cd302e5e4..282d7a0c9d8279fa0a1293573e4313c8223be1b5:/services/keep-balance/balance.go diff --git a/services/keep-balance/balance.go b/services/keep-balance/balance.go index d6a2dde9d7..08a6c5881c 100644 --- a/services/keep-balance/balance.go +++ b/services/keep-balance/balance.go @@ -8,6 +8,8 @@ import ( "bytes" "crypto/md5" "fmt" + "io" + "io/ioutil" "log" "math" "os" @@ -15,24 +17,14 @@ import ( "sort" "strings" "sync" + "syscall" "time" "git.curoverse.com/arvados.git/sdk/go/arvados" "git.curoverse.com/arvados.git/sdk/go/keepclient" + "github.com/sirupsen/logrus" ) -// CheckConfig returns an error if anything is wrong with the given -// config and runOptions. -func CheckConfig(config Config, runOptions RunOptions) error { - if len(config.KeepServiceList.Items) > 0 && config.KeepServiceTypes != nil { - return fmt.Errorf("cannot specify both KeepServiceList and KeepServiceTypes in config") - } - if !runOptions.Once && config.RunPeriod == arvados.Duration(0) { - return fmt.Errorf("you must either use the -once flag, or specify RunPeriod in config") - } - return nil -} - // Balancer compares the contents of keepstore servers with the // collections stored in Arvados, and issues pull/trash requests // needed to get (closer to) the optimal data layout. @@ -43,11 +35,15 @@ func CheckConfig(config Config, runOptions RunOptions) error { // 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.FieldLogger + Dumper logrus.FieldLogger + Metrics *metrics + + LostBlocksFile string + *BlockStateMap KeepServices map[string]*KeepService DefaultReplication int - Logger *log.Logger - Dumper *log.Logger MinMtime int64 classes []string @@ -58,6 +54,7 @@ type Balancer struct { errors []error stats balancerStats mutex sync.Mutex + lostBlocks io.Writer } // Run performs a balance operation using the given config and @@ -72,13 +69,31 @@ type Balancer struct { func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions RunOptions, err error) { nextRunOptions = runOptions - bal.Dumper = runOptions.Dumper - bal.Logger = runOptions.Logger - if bal.Logger == nil { - bal.Logger = log.New(os.Stderr, "", log.LstdFlags) - } + defer bal.time("sweep", "wall clock time to run one full sweep")() - defer timeMe(bal.Logger, "Run")() + 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 { + bal.lostBlocks = ioutil.Discard + } if len(config.KeepServiceList.Items) > 0 { err = bal.SetKeepServices(config.KeepServiceList) @@ -95,6 +110,7 @@ func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions R return } } + bal.cleanupMounts() if err = bal.CheckSanityEarly(&config.Client); err != nil { return @@ -122,6 +138,17 @@ 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) if err != nil { @@ -169,6 +196,38 @@ func (bal *Balancer) DiscoverKeepServices(c *arvados.Client, okTypes []string) e }) } +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 + } + } + } + // Drop the readonly mounts whose device is mounted RW + // elsewhere. + 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]) + } else { + dedup = append(dedup, mnt) + } + } + srv.mounts = dedup + } + for _, srv := range bal.KeepServices { + for _, mnt := range srv.mounts { + if mnt.Replication <= 0 { + log.Printf("%s: mount %s reports replication=%d, using replication=1", srv, mnt.UUID, mnt.Replication) + mnt.Replication = 1 + } + } + } +} + // CheckSanityEarly checks for configuration and runtime errors that // can be detected before GetCurrentState() and ComputeChangeSets() // are called. @@ -189,6 +248,24 @@ func (bal *Balancer) CheckSanityEarly(c *arvados.Client) error { return fmt.Errorf("config error: %s: proxy servers cannot be balanced", srv) } } + + 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 } @@ -236,7 +313,7 @@ func (bal *Balancer) ClearTrashLists(c *arvados.Client) error { // // It encodes the resulting information in BlockStateMap. func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) error { - defer timeMe(bal.Logger, "GetCurrentState")() + defer bal.time("get_state", "wall clock time to get current state")() bal.BlockStateMap = NewBlockStateMap() dd, err := c.DiscoveryDocument() @@ -246,35 +323,60 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro bal.DefaultReplication = dd.DefaultCollectionReplication bal.MinMtime = time.Now().UnixNano() - dd.BlobSignatureTTL*1e9 - errs := make(chan error, 2+len(bal.KeepServices)) + errs := make(chan error, 1) wg := sync.WaitGroup{} - // Start one goroutine for each KeepService: retrieve the - // index, and add the returned blocks to BlockStateMap. + // When a device is mounted more than once, we will get its + // index only once, and call AddReplicas on all of the mounts. + // equivMount keys are the mounts that will be indexed, and + // each value is a list of mounts to apply the received index + // to. + equivMount := map[*KeepMount][]*KeepMount{} + // deviceMount maps each device ID to the one mount that will + // be indexed for that device. + deviceMount := map[string]*KeepMount{} for _, srv := range bal.KeepServices { + for _, mnt := range srv.mounts { + equiv := deviceMount[mnt.DeviceID] + if equiv == nil { + equiv = mnt + if mnt.DeviceID != "" { + deviceMount[mnt.DeviceID] = equiv + } + } + equivMount[equiv] = append(equivMount[equiv], mnt) + } + } + + // Start one goroutine for each (non-redundant) mount: + // retrieve the index, and add the returned blocks to + // BlockStateMap. + for _, mounts := range equivMount { wg.Add(1) - go func(srv *KeepService) { + go func(mounts []*KeepMount) { defer wg.Done() - bal.logf("%s: retrieve indexes", srv) - for _, mount := range srv.mounts { - bal.logf("%s: retrieve index", mount) - idx, err := srv.IndexMount(c, mount.UUID, "") - if err != nil { - errs <- fmt.Errorf("%s: retrieve index: %v", mount, err) - return - } - if len(errs) > 0 { - // Some other goroutine encountered an - // error -- any further effort here - // will be wasted. - return + bal.logf("mount %s: retrieve index from %s", mounts[0], mounts[0].KeepService) + idx, err := mounts[0].KeepService.IndexMount(c, mounts[0].UUID, "") + if err != nil { + select { + case errs <- fmt.Errorf("%s: retrieve index: %v", mounts[0], err): + default: } - bal.logf("%s: add %d replicas to map", mount, len(idx)) + return + } + if len(errs) > 0 { + // Some other goroutine encountered an + // error -- any further effort here + // will be wasted. + return + } + for _, mount := range mounts { + bal.logf("%s: add %d entries to map", mount, len(idx)) bal.BlockStateMap.AddReplicas(mount, idx) - bal.logf("%s: done", mount) + bal.logf("%s: added %d entries to map at %dx (%d replicas)", mount, len(idx), mount.Replication, len(idx)*mount.Replication) } - bal.logf("%s: done", srv) - }(srv) + bal.logf("mount %s: index done", mounts[0]) + }(mounts) } // collQ buffers incoming collections so we can start fetching @@ -290,8 +392,11 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro defer wg.Done() for coll := range collQ { err := bal.addCollection(coll) - if err != nil { - errs <- err + if err != nil || len(errs) > 0 { + select { + case errs <- err: + default: + } for range collQ { } return @@ -321,7 +426,10 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro }) close(collQ) if err != nil { - errs <- err + select { + case errs <- err: + default: + } } }() @@ -335,17 +443,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) + // 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 } @@ -358,7 +469,7 @@ func (bal *Balancer) addCollection(coll arvados.Collection) error { func (bal *Balancer) ComputeChangeSets() { // This just calls balanceBlock() once for each block, using a // pool of worker goroutines. - defer timeMe(bal.Logger, "ComputeChangeSets")() + defer bal.time("changeset_compute", "wall clock time to compute changesets")() bal.setupLookupTables() type balanceTask struct { @@ -396,7 +507,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 { @@ -481,7 +592,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba slots = append(slots, slot{ mnt: mnt, repl: repl, - want: repl != nil && (mnt.ReadOnly || repl.Mtime >= bal.MinMtime), + want: repl != nil && mnt.ReadOnly, }) } } @@ -503,10 +614,14 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba 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] { - have++ + 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{ @@ -525,14 +640,14 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // Prefer a mount that satisfies the // desired class. return bal.mountsByClass[class][si.mnt] - } else if wanti, wantj := si.want, si.want; wanti != wantj { + } else if si.want != sj.want { // Prefer a mount that will have a // replica no matter what we do here // -- either because it already has an // untrashable replica, or because we // already need it to satisfy a // different storage class. - return slots[i].want + return si.want } else if orderi, orderj := srvRendezvous[si.mnt.KeepService], srvRendezvous[sj.mnt.KeepService]; orderi != orderj { // Prefer a better rendezvous // position. @@ -551,32 +666,48 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba } }) - // Servers and mounts (with or without existing + // Servers/mounts/devices (with or without existing // replicas) that are part of the best achievable // layout for this storage class. wantSrv := map[*KeepService]bool{} wantMnt := map[*KeepMount]bool{} + wantDev := map[string]bool{} // Positions (with existing replicas) that have been // protected (via unsafeToDelete) to ensure we don't // reduce replication below desired level when // trashing replicas that aren't optimal positions for // any storage class. protMnt := map[*KeepMount]bool{} + // Replication planned so far (corresponds to wantMnt). + replWant := 0 + // Protected replication (corresponds to protMnt). + replProt := 0 // trySlot tries using a slot to meet requirements, // and returns true if all requirements are met. trySlot := func(i int) bool { slot := slots[i] - if len(protMnt) < desired && slot.repl != nil { + if wantMnt[slot.mnt] || wantDev[slot.mnt.DeviceID] { + // Already allocated a replica to this + // backend device, possibly on a + // different server. + return false + } + if replProt < desired && slot.repl != nil && !protMnt[slot.mnt] { unsafeToDelete[slot.repl.Mtime] = true protMnt[slot.mnt] = true + replProt += slot.mnt.Replication } - if len(wantMnt) < desired && (slot.repl != nil || !slot.mnt.ReadOnly) { + if replWant < desired && (slot.repl != nil || !slot.mnt.ReadOnly) { slots[i].want = true wantSrv[slot.mnt.KeepService] = true wantMnt[slot.mnt] = true + if slot.mnt.DeviceID != "" { + wantDev[slot.mnt.DeviceID] = true + } + replWant += slot.mnt.Replication } - return len(protMnt) >= desired && len(wantMnt) >= desired + return replProt >= desired && replWant >= desired } // First try to achieve desired replication without @@ -603,7 +734,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba if slot.repl == nil || !bal.mountsByClass[class][slot.mnt] { continue } - if safe++; safe >= desired { + if safe += slot.mnt.Replication; safe >= desired { break } } @@ -619,19 +750,36 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba 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] { + unsafeToDelete[slot.repl.Mtime] = true + } + } } // TODO: If multiple replicas are trashable, prefer the oldest // 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++ + want += slot.mnt.Replication } if slot.repl != nil { - have++ + have += slot.mnt.Replication + } + if slot.mnt.DeviceID != "" { + countedDev[slot.mnt.DeviceID] = true } } @@ -640,24 +788,24 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba // TODO: request a Touch if Mtime is duplicated. var change int switch { - case !underreplicated && slot.repl != nil && !slot.want && !unsafeToDelete[slot.repl.Mtime]: + case !underreplicated && !slot.want && slot.repl != nil && slot.repl.Mtime < bal.MinMtime && !unsafeToDelete[slot.repl.Mtime]: slot.mnt.KeepService.AddTrash(Trash{ SizedDigest: blkid, Mtime: slot.repl.Mtime, From: slot.mnt, }) change = changeTrash - case len(blk.Replicas) == 0: - change = changeNone - case slot.repl == nil && slot.want && !slot.mnt.ReadOnly: + case len(blk.Replicas) > 0 && slot.repl == nil && slot.want && !slot.mnt.ReadOnly: slot.mnt.KeepService.AddPull(Pull{ SizedDigest: blkid, From: blk.Replicas[0].KeepMount.KeepService, To: slot.mnt, }) change = changePull - default: + case slot.repl != nil: change = changeStay + default: + change = changeNone } if bal.Dumper != nil { var mtime int64 @@ -669,7 +817,7 @@ 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 have=%d want=%v %v %v", blkid, blk.RefCount, have, want, blk.Desired, changes) } return balanceResult{ blk: blk, @@ -704,6 +852,26 @@ type balancerStats struct { trashes int replHistogram []int classStats map[string]replicationStats + + // collectionBytes / collectionBlockBytes = deduplication ratio + collectionBytes int64 // sum(bytes in referenced blocks) across all collections + collectionBlockBytes int64 // sum(block size) across all blocks referenced by collections + collectionBlockRefs int64 // sum(number of blocks referenced) across all collections + collectionBlocks int64 // number of blocks referenced by any collection +} + +func (s *balancerStats) dedupByteRatio() float64 { + if s.collectionBlockBytes == 0 { + return 0 + } + return float64(s.collectionBytes) / float64(s.collectionBlockBytes) +} + +func (s *balancerStats) dedupBlockRatio() float64 { + if s.collectionBlocks == 0 { + return 0 + } + return float64(s.collectionBlockRefs) / float64(s.collectionBlocks) } type replicationStats struct { @@ -727,6 +895,13 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { surplus := result.have - result.want bytes := result.blkid.Size() + if rc := int64(result.blk.RefCount); rc > 0 { + s.collectionBytes += rc * bytes + s.collectionBlockBytes += bytes + s.collectionBlockRefs += rc + s.collectionBlocks++ + } + for class, state := range result.classState { cs := s.classStats[class] if state.unachievable { @@ -755,6 +930,11 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { s.lost.replicas -= surplus s.lost.blocks++ s.lost.bytes += bytes * int64(-surplus) + 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 surplus < 0: s.underrep.replicas -= surplus s.underrep.blocks++ @@ -773,7 +953,7 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { case surplus > 0: s.overrep.replicas += surplus s.overrep.blocks++ - s.overrep.bytes += bytes * int64(len(result.blk.Replicas)-result.want) + s.overrep.bytes += bytes * int64(result.have-result.want) default: s.justright.replicas += result.want s.justright.blocks++ @@ -785,22 +965,23 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) { s.desired.blocks++ s.desired.bytes += bytes * int64(result.want) } - if len(result.blk.Replicas) > 0 { - s.current.replicas += len(result.blk.Replicas) + if result.have > 0 { + s.current.replicas += result.have s.current.blocks++ - s.current.bytes += bytes * int64(len(result.blk.Replicas)) + s.current.bytes += bytes * int64(result.have) } - for len(s.replHistogram) <= len(result.blk.Replicas) { + for len(s.replHistogram) <= result.have { s.replHistogram = append(s.replHistogram, 0) } - s.replHistogram[len(result.blk.Replicas)]++ + s.replHistogram[result.have]++ } for _, srv := range bal.KeepServices { s.pulls += len(srv.ChangeSet.Pulls) s.trashes += len(srv.ChangeSet.Trashes) } bal.stats = s + bal.Metrics.UpdateStats(s) } // PrintStatistics writes statistics about the computed changes to @@ -894,6 +1075,7 @@ func (bal *Balancer) CheckSanityLate() error { // existing blocks that are either underreplicated or poorly // distributed according to rendezvous hashing. func (bal *Balancer) CommitPulls(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) @@ -904,6 +1086,7 @@ func (bal *Balancer) CommitPulls(c *arvados.Client) error { // keepstore servers. This has the effect of deleting blocks that are // overreplicated or unreferenced. func (bal *Balancer) CommitTrash(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) @@ -917,7 +1100,6 @@ func (bal *Balancer) commitAsync(c *arvados.Client, label string, f func(srv *Ke var err error defer func() { errs <- err }() label := fmt.Sprintf("%s: %v", srv, label) - defer timeMe(bal.Logger, label)() err = f(srv) if err != nil { err = fmt.Errorf("%s: %v", label, err) @@ -941,6 +1123,17 @@ func (bal *Balancer) logf(f string, args ...interface{}) { } } +func (bal *Balancer) time(name, help string) func() { + observer := bal.Metrics.DurationObserver(name+"_seconds", help) + t0 := time.Now() + bal.Logger.Printf("%s: start", name) + return func() { + dur := time.Since(t0) + observer.Observe(dur.Seconds()) + bal.Logger.Printf("%s: took %vs", name, dur.Seconds()) + } +} + // Rendezvous hash sort function. Less efficient than sorting on // precomputed rendezvous hashes, but also rarely used. func rendezvousLess(i, j string, blkid arvados.SizedDigest) bool {