Merge branch '15781-multi-value-property-search'
[arvados.git] / services / keep-balance / balance.go
index d86234a936cc96702f3a79d12c10d04548c0faa2..3c35d304cb395cf97485cd462f0f78ae31b523d6 100644 (file)
@@ -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"
+       "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/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
@@ -69,35 +66,49 @@ type Balancer struct {
 // Typical usage:
 //
 //   runOptions, err = (&Balancer{}).Run(config, runOptions)
-func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions RunOptions, err error) {
+func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, 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 timeMe(bal.Logger, "Run")()
+       defer bal.time("sweep", "wall clock time to run one full sweep")()
 
-       if len(config.KeepServiceList.Items) > 0 {
-               err = bal.SetKeepServices(config.KeepServiceList)
+       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
        }
        rs := bal.rendezvousState()
@@ -106,7 +117,7 @@ func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions R
                        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(client); err != nil {
                        return
                }
                // The current rendezvous state becomes "safe" (i.e.,
@@ -115,7 +126,7 @@ 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(client, cluster.Collections.BalanceCollectionBatch, cluster.Collections.BalanceCollectionBuffers); err != nil {
                return
        }
        bal.ComputeChangeSets()
@@ -123,15 +134,26 @@ 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(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(client)
        }
        return
 }
@@ -150,15 +172,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{},
@@ -222,6 +240,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
 }
 
@@ -269,7 +305,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()
@@ -279,7 +315,7 @@ 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{}
 
        // When a device is mounted more than once, we will get its
@@ -314,7 +350,10 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
                        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 {
-                               errs <- fmt.Errorf("%s: retrieve index: %v", mounts[0], err)
+                               select {
+                               case errs <- fmt.Errorf("%s: retrieve index: %v", mounts[0], err):
+                               default:
+                               }
                                return
                        }
                        if len(errs) > 0 {
@@ -324,9 +363,9 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
                                return
                        }
                        for _, mount := range mounts {
-                               bal.logf("%s: add %d replicas to map", mount, len(idx))
+                               bal.logf("%s: add %d entries to map", mount, len(idx))
                                bal.BlockStateMap.AddReplicas(mount, idx)
-                               bal.logf("%s: added %d replicas", mount, len(idx))
+                               bal.logf("%s: added %d entries to map at %dx (%d replicas)", mount, len(idx), mount.Replication, len(idx)*mount.Replication)
                        }
                        bal.logf("mount %s: index done", mounts[0])
                }(mounts)
@@ -345,8 +384,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
@@ -376,7 +418,10 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
                        })
                close(collQ)
                if err != nil {
-                       errs <- err
+                       select {
+                       case errs <- err:
+                       default:
+                       }
                }
        }()
 
@@ -390,17 +435,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 block 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
 }
 
@@ -413,7 +461,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 {
@@ -451,7 +499,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 {
@@ -467,7 +515,7 @@ func (bal *Balancer) setupLookupTables() {
                                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}
@@ -499,24 +547,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)
@@ -536,7 +591,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,
                        })
                }
        }
@@ -553,26 +608,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
                }
@@ -584,14 +622,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.
@@ -685,16 +723,6 @@ 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
@@ -710,36 +738,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.repl != nil && !slot.want && !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{
@@ -748,8 +778,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
@@ -761,17 +793,48 @@ 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.DeviceID] {
+                       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
+               }
+               if slot.mnt.DeviceID != "" {
+                       countedDev[slot.mnt.DeviceID] = true
+               }
+       }
+       if repl < needRepl {
+               bbs.unachievable = true
+       }
+       return
+}
+
 type blocksNBytes struct {
        replicas int
        blocks   int
@@ -782,6 +845,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
@@ -796,19 +866,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
 }
 
-type replicationStats struct {
-       desired      blocksNBytes
-       surplus      blocksNBytes
-       short        blocksNBytes
-       unachievable blocksNBytes
+func (s *balancerStats) dedupByteRatio() float64 {
+       if s.collectionBlockBytes == 0 {
+               return 0
+       }
+       return float64(s.collectionBytes) / float64(s.collectionBlockBytes)
 }
 
-type balancedBlockState struct {
-       desired      int
-       surplus      int
-       unachievable bool
+func (s *balancerStats) dedupBlockRatio() float64 {
+       if s.collectionBlocks == 0 {
+               return 0
+       }
+       return float64(s.collectionBlockRefs) / float64(s.collectionBlocks)
 }
 
 func (bal *Balancer) collectStatistics(results <-chan balanceResult) {
@@ -816,42 +893,63 @@ func (bal *Balancer) collectStatistics(results <-chan balanceResult) {
        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 {
+                       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 {
+                               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.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.unneeded > 0 {
+                               cs.unneeded.replicas += state.unneeded
+                               cs.unneeded.blocks++
+                               cs.unneeded.bytes += bytes * int64(state.unneeded)
+                       }
+                       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 {
@@ -859,40 +957,41 @@ 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)
                s.trashes += len(srv.ChangeSet.Trashes)
        }
        bal.stats = s
+       bal.Metrics.UpdateStats(s)
 }
 
 // PrintStatistics writes statistics about the computed changes to
@@ -909,9 +1008,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("===")
@@ -927,7 +1026,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 {
@@ -986,6 +1085,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)
@@ -996,6 +1096,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)
@@ -1009,7 +1110,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)
@@ -1033,6 +1133,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 {