/
segment.go
278 lines (237 loc) · 7.56 KB
/
segment.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
// _ _
// __ _____ __ ___ ___ __ _| |_ ___
// \ \ /\ / / _ \/ _` \ \ / / |/ _` | __/ _ \
// \ V V / __/ (_| |\ V /| | (_| | || __/
// \_/\_/ \___|\__,_| \_/ |_|\__,_|\__\___|
//
// Copyright © 2016 - 2024 Weaviate B.V. All rights reserved.
//
// CONTACT: hello@weaviate.io
//
package lsmkv
import (
"bufio"
"bytes"
"fmt"
"io"
"os"
"github.com/edsrzf/mmap-go"
"github.com/sirupsen/logrus"
"github.com/weaviate/weaviate/adapters/repos/db/lsmkv/segmentindex"
"github.com/weaviate/weaviate/entities/lsmkv"
"github.com/willf/bloom"
)
type segment struct {
path string
level uint16
secondaryIndexCount uint16
version uint16
segmentStartPos uint64
segmentEndPos uint64
dataStartPos uint64
dataEndPos uint64
contents []byte
contentFile *os.File
strategy segmentindex.Strategy
index diskIndex
secondaryIndices []diskIndex
logger logrus.FieldLogger
metrics *Metrics
size int64
mmapContents bool
useBloomFilter bool // see bucket for more datails
bloomFilter *bloom.BloomFilter
secondaryBloomFilters []*bloom.BloomFilter
bloomFilterMetrics *bloomFilterMetrics
// the net addition this segment adds with respect to all previous segments
calcCountNetAdditions bool // see bucket for more datails
countNetAdditions int
}
type diskIndex interface {
// Get return lsmkv.NotFound in case no node can be found
Get(key []byte) (segmentindex.Node, error)
// Seek returns lsmkv.NotFound in case the seek value is larger than
// the highest value in the collection, otherwise it returns the next highest
// value (or the exact value if present)
Seek(key []byte) (segmentindex.Node, error)
// AllKeys in no specific order, e.g. for building a bloom filter
AllKeys() ([][]byte, error)
// Size of the index in bytes
Size() int
}
func newSegment(path string, logger logrus.FieldLogger, metrics *Metrics,
existsLower existsOnLowerSegmentsFn, mmapContents bool,
useBloomFilter bool, calcCountNetAdditions bool, overwriteDerived bool,
) (*segment, error) {
file, err := os.Open(path)
if err != nil {
return nil, fmt.Errorf("open file: %w", err)
}
fileInfo, err := file.Stat()
if err != nil {
return nil, fmt.Errorf("stat file: %w", err)
}
contents, err := mmap.MapRegion(file, int(fileInfo.Size()), mmap.RDONLY, 0, 0)
if err != nil {
return nil, fmt.Errorf("mmap file: %w", err)
}
header, err := segmentindex.ParseHeader(bytes.NewReader(contents[:segmentindex.HeaderSize]))
if err != nil {
return nil, fmt.Errorf("parse header: %w", err)
}
switch header.Strategy {
case segmentindex.StrategyReplace, segmentindex.StrategySetCollection,
segmentindex.StrategyMapCollection, segmentindex.StrategyRoaringSet:
default:
return nil, fmt.Errorf("unsupported strategy in segment")
}
primaryIndex, err := header.PrimaryIndex(contents)
if err != nil {
return nil, fmt.Errorf("extract primary index position: %w", err)
}
primaryDiskIndex := segmentindex.NewDiskTree(primaryIndex)
seg := &segment{
level: header.Level,
path: path,
contents: contents,
version: header.Version,
secondaryIndexCount: header.SecondaryIndices,
segmentStartPos: header.IndexStart,
segmentEndPos: uint64(fileInfo.Size()),
strategy: header.Strategy,
dataStartPos: segmentindex.HeaderSize, // fixed value that's the same for all strategies
dataEndPos: header.IndexStart,
index: primaryDiskIndex,
logger: logger,
metrics: metrics,
size: fileInfo.Size(),
mmapContents: mmapContents,
useBloomFilter: useBloomFilter,
calcCountNetAdditions: calcCountNetAdditions,
}
// Using pread strategy requires file to remain open for segment lifetime
if seg.mmapContents {
defer file.Close()
} else {
seg.contentFile = file
}
if seg.secondaryIndexCount > 0 {
seg.secondaryIndices = make([]diskIndex, seg.secondaryIndexCount)
for i := range seg.secondaryIndices {
secondary, err := header.SecondaryIndex(contents, uint16(i))
if err != nil {
return nil, fmt.Errorf("get position for secondary index at %d: %w", i, err)
}
seg.secondaryIndices[i] = segmentindex.NewDiskTree(secondary)
}
}
if seg.useBloomFilter {
if err := seg.initBloomFilters(metrics, overwriteDerived); err != nil {
return nil, err
}
}
if seg.calcCountNetAdditions {
if err := seg.initCountNetAdditions(existsLower, overwriteDerived); err != nil {
return nil, err
}
}
return seg, nil
}
func (s *segment) close() error {
var munmapErr, fileCloseErr error
m := mmap.MMap(s.contents)
munmapErr = m.Unmap()
if s.contentFile != nil {
fileCloseErr = s.contentFile.Close()
}
if munmapErr != nil || fileCloseErr != nil {
return fmt.Errorf("close segment: munmap: %v, close contents file: %w", munmapErr, fileCloseErr)
}
return nil
}
func (s *segment) drop() error {
// support for persisting bloom filters and cnas was added in v1.17,
// therefore the files may not be present on segments created with previous
// versions. By using RemoveAll, which does not error on NotExists, these
// drop calls are backward-compatible:
if err := os.RemoveAll(s.bloomFilterPath()); err != nil {
return fmt.Errorf("drop bloom filter: %w", err)
}
for i := 0; i < int(s.secondaryIndexCount); i++ {
if err := os.RemoveAll(s.bloomFilterSecondaryPath(i)); err != nil {
return fmt.Errorf("drop bloom filter: %w", err)
}
}
if err := os.RemoveAll(s.countNetPath()); err != nil {
return fmt.Errorf("drop count net additions file: %w", err)
}
// for the segment itself, we're not using RemoveAll, but Remove. If there
// was a NotExists error here, something would be seriously wrong, and we
// don't want to ignore it.
if err := os.Remove(s.path); err != nil {
return fmt.Errorf("drop segment: %w", err)
}
return nil
}
// Size returns the total size of the segment in bytes, including the header
// and index
func (s *segment) Size() int {
return int(s.size)
}
// PayloadSize is only the payload of the index, excluding the index
func (s *segment) PayloadSize() int {
return int(s.dataEndPos)
}
type nodeReader struct {
r io.Reader
}
func (n *nodeReader) Read(b []byte) (int, error) {
return n.r.Read(b)
}
type nodeOffset struct {
start, end uint64
}
func (s *segment) newNodeReader(offset nodeOffset) (*nodeReader, error) {
var (
r io.Reader
err error
)
if s.mmapContents {
contents := s.contents[offset.start:]
if offset.end != 0 {
contents = s.contents[offset.start:offset.end]
}
r, err = s.bytesReaderFrom(contents)
} else {
r, err = s.bufferedReaderAt(offset.start)
}
if err != nil {
return nil, fmt.Errorf("new nodeReader: %w", err)
}
return &nodeReader{r: r}, nil
}
func (s *segment) copyNode(b []byte, offset nodeOffset) error {
if s.mmapContents {
copy(b, s.contents[offset.start:offset.end])
return nil
}
n, err := s.newNodeReader(offset)
if err != nil {
return fmt.Errorf("copy node: %w", err)
}
_, err = io.ReadFull(n, b)
return err
}
func (s *segment) bytesReaderFrom(in []byte) (*bytes.Reader, error) {
if len(in) == 0 {
return nil, lsmkv.NotFound
}
return bytes.NewReader(in), nil
}
func (s *segment) bufferedReaderAt(offset uint64) (*bufio.Reader, error) {
if s.contentFile == nil {
return nil, fmt.Errorf("nil contentFile for segment at %s", s.path)
}
r := io.NewSectionReader(s.contentFile, int64(offset), s.size)
return bufio.NewReader(r), nil
}