Merge branch '8784-dir-listings'
[arvados.git] / services / keep-balance / balance.go
1 // Copyright (C) The Arvados Authors. All rights reserved.
2 //
3 // SPDX-License-Identifier: AGPL-3.0
4
5 package main
6
7 import (
8         "fmt"
9         "log"
10         "math"
11         "os"
12         "runtime"
13         "sort"
14         "strings"
15         "sync"
16         "time"
17
18         "git.curoverse.com/arvados.git/sdk/go/arvados"
19         "git.curoverse.com/arvados.git/sdk/go/keepclient"
20 )
21
22 // CheckConfig returns an error if anything is wrong with the given
23 // config and runOptions.
24 func CheckConfig(config Config, runOptions RunOptions) error {
25         if len(config.KeepServiceList.Items) > 0 && config.KeepServiceTypes != nil {
26                 return fmt.Errorf("cannot specify both KeepServiceList and KeepServiceTypes in config")
27         }
28         if !runOptions.Once && config.RunPeriod == arvados.Duration(0) {
29                 return fmt.Errorf("you must either use the -once flag, or specify RunPeriod in config")
30         }
31         return nil
32 }
33
34 // Balancer compares the contents of keepstore servers with the
35 // collections stored in Arvados, and issues pull/trash requests
36 // needed to get (closer to) the optimal data layout.
37 //
38 // In the optimal data layout: every data block referenced by a
39 // collection is replicated at least as many times as desired by the
40 // collection; there are no unreferenced data blocks older than
41 // BlobSignatureTTL; and all N existing replicas of a given data block
42 // are in the N best positions in rendezvous probe order.
43 type Balancer struct {
44         *BlockStateMap
45         KeepServices       map[string]*KeepService
46         DefaultReplication int
47         Logger             *log.Logger
48         Dumper             *log.Logger
49         MinMtime           int64
50
51         collScanned  int
52         serviceRoots map[string]string
53         errors       []error
54         mutex        sync.Mutex
55 }
56
57 // Run performs a balance operation using the given config and
58 // runOptions, and returns RunOptions suitable for passing to a
59 // subsequent balance operation.
60 //
61 // Run should only be called once on a given Balancer object.
62 //
63 // Typical usage:
64 //
65 //   runOptions, err = (&Balancer{}).Run(config, runOptions)
66 func (bal *Balancer) Run(config Config, runOptions RunOptions) (nextRunOptions RunOptions, err error) {
67         nextRunOptions = runOptions
68
69         bal.Dumper = runOptions.Dumper
70         bal.Logger = runOptions.Logger
71         if bal.Logger == nil {
72                 bal.Logger = log.New(os.Stderr, "", log.LstdFlags)
73         }
74
75         defer timeMe(bal.Logger, "Run")()
76
77         if len(config.KeepServiceList.Items) > 0 {
78                 err = bal.SetKeepServices(config.KeepServiceList)
79         } else {
80                 err = bal.DiscoverKeepServices(&config.Client, config.KeepServiceTypes)
81         }
82         if err != nil {
83                 return
84         }
85
86         if err = bal.CheckSanityEarly(&config.Client); err != nil {
87                 return
88         }
89         rs := bal.rendezvousState()
90         if runOptions.CommitTrash && rs != runOptions.SafeRendezvousState {
91                 if runOptions.SafeRendezvousState != "" {
92                         bal.logf("notice: KeepServices list has changed since last run")
93                 }
94                 bal.logf("clearing existing trash lists, in case the new rendezvous order differs from previous run")
95                 if err = bal.ClearTrashLists(&config.Client); err != nil {
96                         return
97                 }
98                 // The current rendezvous state becomes "safe" (i.e.,
99                 // OK to compute changes for that state without
100                 // clearing existing trash lists) only now, after we
101                 // succeed in clearing existing trash lists.
102                 nextRunOptions.SafeRendezvousState = rs
103         }
104         if err = bal.GetCurrentState(&config.Client, config.CollectionBatchSize, config.CollectionBuffers); err != nil {
105                 return
106         }
107         bal.ComputeChangeSets()
108         bal.PrintStatistics()
109         if err = bal.CheckSanityLate(); err != nil {
110                 return
111         }
112         if runOptions.CommitPulls {
113                 err = bal.CommitPulls(&config.Client)
114                 if err != nil {
115                         // Skip trash if we can't pull. (Too cautious?)
116                         return
117                 }
118         }
119         if runOptions.CommitTrash {
120                 err = bal.CommitTrash(&config.Client)
121         }
122         return
123 }
124
125 // SetKeepServices sets the list of KeepServices to operate on.
126 func (bal *Balancer) SetKeepServices(srvList arvados.KeepServiceList) error {
127         bal.KeepServices = make(map[string]*KeepService)
128         for _, srv := range srvList.Items {
129                 bal.KeepServices[srv.UUID] = &KeepService{
130                         KeepService: srv,
131                         ChangeSet:   &ChangeSet{},
132                 }
133         }
134         return nil
135 }
136
137 // DiscoverKeepServices sets the list of KeepServices by calling the
138 // API to get a list of all services, and selecting the ones whose
139 // ServiceType is in okTypes.
140 func (bal *Balancer) DiscoverKeepServices(c *arvados.Client, okTypes []string) error {
141         bal.KeepServices = make(map[string]*KeepService)
142         ok := make(map[string]bool)
143         for _, t := range okTypes {
144                 ok[t] = true
145         }
146         return c.EachKeepService(func(srv arvados.KeepService) error {
147                 if ok[srv.ServiceType] {
148                         bal.KeepServices[srv.UUID] = &KeepService{
149                                 KeepService: srv,
150                                 ChangeSet:   &ChangeSet{},
151                         }
152                 } else {
153                         bal.logf("skipping %v with service type %q", srv.UUID, srv.ServiceType)
154                 }
155                 return nil
156         })
157 }
158
159 // CheckSanityEarly checks for configuration and runtime errors that
160 // can be detected before GetCurrentState() and ComputeChangeSets()
161 // are called.
162 //
163 // If it returns an error, it is pointless to run GetCurrentState or
164 // ComputeChangeSets: after doing so, the statistics would be
165 // meaningless and it would be dangerous to run any Commit methods.
166 func (bal *Balancer) CheckSanityEarly(c *arvados.Client) error {
167         u, err := c.CurrentUser()
168         if err != nil {
169                 return fmt.Errorf("CurrentUser(): %v", err)
170         }
171         if !u.IsActive || !u.IsAdmin {
172                 return fmt.Errorf("current user (%s) is not an active admin user", u.UUID)
173         }
174         for _, srv := range bal.KeepServices {
175                 if srv.ServiceType == "proxy" {
176                         return fmt.Errorf("config error: %s: proxy servers cannot be balanced", srv)
177                 }
178         }
179         return nil
180 }
181
182 // rendezvousState returns a fingerprint (e.g., a sorted list of
183 // UUID+host+port) of the current set of keep services.
184 func (bal *Balancer) rendezvousState() string {
185         srvs := make([]string, 0, len(bal.KeepServices))
186         for _, srv := range bal.KeepServices {
187                 srvs = append(srvs, srv.String())
188         }
189         sort.Strings(srvs)
190         return strings.Join(srvs, "; ")
191 }
192
193 // ClearTrashLists sends an empty trash list to each keep
194 // service. Calling this before GetCurrentState avoids races.
195 //
196 // When a block appears in an index, we assume that replica will still
197 // exist after we delete other replicas on other servers. However,
198 // it's possible that a previous rebalancing operation made different
199 // decisions (e.g., servers were added/removed, and rendezvous order
200 // changed). In this case, the replica might already be on that
201 // server's trash list, and it might be deleted before we send a
202 // replacement trash list.
203 //
204 // We avoid this problem if we clear all trash lists before getting
205 // indexes. (We also assume there is only one rebalancing process
206 // running at a time.)
207 func (bal *Balancer) ClearTrashLists(c *arvados.Client) error {
208         for _, srv := range bal.KeepServices {
209                 srv.ChangeSet = &ChangeSet{}
210         }
211         return bal.CommitTrash(c)
212 }
213
214 // GetCurrentState determines the current replication state, and the
215 // desired replication level, for every block that is either
216 // retrievable or referenced.
217 //
218 // It determines the current replication state by reading the block index
219 // from every known Keep service.
220 //
221 // It determines the desired replication level by retrieving all
222 // collection manifests in the database (API server).
223 //
224 // It encodes the resulting information in BlockStateMap.
225 func (bal *Balancer) GetCurrentState(c *arvados.Client, pageSize, bufs int) error {
226         defer timeMe(bal.Logger, "GetCurrentState")()
227         bal.BlockStateMap = NewBlockStateMap()
228
229         dd, err := c.DiscoveryDocument()
230         if err != nil {
231                 return err
232         }
233         bal.DefaultReplication = dd.DefaultCollectionReplication
234         bal.MinMtime = time.Now().UnixNano() - dd.BlobSignatureTTL*1e9
235
236         errs := make(chan error, 2+len(bal.KeepServices))
237         wg := sync.WaitGroup{}
238
239         // Start one goroutine for each KeepService: retrieve the
240         // index, and add the returned blocks to BlockStateMap.
241         for _, srv := range bal.KeepServices {
242                 wg.Add(1)
243                 go func(srv *KeepService) {
244                         defer wg.Done()
245                         bal.logf("%s: retrieve index", srv)
246                         idx, err := srv.Index(c, "")
247                         if err != nil {
248                                 errs <- fmt.Errorf("%s: %v", srv, err)
249                                 return
250                         }
251                         if len(errs) > 0 {
252                                 // Some other goroutine encountered an
253                                 // error -- any further effort here
254                                 // will be wasted.
255                                 return
256                         }
257                         bal.logf("%s: add %d replicas to map", srv, len(idx))
258                         bal.BlockStateMap.AddReplicas(srv, idx)
259                         bal.logf("%s: done", srv)
260                 }(srv)
261         }
262
263         // collQ buffers incoming collections so we can start fetching
264         // the next page without waiting for the current page to
265         // finish processing.
266         collQ := make(chan arvados.Collection, bufs)
267
268         // Start a goroutine to process collections. (We could use a
269         // worker pool here, but even with a single worker we already
270         // process collections much faster than we can retrieve them.)
271         wg.Add(1)
272         go func() {
273                 defer wg.Done()
274                 for coll := range collQ {
275                         err := bal.addCollection(coll)
276                         if err != nil {
277                                 errs <- err
278                                 for range collQ {
279                                 }
280                                 return
281                         }
282                         bal.collScanned++
283                 }
284         }()
285
286         // Start a goroutine to retrieve all collections from the
287         // Arvados database and send them to collQ for processing.
288         wg.Add(1)
289         go func() {
290                 defer wg.Done()
291                 err = EachCollection(c, pageSize,
292                         func(coll arvados.Collection) error {
293                                 collQ <- coll
294                                 if len(errs) > 0 {
295                                         // some other GetCurrentState
296                                         // error happened: no point
297                                         // getting any more
298                                         // collections.
299                                         return fmt.Errorf("")
300                                 }
301                                 return nil
302                         }, func(done, total int) {
303                                 bal.logf("collections: %d/%d", done, total)
304                         })
305                 close(collQ)
306                 if err != nil {
307                         errs <- err
308                 }
309         }()
310
311         wg.Wait()
312         if len(errs) > 0 {
313                 return <-errs
314         }
315         return nil
316 }
317
318 func (bal *Balancer) addCollection(coll arvados.Collection) error {
319         blkids, err := coll.SizedDigests()
320         if err != nil {
321                 bal.mutex.Lock()
322                 bal.errors = append(bal.errors, fmt.Errorf("%v: %v", coll.UUID, err))
323                 bal.mutex.Unlock()
324                 return nil
325         }
326         repl := bal.DefaultReplication
327         if coll.ReplicationDesired != nil {
328                 repl = *coll.ReplicationDesired
329         }
330         debugf("%v: %d block x%d", coll.UUID, len(blkids), repl)
331         bal.BlockStateMap.IncreaseDesired(repl, blkids)
332         return nil
333 }
334
335 // ComputeChangeSets compares, for each known block, the current and
336 // desired replication states. If it is possible to get closer to the
337 // desired state by copying or deleting blocks, it adds those changes
338 // to the relevant KeepServices' ChangeSets.
339 //
340 // It does not actually apply any of the computed changes.
341 func (bal *Balancer) ComputeChangeSets() {
342         // This just calls balanceBlock() once for each block, using a
343         // pool of worker goroutines.
344         defer timeMe(bal.Logger, "ComputeChangeSets")()
345         bal.setupServiceRoots()
346
347         type balanceTask struct {
348                 blkid arvados.SizedDigest
349                 blk   *BlockState
350         }
351         nWorkers := 1 + runtime.NumCPU()
352         todo := make(chan balanceTask, nWorkers)
353         var wg sync.WaitGroup
354         for i := 0; i < nWorkers; i++ {
355                 wg.Add(1)
356                 go func() {
357                         for work := range todo {
358                                 bal.balanceBlock(work.blkid, work.blk)
359                         }
360                         wg.Done()
361                 }()
362         }
363         bal.BlockStateMap.Apply(func(blkid arvados.SizedDigest, blk *BlockState) {
364                 todo <- balanceTask{
365                         blkid: blkid,
366                         blk:   blk,
367                 }
368         })
369         close(todo)
370         wg.Wait()
371 }
372
373 func (bal *Balancer) setupServiceRoots() {
374         bal.serviceRoots = make(map[string]string)
375         for _, srv := range bal.KeepServices {
376                 bal.serviceRoots[srv.UUID] = srv.UUID
377         }
378 }
379
380 const (
381         changeStay = iota
382         changePull
383         changeTrash
384         changeNone
385 )
386
387 var changeName = map[int]string{
388         changeStay:  "stay",
389         changePull:  "pull",
390         changeTrash: "trash",
391         changeNone:  "none",
392 }
393
394 // balanceBlock compares current state to desired state for a single
395 // block, and makes the appropriate ChangeSet calls.
396 func (bal *Balancer) balanceBlock(blkid arvados.SizedDigest, blk *BlockState) {
397         debugf("balanceBlock: %v %+v", blkid, blk)
398         uuids := keepclient.NewRootSorter(bal.serviceRoots, string(blkid[:32])).GetSortedRoots()
399         hasRepl := make(map[string]Replica, len(bal.serviceRoots))
400         for _, repl := range blk.Replicas {
401                 hasRepl[repl.UUID] = repl
402                 // TODO: when multiple copies are on one server, use
403                 // the oldest one that doesn't have a timestamp
404                 // collision with other replicas.
405         }
406         // number of replicas already found in positions better than
407         // the position we're contemplating now.
408         reportedBestRepl := 0
409         // To be safe we assume two replicas with the same Mtime are
410         // in fact the same replica being reported more than
411         // once. len(uniqueBestRepl) is the number of distinct
412         // replicas in the best rendezvous positions we've considered
413         // so far.
414         uniqueBestRepl := make(map[int64]bool, len(bal.serviceRoots))
415         // pulls is the number of Pull changes we have already
416         // requested. (For purposes of deciding whether to Pull to
417         // rendezvous position N, we should assume all pulls we have
418         // requested on rendezvous positions M<N will be successful.)
419         pulls := 0
420         var changes []string
421         for _, uuid := range uuids {
422                 change := changeNone
423                 srv := bal.KeepServices[uuid]
424                 // TODO: request a Touch if Mtime is duplicated.
425                 repl, ok := hasRepl[srv.UUID]
426                 if ok {
427                         // This service has a replica. We should
428                         // delete it if [1] we already have enough
429                         // distinct replicas in better rendezvous
430                         // positions and [2] this replica's Mtime is
431                         // distinct from all of the better replicas'
432                         // Mtimes.
433                         if !srv.ReadOnly &&
434                                 repl.Mtime < bal.MinMtime &&
435                                 len(uniqueBestRepl) >= blk.Desired &&
436                                 !uniqueBestRepl[repl.Mtime] {
437                                 srv.AddTrash(Trash{
438                                         SizedDigest: blkid,
439                                         Mtime:       repl.Mtime,
440                                 })
441                                 change = changeTrash
442                         } else {
443                                 change = changeStay
444                         }
445                         uniqueBestRepl[repl.Mtime] = true
446                         reportedBestRepl++
447                 } else if pulls+reportedBestRepl < blk.Desired &&
448                         len(blk.Replicas) > 0 &&
449                         !srv.ReadOnly {
450                         // This service doesn't have a replica. We
451                         // should pull one to this server if we don't
452                         // already have enough (existing+requested)
453                         // replicas in better rendezvous positions.
454                         srv.AddPull(Pull{
455                                 SizedDigest: blkid,
456                                 Source:      blk.Replicas[0].KeepService,
457                         })
458                         pulls++
459                         change = changePull
460                 }
461                 if bal.Dumper != nil {
462                         changes = append(changes, fmt.Sprintf("%s:%d=%s,%d", srv.ServiceHost, srv.ServicePort, changeName[change], repl.Mtime))
463                 }
464         }
465         if bal.Dumper != nil {
466                 bal.Dumper.Printf("%s have=%d want=%d %s", blkid, len(blk.Replicas), blk.Desired, strings.Join(changes, " "))
467         }
468 }
469
470 type blocksNBytes struct {
471         replicas int
472         blocks   int
473         bytes    int64
474 }
475
476 func (bb blocksNBytes) String() string {
477         return fmt.Sprintf("%d replicas (%d blocks, %d bytes)", bb.replicas, bb.blocks, bb.bytes)
478 }
479
480 type balancerStats struct {
481         lost, overrep, unref, garbage, underrep, justright blocksNBytes
482         desired, current                                   blocksNBytes
483         pulls, trashes                                     int
484         replHistogram                                      []int
485 }
486
487 func (bal *Balancer) getStatistics() (s balancerStats) {
488         s.replHistogram = make([]int, 2)
489         bal.BlockStateMap.Apply(func(blkid arvados.SizedDigest, blk *BlockState) {
490                 surplus := len(blk.Replicas) - blk.Desired
491                 bytes := blkid.Size()
492                 switch {
493                 case len(blk.Replicas) == 0 && blk.Desired > 0:
494                         s.lost.replicas -= surplus
495                         s.lost.blocks++
496                         s.lost.bytes += bytes * int64(-surplus)
497                 case len(blk.Replicas) < blk.Desired:
498                         s.underrep.replicas -= surplus
499                         s.underrep.blocks++
500                         s.underrep.bytes += bytes * int64(-surplus)
501                 case len(blk.Replicas) > 0 && blk.Desired == 0:
502                         counter := &s.garbage
503                         for _, r := range blk.Replicas {
504                                 if r.Mtime >= bal.MinMtime {
505                                         counter = &s.unref
506                                         break
507                                 }
508                         }
509                         counter.replicas += surplus
510                         counter.blocks++
511                         counter.bytes += bytes * int64(surplus)
512                 case len(blk.Replicas) > blk.Desired:
513                         s.overrep.replicas += surplus
514                         s.overrep.blocks++
515                         s.overrep.bytes += bytes * int64(len(blk.Replicas)-blk.Desired)
516                 default:
517                         s.justright.replicas += blk.Desired
518                         s.justright.blocks++
519                         s.justright.bytes += bytes * int64(blk.Desired)
520                 }
521
522                 if blk.Desired > 0 {
523                         s.desired.replicas += blk.Desired
524                         s.desired.blocks++
525                         s.desired.bytes += bytes * int64(blk.Desired)
526                 }
527                 if len(blk.Replicas) > 0 {
528                         s.current.replicas += len(blk.Replicas)
529                         s.current.blocks++
530                         s.current.bytes += bytes * int64(len(blk.Replicas))
531                 }
532
533                 for len(s.replHistogram) <= len(blk.Replicas) {
534                         s.replHistogram = append(s.replHistogram, 0)
535                 }
536                 s.replHistogram[len(blk.Replicas)]++
537         })
538         for _, srv := range bal.KeepServices {
539                 s.pulls += len(srv.ChangeSet.Pulls)
540                 s.trashes += len(srv.ChangeSet.Trashes)
541         }
542         return
543 }
544
545 // PrintStatistics writes statistics about the computed changes to
546 // bal.Logger. It should not be called until ComputeChangeSets has
547 // finished.
548 func (bal *Balancer) PrintStatistics() {
549         s := bal.getStatistics()
550         bal.logf("===")
551         bal.logf("%s lost (0=have<want)", s.lost)
552         bal.logf("%s underreplicated (0<have<want)", s.underrep)
553         bal.logf("%s just right (have=want)", s.justright)
554         bal.logf("%s overreplicated (have>want>0)", s.overrep)
555         bal.logf("%s unreferenced (have>want=0, new)", s.unref)
556         bal.logf("%s garbage (have>want=0, old)", s.garbage)
557         bal.logf("===")
558         bal.logf("%s total commitment (excluding unreferenced)", s.desired)
559         bal.logf("%s total usage", s.current)
560         bal.logf("===")
561         for _, srv := range bal.KeepServices {
562                 bal.logf("%s: %v\n", srv, srv.ChangeSet)
563         }
564         bal.logf("===")
565         bal.printHistogram(s, 60)
566         bal.logf("===")
567 }
568
569 func (bal *Balancer) printHistogram(s balancerStats, hashColumns int) {
570         bal.logf("Replication level distribution (counting N replicas on a single server as N):")
571         maxCount := 0
572         for _, count := range s.replHistogram {
573                 if maxCount < count {
574                         maxCount = count
575                 }
576         }
577         hashes := strings.Repeat("#", hashColumns)
578         countWidth := 1 + int(math.Log10(float64(maxCount+1)))
579         scaleCount := 10 * float64(hashColumns) / math.Floor(1+10*math.Log10(float64(maxCount+1)))
580         for repl, count := range s.replHistogram {
581                 nHashes := int(scaleCount * math.Log10(float64(count+1)))
582                 bal.logf("%2d: %*d %s", repl, countWidth, count, hashes[:nHashes])
583         }
584 }
585
586 // CheckSanityLate checks for configuration and runtime errors after
587 // GetCurrentState() and ComputeChangeSets() have finished.
588 //
589 // If it returns an error, it is dangerous to run any Commit methods.
590 func (bal *Balancer) CheckSanityLate() error {
591         if bal.errors != nil {
592                 for _, err := range bal.errors {
593                         bal.logf("deferred error: %v", err)
594                 }
595                 return fmt.Errorf("cannot proceed safely after deferred errors")
596         }
597
598         if bal.collScanned == 0 {
599                 return fmt.Errorf("received zero collections")
600         }
601
602         anyDesired := false
603         bal.BlockStateMap.Apply(func(_ arvados.SizedDigest, blk *BlockState) {
604                 if blk.Desired > 0 {
605                         anyDesired = true
606                 }
607         })
608         if !anyDesired {
609                 return fmt.Errorf("zero blocks have desired replication>0")
610         }
611
612         if dr := bal.DefaultReplication; dr < 1 {
613                 return fmt.Errorf("Default replication (%d) is less than 1", dr)
614         }
615
616         // TODO: no two services have identical indexes
617         // TODO: no collisions (same md5, different size)
618         return nil
619 }
620
621 // CommitPulls sends the computed lists of pull requests to the
622 // keepstore servers. This has the effect of increasing replication of
623 // existing blocks that are either underreplicated or poorly
624 // distributed according to rendezvous hashing.
625 func (bal *Balancer) CommitPulls(c *arvados.Client) error {
626         return bal.commitAsync(c, "send pull list",
627                 func(srv *KeepService) error {
628                         return srv.CommitPulls(c)
629                 })
630 }
631
632 // CommitTrash sends the computed lists of trash requests to the
633 // keepstore servers. This has the effect of deleting blocks that are
634 // overreplicated or unreferenced.
635 func (bal *Balancer) CommitTrash(c *arvados.Client) error {
636         return bal.commitAsync(c, "send trash list",
637                 func(srv *KeepService) error {
638                         return srv.CommitTrash(c)
639                 })
640 }
641
642 func (bal *Balancer) commitAsync(c *arvados.Client, label string, f func(srv *KeepService) error) error {
643         errs := make(chan error)
644         for _, srv := range bal.KeepServices {
645                 go func(srv *KeepService) {
646                         var err error
647                         defer func() { errs <- err }()
648                         label := fmt.Sprintf("%s: %v", srv, label)
649                         defer timeMe(bal.Logger, label)()
650                         err = f(srv)
651                         if err != nil {
652                                 err = fmt.Errorf("%s: %v", label, err)
653                         }
654                 }(srv)
655         }
656         var lastErr error
657         for range bal.KeepServices {
658                 if err := <-errs; err != nil {
659                         bal.logf("%v", err)
660                         lastErr = err
661                 }
662         }
663         close(errs)
664         return lastErr
665 }
666
667 func (bal *Balancer) logf(f string, args ...interface{}) {
668         if bal.Logger != nil {
669                 bal.Logger.Printf(f, args...)
670         }
671 }