github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/m3ninx/index/segment/fst/segment.go (about)

     1  // Copyright (c) 2018 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 fst
    22  
    23  import (
    24  	"errors"
    25  	"fmt"
    26  	"io"
    27  	"sync"
    28  
    29  	"github.com/m3db/m3/src/m3ninx/doc"
    30  	"github.com/m3db/m3/src/m3ninx/generated/proto/fswriter"
    31  	"github.com/m3db/m3/src/m3ninx/index"
    32  	"github.com/m3db/m3/src/m3ninx/index/segment"
    33  	sgmt "github.com/m3db/m3/src/m3ninx/index/segment"
    34  	"github.com/m3db/m3/src/m3ninx/index/segment/fst/encoding"
    35  	"github.com/m3db/m3/src/m3ninx/index/segment/fst/encoding/docs"
    36  	"github.com/m3db/m3/src/m3ninx/postings"
    37  	"github.com/m3db/m3/src/m3ninx/postings/pilosa"
    38  	"github.com/m3db/m3/src/m3ninx/postings/roaring"
    39  	"github.com/m3db/m3/src/m3ninx/x"
    40  	"github.com/m3db/m3/src/x/context"
    41  	xerrors "github.com/m3db/m3/src/x/errors"
    42  	"github.com/m3db/m3/src/x/mmap"
    43  
    44  	pilosaroaring "github.com/m3dbx/pilosa/roaring"
    45  	"github.com/m3dbx/vellum"
    46  )
    47  
    48  var (
    49  	errReaderClosed                         = errors.New("segment is closed")
    50  	errReaderFinalized                      = errors.New("segment is finalized")
    51  	errReaderNilRegexp                      = errors.New("nil regexp provided")
    52  	errDocumentsDataUnset                   = errors.New("documents data bytes are not set")
    53  	errDocumentsIdxUnset                    = errors.New("documents index bytes are not set")
    54  	errPostingsDataUnset                    = errors.New("postings data bytes are not set")
    55  	errFSTTermsDataUnset                    = errors.New("fst terms data bytes are not set")
    56  	errFSTFieldsDataUnset                   = errors.New("fst fields data bytes are not set")
    57  	errUnsupportedFeatureFieldsPostingsList = errors.New(
    58  		"fst unsupported operation on old segment version: missing field postings list",
    59  	)
    60  )
    61  
    62  // SegmentData represent the collection of required parameters to construct a Segment.
    63  type SegmentData struct {
    64  	Version  Version
    65  	Metadata []byte
    66  
    67  	DocsData      mmap.Descriptor
    68  	DocsIdxData   mmap.Descriptor
    69  	PostingsData  mmap.Descriptor
    70  	FSTTermsData  mmap.Descriptor
    71  	FSTFieldsData mmap.Descriptor
    72  
    73  	// DocsReader is an alternative to specifying
    74  	// the docs data and docs idx data if the documents
    75  	// already reside in memory and we want to use the
    76  	// in memory references instead.
    77  	DocsReader docs.Reader
    78  
    79  	Closer io.Closer
    80  }
    81  
    82  // Validate validates the provided segment data, returning an error if it's not.
    83  func (sd SegmentData) Validate() error {
    84  	if err := sd.Version.Supported(); err != nil {
    85  		return err
    86  	}
    87  
    88  	if sd.PostingsData.Bytes == nil {
    89  		return errPostingsDataUnset
    90  	}
    91  
    92  	if sd.FSTTermsData.Bytes == nil {
    93  		return errFSTTermsDataUnset
    94  	}
    95  
    96  	if sd.FSTFieldsData.Bytes == nil {
    97  		return errFSTFieldsDataUnset
    98  	}
    99  
   100  	if sd.DocsReader == nil {
   101  		if sd.DocsData.Bytes == nil {
   102  			return errDocumentsDataUnset
   103  		}
   104  
   105  		if sd.DocsIdxData.Bytes == nil {
   106  			return errDocumentsIdxUnset
   107  		}
   108  	}
   109  
   110  	return nil
   111  }
   112  
   113  // NewSegment returns a new Segment backed by the provided options.
   114  // NB(prateek): this method only assumes ownership of the data if it returns a nil error,
   115  // otherwise, the user is expected to handle the lifecycle of the input.
   116  func NewSegment(data SegmentData, opts Options) (Segment, error) {
   117  	if err := data.Validate(); err != nil {
   118  		return nil, err
   119  	}
   120  
   121  	metadata := fswriter.Metadata{}
   122  	if err := metadata.Unmarshal(data.Metadata); err != nil {
   123  		return nil, err
   124  	}
   125  
   126  	if metadata.PostingsFormat != fswriter.PostingsFormat_PILOSAV1_POSTINGS_FORMAT {
   127  		return nil, fmt.Errorf("unsupported postings format: %v", metadata.PostingsFormat.String())
   128  	}
   129  
   130  	fieldsFST, err := vellum.Load(data.FSTFieldsData.Bytes)
   131  	if err != nil {
   132  		return nil, fmt.Errorf("unable to load fields fst: %v", err)
   133  	}
   134  
   135  	var (
   136  		docsThirdPartyReader  = data.DocsReader
   137  		docsDataReader        *docs.DataReader
   138  		docsEncodedDataReader *docs.EncodedDataReader
   139  		docsIndexReader       *docs.IndexReader
   140  	)
   141  	if docsThirdPartyReader == nil {
   142  		docsDataReader = docs.NewDataReader(data.DocsData.Bytes)
   143  		docsIndexReader, err = docs.NewIndexReader(data.DocsIdxData.Bytes)
   144  		if err != nil {
   145  			return nil, fmt.Errorf("unable to load documents index: %v", err)
   146  		}
   147  	}
   148  	docsEncodedDataReader = docs.NewEncodedDataReader(data.DocsData.Bytes)
   149  
   150  	s := &fsSegment{
   151  		fieldsFST:             fieldsFST,
   152  		docsDataReader:        docsDataReader,
   153  		docsEncodedDataReader: docsEncodedDataReader,
   154  		docsIndexReader:       docsIndexReader,
   155  		docsThirdPartyReader:  docsThirdPartyReader,
   156  
   157  		data:    data,
   158  		opts:    opts,
   159  		numDocs: metadata.NumDocs,
   160  	}
   161  
   162  	// NB(r): The segment uses the context finalization to finalize
   163  	// resources. Finalize is called after Close is called and all
   164  	// the segment readers have also been closed.
   165  	s.ctx = opts.ContextPool().Get()
   166  	s.ctx.RegisterFinalizer(s)
   167  
   168  	return s, nil
   169  }
   170  
   171  // Ensure FST segment implements ImmutableSegment so can be casted upwards
   172  // and mmap's can be freed.
   173  var _ segment.ImmutableSegment = (*fsSegment)(nil)
   174  
   175  type fsSegment struct {
   176  	sync.RWMutex
   177  	ctx                   context.Context
   178  	closed                bool
   179  	finalized             bool
   180  	fieldsFST             *vellum.FST
   181  	docsDataReader        *docs.DataReader
   182  	docsEncodedDataReader *docs.EncodedDataReader
   183  	docsIndexReader       *docs.IndexReader
   184  	docsThirdPartyReader  docs.Reader
   185  	data                  SegmentData
   186  	opts                  Options
   187  
   188  	numDocs int64
   189  }
   190  
   191  func (r *fsSegment) SegmentData(ctx context.Context) (SegmentData, error) {
   192  	r.RLock()
   193  	defer r.RUnlock()
   194  	if r.closed {
   195  		return SegmentData{}, errReaderClosed
   196  	}
   197  
   198  	// NB(r): Ensure that we do not release, mmaps, etc
   199  	// until all readers have been closed.
   200  	r.ctx.DependsOn(ctx)
   201  	return r.data, nil
   202  }
   203  
   204  func (r *fsSegment) Size() int64 {
   205  	r.RLock()
   206  	defer r.RUnlock()
   207  	if r.closed {
   208  		return 0
   209  	}
   210  	return r.numDocs
   211  }
   212  
   213  func (r *fsSegment) ContainsID(docID []byte) (bool, error) {
   214  	r.RLock()
   215  	defer r.RUnlock()
   216  	if r.closed {
   217  		return false, errReaderClosed
   218  	}
   219  
   220  	termsFST, exists, err := r.retrieveTermsFSTWithRLock(doc.IDReservedFieldName)
   221  	if err != nil {
   222  		return false, err
   223  	}
   224  
   225  	if !exists {
   226  		return false, fmt.Errorf("internal error while retrieving id FST: %v", err)
   227  	}
   228  
   229  	_, exists, err = termsFST.Get(docID)
   230  	closeErr := termsFST.Close()
   231  	if err != nil {
   232  		return false, err
   233  	}
   234  
   235  	return exists, closeErr
   236  }
   237  
   238  func (r *fsSegment) ContainsField(field []byte) (bool, error) {
   239  	r.RLock()
   240  	defer r.RUnlock()
   241  	if r.closed {
   242  		return false, errReaderClosed
   243  	}
   244  	return r.fieldsFST.Contains(field)
   245  }
   246  
   247  func (r *fsSegment) Reader() (sgmt.Reader, error) {
   248  	r.RLock()
   249  	defer r.RUnlock()
   250  	if r.closed {
   251  		return nil, errReaderClosed
   252  	}
   253  
   254  	reader := newReader(r, r.opts)
   255  
   256  	// NB(r): Ensure that we do not release, mmaps, etc
   257  	// until all readers have been closed.
   258  	r.ctx.DependsOn(reader.ctx)
   259  
   260  	return reader, nil
   261  }
   262  
   263  func (r *fsSegment) Close() error {
   264  	r.Lock()
   265  	if r.closed {
   266  		r.Unlock()
   267  		return errReaderClosed
   268  	}
   269  	r.closed = true
   270  	r.Unlock()
   271  	// NB(r): Inform context we are done, once all segment readers are
   272  	// closed the segment Finalize will be called async.
   273  	r.ctx.Close()
   274  	return nil
   275  }
   276  
   277  func (r *fsSegment) Finalize() {
   278  	r.Lock()
   279  	r.fieldsFST.Close()
   280  	if r.data.Closer != nil {
   281  		r.data.Closer.Close()
   282  	}
   283  	r.finalized = true
   284  	r.Unlock()
   285  }
   286  
   287  func (r *fsSegment) FieldsIterable() sgmt.FieldsIterable {
   288  	return r
   289  }
   290  
   291  func (r *fsSegment) Fields() (sgmt.FieldsIterator, error) {
   292  	r.RLock()
   293  	defer r.RUnlock()
   294  	if r.closed {
   295  		return nil, errReaderClosed
   296  	}
   297  
   298  	iter := newFSTTermsIter()
   299  	iter.reset(fstTermsIterOpts{
   300  		seg:         r,
   301  		fst:         r.fieldsFST,
   302  		finalizeFST: false,
   303  	})
   304  	return iter, nil
   305  }
   306  
   307  func (r *fsSegment) TermsIterable() sgmt.TermsIterable {
   308  	return &termsIterable{
   309  		r:            r,
   310  		fieldsIter:   newFSTTermsIter(),
   311  		postingsIter: newFSTTermsPostingsIter(),
   312  	}
   313  }
   314  
   315  func (r *fsSegment) FreeMmap() error {
   316  	multiErr := xerrors.NewMultiError()
   317  
   318  	// NB(bodu): PostingsData, FSTTermsData and FSTFieldsData always present.
   319  	if err := mmap.MadviseDontNeed(r.data.PostingsData); err != nil {
   320  		multiErr = multiErr.Add(err)
   321  	}
   322  	if err := mmap.MadviseDontNeed(r.data.FSTTermsData); err != nil {
   323  		multiErr = multiErr.Add(err)
   324  	}
   325  	if err := mmap.MadviseDontNeed(r.data.FSTFieldsData); err != nil {
   326  		multiErr = multiErr.Add(err)
   327  	}
   328  
   329  	// DocsData and DocsIdxData are not always present.
   330  	if r.data.DocsData.Bytes != nil {
   331  		if err := mmap.MadviseDontNeed(r.data.DocsData); err != nil {
   332  			multiErr = multiErr.Add(err)
   333  		}
   334  	}
   335  	if r.data.DocsIdxData.Bytes != nil {
   336  		if err := mmap.MadviseDontNeed(r.data.DocsIdxData); err != nil {
   337  			multiErr = multiErr.Add(err)
   338  		}
   339  	}
   340  
   341  	return multiErr.FinalError()
   342  }
   343  
   344  // termsIterable allows multiple term lookups to share the same roaring
   345  // bitmap being unpacked for use when iterating over an entire segment
   346  type termsIterable struct {
   347  	r            *fsSegment
   348  	fieldsIter   *fstTermsIter
   349  	postingsIter *fstTermsPostingsIter
   350  }
   351  
   352  func newTermsIterable(r *fsSegment) *termsIterable {
   353  	return &termsIterable{
   354  		r:            r,
   355  		fieldsIter:   newFSTTermsIter(),
   356  		postingsIter: newFSTTermsPostingsIter(),
   357  	}
   358  }
   359  
   360  func (i *termsIterable) Terms(field []byte) (sgmt.TermsIterator, error) {
   361  	i.r.RLock()
   362  	defer i.r.RUnlock()
   363  	if i.r.closed {
   364  		return nil, errReaderClosed
   365  	}
   366  	return i.termsNotClosedMaybeFinalizedWithRLock(field)
   367  }
   368  
   369  func (i *termsIterable) termsNotClosedMaybeFinalizedWithRLock(
   370  	field []byte,
   371  ) (sgmt.TermsIterator, error) {
   372  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   373  	// calling match field after this segment is finalized.
   374  	if i.r.finalized {
   375  		return nil, errReaderFinalized
   376  	}
   377  
   378  	termsFST, exists, err := i.r.retrieveTermsFSTWithRLock(field)
   379  	if err != nil {
   380  		return nil, err
   381  	}
   382  
   383  	if !exists {
   384  		return sgmt.EmptyTermsIterator, nil
   385  	}
   386  
   387  	i.fieldsIter.reset(fstTermsIterOpts{
   388  		seg:         i.r,
   389  		fst:         termsFST,
   390  		finalizeFST: true,
   391  	})
   392  	i.postingsIter.reset(i.r, i.fieldsIter)
   393  	return i.postingsIter, nil
   394  }
   395  
   396  func (i *termsIterable) fieldsNotClosedMaybeFinalizedWithRLock() (sgmt.FieldsPostingsListIterator, error) {
   397  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   398  	// calling match field after this segment is finalized.
   399  	if i.r.finalized {
   400  		return nil, errReaderFinalized
   401  	}
   402  
   403  	i.fieldsIter.reset(fstTermsIterOpts{
   404  		seg:         i.r,
   405  		fst:         i.r.fieldsFST,
   406  		finalizeFST: false,
   407  		fieldsFST:   true,
   408  	})
   409  	i.postingsIter.reset(i.r, i.fieldsIter)
   410  	return i.postingsIter, nil
   411  }
   412  
   413  func (r *fsSegment) UnmarshalPostingsListBitmap(b *pilosaroaring.Bitmap, offset uint64) error {
   414  	r.RLock()
   415  	defer r.RUnlock()
   416  	if r.closed {
   417  		return errReaderClosed
   418  	}
   419  
   420  	return r.unmarshalPostingsListBitmapNotClosedMaybeFinalizedWithLock(b, offset)
   421  }
   422  
   423  func (r *fsSegment) unmarshalPostingsListBitmapNotClosedMaybeFinalizedWithLock(b *pilosaroaring.Bitmap, offset uint64) error {
   424  	if r.finalized {
   425  		return errReaderFinalized
   426  	}
   427  
   428  	postingsBytes, err := r.retrieveBytesWithRLock(r.data.PostingsData.Bytes, offset)
   429  	if err != nil {
   430  		return fmt.Errorf("unable to retrieve postings data: %v", err)
   431  	}
   432  
   433  	b.Reset()
   434  	return b.UnmarshalBinary(postingsBytes)
   435  }
   436  
   437  func (r *fsSegment) matchFieldNotClosedMaybeFinalizedWithRLock(
   438  	field []byte,
   439  ) (postings.List, error) {
   440  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   441  	// calling match field after this segment is finalized.
   442  	if r.finalized {
   443  		return nil, errReaderFinalized
   444  	}
   445  
   446  	if !r.data.Version.supportsFieldPostingsList() {
   447  		// i.e. don't have the field level postings list, so fall back to regexp
   448  		return r.matchRegexpNotClosedMaybeFinalizedWithRLock(field, index.DotStarCompiledRegex())
   449  	}
   450  
   451  	termsFSTOffset, exists, err := r.fieldsFST.Get(field)
   452  	if err != nil {
   453  		return nil, err
   454  	}
   455  	if !exists {
   456  		// i.e. we don't know anything about the term, so can early return an empty postings list
   457  		return r.opts.PostingsListPool().Get(), nil
   458  	}
   459  
   460  	fieldData, err := r.unmarshalFieldDataNotClosedMaybeFinalizedWithRLock(termsFSTOffset)
   461  	if err != nil {
   462  		return nil, err
   463  	}
   464  
   465  	postingsOffset := fieldData.FieldPostingsListOffset
   466  	return r.retrievePostingsListWithRLock(postingsOffset)
   467  }
   468  
   469  func (r *fsSegment) unmarshalFieldDataNotClosedMaybeFinalizedWithRLock(
   470  	fieldDataOffset uint64,
   471  ) (fswriter.FieldData, error) {
   472  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   473  	// calling match field after this segment is finalized.
   474  	if r.finalized {
   475  		return fswriter.FieldData{}, errReaderFinalized
   476  	}
   477  	if !r.data.Version.supportsFieldPostingsList() {
   478  		return fswriter.FieldData{}, errUnsupportedFeatureFieldsPostingsList
   479  	}
   480  
   481  	protoBytes, _, err := r.retrieveTermsBytesWithRLock(r.data.FSTTermsData.Bytes, fieldDataOffset)
   482  	if err != nil {
   483  		return fswriter.FieldData{}, err
   484  	}
   485  
   486  	var fieldData fswriter.FieldData
   487  	if err := fieldData.Unmarshal(protoBytes); err != nil {
   488  		return fswriter.FieldData{}, err
   489  	}
   490  	return fieldData, nil
   491  }
   492  
   493  func (r *fsSegment) matchTermNotClosedMaybeFinalizedWithRLock(
   494  	field, term []byte,
   495  ) (postings.List, error) {
   496  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   497  	// calling match field after this segment is finalized.
   498  	if r.finalized {
   499  		return nil, errReaderFinalized
   500  	}
   501  
   502  	termsFST, exists, err := r.retrieveTermsFSTWithRLock(field)
   503  	if err != nil {
   504  		return nil, err
   505  	}
   506  
   507  	if !exists {
   508  		// i.e. we don't know anything about the field, so can early return an empty postings list
   509  		return r.opts.PostingsListPool().Get(), nil
   510  	}
   511  
   512  	fstCloser := x.NewSafeCloser(termsFST)
   513  	defer fstCloser.Close()
   514  
   515  	postingsOffset, exists, err := termsFST.Get(term)
   516  	if err != nil {
   517  		return nil, err
   518  	}
   519  
   520  	if !exists {
   521  		// i.e. we don't know anything about the term, so can early return an empty postings list
   522  		return r.opts.PostingsListPool().Get(), nil
   523  	}
   524  
   525  	pl, err := r.retrievePostingsListWithRLock(postingsOffset)
   526  	if err != nil {
   527  		return nil, err
   528  	}
   529  
   530  	if err := fstCloser.Close(); err != nil {
   531  		return nil, err
   532  	}
   533  
   534  	return pl, nil
   535  }
   536  
   537  func (r *fsSegment) matchRegexpNotClosedMaybeFinalizedWithRLock(
   538  	field []byte,
   539  	compiled index.CompiledRegex,
   540  ) (postings.List, error) {
   541  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   542  	// calling match field after this segment is finalized.
   543  	if r.finalized {
   544  		return nil, errReaderFinalized
   545  	}
   546  
   547  	re := compiled.FST
   548  	if re == nil {
   549  		return nil, errReaderNilRegexp
   550  	}
   551  
   552  	termsFST, exists, err := r.retrieveTermsFSTWithRLock(field)
   553  	if err != nil {
   554  		return nil, err
   555  	}
   556  
   557  	if !exists {
   558  		// i.e. we don't know anything about the field, so can early return an empty postings list
   559  		return r.opts.PostingsListPool().Get(), nil
   560  	}
   561  
   562  	var (
   563  		fstCloser     = x.NewSafeCloser(termsFST)
   564  		iter, iterErr = termsFST.Search(re, compiled.PrefixBegin, compiled.PrefixEnd)
   565  		iterCloser    = x.NewSafeCloser(iter)
   566  		// NB(prateek): way quicker to union the PLs together at the end, rathen than one at a time.
   567  		pls []postings.List // TODO: pool this slice allocation
   568  	)
   569  	defer func() {
   570  		iterCloser.Close()
   571  		fstCloser.Close()
   572  	}()
   573  
   574  	for {
   575  		if iterErr == vellum.ErrIteratorDone {
   576  			break
   577  		}
   578  
   579  		if iterErr != nil {
   580  			return nil, iterErr
   581  		}
   582  
   583  		_, postingsOffset := iter.Current()
   584  		nextPl, err := r.retrievePostingsListWithRLock(postingsOffset)
   585  		if err != nil {
   586  			return nil, err
   587  		}
   588  		pls = append(pls, nextPl)
   589  		iterErr = iter.Next()
   590  	}
   591  
   592  	pl, err := roaring.Union(pls)
   593  	if err != nil {
   594  		return nil, err
   595  	}
   596  
   597  	if err := iterCloser.Close(); err != nil {
   598  		return nil, err
   599  	}
   600  
   601  	if err := fstCloser.Close(); err != nil {
   602  		return nil, err
   603  	}
   604  
   605  	return pl, nil
   606  }
   607  
   608  func (r *fsSegment) matchAllNotClosedMaybeFinalizedWithRLock() (postings.MutableList, error) {
   609  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   610  	// calling match field after this segment is finalized.
   611  	if r.finalized {
   612  		return nil, errReaderFinalized
   613  	}
   614  
   615  	pl := r.opts.PostingsListPool().Get()
   616  	err := pl.AddRange(0, postings.ID(r.numDocs))
   617  	if err != nil {
   618  		return nil, err
   619  	}
   620  
   621  	return pl, nil
   622  }
   623  
   624  func (r *fsSegment) metadataNotClosedMaybeFinalizedWithRLock(id postings.ID) (doc.Metadata, error) {
   625  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   626  	// calling match field after this segment is finalized.
   627  	if r.finalized {
   628  		return doc.Metadata{}, errReaderFinalized
   629  	}
   630  
   631  	// If using docs slice reader, return from the in memory slice reader
   632  	if r.docsThirdPartyReader != nil {
   633  		return r.docsThirdPartyReader.Read(id)
   634  	}
   635  
   636  	offset, err := r.docsIndexReader.Read(id)
   637  	if err != nil {
   638  		return doc.Metadata{}, err
   639  	}
   640  
   641  	return r.docsDataReader.Read(offset)
   642  }
   643  
   644  func (r *fsSegment) metadataIteratorNotClosedMaybeFinalizedWithRLock(
   645  	retriever index.MetadataRetriever,
   646  	pl postings.List,
   647  ) (doc.MetadataIterator, error) {
   648  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   649  	// calling match field after this segment is finalized.
   650  	if r.finalized {
   651  		return nil, errReaderFinalized
   652  	}
   653  
   654  	return index.NewIDDocIterator(retriever, pl.Iterator()), nil
   655  }
   656  
   657  func (r *fsSegment) docNotClosedMaybeFinalizedWithRLock(id postings.ID) (doc.Document, error) {
   658  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   659  	// calling match field after this segment is finalized.
   660  	if r.finalized {
   661  		return doc.Document{}, errReaderFinalized
   662  	}
   663  
   664  	// If using docs slice reader, return from the in memory slice reader
   665  	if r.docsThirdPartyReader != nil {
   666  		m, err := r.docsThirdPartyReader.Read(id)
   667  		if err != nil {
   668  			return doc.Document{}, err
   669  		}
   670  
   671  		return doc.NewDocumentFromMetadata(m), nil
   672  	}
   673  
   674  	offset, err := r.docsIndexReader.Read(id)
   675  	if err != nil {
   676  		return doc.Document{}, err
   677  	}
   678  
   679  	e, err := r.docsEncodedDataReader.Read(offset)
   680  	if err != nil {
   681  		return doc.Document{}, err
   682  	}
   683  
   684  	return doc.NewDocumentFromEncoded(e), nil
   685  }
   686  
   687  func (r *fsSegment) docsNotClosedMaybeFinalizedWithRLock(
   688  	retriever index.DocRetriever,
   689  	pl postings.List,
   690  ) (doc.Iterator, error) {
   691  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   692  	// calling match field after this segment is finalized.
   693  	if r.finalized {
   694  		return nil, errReaderFinalized
   695  	}
   696  
   697  	return index.NewIterator(retriever, pl.Iterator()), nil
   698  }
   699  
   700  func (r *fsSegment) allDocsNotClosedMaybeFinalizedWithRLock(
   701  	retriever index.MetadataRetriever,
   702  ) (index.IDDocIterator, error) {
   703  	// NB(r): Not closed, but could be finalized (i.e. closed segment reader)
   704  	// calling match field after this segment is finalized.
   705  	if r.finalized {
   706  		return nil, errReaderFinalized
   707  	}
   708  
   709  	pi := postings.NewRangeIterator(0, postings.ID(r.numDocs))
   710  	return index.NewIDDocIterator(retriever, pi), nil
   711  }
   712  
   713  func (r *fsSegment) retrievePostingsListWithRLock(postingsOffset uint64) (postings.List, error) {
   714  	postingsBytes, err := r.retrieveBytesWithRLock(r.data.PostingsData.Bytes, postingsOffset)
   715  	if err != nil {
   716  		return nil, fmt.Errorf("unable to retrieve postings data: %v", err)
   717  	}
   718  
   719  	return pilosa.Unmarshal(postingsBytes)
   720  }
   721  
   722  func (r *fsSegment) retrieveTermsFSTWithRLock(field []byte) (*vellum.FST, bool, error) {
   723  	termsFSTOffset, exists, err := r.fieldsFST.Get(field)
   724  	if err != nil {
   725  		return nil, false, err
   726  	}
   727  
   728  	if !exists {
   729  		return nil, false, nil
   730  	}
   731  
   732  	termsFSTBytes, err := r.retrieveBytesWithRLock(r.data.FSTTermsData.Bytes, termsFSTOffset)
   733  	if err != nil {
   734  		return nil, false, fmt.Errorf("error while decoding terms fst: %v", err)
   735  	}
   736  
   737  	termsFST, err := vellum.Load(termsFSTBytes)
   738  	if err != nil {
   739  		return nil, false, fmt.Errorf("error while loading terms fst: %v", err)
   740  	}
   741  
   742  	return termsFST, true, nil
   743  }
   744  
   745  // retrieveTermsBytesWithRLock assumes the base []byte slice is a collection of
   746  // (protobuf payload, proto payload size, fst payload, fst payload size, magicNumber) tuples;
   747  // where all sizes/magicNumber are strictly uint64 (i.e. 8 bytes). It assumes the 8 bytes
   748  // preceding the offset are the magicNumber, the 8 bytes before that are the fst payload size,
   749  // and the `size` bytes before that are the payload, 8 bytes preceeding that are
   750  // `proto payload size`, and the `proto payload size` bytes before that are the proto payload.
   751  // It retrieves the payload while doing bounds checks to ensure no segfaults.
   752  func (r *fsSegment) retrieveTermsBytesWithRLock(base []byte, offset uint64) (proto []byte, fst []byte, err error) {
   753  	const sizeofUint64 = 8
   754  	var (
   755  		magicNumberEnd   = int64(offset) // to prevent underflows
   756  		magicNumberStart = magicNumberEnd - sizeofUint64
   757  	)
   758  	if magicNumberEnd > int64(len(base)) || magicNumberStart < 0 {
   759  		return nil, nil, fmt.Errorf("base bytes too small, length: %d, base-offset: %d", len(base), magicNumberEnd)
   760  	}
   761  	magicNumberBytes := base[magicNumberStart:magicNumberEnd]
   762  	d := encoding.NewDecoder(magicNumberBytes)
   763  	n, err := d.Uint64()
   764  	if err != nil {
   765  		return nil, nil, fmt.Errorf("error while decoding magicNumber: %v", err)
   766  	}
   767  	if n != uint64(magicNumber) {
   768  		return nil, nil, fmt.Errorf("mismatch while decoding magicNumber: %d", n)
   769  	}
   770  
   771  	var (
   772  		sizeEnd   = magicNumberStart
   773  		sizeStart = sizeEnd - sizeofUint64
   774  	)
   775  	if sizeStart < 0 {
   776  		return nil, nil, fmt.Errorf("base bytes too small, length: %d, size-offset: %d", len(base), sizeStart)
   777  	}
   778  	sizeBytes := base[sizeStart:sizeEnd]
   779  	d.Reset(sizeBytes)
   780  	size, err := d.Uint64()
   781  	if err != nil {
   782  		return nil, nil, fmt.Errorf("error while decoding size: %v", err)
   783  	}
   784  
   785  	var (
   786  		payloadEnd   = sizeStart
   787  		payloadStart = payloadEnd - int64(size)
   788  	)
   789  	if payloadStart < 0 {
   790  		return nil, nil, fmt.Errorf("base bytes too small, length: %d, payload-start: %d, payload-size: %d",
   791  			len(base), payloadStart, size)
   792  	}
   793  
   794  	var (
   795  		fstBytes       = base[payloadStart:payloadEnd]
   796  		protoSizeEnd   = payloadStart
   797  		protoSizeStart = protoSizeEnd - sizeofUint64
   798  	)
   799  	if protoSizeStart < 0 {
   800  		return nil, nil, fmt.Errorf("base bytes too small, length: %d, proto-size-offset: %d", len(base), protoSizeStart)
   801  	}
   802  
   803  	protoSizeBytes := base[protoSizeStart:protoSizeEnd]
   804  	d.Reset(protoSizeBytes)
   805  	protoSize, err := d.Uint64()
   806  	if err != nil {
   807  		return nil, nil, fmt.Errorf("error while decoding size: proto %v", err)
   808  	}
   809  
   810  	var (
   811  		protoEnd   = protoSizeStart
   812  		protoStart = protoEnd - int64(protoSize)
   813  	)
   814  	if protoStart < 0 {
   815  		return nil, nil, fmt.Errorf("base bytes too small, length: %d, proto-start: %d", len(base), protoStart)
   816  	}
   817  	protoBytes := base[protoStart:protoEnd]
   818  
   819  	return protoBytes, fstBytes, nil
   820  }
   821  
   822  // retrieveBytesWithRLock assumes the base []byte slice is a collection of (payload, size, magicNumber) triples,
   823  // where size/magicNumber are strictly uint64 (i.e. 8 bytes). It assumes the 8 bytes preceding the offset
   824  // are the magicNumber, the 8 bytes before that are the size, and the `size` bytes before that are the
   825  // payload. It retrieves the payload while doing bounds checks to ensure no segfaults.
   826  func (r *fsSegment) retrieveBytesWithRLock(base []byte, offset uint64) ([]byte, error) {
   827  	const sizeofUint64 = 8
   828  	var (
   829  		magicNumberEnd   = int64(offset) // to prevent underflows
   830  		magicNumberStart = offset - sizeofUint64
   831  	)
   832  	if magicNumberEnd > int64(len(base)) || magicNumberStart < 0 {
   833  		return nil, fmt.Errorf("base bytes too small, length: %d, base-offset: %d", len(base), magicNumberEnd)
   834  	}
   835  	magicNumberBytes := base[magicNumberStart:magicNumberEnd]
   836  	d := encoding.NewDecoder(magicNumberBytes)
   837  	n, err := d.Uint64()
   838  	if err != nil {
   839  		return nil, fmt.Errorf("error while decoding magicNumber: %v", err)
   840  	}
   841  	if n != uint64(magicNumber) {
   842  		return nil, fmt.Errorf("mismatch while decoding magicNumber: %d", n)
   843  	}
   844  
   845  	var (
   846  		sizeEnd   = magicNumberStart
   847  		sizeStart = sizeEnd - sizeofUint64
   848  	)
   849  	if sizeStart < 0 {
   850  		return nil, fmt.Errorf("base bytes too small, length: %d, size-offset: %d", len(base), sizeStart)
   851  	}
   852  	sizeBytes := base[sizeStart:sizeEnd]
   853  	d.Reset(sizeBytes)
   854  	size, err := d.Uint64()
   855  	if err != nil {
   856  		return nil, fmt.Errorf("error while decoding size: %v", err)
   857  	}
   858  
   859  	var (
   860  		payloadEnd   = sizeStart
   861  		payloadStart = payloadEnd - size
   862  	)
   863  	if payloadStart < 0 {
   864  		return nil, fmt.Errorf("base bytes too small, length: %d, payload-start: %d, payload-size: %d",
   865  			len(base), payloadStart, size)
   866  	}
   867  
   868  	return base[payloadStart:payloadEnd], nil
   869  }
   870  
   871  var _ sgmt.Reader = (*fsSegmentReader)(nil)
   872  
   873  // fsSegmentReader is not thread safe for use and relies on the underlying
   874  // segment for synchronization.
   875  type fsSegmentReader struct {
   876  	closed        bool
   877  	ctx           context.Context
   878  	fsSegment     *fsSegment
   879  	termsIterable *termsIterable
   880  }
   881  
   882  func newReader(
   883  	fsSegment *fsSegment,
   884  	opts Options,
   885  ) *fsSegmentReader {
   886  	return &fsSegmentReader{
   887  		ctx:       opts.ContextPool().Get(),
   888  		fsSegment: fsSegment,
   889  	}
   890  }
   891  
   892  func (sr *fsSegmentReader) Fields() (sgmt.FieldsIterator, error) {
   893  	if sr.closed {
   894  		return nil, errReaderClosed
   895  	}
   896  
   897  	sr.fsSegment.RLock()
   898  	defer sr.fsSegment.RUnlock()
   899  	if sr.fsSegment.finalized {
   900  		return nil, errReaderFinalized
   901  	}
   902  
   903  	iter := newFSTTermsIter()
   904  	iter.reset(fstTermsIterOpts{
   905  		seg:         sr.fsSegment,
   906  		fst:         sr.fsSegment.fieldsFST,
   907  		finalizeFST: false,
   908  	})
   909  	return iter, nil
   910  }
   911  
   912  func (sr *fsSegmentReader) ContainsField(field []byte) (bool, error) {
   913  	if sr.closed {
   914  		return false, errReaderClosed
   915  	}
   916  
   917  	sr.fsSegment.RLock()
   918  	defer sr.fsSegment.RUnlock()
   919  	if sr.fsSegment.finalized {
   920  		return false, errReaderFinalized
   921  	}
   922  
   923  	return sr.fsSegment.fieldsFST.Contains(field)
   924  }
   925  
   926  func (sr *fsSegmentReader) FieldsPostingsList() (sgmt.FieldsPostingsListIterator, error) {
   927  	if sr.closed {
   928  		return nil, errReaderClosed
   929  	}
   930  	fieldsIterable := newTermsIterable(sr.fsSegment)
   931  	sr.fsSegment.RLock()
   932  	iter, err := fieldsIterable.fieldsNotClosedMaybeFinalizedWithRLock()
   933  	sr.fsSegment.RUnlock()
   934  	return iter, err
   935  }
   936  
   937  func (sr *fsSegmentReader) Terms(field []byte) (sgmt.TermsIterator, error) {
   938  	if sr.closed {
   939  		return nil, errReaderClosed
   940  	}
   941  	if sr.termsIterable == nil {
   942  		sr.termsIterable = newTermsIterable(sr.fsSegment)
   943  	}
   944  	sr.fsSegment.RLock()
   945  	iter, err := sr.termsIterable.termsNotClosedMaybeFinalizedWithRLock(field)
   946  	sr.fsSegment.RUnlock()
   947  	return iter, err
   948  }
   949  
   950  func (sr *fsSegmentReader) MatchField(field []byte) (postings.List, error) {
   951  	if sr.closed {
   952  		return nil, errReaderClosed
   953  	}
   954  	// NB(r): We are allowed to call match field after Close called on
   955  	// the segment but not after it is finalized.
   956  	sr.fsSegment.RLock()
   957  	pl, err := sr.fsSegment.matchFieldNotClosedMaybeFinalizedWithRLock(field)
   958  	sr.fsSegment.RUnlock()
   959  	return pl, err
   960  }
   961  
   962  func (sr *fsSegmentReader) MatchTerm(field []byte, term []byte) (postings.List, error) {
   963  	if sr.closed {
   964  		return nil, errReaderClosed
   965  	}
   966  	// NB(r): We are allowed to call match field after Close called on
   967  	// the segment but not after it is finalized.
   968  	sr.fsSegment.RLock()
   969  	pl, err := sr.fsSegment.matchTermNotClosedMaybeFinalizedWithRLock(field, term)
   970  	sr.fsSegment.RUnlock()
   971  	return pl, err
   972  }
   973  
   974  func (sr *fsSegmentReader) MatchRegexp(
   975  	field []byte,
   976  	compiled index.CompiledRegex,
   977  ) (postings.List, error) {
   978  	if sr.closed {
   979  		return nil, errReaderClosed
   980  	}
   981  	// NB(r): We are allowed to call match field after Close called on
   982  	// the segment but not after it is finalized.
   983  	sr.fsSegment.RLock()
   984  	pl, err := sr.fsSegment.matchRegexpNotClosedMaybeFinalizedWithRLock(field, compiled)
   985  	sr.fsSegment.RUnlock()
   986  	return pl, err
   987  }
   988  
   989  func (sr *fsSegmentReader) MatchAll() (postings.List, error) {
   990  	if sr.closed {
   991  		return nil, errReaderClosed
   992  	}
   993  	// NB(r): We are allowed to call match field after Close called on
   994  	// the segment but not after it is finalized.
   995  	sr.fsSegment.RLock()
   996  	pl, err := sr.fsSegment.matchAllNotClosedMaybeFinalizedWithRLock()
   997  	sr.fsSegment.RUnlock()
   998  	return pl, err
   999  }
  1000  
  1001  func (sr *fsSegmentReader) Metadata(id postings.ID) (doc.Metadata, error) {
  1002  	if sr.closed {
  1003  		return doc.Metadata{}, errReaderClosed
  1004  	}
  1005  	// NB(r): We are allowed to call match field after Close called on
  1006  	// the segment but not after it is finalized.
  1007  	sr.fsSegment.RLock()
  1008  	pl, err := sr.fsSegment.metadataNotClosedMaybeFinalizedWithRLock(id)
  1009  	sr.fsSegment.RUnlock()
  1010  	return pl, err
  1011  }
  1012  
  1013  func (sr *fsSegmentReader) MetadataIterator(pl postings.List) (doc.MetadataIterator, error) {
  1014  	if sr.closed {
  1015  		return nil, errReaderClosed
  1016  	}
  1017  	// NB(r): We are allowed to call match field after Close called on
  1018  	// the segment but not after it is finalized.
  1019  	// Also make sure the doc retriever is the reader not the segment so that
  1020  	// is closed check is not performed and only the is finalized check.
  1021  	sr.fsSegment.RLock()
  1022  	iter, err := sr.fsSegment.metadataIteratorNotClosedMaybeFinalizedWithRLock(sr, pl)
  1023  	sr.fsSegment.RUnlock()
  1024  	return iter, err
  1025  }
  1026  
  1027  func (sr *fsSegmentReader) Doc(id postings.ID) (doc.Document, error) {
  1028  	if sr.closed {
  1029  		return doc.Document{}, errReaderClosed
  1030  	}
  1031  	// NB(r): We are allowed to call match field after Close called on
  1032  	// the segment but not after it is finalized.
  1033  	sr.fsSegment.RLock()
  1034  	pl, err := sr.fsSegment.docNotClosedMaybeFinalizedWithRLock(id)
  1035  	sr.fsSegment.RUnlock()
  1036  	return pl, err
  1037  }
  1038  
  1039  func (sr *fsSegmentReader) Docs(pl postings.List) (doc.Iterator, error) {
  1040  	if sr.closed {
  1041  		return nil, errReaderClosed
  1042  	}
  1043  	// NB(r): We are allowed to call match field after Close called on
  1044  	// the segment but not after it is finalized.
  1045  	// Also make sure the doc retriever is the reader not the segment so that
  1046  	// is closed check is not performed and only the is finalized check.
  1047  	sr.fsSegment.RLock()
  1048  	iter, err := sr.fsSegment.docsNotClosedMaybeFinalizedWithRLock(sr, pl)
  1049  	sr.fsSegment.RUnlock()
  1050  	return iter, err
  1051  }
  1052  
  1053  func (sr *fsSegmentReader) AllDocs() (index.IDDocIterator, error) {
  1054  	if sr.closed {
  1055  		return nil, errReaderClosed
  1056  	}
  1057  	// NB(r): We are allowed to call match field after Close called on
  1058  	// the segment but not after it is finalized.
  1059  	// Also make sure the doc retriever is the reader not the segment so that
  1060  	// is closed check is not performed and only the is finalized check.
  1061  	sr.fsSegment.RLock()
  1062  	iter, err := sr.fsSegment.allDocsNotClosedMaybeFinalizedWithRLock(sr)
  1063  	sr.fsSegment.RUnlock()
  1064  	return iter, err
  1065  }
  1066  
  1067  func (sr *fsSegmentReader) Close() error {
  1068  	if sr.closed {
  1069  		return errReaderClosed
  1070  	}
  1071  	sr.closed = true
  1072  	// Close the context so that segment doesn't need to track this any longer.
  1073  	sr.ctx.Close()
  1074  	return nil
  1075  }