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