0
0
Fork 0

Merge pull request #724 from blevesearch/scorch

Scorch
This commit is contained in:
Marty Schoch 2018-01-05 17:24:46 -05:00 committed by GitHub
commit 0456569b62
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
78 changed files with 15138 additions and 57 deletions

25
cmd/bleve/cmd/scorch.go Normal file
View File

@ -0,0 +1,25 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package cmd
import (
"github.com/blevesearch/bleve/cmd/bleve/cmd/scorch"
)
// make scorch command-line tool a bleve sub-command
func init() {
RootCmd.AddCommand(scorch.RootCmd)
}

View File

@ -0,0 +1,59 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"strconv"
"github.com/blevesearch/bleve/index/scorch/mergeplan"
"github.com/spf13/cobra"
)
// asciiCmd represents the snapshots command
var asciiCmd = &cobra.Command{
Use: "ascii",
Short: "ascii prints details an ascii representation of the snapshots in the index",
Long: `The ascii command prints an ascii representation of the snapshots in the index.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("snapshot epoch required")
} else if len(args) < 3 {
snapshotEpoch, err := strconv.ParseUint(args[1], 10, 64)
if err != nil {
return err
}
snapshot, err := index.LoadSnapshot(snapshotEpoch)
if err != nil {
return err
}
segments := snapshot.Segments()
var mergePlanSegments []mergeplan.Segment
for _, v := range segments {
mergePlanSegments = append(mergePlanSegments, v)
}
str := mergeplan.ToBarChart(args[1], 25, mergePlanSegments, nil)
fmt.Printf("%s\n", str)
}
return nil
},
}
func init() {
RootCmd.AddCommand(asciiCmd)
}

View File

@ -0,0 +1,55 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"strconv"
"github.com/spf13/cobra"
)
// deletedCmd represents the deleted command
var deletedCmd = &cobra.Command{
Use: "deleted",
Short: "deleted prints the deleted bitmap for segments in the index snapshot",
Long: `The delete command prints the deleted bitmap for segments in the index snapshot.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("snapshot epoch required")
} else if len(args) < 3 {
snapshotEpoch, err := strconv.ParseUint(args[1], 10, 64)
if err != nil {
return err
}
snapshot, err := index.LoadSnapshot(snapshotEpoch)
if err != nil {
return err
}
segments := snapshot.Segments()
for i, segmentSnap := range segments {
deleted := segmentSnap.Deleted()
fmt.Printf("%d %v\n", i, deleted)
}
}
return nil
},
}
func init() {
RootCmd.AddCommand(deletedCmd)
}

View File

@ -0,0 +1,59 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"github.com/spf13/cobra"
)
// dictCmd represents the dict command
var infoCmd = &cobra.Command{
Use: "info",
Short: "info prints basic info about the index",
Long: `The info command prints basic info about the index.`,
RunE: func(cmd *cobra.Command, args []string) error {
reader, err := index.Reader()
if err != nil {
return err
}
count, err := reader.DocCount()
if err != nil {
return err
}
fmt.Printf("count: %d\n", count)
// var numSnapshots int
// var rootSnapshot uint64
// index.VisitBoltSnapshots(func(snapshotEpoch uint64) error {
// if rootSnapshot == 0 {
// rootSnapshot = snapshotEpoch
// }
// numSnapshots++
// return nil
// })
// fmt.Printf("has %d snapshot(s), root: %d\n", numSnapshots, rootSnapshot)
return nil
},
}
func init() {
RootCmd.AddCommand(infoCmd)
}

View File

@ -0,0 +1,61 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"strconv"
"github.com/spf13/cobra"
)
var ascii bool
// internalCmd represents the snapshots command
var internalCmd = &cobra.Command{
Use: "internal",
Short: "internal prints the internal k/v pairs in a snapshot",
Long: `The internal command prints the internal k/v pairs in a snapshot.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("snapshot epoch required")
} else if len(args) < 3 {
snapshotEpoch, err := strconv.ParseUint(args[1], 10, 64)
if err != nil {
return err
}
snapshot, err := index.LoadSnapshot(snapshotEpoch)
if err != nil {
return err
}
internal := snapshot.Internal()
for k, v := range internal {
if ascii {
fmt.Printf("%s %s\n", k, string(v))
} else {
fmt.Printf("%x %x\n", k, v)
}
}
}
return nil
},
}
func init() {
RootCmd.AddCommand(internalCmd)
internalCmd.Flags().BoolVarP(&ascii, "ascii", "a", false, "print key/value in ascii")
}

View File

@ -0,0 +1,70 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"os"
"github.com/blevesearch/bleve/index/scorch"
"github.com/spf13/cobra"
)
var index *scorch.Scorch
// RootCmd represents the base command when called without any subcommands
var RootCmd = &cobra.Command{
Use: "scorch",
Short: "command-line tool to interact with a scorch index",
Long: `Scorch is a command-line tool to interact with a scorch index.`,
PersistentPreRunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 1 {
return fmt.Errorf("must specify path to scorch index")
}
readOnly := true
config := map[string]interface{}{
"read_only": readOnly,
"path": args[0],
}
idx, err := scorch.NewScorch(scorch.Name, config, nil)
if err != nil {
return err
}
err = idx.Open()
if err != nil {
return fmt.Errorf("error opening: %v", err)
}
index = idx.(*scorch.Scorch)
return nil
},
PersistentPostRunE: func(cmd *cobra.Command, args []string) error {
return nil
},
}
// Execute adds all child commands to the root command sets flags appropriately.
// This is called by main.main(). It only needs to happen once to the rootCmd.
func Execute() {
if err := RootCmd.Execute(); err != nil {
fmt.Println(err)
os.Exit(-1)
}
}

View File

@ -0,0 +1,64 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"strconv"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
"github.com/spf13/cobra"
)
// snapshotCmd represents the snapshot command
var snapshotCmd = &cobra.Command{
Use: "snapshot",
Short: "info prints details about the snapshots in the index",
Long: `The snapshot command prints details about the snapshots in the index.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
snapshotEpochs, err := index.RootBoltSnapshotEpochs()
if err != nil {
return err
}
for _, snapshotEpoch := range snapshotEpochs {
fmt.Printf("%d\n", snapshotEpoch)
}
} else if len(args) < 3 {
snapshotEpoch, err := strconv.ParseUint(args[1], 10, 64)
if err != nil {
return err
}
snapshot, err := index.LoadSnapshot(snapshotEpoch)
if err != nil {
return err
}
segments := snapshot.Segments()
for i, segmentSnap := range segments {
segment := segmentSnap.Segment()
if segment, ok := segment.(*zap.Segment); ok {
fmt.Printf("%d %s\n", i, segment.Path())
}
}
}
return nil
},
}
func init() {
RootCmd.AddCommand(snapshotCmd)
}

25
cmd/bleve/cmd/zap.go Normal file
View File

@ -0,0 +1,25 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package cmd
import (
"github.com/blevesearch/bleve/cmd/bleve/cmd/zap"
)
// make zap command-line tool a bleve sub-command
func init() {
RootCmd.AddCommand(zap.RootCmd)
}

72
cmd/bleve/cmd/zap/dict.go Normal file
View File

@ -0,0 +1,72 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"encoding/binary"
"fmt"
"github.com/couchbase/vellum"
"github.com/spf13/cobra"
)
// dictCmd represents the dict command
var dictCmd = &cobra.Command{
Use: "dict [path] [field]",
Short: "dict prints the term dictionary for the specified field",
Long: `The dict command lets you print the term dictionary for the specified field.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("must specify field")
}
data := segment.Data()
addr, err := segment.DictAddr(args[1])
if err != nil {
return fmt.Errorf("error determing address: %v", err)
}
fmt.Printf("dictionary for field starts at %d (%x)\n", addr, addr)
vellumLen, read := binary.Uvarint(data[addr : addr+binary.MaxVarintLen64])
fmt.Printf("vellum length: %d\n", vellumLen)
fstBytes := data[addr+uint64(read) : addr+uint64(read)+vellumLen]
fmt.Printf("raw vellum data % x\n", fstBytes)
fmt.Printf("dictionary:\n\n")
if fstBytes != nil {
fst, err := vellum.Load(fstBytes)
if err != nil {
return fmt.Errorf("dictionary field %s vellum err: %v", args[1], err)
}
itr, err := fst.Iterator(nil, nil)
for err == nil {
currTerm, currVal := itr.Current()
fmt.Printf("%s - %d (%x)\n", currTerm, currVal, currVal)
err = itr.Next()
}
if err != nil && err != vellum.ErrIteratorDone {
return fmt.Errorf("error iterating dictionary: %v", err)
}
}
return nil
},
}
func init() {
RootCmd.AddCommand(dictCmd)
}

View File

@ -0,0 +1,274 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"fmt"
"log"
"math"
"sort"
"strconv"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
"github.com/golang/snappy"
"github.com/spf13/cobra"
)
// docvalueCmd represents the docvalue command
var docvalueCmd = &cobra.Command{
Use: "docvalue [path] <field> optional <docNum> optional",
Short: "docvalue prints the docvalue details by field, and docNum",
Long: `The docvalue command lets you explore the docValues in order of field and by doc number.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 1 {
return fmt.Errorf("must specify index file path")
}
data := segment.Data()
crcOffset := len(data) - 4
verOffset := crcOffset - 4
chunkOffset := verOffset - 4
fieldsOffset := chunkOffset - 16
fieldsIndexOffset := binary.BigEndian.Uint64(data[fieldsOffset : fieldsOffset+8])
fieldsIndexEnd := uint64(len(data) - zap.FooterSize)
// iterate through fields index
var fieldInv []string
var id, read, fieldLoc uint64
var nread int
for fieldsIndexOffset+(8*id) < fieldsIndexEnd {
addr := binary.BigEndian.Uint64(data[fieldsIndexOffset+(8*id) : fieldsIndexOffset+(8*id)+8])
var n uint64
_, read := binary.Uvarint(data[addr+n : fieldsIndexEnd])
n += uint64(read)
var nameLen uint64
nameLen, read = binary.Uvarint(data[addr+n : fieldsIndexEnd])
n += uint64(read)
name := string(data[addr+n : addr+n+nameLen])
id++
fieldInv = append(fieldInv, name)
}
dvLoc := segment.DocValueOffset()
fieldDvLoc, total, fdvread := uint64(0), uint64(0), int(0)
var fieldName string
var fieldID uint16
// if no fields are specified then print the docValue offsets for all fields set
for id, field := range fieldInv {
fieldLoc, fdvread = binary.Uvarint(data[dvLoc+read : dvLoc+read+binary.MaxVarintLen64])
if fdvread <= 0 {
return fmt.Errorf("loadDvIterators: failed to read the docvalue offsets for field %d", fieldID)
}
read += uint64(fdvread)
if fieldLoc == math.MaxUint64 {
fmt.Printf("fieldID: %d '%s' docvalue at %d (%x) not persisted \n", id, field, fieldLoc, fieldLoc)
continue
}
var offset, clen, numChunks uint64
numChunks, nread = binary.Uvarint(data[fieldLoc : fieldLoc+binary.MaxVarintLen64])
if nread <= 0 {
return fmt.Errorf("failed to read the field "+
"doc values for field %s", fieldName)
}
offset += uint64(nread)
// read the length of chunks
totalSize := uint64(0)
chunkLens := make([]uint64, numChunks)
for i := 0; i < int(numChunks); i++ {
clen, nread = binary.Uvarint(data[fieldLoc+offset : fieldLoc+offset+binary.MaxVarintLen64])
if nread <= 0 {
return fmt.Errorf("corrupted chunk length for chunk number: %d", i)
}
chunkLens[i] = clen
totalSize += clen
offset += uint64(nread)
}
total += totalSize
if len(args) == 1 {
// if no field args are given, then print out the dv locations for all fields
mbsize := float64(totalSize) / (1024 * 1024)
fmt.Printf("fieldID: %d '%s' docvalue at %d (%x) numChunks %d diskSize %.3f MB\n", id, field, fieldLoc, fieldLoc, numChunks, mbsize)
continue
}
if field != args[1] {
continue
} else {
fieldDvLoc = fieldLoc
fieldName = field
fieldID = uint16(id)
}
}
mbsize := float64(total) / (1024 * 1024)
fmt.Printf("Total Doc Values Size on Disk: %.3f MB\n", mbsize)
// done with the fields dv locs printing for the given zap file
if len(args) == 1 {
return nil
}
if fieldName == "" || fieldDvLoc == 0 {
return fmt.Errorf("no field found for given field arg: %s", args[1])
}
// read the number of chunks
var offset, clen, numChunks uint64
numChunks, nread = binary.Uvarint(data[fieldDvLoc : fieldDvLoc+binary.MaxVarintLen64])
if nread <= 0 {
return fmt.Errorf("failed to read the field "+
"doc values for field %s", fieldName)
}
offset += uint64(nread)
if len(args) == 2 {
fmt.Printf("number of chunks: %d\n", numChunks)
}
// read the length of chunks
chunkLens := make([]uint64, numChunks)
for i := 0; i < int(numChunks); i++ {
clen, nread = binary.Uvarint(data[fieldDvLoc+offset : fieldDvLoc+offset+binary.MaxVarintLen64])
if nread <= 0 {
return fmt.Errorf("corrupted chunk length for chunk number: %d", i)
}
chunkLens[i] = clen
offset += uint64(nread)
if len(args) == 2 {
fmt.Printf("chunk: %d size: %d \n", i, clen)
}
/*
TODO => dump all chunk headers??
if len(args) == 3 && args[2] == ">" {
dumpChunkDocIDs(data, )
}*/
}
if len(args) == 2 {
return nil
}
localDocNum, err := strconv.Atoi(args[2])
if err != nil {
return fmt.Errorf("unable to parse doc number: %v", err)
}
if localDocNum >= int(segment.NumDocs()) {
return fmt.Errorf("invalid doc number %d (valid 0 - %d)", localDocNum, segment.NumDocs()-1)
}
// find the chunkNumber where the docValues are stored
docInChunk := uint64(localDocNum) / uint64(segment.ChunkFactor())
if numChunks < docInChunk {
return fmt.Errorf("no chunk exists for chunk number: %d for docID: %d", docInChunk, localDocNum)
}
destChunkDataLoc := fieldDvLoc + offset
for i := 0; i < int(docInChunk); i++ {
destChunkDataLoc += chunkLens[i]
}
curChunkSize := chunkLens[docInChunk]
// read the number of docs reside in the chunk
numDocs := uint64(0)
numDocs, nread = binary.Uvarint(data[destChunkDataLoc : destChunkDataLoc+binary.MaxVarintLen64])
if nread <= 0 {
return fmt.Errorf("failed to read the target chunk: %d", docInChunk)
}
chunkMetaLoc := destChunkDataLoc + uint64(nread)
offset = uint64(0)
curChunkHeader := make([]zap.MetaData, int(numDocs))
for i := 0; i < int(numDocs); i++ {
curChunkHeader[i].DocID, nread = binary.Uvarint(data[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(nread)
curChunkHeader[i].DocDvLoc, nread = binary.Uvarint(data[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(nread)
curChunkHeader[i].DocDvLen, nread = binary.Uvarint(data[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(nread)
}
compressedDataLoc := chunkMetaLoc + offset
dataLength := destChunkDataLoc + curChunkSize - compressedDataLoc
curChunkData := data[compressedDataLoc : compressedDataLoc+dataLength]
start, length := getDocValueLocs(uint64(localDocNum), curChunkHeader)
if start == math.MaxUint64 || length == math.MaxUint64 {
fmt.Printf("no field values found for docID %d\n", localDocNum)
fmt.Printf("Try docIDs present in chunk: %s\n", assortDocID(curChunkHeader))
return nil
}
// uncompress the already loaded data
uncompressed, err := snappy.Decode(nil, curChunkData)
if err != nil {
log.Printf("snappy err %+v ", err)
return err
}
var termSeparator byte = 0xff
var termSeparatorSplitSlice = []byte{termSeparator}
// pick the terms for the given docID
uncompressed = uncompressed[start : start+length]
for {
i := bytes.Index(uncompressed, termSeparatorSplitSlice)
if i < 0 {
break
}
fmt.Printf(" %s ", uncompressed[0:i])
uncompressed = uncompressed[i+1:]
}
fmt.Printf(" \n ")
return nil
},
}
func getDocValueLocs(docID uint64, metaHeader []zap.MetaData) (uint64, uint64) {
i := sort.Search(len(metaHeader), func(i int) bool {
return metaHeader[i].DocID >= docID
})
if i < len(metaHeader) && metaHeader[i].DocID == docID {
return metaHeader[i].DocDvLoc, metaHeader[i].DocDvLen
}
return math.MaxUint64, math.MaxUint64
}
func assortDocID(metaHeader []zap.MetaData) string {
docIDs := ""
for _, meta := range metaHeader {
id := fmt.Sprintf("%d", meta.DocID)
docIDs += id + ", "
}
return docIDs
}
func init() {
RootCmd.AddCommand(docvalueCmd)
}

View File

@ -0,0 +1,124 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"encoding/binary"
"fmt"
"log"
"github.com/couchbase/vellum"
"github.com/spf13/cobra"
)
// exploreCmd represents the explore command
var exploreCmd = &cobra.Command{
Use: "explore [path] [field] <term> <docNum>",
Short: "explores the index by field, then term (optional), and then docNum (optional)",
Long: `The explore command lets you explore the index in order of field, then optionally by term, then optionally again by doc number.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("must specify field")
}
data := segment.Data()
addr, err := segment.DictAddr(args[1])
if err != nil {
return fmt.Errorf("error determing address: %v", err)
}
fmt.Printf("dictionary for field starts at %d (%x)\n", addr, addr)
vellumLen, read := binary.Uvarint(data[addr : addr+binary.MaxVarintLen64])
fmt.Printf("vellum length: %d\n", vellumLen)
fstBytes := data[addr+uint64(read) : addr+uint64(read)+vellumLen]
fmt.Printf("raw vellum data % x\n", fstBytes)
if len(args) >= 3 {
if fstBytes != nil {
fst, err := vellum.Load(fstBytes)
if err != nil {
return fmt.Errorf("dictionary field %s vellum err: %v", args[1], err)
}
postingsAddr, exists, err := fst.Get([]byte(args[2]))
if err != nil {
return fmt.Errorf("error looking for term : %v", err)
}
if exists {
fmt.Printf("postings list begins at %d (%x)\n", postingsAddr, postingsAddr)
var n uint64
freqAddr, read := binary.Uvarint(data[postingsAddr : postingsAddr+binary.MaxVarintLen64])
n += uint64(read)
var locAddr uint64
locAddr, read = binary.Uvarint(data[postingsAddr+n : postingsAddr+n+binary.MaxVarintLen64])
n += uint64(read)
var postingListLen uint64
postingListLen, _ = binary.Uvarint(data[postingsAddr+n : postingsAddr+n+binary.MaxVarintLen64])
fmt.Printf("Posting List Length: %d\n", postingListLen)
fmt.Printf("Freq details at: %d (%x)\n", freqAddr, freqAddr)
numChunks, r2 := binary.Uvarint(data[freqAddr : freqAddr+binary.MaxVarintLen64])
n = uint64(r2)
var freqOffsets []uint64
for j := uint64(0); j < numChunks; j++ {
chunkLen, r3 := binary.Uvarint(data[freqAddr+n : freqAddr+n+binary.MaxVarintLen64])
n += uint64(r3)
freqOffsets = append(freqOffsets, chunkLen)
}
running := freqAddr + n
for k, offset := range freqOffsets {
fmt.Printf("freq chunk: %d, len %d, start at %d (%x) end %d (%x)\n", k, offset, running, running, running+offset, running+offset)
running += offset
}
fmt.Printf("Loc details at: %d (%x)\n", locAddr, locAddr)
numLChunks, r4 := binary.Uvarint(data[locAddr : locAddr+binary.MaxVarintLen64])
n = uint64(r4)
fmt.Printf("there are %d loc chunks\n", numLChunks)
var locOffsets []uint64
for j := uint64(0); j < numLChunks; j++ {
log.Printf("reading from %d(%x)\n", locAddr+n, locAddr+n)
log.Printf("data i see here: % x\n", data[locAddr+n:locAddr+n+binary.MaxVarintLen64])
lchunkLen, r4 := binary.Uvarint(data[locAddr+n : locAddr+n+binary.MaxVarintLen64])
n += uint64(r4)
log.Printf("see chunk len %d(%x)\n", lchunkLen, lchunkLen)
locOffsets = append(locOffsets, lchunkLen)
}
running2 := locAddr + n
for k, offset := range locOffsets {
fmt.Printf("loc chunk: %d, len %d(%x), start at %d (%x) end %d (%x)\n", k, offset, offset, running2, running2, running2+offset, running2+offset)
running2 += offset
}
} else {
fmt.Printf("dictionary does not contain term '%s'\n", args[2])
}
}
}
return nil
},
}
func init() {
RootCmd.AddCommand(exploreCmd)
}

View File

@ -0,0 +1,66 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"encoding/binary"
"fmt"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
"github.com/spf13/cobra"
)
// fieldsCmd represents the fields command
var fieldsCmd = &cobra.Command{
Use: "fields [path]",
Short: "fields prints the fields in the specified file",
Long: `The fields command lets you print the fields in the specified file.`,
RunE: func(cmd *cobra.Command, args []string) error {
data := segment.Data()
crcOffset := len(data) - 4
verOffset := crcOffset - 4
chunkOffset := verOffset - 4
fieldsOffset := chunkOffset - 16
fieldsIndexOffset := binary.BigEndian.Uint64(data[fieldsOffset : fieldsOffset+8])
fieldsIndexEnd := uint64(len(data) - zap.FooterSize)
// iterate through fields index
var fieldID uint64
for fieldsIndexOffset+(8*fieldID) < fieldsIndexEnd {
addr := binary.BigEndian.Uint64(data[fieldsIndexOffset+(8*fieldID) : fieldsIndexOffset+(8*fieldID)+8])
var n uint64
dictLoc, read := binary.Uvarint(data[addr+n : fieldsIndexEnd])
n += uint64(read)
var nameLen uint64
nameLen, read = binary.Uvarint(data[addr+n : fieldsIndexEnd])
n += uint64(read)
name := string(data[addr+n : addr+n+nameLen])
fmt.Printf("field %d '%s' starts at %d (%x)\n", fieldID, name, dictLoc, dictLoc)
fieldID++
}
return nil
},
}
func init() {
RootCmd.AddCommand(fieldsCmd)
}

View File

@ -0,0 +1,44 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"fmt"
"github.com/spf13/cobra"
)
// footerCmd represents the footer command
var footerCmd = &cobra.Command{
Use: "footer [path]",
Short: "prints the contents of the zap footer",
Long: `The footer command will print the contents of the footer.`,
RunE: func(cmd *cobra.Command, args []string) error {
data := segment.Data()
fmt.Printf("Length: %d\n", len(data))
fmt.Printf("CRC: %#x\n", segment.CRC())
fmt.Printf("Version: %d\n", segment.Version())
fmt.Printf("Chunk Factor: %d\n", segment.ChunkFactor())
fmt.Printf("Fields Idx: %d (%#x)\n", segment.FieldsIndexOffset(), segment.FieldsIndexOffset())
fmt.Printf("Stored Idx: %d (%#x)\n", segment.StoredIndexOffset(), segment.StoredIndexOffset())
fmt.Printf("DocValue Idx: %d (%#x)\n", segment.DocValueOffset(), segment.DocValueOffset())
fmt.Printf("Num Docs: %d\n", segment.NumDocs())
return nil
},
}
func init() {
RootCmd.AddCommand(footerCmd)
}

58
cmd/bleve/cmd/zap/root.go Normal file
View File

@ -0,0 +1,58 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"fmt"
"os"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
"github.com/spf13/cobra"
)
var segment *zap.Segment
// RootCmd represents the base command when called without any subcommands
var RootCmd = &cobra.Command{
Use: "zap",
Short: "command-line tool to interact with a zap file",
Long: `Zap is a command-line tool to interact with a zap file.`,
PersistentPreRunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 1 {
return fmt.Errorf("must specify path to zap file")
}
segInf, err := zap.Open(args[0])
if err != nil {
return fmt.Errorf("error opening zap file: %v", err)
}
segment = segInf.(*zap.Segment)
return nil
},
PersistentPostRunE: func(cmd *cobra.Command, args []string) error {
return nil
},
}
// Execute adds all child commands to the root command sets flags appropriately.
// This is called by main.main(). It only needs to happen once to the rootCmd.
func Execute() {
if err := RootCmd.Execute(); err != nil {
fmt.Println(err)
os.Exit(-1)
}
}

View File

@ -0,0 +1,73 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"encoding/binary"
"fmt"
"strconv"
"github.com/golang/snappy"
"github.com/spf13/cobra"
)
// storedCmd represents the stored command
var storedCmd = &cobra.Command{
Use: "stored [path] [docNum]",
Short: "prints the stored section for a doc number",
Long: `The stored command will print the raw stored data bytes for the specified document number.`,
RunE: func(cmd *cobra.Command, args []string) error {
if len(args) < 2 {
return fmt.Errorf("must specify doc number")
}
docNum, err := strconv.Atoi(args[1])
if err != nil {
return fmt.Errorf("unable to parse doc number: %v", err)
}
if docNum >= int(segment.NumDocs()) {
return fmt.Errorf("invalid doc number %d (valid 0 - %d)", docNum, segment.NumDocs()-1)
}
data := segment.Data()
storedIdx := segment.StoredIndexOffset()
// read docNum entry in the index
indexPos := storedIdx + (8 * uint64(docNum))
storedStartAddr := binary.BigEndian.Uint64(data[indexPos : indexPos+8])
fmt.Printf("Stored field starts at %d (%#x)\n", storedStartAddr, storedStartAddr)
var n uint64
metaLen, read := binary.Uvarint(data[storedStartAddr : storedStartAddr+binary.MaxVarintLen64])
n += uint64(read)
fmt.Printf("Meta Len: %d\n", metaLen)
var dataLen uint64
dataLen, read = binary.Uvarint(data[storedStartAddr+n : storedStartAddr+n+binary.MaxVarintLen64])
n += uint64(read)
fmt.Printf("Data Len: %d\n", dataLen)
meta := data[storedStartAddr+n : storedStartAddr+n+metaLen]
fmt.Printf("Raw meta: % x\n", meta)
raw := data[storedStartAddr+n+metaLen : storedStartAddr+n+metaLen+dataLen]
fmt.Printf("Raw data (len %d): % x\n", len(raw), raw)
uncompressed, err := snappy.Decode(nil, raw)
if err != nil {
panic(err)
}
fmt.Printf("Uncompressed data (len %d): % x\n", len(uncompressed), uncompressed)
return nil
},
}
func init() {
RootCmd.AddCommand(storedCmd)
}

View File

@ -25,6 +25,9 @@ import (
"github.com/blevesearch/bleve/index/upsidedown"
"github.com/blevesearch/bleve/registry"
"github.com/blevesearch/bleve/search/highlight/highlighter/html"
// force import of scorch so its accessible by default
_ "github.com/blevesearch/bleve/index/scorch"
)
var bleveExpVar = expvar.NewMap("bleve")

View File

@ -500,33 +500,6 @@ func TestHandlers(t *testing.T) {
Status: http.StatusNotFound,
ResponseBody: []byte(`no such index 'tix'`),
},
{
Desc: "debug doc",
Handler: debugHandler,
Path: "/ti1/a/debug",
Method: "GET",
Params: url.Values{
"indexName": []string{"ti1"},
"docID": []string{"a"},
},
Status: http.StatusOK,
ResponseMatch: map[string]bool{
`"key"`: true,
`"val"`: true,
},
},
{
Desc: "debug doc invalid index",
Handler: debugHandler,
Path: "/ti1/a/debug",
Method: "GET",
Params: url.Values{
"indexName": []string{"tix"},
"docID": []string{"a"},
},
Status: http.StatusNotFound,
ResponseBody: []byte(`no such index 'tix'`),
},
{
Desc: "create alias",
Handler: aliasHandler,

View File

@ -14,7 +14,10 @@
package index
import "github.com/blevesearch/bleve/document"
import (
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
)
type IndexRow interface {
KeySize() int
@ -29,6 +32,11 @@ type IndexRow interface {
type AnalysisResult struct {
DocID string
Rows []IndexRow
// scorch
Document *document.Document
Analyzed []analysis.TokenFrequencies
Length []int
}
type AnalysisWork struct {

510
index/scorch/README.md Normal file
View File

@ -0,0 +1,510 @@
# scorch
## Definitions
Batch
- A collection of Documents to mutate in the index.
Document
- Has a unique identifier (arbitrary bytes).
- Is comprised of a list of fields.
Field
- Has a name (string).
- Has a type (text, number, date, geopoint).
- Has a value (depending on type).
- Can be indexed, stored, or both.
- If indexed, can be analyzed.
-m If indexed, can optionally store term vectors.
## Scope
Scorch *MUST* implement the bleve.index API without requiring any changes to this API.
Scorch *MAY* introduce new interfaces, which can be discovered to allow use of new capabilities not in the current API.
## Implementation
The scorch implementation starts with the concept of a segmented index.
A segment is simply a slice, subset, or portion of the entire index. A segmented index is one which is composed of one or more segments. Although segments are created in a particular order, knowing this ordering is not required to achieve correct semantics when querying. Because there is no ordering, this means that when searching an index, you can (and should) search all the segments concurrently.
### Internal Wrapper
In order to accommodate the existing APIs while also improving the implementation, the scorch implementation includes some wrapper functionality that must be described.
#### \_id field
In scorch, field 0 is prearranged to be named \_id. All documents have a value for this field, which is the documents external identifier. In this version the field *MUST* be both indexed AND stored. The scorch wrapper adds this field, as it will not be present in the Document from the calling bleve code.
NOTE: If a document already contains a field \_id, it will be replaced. If this is problematic, the caller must ensure such a scenario does not happen.
### Proposed Structures
```
type Segment interface {
Dictionary(field string) TermDictionary
}
type TermDictionary interface {
PostingsList(term string, excluding PostingsList) PostingsList
}
type PostingsList interface {
Next() Posting
And(other PostingsList) PostingsList
Or(other PostingsList) PostingsList
}
type Posting interface {
Number() uint64
Frequency() uint64
Norm() float64
Locations() Locations
}
type Locations interface {
Start() uint64
End() uint64
Pos() uint64
ArrayPositions() ...
}
type DeletedDocs {
}
type SegmentSnapshot struct {
segment Segment
deleted PostingsList
}
type IndexSnapshot struct {
segment []SegmentSnapshot
}
```
**What about errors?**
**What about memory mgmnt or context?**
**Postings List separate iterator to separate stateful from stateless**
### Mutating the Index
The bleve.index API has methods for directly making individual mutations (Update/Delete/SetInternal/DeleteInternal), however for this first implementation, we assume that all of these calls can simply be turned into a Batch of size 1. This may be highly inefficient, but it will be correct. This decision is made based on the fact that Couchbase FTS always uses Batches.
NOTE: As a side-effect of this decision, it should be clear that performance tuning may depend on the batch size, which may in-turn require changes in FTS.
From this point forward, only Batch mutations will be discussed.
Sequence of Operations:
1. For each document in the batch, search through all existing segments. The goal is to build up a per-segment bitset which tells us which documents in that segment are obsoleted by the addition of the new segment we're currently building. NOTE: we're not ready for this change to take effect yet, so rather than this operation mutating anything, they simply return bitsets, which we can apply later. Logically, this is something like:
```
foreach segment {
dict := segment.Dictionary("\_id")
postings := empty postings list
foreach docID {
postings = postings.Or(dict.PostingsList(docID, nil))
}
}
```
NOTE: it is illustrated above as nested for loops, but some or all of these could be concurrently. The end result is that for each segment, we have (possibly empty) bitset.
2. Also concurrent with 1, the documents in the batch are analyzed. This analysis proceeds using the existing analyzer pool.
3. (after 2 completes) Analyzed documents are fed into a function which builds a new Segment representing this information.
4. We now have everything we need to update the state of the system to include this new snapshot.
- Acquire a lock
- Create a new IndexSnapshot
- For each SegmentSnapshot in the IndexSnapshot, take the deleted PostingsList and OR it with the new postings list for this Segment. Construct a new SegmentSnapshot for the segment using this new deleted PostingsList. Append this SegmentSnapshot to the IndexSnapshot.
- Create a new SegmentSnapshot wrapping our new segment with nil deleted docs.
- Append the new SegmentSnapshot to the IndexSnapshot
- Release the lock
An ASCII art example:
```
0 - Empty Index
No segments
IndexSnapshot
segments []
deleted []
1 - Index Batch [ A B C ]
segment 0
numbers [ 1 2 3 ]
\_id [ A B C ]
IndexSnapshot
segments [ 0 ]
deleted [ nil ]
2 - Index Batch [ B' ]
segment 0 1
numbers [ 1 2 3 ] [ 1 ]
\_id [ A B C ] [ B ]
Compute bitset segment-0-deleted-by-1:
[ 0 1 0 ]
OR it with previous (nil) (call it 0-1)
[ 0 1 0 ]
IndexSnapshot
segments [ 0 1 ]
deleted [ 0-1 nil ]
3 - Index Batch [ C' ]
segment 0 1 2
numbers [ 1 2 3 ] [ 1 ] [ 1 ]
\_id [ A B C ] [ B ] [ C ]
Compute bitset segment-0-deleted-by-2:
[ 0 0 1 ]
OR it with previous ([ 0 1 0 ]) (call it 0-12)
[ 0 1 1 ]
Compute bitset segment-1-deleted-by-2:
[ 0 ]
OR it with previous (nil)
still just nil
IndexSnapshot
segments [ 0 1 2 ]
deleted [ 0-12 nil nil ]
```
**is there opportunity to stop early when doc is found in one segment**
**also, more efficient way to find bits for long lists of ids?**
### Searching
In the bleve.index API all searching starts by getting an IndexReader, which represents a snapshot of the index at a point in time.
As described in the section above, our index implementation maintains a pointer to the current IndexSnapshot. When a caller gets an IndexReader, they get a copy of this pointer, and can use it as long as they like. The IndexSnapshot contains SegmentSnapshots, which only contain pointers to immutable segments. The deleted posting lists associated with a segment change over time, but the particular deleted posting list in YOUR snapshot is immutable. This gives a stable view of the data.
#### Term Search
Term search is the only searching primitive exposed in today's bleve.index API. This ultimately could limit our ability to take advantage of the indexing improvements, but it also means it will be easier to get a first version of this working.
A term search for term T in field F will look something like this:
```
searchResultPostings = empty
foreach segment {
dict := segment.Dictionary(F)
segmentResultPostings = dict.PostingsList(T, segmentSnapshotDeleted)
// make segmentLocal numbers into global numbers, and flip bits in searchResultPostings
}
```
The searchResultPostings will be a new implementation of the TermFieldReader inteface.
As a reminder this interface is:
```
// TermFieldReader is the interface exposing the enumeration of documents
// containing a given term in a given field. Documents are returned in byte
// lexicographic order over their identifiers.
type TermFieldReader interface {
// Next returns the next document containing the term in this field, or nil
// when it reaches the end of the enumeration. The preAlloced TermFieldDoc
// is optional, and when non-nil, will be used instead of allocating memory.
Next(preAlloced *TermFieldDoc) (*TermFieldDoc, error)
// Advance resets the enumeration at specified document or its immediate
// follower.
Advance(ID IndexInternalID, preAlloced *TermFieldDoc) (*TermFieldDoc, error)
// Count returns the number of documents contains the term in this field.
Count() uint64
Close() error
}
```
At first glance this appears problematic, we have no way to return documents in order of their identifiers. But it turns out the wording of this perhaps too strong, or a bit ambiguous. Originally, this referred to the external identifiers, but with the introduction of a distinction between internal/external identifiers, returning them in order of their internal identifiers is also acceptable. **ASIDE**: the reason for this is that most callers just use Next() and literally don't care what the order is, they could be in any order and it would be fine. There is only one search that cares and that is the ConjunctionSearcher, which relies on Next/Advance having very specific semantics. Later in this document we will have a proposal to split into multiple interfaces:
- The weakest interface, only supports Next() no ordering at all.
- Ordered, supporting Advance()
- And/Or'able capable of internally efficiently doing these ops with like interfaces (if not capable then can always fall back to external walking)
But, the good news is that we don't even have to do that for our first implementation. As long as the global numbers we use for internal identifiers are consistent within this IndexSnapshot, then Next() will be ordered by ascending document number, and Advance() will still work correctly.
NOTE: there is another place where we rely on the ordering of these hits, and that is in the "\_id" sort order. Previously this was the natural order, and a NOOP for the collector, now it must be implemented by actually sorting on the "\_id" field. We probably should introduce at least a marker interface to detect this.
An ASCII art example:
```
Let's start with the IndexSnapshot we ended with earlier:
3 - Index Batch [ C' ]
segment 0 1 2
numbers [ 1 2 3 ] [ 1 ] [ 1 ]
\_id [ A B C ] [ B ] [ C ]
Compute bitset segment-0-deleted-by-2:
[ 0 0 1 ]
OR it with previous ([ 0 1 0 ]) (call it 0-12)
[ 0 1 1 ]
Compute bitset segment-1-deleted-by-2:
[ 0 0 0 ]
OR it with previous (nil)
still just nil
IndexSnapshot
segments [ 0 1 2 ]
deleted [ 0-12 nil nil ]
Now let's search for the term 'cat' in the field 'desc' and let's assume that Document C (both versions) would match it.
Concurrently:
- Segment 0
- Get Term Dictionary For Field 'desc'
- From it get Postings List for term 'cat' EXCLUDING 0-12
- raw segment matches [ 0 0 1 ] but excluding [ 0 1 1 ] gives [ 0 0 0 ]
- Segment 1
- Get Term Dictionary For Field 'desc'
- From it get Postings List for term 'cat' excluding nil
- [ 0 ]
- Segment 2
- Get Term Dictionary For Field 'desc'
- From it get Postings List for term 'cat' excluding nil
- [ 1 ]
Map local bitsets into global number space (global meaning cross-segment but still unique to this snapshot)
IndexSnapshot already should have mapping something like:
0 - Offset 0
1 - Offset 3 (because segment 0 had 3 docs)
2 - Offset 4 (becuase segment 1 had 1 doc)
This maps to search result bitset:
[ 0 0 0 0 1]
Caller would call Next() and get doc number 5 (assuming 1 based indexing for now)
Caller could then ask to get term locations, stored fields, external doc ID for document number 5. Internally in the IndexSnapshot, we can now convert that back, and realize doc number 5 comes from segment 2, 5-4=1 so we're looking for doc number 1 in segment 2. That happens to be C...
```
#### Future improvements
In the future, interfaces to detect these non-serially operating TermFieldReaders could expose their own And() and Or() up to the higher level Conjunction/Disjunction searchers. Doing this alone offers some win, but also means there would be greater burden on the Searcher code rewriting logical expressions for maximum performance.
Another related topic is that of peak memory usage. With serially operating TermFieldReaders it was necessary to start them all at the same time and operate in unison. However, with these non-serially operating TermFieldReaders we have the option of doing a few at a time, consolidating them, dispoting the intermediaries, and then doing a few more. For very complex queries with many clauses this could reduce peak memory usage.
### Memory Tracking
All segments must be able to produce two statistics, an estimate of their explicit memory usage, and their actual size on disk (if any). For in-memory segments, disk usage could be zero, and the memory usage represents the entire information content. For mmap-based disk segments, the memory could be as low as the size of tracking structure itself (say just a few pointers).
This would allow the implementation to throttle or block incoming mutations when a threshold memory usage has (or would be) exceeded.
### Persistence
Obviously, we want to support (but maybe not require) asynchronous persistence of segments. My expectation is that segments are initially built in memory. At some point they are persisted to disk. This poses some interesting challenges.
At runtime, the state of an index (it's IndexSnapshot) is not only the contents of the segments, but also the bitmasks of deleted documents. These bitmasks indirectly encode an ordering in which the segments were added. The reason is that the bitmasks encode which items have been obsoleted by other (subsequent or more future) segments. In the runtime implementation we compute bitmask deltas and then merge them at the same time we bring the new segment in. One idea is that we could take a similar approach on disk. When we persist a segment, we persist the bitmask deltas of segments known to exist at that time, and eventually these can get merged up into a base segment deleted bitmask.
This also relates to the topic rollback, addressed next...
### Rollback
One desirable property in the Couchbase ecosystem is the ability to rollback to some previous (though typically not long ago) state. One idea for keeping this property in this design is to protect some of the most recent segments from merging. Then, if necessary, they could be "undone" to reveal previous states of the system. In these scenarios "undone" has to properly undo the deleted bitmasks on the other segments. Again, the current thinking is that rather than "undo" anything, it could be work that was deferred in the first place, thus making it easier to logically undo.
Another possibly related approach would be to tie this into our existing snapshot mechanism. Perhaps simulating a slow reader (holding onto index snapshots) for some period of time, can be the mechanism to achieve the desired end goal.
### Internal Storage
The bleve.index API has support for "internal storage". The ability to store information under a separate name space.
This is not used for high volume storage, so it is tempting to think we could just put a small k/v store alongside the rest of the index. But, the reality is that this storage is used to maintain key information related to the rollback scenario. Because of this, its crucial that ordering and overwriting of key/value pairs correspond with actual segment persistence in the index. Based on this, I believe its important to put the internal key/value pairs inside the segments themselves. But, this also means that they must follow a similar "deleted" bitmask approach to obsolete values in older segments. But, this also seems to substantially increase the complexity of the solution because of the separate name space, it would appear to require its own bitmask. Further keys aren't numeric, which then implies yet another mapping from internal key to number, etc.
More thought is required here.
### Merging
The segmented index approach requires merging to prevent the number of segments from growing too large.
Recent experience with LSMs has taught us that having the correct merge strategy can make a huge difference in the overall performance of the system. In particular, a simple merge strategy which merges segments too aggressively can lead to high write amplification and unnecessarily rendering cached data useless.
A few simple principles have been identified.
- Roughly we merge multiple smaller segments into a single larger one.
- The larger a segment gets the less likely we should be to ever merge it.
- Segments with large numbers of deleted/obsoleted items are good candidates as the merge will result in a space savings.
- Segments with all items deleted/obsoleted can be dropped.
Merging of a segment should be able to proceed even if that segment is held by an ongoing snapshot, it should only delay the removal of it.
## TODO
- need reference counting on the segments, to know when we can safely remove?
- how well will bitmaps perform when large and possibly mmap'd?
-----
thinking out loud on storage
- fields
- field name - field id
- term dictionary
- field id - FST (values postings ids)
- postings
- posting id - postings list
- freqs
- posting id - freqs list
- norms
- posting id - norms list
- stored
- docNum
- field id - field values
----
race dialog with steve:
state: 2, 4, 8
- introducing new segment X
- deleted bitmasks, 2, 4, 8
- merger, merge 4 and 8
new segment Y
- merger wins
state: 2, 9
introducer: need to recompute bitmask for 9, could lose again and keep losing race
- introducer wins
state: 2, 4, 8, X
2-X, 4-X, 8-X, nil
merger finishes: new segment Y, is not valid, need to be recomputed
### Bolt Segment Proposal
Bucket
"f" field storage
Key Val
field name field id (var uint16)
// TODO field location bits
"d" term dictionary storage
Key Val
field id (var uint16) Vellum FST (mapping term to posting id uint64)
"p" postings list storage
Key Val
posting id (var uint64) Roaring Bitmap Serialization (doc numbers) - see FromBuffer
"x" chunked data storage
Key Val
chunk id (var uint64) sub-bucket
Key Val
posting id (var uint64) sub-bucket
ALL Compressed Integer Encoding []uint64
Key Val
"f" freqs 1 value per hit
"n" norms 1 value per hit
"i" fields <freq> values per hit
"s" start <freq> values per hit
"e" end <freq> values per hit
"p" pos <freq> values per hit
"a" array pos
<freq> entries
each entry is count
followed by <count> uint64
"s" stored field data
Key Val
doc num (var uint64) sub-bucket
Key Val
"m" mossy-like meta packed
16 bits - field id
8 bits - field type
2? bits - array pos length
X bits - offset
X bits - length
"d" raw []byte data (possibly compressed, need segment level config?)
"a" array position info, packed slice uint64
Notes:
It is assumed that each IndexReader (snapshot) starts a new Bolt TX (read-only) immediately, and holds it up until it is no longer needed. This allows us to use (unsafely) the raw bytes coming out of BoltDB as return values. Bolt guarantees they will be safe for the duration of the transaction (which we arrange to be the life of the index snapshot).
Only physically store the fields in one direction, even though at runtime we need both. Upon opening the index, we can read in all the k/v pairs in the "f" bucket. We use the unsafe package to create a []string inverted mapping pointing to the underlying []byte in the BoltDB values.
The term dictionary is stored opaquely as Vellum FST for each field. When accessing these keys, the []byte return to us is mmap'd by bolt under the hood. We then pass this to vellum using its []byte API, which then operates on it without ever forcing whole thing into memory unless needed.
We do not need to persist the dictkeys slice since it is only there to support the dictionary iterator prefix/range searches, which are supported directly by the FST.
Theory of operation of chunked storage is as follows. The postings list iterators only allow starting at the beginning, and have no "advance" capability. In the memory version, this means we always know the Nth hit in the postings list is the Nth entry in some other densely packed slice. However, while OK when everything is in RAM, this is not as suitable for a structure on disk, where wading through detailed info of records you don't care about is too expensive. Instead, we assume some fixed chunking, say 1024. All detailed info for document number N can be found inside of chunk N/1024. Now, the Advance operation still has to Next it's way through the posting list. But, now when it reaches a hit, it knows the chunk index as well as the hit index inside that chunk. Further, we push the chunk offsets to the top of the bolt structure, under the theory that we're likely to access data inside a chunk at the same time. For example, you're likely to access the frequency and norm values for a document hit together, so by organizing by chunk first, we increase the likelihood that this info is nearby on disk.
The "f" and "n" sub-buckets inside a posting have 1 entry for each hit. (you must next-next-next within the chunk)
The "i", "s", "e", "p", sub-buckets have <freq> entries for each hit. (you must have read and know the freq)
The "a" sub-bucket has <freq> groupings, where each grouping starts with a count, followed by <count> entries.
For example, lets say hit docNum 27 has freq of 2. The first location for the hit has array positions (0, 1) length 2, and the second location for the hit has array positions (1, 3, 2) length 3. The entries in the slice for this hit look like:
2 0 1 3 1 3 2
^ ^
| next entry, number of ints to follow for it
number of ints to follow for this entry

56
index/scorch/event.go Normal file
View File

@ -0,0 +1,56 @@
// Copyright (c) 2018 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import "time"
// RegistryAsyncErrorCallbacks should be treated as read-only after
// process init()'ialization.
var RegistryAsyncErrorCallbacks = map[string]func(error){}
// RegistryEventCallbacks should be treated as read-only after
// process init()'ialization.
var RegistryEventCallbacks = map[string]func(Event){}
// Event represents the information provided in an OnEvent() callback.
type Event struct {
Kind EventKind
Scorch *Scorch
Duration time.Duration
}
// EventKind represents an event code for OnEvent() callbacks.
type EventKind int
// EventKindCloseStart is fired when a Scorch.Close() has begun.
var EventKindCloseStart = EventKind(1)
// EventKindClose is fired when a scorch index has been fully closed.
var EventKindClose = EventKind(2)
// EventKindMergerProgress is fired when the merger has completed a
// round of merge processing.
var EventKindMergerProgress = EventKind(3)
// EventKindPersisterProgress is fired when the persister has completed
// a round of persistence processing.
var EventKindPersisterProgress = EventKind(4)
// EventKindBatchIntroductionStart is fired when Batch() is invoked which
// introduces a new segment.
var EventKindBatchIntroductionStart = EventKind(5)
// EventKindBatchIntroduction is fired when Batch() completes.
var EventKindBatchIntroduction = EventKind(6)

View File

@ -0,0 +1,73 @@
// Copyright (c) 2018 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestEventBatchIntroductionStart(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
var count int
RegistryEventCallbacks["test"] = func(e Event) {
if e.Kind == EventKindBatchIntroductionStart {
count++
}
}
ourConfig := make(map[string]interface{}, len(testConfig))
for k, v := range testConfig {
ourConfig[k] = v
}
ourConfig["eventCallbackName"] = "test"
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, ourConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
doc := document.NewDocument("1")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test")))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
if count != 1 {
t.Fatalf("expected to see 1 batch introduction event event, saw %d", count)
}
}

View File

@ -0,0 +1,184 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestIndexFieldDict(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, testConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
defer func() {
cerr := idx.Close()
if cerr != nil {
t.Fatal(cerr)
}
}()
var expectedCount uint64
doc := document.NewDocument("1")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test")))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
doc = document.NewDocument("2")
doc.AddField(document.NewTextFieldWithAnalyzer("name", []uint64{}, []byte("test test test"), testAnalyzer))
doc.AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("eat more rice"), document.IndexField|document.IncludeTermVectors, testAnalyzer))
doc.AddField(document.NewTextFieldCustom("prefix", []uint64{}, []byte("bob cat cats catting dog doggy zoo"), document.IndexField|document.IncludeTermVectors, testAnalyzer))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
indexReader, err := idx.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
dict, err := indexReader.FieldDict("name")
if err != nil {
t.Errorf("error creating reader: %v", err)
}
defer func() {
err := dict.Close()
if err != nil {
t.Fatal(err)
}
}()
termCount := 0
curr, err := dict.Next()
for err == nil && curr != nil {
termCount++
if curr.Term != "test" {
t.Errorf("expected term to be 'test', got '%s'", curr.Term)
}
curr, err = dict.Next()
}
if termCount != 1 {
t.Errorf("expected 1 term for this field, got %d", termCount)
}
dict2, err := indexReader.FieldDict("desc")
if err != nil {
t.Fatalf("error creating reader: %v", err)
}
defer func() {
err := dict2.Close()
if err != nil {
t.Fatal(err)
}
}()
termCount = 0
terms := make([]string, 0)
curr, err = dict2.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict2.Next()
}
if termCount != 3 {
t.Errorf("expected 3 term for this field, got %d", termCount)
}
expectedTerms := []string{"eat", "more", "rice"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
// test start and end range
dict3, err := indexReader.FieldDictRange("desc", []byte("fun"), []byte("nice"))
if err != nil {
t.Errorf("error creating reader: %v", err)
}
defer func() {
err := dict3.Close()
if err != nil {
t.Fatal(err)
}
}()
termCount = 0
terms = make([]string, 0)
curr, err = dict3.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict3.Next()
}
if termCount != 1 {
t.Errorf("expected 1 term for this field, got %d", termCount)
}
expectedTerms = []string{"more"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
// test use case for prefix
dict4, err := indexReader.FieldDictPrefix("prefix", []byte("cat"))
if err != nil {
t.Errorf("error creating reader: %v", err)
}
defer func() {
err := dict4.Close()
if err != nil {
t.Fatal(err)
}
}()
termCount = 0
terms = make([]string, 0)
curr, err = dict4.Next()
for err == nil && curr != nil {
termCount++
terms = append(terms, curr.Term)
curr, err = dict4.Next()
}
if termCount != 3 {
t.Errorf("expected 3 term for this field, got %d", termCount)
}
expectedTerms = []string{"cat", "cats", "catting"}
if !reflect.DeepEqual(expectedTerms, terms) {
t.Errorf("expected %#v, got %#v", expectedTerms, terms)
}
}

316
index/scorch/introducer.go Normal file
View File

@ -0,0 +1,316 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"sync/atomic"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index/scorch/segment"
)
type segmentIntroduction struct {
id uint64
data segment.Segment
obsoletes map[uint64]*roaring.Bitmap
ids []string
internal map[string][]byte
applied chan error
persisted chan error
}
type epochWatcher struct {
epoch uint64
notifyCh notificationChan
}
type snapshotReversion struct {
snapshot *IndexSnapshot
applied chan error
persisted chan error
}
func (s *Scorch) mainLoop() {
var epochWatchers []*epochWatcher
OUTER:
for {
select {
case <-s.closeCh:
break OUTER
case epochWatcher := <-s.introducerNotifier:
epochWatchers = append(epochWatchers, epochWatcher)
case nextMerge := <-s.merges:
s.introduceMerge(nextMerge)
case next := <-s.introductions:
err := s.introduceSegment(next)
if err != nil {
continue OUTER
}
case revertTo := <-s.revertToSnapshots:
err := s.revertToSnapshot(revertTo)
if err != nil {
continue OUTER
}
}
var epochCurr uint64
s.rootLock.RLock()
if s.root != nil {
epochCurr = s.root.epoch
}
s.rootLock.RUnlock()
var epochWatchersNext []*epochWatcher
for _, w := range epochWatchers {
if w.epoch < epochCurr {
close(w.notifyCh)
} else {
epochWatchersNext = append(epochWatchersNext, w)
}
}
epochWatchers = epochWatchersNext
}
s.asyncTasks.Done()
}
func (s *Scorch) introduceSegment(next *segmentIntroduction) error {
// acquire lock
s.rootLock.Lock()
nsegs := len(s.root.segment)
// prepare new index snapshot
newSnapshot := &IndexSnapshot{
parent: s,
segment: make([]*SegmentSnapshot, nsegs, nsegs+1),
offsets: make([]uint64, nsegs, nsegs+1),
internal: make(map[string][]byte, len(s.root.internal)),
epoch: s.nextSnapshotEpoch,
refs: 1,
}
s.nextSnapshotEpoch++
// iterate through current segments
var running uint64
for i := range s.root.segment {
// see if optimistic work included this segment
delta, ok := next.obsoletes[s.root.segment[i].id]
if !ok {
var err error
delta, err = s.root.segment[i].segment.DocNumbers(next.ids)
if err != nil {
next.applied <- fmt.Errorf("error computing doc numbers: %v", err)
close(next.applied)
_ = newSnapshot.DecRef()
return err
}
}
newSnapshot.segment[i] = &SegmentSnapshot{
id: s.root.segment[i].id,
segment: s.root.segment[i].segment,
cachedDocs: s.root.segment[i].cachedDocs,
}
s.root.segment[i].segment.AddRef()
// apply new obsoletions
if s.root.segment[i].deleted == nil {
newSnapshot.segment[i].deleted = delta
} else {
newSnapshot.segment[i].deleted = roaring.Or(s.root.segment[i].deleted, delta)
}
newSnapshot.offsets[i] = running
running += s.root.segment[i].Count()
}
// append new segment, if any, to end of the new index snapshot
if next.data != nil {
newSegmentSnapshot := &SegmentSnapshot{
id: next.id,
segment: next.data, // take ownership of next.data's ref-count
cachedDocs: &cachedDocs{cache: nil},
}
newSnapshot.segment = append(newSnapshot.segment, newSegmentSnapshot)
newSnapshot.offsets = append(newSnapshot.offsets, running)
// increment numItemsIntroduced which tracks the number of items
// queued for persistence.
atomic.AddUint64(&s.stats.numItemsIntroduced, newSegmentSnapshot.Count())
}
// copy old values
for key, oldVal := range s.root.internal {
newSnapshot.internal[key] = oldVal
}
// set new values and apply deletes
for key, newVal := range next.internal {
if newVal != nil {
newSnapshot.internal[key] = newVal
} else {
delete(newSnapshot.internal, key)
}
}
if next.persisted != nil {
s.rootPersisted = append(s.rootPersisted, next.persisted)
}
// swap in new index snapshot
rootPrev := s.root
s.root = newSnapshot
// release lock
s.rootLock.Unlock()
if rootPrev != nil {
_ = rootPrev.DecRef()
}
close(next.applied)
return nil
}
func (s *Scorch) introduceMerge(nextMerge *segmentMerge) {
// acquire lock
s.rootLock.Lock()
// prepare new index snapshot
currSize := len(s.root.segment)
newSize := currSize + 1 - len(nextMerge.old)
newSnapshot := &IndexSnapshot{
parent: s,
segment: make([]*SegmentSnapshot, 0, newSize),
offsets: make([]uint64, 0, newSize),
internal: s.root.internal,
epoch: s.nextSnapshotEpoch,
refs: 1,
}
s.nextSnapshotEpoch++
// iterate through current segments
newSegmentDeleted := roaring.NewBitmap()
var running uint64
for i := range s.root.segment {
segmentID := s.root.segment[i].id
if segSnapAtMerge, ok := nextMerge.old[segmentID]; ok {
// this segment is going away, see if anything else was deleted since we started the merge
if s.root.segment[i].deleted != nil {
// assume all these deletes are new
deletedSince := s.root.segment[i].deleted
// if we already knew about some of them, remove
if segSnapAtMerge.deleted != nil {
deletedSince = roaring.AndNot(s.root.segment[i].deleted, segSnapAtMerge.deleted)
}
deletedSinceItr := deletedSince.Iterator()
for deletedSinceItr.HasNext() {
oldDocNum := deletedSinceItr.Next()
newDocNum := nextMerge.oldNewDocNums[segmentID][oldDocNum]
newSegmentDeleted.Add(uint32(newDocNum))
}
}
} else {
// this segment is staying
newSnapshot.segment = append(newSnapshot.segment, &SegmentSnapshot{
id: s.root.segment[i].id,
segment: s.root.segment[i].segment,
deleted: s.root.segment[i].deleted,
cachedDocs: s.root.segment[i].cachedDocs,
})
s.root.segment[i].segment.AddRef()
newSnapshot.offsets = append(newSnapshot.offsets, running)
running += s.root.segment[i].Count()
}
}
// put new segment at end
newSnapshot.segment = append(newSnapshot.segment, &SegmentSnapshot{
id: nextMerge.id,
segment: nextMerge.new, // take ownership for nextMerge.new's ref-count
deleted: newSegmentDeleted,
cachedDocs: &cachedDocs{cache: nil},
})
newSnapshot.offsets = append(newSnapshot.offsets, running)
// swap in new segment
rootPrev := s.root
s.root = newSnapshot
// release lock
s.rootLock.Unlock()
if rootPrev != nil {
_ = rootPrev.DecRef()
}
// notify merger we incorporated this
close(nextMerge.notify)
}
func (s *Scorch) revertToSnapshot(revertTo *snapshotReversion) error {
if revertTo.snapshot == nil {
err := fmt.Errorf("Cannot revert to a nil snapshot")
revertTo.applied <- err
return err
}
// acquire lock
s.rootLock.Lock()
// prepare a new index snapshot, based on next snapshot
newSnapshot := &IndexSnapshot{
parent: s,
segment: make([]*SegmentSnapshot, len(revertTo.snapshot.segment)),
offsets: revertTo.snapshot.offsets,
internal: revertTo.snapshot.internal,
epoch: s.nextSnapshotEpoch,
refs: 1,
}
s.nextSnapshotEpoch++
// iterate through segments
for i, segmentSnapshot := range revertTo.snapshot.segment {
newSnapshot.segment[i] = &SegmentSnapshot{
id: segmentSnapshot.id,
segment: segmentSnapshot.segment,
deleted: segmentSnapshot.deleted,
cachedDocs: segmentSnapshot.cachedDocs,
}
newSnapshot.segment[i].segment.AddRef()
// remove segment from ineligibleForRemoval map
filename := zapFileName(segmentSnapshot.id)
delete(s.ineligibleForRemoval, filename)
}
if revertTo.persisted != nil {
s.rootPersisted = append(s.rootPersisted, revertTo.persisted)
}
// swap in new snapshot
rootPrev := s.root
s.root = newSnapshot
// release lock
s.rootLock.Unlock()
if rootPrev != nil {
_ = rootPrev.DecRef()
}
close(revertTo.applied)
return nil
}

189
index/scorch/merge.go Normal file
View File

@ -0,0 +1,189 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"os"
"sync/atomic"
"time"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index/scorch/mergeplan"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
)
func (s *Scorch) mergerLoop() {
var lastEpochMergePlanned uint64
OUTER:
for {
select {
case <-s.closeCh:
break OUTER
default:
// check to see if there is a new snapshot to persist
s.rootLock.RLock()
ourSnapshot := s.root
ourSnapshot.AddRef()
s.rootLock.RUnlock()
if ourSnapshot.epoch != lastEpochMergePlanned {
startTime := time.Now()
// lets get started
err := s.planMergeAtSnapshot(ourSnapshot)
if err != nil {
s.fireAsyncError(fmt.Errorf("merging err: %v", err))
_ = ourSnapshot.DecRef()
continue OUTER
}
lastEpochMergePlanned = ourSnapshot.epoch
s.fireEvent(EventKindMergerProgress, time.Since(startTime))
}
_ = ourSnapshot.DecRef()
// tell the persister we're waiting for changes
// first make a notification chan
notifyUs := make(notificationChan)
// give it to the persister
select {
case <-s.closeCh:
break OUTER
case s.persisterNotifier <- notifyUs:
}
// check again
s.rootLock.RLock()
ourSnapshot = s.root
ourSnapshot.AddRef()
s.rootLock.RUnlock()
if ourSnapshot.epoch != lastEpochMergePlanned {
startTime := time.Now()
// lets get started
err := s.planMergeAtSnapshot(ourSnapshot)
if err != nil {
_ = ourSnapshot.DecRef()
continue OUTER
}
lastEpochMergePlanned = ourSnapshot.epoch
s.fireEvent(EventKindMergerProgress, time.Since(startTime))
}
_ = ourSnapshot.DecRef()
// now wait for it (but also detect close)
select {
case <-s.closeCh:
break OUTER
case <-notifyUs:
// woken up, next loop should pick up work
}
}
}
s.asyncTasks.Done()
}
func (s *Scorch) planMergeAtSnapshot(ourSnapshot *IndexSnapshot) error {
// build list of zap segments in this snapshot
var onlyZapSnapshots []mergeplan.Segment
for _, segmentSnapshot := range ourSnapshot.segment {
if _, ok := segmentSnapshot.segment.(*zap.Segment); ok {
onlyZapSnapshots = append(onlyZapSnapshots, segmentSnapshot)
}
}
// give this list to the planner
resultMergePlan, err := mergeplan.Plan(onlyZapSnapshots, nil)
if err != nil {
return fmt.Errorf("merge planning err: %v", err)
}
if resultMergePlan == nil {
// nothing to do
return nil
}
// process tasks in serial for now
var notifications []notificationChan
for _, task := range resultMergePlan.Tasks {
oldMap := make(map[uint64]*SegmentSnapshot)
newSegmentID := atomic.AddUint64(&s.nextSegmentID, 1)
segmentsToMerge := make([]*zap.Segment, 0, len(task.Segments))
docsToDrop := make([]*roaring.Bitmap, 0, len(task.Segments))
for _, planSegment := range task.Segments {
if segSnapshot, ok := planSegment.(*SegmentSnapshot); ok {
oldMap[segSnapshot.id] = segSnapshot
if zapSeg, ok := segSnapshot.segment.(*zap.Segment); ok {
segmentsToMerge = append(segmentsToMerge, zapSeg)
docsToDrop = append(docsToDrop, segSnapshot.deleted)
}
}
}
filename := zapFileName(newSegmentID)
s.markIneligibleForRemoval(filename)
path := s.path + string(os.PathSeparator) + filename
newDocNums, err := zap.Merge(segmentsToMerge, docsToDrop, path, 1024)
if err != nil {
s.unmarkIneligibleForRemoval(filename)
return fmt.Errorf("merging failed: %v", err)
}
segment, err := zap.Open(path)
if err != nil {
s.unmarkIneligibleForRemoval(filename)
return err
}
sm := &segmentMerge{
id: newSegmentID,
old: oldMap,
oldNewDocNums: make(map[uint64][]uint64),
new: segment,
notify: make(notificationChan),
}
notifications = append(notifications, sm.notify)
for i, segNewDocNums := range newDocNums {
sm.oldNewDocNums[task.Segments[i].Id()] = segNewDocNums
}
// give it to the introducer
select {
case <-s.closeCh:
return nil
case s.merges <- sm:
}
}
for _, notification := range notifications {
select {
case <-s.closeCh:
return nil
case <-notification:
}
}
return nil
}
type segmentMerge struct {
id uint64
old map[uint64]*SegmentSnapshot
oldNewDocNums map[uint64][]uint64
new segment.Segment
notify notificationChan
}

View File

@ -0,0 +1,369 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
// Package mergeplan provides a segment merge planning approach that's
// inspired by Lucene's TieredMergePolicy.java and descriptions like
// http://blog.mikemccandless.com/2011/02/visualizing-lucenes-segment-merges.html
package mergeplan
import (
"fmt"
"math"
"sort"
"strings"
)
// A Segment represents the information that the planner needs to
// calculate segment merging.
type Segment interface {
// Unique id of the segment -- used for sorting.
Id() uint64
// Full segment size (the size before any logical deletions).
FullSize() int64
// Size of the live data of the segment; i.e., FullSize() minus
// any logical deletions.
LiveSize() int64
}
// Plan() will functionally compute a merge plan. A segment will be
// assigned to at most a single MergeTask in the output MergePlan. A
// segment not assigned to any MergeTask means the segment should
// remain unmerged.
func Plan(segments []Segment, o *MergePlanOptions) (*MergePlan, error) {
return plan(segments, o)
}
// A MergePlan is the result of the Plan() API.
//
// The planner doesnt know how or whether these tasks are executed --
// thats up to a separate merge execution system, which might execute
// these tasks concurrently or not, and which might execute all the
// tasks or not.
type MergePlan struct {
Tasks []*MergeTask
}
// A MergeTask represents several segments that should be merged
// together into a single segment.
type MergeTask struct {
Segments []Segment
}
// The MergePlanOptions is designed to be reusable between planning calls.
type MergePlanOptions struct {
// Max # segments per logarithmic tier, or max width of any
// logarithmic “step”. Smaller values mean more merging but fewer
// segments. Should be >= SegmentsPerMergeTask, else you'll have
// too much merging.
MaxSegmentsPerTier int
// Max size of any segment produced after merging. Actual
// merging, however, may produce segment sizes different than the
// planners predicted sizes.
MaxSegmentSize int64
// The growth factor for each tier in a staircase of idealized
// segments computed by CalcBudget().
TierGrowth float64
// The number of segments in any resulting MergeTask. e.g.,
// len(result.Tasks[ * ].Segments) == SegmentsPerMergeTask.
SegmentsPerMergeTask int
// Small segments are rounded up to this size, i.e., treated as
// equal (floor) size for consideration. This is to prevent lots
// of tiny segments from resulting in a long tail in the index.
FloorSegmentSize int64
// Controls how aggressively merges that reclaim more deletions
// are favored. Higher values will more aggressively target
// merges that reclaim deletions, but be careful not to go so high
// that way too much merging takes place; a value of 3.0 is
// probably nearly too high. A value of 0.0 means deletions don't
// impact merge selection.
ReclaimDeletesWeight float64
// Optional, defaults to mergeplan.CalcBudget().
CalcBudget func(totalSize int64, firstTierSize int64,
o *MergePlanOptions) (budgetNumSegments int)
// Optional, defaults to mergeplan.ScoreSegments().
ScoreSegments func(segments []Segment, o *MergePlanOptions) float64
// Optional.
Logger func(string)
}
// Returns the higher of the input or FloorSegmentSize.
func (o *MergePlanOptions) RaiseToFloorSegmentSize(s int64) int64 {
if s > o.FloorSegmentSize {
return s
}
return o.FloorSegmentSize
}
// Suggested default options.
var DefaultMergePlanOptions = MergePlanOptions{
MaxSegmentsPerTier: 10,
MaxSegmentSize: 5000000,
TierGrowth: 10.0,
SegmentsPerMergeTask: 10,
FloorSegmentSize: 2000,
ReclaimDeletesWeight: 2.0,
}
// -------------------------------------------
func plan(segmentsIn []Segment, o *MergePlanOptions) (*MergePlan, error) {
if len(segmentsIn) <= 1 {
return nil, nil
}
if o == nil {
o = &DefaultMergePlanOptions
}
segments := append([]Segment(nil), segmentsIn...) // Copy.
sort.Sort(byLiveSizeDescending(segments))
var minLiveSize int64 = math.MaxInt64
var eligibles []Segment
var eligiblesLiveSize int64
for _, segment := range segments {
if minLiveSize > segment.LiveSize() {
minLiveSize = segment.LiveSize()
}
// Only small-enough segments are eligible.
if segment.LiveSize() < o.MaxSegmentSize/2 {
eligibles = append(eligibles, segment)
eligiblesLiveSize += segment.LiveSize()
}
}
minLiveSize = o.RaiseToFloorSegmentSize(minLiveSize)
calcBudget := o.CalcBudget
if calcBudget == nil {
calcBudget = CalcBudget
}
budgetNumSegments := CalcBudget(eligiblesLiveSize, minLiveSize, o)
scoreSegments := o.ScoreSegments
if scoreSegments == nil {
scoreSegments = ScoreSegments
}
rv := &MergePlan{}
var empties []Segment
for _, eligible := range eligibles {
if eligible.LiveSize() <= 0 {
empties = append(empties, eligible)
}
}
if len(empties) > 0 {
rv.Tasks = append(rv.Tasks, &MergeTask{Segments: empties})
eligibles = removeSegments(eligibles, empties)
}
// While were over budget, keep looping, which might produce
// another MergeTask.
for len(eligibles) > budgetNumSegments {
// Track a current best roster as we examine and score
// potential rosters of merges.
var bestRoster []Segment
var bestRosterScore float64 // Lower score is better.
for startIdx := 0; startIdx < len(eligibles)-o.SegmentsPerMergeTask; startIdx++ {
var roster []Segment
var rosterLiveSize int64
for idx := startIdx; idx < len(eligibles) && len(roster) < o.SegmentsPerMergeTask; idx++ {
eligible := eligibles[idx]
if rosterLiveSize+eligible.LiveSize() < o.MaxSegmentSize {
roster = append(roster, eligible)
rosterLiveSize += eligible.LiveSize()
}
}
if len(roster) > 0 {
rosterScore := scoreSegments(roster, o)
if len(bestRoster) <= 0 || rosterScore < bestRosterScore {
bestRoster = roster
bestRosterScore = rosterScore
}
}
}
if len(bestRoster) <= 0 {
return rv, nil
}
rv.Tasks = append(rv.Tasks, &MergeTask{Segments: bestRoster})
eligibles = removeSegments(eligibles, bestRoster)
}
return rv, nil
}
// Compute the number of segments that would be needed to cover the
// totalSize, by climbing up a logarithmically growing staircase of
// segment tiers.
func CalcBudget(totalSize int64, firstTierSize int64, o *MergePlanOptions) (
budgetNumSegments int) {
tierSize := firstTierSize
if tierSize < 1 {
tierSize = 1
}
maxSegmentsPerTier := o.MaxSegmentsPerTier
if maxSegmentsPerTier < 1 {
maxSegmentsPerTier = 1
}
tierGrowth := o.TierGrowth
if tierGrowth < 1.0 {
tierGrowth = 1.0
}
for totalSize > 0 {
segmentsInTier := float64(totalSize) / float64(tierSize)
if segmentsInTier < float64(maxSegmentsPerTier) {
budgetNumSegments += int(math.Ceil(segmentsInTier))
break
}
budgetNumSegments += maxSegmentsPerTier
totalSize -= int64(maxSegmentsPerTier) * tierSize
tierSize = int64(float64(tierSize) * tierGrowth)
}
return budgetNumSegments
}
// Of note, removeSegments() keeps the ordering of the results stable.
func removeSegments(segments []Segment, toRemove []Segment) []Segment {
rv := make([]Segment, 0, len(segments)-len(toRemove))
OUTER:
for _, segment := range segments {
for _, r := range toRemove {
if segment == r {
continue OUTER
}
}
rv = append(rv, segment)
}
return rv
}
// Smaller result score is better.
func ScoreSegments(segments []Segment, o *MergePlanOptions) float64 {
var totBeforeSize int64
var totAfterSize int64
var totAfterSizeFloored int64
for _, segment := range segments {
totBeforeSize += segment.FullSize()
totAfterSize += segment.LiveSize()
totAfterSizeFloored += o.RaiseToFloorSegmentSize(segment.LiveSize())
}
if totBeforeSize <= 0 || totAfterSize <= 0 || totAfterSizeFloored <= 0 {
return 0
}
// Roughly guess the "balance" of the segments -- whether the
// segments are about the same size.
balance :=
float64(o.RaiseToFloorSegmentSize(segments[0].LiveSize())) /
float64(totAfterSizeFloored)
// Gently favor smaller merges over bigger ones. We don't want to
// make the exponent too large else we end up with poor merges of
// small segments in order to avoid the large merges.
score := balance * math.Pow(float64(totAfterSize), 0.05)
// Strongly favor merges that reclaim deletes.
nonDelRatio := float64(totAfterSize) / float64(totBeforeSize)
score *= math.Pow(nonDelRatio, o.ReclaimDeletesWeight)
return score
}
// ------------------------------------------
// ToBarChart returns an ASCII rendering of the segments and the plan.
// The barMax is the max width of the bars in the bar chart.
func ToBarChart(prefix string, barMax int, segments []Segment, plan *MergePlan) string {
rv := make([]string, 0, len(segments))
var maxFullSize int64
for _, segment := range segments {
if maxFullSize < segment.FullSize() {
maxFullSize = segment.FullSize()
}
}
if maxFullSize < 0 {
maxFullSize = 1
}
for _, segment := range segments {
barFull := int(segment.FullSize())
barLive := int(segment.LiveSize())
if maxFullSize > int64(barMax) {
barFull = int(float64(barMax) * float64(barFull) / float64(maxFullSize))
barLive = int(float64(barMax) * float64(barLive) / float64(maxFullSize))
}
barKind := " "
barChar := "."
if plan != nil {
TASK_LOOP:
for taski, task := range plan.Tasks {
for _, taskSegment := range task.Segments {
if taskSegment == segment {
barKind = "*"
barChar = fmt.Sprintf("%d", taski)
break TASK_LOOP
}
}
}
}
bar :=
strings.Repeat(barChar, barLive)[0:barLive] +
strings.Repeat("x", barFull-barLive)[0:barFull-barLive]
rv = append(rv, fmt.Sprintf("%s %5d: %5d /%5d - %s %s", prefix,
segment.Id(),
segment.LiveSize(),
segment.FullSize(),
barKind, bar))
}
return strings.Join(rv, "\n")
}

View File

@ -0,0 +1,463 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mergeplan
import (
"encoding/json"
"fmt"
"os"
"reflect"
"sort"
"testing"
)
// Implements the Segment interface for testing,
type segment struct {
MyId uint64
MyFullSize int64
MyLiveSize int64
}
func (s *segment) Id() uint64 { return s.MyId }
func (s *segment) FullSize() int64 { return s.MyFullSize }
func (s *segment) LiveSize() int64 { return s.MyLiveSize }
func makeLinearSegments(n int) (rv []Segment) {
for i := 0; i < n; i++ {
rv = append(rv, &segment{
MyId: uint64(i),
MyFullSize: int64(i),
MyLiveSize: int64(i),
})
}
return rv
}
// ----------------------------------------
func TestSimplePlan(t *testing.T) {
segs := makeLinearSegments(10)
tests := []struct {
Desc string
Segments []Segment
Options *MergePlanOptions
ExpectPlan *MergePlan
ExpectErr error
}{
{"nil segments",
nil, nil, nil, nil},
{"empty segments",
[]Segment{}, nil, nil, nil},
{"1 segment",
[]Segment{segs[1]},
nil,
nil,
nil,
},
{"2 segments",
[]Segment{
segs[1],
segs[2],
},
nil,
&MergePlan{},
nil,
},
{"3 segments",
[]Segment{
segs[1],
segs[2],
segs[9],
},
nil,
&MergePlan{},
nil,
},
{"many segments",
[]Segment{
segs[1],
segs[2],
segs[3],
segs[4],
segs[5],
segs[6],
},
&MergePlanOptions{
MaxSegmentsPerTier: 1,
MaxSegmentSize: 1000,
TierGrowth: 2.0,
SegmentsPerMergeTask: 2,
FloorSegmentSize: 1,
},
&MergePlan{
Tasks: []*MergeTask{
&MergeTask{
Segments: []Segment{
segs[6],
segs[5],
},
},
},
},
nil,
},
}
for testi, test := range tests {
plan, err := Plan(test.Segments, test.Options)
if err != test.ExpectErr {
testj, _ := json.Marshal(&test)
t.Errorf("testi: %d, test: %s, got err: %v",
testi, testj, err)
}
if !reflect.DeepEqual(plan, test.ExpectPlan) {
testj, _ := json.Marshal(&test)
planj, _ := json.Marshal(&plan)
t.Errorf("testi: %d, test: %s, got plan: %s",
testi, testj, planj)
}
}
}
// ----------------------------------------
func TestSort(t *testing.T) {
segs := makeLinearSegments(10)
sort.Sort(byLiveSizeDescending(segs))
for i := 1; i < len(segs); i++ {
if segs[i].LiveSize() >= segs[i-1].LiveSize() {
t.Errorf("not descending")
}
}
}
// ----------------------------------------
func TestCalcBudget(t *testing.T) {
tests := []struct {
totalSize int64
firstTierSize int64
o MergePlanOptions
expect int
}{
{0, 0, MergePlanOptions{}, 0},
{1, 0, MergePlanOptions{}, 1},
{9, 0, MergePlanOptions{}, 9},
{1, 1,
MergePlanOptions{
MaxSegmentsPerTier: 1,
MaxSegmentSize: 1000,
TierGrowth: 2.0,
SegmentsPerMergeTask: 2,
FloorSegmentSize: 1,
},
1,
},
{21, 1,
MergePlanOptions{
MaxSegmentsPerTier: 1,
MaxSegmentSize: 1000,
TierGrowth: 2.0,
SegmentsPerMergeTask: 2,
FloorSegmentSize: 1,
},
5,
},
{21, 1,
MergePlanOptions{
MaxSegmentsPerTier: 2,
MaxSegmentSize: 1000,
TierGrowth: 2.0,
SegmentsPerMergeTask: 2,
FloorSegmentSize: 1,
},
7,
},
}
for testi, test := range tests {
res := CalcBudget(test.totalSize, test.firstTierSize, &test.o)
if res != test.expect {
t.Errorf("testi: %d, test: %#v, res: %v",
testi, test, res)
}
}
}
// ----------------------------------------
func TestInsert1SameSizedSegmentBetweenMerges(t *testing.T) {
o := &MergePlanOptions{
MaxSegmentSize: 1000,
MaxSegmentsPerTier: 3,
TierGrowth: 3.0,
SegmentsPerMergeTask: 3,
}
spec := testCyclesSpec{
descrip: "i1sssbm",
verbose: os.Getenv("VERBOSE") == "i1sssbm" || os.Getenv("VERBOSE") == "y",
n: 200,
o: o,
beforePlan: func(spec *testCyclesSpec) {
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: 1,
MyLiveSize: 1,
})
spec.nextSegmentId++
},
}
spec.runCycles(t)
}
func TestInsertManySameSizedSegmentsBetweenMerges(t *testing.T) {
o := &MergePlanOptions{
MaxSegmentSize: 1000,
MaxSegmentsPerTier: 3,
TierGrowth: 3.0,
SegmentsPerMergeTask: 3,
}
spec := testCyclesSpec{
descrip: "imsssbm",
verbose: os.Getenv("VERBOSE") == "imsssbm" || os.Getenv("VERBOSE") == "y",
n: 20,
o: o,
beforePlan: func(spec *testCyclesSpec) {
for i := 0; i < 10; i++ {
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: 1,
MyLiveSize: 1,
})
spec.nextSegmentId++
}
},
}
spec.runCycles(t)
}
func TestInsertManySameSizedSegmentsWithDeletionsBetweenMerges(t *testing.T) {
o := &MergePlanOptions{
MaxSegmentSize: 1000,
MaxSegmentsPerTier: 3,
TierGrowth: 3.0,
SegmentsPerMergeTask: 3,
}
spec := testCyclesSpec{
descrip: "imssswdbm",
verbose: os.Getenv("VERBOSE") == "imssswdbm" || os.Getenv("VERBOSE") == "y",
n: 20,
o: o,
beforePlan: func(spec *testCyclesSpec) {
for i := 0; i < 10; i++ {
// Deletions are a shrinking of the live size.
for i, seg := range spec.segments {
if (spec.cycle+i)%5 == 0 {
s := seg.(*segment)
if s.MyLiveSize > 0 {
s.MyLiveSize -= 1
}
}
}
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: 1,
MyLiveSize: 1,
})
spec.nextSegmentId++
}
},
}
spec.runCycles(t)
}
func TestInsertManyDifferentSizedSegmentsBetweenMerges(t *testing.T) {
o := &MergePlanOptions{
MaxSegmentSize: 1000,
MaxSegmentsPerTier: 3,
TierGrowth: 3.0,
SegmentsPerMergeTask: 3,
}
spec := testCyclesSpec{
descrip: "imdssbm",
verbose: os.Getenv("VERBOSE") == "imdssbm" || os.Getenv("VERBOSE") == "y",
n: 20,
o: o,
beforePlan: func(spec *testCyclesSpec) {
for i := 0; i < 10; i++ {
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: int64(1 + (i % 5)),
MyLiveSize: int64(1 + (i % 5)),
})
spec.nextSegmentId++
}
},
}
spec.runCycles(t)
}
func TestManySameSizedSegmentsWithDeletesBetweenMerges(t *testing.T) {
o := &MergePlanOptions{
MaxSegmentSize: 1000,
MaxSegmentsPerTier: 3,
TierGrowth: 3.0,
SegmentsPerMergeTask: 3,
}
var numPlansWithTasks int
spec := testCyclesSpec{
descrip: "mssswdbm",
verbose: os.Getenv("VERBOSE") == "mssswdbm" || os.Getenv("VERBOSE") == "y",
n: 20,
o: o,
beforePlan: func(spec *testCyclesSpec) {
// Deletions are a shrinking of the live size.
for i, seg := range spec.segments {
if (spec.cycle+i)%5 == 0 {
s := seg.(*segment)
if s.MyLiveSize > 0 {
s.MyLiveSize -= 1
}
}
}
for i := 0; i < 10; i++ {
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: 1,
MyLiveSize: 1,
})
spec.nextSegmentId++
}
},
afterPlan: func(spec *testCyclesSpec, plan *MergePlan) {
if plan != nil && len(plan.Tasks) > 0 {
numPlansWithTasks++
}
},
}
spec.runCycles(t)
if numPlansWithTasks <= 0 {
t.Errorf("expected some plans with tasks")
}
}
// ----------------------------------------
type testCyclesSpec struct {
descrip string
verbose bool
n int // Number of cycles to run.
o *MergePlanOptions
beforePlan func(*testCyclesSpec)
afterPlan func(*testCyclesSpec, *MergePlan)
cycle int
segments []Segment
nextSegmentId uint64
}
func (spec *testCyclesSpec) runCycles(t *testing.T) {
numPlansWithTasks := 0
for spec.cycle < spec.n {
if spec.verbose {
emit(spec.descrip, spec.cycle, 0, spec.segments, nil)
}
if spec.beforePlan != nil {
spec.beforePlan(spec)
}
if spec.verbose {
emit(spec.descrip, spec.cycle, 1, spec.segments, nil)
}
plan, err := Plan(spec.segments, spec.o)
if err != nil {
t.Fatalf("expected no err, got: %v", err)
}
if spec.afterPlan != nil {
spec.afterPlan(spec, plan)
}
if spec.verbose {
emit(spec.descrip, spec.cycle, 2, spec.segments, plan)
}
if plan != nil {
if len(plan.Tasks) > 0 {
numPlansWithTasks++
}
for _, task := range plan.Tasks {
spec.segments = removeSegments(spec.segments, task.Segments)
var totLiveSize int64
for _, segment := range task.Segments {
totLiveSize += segment.LiveSize()
}
if totLiveSize > 0 {
spec.segments = append(spec.segments, &segment{
MyId: spec.nextSegmentId,
MyFullSize: totLiveSize,
MyLiveSize: totLiveSize,
})
spec.nextSegmentId++
}
}
}
spec.cycle++
}
if numPlansWithTasks <= 0 {
t.Errorf("expected some plans with tasks")
}
}
func emit(descrip string, cycle int, step int, segments []Segment, plan *MergePlan) {
if os.Getenv("VERBOSE") == "" {
return
}
suffix := ""
if plan != nil && len(plan.Tasks) > 0 {
suffix = "hasPlan"
}
fmt.Printf("%s %d.%d ---------- %s\n", descrip, cycle, step, suffix)
fmt.Printf("%s\n", ToBarChart(descrip, 100, segments, plan))
}

View File

@ -0,0 +1,28 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mergeplan
type byLiveSizeDescending []Segment
func (a byLiveSizeDescending) Len() int { return len(a) }
func (a byLiveSizeDescending) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
func (a byLiveSizeDescending) Less(i, j int) bool {
if a[i].LiveSize() != a[j].LiveSize() {
return a[i].LiveSize() > a[j].LiveSize()
}
return a[i].Id() < a[j].Id()
}

645
index/scorch/persister.go Normal file
View File

@ -0,0 +1,645 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"bytes"
"fmt"
"io/ioutil"
"log"
"os"
"path/filepath"
"strconv"
"strings"
"sync/atomic"
"time"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
"github.com/blevesearch/bleve/index/scorch/segment/zap"
"github.com/boltdb/bolt"
)
type notificationChan chan struct{}
func (s *Scorch) persisterLoop() {
defer s.asyncTasks.Done()
var notifyChs []notificationChan
var lastPersistedEpoch uint64
OUTER:
for {
select {
case <-s.closeCh:
break OUTER
case notifyCh := <-s.persisterNotifier:
notifyChs = append(notifyChs, notifyCh)
default:
}
var ourSnapshot *IndexSnapshot
var ourPersisted []chan error
// check to see if there is a new snapshot to persist
s.rootLock.Lock()
if s.root != nil && s.root.epoch > lastPersistedEpoch {
ourSnapshot = s.root
ourSnapshot.AddRef()
ourPersisted = s.rootPersisted
s.rootPersisted = nil
}
s.rootLock.Unlock()
if ourSnapshot != nil {
startTime := time.Now()
err := s.persistSnapshot(ourSnapshot)
for _, ch := range ourPersisted {
if err != nil {
ch <- err
}
close(ch)
}
if err != nil {
s.fireAsyncError(fmt.Errorf("got err persisting snapshot: %v", err))
_ = ourSnapshot.DecRef()
continue OUTER
}
lastPersistedEpoch = ourSnapshot.epoch
for _, notifyCh := range notifyChs {
close(notifyCh)
}
notifyChs = nil
_ = ourSnapshot.DecRef()
changed := false
s.rootLock.RLock()
if s.root != nil && s.root.epoch != lastPersistedEpoch {
changed = true
}
s.rootLock.RUnlock()
s.fireEvent(EventKindPersisterProgress, time.Since(startTime))
if changed {
continue OUTER
}
}
// tell the introducer we're waiting for changes
w := &epochWatcher{
epoch: lastPersistedEpoch,
notifyCh: make(notificationChan, 1),
}
select {
case <-s.closeCh:
break OUTER
case s.introducerNotifier <- w:
}
s.removeOldData() // might as well cleanup while waiting
select {
case <-s.closeCh:
break OUTER
case <-w.notifyCh:
// woken up, next loop should pick up work
}
}
}
func (s *Scorch) persistSnapshot(snapshot *IndexSnapshot) error {
// start a write transaction
tx, err := s.rootBolt.Begin(true)
if err != nil {
return err
}
// defer fsync of the rootbolt
defer func() {
if err == nil {
err = s.rootBolt.Sync()
}
}()
// defer commit/rollback transaction
defer func() {
if err == nil {
err = tx.Commit()
} else {
_ = tx.Rollback()
}
}()
snapshotsBucket, err := tx.CreateBucketIfNotExists(boltSnapshotsBucket)
if err != nil {
return err
}
newSnapshotKey := segment.EncodeUvarintAscending(nil, snapshot.epoch)
snapshotBucket, err := snapshotsBucket.CreateBucketIfNotExists(newSnapshotKey)
if err != nil {
return err
}
// persist internal values
internalBucket, err := snapshotBucket.CreateBucketIfNotExists(boltInternalKey)
if err != nil {
return err
}
// TODO optimize writing these in order?
for k, v := range snapshot.internal {
err = internalBucket.Put([]byte(k), v)
if err != nil {
return err
}
}
var filenames []string
newSegmentPaths := make(map[uint64]string)
// first ensure that each segment in this snapshot has been persisted
for i, segmentSnapshot := range snapshot.segment {
snapshotSegmentKey := segment.EncodeUvarintAscending(nil, uint64(i))
snapshotSegmentBucket, err2 := snapshotBucket.CreateBucketIfNotExists(snapshotSegmentKey)
if err2 != nil {
return err2
}
switch seg := segmentSnapshot.segment.(type) {
case *mem.Segment:
// need to persist this to disk
filename := zapFileName(segmentSnapshot.id)
path := s.path + string(os.PathSeparator) + filename
err2 := zap.PersistSegment(seg, path, 1024)
if err2 != nil {
return fmt.Errorf("error persisting segment: %v", err2)
}
newSegmentPaths[segmentSnapshot.id] = path
err = snapshotSegmentBucket.Put(boltPathKey, []byte(filename))
if err != nil {
return err
}
filenames = append(filenames, filename)
case *zap.Segment:
path := seg.Path()
filename := strings.TrimPrefix(path, s.path+string(os.PathSeparator))
err = snapshotSegmentBucket.Put(boltPathKey, []byte(filename))
if err != nil {
return err
}
filenames = append(filenames, filename)
default:
return fmt.Errorf("unknown segment type: %T", seg)
}
// store current deleted bits
var roaringBuf bytes.Buffer
if segmentSnapshot.deleted != nil {
_, err = segmentSnapshot.deleted.WriteTo(&roaringBuf)
if err != nil {
return fmt.Errorf("error persisting roaring bytes: %v", err)
}
err = snapshotSegmentBucket.Put(boltDeletedKey, roaringBuf.Bytes())
if err != nil {
return err
}
}
}
// only alter the root if we actually persisted a segment
// (sometimes its just a new snapshot, possibly with new internal values)
if len(newSegmentPaths) > 0 {
// now try to open all the new snapshots
newSegments := make(map[uint64]segment.Segment)
for segmentID, path := range newSegmentPaths {
newSegments[segmentID], err = zap.Open(path)
if err != nil {
for _, s := range newSegments {
if s != nil {
_ = s.Close() // cleanup segments that were successfully opened
}
}
return fmt.Errorf("error opening new segment at %s, %v", path, err)
}
}
s.rootLock.Lock()
newIndexSnapshot := &IndexSnapshot{
parent: s,
epoch: s.nextSnapshotEpoch,
segment: make([]*SegmentSnapshot, len(s.root.segment)),
offsets: make([]uint64, len(s.root.offsets)),
internal: make(map[string][]byte, len(s.root.internal)),
refs: 1,
}
s.nextSnapshotEpoch++
for i, segmentSnapshot := range s.root.segment {
// see if this segment has been replaced
if replacement, ok := newSegments[segmentSnapshot.id]; ok {
newSegmentSnapshot := &SegmentSnapshot{
id: segmentSnapshot.id,
segment: replacement,
deleted: segmentSnapshot.deleted,
cachedDocs: segmentSnapshot.cachedDocs,
}
newIndexSnapshot.segment[i] = newSegmentSnapshot
// update items persisted incase of a new segment snapshot
atomic.AddUint64(&s.stats.numItemsPersisted, newSegmentSnapshot.Count())
} else {
newIndexSnapshot.segment[i] = s.root.segment[i]
newIndexSnapshot.segment[i].segment.AddRef()
}
newIndexSnapshot.offsets[i] = s.root.offsets[i]
}
for k, v := range s.root.internal {
newIndexSnapshot.internal[k] = v
}
for _, filename := range filenames {
delete(s.ineligibleForRemoval, filename)
}
rootPrev := s.root
s.root = newIndexSnapshot
s.rootLock.Unlock()
if rootPrev != nil {
_ = rootPrev.DecRef()
}
}
return nil
}
func zapFileName(epoch uint64) string {
return fmt.Sprintf("%012x.zap", epoch)
}
// bolt snapshot code
var boltSnapshotsBucket = []byte{'s'}
var boltPathKey = []byte{'p'}
var boltDeletedKey = []byte{'d'}
var boltInternalKey = []byte{'i'}
func (s *Scorch) loadFromBolt() error {
return s.rootBolt.View(func(tx *bolt.Tx) error {
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return nil
}
foundRoot := false
c := snapshots.Cursor()
for k, _ := c.Last(); k != nil; k, _ = c.Prev() {
_, snapshotEpoch, err := segment.DecodeUvarintAscending(k)
if err != nil {
log.Printf("unable to parse segment epoch %x, continuing", k)
continue
}
if foundRoot {
s.eligibleForRemoval = append(s.eligibleForRemoval, snapshotEpoch)
continue
}
snapshot := snapshots.Bucket(k)
if snapshot == nil {
log.Printf("snapshot key, but bucket missing %x, continuing", k)
s.eligibleForRemoval = append(s.eligibleForRemoval, snapshotEpoch)
continue
}
indexSnapshot, err := s.loadSnapshot(snapshot)
if err != nil {
log.Printf("unable to load snapshot, %v, continuing", err)
s.eligibleForRemoval = append(s.eligibleForRemoval, snapshotEpoch)
continue
}
indexSnapshot.epoch = snapshotEpoch
// set the nextSegmentID
s.nextSegmentID, err = s.maxSegmentIDOnDisk()
if err != nil {
return err
}
s.nextSegmentID++
s.nextSnapshotEpoch = snapshotEpoch + 1
s.rootLock.Lock()
if s.root != nil {
_ = s.root.DecRef()
}
s.root = indexSnapshot
s.rootLock.Unlock()
foundRoot = true
}
return nil
})
}
// LoadSnapshot loads the segment with the specified epoch
// NOTE: this is currently ONLY intended to be used by the command-line tool
func (s *Scorch) LoadSnapshot(epoch uint64) (rv *IndexSnapshot, err error) {
err = s.rootBolt.View(func(tx *bolt.Tx) error {
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return nil
}
snapshotKey := segment.EncodeUvarintAscending(nil, epoch)
snapshot := snapshots.Bucket(snapshotKey)
if snapshot == nil {
return nil
}
rv, err = s.loadSnapshot(snapshot)
return err
})
if err != nil {
return nil, err
}
return rv, nil
}
func (s *Scorch) loadSnapshot(snapshot *bolt.Bucket) (*IndexSnapshot, error) {
rv := &IndexSnapshot{
parent: s,
internal: make(map[string][]byte),
refs: 1,
}
var running uint64
c := snapshot.Cursor()
for k, _ := c.First(); k != nil; k, _ = c.Next() {
if k[0] == boltInternalKey[0] {
internalBucket := snapshot.Bucket(k)
err := internalBucket.ForEach(func(key []byte, val []byte) error {
copiedVal := append([]byte(nil), val...)
rv.internal[string(key)] = copiedVal
return nil
})
if err != nil {
_ = rv.DecRef()
return nil, err
}
} else {
segmentBucket := snapshot.Bucket(k)
if segmentBucket == nil {
_ = rv.DecRef()
return nil, fmt.Errorf("segment key, but bucket missing % x", k)
}
segmentSnapshot, err := s.loadSegment(segmentBucket)
if err != nil {
_ = rv.DecRef()
return nil, fmt.Errorf("failed to load segment: %v", err)
}
_, segmentSnapshot.id, err = segment.DecodeUvarintAscending(k)
if err != nil {
_ = rv.DecRef()
return nil, fmt.Errorf("failed to decode segment id: %v", err)
}
rv.segment = append(rv.segment, segmentSnapshot)
rv.offsets = append(rv.offsets, running)
running += segmentSnapshot.segment.Count()
}
}
return rv, nil
}
func (s *Scorch) loadSegment(segmentBucket *bolt.Bucket) (*SegmentSnapshot, error) {
pathBytes := segmentBucket.Get(boltPathKey)
if pathBytes == nil {
return nil, fmt.Errorf("segment path missing")
}
segmentPath := s.path + string(os.PathSeparator) + string(pathBytes)
segment, err := zap.Open(segmentPath)
if err != nil {
return nil, fmt.Errorf("error opening bolt segment: %v", err)
}
rv := &SegmentSnapshot{
segment: segment,
cachedDocs: &cachedDocs{cache: nil},
}
deletedBytes := segmentBucket.Get(boltDeletedKey)
if deletedBytes != nil {
deletedBitmap := roaring.NewBitmap()
r := bytes.NewReader(deletedBytes)
_, err := deletedBitmap.ReadFrom(r)
if err != nil {
_ = segment.Close()
return nil, fmt.Errorf("error reading deleted bytes: %v", err)
}
rv.deleted = deletedBitmap
}
return rv, nil
}
type uint64Descending []uint64
func (p uint64Descending) Len() int { return len(p) }
func (p uint64Descending) Less(i, j int) bool { return p[i] > p[j] }
func (p uint64Descending) Swap(i, j int) { p[i], p[j] = p[j], p[i] }
func (s *Scorch) removeOldData() {
removed, err := s.removeOldBoltSnapshots()
if err != nil {
s.fireAsyncError(fmt.Errorf("got err removing old bolt snapshots: %v", err))
}
if removed > 0 {
err = s.removeOldZapFiles()
if err != nil {
s.fireAsyncError(fmt.Errorf("got err removing old zap files: %v", err))
}
}
}
// NumSnapshotsToKeep represents how many recent, old snapshots to
// keep around per Scorch instance. Useful for apps that require
// rollback'ability.
var NumSnapshotsToKeep = 1
// Removes enough snapshots from the rootBolt so that the
// s.eligibleForRemoval stays under the NumSnapshotsToKeep policy.
func (s *Scorch) removeOldBoltSnapshots() (numRemoved int, err error) {
persistedEpochs, err := s.RootBoltSnapshotEpochs()
if err != nil {
return 0, err
}
if len(persistedEpochs) <= NumSnapshotsToKeep {
// we need to keep everything
return 0, nil
}
// make a map of epochs to protect from deletion
protectedEpochs := make(map[uint64]struct{}, NumSnapshotsToKeep)
for _, epoch := range persistedEpochs[0:NumSnapshotsToKeep] {
protectedEpochs[epoch] = struct{}{}
}
var epochsToRemove []uint64
var newEligible []uint64
s.rootLock.Lock()
for _, epoch := range s.eligibleForRemoval {
if _, ok := protectedEpochs[epoch]; ok {
// protected
newEligible = append(newEligible, epoch)
} else {
epochsToRemove = append(epochsToRemove, epoch)
}
}
s.eligibleForRemoval = newEligible
s.rootLock.Unlock()
if len(epochsToRemove) <= 0 {
return 0, nil
}
tx, err := s.rootBolt.Begin(true)
if err != nil {
return 0, err
}
defer func() {
if err == nil {
err = tx.Commit()
} else {
_ = tx.Rollback()
}
if err == nil {
err = s.rootBolt.Sync()
}
}()
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return 0, nil
}
for _, epochToRemove := range epochsToRemove {
k := segment.EncodeUvarintAscending(nil, epochToRemove)
err = snapshots.DeleteBucket(k)
if err == bolt.ErrBucketNotFound {
err = nil
}
if err == nil {
numRemoved++
}
}
return numRemoved, err
}
func (s *Scorch) maxSegmentIDOnDisk() (uint64, error) {
currFileInfos, err := ioutil.ReadDir(s.path)
if err != nil {
return 0, err
}
var rv uint64
for _, finfo := range currFileInfos {
fname := finfo.Name()
if filepath.Ext(fname) == ".zap" {
prefix := strings.TrimSuffix(fname, ".zap")
id, err2 := strconv.ParseUint(prefix, 16, 64)
if err2 != nil {
return 0, err2
}
if id > rv {
rv = id
}
}
}
return rv, err
}
// Removes any *.zap files which aren't listed in the rootBolt.
func (s *Scorch) removeOldZapFiles() error {
liveFileNames, err := s.loadZapFileNames()
if err != nil {
return err
}
currFileInfos, err := ioutil.ReadDir(s.path)
if err != nil {
return err
}
s.rootLock.RLock()
for _, finfo := range currFileInfos {
fname := finfo.Name()
if filepath.Ext(fname) == ".zap" {
if _, exists := liveFileNames[fname]; !exists && !s.ineligibleForRemoval[fname] {
err := os.Remove(s.path + string(os.PathSeparator) + fname)
if err != nil {
log.Printf("got err removing file: %s, err: %v", fname, err)
}
}
}
}
s.rootLock.RUnlock()
return nil
}
func (s *Scorch) RootBoltSnapshotEpochs() ([]uint64, error) {
var rv []uint64
err := s.rootBolt.View(func(tx *bolt.Tx) error {
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return nil
}
sc := snapshots.Cursor()
for sk, _ := sc.Last(); sk != nil; sk, _ = sc.Prev() {
_, snapshotEpoch, err := segment.DecodeUvarintAscending(sk)
if err != nil {
continue
}
rv = append(rv, snapshotEpoch)
}
return nil
})
return rv, err
}
// Returns the *.zap file names that are listed in the rootBolt.
func (s *Scorch) loadZapFileNames() (map[string]struct{}, error) {
rv := map[string]struct{}{}
err := s.rootBolt.View(func(tx *bolt.Tx) error {
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return nil
}
sc := snapshots.Cursor()
for sk, _ := sc.First(); sk != nil; sk, _ = sc.Next() {
snapshot := snapshots.Bucket(sk)
if snapshot == nil {
continue
}
segc := snapshot.Cursor()
for segk, _ := segc.First(); segk != nil; segk, _ = segc.Next() {
if segk[0] == boltInternalKey[0] {
continue
}
segmentBucket := snapshot.Bucket(segk)
if segmentBucket == nil {
continue
}
pathBytes := segmentBucket.Get(boltPathKey)
if pathBytes == nil {
continue
}
pathString := string(pathBytes)
rv[string(pathString)] = struct{}{}
}
}
return nil
})
return rv, err
}

110
index/scorch/reader.go Normal file
View File

@ -0,0 +1,110 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
type Reader struct {
root *IndexSnapshot // Owns 1 ref-count on the index snapshot.
}
func (r *Reader) TermFieldReader(term []byte, field string, includeFreq,
includeNorm, includeTermVectors bool) (index.TermFieldReader, error) {
return r.root.TermFieldReader(term, field, includeFreq, includeNorm, includeTermVectors)
}
// DocIDReader returns an iterator over all doc ids
// The caller must close returned instance to release associated resources.
func (r *Reader) DocIDReaderAll() (index.DocIDReader, error) {
return r.root.DocIDReaderAll()
}
func (r *Reader) DocIDReaderOnly(ids []string) (index.DocIDReader, error) {
return r.root.DocIDReaderOnly(ids)
}
func (r *Reader) FieldDict(field string) (index.FieldDict, error) {
return r.root.FieldDict(field)
}
// FieldDictRange is currently defined to include the start and end terms
func (r *Reader) FieldDictRange(field string, startTerm []byte,
endTerm []byte) (index.FieldDict, error) {
return r.root.FieldDictRange(field, startTerm, endTerm)
}
func (r *Reader) FieldDictPrefix(field string,
termPrefix []byte) (index.FieldDict, error) {
return r.root.FieldDictPrefix(field, termPrefix)
}
func (r *Reader) Document(id string) (*document.Document, error) {
return r.root.Document(id)
}
func (r *Reader) DocumentVisitFieldTerms(id index.IndexInternalID, fields []string,
visitor index.DocumentFieldTermVisitor) error {
return r.root.DocumentVisitFieldTerms(id, fields, visitor)
}
func (r *Reader) Fields() ([]string, error) {
return r.root.Fields()
}
func (r *Reader) GetInternal(key []byte) ([]byte, error) {
return r.root.GetInternal(key)
}
func (r *Reader) DocCount() (uint64, error) {
return r.root.DocCount()
}
func (r *Reader) ExternalID(id index.IndexInternalID) (string, error) {
return r.root.ExternalID(id)
}
func (r *Reader) InternalID(id string) (index.IndexInternalID, error) {
return r.root.InternalID(id)
}
func (r *Reader) DumpAll() chan interface{} {
rv := make(chan interface{})
go func() {
close(rv)
}()
return rv
}
func (r *Reader) DumpDoc(id string) chan interface{} {
rv := make(chan interface{})
go func() {
close(rv)
}()
return rv
}
func (r *Reader) DumpFields() chan interface{} {
rv := make(chan interface{})
go func() {
close(rv)
}()
return rv
}
func (r *Reader) Close() error {
return r.root.DecRef()
}

663
index/scorch/reader_test.go Normal file
View File

@ -0,0 +1,663 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestIndexReader(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, testConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
var expectedCount uint64
doc := document.NewDocument("1")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test")))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
doc = document.NewDocument("2")
doc.AddField(document.NewTextFieldWithAnalyzer("name", []uint64{}, []byte("test test test"), testAnalyzer))
doc.AddField(document.NewTextFieldCustom("desc", []uint64{}, []byte("eat more rice"), document.IndexField|document.IncludeTermVectors, testAnalyzer))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
indexReader, err := idx.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Fatal(err)
}
}()
// first look for a term that doesn't exist
reader, err := indexReader.TermFieldReader([]byte("nope"), "name", true, true, true)
if err != nil {
t.Errorf("Error accessing term field reader: %v", err)
}
count := reader.Count()
if count != 0 {
t.Errorf("Expected doc count to be: %d got: %d", 0, count)
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
reader, err = indexReader.TermFieldReader([]byte("test"), "name", true, true, true)
if err != nil {
t.Errorf("Error accessing term field reader: %v", err)
}
expectedCount = 2
count = reader.Count()
if count != expectedCount {
t.Errorf("Exptected doc count to be: %d got: %d", expectedCount, count)
}
var match *index.TermFieldDoc
var actualCount uint64
match, err = reader.Next(nil)
for err == nil && match != nil {
match, err = reader.Next(nil)
if err != nil {
t.Errorf("unexpected error reading next")
}
actualCount++
}
if actualCount != count {
t.Errorf("count was 2, but only saw %d", actualCount)
}
internalIDBogus, err := indexReader.InternalID("a-bogus-docId")
if err != nil {
t.Fatal(err)
}
if internalIDBogus != nil {
t.Errorf("expected bogus docId to have nil InternalID")
}
internalID2, err := indexReader.InternalID("2")
if err != nil {
t.Fatal(err)
}
expectedMatch := &index.TermFieldDoc{
ID: internalID2,
Freq: 1,
Norm: 0.5773502588272095,
Vectors: []*index.TermFieldVector{
{
Field: "desc",
Pos: 3,
Start: 9,
End: 13,
},
},
}
tfr, err := indexReader.TermFieldReader([]byte("rice"), "desc", true, true, true)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
match, err = tfr.Next(nil)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if !reflect.DeepEqual(expectedMatch, match) {
t.Errorf("got %#v, expected %#v", match, expectedMatch)
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// now test usage of advance
reader, err = indexReader.TermFieldReader([]byte("test"), "name", true, true, true)
if err != nil {
t.Errorf("Error accessing term field reader: %v", err)
}
match, err = reader.Advance(internalID2, nil)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if match == nil {
t.Fatalf("Expected match, got nil")
}
if !match.ID.Equals(internalID2) {
t.Errorf("Expected ID '2', got '%s'", match.ID)
}
// NOTE: no point in changing this to internal id 3, there is no id 3
// the test is looking for something that doens't exist and this doesn't
match, err = reader.Advance(index.IndexInternalID("3"), nil)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if match != nil {
t.Errorf("expected nil, got %v", match)
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// now test creating a reader for a field that doesn't exist
reader, err = indexReader.TermFieldReader([]byte("water"), "doesnotexist", true, true, true)
if err != nil {
t.Errorf("Error accessing term field reader: %v", err)
}
count = reader.Count()
if count != 0 {
t.Errorf("expected count 0 for reader of non-existent field")
}
match, err = reader.Next(nil)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if match != nil {
t.Errorf("expected nil, got %v", match)
}
match, err = reader.Advance(index.IndexInternalID("anywhere"), nil)
if err != nil {
t.Errorf("unexpected error: %v", err)
}
if match != nil {
t.Errorf("expected nil, got %v", match)
}
}
func TestIndexDocIdReader(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, testConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
var expectedCount uint64
doc := document.NewDocument("1")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test")))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
doc = document.NewDocument("2")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test test test")))
doc.AddField(document.NewTextFieldWithIndexingOptions("desc", []uint64{}, []byte("eat more rice"), document.IndexField|document.IncludeTermVectors))
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
expectedCount++
indexReader, err := idx.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Error(err)
}
}()
// first get all doc ids
reader, err := indexReader.DocIDReaderAll()
if err != nil {
t.Errorf("Error accessing doc id reader: %v", err)
}
defer func() {
err := reader.Close()
if err != nil {
t.Fatal(err)
}
}()
id, err := reader.Next()
count := uint64(0)
for id != nil {
count++
id, err = reader.Next()
}
if count != expectedCount {
t.Errorf("expected %d, got %d", expectedCount, count)
}
// try it again, but jump to the second doc this time
reader2, err := indexReader.DocIDReaderAll()
if err != nil {
t.Errorf("Error accessing doc id reader: %v", err)
}
defer func() {
err := reader2.Close()
if err != nil {
t.Error(err)
}
}()
internalID2, err := indexReader.InternalID("2")
if err != nil {
t.Fatal(err)
}
id, err = reader2.Advance(internalID2)
if err != nil {
t.Error(err)
}
if !id.Equals(internalID2) {
t.Errorf("expected to find id '2', got '%s'", id)
}
// again 3 doesn't exist cannot use internal id for 3 as there is none
// the important aspect is that this id doesn't exist, so its ok
id, err = reader2.Advance(index.IndexInternalID("3"))
if err != nil {
t.Error(err)
}
if id != nil {
t.Errorf("expected to find id '', got '%s'", id)
}
}
func TestIndexDocIdOnlyReader(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, testConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
doc := document.NewDocument("1")
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
doc = document.NewDocument("3")
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
doc = document.NewDocument("5")
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
doc = document.NewDocument("7")
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
doc = document.NewDocument("9")
err = idx.Update(doc)
if err != nil {
t.Errorf("Error updating index: %v", err)
}
indexReader, err := idx.Reader()
if err != nil {
t.Error(err)
}
defer func() {
err := indexReader.Close()
if err != nil {
t.Error(err)
}
}()
onlyIds := []string{"1", "5", "9"}
reader, err := indexReader.DocIDReaderOnly(onlyIds)
if err != nil {
t.Errorf("Error accessing doc id reader: %v", err)
}
defer func() {
err := reader.Close()
if err != nil {
t.Fatal(err)
}
}()
id, err := reader.Next()
count := uint64(0)
for id != nil {
count++
id, err = reader.Next()
if err != nil {
t.Fatal(err)
}
}
if count != 3 {
t.Errorf("expected 3, got %d", count)
}
// commented out because advance works with internal ids
// this test presumes we see items in external doc id order
// which is no longer the case, so simply converting external ids
// to internal ones is not logically correct
// not removing though because we need some way to test Advance()
// // try it again, but jump
// reader2, err := indexReader.DocIDReaderOnly(onlyIds)
// if err != nil {
// t.Errorf("Error accessing doc id reader: %v", err)
// }
// defer func() {
// err := reader2.Close()
// if err != nil {
// t.Error(err)
// }
// }()
//
// id, err = reader2.Advance(index.IndexInternalID("5"))
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("5")) {
// t.Errorf("expected to find id '5', got '%s'", id)
// }
//
// id, err = reader2.Advance(index.IndexInternalID("a"))
// if err != nil {
// t.Error(err)
// }
// if id != nil {
// t.Errorf("expected to find id '', got '%s'", id)
// }
// some keys aren't actually there
onlyIds = []string{"0", "2", "4", "5", "6", "8", "a"}
reader3, err := indexReader.DocIDReaderOnly(onlyIds)
if err != nil {
t.Errorf("Error accessing doc id reader: %v", err)
}
defer func() {
err := reader3.Close()
if err != nil {
t.Error(err)
}
}()
id, err = reader3.Next()
count = uint64(0)
for id != nil {
count++
id, err = reader3.Next()
}
if count != 1 {
t.Errorf("expected 1, got %d", count)
}
// commented out because advance works with internal ids
// this test presumes we see items in external doc id order
// which is no longer the case, so simply converting external ids
// to internal ones is not logically correct
// not removing though because we need some way to test Advance()
// // mix advance and next
// onlyIds = []string{"0", "1", "3", "5", "6", "9"}
// reader4, err := indexReader.DocIDReaderOnly(onlyIds)
// if err != nil {
// t.Errorf("Error accessing doc id reader: %v", err)
// }
// defer func() {
// err := reader4.Close()
// if err != nil {
// t.Error(err)
// }
// }()
//
// // first key is "1"
// id, err = reader4.Next()
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("1")) {
// t.Errorf("expected to find id '1', got '%s'", id)
// }
//
// // advancing to key we dont have gives next
// id, err = reader4.Advance(index.IndexInternalID("2"))
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("3")) {
// t.Errorf("expected to find id '3', got '%s'", id)
// }
//
// // next after advance works
// id, err = reader4.Next()
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("5")) {
// t.Errorf("expected to find id '5', got '%s'", id)
// }
//
// // advancing to key we do have works
// id, err = reader4.Advance(index.IndexInternalID("9"))
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("9")) {
// t.Errorf("expected to find id '9', got '%s'", id)
// }
//
// // advance backwards at end
// id, err = reader4.Advance(index.IndexInternalID("4"))
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("5")) {
// t.Errorf("expected to find id '5', got '%s'", id)
// }
//
// // next after advance works
// id, err = reader4.Next()
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("9")) {
// t.Errorf("expected to find id '9', got '%s'", id)
// }
//
// // advance backwards to key that exists, but not in only set
// id, err = reader4.Advance(index.IndexInternalID("7"))
// if err != nil {
// t.Error(err)
// }
// if !id.Equals(index.IndexInternalID("9")) {
// t.Errorf("expected to find id '9', got '%s'", id)
// }
}
func TestSegmentIndexAndLocalDocNumFromGlobal(t *testing.T) {
tests := []struct {
offsets []uint64
globalDocNum uint64
segmentIndex int
localDocNum uint64
}{
// just 1 segment
{
offsets: []uint64{0},
globalDocNum: 0,
segmentIndex: 0,
localDocNum: 0,
},
{
offsets: []uint64{0},
globalDocNum: 1,
segmentIndex: 0,
localDocNum: 1,
},
{
offsets: []uint64{0},
globalDocNum: 25,
segmentIndex: 0,
localDocNum: 25,
},
// now 2 segments, 30 docs in first
{
offsets: []uint64{0, 30},
globalDocNum: 0,
segmentIndex: 0,
localDocNum: 0,
},
{
offsets: []uint64{0, 30},
globalDocNum: 1,
segmentIndex: 0,
localDocNum: 1,
},
{
offsets: []uint64{0, 30},
globalDocNum: 25,
segmentIndex: 0,
localDocNum: 25,
},
{
offsets: []uint64{0, 30},
globalDocNum: 30,
segmentIndex: 1,
localDocNum: 0,
},
{
offsets: []uint64{0, 30},
globalDocNum: 35,
segmentIndex: 1,
localDocNum: 5,
},
// lots of segments
{
offsets: []uint64{0, 30, 40, 70, 99, 172, 800, 25000},
globalDocNum: 0,
segmentIndex: 0,
localDocNum: 0,
},
{
offsets: []uint64{0, 30, 40, 70, 99, 172, 800, 25000},
globalDocNum: 25,
segmentIndex: 0,
localDocNum: 25,
},
{
offsets: []uint64{0, 30, 40, 70, 99, 172, 800, 25000},
globalDocNum: 35,
segmentIndex: 1,
localDocNum: 5,
},
{
offsets: []uint64{0, 30, 40, 70, 99, 172, 800, 25000},
globalDocNum: 100,
segmentIndex: 4,
localDocNum: 1,
},
{
offsets: []uint64{0, 30, 40, 70, 99, 172, 800, 25000},
globalDocNum: 825,
segmentIndex: 6,
localDocNum: 25,
},
}
for _, test := range tests {
i := &IndexSnapshot{
offsets: test.offsets,
refs: 1,
}
gotSegmentIndex, gotLocalDocNum := i.segmentIndexAndLocalDocNumFromGlobal(test.globalDocNum)
if gotSegmentIndex != test.segmentIndex {
t.Errorf("got segment index %d expected %d for offsets %v globalDocNum %d", gotSegmentIndex, test.segmentIndex, test.offsets, test.globalDocNum)
}
if gotLocalDocNum != test.localDocNum {
t.Errorf("got localDocNum %d expected %d for offsets %v globalDocNum %d", gotLocalDocNum, test.localDocNum, test.offsets, test.globalDocNum)
}
err := i.DecRef()
if err != nil {
t.Errorf("expected no err, got: %v", err)
}
}
}

439
index/scorch/scorch.go Normal file
View File

@ -0,0 +1,439 @@
// Copyright (c) 2018 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"encoding/json"
"fmt"
"os"
"sync"
"sync/atomic"
"time"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
"github.com/blevesearch/bleve/index/store"
"github.com/blevesearch/bleve/registry"
"github.com/boltdb/bolt"
)
const Name = "scorch"
const Version uint8 = 1
// UnInvertIndex is implemented by various scorch index implementations
// to provide the un inverting of the postings or other indexed values.
type UnInvertIndex interface {
// apparently need better namings here..
VisitDocumentFieldTerms(localDocNum uint64, fields []string,
visitor index.DocumentFieldTermVisitor) error
}
type Scorch struct {
readOnly bool
version uint8
config map[string]interface{}
analysisQueue *index.AnalysisQueue
stats *Stats
nextSegmentID uint64
path string
unsafeBatch bool
rootLock sync.RWMutex
root *IndexSnapshot // holds 1 ref-count on the root
rootPersisted []chan error // closed when root is persisted
nextSnapshotEpoch uint64
eligibleForRemoval []uint64 // Index snapshot epochs that are safe to GC.
ineligibleForRemoval map[string]bool // Filenames that should not be GC'ed yet.
closeCh chan struct{}
introductions chan *segmentIntroduction
merges chan *segmentMerge
introducerNotifier chan *epochWatcher
revertToSnapshots chan *snapshotReversion
persisterNotifier chan notificationChan
rootBolt *bolt.DB
asyncTasks sync.WaitGroup
onEvent func(event Event)
onAsyncError func(err error)
}
func NewScorch(storeName string,
config map[string]interface{},
analysisQueue *index.AnalysisQueue) (index.Index, error) {
rv := &Scorch{
version: Version,
config: config,
analysisQueue: analysisQueue,
stats: &Stats{},
nextSnapshotEpoch: 1,
closeCh: make(chan struct{}),
ineligibleForRemoval: map[string]bool{},
}
rv.root = &IndexSnapshot{parent: rv, refs: 1}
ro, ok := config["read_only"].(bool)
if ok {
rv.readOnly = ro
}
ub, ok := config["unsafe_batch"].(bool)
if ok {
rv.unsafeBatch = ub
}
ecbName, ok := config["eventCallbackName"].(string)
if ok {
rv.onEvent = RegistryEventCallbacks[ecbName]
}
aecbName, ok := config["asyncErrorCallbackName"].(string)
if ok {
rv.onAsyncError = RegistryAsyncErrorCallbacks[aecbName]
}
return rv, nil
}
func (s *Scorch) fireEvent(kind EventKind, dur time.Duration) {
if s.onEvent != nil {
s.onEvent(Event{Kind: kind, Scorch: s, Duration: dur})
}
}
func (s *Scorch) fireAsyncError(err error) {
if s.onAsyncError != nil {
s.onAsyncError(err)
}
}
func (s *Scorch) Open() error {
var ok bool
s.path, ok = s.config["path"].(string)
if !ok {
return fmt.Errorf("must specify path")
}
if s.path == "" {
s.unsafeBatch = true
}
var rootBoltOpt *bolt.Options
if s.readOnly {
rootBoltOpt = &bolt.Options{
ReadOnly: true,
}
} else {
if s.path != "" {
err := os.MkdirAll(s.path, 0700)
if err != nil {
return err
}
}
}
rootBoltPath := s.path + string(os.PathSeparator) + "root.bolt"
var err error
if s.path != "" {
s.rootBolt, err = bolt.Open(rootBoltPath, 0600, rootBoltOpt)
if err != nil {
return err
}
// now see if there is any existing state to load
err = s.loadFromBolt()
if err != nil {
_ = s.Close()
return err
}
}
s.introductions = make(chan *segmentIntroduction)
s.merges = make(chan *segmentMerge)
s.introducerNotifier = make(chan *epochWatcher, 1)
s.revertToSnapshots = make(chan *snapshotReversion)
s.persisterNotifier = make(chan notificationChan)
if !s.readOnly && s.path != "" {
err := s.removeOldZapFiles() // Before persister or merger create any new files.
if err != nil {
_ = s.Close()
return err
}
}
s.asyncTasks.Add(1)
go s.mainLoop()
if !s.readOnly && s.path != "" {
s.asyncTasks.Add(1)
go s.persisterLoop()
s.asyncTasks.Add(1)
go s.mergerLoop()
}
return nil
}
func (s *Scorch) Close() (err error) {
startTime := time.Now()
defer func() {
s.fireEvent(EventKindClose, time.Since(startTime))
}()
s.fireEvent(EventKindCloseStart, 0)
// signal to async tasks we want to close
close(s.closeCh)
// wait for them to close
s.asyncTasks.Wait()
// now close the root bolt
if s.rootBolt != nil {
err = s.rootBolt.Close()
s.rootLock.Lock()
if s.root != nil {
_ = s.root.DecRef()
}
s.root = nil
s.rootLock.Unlock()
}
return
}
func (s *Scorch) Update(doc *document.Document) error {
b := index.NewBatch()
b.Update(doc)
return s.Batch(b)
}
func (s *Scorch) Delete(id string) error {
b := index.NewBatch()
b.Delete(id)
return s.Batch(b)
}
// Batch applices a batch of changes to the index atomically
func (s *Scorch) Batch(batch *index.Batch) error {
start := time.Now()
defer func() {
s.fireEvent(EventKindBatchIntroduction, time.Since(start))
}()
resultChan := make(chan *index.AnalysisResult, len(batch.IndexOps))
var numUpdates uint64
var numDeletes uint64
var numPlainTextBytes uint64
var ids []string
for docID, doc := range batch.IndexOps {
if doc != nil {
// insert _id field
doc.AddField(document.NewTextFieldCustom("_id", nil, []byte(doc.ID), document.IndexField|document.StoreField, nil))
numUpdates++
numPlainTextBytes += doc.NumPlainTextBytes()
} else {
numDeletes++
}
ids = append(ids, docID)
}
// FIXME could sort ids list concurrent with analysis?
go func() {
for _, doc := range batch.IndexOps {
if doc != nil {
aw := index.NewAnalysisWork(s, doc, resultChan)
// put the work on the queue
s.analysisQueue.Queue(aw)
}
}
}()
// wait for analysis result
analysisResults := make([]*index.AnalysisResult, int(numUpdates))
var itemsDeQueued uint64
for itemsDeQueued < numUpdates {
result := <-resultChan
analysisResults[itemsDeQueued] = result
itemsDeQueued++
}
close(resultChan)
atomic.AddUint64(&s.stats.analysisTime, uint64(time.Since(start)))
// notify handlers that we're about to introduce a segment
s.fireEvent(EventKindBatchIntroductionStart, 0)
var newSegment segment.Segment
if len(analysisResults) > 0 {
newSegment = mem.NewFromAnalyzedDocs(analysisResults)
}
err := s.prepareSegment(newSegment, ids, batch.InternalOps)
if err != nil {
if newSegment != nil {
_ = newSegment.Close()
}
atomic.AddUint64(&s.stats.errors, 1)
} else {
atomic.AddUint64(&s.stats.updates, numUpdates)
atomic.AddUint64(&s.stats.deletes, numDeletes)
atomic.AddUint64(&s.stats.batches, 1)
atomic.AddUint64(&s.stats.numPlainTextBytesIndexed, numPlainTextBytes)
}
return err
}
func (s *Scorch) prepareSegment(newSegment segment.Segment, ids []string,
internalOps map[string][]byte) error {
// new introduction
introduction := &segmentIntroduction{
id: atomic.AddUint64(&s.nextSegmentID, 1),
data: newSegment,
ids: ids,
obsoletes: make(map[uint64]*roaring.Bitmap),
internal: internalOps,
applied: make(chan error),
}
if !s.unsafeBatch {
introduction.persisted = make(chan error, 1)
}
// get read lock, to optimistically prepare obsoleted info
s.rootLock.RLock()
for _, seg := range s.root.segment {
delta, err := seg.segment.DocNumbers(ids)
if err != nil {
s.rootLock.RUnlock()
return err
}
introduction.obsoletes[seg.id] = delta
}
s.rootLock.RUnlock()
s.introductions <- introduction
// block until this segment is applied
err := <-introduction.applied
if err != nil {
return err
}
if introduction.persisted != nil {
err = <-introduction.persisted
}
return err
}
func (s *Scorch) SetInternal(key, val []byte) error {
b := index.NewBatch()
b.SetInternal(key, val)
return s.Batch(b)
}
func (s *Scorch) DeleteInternal(key []byte) error {
b := index.NewBatch()
b.DeleteInternal(key)
return s.Batch(b)
}
// Reader returns a low-level accessor on the index data. Close it to
// release associated resources.
func (s *Scorch) Reader() (index.IndexReader, error) {
s.rootLock.RLock()
rv := &Reader{root: s.root}
rv.root.AddRef()
s.rootLock.RUnlock()
return rv, nil
}
func (s *Scorch) Stats() json.Marshaler {
return s.stats
}
func (s *Scorch) StatsMap() map[string]interface{} {
return s.stats.statsMap()
}
func (s *Scorch) Analyze(d *document.Document) *index.AnalysisResult {
rv := &index.AnalysisResult{
Document: d,
Analyzed: make([]analysis.TokenFrequencies, len(d.Fields)+len(d.CompositeFields)),
Length: make([]int, len(d.Fields)+len(d.CompositeFields)),
}
for i, field := range d.Fields {
if field.Options().IsIndexed() {
fieldLength, tokenFreqs := field.Analyze()
rv.Analyzed[i] = tokenFreqs
rv.Length[i] = fieldLength
if len(d.CompositeFields) > 0 {
// see if any of the composite fields need this
for _, compositeField := range d.CompositeFields {
compositeField.Compose(field.Name(), fieldLength, tokenFreqs)
}
}
}
}
return rv
}
func (s *Scorch) Advanced() (store.KVStore, error) {
return nil, nil
}
func (s *Scorch) AddEligibleForRemoval(epoch uint64) {
s.rootLock.Lock()
if s.root == nil || s.root.epoch != epoch {
s.eligibleForRemoval = append(s.eligibleForRemoval, epoch)
}
s.rootLock.Unlock()
}
func (s *Scorch) MemoryUsed() uint64 {
var memUsed uint64
s.rootLock.RLock()
for _, segmentSnapshot := range s.root.segment {
memUsed += 8 /* size of id -> uint64 */ +
segmentSnapshot.segment.SizeInBytes()
if segmentSnapshot.deleted != nil {
memUsed += segmentSnapshot.deleted.GetSizeInBytes()
}
memUsed += segmentSnapshot.cachedDocs.sizeInBytes()
}
s.rootLock.RUnlock()
return memUsed
}
func (s *Scorch) markIneligibleForRemoval(filename string) {
s.rootLock.Lock()
s.ineligibleForRemoval[filename] = true
s.rootLock.Unlock()
}
func (s *Scorch) unmarkIneligibleForRemoval(filename string) {
s.rootLock.Lock()
delete(s.ineligibleForRemoval, filename)
s.rootLock.Unlock()
}
func init() {
registry.RegisterIndexType(Name, NewScorch)
}

1640
index/scorch/scorch_test.go Normal file

File diff suppressed because it is too large Load Diff

View File

@ -0,0 +1,95 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package segment
import (
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
)
type EmptySegment struct{}
func (e *EmptySegment) Dictionary(field string) (TermDictionary, error) {
return &EmptyDictionary{}, nil
}
func (e *EmptySegment) VisitDocument(num uint64, visitor DocumentFieldValueVisitor) error {
return nil
}
func (e *EmptySegment) Count() uint64 {
return 0
}
func (e *EmptySegment) DocNumbers([]string) (*roaring.Bitmap, error) {
r := roaring.NewBitmap()
return r, nil
}
func (e *EmptySegment) Fields() []string {
return []string{}
}
func (e *EmptySegment) Close() error {
return nil
}
func (e *EmptySegment) AddRef() {
}
func (e *EmptySegment) DecRef() error {
return nil
}
type EmptyDictionary struct{}
func (e *EmptyDictionary) PostingsList(term string,
except *roaring.Bitmap) (PostingsList, error) {
return &EmptyPostingsList{}, nil
}
func (e *EmptyDictionary) Iterator() DictionaryIterator {
return &EmptyDictionaryIterator{}
}
func (e *EmptyDictionary) PrefixIterator(prefix string) DictionaryIterator {
return &EmptyDictionaryIterator{}
}
func (e *EmptyDictionary) RangeIterator(start, end string) DictionaryIterator {
return &EmptyDictionaryIterator{}
}
type EmptyDictionaryIterator struct{}
func (e *EmptyDictionaryIterator) Next() (*index.DictEntry, error) {
return nil, nil
}
type EmptyPostingsList struct{}
func (e *EmptyPostingsList) Iterator() PostingsIterator {
return &EmptyPostingsIterator{}
}
func (e *EmptyPostingsList) Count() uint64 {
return 0
}
type EmptyPostingsIterator struct{}
func (e *EmptyPostingsIterator) Next() (Posting, error) {
return nil, nil
}

View File

@ -0,0 +1,94 @@
// Copyright 2014 The Cockroach Authors.
//
// 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.
// This code originated from:
// https://github.com/cockroachdb/cockroach/blob/2dd65dde5d90c157f4b93f92502ca1063b904e1d/pkg/util/encoding/encoding.go
// Modified to not use pkg/errors
package segment
import "fmt"
const (
MaxVarintSize = 9
// IntMin is chosen such that the range of int tags does not overlap the
// ascii character set that is frequently used in testing.
IntMin = 0x80 // 128
intMaxWidth = 8
intZero = IntMin + intMaxWidth // 136
intSmall = IntMax - intZero - intMaxWidth // 109
// IntMax is the maximum int tag value.
IntMax = 0xfd // 253
)
// EncodeUvarintAscending encodes the uint64 value using a variable length
// (length-prefixed) representation. The length is encoded as a single
// byte indicating the number of encoded bytes (-8) to follow. See
// EncodeVarintAscending for rationale. The encoded bytes are appended to the
// supplied buffer and the final buffer is returned.
func EncodeUvarintAscending(b []byte, v uint64) []byte {
switch {
case v <= intSmall:
return append(b, intZero+byte(v))
case v <= 0xff:
return append(b, IntMax-7, byte(v))
case v <= 0xffff:
return append(b, IntMax-6, byte(v>>8), byte(v))
case v <= 0xffffff:
return append(b, IntMax-5, byte(v>>16), byte(v>>8), byte(v))
case v <= 0xffffffff:
return append(b, IntMax-4, byte(v>>24), byte(v>>16), byte(v>>8), byte(v))
case v <= 0xffffffffff:
return append(b, IntMax-3, byte(v>>32), byte(v>>24), byte(v>>16), byte(v>>8),
byte(v))
case v <= 0xffffffffffff:
return append(b, IntMax-2, byte(v>>40), byte(v>>32), byte(v>>24), byte(v>>16),
byte(v>>8), byte(v))
case v <= 0xffffffffffffff:
return append(b, IntMax-1, byte(v>>48), byte(v>>40), byte(v>>32), byte(v>>24),
byte(v>>16), byte(v>>8), byte(v))
default:
return append(b, IntMax, byte(v>>56), byte(v>>48), byte(v>>40), byte(v>>32),
byte(v>>24), byte(v>>16), byte(v>>8), byte(v))
}
}
// DecodeUvarintAscending decodes a varint encoded uint64 from the input
// buffer. The remainder of the input buffer and the decoded uint64
// are returned.
func DecodeUvarintAscending(b []byte) ([]byte, uint64, error) {
if len(b) == 0 {
return nil, 0, fmt.Errorf("insufficient bytes to decode uvarint value")
}
length := int(b[0]) - intZero
b = b[1:] // skip length byte
if length <= intSmall {
return b, uint64(length), nil
}
length -= intSmall
if length < 0 || length > 8 {
return nil, 0, fmt.Errorf("invalid uvarint length of %d", length)
} else if len(b) < length {
return nil, 0, fmt.Errorf("insufficient bytes to decode uvarint value: %q", b)
}
var v uint64
// It is faster to range over the elements in a slice than to index
// into the slice on each loop iteration.
for _, t := range b[:length] {
v = (v << 8) | uint64(t)
}
return b[length:], v, nil
}

View File

@ -0,0 +1,96 @@
// Copyright 2014 The Cockroach Authors.
//
// 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.
// This code originated from:
// https://github.com/cockroachdb/cockroach/blob/2dd65dde5d90c157f4b93f92502ca1063b904e1d/pkg/util/encoding/encoding_test.go
// Modified to only test the parts we borrowed
package segment
import (
"bytes"
"math"
"testing"
)
type testCaseUint64 struct {
value uint64
expEnc []byte
}
func TestEncodeDecodeUvarint(t *testing.T) {
testBasicEncodeDecodeUint64(EncodeUvarintAscending, DecodeUvarintAscending, false, t)
testCases := []testCaseUint64{
{0, []byte{0x88}},
{1, []byte{0x89}},
{109, []byte{0xf5}},
{110, []byte{0xf6, 0x6e}},
{1 << 8, []byte{0xf7, 0x01, 0x00}},
{math.MaxUint64, []byte{0xfd, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff, 0xff}},
}
testCustomEncodeUint64(testCases, EncodeUvarintAscending, t)
}
func testBasicEncodeDecodeUint64(
encFunc func([]byte, uint64) []byte,
decFunc func([]byte) ([]byte, uint64, error),
descending bool, t *testing.T,
) {
testCases := []uint64{
0, 1,
1<<8 - 1, 1 << 8,
1<<16 - 1, 1 << 16,
1<<24 - 1, 1 << 24,
1<<32 - 1, 1 << 32,
1<<40 - 1, 1 << 40,
1<<48 - 1, 1 << 48,
1<<56 - 1, 1 << 56,
math.MaxUint64 - 1, math.MaxUint64,
}
var lastEnc []byte
for i, v := range testCases {
enc := encFunc(nil, v)
if i > 0 {
if (descending && bytes.Compare(enc, lastEnc) >= 0) ||
(!descending && bytes.Compare(enc, lastEnc) < 0) {
t.Errorf("ordered constraint violated for %d: [% x] vs. [% x]", v, enc, lastEnc)
}
}
b, decode, err := decFunc(enc)
if err != nil {
t.Error(err)
continue
}
if len(b) != 0 {
t.Errorf("leftover bytes: [% x]", b)
}
if decode != v {
t.Errorf("decode yielded different value than input: %d vs. %d", decode, v)
}
lastEnc = enc
}
}
func testCustomEncodeUint64(
testCases []testCaseUint64, encFunc func([]byte, uint64) []byte, t *testing.T,
) {
for _, test := range testCases {
enc := encFunc(nil, test.value)
if !bytes.Equal(enc, test.expEnc) {
t.Errorf("expected [% x]; got [% x] (value: %d)", test.expEnc, enc, test.value)
}
}
}

View File

@ -0,0 +1,250 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"math"
"sort"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
// NewFromAnalyzedDocs places the analyzed document mutations into this segment
func NewFromAnalyzedDocs(results []*index.AnalysisResult) *Segment {
s := New()
// ensure that _id field get fieldID 0
s.getOrDefineField("_id")
// walk each doc
for _, result := range results {
s.processDocument(result)
}
// go back and sort the dictKeys
for _, dict := range s.DictKeys {
sort.Strings(dict)
}
// compute memory usage of segment
s.updateSizeInBytes()
// professional debugging
//
// log.Printf("fields: %v\n", s.FieldsMap)
// log.Printf("fieldsInv: %v\n", s.FieldsInv)
// log.Printf("fieldsLoc: %v\n", s.FieldsLoc)
// log.Printf("dicts: %v\n", s.Dicts)
// log.Printf("dict keys: %v\n", s.DictKeys)
// for i, posting := range s.Postings {
// log.Printf("posting %d: %v\n", i, posting)
// }
// for i, freq := range s.Freqs {
// log.Printf("freq %d: %v\n", i, freq)
// }
// for i, norm := range s.Norms {
// log.Printf("norm %d: %v\n", i, norm)
// }
// for i, field := range s.Locfields {
// log.Printf("field %d: %v\n", i, field)
// }
// for i, start := range s.Locstarts {
// log.Printf("start %d: %v\n", i, start)
// }
// for i, end := range s.Locends {
// log.Printf("end %d: %v\n", i, end)
// }
// for i, pos := range s.Locpos {
// log.Printf("pos %d: %v\n", i, pos)
// }
// for i, apos := range s.Locarraypos {
// log.Printf("apos %d: %v\n", i, apos)
// }
// log.Printf("stored: %v\n", s.Stored)
// log.Printf("stored types: %v\n", s.StoredTypes)
// log.Printf("stored pos: %v\n", s.StoredPos)
return s
}
func (s *Segment) processDocument(result *index.AnalysisResult) {
// used to collate information across fields
docMap := map[uint16]analysis.TokenFrequencies{}
fieldLens := map[uint16]int{}
docNum := uint64(s.addDocument())
processField := func(field uint16, name string, l int, tf analysis.TokenFrequencies) {
fieldLens[field] += l
if existingFreqs, ok := docMap[field]; ok {
existingFreqs.MergeAll(name, tf)
} else {
docMap[field] = tf
}
}
storeField := func(docNum uint64, field uint16, typ byte, val []byte, pos []uint64) {
s.Stored[docNum][field] = append(s.Stored[docNum][field], val)
s.StoredTypes[docNum][field] = append(s.StoredTypes[docNum][field], typ)
s.StoredPos[docNum][field] = append(s.StoredPos[docNum][field], pos)
}
// walk each composite field
for _, field := range result.Document.CompositeFields {
fieldID := uint16(s.getOrDefineField(field.Name()))
l, tf := field.Analyze()
processField(fieldID, field.Name(), l, tf)
}
// walk each field
for i, field := range result.Document.Fields {
fieldID := uint16(s.getOrDefineField(field.Name()))
l := result.Length[i]
tf := result.Analyzed[i]
processField(fieldID, field.Name(), l, tf)
if field.Options().IsStored() {
storeField(docNum, fieldID, encodeFieldType(field), field.Value(), field.ArrayPositions())
}
// TODO with mapping changes for dv
//if field.Options().IncludeDocValues() {
s.DocValueFields[fieldID] = true
//}
}
// now that its been rolled up into docMap, walk that
for fieldID, tokenFrequencies := range docMap {
for term, tokenFreq := range tokenFrequencies {
fieldTermPostings := s.Dicts[fieldID][term]
// FIXME this if/else block has duplicate code that has resulted in
// bugs fixed/missed more than once, need to refactor
if fieldTermPostings == 0 {
// need to build new posting
bs := roaring.New()
bs.AddInt(int(docNum))
newPostingID := uint64(len(s.Postings) + 1)
// add this new bitset to the postings slice
s.Postings = append(s.Postings, bs)
locationBS := roaring.New()
s.PostingsLocs = append(s.PostingsLocs, locationBS)
// add this to the details slice
s.Freqs = append(s.Freqs, []uint64{uint64(tokenFreq.Frequency())})
s.Norms = append(s.Norms, []float32{float32(1.0 / math.Sqrt(float64(fieldLens[fieldID])))})
// add to locations
var locfields []uint16
var locstarts []uint64
var locends []uint64
var locpos []uint64
var locarraypos [][]uint64
if len(tokenFreq.Locations) > 0 {
locationBS.AddInt(int(docNum))
}
for _, loc := range tokenFreq.Locations {
var locf = fieldID
if loc.Field != "" {
locf = uint16(s.getOrDefineField(loc.Field))
}
locfields = append(locfields, locf)
locstarts = append(locstarts, uint64(loc.Start))
locends = append(locends, uint64(loc.End))
locpos = append(locpos, uint64(loc.Position))
if len(loc.ArrayPositions) > 0 {
locarraypos = append(locarraypos, loc.ArrayPositions)
} else {
locarraypos = append(locarraypos, nil)
}
}
s.Locfields = append(s.Locfields, locfields)
s.Locstarts = append(s.Locstarts, locstarts)
s.Locends = append(s.Locends, locends)
s.Locpos = append(s.Locpos, locpos)
s.Locarraypos = append(s.Locarraypos, locarraypos)
// record it
s.Dicts[fieldID][term] = newPostingID
// this term was new for this field, add it to dictKeys
s.DictKeys[fieldID] = append(s.DictKeys[fieldID], term)
} else {
// posting already started for this field/term
// the actual offset is - 1, because 0 is zero value
bs := s.Postings[fieldTermPostings-1]
bs.AddInt(int(docNum))
locationBS := s.PostingsLocs[fieldTermPostings-1]
s.Freqs[fieldTermPostings-1] = append(s.Freqs[fieldTermPostings-1], uint64(tokenFreq.Frequency()))
s.Norms[fieldTermPostings-1] = append(s.Norms[fieldTermPostings-1], float32(1.0/math.Sqrt(float64(fieldLens[fieldID]))))
if len(tokenFreq.Locations) > 0 {
locationBS.AddInt(int(docNum))
}
for _, loc := range tokenFreq.Locations {
var locf = fieldID
if loc.Field != "" {
locf = uint16(s.getOrDefineField(loc.Field))
}
s.Locfields[fieldTermPostings-1] = append(s.Locfields[fieldTermPostings-1], locf)
s.Locstarts[fieldTermPostings-1] = append(s.Locstarts[fieldTermPostings-1], uint64(loc.Start))
s.Locends[fieldTermPostings-1] = append(s.Locends[fieldTermPostings-1], uint64(loc.End))
s.Locpos[fieldTermPostings-1] = append(s.Locpos[fieldTermPostings-1], uint64(loc.Position))
if len(loc.ArrayPositions) > 0 {
s.Locarraypos[fieldTermPostings-1] = append(s.Locarraypos[fieldTermPostings-1], loc.ArrayPositions)
} else {
s.Locarraypos[fieldTermPostings-1] = append(s.Locarraypos[fieldTermPostings-1], nil)
}
}
}
}
}
}
func (s *Segment) getOrDefineField(name string) int {
fieldID, ok := s.FieldsMap[name]
if !ok {
fieldID = uint16(len(s.FieldsInv) + 1)
s.FieldsMap[name] = fieldID
s.FieldsInv = append(s.FieldsInv, name)
s.Dicts = append(s.Dicts, make(map[string]uint64))
s.DictKeys = append(s.DictKeys, make([]string, 0))
}
return int(fieldID - 1)
}
func (s *Segment) addDocument() int {
docNum := len(s.Stored)
s.Stored = append(s.Stored, map[uint16][][]byte{})
s.StoredTypes = append(s.StoredTypes, map[uint16][]byte{})
s.StoredPos = append(s.StoredPos, map[uint16][][]uint64{})
return docNum
}
func encodeFieldType(f document.Field) byte {
fieldType := byte('x')
switch f.(type) {
case *document.TextField:
fieldType = 't'
case *document.NumericField:
fieldType = 'n'
case *document.DateTimeField:
fieldType = 'd'
case *document.BooleanField:
fieldType = 'b'
case *document.GeoPointField:
fieldType = 'g'
case *document.CompositeField:
fieldType = 'c'
}
return fieldType
}

View File

@ -0,0 +1,102 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"sort"
"strings"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
)
// Dictionary is the in-memory representation of the term dictionary
type Dictionary struct {
segment *Segment
field string
fieldID uint16
}
// PostingsList returns the postings list for the specified term
func (d *Dictionary) PostingsList(term string,
except *roaring.Bitmap) (segment.PostingsList, error) {
return &PostingsList{
dictionary: d,
term: term,
postingsID: d.segment.Dicts[d.fieldID][term],
except: except,
}, nil
}
// Iterator returns an iterator for this dictionary
func (d *Dictionary) Iterator() segment.DictionaryIterator {
return &DictionaryIterator{
d: d,
}
}
// PrefixIterator returns an iterator which only visits terms having the
// the specified prefix
func (d *Dictionary) PrefixIterator(prefix string) segment.DictionaryIterator {
offset := sort.SearchStrings(d.segment.DictKeys[d.fieldID], prefix)
return &DictionaryIterator{
d: d,
prefix: prefix,
offset: offset,
}
}
// RangeIterator returns an iterator which only visits terms between the
// start and end terms. NOTE: bleve.index API specifies the end is inclusive.
func (d *Dictionary) RangeIterator(start, end string) segment.DictionaryIterator {
offset := sort.SearchStrings(d.segment.DictKeys[d.fieldID], start)
return &DictionaryIterator{
d: d,
offset: offset,
end: end,
}
}
// DictionaryIterator is an iterator for term dictionary
type DictionaryIterator struct {
d *Dictionary
prefix string
end string
offset int
}
// Next returns the next entry in the dictionary
func (d *DictionaryIterator) Next() (*index.DictEntry, error) {
if d.offset > len(d.d.segment.DictKeys[d.d.fieldID])-1 {
return nil, nil
}
next := d.d.segment.DictKeys[d.d.fieldID][d.offset]
// check prefix
if d.prefix != "" && !strings.HasPrefix(next, d.prefix) {
return nil, nil
}
// check end (bleve.index API demands inclusive end)
if d.end != "" && next > d.end {
return nil, nil
}
d.offset++
postingID := d.d.segment.Dicts[d.d.fieldID][next]
return &index.DictEntry{
Term: next,
Count: d.d.segment.Postings[postingID-1].GetCardinality(),
}, nil
}

View File

@ -0,0 +1,160 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"reflect"
"testing"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestDictionary(t *testing.T) {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("desc", nil, []byte("apple ball cat dog egg fish bat"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 5,
Position: 1,
Term: []byte("apple"),
},
&analysis.Token{
Start: 6,
End: 10,
Position: 2,
Term: []byte("ball"),
},
&analysis.Token{
Start: 11,
End: 14,
Position: 3,
Term: []byte("cat"),
},
&analysis.Token{
Start: 15,
End: 18,
Position: 4,
Term: []byte("dog"),
},
&analysis.Token{
Start: 19,
End: 22,
Position: 5,
Term: []byte("egg"),
},
&analysis.Token{
Start: 20,
End: 24,
Position: 6,
Term: []byte("fish"),
},
&analysis.Token{
Start: 25,
End: 28,
Position: 7,
Term: []byte("bat"),
},
}, nil, true),
},
Length: []int{
1,
7,
},
},
}
segment := NewFromAnalyzedDocs(results)
if segment == nil {
t.Fatalf("segment nil, not expected")
}
dict, err := segment.Dictionary("desc")
if err != nil {
t.Fatal(err)
}
// test basic full iterator
expected := []string{"apple", "ball", "bat", "cat", "dog", "egg", "fish"}
var got []string
itr := dict.Iterator()
next, err := itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
// test prefix iterator
expected = []string{"ball", "bat"}
got = got[:0]
itr = dict.PrefixIterator("b")
next, err = itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
// test range iterator
expected = []string{"cat", "dog", "egg"}
got = got[:0]
itr = dict.RangeIterator("cat", "egg")
next, err = itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
}

View File

@ -0,0 +1,178 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index/scorch/segment"
)
// PostingsList is an in-memory represenation of a postings list
type PostingsList struct {
dictionary *Dictionary
term string
postingsID uint64
except *roaring.Bitmap
}
// Count returns the number of items on this postings list
func (p *PostingsList) Count() uint64 {
var rv uint64
if p.postingsID > 0 {
rv = p.dictionary.segment.Postings[p.postingsID-1].GetCardinality()
if p.except != nil {
except := p.except.GetCardinality()
if except > rv {
// avoid underflow
except = rv
}
rv -= except
}
}
return rv
}
// Iterator returns an iterator for this postings list
func (p *PostingsList) Iterator() segment.PostingsIterator {
rv := &PostingsIterator{
postings: p,
}
if p.postingsID > 0 {
allbits := p.dictionary.segment.Postings[p.postingsID-1]
rv.locations = p.dictionary.segment.PostingsLocs[p.postingsID-1]
rv.all = allbits.Iterator()
if p.except != nil {
allExcept := allbits.Clone()
allExcept.AndNot(p.except)
rv.actual = allExcept.Iterator()
} else {
rv.actual = allbits.Iterator()
}
}
return rv
}
// PostingsIterator provides a way to iterate through the postings list
type PostingsIterator struct {
postings *PostingsList
all roaring.IntIterable
locations *roaring.Bitmap
offset int
locoffset int
actual roaring.IntIterable
}
// Next returns the next posting on the postings list, or nil at the end
func (i *PostingsIterator) Next() (segment.Posting, error) {
if i.actual == nil || !i.actual.HasNext() {
return nil, nil
}
n := i.actual.Next()
allN := i.all.Next()
// n is the next actual hit (excluding some postings)
// allN is the next hit in the full postings
// if they don't match, adjust offsets to factor in item we're skipping over
// incr the all iterator, and check again
for allN != n {
i.locoffset += int(i.postings.dictionary.segment.Freqs[i.postings.postingsID-1][i.offset])
i.offset++
allN = i.all.Next()
}
rv := &Posting{
iterator: i,
docNum: uint64(n),
offset: i.offset,
locoffset: i.locoffset,
hasLoc: i.locations.Contains(n),
}
i.locoffset += int(i.postings.dictionary.segment.Freqs[i.postings.postingsID-1][i.offset])
i.offset++
return rv, nil
}
// Posting is a single entry in a postings list
type Posting struct {
iterator *PostingsIterator
docNum uint64
offset int
locoffset int
hasLoc bool
}
// Number returns the document number of this posting in this segment
func (p *Posting) Number() uint64 {
return p.docNum
}
// Frequency returns the frequence of occurance of this term in this doc/field
func (p *Posting) Frequency() uint64 {
return p.iterator.postings.dictionary.segment.Freqs[p.iterator.postings.postingsID-1][p.offset]
}
// Norm returns the normalization factor for this posting
func (p *Posting) Norm() float64 {
return float64(p.iterator.postings.dictionary.segment.Norms[p.iterator.postings.postingsID-1][p.offset])
}
// Locations returns the location information for each occurance
func (p *Posting) Locations() []segment.Location {
if !p.hasLoc {
return nil
}
freq := int(p.Frequency())
rv := make([]segment.Location, freq)
for i := 0; i < freq; i++ {
rv[i] = &Location{
p: p,
offset: p.locoffset + i,
}
}
return rv
}
// Location represents the location of a single occurance
type Location struct {
p *Posting
offset int
}
// Field returns the name of the field (useful in composite fields to know
// which original field the value came from)
func (l *Location) Field() string {
return l.p.iterator.postings.dictionary.segment.FieldsInv[l.p.iterator.postings.dictionary.segment.Locfields[l.p.iterator.postings.postingsID-1][l.offset]]
}
// Start returns the start byte offset of this occurance
func (l *Location) Start() uint64 {
return l.p.iterator.postings.dictionary.segment.Locstarts[l.p.iterator.postings.postingsID-1][l.offset]
}
// End returns the end byte offset of this occurance
func (l *Location) End() uint64 {
return l.p.iterator.postings.dictionary.segment.Locends[l.p.iterator.postings.postingsID-1][l.offset]
}
// Pos returns the 1-based phrase position of this occurance
func (l *Location) Pos() uint64 {
return l.p.iterator.postings.dictionary.segment.Locpos[l.p.iterator.postings.postingsID-1][l.offset]
}
// ArrayPositions returns the array position vector associated with this occurance
func (l *Location) ArrayPositions() []uint64 {
return l.p.iterator.postings.dictionary.segment.Locarraypos[l.p.iterator.postings.postingsID-1][l.offset]
}

View File

@ -0,0 +1,252 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"fmt"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index/scorch/segment"
)
// _id field is always guaranteed to have fieldID of 0
const idFieldID uint16 = 0
// KNOWN ISSUES
// - LIMITATION - we decided whether or not to store term vectors for a field
// at the segment level, based on the first definition of a
// field we see. in normal bleve usage this is fine, all
// instances of a field definition will be the same. however,
// advanced users may violate this and provide unique field
// definitions with each document. this segment does not
// support this usage.
// TODO
// - need better testing of multiple docs, iterating freqs, locations and
// and verifying the correct results are returned
// Segment is an in memory implementation of scorch.Segment
type Segment struct {
// FieldsMap name -> id+1
FieldsMap map[string]uint16
// fields id -> name
FieldsInv []string
// term dictionary
// field id -> term -> posting id + 1
Dicts []map[string]uint64
// term dictionary keys
// field id -> []dictionary keys
DictKeys [][]string
// Postings list
// Postings list id -> Postings bitmap
Postings []*roaring.Bitmap
// Postings List has locations
PostingsLocs []*roaring.Bitmap
// term frequencies
// postings list id -> Freqs (one for each hit in bitmap)
Freqs [][]uint64
// field Norms
// postings list id -> Norms (one for each hit in bitmap)
Norms [][]float32
// field/start/end/pos/locarraypos
// postings list id -> start/end/pos/locarraypos (one for each freq)
Locfields [][]uint16
Locstarts [][]uint64
Locends [][]uint64
Locpos [][]uint64
Locarraypos [][][]uint64
// Stored field values
// docNum -> field id -> slice of values (each value []byte)
Stored []map[uint16][][]byte
// stored field types
// docNum -> field id -> slice of types (each type byte)
StoredTypes []map[uint16][]byte
// stored field array positions
// docNum -> field id -> slice of array positions (each is []uint64)
StoredPos []map[uint16][][]uint64
// for storing the docValue persisted fields
DocValueFields map[uint16]bool
// footprint of the segment, updated when analyzed document mutations
// are added into the segment
sizeInBytes uint64
}
// New builds a new empty Segment
func New() *Segment {
return &Segment{
FieldsMap: map[string]uint16{},
DocValueFields: map[uint16]bool{},
}
}
func (s *Segment) updateSizeInBytes() {
var sizeInBytes uint64
for k, _ := range s.FieldsMap {
sizeInBytes += uint64(len(k)*2 /* FieldsMap + FieldsInv */ +
2 /* size of uint16 */)
}
for _, entry := range s.Dicts {
for k, _ := range entry {
sizeInBytes += uint64(len(k)*2 /* Dicts + DictKeys */ +
8 /* size of uint64 */)
}
}
for i := 0; i < len(s.Postings); i++ {
sizeInBytes += s.Postings[i].GetSizeInBytes() + s.PostingsLocs[i].GetSizeInBytes()
}
for i := 0; i < len(s.Freqs); i++ {
sizeInBytes += uint64(len(s.Freqs[i])*8 /* size of uint64 */ +
len(s.Norms[i])*4 /* size of float32 */)
}
for i := 0; i < len(s.Locfields); i++ {
sizeInBytes += uint64(len(s.Locfields[i])*2 /* size of uint16 */ +
len(s.Locstarts[i])*8 /* size of uint64 */ +
len(s.Locends[i])*8 /* size of uint64 */ +
len(s.Locpos[i])*8 /* size of uint64 */)
for j := 0; j < len(s.Locarraypos[i]); j++ {
sizeInBytes += uint64(len(s.Locarraypos[i][j]) * 8 /* size of uint64 */)
}
}
for i := 0; i < len(s.Stored); i++ {
for _, v := range s.Stored[i] {
sizeInBytes += uint64(2 /* size of uint16 */)
for _, arr := range v {
sizeInBytes += uint64(len(arr))
}
}
for _, v := range s.StoredTypes[i] {
sizeInBytes += uint64(2 /* size of uint16 */ + len(v))
}
for _, v := range s.StoredPos[i] {
sizeInBytes += uint64(2 /* size of uint16 */)
for _, arr := range v {
sizeInBytes += uint64(len(arr) * 8 /* size of uint64 */)
}
}
}
sizeInBytes += uint64(8 /* size of sizeInBytes -> uint64*/)
s.sizeInBytes = sizeInBytes
}
func (s *Segment) SizeInBytes() uint64 {
return s.sizeInBytes
}
func (s *Segment) AddRef() {
}
func (s *Segment) DecRef() error {
return nil
}
// Fields returns the field names used in this segment
func (s *Segment) Fields() []string {
return s.FieldsInv
}
// VisitDocument invokes the DocFieldValueVistor for each stored field
// for the specified doc number
func (s *Segment) VisitDocument(num uint64, visitor segment.DocumentFieldValueVisitor) error {
// ensure document number exists
if int(num) > len(s.Stored)-1 {
return nil
}
docFields := s.Stored[int(num)]
for field, values := range docFields {
for i, value := range values {
keepGoing := visitor(s.FieldsInv[field], s.StoredTypes[int(num)][field][i], value, s.StoredPos[int(num)][field][i])
if !keepGoing {
return nil
}
}
}
return nil
}
func (s *Segment) getField(name string) (int, error) {
fieldID, ok := s.FieldsMap[name]
if !ok {
return 0, fmt.Errorf("no field named %s", name)
}
return int(fieldID - 1), nil
}
// Dictionary returns the term dictionary for the specified field
func (s *Segment) Dictionary(field string) (segment.TermDictionary, error) {
fieldID, err := s.getField(field)
if err != nil {
// no such field, return empty dictionary
return &segment.EmptyDictionary{}, nil
}
return &Dictionary{
segment: s,
field: field,
fieldID: uint16(fieldID),
}, nil
}
// Count returns the number of documents in this segment
// (this has no notion of deleted docs)
func (s *Segment) Count() uint64 {
return uint64(len(s.Stored))
}
// DocNumbers returns a bitset corresponding to the doc numbers of all the
// provided _id strings
func (s *Segment) DocNumbers(ids []string) (*roaring.Bitmap, error) {
rv := roaring.New()
// guard against empty segment
if len(s.FieldsMap) > 0 {
idDictionary := s.Dicts[idFieldID]
for _, id := range ids {
postingID := idDictionary[id]
if postingID > 0 {
rv.Or(s.Postings[postingID-1])
}
}
}
return rv, nil
}
// Close releases all resources associated with this segment
func (s *Segment) Close() error {
return nil
}

View File

@ -0,0 +1,699 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package mem
import (
"math"
"reflect"
"testing"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestEmpty(t *testing.T) {
emptySegment := New()
if emptySegment.Count() != 0 {
t.Errorf("expected count 0, got %d", emptySegment.Count())
}
dict, err := emptySegment.Dictionary("name")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("marty", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 0 {
t.Errorf("expected count to be 0, got %d", count)
}
// now try and visit a document
err = emptySegment.VisitDocument(0, func(field string, typ byte, value []byte, pos []uint64) bool {
t.Errorf("document visitor called, not expected")
return true
})
if err != nil {
t.Fatal(err)
}
}
func TestSingle(t *testing.T) {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("wow"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, nil),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("wow"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
}
// fix up composite fields
for _, ar := range results {
for i, f := range ar.Document.Fields {
for _, cf := range ar.Document.CompositeFields {
cf.Compose(f.Name(), ar.Length[i], ar.Analyzed[i])
}
}
}
segment := NewFromAnalyzedDocs(results)
if segment == nil {
t.Fatalf("segment nil, not expected")
}
if segment.SizeInBytes() <= 0 {
t.Fatalf("segment size not updated")
}
expectFields := map[string]struct{}{
"_id": struct{}{},
"_all": struct{}{},
"name": struct{}{},
"desc": struct{}{},
"tag": struct{}{},
}
fields := segment.Fields()
if len(fields) != len(expectFields) {
t.Errorf("expected %d fields, only got %d", len(expectFields), len(fields))
}
for _, field := range fields {
if _, ok := expectFields[field]; !ok {
t.Errorf("got unexpected field: %s", field)
}
}
if segment.Count() != 1 {
t.Errorf("expected count 1, got %d", segment.Count())
}
// check the _id field
dict, err := segment.Dictionary("_id")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("a", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
if nextPosting.Norm() != 1.0 {
t.Errorf("expected norm 1.0, got %f", nextPosting.Norm())
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// check the name field
dict, err = segment.Dictionary("name")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("wow", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
if nextPosting.Norm() != 1.0 {
t.Errorf("expected norm 1.0, got %f", nextPosting.Norm())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "name" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 3 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
if loc.ArrayPositions() != nil {
t.Errorf("expect loc array pos to be nil, got %v", loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// check the _all field (composite)
dict, err = segment.Dictionary("_all")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("wow", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
expectedNorm := float32(1.0 / math.Sqrt(float64(6)))
if nextPosting.Norm() != float64(expectedNorm) {
t.Errorf("expected norm %f, got %f", expectedNorm, nextPosting.Norm())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "name" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 3 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
if loc.ArrayPositions() != nil {
t.Errorf("expect loc array pos to be nil, got %v", loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// now try a field with array positions
dict, err = segment.Dictionary("tag")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("dark", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "tag" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 4 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
expectArrayPos := []uint64{1}
if !reflect.DeepEqual(loc.ArrayPositions(), expectArrayPos) {
t.Errorf("expect loc array pos to be %v, got %v", expectArrayPos, loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
// now try and visit a document
var fieldValuesSeen int
err = segment.VisitDocument(0, func(field string, typ byte, value []byte, pos []uint64) bool {
fieldValuesSeen++
return true
})
if err != nil {
t.Fatal(err)
}
if fieldValuesSeen != 5 {
t.Errorf("expected 5 field values, got %d", fieldValuesSeen)
}
}
func TestMultiple(t *testing.T) {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("wow"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, nil),
},
}
doc2 := &document.Document{
ID: "b",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("b"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("who"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, nil),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("wow"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
&index.AnalysisResult{
Document: doc2,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("b"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("who"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
}
// fix up composite fields
for _, ar := range results {
for i, f := range ar.Document.Fields {
for _, cf := range ar.Document.CompositeFields {
cf.Compose(f.Name(), ar.Length[i], ar.Analyzed[i])
}
}
}
segment := NewFromAnalyzedDocs(results)
if segment == nil {
t.Fatalf("segment nil, not expected")
}
if segment.Count() != 2 {
t.Errorf("expected count 2, got %d", segment.Count())
}
// check the desc field
dict, err := segment.Dictionary("desc")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("thing", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 2 {
t.Errorf("expected count to be 2, got %d", count)
}
// get docnum of a
exclude, err := segment.DocNumbers([]string{"a"})
if err != nil {
t.Fatal(err)
}
// look for term 'thing' excluding doc 'a'
postingsListExcluding, err := dict.PostingsList("thing", exclude)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsListExcludingCount := postingsListExcluding.Count()
if postingsListExcludingCount != 1 {
t.Errorf("expected count from postings list to be 1, got %d", postingsListExcludingCount)
}
postingsItrExcluding := postingsListExcluding.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItrExcluding.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItrExcluding.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
}

View File

@ -0,0 +1,99 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package segment
import (
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
)
// DocumentFieldValueVisitor defines a callback to be visited for each
// stored field value. The return value determines if the visitor
// should keep going. Returning true continues visiting, false stops.
type DocumentFieldValueVisitor func(field string, typ byte, value []byte, pos []uint64) bool
type Segment interface {
Dictionary(field string) (TermDictionary, error)
VisitDocument(num uint64, visitor DocumentFieldValueVisitor) error
Count() uint64
DocNumbers([]string) (*roaring.Bitmap, error)
Fields() []string
Close() error
SizeInBytes() uint64
AddRef()
DecRef() error
}
type TermDictionary interface {
PostingsList(term string, except *roaring.Bitmap) (PostingsList, error)
Iterator() DictionaryIterator
PrefixIterator(prefix string) DictionaryIterator
RangeIterator(start, end string) DictionaryIterator
}
type DictionaryIterator interface {
Next() (*index.DictEntry, error)
}
type PostingsList interface {
Iterator() PostingsIterator
Count() uint64
// NOTE deferred for future work
// And(other PostingsList) PostingsList
// Or(other PostingsList) PostingsList
}
type PostingsIterator interface {
// The caller is responsible for copying whatever it needs from
// the returned Posting instance before calling Next(), as some
// implementations may return a shared instance to reduce memory
// allocations.
Next() (Posting, error)
}
type Posting interface {
Number() uint64
Frequency() uint64
Norm() float64
Locations() []Location
}
type Location interface {
Field() string
Start() uint64
End() uint64
Pos() uint64
ArrayPositions() []uint64
}
// DocumentFieldTermVisitable is implemented by various scorch segment
// implementations to provide the un inverting of the postings
// or other indexed values.
type DocumentFieldTermVisitable interface {
VisitDocumentFieldTerms(localDocNum uint64, fields []string,
visitor index.DocumentFieldTermVisitor) error
}

View File

@ -0,0 +1,167 @@
# zap file format
The file is written in the reverse order that we typically access data. This helps us write in one pass since later sections of the file require file offsets of things we've already written.
Current usage:
- mmap the entire file
- crc-32 bytes and version are in fixed position at end of the file
- reading remainder of footer could be version specific
- remainder of footer gives us:
- 3 important offsets (docValue , fields index and stored data index)
- 2 important values (number of docs and chunk factor)
- field data is processed once and memoized onto the heap so that we never have to go back to disk for it
- access to stored data by doc number means first navigating to the stored data index, then accessing a fixed position offset into that slice, which gives us the actual address of the data. the first bytes of that section tell us the size of data so that we know where it ends.
- access to all other indexed data follows the following pattern:
- first know the field name -> convert to id
- next navigate to term dictionary for that field
- some operations stop here and do dictionary ops
- next use dictionary to navigate to posting list for a specific term
- walk posting list
- if necessary, walk posting details as we go
- if location info is desired, consult location bitmap to see if it is there
## stored fields section
- for each document
- preparation phase:
- produce a slice of metadata bytes and data bytes
- produce these slices in field id order
- field value is appended to the data slice
- metadata slice is govarint encoded with the following values for each field value
- field id (uint16)
- field type (byte)
- field value start offset in uncompressed data slice (uint64)
- field value length (uint64)
- field number of array positions (uint64)
- one additional value for each array position (uint64)
- compress the data slice using snappy
- file writing phase:
- remember the start offset for this document
- write out meta data length (varint uint64)
- write out compressed data length (varint uint64)
- write out the metadata bytes
- write out the compressed data bytes
## stored fields idx
- for each document
- write start offset (remembered from previous section) of stored data (big endian uint64)
With this index and a known document number, we have direct access to all the stored field data.
## posting details (freq/norm) section
- for each posting list
- produce a slice containing multiple consecutive chunks (each chunk is govarint stream)
- produce a slice remembering offsets of where each chunk starts
- preparation phase:
- for each hit in the posting list
- if this hit is in next chunk close out encoding of last chunk and record offset start of next
- encode term frequency (uint64)
- encode norm factor (float32)
- file writing phase:
- remember start position for this posting list details
- write out number of chunks that follow (varint uint64)
- write out length of each chunk (each a varint uint64)
- write out the byte slice containing all the chunk data
If you know the doc number you're interested in, this format lets you jump to the correct chunk (docNum/chunkFactor) directly and then seek within that chunk until you find it.
## posting details (location) section
- for each posting list
- produce a slice containing multiple consecutive chunks (each chunk is govarint stream)
- produce a slice remembering offsets of where each chunk starts
- preparation phase:
- for each hit in the posting list
- if this hit is in next chunk close out encoding of last chunk and record offset start of next
- encode field (uint16)
- encode field pos (uint64)
- encode field start (uint64)
- encode field end (uint64)
- encode number of array positions to follow (uint64)
- encode each array position (each uint64)
- file writing phase:
- remember start position for this posting list details
- write out number of chunks that follow (varint uint64)
- write out length of each chunk (each a varint uint64)
- write out the byte slice containing all the chunk data
If you know the doc number you're interested in, this format lets you jump to the correct chunk (docNum/chunkFactor) directly and then seek within that chunk until you find it.
## bitmaps of hits with location info
- for each posting list
- preparation phase:
- encode roaring bitmap (inidicating which hits have location details indexed) posting list to bytes (so we know the length)
- file writing phase:
- remember the start position for this bitmap
- write length of encoded roaring bitmap
- write the serialized roaring bitmap data
## postings list section
- for each posting list
- preparation phase:
- encode roaring bitmap posting list to bytes (so we know the length)
- file writing phase:
- remember the start position for this posting list
- write freq/norm details offset (remembered from previous, as varint uint64)
- write location details offset (remembered from previous, as varint uint64)
- write location bitmap offset (remembered from pervious, as varint uint64)
- write length of encoded roaring bitmap
- write the serialized roaring bitmap data
## dictionary
- for each field
- preparation phase:
- encode vellum FST with dictionary data pointing to file offset of posting list (remembered from previous)
- file writing phase:
- remember the start position of this persistDictionary
- write length of vellum data (varint uint64)
- write out vellum data
## fields section
- for each field
- file writing phase:
- remember start offset for each field
- write dictionary address (remembered from previous) (varint uint64)
- write length of field name (varint uint64)
- write field name bytes
## fields idx
- for each field
- file writing phase:
- write big endian uint64 of start offset for each field
NOTE: currently we don't know or record the length of this fields index. Instead we rely on the fact that we know it immediately precedes a footer of known size.
## fields DocValue
- for each field
- preparation phase:
- produce a slice containing multiple consecutive chunks, where each chunk is composed of a meta section followed by compressed columnar field data
- produce a slice remembering the length of each chunk
- file writing phase:
- remember the start position of this first field DocValue offset in the footer
- write out number of chunks that follow (varint uint64)
- write out length of each chunk (each a varint uint64)
- write out the byte slice containing all the chunk data
NOTE: currently the meta header inside each chunk gives clue to the location offsets and size of the data pertaining to a given docID and any
read operation leverage that meta information to extract the document specific data from the file.
## footer
- file writing phase
- write number of docs (big endian uint64)
- write stored field index location (big endian uint64)
- write field index location (big endian uint64)
- write field docValue location (big endian uint64)
- write out chunk factor (big endian uint32)
- write out version (big endian uint32)
- write out file CRC of everything preceding this (big endian uint32)

View File

@ -0,0 +1,534 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bufio"
"bytes"
"encoding/binary"
"math"
"os"
"sort"
"github.com/Smerity/govarint"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
"github.com/couchbase/vellum"
"github.com/golang/snappy"
)
const version uint32 = 2
const fieldNotUninverted = math.MaxUint64
// PersistSegment takes the in-memory segment and persists it to the specified
// path in the zap file format.
func PersistSegment(memSegment *mem.Segment, path string, chunkFactor uint32) (err error) {
flag := os.O_RDWR | os.O_CREATE
f, err := os.OpenFile(path, flag, 0600)
if err != nil {
return err
}
// bufer the output
br := bufio.NewWriter(f)
// wrap it for counting (tracking offsets)
cr := NewCountHashWriter(br)
var storedIndexOffset uint64
var dictLocs []uint64
docValueOffset := uint64(fieldNotUninverted)
if len(memSegment.Stored) > 0 {
storedIndexOffset, err = persistStored(memSegment, cr)
if err != nil {
return err
}
var freqOffsets, locOffsets []uint64
freqOffsets, locOffsets, err = persistPostingDetails(memSegment, cr, chunkFactor)
if err != nil {
return err
}
var postingsListLocs []uint64
postingsListLocs, err = persistPostingsLocs(memSegment, cr)
if err != nil {
return err
}
var postingsLocs []uint64
postingsLocs, err = persistPostingsLists(memSegment, cr, postingsListLocs, freqOffsets, locOffsets)
if err != nil {
return err
}
dictLocs, err = persistDictionary(memSegment, cr, postingsLocs)
if err != nil {
return err
}
docValueOffset, err = persistFieldDocValues(cr, chunkFactor, memSegment)
if err != nil {
return err
}
} else {
dictLocs = make([]uint64, len(memSegment.FieldsInv))
}
var fieldIndexStart uint64
fieldIndexStart, err = persistFields(memSegment.FieldsInv, cr, dictLocs)
if err != nil {
return err
}
err = persistFooter(uint64(len(memSegment.Stored)), storedIndexOffset,
fieldIndexStart, docValueOffset, chunkFactor, cr)
if err != nil {
return err
}
err = br.Flush()
if err != nil {
return err
}
err = f.Sync()
if err != nil {
return err
}
err = f.Close()
if err != nil {
return err
}
return nil
}
func persistStored(memSegment *mem.Segment, w *CountHashWriter) (uint64, error) {
var curr int
var metaBuf bytes.Buffer
var data, compressed []byte
docNumOffsets := make(map[int]uint64, len(memSegment.Stored))
for docNum, storedValues := range memSegment.Stored {
if docNum != 0 {
// reset buffer if necessary
metaBuf.Reset()
data = data[:0]
compressed = compressed[:0]
curr = 0
}
metaEncoder := govarint.NewU64Base128Encoder(&metaBuf)
// encode fields in order
for fieldID := range memSegment.FieldsInv {
if storedFieldValues, ok := storedValues[uint16(fieldID)]; ok {
// has stored values for this field
num := len(storedFieldValues)
// process each value
for i := 0; i < num; i++ {
// encode field
_, err2 := metaEncoder.PutU64(uint64(fieldID))
if err2 != nil {
return 0, err2
}
// encode type
_, err2 = metaEncoder.PutU64(uint64(memSegment.StoredTypes[docNum][uint16(fieldID)][i]))
if err2 != nil {
return 0, err2
}
// encode start offset
_, err2 = metaEncoder.PutU64(uint64(curr))
if err2 != nil {
return 0, err2
}
// end len
_, err2 = metaEncoder.PutU64(uint64(len(storedFieldValues[i])))
if err2 != nil {
return 0, err2
}
// encode number of array pos
_, err2 = metaEncoder.PutU64(uint64(len(memSegment.StoredPos[docNum][uint16(fieldID)][i])))
if err2 != nil {
return 0, err2
}
// encode all array positions
for j := 0; j < len(memSegment.StoredPos[docNum][uint16(fieldID)][i]); j++ {
_, err2 = metaEncoder.PutU64(memSegment.StoredPos[docNum][uint16(fieldID)][i][j])
if err2 != nil {
return 0, err2
}
}
// append data
data = append(data, storedFieldValues[i]...)
// update curr
curr += len(storedFieldValues[i])
}
}
}
metaEncoder.Close()
metaBytes := metaBuf.Bytes()
// compress the data
compressed = snappy.Encode(compressed, data)
// record where we're about to start writing
docNumOffsets[docNum] = uint64(w.Count())
// write out the meta len and compressed data len
_, err := writeUvarints(w, uint64(len(metaBytes)), uint64(len(compressed)))
if err != nil {
return 0, err
}
// now write the meta
_, err = w.Write(metaBytes)
if err != nil {
return 0, err
}
// now write the compressed data
_, err = w.Write(compressed)
if err != nil {
return 0, err
}
}
// return value is the start of the stored index
rv := uint64(w.Count())
// now write out the stored doc index
for docNum := range memSegment.Stored {
err := binary.Write(w, binary.BigEndian, docNumOffsets[docNum])
if err != nil {
return 0, err
}
}
return rv, nil
}
func persistPostingDetails(memSegment *mem.Segment, w *CountHashWriter, chunkFactor uint32) ([]uint64, []uint64, error) {
var freqOffsets, locOfffsets []uint64
tfEncoder := newChunkedIntCoder(uint64(chunkFactor), uint64(len(memSegment.Stored)-1))
for postingID := range memSegment.Postings {
if postingID != 0 {
tfEncoder.Reset()
}
postingsListItr := memSegment.Postings[postingID].Iterator()
var offset int
for postingsListItr.HasNext() {
docNum := uint64(postingsListItr.Next())
// put freq
err := tfEncoder.Add(docNum, memSegment.Freqs[postingID][offset])
if err != nil {
return nil, nil, err
}
// put norm
norm := memSegment.Norms[postingID][offset]
normBits := math.Float32bits(norm)
err = tfEncoder.Add(docNum, uint64(normBits))
if err != nil {
return nil, nil, err
}
offset++
}
// record where this postings freq info starts
freqOffsets = append(freqOffsets, uint64(w.Count()))
tfEncoder.Close()
_, err := tfEncoder.Write(w)
if err != nil {
return nil, nil, err
}
}
// now do it again for the locations
locEncoder := newChunkedIntCoder(uint64(chunkFactor), uint64(len(memSegment.Stored)-1))
for postingID := range memSegment.Postings {
if postingID != 0 {
locEncoder.Reset()
}
postingsListItr := memSegment.Postings[postingID].Iterator()
var offset int
var locOffset int
for postingsListItr.HasNext() {
docNum := uint64(postingsListItr.Next())
for i := 0; i < int(memSegment.Freqs[postingID][offset]); i++ {
if len(memSegment.Locfields[postingID]) > 0 {
// put field
err := locEncoder.Add(docNum, uint64(memSegment.Locfields[postingID][locOffset]))
if err != nil {
return nil, nil, err
}
// put pos
err = locEncoder.Add(docNum, memSegment.Locpos[postingID][locOffset])
if err != nil {
return nil, nil, err
}
// put start
err = locEncoder.Add(docNum, memSegment.Locstarts[postingID][locOffset])
if err != nil {
return nil, nil, err
}
// put end
err = locEncoder.Add(docNum, memSegment.Locends[postingID][locOffset])
if err != nil {
return nil, nil, err
}
// put array positions
num := len(memSegment.Locarraypos[postingID][locOffset])
// put the number of array positions to follow
err = locEncoder.Add(docNum, uint64(num))
if err != nil {
return nil, nil, err
}
// put each array position
for j := 0; j < num; j++ {
err = locEncoder.Add(docNum, memSegment.Locarraypos[postingID][locOffset][j])
if err != nil {
return nil, nil, err
}
}
}
locOffset++
}
offset++
}
// record where this postings loc info starts
locOfffsets = append(locOfffsets, uint64(w.Count()))
locEncoder.Close()
_, err := locEncoder.Write(w)
if err != nil {
return nil, nil, err
}
}
return freqOffsets, locOfffsets, nil
}
func persistPostingsLocs(memSegment *mem.Segment, w *CountHashWriter) (rv []uint64, err error) {
for postingID := range memSegment.PostingsLocs {
// record where we start this posting loc
rv = append(rv, uint64(w.Count()))
// write out the length and bitmap
_, err = writeRoaringWithLen(memSegment.PostingsLocs[postingID], w)
if err != nil {
return nil, err
}
}
return rv, nil
}
func persistPostingsLists(memSegment *mem.Segment, w *CountHashWriter,
postingsListLocs, freqOffsets, locOffsets []uint64) (rv []uint64, err error) {
for postingID := range memSegment.Postings {
// record where we start this posting list
rv = append(rv, uint64(w.Count()))
// write out the term info, loc info, and loc posting list offset
_, err = writeUvarints(w, freqOffsets[postingID],
locOffsets[postingID], postingsListLocs[postingID])
if err != nil {
return nil, err
}
// write out the length and bitmap
_, err = writeRoaringWithLen(memSegment.Postings[postingID], w)
if err != nil {
return nil, err
}
}
return rv, nil
}
func persistDictionary(memSegment *mem.Segment, w *CountHashWriter, postingsLocs []uint64) ([]uint64, error) {
var rv []uint64
var buffer bytes.Buffer
for fieldID, fieldTerms := range memSegment.DictKeys {
if fieldID != 0 {
buffer.Reset()
}
// start a new vellum for this field
builder, err := vellum.New(&buffer, nil)
if err != nil {
return nil, err
}
dict := memSegment.Dicts[fieldID]
// now walk the dictionary in order of fieldTerms (already sorted)
for i := range fieldTerms {
postingID := dict[fieldTerms[i]] - 1
postingsAddr := postingsLocs[postingID]
err = builder.Insert([]byte(fieldTerms[i]), postingsAddr)
if err != nil {
return nil, err
}
}
err = builder.Close()
if err != nil {
return nil, err
}
// record where this dictionary starts
rv = append(rv, uint64(w.Count()))
vellumData := buffer.Bytes()
// write out the length of the vellum data
buf := make([]byte, binary.MaxVarintLen64)
// write out the number of chunks
n := binary.PutUvarint(buf, uint64(len(vellumData)))
_, err = w.Write(buf[:n])
if err != nil {
return nil, err
}
// write this vellum to disk
_, err = w.Write(vellumData)
if err != nil {
return nil, err
}
}
return rv, nil
}
type docIDRange []uint64
func (a docIDRange) Len() int { return len(a) }
func (a docIDRange) Swap(i, j int) { a[i], a[j] = a[j], a[i] }
func (a docIDRange) Less(i, j int) bool { return a[i] < a[j] }
func persistDocValues(memSegment *mem.Segment, w *CountHashWriter,
chunkFactor uint32) (map[uint16]uint64, error) {
fieldChunkOffsets := make(map[uint16]uint64, len(memSegment.FieldsInv))
fdvEncoder := newChunkedContentCoder(uint64(chunkFactor), uint64(len(memSegment.Stored)-1))
for fieldID := range memSegment.DocValueFields {
field := memSegment.FieldsInv[fieldID]
docTermMap := make(map[uint64][]byte, 0)
dict, err := memSegment.Dictionary(field)
if err != nil {
return nil, err
}
dictItr := dict.Iterator()
next, err := dictItr.Next()
for err == nil && next != nil {
postings, err1 := dict.PostingsList(next.Term, nil)
if err1 != nil {
return nil, err
}
postingsItr := postings.Iterator()
nextPosting, err2 := postingsItr.Next()
for err2 == nil && nextPosting != nil {
docNum := nextPosting.Number()
docTermMap[docNum] = append(docTermMap[docNum], []byte(next.Term)...)
docTermMap[docNum] = append(docTermMap[docNum], termSeparator)
nextPosting, err2 = postingsItr.Next()
}
if err2 != nil {
return nil, err2
}
next, err = dictItr.Next()
}
if err != nil {
return nil, err
}
// sort wrt to docIDs
var docNumbers docIDRange
for k := range docTermMap {
docNumbers = append(docNumbers, k)
}
sort.Sort(docNumbers)
for _, docNum := range docNumbers {
err = fdvEncoder.Add(docNum, docTermMap[docNum])
if err != nil {
return nil, err
}
}
fieldChunkOffsets[fieldID] = uint64(w.Count())
err = fdvEncoder.Close()
if err != nil {
return nil, err
}
// persist the doc value details for this field
_, err = fdvEncoder.Write(w)
if err != nil {
return nil, err
}
// resetting encoder for the next field
fdvEncoder.Reset()
}
return fieldChunkOffsets, nil
}
func persistFieldDocValues(w *CountHashWriter, chunkFactor uint32,
memSegment *mem.Segment) (uint64, error) {
fieldDvOffsets, err := persistDocValues(memSegment, w, chunkFactor)
if err != nil {
return 0, err
}
fieldDocValuesOffset := uint64(w.Count())
buf := make([]byte, binary.MaxVarintLen64)
offset := uint64(0)
ok := true
for fieldID := range memSegment.FieldsInv {
// if the field isn't configured for docValue, then mark
// the offset accordingly
if offset, ok = fieldDvOffsets[uint16(fieldID)]; !ok {
offset = fieldNotUninverted
}
n := binary.PutUvarint(buf, uint64(offset))
_, err := w.Write(buf[:n])
if err != nil {
return 0, err
}
}
return fieldDocValuesOffset, nil
}

View File

@ -0,0 +1,288 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"os"
"testing"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
)
func TestBuild(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
memSegment := buildMemSegment()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatal(err)
}
}
func buildMemSegment() *mem.Segment {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("wow"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, []string{"_id"}),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("wow"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
}
// fix up composite fields
for _, ar := range results {
for i, f := range ar.Document.Fields {
for _, cf := range ar.Document.CompositeFields {
cf.Compose(f.Name(), ar.Length[i], ar.Analyzed[i])
}
}
}
return mem.NewFromAnalyzedDocs(results)
}
func buildMemSegmentMulti() *mem.Segment {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("wow"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, []string{"_id"}),
},
}
doc2 := &document.Document{
ID: "b",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("b"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("who"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, []string{"_id"}),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("wow"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
&index.AnalysisResult{
Document: doc2,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("b"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("who"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
}
// fix up composite fields
for _, ar := range results {
for i, f := range ar.Document.Fields {
for _, cf := range ar.Document.CompositeFields {
cf.Compose(f.Name(), ar.Length[i], ar.Analyzed[i])
}
}
}
segment := mem.NewFromAnalyzedDocs(results)
return segment
}

View File

@ -0,0 +1,167 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"io"
"github.com/golang/snappy"
)
var termSeparator byte = 0xff
var termSeparatorSplitSlice = []byte{termSeparator}
type chunkedContentCoder struct {
final []byte
chunkSize uint64
currChunk uint64
chunkLens []uint64
chunkMetaBuf bytes.Buffer
chunkBuf bytes.Buffer
chunkMeta []MetaData
}
// MetaData represents the data information inside a
// chunk.
type MetaData struct {
DocID uint64 // docid of the data inside the chunk
DocDvLoc uint64 // starting offset for a given docid
DocDvLen uint64 // length of data inside the chunk for the given docid
}
// newChunkedContentCoder returns a new chunk content coder which
// packs data into chunks based on the provided chunkSize
func newChunkedContentCoder(chunkSize uint64,
maxDocNum uint64) *chunkedContentCoder {
total := maxDocNum/chunkSize + 1
rv := &chunkedContentCoder{
chunkSize: chunkSize,
chunkLens: make([]uint64, total),
chunkMeta: []MetaData{},
}
return rv
}
// Reset lets you reuse this chunked content coder. Buffers are reset
// and re used. You cannot change the chunk size.
func (c *chunkedContentCoder) Reset() {
c.currChunk = 0
c.final = c.final[:0]
c.chunkBuf.Reset()
c.chunkMetaBuf.Reset()
for i := range c.chunkLens {
c.chunkLens[i] = 0
}
c.chunkMeta = []MetaData{}
}
// Close indicates you are done calling Add() this allows
// the final chunk to be encoded.
func (c *chunkedContentCoder) Close() error {
return c.flushContents()
}
func (c *chunkedContentCoder) flushContents() error {
// flush the contents, with meta information at first
buf := make([]byte, binary.MaxVarintLen64)
n := binary.PutUvarint(buf, uint64(len(c.chunkMeta)))
_, err := c.chunkMetaBuf.Write(buf[:n])
if err != nil {
return err
}
// write out the metaData slice
for _, meta := range c.chunkMeta {
_, err := writeUvarints(&c.chunkMetaBuf, meta.DocID, meta.DocDvLoc, meta.DocDvLen)
if err != nil {
return err
}
}
// write the metadata to final data
metaData := c.chunkMetaBuf.Bytes()
c.final = append(c.final, c.chunkMetaBuf.Bytes()...)
// write the compressed data to the final data
compressedData := snappy.Encode(nil, c.chunkBuf.Bytes())
c.final = append(c.final, compressedData...)
c.chunkLens[c.currChunk] = uint64(len(compressedData) + len(metaData))
return nil
}
// Add encodes the provided byte slice into the correct chunk for the provided
// doc num. You MUST call Add() with increasing docNums.
func (c *chunkedContentCoder) Add(docNum uint64, vals []byte) error {
chunk := docNum / c.chunkSize
if chunk != c.currChunk {
// flush out the previous chunk details
err := c.flushContents()
if err != nil {
return err
}
// clearing the chunk specific meta for next chunk
c.chunkBuf.Reset()
c.chunkMetaBuf.Reset()
c.chunkMeta = []MetaData{}
c.currChunk = chunk
}
// mark the starting offset for this doc
dvOffset := c.chunkBuf.Len()
dvSize, err := c.chunkBuf.Write(vals)
if err != nil {
return err
}
c.chunkMeta = append(c.chunkMeta, MetaData{
DocID: docNum,
DocDvLoc: uint64(dvOffset),
DocDvLen: uint64(dvSize),
})
return nil
}
// Write commits all the encoded chunked contents to the provided writer.
func (c *chunkedContentCoder) Write(w io.Writer) (int, error) {
var tw int
buf := make([]byte, binary.MaxVarintLen64)
// write out the number of chunks
n := binary.PutUvarint(buf, uint64(len(c.chunkLens)))
nw, err := w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
// write out the chunk lens
for _, chunkLen := range c.chunkLens {
n := binary.PutUvarint(buf, uint64(chunkLen))
nw, err = w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
}
// write out the data
nw, err = w.Write(c.final)
tw += nw
if err != nil {
return tw, err
}
return tw, nil
}

View File

@ -0,0 +1,75 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"reflect"
"testing"
)
func TestChunkContentCoder(t *testing.T) {
tests := []struct {
maxDocNum uint64
chunkSize uint64
docNums []uint64
vals [][]byte
expected string
}{
{
maxDocNum: 0,
chunkSize: 1,
docNums: []uint64{0},
vals: [][]byte{[]byte("bleve")},
// 1 chunk, chunk-0 length 11(b), value
expected: string([]byte{0x1, 0xb, 0x1, 0x0, 0x0, 0x05, 0x05, 0x10, 0x62, 0x6c, 0x65, 0x76, 0x65}),
},
{
maxDocNum: 1,
chunkSize: 1,
docNums: []uint64{0, 1},
vals: [][]byte{
[]byte("upside"),
[]byte("scorch"),
},
expected: string([]byte{0x02, 0x0c, 0x0c, 0x01, 0x00, 0x00, 0x06, 0x06, 0x14,
0x75, 0x70, 0x73, 0x69, 0x64, 0x65, 0x01, 0x01, 0x00, 0x06, 0x06,
0x14, 0x73, 0x63, 0x6f, 0x72, 0x63, 0x68}),
},
}
for _, test := range tests {
cic := newChunkedContentCoder(test.chunkSize, test.maxDocNum)
for i, docNum := range test.docNums {
err := cic.Add(docNum, test.vals[i])
if err != nil {
t.Fatalf("error adding to intcoder: %v", err)
}
}
_ = cic.Close()
var actual bytes.Buffer
_, err := cic.Write(&actual)
if err != nil {
t.Fatalf("error writing: %v", err)
}
if !reflect.DeepEqual(test.expected, string(actual.Bytes())) {
t.Errorf("got % s, expected % s", string(actual.Bytes()), test.expected)
}
}
}

View File

@ -0,0 +1,55 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"hash"
"hash/crc32"
"io"
)
// CountHashWriter is a wrapper around a Writer which counts the number of
// bytes which have been written
type CountHashWriter struct {
w io.Writer
h hash.Hash32
n int
}
// NewCountHashWriter returns a CountHashWriter which wraps the provided Writer
func NewCountHashWriter(w io.Writer) *CountHashWriter {
return &CountHashWriter{
w: w,
h: crc32.NewIEEE(),
}
}
// Write writes the provided bytes to the wrapped writer and counts the bytes
func (c *CountHashWriter) Write(b []byte) (int, error) {
n, err := c.w.Write(b)
c.n += n
_, _ = c.h.Write(b)
return n, err
}
// Count returns the number of bytes written
func (c *CountHashWriter) Count() int {
return c.n
}
// Sum32 returns the CRC-32 hash of the content written to this writer
func (c *CountHashWriter) Sum32() uint32 {
return c.h.Sum32()
}

View File

@ -0,0 +1,180 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"encoding/binary"
"fmt"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/couchbase/vellum"
"github.com/couchbase/vellum/regexp"
)
// Dictionary is the zap representation of the term dictionary
type Dictionary struct {
segment *Segment
field string
fieldID uint16
fst *vellum.FST
}
// PostingsList returns the postings list for the specified term
func (d *Dictionary) PostingsList(term string, except *roaring.Bitmap) (segment.PostingsList, error) {
return d.postingsList(term, except)
}
func (d *Dictionary) postingsList(term string, except *roaring.Bitmap) (*PostingsList, error) {
rv := &PostingsList{
dictionary: d,
term: term,
except: except,
}
if d.fst != nil {
postingsOffset, exists, err := d.fst.Get([]byte(term))
if err != nil {
return nil, fmt.Errorf("vellum err: %v", err)
}
if exists {
rv.postingsOffset = postingsOffset
// read the location of the freq/norm details
var n uint64
var read int
rv.freqOffset, read = binary.Uvarint(d.segment.mm[postingsOffset+n : postingsOffset+binary.MaxVarintLen64])
n += uint64(read)
rv.locOffset, read = binary.Uvarint(d.segment.mm[postingsOffset+n : postingsOffset+n+binary.MaxVarintLen64])
n += uint64(read)
var locBitmapOffset uint64
locBitmapOffset, read = binary.Uvarint(d.segment.mm[postingsOffset+n : postingsOffset+n+binary.MaxVarintLen64])
n += uint64(read)
// go ahead and load loc bitmap
var locBitmapLen uint64
locBitmapLen, read = binary.Uvarint(d.segment.mm[locBitmapOffset : locBitmapOffset+binary.MaxVarintLen64])
locRoaringBytes := d.segment.mm[locBitmapOffset+uint64(read) : locBitmapOffset+uint64(read)+locBitmapLen]
rv.locBitmap = roaring.NewBitmap()
_, err := rv.locBitmap.FromBuffer(locRoaringBytes)
if err != nil {
return nil, fmt.Errorf("error loading roaring bitmap of locations with hits: %v", err)
}
var postingsLen uint64
postingsLen, read = binary.Uvarint(d.segment.mm[postingsOffset+n : postingsOffset+n+binary.MaxVarintLen64])
n += uint64(read)
roaringBytes := d.segment.mm[postingsOffset+n : postingsOffset+n+postingsLen]
bitmap := roaring.NewBitmap()
_, err = bitmap.FromBuffer(roaringBytes)
if err != nil {
return nil, fmt.Errorf("error loading roaring bitmap: %v", err)
}
rv.postings = bitmap
}
}
return rv, nil
}
// Iterator returns an iterator for this dictionary
func (d *Dictionary) Iterator() segment.DictionaryIterator {
rv := &DictionaryIterator{
d: d,
}
if d.fst != nil {
itr, err := d.fst.Iterator(nil, nil)
if err == nil {
rv.itr = itr
}
}
return rv
}
// PrefixIterator returns an iterator which only visits terms having the
// the specified prefix
func (d *Dictionary) PrefixIterator(prefix string) segment.DictionaryIterator {
rv := &DictionaryIterator{
d: d,
}
if d.fst != nil {
r, err := regexp.New(prefix + ".*")
if err == nil {
itr, err := d.fst.Search(r, nil, nil)
if err == nil {
rv.itr = itr
}
}
}
return rv
}
// RangeIterator returns an iterator which only visits terms between the
// start and end terms. NOTE: bleve.index API specifies the end is inclusive.
func (d *Dictionary) RangeIterator(start, end string) segment.DictionaryIterator {
rv := &DictionaryIterator{
d: d,
}
// need to increment the end position to be inclusive
endBytes := []byte(end)
if endBytes[len(endBytes)-1] < 0xff {
endBytes[len(endBytes)-1]++
} else {
endBytes = append(endBytes, 0xff)
}
if d.fst != nil {
itr, err := d.fst.Iterator([]byte(start), endBytes)
if err == nil {
rv.itr = itr
}
}
return rv
}
// DictionaryIterator is an iterator for term dictionary
type DictionaryIterator struct {
d *Dictionary
itr vellum.Iterator
err error
}
// Next returns the next entry in the dictionary
func (i *DictionaryIterator) Next() (*index.DictEntry, error) {
if i.itr == nil || i.err == vellum.ErrIteratorDone {
return nil, nil
} else if i.err != nil {
return nil, i.err
}
term, count := i.itr.Current()
rv := &index.DictEntry{
Term: string(term),
Count: count,
}
i.err = i.itr.Next()
return rv, nil
}

View File

@ -0,0 +1,183 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"os"
"reflect"
"testing"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
)
func buildMemSegmentForDict() *mem.Segment {
doc := &document.Document{
ID: "a",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("a"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("desc", nil, []byte("apple ball cat dog egg fish bat"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("a"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 5,
Position: 1,
Term: []byte("apple"),
},
&analysis.Token{
Start: 6,
End: 10,
Position: 2,
Term: []byte("ball"),
},
&analysis.Token{
Start: 11,
End: 14,
Position: 3,
Term: []byte("cat"),
},
&analysis.Token{
Start: 15,
End: 18,
Position: 4,
Term: []byte("dog"),
},
&analysis.Token{
Start: 19,
End: 22,
Position: 5,
Term: []byte("egg"),
},
&analysis.Token{
Start: 20,
End: 24,
Position: 6,
Term: []byte("fish"),
},
&analysis.Token{
Start: 25,
End: 28,
Position: 7,
Term: []byte("bat"),
},
}, nil, true),
},
Length: []int{
1,
7,
},
},
}
segment := mem.NewFromAnalyzedDocs(results)
return segment
}
func TestDictionary(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
memSegment := buildMemSegmentForDict()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatalf("error persisting segment: %v", err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
dict, err := segment.Dictionary("desc")
if err != nil {
t.Fatal(err)
}
// test basic full iterator
expected := []string{"apple", "ball", "bat", "cat", "dog", "egg", "fish"}
var got []string
itr := dict.Iterator()
next, err := itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
// test prefix iterator
expected = []string{"ball", "bat"}
got = got[:0]
itr = dict.PrefixIterator("b")
next, err = itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
// test range iterator
expected = []string{"cat", "dog", "egg"}
got = got[:0]
itr = dict.RangeIterator("cat", "egg")
next, err = itr.Next()
for next != nil && err == nil {
got = append(got, next.Term)
next, err = itr.Next()
}
if err != nil {
t.Fatalf("dict itr error: %v", err)
}
if !reflect.DeepEqual(expected, got) {
t.Errorf("expected: %v, got: %v", expected, got)
}
}

View File

@ -0,0 +1,180 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"fmt"
"math"
"sort"
"github.com/blevesearch/bleve/index"
"github.com/golang/snappy"
)
type docValueIterator struct {
field string
curChunkNum uint64
numChunks uint64
chunkLens []uint64
dvDataLoc uint64
curChunkHeader []MetaData
curChunkData []byte // compressed data cache
}
func (di *docValueIterator) fieldName() string {
return di.field
}
func (di *docValueIterator) curChunkNumber() uint64 {
return di.curChunkNum
}
func (s *Segment) loadFieldDocValueIterator(field string,
fieldDvLoc uint64) (*docValueIterator, error) {
// get the docValue offset for the given fields
if fieldDvLoc == fieldNotUninverted {
return nil, fmt.Errorf("loadFieldDocValueConfigs: "+
"no docValues found for field: %s", field)
}
// read the number of chunks, chunk lengths
var offset, clen uint64
numChunks, read := binary.Uvarint(s.mm[fieldDvLoc : fieldDvLoc+binary.MaxVarintLen64])
if read <= 0 {
return nil, fmt.Errorf("failed to read the field "+
"doc values for field %s", field)
}
offset += uint64(read)
fdvIter := &docValueIterator{
curChunkNum: math.MaxUint64,
field: field,
chunkLens: make([]uint64, int(numChunks)),
}
for i := 0; i < int(numChunks); i++ {
clen, read = binary.Uvarint(s.mm[fieldDvLoc+offset : fieldDvLoc+offset+binary.MaxVarintLen64])
if read <= 0 {
return nil, fmt.Errorf("corrupted chunk length during segment load")
}
fdvIter.chunkLens[i] = clen
offset += uint64(read)
}
fdvIter.dvDataLoc = fieldDvLoc + offset
return fdvIter, nil
}
func (di *docValueIterator) loadDvChunk(chunkNumber,
localDocNum uint64, s *Segment) error {
// advance to the chunk where the docValues
// reside for the given docID
destChunkDataLoc := di.dvDataLoc
for i := 0; i < int(chunkNumber); i++ {
destChunkDataLoc += di.chunkLens[i]
}
curChunkSize := di.chunkLens[chunkNumber]
// read the number of docs reside in the chunk
numDocs, read := binary.Uvarint(s.mm[destChunkDataLoc : destChunkDataLoc+binary.MaxVarintLen64])
if read <= 0 {
return fmt.Errorf("failed to read the chunk")
}
chunkMetaLoc := destChunkDataLoc + uint64(read)
offset := uint64(0)
di.curChunkHeader = make([]MetaData, int(numDocs))
for i := 0; i < int(numDocs); i++ {
di.curChunkHeader[i].DocID, read = binary.Uvarint(s.mm[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(read)
di.curChunkHeader[i].DocDvLoc, read = binary.Uvarint(s.mm[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(read)
di.curChunkHeader[i].DocDvLen, read = binary.Uvarint(s.mm[chunkMetaLoc+offset : chunkMetaLoc+offset+binary.MaxVarintLen64])
offset += uint64(read)
}
compressedDataLoc := chunkMetaLoc + offset
dataLength := destChunkDataLoc + curChunkSize - compressedDataLoc
di.curChunkData = s.mm[compressedDataLoc : compressedDataLoc+dataLength]
di.curChunkNum = chunkNumber
return nil
}
func (di *docValueIterator) visitDocValues(docID uint64,
visitor index.DocumentFieldTermVisitor) error {
// binary search the term locations for the docID
start, length := di.getDocValueLocs(docID)
if start == math.MaxUint64 || length == math.MaxUint64 {
return nil
}
// uncompress the already loaded data
uncompressed, err := snappy.Decode(nil, di.curChunkData)
if err != nil {
return err
}
// pick the terms for the given docID
uncompressed = uncompressed[start : start+length]
for {
i := bytes.Index(uncompressed, termSeparatorSplitSlice)
if i < 0 {
break
}
visitor(di.field, uncompressed[0:i])
uncompressed = uncompressed[i+1:]
}
return nil
}
func (di *docValueIterator) getDocValueLocs(docID uint64) (uint64, uint64) {
i := sort.Search(len(di.curChunkHeader), func(i int) bool {
return di.curChunkHeader[i].DocID >= docID
})
if i < len(di.curChunkHeader) && di.curChunkHeader[i].DocID == docID {
return di.curChunkHeader[i].DocDvLoc, di.curChunkHeader[i].DocDvLen
}
return math.MaxUint64, math.MaxUint64
}
// VisitDocumentFieldTerms is an implementation of the UnInvertIndex interface
func (s *Segment) VisitDocumentFieldTerms(localDocNum uint64, fields []string,
visitor index.DocumentFieldTermVisitor) error {
fieldID := uint16(0)
ok := true
for _, field := range fields {
if fieldID, ok = s.fieldsMap[field]; !ok {
continue
}
// find the chunkNumber where the docValues are stored
docInChunk := localDocNum / uint64(s.chunkFactor)
if dvIter, exists := s.fieldDvIterMap[fieldID-1]; exists &&
dvIter != nil {
// check if the chunk is already loaded
if docInChunk != dvIter.curChunkNumber() {
err := dvIter.loadDvChunk(docInChunk, localDocNum, s)
if err != nil {
continue
}
}
_ = dvIter.visitDocValues(localDocNum, visitor)
}
}
return nil
}

View File

@ -0,0 +1,125 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"io"
"github.com/Smerity/govarint"
)
type chunkedIntCoder struct {
final []byte
maxDocNum uint64
chunkSize uint64
chunkBuf bytes.Buffer
encoder *govarint.Base128Encoder
chunkLens []uint64
currChunk uint64
}
// newChunkedIntCoder returns a new chunk int coder which packs data into
// chunks based on the provided chunkSize and supports up to the specified
// maxDocNum
func newChunkedIntCoder(chunkSize uint64, maxDocNum uint64) *chunkedIntCoder {
total := maxDocNum/chunkSize + 1
rv := &chunkedIntCoder{
chunkSize: chunkSize,
maxDocNum: maxDocNum,
chunkLens: make([]uint64, total),
}
rv.encoder = govarint.NewU64Base128Encoder(&rv.chunkBuf)
return rv
}
// Reset lets you reuse this chunked int coder. buffers are reset and reused
// from previous use. you cannot change the chunk size or max doc num.
func (c *chunkedIntCoder) Reset() {
c.final = c.final[:0]
c.chunkBuf.Reset()
c.currChunk = 0
for i := range c.chunkLens {
c.chunkLens[i] = 0
}
}
// Add encodes the provided integers into the correct chunk for the provided
// doc num. You MUST call Add() with increasing docNums.
func (c *chunkedIntCoder) Add(docNum uint64, vals ...uint64) error {
chunk := docNum / c.chunkSize
if chunk != c.currChunk {
// starting a new chunk
if c.encoder != nil {
// close out last
c.encoder.Close()
encodingBytes := c.chunkBuf.Bytes()
c.chunkLens[c.currChunk] = uint64(len(encodingBytes))
c.final = append(c.final, encodingBytes...)
c.chunkBuf.Reset()
c.encoder = govarint.NewU64Base128Encoder(&c.chunkBuf)
}
c.currChunk = chunk
}
for _, val := range vals {
_, err := c.encoder.PutU64(val)
if err != nil {
return err
}
}
return nil
}
// Close indicates you are done calling Add() this allows the final chunk
// to be encoded.
func (c *chunkedIntCoder) Close() {
c.encoder.Close()
encodingBytes := c.chunkBuf.Bytes()
c.chunkLens[c.currChunk] = uint64(len(encodingBytes))
c.final = append(c.final, encodingBytes...)
}
// Write commits all the encoded chunked integers to the provided writer.
func (c *chunkedIntCoder) Write(w io.Writer) (int, error) {
var tw int
buf := make([]byte, binary.MaxVarintLen64)
// write out the number of chunks
n := binary.PutUvarint(buf, uint64(len(c.chunkLens)))
nw, err := w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
// write out the chunk lens
for _, chunkLen := range c.chunkLens {
n := binary.PutUvarint(buf, uint64(chunkLen))
nw, err = w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
}
// write out the data
nw, err = w.Write(c.final)
tw += nw
if err != nil {
return tw, err
}
return tw, nil
}

View File

@ -0,0 +1,73 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"reflect"
"testing"
)
func TestChunkIntCoder(t *testing.T) {
tests := []struct {
maxDocNum uint64
chunkSize uint64
docNums []uint64
vals [][]uint64
expected []byte
}{
{
maxDocNum: 0,
chunkSize: 1,
docNums: []uint64{0},
vals: [][]uint64{
[]uint64{3},
},
// 1 chunk, chunk-0 length 1, value 3
expected: []byte{0x1, 0x1, 0x3},
},
{
maxDocNum: 1,
chunkSize: 1,
docNums: []uint64{0, 1},
vals: [][]uint64{
[]uint64{3},
[]uint64{7},
},
// 2 chunks, chunk-0 length 1, chunk-1 length 1, value 3, value 7
expected: []byte{0x2, 0x1, 0x1, 0x3, 0x7},
},
}
for _, test := range tests {
cic := newChunkedIntCoder(test.chunkSize, test.maxDocNum)
for i, docNum := range test.docNums {
err := cic.Add(docNum, test.vals[i]...)
if err != nil {
t.Fatalf("error adding to intcoder: %v", err)
}
}
cic.Close()
var actual bytes.Buffer
_, err := cic.Write(&actual)
if err != nil {
t.Fatalf("error writing: %v", err)
}
if !reflect.DeepEqual(test.expected, actual.Bytes()) {
t.Errorf("got % x, expected % x", actual.Bytes(), test.expected)
}
}
}

View File

@ -0,0 +1,531 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bufio"
"bytes"
"encoding/binary"
"fmt"
"math"
"os"
"sort"
"github.com/RoaringBitmap/roaring"
"github.com/Smerity/govarint"
"github.com/couchbase/vellum"
"github.com/golang/snappy"
)
// Merge takes a slice of zap segments, bit masks describing which documents
// from the may be dropped, and creates a new segment containing the remaining
// data. This new segment is built at the specified path, with the provided
// chunkFactor.
func Merge(segments []*Segment, drops []*roaring.Bitmap, path string,
chunkFactor uint32) ([][]uint64, error) {
flag := os.O_RDWR | os.O_CREATE
f, err := os.OpenFile(path, flag, 0600)
if err != nil {
return nil, err
}
// buffer the output
br := bufio.NewWriter(f)
// wrap it for counting (tracking offsets)
cr := NewCountHashWriter(br)
fieldsInv := mergeFields(segments)
fieldsMap := mapFields(fieldsInv)
newSegDocCount := computeNewDocCount(segments, drops)
var newDocNums [][]uint64
var storedIndexOffset uint64
fieldDvLocsOffset := uint64(fieldNotUninverted)
var dictLocs []uint64
if newSegDocCount > 0 {
storedIndexOffset, newDocNums, err = mergeStoredAndRemap(segments, drops,
fieldsMap, fieldsInv, newSegDocCount, cr)
if err != nil {
return nil, err
}
dictLocs, fieldDvLocsOffset, err = persistMergedRest(segments, drops, fieldsInv, fieldsMap,
newDocNums, newSegDocCount, chunkFactor, cr)
if err != nil {
return nil, err
}
} else {
dictLocs = make([]uint64, len(fieldsInv))
}
var fieldsIndexOffset uint64
fieldsIndexOffset, err = persistFields(fieldsInv, cr, dictLocs)
if err != nil {
return nil, err
}
err = persistFooter(newSegDocCount, storedIndexOffset,
fieldsIndexOffset, fieldDvLocsOffset, chunkFactor, cr)
if err != nil {
return nil, err
}
err = br.Flush()
if err != nil {
return nil, err
}
err = f.Sync()
if err != nil {
return nil, err
}
err = f.Close()
if err != nil {
return nil, err
}
return newDocNums, nil
}
// mapFields takes the fieldsInv list and builds the map
func mapFields(fields []string) map[string]uint16 {
rv := make(map[string]uint16)
for i, fieldName := range fields {
rv[fieldName] = uint16(i)
}
return rv
}
// computeNewDocCount determines how many documents will be in the newly
// merged segment when obsoleted docs are dropped
func computeNewDocCount(segments []*Segment, drops []*roaring.Bitmap) uint64 {
var newSegDocCount uint64
for segI, segment := range segments {
segIAfterDrop := segment.NumDocs()
if drops[segI] != nil {
segIAfterDrop -= drops[segI].GetCardinality()
}
newSegDocCount += segIAfterDrop
}
return newSegDocCount
}
func persistMergedRest(segments []*Segment, drops []*roaring.Bitmap,
fieldsInv []string, fieldsMap map[string]uint16, newDocNums [][]uint64,
newSegDocCount uint64, chunkFactor uint32,
w *CountHashWriter) ([]uint64, uint64, error) {
var bufMaxVarintLen64 []byte = make([]byte, binary.MaxVarintLen64)
var bufLoc []uint64
rv1 := make([]uint64, len(fieldsInv))
fieldDvLocs := make([]uint64, len(fieldsInv))
fieldDvLocsOffset := uint64(fieldNotUninverted)
var vellumBuf bytes.Buffer
// for each field
for fieldID, fieldName := range fieldsInv {
if fieldID != 0 {
vellumBuf.Reset()
}
newVellum, err := vellum.New(&vellumBuf, nil)
if err != nil {
return nil, 0, err
}
// collect FST iterators from all segments for this field
var dicts []*Dictionary
var itrs []vellum.Iterator
for _, segment := range segments {
dict, err2 := segment.dictionary(fieldName)
if err2 != nil {
return nil, 0, err2
}
dicts = append(dicts, dict)
if dict != nil && dict.fst != nil {
itr, err2 := dict.fst.Iterator(nil, nil)
if err2 != nil && err2 != vellum.ErrIteratorDone {
return nil, 0, err2
}
if itr != nil {
itrs = append(itrs, itr)
}
}
}
// create merging iterator
mergeItr, err := vellum.NewMergeIterator(itrs, func(postingOffsets []uint64) uint64 {
// we don't actually use the merged value
return 0
})
tfEncoder := newChunkedIntCoder(uint64(chunkFactor), newSegDocCount-1)
locEncoder := newChunkedIntCoder(uint64(chunkFactor), newSegDocCount-1)
fdvEncoder := newChunkedContentCoder(uint64(chunkFactor), newSegDocCount-1)
docTermMap := make(map[uint64][]byte, 0)
for err == nil {
term, _ := mergeItr.Current()
newRoaring := roaring.NewBitmap()
newRoaringLocs := roaring.NewBitmap()
tfEncoder.Reset()
locEncoder.Reset()
// now go back and get posting list for this term
// but pass in the deleted docs for that segment
for dictI, dict := range dicts {
if dict == nil {
continue
}
postings, err2 := dict.postingsList(string(term), drops[dictI])
if err2 != nil {
return nil, 0, err2
}
postItr := postings.Iterator()
next, err2 := postItr.Next()
for next != nil && err2 == nil {
hitNewDocNum := newDocNums[dictI][next.Number()]
if hitNewDocNum == docDropped {
return nil, 0, fmt.Errorf("see hit with dropped doc num")
}
newRoaring.Add(uint32(hitNewDocNum))
// encode norm bits
norm := next.Norm()
normBits := math.Float32bits(float32(norm))
err3 := tfEncoder.Add(hitNewDocNum, next.Frequency(), uint64(normBits))
if err3 != nil {
return nil, 0, err3
}
locs := next.Locations()
if len(locs) > 0 {
newRoaringLocs.Add(uint32(hitNewDocNum))
for _, loc := range locs {
if cap(bufLoc) < 5+len(loc.ArrayPositions()) {
bufLoc = make([]uint64, 0, 5+len(loc.ArrayPositions()))
}
args := bufLoc[0:0]
args = append(args, uint64(fieldsMap[loc.Field()]))
args = append(args, loc.Pos())
args = append(args, loc.Start())
args = append(args, loc.End())
args = append(args, uint64(len(loc.ArrayPositions())))
args = append(args, loc.ArrayPositions()...)
err = locEncoder.Add(hitNewDocNum, args...)
if err != nil {
return nil, 0, err
}
}
}
docTermMap[hitNewDocNum] = append(docTermMap[hitNewDocNum], []byte(term)...)
docTermMap[hitNewDocNum] = append(docTermMap[hitNewDocNum], termSeparator)
next, err2 = postItr.Next()
}
if err != nil {
return nil, 0, err
}
}
tfEncoder.Close()
locEncoder.Close()
if newRoaring.GetCardinality() > 0 {
// this field/term actually has hits in the new segment, lets write it down
freqOffset := uint64(w.Count())
_, err = tfEncoder.Write(w)
if err != nil {
return nil, 0, err
}
locOffset := uint64(w.Count())
_, err = locEncoder.Write(w)
if err != nil {
return nil, 0, err
}
postingLocOffset := uint64(w.Count())
_, err = writeRoaringWithLen(newRoaringLocs, w)
if err != nil {
return nil, 0, err
}
postingOffset := uint64(w.Count())
// write out the start of the term info
buf := bufMaxVarintLen64
n := binary.PutUvarint(buf, freqOffset)
_, err = w.Write(buf[:n])
if err != nil {
return nil, 0, err
}
// write out the start of the loc info
n = binary.PutUvarint(buf, locOffset)
_, err = w.Write(buf[:n])
if err != nil {
return nil, 0, err
}
// write out the start of the loc posting list
n = binary.PutUvarint(buf, postingLocOffset)
_, err = w.Write(buf[:n])
if err != nil {
return nil, 0, err
}
_, err = writeRoaringWithLen(newRoaring, w)
if err != nil {
return nil, 0, err
}
err = newVellum.Insert(term, postingOffset)
if err != nil {
return nil, 0, err
}
}
err = mergeItr.Next()
}
if err != nil && err != vellum.ErrIteratorDone {
return nil, 0, err
}
dictOffset := uint64(w.Count())
err = newVellum.Close()
if err != nil {
return nil, 0, err
}
vellumData := vellumBuf.Bytes()
// write out the length of the vellum data
buf := bufMaxVarintLen64
// write out the number of chunks
n := binary.PutUvarint(buf, uint64(len(vellumData)))
_, err = w.Write(buf[:n])
if err != nil {
return nil, 0, err
}
// write this vellum to disk
_, err = w.Write(vellumData)
if err != nil {
return nil, 0, err
}
rv1[fieldID] = dictOffset
// update the doc value
var docNumbers docIDRange
for k := range docTermMap {
docNumbers = append(docNumbers, k)
}
sort.Sort(docNumbers)
for _, docNum := range docNumbers {
err = fdvEncoder.Add(docNum, docTermMap[docNum])
if err != nil {
return nil, 0, err
}
}
// get the field doc value offset
fieldDvLocs[fieldID] = uint64(w.Count())
err = fdvEncoder.Close()
if err != nil {
return nil, 0, err
}
// persist the doc value details for this field
_, err = fdvEncoder.Write(w)
if err != nil {
return nil, 0, err
}
}
fieldDvLocsOffset = uint64(w.Count())
buf := make([]byte, binary.MaxVarintLen64)
for _, offset := range fieldDvLocs {
n := binary.PutUvarint(buf, uint64(offset))
_, err := w.Write(buf[:n])
if err != nil {
return nil, 0, err
}
}
return rv1, fieldDvLocsOffset, nil
}
const docDropped = math.MaxUint64
func mergeStoredAndRemap(segments []*Segment, drops []*roaring.Bitmap,
fieldsMap map[string]uint16, fieldsInv []string, newSegDocCount uint64,
w *CountHashWriter) (uint64, [][]uint64, error) {
var rv [][]uint64
var newDocNum int
var curr int
var metaBuf bytes.Buffer
var data, compressed []byte
vals := make([][][]byte, len(fieldsInv))
typs := make([][]byte, len(fieldsInv))
poss := make([][][]uint64, len(fieldsInv))
docNumOffsets := make([]uint64, newSegDocCount)
// for each segment
for segI, segment := range segments {
var segNewDocNums []uint64
// for each doc num
for docNum := uint64(0); docNum < segment.numDocs; docNum++ {
metaBuf.Reset()
data = data[:0]
compressed = compressed[:0]
curr = 0
metaEncoder := govarint.NewU64Base128Encoder(&metaBuf)
if drops[segI] != nil && drops[segI].Contains(uint32(docNum)) {
segNewDocNums = append(segNewDocNums, docDropped)
} else {
segNewDocNums = append(segNewDocNums, uint64(newDocNum))
// collect all the data
for i := 0; i < len(fieldsInv); i++ {
vals[i] = vals[i][:0]
typs[i] = typs[i][:0]
poss[i] = poss[i][:0]
}
err := segment.VisitDocument(docNum, func(field string, typ byte, value []byte, pos []uint64) bool {
fieldID := int(fieldsMap[field])
vals[fieldID] = append(vals[fieldID], value)
typs[fieldID] = append(typs[fieldID], typ)
poss[fieldID] = append(poss[fieldID], pos)
return true
})
if err != nil {
return 0, nil, err
}
// now walk the fields in order
for fieldID := range fieldsInv {
storedFieldValues := vals[int(fieldID)]
// has stored values for this field
num := len(storedFieldValues)
// process each value
for i := 0; i < num; i++ {
// encode field
_, err2 := metaEncoder.PutU64(uint64(fieldID))
if err2 != nil {
return 0, nil, err2
}
// encode type
_, err2 = metaEncoder.PutU64(uint64(typs[int(fieldID)][i]))
if err2 != nil {
return 0, nil, err2
}
// encode start offset
_, err2 = metaEncoder.PutU64(uint64(curr))
if err2 != nil {
return 0, nil, err2
}
// end len
_, err2 = metaEncoder.PutU64(uint64(len(storedFieldValues[i])))
if err2 != nil {
return 0, nil, err2
}
// encode number of array pos
_, err2 = metaEncoder.PutU64(uint64(len(poss[int(fieldID)][i])))
if err2 != nil {
return 0, nil, err2
}
// encode all array positions
for j := 0; j < len(poss[int(fieldID)][i]); j++ {
_, err2 = metaEncoder.PutU64(poss[int(fieldID)][i][j])
if err2 != nil {
return 0, nil, err2
}
}
// append data
data = append(data, storedFieldValues[i]...)
// update curr
curr += len(storedFieldValues[i])
}
}
metaEncoder.Close()
metaBytes := metaBuf.Bytes()
compressed = snappy.Encode(compressed, data)
// record where we're about to start writing
docNumOffsets[newDocNum] = uint64(w.Count())
// write out the meta len and compressed data len
_, err = writeUvarints(w,
uint64(len(metaBytes)), uint64(len(compressed)))
if err != nil {
return 0, nil, err
}
// now write the meta
_, err = w.Write(metaBytes)
if err != nil {
return 0, nil, err
}
// now write the compressed data
_, err = w.Write(compressed)
if err != nil {
return 0, nil, err
}
newDocNum++
}
}
rv = append(rv, segNewDocNums)
}
// return value is the start of the stored index
offset := uint64(w.Count())
// now write out the stored doc index
for docNum := range docNumOffsets {
err := binary.Write(w, binary.BigEndian, docNumOffsets[docNum])
if err != nil {
return 0, nil, err
}
}
return offset, rv, nil
}
// mergeFields builds a unified list of fields used across all the input segments
func mergeFields(segments []*Segment) []string {
fieldsMap := map[string]struct{}{}
for _, segment := range segments {
fields := segment.Fields()
for _, field := range fields {
fieldsMap[field] = struct{}{}
}
}
rv := make([]string, 0, len(fieldsMap))
// ensure _id stays first
rv = append(rv, "_id")
for k := range fieldsMap {
if k != "_id" {
rv = append(rv, k)
}
}
return rv
}

View File

@ -0,0 +1,294 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"os"
"testing"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/analysis"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment/mem"
)
func TestMerge(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
_ = os.RemoveAll("/tmp/scorch2.zap")
_ = os.RemoveAll("/tmp/scorch3.zap")
memSegment := buildMemSegmentMulti()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatal(err)
}
memSegment2 := buildMemSegmentMulti2()
err = PersistSegment(memSegment2, "/tmp/scorch2.zap", 1024)
if err != nil {
t.Fatal(err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
segment2, err := Open("/tmp/scorch2.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment2.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
segsToMerge := make([]*Segment, 2)
segsToMerge[0] = segment.(*Segment)
segsToMerge[1] = segment2.(*Segment)
_, err = Merge(segsToMerge, []*roaring.Bitmap{nil, nil}, "/tmp/scorch3.zap", 1024)
if err != nil {
t.Fatal(err)
}
}
func TestMergeAndDrop(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
_ = os.RemoveAll("/tmp/scorch2.zap")
_ = os.RemoveAll("/tmp/scorch3.zap")
memSegment := buildMemSegmentMulti()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatal(err)
}
memSegment2 := buildMemSegmentMulti2()
err = PersistSegment(memSegment2, "/tmp/scorch2.zap", 1024)
if err != nil {
t.Fatal(err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
segment2, err := Open("/tmp/scorch2.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment2.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
segsToMerge := make([]*Segment, 2)
segsToMerge[0] = segment.(*Segment)
segsToMerge[1] = segment2.(*Segment)
docsToDrop := make([]*roaring.Bitmap, 2)
docsToDrop[0] = roaring.NewBitmap()
docsToDrop[0].AddInt(1)
docsToDrop[1] = roaring.NewBitmap()
docsToDrop[1].AddInt(1)
_, err = Merge(segsToMerge, docsToDrop, "/tmp/scorch3.zap", 1024)
if err != nil {
t.Fatal(err)
}
}
func buildMemSegmentMulti2() *mem.Segment {
doc := &document.Document{
ID: "c",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("c"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("mat"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, []string{"_id"}),
},
}
doc2 := &document.Document{
ID: "d",
Fields: []document.Field{
document.NewTextFieldCustom("_id", nil, []byte("d"), document.IndexField|document.StoreField, nil),
document.NewTextFieldCustom("name", nil, []byte("joa"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("desc", nil, []byte("some thing"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{0}, []byte("cold"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
document.NewTextFieldCustom("tag", []uint64{1}, []byte("dark"), document.IndexField|document.StoreField|document.IncludeTermVectors, nil),
},
CompositeFields: []*document.CompositeField{
document.NewCompositeField("_all", true, nil, []string{"_id"}),
},
}
// forge analyzed docs
results := []*index.AnalysisResult{
&index.AnalysisResult{
Document: doc,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("c"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("mat"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
&index.AnalysisResult{
Document: doc2,
Analyzed: []analysis.TokenFrequencies{
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 1,
Position: 1,
Term: []byte("d"),
},
}, nil, false),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 3,
Position: 1,
Term: []byte("joa"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("some"),
},
&analysis.Token{
Start: 5,
End: 10,
Position: 2,
Term: []byte("thing"),
},
}, nil, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("cold"),
},
}, []uint64{0}, true),
analysis.TokenFrequency(analysis.TokenStream{
&analysis.Token{
Start: 0,
End: 4,
Position: 1,
Term: []byte("dark"),
},
}, []uint64{1}, true),
},
Length: []int{
1,
1,
2,
1,
1,
},
},
}
// fix up composite fields
for _, ar := range results {
for i, f := range ar.Document.Fields {
for _, cf := range ar.Document.CompositeFields {
cf.Compose(f.Name(), ar.Length[i], ar.Analyzed[i])
}
}
}
segment := mem.NewFromAnalyzedDocs(results)
return segment
}

View File

@ -0,0 +1,366 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"fmt"
"math"
"github.com/RoaringBitmap/roaring"
"github.com/Smerity/govarint"
"github.com/blevesearch/bleve/index/scorch/segment"
)
// PostingsList is an in-memory represenation of a postings list
type PostingsList struct {
dictionary *Dictionary
term string
postingsOffset uint64
freqOffset uint64
locOffset uint64
locBitmap *roaring.Bitmap
postings *roaring.Bitmap
except *roaring.Bitmap
postingKey []byte
}
// Iterator returns an iterator for this postings list
func (p *PostingsList) Iterator() segment.PostingsIterator {
rv := &PostingsIterator{
postings: p,
}
if p.postings != nil {
// prepare the freq chunk details
var n uint64
var read int
var numFreqChunks uint64
numFreqChunks, read = binary.Uvarint(p.dictionary.segment.mm[p.freqOffset+n : p.freqOffset+n+binary.MaxVarintLen64])
n += uint64(read)
rv.freqChunkLens = make([]uint64, int(numFreqChunks))
for i := 0; i < int(numFreqChunks); i++ {
rv.freqChunkLens[i], read = binary.Uvarint(p.dictionary.segment.mm[p.freqOffset+n : p.freqOffset+n+binary.MaxVarintLen64])
n += uint64(read)
}
rv.freqChunkStart = p.freqOffset + n
// prepare the loc chunk details
n = 0
var numLocChunks uint64
numLocChunks, read = binary.Uvarint(p.dictionary.segment.mm[p.locOffset+n : p.locOffset+n+binary.MaxVarintLen64])
n += uint64(read)
rv.locChunkLens = make([]uint64, int(numLocChunks))
for i := 0; i < int(numLocChunks); i++ {
rv.locChunkLens[i], read = binary.Uvarint(p.dictionary.segment.mm[p.locOffset+n : p.locOffset+n+binary.MaxVarintLen64])
n += uint64(read)
}
rv.locChunkStart = p.locOffset + n
rv.locBitmap = p.locBitmap
rv.all = p.postings.Iterator()
if p.except != nil {
allExcept := roaring.AndNot(p.postings, p.except)
rv.actual = allExcept.Iterator()
} else {
rv.actual = p.postings.Iterator()
}
}
return rv
}
// Count returns the number of items on this postings list
func (p *PostingsList) Count() uint64 {
if p.postings != nil {
n := p.postings.GetCardinality()
if p.except != nil {
e := p.except.GetCardinality()
if e > n {
e = n
}
return n - e
}
return n
}
return 0
}
// PostingsIterator provides a way to iterate through the postings list
type PostingsIterator struct {
postings *PostingsList
all roaring.IntIterable
offset int
locoffset int
actual roaring.IntIterable
currChunk uint32
currChunkFreqNorm []byte
currChunkLoc []byte
freqNormDecoder *govarint.Base128Decoder
locDecoder *govarint.Base128Decoder
freqChunkLens []uint64
freqChunkStart uint64
locChunkLens []uint64
locChunkStart uint64
locBitmap *roaring.Bitmap
next Posting
}
func (i *PostingsIterator) loadChunk(chunk int) error {
if chunk >= len(i.freqChunkLens) || chunk >= len(i.locChunkLens) {
return fmt.Errorf("tried to load chunk that doesn't exist %d/(%d %d)", chunk, len(i.freqChunkLens), len(i.locChunkLens))
}
// load correct chunk bytes
start := i.freqChunkStart
for j := 0; j < chunk; j++ {
start += i.freqChunkLens[j]
}
end := start + i.freqChunkLens[chunk]
i.currChunkFreqNorm = i.postings.dictionary.segment.mm[start:end]
i.freqNormDecoder = govarint.NewU64Base128Decoder(bytes.NewReader(i.currChunkFreqNorm))
start = i.locChunkStart
for j := 0; j < chunk; j++ {
start += i.locChunkLens[j]
}
end = start + i.locChunkLens[chunk]
i.currChunkLoc = i.postings.dictionary.segment.mm[start:end]
i.locDecoder = govarint.NewU64Base128Decoder(bytes.NewReader(i.currChunkLoc))
i.currChunk = uint32(chunk)
return nil
}
func (i *PostingsIterator) readFreqNorm() (uint64, uint64, error) {
freq, err := i.freqNormDecoder.GetU64()
if err != nil {
return 0, 0, fmt.Errorf("error reading frequency: %v", err)
}
normBits, err := i.freqNormDecoder.GetU64()
if err != nil {
return 0, 0, fmt.Errorf("error reading norm: %v", err)
}
return freq, normBits, err
}
// readLocation processes all the integers on the stream representing a single
// location. if you care about it, pass in a non-nil location struct, and we
// will fill it. if you don't care about it, pass in nil and we safely consume
// the contents.
func (i *PostingsIterator) readLocation(l *Location) error {
// read off field
fieldID, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading location field: %v", err)
}
// read off pos
pos, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading location pos: %v", err)
}
// read off start
start, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading location start: %v", err)
}
// read off end
end, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading location end: %v", err)
}
// read off num array pos
numArrayPos, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading location num array pos: %v", err)
}
// group these together for less branching
if l != nil {
l.field = i.postings.dictionary.segment.fieldsInv[fieldID]
l.pos = pos
l.start = start
l.end = end
if numArrayPos > 0 {
l.ap = make([]uint64, int(numArrayPos))
}
}
// read off array positions
for k := 0; k < int(numArrayPos); k++ {
ap, err := i.locDecoder.GetU64()
if err != nil {
return fmt.Errorf("error reading array position: %v", err)
}
if l != nil {
l.ap[k] = ap
}
}
return nil
}
// Next returns the next posting on the postings list, or nil at the end
func (i *PostingsIterator) Next() (segment.Posting, error) {
if i.actual == nil || !i.actual.HasNext() {
return nil, nil
}
n := i.actual.Next()
nChunk := n / i.postings.dictionary.segment.chunkFactor
allN := i.all.Next()
allNChunk := allN / i.postings.dictionary.segment.chunkFactor
// n is the next actual hit (excluding some postings)
// allN is the next hit in the full postings
// if they don't match, adjust offsets to factor in item we're skipping over
// incr the all iterator, and check again
for allN != n {
// in different chunks, reset offsets
if allNChunk != nChunk {
i.locoffset = 0
i.offset = 0
} else {
if i.currChunk != nChunk || i.currChunkFreqNorm == nil {
err := i.loadChunk(int(nChunk))
if err != nil {
return nil, fmt.Errorf("error loading chunk: %v", err)
}
}
// read off freq/offsets even though we don't care about them
freq, _, err := i.readFreqNorm()
if err != nil {
return nil, err
}
if i.locBitmap.Contains(allN) {
for j := 0; j < int(freq); j++ {
err := i.readLocation(nil)
if err != nil {
return nil, err
}
}
}
// in same chunk, need to account for offsets
i.offset++
}
allN = i.all.Next()
}
if i.currChunk != nChunk || i.currChunkFreqNorm == nil {
err := i.loadChunk(int(nChunk))
if err != nil {
return nil, fmt.Errorf("error loading chunk: %v", err)
}
}
i.next = Posting{} // clear the struct.
rv := &i.next
rv.iterator = i
rv.docNum = uint64(n)
var err error
var normBits uint64
rv.freq, normBits, err = i.readFreqNorm()
if err != nil {
return nil, err
}
rv.norm = math.Float32frombits(uint32(normBits))
if i.locBitmap.Contains(n) {
// read off 'freq' locations
rv.locs = make([]segment.Location, rv.freq)
locs := make([]Location, rv.freq)
for j := 0; j < int(rv.freq); j++ {
err := i.readLocation(&locs[j])
if err != nil {
return nil, err
}
rv.locs[j] = &locs[j]
}
}
return rv, nil
}
// Posting is a single entry in a postings list
type Posting struct {
iterator *PostingsIterator
docNum uint64
freq uint64
norm float32
locs []segment.Location
}
// Number returns the document number of this posting in this segment
func (p *Posting) Number() uint64 {
return p.docNum
}
// Frequency returns the frequence of occurance of this term in this doc/field
func (p *Posting) Frequency() uint64 {
return p.freq
}
// Norm returns the normalization factor for this posting
func (p *Posting) Norm() float64 {
return float64(p.norm)
}
// Locations returns the location information for each occurance
func (p *Posting) Locations() []segment.Location {
return p.locs
}
// Location represents the location of a single occurance
type Location struct {
field string
pos uint64
start uint64
end uint64
ap []uint64
}
// Field returns the name of the field (useful in composite fields to know
// which original field the value came from)
func (l *Location) Field() string {
return l.field
}
// Start returns the start byte offset of this occurance
func (l *Location) Start() uint64 {
return l.start
}
// End returns the end byte offset of this occurance
func (l *Location) End() uint64 {
return l.end
}
// Pos returns the 1-based phrase position of this occurance
func (l *Location) Pos() uint64 {
return l.pos
}
// ArrayPositions returns the array position vector associated with this occurance
func (l *Location) ArrayPositions() []uint64 {
return l.ap
}

View File

@ -0,0 +1,31 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import "encoding/binary"
func (s *Segment) getStoredMetaAndCompressed(docNum uint64) ([]byte, []byte) {
docStoredStartAddr := s.storedIndexOffset + (8 * docNum)
docStoredStart := binary.BigEndian.Uint64(s.mm[docStoredStartAddr : docStoredStartAddr+8])
var n uint64
metaLen, read := binary.Uvarint(s.mm[docStoredStart : docStoredStart+binary.MaxVarintLen64])
n += uint64(read)
var dataLen uint64
dataLen, read = binary.Uvarint(s.mm[docStoredStart+n : docStoredStart+n+binary.MaxVarintLen64])
n += uint64(read)
meta := s.mm[docStoredStart+n : docStoredStart+n+metaLen]
data := s.mm[docStoredStart+n+metaLen : docStoredStart+n+metaLen+dataLen]
return meta, data
}

View File

@ -0,0 +1,418 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"fmt"
"io"
"os"
"sync"
"github.com/RoaringBitmap/roaring"
"github.com/Smerity/govarint"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/couchbase/vellum"
mmap "github.com/edsrzf/mmap-go"
"github.com/golang/snappy"
)
// Open returns a zap impl of a segment
func Open(path string) (segment.Segment, error) {
f, err := os.Open(path)
if err != nil {
return nil, err
}
mm, err := mmap.Map(f, mmap.RDONLY, 0)
if err != nil {
// mmap failed, try to close the file
_ = f.Close()
return nil, err
}
rv := &Segment{
f: f,
mm: mm,
path: path,
fieldsMap: make(map[string]uint16),
fieldDvIterMap: make(map[uint16]*docValueIterator),
refs: 1,
}
err = rv.loadConfig()
if err != nil {
_ = rv.Close()
return nil, err
}
err = rv.loadFields()
if err != nil {
_ = rv.Close()
return nil, err
}
err = rv.loadDvIterators()
if err != nil {
_ = rv.Close()
return nil, err
}
return rv, nil
}
// Segment implements the segment.Segment inteface over top the zap file format
type Segment struct {
f *os.File
mm mmap.MMap
path string
crc uint32
version uint32
chunkFactor uint32
numDocs uint64
storedIndexOffset uint64
fieldsIndexOffset uint64
fieldsMap map[string]uint16
fieldsInv []string
fieldsOffsets []uint64
docValueOffset uint64
fieldDvIterMap map[uint16]*docValueIterator // naive chunk cache per field
m sync.Mutex // Protects the fields that follow.
refs int64
}
func (s *Segment) SizeInBytes() uint64 {
// 4 /* size of crc -> uint32 */ +
// 4 /* size of version -> uint32 */ +
// 4 /* size of chunkFactor -> uint32 */ +
// 8 /* size of numDocs -> uint64 */ +
// 8 /* size of storedIndexOffset -> uint64 */ +
// 8 /* size of fieldsIndexOffset -> uint64 */
sizeOfUints := 36
sizeInBytes := len(s.mm) + len(s.path) + sizeOfUints
for k, _ := range s.fieldsMap {
sizeInBytes += len(k) + 2 /* size of uint16 */
}
for _, entry := range s.fieldsInv {
sizeInBytes += len(entry)
}
sizeInBytes += len(s.fieldsOffsets) * 8 /* size of uint64 */
sizeInBytes += 8 /* size of refs -> int64 */
return uint64(sizeInBytes)
}
func (s *Segment) AddRef() {
s.m.Lock()
s.refs++
s.m.Unlock()
}
func (s *Segment) DecRef() (err error) {
s.m.Lock()
s.refs--
if s.refs == 0 {
err = s.closeActual()
}
s.m.Unlock()
return err
}
func (s *Segment) loadConfig() error {
crcOffset := len(s.mm) - 4
s.crc = binary.BigEndian.Uint32(s.mm[crcOffset : crcOffset+4])
verOffset := crcOffset - 4
s.version = binary.BigEndian.Uint32(s.mm[verOffset : verOffset+4])
if s.version != version {
return fmt.Errorf("unsupported version %d", s.version)
}
chunkOffset := verOffset - 4
s.chunkFactor = binary.BigEndian.Uint32(s.mm[chunkOffset : chunkOffset+4])
docValueOffset := chunkOffset - 8
s.docValueOffset = binary.BigEndian.Uint64(s.mm[docValueOffset : docValueOffset+8])
fieldsOffset := docValueOffset - 8
s.fieldsIndexOffset = binary.BigEndian.Uint64(s.mm[fieldsOffset : fieldsOffset+8])
storedOffset := fieldsOffset - 8
s.storedIndexOffset = binary.BigEndian.Uint64(s.mm[storedOffset : storedOffset+8])
docNumOffset := storedOffset - 8
s.numDocs = binary.BigEndian.Uint64(s.mm[docNumOffset : docNumOffset+8])
return nil
}
func (s *Segment) loadFields() error {
// NOTE for now we assume the fields index immediately preceeds the footer
// if this changes, need to adjust accordingly (or store epxlicit length)
fieldsIndexEnd := uint64(len(s.mm) - FooterSize)
// iterate through fields index
var fieldID uint64
for s.fieldsIndexOffset+(8*fieldID) < fieldsIndexEnd {
addr := binary.BigEndian.Uint64(s.mm[s.fieldsIndexOffset+(8*fieldID) : s.fieldsIndexOffset+(8*fieldID)+8])
var n uint64
dictLoc, read := binary.Uvarint(s.mm[addr+n : fieldsIndexEnd])
n += uint64(read)
s.fieldsOffsets = append(s.fieldsOffsets, dictLoc)
var nameLen uint64
nameLen, read = binary.Uvarint(s.mm[addr+n : fieldsIndexEnd])
n += uint64(read)
name := string(s.mm[addr+n : addr+n+nameLen])
s.fieldsInv = append(s.fieldsInv, name)
s.fieldsMap[name] = uint16(fieldID + 1)
fieldID++
}
return nil
}
// Dictionary returns the term dictionary for the specified field
func (s *Segment) Dictionary(field string) (segment.TermDictionary, error) {
dict, err := s.dictionary(field)
if err == nil && dict == nil {
return &segment.EmptyDictionary{}, nil
}
return dict, err
}
func (s *Segment) dictionary(field string) (rv *Dictionary, err error) {
rv = &Dictionary{
segment: s,
field: field,
}
rv.fieldID = s.fieldsMap[field]
if rv.fieldID > 0 {
rv.fieldID = rv.fieldID - 1
dictStart := s.fieldsOffsets[rv.fieldID]
if dictStart > 0 {
// read the length of the vellum data
vellumLen, read := binary.Uvarint(s.mm[dictStart : dictStart+binary.MaxVarintLen64])
fstBytes := s.mm[dictStart+uint64(read) : dictStart+uint64(read)+vellumLen]
if fstBytes != nil {
rv.fst, err = vellum.Load(fstBytes)
if err != nil {
return nil, fmt.Errorf("dictionary field %s vellum err: %v", field, err)
}
}
}
} else {
return nil, nil
}
return rv, nil
}
// VisitDocument invokes the DocFieldValueVistor for each stored field
// for the specified doc number
func (s *Segment) VisitDocument(num uint64, visitor segment.DocumentFieldValueVisitor) error {
// first make sure this is a valid number in this segment
if num < s.numDocs {
meta, compressed := s.getStoredMetaAndCompressed(num)
uncompressed, err := snappy.Decode(nil, compressed)
if err != nil {
return err
}
// now decode meta and process
reader := bytes.NewReader(meta)
decoder := govarint.NewU64Base128Decoder(reader)
keepGoing := true
for keepGoing {
field, err := decoder.GetU64()
if err == io.EOF {
break
}
if err != nil {
return err
}
typ, err := decoder.GetU64()
if err != nil {
return err
}
offset, err := decoder.GetU64()
if err != nil {
return err
}
l, err := decoder.GetU64()
if err != nil {
return err
}
numap, err := decoder.GetU64()
if err != nil {
return err
}
var arrayPos []uint64
if numap > 0 {
arrayPos = make([]uint64, numap)
for i := 0; i < int(numap); i++ {
ap, err := decoder.GetU64()
if err != nil {
return err
}
arrayPos[i] = ap
}
}
value := uncompressed[offset : offset+l]
keepGoing = visitor(s.fieldsInv[field], byte(typ), value, arrayPos)
}
}
return nil
}
// Count returns the number of documents in this segment.
func (s *Segment) Count() uint64 {
return s.numDocs
}
// DocNumbers returns a bitset corresponding to the doc numbers of all the
// provided _id strings
func (s *Segment) DocNumbers(ids []string) (*roaring.Bitmap, error) {
rv := roaring.New()
if len(s.fieldsMap) > 0 {
idDict, err := s.dictionary("_id")
if err != nil {
return nil, err
}
for _, id := range ids {
postings, err := idDict.postingsList(id, nil)
if err != nil {
return nil, err
}
if postings.postings != nil {
rv.Or(postings.postings)
}
}
}
return rv, nil
}
// Fields returns the field names used in this segment
func (s *Segment) Fields() []string {
return s.fieldsInv
}
// Path returns the path of this segment on disk
func (s *Segment) Path() string {
return s.path
}
// Close releases all resources associated with this segment
func (s *Segment) Close() (err error) {
return s.DecRef()
}
func (s *Segment) closeActual() (err error) {
if s.mm != nil {
err = s.mm.Unmap()
}
// try to close file even if unmap failed
if s.f != nil {
err2 := s.f.Close()
if err == nil {
// try to return first error
err = err2
}
}
return
}
// some helpers i started adding for the command-line utility
// Data returns the underlying mmaped data slice
func (s *Segment) Data() []byte {
return s.mm
}
// CRC returns the CRC value stored in the file footer
func (s *Segment) CRC() uint32 {
return s.crc
}
// Version returns the file version in the file footer
func (s *Segment) Version() uint32 {
return s.version
}
// ChunkFactor returns the chunk factor in the file footer
func (s *Segment) ChunkFactor() uint32 {
return s.chunkFactor
}
// FieldsIndexOffset returns the fields index offset in the file footer
func (s *Segment) FieldsIndexOffset() uint64 {
return s.fieldsIndexOffset
}
// StoredIndexOffset returns the stored value index offset in the file footer
func (s *Segment) StoredIndexOffset() uint64 {
return s.storedIndexOffset
}
// DocValueOffset returns the docValue offset in the file footer
func (s *Segment) DocValueOffset() uint64 {
return s.docValueOffset
}
// NumDocs returns the number of documents in the file footer
func (s *Segment) NumDocs() uint64 {
return s.numDocs
}
// DictAddr is a helper function to compute the file offset where the
// dictionary is stored for the specified field.
func (s *Segment) DictAddr(field string) (uint64, error) {
var fieldID uint16
var ok bool
if fieldID, ok = s.fieldsMap[field]; !ok {
return 0, fmt.Errorf("no such field '%s'", field)
}
return s.fieldsOffsets[fieldID-1], nil
}
func (s *Segment) loadDvIterators() error {
if s.docValueOffset == fieldNotUninverted {
return nil
}
var read uint64
for fieldID, field := range s.fieldsInv {
fieldLoc, n := binary.Uvarint(s.mm[s.docValueOffset+read : s.docValueOffset+read+binary.MaxVarintLen64])
if n <= 0 {
return fmt.Errorf("loadDvIterators: failed to read the docvalue offsets for field %d", fieldID)
}
s.fieldDvIterMap[uint16(fieldID)], _ = s.loadFieldDocValueIterator(field, fieldLoc)
read += uint64(n)
}
return nil
}

View File

@ -0,0 +1,517 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"math"
"os"
"reflect"
"testing"
)
func TestOpen(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
memSegment := buildMemSegment()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatalf("error persisting segment: %v", err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
expectFields := map[string]struct{}{
"_id": struct{}{},
"_all": struct{}{},
"name": struct{}{},
"desc": struct{}{},
"tag": struct{}{},
}
fields := segment.Fields()
if len(fields) != len(expectFields) {
t.Errorf("expected %d fields, only got %d", len(expectFields), len(fields))
}
for _, field := range fields {
if _, ok := expectFields[field]; !ok {
t.Errorf("got unexpected field: %s", field)
}
}
docCount := segment.Count()
if docCount != 1 {
t.Errorf("expected count 1, got %d", docCount)
}
// check the _id field
dict, err := segment.Dictionary("_id")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("a", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
if nextPosting.Norm() != 1.0 {
t.Errorf("expected norm 1.0, got %f", nextPosting.Norm())
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// check the name field
dict, err = segment.Dictionary("name")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("wow", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
if nextPosting.Norm() != 1.0 {
t.Errorf("expected norm 1.0, got %f", nextPosting.Norm())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "name" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 3 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
if loc.ArrayPositions() != nil {
t.Errorf("expect loc array pos to be nil, got %v", loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// check the _all field (composite)
dict, err = segment.Dictionary("_all")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("wow", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
count++
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
expectedNorm := float32(1.0 / math.Sqrt(float64(5)))
if nextPosting.Norm() != float64(expectedNorm) {
t.Errorf("expected norm %f, got %f", expectedNorm, nextPosting.Norm())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "name" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 3 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
if loc.ArrayPositions() != nil {
t.Errorf("expect loc array pos to be nil, got %v", loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
// now try a field with array positions
dict, err = segment.Dictionary("tag")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err = dict.PostingsList("dark", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr = postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
nextPosting, err = postingsItr.Next()
for nextPosting != nil && err == nil {
if nextPosting.Frequency() != 1 {
t.Errorf("expected frequency 1, got %d", nextPosting.Frequency())
}
if nextPosting.Number() != 0 {
t.Errorf("expected doc number 0, got %d", nextPosting.Number())
}
var numLocs uint64
for _, loc := range nextPosting.Locations() {
numLocs++
if loc.Field() != "tag" {
t.Errorf("expected loc field to be 'name', got '%s'", loc.Field())
}
if loc.Start() != 0 {
t.Errorf("expected loc start to be 0, got %d", loc.Start())
}
if loc.End() != 4 {
t.Errorf("expected loc end to be 3, got %d", loc.End())
}
if loc.Pos() != 1 {
t.Errorf("expected loc pos to be 1, got %d", loc.Pos())
}
expectArrayPos := []uint64{1}
if !reflect.DeepEqual(loc.ArrayPositions(), expectArrayPos) {
t.Errorf("expect loc array pos to be %v, got %v", expectArrayPos, loc.ArrayPositions())
}
}
if numLocs != nextPosting.Frequency() {
t.Errorf("expected %d locations, got %d", nextPosting.Frequency(), numLocs)
}
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
// now try and visit a document
var fieldValuesSeen int
err = segment.VisitDocument(0, func(field string, typ byte, value []byte, pos []uint64) bool {
fieldValuesSeen++
return true
})
if err != nil {
t.Fatal(err)
}
if fieldValuesSeen != 5 {
t.Errorf("expected 5 field values, got %d", fieldValuesSeen)
}
}
func TestOpenMulti(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
memSegment := buildMemSegmentMulti()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1024)
if err != nil {
t.Fatalf("error persisting segment: %v", err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
if segment.Count() != 2 {
t.Errorf("expected count 2, got %d", segment.Count())
}
// check the desc field
dict, err := segment.Dictionary("desc")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("thing", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 2 {
t.Errorf("expected count to be 2, got %d", count)
}
// get docnum of a
exclude, err := segment.DocNumbers([]string{"a"})
if err != nil {
t.Fatal(err)
}
// look for term 'thing' excluding doc 'a'
postingsListExcluding, err := dict.PostingsList("thing", exclude)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsListExcludingCount := postingsListExcluding.Count()
if postingsListExcludingCount != 1 {
t.Errorf("expected count from postings list to be 1, got %d", postingsListExcludingCount)
}
postingsItrExcluding := postingsListExcluding.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItrExcluding.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItrExcluding.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
}
func TestOpenMultiWithTwoChunks(t *testing.T) {
_ = os.RemoveAll("/tmp/scorch.zap")
memSegment := buildMemSegmentMulti()
err := PersistSegment(memSegment, "/tmp/scorch.zap", 1)
if err != nil {
t.Fatalf("error persisting segment: %v", err)
}
segment, err := Open("/tmp/scorch.zap")
if err != nil {
t.Fatalf("error opening segment: %v", err)
}
defer func() {
cerr := segment.Close()
if cerr != nil {
t.Fatalf("error closing segment: %v", err)
}
}()
if segment.Count() != 2 {
t.Errorf("expected count 2, got %d", segment.Count())
}
// check the desc field
dict, err := segment.Dictionary("desc")
if err != nil {
t.Fatal(err)
}
if dict == nil {
t.Fatal("got nil dict, expected non-nil")
}
postingsList, err := dict.PostingsList("thing", nil)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItr := postingsList.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count := 0
nextPosting, err := postingsItr.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItr.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 2 {
t.Errorf("expected count to be 2, got %d", count)
}
// get docnum of a
exclude, err := segment.DocNumbers([]string{"a"})
if err != nil {
t.Fatal(err)
}
// look for term 'thing' excluding doc 'a'
postingsListExcluding, err := dict.PostingsList("thing", exclude)
if err != nil {
t.Fatal(err)
}
if postingsList == nil {
t.Fatal("got nil postings list, expected non-nil")
}
postingsItrExcluding := postingsListExcluding.Iterator()
if postingsItr == nil {
t.Fatal("got nil iterator, expected non-nil")
}
count = 0
nextPosting, err = postingsItrExcluding.Next()
for nextPosting != nil && err == nil {
count++
nextPosting, err = postingsItrExcluding.Next()
}
if err != nil {
t.Fatal(err)
}
if count != 1 {
t.Errorf("expected count to be 1, got %d", count)
}
}

View File

@ -0,0 +1,144 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package zap
import (
"bytes"
"encoding/binary"
"io"
"github.com/RoaringBitmap/roaring"
)
// writes out the length of the roaring bitmap in bytes as varint
// then writs out the roaring bitmap itself
func writeRoaringWithLen(r *roaring.Bitmap, w io.Writer) (int, error) {
var buffer bytes.Buffer
// write out postings list to memory so we know the len
postingsListLen, err := r.WriteTo(&buffer)
if err != nil {
return 0, err
}
var tw int
// write out the length of this postings list
buf := make([]byte, binary.MaxVarintLen64)
n := binary.PutUvarint(buf, uint64(postingsListLen))
nw, err := w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
// write out the postings list itself
nw, err = w.Write(buffer.Bytes())
tw += nw
if err != nil {
return tw, err
}
return tw, nil
}
func persistFields(fieldsInv []string, w *CountHashWriter, dictLocs []uint64) (uint64, error) {
var rv uint64
var fieldStarts []uint64
for fieldID, fieldName := range fieldsInv {
// record start of this field
fieldStarts = append(fieldStarts, uint64(w.Count()))
// write out the dict location and field name length
_, err := writeUvarints(w, dictLocs[fieldID], uint64(len(fieldName)))
if err != nil {
return 0, err
}
// write out the field name
_, err = w.Write([]byte(fieldName))
if err != nil {
return 0, err
}
}
// now write out the fields index
rv = uint64(w.Count())
for fieldID := range fieldsInv {
err := binary.Write(w, binary.BigEndian, fieldStarts[fieldID])
if err != nil {
return 0, err
}
}
return rv, nil
}
// FooterSize is the size of the footer record in bytes
// crc + ver + chunk + field offset + stored offset + num docs + docValueOffset
const FooterSize = 4 + 4 + 4 + 8 + 8 + 8 + 8
func persistFooter(numDocs, storedIndexOffset, fieldIndexOffset, docValueOffset uint64,
chunkFactor uint32, w *CountHashWriter) error {
// write out the number of docs
err := binary.Write(w, binary.BigEndian, numDocs)
if err != nil {
return err
}
// write out the stored field index location:
err = binary.Write(w, binary.BigEndian, storedIndexOffset)
if err != nil {
return err
}
// write out the field index location
err = binary.Write(w, binary.BigEndian, fieldIndexOffset)
if err != nil {
return err
}
// write out the fieldDocValue location
err = binary.Write(w, binary.BigEndian, docValueOffset)
if err != nil {
return err
}
// write out 32-bit chunk factor
err = binary.Write(w, binary.BigEndian, chunkFactor)
if err != nil {
return err
}
// write out 32-bit version
err = binary.Write(w, binary.BigEndian, version)
if err != nil {
return err
}
// write out CRC-32 of everything upto but not including this CRC
err = binary.Write(w, binary.BigEndian, w.Sum32())
if err != nil {
return err
}
return nil
}
func writeUvarints(w io.Writer, vals ...uint64) (tw int, err error) {
buf := make([]byte, binary.MaxVarintLen64)
for _, val := range vals {
n := binary.PutUvarint(buf, val)
var nw int
nw, err = w.Write(buf[:n])
tw += nw
if err != nil {
return tw, err
}
}
return tw, err
}

View File

@ -0,0 +1,440 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"bytes"
"container/heap"
"encoding/binary"
"fmt"
"sort"
"sync"
"sync/atomic"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
)
type asynchSegmentResult struct {
dictItr segment.DictionaryIterator
index int
docs *roaring.Bitmap
postings segment.PostingsList
err error
}
type IndexSnapshot struct {
parent *Scorch
segment []*SegmentSnapshot
offsets []uint64
internal map[string][]byte
epoch uint64
m sync.Mutex // Protects the fields that follow.
refs int64
}
func (i *IndexSnapshot) Segments() []*SegmentSnapshot {
return i.segment
}
func (i *IndexSnapshot) Internal() map[string][]byte {
return i.internal
}
func (i *IndexSnapshot) AddRef() {
i.m.Lock()
i.refs++
i.m.Unlock()
}
func (i *IndexSnapshot) DecRef() (err error) {
i.m.Lock()
i.refs--
if i.refs == 0 {
for _, s := range i.segment {
if s != nil {
err2 := s.segment.DecRef()
if err == nil {
err = err2
}
}
}
if i.parent != nil {
go i.parent.AddEligibleForRemoval(i.epoch)
}
}
i.m.Unlock()
return err
}
func (i *IndexSnapshot) newIndexSnapshotFieldDict(field string, makeItr func(i segment.TermDictionary) segment.DictionaryIterator) (*IndexSnapshotFieldDict, error) {
results := make(chan *asynchSegmentResult)
for index, segment := range i.segment {
go func(index int, segment *SegmentSnapshot) {
dict, err := segment.Dictionary(field)
if err != nil {
results <- &asynchSegmentResult{err: err}
} else {
results <- &asynchSegmentResult{dictItr: makeItr(dict)}
}
}(index, segment)
}
var err error
rv := &IndexSnapshotFieldDict{
snapshot: i,
cursors: make([]*segmentDictCursor, 0, len(i.segment)),
}
for count := 0; count < len(i.segment); count++ {
asr := <-results
if asr.err != nil && err == nil {
err = asr.err
} else {
next, err2 := asr.dictItr.Next()
if err2 != nil && err == nil {
err = err2
}
if next != nil {
rv.cursors = append(rv.cursors, &segmentDictCursor{
itr: asr.dictItr,
curr: next,
})
}
}
}
// after ensuring we've read all items on channel
if err != nil {
return nil, err
}
// prepare heap
heap.Init(rv)
return rv, nil
}
func (i *IndexSnapshot) FieldDict(field string) (index.FieldDict, error) {
return i.newIndexSnapshotFieldDict(field, func(i segment.TermDictionary) segment.DictionaryIterator {
return i.Iterator()
})
}
func (i *IndexSnapshot) FieldDictRange(field string, startTerm []byte,
endTerm []byte) (index.FieldDict, error) {
return i.newIndexSnapshotFieldDict(field, func(i segment.TermDictionary) segment.DictionaryIterator {
return i.RangeIterator(string(startTerm), string(endTerm))
})
}
func (i *IndexSnapshot) FieldDictPrefix(field string,
termPrefix []byte) (index.FieldDict, error) {
return i.newIndexSnapshotFieldDict(field, func(i segment.TermDictionary) segment.DictionaryIterator {
return i.PrefixIterator(string(termPrefix))
})
}
func (i *IndexSnapshot) DocIDReaderAll() (index.DocIDReader, error) {
results := make(chan *asynchSegmentResult)
for index, segment := range i.segment {
go func(index int, segment *SegmentSnapshot) {
results <- &asynchSegmentResult{
index: index,
docs: segment.DocNumbersLive(),
}
}(index, segment)
}
return i.newDocIDReader(results)
}
func (i *IndexSnapshot) DocIDReaderOnly(ids []string) (index.DocIDReader, error) {
results := make(chan *asynchSegmentResult)
for index, segment := range i.segment {
go func(index int, segment *SegmentSnapshot) {
docs, err := segment.DocNumbers(ids)
if err != nil {
results <- &asynchSegmentResult{err: err}
} else {
results <- &asynchSegmentResult{
index: index,
docs: docs,
}
}
}(index, segment)
}
return i.newDocIDReader(results)
}
func (i *IndexSnapshot) newDocIDReader(results chan *asynchSegmentResult) (index.DocIDReader, error) {
rv := &IndexSnapshotDocIDReader{
snapshot: i,
iterators: make([]roaring.IntIterable, len(i.segment)),
}
var err error
for count := 0; count < len(i.segment); count++ {
asr := <-results
if asr.err != nil && err != nil {
err = asr.err
} else {
rv.iterators[asr.index] = asr.docs.Iterator()
}
}
if err != nil {
return nil, err
}
return rv, nil
}
func (i *IndexSnapshot) Fields() ([]string, error) {
// FIXME not making this concurrent for now as it's not used in hot path
// of any searches at the moment (just a debug aid)
fieldsMap := map[string]struct{}{}
for _, segment := range i.segment {
fields := segment.Fields()
for _, field := range fields {
fieldsMap[field] = struct{}{}
}
}
rv := make([]string, 0, len(fieldsMap))
for k := range fieldsMap {
rv = append(rv, k)
}
return rv, nil
}
func (i *IndexSnapshot) GetInternal(key []byte) ([]byte, error) {
return i.internal[string(key)], nil
}
func (i *IndexSnapshot) DocCount() (uint64, error) {
var rv uint64
for _, segment := range i.segment {
rv += segment.Count()
}
return rv, nil
}
func (i *IndexSnapshot) Document(id string) (rv *document.Document, err error) {
// FIXME could be done more efficiently directly, but reusing for simplicity
tfr, err := i.TermFieldReader([]byte(id), "_id", false, false, false)
if err != nil {
return nil, err
}
defer func() {
if cerr := tfr.Close(); err == nil && cerr != nil {
err = cerr
}
}()
next, err := tfr.Next(nil)
if err != nil {
return nil, err
}
if next == nil {
// no such doc exists
return nil, nil
}
docNum, err := docInternalToNumber(next.ID)
if err != nil {
return nil, err
}
segmentIndex, localDocNum := i.segmentIndexAndLocalDocNumFromGlobal(docNum)
rv = document.NewDocument(id)
err = i.segment[segmentIndex].VisitDocument(localDocNum, func(name string, typ byte, value []byte, pos []uint64) bool {
if name == "_id" {
return true
}
switch typ {
case 't':
rv.AddField(document.NewTextField(name, pos, value))
case 'n':
rv.AddField(document.NewNumericFieldFromBytes(name, pos, value))
case 'd':
rv.AddField(document.NewDateTimeFieldFromBytes(name, pos, value))
case 'b':
rv.AddField(document.NewBooleanFieldFromBytes(name, pos, value))
case 'g':
rv.AddField(document.NewGeoPointFieldFromBytes(name, pos, value))
}
return true
})
if err != nil {
return nil, err
}
return rv, nil
}
func (i *IndexSnapshot) segmentIndexAndLocalDocNumFromGlobal(docNum uint64) (int, uint64) {
segmentIndex := sort.Search(len(i.offsets),
func(x int) bool {
return i.offsets[x] > docNum
}) - 1
localDocNum := docNum - i.offsets[segmentIndex]
return int(segmentIndex), localDocNum
}
func (i *IndexSnapshot) ExternalID(id index.IndexInternalID) (string, error) {
docNum, err := docInternalToNumber(id)
if err != nil {
return "", err
}
segmentIndex, localDocNum := i.segmentIndexAndLocalDocNumFromGlobal(docNum)
var found bool
var rv string
err = i.segment[segmentIndex].VisitDocument(localDocNum, func(field string, typ byte, value []byte, pos []uint64) bool {
if field == "_id" {
found = true
rv = string(value)
return false
}
return true
})
if err != nil {
return "", err
}
if found {
return rv, nil
}
return "", fmt.Errorf("document number %d not found", docNum)
}
func (i *IndexSnapshot) InternalID(id string) (rv index.IndexInternalID, err error) {
// FIXME could be done more efficiently directly, but reusing for simplicity
tfr, err := i.TermFieldReader([]byte(id), "_id", false, false, false)
if err != nil {
return nil, err
}
defer func() {
if cerr := tfr.Close(); err == nil && cerr != nil {
err = cerr
}
}()
next, err := tfr.Next(nil)
if err != nil || next == nil {
return nil, err
}
return next.ID, nil
}
func (i *IndexSnapshot) TermFieldReader(term []byte, field string, includeFreq,
includeNorm, includeTermVectors bool) (index.TermFieldReader, error) {
rv := &IndexSnapshotTermFieldReader{
term: term,
field: field,
snapshot: i,
postings: make([]segment.PostingsList, len(i.segment)),
iterators: make([]segment.PostingsIterator, len(i.segment)),
includeFreq: includeFreq,
includeNorm: includeNorm,
includeTermVectors: includeTermVectors,
}
for i, segment := range i.segment {
dict, err := segment.Dictionary(field)
if err != nil {
return nil, err
}
pl, err := dict.PostingsList(string(term), nil)
if err != nil {
return nil, err
}
rv.postings[i] = pl
rv.iterators[i] = pl.Iterator()
}
atomic.AddUint64(&i.parent.stats.termSearchersStarted, uint64(1))
return rv, nil
}
func docNumberToBytes(buf []byte, in uint64) []byte {
if len(buf) != 8 {
if cap(buf) >= 8 {
buf = buf[0:8]
} else {
buf = make([]byte, 8)
}
}
binary.BigEndian.PutUint64(buf, in)
return buf
}
func docInternalToNumber(in index.IndexInternalID) (uint64, error) {
var res uint64
err := binary.Read(bytes.NewReader(in), binary.BigEndian, &res)
if err != nil {
return 0, err
}
return res, nil
}
func (i *IndexSnapshot) DocumentVisitFieldTerms(id index.IndexInternalID,
fields []string, visitor index.DocumentFieldTermVisitor) error {
docNum, err := docInternalToNumber(id)
if err != nil {
return err
}
segmentIndex, localDocNum := i.segmentIndexAndLocalDocNumFromGlobal(docNum)
if segmentIndex >= len(i.segment) {
return nil
}
ss := i.segment[segmentIndex]
if zaps, ok := ss.segment.(segment.DocumentFieldTermVisitable); ok {
return zaps.VisitDocumentFieldTerms(localDocNum, fields, visitor)
}
// else fallback to the in memory fieldCache
err = ss.cachedDocs.prepareFields(fields, ss)
if err != nil {
return err
}
for _, field := range fields {
if cachedFieldDocs, exists := ss.cachedDocs.cache[field]; exists {
if tlist, exists := cachedFieldDocs.docs[localDocNum]; exists {
for {
i := bytes.Index(tlist, TermSeparatorSplitSlice)
if i < 0 {
break
}
visitor(field, tlist[0:i])
tlist = tlist[i+1:]
}
}
}
}
return nil
}

View File

@ -0,0 +1,92 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"container/heap"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
)
type segmentDictCursor struct {
itr segment.DictionaryIterator
curr *index.DictEntry
}
type IndexSnapshotFieldDict struct {
snapshot *IndexSnapshot
cursors []*segmentDictCursor
}
func (i *IndexSnapshotFieldDict) Len() int { return len(i.cursors) }
func (i *IndexSnapshotFieldDict) Less(a, b int) bool {
return i.cursors[a].curr.Term < i.cursors[b].curr.Term
}
func (i *IndexSnapshotFieldDict) Swap(a, b int) {
i.cursors[a], i.cursors[b] = i.cursors[b], i.cursors[a]
}
func (i *IndexSnapshotFieldDict) Push(x interface{}) {
i.cursors = append(i.cursors, x.(*segmentDictCursor))
}
func (i *IndexSnapshotFieldDict) Pop() interface{} {
n := len(i.cursors)
x := i.cursors[n-1]
i.cursors = i.cursors[0 : n-1]
return x
}
func (i *IndexSnapshotFieldDict) Next() (*index.DictEntry, error) {
if len(i.cursors) <= 0 {
return nil, nil
}
rv := i.cursors[0].curr
next, err := i.cursors[0].itr.Next()
if err != nil {
return nil, err
}
if next == nil {
// at end of this cursor, remove it
heap.Pop(i)
} else {
// modified heap, fix it
i.cursors[0].curr = next
heap.Fix(i, 0)
}
// look for any other entries with the exact same term
for len(i.cursors) > 0 && i.cursors[0].curr.Term == rv.Term {
rv.Count += i.cursors[0].curr.Count
next, err := i.cursors[0].itr.Next()
if err != nil {
return nil, err
}
if next == nil {
// at end of this cursor, remove it
heap.Pop(i)
} else {
// modified heap, fix it
i.cursors[0].curr = next
heap.Fix(i, 0)
}
}
return rv, nil
}
func (i *IndexSnapshotFieldDict) Close() error {
return nil
}

View File

@ -0,0 +1,67 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"bytes"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
)
type IndexSnapshotDocIDReader struct {
snapshot *IndexSnapshot
iterators []roaring.IntIterable
segmentOffset int
}
func (i *IndexSnapshotDocIDReader) Next() (index.IndexInternalID, error) {
for i.segmentOffset < len(i.iterators) {
if !i.iterators[i.segmentOffset].HasNext() {
i.segmentOffset++
continue
}
next := i.iterators[i.segmentOffset].Next()
// make segment number into global number by adding offset
globalOffset := i.snapshot.offsets[i.segmentOffset]
return docNumberToBytes(nil, uint64(next)+globalOffset), nil
}
return nil, nil
}
func (i *IndexSnapshotDocIDReader) Advance(ID index.IndexInternalID) (index.IndexInternalID, error) {
// FIXME do something better
next, err := i.Next()
if err != nil {
return nil, err
}
if next == nil {
return nil, nil
}
for bytes.Compare(next, ID) < 0 {
next, err = i.Next()
if err != nil {
return nil, err
}
if next == nil {
break
}
}
return next, nil
}
func (i *IndexSnapshotDocIDReader) Close() error {
return nil
}

View File

@ -0,0 +1,132 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"bytes"
"sync/atomic"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
)
type IndexSnapshotTermFieldReader struct {
term []byte
field string
snapshot *IndexSnapshot
postings []segment.PostingsList
iterators []segment.PostingsIterator
segmentOffset int
includeFreq bool
includeNorm bool
includeTermVectors bool
currPosting segment.Posting
currID index.IndexInternalID
}
func (i *IndexSnapshotTermFieldReader) Next(preAlloced *index.TermFieldDoc) (*index.TermFieldDoc, error) {
rv := preAlloced
if rv == nil {
rv = &index.TermFieldDoc{}
}
// find the next hit
for i.segmentOffset < len(i.postings) {
next, err := i.iterators[i.segmentOffset].Next()
if err != nil {
return nil, err
}
if next != nil {
// make segment number into global number by adding offset
globalOffset := i.snapshot.offsets[i.segmentOffset]
nnum := next.Number()
rv.ID = docNumberToBytes(rv.ID, nnum+globalOffset)
i.postingToTermFieldDoc(next, rv)
i.currID = rv.ID
i.currPosting = next
return rv, nil
}
i.segmentOffset++
}
return nil, nil
}
func (i *IndexSnapshotTermFieldReader) postingToTermFieldDoc(next segment.Posting, rv *index.TermFieldDoc) {
if i.includeFreq {
rv.Freq = next.Frequency()
}
if i.includeNorm {
rv.Norm = next.Norm()
}
if i.includeTermVectors {
locs := next.Locations()
rv.Vectors = make([]*index.TermFieldVector, len(locs))
for i, loc := range locs {
rv.Vectors[i] = &index.TermFieldVector{
Start: loc.Start(),
End: loc.End(),
Pos: loc.Pos(),
ArrayPositions: loc.ArrayPositions(),
Field: loc.Field(),
}
}
}
}
func (i *IndexSnapshotTermFieldReader) Advance(ID index.IndexInternalID, preAlloced *index.TermFieldDoc) (*index.TermFieldDoc, error) {
// FIXME do something better
// for now, if we need to seek backwards, then restart from the beginning
if i.currPosting != nil && bytes.Compare(i.currID, ID) >= 0 {
i2, err := i.snapshot.TermFieldReader(i.term, i.field,
i.includeFreq, i.includeNorm, i.includeTermVectors)
if err != nil {
return nil, err
}
*i = *(i2.(*IndexSnapshotTermFieldReader))
}
// FIXME do something better
next, err := i.Next(preAlloced)
if err != nil {
return nil, err
}
if next == nil {
return nil, nil
}
for bytes.Compare(next.ID, ID) < 0 {
next, err = i.Next(preAlloced)
if err != nil {
return nil, err
}
if next == nil {
break
}
}
return next, nil
}
func (i *IndexSnapshotTermFieldReader) Count() uint64 {
var rv uint64
for _, posting := range i.postings {
rv += posting.Count()
}
return rv
}
func (i *IndexSnapshotTermFieldReader) Close() error {
if i.snapshot != nil {
atomic.AddUint64(&i.snapshot.parent.stats.termSearchersFinished, uint64(1))
}
return nil
}

View File

@ -0,0 +1,181 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"fmt"
"log"
"github.com/blevesearch/bleve/index/scorch/segment"
"github.com/boltdb/bolt"
)
type RollbackPoint struct {
epoch uint64
meta map[string][]byte
}
func (r *RollbackPoint) GetInternal(key []byte) []byte {
return r.meta[string(key)]
}
// RollbackPoints returns an array of rollback points available
// for the application to make a decision on where to rollback
// to. A nil return value indicates that there are no available
// rollback points.
func (s *Scorch) RollbackPoints() ([]*RollbackPoint, error) {
if s.rootBolt == nil {
return nil, fmt.Errorf("RollbackPoints: root is nil")
}
// start a read-only bolt transaction
tx, err := s.rootBolt.Begin(false)
if err != nil {
return nil, fmt.Errorf("RollbackPoints: failed to start" +
" read-only transaction")
}
// read-only bolt transactions to be rolled back
defer func() {
_ = tx.Rollback()
}()
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return nil, fmt.Errorf("RollbackPoints: no snapshots available")
}
rollbackPoints := []*RollbackPoint{}
c1 := snapshots.Cursor()
for k, _ := c1.Last(); k != nil; k, _ = c1.Prev() {
_, snapshotEpoch, err := segment.DecodeUvarintAscending(k)
if err != nil {
log.Printf("RollbackPoints:"+
" unable to parse segment epoch %x, continuing", k)
continue
}
snapshot := snapshots.Bucket(k)
if snapshot == nil {
log.Printf("RollbackPoints:"+
" snapshot key, but bucket missing %x, continuing", k)
continue
}
meta := map[string][]byte{}
c2 := snapshot.Cursor()
for j, _ := c2.First(); j != nil; j, _ = c2.Next() {
if j[0] == boltInternalKey[0] {
internalBucket := snapshot.Bucket(j)
err = internalBucket.ForEach(func(key []byte, val []byte) error {
copiedVal := append([]byte(nil), val...)
meta[string(key)] = copiedVal
return nil
})
if err != nil {
break
}
}
}
if err != nil {
log.Printf("RollbackPoints:"+
" failed in fetching internal data: %v", err)
continue
}
rollbackPoints = append(rollbackPoints, &RollbackPoint{
epoch: snapshotEpoch,
meta: meta,
})
}
return rollbackPoints, nil
}
// Rollback atomically and durably (if unsafeBatch is unset) brings
// the store back to the point in time as represented by the
// RollbackPoint. Rollback() should only be passed a RollbackPoint
// that came from the same store using the RollbackPoints() API.
func (s *Scorch) Rollback(to *RollbackPoint) error {
if to == nil {
return fmt.Errorf("Rollback: RollbackPoint is nil")
}
if s.rootBolt == nil {
return fmt.Errorf("Rollback: root is nil")
}
revert := &snapshotReversion{}
s.rootLock.Lock()
err := s.rootBolt.View(func(tx *bolt.Tx) error {
snapshots := tx.Bucket(boltSnapshotsBucket)
if snapshots == nil {
return fmt.Errorf("Rollback: no snapshots available")
}
pos := segment.EncodeUvarintAscending(nil, to.epoch)
snapshot := snapshots.Bucket(pos)
if snapshot == nil {
return fmt.Errorf("Rollback: snapshot not found")
}
indexSnapshot, err := s.loadSnapshot(snapshot)
if err != nil {
return fmt.Errorf("Rollback: unable to load snapshot: %v", err)
}
// add segments referenced by loaded index snapshot to the
// ineligibleForRemoval map
for _, segSnap := range indexSnapshot.segment {
filename := zapFileName(segSnap.id)
s.ineligibleForRemoval[filename] = true
}
revert.snapshot = indexSnapshot
revert.applied = make(chan error)
if !s.unsafeBatch {
revert.persisted = make(chan error)
}
return nil
})
s.rootLock.Unlock()
if err != nil {
return err
}
// introduce the reversion
s.revertToSnapshots <- revert
// block until this snapshot is applied
err = <-revert.applied
if err != nil {
return fmt.Errorf("Rollback: failed with err: %v", err)
}
if revert.persisted != nil {
err = <-revert.persisted
}
return err
}

View File

@ -0,0 +1,168 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"testing"
"github.com/blevesearch/bleve/document"
"github.com/blevesearch/bleve/index"
)
func TestIndexRollback(t *testing.T) {
defer func() {
err := DestroyTest()
if err != nil {
t.Fatal(err)
}
}()
analysisQueue := index.NewAnalysisQueue(1)
idx, err := NewScorch(Name, testConfig, analysisQueue)
if err != nil {
t.Fatal(err)
}
err = idx.Open()
if err != nil {
t.Fatalf("error opening index: %v", err)
}
defer func() {
err := idx.Close()
if err != nil {
t.Fatal(err)
}
}()
// create a batch, insert 2 new documents
batch := index.NewBatch()
doc := document.NewDocument("1")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test1")))
batch.Update(doc)
doc = document.NewDocument("2")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test2")))
batch.Update(doc)
err = idx.Batch(batch)
if err != nil {
t.Fatal(err)
}
sh, ok := idx.(*Scorch)
if !ok {
t.Fatalf("Not a scorch index?")
}
// fetch rollback points available as of here
rollbackPoints, err := sh.RollbackPoints()
if err != nil || len(rollbackPoints) == 0 {
t.Fatal(err, len(rollbackPoints))
}
// set this as a rollback point for the future
rollbackPoint := rollbackPoints[0]
// create another batch, insert 2 new documents, and delete an existing one
batch = index.NewBatch()
doc = document.NewDocument("3")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test3")))
batch.Update(doc)
doc = document.NewDocument("4")
doc.AddField(document.NewTextField("name", []uint64{}, []byte("test4")))
batch.Update(doc)
batch.Delete("1")
err = idx.Batch(batch)
if err != nil {
t.Fatal(err)
}
reader, err := idx.Reader()
if err != nil {
t.Fatal(err)
}
docCount, err := reader.DocCount()
if err != nil {
t.Fatal(err)
}
// expect docs 2, 3, 4
if docCount != 3 {
t.Fatalf("unexpected doc count: %v", docCount)
}
ret, err := reader.Document("1")
if err != nil || ret != nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("2")
if err != nil || ret == nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("3")
if err != nil || ret == nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("4")
if err != nil || ret == nil {
t.Fatal(ret, err)
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
// rollback to the selected rollback point
err = sh.Rollback(rollbackPoint)
if err != nil {
t.Fatal(err)
}
reader, err = idx.Reader()
if err != nil {
t.Fatal(err)
}
docCount, err = reader.DocCount()
if err != nil {
t.Fatal(err)
}
// expect only docs 1, 2
if docCount != 2 {
t.Fatalf("unexpected doc count: %v", docCount)
}
ret, err = reader.Document("1")
if err != nil || ret == nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("2")
if err != nil || ret == nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("3")
if err != nil || ret != nil {
t.Fatal(ret, err)
}
ret, err = reader.Document("4")
if err != nil || ret != nil {
t.Fatal(ret, err)
}
err = reader.Close()
if err != nil {
t.Fatal(err)
}
}

View File

@ -0,0 +1,274 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"sync"
"github.com/RoaringBitmap/roaring"
"github.com/blevesearch/bleve/index"
"github.com/blevesearch/bleve/index/scorch/segment"
)
var TermSeparator byte = 0xff
var TermSeparatorSplitSlice = []byte{TermSeparator}
type SegmentDictionarySnapshot struct {
s *SegmentSnapshot
d segment.TermDictionary
}
func (s *SegmentDictionarySnapshot) PostingsList(term string, except *roaring.Bitmap) (segment.PostingsList, error) {
// TODO: if except is non-nil, perhaps need to OR it with s.s.deleted?
return s.d.PostingsList(term, s.s.deleted)
}
func (s *SegmentDictionarySnapshot) Iterator() segment.DictionaryIterator {
return s.d.Iterator()
}
func (s *SegmentDictionarySnapshot) PrefixIterator(prefix string) segment.DictionaryIterator {
return s.d.PrefixIterator(prefix)
}
func (s *SegmentDictionarySnapshot) RangeIterator(start, end string) segment.DictionaryIterator {
return s.d.RangeIterator(start, end)
}
type SegmentSnapshot struct {
id uint64
segment segment.Segment
deleted *roaring.Bitmap
cachedDocs *cachedDocs
}
func (s *SegmentSnapshot) Segment() segment.Segment {
return s.segment
}
func (s *SegmentSnapshot) Deleted() *roaring.Bitmap {
return s.deleted
}
func (s *SegmentSnapshot) Id() uint64 {
return s.id
}
func (s *SegmentSnapshot) FullSize() int64 {
return int64(s.segment.Count())
}
func (s SegmentSnapshot) LiveSize() int64 {
return int64(s.Count())
}
func (s *SegmentSnapshot) Close() error {
return s.segment.Close()
}
func (s *SegmentSnapshot) VisitDocument(num uint64, visitor segment.DocumentFieldValueVisitor) error {
return s.segment.VisitDocument(num, visitor)
}
func (s *SegmentSnapshot) DocumentVisitFieldTerms(num uint64, fields []string,
visitor index.DocumentFieldTermVisitor) error {
collection := make(map[string][][]byte)
// collect field indexed values
for _, field := range fields {
dict, err := s.Dictionary(field)
if err != nil {
return err
}
dictItr := dict.Iterator()
var next *index.DictEntry
next, err = dictItr.Next()
for next != nil && err == nil {
postings, err2 := dict.PostingsList(next.Term, nil)
if err2 != nil {
return err2
}
postingsItr := postings.Iterator()
nextPosting, err2 := postingsItr.Next()
for err2 == nil && nextPosting != nil && nextPosting.Number() <= num {
if nextPosting.Number() == num {
// got what we're looking for
collection[field] = append(collection[field], []byte(next.Term))
}
nextPosting, err = postingsItr.Next()
}
if err2 != nil {
return err
}
next, err = dictItr.Next()
}
if err != nil {
return err
}
}
// invoke callback
for field, values := range collection {
for _, value := range values {
visitor(field, value)
}
}
return nil
}
func (s *SegmentSnapshot) Count() uint64 {
rv := s.segment.Count()
if s.deleted != nil {
rv -= s.deleted.GetCardinality()
}
return rv
}
func (s *SegmentSnapshot) Dictionary(field string) (segment.TermDictionary, error) {
d, err := s.segment.Dictionary(field)
if err != nil {
return nil, err
}
return &SegmentDictionarySnapshot{
s: s,
d: d,
}, nil
}
func (s *SegmentSnapshot) DocNumbers(docIDs []string) (*roaring.Bitmap, error) {
rv, err := s.segment.DocNumbers(docIDs)
if err != nil {
return nil, err
}
if s.deleted != nil {
rv.AndNot(s.deleted)
}
return rv, nil
}
// DocNumbersLive returns bitsit containing doc numbers for all live docs
func (s *SegmentSnapshot) DocNumbersLive() *roaring.Bitmap {
rv := roaring.NewBitmap()
rv.AddRange(0, s.segment.Count())
if s.deleted != nil {
rv.AndNot(s.deleted)
}
return rv
}
func (s *SegmentSnapshot) Fields() []string {
return s.segment.Fields()
}
type cachedFieldDocs struct {
readyCh chan struct{} // closed when the cachedFieldDocs.docs is ready to be used.
err error // Non-nil if there was an error when preparing this cachedFieldDocs.
docs map[uint64][]byte // Keyed by localDocNum, value is a list of terms delimited by 0xFF.
}
func (cfd *cachedFieldDocs) prepareFields(field string, ss *SegmentSnapshot) {
defer close(cfd.readyCh)
dict, err := ss.segment.Dictionary(field)
if err != nil {
cfd.err = err
return
}
dictItr := dict.Iterator()
next, err := dictItr.Next()
for err == nil && next != nil {
postings, err1 := dict.PostingsList(next.Term, nil)
if err1 != nil {
cfd.err = err1
return
}
postingsItr := postings.Iterator()
nextPosting, err2 := postingsItr.Next()
for err2 == nil && nextPosting != nil {
docNum := nextPosting.Number()
cfd.docs[docNum] = append(cfd.docs[docNum], []byte(next.Term)...)
cfd.docs[docNum] = append(cfd.docs[docNum], TermSeparator)
nextPosting, err2 = postingsItr.Next()
}
if err2 != nil {
cfd.err = err2
return
}
next, err = dictItr.Next()
}
if err != nil {
cfd.err = err
return
}
}
type cachedDocs struct {
m sync.Mutex // As the cache is asynchronously prepared, need a lock
cache map[string]*cachedFieldDocs // Keyed by field
}
func (c *cachedDocs) prepareFields(wantedFields []string, ss *SegmentSnapshot) error {
c.m.Lock()
if c.cache == nil {
c.cache = make(map[string]*cachedFieldDocs, len(ss.Fields()))
}
for _, field := range wantedFields {
_, exists := c.cache[field]
if !exists {
c.cache[field] = &cachedFieldDocs{
readyCh: make(chan struct{}),
docs: make(map[uint64][]byte),
}
go c.cache[field].prepareFields(field, ss)
}
}
for _, field := range wantedFields {
cachedFieldDocs := c.cache[field]
c.m.Unlock()
<-cachedFieldDocs.readyCh
if cachedFieldDocs.err != nil {
return cachedFieldDocs.err
}
c.m.Lock()
}
c.m.Unlock()
return nil
}
func (c *cachedDocs) sizeInBytes() uint64 {
sizeInBytes := 0
c.m.Lock()
for k, v := range c.cache { // cachedFieldDocs
sizeInBytes += len(k)
if v != nil {
for _, entry := range v.docs { // docs
sizeInBytes += 8 /* size of uint64 */ + len(entry)
}
}
}
c.m.Unlock()
return uint64(sizeInBytes)
}

54
index/scorch/stats.go Normal file
View File

@ -0,0 +1,54 @@
// Copyright (c) 2017 Couchbase, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
// See the License for the specific language governing permissions and
// limitations under the License.
package scorch
import (
"encoding/json"
"sync/atomic"
)
// Stats tracks statistics about the index
type Stats struct {
updates, deletes, batches, errors uint64
analysisTime, indexTime uint64
termSearchersStarted uint64
termSearchersFinished uint64
numPlainTextBytesIndexed uint64
numItemsIntroduced uint64
numItemsPersisted uint64
}
func (s *Stats) statsMap() map[string]interface{} {
m := map[string]interface{}{}
m["updates"] = atomic.LoadUint64(&s.updates)
m["deletes"] = atomic.LoadUint64(&s.deletes)
m["batches"] = atomic.LoadUint64(&s.batches)
m["errors"] = atomic.LoadUint64(&s.errors)
m["analysis_time"] = atomic.LoadUint64(&s.analysisTime)
m["index_time"] = atomic.LoadUint64(&s.indexTime)
m["term_searchers_started"] = atomic.LoadUint64(&s.termSearchersStarted)
m["term_searchers_finished"] = atomic.LoadUint64(&s.termSearchersFinished)
m["num_plain_text_bytes_indexed"] = atomic.LoadUint64(&s.numPlainTextBytesIndexed)
m["num_items_introduced"] = atomic.LoadUint64(&s.numItemsIntroduced)
m["num_items_persisted"] = atomic.LoadUint64(&s.numItemsPersisted)
return m
}
// MarshalJSON implements json.Marshaler
func (s *Stats) MarshalJSON() ([]byte, error) {
m := s.statsMap()
return json.Marshal(m)
}

View File

@ -13,7 +13,7 @@
// limitations under the License.
// Package moss provides a KVStore implementation based on the
// github.com/couchbaselabs/moss library.
// github.com/couchbase/moss library.
package moss

View File

@ -13,7 +13,7 @@
// limitations under the License.
// Package moss provides a KVStore implementation based on the
// github.com/couchbaselabs/moss library.
// github.com/couchbase/moss library.
package moss

View File

@ -674,16 +674,19 @@ func TestIndexMetadataRaceBug198(t *testing.T) {
}
}()
wg := sync.WaitGroup{}
wg.Add(1)
done := make(chan struct{})
go func() {
for {
select {
case <-done:
wg.Done()
return
default:
_, err := index.DocCount()
if err != nil {
t.Fatal(err)
_, err2 := index.DocCount()
if err2 != nil {
t.Fatal(err2)
}
}
}
@ -701,6 +704,7 @@ func TestIndexMetadataRaceBug198(t *testing.T) {
}
}
close(done)
wg.Wait()
}
func TestSortMatchSearch(t *testing.T) {
@ -1561,6 +1565,12 @@ func TestBatchRaceBug260(t *testing.T) {
if err != nil {
t.Fatal(err)
}
defer func() {
err := i.Close()
if err != nil {
t.Fatal(err)
}
}()
b := i.NewBatch()
err = b.Index("1", 1)
if err != nil {

View File

@ -339,7 +339,7 @@ func (im *IndexMappingImpl) newWalkContext(doc *document.Document, dm *DocumentM
doc: doc,
im: im,
dm: dm,
excludedFromAll: []string{},
excludedFromAll: []string{"_id"},
}
}

View File

@ -37,12 +37,12 @@ func (ap ArrayPositions) Equals(other ArrayPositions) bool {
type Location struct {
// Pos is the position of the term within the field, starting at 1
Pos uint64 `json:"pos"`
Pos uint64 `json:"pos"`
// Start and End are the byte offsets of the term in the field
Start uint64 `json:"start"`
End uint64 `json:"end"`
Start uint64 `json:"start"`
End uint64 `json:"end"`
// ArrayPositions contains the positions of the term within any elements.
ArrayPositions ArrayPositions `json:"array_positions"`
}

View File

@ -57,25 +57,25 @@ func NewConjunctionSearcher(indexReader index.IndexReader, qsearchers []search.S
func (s *ConjunctionSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
for _, termSearcher := range s.searchers {
sumOfSquaredWeights += termSearcher.Weight()
for _, searcher := range s.searchers {
sumOfSquaredWeights += searcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
for _, termSearcher := range s.searchers {
termSearcher.SetQueryNorm(s.queryNorm)
for _, searcher := range s.searchers {
searcher.SetQueryNorm(s.queryNorm)
}
}
func (s *ConjunctionSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, termSearcher := range s.searchers {
for i, searcher := range s.searchers {
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return err
}
@ -160,11 +160,11 @@ OUTER:
// we know all the searchers are pointing at the same thing
// so they all need to be bumped
for i, termSearcher := range s.searchers {
for i, searcher := range s.searchers {
if s.currs[i] != rv {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return nil, err
}
@ -184,6 +184,9 @@ func (s *ConjunctionSearcher) Advance(ctx *search.SearchContext, ID index.IndexI
}
}
for i := range s.searchers {
if s.currs[i] != nil && s.currs[i].IndexInternalID.Compare(ID) >= 0 {
continue
}
err := s.advanceChild(ctx, i, ID)
if err != nil {
return nil, err

View File

@ -93,25 +93,25 @@ func newDisjunctionSearcher(indexReader index.IndexReader,
func (s *DisjunctionSearcher) computeQueryNorm() {
// first calculate sum of squared weights
sumOfSquaredWeights := 0.0
for _, termSearcher := range s.searchers {
sumOfSquaredWeights += termSearcher.Weight()
for _, searcher := range s.searchers {
sumOfSquaredWeights += searcher.Weight()
}
// now compute query norm from this
s.queryNorm = 1.0 / math.Sqrt(sumOfSquaredWeights)
// finally tell all the downstream searchers the norm
for _, termSearcher := range s.searchers {
termSearcher.SetQueryNorm(s.queryNorm)
for _, searcher := range s.searchers {
searcher.SetQueryNorm(s.queryNorm)
}
}
func (s *DisjunctionSearcher) initSearchers(ctx *search.SearchContext) error {
var err error
// get all searchers pointing at their first match
for i, termSearcher := range s.searchers {
for i, searcher := range s.searchers {
if s.currs[i] != nil {
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Next(ctx)
s.currs[i], err = searcher.Next(ctx)
if err != nil {
return err
}
@ -221,11 +221,14 @@ func (s *DisjunctionSearcher) Advance(ctx *search.SearchContext,
}
// get all searchers pointing at their first match
var err error
for i, termSearcher := range s.searchers {
for i, searcher := range s.searchers {
if s.currs[i] != nil {
if s.currs[i].IndexInternalID.Compare(ID) >= 0 {
continue
}
ctx.DocumentMatchPool.Put(s.currs[i])
}
s.currs[i], err = termSearcher.Advance(ctx, ID)
s.currs[i], err = searcher.Advance(ctx, ID)
if err != nil {
return nil, err
}

View File

@ -313,6 +313,15 @@ func (s *PhraseSearcher) Advance(ctx *search.SearchContext, ID index.IndexIntern
return nil, err
}
}
if s.currMust != nil {
if s.currMust.IndexInternalID.Compare(ID) >= 0 {
return s.Next(ctx)
}
ctx.DocumentMatchPool.Put(s.currMust)
}
if s.currMust == nil {
return nil, nil
}
var err error
s.currMust, err = s.mustSearcher.Advance(ctx, ID)
if err != nil {

497
test/versus_test.go Normal file
View File

@ -0,0 +1,497 @@
// 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 test
import (
"bytes"
"encoding/json"
"fmt"
"math"
"math/rand"
"os"
"reflect"
"strconv"
"strings"
"testing"
"text/template"
"github.com/blevesearch/bleve"
"github.com/blevesearch/bleve/index/scorch"
"github.com/blevesearch/bleve/index/store/boltdb"
"github.com/blevesearch/bleve/index/upsidedown"
"github.com/blevesearch/bleve/mapping"
"github.com/blevesearch/bleve/search"
)
// Tests scorch indexer versus upsidedown/bolt indexer against various
// templated queries. Example usage from the bleve top-level directory...
//
// go test -v -run TestScorchVersusUpsideDownBolt ./test
// VERBOSE=1 FOCUS=Trista go test -v -run TestScorchVersusUpsideDownBolt ./test
//
func TestScorchVersusUpsideDownBoltAll(t *testing.T) {
(&VersusTest{
t: t,
NumDocs: 1000,
MaxWordsPerDoc: 20,
NumWords: 10,
BatchSize: 10,
NumAttemptsPerSearch: 100,
}).run(scorch.Name, boltdb.Name, upsidedown.Name, boltdb.Name, nil, nil)
}
func TestScorchVersusUpsideDownBoltSmallMNSAM(t *testing.T) {
(&VersusTest{
t: t,
Focus: "must-not-same-as-must",
NumDocs: 5,
MaxWordsPerDoc: 2,
NumWords: 1,
BatchSize: 1,
NumAttemptsPerSearch: 1,
}).run(scorch.Name, boltdb.Name, upsidedown.Name, boltdb.Name, nil, nil)
}
func TestScorchVersusUpsideDownBoltSmallCMP11(t *testing.T) {
(&VersusTest{
t: t,
Focus: "conjuncts-match-phrase-1-1",
NumDocs: 30,
MaxWordsPerDoc: 8,
NumWords: 2,
BatchSize: 1,
NumAttemptsPerSearch: 1,
}).run(scorch.Name, boltdb.Name, upsidedown.Name, boltdb.Name, nil, nil)
}
// -------------------------------------------------------
// Templates used to compare search results in the "versus" tests.
var testVersusSearchTemplates = []string{
`{
"about": "expected to return zero hits",
"query": {
"query": "title:notARealTitle"
}
}`,
`{
"about": "try straight word()'s",
"query": {
"query": "body:{{word}}"
}
}`,
`{
"about": "conjuncts on same term",
"query": {
"conjuncts": [
{ "field": "body", "term": "{{word}}", "boost": 1.0 },
{ "field": "body", "term": "{{word}}", "boost": 1.0 }
]
}
}`,
`{
"about": "disjuncts on same term",
"query": {
"disjuncts": [
{ "field": "body", "term": "{{word}}", "boost": 1.0 },
{ "field": "body", "term": "{{word}}", "boost": 1.0 }
]
}
}`,
`{
"about": "never-matching-title-conjuncts",
"query": {
"conjuncts": [
{"field": "body", "match": "{{word}}"},
{"field": "body", "match": "{{word}}"},
{"field": "title", "match": "notAnActualTitle"}
]
}
}`,
`{
"about": "never-matching-title-disjuncts",
"query": {
"disjuncts": [
{"field": "body", "match": "{{word}}"},
{"field": "body", "match": "{{word}}"},
{"field": "title", "match": "notAnActualTitle"}
]
}
}`,
`{
"about": "must-not-never-matches",
"query": {
"must_not": {"disjuncts": [
{"field": "title", "match": "notAnActualTitle"}
]},
"should": {"disjuncts": [
{"field": "body", "match": "{{word}}"}
]}
}
}`,
`{
"about": "must-not-only",
"query": {
"must_not": {"disjuncts": [
{"field": "body", "term": "{{word}}"}
]}
}
}`,
`{
"about": "must-not-same-as-must -- see: MB-27291",
"query": {
"must_not": {"disjuncts": [
{"field": "body", "match": "{{word}}"}
]},
"must": {"conjuncts": [
{"field": "body", "match": "{{word}}"}
]}
}
}`,
`{
"about": "must-not-same-as-should",
"query": {
"must_not": {"disjuncts": [
{"field": "body", "match": "{{word}}"}
]},
"should": {"disjuncts": [
{"field": "body", "match": "{{word}}"}
]}
}
}`,
`{
"about": "inspired by testrunner RQG issue -- see: MB-27291",
"query": {
"must_not": {"disjuncts": [
{"field": "title", "match": "Trista Allen"},
{"field": "body", "match": "{{word}}"}
]},
"should": {"disjuncts": [
{"field": "title", "match": "Kallie Safiya Amara"},
{"field": "body", "match": "{{word}}"}
]}
}
}`,
`{
"about": "conjuncts-match-phrase-1-1 inspired by testrunner RQG issue -- see: MB-27291",
"query": {
"conjuncts": [
{"field": "body", "match": "{{bodyWord 0}}"},
{"field": "body", "match_phrase": "{{bodyWord 1}} {{bodyWord 1}}"}
]
}
}`,
`{
"about": "conjuncts-match-phrase-1-2 inspired by testrunner RQG issue -- see: MB-27291 -- FAILS!!",
"query": {
"conjuncts": [
{"field": "body", "match": "{{bodyWord 0}}"},
{"field": "body", "match_phrase": "{{bodyWord 1}} {{bodyWord 2}}"}
]
}
}`,
}
// -------------------------------------------------------
type VersusTest struct {
t *testing.T
// Use environment variable VERBOSE=<integer> that's > 0 for more
// verbose output.
Verbose int
// Allow user to focus on particular search templates, where
// where the search template must contain the Focus string.
Focus string
NumDocs int // Number of docs to insert.
MaxWordsPerDoc int // Max number words in each doc's Body field.
NumWords int // Total number of words in the dictionary.
BatchSize int // Batch size when inserting docs.
NumAttemptsPerSearch int // For each search template, number of searches to try.
// The Bodies is an array with length NumDocs, where each entry
// is the words in a doc's Body field.
Bodies [][]string
CurAttempt int
TotAttempts int
}
// -------------------------------------------------------
func testVersusSearches(vt *VersusTest, searchTemplates []string, idxA, idxB bleve.Index) {
t := vt.t
funcMap := template.FuncMap{
// Returns a word. The word may or may not be in any
// document's body.
"word": func() string {
return vt.genWord(vt.CurAttempt % vt.NumWords)
},
// Picks a document and returns the i'th word in that
// document's body. You can use this in searches to
// definitely find at least one document.
"bodyWord": func(i int) string {
body := vt.Bodies[vt.CurAttempt%len(vt.Bodies)]
if len(body) <= 0 {
return ""
}
return body[i%len(body)]
},
}
// Optionally allow call to focus on a particular search templates,
// where the search template must contain the vt.Focus string.
if vt.Focus == "" {
vt.Focus = os.Getenv("FOCUS")
}
for i, searchTemplate := range searchTemplates {
if vt.Focus != "" && !strings.Contains(searchTemplate, vt.Focus) {
continue
}
tmpl, err := template.New("search").Funcs(funcMap).Parse(searchTemplate)
if err != nil {
t.Fatalf("could not parse search template: %s, err: %v", searchTemplate, err)
}
for j := 0; j < vt.NumAttemptsPerSearch; j++ {
vt.CurAttempt = j
var buf bytes.Buffer
err = tmpl.Execute(&buf, vt)
if err != nil {
t.Fatalf("could not execute search template: %s, err: %v", searchTemplate, err)
}
bufBytes := buf.Bytes()
if vt.Verbose > 0 {
fmt.Printf(" %s\n", bufBytes)
}
var search bleve.SearchRequest
err = json.Unmarshal(bufBytes, &search)
if err != nil {
t.Fatalf("could not unmarshal search: %s, err: %v", bufBytes, err)
}
search.Size = vt.NumDocs * 10 // Crank up limit to get all results.
searchA := search
searchB := search
resA, errA := idxA.Search(&searchA)
resB, errB := idxB.Search(&searchB)
if errA != errB {
t.Errorf("search: (%d) %s,\n err mismatch, errA: %v, errB: %v",
i, bufBytes, errA, errB)
}
// Scores might have float64 vs float32 wobbles, so truncate precision.
resA.MaxScore = math.Trunc(resA.MaxScore*1000.0) / 1000.0
resB.MaxScore = math.Trunc(resB.MaxScore*1000.0) / 1000.0
// Timings may be different between A & B, so force equality.
resA.Took = resB.Took
// Hits might have different ordering since some indexers
// (like upsidedown) have a natural secondary sort on id
// while others (like scorch) don't. So, we compare by
// putting the hits from A & B into maps.
hitsA := hitsById(resA)
hitsB := hitsById(resB)
if !reflect.DeepEqual(hitsA, hitsB) {
t.Errorf("=========\nsearch: (%d) %s,\n res hits mismatch,\n len(hitsA): %d,\n len(hitsB): %d",
i, bufBytes, len(hitsA), len(hitsB))
t.Errorf("\n hitsA: %#v,\n hitsB: %#v",
hitsA, hitsB)
for id, hitA := range hitsA {
hitB := hitsB[id]
if !reflect.DeepEqual(hitA, hitB) {
t.Errorf("\n driving from hitsA\n hitA: %#v,\n hitB: %#v", hitA, hitB)
idx, _ := strconv.Atoi(id)
t.Errorf("\n doc: %d, body: %s", idx, strings.Join(vt.Bodies[idx], " "))
}
}
for id, hitB := range hitsB {
hitA := hitsA[id]
if !reflect.DeepEqual(hitA, hitB) {
t.Errorf("\n driving from hitsB\n hitA: %#v,\n hitB: %#v", hitA, hitB)
idx, _ := strconv.Atoi(id)
t.Errorf("\n doc: %d, body: %s", idx, strings.Join(vt.Bodies[idx], " "))
}
}
}
resA.Hits = nil
resB.Hits = nil
if !reflect.DeepEqual(resA, resB) {
resAj, _ := json.Marshal(resA)
resBj, _ := json.Marshal(resB)
t.Errorf("search: (%d) %s,\n res mismatch,\n resA: %s,\n resB: %s",
i, bufBytes, resAj, resBj)
}
if vt.Verbose > 0 {
fmt.Printf(" Total: (%t) %d\n", resA.Total == resB.Total, resA.Total)
}
vt.TotAttempts++
}
}
}
// Organizes the hits into a map keyed by id.
func hitsById(res *bleve.SearchResult) map[string]*search.DocumentMatch {
rv := make(map[string]*search.DocumentMatch, len(res.Hits))
for _, hit := range res.Hits {
// Clear out or truncate precision of hit fields that might be
// different across different indexer implementations.
hit.Index = ""
hit.Score = math.Trunc(hit.Score*1000.0) / 1000.0
hit.IndexInternalID = nil
hit.HitNumber = 0
rv[hit.ID] = hit
}
return rv
}
// -------------------------------------------------------
func (vt *VersusTest) run(indexTypeA, kvStoreA, indexTypeB, kvStoreB string,
cb func(versusTest *VersusTest, searchTemplates []string, idxA, idxB bleve.Index),
searchTemplates []string) {
if cb == nil {
cb = testVersusSearches
}
if searchTemplates == nil {
searchTemplates = testVersusSearchTemplates
}
if vt.Verbose <= 0 {
vt.Verbose, _ = strconv.Atoi(os.Getenv("VERBOSE"))
}
dirA := "/tmp/bleve-versus-test-a"
dirB := "/tmp/bleve-versus-test-b"
defer func() {
_ = os.RemoveAll(dirA)
_ = os.RemoveAll(dirB)
}()
_ = os.RemoveAll(dirA)
_ = os.RemoveAll(dirB)
imA := vt.makeIndexMapping()
imB := vt.makeIndexMapping()
kvConfigA := map[string]interface{}{}
kvConfigB := map[string]interface{}{}
idxA, err := bleve.NewUsing(dirA, imA, indexTypeA, kvStoreA, kvConfigA)
if err != nil || idxA == nil {
vt.t.Fatalf("new using err: %v", err)
}
defer func() { _ = idxA.Close() }()
idxB, err := bleve.NewUsing(dirB, imB, indexTypeB, kvStoreB, kvConfigB)
if err != nil || idxB == nil {
vt.t.Fatalf("new using err: %v", err)
}
defer func() { _ = idxB.Close() }()
rand.Seed(0)
if vt.Bodies == nil {
vt.Bodies = vt.genBodies()
}
vt.insertBodies(idxA)
vt.insertBodies(idxB)
cb(vt, searchTemplates, idxA, idxB)
}
// -------------------------------------------------------
func (vt *VersusTest) makeIndexMapping() mapping.IndexMapping {
standardFM := bleve.NewTextFieldMapping()
standardFM.Store = false
standardFM.IncludeInAll = false
standardFM.IncludeTermVectors = true
standardFM.Analyzer = "standard"
dm := bleve.NewDocumentMapping()
dm.AddFieldMappingsAt("title", standardFM)
dm.AddFieldMappingsAt("body", standardFM)
im := bleve.NewIndexMapping()
im.DefaultMapping = dm
im.DefaultAnalyzer = "standard"
return im
}
func (vt *VersusTest) insertBodies(idx bleve.Index) {
batch := idx.NewBatch()
for i, bodyWords := range vt.Bodies {
title := fmt.Sprintf("%d", i)
body := strings.Join(bodyWords, " ")
err := batch.Index(title, map[string]interface{}{"title": title, "body": body})
if err != nil {
vt.t.Fatalf("batch.Index err: %v", err)
}
if i%vt.BatchSize == 0 {
err = idx.Batch(batch)
if err != nil {
vt.t.Fatalf("batch err: %v", err)
}
batch.Reset()
}
}
err := idx.Batch(batch)
if err != nil {
vt.t.Fatalf("last batch err: %v", err)
}
}
func (vt *VersusTest) genBodies() (rv [][]string) {
for i := 0; i < vt.NumDocs; i++ {
rv = append(rv, vt.genBody())
}
return rv
}
func (vt *VersusTest) genBody() (rv []string) {
m := rand.Intn(vt.MaxWordsPerDoc)
for j := 0; j < m; j++ {
rv = append(rv, vt.genWord(rand.Intn(vt.NumWords)))
}
return rv
}
func (vt *VersusTest) genWord(i int) string {
return fmt.Sprintf("%x", i)
}

4
vendor/manifest vendored
View File

@ -21,7 +21,7 @@
"importpath": "github.com/boltdb/bolt",
"repository": "https://github.com/boltdb/bolt",
"vcs": "",
"revision": "144418e1475d8bf7abbdc48583500f1a20c62ea7",
"revision": "9da31745363232bc1e27dbab3569e77383a51585",
"branch": "master",
"notests": true
},
@ -127,4 +127,4 @@
"notests": true
}
]
}
}