github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/persist/fs/seek_manager.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 fs 22 23 import ( 24 "errors" 25 "fmt" 26 "sync" 27 "syscall" 28 "time" 29 30 "github.com/m3db/m3/src/dbnode/namespace" 31 "github.com/m3db/m3/src/dbnode/retention" 32 "github.com/m3db/m3/src/dbnode/sharding" 33 "github.com/m3db/m3/src/dbnode/storage/block" 34 xerrors "github.com/m3db/m3/src/x/errors" 35 "github.com/m3db/m3/src/x/ident" 36 "github.com/m3db/m3/src/x/pool" 37 xsync "github.com/m3db/m3/src/x/sync" 38 xtime "github.com/m3db/m3/src/x/time" 39 40 "go.uber.org/zap" 41 ) 42 43 const ( 44 seekManagerCloseInterval = time.Second 45 reusableSeekerResourcesPoolSize = 10 46 concurrentCacheShardIndices = 16 47 ) 48 49 var ( 50 errSeekerManagerAlreadyOpenOrClosed = errors.New( 51 "seeker manager already open or is closed", 52 ) 53 errSeekerManagerAlreadyClosed = errors.New( 54 "seeker manager already closed", 55 ) 56 errSeekerManagerFileSetNotFound = errors.New( 57 "seeker manager lookup fileset not found", 58 ) 59 errNoAvailableSeekers = errors.New( 60 "no available seekers", 61 ) 62 errSeekersDontExist = errors.New( 63 "seekers don't exist", 64 ) 65 errCantCloseSeekerManagerWhileSeekersAreBorrowed = errors.New( 66 "cant close seeker manager while seekers are borrowed", 67 ) 68 errReturnedUnmanagedSeeker = errors.New( 69 "cant return a seeker not managed by the seeker manager", 70 ) 71 errUpdateOpenLeaseSeekerManagerNotOpen = errors.New( 72 "cant update open lease because seeker manager is not open", 73 ) 74 errCacheShardIndicesSeekerManagerNotOpen = errors.New( 75 "cant cache shard indices because seeker manager is not open", 76 ) 77 errConcurrentUpdateOpenLeaseNotAllowed = errors.New( 78 "concurrent open lease updates are not allowed", 79 ) 80 errOutOfOrderUpdateOpenLease = errors.New( 81 "received update open lease volumes out of order", 82 ) 83 errShardNotExists = errors.New( 84 "shard not exists", 85 ) 86 ) 87 88 type openAnyUnopenSeekersFn func(*seekersByTime) error 89 90 type newOpenSeekerFn func( 91 shard uint32, 92 blockStart xtime.UnixNano, 93 volume int, 94 ) (DataFileSetSeeker, error) 95 96 type seekerManagerStatus int 97 98 const ( 99 seekerManagerNotOpen seekerManagerStatus = iota 100 seekerManagerOpen 101 seekerManagerClosed 102 ) 103 104 // seekerManager provides functionality around borrowableSeekers such as 105 // opening and closing them, as well as lending them out to a Retriever. 106 // There is a single seekerManager per namespace which contains all 107 // open seekers for all shards and blocks within that namespace. 108 type seekerManager struct { 109 sync.RWMutex 110 111 opts Options 112 blockRetrieverOpts BlockRetrieverOptions 113 fetchConcurrency int 114 logger *zap.Logger 115 116 bytesPool pool.CheckedBytesPool 117 filePathPrefix string 118 119 status seekerManagerStatus 120 shardSet sharding.ShardSet 121 updateOpenLeasesInProgress map[block.HashableLeaseDescriptor]struct{} 122 123 cacheShardIndicesWorkers xsync.WorkerPool 124 125 // seekersByShardIdx provides access to all seekers, first partitioned by 126 // shard and then by block start. 127 seekersByShardIdx []*seekersByTime 128 namespace ident.ID 129 namespaceMetadata namespace.Metadata 130 unreadBuf seekerUnreadBuf 131 openAnyUnopenSeekersFn openAnyUnopenSeekersFn 132 newOpenSeekerFn newOpenSeekerFn 133 sleepFn func(d time.Duration) 134 openCloseLoopDoneCh chan struct{} 135 136 // Pool of seeker resources that can be used to open new seekers. 137 reusableSeekerResourcesPool pool.ObjectPool 138 } 139 140 type seekerUnreadBuf struct { 141 sync.RWMutex 142 value []byte 143 } 144 145 // seekersAndBloom contains a slice of seekers for a given shard/blockStart. One of the seekers will be the original, 146 // and the others will be clones. The bloomFilter field is a reference to the underlying bloom filter that the 147 // original seeker and all of its clones share. 148 type seekersAndBloom struct { 149 wg *sync.WaitGroup 150 seekers []borrowableSeeker 151 bloomFilter *ManagedConcurrentBloomFilter 152 volume int 153 } 154 155 func (s seekersAndBloom) closeWithLock() error { 156 multiErr := xerrors.NewMultiError() 157 for _, seeker := range s.seekers { 158 multiErr = multiErr.Add(seeker.seeker.Close()) 159 } 160 return multiErr.FinalError() 161 } 162 163 // Returns true if any seekers are borrowed. 164 func (s seekersAndBloom) anyBorrowedWithLock() bool { 165 for _, seeker := range s.seekers { 166 if seeker.isBorrowed { 167 return true 168 } 169 } 170 return false 171 } 172 173 // borrowableSeeker is just a seeker with an additional field for keeping 174 // track of whether or not it has been borrowed. 175 type borrowableSeeker struct { 176 seeker ConcurrentDataFileSetSeeker 177 isBorrowed bool 178 } 179 180 // seekersByTime contains all seekers for a specific shard, accessible by 181 // blockStart. The accessed field allows for pre-caching those seekers. 182 type seekersByTime struct { 183 sync.RWMutex 184 shard uint32 185 accessed bool 186 seekers map[xtime.UnixNano]rotatableSeekers 187 } 188 189 // rotatableSeekers is a wrapper around seekersAndBloom that allows for rotating 190 // out stale seekers. This is required so that the active seekers can be rotated 191 // to inactive while the seeker manager waits for any outstanding stale seekers 192 // to be returned. 193 type rotatableSeekers struct { 194 active seekersAndBloom 195 inactive seekersAndBloom 196 } 197 198 type seekerManagerPendingClose struct { 199 shard uint32 200 blockStart xtime.UnixNano 201 } 202 203 // NewSeekerManager returns a new TSDB file set seeker manager. 204 func NewSeekerManager( 205 bytesPool pool.CheckedBytesPool, 206 opts Options, 207 blockRetrieverOpts BlockRetrieverOptions, 208 ) DataFileSetSeekerManager { 209 reusableSeekerResourcesPool := pool.NewObjectPool( 210 pool.NewObjectPoolOptions(). 211 SetSize(reusableSeekerResourcesPoolSize). 212 SetRefillHighWatermark(0). 213 SetRefillLowWatermark(0)) 214 reusableSeekerResourcesPool.Init(func() interface{} { 215 return NewReusableSeekerResources(opts) 216 }) 217 218 // NB(r): Since this is mainly IO bound work, perfectly 219 // fine to do this in parallel. 220 cacheShardIndicesWorkers := xsync.NewWorkerPool(concurrentCacheShardIndices) 221 cacheShardIndicesWorkers.Init() 222 223 m := &seekerManager{ 224 bytesPool: bytesPool, 225 filePathPrefix: opts.FilePathPrefix(), 226 opts: opts, 227 blockRetrieverOpts: blockRetrieverOpts, 228 fetchConcurrency: blockRetrieverOpts.FetchConcurrency(), 229 cacheShardIndicesWorkers: cacheShardIndicesWorkers, 230 logger: opts.InstrumentOptions().Logger(), 231 openCloseLoopDoneCh: make(chan struct{}), 232 reusableSeekerResourcesPool: reusableSeekerResourcesPool, 233 updateOpenLeasesInProgress: make(map[block.HashableLeaseDescriptor]struct{}), 234 } 235 m.openAnyUnopenSeekersFn = m.openAnyUnopenSeekers 236 m.newOpenSeekerFn = m.newOpenSeeker 237 m.sleepFn = time.Sleep 238 return m 239 } 240 241 // Open opens the seekerManager, which starts background processes such as 242 // the openCloseLoop, ensuring open file descriptors for the file sets accesible 243 // through the seekers. 244 func (m *seekerManager) Open( 245 nsMetadata namespace.Metadata, 246 shardSet sharding.ShardSet, 247 ) error { 248 m.Lock() 249 if m.status != seekerManagerNotOpen { 250 m.Unlock() 251 return errSeekerManagerAlreadyOpenOrClosed 252 } 253 254 m.namespace = nsMetadata.ID() 255 m.namespaceMetadata = nsMetadata 256 m.shardSet = shardSet 257 m.status = seekerManagerOpen 258 m.Unlock() 259 go m.openCloseLoop() 260 261 // Register for updates to block leases. 262 // NB(rartoul): This should be safe to do within the context of the lock 263 // because the block.LeaseManager does not yet have a handle on the SeekerManager 264 // so they can't deadlock trying to acquire each other's locks, but do it outside 265 // of the lock just to be safe. 266 m.blockRetrieverOpts.BlockLeaseManager().RegisterLeaser(m) 267 268 return nil 269 } 270 271 func (m *seekerManager) CacheShardIndices(shards []uint32) error { 272 m.RLock() 273 if m.status == seekerManagerNotOpen { 274 m.RUnlock() 275 return errCacheShardIndicesSeekerManagerNotOpen 276 } 277 m.RUnlock() 278 279 var ( 280 multiErr = xerrors.NewMultiError() 281 resultsLock sync.Mutex 282 wg sync.WaitGroup 283 ) 284 for _, shard := range shards { 285 byTime, ok := m.seekersByTime(shard) 286 if !ok { 287 multiErr = multiErr.Add(errShardNotExists) 288 continue 289 } 290 291 byTime.Lock() 292 // Track accessed to precache in open/close loop 293 byTime.accessed = true 294 byTime.Unlock() 295 296 wg.Add(1) 297 m.cacheShardIndicesWorkers.Go(func() { 298 defer wg.Done() 299 if err := m.openAnyUnopenSeekersFn(byTime); err != nil { 300 resultsLock.Lock() 301 multiErr = multiErr.Add(err) 302 resultsLock.Unlock() 303 } 304 }) 305 } 306 307 wg.Wait() 308 return multiErr.FinalError() 309 } 310 311 func (m *seekerManager) AssignShardSet(shardSet sharding.ShardSet) { 312 m.Lock() 313 m.shardSet = shardSet 314 m.Unlock() 315 } 316 317 func (m *seekerManager) Test( 318 id ident.ID, 319 shard uint32, 320 start xtime.UnixNano, 321 ) (bool, error) { 322 byTime, ok := m.seekersByTime(shard) 323 if !ok { 324 return false, errShardNotExists 325 } 326 327 // Try fast RLock() first. 328 byTime.RLock() 329 if seekers, ok := byTime.seekers[start]; ok && seekers.active.wg == nil { 330 // Seekers are open: good to test but still hold RLock while doing so 331 idExists := seekers.active.bloomFilter.Test(id.Bytes()) 332 byTime.RUnlock() 333 return idExists, nil 334 } 335 336 byTime.RUnlock() 337 338 byTime.Lock() 339 defer byTime.Unlock() 340 341 // Check if raced with another call to this method 342 if seekers, ok := byTime.seekers[start]; ok && seekers.active.wg == nil { 343 return seekers.active.bloomFilter.Test(id.Bytes()), nil 344 } 345 346 openSeekersAndBloom, err := m.getOrOpenSeekersWithLock(start, byTime) 347 if err != nil { 348 return false, err 349 } 350 351 return openSeekersAndBloom.bloomFilter.Test(id.Bytes()), nil 352 } 353 354 // Borrow returns a "borrowed" seeker which the caller has exclusive access to 355 // until it's returned later. 356 func (m *seekerManager) Borrow( 357 shard uint32, 358 start xtime.UnixNano, 359 ) (ConcurrentDataFileSetSeeker, error) { 360 byTime, ok := m.seekersByTime(shard) 361 if !ok { 362 return nil, errShardNotExists 363 } 364 365 byTime.Lock() 366 defer byTime.Unlock() 367 // Track accessed to precache in open/close loop 368 byTime.accessed = true 369 370 openSeekersAndBloom, err := m.getOrOpenSeekersWithLock(start, byTime) 371 if err != nil { 372 return nil, err 373 } 374 375 seekers := openSeekersAndBloom.seekers 376 availableSeekerIdx := -1 377 availableSeeker := borrowableSeeker{} 378 for i, seeker := range seekers { 379 if !seeker.isBorrowed { 380 availableSeekerIdx = i 381 availableSeeker = seeker 382 break 383 } 384 } 385 386 // Should not occur in the case of a well-behaved caller 387 if availableSeekerIdx == -1 { 388 return nil, errNoAvailableSeekers 389 } 390 391 availableSeeker.isBorrowed = true 392 seekers[availableSeekerIdx] = availableSeeker 393 return availableSeeker.seeker, nil 394 } 395 396 func (m *seekerManager) shardExistsWithLock(shard uint32) bool { 397 _, err := m.shardSet.LookupStateByID(shard) 398 // NB(bodu): LookupStateByID returns ErrInvalidShardID when shard 399 // does not exist in the shard map which means the shard is not available. 400 return err == nil 401 } 402 403 func (m *seekerManager) Return( 404 shard uint32, 405 start xtime.UnixNano, 406 seeker ConcurrentDataFileSetSeeker, 407 ) error { 408 byTime, ok := m.seekersByTime(shard) 409 if !ok { 410 return errShardNotExists 411 } 412 413 byTime.Lock() 414 defer byTime.Unlock() 415 416 seekers, ok := byTime.seekers[start] 417 // Should never happen - This either means that the caller (DataBlockRetriever) 418 // is trying to return seekers that it never requested, OR its trying to return 419 // seekers after the openCloseLoop has already determined that they were all 420 // no longer in use and safe to close. Either way it indicates there is 421 // a bug in the code. 422 if !ok { 423 return errSeekersDontExist 424 } 425 426 returned, err := m.returnSeekerWithLock(seekers, seeker) 427 if err != nil { 428 return err 429 } 430 431 // Should never happen with a well behaved caller. Either they are trying to return a seeker 432 // that we're not managing, or they provided the wrong shard/start. 433 if !returned { 434 return errReturnedUnmanagedSeeker 435 } 436 437 return nil 438 } 439 440 // returnSeekerWithLock encapsulates all the logic for returning a seeker, 441 // including distinguishing between active and inactive seekers. For more 442 // details on this read the comment above the UpdateOpenLease() method. 443 func (m *seekerManager) returnSeekerWithLock( 444 seekers rotatableSeekers, seeker ConcurrentDataFileSetSeeker, 445 ) (bool, error) { 446 // Check if the seeker being returned is an active seeker first. 447 if m.markBorrowedSeekerAsReturned(&seekers.active, seeker) { 448 // We can return right away if we've returned an active seeker. 449 return true, nil 450 } 451 452 // If no match was found in the active seekers, it's possible that an inactive seeker is being returned. 453 if m.markBorrowedSeekerAsReturned(&seekers.inactive, seeker) { 454 // The goroutine that returns the last outstanding inactive seeker is responsible for notifying any 455 // goroutines waiting for all inactive seekers to be returned and clearing out the inactive seekers 456 // state entirely. 457 if seekers.inactive.anyBorrowedWithLock() { 458 return true, nil 459 } 460 461 err := seekers.inactive.closeWithLock() 462 if seekers.inactive.wg != nil { 463 // Signal completion regardless of any errors encountered while closing. 464 seekers.inactive.wg.Done() 465 seekers.inactive.wg = nil 466 } 467 return true, err 468 } 469 470 return false, nil 471 } 472 473 func (m *seekerManager) markBorrowedSeekerAsReturned( 474 seekers *seekersAndBloom, seeker ConcurrentDataFileSetSeeker, 475 ) bool { 476 for i, compareSeeker := range seekers.seekers { 477 if seeker == compareSeeker.seeker { 478 compareSeeker.isBorrowed = false 479 seekers.seekers[i] = compareSeeker 480 return true 481 } 482 } 483 return false 484 } 485 486 // UpdateOpenLease() implements block.Leaser. The contract of this API is that once the function 487 // returns successfully any resources associated with the previous lease should have been 488 // released (in this case the Seeker / files for the previous volume) and the resources associated 489 // with the new lease should have been acquired (the seeker for the provided volume). 490 // 491 // Practically speaking, the goal of this function is to open a new seeker for the latest volume and 492 // then "hot-swap" it so that by the time this function returns there are no more outstanding reads 493 // using the old seekers, all the old seekers have been closed, and all subsequent reads will use the 494 // seekers associated with the latest volume. 495 // 496 // The bulk of the complexity of this function is caused by the desire to avoid the hot-swap from 497 // causing any latency spikes. To accomplish this, the following is performed: 498 // 499 // 1. Open the new seeker outside the context of any locks. 500 // 2. Acquire a lock on the seekers that need to be swapped and rotate the existing "active" seekers 501 // to be "inactive" and set the newly opened seekers as "active". This operation is extremely cheap 502 // and ensures that all subsequent reads will use the seekers for the latest volume instead of the 503 // previous. In addition, this phase also creates a waitgroup for the inactive seekers that will be 504 // be used to "wait" for all of the existing seekers that are currently borrowed to be returned. 505 // 3. Release the lock so that reads can continue uninterrupted and call waitgroup.Wait() to wait for all 506 // the currently borrowed "inactive" seekers (if any) to be returned. 507 // 4. Every call to Return() for an "inactive" seeker will check if it's the last borrowed inactive seeker, 508 // and if so, will close all the inactive seekers and call wg.Done() which will notify the goroutine 509 // running the UpdateOpenLease() function that all inactive seekers have been returned and closed at 510 // which point the function will return successfully. 511 func (m *seekerManager) UpdateOpenLease( 512 descriptor block.LeaseDescriptor, 513 state block.LeaseState, 514 ) (block.UpdateOpenLeaseResult, error) { 515 hashableDescriptor := block.NewHashableLeaseDescriptor(descriptor) 516 noop, err := m.startUpdateOpenLease(descriptor.Namespace, hashableDescriptor) 517 if err != nil { 518 return 0, err 519 } 520 if noop { 521 return block.NoOpenLease, nil 522 } 523 524 defer func() { 525 m.Lock() 526 // Was added by startUpdateOpenLease(). 527 delete(m.updateOpenLeasesInProgress, hashableDescriptor) 528 m.Unlock() 529 }() 530 531 wg, updateLeaseResult, err := m.updateOpenLeaseHotSwapSeekers(descriptor, state) 532 if err != nil { 533 return 0, err 534 } 535 if wg != nil { 536 // Wait for all the inactive seekers to be returned and closed because the contract 537 // of this API is that the Leaser (SeekerManager) should have relinquished any resources 538 // associated with the old lease by the time this function returns. 539 wg.Wait() 540 } 541 542 return updateLeaseResult, nil 543 } 544 545 func (m *seekerManager) startUpdateOpenLease( 546 namespace ident.ID, 547 hashableDescriptor block.HashableLeaseDescriptor, 548 ) (bool, error) { 549 m.Lock() 550 defer m.Unlock() 551 552 if m.status != seekerManagerOpen { 553 return false, errUpdateOpenLeaseSeekerManagerNotOpen 554 } 555 if _, ok := m.updateOpenLeasesInProgress[hashableDescriptor]; ok { 556 // Prevent UpdateOpenLease() calls from happening concurrently 557 // (at the granularity of block.LeaseDescriptor). 558 return false, errConcurrentUpdateOpenLeaseNotAllowed 559 } 560 if !m.namespace.Equal(namespace) { 561 return true, nil 562 } 563 564 m.updateOpenLeasesInProgress[hashableDescriptor] = struct{}{} 565 566 return false, nil 567 } 568 569 // updateOpenLeaseHotSwapSeekers encapsulates all of the logic for swapping the 570 // existing seekers with the new ones as dictated by the call to UpdateOpenLease(). 571 // For details of the algorithm review the comment above the UpdateOpenLease() method. 572 func (m *seekerManager) updateOpenLeaseHotSwapSeekers( 573 descriptor block.LeaseDescriptor, 574 state block.LeaseState, 575 ) (*sync.WaitGroup, block.UpdateOpenLeaseResult, error) { 576 newActiveSeekers, err := m.newSeekersAndBloom(descriptor.Shard, descriptor.BlockStart, state.Volume) 577 if err != nil { 578 return nil, 0, err 579 } 580 581 byTime, ok := m.seekersByTime(descriptor.Shard) 582 if !ok { 583 return nil, 0, errShardNotExists 584 } 585 586 var ( 587 blockStartNano = descriptor.BlockStart 588 updateOpenLeaseResult = block.NoOpenLease 589 ) 590 seekers, ok := m.acquireByTimeLockWaitGroupAware(blockStartNano, byTime) 591 defer byTime.Unlock() 592 if !ok { 593 // No existing seekers, so just set the newly created ones and be done. 594 seekers.active = newActiveSeekers 595 byTime.seekers[blockStartNano] = seekers 596 return nil, updateOpenLeaseResult, nil 597 } 598 599 // Existing seekers exist. 600 updateOpenLeaseResult = block.UpdateOpenLease 601 if seekers.active.volume > state.Volume { 602 // Ignore any close errors because its not relevant from the callers perspective. 603 m.closeSeekersAndLogError(descriptor, newActiveSeekers) 604 return nil, 0, errOutOfOrderUpdateOpenLease 605 } 606 607 seekers.inactive = seekers.active 608 seekers.active = newActiveSeekers 609 610 var wg *sync.WaitGroup 611 if seekers.inactive.anyBorrowedWithLock() { 612 // If any of the seekers are borrowed setup a waitgroup which will be used to 613 // signal when they've all been returned (the last seeker that is returned via 614 // the Return() API will call wg.Done()). 615 wg = &sync.WaitGroup{} 616 wg.Add(1) 617 seekers.inactive.wg = wg 618 } else { 619 // If none of the existing seekers are currently borrowed then we can just close them all. 620 m.closeSeekersAndLogError(descriptor, seekers.inactive) 621 seekers.inactive = seekersAndBloom{} 622 } 623 byTime.seekers[blockStartNano] = seekers 624 625 return wg, updateOpenLeaseResult, nil 626 } 627 628 // acquireByTimeLockWaitGroupAware grabs a lock on the shard and checks if 629 // seekers exist for a given blockStart. If a waitgroup is present, meaning 630 // a different goroutine is currently trying to open those seekers, it will 631 // wait for that operation to complete first, before returning the seekers 632 // while the lock on the shard is still being held. 633 func (m *seekerManager) acquireByTimeLockWaitGroupAware( 634 blockStart xtime.UnixNano, 635 byTime *seekersByTime, 636 ) (seekers rotatableSeekers, ok bool) { 637 // It's possible that another goroutine is currently trying to open seekers for this blockStart. If so, this 638 // goroutine will need to wait for the other goroutine to finish before proceeding. The check is performed in 639 // a loop because each iteration relinquishes the lock temporarily. Once the lock is reacquired the same 640 // conditions need to be checked again until this Goroutine finds that either: 641 // 642 // a) Seekers are already present for this blockStart in which case this function can return while holding the 643 // lock. 644 // or 645 // b) Seeks are not present for this blockStart and no other goroutines are currently trying to open them, in 646 // which case this function can also return while holding the lock. 647 for { 648 byTime.Lock() 649 seekers, ok = byTime.seekers[blockStart] 650 651 if !ok || seekers.active.wg == nil { 652 // Exit the loop still holding the lock. 653 return seekers, ok 654 } 655 656 // If another goroutine is currently trying to open seekers for this block start 657 // then wait for that operation to complete. 658 wg := seekers.active.wg 659 byTime.Unlock() 660 wg.Wait() 661 } 662 } 663 664 // closeSeekersAndLogError is a helper function that closes all the seekers in a slice of borrowableSeeker 665 // and emits a log if any errors occurred. 666 func (m *seekerManager) closeSeekersAndLogError(descriptor block.LeaseDescriptor, seekers seekersAndBloom) { 667 if err := seekers.closeWithLock(); err != nil { 668 // Log the error but don't return it since its not relevant from 669 // the callers perspective. 670 m.logger.Error( 671 "error closing seeker in update open lease", 672 zap.Error(err), 673 zap.String("namespace", descriptor.Namespace.String()), 674 zap.Int("shard", int(descriptor.Shard)), 675 zap.Time("blockStart", descriptor.BlockStart.ToTime())) 676 } 677 } 678 679 // getOrOpenSeekersWithLock checks if the seekers are already open / initialized. If they are, then it 680 // returns them. Then, it checks if a different goroutine is in the process of opening them , if so it 681 // registers itself as waiting until the other goroutine completes. If neither of those conditions occur, 682 // then it begins the process of opening the seekers itself. First, it creates a waitgroup that other 683 // goroutines can use so that they're notified when the seekers are open. This is useful because it allows 684 // us to prevent multiple goroutines from trying to open the same seeker without having to hold onto a lock 685 // of the seekersByTime struct during a I/O heavy workload. Once the wg is created, we relinquish the lock, 686 // open the Seeker (I/O heavy), re-acquire the lock (so that the waiting goroutines don't get it before us), 687 // and then notify the waiting goroutines that we've finished. 688 func (m *seekerManager) getOrOpenSeekersWithLock( 689 start xtime.UnixNano, 690 byTime *seekersByTime, 691 ) (seekersAndBloom, error) { 692 seekers, ok := byTime.seekers[start] 693 if ok && seekers.active.wg == nil { 694 // Seekers are already open 695 return seekers.active, nil 696 } 697 698 if seekers.active.wg != nil { 699 // Seekers are being initialized / opened, wait for the that to complete 700 byTime.Unlock() 701 seekers.active.wg.Wait() 702 byTime.Lock() 703 // Need to do the lookup again recursively to see the new state 704 return m.getOrOpenSeekersWithLock(start, byTime) 705 } 706 707 // Seekers need to be opened. 708 // We're going to release the lock temporarily, so we initialize a WaitGroup 709 // that other routines which would have otherwise attempted to also open this 710 // same seeker can use instead to wait for us to finish. 711 wg := &sync.WaitGroup{} 712 seekers.active.wg = wg 713 seekers.active.wg.Add(1) 714 byTime.seekers[start] = seekers 715 byTime.Unlock() 716 717 activeSeekers, err := m.openLatestSeekersWithActiveWaitGroup(start, seekers, byTime) 718 // Lock must be held when function returns. 719 byTime.Lock() 720 // Signal to other waiting goroutines that this goroutine is done attempting to open 721 // the seekers. This is done *after* acquiring the lock so that other goroutines that 722 // were waiting won't acquire the lock before this goroutine does. 723 wg.Done() 724 if err != nil { 725 // Delete the seekersByTime struct so that the process can be restarted by the next 726 // goroutine (since this one errored out). 727 delete(byTime.seekers, start) 728 return seekersAndBloom{}, err 729 } 730 731 seekers.active = activeSeekers 732 byTime.seekers[start] = seekers 733 return activeSeekers, nil 734 } 735 736 // openLatestSeekersWithActiveWaitGroup opens the latest seekers for the provided block start. Similar 737 // to the withLock() convention, the caller of this function is expected to be the owner of the waitgroup 738 // that is being used to signal that seekers have completed opening. 739 func (m *seekerManager) openLatestSeekersWithActiveWaitGroup( 740 start xtime.UnixNano, 741 seekers rotatableSeekers, 742 byTime *seekersByTime, 743 ) (seekersAndBloom, error) { 744 // Open first one - Do this outside the context of the lock because opening 745 // a seeker can be an expensive operation (validating index files). 746 blm := m.blockRetrieverOpts.BlockLeaseManager() 747 state, err := blm.OpenLatestLease(m, block.LeaseDescriptor{ 748 Namespace: m.namespace, 749 Shard: byTime.shard, 750 BlockStart: start, 751 }) 752 if err != nil { 753 return seekersAndBloom{}, fmt.Errorf("err opening latest lease: %v", err) 754 } 755 756 return m.newSeekersAndBloom(byTime.shard, start, state.Volume) 757 } 758 759 func (m *seekerManager) newSeekersAndBloom( 760 shard uint32, blockStart xtime.UnixNano, volume int, 761 ) (seekersAndBloom, error) { 762 seeker, err := m.newOpenSeekerFn(shard, blockStart, volume) 763 if err != nil { 764 return seekersAndBloom{}, err 765 } 766 767 newSeekersAndBloom, err := m.seekersAndBloomFromSeeker(seeker, volume) 768 if err != nil { 769 return seekersAndBloom{}, err 770 } 771 772 return newSeekersAndBloom, nil 773 } 774 775 func (m *seekerManager) seekersAndBloomFromSeeker( 776 seeker DataFileSetSeeker, volume int, 777 ) (seekersAndBloom, error) { 778 borrowableSeekers := make([]borrowableSeeker, 0, m.fetchConcurrency) 779 borrowableSeekers = append(borrowableSeekers, borrowableSeeker{seeker: seeker}) 780 // Clone remaining seekers from the original - No need to release the lock, cloning is cheap. 781 for i := 0; i < m.fetchConcurrency-1; i++ { 782 clone, err := seeker.ConcurrentClone() 783 if err != nil { 784 multiErr := xerrors.NewMultiError() 785 multiErr = multiErr.Add(err) 786 for _, seeker := range borrowableSeekers { 787 // Don't leak successfully opened seekers 788 multiErr = multiErr.Add(seeker.seeker.Close()) 789 } 790 return seekersAndBloom{}, multiErr.FinalError() 791 } 792 borrowableSeekers = append(borrowableSeekers, borrowableSeeker{seeker: clone}) 793 } 794 795 return seekersAndBloom{ 796 seekers: borrowableSeekers, 797 bloomFilter: borrowableSeekers[0].seeker.ConcurrentIDBloomFilter(), 798 volume: volume, 799 }, nil 800 } 801 802 func (m *seekerManager) openAnyUnopenSeekers(byTime *seekersByTime) error { 803 start := m.earliestSeekableBlockStart() 804 end := m.latestSeekableBlockStart() 805 blockSize := m.namespaceMetadata.Options().RetentionOptions().BlockSize() 806 multiErr := xerrors.NewMultiError() 807 808 for t := start; !t.After(end); t = t.Add(blockSize) { 809 byTime.Lock() 810 _, err := m.getOrOpenSeekersWithLock(t, byTime) 811 byTime.Unlock() 812 if err != nil && err != errSeekerManagerFileSetNotFound { 813 multiErr = multiErr.Add(err) 814 } 815 } 816 817 return multiErr.FinalError() 818 } 819 820 func (m *seekerManager) newOpenSeeker( 821 shard uint32, 822 blockStart xtime.UnixNano, 823 volume int, 824 ) (DataFileSetSeeker, error) { 825 exists, err := DataFileSetExists( 826 m.filePathPrefix, m.namespace, shard, blockStart, volume) 827 if err != nil { 828 return nil, err 829 } 830 if !exists { 831 return nil, errSeekerManagerFileSetNotFound 832 } 833 834 // NB(r): Use a lock on the unread buffer to avoid multiple 835 // goroutines reusing the unread buffer that we share between the seekers 836 // when we open each seeker. 837 m.unreadBuf.Lock() 838 defer m.unreadBuf.Unlock() 839 840 seekerIface := NewSeeker( 841 m.filePathPrefix, 842 m.opts.DataReaderBufferSize(), 843 m.opts.InfoReaderBufferSize(), 844 m.bytesPool, 845 true, 846 m.opts, 847 ) 848 seeker := seekerIface.(*seeker) 849 850 // Set the unread buffer to reuse it amongst all seekers. 851 seeker.setUnreadBuffer(m.unreadBuf.value) 852 853 resources := m.getSeekerResources() 854 err = seeker.Open(m.namespace, shard, blockStart, volume, resources) 855 m.putSeekerResources(resources) 856 // This is expected is the cleanup manager may have 857 // removed out of retention index filesets. 858 if xerrors.Is(err, syscall.ENOENT) { 859 return nil, errSeekerManagerFileSetNotFound 860 } 861 if err != nil { 862 return nil, err 863 } 864 865 // Retrieve the buffer, it may have changed due to 866 // growing. Also release reference to the unread buffer. 867 m.unreadBuf.value = seeker.unreadBuffer() 868 seeker.setUnreadBuffer(nil) 869 870 return seeker, nil 871 } 872 873 func (m *seekerManager) seekersByTime(shard uint32) (*seekersByTime, bool) { 874 m.RLock() 875 if !m.shardExistsWithLock(shard) { 876 m.RUnlock() 877 return nil, false 878 } 879 880 if int(shard) < len(m.seekersByShardIdx) { 881 byTime := m.seekersByShardIdx[shard] 882 m.RUnlock() 883 return byTime, true 884 } 885 m.RUnlock() 886 887 m.Lock() 888 defer m.Unlock() 889 if !m.shardExistsWithLock(shard) { 890 return nil, false 891 } 892 893 // Check if raced with another call to this method 894 if int(shard) < len(m.seekersByShardIdx) { 895 byTime := m.seekersByShardIdx[shard] 896 return byTime, true 897 } 898 899 seekersByShardIdx := make([]*seekersByTime, shard+1) 900 idx := copy(seekersByShardIdx, m.seekersByShardIdx) 901 for ; idx < len(seekersByShardIdx); idx++ { 902 seekersByShardIdx[idx] = &seekersByTime{ 903 shard: uint32(idx), 904 seekers: make(map[xtime.UnixNano]rotatableSeekers), 905 } 906 } 907 908 m.seekersByShardIdx = seekersByShardIdx 909 byTime := m.seekersByShardIdx[shard] 910 911 return byTime, true 912 } 913 914 func (m *seekerManager) Close() error { 915 m.RLock() 916 917 if m.status == seekerManagerClosed { 918 m.RUnlock() 919 return errSeekerManagerAlreadyClosed 920 } 921 922 // Make sure all seekers are returned before allowing the SeekerManager to be closed. 923 // Actual cleanup of the seekers themselves will be handled by the openCloseLoop. 924 for _, byTime := range m.seekersByShardIdx { 925 byTime.RLock() 926 for _, seekersForBlock := range byTime.seekers { 927 // Ensure active seekers are all returned. 928 if seekersForBlock.active.anyBorrowedWithLock() { 929 byTime.RUnlock() 930 m.RUnlock() 931 return errCantCloseSeekerManagerWhileSeekersAreBorrowed 932 } 933 934 // Ensure inactive seekers are all returned. 935 if seekersForBlock.inactive.anyBorrowedWithLock() { 936 byTime.RUnlock() 937 m.RUnlock() 938 return errCantCloseSeekerManagerWhileSeekersAreBorrowed 939 } 940 } 941 byTime.RUnlock() 942 } 943 m.RUnlock() 944 945 m.Lock() 946 m.status = seekerManagerClosed 947 m.Unlock() 948 949 // Unregister for lease updates since all the seekers are going to be closed. 950 // NB(rartoul): Perform this outside the lock to prevent deadlock issues where 951 // the block.LeaseManager is trying to acquire the SeekerManager's lock (via 952 // a call to UpdateOpenLease) and the SeekerManager is trying to acquire the 953 // block.LeaseManager's lock (via a call to UnregisterLeaser). 954 m.blockRetrieverOpts.BlockLeaseManager().UnregisterLeaser(m) 955 956 <-m.openCloseLoopDoneCh 957 return nil 958 } 959 960 func (m *seekerManager) earliestSeekableBlockStart() xtime.UnixNano { 961 nowFn := m.opts.ClockOptions().NowFn() 962 now := xtime.ToUnixNano(nowFn()) 963 ropts := m.namespaceMetadata.Options().RetentionOptions() 964 return retention.FlushTimeStart(ropts, now) 965 } 966 967 func (m *seekerManager) latestSeekableBlockStart() xtime.UnixNano { 968 nowFn := m.opts.ClockOptions().NowFn() 969 now := xtime.ToUnixNano(nowFn()) 970 ropts := m.namespaceMetadata.Options().RetentionOptions() 971 return now.Truncate(ropts.BlockSize()) 972 } 973 974 // openCloseLoop ensures to keep seekers open for those times where they are 975 // available and closes them when they fall out of retention and expire. 976 func (m *seekerManager) openCloseLoop() { 977 var ( 978 shouldTryOpen []*seekersByTime 979 shouldClose []seekerManagerPendingClose 980 closing []seekersAndBloom 981 ) 982 resetSlices := func() { 983 for i := range shouldTryOpen { 984 shouldTryOpen[i] = nil 985 } 986 shouldTryOpen = shouldTryOpen[:0] 987 for i := range shouldClose { 988 shouldClose[i] = seekerManagerPendingClose{} 989 } 990 shouldClose = shouldClose[:0] 991 for i := range closing { 992 closing[i] = seekersAndBloom{} 993 } 994 closing = closing[:0] 995 } 996 997 for { 998 earliestSeekableBlockStart := m.earliestSeekableBlockStart() 999 1000 m.RLock() 1001 if m.status != seekerManagerOpen { 1002 m.RUnlock() 1003 break 1004 } 1005 1006 for _, byTime := range m.seekersByShardIdx { 1007 byTime.RLock() 1008 accessed := byTime.accessed 1009 byTime.RUnlock() 1010 if !accessed { 1011 continue 1012 } 1013 shouldTryOpen = append(shouldTryOpen, byTime) 1014 } 1015 m.RUnlock() 1016 1017 // Try opening any unopened times for accessed seekers 1018 for _, byTime := range shouldTryOpen { 1019 m.openAnyUnopenSeekersFn(byTime) 1020 } 1021 1022 m.RLock() 1023 for shard, byTime := range m.seekersByShardIdx { 1024 byTime.RLock() 1025 for blockStart := range byTime.seekers { 1026 if blockStart.Before(earliestSeekableBlockStart) || 1027 // Close seekers for shards that are no longer available. This 1028 // ensure that seekers are eventually consistent w/ shard state. 1029 !m.shardExistsWithLock(uint32(shard)) { 1030 shouldClose = append(shouldClose, seekerManagerPendingClose{ 1031 shard: uint32(shard), 1032 blockStart: blockStart, 1033 }) 1034 } 1035 } 1036 byTime.RUnlock() 1037 } 1038 1039 if len(shouldClose) > 0 { 1040 for _, elem := range shouldClose { 1041 byTime := m.seekersByShardIdx[elem.shard] 1042 byTime.Lock() 1043 seekers := byTime.seekers[elem.blockStart] 1044 allSeekersAreReturned := true 1045 1046 // Ensure no active seekers are still borrowed. 1047 if seekers.active.anyBorrowedWithLock() { 1048 allSeekersAreReturned = false 1049 } 1050 1051 // Ensure no inactive seekers are still borrowed. 1052 if seekers.inactive.anyBorrowedWithLock() { 1053 allSeekersAreReturned = false 1054 } 1055 1056 // Never close seekers unless they've all been returned because 1057 // some of them are clones of the original and can't be used once 1058 // the parent is closed (because they share underlying resources) 1059 if allSeekersAreReturned { 1060 closing = append(closing, seekers.active) 1061 closing = append(closing, seekers.inactive) 1062 delete(byTime.seekers, elem.blockStart) 1063 } 1064 byTime.Unlock() 1065 } 1066 } 1067 m.RUnlock() 1068 1069 // Close after releasing lock so any IO is done out of lock 1070 for _, seekersAndBloom := range closing { 1071 if err := seekersAndBloom.closeWithLock(); err != nil { 1072 m.logger.Error("err closing seekersAndBloom in SeekerManager openCloseLoop", zap.Error(err)) 1073 } 1074 } 1075 1076 m.sleepFn(seekManagerCloseInterval) 1077 1078 resetSlices() 1079 } 1080 1081 // Release all resources 1082 m.Lock() 1083 for _, byTime := range m.seekersByShardIdx { 1084 byTime.Lock() 1085 for _, seekersForBlock := range byTime.seekers { 1086 // Close the active seekers. 1087 if err := seekersForBlock.active.closeWithLock(); err != nil { 1088 m.logger.Error("err closing seeker in SeekerManager at end of openCloseLoop", zap.Error(err)) 1089 } 1090 1091 // Close the inactive seekers. 1092 if err := seekersForBlock.inactive.closeWithLock(); err != nil { 1093 m.logger.Error("err closing seeker in SeekerManager at end of openCloseLoop", zap.Error(err)) 1094 } 1095 } 1096 byTime.seekers = nil 1097 byTime.Unlock() 1098 } 1099 m.seekersByShardIdx = nil 1100 m.Unlock() 1101 1102 m.openCloseLoopDoneCh <- struct{}{} 1103 } 1104 1105 func (m *seekerManager) getSeekerResources() ReusableSeekerResources { 1106 return m.reusableSeekerResourcesPool.Get().(ReusableSeekerResources) 1107 } 1108 1109 func (m *seekerManager) putSeekerResources(r ReusableSeekerResources) { 1110 m.reusableSeekerResourcesPool.Put(r) 1111 }