github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/storage/bootstrap/bootstrapper/peers/source.go (about) 1 // Copyright (c) 2016 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package peers 22 23 import ( 24 "errors" 25 "fmt" 26 "io" 27 "sync" 28 "time" 29 30 "github.com/opentracing/opentracing-go" 31 "go.uber.org/zap" 32 "go.uber.org/zap/zapcore" 33 34 "github.com/m3db/m3/src/cluster/shard" 35 "github.com/m3db/m3/src/dbnode/client" 36 "github.com/m3db/m3/src/dbnode/namespace" 37 "github.com/m3db/m3/src/dbnode/persist" 38 "github.com/m3db/m3/src/dbnode/persist/fs" 39 "github.com/m3db/m3/src/dbnode/storage/block" 40 "github.com/m3db/m3/src/dbnode/storage/bootstrap" 41 "github.com/m3db/m3/src/dbnode/storage/bootstrap/bootstrapper" 42 "github.com/m3db/m3/src/dbnode/storage/bootstrap/result" 43 "github.com/m3db/m3/src/dbnode/storage/index" 44 "github.com/m3db/m3/src/dbnode/storage/index/compaction" 45 "github.com/m3db/m3/src/dbnode/storage/index/convert" 46 "github.com/m3db/m3/src/dbnode/storage/series" 47 "github.com/m3db/m3/src/dbnode/topology" 48 "github.com/m3db/m3/src/m3ninx/doc" 49 "github.com/m3db/m3/src/m3ninx/index/segment/fst" 50 idxpersist "github.com/m3db/m3/src/m3ninx/persist" 51 "github.com/m3db/m3/src/x/context" 52 "github.com/m3db/m3/src/x/ident" 53 "github.com/m3db/m3/src/x/instrument" 54 xresource "github.com/m3db/m3/src/x/resource" 55 xsync "github.com/m3db/m3/src/x/sync" 56 xtime "github.com/m3db/m3/src/x/time" 57 ) 58 59 var errNamespaceNotFound = errors.New("namespace not found") 60 61 const readSeriesBlocksWorkerChannelSize = 512 62 63 type peersSource struct { 64 opts Options 65 newPersistManager func() (persist.Manager, error) 66 log *zap.Logger 67 instrumentation *instrumentation 68 } 69 70 type persistenceFlush struct { 71 nsMetadata namespace.Metadata 72 shard uint32 73 shardResult result.ShardResult 74 timeRange xtime.Range 75 } 76 77 func newPeersSource(opts Options) (bootstrap.Source, error) { 78 if err := opts.Validate(); err != nil { 79 return nil, err 80 } 81 82 instrumentation := newInstrumentation(opts) 83 return &peersSource{ 84 opts: opts, 85 newPersistManager: func() (persist.Manager, error) { 86 return fs.NewPersistManager(opts.FilesystemOptions()) 87 }, 88 log: instrumentation.log, 89 instrumentation: instrumentation, 90 }, nil 91 } 92 93 type shardPeerAvailability struct { 94 numPeers int 95 numAvailablePeers int 96 } 97 98 func (s *peersSource) AvailableData( 99 nsMetadata namespace.Metadata, 100 shardTimeRanges result.ShardTimeRanges, 101 _ bootstrap.Cache, 102 runOpts bootstrap.RunOptions, 103 ) (result.ShardTimeRanges, error) { 104 if err := s.validateRunOpts(runOpts); err != nil { 105 return nil, err 106 } 107 return s.peerAvailability(nsMetadata, shardTimeRanges, runOpts) 108 } 109 110 func (s *peersSource) AvailableIndex( 111 nsMetadata namespace.Metadata, 112 shardTimeRanges result.ShardTimeRanges, 113 _ bootstrap.Cache, 114 runOpts bootstrap.RunOptions, 115 ) (result.ShardTimeRanges, error) { 116 if err := s.validateRunOpts(runOpts); err != nil { 117 return nil, err 118 } 119 return s.peerAvailability(nsMetadata, shardTimeRanges, runOpts) 120 } 121 122 func (s *peersSource) Read( 123 ctx context.Context, 124 namespaces bootstrap.Namespaces, 125 cache bootstrap.Cache, 126 ) (bootstrap.NamespaceResults, error) { 127 instrCtx := s.instrumentation.peersBootstrapperSourceReadStarted(ctx) 128 defer instrCtx.finish() 129 130 timeRangesEmpty := true 131 for _, elem := range namespaces.Namespaces.Iter() { 132 namespace := elem.Value() 133 dataRangesNotEmpty := !namespace.DataRunOptions.ShardTimeRanges.IsEmpty() 134 135 indexEnabled := namespace.Metadata.Options().IndexOptions().Enabled() 136 indexRangesNotEmpty := indexEnabled && !namespace.IndexRunOptions.ShardTimeRanges.IsEmpty() 137 if dataRangesNotEmpty || indexRangesNotEmpty { 138 timeRangesEmpty = false 139 break 140 } 141 } 142 if timeRangesEmpty { 143 // Return empty result with no unfulfilled ranges. 144 return bootstrap.NewNamespaceResults(namespaces), nil 145 } 146 147 results := bootstrap.NamespaceResults{ 148 Results: bootstrap.NewNamespaceResultsMap(bootstrap.NamespaceResultsMapOptions{}), 149 } 150 151 // NB(r): Perform all data bootstrapping first then index bootstrapping 152 // to more clearly delineate which process is slower than the other. 153 instrCtx.bootstrapDataStarted() 154 for _, elem := range namespaces.Namespaces.Iter() { 155 namespace := elem.Value() 156 md := namespace.Metadata 157 158 r, err := s.readData(md, namespace.DataAccumulator, 159 namespace.DataRunOptions.ShardTimeRanges, 160 namespace.DataRunOptions.RunOptions) 161 if err != nil { 162 return bootstrap.NamespaceResults{}, err 163 } 164 165 results.Results.Set(md.ID(), bootstrap.NamespaceResult{ 166 Metadata: md, 167 Shards: namespace.Shards, 168 DataResult: r, 169 }) 170 } 171 instrCtx.bootstrapDataCompleted() 172 // NB(bodu): We need to evict the info file cache before reading index data since we've 173 // maybe fetched blocks from peers so the cached info file state is now stale. 174 cache.Evict() 175 176 instrCtx.bootstrapIndexStarted() 177 for _, elem := range namespaces.Namespaces.Iter() { 178 namespace := elem.Value() 179 md := namespace.Metadata 180 if !md.Options().IndexOptions().Enabled() { 181 s.log.Info("skipping bootstrap for namespace based on options", 182 zap.Stringer("namespace", md.ID())) 183 184 // Not bootstrapping for index. 185 continue 186 } 187 188 var ( 189 opts = namespace.IndexRunOptions.RunOptions 190 r result.IndexBootstrapResult 191 err error 192 ) 193 if s.shouldPersist(opts) { 194 // Only attempt to bootstrap index if we've persisted tsdb data. 195 r, err = s.readIndex(md, 196 namespace.IndexRunOptions.ShardTimeRanges, 197 instrCtx.span, 198 cache, 199 opts, 200 ) 201 if err != nil { 202 return bootstrap.NamespaceResults{}, err 203 } 204 } else { 205 // Copy data unfulfilled ranges over to index results 206 // we did not persist any tsdb data (e.g. snapshot data). 207 dataNsResult, ok := results.Results.Get(md.ID()) 208 if !ok { 209 return bootstrap.NamespaceResults{}, errNamespaceNotFound 210 } 211 r = result.NewIndexBootstrapResult() 212 r.SetUnfulfilled(dataNsResult.DataResult.Unfulfilled().Copy()) 213 } 214 215 result, ok := results.Results.Get(md.ID()) 216 if !ok { 217 err = fmt.Errorf("missing expected result for namespace: %s", 218 md.ID().String()) 219 return bootstrap.NamespaceResults{}, err 220 } 221 222 result.IndexResult = r 223 224 results.Results.Set(md.ID(), result) 225 } 226 instrCtx.bootstrapIndexCompleted() 227 228 return results, nil 229 } 230 231 func (s *peersSource) readData( 232 nsMetadata namespace.Metadata, 233 accumulator bootstrap.NamespaceDataAccumulator, 234 shardTimeRanges result.ShardTimeRanges, 235 opts bootstrap.RunOptions, 236 ) (result.DataBootstrapResult, error) { 237 if err := s.validateRunOpts(opts); err != nil { 238 return nil, err 239 } 240 241 if shardTimeRanges.IsEmpty() { 242 return result.NewDataBootstrapResult(), nil 243 } 244 245 shouldPersist := s.shouldPersist(opts) 246 result := result.NewDataBootstrapResult() 247 session, err := s.opts.AdminClient().DefaultAdminSession() 248 if err != nil { 249 s.log.Error("peers bootstrapper cannot get default admin session", zap.Error(err)) 250 result.SetUnfulfilled(shardTimeRanges) 251 return nil, err 252 } 253 254 var ( 255 resultLock sync.Mutex 256 persistenceMaxQueueSize = s.opts.PersistenceMaxQueueSize() 257 persistenceQueue = make(chan persistenceFlush, persistenceMaxQueueSize) 258 resultOpts = s.opts.ResultOptions() 259 count = shardTimeRanges.Len() 260 concurrency = s.opts.DefaultShardConcurrency() 261 blockSize = nsMetadata.Options().RetentionOptions().BlockSize() 262 persistWg = &sync.WaitGroup{} 263 persistClosers []io.Closer 264 ) 265 if shouldPersist { 266 concurrency = s.opts.ShardPersistenceConcurrency() 267 } 268 269 instrCtx := s.instrumentation.bootstrapShardsStarted(nsMetadata.ID(), count, concurrency, shouldPersist) 270 defer instrCtx.bootstrapShardsCompleted() 271 if shouldPersist { 272 // Spin up persist workers. 273 for i := 0; i < s.opts.ShardPersistenceFlushConcurrency(); i++ { 274 closer, err := s.startPersistenceQueueWorkerLoop(opts, 275 persistWg, persistenceQueue, result, &resultLock) 276 if err != nil { 277 return nil, err 278 } 279 280 persistClosers = append(persistClosers, closer) 281 } 282 } 283 284 var ( 285 wg sync.WaitGroup 286 workers = xsync.NewWorkerPool(concurrency) 287 ) 288 workers.Init() 289 for shard, ranges := range shardTimeRanges.Iter() { 290 shard, ranges := shard, ranges 291 wg.Add(1) 292 workers.Go(func() { 293 defer wg.Done() 294 s.fetchBootstrapBlocksFromPeers(shard, ranges, nsMetadata, session, 295 accumulator, resultOpts, result, &resultLock, shouldPersist, 296 persistenceQueue, blockSize) 297 }) 298 } 299 300 wg.Wait() 301 close(persistenceQueue) 302 if shouldPersist { 303 // Wait for the persistenceQueue workers to finish flushing everything. 304 persistWg.Wait() 305 306 // Close any persist closers to finalize files written. 307 for _, closer := range persistClosers { 308 if err := closer.Close(); err != nil { 309 return nil, err 310 } 311 } 312 } 313 314 return result, nil 315 } 316 317 func (s *peersSource) startPersistenceQueueWorkerLoop( 318 opts bootstrap.RunOptions, 319 persistWg *sync.WaitGroup, 320 persistenceQueue chan persistenceFlush, 321 bootstrapResult result.DataBootstrapResult, 322 lock *sync.Mutex, 323 ) (io.Closer, error) { 324 persistMgr, err := s.newPersistManager() 325 if err != nil { 326 return nil, err 327 } 328 329 persistFlush, err := persistMgr.StartFlushPersist() 330 if err != nil { 331 return nil, err 332 } 333 334 persistWg.Add(1) 335 go func() { 336 defer persistWg.Done() 337 s.runPersistenceQueueWorkerLoop(opts, persistenceQueue, 338 persistFlush, bootstrapResult, lock) 339 }() 340 341 return xresource.CloserFn(persistFlush.DoneFlush), nil 342 } 343 344 // runPersistenceQueueWorkerLoop is meant to be run in its own goroutine, and it creates a worker that 345 // loops through the persistenceQueue and performs a flush for each entry, ensuring that 346 // no more than one flush is ever happening at once. Once the persistenceQueue channel 347 // is closed, and the worker has completed flushing all the remaining entries, it will close the 348 // provided doneCh so that callers can block until everything has been successfully flushed. 349 func (s *peersSource) runPersistenceQueueWorkerLoop( 350 opts bootstrap.RunOptions, 351 persistenceQueue chan persistenceFlush, 352 persistFlush persist.FlushPreparer, 353 bootstrapResult result.DataBootstrapResult, 354 lock *sync.Mutex, 355 ) { 356 // Track async cleanup tasks. 357 asyncTasks := &sync.WaitGroup{} 358 359 // Wait for cleanups to all occur before returning from worker. 360 defer asyncTasks.Wait() 361 362 // If performing a bootstrap with persistence enabled then flush one 363 // at a time as shard results are gathered. 364 for flush := range persistenceQueue { 365 err := s.flush(opts, persistFlush, flush.nsMetadata, flush.shard, 366 flush.shardResult, flush.timeRange, asyncTasks) 367 if err == nil { 368 continue 369 } 370 371 // Remove results and make unfulfilled if an error occurred. 372 s.log.Error("peers bootstrapper bootstrap with persistence flush encountered error", 373 zap.Error(err)) 374 375 // Make unfulfilled. 376 lock.Lock() 377 unfulfilled := bootstrapResult.Unfulfilled().Copy() 378 unfulfilled.AddRanges(result.NewShardTimeRanges().Set( 379 flush.shard, 380 xtime.NewRanges(flush.timeRange), 381 )) 382 bootstrapResult.SetUnfulfilled(unfulfilled) 383 lock.Unlock() 384 } 385 } 386 387 type seriesBlocks struct { 388 resolver bootstrap.SeriesRefResolver 389 blocks block.DatabaseSeriesBlocks 390 } 391 392 // fetchBootstrapBlocksFromPeers loops through all the provided ranges for a given shard and 393 // fetches all the bootstrap blocks from the appropriate peers. 394 // Persistence enabled case: Immediately add the results to the bootstrap result 395 // Persistence disabled case: Don't add the results yet, but push a flush into the 396 // persistenceQueue. The persistenceQueue worker will eventually 397 // add the results once its performed the flush. 398 func (s *peersSource) fetchBootstrapBlocksFromPeers( 399 shard uint32, 400 ranges xtime.Ranges, 401 nsMetadata namespace.Metadata, 402 session client.AdminSession, 403 accumulator bootstrap.NamespaceDataAccumulator, 404 bopts result.Options, 405 bootstrapResult result.DataBootstrapResult, 406 lock *sync.Mutex, 407 shouldPersist bool, 408 persistenceQueue chan persistenceFlush, 409 blockSize time.Duration, 410 ) { 411 it := ranges.Iter() 412 tagsIter := ident.NewTagsIterator(ident.Tags{}) 413 unfulfill := func(r xtime.Range) { 414 lock.Lock() 415 unfulfilled := bootstrapResult.Unfulfilled() 416 unfulfilled.AddRanges(result.NewShardTimeRanges().Set(shard, xtime.NewRanges(r))) 417 lock.Unlock() 418 } 419 for it.Next() { 420 currRange := it.Value() 421 422 for blockStart := currRange.Start; blockStart.Before(currRange.End); blockStart = blockStart.Add(blockSize) { 423 blockEnd := blockStart.Add(blockSize) 424 shardResult, err := session.FetchBootstrapBlocksFromPeers( 425 nsMetadata, shard, blockStart, blockEnd, bopts) 426 s.logFetchBootstrapBlocksFromPeersOutcome(shard, shardResult, err) 427 428 if err != nil { 429 // No result to add for this bootstrap. 430 unfulfill(currRange) 431 continue 432 } 433 434 if shouldPersist { 435 persistenceQueue <- persistenceFlush{ 436 nsMetadata: nsMetadata, 437 shard: shard, 438 shardResult: shardResult, 439 timeRange: xtime.Range{Start: blockStart, End: blockEnd}, 440 } 441 continue 442 } 443 444 dataCh := make(chan seriesBlocks, readSeriesBlocksWorkerChannelSize) 445 go func() { 446 defer close(dataCh) 447 for _, elem := range shardResult.AllSeries().Iter() { 448 entry := elem.Value() 449 tagsIter.Reset(entry.Tags) 450 ref, owned, err := accumulator.CheckoutSeriesWithLock(shard, entry.ID, tagsIter) 451 if err != nil { 452 if !owned { 453 // Only if we own this shard do we care consider this an 454 // error in bootstrapping. 455 continue 456 } 457 unfulfill(currRange) 458 s.log.Error("could not checkout series", zap.Error(err)) 459 continue 460 } 461 462 dataCh <- seriesBlocks{ 463 resolver: ref.Resolver, 464 blocks: entry.Blocks, 465 } 466 467 // Safe to finalize these IDs and Tags, shard result no longer used. 468 entry.ID.Finalize() 469 entry.Tags.Finalize() 470 } 471 }() 472 473 for seriesBlocks := range dataCh { 474 seriesRef, err := seriesBlocks.resolver.SeriesRef() 475 if err != nil { 476 s.log.Error("could not resolve seriesRef", zap.Error(err)) 477 unfulfill(currRange) 478 continue 479 } 480 481 for _, bl := range seriesBlocks.blocks.AllBlocks() { 482 if err := seriesRef.LoadBlock(bl, series.WarmWrite); err != nil { 483 unfulfill(currRange) 484 s.log.Error("could not load series block", zap.Error(err)) 485 } 486 } 487 } 488 } 489 } 490 } 491 492 func (s *peersSource) logFetchBootstrapBlocksFromPeersOutcome( 493 shard uint32, 494 shardResult result.ShardResult, 495 err error, 496 ) { 497 if err != nil { 498 s.log.Error("error fetching bootstrap blocks", 499 zap.Uint32("shard", shard), 500 zap.Error(err), 501 ) 502 return 503 } 504 505 shardBlockSeriesCounter := map[xtime.UnixNano]int64{} 506 for _, entry := range shardResult.AllSeries().Iter() { // nolint 507 series := entry.Value() 508 for blockStart := range series.Blocks.AllBlocks() { 509 shardBlockSeriesCounter[blockStart]++ 510 } 511 } 512 513 for block, numSeries := range shardBlockSeriesCounter { 514 s.log.Info("peer bootstrapped shard", 515 zap.Uint32("shard", shard), 516 zap.Int64("numSeries", numSeries), 517 zap.Time("blockStart", block.ToTime()), 518 ) 519 } 520 } 521 522 // flush is used to flush peer-bootstrapped shards to disk as they finish so 523 // that we're not (necessarily) holding everything in memory at once. 524 // flush starts by looping through every block in a timerange for 525 // a given shard, and then subsequently looping through every series in that 526 // shard/block and flushing it to disk. Depending on the series caching policy, 527 // the series will either be held in memory, or removed from memory once 528 // flushing has completed. 529 // In addition, if the caching policy is not CacheAll, then 530 // at the end we remove all the series objects from the shard result as well 531 // (since all their corresponding blocks have been removed anyways) to prevent 532 // a huge memory spike caused by adding lots of unused series to the Shard 533 // object and then immediately evicting them in the next tick. 534 func (s *peersSource) flush( 535 opts bootstrap.RunOptions, 536 flush persist.FlushPreparer, 537 nsMetadata namespace.Metadata, 538 shard uint32, 539 shardResult result.ShardResult, 540 tr xtime.Range, 541 asyncTasks *sync.WaitGroup, 542 ) error { 543 persistConfig := opts.PersistConfig() 544 if persistConfig.FileSetType != persist.FileSetFlushType { 545 // Should never happen. 546 iOpts := s.opts.ResultOptions().InstrumentOptions() 547 instrument.EmitAndLogInvariantViolation(iOpts, func(l *zap.Logger) { 548 l.With( 549 zap.Stringer("namespace", nsMetadata.ID()), 550 zap.Any("filesetType", persistConfig.FileSetType), 551 ).Error("error tried to persist data in peers bootstrapper with non-flush fileset type") 552 }) 553 return instrument.InvariantErrorf( 554 "tried to flush with unexpected fileset type: %v", persistConfig.FileSetType) 555 } 556 557 seriesCachePolicy := s.opts.ResultOptions().SeriesCachePolicy() 558 if seriesCachePolicy == series.CacheAll { 559 // Should never happen. 560 iOpts := s.opts.ResultOptions().InstrumentOptions() 561 instrument.EmitAndLogInvariantViolation(iOpts, func(l *zap.Logger) { 562 l.With( 563 zap.Stringer("namespace", nsMetadata.ID()), 564 zap.Any("cachePolicy", seriesCachePolicy), 565 ).Error("error tried to persist data in peers bootstrapper with invalid cache policy") 566 }) 567 return instrument.InvariantErrorf( 568 "tried to persist data in peers bootstrapper with invalid cache policy: %v", seriesCachePolicy) 569 } 570 571 var ( 572 ropts = nsMetadata.Options().RetentionOptions() 573 blockSize = ropts.BlockSize() 574 ) 575 for start := tr.Start; start.Before(tr.End); start = start.Add(blockSize) { 576 prepareOpts := persist.DataPrepareOptions{ 577 NamespaceMetadata: nsMetadata, 578 FileSetType: persistConfig.FileSetType, 579 Shard: shard, 580 BlockStart: start, 581 // When bootstrapping, the volume index will always be 0. However, 582 // if we want to be able to snapshot and flush while bootstrapping, 583 // this may not be the case, e.g. if a flush occurs before a 584 // bootstrap, then the bootstrap volume index will be >0. In order 585 // to support this, bootstrapping code will need to incorporate 586 // merging logic and flush version/volume index will need to be 587 // synchronized between processes. 588 VolumeIndex: 0, 589 // If we've peer bootstrapped this shard/block combination AND the fileset 590 // already exists on disk, then that means either: 591 // 1) The Filesystem bootstrapper was unable to bootstrap the fileset 592 // files on disk, even though they have a checkpoint file. This 593 // could either be the result of data corruption, or a 594 // backwards-incompatible change to the file-format. 595 // 2) The Filesystem bootstrapper is not enabled, in which case it makes 596 // complete sense to replaces the fileset on disk with the one which 597 // we just peer-bootstrapped because the operator has already made it 598 // clear that they only want data to be returned if it came from peers 599 // (they made this decision by turning off the Filesystem bootstrapper). 600 // 3) We have received a shard/block we previously owned. For example, when a 601 // node was added to this replication group and was later removed. 602 // Although we take writes while bootstrapping, we do not allow flushes 603 // so it is safe to delete on disk data. 604 DeleteIfExists: true, 605 } 606 prepared, err := flush.PrepareData(prepareOpts) 607 if err != nil { 608 return err 609 } 610 611 var blockErr error 612 for _, entry := range shardResult.AllSeries().Iter() { 613 s := entry.Value() 614 bl, ok := s.Blocks.BlockAt(start) 615 if !ok { 616 continue 617 } 618 619 checksum, err := bl.Checksum() 620 if err != nil { 621 blockErr = err // Need to call prepared.Close, avoid return 622 break 623 } 624 625 // Discard and finalize the block. 626 segment := bl.Discard() 627 628 // Remove from map. 629 s.Blocks.RemoveBlockAt(start) 630 631 metadata := persist.NewMetadataFromIDAndTags(s.ID, s.Tags, 632 persist.MetadataOptions{}) 633 err = prepared.Persist(metadata, segment, checksum) 634 if err != nil { 635 blockErr = err // Need to call prepared.Close, avoid return 636 break 637 } 638 } 639 640 // Always close before attempting to check if block error occurred, 641 // avoid using a defer here as this needs to be done for each inner loop 642 err = prepared.Close() 643 if blockErr != nil { 644 // A block error is more interesting to bubble up than a close error 645 err = blockErr 646 } 647 648 if err != nil { 649 return err 650 } 651 } 652 653 // Perform cleanup async but allow caller to wait on them. 654 // This allows to progress to next flush faster. 655 asyncTasks.Add(1) 656 go func() { 657 defer asyncTasks.Done() 658 659 // Since we've persisted the data to disk, we don't want to keep all the series in the shard 660 // result. Otherwise if we leave them in, then they will all get loaded into the shard object, 661 // and then immediately evicted on the next tick which causes unnecessary memory pressure 662 // during peer bootstrapping. 663 numSeriesTriedToRemoveWithRemainingBlocks := 0 664 for _, entry := range shardResult.AllSeries().Iter() { 665 series := entry.Value() 666 numBlocksRemaining := len(series.Blocks.AllBlocks()) 667 // Should never happen since we removed all the block in the previous loop and fetching 668 // bootstrap blocks should always be exclusive on the end side. 669 if numBlocksRemaining > 0 { 670 numSeriesTriedToRemoveWithRemainingBlocks++ 671 continue 672 } 673 674 shardResult.RemoveSeries(series.ID) 675 series.Blocks.Close() 676 // Safe to finalize these IDs and Tags because the prepared object was the only other thing 677 // using them, and it has been closed. 678 series.ID.Finalize() 679 series.Tags.Finalize() 680 } 681 if numSeriesTriedToRemoveWithRemainingBlocks > 0 { 682 iOpts := s.opts.ResultOptions().InstrumentOptions() 683 instrument.EmitAndLogInvariantViolation(iOpts, func(l *zap.Logger) { 684 l.With( 685 zap.Int64("start", tr.Start.Seconds()), 686 zap.Int64("end", tr.End.Seconds()), 687 zap.Int("numTimes", numSeriesTriedToRemoveWithRemainingBlocks), 688 ).Error("error tried to remove series that still has blocks") 689 }) 690 } 691 }() 692 693 return nil 694 } 695 696 func (s *peersSource) readIndex( 697 ns namespace.Metadata, 698 shardTimeRanges result.ShardTimeRanges, 699 span opentracing.Span, 700 cache bootstrap.Cache, 701 opts bootstrap.RunOptions, 702 ) (result.IndexBootstrapResult, error) { 703 if err := s.validateRunOpts(opts); err != nil { 704 return nil, err 705 } 706 707 // FOLLOWUP(r): Try to reuse any metadata fetched during the ReadData(...) 708 // call rather than going to the network again 709 r := result.NewIndexBootstrapResult() 710 if shardTimeRanges.IsEmpty() { 711 return r, nil 712 } 713 714 var ( 715 count = shardTimeRanges.Len() 716 indexBlockSize = ns.Options().IndexOptions().BlockSize() 717 runtimeOpts = s.opts.RuntimeOptionsManager().Get() 718 fsOpts = s.opts.FilesystemOptions() 719 idxOpts = ns.Options().IndexOptions() 720 readerPool = bootstrapper.NewReaderPool(bootstrapper.NewReaderPoolOptions{ 721 Alloc: func() (fs.DataFileSetReader, error) { 722 bytesPool := s.opts.ResultOptions().DatabaseBlockOptions().BytesPool() 723 return fs.NewReader(bytesPool, fsOpts) 724 }, 725 }) 726 resultLock = &sync.Mutex{} 727 indexSegmentConcurrency = s.opts.IndexSegmentConcurrency() 728 readersCh = make(chan bootstrapper.TimeWindowReaders, indexSegmentConcurrency) 729 ) 730 s.log.Info("peers bootstrapper bootstrapping index for ranges", 731 zap.Int("shards", count)) 732 733 go bootstrapper.EnqueueReaders(bootstrapper.EnqueueReadersOptions{ 734 NsMD: ns, 735 RunOpts: opts, 736 RuntimeOpts: runtimeOpts, 737 FsOpts: fsOpts, 738 ShardTimeRanges: shardTimeRanges, 739 ReaderPool: readerPool, 740 ReadersCh: readersCh, 741 BlockSize: indexBlockSize, 742 // NB(bodu): We only read metadata when performing a peers bootstrap 743 // so we do not need to sort the data fileset reader. 744 ReadMetadataOnly: true, 745 Logger: s.instrumentation.log, 746 Span: span, 747 NowFn: s.instrumentation.nowFn, 748 Cache: cache, 749 }) 750 751 var buildWg sync.WaitGroup 752 for i := 0; i < indexSegmentConcurrency; i++ { 753 alloc := s.opts.ResultOptions().IndexDocumentsBuilderAllocator() 754 segBuilder, err := alloc() 755 if err != nil { 756 return nil, err 757 } 758 759 builder := result.NewIndexBuilder(segBuilder) 760 761 indexOpts := s.opts.IndexOptions() 762 compactor, err := compaction.NewCompactor(indexOpts.MetadataArrayPool(), 763 index.MetadataArrayPoolCapacity, 764 indexOpts.SegmentBuilderOptions(), 765 indexOpts.FSTSegmentOptions(), 766 compaction.CompactorOptions{ 767 FSTWriterOptions: &fst.WriterOptions{ 768 // DisableRegistry is set to true to trade a larger FST size 769 // for a faster FST compaction since we want to reduce the end 770 // to end latency for time to first index a metric. 771 DisableRegistry: true, 772 }, 773 }) 774 if err != nil { 775 return nil, err 776 } 777 778 persistManager, err := s.newPersistManager() 779 if err != nil { 780 return nil, err 781 } 782 783 buildWg.Add(1) 784 go func() { 785 s.processReadersWorker(ns, r, readersCh, builder, readerPool, idxOpts, 786 &bootstrapper.SharedPersistManager{Mgr: persistManager}, 787 &bootstrapper.SharedCompactor{Compactor: compactor}, 788 resultLock) 789 buildWg.Done() 790 }() 791 } 792 793 buildWg.Wait() 794 795 return r, nil 796 } 797 798 func (s *peersSource) processReadersWorker( 799 ns namespace.Metadata, 800 r result.IndexBootstrapResult, 801 readersCh <-chan bootstrapper.TimeWindowReaders, 802 builder *result.IndexBuilder, 803 readerPool *bootstrapper.ReaderPool, 804 idxOpts namespace.IndexOptions, 805 persistManager *bootstrapper.SharedPersistManager, 806 compactor *bootstrapper.SharedCompactor, 807 resultLock *sync.Mutex, 808 ) { 809 for timeWindowReaders := range readersCh { 810 // NB(bodu): Since we are re-using the same builder for all bootstrapped index blocks, 811 // it is not thread safe and requires reset after every processed index block. 812 builder.Builder().Reset() 813 814 // NB(bodu): This is fetching the data for all shards for a block of time. 815 remainingRanges, timesWithErrors := s.processReaders( 816 ns, 817 r, 818 builder, 819 timeWindowReaders, 820 readerPool, 821 idxOpts, 822 persistManager, 823 compactor, 824 resultLock, 825 ) 826 s.markRunResultErrorsAndUnfulfilled(resultLock, r, timeWindowReaders.Ranges, 827 remainingRanges, timesWithErrors) 828 } 829 } 830 831 func (s *peersSource) processReaders( 832 ns namespace.Metadata, 833 r result.IndexBootstrapResult, 834 builder *result.IndexBuilder, 835 timeWindowReaders bootstrapper.TimeWindowReaders, 836 readerPool *bootstrapper.ReaderPool, 837 idxOpts namespace.IndexOptions, 838 persistManager *bootstrapper.SharedPersistManager, 839 compactor *bootstrapper.SharedCompactor, 840 resultLock *sync.Mutex, 841 ) (result.ShardTimeRanges, []xtime.UnixNano) { 842 var ( 843 metadataPool = s.opts.IndexOptions().MetadataArrayPool() 844 batch = metadataPool.Get() 845 timesWithErrors []xtime.UnixNano 846 totalEntries int 847 ) 848 849 defer func() { 850 metadataPool.Put(batch) 851 // Return readers to pool. 852 for _, shardReaders := range timeWindowReaders.Readers { 853 for _, r := range shardReaders.Readers { 854 if err := r.Close(); err == nil { 855 readerPool.Put(r) 856 } 857 } 858 } 859 }() 860 861 requestedRanges := timeWindowReaders.Ranges 862 remainingRanges := requestedRanges.Copy() 863 for shard, shardReaders := range timeWindowReaders.Readers { 864 shard := uint32(shard) 865 readers := shardReaders.Readers 866 867 for _, reader := range readers { 868 var ( 869 timeRange = reader.Range() 870 start = timeRange.Start 871 err error 872 ) 873 874 resultLock.Lock() 875 r.IndexResults().AddBlockIfNotExists(start, idxOpts) 876 resultLock.Unlock() 877 numEntries := reader.Entries() 878 for i := 0; err == nil && i < numEntries; i++ { 879 batch, err = s.readNextEntryAndMaybeIndex(reader, batch, builder) 880 totalEntries++ 881 } 882 883 // NB(bodu): Only flush if we've experienced no errors up until this point. 884 if err == nil && len(batch) > 0 { 885 batch, err = builder.FlushBatch(batch) 886 } 887 888 // Validate the read results 889 if err == nil { 890 err = reader.ValidateMetadata() 891 } 892 893 if err == nil { 894 // Mark index block as fulfilled. 895 fulfilled := result.NewShardTimeRanges().Set( 896 shard, 897 xtime.NewRanges(timeRange), 898 ) 899 resultLock.Lock() 900 err = r.IndexResults().MarkFulfilled(start, fulfilled, 901 // NB(bodu): By default, we always load bootstrapped data into the default index volume. 902 idxpersist.DefaultIndexVolumeType, idxOpts) 903 resultLock.Unlock() 904 } 905 906 if err == nil { 907 remainingRanges.Subtract(result.NewShardTimeRanges().Set( 908 shard, 909 xtime.NewRanges(timeRange), 910 )) 911 } else { 912 s.log.Error("error processing readers", zap.Error(err), 913 zap.Time("timeRange.start", start.ToTime())) 914 timesWithErrors = append(timesWithErrors, timeRange.Start) 915 } 916 } 917 } 918 if totalEntries == 0 { 919 // NB(r): Do not try to build a segment if no entries to index. 920 return remainingRanges, timesWithErrors 921 } 922 923 // Only persist to disk if the requested ranges were completely fulfilled. 924 // Otherwise, this is the latest index segment and should only exist in mem. 925 var ( 926 iopts = s.opts.ResultOptions().InstrumentOptions() 927 shouldPersist = remainingRanges.IsEmpty() 928 min, max = requestedRanges.MinMax() 929 indexBlockSize = ns.Options().IndexOptions().BlockSize() 930 blockStart = min.Truncate(indexBlockSize) 931 blockEnd = blockStart.Add(indexBlockSize) 932 indexBlock result.IndexBlock 933 err error 934 ) 935 936 // NB(bodu): Assume if we're bootstrapping data from disk that it is the "default" index volume type. 937 resultLock.Lock() 938 existingIndexBlock, ok := bootstrapper.GetDefaultIndexBlockForBlockStart( 939 r.IndexResults(), blockStart) 940 resultLock.Unlock() 941 942 if !ok { 943 err := fmt.Errorf("could not find index block in results: time=%s, ts=%d", 944 blockStart.String(), blockStart) 945 instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { 946 l.Error("peers bootstrap failed", 947 zap.Error(err), 948 zap.Stringer("namespace", ns.ID()), 949 zap.Stringer("requestedRanges", requestedRanges)) 950 }) 951 } 952 953 buildIndexLogFields := []zapcore.Field{ 954 zap.Bool("shouldPersist", shouldPersist), 955 zap.Int("totalEntries", totalEntries), 956 zap.String("requestedRanges", fmt.Sprintf("%v - %v", min, max)), 957 zap.String("timesWithErrors", fmt.Sprintf("%v", timesWithErrors)), 958 zap.String("remainingRanges", remainingRanges.SummaryString()), 959 } 960 if shouldPersist { 961 s.log.Debug("building file set index segment", buildIndexLogFields...) 962 indexBlock, err = bootstrapper.PersistBootstrapIndexSegment( 963 ns, 964 requestedRanges, 965 builder.Builder(), 966 persistManager, 967 s.opts.IndexClaimsManager(), 968 s.opts.ResultOptions(), 969 existingIndexBlock.Fulfilled(), 970 blockStart, 971 blockEnd, 972 ) 973 if errors.Is(err, fs.ErrIndexOutOfRetention) { 974 // Bail early if the index segment is already out of retention. 975 // This can happen when the edge of requested ranges at time of data bootstrap 976 // is now out of retention. 977 s.instrumentation.outOfRetentionIndexSegmentSkipped(buildIndexLogFields) 978 return remainingRanges, timesWithErrors 979 } else if err != nil { 980 instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { 981 l.Error("persist fs index bootstrap failed", 982 zap.Stringer("namespace", ns.ID()), 983 zap.Stringer("requestedRanges", requestedRanges), 984 zap.Error(err)) 985 }) 986 } 987 } else { 988 s.log.Info("building in-memory index segment", buildIndexLogFields...) 989 indexBlock, err = bootstrapper.BuildBootstrapIndexSegment( 990 ns, 991 requestedRanges, 992 builder.Builder(), 993 compactor, 994 s.opts.ResultOptions(), 995 s.opts.IndexOptions().MmapReporter(), 996 blockStart, 997 blockEnd, 998 ) 999 if errors.Is(err, fs.ErrIndexOutOfRetention) { 1000 // Bail early if the index segment is already out of retention. 1001 // This can happen when the edge of requested ranges at time of data bootstrap 1002 // is now out of retention. 1003 s.instrumentation.outOfRetentionIndexSegmentSkipped(buildIndexLogFields) 1004 return remainingRanges, timesWithErrors 1005 } else if err != nil { 1006 instrument.EmitAndLogInvariantViolation(iopts, func(l *zap.Logger) { 1007 l.Error("build fs index bootstrap failed", 1008 zap.Stringer("namespace", ns.ID()), 1009 zap.Stringer("requestedRanges", requestedRanges), 1010 zap.Error(err)) 1011 }) 1012 } 1013 } 1014 1015 // Merge segments and fulfilled time ranges. 1016 segments := indexBlock.Segments() 1017 for _, seg := range existingIndexBlock.Segments() { 1018 segments = append(segments, seg) 1019 } 1020 newFulfilled := existingIndexBlock.Fulfilled().Copy() 1021 newFulfilled.AddRanges(indexBlock.Fulfilled()) 1022 1023 // Replace index block for default index volume type. 1024 resultLock.Lock() 1025 r.IndexResults()[blockStart]. 1026 SetBlock(idxpersist.DefaultIndexVolumeType, result.NewIndexBlock(segments, newFulfilled)) 1027 resultLock.Unlock() 1028 1029 return remainingRanges, timesWithErrors 1030 } 1031 1032 func (s *peersSource) readNextEntryAndMaybeIndex( 1033 r fs.DataFileSetReader, 1034 batch []doc.Metadata, 1035 builder *result.IndexBuilder, 1036 ) ([]doc.Metadata, error) { 1037 // If performing index run, then simply read the metadata and add to segment. 1038 entry, err := r.StreamingReadMetadata() 1039 if err != nil { 1040 return batch, err 1041 } 1042 1043 d, err := convert.FromSeriesIDAndEncodedTags(entry.ID, entry.EncodedTags) 1044 if err != nil { 1045 return batch, err 1046 } 1047 1048 batch = append(batch, d) 1049 1050 if len(batch) >= index.MetadataArrayPoolCapacity { 1051 return builder.FlushBatch(batch) 1052 } 1053 1054 return batch, nil 1055 } 1056 1057 // markRunResultErrorsAndUnfulfilled checks the list of times that had errors and makes 1058 // sure that we don't return any blocks or bloom filters for them. In addition, 1059 // it looks at any remaining (unfulfilled) ranges and makes sure they're marked 1060 // as unfulfilled. 1061 func (s *peersSource) markRunResultErrorsAndUnfulfilled( 1062 resultLock *sync.Mutex, 1063 results result.IndexBootstrapResult, 1064 requestedRanges result.ShardTimeRanges, 1065 remainingRanges result.ShardTimeRanges, 1066 timesWithErrors []xtime.UnixNano, 1067 ) { 1068 // NB(xichen): this is the exceptional case where we encountered errors due to files 1069 // being corrupted, which should be fairly rare so we can live with the overhead. We 1070 // experimented with adding the series to a temporary map and only adding the temporary map 1071 // to the final result but adding series to large map with string keys is expensive, and 1072 // the current implementation saves the extra overhead of merging temporary map with the 1073 // final result. 1074 if len(timesWithErrors) > 0 { 1075 timesWithErrorsString := make([]string, len(timesWithErrors)) 1076 for i := range timesWithErrors { 1077 timesWithErrorsString[i] = timesWithErrors[i].String() 1078 } 1079 s.log.Info("encountered errors for range", 1080 zap.String("requestedRanges", remainingRanges.SummaryString()), 1081 zap.Strings("timesWithErrors", timesWithErrorsString)) 1082 } 1083 1084 if !remainingRanges.IsEmpty() { 1085 resultLock.Lock() 1086 results.Unfulfilled().AddRanges(remainingRanges) 1087 resultLock.Unlock() 1088 } 1089 } 1090 1091 func (s *peersSource) peerAvailability( 1092 _ namespace.Metadata, 1093 shardTimeRanges result.ShardTimeRanges, 1094 runOpts bootstrap.RunOptions, 1095 ) (result.ShardTimeRanges, error) { 1096 var ( 1097 peerAvailabilityByShard = map[topology.ShardID]*shardPeerAvailability{} 1098 initialTopologyState = runOpts.InitialTopologyState() 1099 ) 1100 1101 for shardIDUint := range shardTimeRanges.Iter() { 1102 shardID := topology.ShardID(shardIDUint) 1103 shardPeers, ok := peerAvailabilityByShard[shardID] 1104 if !ok { 1105 shardPeers = &shardPeerAvailability{} 1106 peerAvailabilityByShard[shardID] = shardPeers 1107 } 1108 hostShardStates, ok := initialTopologyState.ShardStates[shardID] 1109 if !ok { 1110 // This shard was not part of the topology when the bootstrapping 1111 // process began. 1112 continue 1113 } 1114 1115 shardPeers.numPeers = len(hostShardStates) 1116 for _, hostShardState := range hostShardStates { 1117 if hostShardState.Host.ID() == initialTopologyState.Origin.ID() { 1118 // Don't take self into account 1119 continue 1120 } 1121 1122 shardState := hostShardState.ShardState 1123 1124 switch shardState { 1125 // Don't want to peer bootstrap from a node that has not yet completely 1126 // taken ownership of the shard. 1127 case shard.Initializing: 1128 // Success cases - We can bootstrap from this host, which is enough to 1129 // mark this shard as bootstrappable. 1130 case shard.Leaving: 1131 fallthrough 1132 case shard.Available: 1133 shardPeers.numAvailablePeers++ 1134 case shard.Unknown: 1135 fallthrough 1136 default: 1137 return nil, fmt.Errorf("unknown shard state: %v", shardState) 1138 } 1139 } 1140 } 1141 1142 var ( 1143 runtimeOpts = s.opts.RuntimeOptionsManager().Get() 1144 bootstrapConsistencyLevel = runtimeOpts.ClientBootstrapConsistencyLevel() 1145 majorityReplicas = initialTopologyState.MajorityReplicas 1146 availableShardTimeRanges = result.NewShardTimeRanges() 1147 ) 1148 for shardIDUint := range shardTimeRanges.Iter() { 1149 var ( 1150 shardID = topology.ShardID(shardIDUint) 1151 shardPeers = peerAvailabilityByShard[shardID] 1152 1153 total = shardPeers.numPeers 1154 available = shardPeers.numAvailablePeers 1155 ) 1156 1157 if available == 0 { 1158 // Can't peer bootstrap if there are no available peers. 1159 s.log.Debug("0 available peers, unable to peer bootstrap", 1160 zap.Int("total", total), 1161 zap.Uint32("shard", shardIDUint)) 1162 continue 1163 } 1164 1165 if !topology.ReadConsistencyAchieved( 1166 bootstrapConsistencyLevel, majorityReplicas, total, available) { 1167 s.log.Debug("read consistency not achieved, unable to peer bootstrap", 1168 zap.Any("level", bootstrapConsistencyLevel), 1169 zap.Int("replicas", majorityReplicas), 1170 zap.Int("total", total), 1171 zap.Int("available", available)) 1172 continue 1173 } 1174 1175 // Optimistically assume that the peers will be able to provide 1176 // all the data. This assumption is safe, as the shard/block ranges 1177 // will simply be marked unfulfilled if the peers are not able to 1178 // satisfy the requests. 1179 if tr, ok := shardTimeRanges.Get(shardIDUint); ok { 1180 availableShardTimeRanges.Set(shardIDUint, tr) 1181 } 1182 } 1183 1184 return availableShardTimeRanges, nil 1185 } 1186 1187 func (s *peersSource) validateRunOpts(runOpts bootstrap.RunOptions) error { 1188 persistConfig := runOpts.PersistConfig() 1189 if persistConfig.FileSetType != persist.FileSetFlushType && 1190 persistConfig.FileSetType != persist.FileSetSnapshotType { 1191 // Should never happen 1192 return fmt.Errorf("unknown persist config fileset file type: %v", persistConfig.FileSetType) 1193 } 1194 1195 return nil 1196 } 1197 1198 func (s *peersSource) shouldPersist(runOpts bootstrap.RunOptions) bool { 1199 persistConfig := runOpts.PersistConfig() 1200 1201 return persistConfig.Enabled && 1202 persistConfig.FileSetType == persist.FileSetFlushType && 1203 // TODO(bodu): We should migrate to series.CacheLRU only. 1204 s.opts.ResultOptions().SeriesCachePolicy() != series.CacheAll 1205 }