Adding upstream version 2.5.1.
Signed-off-by: Daniel Baumann <daniel@debian.org>
This commit is contained in:
parent
c71cb8b61d
commit
982828099e
783 changed files with 150650 additions and 0 deletions
85
index/upsidedown/store/boltdb/iterator.go
Normal file
85
index/upsidedown/store/boltdb/iterator.go
Normal file
|
@ -0,0 +1,85 @@
|
|||
// 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 boltdb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
|
||||
bolt "go.etcd.io/bbolt"
|
||||
)
|
||||
|
||||
type Iterator struct {
|
||||
store *Store
|
||||
tx *bolt.Tx
|
||||
cursor *bolt.Cursor
|
||||
prefix []byte
|
||||
start []byte
|
||||
end []byte
|
||||
valid bool
|
||||
key []byte
|
||||
val []byte
|
||||
}
|
||||
|
||||
func (i *Iterator) updateValid() {
|
||||
i.valid = (i.key != nil)
|
||||
if i.valid {
|
||||
if i.prefix != nil {
|
||||
i.valid = bytes.HasPrefix(i.key, i.prefix)
|
||||
} else if i.end != nil {
|
||||
i.valid = bytes.Compare(i.key, i.end) < 0
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (i *Iterator) Seek(k []byte) {
|
||||
if i.start != nil && bytes.Compare(k, i.start) < 0 {
|
||||
k = i.start
|
||||
}
|
||||
if i.prefix != nil && !bytes.HasPrefix(k, i.prefix) {
|
||||
if bytes.Compare(k, i.prefix) < 0 {
|
||||
k = i.prefix
|
||||
} else {
|
||||
i.valid = false
|
||||
return
|
||||
}
|
||||
}
|
||||
i.key, i.val = i.cursor.Seek(k)
|
||||
i.updateValid()
|
||||
}
|
||||
|
||||
func (i *Iterator) Next() {
|
||||
i.key, i.val = i.cursor.Next()
|
||||
i.updateValid()
|
||||
}
|
||||
|
||||
func (i *Iterator) Current() ([]byte, []byte, bool) {
|
||||
return i.key, i.val, i.valid
|
||||
}
|
||||
|
||||
func (i *Iterator) Key() []byte {
|
||||
return i.key
|
||||
}
|
||||
|
||||
func (i *Iterator) Value() []byte {
|
||||
return i.val
|
||||
}
|
||||
|
||||
func (i *Iterator) Valid() bool {
|
||||
return i.valid
|
||||
}
|
||||
|
||||
func (i *Iterator) Close() error {
|
||||
return nil
|
||||
}
|
73
index/upsidedown/store/boltdb/reader.go
Normal file
73
index/upsidedown/store/boltdb/reader.go
Normal file
|
@ -0,0 +1,73 @@
|
|||
// 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 boltdb
|
||||
|
||||
import (
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
)
|
||||
|
||||
type Reader struct {
|
||||
store *Store
|
||||
tx *bolt.Tx
|
||||
bucket *bolt.Bucket
|
||||
}
|
||||
|
||||
func (r *Reader) Get(key []byte) ([]byte, error) {
|
||||
var rv []byte
|
||||
v := r.bucket.Get(key)
|
||||
if v != nil {
|
||||
rv = make([]byte, len(v))
|
||||
copy(rv, v)
|
||||
}
|
||||
return rv, nil
|
||||
}
|
||||
|
||||
func (r *Reader) MultiGet(keys [][]byte) ([][]byte, error) {
|
||||
return store.MultiGet(r, keys)
|
||||
}
|
||||
|
||||
func (r *Reader) PrefixIterator(prefix []byte) store.KVIterator {
|
||||
cursor := r.bucket.Cursor()
|
||||
|
||||
rv := &Iterator{
|
||||
store: r.store,
|
||||
tx: r.tx,
|
||||
cursor: cursor,
|
||||
prefix: prefix,
|
||||
}
|
||||
|
||||
rv.Seek(prefix)
|
||||
return rv
|
||||
}
|
||||
|
||||
func (r *Reader) RangeIterator(start, end []byte) store.KVIterator {
|
||||
cursor := r.bucket.Cursor()
|
||||
|
||||
rv := &Iterator{
|
||||
store: r.store,
|
||||
tx: r.tx,
|
||||
cursor: cursor,
|
||||
start: start,
|
||||
end: end,
|
||||
}
|
||||
|
||||
rv.Seek(start)
|
||||
return rv
|
||||
}
|
||||
|
||||
func (r *Reader) Close() error {
|
||||
return r.tx.Rollback()
|
||||
}
|
28
index/upsidedown/store/boltdb/stats.go
Normal file
28
index/upsidedown/store/boltdb/stats.go
Normal file
|
@ -0,0 +1,28 @@
|
|||
// 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 boltdb
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
)
|
||||
|
||||
type stats struct {
|
||||
s *Store
|
||||
}
|
||||
|
||||
func (s *stats) MarshalJSON() ([]byte, error) {
|
||||
bs := s.s.db.Stats()
|
||||
return util.MarshalJSON(bs)
|
||||
}
|
184
index/upsidedown/store/boltdb/store.go
Normal file
184
index/upsidedown/store/boltdb/store.go
Normal file
|
@ -0,0 +1,184 @@
|
|||
// 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 boltdb implements a store.KVStore on top of BoltDB. It supports the
|
||||
// following options:
|
||||
//
|
||||
// "bucket" (string): the name of BoltDB bucket to use, defaults to "bleve".
|
||||
//
|
||||
// "nosync" (bool): if true, set boltdb.DB.NoSync to true. It speeds up index
|
||||
// operations in exchange of losing integrity guarantees if indexation aborts
|
||||
// without closing the index. Use it when rebuilding indexes from zero.
|
||||
package boltdb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
)
|
||||
|
||||
const (
|
||||
Name = "boltdb"
|
||||
defaultCompactBatchSize = 100
|
||||
)
|
||||
|
||||
type Store struct {
|
||||
path string
|
||||
bucket string
|
||||
db *bolt.DB
|
||||
noSync bool
|
||||
fillPercent float64
|
||||
mo store.MergeOperator
|
||||
}
|
||||
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (store.KVStore, error) {
|
||||
path, ok := config["path"].(string)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("must specify path")
|
||||
}
|
||||
if path == "" {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
bucket, ok := config["bucket"].(string)
|
||||
if !ok {
|
||||
bucket = "bleve"
|
||||
}
|
||||
|
||||
noSync, _ := config["nosync"].(bool)
|
||||
|
||||
fillPercent, ok := config["fillPercent"].(float64)
|
||||
if !ok {
|
||||
fillPercent = bolt.DefaultFillPercent
|
||||
}
|
||||
|
||||
bo := &bolt.Options{}
|
||||
ro, ok := config["read_only"].(bool)
|
||||
if ok {
|
||||
bo.ReadOnly = ro
|
||||
}
|
||||
|
||||
if initialMmapSize, ok := config["initialMmapSize"].(int); ok {
|
||||
bo.InitialMmapSize = initialMmapSize
|
||||
} else if initialMmapSize, ok := config["initialMmapSize"].(float64); ok {
|
||||
bo.InitialMmapSize = int(initialMmapSize)
|
||||
}
|
||||
|
||||
db, err := bolt.Open(path, 0600, bo)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
db.NoSync = noSync
|
||||
|
||||
if !bo.ReadOnly {
|
||||
err = db.Update(func(tx *bolt.Tx) error {
|
||||
_, err := tx.CreateBucketIfNotExists([]byte(bucket))
|
||||
|
||||
return err
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
rv := Store{
|
||||
path: path,
|
||||
bucket: bucket,
|
||||
db: db,
|
||||
mo: mo,
|
||||
noSync: noSync,
|
||||
fillPercent: fillPercent,
|
||||
}
|
||||
return &rv, nil
|
||||
}
|
||||
|
||||
func (bs *Store) Close() error {
|
||||
return bs.db.Close()
|
||||
}
|
||||
|
||||
func (bs *Store) Reader() (store.KVReader, error) {
|
||||
tx, err := bs.db.Begin(false)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Reader{
|
||||
store: bs,
|
||||
tx: tx,
|
||||
bucket: tx.Bucket([]byte(bs.bucket)),
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (bs *Store) Writer() (store.KVWriter, error) {
|
||||
return &Writer{
|
||||
store: bs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (bs *Store) Stats() json.Marshaler {
|
||||
return &stats{
|
||||
s: bs,
|
||||
}
|
||||
}
|
||||
|
||||
// CompactWithBatchSize removes DictionaryTerm entries with a count of zero (in batchSize batches)
|
||||
// Removing entries is a workaround for github issue #374.
|
||||
func (bs *Store) CompactWithBatchSize(batchSize int) error {
|
||||
for {
|
||||
cnt := 0
|
||||
err := bs.db.Batch(func(tx *bolt.Tx) error {
|
||||
c := tx.Bucket([]byte(bs.bucket)).Cursor()
|
||||
prefix := []byte("d")
|
||||
|
||||
for k, v := c.Seek(prefix); bytes.HasPrefix(k, prefix); k, v = c.Next() {
|
||||
if bytes.Equal(v, []byte{0}) {
|
||||
cnt++
|
||||
if err := c.Delete(); err != nil {
|
||||
return err
|
||||
}
|
||||
if cnt == batchSize {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
if cnt == 0 {
|
||||
break
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// Compact calls CompactWithBatchSize with a default batch size of 100. This is a workaround
|
||||
// for github issue #374.
|
||||
func (bs *Store) Compact() error {
|
||||
return bs.CompactWithBatchSize(defaultCompactBatchSize)
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
148
index/upsidedown/store/boltdb/store_test.go
Normal file
148
index/upsidedown/store/boltdb/store_test.go
Normal file
|
@ -0,0 +1,148 @@
|
|||
// 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.
|
||||
|
||||
//go:build !darwin || !arm64
|
||||
|
||||
package boltdb
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
bolt "go.etcd.io/bbolt"
|
||||
)
|
||||
|
||||
func open(t *testing.T, mo store.MergeOperator) store.KVStore {
|
||||
rv, err := New(mo, map[string]interface{}{"path": "test"})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return rv
|
||||
}
|
||||
|
||||
func cleanup(t *testing.T, s store.KVStore) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
err = os.RemoveAll("test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestBoltDBKVCrud(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBReaderIsolation(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBReaderOwnsGetBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBWriterOwnsBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBPrefixIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBPrefixIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBRangeIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBRangeIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBMerge(t *testing.T) {
|
||||
s := open(t, &test.TestMergeCounter{})
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
||||
|
||||
func TestBoltDBConfig(t *testing.T) {
|
||||
var tests = []struct {
|
||||
in map[string]interface{}
|
||||
path string
|
||||
bucket string
|
||||
noSync bool
|
||||
fillPercent float64
|
||||
}{
|
||||
{
|
||||
map[string]interface{}{"path": "test", "bucket": "mybucket", "nosync": true, "fillPercent": 0.75},
|
||||
"test",
|
||||
"mybucket",
|
||||
true,
|
||||
0.75,
|
||||
},
|
||||
{
|
||||
map[string]interface{}{"path": "test"},
|
||||
"test",
|
||||
"bleve",
|
||||
false,
|
||||
bolt.DefaultFillPercent,
|
||||
},
|
||||
}
|
||||
|
||||
for _, test := range tests {
|
||||
kv, err := New(nil, test.in)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
bs, ok := kv.(*Store)
|
||||
if !ok {
|
||||
t.Fatal("failed type assertion to *boltdb.Store")
|
||||
}
|
||||
if bs.path != test.path {
|
||||
t.Fatalf("path: expected %q, got %q", test.path, bs.path)
|
||||
}
|
||||
if bs.bucket != test.bucket {
|
||||
t.Fatalf("bucket: expected %q, got %q", test.bucket, bs.bucket)
|
||||
}
|
||||
if bs.noSync != test.noSync {
|
||||
t.Fatalf("noSync: expected %t, got %t", test.noSync, bs.noSync)
|
||||
}
|
||||
if bs.fillPercent != test.fillPercent {
|
||||
t.Fatalf("fillPercent: expected %f, got %f", test.fillPercent, bs.fillPercent)
|
||||
}
|
||||
cleanup(t, kv)
|
||||
}
|
||||
}
|
95
index/upsidedown/store/boltdb/writer.go
Normal file
95
index/upsidedown/store/boltdb/writer.go
Normal file
|
@ -0,0 +1,95 @@
|
|||
// 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 boltdb
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
store *Store
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatch() store.KVBatch {
|
||||
return store.NewEmulatedBatch(w.store.mo)
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatchEx(options store.KVBatchOptions) ([]byte, store.KVBatch, error) {
|
||||
return make([]byte, options.TotalBytes), w.NewBatch(), nil
|
||||
}
|
||||
|
||||
func (w *Writer) ExecuteBatch(batch store.KVBatch) (err error) {
|
||||
|
||||
emulatedBatch, ok := batch.(*store.EmulatedBatch)
|
||||
if !ok {
|
||||
return fmt.Errorf("wrong type of batch")
|
||||
}
|
||||
|
||||
tx, err := w.store.db.Begin(true)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
// defer function to ensure that once started,
|
||||
// we either Commit tx or Rollback
|
||||
defer func() {
|
||||
// if nothing went wrong, commit
|
||||
if err == nil {
|
||||
// careful to catch error here too
|
||||
err = tx.Commit()
|
||||
} else {
|
||||
// caller should see error that caused abort,
|
||||
// not success or failure of Rollback itself
|
||||
_ = tx.Rollback()
|
||||
}
|
||||
}()
|
||||
|
||||
bucket := tx.Bucket([]byte(w.store.bucket))
|
||||
bucket.FillPercent = w.store.fillPercent
|
||||
|
||||
for k, mergeOps := range emulatedBatch.Merger.Merges {
|
||||
kb := []byte(k)
|
||||
existingVal := bucket.Get(kb)
|
||||
mergedVal, fullMergeOk := w.store.mo.FullMerge(kb, existingVal, mergeOps)
|
||||
if !fullMergeOk {
|
||||
err = fmt.Errorf("merge operator returned failure")
|
||||
return
|
||||
}
|
||||
err = bucket.Put(kb, mergedVal)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
for _, op := range emulatedBatch.Ops {
|
||||
if op.V != nil {
|
||||
err = bucket.Put(op.K, op.V)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
} else {
|
||||
err = bucket.Delete(op.K)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
func (w *Writer) Close() error {
|
||||
return nil
|
||||
}
|
50
index/upsidedown/store/goleveldb/batch.go
Normal file
50
index/upsidedown/store/goleveldb/batch.go
Normal file
|
@ -0,0 +1,50 @@
|
|||
// 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 goleveldb
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/goleveldb/leveldb"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Batch struct {
|
||||
store *Store
|
||||
merge *store.EmulatedMerge
|
||||
batch *leveldb.Batch
|
||||
}
|
||||
|
||||
func (b *Batch) Set(key, val []byte) {
|
||||
b.batch.Put(key, val)
|
||||
}
|
||||
|
||||
func (b *Batch) Delete(key []byte) {
|
||||
b.batch.Delete(key)
|
||||
}
|
||||
|
||||
func (b *Batch) Merge(key, val []byte) {
|
||||
b.merge.Merge(key, val)
|
||||
}
|
||||
|
||||
func (b *Batch) Reset() {
|
||||
b.batch.Reset()
|
||||
b.merge = store.NewEmulatedMerge(b.store.mo)
|
||||
}
|
||||
|
||||
func (b *Batch) Close() error {
|
||||
b.batch.Reset()
|
||||
b.batch = nil
|
||||
b.merge = nil
|
||||
return nil
|
||||
}
|
66
index/upsidedown/store/goleveldb/config.go
Normal file
66
index/upsidedown/store/goleveldb/config.go
Normal file
|
@ -0,0 +1,66 @@
|
|||
// 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 goleveldb
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/goleveldb/leveldb/filter"
|
||||
"github.com/blevesearch/goleveldb/leveldb/opt"
|
||||
)
|
||||
|
||||
func applyConfig(o *opt.Options, config map[string]interface{}) (*opt.Options, error) {
|
||||
|
||||
ro, ok := config["read_only"].(bool)
|
||||
if ok {
|
||||
o.ReadOnly = ro
|
||||
}
|
||||
|
||||
cim, ok := config["create_if_missing"].(bool)
|
||||
if ok {
|
||||
o.ErrorIfMissing = !cim
|
||||
}
|
||||
|
||||
eie, ok := config["error_if_exists"].(bool)
|
||||
if ok {
|
||||
o.ErrorIfExist = eie
|
||||
}
|
||||
|
||||
wbs, ok := config["write_buffer_size"].(float64)
|
||||
if ok {
|
||||
o.WriteBuffer = int(wbs)
|
||||
}
|
||||
|
||||
bs, ok := config["block_size"].(float64)
|
||||
if ok {
|
||||
o.BlockSize = int(bs)
|
||||
}
|
||||
|
||||
bri, ok := config["block_restart_interval"].(float64)
|
||||
if ok {
|
||||
o.BlockRestartInterval = int(bri)
|
||||
}
|
||||
|
||||
lcc, ok := config["lru_cache_capacity"].(float64)
|
||||
if ok {
|
||||
o.BlockCacheCapacity = int(lcc)
|
||||
}
|
||||
|
||||
bfbpk, ok := config["bloom_filter_bits_per_key"].(float64)
|
||||
if ok {
|
||||
bf := filter.NewBloomFilter(int(bfbpk))
|
||||
o.Filter = bf
|
||||
}
|
||||
|
||||
return o, nil
|
||||
}
|
54
index/upsidedown/store/goleveldb/iterator.go
Normal file
54
index/upsidedown/store/goleveldb/iterator.go
Normal file
|
@ -0,0 +1,54 @@
|
|||
// 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 goleveldb
|
||||
|
||||
import "github.com/blevesearch/goleveldb/leveldb/iterator"
|
||||
|
||||
type Iterator struct {
|
||||
store *Store
|
||||
iterator iterator.Iterator
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Seek(key []byte) {
|
||||
ldi.iterator.Seek(key)
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Next() {
|
||||
ldi.iterator.Next()
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Current() ([]byte, []byte, bool) {
|
||||
if ldi.Valid() {
|
||||
return ldi.Key(), ldi.Value(), true
|
||||
}
|
||||
return nil, nil, false
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Key() []byte {
|
||||
return ldi.iterator.Key()
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Value() []byte {
|
||||
return ldi.iterator.Value()
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Valid() bool {
|
||||
return ldi.iterator.Valid()
|
||||
}
|
||||
|
||||
func (ldi *Iterator) Close() error {
|
||||
ldi.iterator.Release()
|
||||
return nil
|
||||
}
|
68
index/upsidedown/store/goleveldb/reader.go
Normal file
68
index/upsidedown/store/goleveldb/reader.go
Normal file
|
@ -0,0 +1,68 @@
|
|||
// Copyright (c) 2014 Couchbase, Inc.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package goleveldb
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/goleveldb/leveldb"
|
||||
"github.com/blevesearch/goleveldb/leveldb/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Reader struct {
|
||||
store *Store
|
||||
snapshot *leveldb.Snapshot
|
||||
}
|
||||
|
||||
func (r *Reader) Get(key []byte) ([]byte, error) {
|
||||
b, err := r.snapshot.Get(key, r.store.defaultReadOptions)
|
||||
if err == leveldb.ErrNotFound {
|
||||
return nil, nil
|
||||
}
|
||||
return b, err
|
||||
}
|
||||
|
||||
func (r *Reader) MultiGet(keys [][]byte) ([][]byte, error) {
|
||||
return store.MultiGet(r, keys)
|
||||
}
|
||||
|
||||
func (r *Reader) PrefixIterator(prefix []byte) store.KVIterator {
|
||||
byteRange := util.BytesPrefix(prefix)
|
||||
iter := r.snapshot.NewIterator(byteRange, r.store.defaultReadOptions)
|
||||
iter.First()
|
||||
rv := Iterator{
|
||||
store: r.store,
|
||||
iterator: iter,
|
||||
}
|
||||
return &rv
|
||||
}
|
||||
|
||||
func (r *Reader) RangeIterator(start, end []byte) store.KVIterator {
|
||||
byteRange := &util.Range{
|
||||
Start: start,
|
||||
Limit: end,
|
||||
}
|
||||
iter := r.snapshot.NewIterator(byteRange, r.store.defaultReadOptions)
|
||||
iter.First()
|
||||
rv := Iterator{
|
||||
store: r.store,
|
||||
iterator: iter,
|
||||
}
|
||||
return &rv
|
||||
}
|
||||
|
||||
func (r *Reader) Close() error {
|
||||
r.snapshot.Release()
|
||||
return nil
|
||||
}
|
152
index/upsidedown/store/goleveldb/store.go
Normal file
152
index/upsidedown/store/goleveldb/store.go
Normal file
|
@ -0,0 +1,152 @@
|
|||
// 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 goleveldb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"os"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
"github.com/blevesearch/goleveldb/leveldb"
|
||||
"github.com/blevesearch/goleveldb/leveldb/opt"
|
||||
"github.com/blevesearch/goleveldb/leveldb/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
const (
|
||||
Name = "goleveldb"
|
||||
defaultCompactBatchSize = 250
|
||||
)
|
||||
|
||||
type Store struct {
|
||||
path string
|
||||
opts *opt.Options
|
||||
db *leveldb.DB
|
||||
mo store.MergeOperator
|
||||
|
||||
defaultWriteOptions *opt.WriteOptions
|
||||
defaultReadOptions *opt.ReadOptions
|
||||
}
|
||||
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (store.KVStore, error) {
|
||||
|
||||
path, ok := config["path"].(string)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("must specify path")
|
||||
}
|
||||
if path == "" {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
opts, err := applyConfig(&opt.Options{}, config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
db, err := leveldb.OpenFile(path, opts)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rv := Store{
|
||||
path: path,
|
||||
opts: opts,
|
||||
db: db,
|
||||
mo: mo,
|
||||
defaultReadOptions: &opt.ReadOptions{},
|
||||
defaultWriteOptions: &opt.WriteOptions{},
|
||||
}
|
||||
rv.defaultWriteOptions.Sync = true
|
||||
return &rv, nil
|
||||
}
|
||||
|
||||
func (ldbs *Store) Close() error {
|
||||
return ldbs.db.Close()
|
||||
}
|
||||
|
||||
func (ldbs *Store) Reader() (store.KVReader, error) {
|
||||
snapshot, _ := ldbs.db.GetSnapshot()
|
||||
return &Reader{
|
||||
store: ldbs,
|
||||
snapshot: snapshot,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (ldbs *Store) Writer() (store.KVWriter, error) {
|
||||
return &Writer{
|
||||
store: ldbs,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// CompactWithBatchSize removes DictionaryTerm entries with a count of zero (in batchSize batches), then
|
||||
// compacts the underlying goleveldb store. Removing entries is a workaround for github issue #374.
|
||||
func (ldbs *Store) CompactWithBatchSize(batchSize int) error {
|
||||
// workaround for github issue #374 - remove DictionaryTerm keys with count=0
|
||||
batch := &leveldb.Batch{}
|
||||
for {
|
||||
t, err := ldbs.db.OpenTransaction()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
iter := t.NewIterator(util.BytesPrefix([]byte("d")), ldbs.defaultReadOptions)
|
||||
|
||||
for iter.Next() {
|
||||
if bytes.Equal(iter.Value(), []byte{0}) {
|
||||
k := append([]byte{}, iter.Key()...)
|
||||
batch.Delete(k)
|
||||
}
|
||||
if batch.Len() == batchSize {
|
||||
break
|
||||
}
|
||||
}
|
||||
iter.Release()
|
||||
if iter.Error() != nil {
|
||||
t.Discard()
|
||||
return iter.Error()
|
||||
}
|
||||
|
||||
if batch.Len() > 0 {
|
||||
err := t.Write(batch, ldbs.defaultWriteOptions)
|
||||
if err != nil {
|
||||
t.Discard()
|
||||
return err
|
||||
}
|
||||
err = t.Commit()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
} else {
|
||||
t.Discard()
|
||||
break
|
||||
}
|
||||
batch.Reset()
|
||||
}
|
||||
|
||||
return ldbs.db.CompactRange(util.Range{Start: nil, Limit: nil})
|
||||
}
|
||||
|
||||
// Compact compacts the underlying goleveldb store. The current implementation includes a workaround
|
||||
// for github issue #374 (see CompactWithBatchSize).
|
||||
func (ldbs *Store) Compact() error {
|
||||
return ldbs.CompactWithBatchSize(defaultCompactBatchSize)
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
99
index/upsidedown/store/goleveldb/store_test.go
Normal file
99
index/upsidedown/store/goleveldb/store_test.go
Normal file
|
@ -0,0 +1,99 @@
|
|||
// 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 goleveldb
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
)
|
||||
|
||||
func open(t *testing.T, mo store.MergeOperator) store.KVStore {
|
||||
rv, err := New(mo, map[string]interface{}{
|
||||
"path": "test",
|
||||
"create_if_missing": true,
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return rv
|
||||
}
|
||||
|
||||
func cleanup(t *testing.T, s store.KVStore) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
err = os.RemoveAll("test")
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGoLevelDBKVCrud(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBReaderIsolation(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBReaderOwnsGetBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBWriterOwnsBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBPrefixIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBPrefixIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBRangeIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBRangeIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestGoLevelDBMerge(t *testing.T) {
|
||||
s := open(t, &test.TestMergeCounter{})
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
68
index/upsidedown/store/goleveldb/writer.go
Normal file
68
index/upsidedown/store/goleveldb/writer.go
Normal file
|
@ -0,0 +1,68 @@
|
|||
// Copyright (c) 2014 Couchbase, Inc.
|
||||
//
|
||||
// Licensed under the Apache License, Version 2.0 (the "License");
|
||||
// you may not use this file except in compliance with the License.
|
||||
// You may obtain a copy of the License at
|
||||
//
|
||||
// http://www.apache.org/licenses/LICENSE-2.0
|
||||
//
|
||||
// Unless required by applicable law or agreed to in writing, software
|
||||
// distributed under the License is distributed on an "AS IS" BASIS,
|
||||
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
// See the License for the specific language governing permissions and
|
||||
// limitations under the License.
|
||||
|
||||
package goleveldb
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
"github.com/blevesearch/goleveldb/leveldb"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
store *Store
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatch() store.KVBatch {
|
||||
rv := Batch{
|
||||
store: w.store,
|
||||
merge: store.NewEmulatedMerge(w.store.mo),
|
||||
batch: new(leveldb.Batch),
|
||||
}
|
||||
return &rv
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatchEx(options store.KVBatchOptions) ([]byte, store.KVBatch, error) {
|
||||
return make([]byte, options.TotalBytes), w.NewBatch(), nil
|
||||
}
|
||||
|
||||
func (w *Writer) ExecuteBatch(b store.KVBatch) error {
|
||||
batch, ok := b.(*Batch)
|
||||
if !ok {
|
||||
return fmt.Errorf("wrong type of batch")
|
||||
}
|
||||
|
||||
// first process merges
|
||||
for k, mergeOps := range batch.merge.Merges {
|
||||
kb := []byte(k)
|
||||
existingVal, err := w.store.db.Get(kb, w.store.defaultReadOptions)
|
||||
if err != nil && err != leveldb.ErrNotFound {
|
||||
return err
|
||||
}
|
||||
mergedVal, fullMergeOk := w.store.mo.FullMerge(kb, existingVal, mergeOps)
|
||||
if !fullMergeOk {
|
||||
return fmt.Errorf("merge operator returned failure")
|
||||
}
|
||||
// add the final merge to this batch
|
||||
batch.batch.Put(kb, mergedVal)
|
||||
}
|
||||
|
||||
// now execute the batch
|
||||
return w.store.db.Write(batch.batch, w.store.defaultWriteOptions)
|
||||
}
|
||||
|
||||
func (w *Writer) Close() error {
|
||||
return nil
|
||||
}
|
152
index/upsidedown/store/gtreap/iterator.go
Normal file
152
index/upsidedown/store/gtreap/iterator.go
Normal file
|
@ -0,0 +1,152 @@
|
|||
// 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 gtreap provides an in-memory implementation of the
|
||||
// KVStore interfaces using the gtreap balanced-binary treap,
|
||||
// copy-on-write data structure.
|
||||
package gtreap
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"sync"
|
||||
|
||||
"github.com/blevesearch/gtreap"
|
||||
)
|
||||
|
||||
type Iterator struct {
|
||||
t *gtreap.Treap
|
||||
|
||||
m sync.Mutex
|
||||
cancelCh chan struct{}
|
||||
nextCh chan *Item
|
||||
curr *Item
|
||||
currOk bool
|
||||
|
||||
prefix []byte
|
||||
start []byte
|
||||
end []byte
|
||||
}
|
||||
|
||||
func (w *Iterator) Seek(k []byte) {
|
||||
if w.start != nil && bytes.Compare(k, w.start) < 0 {
|
||||
k = w.start
|
||||
}
|
||||
if w.prefix != nil && !bytes.HasPrefix(k, w.prefix) {
|
||||
if bytes.Compare(k, w.prefix) < 0 {
|
||||
k = w.prefix
|
||||
} else {
|
||||
var end []byte
|
||||
for i := len(w.prefix) - 1; i >= 0; i-- {
|
||||
c := w.prefix[i]
|
||||
if c < 0xff {
|
||||
end = make([]byte, i+1)
|
||||
copy(end, w.prefix)
|
||||
end[i] = c + 1
|
||||
break
|
||||
}
|
||||
}
|
||||
k = end
|
||||
}
|
||||
}
|
||||
w.restart(&Item{k: k})
|
||||
}
|
||||
|
||||
func (w *Iterator) restart(start *Item) *Iterator {
|
||||
cancelCh := make(chan struct{})
|
||||
nextCh := make(chan *Item, 1)
|
||||
|
||||
w.m.Lock()
|
||||
if w.cancelCh != nil {
|
||||
close(w.cancelCh)
|
||||
}
|
||||
w.cancelCh = cancelCh
|
||||
w.nextCh = nextCh
|
||||
w.curr = nil
|
||||
w.currOk = false
|
||||
w.m.Unlock()
|
||||
|
||||
go func() {
|
||||
if start != nil {
|
||||
w.t.VisitAscend(start, func(itm gtreap.Item) bool {
|
||||
select {
|
||||
case <-cancelCh:
|
||||
return false
|
||||
case nextCh <- itm.(*Item):
|
||||
return true
|
||||
}
|
||||
})
|
||||
}
|
||||
close(nextCh)
|
||||
}()
|
||||
|
||||
w.Next()
|
||||
|
||||
return w
|
||||
}
|
||||
|
||||
func (w *Iterator) Next() {
|
||||
w.m.Lock()
|
||||
nextCh := w.nextCh
|
||||
w.m.Unlock()
|
||||
w.curr, w.currOk = <-nextCh
|
||||
}
|
||||
|
||||
func (w *Iterator) Current() ([]byte, []byte, bool) {
|
||||
w.m.Lock()
|
||||
defer w.m.Unlock()
|
||||
if !w.currOk || w.curr == nil {
|
||||
return nil, nil, false
|
||||
}
|
||||
if w.prefix != nil && !bytes.HasPrefix(w.curr.k, w.prefix) {
|
||||
return nil, nil, false
|
||||
} else if w.end != nil && bytes.Compare(w.curr.k, w.end) >= 0 {
|
||||
return nil, nil, false
|
||||
}
|
||||
return w.curr.k, w.curr.v, w.currOk
|
||||
}
|
||||
|
||||
func (w *Iterator) Key() []byte {
|
||||
k, _, ok := w.Current()
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
return k
|
||||
}
|
||||
|
||||
func (w *Iterator) Value() []byte {
|
||||
_, v, ok := w.Current()
|
||||
if !ok {
|
||||
return nil
|
||||
}
|
||||
return v
|
||||
}
|
||||
|
||||
func (w *Iterator) Valid() bool {
|
||||
_, _, ok := w.Current()
|
||||
return ok
|
||||
}
|
||||
|
||||
func (w *Iterator) Close() error {
|
||||
w.m.Lock()
|
||||
if w.cancelCh != nil {
|
||||
close(w.cancelCh)
|
||||
}
|
||||
w.cancelCh = nil
|
||||
w.nextCh = nil
|
||||
w.curr = nil
|
||||
w.currOk = false
|
||||
w.m.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
66
index/upsidedown/store/gtreap/reader.go
Normal file
66
index/upsidedown/store/gtreap/reader.go
Normal file
|
@ -0,0 +1,66 @@
|
|||
// 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 gtreap provides an in-memory implementation of the
|
||||
// KVStore interfaces using the gtreap balanced-binary treap,
|
||||
// copy-on-write data structure.
|
||||
package gtreap
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/upsidedown_store_api"
|
||||
|
||||
"github.com/blevesearch/gtreap"
|
||||
)
|
||||
|
||||
type Reader struct {
|
||||
t *gtreap.Treap
|
||||
}
|
||||
|
||||
func (w *Reader) Get(k []byte) (v []byte, err error) {
|
||||
var rv []byte
|
||||
itm := w.t.Get(&Item{k: k})
|
||||
if itm != nil {
|
||||
rv = make([]byte, len(itm.(*Item).v))
|
||||
copy(rv, itm.(*Item).v)
|
||||
return rv, nil
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (r *Reader) MultiGet(keys [][]byte) ([][]byte, error) {
|
||||
return store.MultiGet(r, keys)
|
||||
}
|
||||
|
||||
func (w *Reader) PrefixIterator(k []byte) store.KVIterator {
|
||||
rv := Iterator{
|
||||
t: w.t,
|
||||
prefix: k,
|
||||
}
|
||||
rv.restart(&Item{k: k})
|
||||
return &rv
|
||||
}
|
||||
|
||||
func (w *Reader) RangeIterator(start, end []byte) store.KVIterator {
|
||||
rv := Iterator{
|
||||
t: w.t,
|
||||
start: start,
|
||||
end: end,
|
||||
}
|
||||
rv.restart(&Item{k: start})
|
||||
return &rv
|
||||
}
|
||||
|
||||
func (w *Reader) Close() error {
|
||||
return nil
|
||||
}
|
85
index/upsidedown/store/gtreap/store.go
Normal file
85
index/upsidedown/store/gtreap/store.go
Normal 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 gtreap provides an in-memory implementation of the
|
||||
// KVStore interfaces using the gtreap balanced-binary treap,
|
||||
// copy-on-write data structure.
|
||||
|
||||
package gtreap
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"fmt"
|
||||
"os"
|
||||
"sync"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
"github.com/blevesearch/gtreap"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
const Name = "gtreap"
|
||||
|
||||
type Store struct {
|
||||
m sync.Mutex
|
||||
t *gtreap.Treap
|
||||
mo store.MergeOperator
|
||||
}
|
||||
|
||||
type Item struct {
|
||||
k []byte
|
||||
v []byte
|
||||
}
|
||||
|
||||
func itemCompare(a, b interface{}) int {
|
||||
return bytes.Compare(a.(*Item).k, b.(*Item).k)
|
||||
}
|
||||
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (store.KVStore, error) {
|
||||
path, ok := config["path"].(string)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("must specify path")
|
||||
}
|
||||
if path != "" {
|
||||
return nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
rv := Store{
|
||||
t: gtreap.NewTreap(itemCompare),
|
||||
mo: mo,
|
||||
}
|
||||
return &rv, nil
|
||||
}
|
||||
|
||||
func (s *Store) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *Store) Reader() (store.KVReader, error) {
|
||||
s.m.Lock()
|
||||
t := s.t
|
||||
s.m.Unlock()
|
||||
return &Reader{t: t}, nil
|
||||
}
|
||||
|
||||
func (s *Store) Writer() (store.KVWriter, error) {
|
||||
return &Writer{s: s}, nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
93
index/upsidedown/store/gtreap/store_test.go
Normal file
93
index/upsidedown/store/gtreap/store_test.go
Normal file
|
@ -0,0 +1,93 @@
|
|||
// 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 gtreap
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
)
|
||||
|
||||
func open(t *testing.T, mo store.MergeOperator) store.KVStore {
|
||||
rv, err := New(mo, map[string]interface{}{
|
||||
"path": "",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return rv
|
||||
}
|
||||
|
||||
func cleanup(t *testing.T, s store.KVStore) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestGTreapKVCrud(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapReaderIsolation(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapReaderOwnsGetBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapWriterOwnsBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapPrefixIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapPrefixIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapRangeIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapRangeIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestGTreapMerge(t *testing.T) {
|
||||
s := open(t, &test.TestMergeCounter{})
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
76
index/upsidedown/store/gtreap/writer.go
Normal file
76
index/upsidedown/store/gtreap/writer.go
Normal 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 gtreap provides an in-memory implementation of the
|
||||
// KVStore interfaces using the gtreap balanced-binary treap,
|
||||
// copy-on-write data structure.
|
||||
package gtreap
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math/rand"
|
||||
|
||||
"github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
s *Store
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatch() store.KVBatch {
|
||||
return store.NewEmulatedBatch(w.s.mo)
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatchEx(options store.KVBatchOptions) ([]byte, store.KVBatch, error) {
|
||||
return make([]byte, options.TotalBytes), w.NewBatch(), nil
|
||||
}
|
||||
|
||||
func (w *Writer) ExecuteBatch(batch store.KVBatch) error {
|
||||
|
||||
emulatedBatch, ok := batch.(*store.EmulatedBatch)
|
||||
if !ok {
|
||||
return fmt.Errorf("wrong type of batch")
|
||||
}
|
||||
|
||||
w.s.m.Lock()
|
||||
for k, mergeOps := range emulatedBatch.Merger.Merges {
|
||||
kb := []byte(k)
|
||||
var existingVal []byte
|
||||
existingItem := w.s.t.Get(&Item{k: kb})
|
||||
if existingItem != nil {
|
||||
existingVal = w.s.t.Get(&Item{k: kb}).(*Item).v
|
||||
}
|
||||
mergedVal, fullMergeOk := w.s.mo.FullMerge(kb, existingVal, mergeOps)
|
||||
if !fullMergeOk {
|
||||
return fmt.Errorf("merge operator returned failure")
|
||||
}
|
||||
w.s.t = w.s.t.Upsert(&Item{k: kb, v: mergedVal}, rand.Int())
|
||||
}
|
||||
|
||||
for _, op := range emulatedBatch.Ops {
|
||||
if op.V != nil {
|
||||
w.s.t = w.s.t.Upsert(&Item{k: op.K, v: op.V}, rand.Int())
|
||||
} else {
|
||||
w.s.t = w.s.t.Delete(&Item{k: op.K})
|
||||
}
|
||||
}
|
||||
w.s.m.Unlock()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *Writer) Close() error {
|
||||
w.s = nil
|
||||
return nil
|
||||
}
|
46
index/upsidedown/store/metrics/batch.go
Normal file
46
index/upsidedown/store/metrics/batch.go
Normal file
|
@ -0,0 +1,46 @@
|
|||
// 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 metrics
|
||||
|
||||
import store "github.com/blevesearch/upsidedown_store_api"
|
||||
|
||||
type Batch struct {
|
||||
s *Store
|
||||
o store.KVBatch
|
||||
}
|
||||
|
||||
func (b *Batch) Set(key, val []byte) {
|
||||
b.o.Set(key, val)
|
||||
}
|
||||
|
||||
func (b *Batch) Delete(key []byte) {
|
||||
b.o.Delete(key)
|
||||
}
|
||||
|
||||
func (b *Batch) Merge(key, val []byte) {
|
||||
b.s.timerBatchMerge.Time(func() {
|
||||
b.o.Merge(key, val)
|
||||
})
|
||||
}
|
||||
|
||||
func (b *Batch) Reset() {
|
||||
b.o.Reset()
|
||||
}
|
||||
|
||||
func (b *Batch) Close() error {
|
||||
err := b.o.Close()
|
||||
b.o = nil
|
||||
return err
|
||||
}
|
58
index/upsidedown/store/metrics/iterator.go
Normal file
58
index/upsidedown/store/metrics/iterator.go
Normal file
|
@ -0,0 +1,58 @@
|
|||
// 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 metrics
|
||||
|
||||
import store "github.com/blevesearch/upsidedown_store_api"
|
||||
|
||||
type Iterator struct {
|
||||
s *Store
|
||||
o store.KVIterator
|
||||
}
|
||||
|
||||
func (i *Iterator) Seek(x []byte) {
|
||||
i.s.timerIteratorSeek.Time(func() {
|
||||
i.o.Seek(x)
|
||||
})
|
||||
}
|
||||
|
||||
func (i *Iterator) Next() {
|
||||
i.s.timerIteratorNext.Time(func() {
|
||||
i.o.Next()
|
||||
})
|
||||
}
|
||||
|
||||
func (i *Iterator) Current() ([]byte, []byte, bool) {
|
||||
return i.o.Current()
|
||||
}
|
||||
|
||||
func (i *Iterator) Key() []byte {
|
||||
return i.o.Key()
|
||||
}
|
||||
|
||||
func (i *Iterator) Value() []byte {
|
||||
return i.o.Value()
|
||||
}
|
||||
|
||||
func (i *Iterator) Valid() bool {
|
||||
return i.o.Valid()
|
||||
}
|
||||
|
||||
func (i *Iterator) Close() error {
|
||||
err := i.o.Close()
|
||||
if err != nil {
|
||||
i.s.AddError("Iterator.Close", err, nil)
|
||||
}
|
||||
return err
|
||||
}
|
141
index/upsidedown/store/metrics/metrics_test.go
Normal file
141
index/upsidedown/store/metrics/metrics_test.go
Normal file
|
@ -0,0 +1,141 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"testing"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
|
||||
)
|
||||
|
||||
func TestMetricsStore(t *testing.T) {
|
||||
_, err := New(nil, map[string]interface{}{})
|
||||
if err == nil {
|
||||
t.Errorf("expected err when bad config")
|
||||
}
|
||||
|
||||
_, err = New(nil, map[string]interface{}{
|
||||
"kvStoreName_actual": "some-invalid-kvstore-name",
|
||||
})
|
||||
if err == nil {
|
||||
t.Errorf("expected err when unknown kvStoreName_actual")
|
||||
}
|
||||
|
||||
s, err := New(nil, map[string]interface{}{
|
||||
"kvStoreName_actual": gtreap.Name,
|
||||
"path": "",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
b := bytes.NewBuffer(nil)
|
||||
err = s.(*Store).WriteJSON(b)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
if b.Len() <= 0 {
|
||||
t.Errorf("expected some output from WriteJSON")
|
||||
}
|
||||
var m map[string]interface{}
|
||||
err = json.Unmarshal(b.Bytes(), &m)
|
||||
if err != nil {
|
||||
t.Errorf("expected WriteJSON to be unmarshallable")
|
||||
}
|
||||
if len(m) == 0 {
|
||||
t.Errorf("expected some entries")
|
||||
}
|
||||
|
||||
b = bytes.NewBuffer(nil)
|
||||
s.(*Store).WriteCSVHeader(b)
|
||||
if b.Len() <= 0 {
|
||||
t.Errorf("expected some output from WriteCSVHeader")
|
||||
}
|
||||
|
||||
b = bytes.NewBuffer(nil)
|
||||
s.(*Store).WriteCSV(b)
|
||||
if b.Len() <= 0 {
|
||||
t.Errorf("expected some output from WriteCSV")
|
||||
}
|
||||
}
|
||||
|
||||
func TestErrors(t *testing.T) {
|
||||
s, err := New(nil, map[string]interface{}{
|
||||
"kvStoreName_actual": gtreap.Name,
|
||||
"path": "",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
x, ok := s.(*Store)
|
||||
if !ok {
|
||||
t.Errorf("expecting a Store")
|
||||
}
|
||||
|
||||
x.AddError("foo", fmt.Errorf("Foo"), []byte("fooKey"))
|
||||
x.AddError("bar", fmt.Errorf("Bar"), nil)
|
||||
x.AddError("baz", fmt.Errorf("Baz"), []byte("bazKey"))
|
||||
|
||||
b := bytes.NewBuffer(nil)
|
||||
err = x.WriteJSON(b)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
var m map[string]interface{}
|
||||
err = json.Unmarshal(b.Bytes(), &m)
|
||||
if err != nil {
|
||||
t.Errorf("expected unmarshallable writeJSON, err: %v, b: %s",
|
||||
err, b.Bytes())
|
||||
}
|
||||
|
||||
errorsi, ok := m["Errors"]
|
||||
if !ok || errorsi == nil {
|
||||
t.Errorf("expected errorsi")
|
||||
}
|
||||
errors, ok := errorsi.([]interface{})
|
||||
if !ok || errors == nil {
|
||||
t.Errorf("expected errorsi is array")
|
||||
}
|
||||
if len(errors) != 3 {
|
||||
t.Errorf("expected errors len 3")
|
||||
}
|
||||
|
||||
e := errors[0].(map[string]interface{})
|
||||
if e["Op"].(string) != "foo" ||
|
||||
e["Err"].(string) != "Foo" ||
|
||||
len(e["Time"].(string)) < 10 ||
|
||||
e["Key"].(string) != "fooKey" {
|
||||
t.Errorf("expected foo, %#v", e)
|
||||
}
|
||||
e = errors[1].(map[string]interface{})
|
||||
if e["Op"].(string) != "bar" ||
|
||||
e["Err"].(string) != "Bar" ||
|
||||
len(e["Time"].(string)) < 10 ||
|
||||
e["Key"].(string) != "" {
|
||||
t.Errorf("expected bar, %#v", e)
|
||||
}
|
||||
e = errors[2].(map[string]interface{})
|
||||
if e["Op"].(string) != "baz" ||
|
||||
e["Err"].(string) != "Baz" ||
|
||||
len(e["Time"].(string)) < 10 ||
|
||||
e["Key"].(string) != "bazKey" {
|
||||
t.Errorf("expected baz, %#v", e)
|
||||
}
|
||||
}
|
64
index/upsidedown/store/metrics/reader.go
Normal file
64
index/upsidedown/store/metrics/reader.go
Normal file
|
@ -0,0 +1,64 @@
|
|||
// 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 metrics
|
||||
|
||||
import store "github.com/blevesearch/upsidedown_store_api"
|
||||
|
||||
type Reader struct {
|
||||
s *Store
|
||||
o store.KVReader
|
||||
}
|
||||
|
||||
func (r *Reader) Get(key []byte) (v []byte, err error) {
|
||||
r.s.timerReaderGet.Time(func() {
|
||||
v, err = r.o.Get(key)
|
||||
if err != nil {
|
||||
r.s.AddError("Reader.Get", err, key)
|
||||
}
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (r *Reader) MultiGet(keys [][]byte) (vals [][]byte, err error) {
|
||||
r.s.timerReaderMultiGet.Time(func() {
|
||||
vals, err = r.o.MultiGet(keys)
|
||||
if err != nil {
|
||||
r.s.AddError("Reader.MultiGet", err, nil)
|
||||
}
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (r *Reader) PrefixIterator(prefix []byte) (i store.KVIterator) {
|
||||
r.s.timerReaderPrefixIterator.Time(func() {
|
||||
i = &Iterator{s: r.s, o: r.o.PrefixIterator(prefix)}
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (r *Reader) RangeIterator(start, end []byte) (i store.KVIterator) {
|
||||
r.s.timerReaderRangeIterator.Time(func() {
|
||||
i = &Iterator{s: r.s, o: r.o.RangeIterator(start, end)}
|
||||
})
|
||||
return
|
||||
}
|
||||
|
||||
func (r *Reader) Close() error {
|
||||
err := r.o.Close()
|
||||
if err != nil {
|
||||
r.s.AddError("Reader.Close", err, nil)
|
||||
}
|
||||
return err
|
||||
}
|
50
index/upsidedown/store/metrics/stats.go
Normal file
50
index/upsidedown/store/metrics/stats.go
Normal file
|
@ -0,0 +1,50 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type stats struct {
|
||||
s *Store
|
||||
}
|
||||
|
||||
func (s *stats) statsMap() map[string]interface{} {
|
||||
ms := map[string]interface{}{}
|
||||
|
||||
ms["metrics"] = map[string]interface{}{
|
||||
"reader_get": TimerMap(s.s.timerReaderGet),
|
||||
"reader_multi_get": TimerMap(s.s.timerReaderMultiGet),
|
||||
"reader_prefix_iterator": TimerMap(s.s.timerReaderPrefixIterator),
|
||||
"reader_range_iterator": TimerMap(s.s.timerReaderRangeIterator),
|
||||
"writer_execute_batch": TimerMap(s.s.timerWriterExecuteBatch),
|
||||
"iterator_seek": TimerMap(s.s.timerIteratorSeek),
|
||||
"iterator_next": TimerMap(s.s.timerIteratorNext),
|
||||
"batch_merge": TimerMap(s.s.timerBatchMerge),
|
||||
}
|
||||
|
||||
if o, ok := s.s.o.(store.KVStoreStats); ok {
|
||||
ms["kv"] = o.StatsMap()
|
||||
}
|
||||
|
||||
return ms
|
||||
}
|
||||
|
||||
func (s *stats) MarshalJSON() ([]byte, error) {
|
||||
m := s.statsMap()
|
||||
return util.MarshalJSON(m)
|
||||
}
|
277
index/upsidedown/store/metrics/store.go
Normal file
277
index/upsidedown/store/metrics/store.go
Normal file
|
@ -0,0 +1,277 @@
|
|||
// 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 metrics provides a bleve.store.KVStore implementation that
|
||||
// wraps another, real KVStore implementation, and uses go-metrics to
|
||||
// track runtime performance metrics.
|
||||
package metrics
|
||||
|
||||
import (
|
||||
"container/list"
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"io"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
"github.com/blevesearch/go-metrics"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
const Name = "metrics"
|
||||
|
||||
type Store struct {
|
||||
o store.KVStore
|
||||
|
||||
timerReaderGet metrics.Timer
|
||||
timerReaderMultiGet metrics.Timer
|
||||
timerReaderPrefixIterator metrics.Timer
|
||||
timerReaderRangeIterator metrics.Timer
|
||||
timerWriterExecuteBatch metrics.Timer
|
||||
timerIteratorSeek metrics.Timer
|
||||
timerIteratorNext metrics.Timer
|
||||
timerBatchMerge metrics.Timer
|
||||
|
||||
m sync.Mutex // Protects the fields that follow.
|
||||
errors *list.List // Capped list of StoreError's.
|
||||
|
||||
s *stats
|
||||
}
|
||||
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (store.KVStore, error) {
|
||||
|
||||
name, ok := config["kvStoreName_actual"].(string)
|
||||
if !ok || name == "" {
|
||||
return nil, fmt.Errorf("metrics: missing kvStoreName_actual,"+
|
||||
" config: %#v", config)
|
||||
}
|
||||
|
||||
if name == Name {
|
||||
return nil, fmt.Errorf("metrics: circular kvStoreName_actual")
|
||||
}
|
||||
|
||||
ctr := registry.KVStoreConstructorByName(name)
|
||||
if ctr == nil {
|
||||
return nil, fmt.Errorf("metrics: no kv store constructor,"+
|
||||
" kvStoreName_actual: %s", name)
|
||||
}
|
||||
|
||||
kvs, err := ctr(mo, config)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rv := &Store{
|
||||
o: kvs,
|
||||
|
||||
timerReaderGet: metrics.NewTimer(),
|
||||
timerReaderMultiGet: metrics.NewTimer(),
|
||||
timerReaderPrefixIterator: metrics.NewTimer(),
|
||||
timerReaderRangeIterator: metrics.NewTimer(),
|
||||
timerWriterExecuteBatch: metrics.NewTimer(),
|
||||
timerIteratorSeek: metrics.NewTimer(),
|
||||
timerIteratorNext: metrics.NewTimer(),
|
||||
timerBatchMerge: metrics.NewTimer(),
|
||||
|
||||
errors: list.New(),
|
||||
}
|
||||
|
||||
rv.s = &stats{s: rv}
|
||||
|
||||
return rv, nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Store) Close() error {
|
||||
return s.o.Close()
|
||||
}
|
||||
|
||||
func (s *Store) Reader() (store.KVReader, error) {
|
||||
o, err := s.o.Reader()
|
||||
if err != nil {
|
||||
s.AddError("Reader", err, nil)
|
||||
return nil, err
|
||||
}
|
||||
return &Reader{s: s, o: o}, nil
|
||||
}
|
||||
|
||||
func (s *Store) Writer() (store.KVWriter, error) {
|
||||
o, err := s.o.Writer()
|
||||
if err != nil {
|
||||
s.AddError("Writer", err, nil)
|
||||
return nil, err
|
||||
}
|
||||
return &Writer{s: s, o: o}, nil
|
||||
}
|
||||
|
||||
// Metric specific code below:
|
||||
|
||||
const MaxErrors = 100
|
||||
|
||||
type StoreError struct {
|
||||
Time string
|
||||
Op string
|
||||
Err string
|
||||
Key string
|
||||
}
|
||||
|
||||
func (s *Store) AddError(op string, err error, key []byte) {
|
||||
e := &StoreError{
|
||||
Time: time.Now().Format(time.RFC3339Nano),
|
||||
Op: op,
|
||||
Err: fmt.Sprintf("%v", err),
|
||||
Key: string(key),
|
||||
}
|
||||
|
||||
s.m.Lock()
|
||||
for s.errors.Len() >= MaxErrors {
|
||||
s.errors.Remove(s.errors.Front())
|
||||
}
|
||||
s.errors.PushBack(e)
|
||||
s.m.Unlock()
|
||||
}
|
||||
|
||||
func (s *Store) WriteJSON(w io.Writer) (err error) {
|
||||
_, err = w.Write([]byte(`{"TimerReaderGet":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerReaderGet)
|
||||
_, err = w.Write([]byte(`,"TimerReaderMultiGet":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerReaderMultiGet)
|
||||
_, err = w.Write([]byte(`,"TimerReaderPrefixIterator":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerReaderPrefixIterator)
|
||||
_, err = w.Write([]byte(`,"TimerReaderRangeIterator":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerReaderRangeIterator)
|
||||
_, err = w.Write([]byte(`,"TimerWriterExecuteBatch":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerWriterExecuteBatch)
|
||||
_, err = w.Write([]byte(`,"TimerIteratorSeek":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerIteratorSeek)
|
||||
_, err = w.Write([]byte(`,"TimerIteratorNext":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerIteratorNext)
|
||||
_, err = w.Write([]byte(`,"TimerBatchMerge":`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
WriteTimerJSON(w, s.timerBatchMerge)
|
||||
|
||||
_, err = w.Write([]byte(`,"Errors":[`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
s.m.Lock()
|
||||
defer s.m.Unlock()
|
||||
e := s.errors.Front()
|
||||
i := 0
|
||||
for e != nil {
|
||||
se, ok := e.Value.(*StoreError)
|
||||
if ok && se != nil {
|
||||
if i > 0 {
|
||||
_, err = w.Write([]byte(","))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
var buf []byte
|
||||
buf, err = util.MarshalJSON(se)
|
||||
if err == nil {
|
||||
_, err = w.Write(buf)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
e = e.Next()
|
||||
i = i + 1
|
||||
}
|
||||
_, err = w.Write([]byte(`]`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
// see if the underlying implementation has its own stats
|
||||
if o, ok := s.o.(store.KVStoreStats); ok {
|
||||
storeStats := o.Stats()
|
||||
var storeBytes []byte
|
||||
storeBytes, err = util.MarshalJSON(storeStats)
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
_, err = fmt.Fprintf(w, `, "store": %s`, string(storeBytes))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
_, err = w.Write([]byte(`}`))
|
||||
if err != nil {
|
||||
return
|
||||
}
|
||||
|
||||
return
|
||||
}
|
||||
|
||||
func (s *Store) WriteCSVHeader(w io.Writer) {
|
||||
WriteTimerCSVHeader(w, "TimerReaderGet")
|
||||
WriteTimerCSVHeader(w, "TimerReaderPrefixIterator")
|
||||
WriteTimerCSVHeader(w, "TimerReaderRangeIterator")
|
||||
WriteTimerCSVHeader(w, "TimerWtierExecuteBatch")
|
||||
WriteTimerCSVHeader(w, "TimerIteratorSeek")
|
||||
WriteTimerCSVHeader(w, "TimerIteratorNext")
|
||||
WriteTimerCSVHeader(w, "TimerBatchMerge")
|
||||
}
|
||||
|
||||
func (s *Store) WriteCSV(w io.Writer) {
|
||||
WriteTimerCSV(w, s.timerReaderGet)
|
||||
WriteTimerCSV(w, s.timerReaderPrefixIterator)
|
||||
WriteTimerCSV(w, s.timerReaderRangeIterator)
|
||||
WriteTimerCSV(w, s.timerWriterExecuteBatch)
|
||||
WriteTimerCSV(w, s.timerIteratorSeek)
|
||||
WriteTimerCSV(w, s.timerIteratorNext)
|
||||
WriteTimerCSV(w, s.timerBatchMerge)
|
||||
}
|
||||
|
||||
func (s *Store) Stats() json.Marshaler {
|
||||
return s.s
|
||||
}
|
||||
|
||||
func (s *Store) StatsMap() map[string]interface{} {
|
||||
return s.s.statsMap()
|
||||
}
|
95
index/upsidedown/store/metrics/store_test.go
Normal file
95
index/upsidedown/store/metrics/store_test.go
Normal file
|
@ -0,0 +1,95 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/index/upsidedown/store/gtreap"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
)
|
||||
|
||||
func open(t *testing.T, mo store.MergeOperator) store.KVStore {
|
||||
rv, err := New(mo, map[string]interface{}{
|
||||
"kvStoreName_actual": gtreap.Name,
|
||||
"path": "",
|
||||
})
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return rv
|
||||
}
|
||||
|
||||
func cleanup(t *testing.T, s store.KVStore) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMetricsKVCrud(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsReaderIsolation(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsReaderOwnsGetBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsWriterOwnsBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsPrefixIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsPrefixIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsRangeIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsRangeIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMetricsMerge(t *testing.T) {
|
||||
s := open(t, &test.TestMergeCounter{})
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
135
index/upsidedown/store/metrics/util.go
Normal file
135
index/upsidedown/store/metrics/util.go
Normal file
|
@ -0,0 +1,135 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"math"
|
||||
|
||||
"github.com/blevesearch/go-metrics"
|
||||
)
|
||||
|
||||
// NOTE: This is copy & pasted from cbft as otherwise there
|
||||
// would be an import cycle.
|
||||
|
||||
var timerPercentiles = []float64{0.5, 0.75, 0.95, 0.99, 0.999}
|
||||
|
||||
func TimerMap(timer metrics.Timer) map[string]interface{} {
|
||||
|
||||
rv := make(map[string]interface{})
|
||||
t := timer.Snapshot()
|
||||
p := t.Percentiles(timerPercentiles)
|
||||
|
||||
percentileKeys := []string{"median", "75%", "95%", "99%", "99.9%"}
|
||||
percentiles := make(map[string]interface{})
|
||||
for i, pi := range p {
|
||||
if !isNanOrInf(pi) {
|
||||
percentileKey := percentileKeys[i]
|
||||
percentiles[percentileKey] = pi
|
||||
}
|
||||
}
|
||||
|
||||
rateKeys := []string{"1-min", "5-min", "15-min", "mean"}
|
||||
rates := make(map[string]interface{})
|
||||
for i, ri := range []float64{t.Rate1(), t.Rate5(), t.Rate15(), t.RateMean()} {
|
||||
if !isNanOrInf(ri) {
|
||||
rateKey := rateKeys[i]
|
||||
rates[rateKey] = ri
|
||||
}
|
||||
}
|
||||
|
||||
rv["count"] = t.Count()
|
||||
rv["min"] = t.Min()
|
||||
rv["max"] = t.Max()
|
||||
mean := t.Mean()
|
||||
if !isNanOrInf(mean) {
|
||||
rv["mean"] = mean
|
||||
}
|
||||
stddev := t.StdDev()
|
||||
if !isNanOrInf(stddev) {
|
||||
rv["stddev"] = stddev
|
||||
}
|
||||
rv["percentiles"] = percentiles
|
||||
rv["rates"] = rates
|
||||
|
||||
return rv
|
||||
}
|
||||
|
||||
func isNanOrInf(v float64) bool {
|
||||
if math.IsNaN(v) || math.IsInf(v, 0) {
|
||||
return true
|
||||
}
|
||||
return false
|
||||
}
|
||||
|
||||
func WriteTimerJSON(w io.Writer, timer metrics.Timer) {
|
||||
t := timer.Snapshot()
|
||||
p := t.Percentiles(timerPercentiles)
|
||||
|
||||
fmt.Fprintf(w, `{"count":%9d,`, t.Count())
|
||||
fmt.Fprintf(w, `"min":%9d,`, t.Min())
|
||||
fmt.Fprintf(w, `"max":%9d,`, t.Max())
|
||||
fmt.Fprintf(w, `"mean":%12.2f,`, t.Mean())
|
||||
fmt.Fprintf(w, `"stddev":%12.2f,`, t.StdDev())
|
||||
fmt.Fprintf(w, `"percentiles":{`)
|
||||
fmt.Fprintf(w, `"median":%12.2f,`, p[0])
|
||||
fmt.Fprintf(w, `"75%%":%12.2f,`, p[1])
|
||||
fmt.Fprintf(w, `"95%%":%12.2f,`, p[2])
|
||||
fmt.Fprintf(w, `"99%%":%12.2f,`, p[3])
|
||||
fmt.Fprintf(w, `"99.9%%":%12.2f},`, p[4])
|
||||
fmt.Fprintf(w, `"rates":{`)
|
||||
fmt.Fprintf(w, `"1-min":%12.2f,`, t.Rate1())
|
||||
fmt.Fprintf(w, `"5-min":%12.2f,`, t.Rate5())
|
||||
fmt.Fprintf(w, `"15-min":%12.2f,`, t.Rate15())
|
||||
fmt.Fprintf(w, `"mean":%12.2f}}`, t.RateMean())
|
||||
}
|
||||
|
||||
func WriteTimerCSVHeader(w io.Writer, prefix string) {
|
||||
fmt.Fprintf(w, "%s-count,", prefix)
|
||||
fmt.Fprintf(w, "%s-min,", prefix)
|
||||
fmt.Fprintf(w, "%s-max,", prefix)
|
||||
fmt.Fprintf(w, "%s-mean,", prefix)
|
||||
fmt.Fprintf(w, "%s-stddev,", prefix)
|
||||
fmt.Fprintf(w, "%s-percentile-50%%,", prefix)
|
||||
fmt.Fprintf(w, "%s-percentile-75%%,", prefix)
|
||||
fmt.Fprintf(w, "%s-percentile-95%%,", prefix)
|
||||
fmt.Fprintf(w, "%s-percentile-99%%,", prefix)
|
||||
fmt.Fprintf(w, "%s-percentile-99.9%%,", prefix)
|
||||
fmt.Fprintf(w, "%s-rate-1-min,", prefix)
|
||||
fmt.Fprintf(w, "%s-rate-5-min,", prefix)
|
||||
fmt.Fprintf(w, "%s-rate-15-min,", prefix)
|
||||
fmt.Fprintf(w, "%s-rate-mean", prefix)
|
||||
}
|
||||
|
||||
func WriteTimerCSV(w io.Writer, timer metrics.Timer) {
|
||||
t := timer.Snapshot()
|
||||
p := t.Percentiles(timerPercentiles)
|
||||
|
||||
fmt.Fprintf(w, `%d,`, t.Count())
|
||||
fmt.Fprintf(w, `%d,`, t.Min())
|
||||
fmt.Fprintf(w, `%d,`, t.Max())
|
||||
fmt.Fprintf(w, `%f,`, t.Mean())
|
||||
fmt.Fprintf(w, `%f,`, t.StdDev())
|
||||
fmt.Fprintf(w, `%f,`, p[0])
|
||||
fmt.Fprintf(w, `%f,`, p[1])
|
||||
fmt.Fprintf(w, `%f,`, p[2])
|
||||
fmt.Fprintf(w, `%f,`, p[3])
|
||||
fmt.Fprintf(w, `%f,`, p[4])
|
||||
fmt.Fprintf(w, `%f,`, t.Rate1())
|
||||
fmt.Fprintf(w, `%f,`, t.Rate5())
|
||||
fmt.Fprintf(w, `%f,`, t.Rate15())
|
||||
fmt.Fprintf(w, `%f`, t.RateMean())
|
||||
}
|
60
index/upsidedown/store/metrics/writer.go
Normal file
60
index/upsidedown/store/metrics/writer.go
Normal file
|
@ -0,0 +1,60 @@
|
|||
// 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 metrics
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
s *Store
|
||||
o store.KVWriter
|
||||
}
|
||||
|
||||
func (w *Writer) Close() error {
|
||||
err := w.o.Close()
|
||||
if err != nil {
|
||||
w.s.AddError("Writer.Close", err, nil)
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatch() store.KVBatch {
|
||||
return &Batch{s: w.s, o: w.o.NewBatch()}
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatchEx(options store.KVBatchOptions) ([]byte, store.KVBatch, error) {
|
||||
buf, b, err := w.o.NewBatchEx(options)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
return buf, &Batch{s: w.s, o: b}, nil
|
||||
}
|
||||
|
||||
func (w *Writer) ExecuteBatch(b store.KVBatch) (err error) {
|
||||
batch, ok := b.(*Batch)
|
||||
if !ok {
|
||||
return fmt.Errorf("wrong type of batch")
|
||||
}
|
||||
w.s.timerWriterExecuteBatch.Time(func() {
|
||||
err = w.o.ExecuteBatch(batch.o)
|
||||
if err != nil {
|
||||
w.s.AddError("Writer.ExecuteBatch", err, nil)
|
||||
}
|
||||
})
|
||||
return
|
||||
}
|
87
index/upsidedown/store/moss/batch.go
Normal file
87
index/upsidedown/store/moss/batch.go
Normal file
|
@ -0,0 +1,87 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"github.com/couchbase/moss"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Batch struct {
|
||||
store *Store
|
||||
merge *store.EmulatedMerge
|
||||
batch moss.Batch
|
||||
buf []byte // Non-nil when using pre-alloc'ed / NewBatchEx().
|
||||
bufUsed int
|
||||
}
|
||||
|
||||
func (b *Batch) Set(key, val []byte) {
|
||||
var err error
|
||||
if b.buf != nil {
|
||||
b.bufUsed += len(key) + len(val)
|
||||
err = b.batch.AllocSet(key, val)
|
||||
} else {
|
||||
err = b.batch.Set(key, val)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
b.store.Logf("bleve moss batch.Set err: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (b *Batch) Delete(key []byte) {
|
||||
var err error
|
||||
if b.buf != nil {
|
||||
b.bufUsed += len(key)
|
||||
err = b.batch.AllocDel(key)
|
||||
} else {
|
||||
err = b.batch.Del(key)
|
||||
}
|
||||
|
||||
if err != nil {
|
||||
b.store.Logf("bleve moss batch.Delete err: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
func (b *Batch) Merge(key, val []byte) {
|
||||
if b.buf != nil {
|
||||
b.bufUsed += len(key) + len(val)
|
||||
}
|
||||
b.merge.Merge(key, val)
|
||||
}
|
||||
|
||||
func (b *Batch) Reset() {
|
||||
err := b.Close()
|
||||
if err != nil {
|
||||
b.store.Logf("bleve moss batch.Close err: %v", err)
|
||||
return
|
||||
}
|
||||
|
||||
batch, err := b.store.ms.NewBatch(0, 0)
|
||||
if err == nil {
|
||||
b.batch = batch
|
||||
b.merge = store.NewEmulatedMerge(b.store.mo)
|
||||
b.buf = nil
|
||||
b.bufUsed = 0
|
||||
}
|
||||
}
|
||||
|
||||
func (b *Batch) Close() error {
|
||||
b.merge = nil
|
||||
err := b.batch.Close()
|
||||
b.batch = nil
|
||||
return err
|
||||
}
|
87
index/upsidedown/store/moss/iterator.go
Normal file
87
index/upsidedown/store/moss/iterator.go
Normal file
|
@ -0,0 +1,87 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"github.com/couchbase/moss"
|
||||
)
|
||||
|
||||
type Iterator struct {
|
||||
store *Store
|
||||
ss moss.Snapshot
|
||||
iter moss.Iterator
|
||||
start []byte
|
||||
end []byte
|
||||
k []byte
|
||||
v []byte
|
||||
err error
|
||||
}
|
||||
|
||||
func (x *Iterator) Seek(seekToKey []byte) {
|
||||
_ = x.iter.SeekTo(seekToKey)
|
||||
|
||||
x.k, x.v, x.err = x.iter.Current()
|
||||
}
|
||||
|
||||
func (x *Iterator) Next() {
|
||||
_ = x.iter.Next()
|
||||
|
||||
x.k, x.v, x.err = x.iter.Current()
|
||||
}
|
||||
|
||||
func (x *Iterator) Current() ([]byte, []byte, bool) {
|
||||
return x.k, x.v, x.err == nil
|
||||
}
|
||||
|
||||
func (x *Iterator) Key() []byte {
|
||||
if x.err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return x.k
|
||||
}
|
||||
|
||||
func (x *Iterator) Value() []byte {
|
||||
if x.err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return x.v
|
||||
}
|
||||
|
||||
func (x *Iterator) Valid() bool {
|
||||
return x.err == nil
|
||||
}
|
||||
|
||||
func (x *Iterator) Close() error {
|
||||
var err error
|
||||
|
||||
x.ss = nil
|
||||
|
||||
if x.iter != nil {
|
||||
err = x.iter.Close()
|
||||
x.iter = nil
|
||||
}
|
||||
|
||||
x.k = nil
|
||||
x.v = nil
|
||||
x.err = moss.ErrIteratorDone
|
||||
|
||||
return err
|
||||
}
|
||||
|
||||
func (x *Iterator) current() {
|
||||
x.k, x.v, x.err = x.iter.Current()
|
||||
}
|
571
index/upsidedown/store/moss/lower.go
Normal file
571
index/upsidedown/store/moss/lower.go
Normal file
|
@ -0,0 +1,571 @@
|
|||
// Copyright (c) 2016 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 moss provides a KVStore implementation based on the
|
||||
// github.com/couchbase/moss library.
|
||||
|
||||
package moss
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"os"
|
||||
"sync"
|
||||
|
||||
"github.com/couchbase/moss"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
func initLowerLevelStore(
|
||||
config map[string]interface{},
|
||||
lowerLevelStoreName string,
|
||||
lowerLevelStoreConfig map[string]interface{},
|
||||
lowerLevelMaxBatchSize uint64,
|
||||
options moss.CollectionOptions,
|
||||
) (moss.Snapshot, moss.LowerLevelUpdate, store.KVStore, statsFunc, error) {
|
||||
if lowerLevelStoreConfig == nil {
|
||||
lowerLevelStoreConfig = map[string]interface{}{}
|
||||
}
|
||||
|
||||
for k, v := range config {
|
||||
_, exists := lowerLevelStoreConfig[k]
|
||||
if !exists {
|
||||
lowerLevelStoreConfig[k] = v
|
||||
}
|
||||
}
|
||||
|
||||
if lowerLevelStoreName == "mossStore" {
|
||||
return InitMossStore(lowerLevelStoreConfig, options)
|
||||
}
|
||||
|
||||
constructor := registry.KVStoreConstructorByName(lowerLevelStoreName)
|
||||
if constructor == nil {
|
||||
return nil, nil, nil, nil, fmt.Errorf("moss store, initLowerLevelStore,"+
|
||||
" could not find lower level store: %s", lowerLevelStoreName)
|
||||
}
|
||||
|
||||
kvStore, err := constructor(options.MergeOperator, lowerLevelStoreConfig)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
llStore := &llStore{
|
||||
refs: 0,
|
||||
config: config,
|
||||
llConfig: lowerLevelStoreConfig,
|
||||
kvStore: kvStore,
|
||||
logf: options.Log,
|
||||
}
|
||||
|
||||
llUpdate := func(ssHigher moss.Snapshot) (ssLower moss.Snapshot, err error) {
|
||||
return llStore.update(ssHigher, lowerLevelMaxBatchSize)
|
||||
}
|
||||
|
||||
llSnapshot, err := llUpdate(nil)
|
||||
if err != nil {
|
||||
_ = kvStore.Close()
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
return llSnapshot, llUpdate, kvStore, nil, nil // llStore.refs is now 1.
|
||||
}
|
||||
|
||||
// ------------------------------------------------
|
||||
|
||||
// llStore is a lower level store and provides ref-counting around a
|
||||
// bleve store.KVStore.
|
||||
type llStore struct {
|
||||
kvStore store.KVStore
|
||||
|
||||
config map[string]interface{}
|
||||
llConfig map[string]interface{}
|
||||
|
||||
logf func(format string, a ...interface{})
|
||||
|
||||
m sync.Mutex // Protects fields that follow.
|
||||
refs int
|
||||
}
|
||||
|
||||
// llSnapshot represents a lower-level snapshot, wrapping a bleve
|
||||
// store.KVReader, and implements the moss.Snapshot interface.
|
||||
type llSnapshot struct {
|
||||
llStore *llStore // Holds 1 refs on the llStore.
|
||||
kvReader store.KVReader
|
||||
childSnapshots map[string]*llSnapshot
|
||||
|
||||
m sync.Mutex // Protects fields that follow.
|
||||
refs int
|
||||
}
|
||||
|
||||
// llIterator represents a lower-level iterator, wrapping a bleve
|
||||
// store.KVIterator, and implements the moss.Iterator interface.
|
||||
type llIterator struct {
|
||||
llSnapshot *llSnapshot // Holds 1 refs on the llSnapshot.
|
||||
|
||||
// Some lower-level KVReader implementations need a separate
|
||||
// KVReader clone, due to KVReader single-threaded'ness.
|
||||
kvReader store.KVReader
|
||||
|
||||
kvIterator store.KVIterator
|
||||
}
|
||||
|
||||
type readerSource interface {
|
||||
Reader() (store.KVReader, error)
|
||||
}
|
||||
|
||||
// ------------------------------------------------
|
||||
|
||||
func (s *llStore) addRef() *llStore {
|
||||
s.m.Lock()
|
||||
s.refs += 1
|
||||
s.m.Unlock()
|
||||
|
||||
return s
|
||||
}
|
||||
|
||||
func (s *llStore) decRef() {
|
||||
s.m.Lock()
|
||||
s.refs -= 1
|
||||
if s.refs <= 0 {
|
||||
err := s.kvStore.Close()
|
||||
if err != nil {
|
||||
s.logf("llStore kvStore.Close err: %v", err)
|
||||
}
|
||||
}
|
||||
s.m.Unlock()
|
||||
}
|
||||
|
||||
// update() mutates this lower level store with latest data from the
|
||||
// given higher level moss.Snapshot and returns a new moss.Snapshot
|
||||
// that the higher level can use which represents this lower level
|
||||
// store.
|
||||
func (s *llStore) update(ssHigher moss.Snapshot, maxBatchSize uint64) (
|
||||
ssLower moss.Snapshot, err error,
|
||||
) {
|
||||
if ssHigher != nil {
|
||||
iter, err := ssHigher.StartIterator(nil, nil, moss.IteratorOptions{
|
||||
IncludeDeletions: true,
|
||||
SkipLowerLevel: true,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer func() {
|
||||
err = iter.Close()
|
||||
if err != nil {
|
||||
s.logf("llStore iter.Close err: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
kvWriter, err := s.kvStore.Writer()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
defer func() {
|
||||
err = kvWriter.Close()
|
||||
if err != nil {
|
||||
s.logf("llStore kvWriter.Close err: %v", err)
|
||||
}
|
||||
}()
|
||||
|
||||
batch := kvWriter.NewBatch()
|
||||
|
||||
defer func() {
|
||||
if batch != nil {
|
||||
err = batch.Close()
|
||||
if err != nil {
|
||||
s.logf("llStore batch.Close err: %v", err)
|
||||
}
|
||||
}
|
||||
}()
|
||||
|
||||
var readOptions moss.ReadOptions
|
||||
|
||||
i := uint64(0)
|
||||
for {
|
||||
if i%1000000 == 0 {
|
||||
s.logf("llStore.update, i: %d", i)
|
||||
}
|
||||
|
||||
ex, key, val, err := iter.CurrentEx()
|
||||
if err == moss.ErrIteratorDone {
|
||||
break
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
switch ex.Operation {
|
||||
case moss.OperationSet:
|
||||
batch.Set(key, val)
|
||||
|
||||
case moss.OperationDel:
|
||||
batch.Delete(key)
|
||||
|
||||
case moss.OperationMerge:
|
||||
val, err = ssHigher.Get(key, readOptions)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if val != nil {
|
||||
batch.Set(key, val)
|
||||
} else {
|
||||
batch.Delete(key)
|
||||
}
|
||||
|
||||
default:
|
||||
return nil, fmt.Errorf("moss store, update,"+
|
||||
" unexpected operation, ex: %v", ex)
|
||||
}
|
||||
|
||||
i++
|
||||
|
||||
err = iter.Next()
|
||||
if err == moss.ErrIteratorDone {
|
||||
break
|
||||
}
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
if maxBatchSize > 0 && i%maxBatchSize == 0 {
|
||||
err = kvWriter.ExecuteBatch(batch)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = batch.Close()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
batch = kvWriter.NewBatch()
|
||||
}
|
||||
}
|
||||
|
||||
if i > 0 {
|
||||
s.logf("llStore.update, ExecuteBatch,"+
|
||||
" path: %s, total: %d, start", s.llConfig["path"], i)
|
||||
|
||||
err = kvWriter.ExecuteBatch(batch)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
s.logf("llStore.update, ExecuteBatch,"+
|
||||
" path: %s: total: %d, done", s.llConfig["path"], i)
|
||||
}
|
||||
}
|
||||
|
||||
kvReader, err := s.kvStore.Reader()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
s.logf("llStore.update, new reader")
|
||||
|
||||
return &llSnapshot{
|
||||
llStore: s.addRef(),
|
||||
kvReader: kvReader,
|
||||
refs: 1,
|
||||
}, nil
|
||||
}
|
||||
|
||||
// ------------------------------------------------
|
||||
|
||||
func (llss *llSnapshot) addRef() *llSnapshot {
|
||||
llss.m.Lock()
|
||||
llss.refs += 1
|
||||
llss.m.Unlock()
|
||||
|
||||
return llss
|
||||
}
|
||||
|
||||
func (llss *llSnapshot) decRef() {
|
||||
llss.m.Lock()
|
||||
llss.refs -= 1
|
||||
if llss.refs <= 0 {
|
||||
if llss.kvReader != nil {
|
||||
err := llss.kvReader.Close()
|
||||
if err != nil {
|
||||
llss.llStore.logf("llSnapshot kvReader.Close err: %v", err)
|
||||
}
|
||||
|
||||
llss.kvReader = nil
|
||||
}
|
||||
|
||||
if llss.llStore != nil {
|
||||
llss.llStore.decRef()
|
||||
llss.llStore = nil
|
||||
}
|
||||
}
|
||||
llss.m.Unlock()
|
||||
}
|
||||
|
||||
// ChildCollectionNames returns an array of child collection name strings.
|
||||
func (llss *llSnapshot) ChildCollectionNames() ([]string, error) {
|
||||
childCollections := make([]string, len(llss.childSnapshots))
|
||||
idx := 0
|
||||
for name := range llss.childSnapshots {
|
||||
childCollections[idx] = name
|
||||
idx++
|
||||
}
|
||||
return childCollections, nil
|
||||
}
|
||||
|
||||
// ChildCollectionSnapshot returns a Snapshot on a given child
|
||||
// collection by its name.
|
||||
func (llss *llSnapshot) ChildCollectionSnapshot(childCollectionName string) (
|
||||
moss.Snapshot, error,
|
||||
) {
|
||||
childSnapshot, exists := llss.childSnapshots[childCollectionName]
|
||||
if !exists {
|
||||
return nil, nil
|
||||
}
|
||||
childSnapshot.addRef()
|
||||
return childSnapshot, nil
|
||||
}
|
||||
|
||||
func (llss *llSnapshot) Close() error {
|
||||
llss.decRef()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (llss *llSnapshot) Get(key []byte,
|
||||
readOptions moss.ReadOptions,
|
||||
) ([]byte, error) {
|
||||
rs, ok := llss.kvReader.(readerSource)
|
||||
if ok {
|
||||
r2, err := rs.Reader()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
val, err := r2.Get(key)
|
||||
|
||||
_ = r2.Close()
|
||||
|
||||
return val, err
|
||||
}
|
||||
|
||||
return llss.kvReader.Get(key)
|
||||
}
|
||||
|
||||
func (llss *llSnapshot) StartIterator(
|
||||
startKeyInclusive, endKeyExclusive []byte,
|
||||
iteratorOptions moss.IteratorOptions,
|
||||
) (moss.Iterator, error) {
|
||||
rs, ok := llss.kvReader.(readerSource)
|
||||
if ok {
|
||||
r2, err := rs.Reader()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
i2 := r2.RangeIterator(startKeyInclusive, endKeyExclusive)
|
||||
|
||||
return &llIterator{llSnapshot: llss.addRef(), kvReader: r2, kvIterator: i2}, nil
|
||||
}
|
||||
|
||||
i := llss.kvReader.RangeIterator(startKeyInclusive, endKeyExclusive)
|
||||
|
||||
return &llIterator{llSnapshot: llss.addRef(), kvReader: nil, kvIterator: i}, nil
|
||||
}
|
||||
|
||||
// ------------------------------------------------
|
||||
|
||||
func (lli *llIterator) Close() error {
|
||||
var err0 error
|
||||
if lli.kvIterator != nil {
|
||||
err0 = lli.kvIterator.Close()
|
||||
lli.kvIterator = nil
|
||||
}
|
||||
|
||||
var err1 error
|
||||
if lli.kvReader != nil {
|
||||
err1 = lli.kvReader.Close()
|
||||
lli.kvReader = nil
|
||||
}
|
||||
|
||||
lli.llSnapshot.decRef()
|
||||
lli.llSnapshot = nil
|
||||
|
||||
if err0 != nil {
|
||||
return err0
|
||||
}
|
||||
|
||||
if err1 != nil {
|
||||
return err1
|
||||
}
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lli *llIterator) Next() error {
|
||||
lli.kvIterator.Next()
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lli *llIterator) SeekTo(k []byte) error {
|
||||
lli.kvIterator.Seek(k)
|
||||
|
||||
return nil
|
||||
}
|
||||
|
||||
func (lli *llIterator) Current() (key, val []byte, err error) {
|
||||
key, val, ok := lli.kvIterator.Current()
|
||||
if !ok {
|
||||
return nil, nil, moss.ErrIteratorDone
|
||||
}
|
||||
|
||||
return key, val, nil
|
||||
}
|
||||
|
||||
func (lli *llIterator) CurrentEx() (
|
||||
entryEx moss.EntryEx, key, val []byte, err error,
|
||||
) {
|
||||
return moss.EntryEx{}, nil, nil, moss.ErrUnimplemented
|
||||
}
|
||||
|
||||
// ------------------------------------------------
|
||||
|
||||
func InitMossStore(config map[string]interface{}, options moss.CollectionOptions) (
|
||||
moss.Snapshot, moss.LowerLevelUpdate, store.KVStore, statsFunc, error,
|
||||
) {
|
||||
path, ok := config["path"].(string)
|
||||
if !ok {
|
||||
return nil, nil, nil, nil, fmt.Errorf("lower: missing path for InitMossStore config")
|
||||
}
|
||||
if path == "" {
|
||||
return nil, nil, nil, nil, os.ErrInvalid
|
||||
}
|
||||
|
||||
err := os.MkdirAll(path, 0o700)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, fmt.Errorf("lower: InitMossStore mkdir,"+
|
||||
" path: %s, err: %v", path, err)
|
||||
}
|
||||
|
||||
storeOptions := moss.StoreOptions{
|
||||
CollectionOptions: options,
|
||||
}
|
||||
v, ok := config["mossStoreOptions"]
|
||||
if ok {
|
||||
b, err := util.MarshalJSON(v) // Convert from map[string]interface{}.
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
err = util.UnmarshalJSON(b, &storeOptions)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
}
|
||||
|
||||
s, err := moss.OpenStore(path, storeOptions)
|
||||
if err != nil {
|
||||
return nil, nil, nil, nil, fmt.Errorf("lower: moss.OpenStore,"+
|
||||
" path: %s, err: %v", path, err)
|
||||
}
|
||||
|
||||
sw := &mossStoreWrapper{s: s}
|
||||
|
||||
llUpdate := func(ssHigher moss.Snapshot) (moss.Snapshot, error) {
|
||||
ss, err := sw.s.Persist(ssHigher, moss.StorePersistOptions{
|
||||
CompactionConcern: moss.CompactionAllow,
|
||||
})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
sw.AddRef() // Ref-count to be owned by snapshot wrapper.
|
||||
|
||||
return moss.NewSnapshotWrapper(ss, sw), nil
|
||||
}
|
||||
|
||||
llSnapshot, err := llUpdate(nil)
|
||||
if err != nil {
|
||||
_ = s.Close()
|
||||
return nil, nil, nil, nil, err
|
||||
}
|
||||
|
||||
llStats := func() map[string]interface{} {
|
||||
stats, err := s.Stats()
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
return stats
|
||||
}
|
||||
|
||||
return llSnapshot, llUpdate, sw, llStats, nil
|
||||
}
|
||||
|
||||
// mossStoreWrapper implements the bleve.index.store.KVStore
|
||||
// interface, but only barely enough to allow it to be passed around
|
||||
// as a lower-level store. Advanced apps will likely cast the
|
||||
// mossStoreWrapper to access the Actual() method.
|
||||
type mossStoreWrapper struct {
|
||||
m sync.Mutex
|
||||
refs int
|
||||
s *moss.Store
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) AddRef() {
|
||||
w.m.Lock()
|
||||
w.refs++
|
||||
w.m.Unlock()
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) Close() (err error) {
|
||||
w.m.Lock()
|
||||
w.refs--
|
||||
if w.refs <= 0 {
|
||||
err = w.s.Close()
|
||||
w.s = nil
|
||||
}
|
||||
w.m.Unlock()
|
||||
return err
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) Reader() (store.KVReader, error) {
|
||||
return nil, fmt.Errorf("unexpected")
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) Writer() (store.KVWriter, error) {
|
||||
return nil, fmt.Errorf("unexpected")
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) Actual() *moss.Store {
|
||||
w.m.Lock()
|
||||
rv := w.s
|
||||
w.m.Unlock()
|
||||
return rv
|
||||
}
|
||||
|
||||
func (w *mossStoreWrapper) histograms() string {
|
||||
var rv string
|
||||
w.m.Lock()
|
||||
if w.s != nil {
|
||||
rv = w.s.Histograms().String()
|
||||
}
|
||||
w.m.Unlock()
|
||||
return rv
|
||||
}
|
103
index/upsidedown/store/moss/lower_test.go
Normal file
103
index/upsidedown/store/moss/lower_test.go
Normal file
|
@ -0,0 +1,103 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"os"
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
)
|
||||
|
||||
func openWithLower(t *testing.T, mo store.MergeOperator) (string, store.KVStore) {
|
||||
tmpDir, _ := os.MkdirTemp("", "mossStore")
|
||||
|
||||
config := map[string]interface{}{
|
||||
"path": tmpDir,
|
||||
"mossLowerLevelStoreName": "mossStore",
|
||||
}
|
||||
|
||||
rv, err := New(mo, config)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return tmpDir, rv
|
||||
}
|
||||
|
||||
func cleanupWithLower(t *testing.T, s store.KVStore, tmpDir string) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
err = os.RemoveAll(tmpDir)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMossWithLowerKVCrud(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerReaderIsolation(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerReaderOwnsGetBytes(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerWriterOwnsBytes(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerPrefixIterator(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerPrefixIteratorSeek(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerRangeIterator(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerRangeIteratorSeek(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, nil)
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMossWithLowerMerge(t *testing.T) {
|
||||
tmpDir, s := openWithLower(t, &test.TestMergeCounter{})
|
||||
defer cleanupWithLower(t, s, tmpDir)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
97
index/upsidedown/store/moss/reader.go
Normal file
97
index/upsidedown/store/moss/reader.go
Normal file
|
@ -0,0 +1,97 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"github.com/couchbase/moss"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type Reader struct {
|
||||
store *Store
|
||||
ss moss.Snapshot
|
||||
}
|
||||
|
||||
func (r *Reader) Get(k []byte) (v []byte, err error) {
|
||||
v, err = r.ss.Get(k, moss.ReadOptions{})
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if v != nil {
|
||||
return append(make([]byte, 0, len(v)), v...), nil
|
||||
}
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (r *Reader) MultiGet(keys [][]byte) ([][]byte, error) {
|
||||
return store.MultiGet(r, keys)
|
||||
}
|
||||
|
||||
func (r *Reader) PrefixIterator(k []byte) store.KVIterator {
|
||||
kEnd := incrementBytes(k)
|
||||
|
||||
iter, err := r.ss.StartIterator(k, kEnd, moss.IteratorOptions{})
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
rv := &Iterator{
|
||||
store: r.store,
|
||||
ss: r.ss,
|
||||
iter: iter,
|
||||
start: k,
|
||||
end: kEnd,
|
||||
}
|
||||
|
||||
rv.current()
|
||||
|
||||
return rv
|
||||
}
|
||||
|
||||
func (r *Reader) RangeIterator(start, end []byte) store.KVIterator {
|
||||
iter, err := r.ss.StartIterator(start, end, moss.IteratorOptions{})
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
rv := &Iterator{
|
||||
store: r.store,
|
||||
ss: r.ss,
|
||||
iter: iter,
|
||||
start: start,
|
||||
end: end,
|
||||
}
|
||||
|
||||
rv.current()
|
||||
|
||||
return rv
|
||||
}
|
||||
|
||||
func (r *Reader) Close() error {
|
||||
return r.ss.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 {
|
||||
return rv // didn't overflow, so stop
|
||||
}
|
||||
}
|
||||
return nil // overflowed
|
||||
}
|
58
index/upsidedown/store/moss/stats.go
Normal file
58
index/upsidedown/store/moss/stats.go
Normal file
|
@ -0,0 +1,58 @@
|
|||
// 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 moss
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
type stats struct {
|
||||
s *Store
|
||||
}
|
||||
|
||||
func (s *stats) statsMap() map[string]interface{} {
|
||||
ms := map[string]interface{}{}
|
||||
|
||||
var err error
|
||||
ms["moss"], err = s.s.ms.Stats()
|
||||
if err != nil {
|
||||
return ms
|
||||
}
|
||||
|
||||
if s.s.llstore != nil {
|
||||
if o, ok := s.s.llstore.(store.KVStoreStats); ok {
|
||||
ms["kv"] = o.StatsMap()
|
||||
}
|
||||
}
|
||||
|
||||
_, exists := ms["kv"]
|
||||
if !exists && s.s.llstats != nil {
|
||||
ms["kv"] = s.s.llstats()
|
||||
}
|
||||
|
||||
if msw, ok := s.s.llstore.(*mossStoreWrapper); ok {
|
||||
ms["store_histograms"] = msw.histograms()
|
||||
}
|
||||
|
||||
ms["coll_histograms"] = s.s.ms.Histograms().String()
|
||||
|
||||
return ms
|
||||
}
|
||||
|
||||
func (s *stats) MarshalJSON() ([]byte, error) {
|
||||
m := s.statsMap()
|
||||
return util.MarshalJSON(m)
|
||||
}
|
231
index/upsidedown/store/moss/store.go
Normal file
231
index/upsidedown/store/moss/store.go
Normal file
|
@ -0,0 +1,231 @@
|
|||
// Copyright (c) 2016 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 moss provides a KVStore implementation based on the
|
||||
// github.com/couchbase/moss library.
|
||||
|
||||
package moss
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/couchbase/moss"
|
||||
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
"github.com/blevesearch/bleve/v2/util"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
// RegistryCollectionOptions should be treated as read-only after
|
||||
// process init()'ialization.
|
||||
var RegistryCollectionOptions = map[string]moss.CollectionOptions{}
|
||||
|
||||
const Name = "moss"
|
||||
|
||||
type Store struct {
|
||||
m sync.Mutex
|
||||
ms moss.Collection
|
||||
mo store.MergeOperator
|
||||
llstore store.KVStore // May be nil.
|
||||
llstats statsFunc // May be nil.
|
||||
|
||||
s *stats
|
||||
config map[string]interface{}
|
||||
}
|
||||
|
||||
type statsFunc func() map[string]interface{}
|
||||
|
||||
// New initializes a moss storage with values from the optional
|
||||
// config["mossCollectionOptions"] (a JSON moss.CollectionOptions).
|
||||
// Next, values from the RegistryCollectionOptions, named by the
|
||||
// optional config["mossCollectionOptionsName"], take precedence.
|
||||
// Finally, base case defaults are taken from
|
||||
// moss.DefaultCollectionOptions.
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (
|
||||
store.KVStore, error) {
|
||||
options := moss.DefaultCollectionOptions // Copy.
|
||||
|
||||
v, ok := config["mossCollectionOptionsName"]
|
||||
if ok {
|
||||
name, ok := v.(string)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not parse config[mossCollectionOptionsName]: %v", v)
|
||||
}
|
||||
|
||||
options, ok = RegistryCollectionOptions[name] // Copy.
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not find RegistryCollectionOptions, name: %s", name)
|
||||
}
|
||||
}
|
||||
|
||||
options.MergeOperator = mo
|
||||
options.DeferredSort = true
|
||||
|
||||
v, ok = config["mossCollectionOptions"]
|
||||
if ok {
|
||||
b, err := util.MarshalJSON(v) // Convert from map[string]interface{}.
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not marshal config[mossCollectionOptions]: %v, err: %v", v, err)
|
||||
}
|
||||
|
||||
err = util.UnmarshalJSON(b, &options)
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not unmarshal config[mossCollectionOptions]: %v, err: %v", v, err)
|
||||
}
|
||||
}
|
||||
|
||||
// --------------------------------------------------
|
||||
|
||||
if options.Log == nil || options.Debug <= 0 {
|
||||
options.Log = func(format string, a ...interface{}) {}
|
||||
}
|
||||
|
||||
// --------------------------------------------------
|
||||
|
||||
mossLowerLevelStoreName := ""
|
||||
v, ok = config["mossLowerLevelStoreName"]
|
||||
if ok {
|
||||
mossLowerLevelStoreName, ok = v.(string)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not parse config[mossLowerLevelStoreName]: %v", v)
|
||||
}
|
||||
}
|
||||
|
||||
var llStore store.KVStore
|
||||
var llStats statsFunc
|
||||
|
||||
if options.LowerLevelInit == nil &&
|
||||
options.LowerLevelUpdate == nil &&
|
||||
mossLowerLevelStoreName != "" {
|
||||
mossLowerLevelStoreConfig := map[string]interface{}{}
|
||||
v, ok := config["mossLowerLevelStoreConfig"]
|
||||
if ok {
|
||||
mossLowerLevelStoreConfig, ok = v.(map[string]interface{})
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("moss store, initLowerLevelStore,"+
|
||||
" could parse mossLowerLevelStoreConfig: %v", v)
|
||||
}
|
||||
}
|
||||
|
||||
mossLowerLevelMaxBatchSize := uint64(0)
|
||||
v, ok = config["mossLowerLevelMaxBatchSize"]
|
||||
if ok {
|
||||
mossLowerLevelMaxBatchSizeF, ok := v.(float64)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("moss store,"+
|
||||
" could not parse config[mossLowerLevelMaxBatchSize]: %v", v)
|
||||
}
|
||||
|
||||
mossLowerLevelMaxBatchSize = uint64(mossLowerLevelMaxBatchSizeF)
|
||||
}
|
||||
|
||||
lowerLevelInit, lowerLevelUpdate, lowerLevelStore, lowerLevelStats, err :=
|
||||
initLowerLevelStore(config,
|
||||
mossLowerLevelStoreName,
|
||||
mossLowerLevelStoreConfig,
|
||||
mossLowerLevelMaxBatchSize,
|
||||
options)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
options.LowerLevelInit = lowerLevelInit
|
||||
options.LowerLevelUpdate = lowerLevelUpdate
|
||||
|
||||
llStore = lowerLevelStore
|
||||
llStats = lowerLevelStats
|
||||
}
|
||||
|
||||
// --------------------------------------------------
|
||||
|
||||
ms, err := moss.NewCollection(options)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
err = ms.Start()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
rv := Store{
|
||||
ms: ms,
|
||||
mo: mo,
|
||||
llstore: llStore,
|
||||
llstats: llStats,
|
||||
config: config,
|
||||
}
|
||||
rv.s = &stats{s: &rv}
|
||||
return &rv, nil
|
||||
}
|
||||
|
||||
func (s *Store) Close() error {
|
||||
if val, ok := s.config["mossAbortCloseEnabled"]; ok {
|
||||
if v, ok := val.(bool); ok && v {
|
||||
if msw, ok := s.llstore.(*mossStoreWrapper); ok {
|
||||
if s := msw.Actual(); s != nil {
|
||||
_ = s.CloseEx(moss.StoreCloseExOptions{Abort: true})
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
return s.ms.Close()
|
||||
}
|
||||
|
||||
func (s *Store) Reader() (store.KVReader, error) {
|
||||
ss, err := s.ms.Snapshot()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
return &Reader{ss: ss}, nil
|
||||
}
|
||||
|
||||
func (s *Store) Writer() (store.KVWriter, error) {
|
||||
return &Writer{s: s}, nil
|
||||
}
|
||||
|
||||
func (s *Store) Logf(fmt string, args ...interface{}) {
|
||||
options := s.ms.Options()
|
||||
if options.Log != nil {
|
||||
options.Log(fmt, args...)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Store) Stats() json.Marshaler {
|
||||
return s.s
|
||||
}
|
||||
|
||||
func (s *Store) StatsMap() map[string]interface{} {
|
||||
return s.s.statsMap()
|
||||
}
|
||||
|
||||
func (s *Store) LowerLevelStore() store.KVStore {
|
||||
return s.llstore
|
||||
}
|
||||
|
||||
func (s *Store) Collection() moss.Collection {
|
||||
return s.ms
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
91
index/upsidedown/store/moss/store_test.go
Normal file
91
index/upsidedown/store/moss/store_test.go
Normal file
|
@ -0,0 +1,91 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
"github.com/blevesearch/upsidedown_store_api/test"
|
||||
)
|
||||
|
||||
func open(t *testing.T, mo store.MergeOperator) store.KVStore {
|
||||
rv, err := New(mo, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
return rv
|
||||
}
|
||||
|
||||
func cleanup(t *testing.T, s store.KVStore) {
|
||||
err := s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
||||
|
||||
func TestMossKVCrud(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestKVCrud(t, s)
|
||||
}
|
||||
|
||||
func TestMossReaderIsolation(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderIsolation(t, s)
|
||||
}
|
||||
|
||||
func TestMossReaderOwnsGetBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestReaderOwnsGetBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMossWriterOwnsBytes(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestWriterOwnsBytes(t, s)
|
||||
}
|
||||
|
||||
func TestMossPrefixIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMossPrefixIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestPrefixIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMossRangeIterator(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIterator(t, s)
|
||||
}
|
||||
|
||||
func TestMossRangeIteratorSeek(t *testing.T) {
|
||||
s := open(t, nil)
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestRangeIteratorSeek(t, s)
|
||||
}
|
||||
|
||||
func TestMossMerge(t *testing.T) {
|
||||
s := open(t, &test.TestMergeCounter{})
|
||||
defer cleanup(t, s)
|
||||
test.CommonTestMerge(t, s)
|
||||
}
|
97
index/upsidedown/store/moss/writer.go
Normal file
97
index/upsidedown/store/moss/writer.go
Normal file
|
@ -0,0 +1,97 @@
|
|||
// Copyright (c) 2016 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 moss
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
|
||||
"github.com/couchbase/moss"
|
||||
)
|
||||
|
||||
type Writer struct {
|
||||
s *Store
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatch() store.KVBatch {
|
||||
b, err := w.s.ms.NewBatch(0, 0)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
return &Batch{
|
||||
store: w.s,
|
||||
merge: store.NewEmulatedMerge(w.s.mo),
|
||||
batch: b,
|
||||
}
|
||||
}
|
||||
|
||||
func (w *Writer) NewBatchEx(options store.KVBatchOptions) (
|
||||
[]byte, store.KVBatch, error) {
|
||||
numOps := options.NumSets + options.NumDeletes + options.NumMerges
|
||||
|
||||
b, err := w.s.ms.NewBatch(numOps, options.TotalBytes)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
buf, err := b.Alloc(options.TotalBytes)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
}
|
||||
|
||||
return buf, &Batch{
|
||||
store: w.s,
|
||||
merge: store.NewEmulatedMerge(w.s.mo),
|
||||
batch: b,
|
||||
buf: buf,
|
||||
bufUsed: 0,
|
||||
}, nil
|
||||
}
|
||||
|
||||
func (w *Writer) ExecuteBatch(b store.KVBatch) (err error) {
|
||||
batch, ok := b.(*Batch)
|
||||
if !ok {
|
||||
return fmt.Errorf("wrong type of batch")
|
||||
}
|
||||
|
||||
for kStr, mergeOps := range batch.merge.Merges {
|
||||
for _, v := range mergeOps {
|
||||
if batch.buf != nil {
|
||||
kLen := len(kStr)
|
||||
vLen := len(v)
|
||||
kBuf := batch.buf[batch.bufUsed : batch.bufUsed+kLen]
|
||||
vBuf := batch.buf[batch.bufUsed+kLen : batch.bufUsed+kLen+vLen]
|
||||
copy(kBuf, kStr)
|
||||
copy(vBuf, v)
|
||||
batch.bufUsed += kLen + vLen
|
||||
err = batch.batch.AllocMerge(kBuf, vBuf)
|
||||
} else {
|
||||
err = batch.batch.Merge([]byte(kStr), v)
|
||||
}
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return w.s.ms.ExecuteBatch(batch.batch, moss.WriteOptions{})
|
||||
}
|
||||
|
||||
func (w *Writer) Close() error {
|
||||
w.s = nil
|
||||
return nil
|
||||
}
|
121
index/upsidedown/store/null/null.go
Normal file
121
index/upsidedown/store/null/null.go
Normal file
|
@ -0,0 +1,121 @@
|
|||
// 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 null
|
||||
|
||||
import (
|
||||
"github.com/blevesearch/bleve/v2/registry"
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
const Name = "null"
|
||||
|
||||
type Store struct{}
|
||||
|
||||
func New(mo store.MergeOperator, config map[string]interface{}) (store.KVStore, error) {
|
||||
return &Store{}, nil
|
||||
}
|
||||
|
||||
func (i *Store) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *Store) Reader() (store.KVReader, error) {
|
||||
return &reader{}, nil
|
||||
}
|
||||
|
||||
func (i *Store) Writer() (store.KVWriter, error) {
|
||||
return &writer{}, nil
|
||||
}
|
||||
|
||||
type reader struct{}
|
||||
|
||||
func (r *reader) Get(key []byte) ([]byte, error) {
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
func (r *reader) MultiGet(keys [][]byte) ([][]byte, error) {
|
||||
return make([][]byte, len(keys)), nil
|
||||
}
|
||||
|
||||
func (r *reader) PrefixIterator(prefix []byte) store.KVIterator {
|
||||
return &iterator{}
|
||||
}
|
||||
|
||||
func (r *reader) RangeIterator(start, end []byte) store.KVIterator {
|
||||
return &iterator{}
|
||||
}
|
||||
|
||||
func (r *reader) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
type iterator struct{}
|
||||
|
||||
func (i *iterator) SeekFirst() {}
|
||||
func (i *iterator) Seek(k []byte) {}
|
||||
func (i *iterator) Next() {}
|
||||
|
||||
func (i *iterator) Current() ([]byte, []byte, bool) {
|
||||
return nil, nil, false
|
||||
}
|
||||
|
||||
func (i *iterator) Key() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *iterator) Value() []byte {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (i *iterator) Valid() bool {
|
||||
return false
|
||||
}
|
||||
|
||||
func (i *iterator) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
type batch struct{}
|
||||
|
||||
func (i *batch) Set(key, val []byte) {}
|
||||
func (i *batch) Delete(key []byte) {}
|
||||
func (i *batch) Merge(key, val []byte) {}
|
||||
func (i *batch) Reset() {}
|
||||
func (i *batch) Close() error { return nil }
|
||||
|
||||
type writer struct{}
|
||||
|
||||
func (w *writer) NewBatch() store.KVBatch {
|
||||
return &batch{}
|
||||
}
|
||||
|
||||
func (w *writer) NewBatchEx(options store.KVBatchOptions) ([]byte, store.KVBatch, error) {
|
||||
return make([]byte, options.TotalBytes), w.NewBatch(), nil
|
||||
}
|
||||
|
||||
func (w *writer) ExecuteBatch(store.KVBatch) error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (w *writer) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func init() {
|
||||
err := registry.RegisterKVStore(Name, New)
|
||||
if err != nil {
|
||||
panic(err)
|
||||
}
|
||||
}
|
92
index/upsidedown/store/null/null_test.go
Normal file
92
index/upsidedown/store/null/null_test.go
Normal file
|
@ -0,0 +1,92 @@
|
|||
// 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 null
|
||||
|
||||
import (
|
||||
"testing"
|
||||
|
||||
store "github.com/blevesearch/upsidedown_store_api"
|
||||
)
|
||||
|
||||
func TestStore(t *testing.T) {
|
||||
s, err := New(nil, nil)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
NullTestKVStore(t, s)
|
||||
}
|
||||
|
||||
// NullTestKVStore has very different expectations
|
||||
// compared to CommonTestKVStore
|
||||
func NullTestKVStore(t *testing.T, s store.KVStore) {
|
||||
|
||||
writer, err := s.Writer()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
|
||||
batch := writer.NewBatch()
|
||||
batch.Set([]byte("b"), []byte("val-b"))
|
||||
batch.Set([]byte("c"), []byte("val-c"))
|
||||
batch.Set([]byte("d"), []byte("val-d"))
|
||||
batch.Set([]byte("e"), []byte("val-e"))
|
||||
batch.Set([]byte("f"), []byte("val-f"))
|
||||
batch.Set([]byte("g"), []byte("val-g"))
|
||||
batch.Set([]byte("h"), []byte("val-h"))
|
||||
batch.Set([]byte("i"), []byte("val-i"))
|
||||
batch.Set([]byte("j"), []byte("val-j"))
|
||||
|
||||
err = writer.ExecuteBatch(batch)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
err = writer.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
reader, err := s.Reader()
|
||||
if err != nil {
|
||||
t.Error(err)
|
||||
}
|
||||
defer func() {
|
||||
err := reader.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}()
|
||||
it := reader.RangeIterator([]byte("b"), nil)
|
||||
key, val, valid := it.Current()
|
||||
if valid {
|
||||
t.Fatalf("valid true, expected false")
|
||||
}
|
||||
if key != nil {
|
||||
t.Fatalf("expected key nil, got %s", key)
|
||||
}
|
||||
if val != nil {
|
||||
t.Fatalf("expected value nil, got %s", val)
|
||||
}
|
||||
|
||||
err = it.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
||||
err = s.Close()
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
}
|
Loading…
Add table
Add a link
Reference in a new issue