Merge branch 'main' into 18842-arv-mount-disk-config
[arvados.git] / services / keep-balance / balance.go
index 261714605666cb37829a4864d069c58ef8a4b32b..9f581751d938baf8d0f8fdc726bfae92e1f6877d 100644 (file)
@@ -2,12 +2,13 @@
 //
 // SPDX-License-Identifier: AGPL-3.0
 
-package main
+package keepbalance
 
 import (
        "bytes"
        "context"
        "crypto/md5"
+       "errors"
        "fmt"
        "io"
        "io/ioutil"
@@ -18,11 +19,15 @@ import (
        "sort"
        "strings"
        "sync"
+       "sync/atomic"
        "syscall"
        "time"
 
+       "git.arvados.org/arvados.git/lib/controller/dblock"
        "git.arvados.org/arvados.git/sdk/go/arvados"
+       "git.arvados.org/arvados.git/sdk/go/ctxlog"
        "git.arvados.org/arvados.git/sdk/go/keepclient"
+       "github.com/jmoiron/sqlx"
        "github.com/sirupsen/logrus"
 )
 
@@ -36,6 +41,7 @@ import (
 // BlobSignatureTTL; and all N existing replicas of a given data block
 // are in the N best positions in rendezvous probe order.
 type Balancer struct {
+       DB      *sqlx.DB
        Logger  logrus.FieldLogger
        Dumper  logrus.FieldLogger
        Metrics *metrics
@@ -50,7 +56,7 @@ type Balancer struct {
        classes       []string
        mounts        int
        mountsByClass map[string]map[*KeepMount]bool
-       collScanned   int
+       collScanned   int64
        serviceRoots  map[string]string
        errors        []error
        stats         balancerStats
@@ -63,15 +69,21 @@ type Balancer struct {
 // subsequent balance operation.
 //
 // Run should only be called once on a given Balancer object.
-//
-// Typical usage:
-//
-//   runOptions, err = (&Balancer{}).Run(config, runOptions)
-func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, runOptions RunOptions) (nextRunOptions RunOptions, err error) {
+func (bal *Balancer) Run(ctx context.Context, client *arvados.Client, cluster *arvados.Cluster, runOptions RunOptions) (nextRunOptions RunOptions, err error) {
        nextRunOptions = runOptions
 
+       ctxlog.FromContext(ctx).Info("acquiring active lock")
+       if !dblock.KeepBalanceActive.Lock(ctx, func(context.Context) (*sqlx.DB, error) { return bal.DB, nil }) {
+               // context canceled
+               return
+       }
+       defer dblock.KeepBalanceActive.Unlock()
+
        defer bal.time("sweep", "wall clock time to run one full sweep")()
 
+       ctx, cancel := context.WithDeadline(ctx, time.Now().Add(cluster.Collections.BalanceTimeout.Duration()))
+       defer cancel()
+
        var lbFile *os.File
        if bal.LostBlocksFile != "" {
                tmpfn := bal.LostBlocksFile + ".tmp"
@@ -112,13 +124,21 @@ func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, runOp
        if err = bal.CheckSanityEarly(client); err != nil {
                return
        }
+
+       // On a big site, indexing and sending trash/pull lists can
+       // take much longer than the usual 5 minute client
+       // timeout. From here on, we rely on the context deadline
+       // instead, aborting the entire operation if any part takes
+       // too long.
+       client.Timeout = 0
+
        rs := bal.rendezvousState()
        if runOptions.CommitTrash && rs != runOptions.SafeRendezvousState {
                if runOptions.SafeRendezvousState != "" {
                        bal.logf("notice: KeepServices list has changed since last run")
                }
                bal.logf("clearing existing trash lists, in case the new rendezvous order differs from previous run")
-               if err = bal.ClearTrashLists(client); err != nil {
+               if err = bal.ClearTrashLists(ctx, client); err != nil {
                        return
                }
                // The current rendezvous state becomes "safe" (i.e.,
@@ -128,13 +148,7 @@ func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, runOp
                nextRunOptions.SafeRendezvousState = rs
        }
 
-       // Indexing and sending trash/pull lists can take a long time
-       // on a big site. Prefer a long timeout (causing slow recovery
-       // from undetected network problems) to a short timeout
-       // (causing starvation via perpetual timeout/restart cycle).
-       client.Timeout = 24 * time.Hour
-
-       if err = bal.GetCurrentState(client, cluster.Collections.BalanceCollectionBatch, cluster.Collections.BalanceCollectionBuffers); err != nil {
+       if err = bal.GetCurrentState(ctx, client, cluster.Collections.BalanceCollectionBatch, cluster.Collections.BalanceCollectionBuffers); err != nil {
                return
        }
        bal.ComputeChangeSets()
@@ -154,14 +168,23 @@ func (bal *Balancer) Run(client *arvados.Client, cluster *arvados.Cluster, runOp
                lbFile = nil
        }
        if runOptions.CommitPulls {
-               err = bal.CommitPulls(client)
+               err = bal.CommitPulls(ctx, client)
                if err != nil {
                        // Skip trash if we can't pull. (Too cautious?)
                        return
                }
        }
        if runOptions.CommitTrash {
-               err = bal.CommitTrash(client)
+               err = bal.CommitTrash(ctx, client)
+               if err != nil {
+                       return
+               }
+       }
+       if runOptions.CommitConfirmedFields {
+               err = bal.updateCollections(ctx, client, cluster)
+               if err != nil {
+                       return
+               }
        }
        return
 }
@@ -200,8 +223,8 @@ func (bal *Balancer) cleanupMounts() {
        rwdev := map[string]*KeepService{}
        for _, srv := range bal.KeepServices {
                for _, mnt := range srv.mounts {
-                       if !mnt.ReadOnly && mnt.DeviceID != "" {
-                               rwdev[mnt.DeviceID] = srv
+                       if !mnt.ReadOnly {
+                               rwdev[mnt.UUID] = srv
                        }
                }
        }
@@ -210,8 +233,8 @@ func (bal *Balancer) cleanupMounts() {
        for _, srv := range bal.KeepServices {
                var dedup []*KeepMount
                for _, mnt := range srv.mounts {
-                       if mnt.ReadOnly && rwdev[mnt.DeviceID] != nil {
-                               bal.logf("skipping srv %s readonly mount %q because same device %q is mounted read-write on srv %s", srv, mnt.UUID, mnt.DeviceID, rwdev[mnt.DeviceID])
+                       if mnt.ReadOnly && rwdev[mnt.UUID] != nil {
+                               bal.logf("skipping srv %s readonly mount %q because same volume is mounted read-write on srv %s", srv, mnt.UUID, rwdev[mnt.UUID])
                        } else {
                                dedup = append(dedup, mnt)
                        }
@@ -249,6 +272,29 @@ func (bal *Balancer) CheckSanityEarly(c *arvados.Client) error {
                }
        }
 
+       mountProblem := false
+       type deviceMount struct {
+               srv *KeepService
+               mnt *KeepMount
+       }
+       deviceMounted := map[string]deviceMount{} // DeviceID -> mount
+       for _, srv := range bal.KeepServices {
+               for _, mnt := range srv.mounts {
+                       if first, dup := deviceMounted[mnt.DeviceID]; dup && first.mnt.UUID != mnt.UUID && mnt.DeviceID != "" {
+                               bal.logf("config error: device %s is mounted with multiple volume UUIDs: %s on %s, and %s on %s",
+                                       mnt.DeviceID,
+                                       first.mnt.UUID, first.srv,
+                                       mnt.UUID, srv)
+                               mountProblem = true
+                               continue
+                       }
+                       deviceMounted[mnt.DeviceID] = deviceMount{srv, mnt}
+               }
+       }
+       if mountProblem {
+               return errors.New("cannot continue with config errors (see above)")
+       }
+
        var checkPage arvados.CollectionList
        if err = c.RequestAndDecode(&checkPage, "GET", "arvados/v1/collections", nil, arvados.ResourceListParams{
                Limit:              new(int),
@@ -294,11 +340,11 @@ func (bal *Balancer) rendezvousState() string {
 // We avoid this problem if we clear all trash lists before getting
 // indexes. (We also assume there is only one rebalancing process
 // running at a time.)
-func (bal *Balancer) ClearTrashLists(c *arvados.Client) error {
+func (bal *Balancer) ClearTrashLists(ctx context.Context, c *arvados.Client) error {
        for _, srv := range bal.KeepServices {
                srv.ChangeSet = &ChangeSet{}
        }
-       return bal.CommitTrash(c)
+       return bal.CommitTrash(ctx, c)
 }
 
 // GetCurrentState determines the current replication state, and the
@@ -312,8 +358,8 @@ func (bal *Balancer) ClearTrashLists(c *arvados.Client) error {
 // collection manifests in the database (API server).
 //
 // It encodes the resulting information in BlockStateMap.
-func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) error {
-       ctx, cancel := context.WithCancel(context.Background())
+func (bal *Balancer) GetCurrentState(ctx context.Context, c *arvados.Client, pageSize, bufs int) error {
+       ctx, cancel := context.WithCancel(ctx)
        defer cancel()
 
        defer bal.time("get_state", "wall clock time to get current state")()
@@ -340,12 +386,10 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
        deviceMount := map[string]*KeepMount{}
        for _, srv := range bal.KeepServices {
                for _, mnt := range srv.mounts {
-                       equiv := deviceMount[mnt.DeviceID]
+                       equiv := deviceMount[mnt.UUID]
                        if equiv == nil {
                                equiv = mnt
-                               if mnt.DeviceID != "" {
-                                       deviceMount[mnt.DeviceID] = equiv
-                               }
+                               deviceMount[mnt.UUID] = equiv
                        }
                        equivMount[equiv] = append(equivMount[equiv], mnt)
                }
@@ -383,39 +427,14 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
                }(mounts)
        }
 
-       // collQ buffers incoming collections so we can start fetching
-       // the next page without waiting for the current page to
-       // finish processing.
        collQ := make(chan arvados.Collection, bufs)
 
-       // Start a goroutine to process collections. (We could use a
-       // worker pool here, but even with a single worker we already
-       // process collections much faster than we can retrieve them.)
-       wg.Add(1)
-       go func() {
-               defer wg.Done()
-               for coll := range collQ {
-                       err := bal.addCollection(coll)
-                       if err != nil || len(errs) > 0 {
-                               select {
-                               case errs <- err:
-                               default:
-                               }
-                               for range collQ {
-                               }
-                               cancel()
-                               return
-                       }
-                       bal.collScanned++
-               }
-       }()
-
-       // Start a goroutine to retrieve all collections from the
-       // Arvados database and send them to collQ for processing.
+       // Retrieve all collections from the database and send them to
+       // collQ.
        wg.Add(1)
        go func() {
                defer wg.Done()
-               err = EachCollection(c, pageSize,
+               err = EachCollection(ctx, bal.DB, c,
                        func(coll arvados.Collection) error {
                                collQ <- coll
                                if len(errs) > 0 {
@@ -439,6 +458,27 @@ func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) erro
                }
        }()
 
+       // Parse manifests from collQ and pass the block hashes to
+       // BlockStateMap to track desired replication.
+       for i := 0; i < runtime.NumCPU(); i++ {
+               wg.Add(1)
+               go func() {
+                       defer wg.Done()
+                       for coll := range collQ {
+                               err := bal.addCollection(coll)
+                               if err != nil || len(errs) > 0 {
+                                       select {
+                                       case errs <- err:
+                                       default:
+                                       }
+                                       cancel()
+                                       continue
+                               }
+                               atomic.AddInt64(&bal.collScanned, 1)
+                       }
+               }()
+       }
+
        wg.Wait()
        if len(errs) > 0 {
                return <-errs
@@ -455,7 +495,7 @@ func (bal *Balancer) addCollection(coll arvados.Collection) error {
        if coll.ReplicationDesired != nil {
                repl = *coll.ReplicationDesired
        }
-       bal.Logger.Debugf("%v: %d block x%d", coll.UUID, len(blkids), repl)
+       bal.Logger.Debugf("%v: %d blocks x%d", coll.UUID, len(blkids), repl)
        // Pass pdh to IncreaseDesired only if LostBlocksFile is being
        // written -- otherwise it's just a waste of memory.
        pdh := ""
@@ -525,10 +565,6 @@ func (bal *Balancer) setupLookupTables() {
                        // effectively read-only.
                        mnt.ReadOnly = mnt.ReadOnly || srv.ReadOnly
 
-                       if len(mnt.StorageClasses) == 0 {
-                               bal.mountsByClass["default"][mnt] = true
-                               continue
-                       }
                        for class := range mnt.StorageClasses {
                                if mbc := bal.mountsByClass[class]; mbc == nil {
                                        bal.classes = append(bal.classes, class)
@@ -658,7 +694,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba
                                // new/remaining replicas uniformly
                                // across qualifying mounts on a given
                                // server.
-                               return rendezvousLess(si.mnt.DeviceID, sj.mnt.DeviceID, blkid)
+                               return rendezvousLess(si.mnt.UUID, sj.mnt.UUID, blkid)
                        }
                })
 
@@ -683,7 +719,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba
                // and returns true if all requirements are met.
                trySlot := func(i int) bool {
                        slot := slots[i]
-                       if wantMnt[slot.mnt] || wantDev[slot.mnt.DeviceID] {
+                       if wantMnt[slot.mnt] || wantDev[slot.mnt.UUID] {
                                // Already allocated a replica to this
                                // backend device, possibly on a
                                // different server.
@@ -698,9 +734,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba
                                slots[i].want = true
                                wantSrv[slot.mnt.KeepService] = true
                                wantMnt[slot.mnt] = true
-                               if slot.mnt.DeviceID != "" {
-                                       wantDev[slot.mnt.DeviceID] = true
-                               }
+                               wantDev[slot.mnt.UUID] = true
                                replWant += slot.mnt.Replication
                        }
                        return replProt >= desired && replWant >= desired
@@ -742,7 +776,7 @@ func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) ba
                // haven't already been added to unsafeToDelete
                // because the servers report different Mtimes.
                for _, slot := range slots {
-                       if slot.repl != nil && wantDev[slot.mnt.DeviceID] {
+                       if slot.repl != nil && wantDev[slot.mnt.UUID] {
                                unsafeToDelete[slot.repl.Mtime] = true
                        }
                }
@@ -825,7 +859,7 @@ func computeBlockState(slots []slot, onlyCount map[*KeepMount]bool, have, needRe
                if onlyCount != nil && !onlyCount[slot.mnt] {
                        continue
                }
-               if countedDev[slot.mnt.DeviceID] {
+               if countedDev[slot.mnt.UUID] {
                        continue
                }
                switch {
@@ -839,9 +873,7 @@ func computeBlockState(slots []slot, onlyCount map[*KeepMount]bool, have, needRe
                        bbs.pulling++
                        repl += slot.mnt.Replication
                }
-               if slot.mnt.DeviceID != "" {
-                       countedDev[slot.mnt.DeviceID] = true
-               }
+               countedDev[slot.mnt.UUID] = true
        }
        if repl < needRepl {
                bbs.unachievable = true
@@ -1098,22 +1130,22 @@ func (bal *Balancer) CheckSanityLate() error {
 // keepstore servers. This has the effect of increasing replication of
 // existing blocks that are either underreplicated or poorly
 // distributed according to rendezvous hashing.
-func (bal *Balancer) CommitPulls(c *arvados.Client) error {
+func (bal *Balancer) CommitPulls(ctx context.Context, c *arvados.Client) error {
        defer bal.time("send_pull_lists", "wall clock time to send pull lists")()
        return bal.commitAsync(c, "send pull list",
                func(srv *KeepService) error {
-                       return srv.CommitPulls(c)
+                       return srv.CommitPulls(ctx, c)
                })
 }
 
 // CommitTrash sends the computed lists of trash requests to the
 // keepstore servers. This has the effect of deleting blocks that are
 // overreplicated or unreferenced.
-func (bal *Balancer) CommitTrash(c *arvados.Client) error {
+func (bal *Balancer) CommitTrash(ctx context.Context, c *arvados.Client) error {
        defer bal.time("send_trash_lists", "wall clock time to send trash lists")()
        return bal.commitAsync(c, "send trash list",
                func(srv *KeepService) error {
-                       return srv.CommitTrash(c)
+                       return srv.CommitTrash(ctx, c)
                })
 }