-
Notifications
You must be signed in to change notification settings - Fork 678
/
segment_group_compaction.go
488 lines (400 loc) · 13.2 KB
/
segment_group_compaction.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
// _ _
// __ _____ __ ___ ___ __ _| |_ ___
// \ \ /\ / / _ \/ _` \ \ / / |/ _` | __/ _ \
// \ V V / __/ (_| |\ V /| | (_| | || __/
// \_/\_/ \___|\__,_| \_/ |_|\__,_|\__\___|
//
// Copyright © 2016 - 2024 Weaviate B.V. All rights reserved.
//
// CONTACT: hello@weaviate.io
//
package lsmkv
import (
"fmt"
"math"
"os"
"path/filepath"
"strings"
"github.com/pkg/errors"
"github.com/prometheus/client_golang/prometheus"
"github.com/sirupsen/logrus"
"github.com/weaviate/weaviate/adapters/repos/db/lsmkv/segmentindex"
"github.com/weaviate/weaviate/adapters/repos/db/roaringset"
"github.com/weaviate/weaviate/entities/cyclemanager"
)
func (sg *SegmentGroup) bestCompactionCandidatePair() []int {
sg.maintenanceLock.RLock()
defer sg.maintenanceLock.RUnlock()
// if true, the parent shard has indicated that it has
// entered an immutable state. During this time, the
// SegmentGroup should refrain from flushing until its
// shard indicates otherwise
if sg.isReadyOnly() {
return nil
}
// Nothing to compact
if len(sg.segments) < 2 {
return nil
}
// first determine the lowest level with candidates
levels := map[uint16]int{}
lowestPairLevel := uint16(math.MaxUint16)
lowestLevel := uint16(math.MaxUint16)
lowestIndex := -1
secondLowestIndex := -1
pairExists := false
for ind, seg := range sg.segments {
levels[seg.level]++
val := levels[seg.level]
if val > 1 {
if seg.level < lowestPairLevel {
lowestPairLevel = seg.level
pairExists = true
}
}
if seg.level < lowestLevel {
secondLowestIndex = lowestIndex
lowestLevel = seg.level
lowestIndex = ind
}
}
if pairExists {
// now pick any two segments which match the level
var res []int
for i, segment := range sg.segments {
if len(res) >= 2 {
break
}
if segment.level == lowestPairLevel {
res = append(res, i)
}
}
return res
} else {
if sg.compactLeftOverSegments {
// Some segments exist, but none are of the same level
// Merge the two lowest segments
return []int{secondLowestIndex, lowestIndex}
} else {
// No segments of the same level exist, and we are not allowed to merge the lowest segments
// This means we cannot compact. Set COMPACT_LEFTOVER_SEGMENTS to true to compact the remaining segments
return nil
}
}
}
// segmentAtPos retrieves the segment for the given position using a read-lock
func (sg *SegmentGroup) segmentAtPos(pos int) *segment {
sg.maintenanceLock.RLock()
defer sg.maintenanceLock.RUnlock()
return sg.segments[pos]
}
func segmentID(path string) string {
filename := filepath.Base(path)
return strings.TrimSuffix(strings.TrimPrefix(filename, "segment-"), ".db")
}
func (sg *SegmentGroup) compactOnce() (bool, error) {
// Is it safe to only occasionally lock instead of the entire duration? Yes,
// because other than compaction the only change to the segments array could
// be an append because of a new flush cycle, so we do not need to guarantee
// that the array contents stay stable over the duration of an entire
// compaction. We do however need to protect against a read-while-write (race
// condition) on the array. Thus any read from sg.segments need to protected
pair := sg.bestCompactionCandidatePair()
if pair == nil {
// nothing to do
return false, nil
}
if sg.allocChecker != nil {
// allocChecker is optional
if err := sg.allocChecker.CheckAlloc(100 * 1024 * 1024); err != nil {
// if we don't have at least 100MB to spare, don't start a compaction. A
// compaction does not actually need a 100MB, but it will create garbage
// that needs to be cleaned up. If we're so close to the memory limit, we
// can increase stability by preventing anything that's not strictly
// necessary. Compactions can simply resume when the cluster has been
// scaled.
sg.logger.WithFields(logrus.Fields{
"action": "lsm_compaction",
"event": "compaction_skipped_oom",
"path": sg.dir,
}).WithError(err).
Warnf("skipping compaction due to memory pressure")
return false, nil
}
}
leftSegment := sg.segmentAtPos(pair[0])
rightSegment := sg.segmentAtPos(pair[1])
if !sg.compactionFitsSizeLimit(leftSegment, rightSegment) {
// nothing to do this round, let's wait for the next round in the hopes
// that we'll find smaller (lower-level) segments that can still fit.
return false, nil
}
path := filepath.Join(sg.dir, "segment-"+segmentID(leftSegment.path)+"_"+segmentID(rightSegment.path)+".db.tmp")
f, err := os.Create(path)
if err != nil {
return false, err
}
scratchSpacePath := rightSegment.path + "compaction.scratch.d"
// the assumption is that the first element is older, and/or a higher level
level := leftSegment.level
secondaryIndices := leftSegment.secondaryIndexCount
if level == rightSegment.level {
level = level + 1
}
strategy := leftSegment.strategy
cleanupTombstones := !sg.keepTombstones && pair[0] == 0
pathLabel := "n/a"
if sg.metrics != nil && !sg.metrics.groupClasses {
pathLabel = sg.dir
}
switch strategy {
// TODO: call metrics just once with variable strategy label
case segmentindex.StrategyReplace:
c := newCompactorReplace(f, leftSegment.newCursor(),
rightSegment.newCursor(), level, secondaryIndices, scratchSpacePath, cleanupTombstones)
if sg.metrics != nil {
sg.metrics.CompactionReplace.With(prometheus.Labels{"path": pathLabel}).Inc()
defer sg.metrics.CompactionReplace.With(prometheus.Labels{"path": pathLabel}).Dec()
}
if err := c.do(); err != nil {
return false, err
}
case segmentindex.StrategySetCollection:
c := newCompactorSetCollection(f, leftSegment.newCollectionCursor(),
rightSegment.newCollectionCursor(), level, secondaryIndices,
scratchSpacePath, cleanupTombstones)
if sg.metrics != nil {
sg.metrics.CompactionSet.With(prometheus.Labels{"path": pathLabel}).Inc()
defer sg.metrics.CompactionSet.With(prometheus.Labels{"path": pathLabel}).Dec()
}
if err := c.do(); err != nil {
return false, err
}
case segmentindex.StrategyMapCollection:
c := newCompactorMapCollection(f,
leftSegment.newCollectionCursorReusable(),
rightSegment.newCollectionCursorReusable(),
level, secondaryIndices, scratchSpacePath, sg.mapRequiresSorting, cleanupTombstones)
if sg.metrics != nil {
sg.metrics.CompactionMap.With(prometheus.Labels{"path": pathLabel}).Inc()
defer sg.metrics.CompactionMap.With(prometheus.Labels{"path": pathLabel}).Dec()
}
if err := c.do(); err != nil {
return false, err
}
case segmentindex.StrategyRoaringSet:
leftCursor := leftSegment.newRoaringSetCursor()
rightCursor := rightSegment.newRoaringSetCursor()
c := roaringset.NewCompactor(f, leftCursor, rightCursor,
level, scratchSpacePath, cleanupTombstones)
if sg.metrics != nil {
sg.metrics.CompactionRoaringSet.With(prometheus.Labels{"path": pathLabel}).Set(1)
defer sg.metrics.CompactionRoaringSet.With(prometheus.Labels{"path": pathLabel}).Set(0)
}
if err := c.Do(); err != nil {
return false, err
}
default:
return false, errors.Errorf("unrecognized strategy %v", strategy)
}
if err := f.Sync(); err != nil {
return false, errors.Wrap(err, "fsync compacted segment file")
}
if err := f.Close(); err != nil {
return false, errors.Wrap(err, "close compacted segment file")
}
if err := sg.replaceCompactedSegments(pair[0], pair[1], path); err != nil {
return false, errors.Wrap(err, "replace compacted segments")
}
return true, nil
}
func (sg *SegmentGroup) replaceCompactedSegments(old1, old2 int,
newPathTmp string,
) error {
sg.maintenanceLock.RLock()
updatedCountNetAdditions := sg.segments[old1].countNetAdditions +
sg.segments[old2].countNetAdditions
sg.maintenanceLock.RUnlock()
precomputedFiles, err := preComputeSegmentMeta(newPathTmp,
updatedCountNetAdditions, sg.logger,
sg.useBloomFilter, sg.calcCountNetAdditions)
if err != nil {
return fmt.Errorf("precompute segment meta: %w", err)
}
sg.maintenanceLock.Lock()
defer sg.maintenanceLock.Unlock()
leftSegment := sg.segments[old1]
rightSegment := sg.segments[old2]
if err := leftSegment.close(); err != nil {
return errors.Wrap(err, "close disk segment")
}
if err := rightSegment.close(); err != nil {
return errors.Wrap(err, "close disk segment")
}
if err := leftSegment.drop(); err != nil {
return errors.Wrap(err, "drop disk segment")
}
if err := rightSegment.drop(); err != nil {
return errors.Wrap(err, "drop disk segment")
}
sg.segments[old1] = nil
sg.segments[old2] = nil
var newPath string
// the old segments have been deleted, we can now safely remove the .tmp
// extension from the new segment itself and the pre-computed files which
// carried the name of the second old segment
for i, path := range precomputedFiles {
updated, err := sg.stripTmpExtension(path, segmentID(leftSegment.path), segmentID(rightSegment.path))
if err != nil {
return errors.Wrap(err, "strip .tmp extension of new segment")
}
if i == 0 {
// the first element in the list is the segment itself
newPath = updated
}
}
seg, err := newSegment(newPath, sg.logger, sg.metrics, nil,
sg.mmapContents, sg.useBloomFilter, sg.calcCountNetAdditions, false)
if err != nil {
return errors.Wrap(err, "create new segment")
}
sg.segments[old2] = seg
sg.segments = append(sg.segments[:old1], sg.segments[old1+1:]...)
return nil
}
func (sg *SegmentGroup) stripTmpExtension(oldPath, left, right string) (string, error) {
ext := filepath.Ext(oldPath)
if ext != ".tmp" {
return "", errors.Errorf("segment %q did not have .tmp extension", oldPath)
}
newPath := oldPath[:len(oldPath)-len(ext)]
newPath = strings.ReplaceAll(newPath, fmt.Sprintf("%s_%s", left, right), right)
if err := os.Rename(oldPath, newPath); err != nil {
return "", errors.Wrapf(err, "rename %q -> %q", oldPath, newPath)
}
return newPath, nil
}
func (sg *SegmentGroup) compactIfLevelsMatch(shouldAbort cyclemanager.ShouldAbortCallback) bool {
sg.monitorSegments()
compacted, err := sg.compactOnce()
if err != nil {
sg.logger.WithField("action", "lsm_compaction").
WithField("path", sg.dir).
WithError(err).
Errorf("compaction failed")
}
if compacted {
return true
} else {
sg.logger.WithField("action", "lsm_compaction").
WithField("path", sg.dir).
Trace("no segment eligible for compaction")
return false
}
}
func (sg *SegmentGroup) Len() int {
sg.maintenanceLock.RLock()
defer sg.maintenanceLock.RUnlock()
return len(sg.segments)
}
func (sg *SegmentGroup) monitorSegments() {
if sg.metrics == nil || sg.metrics.groupClasses {
return
}
sg.metrics.ActiveSegments.With(prometheus.Labels{
"strategy": sg.strategy,
"path": sg.dir,
}).Set(float64(sg.Len()))
stats := sg.segmentLevelStats()
stats.fillMissingLevels()
stats.report(sg.metrics, sg.strategy, sg.dir)
}
type segmentLevelStats struct {
indexes map[uint16]int
payloads map[uint16]int
count map[uint16]int
}
func newSegmentLevelStats() segmentLevelStats {
return segmentLevelStats{
indexes: map[uint16]int{},
payloads: map[uint16]int{},
count: map[uint16]int{},
}
}
func (sg *SegmentGroup) segmentLevelStats() segmentLevelStats {
sg.maintenanceLock.RLock()
defer sg.maintenanceLock.RUnlock()
stats := newSegmentLevelStats()
for _, seg := range sg.segments {
stats.count[seg.level]++
cur := stats.indexes[seg.level]
cur += seg.index.Size()
stats.indexes[seg.level] = cur
cur = stats.payloads[seg.level]
cur += seg.PayloadSize()
stats.payloads[seg.level] = cur
}
return stats
}
// fill missing levels
//
// Imagine we had exactly two segments of level 4 before, and there were just
// compacted to single segment of level 5. As a result, there should be no
// more segments of level 4. However, our current logic only loops over
// existing segments. As a result, we need to check what the highest level
// is, then for every level lower than the highest check if we are missing
// data. If yes, we need to explicitly set the gauges to 0.
func (s *segmentLevelStats) fillMissingLevels() {
maxLevel := uint16(0)
for level := range s.count {
if level > maxLevel {
maxLevel = level
}
}
if maxLevel > 0 {
for level := uint16(0); level < maxLevel; level++ {
if _, ok := s.count[level]; ok {
continue
}
// there is no entry for this level, we must explicitly set it to 0
s.count[level] = 0
s.indexes[level] = 0
s.payloads[level] = 0
}
}
}
func (s *segmentLevelStats) report(metrics *Metrics,
strategy, dir string,
) {
for level, size := range s.indexes {
metrics.SegmentSize.With(prometheus.Labels{
"strategy": strategy,
"unit": "index",
"level": fmt.Sprint(level),
"path": dir,
}).Set(float64(size))
}
for level, size := range s.payloads {
metrics.SegmentSize.With(prometheus.Labels{
"strategy": strategy,
"unit": "payload",
"level": fmt.Sprint(level),
"path": dir,
}).Set(float64(size))
}
for level, count := range s.count {
metrics.SegmentCount.With(prometheus.Labels{
"strategy": strategy,
"level": fmt.Sprint(level),
"path": dir,
}).Set(float64(count))
}
}
func (sg *SegmentGroup) compactionFitsSizeLimit(left, right *segment) bool {
if sg.maxSegmentSize == 0 {
// no limit is set, always return true
return true
}
totalSize := left.size + right.size
return totalSize <= sg.maxSegmentSize
}