github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/dbnode/persist/fs/retriever.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  // The block retriever is used to stream blocks of data from disk. It controls
    22  // the fetch concurrency on a per-Namespace basis I.E if the server is using
    23  // spinning-disks the concurrency can be set to 1 to serialize all disk fetches
    24  // for a given namespace, and the concurrency be set higher in the case of SSDs.
    25  // This fetch concurrency is primarily implemented via the number of concurrent
    26  // fetchLoops that the retriever creates.
    27  //
    28  // The block retriever also handles batching of requests for data, as well as
    29  // re-arranging the order of requests to increase data locality when seeking
    30  // through and across files.
    31  
    32  package fs
    33  
    34  import (
    35  	stdctx "context"
    36  	"errors"
    37  	"sort"
    38  	"sync"
    39  	"sync/atomic"
    40  	"time"
    41  
    42  	"github.com/m3db/m3/src/dbnode/namespace"
    43  	"github.com/m3db/m3/src/dbnode/sharding"
    44  	"github.com/m3db/m3/src/dbnode/storage/block"
    45  	"github.com/m3db/m3/src/dbnode/storage/limits"
    46  	"github.com/m3db/m3/src/dbnode/ts"
    47  	"github.com/m3db/m3/src/dbnode/x/xio"
    48  	"github.com/m3db/m3/src/x/context"
    49  	"github.com/m3db/m3/src/x/ident"
    50  	"github.com/m3db/m3/src/x/pool"
    51  	xtime "github.com/m3db/m3/src/x/time"
    52  
    53  	"github.com/uber-go/tally"
    54  	"go.uber.org/zap"
    55  )
    56  
    57  var (
    58  	errBlockRetrieverNotOpen             = errors.New("block retriever is not open")
    59  	errBlockRetrieverAlreadyOpenOrClosed = errors.New("block retriever already open or is closed")
    60  	errBlockRetrieverAlreadyClosed       = errors.New("block retriever already closed")
    61  	errNoSeekerMgr                       = errors.New("there is no open seeker manager")
    62  )
    63  
    64  const (
    65  	defaultRetrieveRequestQueueCapacity = 4096
    66  )
    67  
    68  type blockRetrieverStatus int
    69  
    70  type newSeekerMgrFn func(
    71  	bytesPool pool.CheckedBytesPool,
    72  	opts Options,
    73  	blockRetrieverOpts BlockRetrieverOptions,
    74  ) DataFileSetSeekerManager
    75  
    76  const (
    77  	blockRetrieverNotOpen blockRetrieverStatus = iota
    78  	blockRetrieverOpen
    79  	blockRetrieverClosed
    80  )
    81  
    82  type blockRetriever struct {
    83  	sync.RWMutex
    84  
    85  	opts                    BlockRetrieverOptions
    86  	fsOpts                  Options
    87  	logger                  *zap.Logger
    88  	queryLimits             limits.QueryLimits
    89  	bytesReadLimit          limits.LookbackLimit
    90  	seriesBloomFilterMisses tally.Counter
    91  
    92  	newSeekerMgrFn newSeekerMgrFn
    93  
    94  	reqPool    RetrieveRequestPool
    95  	bytesPool  pool.CheckedBytesPool
    96  	idPool     ident.Pool
    97  	nsMetadata namespace.Metadata
    98  
    99  	blockSize               time.Duration
   100  	nsCacheBlocksOnRetrieve bool
   101  
   102  	status                     blockRetrieverStatus
   103  	reqsByShardIdx             []*shardRetrieveRequests
   104  	seekerMgr                  DataFileSetSeekerManager
   105  	notifyFetch                chan struct{}
   106  	fetchLoopsShouldShutdownCh chan struct{}
   107  	fetchLoopsHaveShutdownCh   chan struct{}
   108  }
   109  
   110  // NewBlockRetriever returns a new block retriever for TSDB file sets.
   111  func NewBlockRetriever(
   112  	opts BlockRetrieverOptions,
   113  	fsOpts Options,
   114  ) (DataBlockRetriever, error) {
   115  	if err := opts.Validate(); err != nil {
   116  		return nil, err
   117  	}
   118  
   119  	scope := fsOpts.InstrumentOptions().MetricsScope().SubScope("retriever")
   120  
   121  	return &blockRetriever{
   122  		opts:                    opts,
   123  		fsOpts:                  fsOpts,
   124  		logger:                  fsOpts.InstrumentOptions().Logger(),
   125  		queryLimits:             opts.QueryLimits(),
   126  		bytesReadLimit:          opts.QueryLimits().BytesReadLimit(),
   127  		seriesBloomFilterMisses: scope.Counter("series-bloom-filter-misses"),
   128  		newSeekerMgrFn:          NewSeekerManager,
   129  		reqPool:                 opts.RetrieveRequestPool(),
   130  		bytesPool:               opts.BytesPool(),
   131  		idPool:                  opts.IdentifierPool(),
   132  		status:                  blockRetrieverNotOpen,
   133  		notifyFetch:             make(chan struct{}, 1),
   134  		// We just close this channel when the fetchLoops should shutdown, so no
   135  		// buffering is required
   136  		fetchLoopsShouldShutdownCh: make(chan struct{}),
   137  		fetchLoopsHaveShutdownCh:   make(chan struct{}, opts.FetchConcurrency()),
   138  	}, nil
   139  }
   140  
   141  func (r *blockRetriever) Open(
   142  	ns namespace.Metadata,
   143  	shardSet sharding.ShardSet,
   144  ) error {
   145  	r.Lock()
   146  	defer r.Unlock()
   147  
   148  	if r.status != blockRetrieverNotOpen {
   149  		return errBlockRetrieverAlreadyOpenOrClosed
   150  	}
   151  
   152  	seekerMgr := r.newSeekerMgrFn(r.bytesPool, r.fsOpts, r.opts)
   153  	if err := seekerMgr.Open(ns, shardSet); err != nil {
   154  		return err
   155  	}
   156  
   157  	r.nsMetadata = ns
   158  	r.status = blockRetrieverOpen
   159  	r.seekerMgr = seekerMgr
   160  
   161  	// Cache blockSize result and namespace specific block caching option
   162  	r.blockSize = ns.Options().RetentionOptions().BlockSize()
   163  	r.nsCacheBlocksOnRetrieve = ns.Options().CacheBlocksOnRetrieve()
   164  
   165  	for i := 0; i < r.opts.FetchConcurrency(); i++ {
   166  		go r.fetchLoop(seekerMgr)
   167  	}
   168  	return nil
   169  }
   170  
   171  func (r *blockRetriever) CacheShardIndices(shards []uint32) error {
   172  	r.RLock()
   173  	if r.status != blockRetrieverOpen {
   174  		r.RUnlock()
   175  		return errBlockRetrieverNotOpen
   176  	}
   177  	seekerMgr := r.seekerMgr
   178  	r.RUnlock()
   179  
   180  	// Don't hold the RLock() for duration of CacheShardIndices because
   181  	// it can take a very long time and it could block the regular read
   182  	// path (which sometimes needs to acquire an exclusive lock). In practice
   183  	// this is fine, it just means that the Retriever could be closed while a
   184  	// call to CacheShardIndices is still outstanding.
   185  	return seekerMgr.CacheShardIndices(shards)
   186  }
   187  
   188  func (r *blockRetriever) AssignShardSet(shardSet sharding.ShardSet) {
   189  	// NB(bodu): Block retriever will always be open before calling this method.
   190  	// But have this check anyways to be safe.
   191  	r.RLock()
   192  	defer r.RUnlock()
   193  	if r.status != blockRetrieverOpen {
   194  		return
   195  	}
   196  	r.seekerMgr.AssignShardSet(shardSet)
   197  }
   198  
   199  func (r *blockRetriever) fetchLoop(seekerMgr DataFileSetSeekerManager) {
   200  	var (
   201  		seekerResources    = NewReusableSeekerResources(r.fsOpts)
   202  		retrieverResources = newReusableRetrieverResources()
   203  		inFlight           []*retrieveRequest
   204  		currBatchReqs      []*retrieveRequest
   205  	)
   206  	for {
   207  		// Free references to the inflight requests
   208  		for i := range inFlight {
   209  			inFlight[i] = nil
   210  		}
   211  		inFlight = inFlight[:0]
   212  
   213  		// Select in flight requests
   214  		r.RLock()
   215  		// Move requests from shard retriever reqs into in flight slice
   216  		for _, reqs := range r.reqsByShardIdx {
   217  			reqs.Lock()
   218  			if len(reqs.queued) > 0 {
   219  				inFlight = append(inFlight, reqs.queued...)
   220  				reqs.resetQueued()
   221  			}
   222  			reqs.Unlock()
   223  		}
   224  
   225  		status := r.status
   226  		n := len(inFlight)
   227  		r.RUnlock()
   228  
   229  		// Exit if not open and fulfilled all open requests
   230  		if n == 0 && status != blockRetrieverOpen {
   231  			break
   232  		}
   233  
   234  		// If no fetches then no work to do, yield
   235  		if n == 0 {
   236  			select {
   237  			case <-r.notifyFetch:
   238  				continue
   239  			case <-r.fetchLoopsShouldShutdownCh:
   240  				break
   241  			}
   242  		}
   243  
   244  		// Files are all by shard and block time, the locality of
   245  		// files is therefore foremost by block time as that is when they are
   246  		// all written. Note that this sort does NOT mean that we're going to stripe
   247  		// through different files at once as you might expect at first, but simply
   248  		// that since all the fileset files are written at the end of a block period
   249  		// those files are more likely to be physically located close to each other
   250  		// on disk. In other words, instead of accessing files like this:
   251  		// 		shard1T1 --> shard1T2 --> shard1T3 --> shard2T1 --> shard2T2 --> shard2T3
   252  		// its probably faster to access them like this:
   253  		// 		shard1T1 --> shard2T1 --> shard1T2 --> shard2T2 --> shard1T3 --> shard2T3
   254  		// so we re-arrange the order of the requests to achieve that
   255  		sort.Sort(retrieveRequestByStartAscShardAsc(inFlight))
   256  
   257  		// Iterate through all in flight requests and send them to the seeker in
   258  		// batches of block time + shard.
   259  		currBatchShard := uint32(0)
   260  		currBatchStart := xtime.UnixNano(0)
   261  		currBatchReqs = currBatchReqs[:0]
   262  		for _, req := range inFlight {
   263  			if !req.start.Equal(currBatchStart) ||
   264  				req.shard != currBatchShard {
   265  				// Fetch any outstanding in the current batch
   266  				if len(currBatchReqs) > 0 {
   267  					r.fetchBatch(seekerMgr, currBatchShard, currBatchStart,
   268  						currBatchReqs, seekerResources, retrieverResources)
   269  					for i := range currBatchReqs {
   270  						currBatchReqs[i] = nil
   271  					}
   272  					currBatchReqs = currBatchReqs[:0]
   273  				}
   274  
   275  				// Set the new batch attributes
   276  				currBatchShard = req.shard
   277  				currBatchStart = req.start
   278  			}
   279  
   280  			// Enqueue into the current batch
   281  			currBatchReqs = append(currBatchReqs, req)
   282  		}
   283  
   284  		// Fetch any finally outstanding in the current batch
   285  		if len(currBatchReqs) > 0 {
   286  			r.fetchBatch(seekerMgr, currBatchShard, currBatchStart,
   287  				currBatchReqs, seekerResources, retrieverResources)
   288  			for i := range currBatchReqs {
   289  				currBatchReqs[i] = nil
   290  			}
   291  			currBatchReqs = currBatchReqs[:0]
   292  		}
   293  	}
   294  
   295  	r.fetchLoopsHaveShutdownCh <- struct{}{}
   296  }
   297  
   298  func (r *blockRetriever) fetchBatch(
   299  	seekerMgr DataFileSetSeekerManager,
   300  	shard uint32,
   301  	blockStart xtime.UnixNano,
   302  	allReqs []*retrieveRequest,
   303  	seekerResources ReusableSeekerResources,
   304  	retrieverResources *reusableRetrieverResources,
   305  ) {
   306  	var (
   307  		seeker     ConcurrentDataFileSetSeeker
   308  		callbackWg sync.WaitGroup
   309  	)
   310  
   311  	defer func() {
   312  		filteredReqs := allReqs[:0]
   313  		// Make sure requests are always fulfilled so if there's a code bug
   314  		// then errSeekNotCompleted is returned because req.success is not set
   315  		// rather than we have dangling goroutines stacking up.
   316  		for _, req := range allReqs {
   317  			if !req.waitingForCallback {
   318  				req.onDone()
   319  				continue
   320  			}
   321  
   322  			filteredReqs = append(filteredReqs, req)
   323  		}
   324  
   325  		callbackWg.Wait()
   326  		for _, req := range filteredReqs {
   327  			req.onDone()
   328  		}
   329  
   330  		// Reset resources to free any pointers in the slices still pointing
   331  		// to requests that are now completed and returned to pools.
   332  		retrieverResources.resetAll()
   333  
   334  		if seeker == nil {
   335  			// No borrowed seeker to return.
   336  			return
   337  		}
   338  
   339  		// Return borrowed seeker.
   340  		err := seekerMgr.Return(shard, blockStart, seeker)
   341  		if err != nil {
   342  			r.logger.Error("err returning seeker for shard",
   343  				zap.Uint32("shard", shard),
   344  				zap.Int64("blockStart", blockStart.Seconds()),
   345  				zap.Error(err),
   346  			)
   347  		}
   348  	}()
   349  
   350  	var err error
   351  	seeker, err = seekerMgr.Borrow(shard, blockStart)
   352  	if err != nil {
   353  		for _, req := range allReqs {
   354  			req.err = err
   355  		}
   356  		return
   357  	}
   358  
   359  	retrieverResources.resetDataReqs()
   360  	retrieverResources.dataReqs = append(retrieverResources.dataReqs, allReqs...)
   361  	reqs := retrieverResources.dataReqs
   362  
   363  	var limitErr error
   364  	if err := r.queryLimits.AnyFetchExceeded(); err != nil {
   365  		for _, req := range reqs {
   366  			req.err = err
   367  		}
   368  		return
   369  	}
   370  
   371  	for _, req := range reqs {
   372  		if limitErr != nil {
   373  			req.err = limitErr
   374  			continue
   375  		}
   376  
   377  		select {
   378  		case <-req.stdCtx.Done():
   379  			req.err = req.stdCtx.Err()
   380  			continue
   381  		default:
   382  		}
   383  
   384  		entry, err := seeker.SeekIndexEntry(req.id, seekerResources)
   385  		if err != nil && !errors.Is(err, errSeekIDNotFound) {
   386  			req.err = err
   387  			continue
   388  		}
   389  
   390  		if err := r.bytesReadLimit.Inc(int(entry.Size), req.source); err != nil {
   391  			req.err = err
   392  			limitErr = err
   393  			continue
   394  		}
   395  
   396  		if errors.Is(err, errSeekIDNotFound) {
   397  			req.notFound = true
   398  		}
   399  
   400  		req.indexEntry = entry
   401  	}
   402  
   403  	sort.Sort(retrieveRequestByIndexEntryOffsetAsc(reqs))
   404  	tagDecoderPool := r.fsOpts.TagDecoderPool()
   405  
   406  	blockCachingEnabled := r.opts.CacheBlocksOnRetrieve() && r.nsCacheBlocksOnRetrieve
   407  
   408  	// Seek and execute all requests
   409  	for _, req := range reqs {
   410  		if req.err != nil {
   411  			// Skip requests with error, will already get appropriate callback.
   412  			continue
   413  		}
   414  
   415  		if req.notFound {
   416  			// Only try to seek the ID if it exists and there haven't been any errors so
   417  			// far, otherwise we'll get a checksum mismatch error because the default
   418  			// offset value for indexEntry is zero.
   419  			req.success = true
   420  			req.onCallerOrRetrieverDone()
   421  			continue
   422  		}
   423  
   424  		select {
   425  		case <-req.stdCtx.Done():
   426  			req.err = req.stdCtx.Err()
   427  			continue
   428  		default:
   429  		}
   430  
   431  		data, err := seeker.SeekByIndexEntry(req.indexEntry, seekerResources)
   432  		if err != nil {
   433  			// If not found error is returned here, that's still an error since
   434  			// it's expected to be found if it was found in the index file.
   435  			req.err = err
   436  			continue
   437  		}
   438  
   439  		var (
   440  			seg, onRetrieveSeg ts.Segment
   441  			checksum           = req.indexEntry.DataChecksum
   442  		)
   443  		seg = ts.NewSegment(data, nil, checksum, ts.FinalizeHead)
   444  
   445  		// We don't need to call onRetrieve.OnRetrieveBlock if the ID was not found.
   446  		callOnRetrieve := blockCachingEnabled && req.onRetrieve != nil
   447  		if callOnRetrieve {
   448  			// NB(r): Need to also trigger callback with a copy of the data.
   449  			// This is used by the database to cache the in memory data for
   450  			// consequent fetches.
   451  			dataCopy := r.bytesPool.Get(data.Len())
   452  			onRetrieveSeg = ts.NewSegment(dataCopy, nil, checksum, ts.FinalizeHead)
   453  			dataCopy.AppendAll(data.Bytes())
   454  
   455  			if tags := req.indexEntry.EncodedTags; tags != nil && tags.Len() > 0 {
   456  				decoder := tagDecoderPool.Get()
   457  				// DecRef because we're transferring ownership from the index entry to
   458  				// the tagDecoder which will IncRef().
   459  				tags.DecRef()
   460  				decoder.Reset(tags)
   461  				req.tags = decoder
   462  			}
   463  		} else {
   464  			// If we didn't transfer ownership of the tags to the decoder above, then we
   465  			// no longer need them and we can can finalize them.
   466  			if tags := req.indexEntry.EncodedTags; tags != nil {
   467  				tags.DecRef()
   468  				tags.Finalize()
   469  			}
   470  		}
   471  
   472  		// Complete request.
   473  		req.onRetrieved(seg, req.nsCtx)
   474  		req.success = true
   475  
   476  		if !callOnRetrieve {
   477  			// No need to call the onRetrieve callback, but do need to call
   478  			// onCallerOrRetrieverDone since data requests do not get finalized
   479  			// when req.onDone is called since sometimes they need deferred
   480  			// finalization (when callOnRetrieve is true).
   481  			req.onCallerOrRetrieverDone()
   482  			continue
   483  		}
   484  
   485  		callbackWg.Add(1)
   486  		req.waitingForCallback = true
   487  		go func(r *retrieveRequest) {
   488  			// Call the onRetrieve callback and finalize.
   489  			r.onRetrieve.OnRetrieveBlock(r.id, r.tags, r.start, onRetrieveSeg, r.nsCtx)
   490  			r.onCallerOrRetrieverDone()
   491  			callbackWg.Done()
   492  		}(req)
   493  	}
   494  }
   495  
   496  func (r *blockRetriever) seriesPresentInBloomFilter(
   497  	id ident.ID,
   498  	shard uint32,
   499  	startTime xtime.UnixNano,
   500  ) (bool, error) {
   501  	// Capture variable and RLock() because this slice can be modified in the
   502  	// Open() method
   503  	r.RLock()
   504  	seekerMgr := r.seekerMgr
   505  	r.RUnlock()
   506  
   507  	// This should never happen unless caller tries to use Stream() before Open()
   508  	if seekerMgr == nil {
   509  		return false, errNoSeekerMgr
   510  	}
   511  
   512  	idExists, err := seekerMgr.Test(id, shard, startTime)
   513  	if err != nil {
   514  		return false, err
   515  	}
   516  
   517  	if !idExists {
   518  		r.seriesBloomFilterMisses.Inc(1)
   519  	}
   520  
   521  	return idExists, nil
   522  }
   523  
   524  // streamRequest returns a bool indicating if the ID was found, and any errors.
   525  func (r *blockRetriever) streamRequest(
   526  	ctx context.Context,
   527  	req *retrieveRequest,
   528  	shard uint32,
   529  	id ident.ID,
   530  	startTime xtime.UnixNano,
   531  ) error {
   532  	req.resultWg.Add(1)
   533  	req.shard = shard
   534  
   535  	// NB(r): If the ID is a ident.BytesID then we can just hold
   536  	// onto this ID.
   537  	seriesID := id
   538  	if !seriesID.IsNoFinalize() {
   539  		// NB(r): Clone the ID as we're not positive it will stay valid throughout
   540  		// the lifecycle of the async request.
   541  		seriesID = r.idPool.Clone(id)
   542  	}
   543  
   544  	req.id = seriesID
   545  	req.start = startTime
   546  	req.blockSize = r.blockSize
   547  
   548  	// Ensure to finalize at the end of request
   549  	ctx.RegisterFinalizer(req)
   550  
   551  	reqs, err := r.shardRequests(shard)
   552  	if err != nil {
   553  		return err
   554  	}
   555  
   556  	reqs.Lock()
   557  	reqs.queued = append(reqs.queued, req)
   558  	reqs.Unlock()
   559  
   560  	// Notify fetch loop
   561  	select {
   562  	case r.notifyFetch <- struct{}{}:
   563  	default:
   564  		// Loop busy, already ready to consume notification
   565  	}
   566  
   567  	// The request may not have completed yet, but it has an internal
   568  	// waitgroup which the caller will have to wait for before retrieving
   569  	// the data. This means that even though we're returning nil for error
   570  	// here, the caller may still encounter an error when they attempt to
   571  	// read the data.
   572  	return nil
   573  }
   574  
   575  func (r *blockRetriever) Stream(
   576  	ctx context.Context,
   577  	shard uint32,
   578  	id ident.ID,
   579  	startTime xtime.UnixNano,
   580  	onRetrieve block.OnRetrieveBlock,
   581  	nsCtx namespace.Context,
   582  ) (xio.BlockReader, error) {
   583  	found, err := r.seriesPresentInBloomFilter(id, shard, startTime)
   584  	if err != nil {
   585  		return xio.EmptyBlockReader, err
   586  	}
   587  	// If the ID is not in the seeker's bloom filter, then it's definitely not on
   588  	// disk and we can return immediately.
   589  	if !found {
   590  		return xio.EmptyBlockReader, nil
   591  	}
   592  
   593  	req := r.reqPool.Get()
   594  	// only save the go ctx to ensure we don't accidentally use the m3 ctx after it's been closed by the caller.
   595  	req.stdCtx = ctx.GoContext()
   596  	req.onRetrieve = onRetrieve
   597  
   598  	if source, ok := req.stdCtx.Value(limits.SourceContextKey).([]byte); ok {
   599  		req.source = source
   600  	}
   601  
   602  	err = r.streamRequest(ctx, req, shard, id, startTime)
   603  	if err != nil {
   604  		req.resultWg.Done()
   605  		return xio.EmptyBlockReader, err
   606  	}
   607  
   608  	// The request may not have completed yet, but it has an internal
   609  	// waitgroup which the caller will have to wait for before retrieving
   610  	// the data. This means that even though we're returning nil for error
   611  	// here, the caller may still encounter an error when they attempt to
   612  	// read the data.
   613  	return req.toBlock(), nil
   614  }
   615  
   616  func (r *blockRetriever) shardRequests(
   617  	shard uint32,
   618  ) (*shardRetrieveRequests, error) {
   619  	r.RLock()
   620  	if r.status != blockRetrieverOpen {
   621  		r.RUnlock()
   622  		return nil, errBlockRetrieverNotOpen
   623  	}
   624  	if int(shard) < len(r.reqsByShardIdx) {
   625  		reqs := r.reqsByShardIdx[shard]
   626  		r.RUnlock()
   627  		return reqs, nil
   628  	}
   629  	r.RUnlock()
   630  
   631  	r.Lock()
   632  	defer r.Unlock()
   633  
   634  	// Check if raced with another call to this method
   635  	if r.status != blockRetrieverOpen {
   636  		return nil, errBlockRetrieverNotOpen
   637  	}
   638  	if int(shard) < len(r.reqsByShardIdx) {
   639  		reqs := r.reqsByShardIdx[shard]
   640  		return reqs, nil
   641  	}
   642  
   643  	reqsByShardIdx := make([]*shardRetrieveRequests, shard+1)
   644  
   645  	for i := range reqsByShardIdx {
   646  		if i < len(r.reqsByShardIdx) {
   647  			reqsByShardIdx[i] = r.reqsByShardIdx[i]
   648  			continue
   649  		}
   650  		capacity := defaultRetrieveRequestQueueCapacity
   651  		reqsByShardIdx[i] = &shardRetrieveRequests{
   652  			shard:  uint32(i),
   653  			queued: make([]*retrieveRequest, 0, capacity),
   654  		}
   655  	}
   656  
   657  	r.reqsByShardIdx = reqsByShardIdx
   658  	reqs := r.reqsByShardIdx[shard]
   659  
   660  	return reqs, nil
   661  }
   662  
   663  func (r *blockRetriever) Close() error {
   664  	r.Lock()
   665  	if r.status == blockRetrieverClosed {
   666  		r.Unlock()
   667  		return errBlockRetrieverAlreadyClosed
   668  	}
   669  	r.nsMetadata = nil
   670  	r.status = blockRetrieverClosed
   671  
   672  	r.blockSize = 0
   673  	r.Unlock()
   674  
   675  	close(r.fetchLoopsShouldShutdownCh)
   676  	for i := 0; i < r.opts.FetchConcurrency(); i++ {
   677  		<-r.fetchLoopsHaveShutdownCh
   678  	}
   679  
   680  	return r.seekerMgr.Close()
   681  }
   682  
   683  type shardRetrieveRequests struct {
   684  	sync.Mutex
   685  	shard  uint32
   686  	queued []*retrieveRequest
   687  }
   688  
   689  func (reqs *shardRetrieveRequests) resetQueued() {
   690  	// Free references to the queued requests
   691  	for i := range reqs.queued {
   692  		reqs.queued[i] = nil
   693  	}
   694  	reqs.queued = reqs.queued[:0]
   695  }
   696  
   697  // Don't forget to update the resetForReuse method when adding a new field
   698  type retrieveRequest struct {
   699  	finalized          bool
   700  	waitingForCallback bool
   701  	resultWg           sync.WaitGroup
   702  
   703  	pool *reqPool
   704  
   705  	id         ident.ID
   706  	tags       ident.TagIterator
   707  	start      xtime.UnixNano
   708  	blockSize  time.Duration
   709  	onRetrieve block.OnRetrieveBlock
   710  	nsCtx      namespace.Context
   711  	source     []byte
   712  	stdCtx     stdctx.Context
   713  
   714  	indexEntry IndexEntry
   715  	reader     xio.SegmentReader
   716  
   717  	err error
   718  
   719  	// Finalize requires two calls to finalize (once both the user of the
   720  	// request and the retriever fetch loop is done, and only then, can
   721  	// we free this request) so we track this with an atomic here.
   722  	finalizes uint32
   723  	shard     uint32
   724  
   725  	notFound bool
   726  	success  bool
   727  }
   728  
   729  func (req *retrieveRequest) toBlock() xio.BlockReader {
   730  	return xio.BlockReader{
   731  		SegmentReader: req,
   732  		Start:         req.start,
   733  		BlockSize:     req.blockSize,
   734  	}
   735  }
   736  
   737  func (req *retrieveRequest) onRetrieved(segment ts.Segment, nsCtx namespace.Context) {
   738  	req.nsCtx = nsCtx
   739  	req.Reset(segment)
   740  }
   741  
   742  func (req *retrieveRequest) onDone() {
   743  	var (
   744  		err     = req.err
   745  		success = req.success
   746  	)
   747  
   748  	if err == nil && !success {
   749  		// Require explicit success, otherwise this request
   750  		// was never completed.
   751  		// This helps catch code bugs where this element wasn't explicitly
   752  		// handled as completed during a fetch batch call instead of
   753  		// returning but with no actual result set properly.
   754  		req.err = errSeekNotCompleted
   755  	}
   756  
   757  	req.resultWg.Done()
   758  
   759  	// Do not call onCallerOrRetrieverDone since the OnRetrieveCallback
   760  	// code path will call req.onCallerOrRetrieverDone() when it's done.
   761  	// If encountered an error though, should call it since not waiting for
   762  	// callback to finish or even if not waiting for callback to finish
   763  	// the happy path that calls this pre-emptively has not executed either.
   764  	// That is if-and-only-if request is data request and is successful and
   765  	// will req.onCallerOrRetrieverDone() be called in a deferred manner.
   766  	if !success {
   767  		req.onCallerOrRetrieverDone()
   768  	}
   769  }
   770  
   771  func (req *retrieveRequest) Reset(segment ts.Segment) {
   772  	req.reader.Reset(segment)
   773  }
   774  
   775  func (req *retrieveRequest) ResetWindowed(
   776  	segment ts.Segment,
   777  	start xtime.UnixNano,
   778  	blockSize time.Duration,
   779  ) {
   780  	req.start = start
   781  	req.blockSize = blockSize
   782  	req.Reset(segment)
   783  }
   784  
   785  func (req *retrieveRequest) onCallerOrRetrieverDone() {
   786  	if atomic.AddUint32(&req.finalizes, 1) != 2 {
   787  		return
   788  	}
   789  
   790  	if req.id != nil {
   791  		req.id.Finalize()
   792  		req.id = nil
   793  	}
   794  	if req.tags != nil {
   795  		req.tags.Close()
   796  		req.tags = ident.EmptyTagIterator
   797  	}
   798  	if req.reader != nil {
   799  		req.reader.Finalize()
   800  		req.reader = nil
   801  	}
   802  
   803  	req.pool.Put(req)
   804  }
   805  
   806  func (req *retrieveRequest) SegmentReader() (xio.SegmentReader, error) {
   807  	return req.reader, nil
   808  }
   809  
   810  // NB: be aware to avoid calling Clone() in a hot path, since it copies the
   811  // underlying bytes.
   812  func (req *retrieveRequest) Clone(
   813  	pool pool.CheckedBytesPool,
   814  ) (xio.SegmentReader, error) {
   815  	req.resultWg.Wait() // wait until result is ready
   816  	if req.err != nil {
   817  		return nil, req.err
   818  	}
   819  	return req.reader.Clone(pool)
   820  }
   821  
   822  func (req *retrieveRequest) BlockSize() time.Duration {
   823  	return req.blockSize
   824  }
   825  
   826  func (req *retrieveRequest) Read64() (word uint64, n byte, err error) {
   827  	req.resultWg.Wait()
   828  	if req.err != nil {
   829  		return 0, 0, req.err
   830  	}
   831  	return req.reader.Read64()
   832  }
   833  
   834  func (req *retrieveRequest) Peek64() (word uint64, n byte, err error) {
   835  	req.resultWg.Wait()
   836  	if req.err != nil {
   837  		return 0, 0, req.err
   838  	}
   839  	return req.reader.Peek64()
   840  }
   841  
   842  func (req *retrieveRequest) Segment() (ts.Segment, error) {
   843  	req.resultWg.Wait()
   844  	if req.err != nil {
   845  		return ts.Segment{}, req.err
   846  	}
   847  	return req.reader.Segment()
   848  }
   849  
   850  func (req *retrieveRequest) Finalize() {
   851  	// May not actually finalize the request, depending on if
   852  	// retriever is done too
   853  	if req.finalized {
   854  		return
   855  	}
   856  
   857  	req.resultWg.Wait()
   858  	req.finalized = true
   859  	req.onCallerOrRetrieverDone()
   860  }
   861  
   862  func (req *retrieveRequest) resetForReuse() {
   863  	req.resultWg = sync.WaitGroup{}
   864  	req.finalized = false
   865  	req.finalizes = 0
   866  	req.source = nil
   867  	req.shard = 0
   868  	req.id = nil
   869  	req.tags = ident.EmptyTagIterator
   870  	req.start = 0
   871  	req.blockSize = 0
   872  	req.onRetrieve = nil
   873  	req.indexEntry = IndexEntry{}
   874  	req.reader = nil
   875  	req.err = nil
   876  	req.notFound = false
   877  	req.success = false
   878  	req.stdCtx = nil
   879  }
   880  
   881  type retrieveRequestByStartAscShardAsc []*retrieveRequest
   882  
   883  func (r retrieveRequestByStartAscShardAsc) Len() int      { return len(r) }
   884  func (r retrieveRequestByStartAscShardAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
   885  func (r retrieveRequestByStartAscShardAsc) Less(i, j int) bool {
   886  	if !r[i].start.Equal(r[j].start) {
   887  		return r[i].start.Before(r[j].start)
   888  	}
   889  	return r[i].shard < r[j].shard
   890  }
   891  
   892  type retrieveRequestByIndexEntryOffsetAsc []*retrieveRequest
   893  
   894  func (r retrieveRequestByIndexEntryOffsetAsc) Len() int      { return len(r) }
   895  func (r retrieveRequestByIndexEntryOffsetAsc) Swap(i, j int) { r[i], r[j] = r[j], r[i] }
   896  func (r retrieveRequestByIndexEntryOffsetAsc) Less(i, j int) bool {
   897  	return r[i].indexEntry.Offset < r[j].indexEntry.Offset
   898  }
   899  
   900  // RetrieveRequestPool is the retrieve request pool.
   901  type RetrieveRequestPool interface {
   902  	// Init initializes the request pool.
   903  	Init()
   904  	// Get gets a retrieve request.
   905  	Get() *retrieveRequest
   906  	// Put returns a retrieve request to the pool.
   907  	Put(req *retrieveRequest)
   908  }
   909  
   910  type reqPool struct {
   911  	segmentReaderPool xio.SegmentReaderPool
   912  	pool              pool.ObjectPool
   913  }
   914  
   915  // NewRetrieveRequestPool returns a new retrieve request pool.
   916  func NewRetrieveRequestPool(
   917  	segmentReaderPool xio.SegmentReaderPool,
   918  	opts pool.ObjectPoolOptions,
   919  ) RetrieveRequestPool {
   920  	return &reqPool{
   921  		segmentReaderPool: segmentReaderPool,
   922  		pool:              pool.NewObjectPool(opts),
   923  	}
   924  }
   925  
   926  func (p *reqPool) Init() {
   927  	p.pool.Init(func() interface{} {
   928  		return &retrieveRequest{pool: p}
   929  	})
   930  }
   931  
   932  func (p *reqPool) Get() *retrieveRequest {
   933  	req := p.pool.Get().(*retrieveRequest)
   934  	req.resetForReuse()
   935  	req.reader = p.segmentReaderPool.Get()
   936  	return req
   937  }
   938  
   939  func (p *reqPool) Put(req *retrieveRequest) {
   940  	// Also call reset for reuse to nil any references before
   941  	// putting back in pool to avoid holding strong refs to any
   942  	// shortly lived objects while still in the pool
   943  	req.resetForReuse()
   944  	p.pool.Put(req)
   945  }
   946  
   947  type reusableRetrieverResources struct {
   948  	dataReqs []*retrieveRequest
   949  }
   950  
   951  func newReusableRetrieverResources() *reusableRetrieverResources {
   952  	return &reusableRetrieverResources{}
   953  }
   954  
   955  func (r *reusableRetrieverResources) resetAll() {
   956  	r.resetDataReqs()
   957  }
   958  
   959  func (r *reusableRetrieverResources) resetDataReqs() {
   960  	for i := range r.dataReqs {
   961  		r.dataReqs[i] = nil
   962  	}
   963  	r.dataReqs = r.dataReqs[:0]
   964  }