github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/m3ninx/index/segment/mem/reader.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 mem
    22  
    23  import (
    24  	"errors"
    25  	"sync"
    26  
    27  	"github.com/m3db/m3/src/m3ninx/doc"
    28  	"github.com/m3db/m3/src/m3ninx/index"
    29  	sgmt "github.com/m3db/m3/src/m3ninx/index/segment"
    30  	"github.com/m3db/m3/src/m3ninx/postings"
    31  )
    32  
    33  var (
    34  	errSegmentReaderClosed = errors.New("segment reader is closed")
    35  	errReaderNilRegex      = errors.New("nil regex received")
    36  )
    37  
    38  type reader struct {
    39  	sync.RWMutex
    40  
    41  	segment ReadableSegment
    42  	limits  readerDocRange
    43  	plPool  postings.Pool
    44  
    45  	closed bool
    46  }
    47  
    48  type readerDocRange struct {
    49  	startInclusive postings.ID
    50  	endExclusive   postings.ID
    51  }
    52  
    53  func newReader(s ReadableSegment, l readerDocRange, p postings.Pool) sgmt.Reader {
    54  	return &reader{
    55  		segment: s,
    56  		limits:  l,
    57  		plPool:  p,
    58  	}
    59  }
    60  
    61  func (r *reader) Fields() (sgmt.FieldsIterator, error) {
    62  	return r.segment.Fields()
    63  }
    64  
    65  func (r *reader) ContainsField(field []byte) (bool, error) {
    66  	return r.segment.ContainsField(field)
    67  }
    68  
    69  func (r *reader) Terms(field []byte) (sgmt.TermsIterator, error) {
    70  	return r.segment.Terms(field)
    71  }
    72  
    73  func (r *reader) FieldsPostingsList() (sgmt.FieldsPostingsListIterator, error) {
    74  	return r.segment.FieldsPostingsList()
    75  }
    76  
    77  func (r *reader) MatchField(field []byte) (postings.List, error) {
    78  	// falling back to regexp .* as this segment implementation is only used in tests.
    79  	return r.MatchRegexp(field, index.DotStarCompiledRegex())
    80  }
    81  
    82  func (r *reader) MatchTerm(field, term []byte) (postings.List, error) {
    83  	r.RLock()
    84  	defer r.RUnlock()
    85  	if r.closed {
    86  		return nil, errSegmentReaderClosed
    87  	}
    88  
    89  	// A reader can return IDs in the posting list which are greater than its limit.
    90  	// The reader only guarantees that when fetching the documents associated with a
    91  	// postings list through a call to Docs, IDs greater than or equal to the limit
    92  	// will be filtered out.
    93  	pl, err := r.segment.matchTerm(field, term)
    94  	return pl, err
    95  }
    96  
    97  func (r *reader) MatchRegexp(field []byte, compiled index.CompiledRegex) (postings.List, error) {
    98  	r.RLock()
    99  	defer r.RUnlock()
   100  	if r.closed {
   101  		return nil, errSegmentReaderClosed
   102  	}
   103  
   104  	// A reader can return IDs in the posting list which are greater than its maximum
   105  	// permitted ID. The reader only guarantees that when fetching the documents associated
   106  	// with a postings list through a call to Docs will IDs greater than the maximum be
   107  	// filtered out.
   108  	compileRE := compiled.Simple
   109  	if compileRE == nil {
   110  		return nil, errReaderNilRegex
   111  	}
   112  
   113  	return r.segment.matchRegexp(field, compileRE)
   114  }
   115  
   116  func (r *reader) MatchAll() (postings.List, error) {
   117  	r.RLock()
   118  	defer r.RUnlock()
   119  	if r.closed {
   120  		return nil, errSegmentReaderClosed
   121  	}
   122  
   123  	pl := r.plPool.Get()
   124  	err := pl.AddRange(r.limits.startInclusive, r.limits.endExclusive)
   125  	if err != nil {
   126  		return nil, err
   127  	}
   128  	return pl, nil
   129  }
   130  
   131  func (r *reader) Metadata(id postings.ID) (doc.Metadata, error) {
   132  	r.RLock()
   133  	defer r.RUnlock()
   134  
   135  	return r.getMetadataWithRLock(id)
   136  }
   137  
   138  func (r *reader) MetadataIterator(pl postings.List) (doc.MetadataIterator, error) {
   139  	r.RLock()
   140  	defer r.RUnlock()
   141  	if r.closed {
   142  		return nil, errSegmentReaderClosed
   143  	}
   144  	boundedIter := newBoundedPostingsIterator(pl.Iterator(), r.limits)
   145  	return r.getMetadataIterWithLock(boundedIter), nil
   146  }
   147  
   148  func (r *reader) Doc(id postings.ID) (doc.Document, error) {
   149  	r.RLock()
   150  	defer r.RUnlock()
   151  
   152  	m, err := r.getMetadataWithRLock(id)
   153  	if err != nil {
   154  		return doc.Document{}, err
   155  	}
   156  
   157  	return doc.NewDocumentFromMetadata(m), nil
   158  }
   159  
   160  func (r *reader) Docs(pl postings.List) (doc.Iterator, error) {
   161  	r.RLock()
   162  	defer r.RUnlock()
   163  
   164  	if r.closed {
   165  		return nil, errSegmentReaderClosed
   166  	}
   167  
   168  	boundedIter := newBoundedPostingsIterator(pl.Iterator(), r.limits)
   169  	return index.NewIterator(r, boundedIter), nil
   170  }
   171  
   172  func (r *reader) AllDocs() (index.IDDocIterator, error) {
   173  	r.RLock()
   174  	defer r.RUnlock()
   175  	if r.closed {
   176  		return nil, errSegmentReaderClosed
   177  	}
   178  
   179  	pi := postings.NewRangeIterator(r.limits.startInclusive, r.limits.endExclusive)
   180  	return r.getMetadataIterWithLock(pi), nil
   181  }
   182  
   183  func (r *reader) getMetadataIterWithLock(iter postings.Iterator) index.IDDocIterator {
   184  	return index.NewIDDocIterator(r, iter)
   185  }
   186  
   187  func (r *reader) getMetadataWithRLock(id postings.ID) (doc.Metadata, error) {
   188  	if r.closed {
   189  		return doc.Metadata{}, errSegmentReaderClosed
   190  	}
   191  
   192  	if id < r.limits.startInclusive || id >= r.limits.endExclusive {
   193  		return doc.Metadata{}, index.ErrDocNotFound
   194  	}
   195  
   196  	return r.segment.getDoc(id)
   197  }
   198  
   199  func (r *reader) Close() error {
   200  	r.Lock()
   201  	if r.closed {
   202  		r.Unlock()
   203  		return errSegmentReaderClosed
   204  	}
   205  	r.closed = true
   206  	r.Unlock()
   207  	return nil
   208  }