scorch handles non-updating batches better
This commit improves handling when an incoming batch has internal-data updates only and no doc updates. In this case, a nil segment instead of an empty segment instance is used in the segmentIntroduction. The segmentIntroduction, that is, might now hold only internal-data updates only. To handle synchronous persistence, a new field that's a slice of persisted notification channels is added to the IndexSnapshot struct, which the persister goroutine will close as each IndexSnapshot is persisted. Also, as part of this change, instead of checking the unsafeBatch flag in several places, we instead check for non-nil'ness of these persisted chan's.
This commit is contained in:
parent
cf1dd4cb00
commit
ecbb3d2df4
|
@ -66,12 +66,14 @@ func (s *Scorch) introduceSegment(next *segmentIntroduction) error {
|
|||
// acquire lock
|
||||
s.rootLock.Lock()
|
||||
|
||||
// prepare new index snapshot, with curr size + 1
|
||||
nsegs := len(s.root.segment)
|
||||
|
||||
// prepare new index snapshot
|
||||
newSnapshot := &IndexSnapshot{
|
||||
parent: s,
|
||||
segment: make([]*SegmentSnapshot, len(s.root.segment)+1),
|
||||
offsets: make([]uint64, len(s.root.segment)+1),
|
||||
internal: make(map[string][]byte, len(s.root.segment)),
|
||||
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,
|
||||
}
|
||||
|
@ -95,7 +97,7 @@ func (s *Scorch) introduceSegment(next *segmentIntroduction) error {
|
|||
newSnapshot.segment[i] = &SegmentSnapshot{
|
||||
id: s.root.segment[i].id,
|
||||
segment: s.root.segment[i].segment,
|
||||
notify: s.root.segment[i].notify,
|
||||
persisted: s.root.segment[i].persisted,
|
||||
cachedDocs: s.root.segment[i].cachedDocs,
|
||||
}
|
||||
s.root.segment[i].segment.AddRef()
|
||||
|
@ -110,18 +112,22 @@ func (s *Scorch) introduceSegment(next *segmentIntroduction) error {
|
|||
running += s.root.segment[i].Count()
|
||||
|
||||
}
|
||||
// put new segment at end
|
||||
newSnapshot.segment[len(s.root.segment)] = &SegmentSnapshot{
|
||||
id: next.id,
|
||||
segment: next.data, // Take ownership of next.data's ref-count.
|
||||
cachedDocs: &cachedDocs{cache: nil},
|
||||
}
|
||||
newSnapshot.offsets[len(s.root.segment)] = running
|
||||
if !s.unsafeBatch {
|
||||
newSnapshot.segment[len(s.root.segment)].notify = append(
|
||||
newSnapshot.segment[len(s.root.segment)].notify,
|
||||
next.persisted,
|
||||
)
|
||||
// append new segment, if any, to end of the new index snapshot
|
||||
if next.data != nil {
|
||||
newSnapshot.segment = append(newSnapshot.segment, &SegmentSnapshot{
|
||||
id: next.id,
|
||||
segment: next.data, // take ownership of next.data's ref-count
|
||||
cachedDocs: &cachedDocs{cache: nil},
|
||||
})
|
||||
newSnapshot.offsets = append(newSnapshot.offsets, running)
|
||||
if next.persisted != nil {
|
||||
newSnapshot.segment[nsegs].persisted =
|
||||
append(newSnapshot.segment[nsegs].persisted, next.persisted)
|
||||
}
|
||||
} else { // new segment might be nil when it's an internal data update only
|
||||
if next.persisted != nil {
|
||||
newSnapshot.persisted = append(newSnapshot.persisted, next.persisted)
|
||||
}
|
||||
}
|
||||
// copy old values
|
||||
for key, oldVal := range s.root.internal {
|
||||
|
@ -161,7 +167,7 @@ func (s *Scorch) introduceMerge(nextMerge *segmentMerge) {
|
|||
parent: s,
|
||||
segment: make([]*SegmentSnapshot, 0, newSize),
|
||||
offsets: make([]uint64, 0, newSize),
|
||||
internal: make(map[string][]byte, len(s.root.segment)),
|
||||
internal: s.root.internal,
|
||||
epoch: s.nextSnapshotEpoch,
|
||||
refs: 1,
|
||||
}
|
||||
|
@ -193,8 +199,8 @@ func (s *Scorch) introduceMerge(nextMerge *segmentMerge) {
|
|||
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,
|
||||
persisted: s.root.segment[i].persisted,
|
||||
cachedDocs: s.root.segment[i].cachedDocs,
|
||||
})
|
||||
s.root.segment[i].segment.AddRef()
|
||||
|
@ -206,17 +212,12 @@ func (s *Scorch) introduceMerge(nextMerge *segmentMerge) {
|
|||
// 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.
|
||||
segment: nextMerge.new, // take ownership for nextMerge.new's ref-count
|
||||
deleted: newSegmentDeleted,
|
||||
cachedDocs: &cachedDocs{cache: nil},
|
||||
})
|
||||
newSnapshot.offsets = append(newSnapshot.offsets, running)
|
||||
|
||||
// copy old values
|
||||
for key, oldVal := range s.root.internal {
|
||||
newSnapshot.internal[key] = oldVal
|
||||
}
|
||||
|
||||
// swap in new segment
|
||||
rootPrev := s.root
|
||||
s.root = newSnapshot
|
||||
|
|
|
@ -222,7 +222,10 @@ func (s *Scorch) persistSnapshot(snapshot *IndexSnapshot) error {
|
|||
}
|
||||
|
||||
// get write lock and update the current snapshot with disk-based versions
|
||||
var notifications []chan error
|
||||
snapshot.m.Lock()
|
||||
notifications := snapshot.persisted
|
||||
snapshot.persisted = nil
|
||||
snapshot.m.Unlock()
|
||||
|
||||
s.rootLock.Lock()
|
||||
newIndexSnapshot := &IndexSnapshot{
|
||||
|
@ -244,7 +247,7 @@ func (s *Scorch) persistSnapshot(snapshot *IndexSnapshot) error {
|
|||
}
|
||||
newIndexSnapshot.segment[i] = newSegmentSnapshot
|
||||
// add the old segment snapshots notifications to the list
|
||||
for _, notification := range segmentSnapshot.notify {
|
||||
for _, notification := range segmentSnapshot.persisted {
|
||||
notifications = append(notifications, notification)
|
||||
}
|
||||
} else {
|
||||
|
|
|
@ -215,11 +215,9 @@ func (s *Scorch) Batch(batch *index.Batch) error {
|
|||
|
||||
// wait for analysis result
|
||||
analysisResults := make([]*index.AnalysisResult, int(numUpdates))
|
||||
// newRowsMap := make(map[string][]index.IndexRow)
|
||||
var itemsDeQueued uint64
|
||||
for itemsDeQueued < numUpdates {
|
||||
result := <-resultChan
|
||||
//newRowsMap[result.DocID] = result.Rows
|
||||
analysisResults[itemsDeQueued] = result
|
||||
itemsDeQueued++
|
||||
}
|
||||
|
@ -230,12 +228,10 @@ func (s *Scorch) Batch(batch *index.Batch) error {
|
|||
var newSegment segment.Segment
|
||||
if len(analysisResults) > 0 {
|
||||
newSegment = mem.NewFromAnalyzedDocs(analysisResults)
|
||||
} else {
|
||||
newSegment = mem.New()
|
||||
}
|
||||
|
||||
err := s.prepareSegment(newSegment, ids, batch.InternalOps)
|
||||
if err != nil {
|
||||
if err != nil && newSegment != nil {
|
||||
_ = newSegment.Close()
|
||||
}
|
||||
return err
|
||||
|
@ -278,7 +274,7 @@ func (s *Scorch) prepareSegment(newSegment segment.Segment, ids []string,
|
|||
return err
|
||||
}
|
||||
|
||||
if !s.unsafeBatch {
|
||||
if introduction.persisted != nil {
|
||||
err = <-introduction.persisted
|
||||
}
|
||||
|
||||
|
|
|
@ -48,6 +48,8 @@ type IndexSnapshot struct {
|
|||
|
||||
m sync.Mutex // Protects the fields that follow.
|
||||
refs int64
|
||||
|
||||
persisted []chan error
|
||||
}
|
||||
|
||||
func (i *IndexSnapshot) AddRef() {
|
||||
|
|
|
@ -52,7 +52,8 @@ type SegmentSnapshot struct {
|
|||
segment segment.Segment
|
||||
deleted *roaring.Bitmap
|
||||
|
||||
notify []chan error
|
||||
persisted []chan error // closed when the segment is persisted
|
||||
|
||||
cachedDocs *cachedDocs
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue