-
Notifications
You must be signed in to change notification settings - Fork 12
/
merge.go
616 lines (511 loc) · 16.3 KB
/
merge.go
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
// 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"
seg "github.com/blevesearch/scorch_segment_api/v2"
"github.com/golang/snappy"
)
var DefaultFileMergerBufferSize = 1024 * 1024
const docDropped = math.MaxUint64 // sentinel docNum to represent a deleted doc
// Merge takes a slice of segments and bit masks describing which
// documents may be dropped, and creates a new segment containing the
// remaining data. This new segment is built at the specified path.
func (*ZapPlugin) Merge(segments []seg.Segment, drops []*roaring.Bitmap, path string,
closeCh chan struct{}, s seg.StatsReporter) (
[][]uint64, uint64, error) {
segmentBases := make([]*SegmentBase, len(segments))
for segmenti, segment := range segments {
switch segmentx := segment.(type) {
case *Segment:
segmentBases[segmenti] = &segmentx.SegmentBase
case *SegmentBase:
segmentBases[segmenti] = segmentx
default:
panic(fmt.Sprintf("oops, unexpected segment type: %T", segment))
}
}
return mergeSegmentBases(segmentBases, drops, path, DefaultChunkMode, closeCh, s)
}
func mergeSegmentBases(segmentBases []*SegmentBase, drops []*roaring.Bitmap, path string,
chunkMode uint32, closeCh chan struct{}, s seg.StatsReporter) (
[][]uint64, uint64, error) {
flag := os.O_RDWR | os.O_CREATE
f, err := os.OpenFile(path, flag, 0600)
if err != nil {
return nil, 0, err
}
cleanup := func() {
_ = f.Close()
_ = os.Remove(path)
}
// buffer the output
br := bufio.NewWriterSize(f, DefaultFileMergerBufferSize)
// wrap it for counting (tracking offsets)
cr := NewCountHashWriterWithStatsReporter(br, s)
newDocNums, numDocs, storedIndexOffset, _, _, sectionsIndexOffset, err :=
mergeToWriter(segmentBases, drops, chunkMode, cr, closeCh)
if err != nil {
cleanup()
return nil, 0, err
}
// passing the sectionsIndexOffset as fieldsIndexOffset and the docValueOffset as 0 for the footer
err = persistFooter(numDocs, storedIndexOffset, sectionsIndexOffset, sectionsIndexOffset,
0, chunkMode, cr.Sum32(), cr)
if err != nil {
cleanup()
return nil, 0, err
}
err = br.Flush()
if err != nil {
cleanup()
return nil, 0, err
}
err = f.Sync()
if err != nil {
cleanup()
return nil, 0, err
}
err = f.Close()
if err != nil {
cleanup()
return nil, 0, err
}
return newDocNums, uint64(cr.Count()), nil
}
func mergeToWriter(segments []*SegmentBase, drops []*roaring.Bitmap,
chunkMode uint32, cr *CountHashWriter, closeCh chan struct{}) (
newDocNums [][]uint64, numDocs, storedIndexOffset uint64,
fieldsInv []string, fieldsMap map[string]uint16, sectionsIndexOffset uint64,
err error) {
var fieldsSame bool
fieldsSame, fieldsInv = mergeFields(segments)
fieldsMap = mapFields(fieldsInv)
numDocs = computeNewDocCount(segments, drops)
if isClosed(closeCh) {
return nil, 0, 0, nil, nil, 0, seg.ErrClosed
}
// the merge opaque is especially important when it comes to tracking the file
// offset a field of a particular section is at. This will be used to write the
// offsets in the fields section index of the file (the final merged file).
mergeOpaque := map[int]resetable{}
args := map[string]interface{}{
"chunkMode": chunkMode,
"fieldsSame": fieldsSame,
"fieldsMap": fieldsMap,
"numDocs": numDocs,
}
if numDocs > 0 {
storedIndexOffset, newDocNums, err = mergeStoredAndRemap(segments, drops,
fieldsMap, fieldsInv, fieldsSame, numDocs, cr, closeCh)
if err != nil {
return nil, 0, 0, nil, nil, 0, err
}
// at this point, ask each section implementation to merge itself
for i, x := range segmentSections {
mergeOpaque[int(i)] = x.InitOpaque(args)
err = x.Merge(mergeOpaque, segments, drops, fieldsInv, newDocNums, cr, closeCh)
if err != nil {
return nil, 0, 0, nil, nil, 0, err
}
}
}
// we can persist the fields section index now, this will point
// to the various indexes (each in different section) available for a field.
sectionsIndexOffset, err = persistFieldsSection(fieldsInv, cr, mergeOpaque)
if err != nil {
return nil, 0, 0, nil, nil, 0, err
}
return newDocNums, numDocs, storedIndexOffset, fieldsInv, fieldsMap, sectionsIndexOffset, nil
}
// mapFields takes the fieldsInv list and returns a map of fieldName
// to fieldID+1
func mapFields(fields []string) map[string]uint16 {
rv := make(map[string]uint16, len(fields))
for i, fieldName := range fields {
rv[fieldName] = uint16(i) + 1
}
return rv
}
// computeNewDocCount determines how many documents will be in the newly
// merged segment when obsoleted docs are dropped
func computeNewDocCount(segments []*SegmentBase, drops []*roaring.Bitmap) uint64 {
var newDocCount uint64
for segI, segment := range segments {
newDocCount += segment.numDocs
if drops[segI] != nil {
newDocCount -= drops[segI].GetCardinality()
}
}
return newDocCount
}
func mergeTermFreqNormLocsByCopying(term []byte, postItr *PostingsIterator,
newDocNums []uint64, newRoaring *roaring.Bitmap,
tfEncoder *chunkedIntCoder, locEncoder *chunkedIntCoder) (
lastDocNum uint64, lastFreq uint64, lastNorm uint64, err error) {
nextDocNum, nextFreq, nextNorm, nextFreqNormBytes, nextLocBytes, err :=
postItr.nextBytes()
for err == nil && len(nextFreqNormBytes) > 0 {
hitNewDocNum := newDocNums[nextDocNum]
if hitNewDocNum == docDropped {
return 0, 0, 0, fmt.Errorf("see hit with dropped doc num")
}
newRoaring.Add(uint32(hitNewDocNum))
err = tfEncoder.AddBytes(hitNewDocNum, nextFreqNormBytes)
if err != nil {
return 0, 0, 0, err
}
if len(nextLocBytes) > 0 {
err = locEncoder.AddBytes(hitNewDocNum, nextLocBytes)
if err != nil {
return 0, 0, 0, err
}
}
lastDocNum = hitNewDocNum
lastFreq = nextFreq
lastNorm = nextNorm
nextDocNum, nextFreq, nextNorm, nextFreqNormBytes, nextLocBytes, err =
postItr.nextBytes()
}
return lastDocNum, lastFreq, lastNorm, err
}
func mergeTermFreqNormLocs(fieldsMap map[string]uint16, term []byte, postItr *PostingsIterator,
newDocNums []uint64, newRoaring *roaring.Bitmap,
tfEncoder *chunkedIntCoder, locEncoder *chunkedIntCoder, bufLoc []uint64) (
lastDocNum uint64, lastFreq uint64, lastNorm uint64, bufLocOut []uint64, err error) {
next, err := postItr.Next()
for next != nil && err == nil {
hitNewDocNum := newDocNums[next.Number()]
if hitNewDocNum == docDropped {
return 0, 0, 0, nil, fmt.Errorf("see hit with dropped docNum")
}
newRoaring.Add(uint32(hitNewDocNum))
nextFreq := next.Frequency()
var nextNorm uint64
if pi, ok := next.(*Posting); ok {
nextNorm = pi.NormUint64()
} else {
return 0, 0, 0, nil, fmt.Errorf("unexpected posting type %T", next)
}
locs := next.Locations()
if nextFreq > 0 {
err = tfEncoder.Add(hitNewDocNum,
encodeFreqHasLocs(nextFreq, len(locs) > 0), nextNorm)
} else {
err = tfEncoder.Add(hitNewDocNum,
encodeFreqHasLocs(nextFreq, len(locs) > 0))
}
if err != nil {
return 0, 0, 0, nil, err
}
if len(locs) > 0 {
numBytesLocs := 0
for _, loc := range locs {
ap := loc.ArrayPositions()
numBytesLocs += totalUvarintBytes(uint64(fieldsMap[loc.Field()]-1),
loc.Pos(), loc.Start(), loc.End(), uint64(len(ap)), ap)
}
err = locEncoder.Add(hitNewDocNum, uint64(numBytesLocs))
if err != nil {
return 0, 0, 0, nil, err
}
for _, loc := range locs {
ap := loc.ArrayPositions()
if cap(bufLoc) < 5+len(ap) {
bufLoc = make([]uint64, 0, 5+len(ap))
}
args := bufLoc[0:5]
args[0] = uint64(fieldsMap[loc.Field()] - 1)
args[1] = loc.Pos()
args[2] = loc.Start()
args[3] = loc.End()
args[4] = uint64(len(ap))
args = append(args, ap...)
err = locEncoder.Add(hitNewDocNum, args...)
if err != nil {
return 0, 0, 0, nil, err
}
}
}
lastDocNum = hitNewDocNum
lastFreq = nextFreq
lastNorm = nextNorm
next, err = postItr.Next()
}
return lastDocNum, lastFreq, lastNorm, bufLoc, err
}
func writePostings(postings *roaring.Bitmap, tfEncoder, locEncoder *chunkedIntCoder,
use1HitEncoding func(uint64) (bool, uint64, uint64),
w *CountHashWriter, bufMaxVarintLen64 []byte) (
offset uint64, err error) {
if postings == nil {
return 0, nil
}
termCardinality := postings.GetCardinality()
if termCardinality <= 0 {
return 0, nil
}
if use1HitEncoding != nil {
encodeAs1Hit, docNum1Hit, normBits1Hit := use1HitEncoding(termCardinality)
if encodeAs1Hit {
return FSTValEncode1Hit(docNum1Hit, normBits1Hit), nil
}
}
var tfOffset uint64
tfOffset, _, err = tfEncoder.writeAt(w)
if err != nil {
return 0, err
}
var locOffset uint64
locOffset, _, err = locEncoder.writeAt(w)
if err != nil {
return 0, err
}
postingsOffset := uint64(w.Count())
n := binary.PutUvarint(bufMaxVarintLen64, tfOffset)
_, err = w.Write(bufMaxVarintLen64[:n])
if err != nil {
return 0, err
}
n = binary.PutUvarint(bufMaxVarintLen64, locOffset)
_, err = w.Write(bufMaxVarintLen64[:n])
if err != nil {
return 0, err
}
_, err = writeRoaringWithLen(postings, w, bufMaxVarintLen64)
if err != nil {
return 0, err
}
return postingsOffset, nil
}
type varintEncoder func(uint64) (int, error)
func mergeStoredAndRemap(segments []*SegmentBase, drops []*roaring.Bitmap,
fieldsMap map[string]uint16, fieldsInv []string, fieldsSame bool, newSegDocCount uint64,
w *CountHashWriter, closeCh chan struct{}) (uint64, [][]uint64, error) {
var rv [][]uint64 // The remapped or newDocNums for each segment.
var newDocNum uint64
var curr int
var data, compressed []byte
var metaBuf bytes.Buffer
varBuf := make([]byte, binary.MaxVarintLen64)
metaEncode := func(val uint64) (int, error) {
wb := binary.PutUvarint(varBuf, val)
return metaBuf.Write(varBuf[:wb])
}
vals := make([][][]byte, len(fieldsInv))
typs := make([][]byte, len(fieldsInv))
poss := make([][][]uint64, len(fieldsInv))
var posBuf []uint64
docNumOffsets := make([]uint64, newSegDocCount)
vdc := visitDocumentCtxPool.Get().(*visitDocumentCtx)
defer visitDocumentCtxPool.Put(vdc)
// for each segment
for segI, segment := range segments {
// check for the closure in meantime
if isClosed(closeCh) {
return 0, nil, seg.ErrClosed
}
segNewDocNums := make([]uint64, segment.numDocs)
dropsI := drops[segI]
// optimize when the field mapping is the same across all
// segments and there are no deletions, via byte-copying
// of stored docs bytes directly to the writer
if fieldsSame && (dropsI == nil || dropsI.GetCardinality() == 0) {
err := segment.copyStoredDocs(newDocNum, docNumOffsets, w)
if err != nil {
return 0, nil, err
}
for i := uint64(0); i < segment.numDocs; i++ {
segNewDocNums[i] = newDocNum
newDocNum++
}
rv = append(rv, segNewDocNums)
continue
}
// for each doc num
for docNum := uint64(0); docNum < segment.numDocs; docNum++ {
// TODO: roaring's API limits docNums to 32-bits?
if dropsI != nil && dropsI.Contains(uint32(docNum)) {
segNewDocNums[docNum] = docDropped
continue
}
segNewDocNums[docNum] = newDocNum
curr = 0
metaBuf.Reset()
data = data[:0]
posTemp := posBuf
// 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.visitStoredFields(vdc, docNum, func(field string, typ byte, value []byte, pos []uint64) bool {
fieldID := int(fieldsMap[field]) - 1
if fieldID < 0 {
// no entry for field in fieldsMap
return false
}
vals[fieldID] = append(vals[fieldID], value)
typs[fieldID] = append(typs[fieldID], typ)
// copy array positions to preserve them beyond the scope of this callback
var curPos []uint64
if len(pos) > 0 {
if cap(posTemp) < len(pos) {
posBuf = make([]uint64, len(pos)*len(fieldsInv))
posTemp = posBuf
}
curPos = posTemp[0:len(pos)]
copy(curPos, pos)
posTemp = posTemp[len(pos):]
}
poss[fieldID] = append(poss[fieldID], curPos)
return true
})
if err != nil {
return 0, nil, err
}
// _id field special case optimizes ExternalID() lookups
idFieldVal := vals[uint16(0)][0]
_, err = metaEncode(uint64(len(idFieldVal)))
if err != nil {
return 0, nil, err
}
// now walk the non-"_id" fields in order
for fieldID := 1; fieldID < len(fieldsInv); fieldID++ {
storedFieldValues := vals[fieldID]
stf := typs[fieldID]
spf := poss[fieldID]
var err2 error
curr, data, err2 = persistStoredFieldValues(fieldID,
storedFieldValues, stf, spf, curr, metaEncode, data)
if err2 != nil {
return 0, nil, err2
}
}
metaBytes := metaBuf.Bytes()
compressed = snappy.Encode(compressed[:cap(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(idFieldVal)+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 _id field val (counted as part of the 'compressed' data)
_, err = w.Write(idFieldVal)
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
storedIndexOffset := uint64(w.Count())
// now write out the stored doc index
for _, docNumOffset := range docNumOffsets {
err := binary.Write(w, binary.BigEndian, docNumOffset)
if err != nil {
return 0, nil, err
}
}
return storedIndexOffset, rv, nil
}
// copyStoredDocs writes out a segment's stored doc info, optimized by
// using a single Write() call for the entire set of bytes. The
// newDocNumOffsets is filled with the new offsets for each doc.
func (s *SegmentBase) copyStoredDocs(newDocNum uint64, newDocNumOffsets []uint64,
w *CountHashWriter) error {
if s.numDocs <= 0 {
return nil
}
indexOffset0, storedOffset0, _, _, _ :=
s.getDocStoredOffsets(0) // the segment's first doc
indexOffsetN, storedOffsetN, readN, metaLenN, dataLenN :=
s.getDocStoredOffsets(s.numDocs - 1) // the segment's last doc
storedOffset0New := uint64(w.Count())
storedBytes := s.mem[storedOffset0 : storedOffsetN+readN+metaLenN+dataLenN]
_, err := w.Write(storedBytes)
if err != nil {
return err
}
// remap the storedOffset's for the docs into new offsets relative
// to storedOffset0New, filling the given docNumOffsetsOut array
for indexOffset := indexOffset0; indexOffset <= indexOffsetN; indexOffset += 8 {
storedOffset := binary.BigEndian.Uint64(s.mem[indexOffset : indexOffset+8])
storedOffsetNew := storedOffset - storedOffset0 + storedOffset0New
newDocNumOffsets[newDocNum] = storedOffsetNew
newDocNum += 1
}
return nil
}
// mergeFields builds a unified list of fields used across all the
// input segments, and computes whether the fields are the same across
// segments (which depends on fields to be sorted in the same way
// across segments)
func mergeFields(segments []*SegmentBase) (bool, []string) {
fieldsSame := true
var segment0Fields []string
if len(segments) > 0 {
segment0Fields = segments[0].Fields()
}
fieldsExist := map[string]struct{}{}
for _, segment := range segments {
fields := segment.Fields()
for fieldi, field := range fields {
fieldsExist[field] = struct{}{}
if len(segment0Fields) != len(fields) || segment0Fields[fieldi] != field {
fieldsSame = false
}
}
}
rv := make([]string, 0, len(fieldsExist))
// ensure _id stays first
rv = append(rv, "_id")
for k := range fieldsExist {
if k != "_id" {
rv = append(rv, k)
}
}
sort.Strings(rv[1:]) // leave _id as first
return fieldsSame, rv
}
func isClosed(closeCh chan struct{}) bool {
select {
case <-closeCh:
return true
default:
return false
}
}