0
0
Fork 0

refactor search package to reuse DocumentMatch and ID []byte's

the motivation for this commit is long and detailed and has been
documented externally here:

https://gist.github.com/mschoch/5cc5c9cf4669a5fe8512cb7770d3c1a2

the core of the changes are:

1.  recognize that collector/searcher need only a fixed number
of DocumentMatch instances, and this number can be determined
from the structure of the query, not the size of the data

2.  knowing this, instances can be allocated in bulk, up front
and they can be reused without locking (since all search
operations take place in a single goroutine

3.  combined with previous commits which enabled reuse of
the IndexInternalID []byte, this allows for no allocation/copy
of these bytes as well (by using DocumentMatch Reset() method
when returning entries to the pool
This commit is contained in:
Marty Schoch 2016-08-08 22:21:47 -04:00
parent b857769217
commit 24a2b57e29
32 changed files with 441 additions and 173 deletions

View File

@ -64,13 +64,18 @@ var COLLECT_CHECK_DONE_EVERY = uint64(1024)
func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Searcher, reader index.IndexReader) error {
startTime := time.Now()
var err error
var pre search.DocumentMatch // A single pre-alloc'ed, reused instance.
var next *search.DocumentMatch
// search context with enough pre-allocated document matches
searchContext := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(tksc.k + tksc.skip + searcher.DocumentMatchPoolSize()),
}
select {
case <-ctx.Done():
return ctx.Err()
default:
next, err = searcher.Next(&pre)
next, err = searcher.Next(searchContext)
}
for err == nil && next != nil {
if tksc.total%COLLECT_CHECK_DONE_EVERY == 0 {
@ -80,14 +85,15 @@ func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Sear
default:
}
}
tksc.collectSingle(next)
if tksc.facetsBuilder != nil {
err = tksc.facetsBuilder.Update(next)
if err != nil {
break
}
}
next, err = searcher.Next(pre.Reset())
tksc.collectSingle(searchContext, next)
next, err = searcher.Next(searchContext)
}
// finalize actual results
tksc.actualResults, err = tksc.finalizeResults(reader)
@ -103,43 +109,42 @@ func (tksc *TopScoreCollector) Collect(ctx context.Context, searcher search.Sear
return nil
}
func (tksc *TopScoreCollector) collectSingle(dmIn *search.DocumentMatch) {
func (tksc *TopScoreCollector) collectSingle(ctx *search.SearchContext, d *search.DocumentMatch) {
// increment total hits
tksc.total++
// update max score
if dmIn.Score > tksc.maxScore {
tksc.maxScore = dmIn.Score
if d.Score > tksc.maxScore {
tksc.maxScore = d.Score
}
if dmIn.Score <= tksc.minScore {
if d.Score <= tksc.minScore {
ctx.DocumentMatchPool.Put(d)
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() {
curr := e.Value.(*search.DocumentMatch)
if dm.Score <= curr.Score {
if d.Score <= curr.Score {
tksc.results.InsertBefore(dm, e)
tksc.results.InsertBefore(d, e)
// if we just made the list too long
if tksc.results.Len() > (tksc.k + tksc.skip) {
// remove the head
tksc.minScore = tksc.results.Remove(tksc.results.Front()).(*search.DocumentMatch).Score
removed := tksc.results.Remove(tksc.results.Front()).(*search.DocumentMatch)
tksc.minScore = removed.Score
ctx.DocumentMatchPool.Put(removed)
}
return
}
}
// if we got to the end, we still have to add it
tksc.results.PushBack(dm)
tksc.results.PushBack(d)
if tksc.results.Len() > (tksc.k + tksc.skip) {
// remove the head
tksc.minScore = tksc.results.Remove(tksc.results.Front()).(*search.DocumentMatch).Score
removed := tksc.results.Remove(tksc.results.Front()).(*search.DocumentMatch)
tksc.minScore = removed.Score
ctx.DocumentMatchPool.Put(removed)
}
}

View File

@ -20,7 +20,7 @@ type stubSearcher struct {
matches []*search.DocumentMatch
}
func (ss *stubSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (ss *stubSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if ss.index < len(ss.matches) {
rv := ss.matches[ss.index]
ss.index++
@ -29,7 +29,7 @@ func (ss *stubSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docume
return nil, nil
}
func (ss *stubSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (ss *stubSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
for ss.index < len(ss.matches) && ss.matches[ss.index].IndexInternalID.Compare(ID) < 0 {
ss.index++
@ -61,6 +61,10 @@ func (ss *stubSearcher) Min() int {
return 0
}
func (ss *stubSearcher) DocumentMatchPoolSize() int {
return 0
}
type stubReader struct{}
func (sr *stubReader) TermFieldReader(term []byte, field string, includeFreq, includeNorm, includeTermVectors bool) (index.TermFieldReader, error) {

70
search/pool.go Normal file
View File

@ -0,0 +1,70 @@
// Copyright (c) 2014 Couchbase, Inc.
// Licensed under the Apache License, Version 2.0 (the "License"); you may not use this file
// except in compliance with the License. You may obtain a copy of the License at
// http://www.apache.org/licenses/LICENSE-2.0
// Unless required by applicable law or agreed to in writing, software distributed under the
// License is distributed on an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND,
// either express or implied. See the License for the specific language governing permissions
// and limitations under the License.
package search
// DocumentMatchPoolTooSmall is a callback function that can be executed
// when the DocumentMatchPool does not have sufficient capacity
// By default we just perform just-in-time allocation, but you could log
// a message, or panic, etc.
type DocumentMatchPoolTooSmall func(p *DocumentMatchPool) *DocumentMatch
// DocumentMatchPool manages use/re-use of DocumentMatch instances
// it pre-allocates space from a single large block with the expected
// number of instances. It is not thread-safe as currently all
// aspects of search take place in a single goroutine.
type DocumentMatchPool struct {
avail DocumentMatchCollection
TooSmall DocumentMatchPoolTooSmall
}
func defaultDocumentMatchPoolTooSmall(p *DocumentMatchPool) *DocumentMatch {
return &DocumentMatch{}
}
// NewDocumentMatchPool will build a DocumentMatchPool with memory
// pre-allocated to accomodate the requested number of DocumentMatch
// instances
func NewDocumentMatchPool(size int) *DocumentMatchPool {
avail := make(DocumentMatchCollection, 0, size)
// pre-allocate the expected number of instances
startBlock := make([]DocumentMatch, size)
// make these initial instances available
for i := range startBlock {
avail = append(avail, &startBlock[i])
}
return &DocumentMatchPool{
avail: avail,
TooSmall: defaultDocumentMatchPoolTooSmall,
}
}
// Get returns an available DocumentMatch from the pool
// if the pool was not allocated with sufficient size, an allocation will
// occur to satisfy this request. As a side-effect this will grow the size
// of the pool.
func (p *DocumentMatchPool) Get() *DocumentMatch {
var rv *DocumentMatch
if len(p.avail) > 0 {
rv, p.avail = p.avail[len(p.avail)-1], p.avail[:len(p.avail)-1]
} else {
rv = p.TooSmall(p)
}
return rv
}
// Put returns a DocumentMatch to the pool
func (p *DocumentMatchPool) Put(d *DocumentMatch) {
if d == nil {
return
}
// reset DocumentMatch before returning it to available pool
d.Reset()
p.avail = append(p.avail, d)
}

View File

@ -23,11 +23,7 @@ func NewConjunctionQueryScorer(explain bool) *ConjunctionQueryScorer {
}
}
func (s *ConjunctionQueryScorer) Score(constituents []*search.DocumentMatch) *search.DocumentMatch {
rv := search.DocumentMatch{
IndexInternalID: constituents[0].IndexInternalID,
}
func (s *ConjunctionQueryScorer) Score(ctx *search.SearchContext, constituents []*search.DocumentMatch) *search.DocumentMatch {
var sum float64
var childrenExplanations []*search.Explanation
if s.explain {
@ -44,16 +40,21 @@ func (s *ConjunctionQueryScorer) Score(constituents []*search.DocumentMatch) *se
locations = append(locations, docMatch.Locations)
}
}
rv.Score = sum
newScore := sum
var newExpl *search.Explanation
if s.explain {
rv.Expl = &search.Explanation{Value: sum, Message: "sum of:", Children: childrenExplanations}
newExpl = &search.Explanation{Value: sum, Message: "sum of:", Children: childrenExplanations}
}
// reuse constituents[0] as the return value
rv := constituents[0]
rv.Score = newScore
rv.Expl = newExpl
if len(locations) == 1 {
rv.Locations = locations[0]
} else if len(locations) > 1 {
rv.Locations = search.MergeLocations(locations)
}
return &rv
return rv
}

View File

@ -65,7 +65,7 @@ func (s *ConstantScorer) SetQueryNorm(qnorm float64) {
}
}
func (s *ConstantScorer) Score(id index.IndexInternalID) *search.DocumentMatch {
func (s *ConstantScorer) Score(ctx *search.SearchContext, id index.IndexInternalID) *search.DocumentMatch {
var scoreExplanation *search.Explanation
score := s.constant
@ -92,13 +92,12 @@ func (s *ConstantScorer) Score(id index.IndexInternalID) *search.DocumentMatch {
}
}
rv := search.DocumentMatch{
IndexInternalID: id,
Score: score,
}
rv := ctx.DocumentMatchPool.Get()
rv.IndexInternalID = id
rv.Score = score
if s.explain {
rv.Expl = scoreExplanation
}
return &rv
return rv
}

View File

@ -52,7 +52,10 @@ func TestConstantScorer(t *testing.T) {
}
for _, test := range tests {
actual := scorer.Score(test.termMatch.ID)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(1),
}
actual := scorer.Score(ctx, test.termMatch.ID)
if !reflect.DeepEqual(actual, test.result) {
t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch)
@ -108,7 +111,10 @@ func TestConstantScorerWithQueryNorm(t *testing.T) {
}
for _, test := range tests {
actual := scorer.Score(test.termMatch.ID)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(1),
}
actual := scorer.Score(ctx, test.termMatch.ID)
if !reflect.DeepEqual(actual, test.result) {
t.Errorf("expected %#v got %#v for %#v", test.result, actual, test.termMatch)

View File

@ -25,11 +25,7 @@ func NewDisjunctionQueryScorer(explain bool) *DisjunctionQueryScorer {
}
}
func (s *DisjunctionQueryScorer) Score(constituents []*search.DocumentMatch, countMatch, countTotal int) *search.DocumentMatch {
rv := search.DocumentMatch{
IndexInternalID: constituents[0].IndexInternalID,
}
func (s *DisjunctionQueryScorer) Score(ctx *search.SearchContext, constituents []*search.DocumentMatch, countMatch, countTotal int) *search.DocumentMatch {
var sum float64
var childrenExplanations []*search.Explanation
if s.explain {
@ -53,19 +49,24 @@ func (s *DisjunctionQueryScorer) Score(constituents []*search.DocumentMatch, cou
}
coord := float64(countMatch) / float64(countTotal)
rv.Score = sum * coord
newScore := sum * coord
var newExpl *search.Explanation
if s.explain {
ce := make([]*search.Explanation, 2)
ce[0] = rawExpl
ce[1] = &search.Explanation{Value: coord, Message: fmt.Sprintf("coord(%d/%d)", countMatch, countTotal)}
rv.Expl = &search.Explanation{Value: rv.Score, Message: "product of:", Children: ce}
newExpl = &search.Explanation{Value: newScore, Message: "product of:", Children: ce}
}
// reuse constituents[0] as the return value
rv := constituents[0]
rv.Score = newScore
rv.Expl = newExpl
if len(locations) == 1 {
rv.Locations = locations[0]
} else if len(locations) > 1 {
rv.Locations = search.MergeLocations(locations)
}
return &rv
return rv
}

View File

@ -83,7 +83,7 @@ func (s *TermQueryScorer) SetQueryNorm(qnorm float64) {
}
}
func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc, preAllocated *search.DocumentMatch) *search.DocumentMatch {
func (s *TermQueryScorer) Score(ctx *search.SearchContext, termMatch *index.TermFieldDoc) *search.DocumentMatch {
var scoreExplanation *search.Explanation
// need to compute score
@ -128,10 +128,7 @@ func (s *TermQueryScorer) Score(termMatch *index.TermFieldDoc, preAllocated *sea
}
}
rv := preAllocated
if rv == nil {
rv = &search.DocumentMatch{}
}
rv := ctx.DocumentMatchPool.Get()
rv.IndexInternalID = append(rv.IndexInternalID, termMatch.ID...)
rv.Score = score
if s.explain {

View File

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

View File

@ -106,11 +106,18 @@ 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 }
type Searcher interface {
Next(preAllocated *DocumentMatch) (*DocumentMatch, error)
Advance(ID index.IndexInternalID, preAllocated *DocumentMatch) (*DocumentMatch, error)
Next(ctx *SearchContext) (*DocumentMatch, error)
Advance(ctx *SearchContext, ID index.IndexInternalID) (*DocumentMatch, error)
Close() error
Weight() float64
SetQueryNorm(float64)
Count() uint64
Min() int
DocumentMatchPoolSize() int
}
// SearchContext represents the context around a single search
type SearchContext struct {
DocumentMatchPool *DocumentMatchPool
}

View File

@ -66,25 +66,34 @@ func (s *BooleanSearcher) computeQueryNorm() {
}
}
func (s *BooleanSearcher) initSearchers() error {
func (s *BooleanSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next(nil)
if s.currMust != nil {
ctx.DocumentMatchPool.Put(s.currMust)
}
s.currMust, err = s.mustSearcher.Next(ctx)
if err != nil {
return err
}
}
if s.shouldSearcher != nil {
s.currShould, err = s.shouldSearcher.Next(nil)
if s.currShould != nil {
ctx.DocumentMatchPool.Put(s.currShould)
}
s.currShould, err = s.shouldSearcher.Next(ctx)
if err != nil {
return err
}
}
if s.mustNotSearcher != nil {
s.currMustNot, err = s.mustNotSearcher.Next(nil)
if s.currMustNot != nil {
ctx.DocumentMatchPool.Put(s.currMustNot)
}
s.currMustNot, err = s.mustNotSearcher.Next(ctx)
if err != nil {
return err
}
@ -102,16 +111,22 @@ func (s *BooleanSearcher) initSearchers() error {
return nil
}
func (s *BooleanSearcher) advanceNextMust() error {
func (s *BooleanSearcher) advanceNextMust(ctx *search.SearchContext, skipReturn *search.DocumentMatch) error {
var err error
if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next(nil)
if s.currMust != skipReturn {
ctx.DocumentMatchPool.Put(s.currMust)
}
s.currMust, err = s.mustSearcher.Next(ctx)
if err != nil {
return err
}
} else if s.mustSearcher == nil {
s.currShould, err = s.shouldSearcher.Next(nil)
if s.currShould != skipReturn {
ctx.DocumentMatchPool.Put(s.currShould)
}
s.currShould, err = s.shouldSearcher.Next(ctx)
if err != nil {
return err
}
@ -148,10 +163,10 @@ func (s *BooleanSearcher) SetQueryNorm(qnorm float64) {
}
}
func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *BooleanSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -162,14 +177,17 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
for s.currentID != nil {
if s.currMustNot != nil && s.currMustNot.IndexInternalID.Compare(s.currentID) < 0 {
if s.currMustNot != nil {
ctx.DocumentMatchPool.Put(s.currMustNot)
}
// advance must not searcher to our candidate entry
s.currMustNot, err = s.mustNotSearcher.Advance(s.currentID, nil)
s.currMustNot, err = s.mustNotSearcher.Advance(ctx, s.currentID)
if err != nil {
return nil, err
}
if s.currMustNot != nil && s.currMustNot.IndexInternalID.Equals(s.currentID) {
// the candidate is excluded
err = s.advanceNextMust()
err = s.advanceNextMust(ctx, nil)
if err != nil {
return nil, err
}
@ -177,7 +195,7 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
}
} else if s.currMustNot != nil && s.currMustNot.IndexInternalID.Equals(s.currentID) {
// the candidate is excluded
err = s.advanceNextMust()
err = s.advanceNextMust(ctx, nil)
if err != nil {
return nil, err
}
@ -186,7 +204,10 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
if s.currShould != nil && s.currShould.IndexInternalID.Compare(s.currentID) < 0 {
// advance should searcher to our candidate entry
s.currShould, err = s.shouldSearcher.Advance(s.currentID, nil)
if s.currShould != nil {
ctx.DocumentMatchPool.Put(s.currShould)
}
s.currShould, err = s.shouldSearcher.Advance(ctx, s.currentID)
if err != nil {
return nil, err
}
@ -203,16 +224,16 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
s.currShould,
}
}
rv = s.scorer.Score(cons)
err = s.advanceNextMust()
rv = s.scorer.Score(ctx, cons)
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
break
} else if s.shouldSearcher.Min() == 0 {
// match is OK anyway
rv = s.scorer.Score([]*search.DocumentMatch{s.currMust})
err = s.advanceNextMust()
rv = s.scorer.Score(ctx, []*search.DocumentMatch{s.currMust})
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
@ -231,23 +252,23 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
s.currShould,
}
}
rv = s.scorer.Score(cons)
err = s.advanceNextMust()
rv = s.scorer.Score(ctx, cons)
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
break
} else if s.shouldSearcher == nil || s.shouldSearcher.Min() == 0 {
// match is OK anyway
rv = s.scorer.Score([]*search.DocumentMatch{s.currMust})
err = s.advanceNextMust()
rv = s.scorer.Score(ctx, []*search.DocumentMatch{s.currMust})
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
break
}
err = s.advanceNextMust()
err = s.advanceNextMust(ctx, nil)
if err != nil {
return nil, err
}
@ -255,10 +276,10 @@ func (s *BooleanSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docu
return rv, nil
}
func (s *BooleanSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *BooleanSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -266,19 +287,28 @@ func (s *BooleanSearcher) Advance(ID index.IndexInternalID, preAllocated *search
var err error
if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Advance(ID, nil)
if s.currMust != nil {
ctx.DocumentMatchPool.Put(s.currMust)
}
s.currMust, err = s.mustSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
}
if s.shouldSearcher != nil {
s.currShould, err = s.shouldSearcher.Advance(ID, nil)
if s.currShould != nil {
ctx.DocumentMatchPool.Put(s.currShould)
}
s.currShould, err = s.shouldSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
}
if s.mustNotSearcher != nil {
s.currMustNot, err = s.mustNotSearcher.Advance(ID, nil)
if s.currMustNot != nil {
ctx.DocumentMatchPool.Put(s.currMustNot)
}
s.currMustNot, err = s.mustNotSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
@ -292,7 +322,7 @@ func (s *BooleanSearcher) Advance(ID index.IndexInternalID, preAllocated *search
s.currentID = nil
}
return s.Next(preAllocated)
return s.Next(ctx)
}
func (s *BooleanSearcher) Count() uint64 {
@ -333,3 +363,17 @@ func (s *BooleanSearcher) Close() error {
func (s *BooleanSearcher) Min() int {
return 0
}
func (s *BooleanSearcher) DocumentMatchPoolSize() int {
rv := 3
if s.mustSearcher != nil {
rv += s.mustSearcher.DocumentMatchPoolSize()
}
if s.shouldSearcher != nil {
rv += s.shouldSearcher.DocumentMatchPoolSize()
}
if s.mustNotSearcher != nil {
rv += s.mustNotSearcher.DocumentMatchPoolSize()
}
return rv
}

View File

@ -343,7 +343,10 @@ func TestBooleanSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -355,7 +358,8 @@ func TestBooleanSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -63,11 +63,14 @@ func (s *ConjunctionSearcher) computeQueryNorm() {
}
}
func (s *ConjunctionSearcher) initSearchers() error {
func (s *ConjunctionSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Next(nil)
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
if err != nil {
return err
}
@ -99,9 +102,9 @@ func (s *ConjunctionSearcher) SetQueryNorm(qnorm float64) {
}
}
func (s *ConjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *ConjunctionSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -117,7 +120,10 @@ OUTER:
continue OUTER
}
// this reader doesn't have the currentID, try to advance
s.currs[i], err = termSearcher.Advance(s.currentID, nil)
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Advance(ctx, s.currentID)
if err != nil {
return nil, err
}
@ -137,40 +143,51 @@ OUTER:
}
}
// if we get here, a doc matched all readers, sum the score and add it
rv = s.scorer.Score(s.currs)
rv = s.scorer.Score(ctx, s.currs)
// prepare for next entry
s.currs[0], err = s.searchers[0].Next(nil)
if err != nil {
return nil, err
// we know all the searchers are pointing at the same thing
// so they all need to be advanced
for i, termSearcher := range s.searchers {
if s.currs[i] != rv {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
if err != nil {
return nil, err
}
}
if s.currs[0] == nil {
s.currentID = nil
} else {
s.currentID = s.currs[0].IndexInternalID
}
// don't continue now, wait for the next call to Next()
break
}
return rv, nil
}
func (s *ConjunctionSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *ConjunctionSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var err error
for i, searcher := range s.searchers {
s.currs[i], err = searcher.Advance(ID, nil)
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
}
s.currentID = ID
return s.Next(preAllocated)
return s.Next(ctx)
}
func (s *ConjunctionSearcher) Count() uint64 {
@ -195,3 +212,11 @@ func (s *ConjunctionSearcher) Close() error {
func (s *ConjunctionSearcher) Min() int {
return 0
}
func (s *ConjunctionSearcher) DocumentMatchPoolSize() int {
rv := len(s.currs)
for _, s := range s.searchers {
rv += s.DocumentMatchPoolSize()
}
return rv
}

View File

@ -188,7 +188,10 @@ func TestConjunctionSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(10),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -200,7 +203,7 @@ func TestConjunctionSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -83,11 +83,14 @@ func (s *DisjunctionSearcher) computeQueryNorm() {
}
}
func (s *DisjunctionSearcher) initSearchers() error {
func (s *DisjunctionSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Next(nil)
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
if err != nil {
return err
}
@ -122,9 +125,9 @@ func (s *DisjunctionSearcher) SetQueryNorm(qnorm float64) {
}
}
func (s *DisjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *DisjunctionSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -144,7 +147,7 @@ func (s *DisjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.
if len(matching) >= int(s.min) {
found = true
// score this match
rv = s.scorer.Score(matching, len(matching), len(s.searchers))
rv = s.scorer.Score(ctx, matching, len(matching), len(s.searchers))
}
// reset matching
@ -153,7 +156,10 @@ func (s *DisjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.
for i, curr := range s.currs {
if curr != nil && curr.IndexInternalID.Equals(s.currentID) {
searcher := s.searchers[i]
s.currs[i], err = searcher.Next(nil)
if s.currs[i] != rv {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return nil, err
}
@ -164,9 +170,9 @@ func (s *DisjunctionSearcher) Next(preAllocated *search.DocumentMatch) (*search.
return rv, nil
}
func (s *DisjunctionSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *DisjunctionSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -174,7 +180,10 @@ func (s *DisjunctionSearcher) Advance(ID index.IndexInternalID, preAllocated *se
// get all searchers pointing at their first match
var err error
for i, termSearcher := range s.searchers {
s.currs[i], err = termSearcher.Advance(ID, nil)
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
@ -182,7 +191,7 @@ func (s *DisjunctionSearcher) Advance(ID index.IndexInternalID, preAllocated *se
s.currentID = s.nextSmallestID()
return s.Next(preAllocated)
return s.Next(ctx)
}
func (s *DisjunctionSearcher) Count() uint64 {
@ -207,3 +216,11 @@ func (s *DisjunctionSearcher) Close() error {
func (s *DisjunctionSearcher) Min() int {
return int(s.min) // FIXME just make this an int
}
func (s *DisjunctionSearcher) DocumentMatchPoolSize() int {
rv := len(s.currs)
for _, s := range s.searchers {
rv += s.DocumentMatchPoolSize()
}
return rv
}

View File

@ -109,7 +109,10 @@ func TestDisjunctionSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -121,7 +124,8 @@ func TestDisjunctionSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
@ -159,7 +163,10 @@ func TestDisjunctionAdvance(t *testing.T) {
t.Fatal(err)
}
match, err := martyOrDustinSearcher.Advance(index.IndexInternalID("3"), nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(martyOrDustinSearcher.DocumentMatchPoolSize()),
}
match, err := martyOrDustinSearcher.Advance(ctx, index.IndexInternalID("3"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}

View File

@ -49,7 +49,7 @@ func (s *DocIDSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *DocIDSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *DocIDSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
docidMatch, err := s.reader.Next()
if err != nil {
return nil, err
@ -58,11 +58,11 @@ func (s *DocIDSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docume
return nil, nil
}
docMatch := s.scorer.Score(docidMatch)
docMatch := s.scorer.Score(ctx, docidMatch)
return docMatch, nil
}
func (s *DocIDSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *DocIDSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
docidMatch, err := s.reader.Advance(ID)
if err != nil {
return nil, err
@ -71,7 +71,7 @@ func (s *DocIDSearcher) Advance(ID index.IndexInternalID, preAllocated *search.D
return nil, nil
}
docMatch := s.scorer.Score(docidMatch)
docMatch := s.scorer.Score(ctx, docidMatch)
return docMatch, nil
}
@ -82,3 +82,7 @@ func (s *DocIDSearcher) Close() error {
func (s *DocIDSearcher) Min() int {
return 0
}
func (s *DocIDSearcher) DocumentMatchPoolSize() int {
return 1
}

View File

@ -16,6 +16,7 @@ import (
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/gtreap"
"github.com/blevesearch/bleve/index/upside_down"
"github.com/blevesearch/bleve/search"
)
func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
@ -62,23 +63,29 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(searcher.DocumentMatchPoolSize()),
}
// Check the sequence
for i, id := range wanted {
m, err := searcher.Next(nil)
m, err := searcher.Next(ctx)
if err != nil {
t.Fatal(err)
}
if !index.IndexInternalID(id).Equals(m.IndexInternalID) {
t.Fatalf("expected %v at position %v, got %v", id, i, m.IndexInternalID)
}
ctx.DocumentMatchPool.Put(m)
}
m, err := searcher.Next(nil)
m, err := searcher.Next(ctx)
if err != nil {
t.Fatal(err)
}
if m != nil {
t.Fatalf("expected nil past the end of the sequence, got %v", m.IndexInternalID)
}
ctx.DocumentMatchPool.Put(m)
// Check seeking
for _, id := range wanted {
@ -87,24 +94,26 @@ func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
}
before := id[:1]
for _, target := range []string{before, id} {
m, err := searcher.Advance(index.IndexInternalID(target), nil)
m, err := searcher.Advance(ctx, index.IndexInternalID(target))
if err != nil {
t.Fatal(err)
}
if m == nil || !m.IndexInternalID.Equals(index.IndexInternalID(id)) {
t.Fatalf("advancing to %v returned %v instead of %v", before, m, id)
}
ctx.DocumentMatchPool.Put(m)
}
}
// Seek after the end of the sequence
after := "zzz"
m, err = searcher.Advance(index.IndexInternalID(after), nil)
m, err = searcher.Advance(ctx, index.IndexInternalID(after))
if err != nil {
t.Fatal(err)
}
if m != nil {
t.Fatalf("advancing past the end of the sequence should return nil, got %v", m)
}
ctx.DocumentMatchPool.Put(m)
}
func TestDocIDSearcherEmptySearchEmptyIndex(t *testing.T) {

View File

@ -107,13 +107,13 @@ func (s *FuzzySearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm)
}
func (s *FuzzySearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next(preAllocated)
func (s *FuzzySearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return s.searcher.Next(ctx)
}
func (s *FuzzySearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID, preAllocated)
func (s *FuzzySearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return s.searcher.Advance(ctx, ID)
}
func (s *FuzzySearcher) Close() error {
@ -123,3 +123,7 @@ func (s *FuzzySearcher) Close() error {
func (s *FuzzySearcher) Min() int {
return 0
}
func (s *FuzzySearcher) DocumentMatchPoolSize() int {
return s.searcher.DocumentMatchPoolSize()
}

View File

@ -106,7 +106,10 @@ func TestFuzzySearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -118,7 +121,8 @@ func TestFuzzySearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -46,7 +46,7 @@ func (s *MatchAllSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *MatchAllSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *MatchAllSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
id, err := s.reader.Next()
if err != nil {
return nil, err
@ -57,13 +57,13 @@ func (s *MatchAllSearcher) Next(preAllocated *search.DocumentMatch) (*search.Doc
}
// score match
docMatch := s.scorer.Score(id)
docMatch := s.scorer.Score(ctx, id)
// return doc match
return docMatch, nil
}
func (s *MatchAllSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *MatchAllSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
id, err := s.reader.Advance(ID)
if err != nil {
return nil, err
@ -74,7 +74,7 @@ func (s *MatchAllSearcher) Advance(ID index.IndexInternalID, preAllocated *searc
}
// score match
docMatch := s.scorer.Score(id)
docMatch := s.scorer.Score(ctx, id)
// return doc match
return docMatch, nil
@ -87,3 +87,7 @@ func (s *MatchAllSearcher) Close() error {
func (s *MatchAllSearcher) Min() int {
return 0
}
func (s *MatchAllSearcher) DocumentMatchPoolSize() int {
return 1
}

View File

@ -110,7 +110,10 @@ func TestMatchAllSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -122,7 +125,8 @@ func TestMatchAllSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -36,11 +36,11 @@ func (s *MatchNoneSearcher) SetQueryNorm(qnorm float64) {
}
func (s *MatchNoneSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *MatchNoneSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return nil, nil
}
func (s *MatchNoneSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *MatchNoneSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return nil, nil
}
@ -51,3 +51,7 @@ func (s *MatchNoneSearcher) Close() error {
func (s *MatchNoneSearcher) Min() int {
return 0
}
func (s *MatchNoneSearcher) DocumentMatchPoolSize() int {
return 0
}

View File

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

View File

@ -96,12 +96,12 @@ func (s *NumericRangeSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm)
}
func (s *NumericRangeSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next(preAllocated)
func (s *NumericRangeSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return s.searcher.Next(ctx)
}
func (s *NumericRangeSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID, preAllocated)
func (s *NumericRangeSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return s.searcher.Advance(ctx, ID)
}
func (s *NumericRangeSearcher) Close() error {
@ -215,3 +215,7 @@ func newRangeBytes(minBytes, maxBytes []byte) *termRange {
func (s *NumericRangeSearcher) Min() int {
return 0
}
func (s *NumericRangeSearcher) DocumentMatchPoolSize() int {
return s.searcher.DocumentMatchPoolSize()
}

View File

@ -52,11 +52,11 @@ func (s *PhraseSearcher) computeQueryNorm() {
}
}
func (s *PhraseSearcher) initSearchers() error {
func (s *PhraseSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next(nil)
s.currMust, err = s.mustSearcher.Next(ctx)
if err != nil {
return err
}
@ -66,11 +66,11 @@ func (s *PhraseSearcher) initSearchers() error {
return nil
}
func (s *PhraseSearcher) advanceNextMust() error {
func (s *PhraseSearcher) advanceNextMust(ctx *search.SearchContext) error {
var err error
if s.mustSearcher != nil {
s.currMust, err = s.mustSearcher.Next(nil)
s.currMust, err = s.mustSearcher.Next(ctx)
if err != nil {
return err
}
@ -90,9 +90,9 @@ func (s *PhraseSearcher) SetQueryNorm(qnorm float64) {
s.mustSearcher.SetQueryNorm(qnorm)
}
func (s *PhraseSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *PhraseSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
@ -144,14 +144,14 @@ func (s *PhraseSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docum
// return match
rv = s.currMust
rv.Locations = rvftlm
err := s.advanceNextMust()
err := s.advanceNextMust(ctx)
if err != nil {
return nil, err
}
return rv, nil
}
err := s.advanceNextMust()
err := s.advanceNextMust(ctx)
if err != nil {
return nil, err
}
@ -160,19 +160,19 @@ func (s *PhraseSearcher) Next(preAllocated *search.DocumentMatch) (*search.Docum
return nil, nil
}
func (s *PhraseSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *PhraseSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers()
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var err error
s.currMust, err = s.mustSearcher.Advance(ID, nil)
s.currMust, err = s.mustSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
return s.Next(preAllocated)
return s.Next(ctx)
}
func (s *PhraseSearcher) Count() uint64 {
@ -195,3 +195,7 @@ func (s *PhraseSearcher) Close() error {
func (s *PhraseSearcher) Min() int {
return 0
}
func (s *PhraseSearcher) DocumentMatchPoolSize() int {
return s.mustSearcher.DocumentMatchPoolSize() + 1
}

View File

@ -69,7 +69,10 @@ func TestPhraseSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -81,7 +84,8 @@ func TestPhraseSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -106,13 +106,13 @@ func (s *RegexpSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm)
}
func (s *RegexpSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next(preAllocated)
func (s *RegexpSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return s.searcher.Next(ctx)
}
func (s *RegexpSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID, preAllocated)
func (s *RegexpSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return s.searcher.Advance(ctx, ID)
}
func (s *RegexpSearcher) Close() error {
@ -122,3 +122,7 @@ func (s *RegexpSearcher) Close() error {
func (s *RegexpSearcher) Min() int {
return 0
}
func (s *RegexpSearcher) DocumentMatchPoolSize() int {
return s.searcher.DocumentMatchPoolSize()
}

View File

@ -86,7 +86,10 @@ func TestRegexpSearch(t *testing.T) {
}
}()
next, err := test.searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize()),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
@ -98,7 +101,8 @@ func TestRegexpSearch(t *testing.T) {
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(nil)
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {

View File

@ -53,7 +53,7 @@ func (s *TermSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *TermSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *TermSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
termMatch, err := s.reader.Next(s.tfd.Reset())
if err != nil {
return nil, err
@ -64,13 +64,13 @@ func (s *TermSearcher) Next(preAllocated *search.DocumentMatch) (*search.Documen
}
// score match
docMatch := s.scorer.Score(termMatch, preAllocated)
docMatch := s.scorer.Score(ctx, termMatch)
// return doc match
return docMatch, nil
}
func (s *TermSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
func (s *TermSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
termMatch, err := s.reader.Advance(ID, s.tfd.Reset())
if err != nil {
return nil, err
@ -81,7 +81,7 @@ func (s *TermSearcher) Advance(ID index.IndexInternalID, preAllocated *search.Do
}
// score match
docMatch := s.scorer.Score(termMatch, preAllocated)
docMatch := s.scorer.Score(ctx, termMatch)
// return doc match
return docMatch, nil
@ -94,3 +94,7 @@ func (s *TermSearcher) Close() error {
func (s *TermSearcher) Min() int {
return 0
}
func (s *TermSearcher) DocumentMatchPoolSize() int {
return 1
}

View File

@ -70,13 +70,13 @@ func (s *TermPrefixSearcher) SetQueryNorm(qnorm float64) {
s.searcher.SetQueryNorm(qnorm)
}
func (s *TermPrefixSearcher) Next(preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Next(preAllocated)
func (s *TermPrefixSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return s.searcher.Next(ctx)
}
func (s *TermPrefixSearcher) Advance(ID index.IndexInternalID, preAllocated *search.DocumentMatch) (*search.DocumentMatch, error) {
return s.searcher.Advance(ID, preAllocated)
func (s *TermPrefixSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return s.searcher.Advance(ctx, ID)
}
func (s *TermPrefixSearcher) Close() error {
@ -86,3 +86,7 @@ func (s *TermPrefixSearcher) Close() error {
func (s *TermPrefixSearcher) Min() int {
return 0
}
func (s *TermPrefixSearcher) DocumentMatchPoolSize() int {
return s.searcher.DocumentMatchPoolSize()
}

View File

@ -17,6 +17,7 @@ import (
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/gtreap"
"github.com/blevesearch/bleve/index/upside_down"
"github.com/blevesearch/bleve/search"
)
func TestTermSearcher(t *testing.T) {
@ -163,14 +164,18 @@ func TestTermSearcher(t *testing.T) {
t.Errorf("expected count of 9, got %d", searcher.Count())
}
docMatch, err := searcher.Next(nil)
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(1),
}
docMatch, err := searcher.Next(ctx)
if err != nil {
t.Errorf("expected result, got %v", err)
}
if !docMatch.IndexInternalID.Equals(index.IndexInternalID("a")) {
t.Errorf("expected result ID to be 'a', got '%s", docMatch.IndexInternalID)
}
docMatch, err = searcher.Advance(index.IndexInternalID("c"), nil)
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Advance(ctx, index.IndexInternalID("c"))
if err != nil {
t.Errorf("expected result, got %v", err)
}
@ -179,7 +184,8 @@ func TestTermSearcher(t *testing.T) {
}
// try advancing past end
docMatch, err = searcher.Advance(index.IndexInternalID("z"), nil)
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Advance(ctx, index.IndexInternalID("z"))
if err != nil {
t.Fatal(err)
}
@ -188,7 +194,8 @@ func TestTermSearcher(t *testing.T) {
}
// try pushing next past end
docMatch, err = searcher.Next(nil)
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Next(ctx)
if err != nil {
t.Fatal(err)
}