0
0

Merge pull request #400 from steveyen/WIP-search-optimizations

search optimizations
This commit is contained in:
Marty Schoch 2016-07-29 17:30:35 -04:00 committed by GitHub
commit b158fb147d
36 changed files with 259 additions and 197 deletions

View File

@ -111,12 +111,13 @@ type TermFieldDoc struct {
// lexicographic order over their identifiers. // lexicographic order over their identifiers.
type TermFieldReader interface { type TermFieldReader interface {
// Next returns the next document containing the term in this field, or nil // Next returns the next document containing the term in this field, or nil
// when it reaches the end of the enumeration. // when it reaches the end of the enumeration. The preAlloced TermFieldDoc
Next() (*TermFieldDoc, error) // is optional, and when non-nil, will be used instead of allocating memory.
Next(preAlloced *TermFieldDoc) (*TermFieldDoc, error)
// Advance resets the enumeration at specified document or its immediate // Advance resets the enumeration at specified document or its immediate
// follower. // follower.
Advance(ID string) (*TermFieldDoc, error) Advance(ID string, preAlloced *TermFieldDoc) (*TermFieldDoc, error)
// Count returns the number of documents contains the term in this field. // Count returns the number of documents contains the term in this field.
Count() uint64 Count() uint64
@ -199,3 +200,8 @@ func (b *Batch) Reset() {
b.IndexOps = make(map[string]*document.Document) b.IndexOps = make(map[string]*document.Document)
b.InternalOps = make(map[string][]byte) b.InternalOps = make(map[string][]byte)
} }
func (tfd *TermFieldDoc) Reset() *TermFieldDoc {
*tfd = TermFieldDoc{}
return tfd
}

View File

@ -18,15 +18,14 @@ import (
) )
type Iterator struct { type Iterator struct {
store *Store store *Store
ss moss.Snapshot ss moss.Snapshot
iter moss.Iterator iter moss.Iterator
prefix []byte start []byte
start []byte end []byte
end []byte done bool
done bool k []byte
k []byte v []byte
v []byte
} }
func (x *Iterator) Seek(seekToKey []byte) { func (x *Iterator) Seek(seekToKey []byte) {
@ -60,12 +59,11 @@ func (x *Iterator) Next() {
return return
} }
x.done = true
x.k = nil
x.v = nil
err := x.iter.Next() err := x.iter.Next()
if err != nil { if err != nil {
x.done = true
x.k = nil
x.v = nil
return return
} }
@ -106,7 +104,6 @@ func (x *Iterator) Close() error {
x.iter = nil x.iter = nil
} }
x.prefix = nil
x.done = true x.done = true
x.k = nil x.k = nil
x.v = nil x.v = nil
@ -115,16 +112,11 @@ func (x *Iterator) Close() error {
} }
func (x *Iterator) checkDone() { func (x *Iterator) checkDone() {
x.done = true
x.k = nil
x.v = nil
k, v, err := x.iter.Current() k, v, err := x.iter.Current()
if err != nil { if err != nil {
return x.done = true
} x.k = nil
x.v = nil
if x.prefix != nil && !bytes.HasPrefix(k, x.prefix) {
return return
} }

View File

@ -12,6 +12,8 @@
package moss package moss
import ( import (
"math/big"
"github.com/couchbase/moss" "github.com/couchbase/moss"
"github.com/blevesearch/bleve/index/store" "github.com/blevesearch/bleve/index/store"
@ -37,19 +39,22 @@ func (r *Reader) MultiGet(keys [][]byte) ([][]byte, error) {
return store.MultiGet(r, keys) return store.MultiGet(r, keys)
} }
var bigOne = big.NewInt(1)
func (r *Reader) PrefixIterator(k []byte) store.KVIterator { func (r *Reader) PrefixIterator(k []byte) store.KVIterator {
iter, err := r.ss.StartIterator(k, nil, moss.IteratorOptions{}) kEnd := big.NewInt(0).Add(big.NewInt(0).SetBytes(k), bigOne).Bytes()
iter, err := r.ss.StartIterator(k, kEnd, moss.IteratorOptions{})
if err != nil { if err != nil {
return nil return nil
} }
rv := &Iterator{ rv := &Iterator{
store: r.store, store: r.store,
ss: r.ss, ss: r.ss,
iter: iter, iter: iter,
prefix: k, start: k,
start: k, end: kEnd,
end: nil,
} }
rv.checkDone() rv.checkDone()
@ -64,12 +69,11 @@ func (r *Reader) RangeIterator(start, end []byte) store.KVIterator {
} }
rv := &Iterator{ rv := &Iterator{
store: r.store, store: r.store,
ss: r.ss, ss: r.ss,
iter: iter, iter: iter,
prefix: nil, start: start,
start: start, end: end,
end: end,
} }
rv.checkDone() rv.checkDone()

View File

@ -17,10 +17,11 @@ import (
) )
type UpsideDownCouchTermFieldReader struct { type UpsideDownCouchTermFieldReader struct {
count uint64
indexReader *IndexReader indexReader *IndexReader
iterator store.KVIterator iterator store.KVIterator
count uint64
term []byte term []byte
tfrNext *TermFrequencyRow
field uint16 field uint16
} }
@ -33,9 +34,10 @@ func newUpsideDownCouchTermFieldReader(indexReader *IndexReader, term []byte, fi
if val == nil { if val == nil {
atomic.AddUint64(&indexReader.index.stats.termSearchersStarted, uint64(1)) atomic.AddUint64(&indexReader.index.stats.termSearchersStarted, uint64(1))
return &UpsideDownCouchTermFieldReader{ return &UpsideDownCouchTermFieldReader{
count: 0, count: 0,
term: term, term: term,
field: field, tfrNext: &TermFrequencyRow{},
field: field,
}, nil }, nil
} }
@ -53,6 +55,7 @@ func newUpsideDownCouchTermFieldReader(indexReader *IndexReader, term []byte, fi
iterator: it, iterator: it,
count: dictionaryRow.count, count: dictionaryRow.count,
term: term, term: term,
tfrNext: &TermFrequencyRow{},
field: field, field: field,
}, nil }, nil
} }
@ -61,28 +64,37 @@ func (r *UpsideDownCouchTermFieldReader) Count() uint64 {
return r.count return r.count
} }
func (r *UpsideDownCouchTermFieldReader) Next() (*index.TermFieldDoc, error) { func (r *UpsideDownCouchTermFieldReader) Next(preAlloced *index.TermFieldDoc) (*index.TermFieldDoc, error) {
if r.iterator != nil { if r.iterator != nil {
key, val, valid := r.iterator.Current() key, val, valid := r.iterator.Current()
if valid { if valid {
tfr, err := NewTermFrequencyRowKV(key, val) tfr := r.tfrNext
err := tfr.parseKDoc(key)
if err != nil { if err != nil {
return nil, err return nil, err
} }
rv := index.TermFieldDoc{ err = tfr.parseV(val)
ID: string(tfr.doc), if err != nil {
Freq: tfr.freq, return nil, err
Norm: float64(tfr.norm), }
Vectors: r.indexReader.index.termFieldVectorsFromTermVectors(tfr.vectors), rv := preAlloced
if rv == nil {
rv = &index.TermFieldDoc{}
}
rv.ID = string(tfr.doc)
rv.Freq = tfr.freq
rv.Norm = float64(tfr.norm)
if tfr.vectors != nil {
rv.Vectors = r.indexReader.index.termFieldVectorsFromTermVectors(tfr.vectors)
} }
r.iterator.Next() r.iterator.Next()
return &rv, nil return rv, nil
} }
} }
return nil, nil return nil, nil
} }
func (r *UpsideDownCouchTermFieldReader) Advance(docID string) (*index.TermFieldDoc, error) { func (r *UpsideDownCouchTermFieldReader) Advance(docID string, preAlloced *index.TermFieldDoc) (*index.TermFieldDoc, error) {
if r.iterator != nil { if r.iterator != nil {
tfr := NewTermFrequencyRow(r.term, r.field, []byte(docID), 0, 0) tfr := NewTermFrequencyRow(r.term, r.field, []byte(docID), 0, 0)
r.iterator.Seek(tfr.Key()) r.iterator.Seek(tfr.Key())
@ -92,14 +104,18 @@ func (r *UpsideDownCouchTermFieldReader) Advance(docID string) (*index.TermField
if err != nil { if err != nil {
return nil, err return nil, err
} }
rv := index.TermFieldDoc{ rv := preAlloced
ID: string(tfr.doc), if rv == nil {
Freq: tfr.freq, rv = &index.TermFieldDoc{}
Norm: float64(tfr.norm), }
Vectors: r.indexReader.index.termFieldVectorsFromTermVectors(tfr.vectors), rv.ID = string(tfr.doc)
rv.Freq = tfr.freq
rv.Norm = float64(tfr.norm)
if tfr.vectors != nil {
rv.Vectors = r.indexReader.index.termFieldVectorsFromTermVectors(tfr.vectors)
} }
r.iterator.Next() r.iterator.Next()
return &rv, nil return rv, nil
} }
} }
return nil, nil return nil, nil

View File

@ -98,9 +98,9 @@ func TestIndexReader(t *testing.T) {
var match *index.TermFieldDoc var match *index.TermFieldDoc
var actualCount uint64 var actualCount uint64
match, err = reader.Next() match, err = reader.Next(nil)
for err == nil && match != nil { for err == nil && match != nil {
match, err = reader.Next() match, err = reader.Next(nil)
if err != nil { if err != nil {
t.Errorf("unexpected error reading next") t.Errorf("unexpected error reading next")
} }
@ -127,7 +127,7 @@ func TestIndexReader(t *testing.T) {
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }
match, err = tfr.Next() match, err = tfr.Next(nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }
@ -145,7 +145,7 @@ func TestIndexReader(t *testing.T) {
t.Errorf("Error accessing term field reader: %v", err) t.Errorf("Error accessing term field reader: %v", err)
} }
match, err = reader.Advance("2") match, err = reader.Advance("2", nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }
@ -155,7 +155,7 @@ func TestIndexReader(t *testing.T) {
if match.ID != "2" { if match.ID != "2" {
t.Errorf("Expected ID '2', got '%s'", match.ID) t.Errorf("Expected ID '2', got '%s'", match.ID)
} }
match, err = reader.Advance("3") match, err = reader.Advance("3", nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }
@ -176,14 +176,14 @@ func TestIndexReader(t *testing.T) {
if count != 0 { if count != 0 {
t.Errorf("expected count 0 for reader of non-existant field") t.Errorf("expected count 0 for reader of non-existant field")
} }
match, err = reader.Next() match, err = reader.Next(nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }
if match != nil { if match != nil {
t.Errorf("expected nil, got %v", match) t.Errorf("expected nil, got %v", match)
} }
match, err = reader.Advance("anywhere") match, err = reader.Advance("anywhere", nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }

View File

@ -350,11 +350,11 @@ func (tv *TermVector) String() string {
type TermFrequencyRow struct { type TermFrequencyRow struct {
term []byte term []byte
field uint16
doc []byte doc []byte
freq uint64 freq uint64
norm float32
vectors []*TermVector vectors []*TermVector
norm float32
field uint16
} }
func (tfr *TermFrequencyRow) Term() []byte { func (tfr *TermFrequencyRow) Term() []byte {
@ -483,36 +483,57 @@ func NewTermFrequencyRowWithTermVectors(term []byte, field uint16, docID []byte,
} }
func NewTermFrequencyRowK(key []byte) (*TermFrequencyRow, error) { func NewTermFrequencyRowK(key []byte) (*TermFrequencyRow, error) {
rv := TermFrequencyRow{} rv := &TermFrequencyRow{}
err := rv.parseK(key)
if err != nil {
return nil, err
}
return rv, nil
}
func (tfr *TermFrequencyRow) parseK(key []byte) error {
keyLen := len(key) keyLen := len(key)
if keyLen < 3 { if keyLen < 3 {
return nil, fmt.Errorf("invalid term frequency key, no valid field") return fmt.Errorf("invalid term frequency key, no valid field")
} }
rv.field = binary.LittleEndian.Uint16(key[1:3]) tfr.field = binary.LittleEndian.Uint16(key[1:3])
termEndPos := bytes.IndexByte(key[3:], ByteSeparator) termEndPos := bytes.IndexByte(key[3:], ByteSeparator)
if termEndPos < 0 { if termEndPos < 0 {
return nil, fmt.Errorf("invalid term frequency key, no byte separator terminating term") return fmt.Errorf("invalid term frequency key, no byte separator terminating term")
} }
rv.term = key[3 : 3+termEndPos] tfr.term = key[3 : 3+termEndPos]
docLen := len(key) - (3 + termEndPos + 1) docLen := keyLen - (3 + termEndPos + 1)
if docLen < 1 { if docLen < 1 {
return nil, fmt.Errorf("invalid term frequency key, empty docid") return fmt.Errorf("invalid term frequency key, empty docid")
} }
rv.doc = key[3+termEndPos+1:] tfr.doc = key[3+termEndPos+1:]
return &rv, nil return nil
}
func (tfr *TermFrequencyRow) parseKDoc(key []byte) error {
termEndPos := bytes.IndexByte(key[3:], ByteSeparator)
if termEndPos < 0 {
return fmt.Errorf("invalid term frequency key, no byte separator terminating term")
}
tfr.doc = key[3+termEndPos+1:]
if len(tfr.doc) <= 0 {
return fmt.Errorf("invalid term frequency key, empty docid")
}
return nil
} }
func (tfr *TermFrequencyRow) parseV(value []byte) error { func (tfr *TermFrequencyRow) parseV(value []byte) error {
currOffset := 0 var bytesRead int
bytesRead := 0 tfr.freq, bytesRead = binary.Uvarint(value)
tfr.freq, bytesRead = binary.Uvarint(value[currOffset:])
if bytesRead <= 0 { if bytesRead <= 0 {
return fmt.Errorf("invalid term frequency value, invalid frequency") return fmt.Errorf("invalid term frequency value, invalid frequency")
} }
currOffset += bytesRead currOffset := bytesRead
var norm uint64 var norm uint64
norm, bytesRead = binary.Uvarint(value[currOffset:]) norm, bytesRead = binary.Uvarint(value[currOffset:])

View File

@ -770,6 +770,10 @@ func (udc *UpsideDownCouch) termVectorsFromTokenFreq(field uint16, tf *analysis.
} }
func (udc *UpsideDownCouch) termFieldVectorsFromTermVectors(in []*TermVector) []*index.TermFieldVector { func (udc *UpsideDownCouch) termFieldVectorsFromTermVectors(in []*TermVector) []*index.TermFieldVector {
if len(in) <= 0 {
return nil
}
rv := make([]*index.TermFieldVector, len(in)) rv := make([]*index.TermFieldVector, len(in))
for i, tv := range in { for i, tv := range in {

View File

@ -1124,12 +1124,12 @@ func TestIndexTermReaderCompositeFields(t *testing.T) {
t.Error(err) t.Error(err)
} }
tfd, err := termFieldReader.Next() tfd, err := termFieldReader.Next(nil)
for tfd != nil && err == nil { for tfd != nil && err == nil {
if tfd.ID != "1" { if tfd.ID != "1" {
t.Errorf("expected to find document id 1") t.Errorf("expected to find document id 1")
} }
tfd, err = termFieldReader.Next() tfd, err = termFieldReader.Next(nil)
} }
if err != nil { if err != nil {
t.Error(err) t.Error(err)

View File

@ -57,30 +57,35 @@ func (tksc *TopScoreCollector) Took() time.Duration {
return tksc.took return tksc.took
} }
var COLLECT_CHECK_DONE_EVERY = uint64(1024)
func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Searcher) error { func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Searcher) error {
startTime := time.Now() startTime := time.Now()
var err error var err error
var pre search.DocumentMatch // A single pre-alloc'ed, reused instance.
var next *search.DocumentMatch var next *search.DocumentMatch
select { select {
case <-ctx.Done(): case <-ctx.Done():
return ctx.Err() return ctx.Err()
default: default:
next, err = searcher.Next() next, err = searcher.Next(&pre)
} }
for err == nil && next != nil { for err == nil && next != nil {
select { if tksc.total%COLLECT_CHECK_DONE_EVERY == 0 {
case <-ctx.Done(): select {
return ctx.Err() case <-ctx.Done():
default: return ctx.Err()
tksc.collectSingle(next) default:
if tksc.facetsBuilder != nil {
err = tksc.facetsBuilder.Update(next)
if err != nil {
break
}
} }
next, err = searcher.Next()
} }
tksc.collectSingle(next)
if tksc.facetsBuilder != nil {
err = tksc.facetsBuilder.Update(next)
if err != nil {
break
}
}
next, err = searcher.Next(pre.Reset())
} }
// compute search duration // compute search duration
tksc.took = time.Since(startTime) tksc.took = time.Since(startTime)
@ -90,19 +95,25 @@ func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Sear
return nil return nil
} }
func (tksc *TopScoreCollector) collectSingle(dm *search.DocumentMatch) { func (tksc *TopScoreCollector) collectSingle(dmIn *search.DocumentMatch) {
// increment total hits // increment total hits
tksc.total++ tksc.total++
// update max score // update max score
if dm.Score > tksc.maxScore { if dmIn.Score > tksc.maxScore {
tksc.maxScore = dm.Score tksc.maxScore = dmIn.Score
} }
if dm.Score <= tksc.minScore { if dmIn.Score <= tksc.minScore {
return return
} }
// Because the dmIn will be the single, pre-allocated, reused
// instance, we need to copy the dmIn into a new, standalone
// instance before inserting into our candidate results list.
dm := &search.DocumentMatch{}
*dm = *dmIn
for e := tksc.results.Front(); e != nil; e = e.Next() { for e := tksc.results.Front(); e != nil; e = e.Next() {
curr := e.Value.(*search.DocumentMatch) curr := e.Value.(*search.DocumentMatch)
if dm.Score <= curr.Score { if dm.Score <= curr.Score {

View File

@ -18,7 +18,7 @@ type stubSearcher struct {
matches search.DocumentMatchCollection matches search.DocumentMatchCollection
} }
func (ss *stubSearcher) Next() (*search.DocumentMatch, error) { func (ss *stubSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if ss.index < len(ss.matches) { if ss.index < len(ss.matches) {
rv := ss.matches[ss.index] rv := ss.matches[ss.index]
ss.index++ ss.index++
@ -27,7 +27,7 @@ func (ss *stubSearcher) Next() (*search.DocumentMatch, error) {
return nil, nil return nil, nil
} }
func (ss *stubSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (ss *stubSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
for ss.index < len(ss.matches) && ss.matches[ss.index].ID < ID { for ss.index < len(ss.matches) && ss.matches[ss.index].ID < ID {
ss.index++ ss.index++

View File

@ -83,7 +83,7 @@ func (s *TermQueryScorer) SetQueryNorm(qnorm float64) {
} }
} }
func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc) *search.DocumentMatch { func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc, preAllocated *search.DocumentMatch) *search.DocumentMatch {
var scoreExplanation *search.Explanation var scoreExplanation *search.Explanation
// need to compute score // need to compute score
@ -128,10 +128,12 @@ func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc) *search.DocumentM
} }
} }
rv := search.DocumentMatch{ rv := preAllocated
ID: termMatch.ID, if rv == nil {
Score: score, rv = &search.DocumentMatch{}
} }
rv.ID = termMatch.ID
rv.Score = score
if s.explain { if s.explain {
rv.Expl = scoreExplanation rv.Expl = scoreExplanation
} }
@ -172,5 +174,5 @@ func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc) *search.DocumentM
} }
return &rv return rv
} }

View File

@ -144,7 +144,7 @@ func TestTermScorer(t *testing.T) {
} }
for _, test := range tests { for _, test := range tests {
actual := scorer.Score(test.termMatch) actual := scorer.Score(test.termMatch, nil)
if !reflect.DeepEqual(actual, test.result) { if !reflect.DeepEqual(actual, test.result) {
t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch) t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch)
@ -231,7 +231,7 @@ func TestTermScorerWithQueryNorm(t *testing.T) {
} }
for _, test := range tests { for _, test := range tests {
actual := scorer.Score(test.termMatch) actual := scorer.Score(test.termMatch, nil)
if !reflect.DeepEqual(actual, test.result) { if !reflect.DeepEqual(actual, test.result) {
t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch) t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch)

View File

@ -13,12 +13,12 @@ import (
"math" "math"
) )
var SqrtCache map[int]float64 var SqrtCache []float64
const MaxSqrtCache = 64 const MaxSqrtCache = 64
func init() { func init() {
SqrtCache = make(map[int]float64, MaxSqrtCache) SqrtCache = make([]float64, MaxSqrtCache)
for i := 0; i < MaxSqrtCache; i++ { for i := 0; i < MaxSqrtCache; i++ {
SqrtCache[i] = math.Sqrt(float64(i)) SqrtCache[i] = math.Sqrt(float64(i))
} }

View File

@ -85,6 +85,11 @@ func (dm *DocumentMatch) AddFieldValue(name string, value interface{}) {
dm.Fields[name] = valSlice dm.Fields[name] = valSlice
} }
func (dm *DocumentMatch) Reset() *DocumentMatch {
*dm = DocumentMatch{}
return dm
}
type DocumentMatchCollection []*DocumentMatch type DocumentMatchCollection []*DocumentMatch
func (c DocumentMatchCollection) Len() int { return len(c) } func (c DocumentMatchCollection) Len() int { return len(c) }
@ -92,8 +97,8 @@ func (c DocumentMatchCollection) Swap(i, j int) { c[i], c[j] = c[j], c[i] }
func (c DocumentMatchCollection) Less(i, j int) bool { return c[i].Score > c[j].Score } func (c DocumentMatchCollection) Less(i, j int) bool { return c[i].Score > c[j].Score }
type Searcher interface { type Searcher interface {
Next() (*DocumentMatch, error) Next(preAllocated *DocumentMatch) (*DocumentMatch, error)
Advance(ID string) (*DocumentMatch, error) Advance(ID string, preAllocated *DocumentMatch) (*DocumentMatch, error)
Close() error Close() error
Weight() float64 Weight() float64
SetQueryNorm(float64) SetQueryNorm(float64)

View File

@ -70,21 +70,21 @@ func (s *BooleanSearcher) initSearchers() error {
var err error var err error
// get all searchers pointing at their first match // get all searchers pointing at their first match
if s.mustSearcher != nil { if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next() s.currMust, err = s.mustSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
} }
if s.shouldSearcher != nil { if s.shouldSearcher != nil {
s.currShould, err = s.shouldSearcher.Next() s.currShould, err = s.shouldSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
} }
if s.mustNotSearcher != nil { if s.mustNotSearcher != nil {
s.currMustNot, err = s.mustNotSearcher.Next() s.currMustNot, err = s.mustNotSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -106,12 +106,12 @@ func (s *BooleanSearcher) advanceNextMust() error {
var err error var err error
if s.mustSearcher != nil { if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next() s.currMust, err = s.mustSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
} else if s.mustSearcher == nil { } else if s.mustSearcher == nil {
s.currShould, err = s.shouldSearcher.Next() s.currShould, err = s.shouldSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -148,7 +148,7 @@ func (s *BooleanSearcher) SetQueryNorm(qnorm float64) {
} }
} }
func (s *BooleanSearcher) Next() (*search.DocumentMatch, error) { func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
@ -163,7 +163,7 @@ func (s *BooleanSearcher) Next() (*search.DocumentMatch, error) {
for s.currentID != "" { for s.currentID != "" {
if s.currMustNot != nil && s.currMustNot.ID < s.currentID { if s.currMustNot != nil && s.currMustNot.ID < s.currentID {
// advance must not searcher to our candidate entry // advance must not searcher to our candidate entry
s.currMustNot, err = s.mustNotSearcher.Advance(s.currentID) s.currMustNot, err = s.mustNotSearcher.Advance(s.currentID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -186,7 +186,7 @@ func (s *BooleanSearcher) Next() (*search.DocumentMatch, error) {
if s.currShould != nil && s.currShould.ID < s.currentID { if s.currShould != nil && s.currShould.ID < s.currentID {
// advance should searcher to our candidate entry // advance should searcher to our candidate entry
s.currShould, err = s.shouldSearcher.Advance(s.currentID) s.currShould, err = s.shouldSearcher.Advance(s.currentID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -255,7 +255,7 @@ func (s *BooleanSearcher) Next() (*search.DocumentMatch, error) {
return rv, nil return rv, nil
} }
func (s *BooleanSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *BooleanSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
@ -266,19 +266,19 @@ func (s *BooleanSearcher) Advance(ID string) (*search.DocumentMatch, error) {
var err error var err error
if s.mustSearcher != nil { if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Advance(ID) s.currMust, err = s.mustSearcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
if s.shouldSearcher != nil { if s.shouldSearcher != nil {
s.currShould, err = s.shouldSearcher.Advance(ID) s.currShould, err = s.shouldSearcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
if s.mustNotSearcher != nil { if s.mustNotSearcher != nil {
s.currMustNot, err = s.mustNotSearcher.Advance(ID) s.currMustNot, err = s.mustNotSearcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -292,7 +292,7 @@ func (s *BooleanSearcher) Advance(ID string) (*search.DocumentMatch, error) {
s.currentID = "" s.currentID = ""
} }
return s.Next() return s.Next(preAllocated)
} }
func (s *BooleanSearcher) Count() uint64 { func (s *BooleanSearcher) Count() uint64 {

View File

@ -342,7 +342,7 @@ func TestBooleanSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -354,7 +354,7 @@ func TestBooleanSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -67,7 +67,7 @@ func (s *ConjunctionSearcher) initSearchers() error {
var err error var err error
// get all searchers pointing at their first match // get all searchers pointing at their first match
for i, termSearcher := range s.searchers { for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Next() s.currs[i], err = termSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -99,7 +99,7 @@ func (s *ConjunctionSearcher) SetQueryNorm(qnorm float64) {
} }
} }
func (s *ConjunctionSearcher) Next() (*search.DocumentMatch, error) { func (s *ConjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -117,7 +117,7 @@ OUTER:
continue OUTER continue OUTER
} }
// this reader doesn't have the currentID, try to advance // this reader doesn't have the currentID, try to advance
s.currs[i], err = termSearcher.Advance(s.currentID) s.currs[i], err = termSearcher.Advance(s.currentID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -140,7 +140,7 @@ OUTER:
rv = s.scorer.Score(s.currs) rv = s.scorer.Score(s.currs)
// prepare for next entry // prepare for next entry
s.currs[0], err = s.searchers[0].Next() s.currs[0], err = s.searchers[0].Next(nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -155,7 +155,7 @@ OUTER:
return rv, nil return rv, nil
} }
func (s *ConjunctionSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *ConjunctionSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -164,13 +164,13 @@ func (s *ConjunctionSearcher) Advance(ID string) (*search.DocumentMatch, error)
} }
var err error var err error
for i, searcher := range s.searchers { for i, searcher := range s.searchers {
s.currs[i], err = searcher.Advance(ID) s.currs[i], err = searcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
} }
s.currentID = ID s.currentID = ID
return s.Next() return s.Next(preAllocated)
} }
func (s *ConjunctionSearcher) Count() uint64 { func (s *ConjunctionSearcher) Count() uint64 {

View File

@ -187,7 +187,7 @@ func TestConjunctionSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -199,7 +199,7 @@ func TestConjunctionSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -87,7 +87,7 @@ func (s *DisjunctionSearcher) initSearchers() error {
var err error var err error
// get all searchers pointing at their first match // get all searchers pointing at their first match
for i, termSearcher := range s.searchers { for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Next() s.currs[i], err = termSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -122,7 +122,7 @@ func (s *DisjunctionSearcher) SetQueryNorm(qnorm float64) {
} }
} }
func (s *DisjunctionSearcher) Next() (*search.DocumentMatch, error) { func (s *DisjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -153,7 +153,7 @@ func (s *DisjunctionSearcher) Next() (*search.DocumentMatch, error) {
for i, curr := range s.currs { for i, curr := range s.currs {
if curr != nil && curr.ID == s.currentID { if curr != nil && curr.ID == s.currentID {
searcher := s.searchers[i] searcher := s.searchers[i]
s.currs[i], err = searcher.Next() s.currs[i], err = searcher.Next(nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -164,7 +164,7 @@ func (s *DisjunctionSearcher) Next() (*search.DocumentMatch, error) {
return rv, nil return rv, nil
} }
func (s *DisjunctionSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *DisjunctionSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -174,7 +174,7 @@ func (s *DisjunctionSearcher) Advance(ID string) (*search.DocumentMatch, error)
// get all searchers pointing at their first match // get all searchers pointing at their first match
var err error var err error
for i, termSearcher := range s.searchers { for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Advance(ID) s.currs[i], err = termSearcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -182,7 +182,7 @@ func (s *DisjunctionSearcher) Advance(ID string) (*search.DocumentMatch, error)
s.currentID = s.nextSmallestID() s.currentID = s.nextSmallestID()
return s.Next() return s.Next(preAllocated)
} }
func (s *DisjunctionSearcher) Count() uint64 { func (s *DisjunctionSearcher) Count() uint64 {

View File

@ -108,7 +108,7 @@ func TestDisjunctionSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -120,7 +120,7 @@ func TestDisjunctionSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {
@ -158,7 +158,7 @@ func TestDisjunctionAdvance(t *testing.T) {
t.Fatal(err) t.Fatal(err)
} }
match, err := martyOrDustinSearcher.Advance("3") match, err := martyOrDustinSearcher.Advance("3", nil)
if err != nil { if err != nil {
t.Errorf("unexpected error: %v", err) t.Errorf("unexpected error: %v", err)
} }

View File

@ -77,7 +77,7 @@ func (s *DocIDSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm) s.scorer.SetQueryNorm(qnorm)
} }
func (s *DocIDSearcher) Next() (*search.DocumentMatch, error) { func (s *DocIDSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if s.current >= len(s.ids) { if s.current >= len(s.ids) {
return nil, nil return nil, nil
} }
@ -88,9 +88,9 @@ func (s *DocIDSearcher) Next() (*search.DocumentMatch, error) {
} }
func (s *DocIDSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *DocIDSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
s.current = sort.SearchStrings(s.ids, ID) s.current = sort.SearchStrings(s.ids, ID)
return s.Next() return s.Next(preAllocated)
} }
func (s *DocIDSearcher) Close() error { func (s *DocIDSearcher) Close() error {

View File

@ -68,7 +68,7 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
// Check the sequence // Check the sequence
for i, id := range wanted { for i, id := range wanted {
m, err := searcher.Next() m, err := searcher.Next(nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -76,7 +76,7 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
t.Fatalf("expected %v at position %v, got %v", id, i, m.ID) t.Fatalf("expected %v at position %v, got %v", id, i, m.ID)
} }
} }
m, err := searcher.Next() m, err := searcher.Next(nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -91,7 +91,7 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
} }
before := id[:1] before := id[:1]
for _, target := range []string{before, id} { for _, target := range []string{before, id} {
m, err := searcher.Advance(target) m, err := searcher.Advance(target, nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -102,7 +102,7 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
} }
// Seek after the end of the sequence // Seek after the end of the sequence
after := "zzz" after := "zzz"
m, err = searcher.Advance(after) m, err = searcher.Advance(after, nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }

View File

@ -107,13 +107,13 @@ func (s *FuzzySearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm) s.searcher.SetQueryNorm(qnorm)
} }
func (s *FuzzySearcher) Next() (*search.DocumentMatch, error) { func (s *FuzzySearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next() return s.searcher.Next(preAllocated)
} }
func (s *FuzzySearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *FuzzySearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID) return s.searcher.Advance(ID, preAllocated)
} }
func (s *FuzzySearcher) Close() error { func (s *FuzzySearcher) Close() error {

View File

@ -105,7 +105,7 @@ func TestFuzzySearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -117,7 +117,7 @@ func TestFuzzySearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -46,7 +46,7 @@ func (s *MatchAllSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm) s.scorer.SetQueryNorm(qnorm)
} }
func (s *MatchAllSearcher) Next() (*search.DocumentMatch, error) { func (s *MatchAllSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
id, err := s.reader.Next() id, err := s.reader.Next()
if err != nil { if err != nil {
return nil, err return nil, err
@ -63,7 +63,7 @@ func (s *MatchAllSearcher) Next() (*search.DocumentMatch, error) {
} }
func (s *MatchAllSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *MatchAllSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
id, err := s.reader.Advance(ID) id, err := s.reader.Advance(ID)
if err != nil { if err != nil {
return nil, err return nil, err

View File

@ -109,7 +109,7 @@ func TestMatchAllSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -121,7 +121,7 @@ func TestMatchAllSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -36,11 +36,11 @@ func (s *MatchNoneSearcher) SetQueryNorm(qnorm float64) {
} }
func (s *MatchNoneSearcher) Next() (*search.DocumentMatch, error) { func (s *MatchNoneSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return nil, nil return nil, nil
} }
func (s *MatchNoneSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *MatchNoneSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return nil, nil return nil, nil
} }

View File

@ -51,7 +51,7 @@ func TestMatchNoneSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -63,7 +63,7 @@ func TestMatchNoneSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -96,12 +96,12 @@ func (s *NumericRangeSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm) s.searcher.SetQueryNorm(qnorm)
} }
func (s *NumericRangeSearcher) Next() (*search.DocumentMatch, error) { func (s *NumericRangeSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next() return s.searcher.Next(preAllocated)
} }
func (s *NumericRangeSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *NumericRangeSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID) return s.searcher.Advance(ID, preAllocated)
} }
func (s *NumericRangeSearcher) Close() error { func (s *NumericRangeSearcher) Close() error {

View File

@ -56,7 +56,7 @@ func (s *PhraseSearcher) initSearchers() error {
var err error var err error
// get all searchers pointing at their first match // get all searchers pointing at their first match
if s.mustSearcher != nil { if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next() s.currMust, err = s.mustSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -70,7 +70,7 @@ func (s *PhraseSearcher) advanceNextMust() error {
var err error var err error
if s.mustSearcher != nil { if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next() s.currMust, err = s.mustSearcher.Next(nil)
if err != nil { if err != nil {
return err return err
} }
@ -90,7 +90,7 @@ func (s *PhraseSearcher) SetQueryNorm(qnorm float64) {
s.mustSearcher.SetQueryNorm(qnorm) s.mustSearcher.SetQueryNorm(qnorm)
} }
func (s *PhraseSearcher) Next() (*search.DocumentMatch, error) { func (s *PhraseSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -160,7 +160,7 @@ func (s *PhraseSearcher) Next() (*search.DocumentMatch, error) {
return nil, nil return nil, nil
} }
func (s *PhraseSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *PhraseSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
if !s.initialized { if !s.initialized {
err := s.initSearchers() err := s.initSearchers()
if err != nil { if err != nil {
@ -168,11 +168,11 @@ func (s *PhraseSearcher) Advance(ID string) (*search.DocumentMatch, error) {
} }
} }
var err error var err error
s.currMust, err = s.mustSearcher.Advance(ID) s.currMust, err = s.mustSearcher.Advance(ID, nil)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return s.Next() return s.Next(preAllocated)
} }
func (s *PhraseSearcher) Count() uint64 { func (s *PhraseSearcher) Count() uint64 {

View File

@ -68,7 +68,7 @@ func TestPhraseSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -80,7 +80,7 @@ func TestPhraseSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -106,13 +106,13 @@ func (s *RegexpSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm) s.searcher.SetQueryNorm(qnorm)
} }
func (s *RegexpSearcher) Next() (*search.DocumentMatch, error) { func (s *RegexpSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next() return s.searcher.Next(preAllocated)
} }
func (s *RegexpSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *RegexpSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID) return s.searcher.Advance(ID, preAllocated)
} }
func (s *RegexpSearcher) Close() error { func (s *RegexpSearcher) Close() error {

View File

@ -85,7 +85,7 @@ func TestRegexpSearch(t *testing.T) {
} }
}() }()
next, err := test.searcher.Next() next, err := test.searcher.Next(nil)
i := 0 i := 0
for err == nil && next != nil { for err == nil && next != nil {
if i < len(test.results) { if i < len(test.results) {
@ -97,7 +97,7 @@ func TestRegexpSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl) t.Logf("scoring explanation: %s", next.Expl)
} }
} }
next, err = test.searcher.Next() next, err = test.searcher.Next(nil)
i++ i++
} }
if err != nil { if err != nil {

View File

@ -19,9 +19,10 @@ type TermSearcher struct {
indexReader index.IndexReader indexReader index.IndexReader
term string term string
field string field string
explain bool
reader index.TermFieldReader reader index.TermFieldReader
scorer *scorers.TermQueryScorer scorer *scorers.TermQueryScorer
tfd index.TermFieldDoc
explain bool
} }
func NewTermSearcher(indexReader index.IndexReader, term string, field string, boost float64, explain bool) (*TermSearcher, error) { func NewTermSearcher(indexReader index.IndexReader, term string, field string, boost float64, explain bool) (*TermSearcher, error) {
@ -52,8 +53,8 @@ func (s *TermSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm) s.scorer.SetQueryNorm(qnorm)
} }
func (s *TermSearcher) Next() (*search.DocumentMatch, error) { func (s *TermSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
termMatch, err := s.reader.Next() termMatch, err := s.reader.Next(s.tfd.Reset())
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -63,14 +64,14 @@ func (s *TermSearcher) Next() (*search.DocumentMatch, error) {
} }
// score match // score match
docMatch := s.scorer.Score(termMatch) docMatch := s.scorer.Score(termMatch, preAllocated)
// return doc match // return doc match
return docMatch, nil return docMatch, nil
} }
func (s *TermSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *TermSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
termMatch, err := s.reader.Advance(ID) termMatch, err := s.reader.Advance(ID, s.tfd.Reset())
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -80,7 +81,7 @@ func (s *TermSearcher) Advance(ID string) (*search.DocumentMatch, error) {
} }
// score match // score match
docMatch := s.scorer.Score(termMatch) docMatch := s.scorer.Score(termMatch, preAllocated)
// return doc match // return doc match
return docMatch, nil return docMatch, nil

View File

@ -70,13 +70,13 @@ func (s *TermPrefixSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm) s.searcher.SetQueryNorm(qnorm)
} }
func (s *TermPrefixSearcher) Next() (*search.DocumentMatch, error) { func (s *TermPrefixSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next() return s.searcher.Next(preAllocated)
} }
func (s *TermPrefixSearcher) Advance(ID string) (*search.DocumentMatch, error) { func (s *TermPrefixSearcher) Advance(ID string, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID) return s.searcher.Advance(ID, preAllocated)
} }
func (s *TermPrefixSearcher) Close() error { func (s *TermPrefixSearcher) Close() error {

View File

@ -163,14 +163,14 @@ func TestTermSearcher(t *testing.T) {
t.Errorf("expected count of 9, got %d", searcher.Count()) t.Errorf("expected count of 9, got %d", searcher.Count())
} }
docMatch, err := searcher.Next() docMatch, err := searcher.Next(nil)
if err != nil { if err != nil {
t.Errorf("expected result, got %v", err) t.Errorf("expected result, got %v", err)
} }
if docMatch.ID != "a" { if docMatch.ID != "a" {
t.Errorf("expected result ID to be 'a', got '%s", docMatch.ID) t.Errorf("expected result ID to be 'a', got '%s", docMatch.ID)
} }
docMatch, err = searcher.Advance("c") docMatch, err = searcher.Advance("c", nil)
if err != nil { if err != nil {
t.Errorf("expected result, got %v", err) t.Errorf("expected result, got %v", err)
} }
@ -179,7 +179,7 @@ func TestTermSearcher(t *testing.T) {
} }
// try advancing past end // try advancing past end
docMatch, err = searcher.Advance("z") docMatch, err = searcher.Advance("z", nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -188,7 +188,7 @@ func TestTermSearcher(t *testing.T) {
} }
// try pushing next past end // try pushing next past end
docMatch, err = searcher.Next() docMatch, err = searcher.Next(nil)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }