0
0
Fork 0

made index type configurable + first version of firestorm

This commit is contained in:
Marty Schoch 2015-08-25 14:52:42 -04:00
parent ae19d77b04
commit 84811cf5a0
74 changed files with 7051 additions and 273 deletions

View File

@ -22,6 +22,10 @@ type TokenFreq struct {
Locations []*TokenLocation
}
func (tf *TokenFreq) Frequency() int {
return len(tf.Locations)
}
type TokenFrequencies []*TokenFreq
func (tfs TokenFrequencies) MergeAll(remoteField string, other TokenFrequencies) TokenFrequencies {

View File

@ -15,7 +15,7 @@ import (
"log"
"time"
"github.com/blevesearch/bleve/index/upside_down"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/registry"
// token maps
@ -106,6 +106,10 @@ import (
_ "github.com/blevesearch/bleve/index/store/gtreap"
_ "github.com/blevesearch/bleve/index/store/inmem"
// index types
_ "github.com/blevesearch/bleve/index/firestorm"
_ "github.com/blevesearch/bleve/index/upside_down"
// byte array converters
_ "github.com/blevesearch/bleve/analysis/byte_array_converters/ignore"
_ "github.com/blevesearch/bleve/analysis/byte_array_converters/json"
@ -118,14 +122,15 @@ type configuration struct {
Cache *registry.Cache
DefaultHighlighter string
DefaultKVStore string
DefaultIndexType string
SlowSearchLogThreshold time.Duration
analysisQueue *upside_down.AnalysisQueue
analysisQueue *index.AnalysisQueue
}
func newConfiguration() *configuration {
return &configuration{
Cache: registry.NewCache(),
analysisQueue: upside_down.NewAnalysisQueue(4),
analysisQueue: index.NewAnalysisQueue(4),
}
}
@ -174,6 +179,9 @@ func init() {
// default kv store
Config.DefaultKVStore = "boltdb"
// default index
Config.DefaultIndexType = "upside_down"
bootDuration := time.Since(bootStart)
bleveExpVar.Add("bootDuration", int64(bootDuration))
}

View File

@ -17,6 +17,7 @@ type Document struct {
ID string `json:"id"`
Fields []Field `json:"fields"`
CompositeFields []*CompositeField
Number uint64 `json:"-"`
}
func NewDocument(id string) *Document {

View File

@ -24,6 +24,7 @@ const (
ErrorIndexClosed
ErrorAliasMulti
ErrorAliasEmpty
ErrorUnknownIndexType
)
// Error represents a more strongly typed bleve error for detecting
@ -48,4 +49,5 @@ var errorMessages = map[int]string{
int(ErrorIndexClosed): "index is closed",
int(ErrorAliasMulti): "cannot perform single index operation on multiple index alias",
int(ErrorAliasEmpty): "cannot perform operation on empty alias",
int(ErrorUnknownIndexType): "unknown index type",
}

View File

@ -126,7 +126,7 @@ type Classifier interface {
// The provided mapping will be used for all
// Index/Search operations.
func New(path string, mapping *IndexMapping) (Index, error) {
return newIndexUsing(path, mapping, Config.DefaultKVStore, nil)
return newIndexUsing(path, mapping, Config.DefaultIndexType, Config.DefaultKVStore, nil)
}
// NewUsing creates index at the specified path,
@ -136,8 +136,8 @@ func New(path string, mapping *IndexMapping) (Index, error) {
// The specified kvstore implemenation will be used
// and the provided kvconfig will be passed to its
// constructor.
func NewUsing(path string, mapping *IndexMapping, kvstore string, kvconfig map[string]interface{}) (Index, error) {
return newIndexUsing(path, mapping, kvstore, kvconfig)
func NewUsing(path string, mapping *IndexMapping, indexType string, kvstore string, kvconfig map[string]interface{}) (Index, error) {
return newIndexUsing(path, mapping, indexType, kvstore, kvconfig)
}
// Open index at the specified path, must exist.

73
index/analysis.go Normal file
View File

@ -0,0 +1,73 @@
// 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 index
import "github.com/blevesearch/bleve/document"
type IndexRow interface {
Key() []byte
Value() []byte
}
type AnalysisResult struct {
DocID string
Rows []IndexRow
}
type AnalysisWork struct {
i Index
d *document.Document
rc chan *AnalysisResult
}
func NewAnalysisWork(i Index, d *document.Document, rc chan *AnalysisResult) *AnalysisWork {
return &AnalysisWork{
i: i,
d: d,
rc: rc,
}
}
type AnalysisQueue struct {
queue chan *AnalysisWork
done chan struct{}
}
func (q *AnalysisQueue) Queue(work *AnalysisWork) {
q.queue <- work
}
func (q *AnalysisQueue) Close() {
close(q.done)
}
func NewAnalysisQueue(numWorkers int) *AnalysisQueue {
rv := AnalysisQueue{
queue: make(chan *AnalysisWork),
done: make(chan struct{}),
}
for i := 0; i < numWorkers; i++ {
go AnalysisWorker(rv)
}
return &rv
}
func AnalysisWorker(q AnalysisQueue) {
// read work off the queue
for {
select {
case <-q.done:
return
case w := <-q.queue:
r := w.i.Analyze(w.d)
w.rc <- r
}
}
}

View File

@ -1,4 +1,4 @@
// Copyright (c) 2014 Couchbase, Inc.
// 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
@ -7,25 +7,26 @@
// either express or implied. See the License for the specific language governing permissions
// and limitations under the License.
package upside_down
package index
import (
"sync"
)
type FieldIndexCache struct {
type FieldCache struct {
fieldIndexes map[string]uint16
lastFieldIndex int
mutex sync.RWMutex
}
func NewFieldIndexCache() *FieldIndexCache {
return &FieldIndexCache{
fieldIndexes: make(map[string]uint16),
func NewFieldCache() *FieldCache {
return &FieldCache{
fieldIndexes: make(map[string]uint16),
lastFieldIndex: -1,
}
}
func (f *FieldIndexCache) AddExisting(field string, index uint16) {
func (f *FieldCache) AddExisting(field string, index uint16) {
f.mutex.Lock()
defer f.mutex.Unlock()
f.fieldIndexes[field] = index
@ -34,30 +35,30 @@ func (f *FieldIndexCache) AddExisting(field string, index uint16) {
}
}
func (f *FieldIndexCache) FieldExists(field string) (uint16, bool) {
// FieldNamed returns the index of the field, and whether or not it existed
// before this call. if createIfMissing is true, and new field index is assigned
// but the second return value will still be false
func (f *FieldCache) FieldNamed(field string, createIfMissing bool) (uint16, bool) {
f.mutex.RLock()
defer f.mutex.RUnlock()
if index, ok := f.fieldIndexes[field]; ok {
f.mutex.RUnlock()
return index, true
} else if !createIfMissing {
f.mutex.RUnlock()
return 0, false
}
return 0, false
}
func (f *FieldIndexCache) FieldIndex(field string) (uint16, *FieldRow) {
// trade read lock for write lock
f.mutex.RUnlock()
f.mutex.Lock()
defer f.mutex.Unlock()
index, exists := f.fieldIndexes[field]
if exists {
return index, nil
}
// assign next field id
index = uint16(f.lastFieldIndex + 1)
index := uint16(f.lastFieldIndex + 1)
f.fieldIndexes[field] = index
f.lastFieldIndex = int(index)
return index, NewFieldRow(uint16(index), field)
f.mutex.Unlock()
return index, false
}
func (f *FieldIndexCache) FieldName(index uint16) string {
func (f *FieldCache) FieldIndexed(index uint16) string {
f.mutex.RLock()
defer f.mutex.RUnlock()
for fieldName, fieldIndex := range f.fieldIndexes {

137
index/firestorm/analysis.go Normal file
View File

@ -0,0 +1,137 @@
// 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 firestorm
import (
"math"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func (f *Firestorm) Analyze(d *document.Document) *index.AnalysisResult {
rv := &index.AnalysisResult{
DocID: d.ID,
Rows: make([]index.IndexRow, 0, 100),
}
for _, field := range d.Fields {
fieldIndex, newFieldRow := f.fieldIndexOrNewRow(field.Name())
if newFieldRow != nil {
rv.Rows = append(rv.Rows, newFieldRow)
}
// add the _id row
rv.Rows = append(rv.Rows, NewTermFreqRow(0, nil, []byte(d.ID), d.Number, 0, 0, nil))
if field.Options().IsIndexed() {
fieldLength, tokenFreqs := field.Analyze()
// see if any of the composite fields need this
for _, compositeField := range d.CompositeFields {
compositeField.Compose(field.Name(), fieldLength, tokenFreqs)
}
// encode this field
indexRows := f.indexField(d.ID, d.Number, field, fieldIndex, fieldLength, tokenFreqs)
rv.Rows = append(rv.Rows, indexRows...)
}
if field.Options().IsStored() {
storeRow := f.storeField(d.ID, d.Number, field, fieldIndex)
rv.Rows = append(rv.Rows, storeRow)
}
}
// now index the composite fields
for _, compositeField := range d.CompositeFields {
fieldIndex, newFieldRow := f.fieldIndexOrNewRow(compositeField.Name())
if newFieldRow != nil {
rv.Rows = append(rv.Rows, newFieldRow)
}
if compositeField.Options().IsIndexed() {
fieldLength, tokenFreqs := compositeField.Analyze()
// encode this field
indexRows := f.indexField(d.ID, d.Number, compositeField, fieldIndex, fieldLength, tokenFreqs)
rv.Rows = append(rv.Rows, indexRows...)
}
}
return rv
}
func (f *Firestorm) indexField(docID string, docNum uint64, field document.Field, fieldIndex uint16, fieldLength int, tokenFreqs analysis.TokenFrequencies) []index.IndexRow {
rows := make([]index.IndexRow, 0, 100)
fieldNorm := float32(1.0 / math.Sqrt(float64(fieldLength)))
for _, tf := range tokenFreqs {
var termFreqRow *TermFreqRow
if field.Options().IncludeTermVectors() {
tv, newFieldRows := f.termVectorsFromTokenFreq(fieldIndex, tf)
rows = append(rows, newFieldRows...)
termFreqRow = NewTermFreqRow(fieldIndex, tf.Term, []byte(docID), docNum, uint64(tf.Frequency()), fieldNorm, tv)
} else {
termFreqRow = NewTermFreqRow(fieldIndex, tf.Term, []byte(docID), docNum, uint64(tf.Frequency()), fieldNorm, nil)
}
rows = append(rows, termFreqRow)
}
return rows
}
func (f *Firestorm) termVectorsFromTokenFreq(field uint16, tf *analysis.TokenFreq) ([]*TermVector, []index.IndexRow) {
rv := make([]*TermVector, len(tf.Locations))
newFieldRows := make([]index.IndexRow, 0)
for i, l := range tf.Locations {
var newFieldRow *FieldRow
fieldIndex := field
if l.Field != "" {
// lookup correct field
fieldIndex, newFieldRow = f.fieldIndexOrNewRow(l.Field)
if newFieldRow != nil {
newFieldRows = append(newFieldRows, newFieldRow)
}
}
tv := NewTermVector(fieldIndex, uint64(l.Position), uint64(l.Start), uint64(l.End), l.ArrayPositions)
rv[i] = tv
}
return rv, newFieldRows
}
func (f *Firestorm) storeField(docID string, docNum uint64, field document.Field, fieldIndex uint16) index.IndexRow {
fieldValue := make([]byte, 1+len(field.Value()))
fieldValue[0] = encodeFieldType(field)
copy(fieldValue[1:], field.Value())
storedRow := NewStoredRow([]byte(docID), docNum, fieldIndex, field.ArrayPositions(), fieldValue)
return storedRow
}
func encodeFieldType(f document.Field) byte {
fieldType := byte('x')
switch f.(type) {
case *document.TextField:
fieldType = 't'
case *document.NumericField:
fieldType = 'n'
case *document.DateTimeField:
fieldType = 'd'
case *document.CompositeField:
fieldType = 'c'
}
return fieldType
}

View File

@ -0,0 +1,81 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestAnalysis(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
for _, row := range rows {
err := kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
// warmup to load field cache and set maxRead correctly
f.warmup(kvwriter)
tests := []struct {
d *document.Document
r *index.AnalysisResult
}{
{
d: document.NewDocument("a").
AddField(
document.NewTextFieldWithIndexingOptions("name", nil, []byte("test"), document.IndexField|document.StoreField|document.IncludeTermVectors)),
r: &index.AnalysisResult{
DocID: "a",
Rows: []index.IndexRow{
NewFieldRow(1, "name"),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(1, []byte("test"), []byte("a"), 1, 1, 1.0, []*TermVector{NewTermVector(1, 1, 0, 4, nil)}),
NewStoredRow([]byte("a"), 1, 1, nil, []byte("ttest")),
},
},
},
}
for _, test := range tests {
test.d.Number = 1
actual := f.Analyze(test.d)
if !reflect.DeepEqual(actual, test.r) {
t.Errorf("expected: %v got %v", test.r, actual)
}
}
err = kvwriter.Close()
if err != nil {
t.Fatal(err)
}
}

View File

@ -0,0 +1,77 @@
// 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 firestorm
import (
"os"
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/boltdb"
)
func CreateBoltDB() (store.KVStore, error) {
s := boltdb.New("test", "bleve")
return s, nil
}
func DestroyBoltDB() error {
return os.RemoveAll("test")
}
func BenchmarkBoltDBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateBoltDB, DestroyBoltDB, 1)
}
func BenchmarkBoltDBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateBoltDB, DestroyBoltDB, 2)
}
func BenchmarkBoltDBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateBoltDB, DestroyBoltDB, 4)
}
// batches
func BenchmarkBoltDBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 1, 10)
}
func BenchmarkBoltDBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 2, 10)
}
func BenchmarkBoltDBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 4, 10)
}
func BenchmarkBoltDBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 1, 100)
}
func BenchmarkBoltDBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 2, 100)
}
func BenchmarkBoltDBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 4, 100)
}
func BenchmarkBoltBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 1, 1000)
}
func BenchmarkBoltBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 2, 1000)
}
func BenchmarkBoltBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateBoltDB, DestroyBoltDB, 4, 1000)
}

View File

@ -0,0 +1,143 @@
// 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 firestorm
import (
"strconv"
"testing"
_ "github.com/blevesearch/bleve/analysis/analyzers/standard_analyzer"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/registry"
)
var benchmarkDocBodies = []string{
"A boiling liquid expanding vapor explosion (BLEVE, /ˈblɛviː/ blev-ee) is an explosion caused by the rupture of a vessel containing a pressurized liquid above its boiling point.",
"A boiler explosion is a catastrophic failure of a boiler. As seen today, boiler explosions are of two kinds. One kind is a failure of the pressure parts of the steam and water sides. There can be many different causes, such as failure of the safety valve, corrosion of critical parts of the boiler, or low water level. Corrosion along the edges of lap joints was a common cause of early boiler explosions.",
"A boiler is a closed vessel in which water or other fluid is heated. The fluid does not necessarily boil. (In North America the term \"furnace\" is normally used if the purpose is not actually to boil the fluid.) The heated or vaporized fluid exits the boiler for use in various processes or heating applications,[1][2] including central heating, boiler-based power generation, cooking, and sanitation.",
"A pressure vessel is a closed container designed to hold gases or liquids at a pressure substantially different from the ambient pressure.",
"Pressure (symbol: p or P) is the ratio of force to the area over which that force is distributed.",
"Liquid is one of the four fundamental states of matter (the others being solid, gas, and plasma), and is the only state with a definite volume but no fixed shape.",
"The boiling point of a substance is the temperature at which the vapor pressure of the liquid equals the pressure surrounding the liquid[1][2] and the liquid changes into a vapor.",
"Vapor pressure or equilibrium vapor pressure is defined as the pressure exerted by a vapor in thermodynamic equilibrium with its condensed phases (solid or liquid) at a given temperature in a closed system.",
"Industrial gases are a group of gases that are specifically manufactured for use in a wide range of industries, which include oil and gas, petrochemicals, chemicals, power, mining, steelmaking, metals, environmental protection, medicine, pharmaceuticals, biotechnology, food, water, fertilizers, nuclear power, electronics and aerospace.",
"The expansion ratio of a liquefied and cryogenic substance is the volume of a given amount of that substance in liquid form compared to the volume of the same amount of substance in gaseous form, at room temperature and normal atmospheric pressure.",
}
type KVStoreCreate func() (store.KVStore, error)
type KVStoreDestroy func() error
func CommonBenchmarkIndex(b *testing.B, create KVStoreCreate, destroy KVStoreDestroy, analysisWorkers int) {
cache := registry.NewCache()
analyzer, err := cache.AnalyzerNamed("standard")
if err != nil {
b.Fatal(err)
}
indexDocument := document.NewDocument("").
AddField(document.NewTextFieldWithAnalyzer("body", []uint64{}, []byte(benchmarkDocBodies[0]), analyzer))
b.ResetTimer()
b.StopTimer()
for i := 0; i < b.N; i++ {
s, err := create()
if err != nil {
b.Fatal(err)
}
analysisQueue := index.NewAnalysisQueue(analysisWorkers)
idx := NewFirestorm(s, analysisQueue)
err = idx.Open()
if err != nil {
b.Fatal(err)
}
indexDocument.ID = strconv.Itoa(i)
// just time the indexing portion
b.StartTimer()
err = idx.Update(indexDocument)
if err != nil {
b.Fatal(err)
}
b.StopTimer()
err = idx.Close()
if err != nil {
b.Fatal(err)
}
err = destroy()
if err != nil {
b.Fatal(err)
}
analysisQueue.Close()
}
}
func CommonBenchmarkIndexBatch(b *testing.B, create KVStoreCreate, destroy KVStoreDestroy, analysisWorkers, batchSize int) {
cache := registry.NewCache()
analyzer, err := cache.AnalyzerNamed("standard")
if err != nil {
b.Fatal(err)
}
b.ResetTimer()
b.StopTimer()
for i := 0; i < b.N; i++ {
s, err := create()
if err != nil {
b.Fatal(err)
}
analysisQueue := index.NewAnalysisQueue(analysisWorkers)
idx := NewFirestorm(s, analysisQueue)
err = idx.Open()
if err != nil {
b.Fatal(err)
}
b.StartTimer()
batch := index.NewBatch()
for j := 0; j < 1000; j++ {
if j%batchSize == 0 {
if len(batch.IndexOps) > 0 {
err := idx.Batch(batch)
if err != nil {
b.Fatal(err)
}
}
batch = index.NewBatch()
}
indexDocument := document.NewDocument("").
AddField(document.NewTextFieldWithAnalyzer("body", []uint64{}, []byte(benchmarkDocBodies[j%10]), analyzer))
indexDocument.ID = strconv.Itoa(i) + "-" + strconv.Itoa(j)
batch.Update(indexDocument)
}
// close last batch
if len(batch.IndexOps) > 0 {
err := idx.Batch(batch)
if err != nil {
b.Fatal(err)
}
}
b.StopTimer()
err = idx.Close()
if err != nil {
b.Fatal(err)
}
err = destroy()
if err != nil {
b.Fatal(err)
}
analysisQueue.Close()
}
}

View File

@ -0,0 +1,75 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/cznicb"
)
func CreateCznicB() (store.KVStore, error) {
return cznicb.StoreConstructor(nil)
}
func DestroyCznicB() error {
return nil
}
func BenchmarkCznicBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateCznicB, DestroyCznicB, 1)
}
func BenchmarkCznicBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateCznicB, DestroyCznicB, 2)
}
func BenchmarkCznicBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateCznicB, DestroyCznicB, 4)
}
// batches
func BenchmarkCznicBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 1, 10)
}
func BenchmarkCznicBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 2, 10)
}
func BenchmarkCznicBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 4, 10)
}
func BenchmarkCznicBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 1, 100)
}
func BenchmarkCznicBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 2, 100)
}
func BenchmarkCznicBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 4, 100)
}
func BenchmarkCznicBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 1, 1000)
}
func BenchmarkCznicBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 2, 1000)
}
func BenchmarkCznicBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateCznicB, DestroyCznicB, 4, 1000)
}

View File

@ -0,0 +1,86 @@
// 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.
// +build forestdb
package firestorm
import (
"os"
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/forestdb"
)
func CreateForestDB() (store.KVStore, error) {
err := os.MkdirAll("testdir", 0700)
if err != nil {
return nil, err
}
s, err := forestdb.New("testdir/test", true, nil)
if err != nil {
return nil, err
}
return s, nil
}
func DestroyForestDB() error {
return os.RemoveAll("testdir")
}
func BenchmarkForestDBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateForestDB, DestroyForestDB, 1)
}
func BenchmarkForestDBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateForestDB, DestroyForestDB, 2)
}
func BenchmarkForestDBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateForestDB, DestroyForestDB, 4)
}
// batches
func BenchmarkForestDBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 1, 10)
}
func BenchmarkForestDBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 2, 10)
}
func BenchmarkForestDBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 4, 10)
}
func BenchmarkForestDBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 1, 100)
}
func BenchmarkForestDBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 2, 100)
}
func BenchmarkForestDBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 4, 100)
}
func BenchmarkForestDBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 1, 1000)
}
func BenchmarkForestDBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 2, 1000)
}
func BenchmarkForestDBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateForestDB, DestroyForestDB, 4, 1000)
}

View File

@ -0,0 +1,80 @@
// 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 firestorm
import (
"os"
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/goleveldb"
)
var goLevelDBTestOptions = map[string]interface{}{
"create_if_missing": true,
}
func CreateGoLevelDB() (store.KVStore, error) {
return goleveldb.New("test", goLevelDBTestOptions)
}
func DestroyGoLevelDB() error {
return os.RemoveAll("test")
}
func BenchmarkGoLevelDBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoLevelDB, DestroyGoLevelDB, 1)
}
func BenchmarkGoLevelDBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoLevelDB, DestroyGoLevelDB, 2)
}
func BenchmarkGoLevelDBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoLevelDB, DestroyGoLevelDB, 4)
}
// batches
func BenchmarkGoLevelDBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 1, 10)
}
func BenchmarkGoLevelDBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 2, 10)
}
func BenchmarkGoLevelDBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 4, 10)
}
func BenchmarkGoLevelDBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 1, 100)
}
func BenchmarkGoLevelDBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 2, 100)
}
func BenchmarkGoLevelDBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 4, 100)
}
func BenchmarkGoLevelDBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 1, 1000)
}
func BenchmarkGoLevelDBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 2, 1000)
}
func BenchmarkGoLevelDBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoLevelDB, DestroyGoLevelDB, 4, 1000)
}

View File

@ -0,0 +1,81 @@
// 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.
// +build rocksdb
package firestorm
import (
"os"
"testing"
"github.com/blevesearch/bleve/index/store"
)
var rocksdbTestOptions = map[string]interface{}{
"create_if_missing": true,
}
func CreateGoRocksDB() (store.KVStore, error) {
return rocksdb.New("test", rocksdbTestOptions)
}
func DestroyGoRocksDB() error {
return os.RemoveAll("test")
}
func BenchmarkRocksDBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoRocksDB, DestroyGoRocksDB, 1)
}
func BenchmarkRocksDBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoRocksDB, DestroyGoRocksDB, 2)
}
func BenchmarkRocksDBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGoRocksDB, DestroyGoRocksDB, 4)
}
// batches
func BenchmarkRocksDBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 1, 10)
}
func BenchmarkRocksDBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 2, 10)
}
func BenchmarkRocksDBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 4, 10)
}
func BenchmarkRocksDBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 1, 100)
}
func BenchmarkRocksDBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 2, 100)
}
func BenchmarkRocksDBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 4, 100)
}
func BenchmarkRocksDBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 1, 1000)
}
func BenchmarkRocksDBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 2, 1000)
}
func BenchmarkRocksDBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGoRocksDB, DestroyGoRocksDB, 4, 1000)
}

View File

@ -0,0 +1,75 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/gtreap"
)
func CreateGTreap() (store.KVStore, error) {
return gtreap.StoreConstructor(nil)
}
func DestroyGTreap() error {
return nil
}
func BenchmarkGTreapIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGTreap, DestroyGTreap, 1)
}
func BenchmarkGTreapIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGTreap, DestroyGTreap, 2)
}
func BenchmarkGTreapIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateGTreap, DestroyGTreap, 4)
}
// batches
func BenchmarkGTreapIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 1, 10)
}
func BenchmarkGTreapIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 2, 10)
}
func BenchmarkGTreapIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 4, 10)
}
func BenchmarkGTreapIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 1, 100)
}
func BenchmarkGTreapIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 2, 100)
}
func BenchmarkGTreapIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 4, 100)
}
func BenchmarkGTreapIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 1, 1000)
}
func BenchmarkGTreapIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 2, 1000)
}
func BenchmarkGTreapIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateGTreap, DestroyGTreap, 4, 1000)
}

View File

@ -0,0 +1,75 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/inmem"
)
func CreateInMem() (store.KVStore, error) {
return inmem.New()
}
func DestroyInMem() error {
return nil
}
func BenchmarkInMemIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateInMem, DestroyInMem, 1)
}
func BenchmarkInMemIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateInMem, DestroyInMem, 2)
}
func BenchmarkInMemIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateInMem, DestroyInMem, 4)
}
// batches
func BenchmarkInMemIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 1, 10)
}
func BenchmarkInMemIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 2, 10)
}
func BenchmarkInMemIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 4, 10)
}
func BenchmarkInMemIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 1, 100)
}
func BenchmarkInMemIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 2, 100)
}
func BenchmarkInMemIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 4, 100)
}
func BenchmarkInMemIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 1, 1000)
}
func BenchmarkInMemIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 2, 1000)
}
func BenchmarkInMemIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateInMem, DestroyInMem, 4, 1000)
}

View File

@ -0,0 +1,82 @@
// 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.
// +build leveldb full
package firestorm
import (
"os"
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/leveldb"
)
var leveldbTestOptions = map[string]interface{}{
"create_if_missing": true,
}
func CreateLevelDB() (store.KVStore, error) {
return leveldb.New("test", leveldbTestOptions)
}
func DestroyLevelDB() error {
return os.RemoveAll("test")
}
func BenchmarkLevelDBIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateLevelDB, DestroyLevelDB, 1)
}
func BenchmarkLevelDBIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateLevelDB, DestroyLevelDB, 2)
}
func BenchmarkLevelDBIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateLevelDB, DestroyLevelDB, 4)
}
// batches
func BenchmarkLevelDBIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 1, 10)
}
func BenchmarkLevelDBIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 2, 10)
}
func BenchmarkLevelDBIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 4, 10)
}
func BenchmarkLevelDBIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 1, 100)
}
func BenchmarkLevelDBIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 2, 100)
}
func BenchmarkLevelDBIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 4, 100)
}
func BenchmarkLevelDBIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 1, 1000)
}
func BenchmarkLevelDBIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 2, 1000)
}
func BenchmarkLevelDBIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateLevelDB, DestroyLevelDB, 4, 1000)
}

View File

@ -0,0 +1,75 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/store/null"
)
func CreateNull() (store.KVStore, error) {
return null.New()
}
func DestroyNull() error {
return nil
}
func BenchmarkNullIndexing1Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateNull, DestroyNull, 1)
}
func BenchmarkNullIndexing2Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateNull, DestroyNull, 2)
}
func BenchmarkNullIndexing4Workers(b *testing.B) {
CommonBenchmarkIndex(b, CreateNull, DestroyNull, 4)
}
// batches
func BenchmarkNullIndexing1Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 1, 10)
}
func BenchmarkNullIndexing2Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 2, 10)
}
func BenchmarkNullIndexing4Workers10Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 4, 10)
}
func BenchmarkNullIndexing1Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 1, 100)
}
func BenchmarkNullIndexing2Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 2, 100)
}
func BenchmarkNullIndexing4Workers100Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 4, 100)
}
func BenchmarkNullIndexing1Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 1, 1000)
}
func BenchmarkNullIndexing2Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 2, 1000)
}
func BenchmarkNullIndexing4Workers1000Batch(b *testing.B) {
CommonBenchmarkIndexBatch(b, CreateNull, DestroyNull, 4, 1000)
}

153
index/firestorm/comp.go Normal file
View File

@ -0,0 +1,153 @@
// 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 firestorm
import (
"bytes"
"math/rand"
"sort"
"sync"
"github.com/blevesearch/bleve/document"
"github.com/steveyen/gtreap"
"github.com/willf/bitset"
)
type Compensator struct {
inFlightMutex sync.RWMutex
maxRead uint64
inFlight *gtreap.Treap
deletedMutex sync.RWMutex
deletedDocNumbers *bitset.BitSet
}
func NewCompensator() *Compensator {
rv := Compensator{
inFlight: gtreap.NewTreap(inFlightItemCompare),
deletedDocNumbers: bitset.New(1000000),
}
return &rv
}
type Snapshot struct {
maxRead uint64
inFlight *gtreap.Treap
deletedDocNumbers *bitset.BitSet
}
// returns which doc number is valid
// if none, then 0
func (s *Snapshot) Which(docID []byte, docNumList DocNumberList) uint64 {
sort.Sort(docNumList)
highestValidDocNum := docNumList.HighestValid(s.maxRead)
if highestValidDocNum > 0 && s.Valid(docID, highestValidDocNum) {
return highestValidDocNum
}
return 0
}
func (s *Snapshot) Valid(docID []byte, docNum uint64) bool {
logger.Printf("checking validity of: '%s' - % x - %d", docID, docID, docNum)
if docNum > s.maxRead {
return false
}
logger.Printf("<= maxRead")
inFlightVal := s.inFlight.Get(&InFlightItem{docID: docID})
if inFlightVal != nil && inFlightVal.(*InFlightItem).docNum != docNum {
return false
}
logger.Printf("not in flight")
if s.deletedDocNumbers.Test(uint(docNum)) {
return false
}
logger.Printf("not deleted")
return true
}
func (c *Compensator) Mutate(docID []byte, docNum uint64) {
c.inFlightMutex.Lock()
defer c.inFlightMutex.Unlock()
c.inFlight = c.inFlight.Upsert(&InFlightItem{docID: docID, docNum: docNum}, rand.Int())
if docNum != 0 {
c.maxRead = docNum
}
}
func (c *Compensator) MutateBatch(docs map[string]*document.Document, docNum uint64) {
c.inFlightMutex.Lock()
defer c.inFlightMutex.Unlock()
for docID, doc := range docs {
if doc != nil {
c.inFlight = c.inFlight.Upsert(&InFlightItem{docID: []byte(docID), docNum: doc.Number}, rand.Int())
} else {
c.inFlight = c.inFlight.Upsert(&InFlightItem{docID: []byte(docID), docNum: 0}, rand.Int())
}
}
c.maxRead = docNum
}
func (c *Compensator) Migrate(docID []byte, docNum uint64, oldDocNums []uint64) {
c.inFlightMutex.Lock()
defer c.inFlightMutex.Unlock()
c.deletedMutex.Lock()
defer c.deletedMutex.Unlock()
// clone deleted doc numbers and mutate
newDeletedDocNumbers := c.deletedDocNumbers.Clone()
for _, oldDocNum := range oldDocNums {
newDeletedDocNumbers.Set(uint(oldDocNum))
}
// update pointer
c.deletedDocNumbers = newDeletedDocNumbers
// remove entry from in-flight if it still has same doc num
val := c.inFlight.Get(&InFlightItem{docID: docID})
if val.(*InFlightItem).docNum == docNum {
c.inFlight = c.inFlight.Delete(&InFlightItem{docID: docID})
}
}
func (c *Compensator) GarbageCollect(docNums []uint64) {
c.deletedMutex.Lock()
defer c.deletedMutex.Unlock()
for _, docNum := range docNums {
c.deletedDocNumbers.Clear(uint(docNum))
}
}
func (c *Compensator) Snapshot() *Snapshot {
c.inFlightMutex.RLock()
defer c.inFlightMutex.RUnlock()
c.deletedMutex.RLock()
defer c.deletedMutex.RUnlock()
rv := Snapshot{
maxRead: c.maxRead,
inFlight: c.inFlight,
deletedDocNumbers: c.deletedDocNumbers,
}
return &rv
}
func (c *Compensator) GarbageCount() uint64 {
return uint64(c.deletedDocNumbers.Count())
}
//**************
type InFlightItem struct {
docID []byte
docNum uint64
}
func inFlightItemCompare(a, b interface{}) int {
return bytes.Compare(a.(*InFlightItem).docID, b.(*InFlightItem).docID)
}

View File

@ -0,0 +1,142 @@
// 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 firestorm
import (
"encoding/binary"
"fmt"
"sync"
"sync/atomic"
"time"
)
const DefaultDictUpdateThreshold = 10
var DefaultDictUpdateSleep = 1 * time.Second
type DictUpdater struct {
f *Firestorm
dictUpdateSleep time.Duration
quit chan struct{}
mutex sync.RWMutex
workingSet map[string]int64
closeWait sync.WaitGroup
batchesStarted uint64
batchesFlushed uint64
}
func NewDictUpdater(f *Firestorm) *DictUpdater {
rv := DictUpdater{
f: f,
dictUpdateSleep: DefaultDictUpdateSleep,
workingSet: make(map[string]int64),
batchesStarted: 1,
quit: make(chan struct{}),
}
return &rv
}
func (d *DictUpdater) Notify(term string, usage int64) {
d.mutex.Lock()
defer d.mutex.Unlock()
d.workingSet[term] += usage
}
func (d *DictUpdater) NotifyBatch(termUsages map[string]int64) {
d.mutex.Lock()
defer d.mutex.Unlock()
for term, usage := range termUsages {
d.workingSet[term] += usage
}
}
func (d *DictUpdater) Start() {
d.closeWait.Add(1)
go d.run()
}
func (d *DictUpdater) Stop() {
close(d.quit)
d.closeWait.Wait()
}
func (d *DictUpdater) run() {
tick := time.Tick(d.dictUpdateSleep)
for {
select {
case <-d.quit:
logger.Printf("dictionary updater asked to quit")
d.closeWait.Done()
return
case <-tick:
logger.Printf("dictionary updater ticked")
d.update()
}
}
}
func (d *DictUpdater) update() {
d.mutex.Lock()
oldWorkingSet := d.workingSet
d.workingSet = make(map[string]int64)
atomic.AddUint64(&d.batchesStarted, 1)
d.mutex.Unlock()
// open a writer
writer, err := d.f.store.Writer()
if err != nil {
writer.Close()
logger.Printf("dict updater fatal: %v")
return
}
// prepare batch
wb := writer.NewBatch()
dictionaryTermDelta := make([]byte, 8)
for term, delta := range oldWorkingSet {
binary.LittleEndian.PutUint64(dictionaryTermDelta, uint64(delta))
wb.Merge([]byte(term), dictionaryTermDelta)
}
err = wb.Execute()
if err != nil {
writer.Close()
logger.Printf("dict updater fatal: %v")
return
}
atomic.AddUint64(&d.batchesFlushed, 1)
err = writer.Close()
}
// this is not intended to be used publicly, only for unit tests
// which depend on consistency we no longer provide
func (d *DictUpdater) waitTasksDone(dur time.Duration) error {
timeout := time.After(dur)
tick := time.Tick(100 * time.Millisecond)
for {
select {
// Got a timeout! fail with a timeout error
case <-timeout:
return fmt.Errorf("timeout")
// Got a tick, we should check on doSomething()
case <-tick:
started := atomic.LoadUint64(&d.batchesStarted)
flushed := atomic.LoadUint64(&d.batchesFlushed)
if started == flushed {
return nil
}
}
}
}

View File

@ -0,0 +1,150 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestDictUpdater(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
kv.SetMergeOperator(&mergeOperator)
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
dictBatch := map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): 3,
}
dictExpect := map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): 3,
}
f.dictUpdater.NotifyBatch(dictBatch)
// invoke updater manually
f.dictUpdater.update()
// assert that dictionary rows are correct
reader, err := f.store.Reader()
if err != nil {
t.Fatal(err)
}
for key, _ := range dictBatch {
v, err := reader.Get([]byte(key))
if err != nil {
t.Fatal(err)
}
dr, err := NewDictionaryRowKV([]byte(key), v)
if err != nil {
t.Fatal(err)
}
expect := dictExpect[key]
if int64(dr.Count()) != expect {
t.Errorf("expected %d, got %d", expect, dr.Count())
}
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// update it again
dictBatch = map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): 1,
}
dictExpect = map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): 4,
}
f.dictUpdater.NotifyBatch(dictBatch)
// invoke updater manually
f.dictUpdater.update()
// assert that dictionary rows are correct
reader, err = f.store.Reader()
if err != nil {
t.Fatal(err)
}
for key, _ := range dictBatch {
v, err := reader.Get([]byte(key))
if err != nil {
t.Fatal(err)
}
dr, err := NewDictionaryRowKV([]byte(key), v)
if err != nil {
t.Fatal(err)
}
expect := dictExpect[key]
if int64(dr.Count()) != expect {
t.Errorf("expected %d, got %d", expect, dr.Count())
}
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// update it again (decrement this time)
dictBatch = map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): -2,
}
dictExpect = map[string]int64{
string([]byte{'d', 1, 0, 'c', 'a', 't'}): 2,
}
f.dictUpdater.NotifyBatch(dictBatch)
// invoke updater manually
f.dictUpdater.update()
// assert that dictionary rows are correct
reader, err = f.store.Reader()
if err != nil {
t.Fatal(err)
}
for key, _ := range dictBatch {
v, err := reader.Get([]byte(key))
if err != nil {
t.Fatal(err)
}
dr, err := NewDictionaryRowKV([]byte(key), v)
if err != nil {
t.Fatal(err)
}
expect := dictExpect[key]
if int64(dr.Count()) != expect {
t.Errorf("expected %d, got %d", expect, dr.Count())
}
}
err = reader.Close()
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 firestorm
import (
"bytes"
"encoding/binary"
"io"
"github.com/golang/protobuf/proto"
)
const ByteSeparator byte = 0xff
var DictionaryKeyPrefix = []byte{'d'}
type DictionaryRow struct {
field uint16
term []byte
value DictionaryValue
}
func NewDictionaryRow(field uint16, term []byte, count uint64) *DictionaryRow {
rv := DictionaryRow{
field: field,
term: term,
}
rv.value.Count = proto.Uint64(count)
return &rv
}
func NewDictionaryRowK(key []byte) (*DictionaryRow, error) {
rv := DictionaryRow{}
buf := bytes.NewBuffer(key)
_, err := buf.ReadByte() // type
if err != nil {
return nil, err
}
err = binary.Read(buf, binary.LittleEndian, &rv.field)
if err != nil {
return nil, err
}
rv.term, err = buf.ReadBytes(ByteSeparator)
// there is no separator expected here, should get EOF
if err != io.EOF {
return nil, err
}
return &rv, nil
}
func (dr *DictionaryRow) parseDictionaryV(value []byte) error {
err := dr.value.Unmarshal(value)
if err != nil {
return err
}
return nil
}
func NewDictionaryRowKV(key, value []byte) (*DictionaryRow, error) {
rv, err := NewDictionaryRowK(key)
if err != nil {
return nil, err
}
err = rv.parseDictionaryV(value)
if err != nil {
return nil, err
}
return rv, nil
}
func (dr *DictionaryRow) Count() uint64 {
return dr.value.GetCount()
}
func (dr *DictionaryRow) SetCount(count uint64) {
dr.value.Count = proto.Uint64(count)
}
func (dr *DictionaryRow) Key() []byte {
buf := make([]byte, 3+len(dr.term))
copy(buf[0:], DictionaryKeyPrefix)
binary.LittleEndian.PutUint16(buf[1:3], dr.field)
copy(buf[3:], dr.term)
return buf
}
func (dr *DictionaryRow) Value() []byte {
rv, _ := dr.value.Marshal()
return rv
}
func DictionaryRowKey(field uint16, term []byte) []byte {
buf := make([]byte, 3+len(term))
copy(buf[0:], DictionaryKeyPrefix)
binary.LittleEndian.PutUint16(buf[1:3], field)
copy(buf[3:], term)
return buf
}

View File

@ -0,0 +1,59 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestDictionaryRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewDictionaryRow(0, []byte("test"), 3),
[]byte{DictionaryKeyPrefix[0], 0, 0, 't', 'e', 's', 't'},
[]byte{8, 3},
},
{
NewDictionaryRow(3, []byte("dictionary"), 734),
[]byte{DictionaryKeyPrefix[0], 3, 0, 'd', 'i', 'c', 't', 'i', 'o', 'n', 'a', 'r', 'y'},
[]byte{8, 222, 5},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewDictionaryRowKV(test.outKey, test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected: %#v got: %#v for %d", test.input, row, i)
}
}
}

86
index/firestorm/dump.go Normal file
View File

@ -0,0 +1,86 @@
// 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 firestorm
import (
"bytes"
"fmt"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
)
// the functions in this file are only intended to be used by
// the bleve_dump utility and the debug http handlers
// if your application relies on them, you're doing something wrong
// they may change or be removed at any time
func (f *Firestorm) dumpPrefix(kvreader store.KVReader, rv chan interface{}, prefix []byte) {
visitPrefix(kvreader, prefix, func(key, val []byte) (bool, error) {
row, err := parseFromKeyValue(key, val)
if err != nil {
rv <- err
return false, err
}
rv <- row
return true, nil
})
}
func (f *Firestorm) dumpDoc(kvreader store.KVReader, rv chan interface{}, docID []byte) {
// without a back index we have no choice but to walk the term freq and stored rows
// walk the term freqs
visitPrefix(kvreader, TermFreqKeyPrefix, func(key, val []byte) (bool, error) {
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
rv <- err
return false, err
}
if bytes.Compare(tfr.DocID(), docID) == 0 {
rv <- tfr
}
return true, nil
})
// now walk the stored
visitPrefix(kvreader, StoredKeyPrefix, func(key, val []byte) (bool, error) {
sr, err := NewStoredRowKV(key, val)
if err != nil {
rv <- err
return false, err
}
if bytes.Compare(sr.DocID(), docID) == 0 {
rv <- sr
}
return true, nil
})
}
func parseFromKeyValue(key, value []byte) (index.IndexRow, error) {
if len(key) > 0 {
switch key[0] {
case VersionKey[0]:
return NewVersionRowV(value)
case FieldKeyPrefix[0]:
return NewFieldRowKV(key, value)
case DictionaryKeyPrefix[0]:
return NewDictionaryRowKV(key, value)
case TermFreqKeyPrefix[0]:
return NewTermFreqRowKV(key, value)
case StoredKeyPrefix[0]:
return NewStoredRowKV(key, value)
case InternalKeyPrefix[0]:
return NewInternalRowKV(key, value)
}
return nil, fmt.Errorf("Unknown field type '%s'", string(key[0]))
}
return nil, fmt.Errorf("Invalid empty key")
}

View File

@ -0,0 +1,133 @@
// 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 firestorm
import (
"os"
"testing"
"time"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/boltdb"
)
var dictWaitDuration = 5 * time.Second
func TestDump(t *testing.T) {
defer func() {
err := os.RemoveAll("test")
if err != nil {
t.Fatal(err)
}
}()
s := boltdb.New("test", "bleve")
s.SetMergeOperator(&mergeOperator)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewFirestorm(s, analysisQueue)
err := idx.Open()
if err != nil {
t.Errorf("error opening index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
var expectedCount uint64
docCount, err := idx.DocCount()
if err != nil {
t.Error(err)
}
if docCount != expectedCount {
t.Errorf("Expected document count to be %d got %d", expectedCount, docCount)
}
doc := document.NewDocument("1")
doc.AddField(document.NewTextFieldWithIndexingOptions("name", []uint64{}, []byte("test"), document.IndexField|document.StoreField))
doc.AddField(document.NewNumericFieldWithIndexingOptions("age", []uint64{}, 35.99, document.IndexField|document.StoreField))
dateField, err := document.NewDateTimeFieldWithIndexingOptions("unixEpoch", []uint64{}, time.Unix(0, 0), document.IndexField|document.StoreField)
if err != nil {
t.Error(err)
}
doc.AddField(dateField)
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
doc = document.NewDocument("2")
doc.AddField(document.NewTextFieldWithIndexingOptions("name", []uint64{}, []byte("test2"), document.IndexField|document.StoreField))
doc.AddField(document.NewNumericFieldWithIndexingOptions("age", []uint64{}, 35.99, document.IndexField|document.StoreField))
dateField, err = document.NewDateTimeFieldWithIndexingOptions("unixEpoch", []uint64{}, time.Unix(0, 0), document.IndexField|document.StoreField)
if err != nil {
t.Error(err)
}
doc.AddField(dateField)
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
fieldsCount := 0
fieldsRows := idx.DumpFields()
for _ = range fieldsRows {
fieldsCount++
}
if fieldsCount != 4 { // _id field is automatic
t.Errorf("expected 4 fields, got %d", fieldsCount)
}
// 1 id term
// 1 text term
// 16 numeric terms
// 16 date terms
// 3 stored fields
expectedDocRowCount := int(1 + 1 + (2 * (64 / document.DefaultPrecisionStep)) + 3)
docRowCount := 0
docRows := idx.DumpDoc("1")
for _ = range docRows {
docRowCount++
}
if docRowCount != expectedDocRowCount {
t.Errorf("expected %d rows for document, got %d", expectedDocRowCount, docRowCount)
}
docRowCount = 0
docRows = idx.DumpDoc("2")
for _ = range docRows {
docRowCount++
}
if docRowCount != expectedDocRowCount {
t.Errorf("expected %d rows for document, got %d", expectedDocRowCount, docRowCount)
}
idx.dictUpdater.waitTasksDone(dictWaitDuration)
// 1 version
// fieldsCount field rows
// 2 docs * expectedDocRowCount
// 2 text term row count (2 different text terms)
// 16 numeric term row counts (shared for both docs, same numeric value)
// 16 date term row counts (shared for both docs, same date value)
//
expectedAllRowCount := int(1 + fieldsCount + (2 * expectedDocRowCount) + 2 + int((2 * (64 / document.DefaultPrecisionStep))))
allRowCount := 0
allRows := idx.DumpAll()
for _ = range allRows {
allRowCount++
}
if allRowCount != expectedAllRowCount {
t.Errorf("expected %d rows for all, got %d", expectedAllRowCount, allRowCount)
}
}

101
index/firestorm/field.go Normal file
View File

@ -0,0 +1,101 @@
// 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 firestorm
import (
"bytes"
"encoding/binary"
"fmt"
"github.com/blevesearch/bleve/index/store"
"github.com/golang/protobuf/proto"
)
var FieldKeyPrefix = []byte{'f'}
func (f *Firestorm) fieldIndexOrNewRow(name string) (uint16, *FieldRow) {
index, existed := f.fieldCache.FieldNamed(name, true)
if !existed {
return index, NewFieldRow(uint16(index), name)
}
return index, nil
}
func (f *Firestorm) loadFields(reader store.KVReader) (err error) {
err = visitPrefix(reader, FieldKeyPrefix, func(key, val []byte) (bool, error) {
fieldRow, err := NewFieldRowKV(key, val)
if err != nil {
return false, err
}
f.fieldCache.AddExisting(fieldRow.Name(), fieldRow.Index())
return true, nil
})
return
}
type FieldRow struct {
index uint16
value FieldValue
}
func NewFieldRow(i uint16, name string) *FieldRow {
rv := FieldRow{
index: i,
}
rv.value.Name = proto.String(name)
return &rv
}
func NewFieldRowKV(key, value []byte) (*FieldRow, error) {
rv := FieldRow{}
buf := bytes.NewBuffer(key)
_, err := buf.ReadByte() // type
if err != nil {
return nil, err
}
err = binary.Read(buf, binary.LittleEndian, &rv.index)
if err != nil {
return nil, err
}
err = rv.value.Unmarshal(value)
if err != nil {
return nil, err
}
return &rv, nil
}
func (fr *FieldRow) Key() []byte {
buf := make([]byte, 3)
buf[0] = 'f'
binary.LittleEndian.PutUint16(buf[1:3], fr.index)
return buf
}
func (fr *FieldRow) Value() []byte {
rv, _ := fr.value.Marshal()
return rv
}
func (fr *FieldRow) Index() uint16 {
return fr.index
}
func (fr *FieldRow) Name() string {
return fr.value.GetName()
}
func (fr *FieldRow) String() string {
return fmt.Sprintf("FieldRow - Field: %d - Name: %s\n", fr.index, fr.Name())
}

View File

@ -0,0 +1,59 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestFieldRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewFieldRow(0, "_id"),
[]byte{FieldKeyPrefix[0], 0, 0},
[]byte{10, 3, '_', 'i', 'd'},
},
{
NewFieldRow(1, "name"),
[]byte{FieldKeyPrefix[0], 1, 0},
[]byte{10, 4, 'n', 'a', 'm', 'e'},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewFieldRowKV(test.outKey, test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected: %#v got: %#v for %d", test.input, row, i)
}
}
}

View File

@ -0,0 +1,442 @@
// 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 firestorm
import (
"encoding/json"
"fmt"
"sync"
"sync/atomic"
"time"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/registry"
)
const Name = "firestorm"
var UnsafeBatchUseDetected = fmt.Errorf("bleve.Batch is NOT thread-safe, modification after execution detected")
type Firestorm struct {
store store.KVStore
compensator *Compensator
analysisQueue *index.AnalysisQueue
fieldCache *index.FieldCache
highDocNumber uint64
docCount *uint64
garbageCollector *GarbageCollector
lookuper *Lookuper
dictUpdater *DictUpdater
stats *indexStat
}
func NewFirestorm(s store.KVStore, analysisQueue *index.AnalysisQueue) *Firestorm {
initialCount := uint64(0)
rv := Firestorm{
store: s,
compensator: NewCompensator(),
analysisQueue: analysisQueue,
fieldCache: index.NewFieldCache(),
docCount: &initialCount,
highDocNumber: 0,
stats: &indexStat{},
}
rv.garbageCollector = NewGarbageCollector(&rv)
rv.lookuper = NewLookuper(&rv)
rv.dictUpdater = NewDictUpdater(&rv)
return &rv
}
func (f *Firestorm) Open() (err error) {
// install the merge operator
f.store.SetMergeOperator(&mergeOperator)
// now open the kv store
err = f.store.Open()
if err != nil {
return
}
// start a writer for the open process
var kvwriter store.KVWriter
kvwriter, err = f.store.Writer()
if err != nil {
return
}
defer func() {
if cerr := kvwriter.Close(); err == nil && cerr != nil {
err = cerr
}
}()
// assert correct version, and find out if this is new index
var newIndex bool
newIndex, err = f.checkVersion(kvwriter)
if err != nil {
return
}
if newIndex {
// prepare a new index
f.bootstrap(kvwriter)
} else {
// process existing index before opening
f.warmup(kvwriter)
}
// start the garbage collector
f.garbageCollector.Start()
// start the lookuper
f.lookuper.Start()
// start the dict updater
f.dictUpdater.Start()
return
}
func (f *Firestorm) Close() error {
f.garbageCollector.Stop()
f.lookuper.Stop()
f.dictUpdater.Stop()
return f.store.Close()
}
func (f *Firestorm) DocCount() (uint64, error) {
count := atomic.LoadUint64(f.docCount)
return count, nil
}
func (f *Firestorm) Update(doc *document.Document) (err error) {
// assign this document a number
doc.Number = atomic.AddUint64(&f.highDocNumber, 1)
// do analysis before acquiring write lock
analysisStart := time.Now()
resultChan := make(chan *index.AnalysisResult)
aw := index.NewAnalysisWork(f, doc, resultChan)
// put the work on the queue
go func() {
f.analysisQueue.Queue(aw)
}()
// wait for the result
result := <-resultChan
close(resultChan)
atomic.AddUint64(&f.stats.analysisTime, uint64(time.Since(analysisStart)))
// start a writer for this update
indexStart := time.Now()
var kvwriter store.KVWriter
kvwriter, err = f.store.Writer()
if err != nil {
return
}
defer func() {
if cerr := kvwriter.Close(); err == nil && cerr != nil {
err = cerr
}
}()
var dictionaryDeltas map[string]int64
dictionaryDeltas, err = f.batchRows(kvwriter, result.Rows, nil)
if err != nil {
_ = kvwriter.Close()
atomic.AddUint64(&f.stats.errors, 1)
return
}
f.compensator.Mutate([]byte(doc.ID), doc.Number)
f.lookuper.Notify(doc.Number, []byte(doc.ID))
f.dictUpdater.NotifyBatch(dictionaryDeltas)
atomic.AddUint64(&f.stats.indexTime, uint64(time.Since(indexStart)))
return
}
func (f *Firestorm) Delete(id string) error {
indexStart := time.Now()
f.compensator.Mutate([]byte(id), 0)
f.lookuper.Notify(0, []byte(id))
atomic.AddUint64(&f.stats.indexTime, uint64(time.Since(indexStart)))
return nil
}
func (f *Firestorm) batchRows(writer store.KVWriter, rows []index.IndexRow, deleteKeys [][]byte) (map[string]int64, error) {
// prepare batch
wb := writer.NewBatch()
dictionaryDeltas := make(map[string]int64)
for _, row := range rows {
tfr, ok := row.(*TermFreqRow)
if ok {
if tfr.Field() != 0 {
drk := tfr.DictionaryRowKey()
dictionaryDeltas[string(drk)] += 1
}
}
wb.Set(row.Key(), row.Value())
}
for _, dk := range deleteKeys {
wb.Delete(dk)
}
// write out the batch
err := wb.Execute()
if err != nil {
return nil, err
}
return dictionaryDeltas, nil
}
func (f *Firestorm) Batch(batch *index.Batch) (err error) {
// acquire enough doc numbers for all updates in the batch
// FIXME we actually waste doc numbers because deletes are in the
// same map and we don't need numbers for them
lastDocNumber := atomic.AddUint64(&f.highDocNumber, uint64(len(batch.IndexOps)))
firstDocNumber := lastDocNumber - uint64(len(batch.IndexOps)) + 1
analysisStart := time.Now()
resultChan := make(chan *index.AnalysisResult)
var numUpdates uint64
var docsDeleted uint64
for _, doc := range batch.IndexOps {
if doc != nil {
doc.Number = firstDocNumber // actually assign doc numbers here
firstDocNumber++
numUpdates++
} else {
docsDeleted++
}
}
var detectedUnsafeMutex sync.RWMutex
detectedUnsafe := false
go func() {
sofar := uint64(0)
for _, doc := range batch.IndexOps {
if doc != nil {
sofar++
if sofar > numUpdates {
detectedUnsafeMutex.Lock()
detectedUnsafe = true
detectedUnsafeMutex.Unlock()
return
}
aw := index.NewAnalysisWork(f, doc, resultChan)
// put the work on the queue
f.analysisQueue.Queue(aw)
}
}
}()
newRowsMap := make(map[string][]index.IndexRow)
// wait for the result
var itemsDeQueued uint64
for itemsDeQueued < numUpdates {
result := <-resultChan
newRowsMap[result.DocID] = result.Rows
itemsDeQueued++
}
close(resultChan)
detectedUnsafeMutex.RLock()
defer detectedUnsafeMutex.RUnlock()
if detectedUnsafe {
return UnsafeBatchUseDetected
}
atomic.AddUint64(&f.stats.analysisTime, uint64(time.Since(analysisStart)))
allRows := make([]index.IndexRow, 0)
deleteKeys := make([][]byte, 0)
for docID := range batch.IndexOps {
allRows = append(allRows, newRowsMap[docID]...)
}
// add the internal ops
for internalKey, internalValue := range batch.InternalOps {
if internalValue == nil {
// delete
deleteInternalRow := NewInternalRow([]byte(internalKey), nil)
deleteKeys = append(deleteKeys, deleteInternalRow.Key())
} else {
updateInternalRow := NewInternalRow([]byte(internalKey), internalValue)
allRows = append(allRows, updateInternalRow)
}
}
indexStart := time.Now()
// start a writer for this batch
var kvwriter store.KVWriter
kvwriter, err = f.store.Writer()
if err != nil {
return
}
var dictionaryDeltas map[string]int64
dictionaryDeltas, err = f.batchRows(kvwriter, allRows, deleteKeys)
if err != nil {
_ = kvwriter.Close()
atomic.AddUint64(&f.stats.errors, 1)
return
}
f.compensator.MutateBatch(batch.IndexOps, lastDocNumber)
for docID, doc := range batch.IndexOps {
if doc != nil {
f.lookuper.Notify(doc.Number, []byte(doc.ID))
} else {
f.lookuper.Notify(0, []byte(docID))
}
}
f.dictUpdater.NotifyBatch(dictionaryDeltas)
err = kvwriter.Close()
atomic.AddUint64(&f.stats.indexTime, uint64(time.Since(indexStart)))
if err == nil {
atomic.AddUint64(&f.stats.updates, numUpdates)
atomic.AddUint64(&f.stats.deletes, docsDeleted)
atomic.AddUint64(&f.stats.batches, 1)
} else {
atomic.AddUint64(&f.stats.errors, 1)
}
return
}
func (f *Firestorm) SetInternal(key, val []byte) (err error) {
internalRow := NewInternalRow(key, val)
var writer store.KVWriter
writer, err = f.store.Writer()
if err != nil {
return
}
defer func() {
if cerr := writer.Close(); err == nil && cerr != nil {
err = cerr
}
}()
return writer.Set(internalRow.Key(), internalRow.Value())
}
func (f *Firestorm) DeleteInternal(key []byte) (err error) {
internalRow := NewInternalRow(key, nil)
var writer store.KVWriter
writer, err = f.store.Writer()
if err != nil {
return
}
defer func() {
if cerr := writer.Close(); err == nil && cerr != nil {
err = cerr
}
}()
return writer.Delete(internalRow.Key())
}
func (f *Firestorm) DumpAll() chan interface{} {
rv := make(chan interface{})
go func() {
defer close(rv)
// start an isolated reader for use during the dump
kvreader, err := f.store.Reader()
if err != nil {
rv <- err
return
}
defer func() {
cerr := kvreader.Close()
if cerr != nil {
rv <- cerr
}
}()
f.dumpPrefix(kvreader, rv, nil)
}()
return rv
}
func (f *Firestorm) DumpDoc(docID string) chan interface{} {
rv := make(chan interface{})
go func() {
defer close(rv)
// start an isolated reader for use during the dump
kvreader, err := f.store.Reader()
if err != nil {
rv <- err
return
}
defer func() {
cerr := kvreader.Close()
if cerr != nil {
rv <- cerr
}
}()
f.dumpDoc(kvreader, rv, []byte(docID))
}()
return rv
}
func (f *Firestorm) DumpFields() chan interface{} {
rv := make(chan interface{})
go func() {
defer close(rv)
// start an isolated reader for use during the dump
kvreader, err := f.store.Reader()
if err != nil {
rv <- err
return
}
defer func() {
cerr := kvreader.Close()
if cerr != nil {
rv <- cerr
}
}()
f.dumpPrefix(kvreader, rv, FieldKeyPrefix)
}()
return rv
}
func (f *Firestorm) Reader() (index.IndexReader, error) {
return newFirestormReader(f)
}
func (f *Firestorm) Stats() json.Marshaler {
return f.stats
}
func IndexTypeConstructor(store store.KVStore, analysisQueue *index.AnalysisQueue) (index.Index, error) {
return NewFirestorm(store, analysisQueue), nil
}
func init() {
registry.RegisterIndexType(Name, IndexTypeConstructor)
}

View File

@ -3,7 +3,7 @@
// DO NOT EDIT!
/*
Package firestorm_rows is a generated protocol buffer package.
Package firestorm is a generated protocol buffer package.
It is generated from these files:
firestorm_rows.proto
@ -12,11 +12,11 @@
VersionValue
FieldValue
DictionaryValue
TermVectorEntry
TermFrequencyValue
TermVector
TermFreqValue
StoredValue
*/
package firestorm_rows
package firestorm
import proto "github.com/golang/protobuf/proto"
import math "math"
@ -77,7 +77,7 @@ func (m *DictionaryValue) GetCount() uint64 {
return 0
}
type TermVectorEntry struct {
type TermVector struct {
Field *uint32 `protobuf:"varint,1,opt,name=field" json:"field,omitempty"`
Pos *uint64 `protobuf:"varint,2,opt,name=pos" json:"pos,omitempty"`
Start *uint64 `protobuf:"varint,3,opt,name=start" json:"start,omitempty"`
@ -86,71 +86,71 @@ type TermVectorEntry struct {
XXX_unrecognized []byte `json:"-"`
}
func (m *TermVectorEntry) Reset() { *m = TermVectorEntry{} }
func (m *TermVectorEntry) String() string { return proto.CompactTextString(m) }
func (*TermVectorEntry) ProtoMessage() {}
func (m *TermVector) Reset() { *m = TermVector{} }
func (m *TermVector) String() string { return proto.CompactTextString(m) }
func (*TermVector) ProtoMessage() {}
func (m *TermVectorEntry) GetField() uint32 {
func (m *TermVector) GetField() uint32 {
if m != nil && m.Field != nil {
return *m.Field
}
return 0
}
func (m *TermVectorEntry) GetPos() uint64 {
func (m *TermVector) GetPos() uint64 {
if m != nil && m.Pos != nil {
return *m.Pos
}
return 0
}
func (m *TermVectorEntry) GetStart() uint64 {
func (m *TermVector) GetStart() uint64 {
if m != nil && m.Start != nil {
return *m.Start
}
return 0
}
func (m *TermVectorEntry) GetEnd() uint64 {
func (m *TermVector) GetEnd() uint64 {
if m != nil && m.End != nil {
return *m.End
}
return 0
}
func (m *TermVectorEntry) GetArrayPositions() []uint64 {
func (m *TermVector) GetArrayPositions() []uint64 {
if m != nil {
return m.ArrayPositions
}
return nil
}
type TermFrequencyValue struct {
Freq *uint64 `protobuf:"varint,1,req,name=freq" json:"freq,omitempty"`
Norm *float32 `protobuf:"fixed32,2,opt,name=norm" json:"norm,omitempty"`
Vectors []*TermVectorEntry `protobuf:"bytes,3,rep,name=vectors" json:"vectors,omitempty"`
XXX_unrecognized []byte `json:"-"`
type TermFreqValue struct {
Freq *uint64 `protobuf:"varint,1,req,name=freq" json:"freq,omitempty"`
Norm *float32 `protobuf:"fixed32,2,opt,name=norm" json:"norm,omitempty"`
Vectors []*TermVector `protobuf:"bytes,3,rep,name=vectors" json:"vectors,omitempty"`
XXX_unrecognized []byte `json:"-"`
}
func (m *TermFrequencyValue) Reset() { *m = TermFrequencyValue{} }
func (m *TermFrequencyValue) String() string { return proto.CompactTextString(m) }
func (*TermFrequencyValue) ProtoMessage() {}
func (m *TermFreqValue) Reset() { *m = TermFreqValue{} }
func (m *TermFreqValue) String() string { return proto.CompactTextString(m) }
func (*TermFreqValue) ProtoMessage() {}
func (m *TermFrequencyValue) GetFreq() uint64 {
func (m *TermFreqValue) GetFreq() uint64 {
if m != nil && m.Freq != nil {
return *m.Freq
}
return 0
}
func (m *TermFrequencyValue) GetNorm() float32 {
func (m *TermFreqValue) GetNorm() float32 {
if m != nil && m.Norm != nil {
return *m.Norm
}
return 0
}
func (m *TermFrequencyValue) GetVectors() []*TermVectorEntry {
func (m *TermFreqValue) GetVectors() []*TermVector {
if m != nil {
return m.Vectors
}
@ -378,7 +378,7 @@ func (m *DictionaryValue) Unmarshal(data []byte) error {
return nil
}
func (m *TermVectorEntry) Unmarshal(data []byte) error {
func (m *TermVector) Unmarshal(data []byte) error {
l := len(data)
iNdEx := 0
for iNdEx < l {
@ -509,7 +509,7 @@ func (m *TermVectorEntry) Unmarshal(data []byte) error {
return nil
}
func (m *TermFrequencyValue) Unmarshal(data []byte) error {
func (m *TermFreqValue) Unmarshal(data []byte) error {
var hasFields [1]uint64
l := len(data)
iNdEx := 0
@ -585,7 +585,7 @@ func (m *TermFrequencyValue) Unmarshal(data []byte) error {
if postIndex > l {
return io.ErrUnexpectedEOF
}
m.Vectors = append(m.Vectors, &TermVectorEntry{})
m.Vectors = append(m.Vectors, &TermVector{})
if err := m.Vectors[len(m.Vectors)-1].Unmarshal(data[iNdEx:postIndex]); err != nil {
return err
}
@ -820,7 +820,7 @@ func (m *DictionaryValue) Size() (n int) {
return n
}
func (m *TermVectorEntry) Size() (n int) {
func (m *TermVector) Size() (n int) {
var l int
_ = l
if m.Field != nil {
@ -846,7 +846,7 @@ func (m *TermVectorEntry) Size() (n int) {
return n
}
func (m *TermFrequencyValue) Size() (n int) {
func (m *TermFreqValue) Size() (n int) {
var l int
_ = l
if m.Freq != nil {
@ -976,7 +976,7 @@ func (m *DictionaryValue) MarshalTo(data []byte) (n int, err error) {
return i, nil
}
func (m *TermVectorEntry) Marshal() (data []byte, err error) {
func (m *TermVector) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
@ -986,7 +986,7 @@ func (m *TermVectorEntry) Marshal() (data []byte, err error) {
return data[:n], nil
}
func (m *TermVectorEntry) MarshalTo(data []byte) (n int, err error) {
func (m *TermVector) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int
@ -1024,7 +1024,7 @@ func (m *TermVectorEntry) MarshalTo(data []byte) (n int, err error) {
return i, nil
}
func (m *TermFrequencyValue) Marshal() (data []byte, err error) {
func (m *TermFreqValue) Marshal() (data []byte, err error) {
size := m.Size()
data = make([]byte, size)
n, err := m.MarshalTo(data)
@ -1034,7 +1034,7 @@ func (m *TermFrequencyValue) Marshal() (data []byte, err error) {
return data[:n], nil
}
func (m *TermFrequencyValue) MarshalTo(data []byte) (n int, err error) {
func (m *TermFreqValue) MarshalTo(data []byte) (n int, err error) {
var i int
_ = i
var l int

View File

@ -1,3 +1,5 @@
package firestorm;
message VersionValue {
required uint64 version = 1;
}
@ -10,7 +12,7 @@ message DictionaryValue {
optional uint64 count = 1; // number of documents using this term in this field
}
message TermVectorEntry {
message TermVector {
optional uint32 field = 1; // field optional if redundant, required for composite fields
optional uint64 pos = 2; // positional offset within the field
optional uint64 start = 3; // start byte offset
@ -18,10 +20,10 @@ message TermVectorEntry {
repeated uint64 arrayPositions = 5; // array positions
}
message TermFrequencyValue {
message TermFreqValue {
required uint64 freq = 1; // frequency of the term occurance within this field
optional float norm = 2; // normalization factor
repeated TermVectorEntry vectors = 3; // term vectors
repeated TermVector vectors = 3; // term vectors
}
message StoredValue {

File diff suppressed because it is too large Load Diff

216
index/firestorm/garbage.go Normal file
View File

@ -0,0 +1,216 @@
// 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 firestorm
import (
"math"
"sync"
"time"
)
const DefaultGarbageThreshold = 10
const DefaultMaxDocsPerPass = 1000
var DefaultGarbageSleep = 15 * time.Second
type GarbageCollector struct {
f *Firestorm
garbageThreshold int
garbageSleep time.Duration
maxDocsPerPass int
quit chan struct{}
mutex sync.RWMutex
workingSet map[uint64][]byte
closeWait sync.WaitGroup
}
func NewGarbageCollector(f *Firestorm) *GarbageCollector {
rv := GarbageCollector{
f: f,
garbageThreshold: DefaultGarbageThreshold,
garbageSleep: DefaultGarbageSleep,
maxDocsPerPass: DefaultMaxDocsPerPass,
quit: make(chan struct{}),
workingSet: make(map[uint64][]byte),
}
return &rv
}
func (gc *GarbageCollector) Notify(docNum uint64, docId []byte) {
gc.mutex.Lock()
defer gc.mutex.Unlock()
gc.workingSet[docNum] = docId
}
func (gc *GarbageCollector) Start() {
gc.closeWait.Add(1)
go gc.run()
}
func (gc *GarbageCollector) Stop() {
close(gc.quit)
gc.closeWait.Wait()
}
func (gc *GarbageCollector) run() {
tick := time.Tick(gc.garbageSleep)
for {
select {
case <-gc.quit:
logger.Printf("garbage collector asked to quit")
gc.closeWait.Done()
return
case <-tick:
logger.Printf("garbage collector ticked")
garbageSize := gc.f.compensator.GarbageCount()
docSize, err := gc.f.DocCount()
if err != nil {
logger.Printf("garbage collector error getting doc count: %v")
continue
}
garbageRatio := int(uint64(garbageSize) / docSize)
if garbageRatio > gc.garbageThreshold {
gc.cleanup()
} else {
logger.Printf("garbage ratio only %d, waiting", garbageRatio)
}
}
}
}
func (gc *GarbageCollector) NextBatch(n int) []uint64 {
gc.mutex.RLock()
defer gc.mutex.RUnlock()
rv := make([]uint64, 0, n)
i := 0
for k := range gc.workingSet {
rv = append(rv, k)
i++
if i > n {
break
}
}
return rv
}
func (gc *GarbageCollector) cleanup() {
logger.Printf("garbage collector starting")
// get list of deleted doc numbers to work on this pass
deletedDocNumsList := gc.NextBatch(gc.maxDocsPerPass) //gc.f.deletedDocNumbers.Keys(gc.maxDocsPerPass)
logger.Printf("found %d doc numbers to cleanup", len(deletedDocNumsList))
// put these documents numbers in a map, for faster checking
// and for organized keys to be deleted
deletedDocNums := make(map[uint64][][]byte)
for _, deletedDocNum := range deletedDocNumsList {
deletedDocNums[deletedDocNum] = make([][]byte, 0)
}
reader, err := gc.f.store.Reader()
if err != nil {
logger.Printf("garbage collector fatal: %v", err)
return
}
defer reader.Close()
// walk all the term freq rows (where field > 0)
termFreqStart := TermFreqIteratorStart(0, []byte{ByteSeparator})
termFreqEnd := TermFreqIteratorStart(math.MaxUint16, []byte{ByteSeparator})
dictionaryDeltas := make(map[string]int64)
err = visitRange(reader, termFreqStart, termFreqEnd, func(key, val []byte) (bool, error) {
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return false, err
}
docNum := tfr.DocNum()
if docNumKeys, deleted := deletedDocNums[docNum]; deleted {
// this doc number has been deleted, place key into map
deletedDocNums[docNum] = append(docNumKeys, key)
if tfr.Field() != 0 {
drk := tfr.DictionaryRowKey()
dictionaryDeltas[string(drk)] -= 1
}
}
return true, nil
})
if err != nil {
logger.Printf("garbage collector fatal: %v")
return
}
// walk all the stored rows
err = visitPrefix(reader, StoredKeyPrefix, func(key, val []byte) (bool, error) {
sr, err := NewStoredRowKV(key, val)
if err != nil {
return false, err
}
docNum := sr.DocNum()
if docNumKeys, deleted := deletedDocNums[docNum]; deleted {
// this doc number has been deleted, place key into map
deletedDocNums[docNum] = append(docNumKeys, key)
}
return true, nil
})
if err != nil {
logger.Printf("garbage collector fatal: %v")
return
}
// now process each doc one at a time
for docNum, docKeys := range deletedDocNums {
// delete keys for a doc number
logger.Printf("deleting keys for %d", docNum)
// open a writer
writer, err := gc.f.store.Writer()
if err != nil {
writer.Close()
logger.Printf("garbage collector fatal: %v")
return
}
// prepare batch
wb := writer.NewBatch()
for _, k := range docKeys {
wb.Delete(k)
}
err = wb.Execute()
if err != nil {
writer.Close()
logger.Printf("garbage collector fatal: %v")
return
}
logger.Printf("deleted %d keys", len(docKeys))
// remove it from delete keys list
docID := gc.workingSet[docNum]
delete(gc.workingSet, docNum)
gc.f.compensator.GarbageCollect([]uint64{docNum})
// now delete the original marker row (field 0)
tfidrow := NewTermFreqRow(0, nil, docID, docNum, 0, 0, nil)
markerRowKey := tfidrow.Key()
writer.Delete(markerRowKey)
writer.Close()
}
// updating dictionary in one batch
gc.f.dictUpdater.NotifyBatch(dictionaryDeltas)
logger.Printf("garbage collector finished")
}

View File

@ -0,0 +1,109 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestGarbageCleanup(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []struct {
row index.IndexRow
garbage bool
}{
// needed for warmup to work
{NewFieldRow(0, IDFieldName), false},
// 3 documents, with 2 older versions
{NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil), true},
{NewTermFreqRow(0, nil, []byte("a"), 2, 0, 0.0, nil), false},
{NewTermFreqRow(0, nil, []byte("b"), 3, 0, 0.0, nil), false},
{NewTermFreqRow(0, nil, []byte("c"), 4, 0, 0.0, nil), true},
{NewTermFreqRow(0, nil, []byte("c"), 5, 0, 0.0, nil), false},
// additional records for these docs which should be removed
{NewTermFreqRow(1, []byte("cat"), []byte("a"), 1, 3, 2.0, nil), true},
{NewTermFreqRow(1, []byte("cat"), []byte("c"), 4, 1, 1.0, nil), true},
{NewStoredRow([]byte("a"), 1, 1, nil, []byte("tcat")), true},
{NewStoredRow([]byte("c"), 4, 1, nil, []byte("tcat")), true},
}
for _, row := range rows {
err = kvwriter.Set(row.row.Key(), row.row.Value())
if err != nil {
t.Fatal(err)
}
}
// warmup ensures that deletedDocNums is seeded correctly
err = f.warmup(kvwriter)
if err != nil {
t.Fatal(err)
}
err = kvwriter.Close()
if err != nil {
t.Fatal(err)
}
// now invoke garbage collector cleanup manually
f.garbageCollector.cleanup()
// assert that garbage rows are gone
reader, err := f.store.Reader()
if err != nil {
t.Fatal(err)
}
for _, row := range rows {
v, err := reader.Get(row.row.Key())
if err != nil {
t.Fatal(err)
}
if v != nil && row.garbage {
t.Errorf("garbage row not deleted, key: %s", row.row.Key())
}
if v == nil && !row.garbage {
t.Errorf("non-garbage row deleted, key: %s", row.row.Key())
}
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// assert that deletedDocsNumbers size is 0
if f.compensator.GarbageCount() != 0 {
t.Errorf("expected deletedDocsNumbers size to be 0, got %d", f.compensator.GarbageCount())
}
}

View File

@ -0,0 +1,43 @@
// 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 firestorm
var InternalKeyPrefix = []byte{'d'}
type InternalRow struct {
key []byte
val []byte
}
func NewInternalRow(key, val []byte) *InternalRow {
rv := InternalRow{
key: key,
val: val,
}
return &rv
}
func NewInternalRowKV(key, value []byte) (*InternalRow, error) {
rv := InternalRow{}
rv.key = key[1:]
rv.val = value
return &rv, nil
}
func (ir *InternalRow) Key() []byte {
buf := make([]byte, len(ir.key)+1)
buf[0] = 'i'
copy(buf[1:], ir.key)
return buf
}
func (ir *InternalRow) Value() []byte {
return ir.val
}

View File

@ -0,0 +1,54 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestInternalRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewInternalRow([]byte("key"), []byte("val")),
[]byte{'i', 'k', 'e', 'y'},
[]byte{'v', 'a', 'l'},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewInternalRowKV(test.outKey, test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected: %#v got: %#v for %d", test.input, row, i)
}
}
}

140
index/firestorm/lookup.go Normal file
View File

@ -0,0 +1,140 @@
// 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 firestorm
import (
"fmt"
"sync"
"sync/atomic"
"time"
)
const channelBufferSize = 1000
type lookupTask struct {
docID []byte
docNum uint64
}
type Lookuper struct {
f *Firestorm
workChan chan *lookupTask
quit chan struct{}
closeWait sync.WaitGroup
tasksQueued uint64
tasksDone uint64
}
func NewLookuper(f *Firestorm) *Lookuper {
rv := Lookuper{
f: f,
workChan: make(chan *lookupTask, channelBufferSize),
quit: make(chan struct{}),
}
return &rv
}
func (l *Lookuper) Notify(docNum uint64, docID []byte) {
atomic.AddUint64(&l.tasksQueued, 1)
l.workChan <- &lookupTask{docID: docID, docNum: docNum}
}
func (l *Lookuper) Start() {
l.closeWait.Add(1)
go l.run()
}
func (l *Lookuper) Stop() {
close(l.quit)
l.closeWait.Wait()
}
func (l *Lookuper) run() {
for {
select {
case <-l.quit:
logger.Printf("lookuper asked to quit")
l.closeWait.Done()
return
case task, ok := <-l.workChan:
if !ok {
logger.Printf("lookuper work channel closed unexpectedly, stopping")
return
}
l.lookup(task)
}
}
}
func (l *Lookuper) lookup(task *lookupTask) {
reader, err := l.f.store.Reader()
if err != nil {
logger.Printf("lookuper fatal: %v", err)
return
}
defer reader.Close()
prefix := TermFreqPrefixFieldTermDocId(0, nil, task.docID)
logger.Printf("lookuper prefix - % x", prefix)
docNums := make(DocNumberList, 0)
err = visitPrefix(reader, prefix, func(key, val []byte) (bool, error) {
logger.Printf("lookuper sees key % x", key)
tfk, err := NewTermFreqRowKV(key, val)
if err != nil {
return false, err
}
docNum := tfk.DocNum()
docNums = append(docNums, docNum)
return true, nil
})
if err != nil {
logger.Printf("lookuper fatal: %v", err)
return
}
oldDocNums := make(DocNumberList, 0, len(docNums))
for _, docNum := range docNums {
if task.docNum == 0 || docNum < task.docNum {
oldDocNums = append(oldDocNums, docNum)
}
}
logger.Printf("lookup migrating '%s' - %d - oldDocNums: %v", task.docID, task.docNum, oldDocNums)
l.f.compensator.Migrate(task.docID, task.docNum, oldDocNums)
if len(oldDocNums) == 0 && task.docNum != 0 {
// this was an add, not an update
atomic.AddUint64(l.f.docCount, 1)
} else if len(oldDocNums) > 0 && task.docNum == 0 {
// this was a delete (and it previously existed)
atomic.AddUint64(l.f.docCount, ^uint64(0))
}
atomic.AddUint64(&l.tasksDone, 1)
}
// this is not intended to be used publicly, only for unit tests
// which depend on consistency we no longer provide
func (l *Lookuper) waitTasksDone(d time.Duration) error {
timeout := time.After(d)
tick := time.Tick(100 * time.Millisecond)
for {
select {
// Got a timeout! fail with a timeout error
case <-timeout:
return fmt.Errorf("timeout")
// Got a tick, we should check on doSomething()
case <-tick:
queued := atomic.LoadUint64(&l.tasksQueued)
done := atomic.LoadUint64(&l.tasksDone)
if queued == done {
return nil
}
}
}
}

View File

@ -0,0 +1,85 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestLookups(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []struct {
row index.IndexRow
garbage bool
}{
// needed for warmup to work
{NewFieldRow(0, IDFieldName), false},
// 3 documents, with 2 older versions
{NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil), true},
{NewTermFreqRow(0, nil, []byte("a"), 2, 0, 0.0, nil), false},
{NewTermFreqRow(0, nil, []byte("b"), 3, 0, 0.0, nil), false},
{NewTermFreqRow(0, nil, []byte("c"), 4, 0, 0.0, nil), true},
{NewTermFreqRow(0, nil, []byte("c"), 5, 0, 0.0, nil), false},
}
for _, row := range rows {
err = kvwriter.Set(row.row.Key(), row.row.Value())
if err != nil {
t.Fatal(err)
}
// also see the compensator
if tfr, ok := row.row.(*TermFreqRow); ok {
f.compensator.Mutate(tfr.DocID(), tfr.DocNum())
// expect this mutation to be in the in-flight list
val := f.compensator.inFlight.Get(&InFlightItem{docID: tfr.DocID()})
if val == nil {
t.Errorf("expected key: % x to be in the inflight list", tfr.DocID())
}
f.lookuper.lookup(&lookupTask{docID: tfr.DocID(), docNum: tfr.DocNum()})
// now expect this mutation to NOT be in the in-flight list
val = f.compensator.inFlight.Get(&InFlightItem{docID: tfr.DocID()})
if val != nil {
t.Errorf("expected key: % x to NOT be in the inflight list, got %v", tfr.DocID(), val)
}
}
}
// check that doc count is 3 at the end of this
docCount, err := f.DocCount()
if err != nil {
t.Fatal(err)
}
if docCount != 3 {
t.Errorf("expected doc count 3, got %d", docCount)
}
}

71
index/firestorm/merge.go Normal file
View File

@ -0,0 +1,71 @@
// 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 firestorm
import (
"encoding/binary"
)
var mergeOperator firestormMerge
var dictionaryTermIncr []byte
var dictionaryTermDecr []byte
func init() {
dictionaryTermIncr = make([]byte, 8)
binary.LittleEndian.PutUint64(dictionaryTermIncr, uint64(1))
dictionaryTermDecr = make([]byte, 8)
var negOne = int64(-1)
binary.LittleEndian.PutUint64(dictionaryTermDecr, uint64(negOne))
}
type firestormMerge struct{}
func (m *firestormMerge) FullMerge(key, existingValue []byte, operands [][]byte) ([]byte, bool) {
// set up record based on key
dr, err := NewDictionaryRowK(key)
if err != nil {
return nil, false
}
if len(existingValue) > 0 {
// if existing value, parse it
err = dr.parseDictionaryV(existingValue)
if err != nil {
return nil, false
}
}
// now process operands
for _, operand := range operands {
next := int64(binary.LittleEndian.Uint64(operand))
if next < 0 && uint64(-next) > dr.Count() {
// subtracting next from existing would overflow
dr.SetCount(0)
} else if next < 0 {
dr.SetCount(dr.Count() - uint64(-next))
} else {
dr.SetCount(dr.Count() + uint64(next))
}
}
return dr.Value(), true
}
func (m *firestormMerge) PartialMerge(key, leftOperand, rightOperand []byte) ([]byte, bool) {
left := int64(binary.LittleEndian.Uint64(leftOperand))
right := int64(binary.LittleEndian.Uint64(rightOperand))
rv := make([]byte, 8)
binary.LittleEndian.PutUint64(rv, uint64(left+right))
return rv, true
}
func (m *firestormMerge) Name() string {
return "firestormMerge"
}

View File

@ -0,0 +1,93 @@
// 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 firestorm
import (
"bytes"
"encoding/binary"
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestPartialMerge(t *testing.T) {
tests := []struct {
in [][]byte
out uint64
}{
{
in: [][]byte{dictionaryTermIncr, dictionaryTermIncr, dictionaryTermIncr, dictionaryTermIncr, dictionaryTermIncr},
out: 5,
},
}
mo := &firestormMerge{}
for _, test := range tests {
curr := test.in[0]
for _, next := range test.in[1:] {
var ok bool
curr, ok = mo.PartialMerge([]byte("key"), curr, next)
if !ok {
t.Errorf("expected partial merge ok")
}
}
actual := decodeCount(curr)
if actual != test.out {
t.Errorf("expected %d, got %d", test.out, actual)
}
}
}
func decodeCount(in []byte) uint64 {
buf := bytes.NewBuffer(in)
count, _ := binary.ReadUvarint(buf)
return count
}
func TestFullMerge(t *testing.T) {
tests := []struct {
existing index.IndexRow
operands [][]byte
result index.IndexRow
success bool
}{
{
existing: NewDictionaryRow(1, []byte("term"), 3),
operands: [][]byte{dictionaryTermIncr, dictionaryTermIncr},
result: NewDictionaryRow(1, []byte("term"), 5),
success: true,
},
{
existing: NewDictionaryRow(1, []byte("term"), 3),
operands: [][]byte{dictionaryTermDecr, dictionaryTermDecr},
result: NewDictionaryRow(1, []byte("term"), 1),
success: true,
},
}
mo := &firestormMerge{}
for _, test := range tests {
existingVal := test.existing.Value()
actual, success := mo.FullMerge([]byte("key"), existingVal, test.operands)
if success != test.success {
t.Errorf("expected error %t, got %t", test.success, success)
}
expectedVal := test.result.Value()
if !reflect.DeepEqual(expectedVal, actual) {
t.Errorf("expected result %v, got %v", expectedVal, actual)
}
}
}

218
index/firestorm/reader.go Normal file
View File

@ -0,0 +1,218 @@
// 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 firestorm
import (
"bytes"
"fmt"
"sort"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
)
type firestormReader struct {
f *Firestorm
r store.KVReader
s *Snapshot
docCount uint64
}
func newFirestormReader(f *Firestorm) (index.IndexReader, error) {
r, err := f.store.Reader()
if err != nil {
return nil, fmt.Errorf("error opening store reader: %v", err)
}
docCount, err := f.DocCount()
if err != nil {
return nil, fmt.Errorf("error opening store reader: %v", err)
}
rv := firestormReader{
f: f,
r: r,
s: f.compensator.Snapshot(),
docCount: docCount,
}
return &rv, nil
}
func (r *firestormReader) TermFieldReader(term []byte, field string) (index.TermFieldReader, error) {
fieldIndex, fieldExists := r.f.fieldCache.FieldNamed(field, false)
if fieldExists {
return newFirestormTermFieldReader(r, uint16(fieldIndex), term)
}
return newFirestormTermFieldReader(r, ^uint16(0), []byte{ByteSeparator})
}
func (r *firestormReader) DocIDReader(start, end string) (index.DocIDReader, error) {
return newFirestormDocIDReader(r, start, end)
}
func (r *firestormReader) FieldDict(field string) (index.FieldDict, error) {
return r.FieldDictRange(field, nil, nil)
}
func (r *firestormReader) FieldDictRange(field string, startTerm []byte, endTerm []byte) (index.FieldDict, error) {
fieldIndex, fieldExists := r.f.fieldCache.FieldNamed(field, false)
if fieldExists {
return newFirestormDictionaryReader(r, uint16(fieldIndex), startTerm, endTerm)
}
return newFirestormDictionaryReader(r, ^uint16(0), []byte{ByteSeparator}, []byte{})
}
func (r *firestormReader) FieldDictPrefix(field string, termPrefix []byte) (index.FieldDict, error) {
return r.FieldDictRange(field, termPrefix, incrementBytes(termPrefix))
}
func (r *firestormReader) Document(id string) (*document.Document, error) {
docID := []byte(id)
docNum, err := r.currDocNumForId(docID)
if err != nil {
return nil, err
} else if docNum == 0 {
return nil, nil
}
rv := document.NewDocument(id)
prefix := StoredPrefixDocIDNum(docID, docNum)
err = visitPrefix(r.r, prefix, func(key, val []byte) (bool, error) {
safeVal := val
if !r.r.BytesSafeAfterClose() {
safeVal = make([]byte, len(val))
copy(safeVal, val)
}
row, err := NewStoredRowKV(key, safeVal)
if err != nil {
return false, err
}
if row != nil {
fieldName := r.f.fieldCache.FieldIndexed(row.field)
field := r.decodeFieldType(fieldName, row.arrayPositions, row.value.GetRaw())
if field != nil {
rv.AddField(field)
}
}
return true, nil
})
if err != nil {
return nil, err
}
return rv, nil
}
func (r *firestormReader) decodeFieldType(name string, pos []uint64, value []byte) document.Field {
switch value[0] {
case 't':
return document.NewTextField(name, pos, value[1:])
case 'n':
return document.NewNumericFieldFromBytes(name, pos, value[1:])
case 'd':
return document.NewDateTimeFieldFromBytes(name, pos, value[1:])
}
return nil
}
func (r *firestormReader) currDocNumForId(docID []byte) (uint64, error) {
prefix := TermFreqPrefixFieldTermDocId(0, nil, docID)
docNums := make(DocNumberList, 0)
err := visitPrefix(r.r, prefix, func(key, val []byte) (bool, error) {
tfk, err := NewTermFreqRowKV(key, val)
if err != nil {
return false, err
}
docNum := tfk.DocNum()
docNums = append(docNums, docNum)
return true, nil
})
if err != nil {
return 0, err
}
if len(docNums) > 0 {
sort.Sort(docNums)
return docNums[0], nil
}
return 0, nil
}
func (r *firestormReader) DocumentFieldTerms(id string) (index.FieldTerms, error) {
docID := []byte(id)
docNum, err := r.currDocNumForId(docID)
if err != nil {
return nil, err
} else if docNum == 0 {
return nil, nil
}
rv := make(index.FieldTerms, 0)
// walk the term freqs
visitPrefix(r.r, TermFreqKeyPrefix, func(key, val []byte) (bool, error) {
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return false, err
}
if bytes.Compare(tfr.DocID(), docID) == 0 && tfr.DocNum() == docNum && tfr.Field() != 0 {
fieldName := r.f.fieldCache.FieldIndexed(uint16(tfr.Field()))
terms, ok := rv[fieldName]
if !ok {
terms = make([]string, 0, 1)
}
terms = append(terms, string(tfr.Term()))
rv[fieldName] = terms
}
return true, nil
})
return rv, nil
}
func (r *firestormReader) Fields() ([]string, error) {
fields := make([]string, 0)
err := visitPrefix(r.r, FieldKeyPrefix, func(key, val []byte) (bool, error) {
fieldRow, err := NewFieldRowKV(key, val)
if err != nil {
return false, err
}
fields = append(fields, fieldRow.Name())
return true, nil
})
if err != nil {
return nil, err
}
return fields, nil
}
func (r *firestormReader) GetInternal(key []byte) ([]byte, error) {
internalRow := NewInternalRow(key, nil)
return r.r.Get(internalRow.Key())
}
func (r *firestormReader) DocCount() uint64 {
return r.docCount
}
func (r *firestormReader) Close() error {
return r.r.Close()
}
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
}

View File

@ -0,0 +1,78 @@
// 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 firestorm
import (
"bytes"
"fmt"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
)
type firestormDictionaryReader struct {
r *firestormReader
field uint16
start []byte
end []byte
i store.KVIterator
}
func newFirestormDictionaryReader(r *firestormReader, field uint16, start, end []byte) (*firestormDictionaryReader, error) {
startKey := DictionaryRowKey(field, start)
logger.Printf("start key '%s' - % x", startKey, startKey)
if end == nil {
end = []byte{ByteSeparator}
}
endKey := DictionaryRowKey(field, end)
logger.Printf("end key '%s' - % x", endKey, endKey)
i := r.r.Iterator(startKey)
rv := firestormDictionaryReader{
r: r,
field: field,
start: startKey,
end: endKey,
i: i,
}
return &rv, nil
}
func (r *firestormDictionaryReader) Next() (*index.DictEntry, error) {
key, val, valid := r.i.Current()
if !valid {
return nil, nil
}
logger.Printf("see key '%s' - % x", key, key)
// past end term
if bytes.Compare(key, r.end) > 0 {
return nil, nil
}
currRow, err := NewDictionaryRowKV(key, val)
if err != nil {
return nil, fmt.Errorf("unexpected error parsing dictionary row kv: %v", err)
}
rv := index.DictEntry{
Term: string(currRow.term),
Count: currRow.Count(),
}
// advance the iterator to the next term
r.i.Next()
return &rv, nil
}
func (r *firestormDictionaryReader) Close() error {
if r.i != nil {
return r.i.Close()
}
return nil
}

View File

@ -0,0 +1,201 @@
// 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 firestorm
import (
"reflect"
"regexp"
"testing"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/analysis/tokenizers/regexp_tokenizer"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
var testAnalyzer = &analysis.Analyzer{
Tokenizer: regexp_tokenizer.NewRegexpTokenizer(regexp.MustCompile(`\w+`)),
}
func TestDictionaryReader(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewFieldRow(1, "name"),
NewFieldRow(2, "desc"),
NewFieldRow(3, "prefix"),
}
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
f.warmup(kvwriter)
rows = []index.IndexRow{
// dictionary entries
NewDictionaryRow(1, []byte("test"), 4),
NewDictionaryRow(2, []byte("eat"), 1),
NewDictionaryRow(2, []byte("more"), 1),
NewDictionaryRow(2, []byte("rice"), 1),
NewDictionaryRow(3, []byte("bob"), 1),
NewDictionaryRow(3, []byte("cat"), 1),
NewDictionaryRow(3, []byte("cats"), 1),
NewDictionaryRow(3, []byte("catting"), 1),
NewDictionaryRow(3, []byte("dog"), 1),
NewDictionaryRow(3, []byte("doggy"), 1),
NewDictionaryRow(3, []byte("zoo"), 1),
}
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
// now try it
r, err := f.Reader()
if err != nil {
t.Fatal(err)
}
dict, err := r.FieldDict("name")
if err != nil {
t.Errorf("error creating reader: %v", err)
}
termCount := 0
curr, err := dict.Next()
for err == nil && curr != nil {
termCount++
if curr.Term != "test" {
t.Errorf("expected term to be 'test', got '%s'", curr.Term)
}
curr, err = dict.Next()
}
if termCount != 1 {
t.Errorf("expected 1 term for this field, got %d", termCount)
}
err = dict.Close()
if err != nil {
t.Fatal(err)
}
dict, err = r.FieldDict("desc")
if err != nil {
t.Errorf("error creating reader: %v", err)
}
termCount = 0
terms := make([]string, 0)
curr, err = dict.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict.Next()
}
if termCount != 3 {
t.Errorf("expected 3 term for this field, got %d", termCount)
}
expectedTerms := []string{"eat", "more", "rice"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
err = dict.Close()
if err != nil {
t.Fatal(err)
}
// test start and end range
dict, err = r.FieldDictRange("desc", []byte("fun"), []byte("nice"))
if err != nil {
t.Errorf("error creating reader: %v", err)
}
termCount = 0
terms = make([]string, 0)
curr, err = dict.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict.Next()
}
if termCount != 1 {
t.Errorf("expected 1 term for this field, got %d", termCount)
}
expectedTerms = []string{"more"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
err = dict.Close()
if err != nil {
t.Fatal(err)
}
// test use case for prefix
dict, err = r.FieldDictPrefix("prefix", []byte("cat"))
if err != nil {
t.Errorf("error creating reader: %v", err)
}
termCount = 0
terms = make([]string, 0)
curr, err = dict.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict.Next()
}
if termCount != 3 {
t.Errorf("expected 3 term for this field, got %d", termCount)
}
expectedTerms = []string{"cat", "cats", "catting"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
err = dict.Close()
if err != nil {
t.Fatal(err)
}
err = r.Close()
if err != nil {
t.Fatal(err)
}
}

View File

@ -0,0 +1,122 @@
// 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 firestorm
import (
"bytes"
"github.com/blevesearch/bleve/index/store"
)
type firestormDocIDReader struct {
r *firestormReader
start []byte
end []byte
i store.KVIterator
}
func newFirestormDocIDReader(r *firestormReader, start, end string) (*firestormDocIDReader, error) {
startKey := TermFreqIteratorStart(0, nil)
if start != "" {
startKey = TermFreqPrefixFieldTermDocId(0, nil, []byte(start))
}
logger.Printf("start key '%s' - % x", startKey, startKey)
endKey := TermFreqIteratorStart(0, []byte{ByteSeparator})
if end != "" {
endKey = TermFreqPrefixFieldTermDocId(0, nil, []byte(end))
}
logger.Printf("end key '%s' - % x", endKey, endKey)
i := r.r.Iterator(startKey)
rv := firestormDocIDReader{
r: r,
start: startKey,
end: endKey,
i: i,
}
return &rv, nil
}
func (r *firestormDocIDReader) Next() (string, error) {
if r.i != nil {
key, val, valid := r.i.Current()
for valid && bytes.Compare(key, r.end) <= 0 {
logger.Printf("see key: '%s' - % x", key, key)
tfrsByDocNum := make(map[uint64]*TermFreqRow)
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return "", err
}
tfrsByDocNum[tfr.DocNum()] = tfr
// now we have a possible row, but there may be more rows for the same docid
// find these now
err = r.findNextTfrsWithSameDocId(tfrsByDocNum, tfr.DocID())
if err != nil {
return "", err
}
docNumList := make(DocNumberList, 0, len(tfrsByDocNum))
for dn := range tfrsByDocNum {
docNumList = append(docNumList, dn)
}
logger.Printf("docNumList: %v", docNumList)
highestValidDocNum := r.r.s.Which(tfr.docID, docNumList)
if highestValidDocNum == 0 {
// no valid doc number
key, val, valid = r.i.Current()
continue
}
logger.Printf("highest valid: %d", highestValidDocNum)
tfr = tfrsByDocNum[highestValidDocNum]
return string(tfr.DocID()), nil
}
}
return "", nil
}
// FIXME this is identical to the one in reader_terms.go
func (r *firestormDocIDReader) findNextTfrsWithSameDocId(tfrsByDocNum map[uint64]*TermFreqRow, docID []byte) error {
tfrDocIdPrefix := TermFreqPrefixFieldTermDocId(0, nil, docID)
r.i.Next()
key, val, valid := r.i.Current()
for valid && bytes.HasPrefix(key, tfrDocIdPrefix) {
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return err
}
tfrsByDocNum[tfr.DocNum()] = tfr
r.i.Next()
key, val, valid = r.i.Current()
}
return nil
}
func (r *firestormDocIDReader) Advance(docID string) (string, error) {
if r.i != nil {
tfrDocIdPrefix := TermFreqPrefixFieldTermDocId(0, nil, []byte(docID))
r.i.Seek(tfrDocIdPrefix)
return r.Next()
}
return "", nil
}
func (r *firestormDocIDReader) Close() error {
if r.i != nil {
return r.i.Close()
}
return nil
}

View File

@ -0,0 +1,170 @@
// 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 firestorm
import (
"math/rand"
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestDocIDReaderSomeGarbage(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewFieldRow(1, "desc"),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 2, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 3, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("d"), 4, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("a"), 5, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 6, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("e"), 7, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("g"), 8, 0, 0.0, nil),
// first version of all docs have cat
NewTermFreqRow(1, []byte("cat"), []byte("a"), 1, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("b"), 2, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("c"), 3, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("d"), 4, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("e"), 7, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("g"), 8, 1, 1.0, nil),
// updated version of a still has cat
NewTermFreqRow(1, []byte("cat"), []byte("a"), 5, 1, 1.0, nil),
// updated version of b does NOT have cat
// c has delete in-flight
// d has delete not-yet-garbage-collected
}
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
f.compensator.inFlight = f.compensator.inFlight.Upsert(&InFlightItem{docID: []byte("c"), docNum: 0}, rand.Int())
f.compensator.deletedDocNumbers.Set(4)
// warmup to load field cache and set maxRead correctly
f.warmup(kvwriter)
r, err := f.Reader()
if err != nil {
t.Fatal(err)
}
dr, err := r.DocIDReader("", "")
if err != nil {
t.Fatal(err)
}
expectedDocIds := []string{"a", "b", "e", "g"}
foundDocIds := make([]string, 0)
next, err := dr.Next()
for next != "" && err == nil {
foundDocIds = append(foundDocIds, next)
next, err = dr.Next()
}
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = dr.Close()
if err != nil {
t.Fatal(err)
}
// now test with some doc id ranges
dr, err = r.DocIDReader("b", "f")
if err != nil {
t.Fatal(err)
}
expectedDocIds = []string{"b", "e"}
foundDocIds = make([]string, 0)
next, err = dr.Next()
for next != "" && err == nil {
foundDocIds = append(foundDocIds, next)
next, err = dr.Next()
}
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = dr.Close()
if err != nil {
t.Fatal(err)
}
//now try again and Advance to skip over "e"
dr, err = r.DocIDReader("b", "")
if err != nil {
t.Fatal(err)
}
expectedDocIds = []string{"b", "g"}
foundDocIds = make([]string, 0)
next, err = dr.Next()
if err != nil {
t.Fatal(err)
} else {
foundDocIds = append(foundDocIds, next)
}
next, err = dr.Advance("f")
if err != nil {
t.Fatal(err)
} else {
foundDocIds = append(foundDocIds, next)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = dr.Close()
if err != nil {
t.Fatal(err)
}
err = r.Close()
if err != nil {
t.Fatal(err)
}
}

View File

@ -0,0 +1,159 @@
// 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 firestorm
import (
"bytes"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
)
type firestormTermFieldReader struct {
r *firestormReader
field uint16
term []byte
prefix []byte
count uint64
i store.KVIterator
}
func newFirestormTermFieldReader(r *firestormReader, field uint16, term []byte) (index.TermFieldReader, error) {
dictionaryKey := DictionaryRowKey(field, term)
dictionaryValue, err := r.r.Get(dictionaryKey)
if err != nil {
return nil, err
}
prefix := TermFreqIteratorStart(field, term)
logger.Printf("starting term freq iterator at: '%s' - % x", prefix, prefix)
i := r.r.Iterator(prefix)
rv := firestormTermFieldReader{
r: r,
field: field,
term: term,
prefix: prefix,
i: i,
}
// NOTE: in firestorm the dictionary row is advisory in nature
// it *may* tell us the correct out
// if this record does not exist, it DOES not mean that there isno
// usage, we must scan the term frequencies to be sure
if dictionaryValue != nil {
dictionaryRow, err := NewDictionaryRowKV(dictionaryKey, dictionaryValue)
if err != nil {
return nil, err
}
rv.count = dictionaryRow.Count()
}
return &rv, nil
}
func (r *firestormTermFieldReader) Next() (*index.TermFieldDoc, error) {
if r.i != nil {
key, val, valid := r.i.Current()
for valid && bytes.HasPrefix(key, r.prefix) {
logger.Printf("see key: '%s' - % x", key, key)
tfrsByDocNum := make(map[uint64]*TermFreqRow)
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return nil, err
}
tfrsByDocNum[tfr.DocNum()] = tfr
// now we have a possible row, but there may be more rows for the same docid
// find these now
err = r.findNextTfrsWithSameDocId(tfrsByDocNum, tfr.DocID())
if err != nil {
return nil, err
}
docNumList := make(DocNumberList, 0, len(tfrsByDocNum))
for dn := range tfrsByDocNum {
docNumList = append(docNumList, dn)
}
logger.Printf("docNumList: %v", docNumList)
highestValidDocNum := r.r.s.Which(tfr.docID, docNumList)
if highestValidDocNum == 0 {
// no valid doc number
key, val, valid = r.i.Current()
continue
}
logger.Printf("highest valid: %d", highestValidDocNum)
tfr = tfrsByDocNum[highestValidDocNum]
return &index.TermFieldDoc{
ID: string(tfr.DocID()),
Freq: tfr.Freq(),
Norm: float64(tfr.Norm()),
Vectors: r.termFieldVectorsFromTermVectors(tfr.Vectors()),
}, nil
}
}
return nil, nil
}
func (r *firestormTermFieldReader) findNextTfrsWithSameDocId(tfrsByDocNum map[uint64]*TermFreqRow, docID []byte) error {
tfrDocIdPrefix := TermFreqPrefixFieldTermDocId(r.field, r.term, docID)
r.i.Next()
key, val, valid := r.i.Current()
for valid && bytes.HasPrefix(key, tfrDocIdPrefix) {
tfr, err := NewTermFreqRowKV(key, val)
if err != nil {
return err
}
tfrsByDocNum[tfr.DocNum()] = tfr
r.i.Next()
key, val, valid = r.i.Current()
}
return nil
}
func (r *firestormTermFieldReader) Advance(docID string) (*index.TermFieldDoc, error) {
if r.i != nil {
tfrDocIdPrefix := TermFreqPrefixFieldTermDocId(r.field, r.term, []byte(docID))
r.i.Seek(tfrDocIdPrefix)
return r.Next()
}
return nil, nil
}
func (r *firestormTermFieldReader) Count() uint64 {
return r.count
}
func (r *firestormTermFieldReader) Close() error {
if r.i != nil {
return r.i.Close()
}
return nil
}
func (r *firestormTermFieldReader) termFieldVectorsFromTermVectors(in []*TermVector) []*index.TermFieldVector {
rv := make([]*index.TermFieldVector, len(in))
for i, tv := range in {
fieldName := r.r.f.fieldCache.FieldIndexed(uint16(tv.GetField()))
tfv := index.TermFieldVector{
Field: fieldName,
ArrayPositions: tv.GetArrayPositions(),
Pos: tv.GetPos(),
Start: tv.GetStart(),
End: tv.GetEnd(),
}
rv[i] = &tfv
}
return rv
}

View File

@ -0,0 +1,220 @@
// 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 firestorm
import (
"math/rand"
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestTermReaderNoGarbage(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewFieldRow(1, "desc"),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 2, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 3, 0, 0.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("a"), 1, 3, 2.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("c"), 3, 1, 1.0, nil),
}
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
// warmup to load field cache and set maxRead correctly
f.warmup(kvwriter)
r, err := f.Reader()
if err != nil {
t.Fatal(err)
}
tfr, err := r.TermFieldReader([]byte("cat"), "desc")
if err != nil {
t.Fatal(err)
}
expectedDocIds := []string{"a", "c"}
foundDocIds := make([]string, 0)
next, err := tfr.Next()
for next != nil && err == nil {
foundDocIds = append(foundDocIds, next.ID)
next, err = tfr.Next()
}
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = tfr.Close()
if err != nil {
t.Fatal(err)
}
err = r.Close()
if err != nil {
t.Fatal(err)
}
}
func TestTermReaderSomeGarbage(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewFieldRow(1, "desc"),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 2, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 3, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("d"), 4, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("a"), 5, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 6, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("e"), 7, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("g"), 8, 0, 0.0, nil),
// first version of all docs have cat
NewTermFreqRow(1, []byte("cat"), []byte("a"), 1, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("b"), 2, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("c"), 3, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("d"), 4, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("e"), 7, 1, 1.0, nil),
NewTermFreqRow(1, []byte("cat"), []byte("g"), 8, 1, 1.0, nil),
// updated version of a still has cat
NewTermFreqRow(1, []byte("cat"), []byte("a"), 5, 1, 1.0, nil),
// updated version of b does NOT have cat
// c has delete in-flight
// d has delete not-yet-garbage-collected
}
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
f.compensator.inFlight = f.compensator.inFlight.Upsert(&InFlightItem{docID: []byte("c"), docNum: 0}, rand.Int())
f.compensator.deletedDocNumbers.Set(4)
// warmup to load field cache and set maxRead correctly
f.warmup(kvwriter)
r, err := f.Reader()
if err != nil {
t.Fatal(err)
}
tfr, err := r.TermFieldReader([]byte("cat"), "desc")
if err != nil {
t.Fatal(err)
}
expectedDocIds := []string{"a", "e", "g"}
foundDocIds := make([]string, 0)
next, err := tfr.Next()
for next != nil && err == nil {
foundDocIds = append(foundDocIds, next.ID)
next, err = tfr.Next()
}
if err != nil {
t.Fatal(err)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = tfr.Close()
if err != nil {
t.Fatal(err)
}
// now try again and Advance to skip over "e"
tfr, err = r.TermFieldReader([]byte("cat"), "desc")
if err != nil {
t.Fatal(err)
}
expectedDocIds = []string{"a", "g"}
foundDocIds = make([]string, 0)
next, err = tfr.Next()
if err != nil {
t.Fatal(err)
} else {
foundDocIds = append(foundDocIds, next.ID)
}
next, err = tfr.Advance("f")
if err != nil {
t.Fatal(err)
} else {
foundDocIds = append(foundDocIds, next.ID)
}
if !reflect.DeepEqual(expectedDocIds, foundDocIds) {
t.Errorf("expected: %v, got %v", expectedDocIds, foundDocIds)
}
err = tfr.Close()
if err != nil {
t.Fatal(err)
}
err = r.Close()
if err != nil {
t.Fatal(err)
}
}

31
index/firestorm/stats.go Normal file
View File

@ -0,0 +1,31 @@
// 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 firestorm
import (
"encoding/json"
"sync/atomic"
)
type indexStat struct {
updates, deletes, batches, errors uint64
analysisTime, indexTime uint64
}
func (i *indexStat) MarshalJSON() ([]byte, error) {
m := map[string]interface{}{}
m["updates"] = atomic.LoadUint64(&i.updates)
m["deletes"] = atomic.LoadUint64(&i.deletes)
m["batches"] = atomic.LoadUint64(&i.batches)
m["errors"] = atomic.LoadUint64(&i.errors)
m["analysis_time"] = atomic.LoadUint64(&i.analysisTime)
m["index_time"] = atomic.LoadUint64(&i.indexTime)
return json.Marshal(m)
}

140
index/firestorm/stored.go Normal file
View File

@ -0,0 +1,140 @@
// 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 firestorm
import (
"bytes"
"encoding/binary"
"fmt"
)
var StoredKeyPrefix = []byte{'s'}
type StoredRow struct {
docID []byte
docNum uint64
field uint16
arrayPositions []uint64
value StoredValue
}
func NewStoredRow(docID []byte, docNum uint64, field uint16, arrayPositions []uint64, value []byte) *StoredRow {
rv := StoredRow{
docID: docID,
docNum: docNum,
field: field,
arrayPositions: arrayPositions,
}
if len(arrayPositions) < 1 {
rv.arrayPositions = make([]uint64, 0)
}
rv.value.Raw = value // FIXME review do we need to copy?
return &rv
}
func NewStoredRowKV(key, value []byte) (*StoredRow, error) {
rv := StoredRow{}
buf := bytes.NewBuffer(key)
_, err := buf.ReadByte() // type
if err != nil {
return nil, err
}
rv.docID, err = buf.ReadBytes(ByteSeparator)
if len(rv.docID) < 2 { // 1 for min doc id length, 1 for separator
err = fmt.Errorf("invalid doc length 0")
return nil, err
}
rv.docID = rv.docID[:len(rv.docID)-1] // trim off separator byte
rv.docNum, err = binary.ReadUvarint(buf)
if err != nil {
return nil, err
}
err = binary.Read(buf, binary.LittleEndian, &rv.field)
if err != nil {
return nil, err
}
rv.arrayPositions = make([]uint64, 0)
nextArrayPos, err := binary.ReadUvarint(buf)
for err == nil {
rv.arrayPositions = append(rv.arrayPositions, nextArrayPos)
nextArrayPos, err = binary.ReadUvarint(buf)
}
err = rv.value.Unmarshal(value)
if err != nil {
return nil, err
}
return &rv, nil
}
func (sr *StoredRow) Key() []byte {
docLen := len(sr.docID)
buf := make([]byte, 1+docLen+1+binary.MaxVarintLen64+2+(binary.MaxVarintLen64*len(sr.arrayPositions)))
buf[0] = 's'
copy(buf[1:], sr.docID)
buf[1+docLen] = ByteSeparator
bytesUsed := 1 + docLen + 1
bytesUsed += binary.PutUvarint(buf[bytesUsed:], sr.docNum)
binary.LittleEndian.PutUint16(buf[bytesUsed:], sr.field)
bytesUsed += 2
for _, arrayPosition := range sr.arrayPositions {
varbytes := binary.PutUvarint(buf[bytesUsed:], arrayPosition)
bytesUsed += varbytes
}
return buf[0:bytesUsed]
}
func (sr *StoredRow) Value() []byte {
rv, _ := sr.value.Marshal()
return rv
}
func (sr *StoredRow) DocID() []byte {
return sr.docID
}
func (sr *StoredRow) DocNum() uint64 {
return sr.docNum
}
func (sr *StoredRow) String() string {
return fmt.Sprintf("StoredRow - Field: %d\n", sr.field) +
fmt.Sprintf("DocID '%s' - % x\n", sr.docID, sr.docID) +
fmt.Sprintf("DocNum %d\n", sr.docNum) +
fmt.Sprintf("Array Positions:\n%v", sr.arrayPositions) +
fmt.Sprintf("Value: % x", sr.value.GetRaw())
}
func StoredIteratorStartDocID(docID []byte) []byte {
docLen := len(docID)
buf := make([]byte, 1+docLen+1)
buf[0] = 's'
copy(buf[1:], docID)
buf[1+docLen] = ByteSeparator
return buf
}
func StoredPrefixDocIDNum(docID []byte, docNum uint64) []byte {
docLen := len(docID)
buf := make([]byte, 1+docLen+1+binary.MaxVarintLen64)
buf[0] = 's'
copy(buf[1:], docID)
buf[1+docLen] = ByteSeparator
bytesUsed := 1 + docLen + 1
bytesUsed += binary.PutUvarint(buf[bytesUsed:], docNum)
return buf[0:bytesUsed]
}

View File

@ -0,0 +1,59 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestStoredRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewStoredRow([]byte("doca"), 5, 7, nil, []byte("tcat")),
[]byte{StoredKeyPrefix[0], 'd', 'o', 'c', 'a', ByteSeparator, 5, 7, 0},
[]byte{10, 4, 't', 'c', 'a', 't'},
},
{
NewStoredRow([]byte("doca"), 5, 7, []uint64{1, 1}, []byte("tcat")),
[]byte{StoredKeyPrefix[0], 'd', 'o', 'c', 'a', ByteSeparator, 5, 7, 0, 1, 1},
[]byte{10, 4, 't', 'c', 'a', 't'},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewStoredRowKV(test.outKey, test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected: %#v got: %#v for %d", test.input, row, i)
}
}
}

175
index/firestorm/termfreq.go Normal file
View File

@ -0,0 +1,175 @@
// 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 firestorm
import (
"bytes"
"encoding/binary"
"fmt"
"github.com/golang/protobuf/proto"
)
var TermFreqKeyPrefix = []byte{'t'}
type TermFreqRow struct {
field uint16
term []byte
docID []byte
docNum uint64
value TermFreqValue
}
func NewTermVector(field uint16, pos uint64, start uint64, end uint64, arrayPos []uint64) *TermVector {
rv := TermVector{}
rv.Field = proto.Uint32(uint32(field))
rv.Pos = proto.Uint64(pos)
rv.Start = proto.Uint64(start)
rv.End = proto.Uint64(end)
if len(arrayPos) > 0 {
rv.ArrayPositions = make([]uint64, len(arrayPos))
for i, apv := range arrayPos {
rv.ArrayPositions[i] = apv
}
}
return &rv
}
func NewTermFreqRow(field uint16, term []byte, docID []byte, docNum uint64, freq uint64, norm float32, termVectors []*TermVector) *TermFreqRow {
rv := TermFreqRow{
field: field,
term: term,
docID: docID,
docNum: docNum,
}
rv.value.Freq = proto.Uint64(freq)
rv.value.Norm = proto.Float32(norm)
rv.value.Vectors = termVectors
return &rv
}
func NewTermFreqRowKV(key, value []byte) (*TermFreqRow, error) {
rv := TermFreqRow{}
keyLen := len(key)
if keyLen < 3 {
return nil, fmt.Errorf("invalid term frequency key, no valid field")
}
rv.field = binary.LittleEndian.Uint16(key[1:3])
termStartPos := 3
termEndPos := bytes.IndexByte(key[termStartPos:], ByteSeparator)
if termEndPos < 0 {
return nil, fmt.Errorf("invalid term frequency key, no byte separator terminating term")
}
rv.term = key[termStartPos : termStartPos+termEndPos]
docStartPos := termStartPos + termEndPos + 1
docEndPos := bytes.IndexByte(key[docStartPos:], ByteSeparator)
rv.docID = key[docStartPos : docStartPos+docEndPos]
docNumPos := docStartPos + docEndPos + 1
rv.docNum, _ = binary.Uvarint(key[docNumPos:])
err := rv.value.Unmarshal(value)
if err != nil {
return nil, err
}
return &rv, nil
}
func (tfr *TermFreqRow) Key() []byte {
buf := make([]byte, 3+len(tfr.term)+1+len(tfr.docID)+1+binary.MaxVarintLen64)
buf[0] = 't'
binary.LittleEndian.PutUint16(buf[1:3], tfr.field)
termLen := copy(buf[3:], tfr.term)
buf[3+termLen] = ByteSeparator
docLen := copy(buf[3+termLen+1:], tfr.docID)
buf[3+termLen+1+docLen] = ByteSeparator
used := binary.PutUvarint(buf[3+termLen+1+docLen+1:], tfr.docNum)
return buf[:3+termLen+1+docLen+1+used]
}
func (tfr *TermFreqRow) Value() []byte {
rv, _ := tfr.value.Marshal()
return rv
}
func (tfr *TermFreqRow) String() string {
vectors := ""
for i, v := range tfr.value.GetVectors() {
vectors += fmt.Sprintf("%d - Field: %d Pos: %d Start: %d End: %d ArrayPos: %v - %#v\n", i, v.GetField(), v.GetPos(), v.GetStart(), v.GetEnd(), v.GetArrayPositions(), v.ArrayPositions)
}
return fmt.Sprintf("TermFreqRow - Field: %d\n", tfr.field) +
fmt.Sprintf("Term '%s' - % x\n", tfr.term, tfr.term) +
fmt.Sprintf("DocID '%s' - % x\n", tfr.docID, tfr.docID) +
fmt.Sprintf("DocNum %d\n", tfr.docNum) +
fmt.Sprintf("Freq: %d\n", tfr.value.GetFreq()) +
fmt.Sprintf("Norm: %f\n", tfr.value.GetNorm()) +
fmt.Sprintf("Vectors:\n%s", vectors)
}
func (tfr *TermFreqRow) Field() uint16 {
return tfr.field
}
func (tfr *TermFreqRow) Term() []byte {
return tfr.term
}
func (tfr *TermFreqRow) DocID() []byte {
return tfr.docID
}
func (tfr *TermFreqRow) DocNum() uint64 {
return tfr.docNum
}
func (tfr *TermFreqRow) Norm() float32 {
return tfr.value.GetNorm()
}
func (tfr *TermFreqRow) Freq() uint64 {
return tfr.value.GetFreq()
}
func (tfr *TermFreqRow) Vectors() []*TermVector {
return tfr.value.GetVectors()
}
func (tfr *TermFreqRow) DictionaryRowKey() []byte {
dr := NewDictionaryRow(tfr.field, tfr.term, 0)
return dr.Key()
}
func TermFreqIteratorStart(field uint16, term []byte) []byte {
buf := make([]byte, 3+len(term)+1)
buf[0] = 't'
binary.LittleEndian.PutUint16(buf[1:3], field)
termLen := copy(buf[3:], term)
buf[3+termLen] = ByteSeparator
return buf
}
func TermFreqPrefixFieldTermDocId(field uint16, term []byte, docID []byte) []byte {
buf := make([]byte, 3+len(term)+1+len(docID)+1)
buf[0] = 't'
binary.LittleEndian.PutUint16(buf[1:3], field)
termLen := copy(buf[3:], term)
buf[3+termLen] = ByteSeparator
docLen := copy(buf[3+termLen+1:], docID)
buf[3+termLen+1+docLen] = ByteSeparator
return buf
}

View File

@ -0,0 +1,85 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestTermFreqRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewTermFreqRow(0, []byte("test"), []byte("doca"), 1, 3, 7.0, nil),
[]byte{TermFreqKeyPrefix[0], 0, 0, 't', 'e', 's', 't', ByteSeparator, 'd', 'o', 'c', 'a', ByteSeparator, 1},
[]byte{8, 3, 21, 0, 0, 224, 64},
},
{
NewTermFreqRow(2, []byte("cats"), []byte("docb"), 254, 3, 7.0, nil),
[]byte{TermFreqKeyPrefix[0], 2, 0, 'c', 'a', 't', 's', ByteSeparator, 'd', 'o', 'c', 'b', ByteSeparator, 254, 1},
[]byte{8, 3, 21, 0, 0, 224, 64},
},
{
NewTermFreqRow(2, []byte("cats"), []byte("docb"), 254, 7, 3.0, nil),
[]byte{TermFreqKeyPrefix[0], 2, 0, 'c', 'a', 't', 's', ByteSeparator, 'd', 'o', 'c', 'b', ByteSeparator, 254, 1},
[]byte{8, 7, 21, 0, 0, 64, 64},
},
{
NewTermFreqRow(2, []byte("cats"), []byte("docb"), 254, 7, 3.0, []*TermVector{NewTermVector(2, 1, 0, 5, nil)}),
[]byte{TermFreqKeyPrefix[0], 2, 0, 'c', 'a', 't', 's', ByteSeparator, 'd', 'o', 'c', 'b', ByteSeparator, 254, 1},
[]byte{8, 7, 21, 0, 0, 64, 64, 26, 8, 8, 2, 16, 1, 24, 0, 32, 5},
},
{
NewTermFreqRow(2, []byte("cats"), []byte("docb"), 254, 7, 3.0, []*TermVector{NewTermVector(2, 1, 0, 5, []uint64{0})}),
[]byte{TermFreqKeyPrefix[0], 2, 0, 'c', 'a', 't', 's', ByteSeparator, 'd', 'o', 'c', 'b', ByteSeparator, 254, 1},
[]byte{8, 7, 21, 0, 0, 64, 64, 26, 10, 8, 2, 16, 1, 24, 0, 32, 5, 40, 0},
},
{
NewTermFreqRow(2, []byte("cats"), []byte("docb"), 254, 7, 3.0, []*TermVector{NewTermVector(2, 1, 0, 5, []uint64{0, 1, 2})}),
[]byte{TermFreqKeyPrefix[0], 2, 0, 'c', 'a', 't', 's', ByteSeparator, 'd', 'o', 'c', 'b', ByteSeparator, 254, 1},
[]byte{8, 7, 21, 0, 0, 64, 64, 26, 14, 8, 2, 16, 1, 24, 0, 32, 5, 40, 0, 40, 1, 40, 2},
},
// test empty term, used by _id field
{
NewTermFreqRow(0, []byte{}, []byte("doca"), 1, 0, 0.0, nil),
[]byte{TermFreqKeyPrefix[0], 0, 0, ByteSeparator, 'd', 'o', 'c', 'a', ByteSeparator, 1},
[]byte{8, 0, 21, 0, 0, 0, 0},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewTermFreqRowKV(test.outKey, test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected:\n%vgot:\n%vfor %d", test.input, row, i)
}
}
}

100
index/firestorm/util.go Normal file
View File

@ -0,0 +1,100 @@
// 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 firestorm
import (
"bytes"
"io/ioutil"
"log"
"github.com/blevesearch/bleve/index/store"
)
type KVVisitor func(key, val []byte) (bool, error)
func visitPrefix(reader store.KVReader, prefix []byte, visitor KVVisitor) (err error) {
start := prefix
if start == nil {
start = []byte{0}
}
it := reader.Iterator(start)
defer func() {
if cerr := it.Close(); err == nil && cerr != nil {
err = cerr
}
}()
k, v, valid := it.Current()
for valid && bytes.HasPrefix(k, prefix) {
var cont bool
cont, err = visitor(k, v)
if err != nil {
// visitor encountered an error, stop and return it
return
}
if !cont {
// vistor has requested we stop iteration, return nil
return
}
it.Next()
k, v, valid = it.Current()
}
return
}
func visitRange(reader store.KVReader, start, end []byte, visitor KVVisitor) (err error) {
it := reader.Iterator(start)
defer func() {
if cerr := it.Close(); err == nil && cerr != nil {
err = cerr
}
}()
k, v, valid := it.Current()
for valid && bytes.Compare(k, end) < 0 {
var cont bool
cont, err = visitor(k, v)
if err != nil {
// visitor encountered an error, stop and return it
return
}
if !cont {
// vistor has requested we stop iteration, return nil
return
}
it.Next()
k, v, valid = it.Current()
}
return
}
type DocNumberList []uint64
func (l DocNumberList) Len() int { return len(l) }
func (l DocNumberList) Less(i, j int) bool { return l[i] > l[j] }
func (l DocNumberList) Swap(i, j int) { l[i], l[j] = l[j], l[i] }
// HighestValid returns the highest valid doc number
// from a *SORTED* DocNumberList
// if no doc number in the list is valid, then 0
func (l DocNumberList) HighestValid(maxRead uint64) uint64 {
for _, dn := range l {
if dn <= maxRead {
return dn
}
}
return 0
}
var logger = log.New(ioutil.Discard, "bleve.index.firestorm", 0)
// SetLog sets the logger used for logging
// by default log messages are sent to ioutil.Discard
func SetLog(l *log.Logger) {
logger = l
}

View File

@ -0,0 +1,88 @@
// 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 firestorm
import (
"fmt"
"github.com/golang/protobuf/proto"
"github.com/blevesearch/bleve/index/store"
)
const Version uint64 = 1
var IncompatibleVersion = fmt.Errorf("incompatible version, %d is supported", Version)
var VersionKey = []byte{'v'}
type VersionRow struct {
value VersionValue
}
func NewVersionRow(version uint64) *VersionRow {
rv := VersionRow{}
rv.value.Version = proto.Uint64(version)
return &rv
}
func NewVersionRowV(val []byte) (*VersionRow, error) {
rv := VersionRow{}
err := rv.value.Unmarshal(val)
if err != nil {
return nil, err
}
return &rv, nil
}
func (vr *VersionRow) Key() []byte {
return VersionKey
}
func (vr *VersionRow) Value() []byte {
rv, _ := vr.value.Marshal()
return rv
}
func (vr *VersionRow) Version() uint64 {
return vr.value.GetVersion()
}
func (f *Firestorm) checkVersion(reader store.KVReader) (newIndex bool, err error) {
value, err := reader.Get(VersionKey)
if err != nil {
return
}
if value == nil {
newIndex = true
return
}
var vr *VersionRow
vr, err = NewVersionRowV(value)
if err != nil {
return
}
// assert correct version
if vr.Version() != Version {
err = IncompatibleVersion
return
}
return
}
func (f *Firestorm) storeVersion(writer store.KVWriter) error {
vr := NewVersionRow(Version)
err := writer.Set(vr.Key(), vr.Value())
return err
}

View File

@ -0,0 +1,59 @@
// 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 firestorm
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/index"
)
func TestVersionRows(t *testing.T) {
tests := []struct {
input index.IndexRow
outKey []byte
outVal []byte
}{
{
NewVersionRow(1),
[]byte{VersionKey[0]},
[]byte{8, 1},
},
{
NewVersionRow(1025),
[]byte{VersionKey[0]},
[]byte{8, 129, 8},
},
}
// test going from struct to k/v bytes
for i, test := range tests {
rk := test.input.Key()
if !reflect.DeepEqual(rk, test.outKey) {
t.Errorf("Expected key to be %v got: %v", test.outKey, rk)
}
rv := test.input.Value()
if !reflect.DeepEqual(rv, test.outVal) {
t.Errorf("Expected value to be %v got: %v for %d", test.outVal, rv, i)
}
}
// now test going back from k/v bytes to struct
for i, test := range tests {
row, err := NewVersionRowV(test.outVal)
if err != nil {
t.Errorf("error parsking key/value: %v", err)
}
if !reflect.DeepEqual(row, test.input) {
t.Errorf("Expected: %#v got: %#v for %d", test.input, row, i)
}
}
}

112
index/firestorm/warmup.go Normal file
View File

@ -0,0 +1,112 @@
// 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 firestorm
import (
"bytes"
"fmt"
"sort"
"sync/atomic"
"github.com/blevesearch/bleve/index/store"
)
const IDFieldName = "_id"
func (f *Firestorm) bootstrap(writer store.KVWriter) error {
// record version
err := f.storeVersion(writer)
if err != nil {
return err
}
// define _id field
_, idFieldRow := f.fieldIndexOrNewRow(IDFieldName)
err = writer.Set(idFieldRow.Key(), idFieldRow.Value())
if err != nil {
return err
}
return nil
}
func (f *Firestorm) warmup(writer store.KVWriter) error {
// load all the existing fields
err := f.loadFields(writer)
if err != nil {
return err
}
// walk the term frequency info for _id
// this allows us to find deleted doc numbers
// and seed the doc count
idField, existed := f.fieldCache.FieldNamed(IDFieldName, false)
if !existed {
return fmt.Errorf("_id field missing, cannot proceed")
}
tfkPrefix := TermFreqIteratorStart(idField, nil)
var lastDocId []byte
lastDocNumbers := make(DocNumberList, 1)
err = visitPrefix(writer, tfkPrefix, func(key, val []byte) (bool, error) {
tfk, err := NewTermFreqRowKV(key, val)
if err != nil {
return false, err
}
docID := tfk.DocID()
docNum := tfk.DocNum()
if docNum > f.highDocNumber {
f.highDocNumber = docNum
}
if docNum > f.compensator.maxRead {
f.compensator.maxRead = docNum
}
// check for consecutive records
if bytes.Compare(docID, lastDocId) == 0 {
lastDocNumbers = append(lastDocNumbers, docNum)
} else {
// new doc id
atomic.AddUint64(f.docCount, 1)
// last docID had multiple doc numbers
if len(lastDocNumbers) > 1 {
f.addOldDocNumbers(lastDocNumbers, lastDocId)
// reset size to 1
lastDocNumbers = make(DocNumberList, 1)
}
lastDocNumbers[0] = docNum
lastDocId = docID
}
return true, nil
})
if err != nil {
return err
}
// be sure to finish up check on final row
if len(lastDocNumbers) > 1 {
f.addOldDocNumbers(lastDocNumbers, lastDocId)
}
return nil
}
func (f *Firestorm) addOldDocNumbers(docNumberList DocNumberList, docID []byte) {
sort.Sort(docNumberList)
// high doc number is OK, rest are deleted
for _, dn := range docNumberList[1:] {
// f.deletedDocNumbers.Add(dn, docID)
f.compensator.deletedDocNumbers.Set(uint(dn))
f.garbageCollector.Notify(dn, docID)
}
}

View File

@ -0,0 +1,225 @@
// 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 firestorm
import (
"testing"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
)
func TestBootstrap(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
err = f.bootstrap(kvwriter)
if err != nil {
t.Fatal(err)
}
// assert that version is set
reader, err := kv.Reader()
if err != nil {
t.Fatal(err)
}
val, err := reader.Get(VersionKey)
if err != nil {
t.Fatal(err)
}
verRow, err := NewVersionRowV(val)
if err != nil {
t.Fatal(err)
}
if verRow.Version() != Version {
t.Errorf("expected version %d, got %d", Version, verRow.Version())
}
// assert that field cache has _id
id, existed := f.fieldCache.FieldNamed(IDFieldName, false)
if !existed {
t.Errorf("expect '%s' in field cache", IDFieldName)
}
if id != 0 {
t.Errorf("expected '%s' to have index 0, got %d", IDFieldName, id)
}
// assert that field is recorded in kv store
fRowExpected := NewFieldRow(id, IDFieldName)
fRowKey := fRowExpected.Key()
val, err = reader.Get(fRowKey)
if err != nil {
t.Fatal(err)
}
fRowActual, err := NewFieldRowKV(fRowKey, val)
if err != nil {
t.Fatal(err)
}
if fRowExpected.Name() != fRowActual.Name() {
t.Errorf("expected name '%s' got '%s'", fRowExpected.Name(), fRowActual.Name())
}
// assert that highDocNumber is 0
if f.highDocNumber != 0 {
t.Errorf("expected highDocNumber to be 0, got %d", f.highDocNumber)
}
}
func TestWarmupNoGarbage(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 2, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 3, 0, 0.0, nil),
}
expectedCount := uint64(3)
expectedGarbage := uint64(0)
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
err = f.warmup(kvwriter)
if err != nil {
t.Fatal(err)
}
// assert that doc count is correct
count, err := f.DocCount()
if err != nil {
t.Fatal(err)
}
if count != expectedCount {
t.Errorf("expected doc count %d, got %d", expectedCount, count)
}
// assert that deleted doc numbers size is 0
if f.compensator.GarbageCount() != expectedGarbage {
t.Errorf("expected 0 deleted doc numbers, got %d", f.compensator.GarbageCount())
}
// assert that highDocNumber is 3
if f.highDocNumber != 3 {
t.Errorf("expected highDocNumber to be 3, got %d", f.highDocNumber)
}
}
func TestWarmupSomeGarbage(t *testing.T) {
kv, err := inmem.New()
if err != nil {
t.Fatal(err)
}
aq := index.NewAnalysisQueue(1)
f := NewFirestorm(kv, aq)
err = kv.Open()
if err != nil {
t.Fatal(err)
}
kvwriter, err := f.store.Writer()
if err != nil {
t.Fatal(err)
}
rows := []index.IndexRow{
NewFieldRow(0, IDFieldName),
NewTermFreqRow(0, nil, []byte("a"), 1, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("a"), 2, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("b"), 3, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 4, 0, 0.0, nil),
NewTermFreqRow(0, nil, []byte("c"), 5, 0, 0.0, nil),
}
expectedCount := uint64(3)
expectedGarbage := uint64(2)
for _, row := range rows {
err = kvwriter.Set(row.Key(), row.Value())
if err != nil {
t.Fatal(err)
}
}
err = f.warmup(kvwriter)
if err != nil {
t.Fatal(err)
}
// assert that doc count is correct
count, err := f.DocCount()
if err != nil {
t.Fatal(err)
}
if count != expectedCount {
t.Errorf("expected doc count %d, got %d", expectedCount, count)
}
// assert that deleted doc numbers size is 0
if f.compensator.GarbageCount() != expectedGarbage {
t.Errorf("expected %d deleted doc numbers, got %d", expectedGarbage, f.compensator.GarbageCount())
}
// assert that doc numbers 1 and 4 are on the deleted list
if !f.compensator.deletedDocNumbers.Test(1) {
t.Errorf("expected doc number 1 to be deleted")
}
if !f.compensator.deletedDocNumbers.Test(4) {
t.Errorf("expected doc number 4 to be deleted")
}
// assert that highDocNumber is 5
if f.highDocNumber != 5 {
t.Errorf("expected highDocNumber to be 5, got %d", f.highDocNumber)
}
}

View File

@ -36,6 +36,8 @@ type Index interface {
Reader() (IndexReader, error)
Stats() json.Marshaler
Analyze(d *document.Document) *AnalysisResult
}
type IndexReader interface {

View File

@ -0,0 +1,76 @@
// 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 upside_down
import (
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func (udc *UpsideDownCouch) Analyze(d *document.Document) *index.AnalysisResult {
rv := &index.AnalysisResult{
DocID: d.ID,
Rows: make([]index.IndexRow, 0, 100),
}
// track our back index entries
backIndexTermEntries := make([]*BackIndexTermEntry, 0)
backIndexStoredEntries := make([]*BackIndexStoreEntry, 0)
for _, field := range d.Fields {
fieldIndex, newFieldRow := udc.fieldIndexOrNewRow(field.Name())
if newFieldRow != nil {
rv.Rows = append(rv.Rows, newFieldRow)
}
if field.Options().IsIndexed() {
fieldLength, tokenFreqs := field.Analyze()
// see if any of the composite fields need this
for _, compositeField := range d.CompositeFields {
compositeField.Compose(field.Name(), fieldLength, tokenFreqs)
}
// encode this field
indexRows, indexBackIndexTermEntries := udc.indexField(d.ID, field, fieldIndex, fieldLength, tokenFreqs)
rv.Rows = append(rv.Rows, indexRows...)
backIndexTermEntries = append(backIndexTermEntries, indexBackIndexTermEntries...)
}
if field.Options().IsStored() {
storeRows, indexBackIndexStoreEntries := udc.storeField(d.ID, field, fieldIndex)
rv.Rows = append(rv.Rows, storeRows...)
backIndexStoredEntries = append(backIndexStoredEntries, indexBackIndexStoreEntries...)
}
}
// now index the composite fields
for _, compositeField := range d.CompositeFields {
fieldIndex, newFieldRow := udc.fieldIndexOrNewRow(compositeField.Name())
if newFieldRow != nil {
rv.Rows = append(rv.Rows, newFieldRow)
}
if compositeField.Options().IsIndexed() {
fieldLength, tokenFreqs := compositeField.Analyze()
// encode this field
indexRows, indexBackIndexTermEntries := udc.indexField(d.ID, compositeField, fieldIndex, fieldLength, tokenFreqs)
rv.Rows = append(rv.Rows, indexRows...)
backIndexTermEntries = append(backIndexTermEntries, indexBackIndexTermEntries...)
}
}
// build the back index row
backIndexRow := NewBackIndexRow(d.ID, backIndexTermEntries, backIndexStoredEntries)
rv.Rows = append(rv.Rows, backIndexRow)
return rv
}

View File

@ -1,119 +0,0 @@
// 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 upside_down
import "github.com/blevesearch/bleve/document"
type AnalysisResult struct {
docID string
rows []UpsideDownCouchRow
}
type AnalysisWork struct {
udc *UpsideDownCouch
d *document.Document
rc chan *AnalysisResult
}
type AnalysisQueue struct {
queue chan *AnalysisWork
done chan struct{}
}
func (q *AnalysisQueue) Queue(work *AnalysisWork) {
q.queue <- work
}
func (q *AnalysisQueue) Close() {
close(q.done)
}
func NewAnalysisQueue(numWorkers int) *AnalysisQueue {
rv := AnalysisQueue{
queue: make(chan *AnalysisWork),
done: make(chan struct{}),
}
for i := 0; i < numWorkers; i++ {
go AnalysisWorker(rv)
}
return &rv
}
func AnalysisWorker(q AnalysisQueue) {
// read work off the queue
for {
select {
case <-q.done:
return
case w := <-q.queue:
rv := &AnalysisResult{
docID: w.d.ID,
rows: make([]UpsideDownCouchRow, 0, 100),
}
// track our back index entries
backIndexTermEntries := make([]*BackIndexTermEntry, 0)
backIndexStoredEntries := make([]*BackIndexStoreEntry, 0)
for _, field := range w.d.Fields {
fieldIndex, newFieldRow := w.udc.fieldIndexCache.FieldIndex(field.Name())
if newFieldRow != nil {
rv.rows = append(rv.rows, newFieldRow)
}
if field.Options().IsIndexed() {
fieldLength, tokenFreqs := field.Analyze()
// see if any of the composite fields need this
for _, compositeField := range w.d.CompositeFields {
compositeField.Compose(field.Name(), fieldLength, tokenFreqs)
}
// encode this field
indexRows, indexBackIndexTermEntries := w.udc.indexField(w.d.ID, field, fieldIndex, fieldLength, tokenFreqs)
rv.rows = append(rv.rows, indexRows...)
backIndexTermEntries = append(backIndexTermEntries, indexBackIndexTermEntries...)
}
if field.Options().IsStored() {
storeRows, indexBackIndexStoreEntries := w.udc.storeField(w.d.ID, field, fieldIndex)
rv.rows = append(rv.rows, storeRows...)
backIndexStoredEntries = append(backIndexStoredEntries, indexBackIndexStoreEntries...)
}
}
// now index the composite fields
for _, compositeField := range w.d.CompositeFields {
fieldIndex, newFieldRow := w.udc.fieldIndexCache.FieldIndex(compositeField.Name())
if newFieldRow != nil {
rv.rows = append(rv.rows, newFieldRow)
}
if compositeField.Options().IsIndexed() {
fieldLength, tokenFreqs := compositeField.Analyze()
// encode this field
indexRows, indexBackIndexTermEntries := w.udc.indexField(w.d.ID, compositeField, fieldIndex, fieldLength, tokenFreqs)
rv.rows = append(rv.rows, indexRows...)
backIndexTermEntries = append(backIndexTermEntries, indexBackIndexTermEntries...)
}
}
// build the back index row
backIndexRow := NewBackIndexRow(w.d.ID, backIndexTermEntries, backIndexStoredEntries)
rv.rows = append(rv.rows, backIndexRow)
w.rc <- rv
}
}
}

View File

@ -54,7 +54,7 @@ func CommonBenchmarkIndex(b *testing.B, create KVStoreCreate, destroy KVStoreDes
if err != nil {
b.Fatal(err)
}
analysisQueue := NewAnalysisQueue(analysisWorkers)
analysisQueue := index.NewAnalysisQueue(analysisWorkers)
idx := NewUpsideDownCouch(s, analysisQueue)
err = idx.Open()
@ -97,7 +97,7 @@ func CommonBenchmarkIndexBatch(b *testing.B, create KVStoreCreate, destroy KVSto
if err != nil {
b.Fatal(err)
}
analysisQueue := NewAnalysisQueue(analysisWorkers)
analysisQueue := index.NewAnalysisQueue(analysisWorkers)
idx := NewUpsideDownCouch(s, analysisQueue)
err = idx.Open()

View File

@ -10,11 +10,13 @@
package upside_down
import (
"github.com/blevesearch/bleve/index/store/boltdb"
"os"
"testing"
"time"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/boltdb"
"github.com/blevesearch/bleve/document"
)
@ -28,7 +30,7 @@ func TestDump(t *testing.T) {
s := boltdb.New("test", "bleve")
s.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(s, analysisQueue)
err := idx.Open()
if err != nil {

View File

@ -15,6 +15,7 @@ import (
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/boltdb"
)
@ -28,7 +29,7 @@ func TestIndexFieldDict(t *testing.T) {
s := boltdb.New("test", "bleve")
s.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(s, analysisQueue)
err := idx.Open()
if err != nil {

View File

@ -24,7 +24,7 @@ type IndexReader struct {
}
func (i *IndexReader) TermFieldReader(term []byte, fieldName string) (index.TermFieldReader, error) {
fieldIndex, fieldExists := i.index.fieldIndexCache.FieldExists(fieldName)
fieldIndex, fieldExists := i.index.fieldCache.FieldNamed(fieldName, false)
if fieldExists {
return newUpsideDownCouchTermFieldReader(i, term, uint16(fieldIndex))
}
@ -36,7 +36,7 @@ func (i *IndexReader) FieldDict(fieldName string) (index.FieldDict, error) {
}
func (i *IndexReader) FieldDictRange(fieldName string, startTerm []byte, endTerm []byte) (index.FieldDict, error) {
fieldIndex, fieldExists := i.index.fieldIndexCache.FieldExists(fieldName)
fieldIndex, fieldExists := i.index.fieldCache.FieldNamed(fieldName, false)
if fieldExists {
return newUpsideDownCouchFieldDict(i, uint16(fieldIndex), startTerm, endTerm)
}
@ -87,7 +87,7 @@ func (i *IndexReader) Document(id string) (doc *document.Document, err error) {
return
}
if row != nil {
fieldName := i.index.fieldIndexCache.FieldName(row.field)
fieldName := i.index.fieldCache.FieldIndexed(row.field)
field := decodeFieldType(row.typ, fieldName, row.arrayPositions, row.value)
if field != nil {
doc.AddField(field)
@ -107,7 +107,7 @@ func (i *IndexReader) DocumentFieldTerms(id string) (index.FieldTerms, error) {
}
rv := make(index.FieldTerms, len(back.termEntries))
for _, entry := range back.termEntries {
fieldName := i.index.fieldIndexCache.FieldName(uint16(*entry.Field))
fieldName := i.index.fieldCache.FieldIndexed(uint16(*entry.Field))
terms, ok := rv[fieldName]
if !ok {
terms = make([]string, 0)

View File

@ -29,7 +29,7 @@ func TestIndexReader(t *testing.T) {
s := boltdb.New("test", "bleve")
s.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(s, analysisQueue)
err := idx.Open()
if err != nil {
@ -203,7 +203,7 @@ func TestIndexDocIdReader(t *testing.T) {
s := boltdb.New("test", "bleve")
s.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(s, analysisQueue)
err := idx.Open()
if err != nil {

View File

@ -22,10 +22,13 @@ import (
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/registry"
"github.com/golang/protobuf/proto"
)
const Name = "upside_down"
var VersionKey = []byte{'v'}
var UnsafeBatchUseDetected = fmt.Errorf("bleve.Batch is NOT thread-safe, modification after execution detected")
@ -35,25 +38,25 @@ const Version uint8 = 5
var IncompatibleVersion = fmt.Errorf("incompatible version, %d is supported", Version)
type UpsideDownCouch struct {
version uint8
path string
store store.KVStore
fieldIndexCache *FieldIndexCache
analysisQueue *AnalysisQueue
stats *indexStat
version uint8
path string
store store.KVStore
fieldCache *index.FieldCache
analysisQueue *index.AnalysisQueue
stats *indexStat
m sync.RWMutex
// fields protected by m
docCount uint64
}
func NewUpsideDownCouch(s store.KVStore, analysisQueue *AnalysisQueue) *UpsideDownCouch {
func NewUpsideDownCouch(s store.KVStore, analysisQueue *index.AnalysisQueue) *UpsideDownCouch {
return &UpsideDownCouch{
version: Version,
fieldIndexCache: NewFieldIndexCache(),
store: s,
analysisQueue: analysisQueue,
stats: &indexStat{},
version: Version,
fieldCache: index.NewFieldCache(),
store: s,
analysisQueue: analysisQueue,
stats: &indexStat{},
}
}
@ -90,7 +93,7 @@ func (udc *UpsideDownCouch) loadSchema(kvreader store.KVReader) (err error) {
if err != nil {
return
}
udc.fieldIndexCache.AddExisting(fieldRow.name, fieldRow.index)
udc.fieldCache.AddExisting(fieldRow.name, fieldRow.index)
it.Next()
key, val, valid = it.Current()
@ -263,15 +266,11 @@ func (udc *UpsideDownCouch) Close() error {
func (udc *UpsideDownCouch) Update(doc *document.Document) (err error) {
// do analysis before acquiring write lock
analysisStart := time.Now()
resultChan := make(chan *AnalysisResult)
aw := AnalysisWork{
udc: udc,
d: doc,
rc: resultChan,
}
resultChan := make(chan *index.AnalysisResult)
aw := index.NewAnalysisWork(udc, doc, resultChan)
// put the work on the queue
go func() {
udc.analysisQueue.Queue(&aw)
udc.analysisQueue.Queue(aw)
}()
// wait for the result
@ -306,7 +305,7 @@ func (udc *UpsideDownCouch) Update(doc *document.Document) (err error) {
updateRows := make([]UpsideDownCouchRow, 0)
deleteRows := make([]UpsideDownCouchRow, 0)
addRows, updateRows, deleteRows = udc.mergeOldAndNew(backIndexRow, result.rows, addRows, updateRows, deleteRows)
addRows, updateRows, deleteRows = udc.mergeOldAndNew(backIndexRow, result.Rows, addRows, updateRows, deleteRows)
err = udc.batchRows(kvwriter, addRows, updateRows, deleteRows)
if err == nil && backIndexRow == nil {
@ -323,7 +322,7 @@ func (udc *UpsideDownCouch) Update(doc *document.Document) (err error) {
return
}
func (udc *UpsideDownCouch) mergeOldAndNew(backIndexRow *BackIndexRow, rows, addRows, updateRows, deleteRows []UpsideDownCouchRow) ([]UpsideDownCouchRow, []UpsideDownCouchRow, []UpsideDownCouchRow) {
func (udc *UpsideDownCouch) mergeOldAndNew(backIndexRow *BackIndexRow, rows []index.IndexRow, addRows, updateRows, deleteRows []UpsideDownCouchRow) ([]UpsideDownCouchRow, []UpsideDownCouchRow, []UpsideDownCouchRow) {
existingTermKeys := make(map[string]bool)
for _, key := range backIndexRow.AllTermKeys() {
existingTermKeys[string(key)] = true
@ -377,8 +376,8 @@ func (udc *UpsideDownCouch) mergeOldAndNew(backIndexRow *BackIndexRow, rows, add
return addRows, updateRows, deleteRows
}
func (udc *UpsideDownCouch) storeField(docID string, field document.Field, fieldIndex uint16) ([]UpsideDownCouchRow, []*BackIndexStoreEntry) {
rows := make([]UpsideDownCouchRow, 0, 100)
func (udc *UpsideDownCouch) storeField(docID string, field document.Field, fieldIndex uint16) ([]index.IndexRow, []*BackIndexStoreEntry) {
rows := make([]index.IndexRow, 0, 100)
backIndexStoredEntries := make([]*BackIndexStoreEntry, 0)
fieldType := encodeFieldType(field)
storedRow := NewStoredRow(docID, fieldIndex, field.ArrayPositions(), fieldType, field.Value())
@ -406,9 +405,9 @@ func encodeFieldType(f document.Field) byte {
return fieldType
}
func (udc *UpsideDownCouch) indexField(docID string, field document.Field, fieldIndex uint16, fieldLength int, tokenFreqs analysis.TokenFrequencies) ([]UpsideDownCouchRow, []*BackIndexTermEntry) {
func (udc *UpsideDownCouch) indexField(docID string, field document.Field, fieldIndex uint16, fieldLength int, tokenFreqs analysis.TokenFrequencies) ([]index.IndexRow, []*BackIndexTermEntry) {
rows := make([]UpsideDownCouchRow, 0, 100)
rows := make([]index.IndexRow, 0, 100)
backIndexTermEntries := make([]*BackIndexTermEntry, 0)
fieldNorm := float32(1.0 / math.Sqrt(float64(fieldLength)))
@ -542,16 +541,16 @@ func frequencyFromTokenFreq(tf *analysis.TokenFreq) int {
return len(tf.Locations)
}
func (udc *UpsideDownCouch) termVectorsFromTokenFreq(field uint16, tf *analysis.TokenFreq) ([]*TermVector, []UpsideDownCouchRow) {
func (udc *UpsideDownCouch) termVectorsFromTokenFreq(field uint16, tf *analysis.TokenFreq) ([]*TermVector, []index.IndexRow) {
rv := make([]*TermVector, len(tf.Locations))
newFieldRows := make([]UpsideDownCouchRow, 0)
newFieldRows := make([]index.IndexRow, 0)
for i, l := range tf.Locations {
var newFieldRow *FieldRow
fieldIndex := field
if l.Field != "" {
// lookup correct field
fieldIndex, newFieldRow = udc.fieldIndexCache.FieldIndex(l.Field)
fieldIndex, newFieldRow = udc.fieldIndexOrNewRow(l.Field)
if newFieldRow != nil {
newFieldRows = append(newFieldRows, newFieldRow)
}
@ -573,7 +572,7 @@ func (udc *UpsideDownCouch) termFieldVectorsFromTermVectors(in []*TermVector) []
rv := make([]*index.TermFieldVector, len(in))
for i, tv := range in {
fieldName := udc.fieldIndexCache.FieldName(tv.field)
fieldName := udc.fieldCache.FieldIndexed(tv.field)
tfv := index.TermFieldVector{
Field: fieldName,
ArrayPositions: tv.arrayPositions,
@ -588,7 +587,7 @@ func (udc *UpsideDownCouch) termFieldVectorsFromTermVectors(in []*TermVector) []
func (udc *UpsideDownCouch) Batch(batch *index.Batch) (err error) {
analysisStart := time.Now()
resultChan := make(chan *AnalysisResult)
resultChan := make(chan *index.AnalysisResult)
var numUpdates uint64
for _, doc := range batch.IndexOps {
@ -611,23 +610,19 @@ func (udc *UpsideDownCouch) Batch(batch *index.Batch) (err error) {
detectedUnsafeMutex.Unlock()
return
}
aw := AnalysisWork{
udc: udc,
d: doc,
rc: resultChan,
}
aw := index.NewAnalysisWork(udc, doc, resultChan)
// put the work on the queue
udc.analysisQueue.Queue(&aw)
udc.analysisQueue.Queue(aw)
}
}
}()
newRowsMap := make(map[string][]UpsideDownCouchRow)
newRowsMap := make(map[string][]index.IndexRow)
// wait for the result
var itemsDeQueued uint64
for itemsDeQueued < numUpdates {
result := <-resultChan
newRowsMap[result.docID] = result.rows
newRowsMap[result.DocID] = result.Rows
itemsDeQueued++
}
close(resultChan)
@ -760,3 +755,19 @@ func (udc *UpsideDownCouch) Reader() (index.IndexReader, error) {
func (udc *UpsideDownCouch) Stats() json.Marshaler {
return udc.stats
}
func (udc *UpsideDownCouch) fieldIndexOrNewRow(name string) (uint16, *FieldRow) {
index, existed := udc.fieldCache.FieldNamed(name, true)
if !existed {
return index, NewFieldRow(uint16(index), name)
}
return index, nil
}
func IndexTypeConstructor(store store.KVStore, analysisQueue *index.AnalysisQueue) (index.Index, error) {
return NewUpsideDownCouch(store, analysisQueue), nil
}
func init() {
registry.RegisterIndexType(Name, IndexTypeConstructor)
}

View File

@ -37,7 +37,7 @@ func TestIndexOpenReopen(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -94,7 +94,7 @@ func TestIndexInsert(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -153,7 +153,7 @@ func TestIndexInsertThenDelete(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -248,7 +248,7 @@ func TestIndexInsertThenUpdate(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -315,7 +315,7 @@ func TestIndexInsertMultiple(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -397,7 +397,7 @@ func TestIndexInsertWithStore(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -483,7 +483,7 @@ func TestIndexInternalCRUD(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -576,7 +576,7 @@ func TestIndexBatch(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -673,7 +673,7 @@ func TestIndexInsertUpdateDeleteWithMultipleTypesStored(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -865,7 +865,7 @@ func TestIndexInsertFields(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -924,7 +924,7 @@ func TestIndexUpdateComposites(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -1020,7 +1020,7 @@ func TestIndexFieldsMisc(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -1041,15 +1041,15 @@ func TestIndexFieldsMisc(t *testing.T) {
t.Errorf("Error updating index: %v", err)
}
fieldName1 := idx.fieldIndexCache.FieldName(1)
fieldName1 := idx.fieldCache.FieldIndexed(0)
if fieldName1 != "name" {
t.Errorf("expected field named 'name', got '%s'", fieldName1)
}
fieldName2 := idx.fieldIndexCache.FieldName(2)
fieldName2 := idx.fieldCache.FieldIndexed(1)
if fieldName2 != "title" {
t.Errorf("expected field named 'title', got '%s'", fieldName2)
}
fieldName3 := idx.fieldIndexCache.FieldName(3)
fieldName3 := idx.fieldCache.FieldIndexed(2)
if fieldName3 != "" {
t.Errorf("expected field named '', got '%s'", fieldName3)
}
@ -1066,7 +1066,7 @@ func TestIndexTermReaderCompositeFields(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {
@ -1126,7 +1126,7 @@ func TestIndexDocumentFieldTerms(t *testing.T) {
store := boltdb.New("test", "bleve")
store.SetMergeOperator(&mergeOperator)
analysisQueue := NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
idx := NewUpsideDownCouch(store, analysisQueue)
err := idx.Open()
if err != nil {

View File

@ -20,7 +20,6 @@ import (
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/index/upside_down"
"github.com/blevesearch/bleve/registry"
"github.com/blevesearch/bleve/search"
"github.com/blevesearch/bleve/search/collectors"
@ -46,11 +45,11 @@ func indexStorePath(path string) string {
return path + string(os.PathSeparator) + storePath
}
func newMemIndex(mapping *IndexMapping) (*indexImpl, error) {
func newMemIndex(indexType string, mapping *IndexMapping) (*indexImpl, error) {
rv := indexImpl{
path: "",
m: mapping,
meta: newIndexMeta("mem", nil),
meta: newIndexMeta(indexType, "mem", nil),
stats: &IndexStat{},
}
@ -66,7 +65,15 @@ func newMemIndex(mapping *IndexMapping) (*indexImpl, error) {
}
// open the index
rv.i = upside_down.NewUpsideDownCouch(rv.s, Config.analysisQueue)
indexTypeConstructor := registry.IndexTypeConstructorByName(rv.meta.IndexType)
if indexTypeConstructor == nil {
return nil, ErrorUnknownIndexType
}
rv.i, err = indexTypeConstructor(rv.s, Config.analysisQueue)
if err != nil {
return nil, err
}
err = rv.i.Open()
if err != nil {
return nil, err
@ -90,7 +97,7 @@ func newMemIndex(mapping *IndexMapping) (*indexImpl, error) {
return &rv, nil
}
func newIndexUsing(path string, mapping *IndexMapping, kvstore string, kvconfig map[string]interface{}) (*indexImpl, error) {
func newIndexUsing(path string, mapping *IndexMapping, indexType string, kvstore string, kvconfig map[string]interface{}) (*indexImpl, error) {
// first validate the mapping
err := mapping.validate()
if err != nil {
@ -98,7 +105,7 @@ func newIndexUsing(path string, mapping *IndexMapping, kvstore string, kvconfig
}
if path == "" {
return newMemIndex(mapping)
return newMemIndex(indexType, mapping)
}
if kvconfig == nil {
@ -108,7 +115,7 @@ func newIndexUsing(path string, mapping *IndexMapping, kvstore string, kvconfig
rv := indexImpl{
path: path,
m: mapping,
meta: newIndexMeta(kvstore, kvconfig),
meta: newIndexMeta(indexType, kvstore, kvconfig),
stats: &IndexStat{},
}
storeConstructor := registry.KVStoreConstructorByName(rv.meta.Storage)
@ -131,7 +138,15 @@ func newIndexUsing(path string, mapping *IndexMapping, kvstore string, kvconfig
}
// open the index
rv.i = upside_down.NewUpsideDownCouch(rv.s, Config.analysisQueue)
indexTypeConstructor := registry.IndexTypeConstructorByName(rv.meta.IndexType)
if indexTypeConstructor == nil {
return nil, ErrorUnknownIndexType
}
rv.i, err = indexTypeConstructor(rv.s, Config.analysisQueue)
if err != nil {
return nil, err
}
err = rv.i.Open()
if err != nil {
return nil, err
@ -191,7 +206,15 @@ func openIndexUsing(path string, runtimeConfig map[string]interface{}) (rv *inde
}
// open the index
rv.i = upside_down.NewUpsideDownCouch(rv.s, Config.analysisQueue)
indexTypeConstructor := registry.IndexTypeConstructorByName(rv.meta.IndexType)
if indexTypeConstructor == nil {
return nil, ErrorUnknownIndexType
}
rv.i, err = indexTypeConstructor(rv.s, Config.analysisQueue)
if err != nil {
return nil, err
}
err = rv.i.Open()
if err != nil {
return nil, err

View File

@ -18,14 +18,16 @@ import (
const metaFilename = "index_meta.json"
type indexMeta struct {
Storage string `json:"storage"`
Config map[string]interface{} `json:"config,omitempty"`
Storage string `json:"storage"`
IndexType string `json:"index_type"`
Config map[string]interface{} `json:"config,omitempty"`
}
func newIndexMeta(storage string, config map[string]interface{}) *indexMeta {
func newIndexMeta(indexType string, storage string, config map[string]interface{}) *indexMeta {
return &indexMeta{
Storage: storage,
Config: config,
IndexType: indexType,
Storage: storage,
Config: config,
}
}

47
registry/index_type.go Normal file
View File

@ -0,0 +1,47 @@
// 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 registry
import (
"fmt"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store"
)
func RegisterIndexType(name string, constructor IndexTypeConstructor) {
_, exists := index_types[name]
if exists {
panic(fmt.Errorf("attempted to register duplicate index encoding named '%s'", name))
}
index_types[name] = constructor
}
type IndexTypeConstructor func(store.KVStore, *index.AnalysisQueue) (index.Index, error)
type IndexTypeRegistry map[string]IndexTypeConstructor
func IndexTypeConstructorByName(name string) IndexTypeConstructor {
return index_types[name]
}
func IndexTypesAndInstances() ([]string, []string) {
emptyConfig := map[string]interface{}{}
types := make([]string, 0)
instances := make([]string, 0)
for name, cons := range stores {
_, err := cons(emptyConfig)
if err == nil {
instances = append(instances, name)
} else {
types = append(types, name)
}
}
return types, instances
}

View File

@ -17,6 +17,7 @@ import (
)
var stores = make(KVStoreRegistry, 0)
var index_types = make(IndexTypeRegistry, 0)
var byteArrayConverters = make(ByteArrayConverterRegistry, 0)

View File

@ -25,7 +25,7 @@ var twoDocIndex index.Index //= upside_down.NewUpsideDownCouch(inmem.MustOpen())
func init() {
inMemStore, _ := inmem.New()
analysisQueue := upside_down.NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
twoDocIndex = upside_down.NewUpsideDownCouch(inMemStore, analysisQueue)
err := twoDocIndex.Open()
if err != nil {

View File

@ -14,6 +14,7 @@ import (
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/store/inmem"
"github.com/blevesearch/bleve/index/upside_down"
)
@ -26,7 +27,7 @@ func TestTermSearcher(t *testing.T) {
var queryExplain = true
inMemStore, _ := inmem.New()
analysisQueue := upside_down.NewAnalysisQueue(1)
analysisQueue := index.NewAnalysisQueue(1)
i := upside_down.NewUpsideDownCouch(inMemStore, analysisQueue)
err := i.Open()
if err != nil {

View File

@ -20,7 +20,8 @@ import (
var indexPath = flag.String("index", "", "index path")
var mappingFile = flag.String("mapping", "", "mapping file")
var storeType = flag.String("store", "", "store type")
var storeType = flag.String("store", bleve.Config.DefaultKVStore, "store type")
var indexType = flag.String("indexType", bleve.Config.DefaultIndexType, "index type")
func main() {
@ -44,13 +45,7 @@ func main() {
}
// create the index
var index bleve.Index
var err error
if *storeType != "" {
index, err = bleve.NewUsing(*indexPath, mapping, *storeType, nil)
} else {
index, err = bleve.New(*indexPath, mapping)
}
index, err := bleve.NewUsing(*indexPath, mapping, *indexType, *storeType, nil)
if err != nil {
log.Fatal(err)
}