1
0
Fork 0

Adding upstream version 2.5.1.

Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
Daniel Baumann 2025-05-19 00:20:02 +02:00
parent c71cb8b61d
commit 982828099e
Signed by: daniel
GPG key ID: FBB4F0E80A80222F
783 changed files with 150650 additions and 0 deletions

View file

@ -0,0 +1,117 @@
// 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 searcher
import (
"math"
"regexp"
"github.com/blevesearch/bleve/v2/analysis"
regexpTokenizer "github.com/blevesearch/bleve/v2/analysis/tokenizer/regexp"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
index "github.com/blevesearch/bleve_index_api"
)
var twoDocIndex index.Index
func init() {
twoDocIndex = initTwoDocUpsideDown()
}
func initTwoDocUpsideDown() index.Index {
analysisQueue := index.NewAnalysisQueue(1)
twoDocIndex, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
}, analysisQueue)
if err != nil {
panic(err)
}
initTwoDocs(twoDocIndex)
return twoDocIndex
}
func initTwoDocScorch(dir string) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
twoDocIndex, err := scorch.NewScorch(
scorch.Name,
map[string]interface{}{
"path": dir,
}, analysisQueue)
if err != nil {
panic(err)
}
initTwoDocs(twoDocIndex)
return twoDocIndex
}
func initTwoDocs(twoDocIndex index.Index) {
err := twoDocIndex.Open()
if err != nil {
panic(err)
}
batch := index.NewBatch()
for _, doc := range twoDocIndexDocs {
batch.Update(doc)
}
err = twoDocIndex.Batch(batch)
if err != nil {
panic(err)
}
}
// create a simpler analyzer which will support these tests
var testAnalyzer = &analysis.DefaultAnalyzer{
Tokenizer: regexpTokenizer.NewRegexpTokenizer(regexp.MustCompile(`\w+`)),
}
// sets up some mock data used in many tests in this package
var twoDocIndexDescIndexingOptions = document.DefaultTextIndexingOptions | index.IncludeTermVectors
var twoDocIndexDocs = []*document.Document{
// must have 4/4 beer
document.NewDocument("1").
AddField(document.NewTextField("name", []uint64{}, []byte("marty"))).
AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("beer beer beer beer"), twoDocIndexDescIndexingOptions, testAnalyzer)).
AddField(document.NewTextFieldWithAnalyzer("street", []uint64{}, []byte("couchbase way"), testAnalyzer)),
// must have 1/4 beer
document.NewDocument("2").
AddField(document.NewTextField("name", []uint64{}, []byte("steve"))).
AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("angst beer couch database"), twoDocIndexDescIndexingOptions, testAnalyzer)).
AddField(document.NewTextFieldWithAnalyzer("street", []uint64{}, []byte("couchbase way"), testAnalyzer)).
AddField(document.NewTextFieldWithAnalyzer("title", []uint64{}, []byte("mister"), testAnalyzer)),
// must have 1/4 beer
document.NewDocument("3").
AddField(document.NewTextField("name", []uint64{}, []byte("dustin"))).
AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("apple beer column dank"), twoDocIndexDescIndexingOptions, testAnalyzer)).
AddField(document.NewTextFieldWithAnalyzer("title", []uint64{}, []byte("mister"), testAnalyzer)),
// must have 65/65 beer
document.NewDocument("4").
AddField(document.NewTextField("name", []uint64{}, []byte("ravi"))).
AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer beer"), twoDocIndexDescIndexingOptions, testAnalyzer)),
// must have 0/x beer
document.NewDocument("5").
AddField(document.NewTextField("name", []uint64{}, []byte("bobert"))).
AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("water"), twoDocIndexDescIndexingOptions, testAnalyzer)).
AddField(document.NewTextFieldWithAnalyzer("title", []uint64{}, []byte("mister"), testAnalyzer)),
}
func scoresCloseEnough(a, b float64) bool {
return math.Abs(a-b) < 0.001
}

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,53 @@
// Copyright (c) 2023 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.
//go:build vectors
// +build vectors
package searcher
import (
"context"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func optimizeKNN(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher) error {
var octx index.VectorOptimizableContext
var err error
for _, searcher := range qsearchers {
// Only applicable to KNN Searchers.
o, ok := searcher.(index.VectorOptimizable)
if !ok {
continue
}
octx, err = o.VectorOptimize(ctx, octx)
if err != nil {
return err
}
}
// No KNN searchers.
if octx == nil {
return nil
}
// Postings lists and iterators replaced in the pointer to the
// vector reader
return octx.Finish()
}

View file

@ -0,0 +1,31 @@
// Copyright (c) 2023 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.
//go:build !vectors
// +build !vectors
package searcher
import (
"context"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func optimizeKNN(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher) error {
// No-op
return nil
}

View file

@ -0,0 +1,55 @@
// 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 searcher
import (
"github.com/blevesearch/bleve/v2/search"
)
type OrderedSearcherList []search.Searcher
// sort.Interface
func (otrl OrderedSearcherList) Len() int {
return len(otrl)
}
func (otrl OrderedSearcherList) Less(i, j int) bool {
return otrl[i].Count() < otrl[j].Count()
}
func (otrl OrderedSearcherList) Swap(i, j int) {
otrl[i], otrl[j] = otrl[j], otrl[i]
}
type OrderedPositionalSearcherList struct {
searchers []search.Searcher
index []int
}
// sort.Interface
func (otrl OrderedPositionalSearcherList) Len() int {
return len(otrl.searchers)
}
func (otrl OrderedPositionalSearcherList) Less(i, j int) bool {
return otrl.searchers[i].Count() < otrl.searchers[j].Count()
}
func (otrl OrderedPositionalSearcherList) Swap(i, j int) {
otrl.searchers[i], otrl.searchers[j] = otrl.searchers[j], otrl.searchers[i]
otrl.index[i], otrl.index[j] = otrl.index[j], otrl.index[i]
}

View file

@ -0,0 +1,451 @@
// 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 searcher
import (
"context"
"math"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeBooleanSearcher int
func init() {
var bs BooleanSearcher
reflectStaticSizeBooleanSearcher = int(reflect.TypeOf(bs).Size())
}
type BooleanSearcher struct {
indexReader index.IndexReader
mustSearcher search.Searcher
shouldSearcher search.Searcher
mustNotSearcher search.Searcher
queryNorm float64
currMust *search.DocumentMatch
currShould *search.DocumentMatch
currMustNot *search.DocumentMatch
currentID index.IndexInternalID
min uint64
scorer *scorer.ConjunctionQueryScorer
matches []*search.DocumentMatch
initialized bool
done bool
}
func NewBooleanSearcher(ctx context.Context, indexReader index.IndexReader, mustSearcher search.Searcher, shouldSearcher search.Searcher, mustNotSearcher search.Searcher, options search.SearcherOptions) (*BooleanSearcher, error) {
// build our searcher
rv := BooleanSearcher{
indexReader: indexReader,
mustSearcher: mustSearcher,
shouldSearcher: shouldSearcher,
mustNotSearcher: mustNotSearcher,
scorer: scorer.NewConjunctionQueryScorer(options),
matches: make([]*search.DocumentMatch, 2),
}
rv.computeQueryNorm()
return &rv, nil
}
func (s *BooleanSearcher) Size() int {
sizeInBytes := reflectStaticSizeBooleanSearcher + size.SizeOfPtr
if s.mustSearcher != nil {
sizeInBytes += s.mustSearcher.Size()
}
if s.shouldSearcher != nil {
sizeInBytes += s.shouldSearcher.Size()
}
if s.mustNotSearcher != nil {
sizeInBytes += s.mustNotSearcher.Size()
}
sizeInBytes += s.scorer.Size()
for _, entry := range s.matches {
if entry != nil {
sizeInBytes += entry.Size()
}
}
return sizeInBytes
}
func (s *BooleanSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
if s.mustSearcher != nil {
sumOfSquaredWeights += s.mustSearcher.Weight()
}
if s.shouldSearcher != nil {
sumOfSquaredWeights += s.shouldSearcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
if s.mustSearcher != nil {
s.mustSearcher.SetQueryNorm(s.queryNorm)
}
if s.shouldSearcher != nil {
s.shouldSearcher.SetQueryNorm(s.queryNorm)
}
}
func (s *BooleanSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
if s.mustSearcher != 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 {
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 {
if s.currMustNot != nil {
ctx.DocumentMatchPool.Put(s.currMustNot)
}
s.currMustNot, err = s.mustNotSearcher.Next(ctx)
if err != nil {
return err
}
}
if s.mustSearcher != nil && s.currMust != nil {
s.currentID = s.currMust.IndexInternalID
} else if s.mustSearcher == nil && s.currShould != nil {
s.currentID = s.currShould.IndexInternalID
} else {
s.currentID = nil
}
s.initialized = true
return nil
}
func (s *BooleanSearcher) advanceNextMust(ctx *search.SearchContext, skipReturn *search.DocumentMatch) error {
var err error
if s.mustSearcher != 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.currShould != skipReturn {
ctx.DocumentMatchPool.Put(s.currShould)
}
s.currShould, err = s.shouldSearcher.Next(ctx)
if err != nil {
return err
}
}
if s.mustSearcher != nil && s.currMust != nil {
s.currentID = s.currMust.IndexInternalID
} else if s.mustSearcher == nil && s.currShould != nil {
s.currentID = s.currShould.IndexInternalID
} else {
s.currentID = nil
}
return nil
}
func (s *BooleanSearcher) Weight() float64 {
var rv float64
if s.mustSearcher != nil {
rv += s.mustSearcher.Weight()
}
if s.shouldSearcher != nil {
rv += s.shouldSearcher.Weight()
}
return rv
}
func (s *BooleanSearcher) SetQueryNorm(qnorm float64) {
if s.mustSearcher != nil {
s.mustSearcher.SetQueryNorm(qnorm)
}
if s.shouldSearcher != nil {
s.shouldSearcher.SetQueryNorm(qnorm)
}
}
func (s *BooleanSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if s.done {
return nil, nil
}
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var err error
var rv *search.DocumentMatch
for s.currentID != nil {
if s.currMustNot != nil {
cmp := s.currMustNot.IndexInternalID.Compare(s.currentID)
if cmp < 0 {
ctx.DocumentMatchPool.Put(s.currMustNot)
// advance must not searcher to our candidate entry
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(ctx, nil)
if err != nil {
return nil, err
}
continue
}
} else if cmp == 0 {
// the candidate is excluded
err = s.advanceNextMust(ctx, nil)
if err != nil {
return nil, err
}
continue
}
}
shouldCmpOrNil := 1 // NOTE: shouldCmp will also be 1 when currShould == nil.
if s.currShould != nil {
shouldCmpOrNil = s.currShould.IndexInternalID.Compare(s.currentID)
}
if shouldCmpOrNil < 0 {
ctx.DocumentMatchPool.Put(s.currShould)
// advance should searcher to our candidate entry
s.currShould, err = s.shouldSearcher.Advance(ctx, s.currentID)
if err != nil {
return nil, err
}
if s.currShould != nil && s.currShould.IndexInternalID.Equals(s.currentID) {
// score bonus matches should
var cons []*search.DocumentMatch
if s.currMust != nil {
cons = s.matches
cons[0] = s.currMust
cons[1] = s.currShould
} else {
cons = s.matches[0:1]
cons[0] = s.currShould
}
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
cons := s.matches[0:1]
cons[0] = s.currMust
rv = s.scorer.Score(ctx, cons)
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
break
}
} else if shouldCmpOrNil == 0 {
// score bonus matches should
var cons []*search.DocumentMatch
if s.currMust != nil {
cons = s.matches
cons[0] = s.currMust
cons[1] = s.currShould
} else {
cons = s.matches[0:1]
cons[0] = s.currShould
}
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
cons := s.matches[0:1]
cons[0] = s.currMust
rv = s.scorer.Score(ctx, cons)
err = s.advanceNextMust(ctx, rv)
if err != nil {
return nil, err
}
break
}
err = s.advanceNextMust(ctx, nil)
if err != nil {
return nil, err
}
}
if rv == nil {
s.done = true
}
return rv, nil
}
func (s *BooleanSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if s.done {
return nil, nil
}
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
// Advance the searcher only if the cursor is trailing the lookup ID
if s.currentID == nil || s.currentID.Compare(ID) < 0 {
var err error
if s.mustSearcher != 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 {
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 {
// Additional check for mustNotSearcher, whose cursor isn't tracked by
// currentID to prevent it from moving when the searcher's tracked
// position is already ahead of or at the requested ID.
if s.currMustNot == nil || s.currMustNot.IndexInternalID.Compare(ID) < 0 {
if s.currMustNot != nil {
ctx.DocumentMatchPool.Put(s.currMustNot)
}
s.currMustNot, err = s.mustNotSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
}
}
if s.mustSearcher != nil && s.currMust != nil {
s.currentID = s.currMust.IndexInternalID
} else if s.mustSearcher == nil && s.currShould != nil {
s.currentID = s.currShould.IndexInternalID
} else {
s.currentID = nil
}
}
return s.Next(ctx)
}
func (s *BooleanSearcher) Count() uint64 {
// for now return a worst case
var sum uint64
if s.mustSearcher != nil {
sum += s.mustSearcher.Count()
}
if s.shouldSearcher != nil {
sum += s.shouldSearcher.Count()
}
return sum
}
func (s *BooleanSearcher) Close() error {
var err0, err1, err2 error
if s.mustSearcher != nil {
err0 = s.mustSearcher.Close()
}
if s.shouldSearcher != nil {
err1 = s.shouldSearcher.Close()
}
if s.mustNotSearcher != nil {
err2 = s.mustNotSearcher.Close()
}
if err0 != nil {
return err0
}
if err1 != nil {
return err1
}
if err2 != nil {
return err2
}
return nil
}
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

@ -0,0 +1,382 @@
// 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 searcher
import (
"context"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestBooleanSearch(t *testing.T) {
if twoDocIndex == nil {
t.Fatal("its null")
}
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
// test 0
beerTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher}, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
shouldSearcher, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher, dustinTermSearcher}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher, shouldSearcher, mustNotSearcher, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 1
martyTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
shouldSearcher2, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher2, dustinTermSearcher2}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher2, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher2}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher2, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, nil, shouldSearcher2, mustNotSearcher2, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 2
steveTermSearcher3, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher3, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher3}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher3, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, nil, nil, mustNotSearcher3, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 3
beerTermSearcher4, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher4, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher4}, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher4, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher4, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher4}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher4, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher4, nil, mustNotSearcher4, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 4
beerTermSearcher5, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher5, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher5}, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher5, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher5, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher5, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher5, martyTermSearcher5}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher5, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher5, nil, mustNotSearcher5, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 5
beerTermSearcher6, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher6, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher6}, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher6, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher6, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
shouldSearcher6, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher6, dustinTermSearcher6}, 2, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher6, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher6, shouldSearcher6, nil, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 6
beerTermSearcher7, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher7, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher7}, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher7, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher7, nil, nil, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher7, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
conjunctionSearcher7, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher7, booleanSearcher7}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 7
beerTermSearcher8, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustSearcher8, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher8}, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher8, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher8, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
shouldSearcher8, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher8, dustinTermSearcher8}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher8, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
mustNotSearcher8, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{steveTermSearcher8}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
booleanSearcher8, err := NewBooleanSearcher(context.TODO(), twoDocIndexReader, mustSearcher8, shouldSearcher8, mustNotSearcher8, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher8a, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
conjunctionSearcher8, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{booleanSearcher8, dustinTermSearcher8a}, explainTrue)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
}{
{
searcher: booleanSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 0.9818005051949021,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.808709699395535,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 0.34618161159873423,
},
},
},
{
searcher: booleanSearcher2,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 0.6775110856165737,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.6775110856165737,
},
},
},
// no MUST or SHOULD clauses yields no results
{
searcher: booleanSearcher3,
results: []*search.DocumentMatch{},
},
{
searcher: booleanSearcher4,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.5,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 1.0,
},
},
},
{
searcher: booleanSearcher5,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.5,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 1.0,
},
},
},
{
searcher: booleanSearcher6,
results: []*search.DocumentMatch{},
},
// test a conjunction query with a nested boolean
{
searcher: conjunctionSearcher7,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 2.0097428702814377,
},
},
},
{
searcher: conjunctionSearcher8,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("3"),
Score: 2.0681575785068107,
},
},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if !scoresCloseEnough(next.Score, test.results[i].Score) {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}

View file

@ -0,0 +1,285 @@
// 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 searcher
import (
"context"
"math"
"reflect"
"sort"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeConjunctionSearcher int
func init() {
var cs ConjunctionSearcher
reflectStaticSizeConjunctionSearcher = int(reflect.TypeOf(cs).Size())
}
type ConjunctionSearcher struct {
indexReader index.IndexReader
searchers []search.Searcher
queryNorm float64
currs []*search.DocumentMatch
maxIDIdx int
scorer *scorer.ConjunctionQueryScorer
initialized bool
options search.SearcherOptions
bytesRead uint64
}
func NewConjunctionSearcher(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher, options search.SearcherOptions) (
search.Searcher, error,
) {
// build the sorted downstream searchers
searchers := make(OrderedSearcherList, len(qsearchers))
copy(searchers, qsearchers)
sort.Sort(searchers)
// attempt the "unadorned" conjunction optimization only when we
// do not need extra information like freq-norm's or term vectors
if len(searchers) > 1 &&
options.Score == "none" && !options.IncludeTermVectors {
rv, err := optimizeCompositeSearcher(ctx, "conjunction:unadorned",
indexReader, searchers, options)
if err != nil || rv != nil {
return rv, err
}
}
// build our searcher
rv := ConjunctionSearcher{
indexReader: indexReader,
options: options,
searchers: searchers,
currs: make([]*search.DocumentMatch, len(searchers)),
scorer: scorer.NewConjunctionQueryScorer(options),
}
rv.computeQueryNorm()
// attempt push-down conjunction optimization when there's >1 searchers
if len(searchers) > 1 {
rv, err := optimizeCompositeSearcher(ctx, "conjunction",
indexReader, searchers, options)
if err != nil || rv != nil {
return rv, err
}
}
return &rv, nil
}
func (s *ConjunctionSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
for _, searcher := range s.searchers {
sumOfSquaredWeights += searcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
for _, searcher := range s.searchers {
searcher.SetQueryNorm(s.queryNorm)
}
}
func (s *ConjunctionSearcher) Size() int {
sizeInBytes := reflectStaticSizeConjunctionSearcher + size.SizeOfPtr +
s.scorer.Size()
for _, entry := range s.searchers {
sizeInBytes += entry.Size()
}
for _, entry := range s.currs {
if entry != nil {
sizeInBytes += entry.Size()
}
}
return sizeInBytes
}
func (s *ConjunctionSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, searcher := range s.searchers {
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return err
}
}
s.initialized = true
return nil
}
func (s *ConjunctionSearcher) Weight() float64 {
var rv float64
for _, searcher := range s.searchers {
rv += searcher.Weight()
}
return rv
}
func (s *ConjunctionSearcher) SetQueryNorm(qnorm float64) {
for _, searcher := range s.searchers {
searcher.SetQueryNorm(qnorm)
}
}
func (s *ConjunctionSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var rv *search.DocumentMatch
var err error
OUTER:
for s.maxIDIdx < len(s.currs) && s.currs[s.maxIDIdx] != nil {
maxID := s.currs[s.maxIDIdx].IndexInternalID
i := 0
for i < len(s.currs) {
if s.currs[i] == nil {
return nil, nil
}
if i == s.maxIDIdx {
i++
continue
}
cmp := maxID.Compare(s.currs[i].IndexInternalID)
if cmp == 0 {
i++
continue
}
if cmp < 0 {
// maxID < currs[i], so we found a new maxIDIdx
s.maxIDIdx = i
// advance the positions where [0 <= x < i], since we
// know they were equal to the former max entry
maxID = s.currs[s.maxIDIdx].IndexInternalID
for x := 0; x < i; x++ {
err = s.advanceChild(ctx, x, maxID)
if err != nil {
return nil, err
}
}
continue OUTER
}
// maxID > currs[i], so need to advance searchers[i]
err = s.advanceChild(ctx, i, maxID)
if err != nil {
return nil, err
}
// don't bump i, so that we'll examine the just-advanced
// currs[i] again
}
// if we get here, a doc matched all readers, so score and add it
rv = s.scorer.Score(ctx, s.currs)
// we know all the searchers are pointing at the same thing
// so they all need to be bumped
for i, searcher := range s.searchers {
if s.currs[i] != rv {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return nil, err
}
}
// don't continue now, wait for the next call to Next()
break
}
return rv, nil
}
func (s *ConjunctionSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
for i := range s.searchers {
if s.currs[i] != nil && s.currs[i].IndexInternalID.Compare(ID) >= 0 {
continue
}
err := s.advanceChild(ctx, i, ID)
if err != nil {
return nil, err
}
}
return s.Next(ctx)
}
func (s *ConjunctionSearcher) advanceChild(ctx *search.SearchContext, i int, ID index.IndexInternalID) (err error) {
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = s.searchers[i].Advance(ctx, ID)
return err
}
func (s *ConjunctionSearcher) Count() uint64 {
// for now return a worst case
var sum uint64
for _, searcher := range s.searchers {
sum += searcher.Count()
}
return sum
}
func (s *ConjunctionSearcher) Close() (rv error) {
for _, searcher := range s.searchers {
err := searcher.Close()
if err != nil && rv == nil {
rv = err
}
}
return rv
}
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

@ -0,0 +1,438 @@
// 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 searcher
import (
"context"
"os"
"strings"
"testing"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestConjunctionSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
// test 0
beerTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
beerAndMartySearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher, martyTermSearcher}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 1
angstTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "angst", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
beerTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
angstAndBeerSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{angstTermSearcher, beerTermSearcher2}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 2
beerTermSearcher3, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
jackTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "jack", "name", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
beerAndJackSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher3, jackTermSearcher}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 3
beerTermSearcher4, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
misterTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "mister", "title", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
beerAndMisterSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher4, misterTermSearcher}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 4
couchbaseTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "couchbase", "street", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
misterTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "mister", "title", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
couchbaseAndMisterSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{couchbaseTermSearcher, misterTermSearcher2}, explainTrue)
if err != nil {
t.Fatal(err)
}
// test 5
beerTermSearcher5, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "beer", "desc", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
couchbaseTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "couchbase", "street", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
misterTermSearcher3, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "mister", "title", 5.0, explainTrue)
if err != nil {
t.Fatal(err)
}
couchbaseAndMisterSearcher2, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{couchbaseTermSearcher2, misterTermSearcher3}, explainTrue)
if err != nil {
t.Fatal(err)
}
beerAndCouchbaseAndMisterSearcher, err := NewConjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{beerTermSearcher5, couchbaseAndMisterSearcher2}, explainTrue)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
}{
{
searcher: beerAndMartySearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 2.0097428702814377,
},
},
},
{
searcher: angstAndBeerSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.0807601687084403,
},
},
},
{
searcher: beerAndJackSearcher,
results: []*search.DocumentMatch{},
},
{
searcher: beerAndMisterSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.2877980334016337,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 1.2877980334016337,
},
},
},
{
searcher: couchbaseAndMisterSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.4436599157093672,
},
},
},
{
searcher: beerAndCouchbaseAndMisterSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.441614953806971,
},
},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(10, 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if !scoresCloseEnough(next.Score, test.results[i].Score) {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}
type compositeSearchOptimizationTest struct {
fieldTerms []string
expectEmpty string
}
func TestScorchCompositeSearchOptimizations(t *testing.T) {
dir, _ := os.MkdirTemp("", "scorchTwoDoc")
defer func() {
_ = os.RemoveAll(dir)
}()
twoDocIndex := initTwoDocScorch(dir)
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
tests := []compositeSearchOptimizationTest{
{
fieldTerms: []string{},
expectEmpty: "conjunction,disjunction",
},
{
fieldTerms: []string{"name:marty"},
expectEmpty: "",
},
{
fieldTerms: []string{"name:marty", "desc:beer"},
expectEmpty: "",
},
{
fieldTerms: []string{"name:marty", "name:marty"},
expectEmpty: "",
},
{
fieldTerms: []string{"name:marty", "desc:beer", "title:mister", "street:couchbase"},
expectEmpty: "conjunction",
},
{
fieldTerms: []string{"name:steve", "desc:beer", "title:mister", "street:couchbase"},
expectEmpty: "",
},
{
fieldTerms: []string{"name:NotARealName"},
expectEmpty: "conjunction,disjunction",
},
{
fieldTerms: []string{"name:NotARealName", "name:marty"},
expectEmpty: "conjunction",
},
{
fieldTerms: []string{"name:NotARealName", "name:marty", "desc:beer"},
expectEmpty: "conjunction",
},
{
fieldTerms: []string{"name:NotARealName", "name:marty", "name:marty"},
expectEmpty: "conjunction",
},
{
fieldTerms: []string{"name:NotARealName", "name:marty", "desc:beer", "title:mister", "street:couchbase"},
expectEmpty: "conjunction",
},
}
// The theme of this unit test is that given one of the above
// search test cases -- no matter what searcher options we
// provide, across either conjunctions or disjunctions, whether we
// have optimizations that are enabled or disabled, the set of doc
// ID's from the search results from any of those combinations
// should be the same.
searcherOptionsToCompare := []search.SearcherOptions{
{},
{Explain: true},
{IncludeTermVectors: true},
{IncludeTermVectors: true, Explain: true},
{Score: "none"},
{Score: "none", IncludeTermVectors: true},
{Score: "none", IncludeTermVectors: true, Explain: true},
{Score: "none", Explain: true},
}
testScorchCompositeSearchOptimizations(t, twoDocIndexReader, tests,
searcherOptionsToCompare, "conjunction")
testScorchCompositeSearchOptimizations(t, twoDocIndexReader, tests,
searcherOptionsToCompare, "disjunction")
}
func testScorchCompositeSearchOptimizations(t *testing.T, indexReader index.IndexReader,
tests []compositeSearchOptimizationTest,
searcherOptionsToCompare []search.SearcherOptions,
compositeKind string,
) {
for testi := range tests {
resultsToCompare := map[string]bool{}
testScorchCompositeSearchOptimizationsHelper(t, indexReader, tests, testi,
searcherOptionsToCompare, compositeKind, false, resultsToCompare)
testScorchCompositeSearchOptimizationsHelper(t, indexReader, tests, testi,
searcherOptionsToCompare, compositeKind, true, resultsToCompare)
}
}
func testScorchCompositeSearchOptimizationsHelper(
t *testing.T, indexReader index.IndexReader,
tests []compositeSearchOptimizationTest, testi int,
searcherOptionsToCompare []search.SearcherOptions,
compositeKind string, allowOptimizations bool, resultsToCompare map[string]bool,
) {
// Save the global allowed optimization settings to restore later.
optimizeConjunction := scorch.OptimizeConjunction
optimizeConjunctionUnadorned := scorch.OptimizeConjunctionUnadorned
optimizeDisjunctionUnadorned := scorch.OptimizeDisjunctionUnadorned
optimizeDisjunctionUnadornedMinChildCardinality := scorch.OptimizeDisjunctionUnadornedMinChildCardinality
scorch.OptimizeConjunction = allowOptimizations
scorch.OptimizeConjunctionUnadorned = allowOptimizations
scorch.OptimizeDisjunctionUnadorned = allowOptimizations
if allowOptimizations {
scorch.OptimizeDisjunctionUnadornedMinChildCardinality = uint64(0)
}
defer func() {
scorch.OptimizeConjunction = optimizeConjunction
scorch.OptimizeConjunctionUnadorned = optimizeConjunctionUnadorned
scorch.OptimizeDisjunctionUnadorned = optimizeDisjunctionUnadorned
scorch.OptimizeDisjunctionUnadornedMinChildCardinality = optimizeDisjunctionUnadornedMinChildCardinality
}()
test := tests[testi]
for searcherOptionsI, searcherOptions := range searcherOptionsToCompare {
// Construct the leaf term searchers.
var searchers []search.Searcher
for _, fieldTerm := range test.fieldTerms {
ft := strings.Split(fieldTerm, ":")
field := ft[0]
term := ft[1]
searcher, err := NewTermSearcher(context.TODO(), indexReader, term, field, 1.0, searcherOptions)
if err != nil {
t.Fatal(err)
}
searchers = append(searchers, searcher)
}
// Construct the composite searcher.
var cs search.Searcher
var err error
if compositeKind == "conjunction" {
cs, err = NewConjunctionSearcher(context.TODO(), indexReader, searchers, searcherOptions)
} else {
cs, err = NewDisjunctionSearcher(context.TODO(), indexReader, searchers, 0, searcherOptions)
}
if err != nil {
t.Fatal(err)
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(10, 0),
}
next, err := cs.Next(ctx)
i := 0
for err == nil && next != nil {
docID, err := indexReader.ExternalID(next.IndexInternalID)
if err != nil {
t.Fatal(err)
}
if searcherOptionsI == 0 && allowOptimizations == false {
resultsToCompare[string(docID)] = true
} else {
if !resultsToCompare[string(docID)] {
t.Errorf("missing %s", string(docID))
}
}
next, err = cs.Next(ctx)
if err != nil {
t.Fatalf("error iterating searcher: %v", err)
}
i++
}
if i != len(resultsToCompare) {
t.Errorf("mismatched count, %d vs %d", i, len(resultsToCompare))
}
if i == 0 && !strings.Contains(test.expectEmpty, compositeKind) {
t.Errorf("testi: %d, compositeKind: %s, allowOptimizations: %t,"+
" searcherOptionsI: %d, searcherOptions: %#v,"+
" expected some results but got no results on test: %#v",
testi, compositeKind, allowOptimizations,
searcherOptionsI, searcherOptions, test)
}
}
}

View file

@ -0,0 +1,131 @@
// Copyright (c) 2018 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 searcher
import (
"context"
"fmt"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
// DisjunctionMaxClauseCount is a compile time setting that applications can
// adjust to non-zero value to cause the DisjunctionSearcher to return an
// error instead of exeucting searches when the size exceeds this value.
var DisjunctionMaxClauseCount = 0
// DisjunctionHeapTakeover is a compile time setting that applications can
// adjust to control when the DisjunctionSearcher will switch from a simple
// slice implementation to a heap implementation.
var DisjunctionHeapTakeover = 10
func NewDisjunctionSearcher(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher, min float64, options search.SearcherOptions) (
search.Searcher, error) {
return newDisjunctionSearcher(ctx, indexReader, qsearchers, min, options, true)
}
func optionsDisjunctionOptimizable(options search.SearcherOptions) bool {
rv := options.Score == "none" && !options.IncludeTermVectors
return rv
}
func newDisjunctionSearcher(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher, min float64, options search.SearcherOptions,
limit bool) (search.Searcher, error) {
var disjOverKNN bool
if ctx != nil {
disjOverKNN, _ = ctx.Value(search.IncludeScoreBreakdownKey).(bool)
}
if disjOverKNN {
// The KNN Searcher optimization is a necessary pre-req for the KNN Searchers,
// not an optional optimization like for, say term searchers.
// It's an optimization to repeat search an open vector index when applicable,
// rather than individually opening and searching a vector index.
err := optimizeKNN(ctx, indexReader, qsearchers)
if err != nil {
return nil, err
}
} else {
// attempt the "unadorned" disjunction optimization only when we
// do not need extra information like freq-norm's or term vectors
// and the requested min is simple
if len(qsearchers) > 1 && min <= 1 &&
optionsDisjunctionOptimizable(options) {
rv, err := optimizeCompositeSearcher(ctx, "disjunction:unadorned",
indexReader, qsearchers, options)
if err != nil || rv != nil {
return rv, err
}
}
}
if len(qsearchers) > DisjunctionHeapTakeover {
return newDisjunctionHeapSearcher(ctx, indexReader, qsearchers, min, options,
limit)
}
return newDisjunctionSliceSearcher(ctx, indexReader, qsearchers, min, options,
limit)
}
func optimizeCompositeSearcher(ctx context.Context, optimizationKind string,
indexReader index.IndexReader, qsearchers []search.Searcher,
options search.SearcherOptions) (search.Searcher, error) {
var octx index.OptimizableContext
for _, searcher := range qsearchers {
o, ok := searcher.(index.Optimizable)
if !ok {
return nil, nil
}
var err error
octx, err = o.Optimize(optimizationKind, octx)
if err != nil {
return nil, err
}
if octx == nil {
return nil, nil
}
}
optimized, err := octx.Finish()
if err != nil || optimized == nil {
return nil, err
}
tfr, ok := optimized.(index.TermFieldReader)
if !ok {
return nil, nil
}
return newTermSearcherFromReader(ctx, indexReader, tfr,
[]byte(optimizationKind), "*", 1.0, options)
}
func tooManyClauses(count int) bool {
if DisjunctionMaxClauseCount != 0 && count > DisjunctionMaxClauseCount {
return true
}
return false
}
func tooManyClausesErr(field string, count int) error {
return fmt.Errorf("TooManyClauses over field: `%s` [%d > maxClauseCount,"+
" which is set to %d]", field, count, DisjunctionMaxClauseCount)
}

View file

@ -0,0 +1,367 @@
// Copyright (c) 2018 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 searcher
import (
"bytes"
"container/heap"
"context"
"math"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeDisjunctionHeapSearcher int
var reflectStaticSizeSearcherCurr int
func init() {
var dhs DisjunctionHeapSearcher
reflectStaticSizeDisjunctionHeapSearcher = int(reflect.TypeOf(dhs).Size())
var sc SearcherCurr
reflectStaticSizeSearcherCurr = int(reflect.TypeOf(sc).Size())
}
type SearcherCurr struct {
searcher search.Searcher
curr *search.DocumentMatch
matchingIdx int
}
type DisjunctionHeapSearcher struct {
indexReader index.IndexReader
numSearchers int
scorer *scorer.DisjunctionQueryScorer
min int
queryNorm float64
retrieveScoreBreakdown bool
initialized bool
searchers []search.Searcher
heap []*SearcherCurr
matching []*search.DocumentMatch
matchingIdxs []int
matchingCurrs []*SearcherCurr
bytesRead uint64
}
func newDisjunctionHeapSearcher(ctx context.Context, indexReader index.IndexReader,
searchers []search.Searcher, min float64, options search.SearcherOptions,
limit bool) (
*DisjunctionHeapSearcher, error) {
if limit && tooManyClauses(len(searchers)) {
return nil, tooManyClausesErr("", len(searchers))
}
var retrieveScoreBreakdown bool
if ctx != nil {
retrieveScoreBreakdown, _ = ctx.Value(search.IncludeScoreBreakdownKey).(bool)
}
// build our searcher
rv := DisjunctionHeapSearcher{
indexReader: indexReader,
searchers: searchers,
numSearchers: len(searchers),
scorer: scorer.NewDisjunctionQueryScorer(options),
min: int(min),
matching: make([]*search.DocumentMatch, len(searchers)),
matchingCurrs: make([]*SearcherCurr, len(searchers)),
matchingIdxs: make([]int, len(searchers)),
retrieveScoreBreakdown: retrieveScoreBreakdown,
heap: make([]*SearcherCurr, 0, len(searchers)),
}
rv.computeQueryNorm()
return &rv, nil
}
func (s *DisjunctionHeapSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
for _, searcher := range s.searchers {
sumOfSquaredWeights += searcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
for _, searcher := range s.searchers {
searcher.SetQueryNorm(s.queryNorm)
}
}
func (s *DisjunctionHeapSearcher) Size() int {
sizeInBytes := reflectStaticSizeDisjunctionHeapSearcher + size.SizeOfPtr +
s.scorer.Size()
for _, entry := range s.searchers {
sizeInBytes += entry.Size()
}
for _, entry := range s.matching {
if entry != nil {
sizeInBytes += entry.Size()
}
}
// for matchingCurrs and heap, just use static size * len
// since searchers and document matches already counted above
sizeInBytes += len(s.matchingCurrs) * reflectStaticSizeSearcherCurr
sizeInBytes += len(s.heap) * reflectStaticSizeSearcherCurr
sizeInBytes += len(s.matchingIdxs) * size.SizeOfInt
return sizeInBytes
}
func (s *DisjunctionHeapSearcher) initSearchers(ctx *search.SearchContext) error {
// alloc a single block of SearcherCurrs
block := make([]SearcherCurr, len(s.searchers))
// get all searchers pointing at their first match
for i, searcher := range s.searchers {
curr, err := searcher.Next(ctx)
if err != nil {
return err
}
if curr != nil {
block[i].searcher = searcher
block[i].curr = curr
block[i].matchingIdx = i
heap.Push(s, &block[i])
}
}
err := s.updateMatches()
if err != nil {
return err
}
s.initialized = true
return nil
}
func (s *DisjunctionHeapSearcher) updateMatches() error {
matching := s.matching[:0]
matchingCurrs := s.matchingCurrs[:0]
matchingIdxs := s.matchingIdxs[:0]
if len(s.heap) > 0 {
// top of the heap is our next hit
next := heap.Pop(s).(*SearcherCurr)
matching = append(matching, next.curr)
matchingCurrs = append(matchingCurrs, next)
matchingIdxs = append(matchingIdxs, next.matchingIdx)
// now as long as top of heap matches, keep popping
for len(s.heap) > 0 && bytes.Compare(next.curr.IndexInternalID, s.heap[0].curr.IndexInternalID) == 0 {
next = heap.Pop(s).(*SearcherCurr)
matching = append(matching, next.curr)
matchingCurrs = append(matchingCurrs, next)
matchingIdxs = append(matchingIdxs, next.matchingIdx)
}
}
s.matching = matching
s.matchingCurrs = matchingCurrs
s.matchingIdxs = matchingIdxs
return nil
}
func (s *DisjunctionHeapSearcher) Weight() float64 {
var rv float64
for _, searcher := range s.searchers {
rv += searcher.Weight()
}
return rv
}
func (s *DisjunctionHeapSearcher) SetQueryNorm(qnorm float64) {
for _, searcher := range s.searchers {
searcher.SetQueryNorm(qnorm)
}
}
func (s *DisjunctionHeapSearcher) Next(ctx *search.SearchContext) (
*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var rv *search.DocumentMatch
found := false
for !found && len(s.matching) > 0 {
if len(s.matching) >= s.min {
found = true
if s.retrieveScoreBreakdown {
// just return score and expl breakdown here, since it is a disjunction over knn searchers,
// and the final score and expl is calculated in the knn collector
rv = s.scorer.ScoreAndExplBreakdown(ctx, s.matching, s.matchingIdxs, nil, s.numSearchers)
} else {
// score this match
rv = s.scorer.Score(ctx, s.matching, len(s.matching), s.numSearchers)
}
}
// invoke next on all the matching searchers
for _, matchingCurr := range s.matchingCurrs {
if matchingCurr.curr != rv {
ctx.DocumentMatchPool.Put(matchingCurr.curr)
}
curr, err := matchingCurr.searcher.Next(ctx)
if err != nil {
return nil, err
}
if curr != nil {
matchingCurr.curr = curr
heap.Push(s, matchingCurr)
}
}
err := s.updateMatches()
if err != nil {
return nil, err
}
}
return rv, nil
}
func (s *DisjunctionHeapSearcher) Advance(ctx *search.SearchContext,
ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
// if there is anything in matching, toss it back onto the heap
for _, matchingCurr := range s.matchingCurrs {
heap.Push(s, matchingCurr)
}
s.matching = s.matching[:0]
s.matchingCurrs = s.matchingCurrs[:0]
// find all searchers that actually need to be advanced
// advance them, using s.matchingCurrs as temp storage
for len(s.heap) > 0 && bytes.Compare(s.heap[0].curr.IndexInternalID, ID) < 0 {
searcherCurr := heap.Pop(s).(*SearcherCurr)
ctx.DocumentMatchPool.Put(searcherCurr.curr)
curr, err := searcherCurr.searcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
if curr != nil {
searcherCurr.curr = curr
s.matchingCurrs = append(s.matchingCurrs, searcherCurr)
}
}
// now all of the searchers that we advanced have to be pushed back
for _, matchingCurr := range s.matchingCurrs {
heap.Push(s, matchingCurr)
}
// reset our temp space
s.matchingCurrs = s.matchingCurrs[:0]
err := s.updateMatches()
if err != nil {
return nil, err
}
return s.Next(ctx)
}
func (s *DisjunctionHeapSearcher) Count() uint64 {
// for now return a worst case
var sum uint64
for _, searcher := range s.searchers {
sum += searcher.Count()
}
return sum
}
func (s *DisjunctionHeapSearcher) Close() (rv error) {
for _, searcher := range s.searchers {
err := searcher.Close()
if err != nil && rv == nil {
rv = err
}
}
return rv
}
func (s *DisjunctionHeapSearcher) Min() int {
return s.min
}
func (s *DisjunctionHeapSearcher) DocumentMatchPoolSize() int {
rv := len(s.searchers)
for _, s := range s.searchers {
rv += s.DocumentMatchPoolSize()
}
return rv
}
// a disjunction searcher implements the index.Optimizable interface
// but only activates on an edge case where the disjunction is a
// wrapper around a single Optimizable child searcher
func (s *DisjunctionHeapSearcher) Optimize(kind string, octx index.OptimizableContext) (
index.OptimizableContext, error) {
if len(s.searchers) == 1 {
o, ok := s.searchers[0].(index.Optimizable)
if ok {
return o.Optimize(kind, octx)
}
}
return nil, nil
}
// heap impl
func (s *DisjunctionHeapSearcher) Len() int { return len(s.heap) }
func (s *DisjunctionHeapSearcher) Less(i, j int) bool {
if s.heap[i].curr == nil {
return true
} else if s.heap[j].curr == nil {
return false
}
return bytes.Compare(s.heap[i].curr.IndexInternalID, s.heap[j].curr.IndexInternalID) < 0
}
func (s *DisjunctionHeapSearcher) Swap(i, j int) {
s.heap[i], s.heap[j] = s.heap[j], s.heap[i]
}
func (s *DisjunctionHeapSearcher) Push(x interface{}) {
s.heap = append(s.heap, x.(*SearcherCurr))
}
func (s *DisjunctionHeapSearcher) Pop() interface{} {
old := s.heap
n := len(old)
x := old[n-1]
s.heap = old[0 : n-1]
return x
}

View file

@ -0,0 +1,334 @@
// Copyright (c) 2018 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 searcher
import (
"context"
"math"
"reflect"
"sort"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeDisjunctionSliceSearcher int
func init() {
var ds DisjunctionSliceSearcher
reflectStaticSizeDisjunctionSliceSearcher = int(reflect.TypeOf(ds).Size())
}
type DisjunctionSliceSearcher struct {
indexReader index.IndexReader
searchers []search.Searcher
originalPos []int
numSearchers int
queryNorm float64
retrieveScoreBreakdown bool
currs []*search.DocumentMatch
scorer *scorer.DisjunctionQueryScorer
min int
matching []*search.DocumentMatch
matchingIdxs []int
initialized bool
bytesRead uint64
}
func newDisjunctionSliceSearcher(ctx context.Context, indexReader index.IndexReader,
qsearchers []search.Searcher, min float64, options search.SearcherOptions,
limit bool) (
*DisjunctionSliceSearcher, error,
) {
if limit && tooManyClauses(len(qsearchers)) {
return nil, tooManyClausesErr("", len(qsearchers))
}
var searchers OrderedSearcherList
var originalPos []int
var retrieveScoreBreakdown bool
if ctx != nil {
retrieveScoreBreakdown, _ = ctx.Value(search.IncludeScoreBreakdownKey).(bool)
}
if retrieveScoreBreakdown {
// needed only when kNN is in picture
sortedSearchers := &OrderedPositionalSearcherList{
searchers: make([]search.Searcher, len(qsearchers)),
index: make([]int, len(qsearchers)),
}
for i, searcher := range qsearchers {
sortedSearchers.searchers[i] = searcher
sortedSearchers.index[i] = i
}
sort.Sort(sortedSearchers)
searchers = sortedSearchers.searchers
originalPos = sortedSearchers.index
} else {
searchers = make(OrderedSearcherList, len(qsearchers))
copy(searchers, qsearchers)
sort.Sort(searchers)
}
rv := DisjunctionSliceSearcher{
indexReader: indexReader,
searchers: searchers,
originalPos: originalPos,
numSearchers: len(searchers),
currs: make([]*search.DocumentMatch, len(searchers)),
scorer: scorer.NewDisjunctionQueryScorer(options),
min: int(min),
retrieveScoreBreakdown: retrieveScoreBreakdown,
matching: make([]*search.DocumentMatch, len(searchers)),
matchingIdxs: make([]int, len(searchers)),
}
rv.computeQueryNorm()
return &rv, nil
}
func (s *DisjunctionSliceSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
for _, searcher := range s.searchers {
sumOfSquaredWeights += searcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
for _, searcher := range s.searchers {
searcher.SetQueryNorm(s.queryNorm)
}
}
func (s *DisjunctionSliceSearcher) Size() int {
sizeInBytes := reflectStaticSizeDisjunctionSliceSearcher + size.SizeOfPtr +
s.scorer.Size()
for _, entry := range s.searchers {
sizeInBytes += entry.Size()
}
for _, entry := range s.currs {
if entry != nil {
sizeInBytes += entry.Size()
}
}
for _, entry := range s.matching {
if entry != nil {
sizeInBytes += entry.Size()
}
}
sizeInBytes += len(s.matchingIdxs) * size.SizeOfInt
sizeInBytes += len(s.originalPos) * size.SizeOfInt
return sizeInBytes
}
func (s *DisjunctionSliceSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, searcher := range s.searchers {
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return err
}
}
err = s.updateMatches()
if err != nil {
return err
}
s.initialized = true
return nil
}
func (s *DisjunctionSliceSearcher) updateMatches() error {
matching := s.matching[:0]
matchingIdxs := s.matchingIdxs[:0]
for i := 0; i < len(s.currs); i++ {
curr := s.currs[i]
if curr == nil {
continue
}
if len(matching) > 0 {
cmp := curr.IndexInternalID.Compare(matching[0].IndexInternalID)
if cmp > 0 {
continue
}
if cmp < 0 {
matching = matching[:0]
matchingIdxs = matchingIdxs[:0]
}
}
matching = append(matching, curr)
matchingIdxs = append(matchingIdxs, i)
}
s.matching = matching
s.matchingIdxs = matchingIdxs
return nil
}
func (s *DisjunctionSliceSearcher) Weight() float64 {
var rv float64
for _, searcher := range s.searchers {
rv += searcher.Weight()
}
return rv
}
func (s *DisjunctionSliceSearcher) SetQueryNorm(qnorm float64) {
for _, searcher := range s.searchers {
searcher.SetQueryNorm(qnorm)
}
}
func (s *DisjunctionSliceSearcher) Next(ctx *search.SearchContext) (
*search.DocumentMatch, error,
) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
var err error
var rv *search.DocumentMatch
found := false
for !found && len(s.matching) > 0 {
if len(s.matching) >= s.min {
found = true
if s.retrieveScoreBreakdown {
// just return score and expl breakdown here, since it is a disjunction over knn searchers,
// and the final score and expl is calculated in the knn collector
rv = s.scorer.ScoreAndExplBreakdown(ctx, s.matching, s.matchingIdxs, s.originalPos, s.numSearchers)
} else {
// score this match
rv = s.scorer.Score(ctx, s.matching, len(s.matching), s.numSearchers)
}
}
// invoke next on all the matching searchers
for _, i := range s.matchingIdxs {
searcher := s.searchers[i]
if s.currs[i] != rv {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return nil, err
}
}
err = s.updateMatches()
if err != nil {
return nil, err
}
}
return rv, nil
}
func (s *DisjunctionSliceSearcher) Advance(ctx *search.SearchContext,
ID index.IndexInternalID,
) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
// get all searchers pointing at their first match
var err error
for i, searcher := range s.searchers {
if s.currs[i] != nil {
if s.currs[i].IndexInternalID.Compare(ID) >= 0 {
continue
}
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = searcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
}
err = s.updateMatches()
if err != nil {
return nil, err
}
return s.Next(ctx)
}
func (s *DisjunctionSliceSearcher) Count() uint64 {
// for now return a worst case
var sum uint64
for _, searcher := range s.searchers {
sum += searcher.Count()
}
return sum
}
func (s *DisjunctionSliceSearcher) Close() (rv error) {
for _, searcher := range s.searchers {
err := searcher.Close()
if err != nil && rv == nil {
rv = err
}
}
return rv
}
func (s *DisjunctionSliceSearcher) Min() int {
return s.min
}
func (s *DisjunctionSliceSearcher) DocumentMatchPoolSize() int {
rv := len(s.currs)
for _, s := range s.searchers {
rv += s.DocumentMatchPoolSize()
}
return rv
}
// a disjunction searcher implements the index.Optimizable interface
// but only activates on an edge case where the disjunction is a
// wrapper around a single Optimizable child searcher
func (s *DisjunctionSliceSearcher) Optimize(kind string, octx index.OptimizableContext) (
index.OptimizableContext, error,
) {
if len(s.searchers) == 1 {
o, ok := s.searchers[0].(index.Optimizable)
if ok {
return o.Optimize(kind, octx)
}
}
return nil, nil
}

View file

@ -0,0 +1,223 @@
// 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 searcher
import (
"context"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestDisjunctionSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
martyTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyOrDustinSearcher, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher, dustinTermSearcher}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher2, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyOrDustinSearcher2, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher2, dustinTermSearcher2}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
raviTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "ravi", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
nestedRaviOrMartyOrDustinSearcher, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{raviTermSearcher, martyOrDustinSearcher2}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
}{
{
searcher: martyOrDustinSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 0.6775110856165737,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.6775110856165737,
},
},
},
// test a nested disjunction
{
searcher: nestedRaviOrMartyOrDustinSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 0.2765927424732821,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.2765927424732821,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 0.5531854849465642,
},
},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if !scoresCloseEnough(next.Score, test.results[i].Score) {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}
func TestDisjunctionAdvance(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
martyTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
martyOrDustinSearcher, err := NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher, dustinTermSearcher}, 0, explainTrue)
if err != nil {
t.Fatal(err)
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(martyOrDustinSearcher.DocumentMatchPoolSize(), 0),
}
match, err := martyOrDustinSearcher.Advance(ctx, index.IndexInternalID("3"))
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if match == nil {
t.Errorf("expected 3, got nil")
}
}
func TestDisjunctionSearchTooMany(t *testing.T) {
// set to max to a low non-zero value
DisjunctionMaxClauseCount = 2
defer func() {
// reset it after the test
DisjunctionMaxClauseCount = 0
}()
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
martyTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "marty", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
dustinTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "dustin", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
steveTermSearcher, err := NewTermSearcher(context.TODO(), twoDocIndexReader, "steve", "name", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
_, err = NewDisjunctionSearcher(context.TODO(), twoDocIndexReader, []search.Searcher{martyTermSearcher, dustinTermSearcher, steveTermSearcher}, 0, explainTrue)
if err == nil {
t.Fatal(err)
}
}

View file

@ -0,0 +1,110 @@
// Copyright (c) 2015 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 searcher
import (
"context"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeDocIDSearcher int
func init() {
var ds DocIDSearcher
reflectStaticSizeDocIDSearcher = int(reflect.TypeOf(ds).Size())
}
// DocIDSearcher returns documents matching a predefined set of identifiers.
type DocIDSearcher struct {
reader index.DocIDReader
scorer *scorer.ConstantScorer
count int
}
func NewDocIDSearcher(ctx context.Context, indexReader index.IndexReader, ids []string, boost float64,
options search.SearcherOptions) (searcher *DocIDSearcher, err error) {
reader, err := indexReader.DocIDReaderOnly(ids)
if err != nil {
return nil, err
}
scorer := scorer.NewConstantScorer(1.0, boost, options)
return &DocIDSearcher{
scorer: scorer,
reader: reader,
count: len(ids),
}, nil
}
func (s *DocIDSearcher) Size() int {
return reflectStaticSizeDocIDSearcher + size.SizeOfPtr +
s.reader.Size() +
s.scorer.Size()
}
func (s *DocIDSearcher) Count() uint64 {
return uint64(s.count)
}
func (s *DocIDSearcher) Weight() float64 {
return s.scorer.Weight()
}
func (s *DocIDSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *DocIDSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
docidMatch, err := s.reader.Next()
if err != nil {
return nil, err
}
if docidMatch == nil {
return nil, nil
}
docMatch := s.scorer.Score(ctx, docidMatch)
return docMatch, nil
}
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
}
if docidMatch == nil {
return nil, nil
}
docMatch := s.scorer.Score(ctx, docidMatch)
return docMatch, nil
}
func (s *DocIDSearcher) Close() error {
return s.reader.Close()
}
func (s *DocIDSearcher) Min() int {
return 0
}
func (s *DocIDSearcher) DocumentMatchPoolSize() int {
return 1
}

View file

@ -0,0 +1,146 @@
// Copyright (c) 2015 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 searcher
import (
"context"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/index/upsidedown"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func testDocIDSearcher(t *testing.T, indexed, searched, wanted []string) {
analysisQueue := index.NewAnalysisQueue(1)
i, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
for _, id := range indexed {
doc := document.NewDocument(id)
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
}
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainOff := search.SearcherOptions{Explain: false}
searcher, err := NewDocIDSearcher(context.TODO(), indexReader, searched, 1.0, explainOff)
if err != nil {
t.Fatal(err)
}
defer func() {
err := searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(searcher.DocumentMatchPoolSize(), 0),
}
// Check the sequence
for i, id := range wanted {
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(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 {
if len(id) != 2 {
t.Fatalf("expected identifier must be 2 characters long, got %v", id)
}
before := id[:1]
for _, target := range []string{before, id} {
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(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) {
testDocIDSearcher(t, nil, nil, nil)
}
func TestDocIDSearcherEmptyIndex(t *testing.T) {
testDocIDSearcher(t, nil, []string{"aa", "bb"}, nil)
}
func TestDocIDSearcherEmptySearch(t *testing.T) {
testDocIDSearcher(t, []string{"aa", "bb"}, nil, nil)
}
func TestDocIDSearcherValid(t *testing.T) {
// Test missing, out of order and duplicate inputs
testDocIDSearcher(t, []string{"aa", "bb", "cc"},
[]string{"ee", "bb", "aa", "bb"},
[]string{"aa", "bb"})
}

View file

@ -0,0 +1,104 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeFilteringSearcher int
func init() {
var fs FilteringSearcher
reflectStaticSizeFilteringSearcher = int(reflect.TypeOf(fs).Size())
}
// FilterFunc defines a function which can filter documents
// returning true means keep the document
// returning false means do not keep the document
type FilterFunc func(d *search.DocumentMatch) bool
// FilteringSearcher wraps any other searcher, but checks any Next/Advance
// call against the supplied FilterFunc
type FilteringSearcher struct {
child search.Searcher
accept FilterFunc
}
func NewFilteringSearcher(ctx context.Context, s search.Searcher, filter FilterFunc) *FilteringSearcher {
return &FilteringSearcher{
child: s,
accept: filter,
}
}
func (f *FilteringSearcher) Size() int {
return reflectStaticSizeFilteringSearcher + size.SizeOfPtr +
f.child.Size()
}
func (f *FilteringSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
next, err := f.child.Next(ctx)
for next != nil && err == nil {
if f.accept(next) {
return next, nil
}
next, err = f.child.Next(ctx)
}
return nil, err
}
func (f *FilteringSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
adv, err := f.child.Advance(ctx, ID)
if err != nil {
return nil, err
}
if adv == nil {
return nil, nil
}
if f.accept(adv) {
return adv, nil
}
return f.Next(ctx)
}
func (f *FilteringSearcher) Close() error {
return f.child.Close()
}
func (f *FilteringSearcher) Weight() float64 {
return f.child.Weight()
}
func (f *FilteringSearcher) SetQueryNorm(n float64) {
f.child.SetQueryNorm(n)
}
func (f *FilteringSearcher) Count() uint64 {
return f.child.Count()
}
func (f *FilteringSearcher) Min() int {
return f.child.Min()
}
func (f *FilteringSearcher) DocumentMatchPoolSize() int {
return f.child.DocumentMatchPoolSize()
}

View file

@ -0,0 +1,250 @@
// 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 searcher
import (
"context"
"fmt"
"strings"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
var MaxFuzziness = 2
// AutoFuzzinessHighThreshold is the threshold for the term length
// above which the fuzziness is set to MaxFuzziness when the fuzziness
// mode is set to AutoFuzziness.
var AutoFuzzinessHighThreshold = 5
// AutoFuzzinessLowThreshold is the threshold for the term length
// below which the fuzziness is set to zero when the fuzziness mode
// is set to AutoFuzziness.
// For terms with length between AutoFuzzinessLowThreshold and
// AutoFuzzinessHighThreshold, the fuzziness is set to
// MaxFuzziness - 1.
var AutoFuzzinessLowThreshold = 2
func NewFuzzySearcher(ctx context.Context, indexReader index.IndexReader, term string,
prefix, fuzziness int, field string, boost float64,
options search.SearcherOptions) (search.Searcher, error) {
if fuzziness > MaxFuzziness {
return nil, fmt.Errorf("fuzziness exceeds max (%d)", MaxFuzziness)
}
if fuzziness < 0 {
return nil, fmt.Errorf("invalid fuzziness, negative")
}
if fuzziness == 0 {
// no fuzziness, just do a term search
// check if the call is made from a phrase searcher
// and if so, add the term to the fuzzy term matches
// since the fuzzy candidate terms are not collected
// for a term search, and the only candidate term is
// the term itself
if ctx != nil {
fuzzyTermMatches := ctx.Value(search.FuzzyMatchPhraseKey)
if fuzzyTermMatches != nil {
fuzzyTermMatches.(map[string][]string)[term] = []string{term}
}
}
return NewTermSearcher(ctx, indexReader, term, field, boost, options)
}
// Note: we don't byte slice the term for a prefix because of runes.
prefixTerm := ""
for i, r := range term {
if i < prefix {
prefixTerm += string(r)
} else {
break
}
}
fuzzyCandidates, err := findFuzzyCandidateTerms(ctx, indexReader, term, fuzziness,
field, prefixTerm)
if err != nil {
return nil, err
}
var candidates []string
var editDistances []uint8
var dictBytesRead uint64
if fuzzyCandidates != nil {
candidates = fuzzyCandidates.candidates
editDistances = fuzzyCandidates.editDistances
dictBytesRead = fuzzyCandidates.bytesRead
}
if ctx != nil {
reportIOStats(ctx, dictBytesRead)
search.RecordSearchCost(ctx, search.AddM, dictBytesRead)
fuzzyTermMatches := ctx.Value(search.FuzzyMatchPhraseKey)
if fuzzyTermMatches != nil {
fuzzyTermMatches.(map[string][]string)[term] = candidates
}
}
// check if the candidates are empty or have one term which is the term itself
if len(candidates) == 0 || (len(candidates) == 1 && candidates[0] == term) {
if ctx != nil {
fuzzyTermMatches := ctx.Value(search.FuzzyMatchPhraseKey)
if fuzzyTermMatches != nil {
fuzzyTermMatches.(map[string][]string)[term] = []string{term}
}
}
return NewTermSearcher(ctx, indexReader, term, field, boost, options)
}
return NewMultiTermSearcherBoosted(ctx, indexReader, candidates, field,
boost, editDistances, options, true)
}
func GetAutoFuzziness(term string) int {
termLength := len(term)
if termLength > AutoFuzzinessHighThreshold {
return MaxFuzziness
} else if termLength > AutoFuzzinessLowThreshold {
return MaxFuzziness - 1
}
return 0
}
func NewAutoFuzzySearcher(ctx context.Context, indexReader index.IndexReader, term string,
prefix int, field string, boost float64, options search.SearcherOptions) (search.Searcher, error) {
return NewFuzzySearcher(ctx, indexReader, term, prefix, GetAutoFuzziness(term), field, boost, options)
}
type fuzzyCandidates struct {
candidates []string
editDistances []uint8
bytesRead uint64
}
func reportIOStats(ctx context.Context, bytesRead uint64) {
// The fuzzy, regexp like queries essentially load a dictionary,
// which potentially incurs a cost that must be accounted by
// using the callback to report the value.
if ctx != nil {
statsCallbackFn := ctx.Value(search.SearchIOStatsCallbackKey)
if statsCallbackFn != nil {
statsCallbackFn.(search.SearchIOStatsCallbackFunc)(bytesRead)
}
}
}
func findFuzzyCandidateTerms(ctx context.Context, indexReader index.IndexReader, term string,
fuzziness int, field, prefixTerm string) (rv *fuzzyCandidates, err error) {
rv = &fuzzyCandidates{
candidates: make([]string, 0),
editDistances: make([]uint8, 0),
}
// in case of advanced reader implementations directly call
// the levenshtein automaton based iterator to collect the
// candidate terms
if ir, ok := indexReader.(index.IndexReaderFuzzy); ok {
termSet := make(map[string]struct{})
addCandidateTerm := func(term string, editDistance uint8) error {
if _, exists := termSet[term]; !exists {
termSet[term] = struct{}{}
rv.candidates = append(rv.candidates, term)
rv.editDistances = append(rv.editDistances, editDistance)
if tooManyClauses(len(rv.candidates)) {
return tooManyClausesErr(field, len(rv.candidates))
}
}
return nil
}
fieldDict, a, err := ir.FieldDictFuzzyAutomaton(field, term, fuzziness, prefixTerm)
if err != nil {
return nil, err
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
err = addCandidateTerm(tfd.Term, tfd.EditDistance)
if err != nil {
return nil, err
}
tfd, err = fieldDict.Next()
}
if err != nil {
return nil, err
}
if ctx != nil {
if fts, ok := ctx.Value(search.FieldTermSynonymMapKey).(search.FieldTermSynonymMap); ok {
if ts, exists := fts[field]; exists {
for term := range ts {
if _, exists := termSet[term]; exists {
continue
}
if !strings.HasPrefix(term, prefixTerm) {
continue
}
match, editDistance := a.MatchAndDistance(term)
if match {
err = addCandidateTerm(term, editDistance)
if err != nil {
return nil, err
}
}
}
}
}
}
rv.bytesRead = fieldDict.BytesRead()
return rv, nil
}
var fieldDict index.FieldDict
if len(prefixTerm) > 0 {
fieldDict, err = indexReader.FieldDictPrefix(field, []byte(prefixTerm))
} else {
fieldDict, err = indexReader.FieldDict(field)
}
if err != nil {
return nil, err
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
// enumerate terms and check levenshtein distance
var reuse []int
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
var ld int
var exceeded bool
ld, exceeded, reuse = search.LevenshteinDistanceMaxReuseSlice(term, tfd.Term, fuzziness, reuse)
if !exceeded && ld <= fuzziness {
rv.candidates = append(rv.candidates, tfd.Term)
rv.editDistances = append(rv.editDistances, uint8(ld))
if tooManyClauses(len(rv.candidates)) {
return nil, tooManyClausesErr(field, len(rv.candidates))
}
}
tfd, err = fieldDict.Next()
}
rv.bytesRead = fieldDict.BytesRead()
return rv, err
}

View file

@ -0,0 +1,156 @@
// Copyright (c) 2015 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 searcher
import (
"context"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestFuzzySearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
fuzzySearcherbeet, err := NewFuzzySearcher(context.TODO(), twoDocIndexReader, "beet", 0, 1, "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
fuzzySearcherdouches, err := NewFuzzySearcher(context.TODO(), twoDocIndexReader, "douches", 0, 2, "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
fuzzySearcheraplee, err := NewFuzzySearcher(context.TODO(), twoDocIndexReader, "aplee", 0, 2, "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
fuzzySearcherprefix, err := NewFuzzySearcher(context.TODO(), twoDocIndexReader, "water", 3, 2, "desc", 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
}{
{
searcher: fuzzySearcherbeet,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("2"),
Score: 0.5,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.5,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 0.9999999838027345,
},
},
},
{
searcher: fuzzySearcherdouches,
results: []*search.DocumentMatch{},
},
{
searcher: fuzzySearcheraplee,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("3"),
Score: 0.9581453659370776,
},
},
},
{
searcher: fuzzySearcherprefix,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("5"),
Score: 1.916290731874155,
},
},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if next.Score != test.results[i].Score {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}
func TestFuzzySearchLimitErrors(t *testing.T) {
explainTrue := search.SearcherOptions{Explain: true}
_, err := NewFuzzySearcher(context.TODO(), nil, "water", 3, 3, "desc", 1.0, explainTrue)
if err == nil {
t.Fatal("`fuzziness exceeds max (2)` error expected")
}
_, err = NewFuzzySearcher(context.TODO(), nil, "water", 3, -1, "desc", 1.0, explainTrue)
if err == nil {
t.Fatal("`invalid fuzziness, negative` error expected")
}
}

View file

@ -0,0 +1,306 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/numeric"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
type filterFunc func(key []byte) bool
var (
GeoBitsShift1 = geo.GeoBits << 1
GeoBitsShift1Minus1 = GeoBitsShift1 - 1
)
func NewGeoBoundingBoxSearcher(ctx context.Context, indexReader index.IndexReader, minLon, minLat,
maxLon, maxLat float64, field string, boost float64,
options search.SearcherOptions, checkBoundaries bool) (
search.Searcher, error,
) {
if tp, ok := indexReader.(index.SpatialIndexPlugin); ok {
sp, err := tp.GetSpatialAnalyzerPlugin("s2")
if err == nil {
terms := sp.GetQueryTokens(geo.NewBoundedRectangle(minLat,
minLon, maxLat, maxLon))
boxSearcher, err := NewMultiTermSearcher(ctx, indexReader,
terms, field, boost, options, false)
if err != nil {
return nil, err
}
dvReader, err := indexReader.DocValueReader([]string{field})
if err != nil {
return nil, err
}
return NewFilteringSearcher(ctx, boxSearcher, buildRectFilter(ctx, dvReader,
field, minLon, minLat, maxLon, maxLat)), nil
}
}
// indexes without the spatial plugin override would continue here.
// track list of opened searchers, for cleanup on early exit
var openedSearchers []search.Searcher
cleanupOpenedSearchers := func() {
for _, s := range openedSearchers {
_ = s.Close()
}
}
// do math to produce list of terms needed for this search
onBoundaryTerms, notOnBoundaryTerms, err := ComputeGeoRange(context.TODO(), 0, GeoBitsShift1Minus1,
minLon, minLat, maxLon, maxLat, checkBoundaries, indexReader, field)
if err != nil {
return nil, err
}
var onBoundarySearcher search.Searcher
dvReader, err := indexReader.DocValueReader([]string{field})
if err != nil {
return nil, err
}
if len(onBoundaryTerms) > 0 {
rawOnBoundarySearcher, err := NewMultiTermSearcherBytes(ctx, indexReader,
onBoundaryTerms, field, boost, options, false)
if err != nil {
return nil, err
}
// add filter to check points near the boundary
onBoundarySearcher = NewFilteringSearcher(ctx, rawOnBoundarySearcher,
buildRectFilter(ctx, dvReader, field, minLon, minLat, maxLon, maxLat))
openedSearchers = append(openedSearchers, onBoundarySearcher)
}
var notOnBoundarySearcher search.Searcher
if len(notOnBoundaryTerms) > 0 {
var err error
notOnBoundarySearcher, err = NewMultiTermSearcherBytes(ctx, indexReader,
notOnBoundaryTerms, field, boost, options, false)
if err != nil {
cleanupOpenedSearchers()
return nil, err
}
openedSearchers = append(openedSearchers, notOnBoundarySearcher)
}
if onBoundarySearcher != nil && notOnBoundarySearcher != nil {
rv, err := NewDisjunctionSearcher(ctx, indexReader,
[]search.Searcher{
onBoundarySearcher,
notOnBoundarySearcher,
},
0, options)
if err != nil {
cleanupOpenedSearchers()
return nil, err
}
return rv, nil
} else if onBoundarySearcher != nil {
return onBoundarySearcher, nil
} else if notOnBoundarySearcher != nil {
return notOnBoundarySearcher, nil
}
return NewMatchNoneSearcher(indexReader)
}
var (
geoMaxShift = document.GeoPrecisionStep * 4
geoDetailLevel = ((geo.GeoBits << 1) - geoMaxShift) / 2
)
type closeFunc func() error
func ComputeGeoRange(ctx context.Context, term uint64, shift uint,
sminLon, sminLat, smaxLon, smaxLat float64, checkBoundaries bool,
indexReader index.IndexReader, field string) (
onBoundary [][]byte, notOnBoundary [][]byte, err error,
) {
isIndexed, closeF, err := buildIsIndexedFunc(ctx, indexReader, field)
if closeF != nil {
defer func() {
cerr := closeF()
if cerr != nil {
err = cerr
}
}()
}
grc := &geoRangeCompute{
preallocBytesLen: 32,
preallocBytes: make([]byte, 32),
sminLon: sminLon,
sminLat: sminLat,
smaxLon: smaxLon,
smaxLat: smaxLat,
checkBoundaries: checkBoundaries,
isIndexed: isIndexed,
}
grc.computeGeoRange(term, shift)
return grc.onBoundary, grc.notOnBoundary, nil
}
func buildIsIndexedFunc(ctx context.Context, indexReader index.IndexReader, field string) (isIndexed filterFunc, closeF closeFunc, err error) {
if irr, ok := indexReader.(index.IndexReaderContains); ok {
fieldDict, err := irr.FieldDictContains(field)
if err != nil {
return nil, nil, err
}
isIndexed = func(term []byte) bool {
found, err := fieldDict.Contains(term)
return err == nil && found
}
closeF = func() error {
if fd, ok := fieldDict.(index.FieldDict); ok {
err := fd.Close()
if err != nil {
return err
}
}
return nil
}
} else if indexReader != nil {
isIndexed = func(term []byte) bool {
reader, err := indexReader.TermFieldReader(ctx, term, field, false, false, false)
if err != nil || reader == nil {
return false
}
if reader.Count() == 0 {
_ = reader.Close()
return false
}
_ = reader.Close()
return true
}
} else {
isIndexed = func([]byte) bool {
return true
}
}
return isIndexed, closeF, err
}
func buildRectFilter(ctx context.Context, dvReader index.DocValueReader, field string,
minLon, minLat, maxLon, maxLat float64,
) FilterFunc {
return func(d *search.DocumentMatch) bool {
// check geo matches against all numeric type terms indexed
var lons, lats []float64
var found bool
err := dvReader.VisitDocValues(d.IndexInternalID, func(field string, term []byte) {
// only consider the values which are shifted 0
prefixCoded := numeric.PrefixCoded(term)
shift, err := prefixCoded.Shift()
if err == nil && shift == 0 {
var i64 int64
i64, err = prefixCoded.Int64()
if err == nil {
lons = append(lons, geo.MortonUnhashLon(uint64(i64)))
lats = append(lats, geo.MortonUnhashLat(uint64(i64)))
found = true
}
}
})
if err == nil && found {
bytes := dvReader.BytesRead()
if bytes > 0 {
reportIOStats(ctx, bytes)
search.RecordSearchCost(ctx, search.AddM, bytes)
}
for i := range lons {
if geo.BoundingBoxContains(lons[i], lats[i],
minLon, minLat, maxLon, maxLat) {
return true
}
}
}
return false
}
}
type geoRangeCompute struct {
preallocBytesLen int
preallocBytes []byte
sminLon, sminLat, smaxLon, smaxLat float64
checkBoundaries bool
onBoundary, notOnBoundary [][]byte
isIndexed func(term []byte) bool
}
func (grc *geoRangeCompute) makePrefixCoded(in int64, shift uint) (rv numeric.PrefixCoded) {
if len(grc.preallocBytes) <= 0 {
grc.preallocBytesLen = grc.preallocBytesLen * 2
grc.preallocBytes = make([]byte, grc.preallocBytesLen)
}
rv, grc.preallocBytes, _ = numeric.NewPrefixCodedInt64Prealloc(in, shift, grc.preallocBytes)
return rv
}
func (grc *geoRangeCompute) computeGeoRange(term uint64, shift uint) {
split := term | uint64(0x1)<<shift
var upperMax uint64
if shift < 63 {
upperMax = term | ((uint64(1) << (shift + 1)) - 1)
} else {
upperMax = 0xffffffffffffffff
}
lowerMax := split - 1
grc.relateAndRecurse(term, lowerMax, shift)
grc.relateAndRecurse(split, upperMax, shift)
}
func (grc *geoRangeCompute) relateAndRecurse(start, end uint64, res uint) {
minLon := geo.MortonUnhashLon(start)
minLat := geo.MortonUnhashLat(start)
maxLon := geo.MortonUnhashLon(end)
maxLat := geo.MortonUnhashLat(end)
level := (GeoBitsShift1 - res) >> 1
within := res%document.GeoPrecisionStep == 0 &&
geo.RectWithin(minLon, minLat, maxLon, maxLat,
grc.sminLon, grc.sminLat, grc.smaxLon, grc.smaxLat)
if within || (level == geoDetailLevel &&
geo.RectIntersects(minLon, minLat, maxLon, maxLat,
grc.sminLon, grc.sminLat, grc.smaxLon, grc.smaxLat)) {
codedTerm := grc.makePrefixCoded(int64(start), res)
if grc.isIndexed(codedTerm) {
if !within && grc.checkBoundaries {
grc.onBoundary = append(grc.onBoundary, codedTerm)
} else {
grc.notOnBoundary = append(grc.notOnBoundary, codedTerm)
}
}
} else if level < geoDetailLevel &&
geo.RectIntersects(minLon, minLat, maxLon, maxLat,
grc.sminLon, grc.sminLat, grc.smaxLon, grc.smaxLat) {
grc.computeGeoRange(start, res-1)
}
}

View file

@ -0,0 +1,317 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/upsidedown"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/numeric"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoBoundingBox(t *testing.T) {
tests := []struct {
minLon float64
minLat float64
maxLon float64
maxLat float64
field string
want []string
}{
{10.001, 10.001, 20.002, 20.002, "loc", nil},
{0.001, 0.001, 0.002, 0.002, "loc", []string{"a"}},
{0.001, 0.001, 1.002, 1.002, "loc", []string{"a", "b"}},
{0.001, 0.001, 9.002, 9.002, "loc", []string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}},
// same upper-left, bottom-right point
{25, 25, 25, 25, "loc", nil},
// box that would return points, but points reversed
{0.002, 0.002, 0.001, 0.001, "loc", nil},
}
i := setupGeo(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for _, test := range tests {
got, err := testGeoBoundingBoxSearch(indexReader, test.minLon, test.minLat, test.maxLon, test.maxLat, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for %f %f %f %f %s", test.want, got, test.minLon, test.minLat, test.maxLon, test.maxLat, test.field)
}
}
}
func testGeoBoundingBoxSearch(i index.IndexReader, minLon, minLat, maxLon, maxLat float64, field string) ([]string, error) {
var rv []string
gbs, err := NewGeoBoundingBoxSearcher(context.TODO(), i, minLon, minLat, maxLon, maxLat, field, 1.0, search.SearcherOptions{}, true)
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
rv = append(rv, string(docMatch.IndexInternalID))
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeo(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
doc := document.NewDocument("a")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 0.0015, 0.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("b")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 1.0015, 1.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("c")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 2.0015, 2.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("d")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 3.0015, 3.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("e")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 4.0015, 4.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("f")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 5.0015, 5.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("g")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 6.0015, 6.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("h")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 7.0015, 7.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("i")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 8.0015, 8.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("j")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 9.0015, 9.0015))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}
func TestComputeGeoRange(t *testing.T) {
tests := []struct {
degs float64
onBoundary int
offBoundary int
err string
}{
{0.01, 4, 0, ""},
{0.1, 56, 144, ""},
{100.0, 32768, 258560, ""},
}
for testi, test := range tests {
onBoundaryRes, offBoundaryRes, err := ComputeGeoRange(context.TODO(), 0, GeoBitsShift1Minus1,
-1.0*test.degs, -1.0*test.degs, test.degs, test.degs, true, nil, "")
if (err != nil) != (test.err != "") {
t.Errorf("test: %+v, err: %v", test, err)
}
if len(onBoundaryRes) != test.onBoundary {
t.Errorf("test: %+v, onBoundaryRes: %v", test, len(onBoundaryRes))
}
if len(offBoundaryRes) != test.offBoundary {
t.Errorf("test: %+v, offBoundaryRes: %v", test, len(offBoundaryRes))
}
onBROrig, offBROrig := origComputeGeoRange(0, GeoBitsShift1Minus1,
-1.0*test.degs, -1.0*test.degs, test.degs, test.degs, true)
if !reflect.DeepEqual(onBoundaryRes, onBROrig) {
t.Errorf("testi: %d, test: %+v, onBoundaryRes != onBROrig,\n onBoundaryRes:%v,\n onBROrig: %v",
testi, test, onBoundaryRes, onBROrig)
}
if !reflect.DeepEqual(offBoundaryRes, offBROrig) {
t.Errorf("testi: %d, test: %+v, offBoundaryRes, offBROrig,\n offBoundaryRes: %v,\n offBROrig: %v",
testi, test, offBoundaryRes, offBROrig)
}
}
}
// --------------------------------------------------------------------
func BenchmarkComputeGeoRangePt01(b *testing.B) {
onBoundary := 4
offBoundary := 0
benchmarkComputeGeoRange(b, -0.01, -0.01, 0.01, 0.01, onBoundary, offBoundary)
}
func BenchmarkComputeGeoRangePt1(b *testing.B) {
onBoundary := 56
offBoundary := 144
benchmarkComputeGeoRange(b, -0.1, -0.1, 0.1, 0.1, onBoundary, offBoundary)
}
func BenchmarkComputeGeoRange10(b *testing.B) {
onBoundary := 5464
offBoundary := 53704
benchmarkComputeGeoRange(b, -10.0, -10.0, 10.0, 10.0, onBoundary, offBoundary)
}
func BenchmarkComputeGeoRange100(b *testing.B) {
onBoundary := 32768
offBoundary := 258560
benchmarkComputeGeoRange(b, -100.0, -100.0, 100.0, 100.0, onBoundary, offBoundary)
}
// --------------------------------------------------------------------
func benchmarkComputeGeoRange(b *testing.B,
minLon, minLat, maxLon, maxLat float64, onBoundary, offBoundary int,
) {
checkBoundaries := true
b.ResetTimer()
for i := 0; i < b.N; i++ {
onBoundaryRes, offBoundaryRes, err := ComputeGeoRange(context.TODO(), 0, GeoBitsShift1Minus1, minLon, minLat, maxLon, maxLat, checkBoundaries, nil, "")
if err != nil {
b.Fatalf("expected no err")
}
if len(onBoundaryRes) != onBoundary || len(offBoundaryRes) != offBoundary {
b.Fatalf("boundaries not matching")
}
}
}
// --------------------------------------------------------------------
// original, non-optimized implementation of ComputeGeoRange
func origComputeGeoRange(term uint64, shift uint,
sminLon, sminLat, smaxLon, smaxLat float64,
checkBoundaries bool) (
onBoundary [][]byte, notOnBoundary [][]byte,
) {
split := term | uint64(0x1)<<shift
var upperMax uint64
if shift < 63 {
upperMax = term | ((uint64(1) << (shift + 1)) - 1)
} else {
upperMax = 0xffffffffffffffff
}
lowerMax := split - 1
onBoundary, notOnBoundary = origRelateAndRecurse(term, lowerMax, shift,
sminLon, sminLat, smaxLon, smaxLat, checkBoundaries)
plusOnBoundary, plusNotOnBoundary := origRelateAndRecurse(split, upperMax, shift,
sminLon, sminLat, smaxLon, smaxLat, checkBoundaries)
onBoundary = append(onBoundary, plusOnBoundary...)
notOnBoundary = append(notOnBoundary, plusNotOnBoundary...)
return
}
// original, non-optimized implementation of relateAndRecurse
func origRelateAndRecurse(start, end uint64, res uint,
sminLon, sminLat, smaxLon, smaxLat float64,
checkBoundaries bool) (
onBoundary [][]byte, notOnBoundary [][]byte,
) {
minLon := geo.MortonUnhashLon(start)
minLat := geo.MortonUnhashLat(start)
maxLon := geo.MortonUnhashLon(end)
maxLat := geo.MortonUnhashLat(end)
level := ((geo.GeoBits << 1) - res) >> 1
within := res%document.GeoPrecisionStep == 0 &&
geo.RectWithin(minLon, minLat, maxLon, maxLat,
sminLon, sminLat, smaxLon, smaxLat)
if within || (level == geoDetailLevel &&
geo.RectIntersects(minLon, minLat, maxLon, maxLat,
sminLon, sminLat, smaxLon, smaxLat)) {
if !within && checkBoundaries {
return [][]byte{
numeric.MustNewPrefixCodedInt64(int64(start), res),
}, nil
}
return nil,
[][]byte{
numeric.MustNewPrefixCodedInt64(int64(start), res),
}
} else if level < geoDetailLevel &&
geo.RectIntersects(minLon, minLat, maxLon, maxLat,
sminLon, sminLat, smaxLon, smaxLat) {
return origComputeGeoRange(start, res-1, sminLon, sminLat, smaxLon, smaxLat,
checkBoundaries)
}
return nil, nil
}

View file

@ -0,0 +1,151 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/numeric"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewGeoPointDistanceSearcher(ctx context.Context, indexReader index.IndexReader, centerLon,
centerLat, dist float64, field string, boost float64,
options search.SearcherOptions) (search.Searcher, error) {
var rectSearcher search.Searcher
if tp, ok := indexReader.(index.SpatialIndexPlugin); ok {
sp, err := tp.GetSpatialAnalyzerPlugin("s2")
if err == nil {
terms := sp.GetQueryTokens(geo.NewPointDistance(centerLat,
centerLon, dist))
rectSearcher, err = NewMultiTermSearcher(ctx, indexReader, terms,
field, boost, options, false)
if err != nil {
return nil, err
}
}
}
// indexes without the spatial plugin override would get
// initialized here.
if rectSearcher == nil {
// compute bounding box containing the circle
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat, err :=
geo.RectFromPointDistance(centerLon, centerLat, dist)
if err != nil {
return nil, err
}
// build a searcher for the box
rectSearcher, err = boxSearcher(ctx, indexReader,
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat,
field, boost, options, false)
if err != nil {
return nil, err
}
}
dvReader, err := indexReader.DocValueReader([]string{field})
if err != nil {
return nil, err
}
// wrap it in a filtering searcher which checks the actual distance
return NewFilteringSearcher(ctx, rectSearcher,
buildDistFilter(ctx, dvReader, field, centerLon, centerLat, dist)), nil
}
// boxSearcher builds a searcher for the described bounding box
// if the desired box crosses the dateline, it is automatically split into
// two boxes joined through a disjunction searcher
func boxSearcher(ctx context.Context, indexReader index.IndexReader,
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat float64,
field string, boost float64, options search.SearcherOptions, checkBoundaries bool) (
search.Searcher, error) {
if bottomRightLon < topLeftLon {
// cross date line, rewrite as two parts
leftSearcher, err := NewGeoBoundingBoxSearcher(ctx, indexReader,
-180, bottomRightLat, bottomRightLon, topLeftLat,
field, boost, options, checkBoundaries)
if err != nil {
return nil, err
}
rightSearcher, err := NewGeoBoundingBoxSearcher(ctx, indexReader,
topLeftLon, bottomRightLat, 180, topLeftLat, field, boost, options,
checkBoundaries)
if err != nil {
_ = leftSearcher.Close()
return nil, err
}
boxSearcher, err := NewDisjunctionSearcher(ctx, indexReader,
[]search.Searcher{leftSearcher, rightSearcher}, 0, options)
if err != nil {
_ = leftSearcher.Close()
_ = rightSearcher.Close()
return nil, err
}
return boxSearcher, nil
}
// build geoboundingbox searcher for that bounding box
boxSearcher, err := NewGeoBoundingBoxSearcher(ctx, indexReader,
topLeftLon, bottomRightLat, bottomRightLon, topLeftLat, field, boost,
options, checkBoundaries)
if err != nil {
return nil, err
}
return boxSearcher, nil
}
func buildDistFilter(ctx context.Context, dvReader index.DocValueReader, field string,
centerLon, centerLat, maxDist float64) FilterFunc {
return func(d *search.DocumentMatch) bool {
// check geo matches against all numeric type terms indexed
var lons, lats []float64
var found bool
err := dvReader.VisitDocValues(d.IndexInternalID, func(field string, term []byte) {
// only consider the values which are shifted 0
prefixCoded := numeric.PrefixCoded(term)
shift, err := prefixCoded.Shift()
if err == nil && shift == 0 {
i64, err := prefixCoded.Int64()
if err == nil {
lons = append(lons, geo.MortonUnhashLon(uint64(i64)))
lats = append(lats, geo.MortonUnhashLat(uint64(i64)))
found = true
}
}
})
if err == nil && found {
bytes := dvReader.BytesRead()
if bytes > 0 {
reportIOStats(ctx, bytes)
search.RecordSearchCost(ctx, search.AddM, bytes)
}
for i := range lons {
dist := geo.Haversin(lons[i], lats[i], centerLon, centerLat)
if dist <= maxDist/1000 {
return true
}
}
}
return false
}
}

View file

@ -0,0 +1,157 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoPointDistanceSearcher(t *testing.T) {
tests := []struct {
centerLon float64
centerLat float64
dist float64
field string
want []string
}{
// approx 110567m per degree at equator
{0.0, 0.0, 0, "loc", nil},
{0.0, 0.0, 110567, "loc", []string{"a"}},
{0.0, 0.0, 2 * 110567, "loc", []string{"a", "b"}},
// stretching our approximation here
{0.0, 0.0, 15 * 110567, "loc", []string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"}},
}
i := setupGeo(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for _, test := range tests {
got, err := testGeoPointDistanceSearch(indexReader, test.centerLon, test.centerLat, test.dist, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for %f %f %f %s", test.want, got, test.centerLon, test.centerLat, test.dist, test.field)
}
}
}
func testGeoPointDistanceSearch(i index.IndexReader, centerLon, centerLat, dist float64, field string) ([]string, error) {
var rv []string
gds, err := NewGeoPointDistanceSearcher(context.TODO(), i, centerLon, centerLat, dist, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gds.DocumentMatchPoolSize(), 0),
}
docMatch, err := gds.Next(ctx)
for docMatch != nil && err == nil {
rv = append(rv, string(docMatch.IndexInternalID))
docMatch, err = gds.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func TestGeoPointDistanceCompare(t *testing.T) {
tests := []struct {
docLat, docLon float64
centerLat, centerLon float64
distance string
}{
// Data points originally from MB-33454.
{
docLat: 33.718,
docLon: -116.8293,
centerLat: 39.59000587,
centerLon: -119.22998428,
distance: "10000mi",
},
{
docLat: 41.1305,
docLon: -121.6587,
centerLat: 61.28,
centerLon: -149.34,
distance: "10000mi",
},
}
for testi, test := range tests {
// compares the results from ComputeGeoRange with original, non-optimized version
compare := func(desc string,
minLon, minLat, maxLon, maxLat float64, checkBoundaries bool,
) {
// do math to produce list of terms needed for this search
onBoundaryRes, offBoundaryRes, err := ComputeGeoRange(context.TODO(), 0, GeoBitsShift1Minus1,
minLon, minLat, maxLon, maxLat, checkBoundaries, nil, "")
if err != nil {
t.Fatal(err)
}
onBROrig, offBROrig := origComputeGeoRange(0, GeoBitsShift1Minus1,
minLon, minLat, maxLon, maxLat, checkBoundaries)
if !reflect.DeepEqual(onBoundaryRes, onBROrig) {
t.Fatalf("testi: %d, test: %+v, desc: %s, onBoundaryRes != onBROrig,\n onBoundaryRes:%v,\n onBROrig: %v",
testi, test, desc, onBoundaryRes, onBROrig)
}
if !reflect.DeepEqual(offBoundaryRes, offBROrig) {
t.Fatalf("testi: %d, test: %+v, desc: %s, offBoundaryRes, offBROrig,\n offBoundaryRes: %v,\n offBROrig: %v",
testi, test, desc, offBoundaryRes, offBROrig)
}
}
// follow the general approach of the GeoPointDistanceSearcher...
dist, err := geo.ParseDistance(test.distance)
if err != nil {
t.Fatal(err)
}
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat, err := geo.RectFromPointDistance(test.centerLon, test.centerLat, dist)
if err != nil {
t.Fatal(err)
}
if bottomRightLon < topLeftLon {
// crosses date line, rewrite as two parts
compare("-180/f", -180, bottomRightLat, bottomRightLon, topLeftLat, false)
compare("-180/t", -180, bottomRightLat, bottomRightLon, topLeftLat, true)
compare("180/f", topLeftLon, bottomRightLat, 180, topLeftLat, false)
compare("180/t", topLeftLon, bottomRightLat, 180, topLeftLat, true)
} else {
compare("reg/f", topLeftLon, bottomRightLat, bottomRightLon, topLeftLat, false)
compare("reg/t", topLeftLon, bottomRightLat, bottomRightLon, topLeftLat, true)
}
}
}

View file

@ -0,0 +1,149 @@
// Copyright (c) 2019 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 searcher
import (
"context"
"fmt"
"math"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/numeric"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewGeoBoundedPolygonSearcher(ctx context.Context, indexReader index.IndexReader,
coordinates []geo.Point, field string, boost float64,
options search.SearcherOptions) (search.Searcher, error) {
if len(coordinates) < 3 {
return nil, fmt.Errorf("Too few points specified for the polygon boundary")
}
var rectSearcher search.Searcher
if sr, ok := indexReader.(index.SpatialIndexPlugin); ok {
tp, err := sr.GetSpatialAnalyzerPlugin("s2")
if err == nil {
terms := tp.GetQueryTokens(geo.NewBoundedPolygon(coordinates))
rectSearcher, err = NewMultiTermSearcher(ctx, indexReader, terms,
field, boost, options, false)
if err != nil {
return nil, err
}
}
}
// indexes without the spatial plugin override would get
// initialized here.
if rectSearcher == nil {
// compute the bounding box enclosing the polygon
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat, err :=
geo.BoundingRectangleForPolygon(coordinates)
if err != nil {
return nil, err
}
// build a searcher for the bounding box on the polygon
rectSearcher, err = boxSearcher(ctx, indexReader,
topLeftLon, topLeftLat, bottomRightLon, bottomRightLat,
field, boost, options, true)
if err != nil {
return nil, err
}
}
dvReader, err := indexReader.DocValueReader([]string{field})
if err != nil {
return nil, err
}
// wrap it in a filtering searcher that checks for the polygon inclusivity
return NewFilteringSearcher(ctx, rectSearcher,
buildPolygonFilter(ctx, dvReader, field, coordinates)), nil
}
const float64EqualityThreshold = 1e-6
func almostEqual(a, b float64) bool {
return math.Abs(a-b) <= float64EqualityThreshold
}
// buildPolygonFilter returns true if the point lies inside the
// polygon. It is based on the ray-casting technique as referred
// here: https://wrf.ecse.rpi.edu/nikola/pubdetails/pnpoly.html
func buildPolygonFilter(ctx context.Context, dvReader index.DocValueReader, field string,
coordinates []geo.Point) FilterFunc {
return func(d *search.DocumentMatch) bool {
// check geo matches against all numeric type terms indexed
var lons, lats []float64
var found bool
err := dvReader.VisitDocValues(d.IndexInternalID, func(field string, term []byte) {
// only consider the values which are shifted 0
prefixCoded := numeric.PrefixCoded(term)
shift, err := prefixCoded.Shift()
if err == nil && shift == 0 {
i64, err := prefixCoded.Int64()
if err == nil {
lons = append(lons, geo.MortonUnhashLon(uint64(i64)))
lats = append(lats, geo.MortonUnhashLat(uint64(i64)))
found = true
}
}
})
// Note: this approach works for points which are strictly inside
// the polygon. ie it might fail for certain points on the polygon boundaries.
if err == nil && found {
bytes := dvReader.BytesRead()
if bytes > 0 {
reportIOStats(ctx, bytes)
search.RecordSearchCost(ctx, search.AddM, bytes)
}
nVertices := len(coordinates)
if len(coordinates) < 3 {
return false
}
rayIntersectsSegment := func(point, a, b geo.Point) bool {
return (a.Lat > point.Lat) != (b.Lat > point.Lat) &&
point.Lon < (b.Lon-a.Lon)*(point.Lat-a.Lat)/(b.Lat-a.Lat)+a.Lon
}
for i := range lons {
pt := geo.Point{Lon: lons[i], Lat: lats[i]}
inside := rayIntersectsSegment(pt, coordinates[len(coordinates)-1], coordinates[0])
// check for a direct vertex match
if almostEqual(coordinates[0].Lat, lats[i]) &&
almostEqual(coordinates[0].Lon, lons[i]) {
return true
}
for j := 1; j < nVertices; j++ {
if almostEqual(coordinates[j].Lat, lats[i]) &&
almostEqual(coordinates[j].Lon, lons[i]) {
return true
}
if rayIntersectsSegment(pt, coordinates[j-1], coordinates[j]) {
inside = !inside
}
}
if inside {
return true
}
}
}
return false
}
}

View file

@ -0,0 +1,409 @@
// Copyright (c) 2019 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/upsidedown"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestSimpleGeoPolygons(t *testing.T) {
tests := []struct {
polygon []geo.Point
field string
want []string
}{
// test points inside a triangle & on vertices
// r, s - inside and t,u - on vertices.
{[]geo.Point{{Lon: 1.0, Lat: 1.0}, {Lon: 2.0, Lat: 1.9}, {Lon: 2.0, Lat: 1.0}}, "loc", []string{"r", "s", "t", "u"}},
// non overlapping polygon for the indexed documents
{[]geo.Point{{Lon: 3.0, Lat: 1.0}, {Lon: 4.0, Lat: 2.5}, {Lon: 3.0, Lat: 2}}, "loc", nil},
}
i := setupGeoPolygonPoints(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for _, test := range tests {
got, err := testGeoPolygonSearch(indexReader, test.polygon, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for polygon: %+v", test.want, got, test.polygon)
}
}
}
func TestRealGeoPolygons(t *testing.T) {
tests := []struct {
polygon []geo.Point
field string
want []string
}{
{[]geo.Point{
{Lon: -80.881, Lat: 35.282},
{Lon: -80.858, Lat: 35.281},
{Lon: -80.864, Lat: 35.270},
}, "loc", []string{"k", "l"}},
{[]geo.Point{
{Lon: -82.467, Lat: 36.356},
{Lon: -78.127, Lat: 36.321},
{Lon: -80.555, Lat: 32.932},
{Lon: -84.807, Lat: 33.111},
}, "loc", []string{"k", "l", "m"}},
// same polygon vertices
{[]geo.Point{{Lon: -82.467, Lat: 36.356}, {Lon: -82.467, Lat: 36.356}, {Lon: -82.467, Lat: 36.356}, {Lon: -82.467, Lat: 36.356}}, "loc", nil},
// non-overlaping polygon
{[]geo.Point{{Lon: -89.113, Lat: 36.400}, {Lon: -93.947, Lat: 36.471}, {Lon: -93.947, Lat: 34.031}}, "loc", nil},
// concave polygon with a document `n` residing inside the hands, but outside the polygon
{[]geo.Point{{Lon: -71.65, Lat: 42.446}, {Lon: -71.649, Lat: 42.428}, {Lon: -71.640, Lat: 42.445}, {Lon: -71.649, Lat: 42.435}}, "loc", nil},
// V like concave polygon with a document 'p' residing inside the bottom corner
{[]geo.Point{{Lon: -80.304, Lat: 40.740}, {Lon: -80.038, Lat: 40.239}, {Lon: -79.562, Lat: 40.786}, {Lon: -80.018, Lat: 40.328}}, "loc", []string{"p"}},
{[]geo.Point{
{Lon: -111.918, Lat: 33.515},
{Lon: -111.938, Lat: 33.494},
{Lon: -111.944, Lat: 33.481},
{Lon: -111.886, Lat: 33.517},
{Lon: -111.919, Lat: 33.468},
{Lon: -111.929, Lat: 33.508},
}, "loc", []string{"q"}},
// real points near cb bangalore
{[]geo.Point{
{Lat: 12.974872, Lon: 77.607749},
{Lat: 12.971725, Lon: 77.610110},
{Lat: 12.972530, Lon: 77.606912},
{Lat: 12.975112, Lon: 77.603780},
}, "loc", []string{"amoeba", "communiti"}},
}
i := setupGeoPolygonPoints(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for _, test := range tests {
got, err := testGeoPolygonSearch(indexReader, test.polygon, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for polygon: %+v", test.want, got, test.polygon)
}
}
}
func TestGeoRectanglePolygon(t *testing.T) {
tests := []struct {
polygon []geo.Point
field string
want []string
}{
{
[]geo.Point{{Lon: 0, Lat: 0}, {Lon: 0, Lat: 50}, {Lon: 50, Lat: 50}, {Lon: 50, Lat: 0}, {Lon: 0, Lat: 0}},
"loc",
[]string{"a", "b", "c", "d", "e", "f", "g", "h", "i", "j"},
},
}
i := setupGeo(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for _, test := range tests {
got, err := testGeoPolygonSearch(indexReader, test.polygon, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for polygon: %+v", test.want, got, test.polygon)
}
}
}
func testGeoPolygonSearch(i index.IndexReader, polygon []geo.Point, field string) ([]string, error) {
var rv []string
gbs, err := NewGeoBoundedPolygonSearcher(context.TODO(), i, polygon, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
rv = append(rv, string(docMatch.IndexInternalID))
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeoPolygonPoints(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
doc := document.NewDocument("k")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -80.86469327, 35.2782))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("l")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -80.8713, 35.28138))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("m")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -84.25, 33.153))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("n")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -89.992, 35.063))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("o")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -71.648, 42.437))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("p")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -80.016, 40.314))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("q")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, -111.919, 33.494))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("r")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 1.5, 1.1))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("s")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 2, 1.5))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("t")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 2.0, 1.9))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("u")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 2.0, 1.0))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("amoeba")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 77.60490, 12.97467))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("communiti")
doc.AddField(document.NewGeoPointField("loc", []uint64{}, 77.608237, 12.97237))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}
type geoPoint struct {
title string
lon float64
lat float64
}
// Test points inside a complex self intersecting polygon
func TestComplexGeoPolygons(t *testing.T) {
tests := []struct {
polygon []geo.Point
points []geoPoint
field string
want []string
}{
/*
/\ /\
/__\____/__\
\ /
\/
*/
// a, b, c - inside and d - on vertices.
{
[]geo.Point{
{Lon: 6.0, Lat: 2.0},
{Lon: 3.0, Lat: 4.0},
{Lon: 9.0, Lat: 6.0},
{Lon: 3.0, Lat: 8.0},
{Lon: 6.0, Lat: 10.0},
{Lon: 6.0, Lat: 2.0},
},
[]geoPoint{
{title: "a", lon: 3, lat: 4},
{title: "b", lon: 7, lat: 6},
{title: "c", lon: 4, lat: 8.1},
{title: "d", lon: 6, lat: 10.0},
{title: "e", lon: 5, lat: 6},
{title: "f", lon: 7, lat: 5},
},
"loc",
[]string{"a", "b", "c", "d"},
},
/*
____
\ /
\/
/\
/__\
*/
{
[]geo.Point{
{Lon: 7.0, Lat: 2.0},
{Lon: 1.0, Lat: 8.0},
{Lon: 1.0, Lat: 2.0},
{Lon: 7.0, Lat: 8.0},
{Lon: 7.0, Lat: 2.0},
},
[]geoPoint{
{title: "a", lon: 6, lat: 5},
{title: "b", lon: 5, lat: 5},
{title: "c", lon: 3, lat: 5.0},
{title: "d", lon: 2, lat: 4.0},
{title: "e", lon: 5, lat: 3},
{title: "f", lon: 4, lat: 4},
},
"loc",
[]string{"a", "b", "c", "d"},
},
}
for _, test := range tests {
i := setupComplexGeoPolygonPoints(t, test.points)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
got, err := testGeoPolygonSearch(indexReader, test.polygon, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected %v, got %v for polygon: %+v", test.want, got, test.polygon)
}
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}
}
func setupComplexGeoPolygonPoints(t *testing.T, points []geoPoint) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
for _, point := range points {
doc := document.NewDocument(point.title)
doc.AddField(document.NewGeoPointField("loc", []uint64{}, point.lon, point.lat))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
}
return i
}

View file

@ -0,0 +1,135 @@
// Copyright (c) 2022 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 searcher
import (
"bytes"
"context"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
"github.com/blevesearch/geo/geojson"
"github.com/blevesearch/geo/s2"
)
func NewGeoShapeSearcher(ctx context.Context, indexReader index.IndexReader, shape index.GeoJSON,
relation string, field string, boost float64,
options search.SearcherOptions,
) (search.Searcher, error) {
var err error
var spatialPlugin index.SpatialAnalyzerPlugin
// check for the spatial plugin from the index.
if sr, ok := indexReader.(index.SpatialIndexPlugin); ok {
spatialPlugin, _ = sr.GetSpatialAnalyzerPlugin("s2")
}
if spatialPlugin == nil {
// fallback to the default spatial plugin(s2).
spatialPlugin = geo.GetSpatialAnalyzerPlugin("s2")
}
// obtain the query tokens.
terms := spatialPlugin.GetQueryTokens(shape)
mSearcher, err := NewMultiTermSearcher(ctx, indexReader, terms,
field, boost, options, false)
if err != nil {
return nil, err
}
dvReader, err := indexReader.DocValueReader([]string{field})
if err != nil {
return nil, err
}
return NewFilteringSearcher(ctx, mSearcher, buildRelationFilterOnShapes(ctx, dvReader, field, relation, shape)), nil
}
// Using the same term splitter slice used in the doc values in zap.
// TODO: This needs to be revisited whenever we change the zap
// implementation of doc values.
var termSeparatorSplitSlice = []byte{0xff}
func buildRelationFilterOnShapes(ctx context.Context, dvReader index.DocValueReader, field string,
relation string, shape index.GeoJSON,
) FilterFunc {
// this is for accumulating the shape's actual complete value
// spread across multiple docvalue visitor callbacks.
var dvShapeValue []byte
var startReading, finishReading bool
var reader *bytes.Reader
var bufPool *s2.GeoBufferPool
if bufPoolCallback, ok := ctx.Value(search.GeoBufferPoolCallbackKey).(search.GeoBufferPoolCallbackFunc); ok {
bufPool = bufPoolCallback()
}
return func(d *search.DocumentMatch) bool {
var found bool
err := dvReader.VisitDocValues(d.IndexInternalID,
func(field string, term []byte) {
// only consider the values which are GlueBytes prefixed or
// if it had already started reading the shape bytes from previous callbacks.
if startReading || len(term) > geo.GlueBytesOffset {
if !startReading && bytes.Equal(geo.GlueBytes, term[:geo.GlueBytesOffset]) {
startReading = true
if bytes.Equal(geo.GlueBytes, term[len(term)-geo.GlueBytesOffset:]) {
term = term[:len(term)-geo.GlueBytesOffset]
finishReading = true
}
dvShapeValue = append(dvShapeValue, term[geo.GlueBytesOffset:]...)
} else if startReading && !finishReading {
if len(term) > geo.GlueBytesOffset &&
bytes.Equal(geo.GlueBytes, term[len(term)-geo.GlueBytesOffset:]) {
term = term[:len(term)-geo.GlueBytesOffset]
finishReading = true
}
term = append(termSeparatorSplitSlice, term...)
dvShapeValue = append(dvShapeValue, term...)
}
// apply the filter once the entire docvalue is finished reading.
if finishReading {
v, err := geojson.FilterGeoShapesOnRelation(shape, dvShapeValue, relation, &reader, bufPool)
if err == nil && v {
found = true
}
dvShapeValue = dvShapeValue[:0]
startReading = false
finishReading = false
}
}
})
if err == nil && found {
bytes := dvReader.BytesRead()
if bytes > 0 {
reportIOStats(ctx, bytes)
search.RecordSearchCost(ctx, search.AddM, bytes)
}
return found
}
return false
}
}

View file

@ -0,0 +1,541 @@
// Copyright (c) 2022 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoJsonCircleIntersectsQuery(t *testing.T) {
tests := []struct {
centrePoint []float64
radiusInMeters string
field string
want []string
}{
// test intersecting query circle for polygon1.
{
[]float64{77.68115043640137, 12.94663769274367},
"200m",
"geometry",
[]string{"polygon1"},
},
// test intersecting query circle for polygon1, circle1 and linestring1.
{
[]float64{77.68115043640137, 12.94663769274367},
"750m",
"geometry",
[]string{"polygon1", "circle1", "linestring1"},
},
// test intersecting query circle for linestring2.
{
[]float64{77.69591331481932, 12.92756503709986},
"250m",
"geometry",
[]string{"linestring2"},
},
// test intersecting query circle for circle1.
{[]float64{77.6767, 12.9422}, "250m", "geometry", []string{"circle1"}},
// test intersecting query circle for point1, envelope1 and linestring3.
{
[]float64{81.243896484375, 26.22444694563432},
"90000m",
"geometry",
[]string{"point1", "envelope1", "linestring3"},
},
// test intersecting query circle for envelope.
{
[]float64{79.98458862304688, 25.339061458818374},
"1250m",
"geometry",
[]string{"envelope1"},
},
// test intersecting query circle for multipoint.
{
[]float64{81.87346458435059, 25.41505910223247},
"200m",
"geometry",
[]string{"multipoint1"},
},
// test intersecting query circle for multilinestring.
{
[]float64{81.8669843673706, 25.512661276952272},
"90m",
"geometry",
[]string{"multilinestring1"},
},
}
i := setupGeoJsonShapesIndexForCircleQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeCircleRelationQuery("intersects",
indexReader, test.centrePoint, test.radiusInMeters, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.centrePoint)
}
}
}
func TestGeoJsonCircleWithInQuery(t *testing.T) {
tests := []struct {
centrePoint []float64
radiusInMeters string
field string
want []string
}{
// test query circle containing polygon2 and multilinestring2.
{
[]float64{81.85981750488281, 25.546778150624146},
"3700m",
"geometry",
[]string{"polygon2", "multilinestring2"},
},
// test query circle containing multilinestring2.
{
[]float64{81.85981750488281, 25.546778150624146},
"3250m",
"geometry",
[]string{"multilinestring2"},
},
// test query circle containing multipoint1.
{
[]float64{81.88599586486816, 25.425756968727935},
"1650m",
"geometry",
[]string{"multipoint1"},
},
// test query circle containing circle2.
{
[]float64{82.09362030029297, 25.546313513788725},
"1280m",
"geometry",
[]string{"envelope2", "circle2"},
},
// test query circle containing envelope2 and circle2.
{
[]float64{82.10289001464844, 25.544919592476727},
"700m",
"geometry",
[]string{"envelope2", "circle2"},
},
// test query circle containing point1 and linestring3.
{
[]float64{81.27685546875, 26.1899475672235},
"5600m",
"geometry",
[]string{"point1", "linestring3"},
},
}
i := setupGeoJsonShapesIndexForCircleQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeCircleRelationQuery("within", indexReader, test.centrePoint, test.radiusInMeters, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v", n, test.want, got, test.centrePoint)
}
}
}
func TestGeoJsonCircleContainsQuery(t *testing.T) {
tests := []struct {
centrePoint []float64
radiusInMeters string
field string
want []string
}{
// test query circle within polygon3.
{
[]float64{8.549551963806152, 47.3759038562437},
"180m",
"geometry",
[]string{"polygon3"},
},
// test query circle containing envelope3.
{
[]float64{8.551011085510254, 47.380117626829275},
"75m",
"geometry",
[]string{"envelope3"},
},
// test query circle exceeding envelope3 with a few meters.
{
[]float64{8.551011085510254, 47.380117626829275},
"78m",
"geometry", nil,
},
// test query circle containing circle3.
{
[]float64{8.535819053649902, 47.38297989270074},
"185m",
"geometry",
[]string{"circle3"},
},
}
i := setupGeoJsonShapesIndexForCircleQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeCircleRelationQuery("contains",
indexReader, test.centrePoint, test.radiusInMeters, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.centrePoint)
}
}
}
func runGeoShapeCircleRelationQuery(relation string, i index.IndexReader,
points []float64, radius string, field string,
) ([]string, error) {
var rv []string
s := geo.NewGeoCircle(points, radius)
gbs, err := NewGeoShapeSearcher(context.TODO(), i, s, relation, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
docID, _ := i.ExternalID(docMatch.IndexInternalID)
rv = append(rv, docID)
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeoJsonShapesIndexForCircleQuery(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := scorch.NewScorch(
gtreap.Name,
map[string]interface{}{
"path": "",
"spatialPlugin": "s2",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
polygon1 := [][][][]float64{{{
{77.67248153686523, 12.957679089615821},
{77.67956256866455, 12.948101542434257},
{77.68908977508545, 12.948896200093982},
{77.68934726715086, 12.955211547173878},
{77.68016338348389, 12.954291440344619},
{77.67248153686523, 12.957679089615821},
}}}
doc := document.NewDocument("polygon1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon1, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon2 := [][][][]float64{{{
{81.84951782226561, 25.522692102524033},
{81.8557834625244, 25.521762640415535},
{81.86264991760254, 25.521762640415535},
{81.86676979064941, 25.521607729364224},
{81.89560890197754, 25.542673796271302},
{81.88977241516113, 25.543293330460937},
{81.84951782226561, 25.522692102524033},
}}}
doc = document.NewDocument("polygon2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon2, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon3 := [][][][]float64{{{
{8.548071384429932, 47.379216780040124},
{8.547642230987549, 47.3771680227784},
{8.545818328857422, 47.37677569847655},
{8.546290397644043, 47.37417465983494},
{8.551719188690186, 47.37417465983494},
{8.553242683410645, 47.37679022905829},
{8.548071384429932, 47.379216780040124},
}}}
doc = document.NewDocument("polygon3")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon3, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
point1 := [][][][]float64{{{{81.2439, 26.2244}}}}
doc = document.NewDocument("point1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
point1, "point", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope1 := [][][][]float64{{{
{79.9969482421875, 23.895882703682627},
{80.7220458984375, 25.750424835909385},
}}}
doc = document.NewDocument("envelope1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
envelope1, "envelope", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope2 := [][][][]float64{{{
{82.10409164428711, 25.54360309635522},
{82.10537910461424, 25.544609829984058},
}}}
doc = document.NewDocument("envelope2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
envelope2, "envelope", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope3 := [][][][]float64{{{
{8.545668125152588, 47.37942019840244},
{8.552148342132568, 47.383778974713124},
}}}
doc = document.NewDocument("envelope3")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
envelope3, "envelope", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle1")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{77.67252445220947, 12.936348678099293}, "900m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle2")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{82.10289001464844, 25.544919592476727}, "100m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle3")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{
8.53363037109375,
47.38191927423153,
}, "400m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring := [][][][]float64{{{
{77.68715858459473, 12.944755587650944},
{77.69213676452637, 12.945090185150542},
}}}
doc = document.NewDocument("linestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring1 := [][][][]float64{{{
{77.68913269042969, 12.929614580987227},
{77.70252227783203, 12.929698235482276},
}}}
doc = document.NewDocument("linestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring1, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring2 := [][][][]float64{{{
{81.26792907714844, 26.170845301716813},
{81.30157470703125, 26.18440207077121},
}}}
doc = document.NewDocument("linestring3")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring2, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring := [][][][]float64{{{
{81.86170578002928, 25.430407918899984},
{81.86273574829102, 25.421958559611397},
}, {
{81.88230514526367, 25.437616536907512},
{81.90084457397461, 25.431415601111418},
}, {
{81.86805725097656, 25.514868905100244},
{81.86702728271484, 25.502474677473746},
}}}
doc = document.NewDocument("multilinestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multilinestring, "multilinestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring1 := [][][][]float64{{
{
{81.84642791748047, 25.561335859046192},
{81.84230804443358, 25.550495180470026},
},
{{81.87423706054688, 25.55142441992021}, {81.88453674316406, 25.555141305670045}},
{{81.8642807006836, 25.572175556682115}, {81.87458038330078, 25.567839795359724}},
}}
doc = document.NewDocument("multilinestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multilinestring1, "multilinestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipoint1 := [][][][]float64{{{
{81.87337875366211, 25.432268248708212},
{81.87355041503906, 25.416299483230368},
{81.90118789672852, 25.426067037656946},
}}}
doc = document.NewDocument("multipoint1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multipoint1, "multipoint", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygonWithHole1 := [][][][]float64{{
{
{77.59991168975829, 12.972232910164502},
{77.6039457321167, 12.97582941279006},
{77.60424613952637, 12.98168407323241},
{77.59974002838135, 12.985489528568463},
{77.59321689605713, 12.979300406693417},
{77.59991168975829, 12.972232910164502},
},
{
{77.59682178497314, 12.975787593290978},
{77.60295867919922, 12.975787593290978},
{77.60295867919922, 12.98143316204164},
{77.59682178497314, 12.98143316204164},
{77.59682178497314, 12.975787593290978},
},
}}
doc = document.NewDocument("polygonWithHole1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygonWithHole1, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}

View file

@ -0,0 +1,520 @@
// Copyright (c) 2022 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoJsonEnvelopeWithInQuery(t *testing.T) {
tests := []struct {
points [][]float64
field string
want []string
}{
// test within query envelope for point1.
{
[][]float64{
{76.256103515625, 16.76772739719064},
{76.35772705078125, 16.872890378907783},
},
"geometry",
[]string{"point1"},
},
// test within query envelope for multipoint1.
{
[][]float64{
{81.046142578125, 17.156537255486093},
{81.331787109375, 17.96305758238804},
},
"geometry",
[]string{"multipoint1"},
},
// test within query envelope for partial points in a multipoint1.
{
[][]float64{
{81.05987548828125, 17.16178591271515},
{81.36199951171875, 17.861132899477624},
},
"geometry", nil,
},
// test within query envelope for polygon2 and point1.
{
[][]float64{
{76.00341796875, 16.573022719182777},
{76.717529296875, 17.006888277600524},
},
"geometry",
[]string{"polygon2", "point1"},
},
// test within query envelope for linestring1.
{
[][]float64{
{76.84112548828125, 16.86500518090961},
{77.62115478515625, 17.531439701706244},
},
"geometry",
[]string{"linestring1"},
},
// test within query envelope for multilinestring1.
{
[][]float64{
{81.683349609375, 17.104042525557904},
{81.99234008789062, 17.66495983051931},
},
"geometry",
[]string{"multilinestring1"},
},
// test within query envelope that is intersecting multilinestring1.
{
[][]float64{
{81.65725708007812, 17.2601707001208},
{81.95114135742186, 17.66495983051931},
},
"geometry", nil,
},
// test within query envelope for envelope1 and circle1.
{
[][]float64{
{74.75372314453125, 17.36636733709516},
{75.509033203125, 18.038809662036805},
},
"geometry",
[]string{"envelope1", "circle1"},
},
// test within query envelope for envelope1.
{
[][]float64{
{74.783935546875, 17.38209494787749},
{75.96221923828125, 17.727758609852284},
},
"geometry",
[]string{"envelope1"},
},
}
i := setupGeoJsonShapesIndexForEnvelopeQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeEnvelopeRelationQuery("within",
indexReader, test.points, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.points)
}
}
}
func TestGeoJsonEnvelopeIntersectsQuery(t *testing.T) {
tests := []struct {
points [][]float64
field string
want []string
}{
// test intersecting query envelope for partial points in a multipoint1.
{
[][]float64{
{81.00769042968749, 17.80622614478282},
{81.199951171875, 17.983957957423037},
},
"geometry",
[]string{"multipoint1"},
},
// test intersecting query envelope that is intersecting multilinestring1.
{
[][]float64{
{81.65725708007812, 17.2601707001208},
{81.95114135742186, 17.66495983051931},
},
"geometry",
[]string{"multilinestring1"},
},
// test intersecting query envelope for linestring2.
{
[][]float64{
{81.9854736328125, 18.27369419984127},
{82.14752197265625, 18.633232565431218},
},
"geometry",
[]string{"linestring2"},
},
// test intersecting query envelope for circle2.
{
[][]float64{
{82.6336669921875, 17.82714499951342},
{82.66387939453125, 17.861132899477624},
},
"geometry",
[]string{"circle2"},
},
// test intersecting query envelope for polygon3.
{
[][]float64{
{82.92343139648438, 17.739530934289657},
{82.98797607421874, 17.79184300887134},
},
"geometry",
[]string{"polygon3"},
},
}
i := setupGeoJsonShapesIndexForEnvelopeQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeEnvelopeRelationQuery("intersects", indexReader, test.points, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v", n, test.want, got, test.points)
}
}
}
func TestGeoJsonEnvelopeContainsQuery(t *testing.T) {
tests := []struct {
points [][]float64
field string
want []string
}{
// test envelope contained within polygon1.
{
[][]float64{
{8.548285961151123, 47.376092756617446},
{8.551225662231445, 47.37764752629426},
},
"geometry",
[]string{"polygon1"},
},
// test envelope partially contained within polygon1.
{
[][]float64{
{8.549273014068604, 47.376194471922986},
{8.551654815673828, 47.37827232736301},
},
"geometry", nil,
},
// test envelope partially contained within polygon1.
{
[][]float64{
{8.549273014068604, 47.376194471922986},
{8.551654815673828, 47.37827232736301},
},
"geometry", nil,
},
// test envelope fully contained within circle3.
{
[][]float64{
{8.532772064208984, 47.380379160110856},
{8.534531593322752, 47.38299442157271},
},
"geometry",
[]string{"circle3"},
},
// test envelope partially contained within circle3.
{
[][]float64{
{8.532836437225342, 47.38010309716447},
{8.538415431976318, 47.383081594720466},
},
"geometry", nil,
},
}
i := setupGeoJsonShapesIndexForEnvelopeQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeEnvelopeRelationQuery("contains",
indexReader, test.points, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.points)
}
}
}
func runGeoShapeEnvelopeRelationQuery(relation string, i index.IndexReader,
points [][]float64, field string,
) ([]string, error) {
var rv []string
s := geo.NewGeoEnvelope(points)
gbs, err := NewGeoShapeSearcher(context.TODO(), i, s, relation, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
docID, _ := i.ExternalID(docMatch.IndexInternalID)
rv = append(rv, docID)
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeoJsonShapesIndexForEnvelopeQuery(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := scorch.NewScorch(
gtreap.Name,
map[string]interface{}{
"path": "",
"spatialPlugin": "s2",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
polygon1 := [][][][]float64{{{
{8.548071384429932, 47.379216780040124},
{8.547642230987549, 47.3771680227784},
{8.545818328857422, 47.37677569847655},
{8.546290397644043, 47.37417465983494},
{8.551719188690186, 47.37417465983494},
{8.553242683410645, 47.37679022905829},
{8.548071384429932, 47.379216780040124},
}}}
doc := document.NewDocument("polygon1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon1, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon2 := [][][][]float64{{{
{76.70379638671874, 16.828203242420393},
{76.36322021484375, 16.58881695544584},
{76.70928955078125, 16.720385051694},
{76.70379638671874, 16.828203242420393},
}}}
doc = document.NewDocument("polygon2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon2, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon3 := [][][][]float64{{{
{82.9522705078125, 17.749994573141873},
{82.94952392578125, 17.692436998627272},
{82.87673950195312, 17.64009591883757},
{82.76412963867188, 17.58643052828743},
{82.8094482421875, 17.522272941245202},
{82.99621582031249, 17.64009591883757},
{82.9522705078125, 17.749994573141873},
}}}
doc = document.NewDocument("polygon3")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon3, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope1 := [][][][]float64{{{
{74.89654541015625, 17.403062993328923},
{74.92401123046875, 17.66495983051931},
}}}
doc = document.NewDocument("envelope1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
envelope1, "envelope", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle1")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{75.0531005859375, 17.675427818339383}, "12900m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle2")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{82.69683837890625, 17.902955242676995}, "6000m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle3")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{
8.53363037109375,
47.38191927423153,
}, "400m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
point1 := [][][][]float64{{{{76.29730224609375, 16.796653031618053}}}}
doc = document.NewDocument("point1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
point1, "point",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring1 := [][][][]float64{{{
{76.85211181640624, 17.51048642597462},
{77.24212646484374, 16.93070509876554},
}}}
doc = document.NewDocument("linestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring1, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring2 := [][][][]float64{{{
{81.89208984375, 18.555136195095105},
{82.21343994140625, 18.059701055000478},
}}}
doc = document.NewDocument("linestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring2, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipoint1 := [][][][]float64{{{
{81.24938964843749, 17.602139123350838},
{81.30432128906249, 17.56548361143177},
{81.29058837890625, 17.180155043474496},
{81.09283447265625, 17.87681743233167},
}}}
doc = document.NewDocument("multipoint1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multipoint1, "multipoint", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring := [][][][]float64{{
{
{81.69708251953125, 17.641404631355755},
{81.90994262695312, 17.642713334367667},
},
{{81.6998291015625, 17.620464090732245}, {81.69708251953125, 17.468572623463153}},
{{81.70120239257811, 17.458092664041494}, {81.81243896484375, 17.311310073048123}},
{{81.815185546875, 17.3034434020238}, {81.81243896484375, 17.109292665395643}},
}}
doc = document.NewDocument("multilinestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multilinestring, "multilinestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring1 := [][][][]float64{{
{
{77.6015853881836, 12.990089451715061},
{77.60476112365723, 12.987747683302153},
},
{{77.59875297546387, 12.988751301039581}, {77.59446144104004, 12.98197680263484}},
{{77.60188579559325, 12.982604078764705}, {77.60557651519775, 12.987329508048184}},
}}
doc = document.NewDocument("multilinestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multilinestring1, "multilinestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}

View file

@ -0,0 +1,692 @@
// Copyright (c) 2022 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoJSONIntersectsQueryAgainstGeometryCollection(t *testing.T) {
tests := []struct {
points [][][][][]float64
types []string
field string
want []string
}{
// test intersects geometrycollection query for gc_polygon1_linestring1.
{
[][][][][]float64{
{{{
{-120.80017089843749, 36.54053616262899},
{-120.67932128906249, 36.33725319397006},
{-120.30578613281251, 36.90597988519294},
{-120.80017089843749, 36.54053616262899},
}}},
{{{{-118.24584960937499, 35.32184842037683}, {-117.8668212890625, 35.06597313798418}}}},
},
[]string{"polygon", "linestring"},
"geometry",
[]string{"gc_polygon1_linestring1"},
},
// test intersects geometrycollection query for gc_polygon1_linestring1.
{
[][][][][]float64{
{{
{{-118.3172607421875, 35.250105158539355}, {-117.50976562499999, 35.37561413174875}},
{{-118.69628906249999, 34.6241677899049}, {-118.3172607421875, 35.03899204678081}},
{{-117.94921874999999, 35.146862906756304}, {-117.674560546875, 34.41144164327245}},
}},
{{{
{-117.04284667968749, 35.263561862152095},
{-116.8505859375, 35.263561862152095},
{-116.8505859375, 35.33529320309328},
{-117.04284667968749, 35.33529320309328},
{-117.04284667968749, 35.263561862152095},
}}},
},
[]string{"multilinestring", "polygon"},
"geometry",
[]string{"gc_polygon1_linestring1"},
},
// test intersects geometrycollection query for gc_multipolygon1_multilinestring1.
{
[][][][][]float64{
{{
{{-115.8563232421875, 38.53957267203905}, {-115.58166503906251, 38.54816542304656}},
{{-115.8343505859375, 38.45789034424927}, {-115.81237792968749, 38.19502155795575}},
}},
{{{{-116.64905548095702, 37.94920616351679}}}},
},
[]string{"multilinestring", "point"},
"geometry",
[]string{"gc_multipolygon1_multilinestring1"},
},
// test intersects geometrycollection query for gc_polygon1_linestring1 and gc_multipolygon1_multilinestring1.
{
[][][][][]float64{
{{{{-116.64905548095702, 37.94920616351679}, {-118.29528808593751, 34.52466147177172}}}},
{{
{{-115.8563232421875, 38.53957267203905}, {-115.58166503906251, 38.54816542304656}},
{{-115.8343505859375, 38.45789034424927}, {-115.81237792968749, 38.19502155795575}},
}},
},
[]string{"multipoint", "multilinestring"},
"geometry",
[]string{
"gc_polygon1_linestring1",
"gc_multipolygon1_multilinestring1",
},
},
// test intersects geometrycollection query for gc_polygon1_linestring1 and gc_multipolygon1_multilinestring1.
{
[][][][][]float64{
{{{
{-117.46582031249999, 36.146746777814364},
{-116.70227050781249, 36.146746777814364},
{-116.70227050781249, 36.69485094156225},
{-117.46582031249999, 36.69485094156225},
{-117.46582031249999, 36.146746777814364},
}}, {{
{-115.5267333984375, 38.06106741381201},
{-115.4937744140625, 37.18220222107978},
{-114.93896484374999, 37.304644804751106},
{-115.5267333984375, 38.06106741381201},
}}},
{{
{{-115.8563232421875, 38.53957267203905}, {-115.58166503906251, 38.54816542304656}},
{{-115.8343505859375, 38.45789034424927}, {-115.81237792968749, 38.19502155795575}},
}},
},
[]string{"multipolygon", "multilinestring"},
"geometry",
[]string{"gc_point1_multipoint1"},
},
}
i := setupGeoJsonShapesIndexForGeometryCollectionQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeGeometryCollectionRelationQuery("intersects",
indexReader, test.points, test.types, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.points)
}
}
}
func TestGeoJSONWithInQueryAgainstGeometryCollection(t *testing.T) {
tests := []struct {
points [][][][][]float64
types []string
field string
want []string
}{
// test within geometrycollection query for gc_multipoint2_multipolygon2_multiline2.
{
[][][][][]float64{
{{{{-122.40434646606444, 37.73400071182758}, {-122.39730834960938, 37.73691949864062}}}},
{{{
{-122.42511749267578, 37.760808496517235},
{-122.42314338684082, 37.74248523826606},
{-122.40082740783691, 37.756669194195815},
{-122.42511749267578, 37.760808496517235},
}}},
{
{{
{-122.46339797973633, 37.76637243960179},
{-122.46176719665527, 37.7502901437285},
{-122.43644714355469, 37.75911208915015},
{-122.46339797973633, 37.76637243960179},
}},
{{
{-122.43653297424315, 37.714720253587004},
{-122.40563392639159, 37.714720253587004},
{-122.40563392639159, 37.72904529863455},
{-122.43653297424315, 37.72904529863455},
{-122.43653297424315, 37.714720253587004},
}},
},
},
[]string{"linestring", "polygon", "multipolygon"},
"geometry",
[]string{"gc_multipoint2_multipolygon2_multiline2"},
},
// test within geometrycollection query.
{
[][][][][]float64{
{{{{-122.40434646606444, 37.73400071182758}, {-122.39730834960938, 37.73691949864062}}}},
{
{{
{-122.46339797973633, 37.76637243960179},
{-122.46176719665527, 37.7502901437285},
{-122.43644714355469, 37.75911208915015},
{-122.46339797973633, 37.76637243960179},
}},
{{
{-122.43653297424315, 37.714720253587004},
{-122.40563392639159, 37.714720253587004},
{-122.40563392639159, 37.72904529863455},
{-122.43653297424315, 37.72904529863455},
{-122.43653297424315, 37.714720253587004},
}},
},
},
[]string{"linestring", "multipolygon"},
"geometry", nil,
},
// test within geometrycollection for gc_multipoint2_multipolygon2_multiline2.
{
[][][][][]float64{
{{{
{-122.4491500854492, 37.78170504295941},
{-122.4862289428711, 37.747371884118664},
{-122.43078231811525, 37.6949593672454},
{-122.3799705505371, 37.72945260537779},
{-122.3928451538086, 37.78007695280165},
{-122.4491500854492, 37.78170504295941},
}}},
},
[]string{"polygon"},
"geometry",
[]string{"gc_multipoint2_multipolygon2_multiline2"},
},
// test within geometrycollection for gc_multipolygon3
// gc_multipolygon3's multipolygons within the geometrycollection is covered by the
// query's geometric collection of a polygon and a multipolygon.
{
[][][][][]float64{
{{{
{86.6162109375, 57.26716357153586},
{85.1220703125, 8119},
{84.462890625, 56.27996083172844},
{86.98974609375, 55.70235509327093},
{87.802734375, 56.77680831656842},
{86.6162109375, 57.26716357153586},
}}},
{
{{
{75.1025390625, 54.3549556895541},
{73.1689453125, 54.29088164657006},
{72.7294921875, 53.08082737207479},
{74.091796875, 51.998410382390325},
{76.79443359375, 53.396432127095984},
{75.1025390625, 54.3549556895541},
}},
{{
{80.1123046875, 55.57834467218206},
{78.9697265625, 55.65279803318956},
{78.5302734375, 54.635697306063854},
{79.87060546875, 54.18815548107151},
{80.96923828125, 54.80068486732233},
{80.1123046875, 55.57834467218206},
}},
},
},
[]string{"polygon", "multipolygon"},
"geometry",
[]string{"gc_multipolygon3"},
},
}
i := setupGeoJsonShapesIndexForGeometryCollectionQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeGeometryCollectionRelationQuery("within", indexReader, test.points, test.types, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v", n, test.want, got, test.points)
}
}
}
func TestGeoJSONContainsQueryAgainstGeometryCollection(t *testing.T) {
tests := []struct {
points [][][][][]float64
types []string
field string
want []string
}{
// test contains for a geometrycollection that comprises of a linestring,
// polygon, multipolygon, point and multipoint for polygon2.
{
[][][][][]float64{
// linestring
{{{{7.457013130187988, 46.966401589723894}, {7.482891082763671, 46.94554547022893}}}},
// polygon
{{{
{7.466454505920409, 46.965054389418476},
{7.46143341064453, 46.9641171865865},
{7.466325759887694, 46.96101258493027},
{7.466454505920409, 46.965054389418476},
}}},
// multipolygon
{
{{
{7.4811744689941415, 46.957966385567474},
{7.478899955749511, 46.95492001277476},
{7.484478950500488, 46.95509576976545},
{7.4811744689941415, 46.957966385567474},
}},
{{
{7.466540336608888, 46.94753769790697},
{7.464609146118165, 46.946219320241674},
{7.468342781066894, 46.94592634301753},
{7.466540336608888, 46.94753769790697},
}},
{{
{7.504348754882812, 47.00425575323296},
{7.501087188720703, 47.001680295206874},
{7.507266998291015, 47.00191443288521},
{7.504348754882812, 47.00425575323296},
}},
},
// point
{{{{7.449932098388673, 46.95817142366062}}}},
// multipoint
{{{{7.479157447814942, 46.96370715518446}, {7.4532365798950195, 46.96657730900153}}}},
},
[]string{"linestring", "polygon", "multipolygon", "point", "multipoint"},
"geometry",
[]string{"multipolygon4"},
},
// test contains for a geometrycollection query with one point inside the multipoint lying outside
// polygon2.
{
[][][][][]float64{
// linestring
{{{{7.457013130187988, 46.966401589723894}, {7.482891082763671, 46.94554547022893}}}},
// polygon
{{{
{7.466454505920409, 46.965054389418476},
{7.46143341064453, 46.9641171865865},
{7.466325759887694, 46.96101258493027},
{7.466454505920409, 46.965054389418476},
}}},
// multipolygon
{
{{
{7.4811744689941415, 46.957966385567474},
{7.478899955749511, 46.95492001277476},
{7.484478950500488, 46.95509576976545},
{7.4811744689941415, 46.957966385567474},
}},
{{
{7.466540336608888, 46.94753769790697},
{7.464609146118165, 46.946219320241674},
{7.468342781066894, 46.94592634301753},
{7.466540336608888, 46.94753769790697},
}},
},
// point
{{{{7.449932098388673, 46.95817142366062}}}},
// multipoint
{{{{7.479157447814942, 46.96370715518446}, {7.475638389587402, 46.965200825877794}}}},
},
[]string{"linestring", "polygon", "multipolygon", "point", "multipoint"},
"geometry",
nil,
},
// test contains for a geometrycollection query with one point inside the multipoint lying outside
// polygon2.
{
[][][][][]float64{
// linestring
{{{{7.457013130187988, 46.966401589723894}, {7.482891082763671, 46.94554547022893}}}},
// polygon
{{{
{7.466454505920409, 46.965054389418476},
{7.46143341064453, 46.9641171865865},
{7.466325759887694, 46.96101258493027},
{7.466454505920409, 46.965054389418476},
}}},
// multipolygon
{
{{
{7.4811744689941415, 46.957966385567474},
{7.478899955749511, 46.95492001277476},
{7.484478950500488, 46.95509576976545},
{7.4811744689941415, 46.957966385567474},
}},
{{
{7.466540336608888, 46.94753769790697},
{7.464609146118165, 46.946219320241674},
{7.468342781066894, 46.94592634301753},
{7.466540336608888, 46.94753769790697},
}},
},
// point
{{{{7.449932098388673, 46.95817142366062}}}},
// multipoint
{{{{7.479157447814942, 46.96370715518446}, {7.4532365798950195, 46.96657730900153}}}},
},
[]string{"linestring", "polygon", "multipolygon", "point", "multipoint"},
"geometry",
[]string{"polygon2", "multipolygon4"},
},
}
i := setupGeoJsonShapesIndexForGeometryCollectionQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeGeometryCollectionRelationQuery("contains",
indexReader, test.points, test.types, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.points)
}
}
}
func runGeoShapeGeometryCollectionRelationQuery(relation string, i index.IndexReader,
points [][][][][]float64, types []string, field string,
) ([]string, error) {
var rv []string
s, _, err := geo.NewGeometryCollection(points, types)
if err != nil {
return nil, err
}
gbs, err := NewGeoShapeSearcher(context.TODO(), i, s, relation, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
docID, _ := i.ExternalID(docMatch.IndexInternalID)
rv = append(rv, docID)
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeoJsonShapesIndexForGeometryCollectionQuery(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := scorch.NewScorch(
gtreap.Name,
map[string]interface{}{
"path": "",
"spatialPlugin": "s2",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
// document gc_polygon1_linestring1
polygon1 := [][][][]float64{{{
{-118.15246582031249, 34.876918445772084},
{-118.46557617187499, 34.773203753940734},
{-118.3172607421875, 34.50655662164561},
{-117.91625976562499, 34.4793919710481},
{-117.76245117187499, 34.76417891445512},
{-118.15246582031249, 34.876918445772084},
}}}
linestring1 := [][][][]float64{{{
{-120.78918457031251, 36.87522650673951},
{-118.9215087890625, 34.95349314197422},
}}}
coordinates := [][][][][]float64{polygon1, linestring1}
types := []string{"polygon", "linestring"}
doc := document.NewDocument("gc_polygon1_linestring1")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types, document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
// document gc_multipolygon1_multilinestring1
multipolygon1 := [][][][]float64{
{{
{-117.24609374999999, 37.67512527892127},
{-117.61962890624999, 37.26530995561875},
{-116.597900390625, 37.56199695314352},
{-117.24609374999999, 37.67512527892127},
}},
{{
{-117.60864257812501, 38.71123253895224},
{-117.41638183593749, 38.36750215395045},
{-117.66357421875, 37.93986540897977},
{-116.6473388671875, 37.94852933714952},
{-117.1307373046875, 38.363195134453846},
{-116.75170898437501, 38.7283759182398},
{-117.60864257812501, 38.71123253895224},
}},
}
multilinestring1 := [][][][]float64{{
{{-118.9215087890625, 38.74123075381228}, {-118.78967285156249, 38.43207668538207}},
{{-118.57543945312501, 38.8225909761771}, {-118.45458984375, 38.522384090200845}},
{{-118.94897460937499, 38.788345355085625}, {-118.61938476562499, 38.86965182408357}},
}}
coordinates = [][][][][]float64{multipolygon1, multilinestring1}
types = []string{"multipolygon", "multilinestring"}
doc = document.NewDocument("gc_multipolygon1_multilinestring1")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types, document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
// document gc_point1_multipoint1
point1 := [][][][]float64{{{{-115.10925292968749, 36.20882309283712}}}}
multipoint1 := [][][][]float64{{{
{-117.13623046874999, 36.474306755095235},
{-118.57543945312501, 36.518465989675875},
{-118.58642578124999, 36.90597988519294},
{-119.5477294921875, 37.85316995894978},
}}}
coordinates = [][][][][]float64{point1, multipoint1}
types = []string{"point", "multipoint"}
doc = document.NewDocument("gc_point1_multipoint1")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types, document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
// document gc_multipoint2_multipolygon2_multiline2
multipoint2 := [][][][]float64{{{
{-122.4052906036377, 37.75626203719391},
{-122.42091178894044, 37.74757548736071},
}}}
multipolygon2 := [][][][]float64{
{{
{-122.46168136596681, 37.765151122096945},
{-122.46168136596681, 37.754972691904946},
{-122.45103836059569, 37.754972691904946},
{-122.451810836792, 37.7624370109886},
{-122.46168136596681, 37.765151122096945},
}},
{{
{-122.41902351379395, 37.726194088705576},
{-122.43533134460448, 37.71668926284967},
{-122.40777969360353, 37.71634978222733},
{-122.41902351379395, 37.726194088705576},
}},
}
multilinestring2 := [][][][]float64{{
{{-122.41284370422362, 37.73155698786267}, {-122.40700721740721, 37.73338978839743}},
{{-122.40434646606444, 37.73400071182758}, {-122.39730834960938, 37.73691949864062}},
}}
coordinates = [][][][][]float64{multipoint2, multipolygon2, multilinestring2}
types = []string{"multipoint", "multipolygon", "multiline"}
doc = document.NewDocument("gc_multipoint2_multipolygon2_multiline2")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types, document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
// document gc_multipolygon3
multipolygon3 := [][][][]float64{
{{
{85.60546875, 57.20771009775018},
{86.396484375, 55.99838095535963},
{87.03369140625, 56.71656572651468},
{85.60546875, 57.20771009775018},
}},
{{
{79.56298828125, 55.3915921070334},
{79.60693359375, 54.43171285946844},
{80.39794921875, 54.85131525968606},
{79.56298828125, 55.3915921070334},
}},
{{
{74.35546875, 54.13669645687002},
{74.1796875, 52.802761415419674},
{75.87158203125, 53.44880683542759},
{74.35546875, 54.13669645687002},
}},
}
coordinates = [][][][][]float64{multipolygon3}
types = []string{"multipolygon"}
doc = document.NewDocument("gc_multipolygon3")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types, document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon2 := [][][][]float64{{{
{7.452635765075683, 46.96692874582506},
{7.449803352355956, 46.95817142366062},
{7.4573564529418945, 46.95149263607834},
{7.462162971496582, 46.945955640812095},
{7.483148574829102, 46.945311085627445},
{7.487225532531738, 46.957029058564686},
{7.4793291091918945, 46.96388288331302},
{7.464480400085448, 46.96903731827891},
{7.452635765075683, 46.96692874582506},
}}}
doc = document.NewDocument("polygon2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon2, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipolygon4 := [][][][]float64{
{{
{7.452635765075683, 46.96692874582506},
{7.449803352355956, 46.95817142366062},
{7.4573564529418945, 46.95149263607834},
{7.462162971496582, 46.945955640812095},
{7.483148574829102, 46.945311085627445},
{7.487225532531738, 46.957029058564686},
{7.4793291091918945, 46.96388288331302},
{7.464480400085448, 46.96903731827891},
{7.452635765075683, 46.96692874582506},
}},
{{
{7.4478721618652335, 47.00015837528636},
{7.5110435485839835, 47.00015837528636},
{7.5110435485839835, 47.00683108710118},
{7.4478721618652335, 47.00683108710118},
{7.4478721618652335, 47.00015837528636},
}},
}
doc = document.NewDocument("multipolygon4")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multipolygon4, "multipolygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}

View file

@ -0,0 +1,687 @@
// Copyright (c) 2022 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoJsonLinestringIntersectsQuery(t *testing.T) {
tests := []struct {
line [][]float64
field string
want []string
}{
// test intersecting linestring query for polygon1.
{
[][]float64{
{74.85860824584961, 22.407219759334023},
{74.8663330078125, 22.382936446589863},
},
"geometry",
[]string{"polygon1"},
},
// test intersecting linestring query for polygon1 and polygon2.
{
[][]float64{
{74.82461929321289, 22.393729553598526},
{74.93671417236328, 22.356743809494784},
},
"geometry",
[]string{"polygon1", "polygon2"},
},
// test intersecting linestring query for envelope1.
{
[][]float64{
{74.83938217163086, 22.325782524687973},
{74.8692512512207, 22.311172762889516},
},
"geometry",
[]string{"envelope1"},
},
// test intersecting linestring query for circle.
{
[][]float64{
{74.94546890258789, 22.310815439776572},
{74.93276596069336, 22.303708490145645},
},
"geometry",
[]string{"circle1"},
},
// test intersecting linestring query for linestring1.
{
[][]float64{
{74.938645362854, 22.321614134448936},
{74.94070529937744, 22.320224643365446},
},
"geometry",
[]string{"linestring1"},
},
// test intersecting linestring query for multilinestring1.
{
[][]float64{
{74.9241828918457, 22.307996525380194},
{74.94100570678711, 22.293781977618558},
},
"geometry",
[]string{"multilinestring1"},
},
// test intersecting linestring query for multipolygon1.
{
[][]float64{
{36.22072219848633, 50.007132228568786},
{36.22218132019043, 49.99791917183082},
},
"geometry",
[]string{"multipolygon1"},
},
// test intersecting linestring query for envelope2, circle2,
// multipolygon1 and gc_polygonInGc_multipolygonInGc.
{
[][]float64{
{36.19840621948242, 50.03834418692451},
{36.25720024108887, 50.02136210283289},
},
"geometry",
[]string{"envelope2", "circle2", "multipolygon1", "gc_polygonInGc_multipolygonInGc"},
},
}
i := setupGeoJsonShapesIndexForLinestringQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeLinestringQueryWithRelation("intersects",
indexReader, test.line, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.line)
}
}
}
func TestGeoJsonLinestringContainsQuery(t *testing.T) {
tests := []struct {
line [][]float64
field string
want []string
}{
// test a linestring query for multipolygon1.
{
[][]float64{
{36.21668815612793, 50.040494087443996},
{36.226301193237305, 50.03861982057644},
},
"geometry",
[]string{"multipolygon1"},
},
// test a linestring query with endspoints on two
// different polygons in a multipolygon.
{
[][]float64{
{36.19746208190918, 50.038564693972646},
{36.21565818786621, 50.03718650830641},
},
"geometry", nil,
},
// test a linestring query for envelope2.
{
[][]float64{
{36.25290870666503, 50.03018471417061},
{36.23110771179199, 50.01854955486945},
},
"geometry",
[]string{"envelope2"},
},
// test a linestring query for circle2.
{
[][]float64{
{36.220550537109375, 50.02930252595981},
{36.224327087402344, 50.02847545979485},
},
"geometry",
[]string{"circle2"},
},
// test a linestring query for polygonWithHole2.
{
[][]float64{
{36.27367973327637, 49.89883638369706},
{36.27445220947265, 49.89596137883285},
},
"geometry",
[]string{"polygonWithHole2"},
},
// test a linestring query within the hole of polygonWithHole2.
{[][]float64{
{36.261234283447266, 49.89540847364305},
{36.26243591308594, 49.89087441212101},
}, "geometry", nil},
}
i := setupGeoJsonShapesIndexForLinestringQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeLinestringQueryWithRelation("contains",
indexReader, test.line, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.line)
}
}
}
func TestGeoJsonMultiLinestringContainsQuery(t *testing.T) {
tests := []struct {
line [][][]float64
field string
want []string
}{
// test a multilinestring query for multipolygon1.
{
[][][]float64{
{
{36.21668815612793, 50.040494087443996},
{36.226301193237305, 50.03861982057644},
},
{
{36.226816177368164, 49.999463999158},
{36.234025955200195, 50.00271900853649},
},
},
"geometry",
[]string{"multipolygon1"},
},
// test a multilinestring query that is covered by the geometryCollection.
{
[][][]float64{{
{36.28664016723633, 49.96574238290487},
{36.30251884460449, 49.96369956194569},
}, {
{36.19179725646973, 50.03983258984584},
{36.19420051574707, 50.03801342445342},
}},
"geometry",
[]string{"gc_polygonInGc_multipolygonInGc"},
},
// test a multilinestring query for envelope2.
{
[][][]float64{
{
{36.23213768005371, 50.02913711386621},
{36.25187873840332, 50.02902683882067},
},
{
{36.231794357299805, 50.018935600613254},
{36.2314510345459, 50.025883893582055},
},
},
"geometry",
[]string{"envelope2"},
},
// test a multilinestring query with one linestring outside of envelope2.
{
[][][]float64{
{
{36.23213768005371, 50.02913711386621},
{36.25187873840332, 50.02902683882067},
},
{{36.231794357299805, 50.018935600613254}, {36.2314510345459, 50.025883893582055}},
{{36.25659942626953, 50.024284772330844}, {36.24406814575195, 50.01518531066489}},
},
"geometry", nil,
},
// test a multilinestring query with one linestring
// inside the whole of a polygonWithHole2.
{
[][][]float64{
{
{36.27367973327637, 49.89883638369706},
{36.27445220947265, 49.89596137883285},
},
{{36.261234283447266, 49.89540847364305}, {36.26243591308594, 49.89087441212101}},
},
"geometry", nil,
},
// test a multilinestring query for polygonWithHole2.
{
[][][]float64{
{
{36.27367973327637, 49.89883638369706},
{36.27445220947265, 49.89596137883285},
},
{{36.279258728027344, 49.894302644257856}, {36.28166198730469, 49.887335336408235}},
},
"geometry",
[]string{"polygonWithHole2"},
},
// test a multilinestring query for polygonWithHole2 with last line cross the hole.
{
[][][]float64{
{
{36.27367973327637, 49.89883638369706},
{36.27445220947265, 49.89596137883285},
},
{{36.279258728027344, 49.894302644257856}, {36.28166198730469, 49.887335336408235}},
{{36.254024505615234, 49.89839408640621}, {36.27016067504883, 49.90038439228633}},
},
"geometry", nil,
},
}
i := setupGeoJsonShapesIndexForLinestringQuery(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapeMultiLinestringQueryWithRelation("contains",
indexReader, test.line, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.line)
}
}
}
func runGeoShapeMultiLinestringQueryWithRelation(relation string, i index.IndexReader,
points [][][]float64, field string,
) ([]string, error) {
s := geo.NewGeoJsonMultilinestring(points)
return executeSearch(relation, i, s, field)
}
func runGeoShapeLinestringQueryWithRelation(relation string, i index.IndexReader,
points [][]float64, field string,
) ([]string, error) {
s := geo.NewGeoJsonLinestring(points)
return executeSearch(relation, i, s, field)
}
func executeSearch(relation string, i index.IndexReader,
s index.GeoJSON, field string,
) ([]string, error) {
var rv []string
gbs, err := NewGeoShapeSearcher(context.TODO(), i, s, relation, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
docID, _ := i.ExternalID(docMatch.IndexInternalID)
rv = append(rv, docID)
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
func setupGeoJsonShapesIndexForLinestringQuery(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := scorch.NewScorch(
gtreap.Name,
map[string]interface{}{
"path": "",
"spatialPlugin": "s2",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
polygon1 := [][][][]float64{{{
{74.84642028808594, 22.402776071459712},
{74.83234405517578, 22.39039647758608},
{74.86719131469727, 22.38801566009795},
{74.85139846801758, 22.39103135536648},
{74.86461639404297, 22.394840561182853},
{74.8495101928711, 22.397697397065034},
{74.86186981201172, 22.401982540816856},
{74.84642028808594, 22.402776071459712},
}}}
doc := document.NewDocument("polygon1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, polygon1, "polygon",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygon2 := [][][][]float64{{{
{74.93431091308592, 22.376428433285266},
{74.92898941040039, 22.39103135536648},
{74.9241828918457, 22.37722210974017},
{74.90821838378906, 22.37388863821397},
{74.92504119873047, 22.369920115637292},
{74.92864608764648, 22.355632497760894},
{74.93207931518555, 22.370396344320053},
{74.94855880737305, 22.3743648533201},
{74.93431091308592, 22.376428433285266},
}}}
doc = document.NewDocument("polygon2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, polygon2, "polygon",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope1 := [][][][]float64{{{
{74.86736297607422, 22.307361269208684},
{74.87028121948242, 22.345471522338478},
}}}
doc = document.NewDocument("envelope1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, envelope1, "envelope",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
envelope2 := [][][][]float64{{{
{36.23007774353027, 50.01810835593541},
{36.25333786010742, 50.03068093791795},
}}}
doc = document.NewDocument("envelope2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, envelope2, "envelope",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle1")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry",
[]uint64{}, []float64{74.93671417236328, 22.308314152382284}, "300m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle2")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry",
[]uint64{}, []float64{36.22243881225586, 50.02941280037234}, "600m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring := [][][][]float64{{{
{74.92697238922119, 22.320343743143248},
{74.94036197662354, 22.32054224254707},
}}}
doc = document.NewDocument("linestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, linestring, "linestring",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring1 := [][][][]float64{{{
{77.60188579559325, 12.982604078764705},
{77.60557651519775, 12.987329508048184},
}}}
doc = document.NewDocument("linestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, linestring1, "linestring",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring := [][][][]float64{{
{
{74.92203712463379, 22.3113315728684},
{74.92323875427246, 22.307798008137024},
},
{{74.92405414581299, 22.307559787072712}, {74.92735862731934, 22.310021385140573}},
{{74.9223804473877, 22.311688894660474}, {74.92534160614014, 22.30930673210729}},
}}
doc = document.NewDocument("multilinestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, multilinestring, "multilinestring",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring1 := [][][][]float64{{
{
{77.6015853881836, 12.990089451715061},
{77.60476112365723, 12.987747683302153},
},
{{77.59875297546387, 12.988751301039581}, {77.59446144104004, 12.98197680263484}},
{{77.60188579559325, 12.982604078764705}, {77.60557651519775, 12.987329508048184}},
}}
doc = document.NewDocument("multilinestring2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, multilinestring1, "multilinestring",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipoint1 := [][][][]float64{{{
{77.56618022918701, 12.958180959662695},
{77.56407737731932, 12.951614746607163},
{77.56922721862793, 12.956173473406446},
}}}
doc = document.NewDocument("multipoint1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, multipoint1, "multipoint",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygonWithHole1 := [][][][]float64{{
{
{77.59991168975829, 12.972232910164502},
{77.6039457321167, 12.97582941279006},
{77.60424613952637, 12.98168407323241},
{77.59974002838135, 12.985489528568463},
{77.59321689605713, 12.979300406693417},
{77.59991168975829, 12.972232910164502},
},
{
{77.59682178497314, 12.975787593290978},
{77.60295867919922, 12.975787593290978},
{77.60295867919922, 12.98143316204164},
{77.59682178497314, 12.98143316204164},
{77.59682178497314, 12.975787593290978},
},
}}
doc = document.NewDocument("polygonWithHole1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, polygonWithHole1, "polygon",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygonWithHole2 := [][][][]float64{{
{
{36.261234283447266, 49.90712870720605},
{36.2479305267334, 49.89480027061714},
{36.254539489746094, 49.883408870659736},
{36.280717849731445, 49.883408870659736},
{36.28741264343262, 49.890432041848264},
{36.27788543701172, 49.90276159448742},
{36.261234283447266, 49.90712870720605},
},
{
{36.264581680297844, 49.905249238801304},
{36.25368118286133, 49.89673543545543},
{36.253509521484375, 49.88578690918283},
{36.270332336425774, 49.886174020645804},
{36.27127647399902, 49.89579550794111},
{36.264581680297844, 49.905249238801304},
},
}}
doc = document.NewDocument("polygonWithHole2")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, polygonWithHole2, "polygon",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipolygon1 := [][][][]float64{{{
{36.1875057220459, 50.04363607656457},
{36.192398071289055, 50.034871067327856},
{36.20218276977539, 50.03955696315653},
{36.1875057220459, 50.04363607656457},
}}, // polygon1
{{
{36.2123966217041, 50.03795829715335},
{36.218318939208984, 50.0333273779768},
{36.226558685302734, 50.03867494711694},
{36.217031478881836, 50.04286437899031},
{36.2123966217041, 50.03795829715335},
}}, // polygon2
{{
{36.221065521240234, 50.00365685169585},
{36.226301193237305, 49.998029518286025},
{36.23342514038086, 49.9995743420677},
{36.23531341552734, 50.002994846659156},
{36.231021881103516, 50.00630478067617},
{36.22810363769531, 50.00663576154257},
{36.226043701171875, 50.004815338573046},
{36.221065521240234, 50.00365685169585},
}}}
doc = document.NewDocument("multipolygon1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry",
[]uint64{}, multipolygon1, "multipolygon",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygonInGc := [][][][]float64{{{
{36.1875057220459, 50.04363607656457},
{36.192398071289055, 50.034871067327856},
{36.20218276977539, 50.03955696315653},
{36.1875057220459, 50.04363607656457},
}}}
multipolygonInGc := [][][][]float64{{{
{36.29015922546387, 49.980150089789376},
{36.28337860107422, 49.961656654293485},
{36.307411193847656, 49.96033147865059},
{36.29015922546387, 49.980150089789376},
}}, // polygon1
{{
{36.16106986999512, 50.00387751801547},
{36.161842346191406, 49.9908012905034},
{36.17900848388672, 49.99841572888488},
{36.16106986999512, 50.00387751801547},
}}}
coordinates := [][][][][]float64{polygonInGc, multipolygonInGc}
types := []string{"polygon", "multipolygon"}
doc = document.NewDocument("gc_polygonInGc_multipolygonInGc")
doc.AddField(document.NewGeometryCollectionFieldWithIndexingOptions("geometry",
[]uint64{}, coordinates, types,
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}

View file

@ -0,0 +1,600 @@
// Copyright (c) 2022 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/geo"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestGeoJsonPointContainsQuery(t *testing.T) {
tests := []struct {
point []float64
field string
want []string
}{
// test points inside the polygon1.
{
[]float64{77.58334636688232, 12.948268838994263},
"geometry",
[]string{"polygon1"},
},
// test points inside the circle1.
{
[]float64{77.58553504943848, 12.954040501528555},
"geometry",
[]string{"circle1"},
},
// test points inside the polygon1 and the circle.
{
[]float64{77.59293794631958, 12.948896200093982},
"geometry",
[]string{"polygon1", "circle1"},
},
// test points outside the polygon1 and the circle1.
{
[]float64{77.5614595413208, 12.953287683563568},
"geometry", nil,
},
// test point within the envelope1.
{
[]float64{81.28166198730469, 26.34203746601541},
"geometry",
[]string{"envelope1"},
},
// test point on the linestring vertex.
{
[]float64{77.57776737213135, 12.952074805390097},
"geometry",
[]string{"linestring1"},
},
// test point on the multilinestring vertex.
{
[]float64{77.5779390335083, 12.945006535817749},
"geometry",
[]string{"multilinestring1"},
},
// test point on the multipoint vertex.
{
[]float64{77.56407737731932, 12.951614746607163},
"geometry",
[]string{"multipoint1"},
},
// test point within the polygonWithHole1.
{
[]float64{77.60334491729736, 12.979844051951334},
"geometry",
[]string{"polygonWithHole1"},
},
// test point within the hole of the polygonWithHole1.
{
[]float64{77.60244369506836, 12.976247607394027},
"geometry", nil,
},
}
i := setupGeoJsonShapesIndex(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapePointRelationQuery("contains",
false, indexReader, [][]float64{test.point}, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.point)
}
}
}
func TestGeoJsonMultiPointWithInQuery(t *testing.T) {
tests := []struct {
multipoint [][]float64
field string
want []string
}{
// test multipoint inside the polygon1.
{
[][]float64{
{77.58334636688232, 12.948268838994263},
{77.58467674255371, 12.944295515355652},
},
"geometry",
[]string{"polygon1"},
},
// test multipoint inside the circle1.
{
[][]float64{
{77.58553504943848, 12.954040501528555},
{77.58643627166747, 12.956089827794571},
},
"geometry",
[]string{"circle1"},
},
// test multipoint inside the envelope1.
{
[][]float64{
{81.28166198730469, 26.34203746601541},
{80.94314575195312, 26.346960121309415},
},
"geometry",
[]string{"envelope1"},
},
// test multipoint inside the polygon1 and the circle.
{
[][]float64{
{77.59293794631958, 12.948896200093982},
{77.58532047271729, 12.953789562459688},
},
"geometry",
[]string{"polygon1", "circle1"},
},
// test multipoint (only 1 point outside) outside.
{[][]float64{
{77.58334636688232, 12.948268838994263},
{77.58643627166747, 12.956089827794571},
{77.5615, 12.9533},
}, "geometry", nil},
// test multipoint on the linestring vertex.
{
[][]float64{
{77.5841188430786, 12.957093573282744},
{77.57776737213135, 12.952074805390097},
},
"geometry",
[]string{"linestring1"},
},
// test multipoint outside the linestring vertex.
{
[][]float64{
{77.5841188430786, 12.957093573282744},
{77.57776737213135, 12.952074805390097},
{77.58334636688232, 12.948268838994263},
},
"geometry", nil,
},
// test multipoint on the multilinestring vertex.
{
[][]float64{
{77.5779390335083, 12.94471376293191},
{77.57218837738037, 12.948268838994263},
},
"geometry",
[]string{"multilinestring1"},
},
// test multipoint outside the multilinestring vertex.
{
[][]float64{
{77.5779390335083, 12.94471376293191},
{77.57218837738037, 12.948268838994263},
{77.58532047271729, 12.953789562459688},
},
"geometry", nil,
},
// test multipoint with one inside the hole within the polygonWithHole1.
{
[][]float64{
{77.60334491729736, 12.979844051951334},
{77.60244369506836, 12.976247607394027},
},
"geometry", nil,
},
// test multipoint with all inside the hole witin the polygonWithHole1.
{
[][]float64{
{77.59656429290771, 12.981767710239714},
{77.59888172149658, 12.979969508380469},
},
"geometry", nil,
},
// test multipoint with all inside the polygonWithHole1.
{
[][]float64{
{77.60334491729736, 12.979844051951334},
{77.59656429290771, 12.981767710239714},
{77.59802341461182, 12.9751602999608},
},
"geometry",
[]string{"polygonWithHole1"},
},
}
i := setupGeoJsonShapesIndex(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapePointRelationQuery("contains",
true, indexReader, test.multipoint, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.multipoint)
}
}
}
func TestGeoJsonMultiPointIntersectsQuery(t *testing.T) {
tests := []struct {
multipoint [][]float64
field string
want []string
}{
// test multipoint inside the polygon1.
{
[][]float64{
{77.58334636688232, 12.948268838994263},
{77.58467674255371, 12.944295515355652},
},
"geometry",
[]string{"polygon1"},
},
// test multipoint inside the circle1.
{
[][]float64{
{77.58553504943848, 12.954040501528555},
{77.58643627166747, 12.956089827794571},
},
"geometry",
[]string{"circle1"},
},
// test multipoint inside the envelope1. (1 point outside)
{
[][]float64{
{81.28166198730469, 26.34203746601541},
{80.94314575195312, 26.346960121309415},
{81.12716674804688, 26.353728430338332},
},
"geometry",
[]string{"envelope1"},
},
// test multipoint inside the polygon1 and the circle.
{
[][]float64{
{77.59293794631958, 12.948896200093982},
{77.58532047271729, 12.953789562459688},
},
"geometry",
[]string{"polygon1", "circle1"},
},
// test multipoint (only 1 point outside) intersects.
{
[][]float64{
{77.58334636688232, 12.948268838994263},
{77.58643627166747, 12.956089827794571},
{77.5615, 12.9533},
},
"geometry",
[]string{"polygon1", "circle1"},
},
// test multipoint on the linestring vertex.
{
[][]float64{
{77.5841188430786, 12.957093573282744},
{77.57776737213135, 12.952074805390097},
},
"geometry",
[]string{"linestring1"},
},
// test multipoint outside the linestring vertex.
{
[][]float64{
{77.5841188430786, 12.957093573282744},
{77.57776737213135, 12.952074805390097},
{77.58334636688232, 12.948268838994263},
},
"geometry",
[]string{"polygon1", "linestring1"},
},
// test multipoint on the multilinestring vertex.
{
[][]float64{
{77.5779390335083, 12.94471376293191},
{77.57218837738037, 12.948268838994263},
},
"geometry",
[]string{"multilinestring1"},
},
// test multipoint outside the multilinestring vertex.
{
[][]float64{
{77.5779390335083, 12.94471376293191},
{77.57218837738037, 12.948268838994263},
{77.58532047271729, 12.953789562459688},
},
"geometry",
[]string{"polygon1", "circle1", "multilinestring1"},
},
// test multipoint with one inside the hole within the polygonWithHole1.
{
[][]float64{
{77.60334491729736, 12.979844051951334},
{77.60244369506836, 12.976247607394027},
},
"geometry",
[]string{"polygonWithHole1"},
},
// test multipoint with all inside the hole witin the polygonWithHole1.
{
[][]float64{
{77.60244369506836, 12.976247607394027},
{77.59888172149658, 12.979969508380469},
},
"geometry", nil,
},
// test multipoint with all inside the polygonWithHole1.
{
[][]float64{
{77.60334491729736, 12.979844051951334},
{77.59656429290771, 12.981767710239714},
{77.59802341461182, 12.9751602999608},
},
"geometry",
[]string{"polygonWithHole1"},
},
}
i := setupGeoJsonShapesIndex(t)
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err = indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
for n, test := range tests {
got, err := runGeoShapePointRelationQuery("intersects",
true, indexReader, test.multipoint, test.field)
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("test %d, expected %v, got %v for polygon: %+v",
n, test.want, got, test.multipoint)
}
}
}
func runGeoShapePointRelationQuery(relation string, multi bool,
i index.IndexReader, points [][]float64, field string,
) ([]string, error) {
var rv []string
var s index.GeoJSON
if multi {
s = geo.NewGeoJsonMultiPoint(points)
} else {
s = geo.NewGeoJsonPoint(points[0])
}
gbs, err := NewGeoShapeSearcher(context.TODO(), i, s, relation, field, 1.0, search.SearcherOptions{})
if err != nil {
return nil, err
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(gbs.DocumentMatchPoolSize(), 0),
}
docMatch, err := gbs.Next(ctx)
for docMatch != nil && err == nil {
docID, _ := i.ExternalID(docMatch.IndexInternalID)
rv = append(rv, docID)
docMatch, err = gbs.Next(ctx)
}
if err != nil {
return nil, err
}
return rv, nil
}
type Fatalfable interface {
Fatalf(format string, args ...interface{})
}
func setupGeoJsonShapesIndex(t *testing.T) index.Index {
analysisQueue := index.NewAnalysisQueue(1)
i, err := scorch.NewScorch(
gtreap.Name,
map[string]interface{}{
"path": "",
"spatialPlugin": "s2",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
polygon1 := [][][][]float64{{{
{77.5853419303894, 12.953977766785052},
{77.58405447006226, 12.95393594361393},
{77.5819730758667, 12.9495026476557},
{77.58068561553955, 12.94883346405509},
{77.58019208908081, 12.948331575175299},
{77.57991313934326, 12.943814529775414},
{77.58497714996338, 12.94394000436408},
{77.58517026901245, 12.9446301134728},
{77.58572816848755, 12.945508431393435},
{77.58785247802734, 12.946365833997325},
{77.58967638015747, 12.946428570657417},
{77.59070634841918, 12.947474179333993},
{77.59317398071289, 12.948875288082773},
{77.59167194366454, 12.949962710338657},
{77.59077072143555, 12.950276388953625},
{77.59098529815674, 12.951196510612728},
{77.58729457855225, 12.952472128200755},
{77.5853419303894, 12.953977766785052},
}}}
doc := document.NewDocument("polygon1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygon1, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
// not working envelope
envelope1 := [][][][]float64{{{
{80.93696594238281, 26.33957605983274},
{81.28440856933594, 26.351267272877074},
}}}
doc = document.NewDocument("envelope1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
envelope1, "envelope", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("circle1")
doc.AddField(document.NewGeoCircleFieldWithIndexingOptions("geometry", []uint64{},
[]float64{77.59137153625487, 12.952660333521468}, "900m",
document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
linestring := [][][][]float64{{{
{77.5841188430786, 12.957093573282744},
{77.57776737213135, 12.952074805390097},
}}}
doc = document.NewDocument("linestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
linestring, "linestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multilinestring := [][][][]float64{{{
{77.57227420806883, 12.948687079902895},
{77.57600784301758, 12.954165970968194},
{77.5779390335083, 12.94471376293191},
{77.57218837738037, 12.948268838994263},
{77.57781028747559, 12.951740217268595},
{77.5779390335083, 12.945006535817749},
}}}
doc = document.NewDocument("multilinestring1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multilinestring, "multilinestring", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
multipoint1 := [][][][]float64{{{
{77.56618022918701, 12.958180959662695},
{77.56407737731932, 12.951614746607163},
{77.56922721862793, 12.956173473406446},
}}}
doc = document.NewDocument("multipoint1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
multipoint1, "multipoint", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
polygonWithHole1 := [][][][]float64{{
{
{77.59991168975829, 12.972232910164502},
{77.6039457321167, 12.97582941279006},
{77.60424613952637, 12.98168407323241},
{77.59974002838135, 12.985489528568463},
{77.59321689605713, 12.979300406693417},
{77.59991168975829, 12.972232910164502},
},
{
{77.59682178497314, 12.975787593290978},
{77.60295867919922, 12.975787593290978},
{77.60295867919922, 12.98143316204164},
{77.59682178497314, 12.98143316204164},
{77.59682178497314, 12.975787593290978},
},
}}
doc = document.NewDocument("polygonWithHole1")
doc.AddField(document.NewGeoShapeFieldWithIndexingOptions("geometry", []uint64{},
polygonWithHole1, "polygon", document.DefaultGeoShapeIndexingOptions))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
return i
}

File diff suppressed because it is too large Load diff

View file

@ -0,0 +1,68 @@
// 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 searcher
import (
"context"
"net"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
// netLimits returns the lo and hi bounds inside the network.
func netLimits(n *net.IPNet) (lo net.IP, hi net.IP) {
ones, bits := n.Mask.Size()
netNum := n.IP
if bits == net.IPv4len*8 {
netNum = netNum.To16()
ones += 8 * (net.IPv6len - net.IPv4len)
}
mask := net.CIDRMask(ones, 8*net.IPv6len)
lo = make(net.IP, net.IPv6len)
hi = make(net.IP, net.IPv6len)
for i := 0; i < net.IPv6len; i++ {
lo[i] = netNum[i] & mask[i]
hi[i] = lo[i] | ^mask[i]
}
return lo, hi
}
func NewIPRangeSearcher(ctx context.Context, indexReader index.IndexReader, ipNet *net.IPNet,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
lo, hi := netLimits(ipNet)
fieldDict, err := indexReader.FieldDictRange(field, lo, hi)
if err != nil {
return nil, err
}
defer fieldDict.Close()
var terms []string
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
terms = append(terms, tfd.Term)
if tooManyClauses(len(terms)) {
return nil, tooManyClausesErr(field, len(terms))
}
tfd, err = fieldDict.Next()
}
if err != nil {
return nil, err
}
return NewMultiTermSearcher(ctx, indexReader, terms, field, boost, options, true)
}

View file

@ -0,0 +1,52 @@
// Copyright (c) 2021 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 searcher
import (
"net"
"testing"
)
func Test_netLimits(t *testing.T) {
tests := []struct {
arg string
lo string
hi string
}{
{"128.0.0.0/1", "128.0.0.0", "255.255.255.255"},
{"128.0.0.0/7", "128.0.0.0", "129.255.255.255"},
{"1.1.1.1/8", "1.0.0.0", "1.255.255.255"},
{"1.2.3.0/24", "1.2.3.0", "1.2.3.255"},
{"1.2.2.0/23", "1.2.2.0", "1.2.3.255"},
{"1.2.3.128/25", "1.2.3.128", "1.2.3.255"},
{"1.2.3.0/25", "1.2.3.0", "1.2.3.127"},
{"1.2.3.4/31", "1.2.3.4", "1.2.3.5"},
{"1.2.3.4/32", "1.2.3.4", "1.2.3.4"},
{"2a00:23c8:7283:ff00:1fa8:0:0:0/80", "2a00:23c8:7283:ff00:1fa8::", "2a00:23c8:7283:ff00:1fa8:ffff:ffff:ffff"},
}
for _, tt := range tests {
t.Run(tt.arg, func(t *testing.T) {
_, net, err := net.ParseCIDR(tt.arg)
if err != nil {
t.Fatal(err)
}
lo, hi := netLimits(net)
if lo.String() != tt.lo || hi.String() != tt.hi {
t.Errorf("netLimits(%q) = %s %s, want %s %s", tt.arg, lo, hi, tt.lo, tt.hi)
}
})
}
}

View file

@ -0,0 +1,145 @@
// Copyright (c) 2023 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.
//go:build vectors
// +build vectors
package searcher
import (
"context"
"encoding/json"
"reflect"
"github.com/blevesearch/bleve/v2/mapping"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeKNNSearcher int
func init() {
var ks KNNSearcher
reflectStaticSizeKNNSearcher = int(reflect.TypeOf(ks).Size())
}
type KNNSearcher struct {
field string
vector []float32
k int64
indexReader index.IndexReader
vectorReader index.VectorReader
scorer *scorer.KNNQueryScorer
count uint64
vd index.VectorDoc
}
func NewKNNSearcher(ctx context.Context, i index.IndexReader, m mapping.IndexMapping,
options search.SearcherOptions, field string, vector []float32, k int64,
boost float64, similarityMetric string, searchParams json.RawMessage,
eligibleSelector index.EligibleDocumentSelector) (
search.Searcher, error) {
if vr, ok := i.(index.VectorIndexReader); ok {
vectorReader, err := vr.VectorReader(ctx, vector, field, k, searchParams, eligibleSelector)
if err != nil {
return nil, err
}
knnScorer := scorer.NewKNNQueryScorer(vector, field, boost,
options, similarityMetric)
return &KNNSearcher{
indexReader: i,
vectorReader: vectorReader,
field: field,
vector: vector,
k: k,
scorer: knnScorer,
}, nil
}
return nil, nil
}
func (s *KNNSearcher) VectorOptimize(ctx context.Context, octx index.VectorOptimizableContext) (
index.VectorOptimizableContext, error) {
o, ok := s.vectorReader.(index.VectorOptimizable)
if ok {
return o.VectorOptimize(ctx, octx)
}
return nil, nil
}
func (s *KNNSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (
*search.DocumentMatch, error) {
knnMatch, err := s.vectorReader.Next(s.vd.Reset())
if err != nil {
return nil, err
}
if knnMatch == nil {
return nil, nil
}
docMatch := s.scorer.Score(ctx, knnMatch)
return docMatch, nil
}
func (s *KNNSearcher) Close() error {
return s.vectorReader.Close()
}
func (s *KNNSearcher) Count() uint64 {
return s.vectorReader.Count()
}
func (s *KNNSearcher) DocumentMatchPoolSize() int {
return 1
}
func (s *KNNSearcher) Min() int {
return 0
}
func (s *KNNSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
knnMatch, err := s.vectorReader.Next(s.vd.Reset())
if err != nil {
return nil, err
}
if knnMatch == nil {
return nil, nil
}
docMatch := s.scorer.Score(ctx, knnMatch)
return docMatch, nil
}
func (s *KNNSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *KNNSearcher) Size() int {
return reflectStaticSizeKNNSearcher + size.SizeOfPtr +
s.vectorReader.Size() +
s.vd.Size() +
s.scorer.Size()
}
func (s *KNNSearcher) Weight() float64 {
return s.scorer.Weight()
}

View file

@ -0,0 +1,123 @@
// 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 searcher
import (
"context"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeMatchAllSearcher int
func init() {
var mas MatchAllSearcher
reflectStaticSizeMatchAllSearcher = int(reflect.TypeOf(mas).Size())
}
type MatchAllSearcher struct {
indexReader index.IndexReader
reader index.DocIDReader
scorer *scorer.ConstantScorer
count uint64
}
func NewMatchAllSearcher(ctx context.Context, indexReader index.IndexReader, boost float64, options search.SearcherOptions) (*MatchAllSearcher, error) {
reader, err := indexReader.DocIDReaderAll()
if err != nil {
return nil, err
}
count, err := indexReader.DocCount()
if err != nil {
_ = reader.Close()
return nil, err
}
scorer := scorer.NewConstantScorer(1.0, boost, options)
return &MatchAllSearcher{
indexReader: indexReader,
reader: reader,
scorer: scorer,
count: count,
}, nil
}
func (s *MatchAllSearcher) Size() int {
return reflectStaticSizeMatchAllSearcher + size.SizeOfPtr +
s.reader.Size() +
s.scorer.Size()
}
func (s *MatchAllSearcher) Count() uint64 {
return s.count
}
func (s *MatchAllSearcher) Weight() float64 {
return s.scorer.Weight()
}
func (s *MatchAllSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *MatchAllSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
id, err := s.reader.Next()
if err != nil {
return nil, err
}
if id == nil {
return nil, nil
}
// score match
docMatch := s.scorer.Score(ctx, id)
// return doc match
return docMatch, nil
}
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
}
if id == nil {
return nil, nil
}
// score match
docMatch := s.scorer.Score(ctx, id)
// return doc match
return docMatch, nil
}
func (s *MatchAllSearcher) Close() error {
return s.reader.Close()
}
func (s *MatchAllSearcher) Min() int {
return 0
}
func (s *MatchAllSearcher) DocumentMatchPoolSize() int {
return 1
}

View file

@ -0,0 +1,146 @@
// Copyright (c) 2013 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 searcher
import (
"context"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestMatchAllSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
explainTrue := search.SearcherOptions{Explain: true}
allSearcher, err := NewMatchAllSearcher(context.TODO(), twoDocIndexReader, 1.0, explainTrue)
if err != nil {
t.Fatal(err)
}
allSearcher2, err := NewMatchAllSearcher(context.TODO(), twoDocIndexReader, 1.2, explainTrue)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
queryNorm float64
results []*search.DocumentMatch
}{
{
searcher: allSearcher,
queryNorm: 1.0,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("5"),
Score: 1.0,
},
},
},
{
searcher: allSearcher2,
queryNorm: 0.8333333,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("1"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("3"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("4"),
Score: 1.0,
},
{
IndexInternalID: index.IndexInternalID("5"),
Score: 1.0,
},
},
},
}
for testIndex, test := range tests {
if test.queryNorm != 1.0 {
test.searcher.SetQueryNorm(test.queryNorm)
}
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if !scoresCloseEnough(next.Score, test.results[i].Score) {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}

View file

@ -0,0 +1,76 @@
// 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 searcher
import (
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeMatchNoneSearcher int
func init() {
var mns MatchNoneSearcher
reflectStaticSizeMatchNoneSearcher = int(reflect.TypeOf(mns).Size())
}
type MatchNoneSearcher struct {
indexReader index.IndexReader
}
func NewMatchNoneSearcher(indexReader index.IndexReader) (*MatchNoneSearcher, error) {
return &MatchNoneSearcher{
indexReader: indexReader,
}, nil
}
func (s *MatchNoneSearcher) Size() int {
return reflectStaticSizeMatchNoneSearcher + size.SizeOfPtr
}
func (s *MatchNoneSearcher) Count() uint64 {
return uint64(0)
}
func (s *MatchNoneSearcher) Weight() float64 {
return 0.0
}
func (s *MatchNoneSearcher) SetQueryNorm(qnorm float64) {
}
func (s *MatchNoneSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
return nil, nil
}
func (s *MatchNoneSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
return nil, nil
}
func (s *MatchNoneSearcher) Close() error {
return nil
}
func (s *MatchNoneSearcher) Min() int {
return 0
}
func (s *MatchNoneSearcher) DocumentMatchPoolSize() int {
return 0
}

View file

@ -0,0 +1,85 @@
// Copyright (c) 2013 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 searcher
import (
"testing"
"github.com/blevesearch/bleve/v2/search"
)
func TestMatchNoneSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
noneSearcher, err := NewMatchNoneSearcher(twoDocIndexReader)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
}{
{
searcher: noneSearcher,
results: []*search.DocumentMatch{},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if !scoresCloseEnough(next.Score, test.results[i].Score) {
t.Errorf("expected result %d to have score %v got %v for test %d", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}

View file

@ -0,0 +1,268 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"fmt"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewMultiTermSearcher(ctx context.Context, indexReader index.IndexReader, terms []string,
field string, boost float64, options search.SearcherOptions, limit bool) (
search.Searcher, error) {
if tooManyClauses(len(terms)) {
if optionsDisjunctionOptimizable(options) {
return optimizeMultiTermSearcher(ctx, indexReader, terms, field, boost, options)
}
if limit {
return nil, tooManyClausesErr(field, len(terms))
}
}
qsearchers, err := makeBatchSearchers(ctx, indexReader, terms, field, boost, options)
if err != nil {
return nil, err
}
// build disjunction searcher of these ranges
return newMultiTermSearcherInternal(ctx, indexReader, qsearchers, field, boost,
options, limit)
}
// Works similarly to the multi term searcher but additionally boosts individual terms based on
// their edit distance from the query terms
func NewMultiTermSearcherBoosted(ctx context.Context, indexReader index.IndexReader, terms []string,
field string, boost float64, editDistances []uint8, options search.SearcherOptions, limit bool) (
search.Searcher, error) {
if tooManyClauses(len(terms)) {
if optionsDisjunctionOptimizable(options) {
return optimizeMultiTermSearcher(ctx, indexReader, terms, field, boost, options)
}
if limit {
return nil, tooManyClausesErr(field, len(terms))
}
}
qsearchers, err := makeBatchSearchersBoosted(ctx, indexReader, terms, field, boost, editDistances, options)
if err != nil {
return nil, err
}
// build disjunction searcher of these ranges
return newMultiTermSearcherInternal(ctx, indexReader, qsearchers, field, boost,
options, limit)
}
func NewMultiTermSearcherBytes(ctx context.Context, indexReader index.IndexReader, terms [][]byte,
field string, boost float64, options search.SearcherOptions, limit bool) (
search.Searcher, error) {
if tooManyClauses(len(terms)) {
if optionsDisjunctionOptimizable(options) {
return optimizeMultiTermSearcherBytes(ctx, indexReader, terms, field, boost, options)
}
if limit {
return nil, tooManyClausesErr(field, len(terms))
}
}
qsearchers, err := makeBatchSearchersBytes(ctx, indexReader, terms, field, boost, options)
if err != nil {
return nil, err
}
// build disjunction searcher of these ranges
return newMultiTermSearcherInternal(ctx, indexReader, qsearchers, field, boost,
options, limit)
}
func newMultiTermSearcherInternal(ctx context.Context, indexReader index.IndexReader,
searchers []search.Searcher, field string, boost float64,
options search.SearcherOptions, limit bool) (
search.Searcher, error) {
// build disjunction searcher of these ranges
searcher, err := newDisjunctionSearcher(ctx, indexReader, searchers, 0, options,
limit)
if err != nil {
for _, s := range searchers {
_ = s.Close()
}
return nil, err
}
return searcher, nil
}
func optimizeMultiTermSearcher(ctx context.Context, indexReader index.IndexReader, terms []string,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
var finalSearcher search.Searcher
for len(terms) > 0 {
var batchTerms []string
if len(terms) > DisjunctionMaxClauseCount {
batchTerms = terms[:DisjunctionMaxClauseCount]
terms = terms[DisjunctionMaxClauseCount:]
} else {
batchTerms = terms
terms = nil
}
batch, err := makeBatchSearchers(ctx, indexReader, batchTerms, field, boost, options)
if err != nil {
return nil, err
}
if finalSearcher != nil {
batch = append(batch, finalSearcher)
}
cleanup := func() {
for _, searcher := range batch {
if searcher != nil {
_ = searcher.Close()
}
}
}
finalSearcher, err = optimizeCompositeSearcher(ctx, "disjunction:unadorned",
indexReader, batch, options)
// all searchers in batch should be closed, regardless of error or optimization failure
// either we're returning, or continuing and only finalSearcher is needed for next loop
cleanup()
if err != nil {
return nil, err
}
if finalSearcher == nil {
return nil, fmt.Errorf("unable to optimize")
}
}
return finalSearcher, nil
}
func makeBatchSearchers(ctx context.Context, indexReader index.IndexReader, terms []string, field string,
boost float64, options search.SearcherOptions) ([]search.Searcher, error) {
qsearchers := make([]search.Searcher, len(terms))
qsearchersClose := func() {
for _, searcher := range qsearchers {
if searcher != nil {
_ = searcher.Close()
}
}
}
for i, term := range terms {
var err error
qsearchers[i], err = NewTermSearcher(ctx, indexReader, term, field, boost, options)
if err != nil {
qsearchersClose()
return nil, err
}
}
return qsearchers, nil
}
func makeBatchSearchersBoosted(ctx context.Context, indexReader index.IndexReader, terms []string, field string,
boost float64, editDistances []uint8, options search.SearcherOptions) ([]search.Searcher, error) {
qsearchers := make([]search.Searcher, len(terms))
qsearchersClose := func() {
for _, searcher := range qsearchers {
if searcher != nil {
_ = searcher.Close()
}
}
}
for i, term := range terms {
var err error
var editMultiplier float64
if editDistances != nil {
editMultiplier = 1 / float64(editDistances[i]+1)
}
qsearchers[i], err = NewTermSearcher(ctx, indexReader, term, field, boost*editMultiplier, options)
if err != nil {
qsearchersClose()
return nil, err
}
}
return qsearchers, nil
}
func optimizeMultiTermSearcherBytes(ctx context.Context, indexReader index.IndexReader, terms [][]byte,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
var finalSearcher search.Searcher
for len(terms) > 0 {
var batchTerms [][]byte
if len(terms) > DisjunctionMaxClauseCount {
batchTerms = terms[:DisjunctionMaxClauseCount]
terms = terms[DisjunctionMaxClauseCount:]
} else {
batchTerms = terms
terms = nil
}
batch, err := makeBatchSearchersBytes(ctx, indexReader, batchTerms, field, boost, options)
if err != nil {
return nil, err
}
if finalSearcher != nil {
batch = append(batch, finalSearcher)
}
cleanup := func() {
for _, searcher := range batch {
if searcher != nil {
_ = searcher.Close()
}
}
}
finalSearcher, err = optimizeCompositeSearcher(ctx, "disjunction:unadorned",
indexReader, batch, options)
// all searchers in batch should be closed, regardless of error or optimization failure
// either we're returning, or continuing and only finalSearcher is needed for next loop
cleanup()
if err != nil {
return nil, err
}
if finalSearcher == nil {
return nil, fmt.Errorf("unable to optimize")
}
}
return finalSearcher, nil
}
func makeBatchSearchersBytes(ctx context.Context, indexReader index.IndexReader, terms [][]byte, field string,
boost float64, options search.SearcherOptions) ([]search.Searcher, error) {
qsearchers := make([]search.Searcher, len(terms))
qsearchersClose := func() {
for _, searcher := range qsearchers {
if searcher != nil {
_ = searcher.Close()
}
}
}
for i, term := range terms {
var err error
qsearchers[i], err = NewTermSearcherBytes(ctx, indexReader, term, field, boost, options)
if err != nil {
qsearchersClose()
return nil, err
}
}
return qsearchers, nil
}

View file

@ -0,0 +1,258 @@
// 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 searcher
import (
"bytes"
"context"
"math"
"sort"
"github.com/blevesearch/bleve/v2/numeric"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewNumericRangeSearcher(ctx context.Context, indexReader index.IndexReader,
min *float64, max *float64, inclusiveMin, inclusiveMax *bool, field string,
boost float64, options search.SearcherOptions) (search.Searcher, error) {
// account for unbounded edges
if min == nil {
negInf := math.Inf(-1)
min = &negInf
}
if max == nil {
Inf := math.Inf(1)
max = &Inf
}
if inclusiveMin == nil {
defaultInclusiveMin := true
inclusiveMin = &defaultInclusiveMin
}
if inclusiveMax == nil {
defaultInclusiveMax := false
inclusiveMax = &defaultInclusiveMax
}
// find all the ranges
minInt64 := numeric.Float64ToInt64(*min)
if !*inclusiveMin && minInt64 != math.MaxInt64 {
minInt64++
}
maxInt64 := numeric.Float64ToInt64(*max)
if !*inclusiveMax && maxInt64 != math.MinInt64 {
maxInt64--
}
var fieldDict index.FieldDictContains
var dictBytesRead uint64
var isIndexed filterFunc
var err error
if irr, ok := indexReader.(index.IndexReaderContains); ok {
fieldDict, err = irr.FieldDictContains(field)
if err != nil {
return nil, err
}
isIndexed = func(term []byte) bool {
found, err := fieldDict.Contains(term)
return err == nil && found
}
dictBytesRead = fieldDict.BytesRead()
}
// FIXME hard-coded precision, should match field declaration
termRanges := splitInt64Range(minInt64, maxInt64, 4)
terms := termRanges.Enumerate(isIndexed)
if fieldDict != nil {
if fd, ok := fieldDict.(index.FieldDict); ok {
if err = fd.Close(); err != nil {
return nil, err
}
}
}
if len(terms) < 1 {
// reporting back the IO stats with respect to the dictionary
// loaded, using the context
if ctx != nil {
reportIOStats(ctx, dictBytesRead)
search.RecordSearchCost(ctx, search.AddM, dictBytesRead)
}
// cannot return MatchNoneSearcher because of interaction with
// commit f391b991c20f02681bacd197afc6d8aed444e132
return NewMultiTermSearcherBytes(ctx, indexReader, terms, field,
boost, options, true)
}
// for upside_down
if isIndexed == nil {
terms, err = filterCandidateTerms(indexReader, terms, field)
if err != nil {
return nil, err
}
}
if tooManyClauses(len(terms)) {
return nil, tooManyClausesErr(field, len(terms))
}
if ctx != nil {
reportIOStats(ctx, dictBytesRead)
search.RecordSearchCost(ctx, search.AddM, dictBytesRead)
}
return NewMultiTermSearcherBytes(ctx, indexReader, terms, field,
boost, options, true)
}
func filterCandidateTerms(indexReader index.IndexReader,
terms [][]byte, field string) (rv [][]byte, err error) {
fieldDict, err := indexReader.FieldDictRange(field, terms[0], terms[len(terms)-1])
if err != nil {
return nil, err
}
// enumerate the terms and check against list of terms
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
termBytes := []byte(tfd.Term)
i := sort.Search(len(terms), func(i int) bool { return bytes.Compare(terms[i], termBytes) >= 0 })
if i < len(terms) && bytes.Compare(terms[i], termBytes) == 0 {
rv = append(rv, terms[i])
}
terms = terms[i:]
tfd, err = fieldDict.Next()
}
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
return rv, err
}
type termRange struct {
startTerm []byte
endTerm []byte
}
func (t *termRange) Enumerate(filter filterFunc) [][]byte {
var rv [][]byte
next := t.startTerm
for bytes.Compare(next, t.endTerm) <= 0 {
if filter != nil {
if filter(next) {
rv = append(rv, next)
}
} else {
rv = append(rv, next)
}
next = incrementBytes(next)
}
return rv
}
func incrementBytes(in []byte) []byte {
rv := make([]byte, len(in))
copy(rv, in)
for i := len(rv) - 1; i >= 0; i-- {
rv[i] = rv[i] + 1
if rv[i] != 0 {
// didn't overflow, so stop
break
}
}
return rv
}
type termRanges []*termRange
func (tr termRanges) Enumerate(filter filterFunc) [][]byte {
var rv [][]byte
for _, tri := range tr {
trie := tri.Enumerate(filter)
rv = append(rv, trie...)
}
return rv
}
func splitInt64Range(minBound, maxBound int64, precisionStep uint) termRanges {
rv := make(termRanges, 0)
if minBound > maxBound {
return rv
}
for shift := uint(0); ; shift += precisionStep {
diff := int64(1) << (shift + precisionStep)
mask := ((int64(1) << precisionStep) - int64(1)) << shift
hasLower := (minBound & mask) != int64(0)
hasUpper := (maxBound & mask) != mask
var nextMinBound int64
if hasLower {
nextMinBound = (minBound + diff) &^ mask
} else {
nextMinBound = minBound &^ mask
}
var nextMaxBound int64
if hasUpper {
nextMaxBound = (maxBound - diff) &^ mask
} else {
nextMaxBound = maxBound &^ mask
}
lowerWrapped := nextMinBound < minBound
upperWrapped := nextMaxBound > maxBound
if shift+precisionStep >= 64 || nextMinBound > nextMaxBound ||
lowerWrapped || upperWrapped {
// We are in the lowest precision or the next precision is not available.
rv = append(rv, newRange(minBound, maxBound, shift))
// exit the split recursion loop
break
}
if hasLower {
rv = append(rv, newRange(minBound, minBound|mask, shift))
}
if hasUpper {
rv = append(rv, newRange(maxBound&^mask, maxBound, shift))
}
// recurse to next precision
minBound = nextMinBound
maxBound = nextMaxBound
}
return rv
}
func newRange(minBound, maxBound int64, shift uint) *termRange {
maxBound |= (int64(1) << shift) - int64(1)
minBytes := numeric.MustNewPrefixCodedInt64(minBound, shift)
maxBytes := numeric.MustNewPrefixCodedInt64(maxBound, shift)
return newRangeBytes(minBytes, maxBytes)
}
func newRangeBytes(minBytes, maxBytes []byte) *termRange {
return &termRange{
startTerm: minBytes,
endTerm: maxBytes,
}
}

View file

@ -0,0 +1,60 @@
// 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 searcher
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/numeric"
)
func TestSplitRange(t *testing.T) {
min := numeric.Float64ToInt64(1.0)
max := numeric.Float64ToInt64(5.0)
ranges := splitInt64Range(min, max, 4)
enumerated := ranges.Enumerate(nil)
if len(enumerated) != 135 {
t.Errorf("expected 135 terms, got %d", len(enumerated))
}
}
func TestIncrementBytes(t *testing.T) {
tests := []struct {
in []byte
out []byte
}{
{
in: []byte{0},
out: []byte{1},
},
{
in: []byte{0, 0},
out: []byte{0, 1},
},
{
in: []byte{0, 255},
out: []byte{1, 0},
},
}
for _, test := range tests {
actual := incrementBytes(test.in)
if !reflect.DeepEqual(actual, test.out) {
t.Errorf("expected %#v, got %#v", test.out, actual)
}
}
}

View file

@ -0,0 +1,554 @@
// 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 searcher
import (
"context"
"fmt"
"math"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizePhraseSearcher int
func init() {
var ps PhraseSearcher
reflectStaticSizePhraseSearcher = int(reflect.TypeOf(ps).Size())
}
type PhraseSearcher struct {
mustSearcher search.Searcher
queryNorm float64
currMust *search.DocumentMatch
terms [][]string
path phrasePath
paths []phrasePath
locations []search.Location
initialized bool
// map a term to a list of fuzzy terms that match it
fuzzyTermMatches map[string][]string
}
func (s *PhraseSearcher) Size() int {
sizeInBytes := reflectStaticSizePhraseSearcher + size.SizeOfPtr
if s.mustSearcher != nil {
sizeInBytes += s.mustSearcher.Size()
}
if s.currMust != nil {
sizeInBytes += s.currMust.Size()
}
for _, entry := range s.terms {
sizeInBytes += size.SizeOfSlice
for _, entry1 := range entry {
sizeInBytes += size.SizeOfString + len(entry1)
}
}
return sizeInBytes
}
func NewPhraseSearcher(ctx context.Context, indexReader index.IndexReader, terms []string,
fuzziness int, autoFuzzy bool, field string, boost float64, options search.SearcherOptions) (*PhraseSearcher, error) {
// turn flat terms []string into [][]string
mterms := make([][]string, len(terms))
for i, term := range terms {
mterms[i] = []string{term}
}
return NewMultiPhraseSearcher(ctx, indexReader, mterms, fuzziness, autoFuzzy, field, boost, options)
}
func NewMultiPhraseSearcher(ctx context.Context, indexReader index.IndexReader, terms [][]string,
fuzziness int, autoFuzzy bool, field string, boost float64, options search.SearcherOptions) (*PhraseSearcher, error) {
options.IncludeTermVectors = true
var termPositionSearchers []search.Searcher
var err error
var ts search.Searcher
// The following logic checks if fuzziness is enabled.
// Fuzziness is considered enabled if either:
// a. `fuzziness` is greater than 0, or
// b. `autoFuzzy` is set to true.
// if both conditions are true, `autoFuzzy` takes precedence.
// If enabled, a map will be created to store the matches for fuzzy terms.
fuzzinessEnabled := autoFuzzy || fuzziness > 0
var fuzzyTermMatches map[string][]string
if fuzzinessEnabled {
fuzzyTermMatches = make(map[string][]string)
ctx = context.WithValue(ctx, search.FuzzyMatchPhraseKey, fuzzyTermMatches)
}
// in case of fuzzy multi-phrase, phrase and match-phrase queries we hardcode the
// prefix length to 0, as setting a per word matching prefix length would not
// make sense from a user perspective.
for _, termPos := range terms {
if len(termPos) == 1 && termPos[0] != "" {
// single term
if fuzzinessEnabled {
// fuzzy
if autoFuzzy {
// auto fuzzy
ts, err = NewAutoFuzzySearcher(ctx, indexReader, termPos[0], 0, field, boost, options)
} else {
// non-auto fuzzy
ts, err = NewFuzzySearcher(ctx, indexReader, termPos[0], 0, fuzziness, field, boost, options)
}
} else {
// non-fuzzy
ts, err = NewTermSearcher(ctx, indexReader, termPos[0], field, boost, options)
}
if err != nil {
// close any searchers already opened
for _, ts := range termPositionSearchers {
_ = ts.Close()
}
return nil, fmt.Errorf("phrase searcher error building term searcher: %v", err)
}
termPositionSearchers = append(termPositionSearchers, ts)
} else if len(termPos) > 1 {
// multiple terms
var termSearchers []search.Searcher
for _, term := range termPos {
if term == "" {
continue
}
if fuzzinessEnabled {
// fuzzy
if autoFuzzy {
// auto fuzzy
ts, err = NewAutoFuzzySearcher(ctx, indexReader, term, 0, field, boost, options)
} else {
// non-auto fuzzy
ts, err = NewFuzzySearcher(ctx, indexReader, term, 0, fuzziness, field, boost, options)
}
} else {
// non-fuzzy
ts, err = NewTermSearcher(ctx, indexReader, term, field, boost, options)
}
if err != nil {
// close any searchers already opened
for _, ts := range termPositionSearchers {
_ = ts.Close()
}
return nil, fmt.Errorf("phrase searcher error building term searcher: %v", err)
}
termSearchers = append(termSearchers, ts)
}
disjunction, err := NewDisjunctionSearcher(ctx, indexReader, termSearchers, 1, options)
if err != nil {
// close any searchers already opened
for _, ts := range termPositionSearchers {
_ = ts.Close()
}
return nil, fmt.Errorf("phrase searcher error building term position disjunction searcher: %v", err)
}
termPositionSearchers = append(termPositionSearchers, disjunction)
}
}
if ctx != nil {
if fts, ok := ctx.Value(search.FieldTermSynonymMapKey).(search.FieldTermSynonymMap); ok {
if ts, exists := fts[field]; exists {
if fuzzinessEnabled {
for term, fuzzyTerms := range fuzzyTermMatches {
fuzzySynonymTerms := make([]string, 0, len(fuzzyTerms))
if s, found := ts[term]; found {
fuzzySynonymTerms = append(fuzzySynonymTerms, s...)
}
for _, fuzzyTerm := range fuzzyTerms {
if fuzzyTerm == term {
continue
}
if s, found := ts[fuzzyTerm]; found {
fuzzySynonymTerms = append(fuzzySynonymTerms, s...)
}
}
if len(fuzzySynonymTerms) > 0 {
fuzzyTermMatches[term] = append(fuzzyTermMatches[term], fuzzySynonymTerms...)
}
}
} else {
for _, termPos := range terms {
for _, term := range termPos {
if s, found := ts[term]; found {
if fuzzyTermMatches == nil {
fuzzyTermMatches = make(map[string][]string)
}
fuzzyTermMatches[term] = s
}
}
}
}
}
}
}
mustSearcher, err := NewConjunctionSearcher(ctx, indexReader, termPositionSearchers, options)
if err != nil {
// close any searchers already opened
for _, ts := range termPositionSearchers {
_ = ts.Close()
}
return nil, fmt.Errorf("phrase searcher error building conjunction searcher: %v", err)
}
// build our searcher
rv := PhraseSearcher{
mustSearcher: mustSearcher,
terms: terms,
fuzzyTermMatches: fuzzyTermMatches,
}
rv.computeQueryNorm()
return &rv, nil
}
func (s *PhraseSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
if s.mustSearcher != nil {
sumOfSquaredWeights += s.mustSearcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
if s.mustSearcher != nil {
s.mustSearcher.SetQueryNorm(s.queryNorm)
}
}
func (s *PhraseSearcher) initSearchers(ctx *search.SearchContext) error {
err := s.advanceNextMust(ctx)
if err != nil {
return err
}
s.initialized = true
return nil
}
func (s *PhraseSearcher) advanceNextMust(ctx *search.SearchContext) error {
var err error
if s.mustSearcher != nil {
if s.currMust != nil {
ctx.DocumentMatchPool.Put(s.currMust)
}
s.currMust, err = s.mustSearcher.Next(ctx)
if err != nil {
return err
}
}
return nil
}
func (s *PhraseSearcher) Weight() float64 {
return s.mustSearcher.Weight()
}
func (s *PhraseSearcher) SetQueryNorm(qnorm float64) {
s.mustSearcher.SetQueryNorm(qnorm)
}
func (s *PhraseSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
for s.currMust != nil {
// check this match against phrase constraints
rv := s.checkCurrMustMatch(ctx)
// prepare for next iteration (either loop or subsequent call to Next())
err := s.advanceNextMust(ctx)
if err != nil {
return nil, err
}
// if match satisfied phrase constraints return it as a hit
if rv != nil {
return rv, nil
}
}
return nil, nil
}
// checkCurrMustMatch is solely concerned with determining if the DocumentMatch
// pointed to by s.currMust (which satisifies the pre-condition searcher)
// also satisfies the phrase constraints. if so, it returns a DocumentMatch
// for this document, otherwise nil
func (s *PhraseSearcher) checkCurrMustMatch(ctx *search.SearchContext) *search.DocumentMatch {
s.locations = s.currMust.Complete(s.locations)
locations := s.currMust.Locations
s.currMust.Locations = nil
ftls := s.currMust.FieldTermLocations
// typically we would expect there to only actually be results in
// one field, but we allow for this to not be the case
// but, we note that phrase constraints can only be satisfied within
// a single field, so we can check them each independently
for field, tlm := range locations {
ftls = s.checkCurrMustMatchField(ctx, field, tlm, ftls)
}
if len(ftls) > 0 {
// return match
rv := s.currMust
s.currMust = nil
rv.FieldTermLocations = ftls
return rv
}
return nil
}
// checkCurrMustMatchField is solely concerned with determining if one
// particular field within the currMust DocumentMatch Locations
// satisfies the phrase constraints (possibly more than once). if so,
// the matching field term locations are appended to the provided
// slice
func (s *PhraseSearcher) checkCurrMustMatchField(ctx *search.SearchContext,
field string, tlm search.TermLocationMap,
ftls []search.FieldTermLocation) []search.FieldTermLocation {
if s.path == nil {
s.path = make(phrasePath, 0, len(s.terms))
}
var tlmPtr *search.TermLocationMap = &tlm
if s.fuzzyTermMatches != nil {
// if fuzzy search, we need to expand the tlm to include all the fuzzy matches
// Example - term is "foo" and fuzzy matches are "foo", "fool", "food"
// the non expanded tlm will be:
// foo -> Locations[foo]
// fool -> Locations[fool]
// food -> Locations[food]
// the expanded tlm will be:
// foo -> [Locations[foo], Locations[fool], Locations[food]]
expandedTlm := make(search.TermLocationMap)
s.expandFuzzyMatches(tlm, expandedTlm)
tlmPtr = &expandedTlm
}
s.paths = findPhrasePaths(0, nil, s.terms, *tlmPtr, s.path[:0], 0, s.paths[:0])
for _, p := range s.paths {
for _, pp := range p {
ftls = append(ftls, search.FieldTermLocation{
Field: field,
Term: pp.term,
Location: search.Location{
Pos: pp.loc.Pos,
Start: pp.loc.Start,
End: pp.loc.End,
ArrayPositions: pp.loc.ArrayPositions,
},
})
}
}
return ftls
}
func (s *PhraseSearcher) expandFuzzyMatches(tlm search.TermLocationMap, expandedTlm search.TermLocationMap) {
for term, fuzzyMatches := range s.fuzzyTermMatches {
locations := tlm[term]
for _, fuzzyMatch := range fuzzyMatches {
if fuzzyMatch == term {
continue
}
locations = append(locations, tlm[fuzzyMatch]...)
}
expandedTlm[term] = locations
}
}
type phrasePart struct {
term string
loc *search.Location
}
func (p *phrasePart) String() string {
return fmt.Sprintf("[%s %v]", p.term, p.loc)
}
type phrasePath []phrasePart
func (p phrasePath) MergeInto(in search.TermLocationMap) {
for _, pp := range p {
in[pp.term] = append(in[pp.term], pp.loc)
}
}
func (p phrasePath) String() string {
rv := "["
for i, pp := range p {
if i > 0 {
rv += ", "
}
rv += pp.String()
}
rv += "]"
return rv
}
// findPhrasePaths is a function to identify phrase matches from a set
// of known term locations. it recursive so care must be taken with
// arguments and return values.
//
// prevPos - the previous location, 0 on first invocation
//
// ap - array positions of the first candidate phrase part to
// which further recursive phrase parts must match,
// nil on initial invocation or when there are no array positions
//
// phraseTerms - slice containing the phrase terms,
// may contain empty string as placeholder (don't care)
//
// tlm - the Term Location Map containing all relevant term locations
//
// p - the current path being explored (appended to in recursive calls)
// this is the primary state being built during the traversal
//
// remainingSlop - amount of sloppiness that's allowed, which is the
// sum of the editDistances from each matching phrase part, where 0 means no
// sloppiness allowed (all editDistances must be 0), decremented during recursion
//
// rv - the final result being appended to by all the recursive calls
//
// returns slice of paths, or nil if invocation did not find any successful paths
func findPhrasePaths(prevPos uint64, ap search.ArrayPositions, phraseTerms [][]string,
tlm search.TermLocationMap, p phrasePath, remainingSlop int, rv []phrasePath) []phrasePath {
// no more terms
if len(phraseTerms) < 1 {
// snapshot or copy the recursively built phrasePath p and
// append it to the rv, also optimizing by checking if next
// phrasePath item in the rv (which we're about to overwrite)
// is available for reuse
var pcopy phrasePath
if len(rv) < cap(rv) {
pcopy = rv[:len(rv)+1][len(rv)][:0]
}
return append(rv, append(pcopy, p...))
}
car := phraseTerms[0]
cdr := phraseTerms[1:]
// empty term is treated as match (continue)
if len(car) == 0 || (len(car) == 1 && car[0] == "") {
nextPos := prevPos + 1
if prevPos == 0 {
// if prevPos was 0, don't set it to 1 (as thats not a real abs pos)
nextPos = 0 // don't advance nextPos if prevPos was 0
}
return findPhrasePaths(nextPos, ap, cdr, tlm, p, remainingSlop, rv)
}
// locations for this term
for _, carTerm := range car {
locations := tlm[carTerm]
LOCATIONS_LOOP:
for _, loc := range locations {
if prevPos != 0 && !loc.ArrayPositions.Equals(ap) {
// if the array positions are wrong, can't match, try next location
continue
}
// compute distance from previous phrase term
dist := 0
if prevPos != 0 {
dist = editDistance(prevPos+1, loc.Pos)
}
// if enough slop remaining, continue recursively
if prevPos == 0 || (remainingSlop-dist) >= 0 {
// skip if we've already used this term+loc already
for _, ppart := range p {
if ppart.term == carTerm && ppart.loc == loc {
continue LOCATIONS_LOOP
}
}
// this location works, add it to the path (but not for empty term)
px := append(p, phrasePart{term: carTerm, loc: loc})
rv = findPhrasePaths(loc.Pos, loc.ArrayPositions, cdr, tlm, px, remainingSlop-dist, rv)
}
}
}
return rv
}
func editDistance(p1, p2 uint64) int {
dist := int(p1 - p2)
if dist < 0 {
return -dist
}
return dist
}
func (s *PhraseSearcher) Advance(ctx *search.SearchContext, ID index.IndexInternalID) (*search.DocumentMatch, error) {
if !s.initialized {
err := s.initSearchers(ctx)
if err != nil {
return nil, err
}
}
if s.currMust != nil {
if s.currMust.IndexInternalID.Compare(ID) >= 0 {
return s.Next(ctx)
}
ctx.DocumentMatchPool.Put(s.currMust)
}
if s.currMust == nil {
return nil, nil
}
var err error
s.currMust, err = s.mustSearcher.Advance(ctx, ID)
if err != nil {
return nil, err
}
return s.Next(ctx)
}
func (s *PhraseSearcher) Count() uint64 {
// for now return a worst case
return s.mustSearcher.Count()
}
func (s *PhraseSearcher) Close() error {
if s.mustSearcher != nil {
err := s.mustSearcher.Close()
if err != nil {
return err
}
}
return nil
}
func (s *PhraseSearcher) Min() int {
return 0
}
func (s *PhraseSearcher) DocumentMatchPoolSize() int {
return s.mustSearcher.DocumentMatchPoolSize() + 1
}

View file

@ -0,0 +1,818 @@
// Copyright (c) 2013 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 searcher
import (
"context"
"reflect"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestPhraseSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
soptions := search.SearcherOptions{Explain: true, IncludeTermVectors: true}
phraseSearcher, err := NewPhraseSearcher(context.TODO(), twoDocIndexReader, []string{"angst", "beer"}, 0, false, "desc", 1.0, soptions)
if err != nil {
t.Fatal(err)
}
tests := []struct {
searcher search.Searcher
results []*search.DocumentMatch
locations map[string]map[string][]search.Location
fieldterms [][2]string
}{
{
searcher: phraseSearcher,
results: []*search.DocumentMatch{
{
IndexInternalID: index.IndexInternalID("2"),
Score: 1.0807601687084403,
},
},
locations: map[string]map[string][]search.Location{"desc": {"beer": {{Pos: 2, Start: 6, End: 10}}, "angst": {{Pos: 1, Start: 0, End: 5}}}},
fieldterms: [][2]string{{"desc", "beer"}, {"desc", "angst"}},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
next.Complete(nil)
if i < len(test.results) {
if !next.IndexInternalID.Equals(test.results[i].IndexInternalID) {
t.Errorf("expected result %d to have id %s got %s for test %d\n", i, test.results[i].IndexInternalID, next.IndexInternalID, testIndex)
}
if next.Score != test.results[i].Score {
t.Errorf("expected result %d to have score %v got %v for test %d\n", i, test.results[i].Score, next.Score, testIndex)
t.Logf("scoring explanation: %s\n", next.Expl)
}
for _, ft := range test.fieldterms {
locs := next.Locations[ft[0]][ft[1]]
explocs := test.locations[ft[0]][ft[1]]
if len(explocs) != len(locs) {
t.Fatalf("expected result %d to have %d Locations (%#v) but got %d (%#v) for test %d with field %q and term %q\n", i, len(explocs), explocs, len(locs), locs, testIndex, ft[0], ft[1])
}
for ind, exploc := range explocs {
if !reflect.DeepEqual(*locs[ind], exploc) {
t.Errorf("expected result %d to have Location %v got %v for test %d\n", i, exploc, locs[ind], testIndex)
}
}
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.results) != i {
t.Errorf("expected %d results got %d for test %d", len(test.results), i, testIndex)
}
}
}
func TestMultiPhraseSearch(t *testing.T) {
soptions := search.SearcherOptions{Explain: true, IncludeTermVectors: true}
tests := []struct {
phrase [][]string
docids [][]byte
}{
{
phrase: [][]string{{"angst", "what"}, {"beer"}},
docids: [][]byte{[]byte("2")},
},
}
for i, test := range tests {
reader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
searcher, err := NewMultiPhraseSearcher(context.TODO(), reader, test.phrase, 0, false, "desc", 1.0, soptions)
if err != nil {
t.Error(err)
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(searcher.DocumentMatchPoolSize(), 0),
}
next, err := searcher.Next(ctx)
var actualIds [][]byte
for err == nil && next != nil {
actualIds = append(actualIds, next.IndexInternalID)
ctx.DocumentMatchPool.Put(next)
next, err = searcher.Next(ctx)
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, i)
}
if !reflect.DeepEqual(test.docids, actualIds) {
t.Fatalf("expected ids: %v, got %v", test.docids, actualIds)
}
err = searcher.Close()
if err != nil {
t.Error(err)
}
err = reader.Close()
if err != nil {
t.Error(err)
}
}
}
func TestFuzzyMultiPhraseSearch(t *testing.T) {
soptions := search.SearcherOptions{Explain: true, IncludeTermVectors: true}
tests := []struct {
mphrase [][]string
docids [][]byte
fuzziness int
prefix int
}{
{
mphrase: [][]string{{"pale", "anger"}, {"best"}, {"colon", "porch"}},
docids: [][]byte{[]byte("2"), []byte("3")},
fuzziness: 2,
},
{
mphrase: [][]string{{"pale", "anger"}, {}, {"colon", "porch", "could"}},
docids: nil,
fuzziness: 1,
},
{
mphrase: [][]string{{"app"}, {"best"}, {"volume"}},
docids: [][]byte{[]byte("3")},
fuzziness: 2,
},
{
mphrase: [][]string{{"anger", "pale", "bar"}, {"beard"}, {}, {}},
docids: [][]byte{[]byte("1"), []byte("2"), []byte("3"), []byte("4")},
fuzziness: 2,
},
{
mphrase: [][]string{{"anger", "pale", "bar"}, {}, {"beard"}, {}},
docids: [][]byte{[]byte("1"), []byte("4")},
fuzziness: 2,
},
}
for i, test := range tests {
reader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
searcher, err := NewMultiPhraseSearcher(context.TODO(), reader, test.mphrase, test.fuzziness, false, "desc", 1.0, soptions)
if err != nil {
t.Error(err)
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(searcher.DocumentMatchPoolSize(), 0),
}
next, err := searcher.Next(ctx)
var actualIds [][]byte
for err == nil && next != nil {
actualIds = append(actualIds, next.IndexInternalID)
ctx.DocumentMatchPool.Put(next)
next, err = searcher.Next(ctx)
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, i)
}
if !reflect.DeepEqual(test.docids, actualIds) {
t.Fatalf("expected ids: %v, got %v", test.docids, actualIds)
}
err = searcher.Close()
if err != nil {
t.Error(err)
}
err = reader.Close()
if err != nil {
t.Error(err)
}
}
}
func TestFindPhrasePaths(t *testing.T) {
tests := []struct {
phrase [][]string
tlm search.TermLocationMap
paths []phrasePath
}{
// simplest matching case
{
phrase: [][]string{{"cat"}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
},
"dog": search.Locations{
&search.Location{
Pos: 2,
},
},
},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
},
},
// second term missing, no match
{
phrase: [][]string{{"cat"}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
},
},
paths: nil,
},
// second term exists but in wrong position
{
phrase: [][]string{{"cat"}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
},
"dog": search.Locations{
&search.Location{
Pos: 3,
},
},
},
paths: nil,
},
// matches multiple times
{
phrase: [][]string{{"cat"}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
&search.Location{
Pos: 8,
},
},
"dog": search.Locations{
&search.Location{
Pos: 2,
},
&search.Location{
Pos: 9,
},
},
},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
{
phrasePart{"cat", &search.Location{Pos: 8}},
phrasePart{"dog", &search.Location{Pos: 9}},
},
},
},
// match over gaps
{
phrase: [][]string{{"cat"}, {""}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
},
"dog": search.Locations{
&search.Location{
Pos: 3,
},
},
},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 3}},
},
},
},
// match with leading ""
{
phrase: [][]string{{""}, {"cat"}, {"dog"}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 2,
},
},
"dog": search.Locations{
&search.Location{
Pos: 3,
},
},
},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 2}},
phrasePart{"dog", &search.Location{Pos: 3}},
},
},
},
// match with trailing ""
{
phrase: [][]string{{"cat"}, {"dog"}, {""}},
tlm: search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 2,
},
},
"dog": search.Locations{
&search.Location{
Pos: 3,
},
},
},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 2}},
phrasePart{"dog", &search.Location{Pos: 3}},
},
},
},
}
for i, test := range tests {
actualPaths := findPhrasePaths(0, nil, test.phrase, test.tlm, nil, 0, nil)
if !reflect.DeepEqual(actualPaths, test.paths) {
t.Fatalf("expected: %v got %v for test %d", test.paths, actualPaths, i)
}
}
}
func TestFindPhrasePathsSloppy(t *testing.T) {
tlm := search.TermLocationMap{
"one": search.Locations{
&search.Location{
Pos: 1,
},
},
"two": search.Locations{
&search.Location{
Pos: 2,
},
},
"three": search.Locations{
&search.Location{
Pos: 3,
},
},
"four": search.Locations{
&search.Location{
Pos: 4,
},
},
"five": search.Locations{
&search.Location{
Pos: 5,
},
},
}
tests := []struct {
phrase [][]string
paths []phrasePath
slop int
tlm search.TermLocationMap
}{
// no match
{
phrase: [][]string{{"one"}, {"five"}},
slop: 2,
},
// should match
{
phrase: [][]string{{"one"}, {"five"}},
slop: 3,
paths: []phrasePath{
{
phrasePart{"one", &search.Location{Pos: 1}},
phrasePart{"five", &search.Location{Pos: 5}},
},
},
},
// slop 0 finds exact match
{
phrase: [][]string{{"four"}, {"five"}},
slop: 0,
paths: []phrasePath{
{
phrasePart{"four", &search.Location{Pos: 4}},
phrasePart{"five", &search.Location{Pos: 5}},
},
},
},
// slop 0 does not find exact match (reversed)
{
phrase: [][]string{{"two"}, {"one"}},
slop: 0,
},
// slop 1 finds exact match
{
phrase: [][]string{{"one"}, {"two"}},
slop: 1,
paths: []phrasePath{
{
phrasePart{"one", &search.Location{Pos: 1}},
phrasePart{"two", &search.Location{Pos: 2}},
},
},
},
// slop 1 *still* does not find exact match (reversed) requires at least 2
{
phrase: [][]string{{"two"}, {"one"}},
slop: 1,
},
// slop 2 does finds exact match reversed
{
phrase: [][]string{{"two"}, {"one"}},
slop: 2,
paths: []phrasePath{
{
phrasePart{"two", &search.Location{Pos: 2}},
phrasePart{"one", &search.Location{Pos: 1}},
},
},
},
// slop 2 not enough for this
{
phrase: [][]string{{"three"}, {"one"}},
slop: 2,
},
// slop should be cumulative
{
phrase: [][]string{{"one"}, {"three"}, {"five"}},
slop: 2,
paths: []phrasePath{
{
phrasePart{"one", &search.Location{Pos: 1}},
phrasePart{"three", &search.Location{Pos: 3}},
phrasePart{"five", &search.Location{Pos: 5}},
},
},
},
// should require 6
{
phrase: [][]string{{"five"}, {"three"}, {"one"}},
slop: 5,
},
// so lets try 6
{
phrase: [][]string{{"five"}, {"three"}, {"one"}},
slop: 6,
paths: []phrasePath{
{
phrasePart{"five", &search.Location{Pos: 5}},
phrasePart{"three", &search.Location{Pos: 3}},
phrasePart{"one", &search.Location{Pos: 1}},
},
},
},
// test an append() related edge case, where append()'s
// current behavior needs to be called 3 times starting from a
// nil slice before it grows to a slice with extra capacity --
// hence, 3 initial terms of ark, bat, cat
{
phrase: [][]string{
{"ark"}, {"bat"}, {"cat"}, {"dog"},
},
slop: 1,
paths: []phrasePath{
{
phrasePart{"ark", &search.Location{Pos: 1}},
phrasePart{"bat", &search.Location{Pos: 2}},
phrasePart{"cat", &search.Location{Pos: 3}},
phrasePart{"dog", &search.Location{Pos: 4}},
},
{
phrasePart{"ark", &search.Location{Pos: 1}},
phrasePart{"bat", &search.Location{Pos: 2}},
phrasePart{"cat", &search.Location{Pos: 3}},
phrasePart{"dog", &search.Location{Pos: 5}},
},
},
tlm: search.TermLocationMap{ // ark bat cat dog dog
"ark": search.Locations{
&search.Location{Pos: 1},
},
"bat": search.Locations{
&search.Location{Pos: 2},
},
"cat": search.Locations{
&search.Location{Pos: 3},
},
"dog": search.Locations{
&search.Location{Pos: 4},
&search.Location{Pos: 5},
},
},
},
// test that we don't see multiple hits from the same location
{
phrase: [][]string{
{"cat"}, {"dog"}, {"dog"},
},
slop: 1,
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
phrasePart{"dog", &search.Location{Pos: 3}},
},
},
tlm: search.TermLocationMap{ // cat dog dog
"cat": search.Locations{
&search.Location{Pos: 1},
},
"dog": search.Locations{
&search.Location{Pos: 2},
&search.Location{Pos: 3},
},
},
},
// test that we don't see multiple hits from the same location
{
phrase: [][]string{
{"cat"}, {"dog"},
},
slop: 10,
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 4}},
},
{
phrasePart{"cat", &search.Location{Pos: 3}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
{
phrasePart{"cat", &search.Location{Pos: 3}},
phrasePart{"dog", &search.Location{Pos: 4}},
},
},
tlm: search.TermLocationMap{ // cat dog cat dog
"cat": search.Locations{
&search.Location{Pos: 1},
&search.Location{Pos: 3},
},
"dog": search.Locations{
&search.Location{Pos: 2},
&search.Location{Pos: 4},
},
},
},
}
for i, test := range tests {
tlmToUse := test.tlm
if tlmToUse == nil {
tlmToUse = tlm
}
actualPaths := findPhrasePaths(0, nil, test.phrase, tlmToUse, nil, test.slop, nil)
if !reflect.DeepEqual(actualPaths, test.paths) {
t.Fatalf("expected: %v got %v for test %d", test.paths, actualPaths, i)
}
}
}
func TestFindPhrasePathsSloppyPalyndrome(t *testing.T) {
tlm := search.TermLocationMap{
"one": search.Locations{
&search.Location{
Pos: 1,
},
&search.Location{
Pos: 5,
},
},
"two": search.Locations{
&search.Location{
Pos: 2,
},
&search.Location{
Pos: 4,
},
},
"three": search.Locations{
&search.Location{
Pos: 3,
},
},
}
tests := []struct {
phrase [][]string
paths []phrasePath
slop int
}{
// search non palyndrone, exact match
{
phrase: [][]string{{"two"}, {"three"}},
slop: 0,
paths: []phrasePath{
{
phrasePart{"two", &search.Location{Pos: 2}},
phrasePart{"three", &search.Location{Pos: 3}},
},
},
},
// same with slop 2 (not required) (find it twice)
{
phrase: [][]string{{"two"}, {"three"}},
slop: 2,
paths: []phrasePath{
{
phrasePart{"two", &search.Location{Pos: 2}},
phrasePart{"three", &search.Location{Pos: 3}},
},
{
phrasePart{"two", &search.Location{Pos: 4}},
phrasePart{"three", &search.Location{Pos: 3}},
},
},
},
// palyndrone reversed
{
phrase: [][]string{{"three"}, {"two"}},
slop: 2,
paths: []phrasePath{
{
phrasePart{"three", &search.Location{Pos: 3}},
phrasePart{"two", &search.Location{Pos: 2}},
},
{
phrasePart{"three", &search.Location{Pos: 3}},
phrasePart{"two", &search.Location{Pos: 4}},
},
},
},
}
for i, test := range tests {
actualPaths := findPhrasePaths(0, nil, test.phrase, tlm, nil, test.slop, nil)
if !reflect.DeepEqual(actualPaths, test.paths) {
t.Fatalf("expected: %v got %v for test %d", test.paths, actualPaths, i)
}
}
}
func TestFindMultiPhrasePaths(t *testing.T) {
tlm := search.TermLocationMap{
"cat": search.Locations{
&search.Location{
Pos: 1,
},
},
"dog": search.Locations{
&search.Location{
Pos: 2,
},
},
"frog": search.Locations{
&search.Location{
Pos: 3,
},
},
}
tests := []struct {
phrase [][]string
paths []phrasePath
}{
// simplest, one of two possible terms matches
{
phrase: [][]string{{"cat", "rat"}, {"dog"}},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
},
},
// two possible terms, neither work
{
phrase: [][]string{{"cat", "rat"}, {"chicken"}},
},
// two possible terms, one works, but out of position with next
{
phrase: [][]string{{"cat", "rat"}, {"frog"}},
},
// matches multiple times, with different pairing
{
phrase: [][]string{{"cat", "dog"}, {"dog", "frog"}},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"dog", &search.Location{Pos: 2}},
},
{
phrasePart{"dog", &search.Location{Pos: 2}},
phrasePart{"frog", &search.Location{Pos: 3}},
},
},
},
// multi-match over a gap
{
phrase: [][]string{{"cat", "rat"}, {""}, {"frog"}},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"frog", &search.Location{Pos: 3}},
},
},
},
// multi-match over a gap (same as before, but with empty term list)
{
phrase: [][]string{{"cat", "rat"}, {}, {"frog"}},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"frog", &search.Location{Pos: 3}},
},
},
},
// multi-match over a gap (same once again, but nil term list)
{
phrase: [][]string{{"cat", "rat"}, nil, {"frog"}},
paths: []phrasePath{
{
phrasePart{"cat", &search.Location{Pos: 1}},
phrasePart{"frog", &search.Location{Pos: 3}},
},
},
},
}
for i, test := range tests {
actualPaths := findPhrasePaths(0, nil, test.phrase, tlm, nil, 0, nil)
if !reflect.DeepEqual(actualPaths, test.paths) {
t.Fatalf("expected: %v got %v for test %d", test.paths, actualPaths, i)
}
}
}

View file

@ -0,0 +1,169 @@
// Copyright (c) 2015 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 searcher
import (
"context"
"regexp"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
// The Regexp interface defines the subset of the regexp.Regexp API
// methods that are used by bleve indexes, allowing callers to pass in
// alternate implementations.
type Regexp interface {
FindStringIndex(s string) (loc []int)
LiteralPrefix() (prefix string, complete bool)
String() string
}
// NewRegexpStringSearcher is similar to NewRegexpSearcher, but
// additionally optimizes for index readers that handle regexp's.
func NewRegexpStringSearcher(ctx context.Context, indexReader index.IndexReader, pattern string,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
ir, ok := indexReader.(index.IndexReaderRegexp)
if !ok {
r, err := regexp.Compile(pattern)
if err != nil {
return nil, err
}
return NewRegexpSearcher(ctx, indexReader, r, field, boost, options)
}
fieldDict, a, err := ir.FieldDictRegexpAutomaton(field, pattern)
if err != nil {
return nil, err
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
var termSet = make(map[string]struct{})
var candidateTerms []string
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
if _, exists := termSet[tfd.Term]; !exists {
termSet[tfd.Term] = struct{}{}
candidateTerms = append(candidateTerms, tfd.Term)
tfd, err = fieldDict.Next()
}
}
if err != nil {
return nil, err
}
if ctx != nil {
if fts, ok := ctx.Value(search.FieldTermSynonymMapKey).(search.FieldTermSynonymMap); ok {
if ts, exists := fts[field]; exists {
for term := range ts {
if _, exists := termSet[term]; exists {
continue
}
if a.MatchesRegex(term) {
termSet[term] = struct{}{}
candidateTerms = append(candidateTerms, term)
}
}
}
}
}
return NewMultiTermSearcher(ctx, indexReader, candidateTerms, field, boost,
options, true)
}
// NewRegexpSearcher creates a searcher which will match documents that
// contain terms which match the pattern regexp. The match must be EXACT
// matching the entire term. The provided regexp SHOULD NOT start with ^
// or end with $ as this can intefere with the implementation. Separately,
// matches will be checked to ensure they match the entire term.
func NewRegexpSearcher(ctx context.Context, indexReader index.IndexReader, pattern Regexp,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
var candidateTerms []string
var regexpCandidates *regexpCandidates
prefixTerm, complete := pattern.LiteralPrefix()
if complete {
// there is no pattern
candidateTerms = []string{prefixTerm}
} else {
var err error
regexpCandidates, err = findRegexpCandidateTerms(indexReader, pattern, field,
prefixTerm)
if err != nil {
return nil, err
}
}
var dictBytesRead uint64
if regexpCandidates != nil {
candidateTerms = regexpCandidates.candidates
dictBytesRead = regexpCandidates.bytesRead
}
if ctx != nil {
reportIOStats(ctx, dictBytesRead)
search.RecordSearchCost(ctx, search.AddM, dictBytesRead)
}
return NewMultiTermSearcher(ctx, indexReader, candidateTerms, field, boost,
options, true)
}
type regexpCandidates struct {
candidates []string
bytesRead uint64
}
func findRegexpCandidateTerms(indexReader index.IndexReader,
pattern Regexp, field, prefixTerm string) (rv *regexpCandidates, err error) {
rv = &regexpCandidates{
candidates: make([]string, 0),
}
var fieldDict index.FieldDict
if len(prefixTerm) > 0 {
fieldDict, err = indexReader.FieldDictPrefix(field, []byte(prefixTerm))
} else {
fieldDict, err = indexReader.FieldDict(field)
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
// enumerate the terms and check against regexp
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
matchPos := pattern.FindStringIndex(tfd.Term)
if matchPos != nil && matchPos[0] == 0 && matchPos[1] == len(tfd.Term) {
rv.candidates = append(rv.candidates, tfd.Term)
if tooManyClauses(len(rv.candidates)) {
return rv, tooManyClausesErr(field, len(rv.candidates))
}
}
tfd, err = fieldDict.Next()
}
rv.bytesRead = fieldDict.BytesRead()
return rv, err
}

View file

@ -0,0 +1,171 @@
// Copyright (c) 2015 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 searcher
import (
"context"
"encoding/binary"
"fmt"
"os"
"regexp"
"testing"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestRegexpSearchUpsideDown(t *testing.T) {
twoDocIndex := initTwoDocUpsideDown()
testRegexpSearch(t, twoDocIndex, internalIDMakerUpsideDown, searcherMaker)
_ = twoDocIndex.Close()
}
func TestRegexpStringSearchUpsideDown(t *testing.T) {
twoDocIndex := initTwoDocUpsideDown()
testRegexpSearch(t, twoDocIndex, internalIDMakerUpsideDown, searcherStringMaker)
_ = twoDocIndex.Close()
}
func TestRegexpSearchScorch(t *testing.T) {
dir, _ := os.MkdirTemp("", "scorchTwoDoc")
defer func() {
_ = os.RemoveAll(dir)
}()
twoDocIndex := initTwoDocScorch(dir)
testRegexpSearch(t, twoDocIndex, internalIDMakerScorch, searcherMaker)
_ = twoDocIndex.Close()
}
func TestRegexpStringSearchScorch(t *testing.T) {
dir, _ := os.MkdirTemp("", "scorchTwoDoc")
defer func() {
_ = os.RemoveAll(dir)
}()
twoDocIndex := initTwoDocScorch(dir)
testRegexpSearch(t, twoDocIndex, internalIDMakerScorch, searcherStringMaker)
_ = twoDocIndex.Close()
}
func internalIDMakerUpsideDown(id int) index.IndexInternalID {
return index.IndexInternalID(fmt.Sprintf("%d", id))
}
func internalIDMakerScorch(id int) index.IndexInternalID {
buf := make([]byte, 8)
binary.BigEndian.PutUint64(buf, uint64(id))
return index.IndexInternalID(buf)
}
func searcherMaker(t *testing.T, ir index.IndexReader, re, field string) search.Searcher {
pattern, err := regexp.Compile(re)
if err != nil {
t.Fatal(err)
}
regexpSearcher, err := NewRegexpSearcher(context.TODO(), ir, pattern, field, 1.0,
search.SearcherOptions{Explain: true})
if err != nil {
t.Fatal(err)
}
return regexpSearcher
}
func searcherStringMaker(t *testing.T, ir index.IndexReader, re, field string) search.Searcher {
regexpSearcher, err := NewRegexpStringSearcher(context.TODO(), ir, re, field, 1.0,
search.SearcherOptions{Explain: true})
if err != nil {
t.Fatal(err)
}
return regexpSearcher
}
func testRegexpSearch(t *testing.T, twoDocIndex index.Index,
internalIDMaker func(int) index.IndexInternalID,
searcherMaker func(t *testing.T, ir index.IndexReader, re, field string) search.Searcher,
) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
regexpSearcher := searcherMaker(t, twoDocIndexReader, "ma.*", "name")
regexpSearcherCo := searcherMaker(t, twoDocIndexReader, "co.*", "desc")
tests := []struct {
searcher search.Searcher
id2score map[string]float64
}{
{
searcher: regexpSearcher,
id2score: map[string]float64{
"1": 1.916290731874155,
},
},
{
searcher: regexpSearcherCo,
id2score: map[string]float64{
"2": 0.33875554280828685,
"3": 0.33875554280828685,
},
},
}
for testIndex, test := range tests {
defer func() {
err := test.searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(test.searcher.DocumentMatchPoolSize(), 0),
}
next, err := test.searcher.Next(ctx)
i := 0
for err == nil && next != nil {
exID, _ := twoDocIndexReader.ExternalID(next.IndexInternalID)
if _, ok := test.id2score[exID]; !ok {
t.Errorf("test %d, found unexpected docID = %v, next = %v", testIndex, exID, next)
} else {
score := test.id2score[exID]
if next.Score != score {
t.Errorf("test %d, expected result %d to have score %v got %v,next: %#v",
testIndex, i, score, next.Score, next)
t.Logf("scoring explanation: %s", next.Expl)
}
}
ctx.DocumentMatchPool.Put(next)
next, err = test.searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v for test %d", err, testIndex)
}
if len(test.id2score) != i {
t.Errorf("expected %d results got %d for test %d", len(test.id2score), i, testIndex)
}
}
}

View file

@ -0,0 +1,282 @@
// 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 searcher
import (
"context"
"fmt"
"math"
"reflect"
"github.com/blevesearch/bleve/v2/search"
"github.com/blevesearch/bleve/v2/search/scorer"
"github.com/blevesearch/bleve/v2/size"
index "github.com/blevesearch/bleve_index_api"
)
var reflectStaticSizeTermSearcher int
func init() {
var ts TermSearcher
reflectStaticSizeTermSearcher = int(reflect.TypeOf(ts).Size())
}
type TermSearcher struct {
indexReader index.IndexReader
reader index.TermFieldReader
scorer *scorer.TermQueryScorer
tfd index.TermFieldDoc
}
func NewTermSearcher(ctx context.Context, indexReader index.IndexReader,
term string, field string, boost float64, options search.SearcherOptions) (search.Searcher, error) {
if isTermQuery(ctx) {
ctx = context.WithValue(ctx, search.QueryTypeKey, search.Term)
}
return NewTermSearcherBytes(ctx, indexReader, []byte(term), field, boost, options)
}
func NewTermSearcherBytes(ctx context.Context, indexReader index.IndexReader,
term []byte, field string, boost float64, options search.SearcherOptions) (search.Searcher, error) {
if ctx != nil {
if fts, ok := ctx.Value(search.FieldTermSynonymMapKey).(search.FieldTermSynonymMap); ok {
if ts, exists := fts[field]; exists {
if s, found := ts[string(term)]; found {
return NewSynonymSearcher(ctx, indexReader, term, s, field, boost, options)
}
}
}
}
needFreqNorm := options.Score != "none"
reader, err := indexReader.TermFieldReader(ctx, term, field, needFreqNorm, needFreqNorm, options.IncludeTermVectors)
if err != nil {
return nil, err
}
return newTermSearcherFromReader(ctx, indexReader, reader, term, field, boost, options)
}
func tfIDFScoreMetrics(indexReader index.IndexReader) (uint64, error) {
// default tf-idf stats
count, err := indexReader.DocCount()
if err != nil {
return 0, err
}
if count == 0 {
return 0, nil
}
return count, nil
}
func bm25ScoreMetrics(ctx context.Context, field string,
indexReader index.IndexReader) (uint64, float64, error) {
var count uint64
var fieldCardinality int
var err error
bm25Stats, ok := ctx.Value(search.BM25StatsKey).(*search.BM25Stats)
if !ok {
count, err = indexReader.DocCount()
if err != nil {
return 0, 0, err
}
if bm25Reader, ok := indexReader.(index.BM25Reader); ok {
fieldCardinality, err = bm25Reader.FieldCardinality(field)
if err != nil {
return 0, 0, err
}
}
} else {
count = uint64(bm25Stats.DocCount)
fieldCardinality, ok = bm25Stats.FieldCardinality[field]
if !ok {
return 0, 0, fmt.Errorf("field stat for bm25 not present %s", field)
}
}
if count == 0 && fieldCardinality == 0 {
return 0, 0, nil
}
return count, math.Ceil(float64(fieldCardinality) / float64(count)), nil
}
func newTermSearcherFromReader(ctx context.Context, indexReader index.IndexReader,
reader index.TermFieldReader, term []byte, field string, boost float64,
options search.SearcherOptions) (*TermSearcher, error) {
var count uint64
var avgDocLength float64
var err error
var similarityModel string
// as a fallback case we track certain stats for tf-idf scoring
if ctx != nil {
if similarityModelCallback, ok := ctx.Value(search.
GetScoringModelCallbackKey).(search.GetScoringModelCallbackFn); ok {
similarityModel = similarityModelCallback()
}
}
switch similarityModel {
case index.BM25Scoring:
count, avgDocLength, err = bm25ScoreMetrics(ctx, field, indexReader)
if err != nil {
_ = reader.Close()
return nil, err
}
case index.TFIDFScoring:
fallthrough
default:
count, err = tfIDFScoreMetrics(indexReader)
if err != nil {
_ = reader.Close()
return nil, err
}
}
scorer := scorer.NewTermQueryScorer(term, field, boost, count, reader.Count(), avgDocLength, options)
return &TermSearcher{
indexReader: indexReader,
reader: reader,
scorer: scorer,
}, nil
}
func NewSynonymSearcher(ctx context.Context, indexReader index.IndexReader, term []byte, synonyms []string, field string, boost float64, options search.SearcherOptions) (search.Searcher, error) {
createTermSearcher := func(term []byte, boostVal float64) (search.Searcher, error) {
needFreqNorm := options.Score != "none"
reader, err := indexReader.TermFieldReader(ctx, term, field, needFreqNorm, needFreqNorm, options.IncludeTermVectors)
if err != nil {
return nil, err
}
return newTermSearcherFromReader(ctx, indexReader, reader, term, field, boostVal, options)
}
// create a searcher for the term itself
termSearcher, err := createTermSearcher(term, boost)
if err != nil {
return nil, err
}
// constituent searchers of the disjunction
qsearchers := make([]search.Searcher, 0, len(synonyms)+1)
// helper method to close all the searchers we've created
// in case of an error
qsearchersClose := func() {
for _, searcher := range qsearchers {
if searcher != nil {
_ = searcher.Close()
}
}
}
qsearchers = append(qsearchers, termSearcher)
// create a searcher for each synonym
for _, synonym := range synonyms {
synonymSearcher, err := createTermSearcher([]byte(synonym), boost/2.0)
if err != nil {
qsearchersClose()
return nil, err
}
qsearchers = append(qsearchers, synonymSearcher)
}
// create a disjunction searcher
rv, err := NewDisjunctionSearcher(ctx, indexReader, qsearchers, 0, options)
if err != nil {
qsearchersClose()
return nil, err
}
return rv, nil
}
func (s *TermSearcher) Size() int {
return reflectStaticSizeTermSearcher + size.SizeOfPtr +
s.reader.Size() +
s.tfd.Size() +
s.scorer.Size()
}
func (s *TermSearcher) Count() uint64 {
return s.reader.Count()
}
func (s *TermSearcher) Weight() float64 {
return s.scorer.Weight()
}
func (s *TermSearcher) SetQueryNorm(qnorm float64) {
s.scorer.SetQueryNorm(qnorm)
}
func (s *TermSearcher) Next(ctx *search.SearchContext) (*search.DocumentMatch, error) {
termMatch, err := s.reader.Next(s.tfd.Reset())
if err != nil {
return nil, err
}
if termMatch == nil {
return nil, nil
}
// score match
docMatch := s.scorer.Score(ctx, termMatch)
// return doc match
return docMatch, nil
}
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
}
if termMatch == nil {
return nil, nil
}
// score match
docMatch := s.scorer.Score(ctx, termMatch)
// return doc match
return docMatch, nil
}
func (s *TermSearcher) Close() error {
return s.reader.Close()
}
func (s *TermSearcher) Min() int {
return 0
}
func (s *TermSearcher) DocumentMatchPoolSize() int {
return 1
}
func (s *TermSearcher) Optimize(kind string, octx index.OptimizableContext) (
index.OptimizableContext, error) {
o, ok := s.reader.(index.Optimizable)
if ok {
return o.Optimize(kind, octx)
}
return nil, nil
}
func isTermQuery(ctx context.Context) bool {
if ctx != nil {
// if the ctx already has a value set for query type
// it would've been done at a non term searcher level.
_, ok := ctx.Value(search.QueryTypeKey).(string)
return !ok
}
// if the context is nil, then don't set the query type
return false
}

View file

@ -0,0 +1,86 @@
// 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 searcher
import (
"context"
"strings"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewTermPrefixSearcher(ctx context.Context, indexReader index.IndexReader, prefix string,
field string, boost float64, options search.SearcherOptions) (
search.Searcher, error) {
// find the terms with this prefix
fieldDict, err := indexReader.FieldDictPrefix(field, []byte(prefix))
if err != nil {
return nil, err
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
var terms []string
var termSet = make(map[string]struct{})
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
if _, exists := termSet[tfd.Term]; !exists {
termSet[tfd.Term] = struct{}{}
terms = append(terms, tfd.Term)
if tooManyClauses(len(terms)) {
return nil, tooManyClausesErr(field, len(terms))
}
tfd, err = fieldDict.Next()
}
}
if err != nil {
return nil, err
}
if ctx != nil {
reportIOStats(ctx, fieldDict.BytesRead())
search.RecordSearchCost(ctx, search.AddM, fieldDict.BytesRead())
}
if ctx != nil {
if fts, ok := ctx.Value(search.FieldTermSynonymMapKey).(search.FieldTermSynonymMap); ok {
if ts, exists := fts[field]; exists {
for term := range ts {
if _, exists := termSet[term]; exists {
continue
}
if strings.HasPrefix(term, prefix) {
termSet[term] = struct{}{}
terms = append(terms, term)
if tooManyClauses(len(terms)) {
return nil, tooManyClausesErr(field, len(terms))
}
}
}
}
}
}
// check if the terms are empty or have one term which is the prefix itself
if len(terms) == 0 || (len(terms) == 1 && terms[0] == prefix) {
return NewTermSearcher(ctx, indexReader, prefix, field, boost, options)
}
return NewMultiTermSearcher(ctx, indexReader, terms, field, boost, options, true)
}

View file

@ -0,0 +1,92 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func NewTermRangeSearcher(ctx context.Context, indexReader index.IndexReader,
min, max []byte, inclusiveMin, inclusiveMax *bool, field string,
boost float64, options search.SearcherOptions) (search.Searcher, error) {
if inclusiveMin == nil {
defaultInclusiveMin := true
inclusiveMin = &defaultInclusiveMin
}
if inclusiveMax == nil {
defaultInclusiveMax := false
inclusiveMax = &defaultInclusiveMax
}
if min == nil {
min = []byte{}
}
rangeMax := max
if rangeMax != nil {
// the term dictionary range end has an unfortunate implementation
rangeMax = append(rangeMax, 0)
}
// find the terms with this prefix
fieldDict, err := indexReader.FieldDictRange(field, min, rangeMax)
if err != nil {
return nil, err
}
defer func() {
if cerr := fieldDict.Close(); cerr != nil && err == nil {
err = cerr
}
}()
var terms []string
tfd, err := fieldDict.Next()
for err == nil && tfd != nil {
terms = append(terms, tfd.Term)
tfd, err = fieldDict.Next()
}
if err != nil {
return nil, err
}
if len(terms) < 1 {
return NewMatchNoneSearcher(indexReader)
}
if !*inclusiveMin && min != nil && string(min) == terms[0] {
terms = terms[1:]
// check again, as we might have removed only entry
if len(terms) < 1 {
return NewMatchNoneSearcher(indexReader)
}
}
// if our term list included the max, it would be the last item
if !*inclusiveMax && max != nil && string(max) == terms[len(terms)-1] {
terms = terms[:len(terms)-1]
}
if ctx != nil {
reportIOStats(ctx, fieldDict.BytesRead())
search.RecordSearchCost(ctx, search.AddM, fieldDict.BytesRead())
}
return NewMultiTermSearcher(ctx, indexReader, terms, field, boost, options, true)
}

View file

@ -0,0 +1,285 @@
// Copyright (c) 2017 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 searcher
import (
"context"
"os"
"reflect"
"sort"
"testing"
"github.com/blevesearch/bleve/v2/index/scorch"
"github.com/blevesearch/bleve/v2/search"
)
func TestTermRangeSearch(t *testing.T) {
twoDocIndexReader, err := twoDocIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := twoDocIndexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
tests := []struct {
min []byte
max []byte
inclusiveMin bool
inclusiveMax bool
field string
want []string
}{
{
min: []byte("marty"),
max: []byte("marty"),
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: []string{"1"},
},
{
min: []byte("marty"),
max: []byte("ravi"),
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: []string{"1", "4"},
},
// inclusive max false should exclude ravi
{
min: []byte("marty"),
max: []byte("ravi"),
field: "name",
inclusiveMin: true,
inclusiveMax: false,
want: []string{"1"},
},
// inclusive max false should remove last/only item
{
min: []byte("martz"),
max: []byte("ravi"),
field: "name",
inclusiveMin: true,
inclusiveMax: false,
want: nil,
},
// inclusive min false should remove marty
{
min: []byte("marty"),
max: []byte("ravi"),
field: "name",
inclusiveMin: false,
inclusiveMax: true,
want: []string{"4"},
},
// inclusive min false should remove first/only item
{
min: []byte("marty"),
max: []byte("rav"),
field: "name",
inclusiveMin: false,
inclusiveMax: true,
want: nil,
},
// max nil sees everything after marty
{
min: []byte("marty"),
max: nil,
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: []string{"1", "2", "4"},
},
// min nil sees everything before ravi
{
min: nil,
max: []byte("ravi"),
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: []string{"1", "3", "4", "5"},
},
// min and max nil sees everything
{
min: nil,
max: nil,
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: []string{"1", "2", "3", "4", "5"},
},
// min and max nil sees everything, even with inclusiveMin false
{
min: nil,
max: nil,
field: "name",
inclusiveMin: false,
inclusiveMax: true,
want: []string{"1", "2", "3", "4", "5"},
},
// min and max nil sees everything, even with inclusiveMax false
{
min: nil,
max: nil,
field: "name",
inclusiveMin: true,
inclusiveMax: false,
want: []string{"1", "2", "3", "4", "5"},
},
// min and max nil sees everything, even with both false
{
min: nil,
max: nil,
field: "name",
inclusiveMin: false,
inclusiveMax: false,
want: []string{"1", "2", "3", "4", "5"},
},
// min and max non-nil, but match 0 terms
{
min: []byte("martz"),
max: []byte("rav"),
field: "name",
inclusiveMin: true,
inclusiveMax: true,
want: nil,
},
// min and max same (and term exists), both exlusive
{
min: []byte("marty"),
max: []byte("marty"),
field: "name",
inclusiveMin: false,
inclusiveMax: false,
want: nil,
},
}
for _, test := range tests {
searcher, err := NewTermRangeSearcher(context.TODO(), twoDocIndexReader, test.min, test.max,
&test.inclusiveMin, &test.inclusiveMax, test.field, 1.0, search.SearcherOptions{Explain: true})
if err != nil {
t.Fatal(err)
}
var got []string
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(
searcher.DocumentMatchPoolSize(), 0),
}
next, err := searcher.Next(ctx)
i := 0
for err == nil && next != nil {
got = append(got, string(next.IndexInternalID))
ctx.DocumentMatchPool.Put(next)
next, err = searcher.Next(ctx)
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v", err)
}
if !reflect.DeepEqual(got, test.want) {
t.Errorf("expected: %v, got %v for test %#v", test.want, got, test)
}
}
}
func TestTermRangeSearchTooManyTerms(t *testing.T) {
dir, _ := os.MkdirTemp("", "scorchTwoDoc")
defer func() {
_ = os.RemoveAll(dir)
}()
scorchIndex := initTwoDocScorch(dir)
// use lower limit for this test
origLimit := DisjunctionMaxClauseCount
DisjunctionMaxClauseCount = 2
defer func() {
DisjunctionMaxClauseCount = origLimit
}()
scorchReader, err := scorchIndex.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := scorchReader.Close()
if err != nil {
t.Fatal(err)
}
}()
want := []string{"1", "3", "4", "5"}
truth := true
searcher, err := NewTermRangeSearcher(context.TODO(), scorchReader, []byte("bobert"), []byte("ravi"),
&truth, &truth, "name", 1.0, search.SearcherOptions{Score: "none", IncludeTermVectors: false})
if err != nil {
t.Fatal(err)
}
var got []string
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(
searcher.DocumentMatchPoolSize(), 0),
}
next, err := searcher.Next(ctx)
i := 0
for err == nil && next != nil {
extId, err := scorchReader.ExternalID(next.IndexInternalID)
if err != nil {
t.Fatal(err)
}
got = append(got, extId)
ctx.DocumentMatchPool.Put(next)
next, err = searcher.Next(ctx)
if err != nil {
break
}
i++
}
if err != nil {
t.Fatalf("error iterating searcher: %v", err)
}
err = searcher.Close()
if err != nil {
t.Fatal(err)
}
// check that the expected number of term searchers were started
// 6 = 4 original terms, 1 optimized after first round, then final searcher
// from the last round
statsMap := scorchIndex.(*scorch.Scorch).StatsMap()
if statsMap["term_searchers_started"].(uint64) != 6 {
t.Errorf("expected 6 term searchers started, got %d", statsMap["term_searchers_started"])
}
// check that all started searchers were closed
if statsMap["term_searchers_started"] != statsMap["term_searchers_finished"] {
t.Errorf("expected all term searchers closed, %d started %d closed",
statsMap["term_searchers_started"], statsMap["term_searchers_finished"])
}
sort.Strings(got)
if !reflect.DeepEqual(got, want) {
t.Errorf("expected: %#v, got %#v", want, got)
}
}

View file

@ -0,0 +1,185 @@
// Copyright (c) 2013 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 searcher
import (
"context"
"math"
"testing"
"github.com/blevesearch/bleve/v2/document"
"github.com/blevesearch/bleve/v2/index/upsidedown"
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
"github.com/blevesearch/bleve/v2/search"
index "github.com/blevesearch/bleve_index_api"
)
func TestTermSearcher(t *testing.T) {
queryTerm := "beer"
queryField := "desc"
queryBoost := 3.0
queryExplain := search.SearcherOptions{Explain: true}
analysisQueue := index.NewAnalysisQueue(1)
i, err := upsidedown.NewUpsideDownCouch(
gtreap.Name,
map[string]interface{}{
"path": "",
},
analysisQueue)
if err != nil {
t.Fatal(err)
}
err = i.Open()
if err != nil {
t.Fatal(err)
}
doc := document.NewDocument("a")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("b")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("c")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("d")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("e")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("f")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("g")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("h")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("i")
doc.AddField(document.NewTextField("desc", []uint64{}, []byte("beer")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
doc = document.NewDocument("j")
doc.AddField(document.NewTextField("title", []uint64{}, []byte("cat")))
err = i.Update(doc)
if err != nil {
t.Fatal(err)
}
indexReader, err := i.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
searcher, err := NewTermSearcher(context.TODO(), indexReader, queryTerm, queryField, queryBoost, queryExplain)
if err != nil {
t.Fatal(err)
}
defer func() {
err := searcher.Close()
if err != nil {
t.Fatal(err)
}
}()
searcher.SetQueryNorm(2.0)
docCount, err := indexReader.DocCount()
if err != nil {
t.Fatal(err)
}
idf := 1.0 + math.Log(float64(docCount)/float64(searcher.Count()+1.0))
expectedQueryWeight := 3 * idf * 3 * idf
if expectedQueryWeight != searcher.Weight() {
t.Errorf("expected weight %v got %v", expectedQueryWeight, searcher.Weight())
}
if searcher.Count() != 9 {
t.Errorf("expected count of 9, got %d", searcher.Count())
}
ctx := &search.SearchContext{
DocumentMatchPool: search.NewDocumentMatchPool(1, 0),
}
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)
}
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Advance(ctx, index.IndexInternalID("c"))
if err != nil {
t.Errorf("expected result, got %v", err)
}
if !docMatch.IndexInternalID.Equals(index.IndexInternalID("c")) {
t.Errorf("expected result ID to be 'c' got '%s'", docMatch.IndexInternalID)
}
// try advancing past end
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Advance(ctx, index.IndexInternalID("z"))
if err != nil {
t.Fatal(err)
}
if docMatch != nil {
t.Errorf("expected nil, got %v", docMatch)
}
// try pushing next past end
ctx.DocumentMatchPool.Put(docMatch)
docMatch, err = searcher.Next(ctx)
if err != nil {
t.Fatal(err)
}
if docMatch != nil {
t.Errorf("expected nil, got %v", docMatch)
}
}