"bytes"
"crypto/md5"
"fmt"
+ "io"
+ "io/ioutil"
"log"
"math"
"os"
"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.
// 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
errors []error
stats balancerStats
mutex sync.Mutex
+ lostBlocks io.Writer
}
// Run performs a balance operation using the given config and
// 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
}
+
+ diskService := []string{"disk"}
+ err = bal.DiscoverKeepServices(client, diskService)
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()
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.,
// 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()
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
}
})
}
+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.
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
}
//
// 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()
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
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
})
close(collQ)
if err != nil {
- errs <- err
+ select {
+ case errs <- err:
+ default:
+ }
}
}()
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
}
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 {
blkid arvados.SizedDigest
blk *BlockState
}
- nWorkers := 1 + runtime.NumCPU()
- todo := make(chan balanceTask, nWorkers)
- results := make(chan balanceResult, 16)
- var wg sync.WaitGroup
- for i := 0; i < nWorkers; i++ {
- wg.Add(1)
- go func() {
- for work := range todo {
- results <- bal.balanceBlock(work.blkid, work.blk)
+ workers := runtime.GOMAXPROCS(-1)
+ todo := make(chan balanceTask, workers)
+ go func() {
+ bal.BlockStateMap.Apply(func(blkid arvados.SizedDigest, blk *BlockState) {
+ todo <- balanceTask{
+ blkid: blkid,
+ blk: blk,
}
- wg.Done()
- }()
- }
- bal.BlockStateMap.Apply(func(blkid arvados.SizedDigest, blk *BlockState) {
- todo <- balanceTask{
- blkid: blkid,
- blk: blk,
- }
- })
- close(todo)
+ })
+ close(todo)
+ }()
+ results := make(chan balanceResult, workers)
go func() {
+ var wg sync.WaitGroup
+ for i := 0; i < workers; i++ {
+ wg.Add(1)
+ go func() {
+ for work := range todo {
+ results <- bal.balanceBlock(work.blkid, work.blk)
+ }
+ wg.Done()
+ }()
+ }
wg.Wait()
close(results)
}()
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 {
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}
}
type balanceResult struct {
- blk *BlockState
- blkid arvados.SizedDigest
- have int
- want int
+ blk *BlockState
+ blkid arvados.SizedDigest
+ have int
+ want int
+ classState map[string]balancedBlockState
}
// balanceBlock compares current state to desired state for a single
slots = append(slots, slot{
mnt: mnt,
repl: repl,
- want: repl != nil && (mnt.ReadOnly || repl.Mtime >= bal.MinMtime),
+ want: repl != nil && mnt.ReadOnly,
})
}
}
// 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
}
+
// Sort the slots by desirability.
sort.Slice(slots, func(i, j int) bool {
si, sj := slots[i], slots[j]
// 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.
}
})
- // 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
if slot.repl == nil || !bal.mountsByClass[class][slot.mnt] {
continue
}
- if safe++; safe >= desired {
+ if safe += slot.mnt.Replication; safe >= desired {
break
}
}
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] {
+ 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
}
}
// 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
}
}
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,
- blkid: blkid,
- have: have,
- want: want,
+ blk: blk,
+ blkid: blkid,
+ have: have,
+ want: want,
+ classState: classState,
}
}
}
type balancerStats struct {
- lost, overrep, unref, garbage, underrep, justright blocksNBytes
- desired, current blocksNBytes
- pulls, trashes int
- replHistogram []int
+ lost blocksNBytes
+ overrep blocksNBytes
+ unref blocksNBytes
+ garbage blocksNBytes
+ underrep blocksNBytes
+ unachievable blocksNBytes
+ justright blocksNBytes
+ desired blocksNBytes
+ current blocksNBytes
+ pulls int
+ 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 {
+ 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 {
+ 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.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.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)
+ }
+ s.classStats[class] = cs
+ }
+
switch {
case result.have == 0 && result.want > 0:
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++
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++
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
bal.logf("%s overreplicated (have>want>0)", bal.stats.overrep)
bal.logf("%s unreferenced (have>want=0, new)", bal.stats.unref)
bal.logf("%s garbage (have>want=0, old)", bal.stats.garbage)
+ 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 unachievable", class, cs.unachievable)
+ }
bal.logf("===")
bal.logf("%s total commitment (excluding unreferenced)", bal.stats.desired)
bal.logf("%s total usage", bal.stats.current)
// 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)
// 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)
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)
}
}
+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 {