initial cut of merger that actually introduces changes
This commit is contained in:
parent
c15c3c11cd
commit
f83c9f2a20
|
@ -42,6 +42,82 @@ OUTER:
|
|||
|
||||
case notify = <-s.introducerNotifier:
|
||||
|
||||
case nextMerge := <-s.merges:
|
||||
// acquire lock
|
||||
s.rootLock.Lock()
|
||||
|
||||
// prepare new index snapshot
|
||||
currSize := len(s.root.segment)
|
||||
newSize := currSize + 1 - len(nextMerge.old)
|
||||
newSnapshot := &IndexSnapshot{
|
||||
segment: make([]*SegmentSnapshot, 0, newSize),
|
||||
offsets: make([]uint64, 0, newSize),
|
||||
internal: make(map[string][]byte, len(s.root.segment)),
|
||||
epoch: s.nextSnapshotEpoch,
|
||||
}
|
||||
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,
|
||||
notify: s.root.segment[i].notify,
|
||||
deleted: s.root.segment[i].deleted,
|
||||
})
|
||||
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,
|
||||
deleted: newSegmentDeleted,
|
||||
})
|
||||
newSnapshot.offsets = append(newSnapshot.offsets, running)
|
||||
|
||||
// copy old values
|
||||
for key, oldVal := range s.root.internal {
|
||||
newSnapshot.internal[key] = oldVal
|
||||
}
|
||||
|
||||
// swap in new segment
|
||||
s.root = newSnapshot
|
||||
// release lock
|
||||
s.rootLock.Unlock()
|
||||
|
||||
// notify merger we incorporated this
|
||||
close(nextMerge.notify)
|
||||
|
||||
// notify persister
|
||||
if notify != nil {
|
||||
close(notify)
|
||||
notify = nil
|
||||
}
|
||||
|
||||
case next := <-s.introductions:
|
||||
// acquire lock
|
||||
s.rootLock.Lock()
|
||||
|
|
|
@ -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 scorch
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"log"
|
||||
"os"
|
||||
"sync/atomic"
|
||||
|
||||
"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
|
||||
s.rootLock.RUnlock()
|
||||
|
||||
if ourSnapshot.epoch != lastEpochMergePlanned {
|
||||
// lets get started
|
||||
err := s.planMergeAtSnapshot(ourSnapshot)
|
||||
if err != nil {
|
||||
log.Printf("merging err: %v", err)
|
||||
continue OUTER
|
||||
}
|
||||
lastEpochMergePlanned = ourSnapshot.epoch
|
||||
}
|
||||
|
||||
// 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
|
||||
s.rootLock.RUnlock()
|
||||
|
||||
if ourSnapshot.epoch != lastEpochMergePlanned {
|
||||
// lets get started
|
||||
err := s.planMergeAtSnapshot(ourSnapshot)
|
||||
if err != nil {
|
||||
continue OUTER
|
||||
}
|
||||
lastEpochMergePlanned = ourSnapshot.epoch
|
||||
}
|
||||
|
||||
// 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 := fmt.Sprintf("%x.zap", newSegmentID)
|
||||
path := s.path + string(os.PathSeparator) + filename
|
||||
newDocNums, err := zap.Merge(segmentsToMerge, docsToDrop, path, 1024)
|
||||
if err != nil {
|
||||
return fmt.Errorf("merging failed: %v", err)
|
||||
}
|
||||
segment, err := zap.Open(path)
|
||||
if err != nil {
|
||||
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 {
|
||||
<-notification
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
type segmentMerge struct {
|
||||
id uint64
|
||||
old map[uint64]*SegmentSnapshot
|
||||
oldNewDocNums map[uint64][]uint64
|
||||
new segment.Segment
|
||||
notify notificationChan
|
||||
}
|
|
@ -31,12 +31,14 @@ import (
|
|||
type notificationChan chan struct{}
|
||||
|
||||
func (s *Scorch) persisterLoop() {
|
||||
var notify notificationChan
|
||||
var lastPersistedEpoch uint64
|
||||
OUTER:
|
||||
for {
|
||||
select {
|
||||
case <-s.closeCh:
|
||||
break OUTER
|
||||
case notify = <-s.persisterNotifier:
|
||||
|
||||
default:
|
||||
// check to see if there is a new snapshot to persist
|
||||
|
@ -53,6 +55,10 @@ OUTER:
|
|||
continue OUTER
|
||||
}
|
||||
lastPersistedEpoch = ourSnapshot.epoch
|
||||
if notify != nil {
|
||||
close(notify)
|
||||
notify = nil
|
||||
}
|
||||
}
|
||||
|
||||
// tell the introducer we're waiting for changes
|
||||
|
@ -79,6 +85,10 @@ OUTER:
|
|||
continue OUTER
|
||||
}
|
||||
lastPersistedEpoch = ourSnapshot.epoch
|
||||
if notify != nil {
|
||||
close(notify)
|
||||
notify = nil
|
||||
}
|
||||
}
|
||||
|
||||
// now wait for it (but also detect close)
|
||||
|
|
|
@ -54,7 +54,9 @@ type Scorch struct {
|
|||
|
||||
closeCh chan struct{}
|
||||
introductions chan *segmentIntroduction
|
||||
merges chan *segmentMerge
|
||||
introducerNotifier chan notificationChan
|
||||
persisterNotifier chan notificationChan
|
||||
rootBolt *bolt.DB
|
||||
asyncTasks sync.WaitGroup
|
||||
}
|
||||
|
@ -115,7 +117,9 @@ func (s *Scorch) Open() error {
|
|||
|
||||
s.closeCh = make(chan struct{})
|
||||
s.introductions = make(chan *segmentIntroduction)
|
||||
s.merges = make(chan *segmentMerge)
|
||||
s.introducerNotifier = make(chan notificationChan)
|
||||
s.persisterNotifier = make(chan notificationChan)
|
||||
|
||||
s.asyncTasks.Add(1)
|
||||
go s.mainLoop()
|
||||
|
@ -123,6 +127,8 @@ func (s *Scorch) Open() error {
|
|||
if !s.readOnly && s.path != "" {
|
||||
s.asyncTasks.Add(1)
|
||||
go s.persisterLoop()
|
||||
s.asyncTasks.Add(1)
|
||||
go s.mergerLoop()
|
||||
}
|
||||
|
||||
return nil
|
||||
|
|
|
@ -19,13 +19,13 @@ import (
|
|||
// 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) error {
|
||||
|
||||
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 err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// bufer the output
|
||||
|
@ -41,50 +41,49 @@ func Merge(segments []*Segment, drops []*roaring.Bitmap, path string, chunkFacto
|
|||
|
||||
var newDocNums [][]uint64
|
||||
var storedIndexOffset uint64
|
||||
storedIndexOffset, newDocNums, err = mergeStoredAndRemap(segments, drops,
|
||||
fieldsMap, fieldsInv, newSegDocCount, cr)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
dictLocs := make([]uint64, len(fieldsInv))
|
||||
if newSegDocCount > 0 {
|
||||
storedIndexOffset, newDocNums, err = mergeStoredAndRemap(segments, drops,
|
||||
fieldsMap, fieldsInv, newSegDocCount, cr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// FIXME temp until computed
|
||||
//dictLocs := make([]uint64, len(fieldsInv))
|
||||
|
||||
var dictLocs []uint64
|
||||
dictLocs, err = persistMergedRest(segments, drops, fieldsInv, fieldsMap,
|
||||
newDocNums, newSegDocCount, cr)
|
||||
if err != nil {
|
||||
return err
|
||||
dictLocs, err = persistMergedRest(segments, drops, fieldsInv, fieldsMap,
|
||||
newDocNums, newSegDocCount, cr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
}
|
||||
|
||||
var fieldsIndexOffset uint64
|
||||
fieldsIndexOffset, err = persistMergedFields(fieldsInv, cr, dictLocs)
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = persistFooter(newSegDocCount, storedIndexOffset,
|
||||
fieldsIndexOffset, chunkFactor, cr)
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = br.Flush()
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = f.Sync()
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
err = f.Close()
|
||||
if err != nil {
|
||||
return err
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return nil
|
||||
return newDocNums, nil
|
||||
}
|
||||
|
||||
func mapFields(fields []string) map[string]uint16 {
|
||||
|
@ -161,11 +160,13 @@ func persistMergedRest(segments []*Segment, drops []*roaring.Bitmap,
|
|||
}
|
||||
dicts = append(dicts, dict)
|
||||
|
||||
itr, err2 := dict.fst.Iterator(nil, nil)
|
||||
if err2 != nil {
|
||||
return nil, err2
|
||||
if dict != nil && dict.fst != nil {
|
||||
itr, err2 := dict.fst.Iterator(nil, nil)
|
||||
if err2 != nil {
|
||||
return nil, err2
|
||||
}
|
||||
itrs = append(itrs, itr)
|
||||
}
|
||||
itrs = append(itrs, itr)
|
||||
}
|
||||
|
||||
// create merging iterator
|
||||
|
@ -187,6 +188,9 @@ func persistMergedRest(segments []*Segment, drops []*roaring.Bitmap,
|
|||
// 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, err2
|
||||
|
|
|
@ -54,7 +54,7 @@ func TestMerge(t *testing.T) {
|
|||
segsToMerge[0] = segment.(*Segment)
|
||||
segsToMerge[1] = segment2.(*Segment)
|
||||
|
||||
err = Merge(segsToMerge, []*roaring.Bitmap{nil, nil}, "/tmp/scorch3.zap", 1024)
|
||||
_, err = Merge(segsToMerge, []*roaring.Bitmap{nil, nil}, "/tmp/scorch3.zap", 1024)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
@ -109,7 +109,7 @@ func TestMergeAndDrop(t *testing.T) {
|
|||
docsToDrop[1] = roaring.NewBitmap()
|
||||
docsToDrop[1].AddInt(1)
|
||||
|
||||
err = Merge(segsToMerge, docsToDrop, "/tmp/scorch3.zap", 1024)
|
||||
_, err = Merge(segsToMerge, docsToDrop, "/tmp/scorch3.zap", 1024)
|
||||
if err != nil {
|
||||
t.Fatal(err)
|
||||
}
|
||||
|
|
|
@ -49,6 +49,18 @@ type SegmentSnapshot struct {
|
|||
notify []chan error
|
||||
}
|
||||
|
||||
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()
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue