1 // Copyright (C) The Arvados Authors. All rights reserved.
3 // SPDX-License-Identifier: AGPL-3.0
24 "git.arvados.org/arvados.git/sdk/go/arvados"
25 "git.arvados.org/arvados.git/sdk/go/keepclient"
26 "github.com/sirupsen/logrus"
29 // Balancer compares the contents of keepstore servers with the
30 // collections stored in Arvados, and issues pull/trash requests
31 // needed to get (closer to) the optimal data layout.
33 // In the optimal data layout: every data block referenced by a
34 // collection is replicated at least as many times as desired by the
35 // collection; there are no unreferenced data blocks older than
36 // BlobSignatureTTL; and all N existing replicas of a given data block
37 // are in the N best positions in rendezvous probe order.
38 type Balancer struct {
39 Logger logrus.FieldLogger
40 Dumper logrus.FieldLogger
46 KeepServices map[string]*KeepService
47 DefaultReplication int
52 mountsByClass map[string]map[*KeepMount]bool
54 serviceRoots map[string]string
61 // Run performs a balance operation using the given config and
62 // runOptions, and returns RunOptions suitable for passing to a
63 // subsequent balance operation.
65 // Run should only be called once on a given Balancer object.
69 // runOptions, err = (&Balancer{}).Run(config, runOptions)
70 func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, runOptions RunOptions) (nextRunOptions RunOptions, err error) {
71 nextRunOptions = runOptions
73 defer bal.time("sweep", "wall clock time to run one full sweep")()
75 ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(cluster.Collections.BalanceTimeout.Duration()))
79 if bal.LostBlocksFile != "" {
80 tmpfn := bal.LostBlocksFile + ".tmp"
81 lbFile, err = os.OpenFile(tmpfn, os.O_CREATE|os.O_WRONLY, 0777)
86 err = syscall.Flock(int(lbFile.Fd()), syscall.LOCK_EX|syscall.LOCK_NB)
91 // Remove the tempfile only if we didn't get
92 // as far as successfully renaming it.
97 bal.lostBlocks = lbFile
99 bal.lostBlocks = ioutil.Discard
102 err = bal.DiscoverKeepServices(client)
107 for _, srv := range bal.KeepServices {
108 err = srv.discoverMounts(client)
115 if err = bal.CheckSanityEarly(client); err != nil {
119 // On a big site, indexing and sending trash/pull lists can
120 // take much longer than the usual 5 minute client
121 // timeout. From here on, we rely on the context deadline
122 // instead, aborting the entire operation if any part takes
126 rs := bal.rendezvousState()
127 if runOptions.CommitTrash && rs != runOptions.SafeRendezvousState {
128 if runOptions.SafeRendezvousState != "" {
129 bal.logf("notice: KeepServices list has changed since last run")
131 bal.logf("clearing existing trash lists, in case the new rendezvous order differs from previous run")
132 if err = bal.ClearTrashLists(ctx, client); err != nil {
135 // The current rendezvous state becomes "safe" (i.e.,
136 // OK to compute changes for that state without
137 // clearing existing trash lists) only now, after we
138 // succeed in clearing existing trash lists.
139 nextRunOptions.SafeRendezvousState = rs
142 if err = bal.GetCurrentState(ctx, client, cluster.Collections.BalanceCollectionBatch, cluster.Collections.BalanceCollectionBuffers); err != nil {
145 bal.ComputeChangeSets()
146 bal.PrintStatistics()
147 if err = bal.CheckSanityLate(); err != nil {
155 err = os.Rename(bal.LostBlocksFile+".tmp", bal.LostBlocksFile)
161 if runOptions.CommitPulls {
162 err = bal.CommitPulls(ctx, client)
164 // Skip trash if we can't pull. (Too cautious?)
168 if runOptions.CommitTrash {
169 err = bal.CommitTrash(ctx, client)
174 // SetKeepServices sets the list of KeepServices to operate on.
175 func (bal *Balancer) SetKeepServices(srvList arvados.KeepServiceList) error {
176 bal.KeepServices = make(map[string]*KeepService)
177 for _, srv := range srvList.Items {
178 bal.KeepServices[srv.UUID] = &KeepService{
180 ChangeSet: &ChangeSet{},
186 // DiscoverKeepServices sets the list of KeepServices by calling the
187 // API to get a list of all services, and selecting the ones whose
188 // ServiceType is "disk"
189 func (bal *Balancer) DiscoverKeepServices(c *arvados.Client) error {
190 bal.KeepServices = make(map[string]*KeepService)
191 return c.EachKeepService(func(srv arvados.KeepService) error {
192 if srv.ServiceType == "disk" {
193 bal.KeepServices[srv.UUID] = &KeepService{
195 ChangeSet: &ChangeSet{},
198 bal.logf("skipping %v with service type %q", srv.UUID, srv.ServiceType)
204 func (bal *Balancer) cleanupMounts() {
205 rwdev := map[string]*KeepService{}
206 for _, srv := range bal.KeepServices {
207 for _, mnt := range srv.mounts {
208 if !mnt.ReadOnly && mnt.DeviceID != "" {
209 rwdev[mnt.DeviceID] = srv
213 // Drop the readonly mounts whose device is mounted RW
215 for _, srv := range bal.KeepServices {
216 var dedup []*KeepMount
217 for _, mnt := range srv.mounts {
218 if mnt.ReadOnly && rwdev[mnt.DeviceID] != nil {
219 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])
221 dedup = append(dedup, mnt)
226 for _, srv := range bal.KeepServices {
227 for _, mnt := range srv.mounts {
228 if mnt.Replication <= 0 {
229 log.Printf("%s: mount %s reports replication=%d, using replication=1", srv, mnt.UUID, mnt.Replication)
236 // CheckSanityEarly checks for configuration and runtime errors that
237 // can be detected before GetCurrentState() and ComputeChangeSets()
240 // If it returns an error, it is pointless to run GetCurrentState or
241 // ComputeChangeSets: after doing so, the statistics would be
242 // meaningless and it would be dangerous to run any Commit methods.
243 func (bal *Balancer) CheckSanityEarly(c *arvados.Client) error {
244 u, err := c.CurrentUser()
246 return fmt.Errorf("CurrentUser(): %v", err)
248 if !u.IsActive || !u.IsAdmin {
249 return fmt.Errorf("current user (%s) is not an active admin user", u.UUID)
251 for _, srv := range bal.KeepServices {
252 if srv.ServiceType == "proxy" {
253 return fmt.Errorf("config error: %s: proxy servers cannot be balanced", srv)
257 var checkPage arvados.CollectionList
258 if err = c.RequestAndDecode(&checkPage, "GET", "arvados/v1/collections", nil, arvados.ResourceListParams{
262 IncludeOldVersions: true,
263 Filters: []arvados.Filter{{
270 } else if n := checkPage.ItemsAvailable; n > 0 {
271 return fmt.Errorf("%d collections exist with null modified_at; cannot fetch reliably", n)
277 // rendezvousState returns a fingerprint (e.g., a sorted list of
278 // UUID+host+port) of the current set of keep services.
279 func (bal *Balancer) rendezvousState() string {
280 srvs := make([]string, 0, len(bal.KeepServices))
281 for _, srv := range bal.KeepServices {
282 srvs = append(srvs, srv.String())
285 return strings.Join(srvs, "; ")
288 // ClearTrashLists sends an empty trash list to each keep
289 // service. Calling this before GetCurrentState avoids races.
291 // When a block appears in an index, we assume that replica will still
292 // exist after we delete other replicas on other servers. However,
293 // it's possible that a previous rebalancing operation made different
294 // decisions (e.g., servers were added/removed, and rendezvous order
295 // changed). In this case, the replica might already be on that
296 // server's trash list, and it might be deleted before we send a
297 // replacement trash list.
299 // We avoid this problem if we clear all trash lists before getting
300 // indexes. (We also assume there is only one rebalancing process
301 // running at a time.)
302 func (bal *Balancer) ClearTrashLists(ctx context.Context, c *arvados.Client) error {
303 for _, srv := range bal.KeepServices {
304 srv.ChangeSet = &ChangeSet{}
306 return bal.CommitTrash(ctx, c)
309 // GetCurrentState determines the current replication state, and the
310 // desired replication level, for every block that is either
311 // retrievable or referenced.
313 // It determines the current replication state by reading the block index
314 // from every known Keep service.
316 // It determines the desired replication level by retrieving all
317 // collection manifests in the database (API server).
319 // It encodes the resulting information in BlockStateMap.
320 func (bal *Balancer) GetCurrentState(ctx context.Context, c *arvados.Client, pageSize, bufs int) error {
321 ctx, cancel := context.WithCancel(ctx)
324 defer bal.time("get_state", "wall clock time to get current state")()
325 bal.BlockStateMap = NewBlockStateMap()
327 dd, err := c.DiscoveryDocument()
331 bal.DefaultReplication = dd.DefaultCollectionReplication
332 bal.MinMtime = time.Now().UnixNano() - dd.BlobSignatureTTL*1e9
334 errs := make(chan error, 1)
335 wg := sync.WaitGroup{}
337 // When a device is mounted more than once, we will get its
338 // index only once, and call AddReplicas on all of the mounts.
339 // equivMount keys are the mounts that will be indexed, and
340 // each value is a list of mounts to apply the received index
342 equivMount := map[*KeepMount][]*KeepMount{}
343 // deviceMount maps each device ID to the one mount that will
344 // be indexed for that device.
345 deviceMount := map[string]*KeepMount{}
346 for _, srv := range bal.KeepServices {
347 for _, mnt := range srv.mounts {
348 equiv := deviceMount[mnt.DeviceID]
351 if mnt.DeviceID != "" {
352 deviceMount[mnt.DeviceID] = equiv
355 equivMount[equiv] = append(equivMount[equiv], mnt)
359 // Start one goroutine for each (non-redundant) mount:
360 // retrieve the index, and add the returned blocks to
362 for _, mounts := range equivMount {
364 go func(mounts []*KeepMount) {
366 bal.logf("mount %s: retrieve index from %s", mounts[0], mounts[0].KeepService)
367 idx, err := mounts[0].KeepService.IndexMount(ctx, c, mounts[0].UUID, "")
370 case errs <- fmt.Errorf("%s: retrieve index: %v", mounts[0], err):
377 // Some other goroutine encountered an
378 // error -- any further effort here
382 for _, mount := range mounts {
383 bal.logf("%s: add %d entries to map", mount, len(idx))
384 bal.BlockStateMap.AddReplicas(mount, idx)
385 bal.logf("%s: added %d entries to map at %dx (%d replicas)", mount, len(idx), mount.Replication, len(idx)*mount.Replication)
387 bal.logf("mount %s: index done", mounts[0])
391 // collQ buffers incoming collections so we can start fetching
392 // the next page without waiting for the current page to
393 // finish processing.
394 collQ := make(chan arvados.Collection, bufs)
396 // Start a goroutine to process collections. (We could use a
397 // worker pool here, but even with a single worker we already
398 // process collections much faster than we can retrieve them.)
402 for coll := range collQ {
403 err := bal.addCollection(coll)
404 if err != nil || len(errs) > 0 {
418 // Start a goroutine to retrieve all collections from the
419 // Arvados database and send them to collQ for processing.
423 err = EachCollection(ctx, c, pageSize,
424 func(coll arvados.Collection) error {
427 // some other GetCurrentState
428 // error happened: no point
431 return fmt.Errorf("")
434 }, func(done, total int) {
435 bal.logf("collections: %d/%d", done, total)
454 func (bal *Balancer) addCollection(coll arvados.Collection) error {
455 blkids, err := coll.SizedDigests()
457 return fmt.Errorf("%v: %v", coll.UUID, err)
459 repl := bal.DefaultReplication
460 if coll.ReplicationDesired != nil {
461 repl = *coll.ReplicationDesired
463 bal.Logger.Debugf("%v: %d block x%d", coll.UUID, len(blkids), repl)
464 // Pass pdh to IncreaseDesired only if LostBlocksFile is being
465 // written -- otherwise it's just a waste of memory.
467 if bal.LostBlocksFile != "" {
468 pdh = coll.PortableDataHash
470 bal.BlockStateMap.IncreaseDesired(pdh, coll.StorageClassesDesired, repl, blkids)
474 // ComputeChangeSets compares, for each known block, the current and
475 // desired replication states. If it is possible to get closer to the
476 // desired state by copying or deleting blocks, it adds those changes
477 // to the relevant KeepServices' ChangeSets.
479 // It does not actually apply any of the computed changes.
480 func (bal *Balancer) ComputeChangeSets() {
481 // This just calls balanceBlock() once for each block, using a
482 // pool of worker goroutines.
483 defer bal.time("changeset_compute", "wall clock time to compute changesets")()
484 bal.setupLookupTables()
486 type balanceTask struct {
487 blkid arvados.SizedDigest
490 workers := runtime.GOMAXPROCS(-1)
491 todo := make(chan balanceTask, workers)
493 bal.BlockStateMap.Apply(func(blkid arvados.SizedDigest, blk *BlockState) {
501 results := make(chan balanceResult, workers)
503 var wg sync.WaitGroup
504 for i := 0; i < workers; i++ {
507 for work := range todo {
508 results <- bal.balanceBlock(work.blkid, work.blk)
516 bal.collectStatistics(results)
519 func (bal *Balancer) setupLookupTables() {
520 bal.serviceRoots = make(map[string]string)
521 bal.classes = defaultClasses
522 bal.mountsByClass = map[string]map[*KeepMount]bool{"default": {}}
524 for _, srv := range bal.KeepServices {
525 bal.serviceRoots[srv.UUID] = srv.UUID
526 for _, mnt := range srv.mounts {
529 // All mounts on a read-only service are
530 // effectively read-only.
531 mnt.ReadOnly = mnt.ReadOnly || srv.ReadOnly
533 if len(mnt.StorageClasses) == 0 {
534 bal.mountsByClass["default"][mnt] = true
537 for class := range mnt.StorageClasses {
538 if mbc := bal.mountsByClass[class]; mbc == nil {
539 bal.classes = append(bal.classes, class)
540 bal.mountsByClass[class] = map[*KeepMount]bool{mnt: true}
547 // Consider classes in lexicographic order to avoid flapping
548 // between balancing runs. The outcome of the "prefer a mount
549 // we're already planning to use for a different storage
550 // class" case in balanceBlock depends on the order classes
552 sort.Strings(bal.classes)
562 var changeName = map[int]string{
565 changeTrash: "trash",
569 type balancedBlockState struct {
576 type balanceResult struct {
578 blkid arvados.SizedDigest
580 blockState balancedBlockState
581 classState map[string]balancedBlockState
585 mnt *KeepMount // never nil
586 repl *Replica // replica already stored here (or nil)
587 want bool // we should pull/leave a replica here
590 // balanceBlock compares current state to desired state for a single
591 // block, and makes the appropriate ChangeSet calls.
592 func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) balanceResult {
593 bal.Logger.Debugf("balanceBlock: %v %+v", blkid, blk)
595 // Build a list of all slots (one per mounted volume).
596 slots := make([]slot, 0, bal.mounts)
597 for _, srv := range bal.KeepServices {
598 for _, mnt := range srv.mounts {
600 for r := range blk.Replicas {
601 if blk.Replicas[r].KeepMount == mnt {
602 repl = &blk.Replicas[r]
605 // Initial value of "want" is "have, and can't
606 // delete". These untrashable replicas get
607 // prioritized when sorting slots: otherwise,
608 // non-optimal readonly copies would cause us
610 slots = append(slots, slot{
613 want: repl != nil && mnt.ReadOnly,
618 uuids := keepclient.NewRootSorter(bal.serviceRoots, string(blkid[:32])).GetSortedRoots()
619 srvRendezvous := make(map[*KeepService]int, len(uuids))
620 for i, uuid := range uuids {
621 srv := bal.KeepServices[uuid]
622 srvRendezvous[srv] = i
625 // Below we set underreplicated=true if we find any storage
626 // class that's currently underreplicated -- in that case we
627 // won't want to trash any replicas.
628 underreplicated := false
630 unsafeToDelete := make(map[int64]bool, len(slots))
631 for _, class := range bal.classes {
632 desired := blk.Desired[class]
637 // Sort the slots by desirability.
638 sort.Slice(slots, func(i, j int) bool {
639 si, sj := slots[i], slots[j]
640 if classi, classj := bal.mountsByClass[class][si.mnt], bal.mountsByClass[class][sj.mnt]; classi != classj {
641 // Prefer a mount that satisfies the
643 return bal.mountsByClass[class][si.mnt]
644 } else if si.want != sj.want {
645 // Prefer a mount that will have a
646 // replica no matter what we do here
647 // -- either because it already has an
648 // untrashable replica, or because we
649 // already need it to satisfy a
650 // different storage class.
652 } else if orderi, orderj := srvRendezvous[si.mnt.KeepService], srvRendezvous[sj.mnt.KeepService]; orderi != orderj {
653 // Prefer a better rendezvous
655 return orderi < orderj
656 } else if repli, replj := si.repl != nil, sj.repl != nil; repli != replj {
657 // Prefer a mount that already has a
661 // If pull/trash turns out to be
662 // needed, distribute the
663 // new/remaining replicas uniformly
664 // across qualifying mounts on a given
666 return rendezvousLess(si.mnt.DeviceID, sj.mnt.DeviceID, blkid)
670 // Servers/mounts/devices (with or without existing
671 // replicas) that are part of the best achievable
672 // layout for this storage class.
673 wantSrv := map[*KeepService]bool{}
674 wantMnt := map[*KeepMount]bool{}
675 wantDev := map[string]bool{}
676 // Positions (with existing replicas) that have been
677 // protected (via unsafeToDelete) to ensure we don't
678 // reduce replication below desired level when
679 // trashing replicas that aren't optimal positions for
680 // any storage class.
681 protMnt := map[*KeepMount]bool{}
682 // Replication planned so far (corresponds to wantMnt).
684 // Protected replication (corresponds to protMnt).
687 // trySlot tries using a slot to meet requirements,
688 // and returns true if all requirements are met.
689 trySlot := func(i int) bool {
691 if wantMnt[slot.mnt] || wantDev[slot.mnt.DeviceID] {
692 // Already allocated a replica to this
693 // backend device, possibly on a
697 if replProt < desired && slot.repl != nil && !protMnt[slot.mnt] {
698 unsafeToDelete[slot.repl.Mtime] = true
699 protMnt[slot.mnt] = true
700 replProt += slot.mnt.Replication
702 if replWant < desired && (slot.repl != nil || !slot.mnt.ReadOnly) {
704 wantSrv[slot.mnt.KeepService] = true
705 wantMnt[slot.mnt] = true
706 if slot.mnt.DeviceID != "" {
707 wantDev[slot.mnt.DeviceID] = true
709 replWant += slot.mnt.Replication
711 return replProt >= desired && replWant >= desired
714 // First try to achieve desired replication without
715 // using the same server twice.
717 for i := 0; i < len(slots) && !done; i++ {
718 if !wantSrv[slots[i].mnt.KeepService] {
723 // If that didn't suffice, do another pass without the
724 // "distinct services" restriction. (Achieving the
725 // desired volume replication on fewer than the
726 // desired number of services is better than
727 // underreplicating.)
728 for i := 0; i < len(slots) && !done; i++ {
732 if !underreplicated {
734 for _, slot := range slots {
735 if slot.repl == nil || !bal.mountsByClass[class][slot.mnt] {
738 if safe += slot.mnt.Replication; safe >= desired {
742 underreplicated = safe < desired
745 // Avoid deleting wanted replicas from devices that
746 // are mounted on multiple servers -- even if they
747 // haven't already been added to unsafeToDelete
748 // because the servers report different Mtimes.
749 for _, slot := range slots {
750 if slot.repl != nil && wantDev[slot.mnt.DeviceID] {
751 unsafeToDelete[slot.repl.Mtime] = true
756 // TODO: If multiple replicas are trashable, prefer the oldest
757 // replica that doesn't have a timestamp collision with
760 for i, slot := range slots {
761 // Don't trash (1) any replicas of an underreplicated
762 // block, even if they're in the wrong positions, or
763 // (2) any replicas whose Mtimes are identical to
764 // needed replicas (in case we're really seeing the
765 // same copy via different mounts).
766 if slot.repl != nil && (underreplicated || unsafeToDelete[slot.repl.Mtime]) {
771 classState := make(map[string]balancedBlockState, len(bal.classes))
772 for _, class := range bal.classes {
773 classState[class] = computeBlockState(slots, bal.mountsByClass[class], len(blk.Replicas), blk.Desired[class])
775 blockState := computeBlockState(slots, nil, len(blk.Replicas), 0)
779 for _, slot := range slots {
780 // TODO: request a Touch if Mtime is duplicated.
783 case !slot.want && slot.repl != nil && slot.repl.Mtime < bal.MinMtime:
784 slot.mnt.KeepService.AddTrash(Trash{
786 Mtime: slot.repl.Mtime,
790 case slot.repl == nil && slot.want && len(blk.Replicas) == 0:
793 case slot.repl == nil && slot.want && !slot.mnt.ReadOnly:
794 slot.mnt.KeepService.AddPull(Pull{
796 From: blk.Replicas[0].KeepMount.KeepService,
800 case slot.repl != nil:
805 if bal.Dumper != nil {
807 if slot.repl != nil {
808 mtime = slot.repl.Mtime
810 srv := slot.mnt.KeepService
811 changes = append(changes, fmt.Sprintf("%s:%d/%s=%s,%d", srv.ServiceHost, srv.ServicePort, slot.mnt.UUID, changeName[change], mtime))
814 if bal.Dumper != nil {
815 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)
817 return balanceResult{
821 blockState: blockState,
822 classState: classState,
826 func computeBlockState(slots []slot, onlyCount map[*KeepMount]bool, have, needRepl int) (bbs balancedBlockState) {
828 countedDev := map[string]bool{}
829 for _, slot := range slots {
830 if onlyCount != nil && !onlyCount[slot.mnt] {
833 if countedDev[slot.mnt.DeviceID] {
837 case slot.repl != nil && slot.want:
839 repl += slot.mnt.Replication
840 case slot.repl != nil && !slot.want:
842 repl += slot.mnt.Replication
843 case slot.repl == nil && slot.want && have > 0:
845 repl += slot.mnt.Replication
847 if slot.mnt.DeviceID != "" {
848 countedDev[slot.mnt.DeviceID] = true
852 bbs.unachievable = true
857 type blocksNBytes struct {
863 func (bb blocksNBytes) String() string {
864 return fmt.Sprintf("%d replicas (%d blocks, %d bytes)", bb.replicas, bb.blocks, bb.bytes)
867 type replicationStats struct {
869 unneeded blocksNBytes
871 unachievable blocksNBytes
874 type balancerStats struct {
879 underrep blocksNBytes
880 unachievable blocksNBytes
881 justright blocksNBytes
887 classStats map[string]replicationStats
889 // collectionBytes / collectionBlockBytes = deduplication ratio
890 collectionBytes int64 // sum(bytes in referenced blocks) across all collections
891 collectionBlockBytes int64 // sum(block size) across all blocks referenced by collections
892 collectionBlockRefs int64 // sum(number of blocks referenced) across all collections
893 collectionBlocks int64 // number of blocks referenced by any collection
896 func (s *balancerStats) dedupByteRatio() float64 {
897 if s.collectionBlockBytes == 0 {
900 return float64(s.collectionBytes) / float64(s.collectionBlockBytes)
903 func (s *balancerStats) dedupBlockRatio() float64 {
904 if s.collectionBlocks == 0 {
907 return float64(s.collectionBlockRefs) / float64(s.collectionBlocks)
910 func (bal *Balancer) collectStatistics(results <-chan balanceResult) {
912 s.replHistogram = make([]int, 2)
913 s.classStats = make(map[string]replicationStats, len(bal.classes))
914 for result := range results {
915 bytes := result.blkid.Size()
917 if rc := int64(result.blk.RefCount); rc > 0 {
918 s.collectionBytes += rc * bytes
919 s.collectionBlockBytes += bytes
920 s.collectionBlockRefs += rc
924 for class, state := range result.classState {
925 cs := s.classStats[class]
926 if state.unachievable {
927 cs.unachievable.replicas++
928 cs.unachievable.blocks++
929 cs.unachievable.bytes += bytes
931 if state.needed > 0 {
932 cs.needed.replicas += state.needed
934 cs.needed.bytes += bytes * int64(state.needed)
936 if state.unneeded > 0 {
937 cs.unneeded.replicas += state.unneeded
939 cs.unneeded.bytes += bytes * int64(state.unneeded)
941 if state.pulling > 0 {
942 cs.pulling.replicas += state.pulling
944 cs.pulling.bytes += bytes * int64(state.pulling)
946 s.classStats[class] = cs
949 bs := result.blockState
954 s.lost.bytes += bytes
955 fmt.Fprintf(bal.lostBlocks, "%s", strings.SplitN(string(result.blkid), "+", 2)[0])
956 for pdh := range result.blk.Refs {
957 fmt.Fprintf(bal.lostBlocks, " %s", pdh)
959 fmt.Fprint(bal.lostBlocks, "\n")
961 s.underrep.replicas += bs.pulling
963 s.underrep.bytes += bytes * int64(bs.pulling)
964 case bs.unachievable:
965 s.underrep.replicas++
967 s.underrep.bytes += bytes
968 case bs.unneeded > 0 && bs.needed == 0:
969 // Count as "garbage" if all replicas are old
970 // enough to trash, otherwise count as
972 counter := &s.garbage
973 for _, r := range result.blk.Replicas {
974 if r.Mtime >= bal.MinMtime {
979 counter.replicas += bs.unneeded
981 counter.bytes += bytes * int64(bs.unneeded)
982 case bs.unneeded > 0:
983 s.overrep.replicas += bs.unneeded
985 s.overrep.bytes += bytes * int64(bs.unneeded)
987 s.justright.replicas += bs.needed
989 s.justright.bytes += bytes * int64(bs.needed)
993 s.desired.replicas += bs.needed
995 s.desired.bytes += bytes * int64(bs.needed)
997 if bs.needed+bs.unneeded > 0 {
998 s.current.replicas += bs.needed + bs.unneeded
1000 s.current.bytes += bytes * int64(bs.needed+bs.unneeded)
1003 for len(s.replHistogram) <= bs.needed+bs.unneeded {
1004 s.replHistogram = append(s.replHistogram, 0)
1006 s.replHistogram[bs.needed+bs.unneeded]++
1008 for _, srv := range bal.KeepServices {
1009 s.pulls += len(srv.ChangeSet.Pulls)
1010 s.trashes += len(srv.ChangeSet.Trashes)
1013 bal.Metrics.UpdateStats(s)
1016 // PrintStatistics writes statistics about the computed changes to
1017 // bal.Logger. It should not be called until ComputeChangeSets has
1019 func (bal *Balancer) PrintStatistics() {
1021 bal.logf("%s lost (0=have<want)", bal.stats.lost)
1022 bal.logf("%s underreplicated (0<have<want)", bal.stats.underrep)
1023 bal.logf("%s just right (have=want)", bal.stats.justright)
1024 bal.logf("%s overreplicated (have>want>0)", bal.stats.overrep)
1025 bal.logf("%s unreferenced (have>want=0, new)", bal.stats.unref)
1026 bal.logf("%s garbage (have>want=0, old)", bal.stats.garbage)
1027 for _, class := range bal.classes {
1028 cs := bal.stats.classStats[class]
1030 bal.logf("storage class %q: %s needed", class, cs.needed)
1031 bal.logf("storage class %q: %s unneeded", class, cs.unneeded)
1032 bal.logf("storage class %q: %s pulling", class, cs.pulling)
1033 bal.logf("storage class %q: %s unachievable", class, cs.unachievable)
1036 bal.logf("%s total commitment (excluding unreferenced)", bal.stats.desired)
1037 bal.logf("%s total usage", bal.stats.current)
1039 for _, srv := range bal.KeepServices {
1040 bal.logf("%s: %v\n", srv, srv.ChangeSet)
1043 bal.printHistogram(60)
1047 func (bal *Balancer) printHistogram(hashColumns int) {
1048 bal.logf("Replication level distribution:")
1050 for _, count := range bal.stats.replHistogram {
1051 if maxCount < count {
1055 hashes := strings.Repeat("#", hashColumns)
1056 countWidth := 1 + int(math.Log10(float64(maxCount+1)))
1057 scaleCount := 10 * float64(hashColumns) / math.Floor(1+10*math.Log10(float64(maxCount+1)))
1058 for repl, count := range bal.stats.replHistogram {
1059 nHashes := int(scaleCount * math.Log10(float64(count+1)))
1060 bal.logf("%2d: %*d %s", repl, countWidth, count, hashes[:nHashes])
1064 // CheckSanityLate checks for configuration and runtime errors after
1065 // GetCurrentState() and ComputeChangeSets() have finished.
1067 // If it returns an error, it is dangerous to run any Commit methods.
1068 func (bal *Balancer) CheckSanityLate() error {
1069 if bal.errors != nil {
1070 for _, err := range bal.errors {
1071 bal.logf("deferred error: %v", err)
1073 return fmt.Errorf("cannot proceed safely after deferred errors")
1076 if bal.collScanned == 0 {
1077 return fmt.Errorf("received zero collections")
1081 bal.BlockStateMap.Apply(func(_ arvados.SizedDigest, blk *BlockState) {
1082 for _, desired := range blk.Desired {
1090 return fmt.Errorf("zero blocks have desired replication>0")
1093 if dr := bal.DefaultReplication; dr < 1 {
1094 return fmt.Errorf("Default replication (%d) is less than 1", dr)
1097 // TODO: no two services have identical indexes
1098 // TODO: no collisions (same md5, different size)
1102 // CommitPulls sends the computed lists of pull requests to the
1103 // keepstore servers. This has the effect of increasing replication of
1104 // existing blocks that are either underreplicated or poorly
1105 // distributed according to rendezvous hashing.
1106 func (bal *Balancer) CommitPulls(ctx context.Context, c *arvados.Client) error {
1107 defer bal.time("send_pull_lists", "wall clock time to send pull lists")()
1108 return bal.commitAsync(c, "send pull list",
1109 func(srv *KeepService) error {
1110 return srv.CommitPulls(ctx, c)
1114 // CommitTrash sends the computed lists of trash requests to the
1115 // keepstore servers. This has the effect of deleting blocks that are
1116 // overreplicated or unreferenced.
1117 func (bal *Balancer) CommitTrash(ctx context.Context, c *arvados.Client) error {
1118 defer bal.time("send_trash_lists", "wall clock time to send trash lists")()
1119 return bal.commitAsync(c, "send trash list",
1120 func(srv *KeepService) error {
1121 return srv.CommitTrash(ctx, c)
1125 func (bal *Balancer) commitAsync(c *arvados.Client, label string, f func(srv *KeepService) error) error {
1126 errs := make(chan error)
1127 for _, srv := range bal.KeepServices {
1128 go func(srv *KeepService) {
1130 defer func() { errs <- err }()
1131 label := fmt.Sprintf("%s: %v", srv, label)
1134 err = fmt.Errorf("%s: %v", label, err)
1139 for range bal.KeepServices {
1140 if err := <-errs; err != nil {
1149 func (bal *Balancer) logf(f string, args ...interface{}) {
1150 if bal.Logger != nil {
1151 bal.Logger.Printf(f, args...)
1155 func (bal *Balancer) time(name, help string) func() {
1156 observer := bal.Metrics.DurationObserver(name+"_seconds", help)
1158 bal.Logger.Printf("%s: start", name)
1160 dur := time.Since(t0)
1161 observer.Observe(dur.Seconds())
1162 bal.Logger.Printf("%s: took %vs", name, dur.Seconds())
1166 // Rendezvous hash sort function. Less efficient than sorting on
1167 // precomputed rendezvous hashes, but also rarely used.
1168 func rendezvousLess(i, j string, blkid arvados.SizedDigest) bool {
1169 a := md5.Sum([]byte(string(blkid[:32]) + i))
1170 b := md5.Sum([]byte(string(blkid[:32]) + j))
1171 return bytes.Compare(a[:], b[:]) < 0