-
Notifications
You must be signed in to change notification settings - Fork 0
/
table.go
339 lines (271 loc) · 7.79 KB
/
table.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
package compactor
import (
"context"
"errors"
"fmt"
"os"
"path/filepath"
"time"
"github.com/cortexproject/cortex/pkg/chunk"
chunk_util "github.com/cortexproject/cortex/pkg/chunk/util"
util_log "github.com/cortexproject/cortex/pkg/util/log"
util_math "github.com/cortexproject/cortex/pkg/util/math"
"github.com/go-kit/kit/log/level"
"go.etcd.io/bbolt"
shipper_util "github.com/metrico/loki-apache/pkg/storage/stores/shipper/util"
)
const (
compactMinDBs = 4
uploaderName = "compactor"
readDBsParallelism = 50
batchSize = 1000
)
var bucketName = []byte("index")
type indexEntry struct {
k, v []byte
}
type table struct {
name string
workingDirectory string
storageClient chunk.ObjectClient
compactedDB *bbolt.DB
ctx context.Context
quit chan struct{}
}
func newTable(ctx context.Context, workingDirectory string, objectClient chunk.ObjectClient) (*table, error) {
err := chunk_util.EnsureDirectory(workingDirectory)
if err != nil {
return nil, err
}
table := table{
ctx: ctx,
name: filepath.Base(workingDirectory),
workingDirectory: workingDirectory,
storageClient: objectClient,
quit: make(chan struct{}),
}
return &table, nil
}
func (t *table) compact() error {
// The forward slash here needs to stay because we are trying to list contents of a directory without it we will get the name of the same directory back with hosted object stores.
// This is due to the object stores not having a concept of directories.
objects, _, err := t.storageClient.List(t.ctx, t.name+delimiter, delimiter)
if err != nil {
return err
}
level.Info(util_log.Logger).Log("msg", "listed files", "count", len(objects))
if len(objects) < compactMinDBs {
level.Info(util_log.Logger).Log("msg", fmt.Sprintf("skipping compaction since we have just %d files in storage", len(objects)))
return nil
}
defer func() {
err := t.cleanup()
if err != nil {
level.Error(util_log.Logger).Log("msg", "failed to cleanup table", "name", t.name)
}
}()
t.compactedDB, err = shipper_util.SafeOpenBoltdbFile(filepath.Join(t.workingDirectory, fmt.Sprint(time.Now().Unix())))
if err != nil {
return err
}
level.Info(util_log.Logger).Log("msg", "starting compaction of dbs")
errChan := make(chan error)
readObjectChan := make(chan string)
n := util_math.Min(len(objects), readDBsParallelism)
// read files parallelly
for i := 0; i < n; i++ {
go func() {
var err error
defer func() {
errChan <- err
}()
for {
select {
case objectKey, ok := <-readObjectChan:
if !ok {
return
}
// The s3 client can also return the directory itself in the ListObjects.
if shipper_util.IsDirectory(objectKey) {
continue
}
var dbName string
dbName, err = shipper_util.GetDBNameFromObjectKey(objectKey)
if err != nil {
return
}
downloadAt := filepath.Join(t.workingDirectory, dbName)
err = shipper_util.GetFileFromStorage(t.ctx, t.storageClient, objectKey, downloadAt)
if err != nil {
return
}
err = t.readFile(downloadAt)
if err != nil {
level.Error(util_log.Logger).Log("msg", "error reading file", "err", err)
return
}
case <-t.quit:
return
case <-t.ctx.Done():
return
}
}
}()
}
// send all files to readObjectChan
go func() {
for _, object := range objects {
select {
case readObjectChan <- object.Key:
case <-t.quit:
break
case <-t.ctx.Done():
break
}
}
level.Debug(util_log.Logger).Log("msg", "closing readObjectChan")
close(readObjectChan)
}()
var firstErr error
// read all the errors
for i := 0; i < n; i++ {
err := <-errChan
if err != nil && firstErr == nil {
firstErr = err
close(t.quit)
}
}
if firstErr != nil {
return firstErr
}
// check whether we stopped compaction due to context being cancelled.
select {
case <-t.ctx.Done():
return nil
default:
}
level.Info(util_log.Logger).Log("msg", "finished compacting the dbs")
// upload the compacted db
err = t.upload()
if err != nil {
return err
}
// remove source files from storage which were compacted
return t.removeObjectsFromStorage(objects)
}
func (t *table) cleanup() error {
if t.compactedDB != nil {
err := t.compactedDB.Close()
if err != nil {
return err
}
}
return os.RemoveAll(t.workingDirectory)
}
// writeBatch writes a batch to compactedDB
func (t *table) writeBatch(batch []indexEntry) error {
return t.compactedDB.Batch(func(tx *bbolt.Tx) error {
b, err := tx.CreateBucketIfNotExists(bucketName)
if err != nil {
return err
}
for _, w := range batch {
err = b.Put(w.k, w.v)
if err != nil {
return err
}
}
return nil
})
}
// readFile reads a boltdb file from a path and writes the index in batched mode to compactedDB
func (t *table) readFile(path string) error {
level.Debug(util_log.Logger).Log("msg", "reading file for compaction", "path", path)
db, err := shipper_util.SafeOpenBoltdbFile(path)
if err != nil {
return err
}
defer func() {
if err := db.Close(); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to close db", "path", path, "err", err)
}
if err = os.Remove(path); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to remove file", "path", path, "err", err)
}
}()
writeBatch := make([]indexEntry, 0, batchSize)
return db.View(func(tx *bbolt.Tx) error {
b := tx.Bucket(bucketName)
if b == nil {
return errors.New("bucket not found")
}
err := b.ForEach(func(k, v []byte) error {
ie := indexEntry{
k: make([]byte, len(k)),
v: make([]byte, len(v)),
}
// make a copy since k, v are only valid for the life of the transaction.
// See: https://godoc.org/github.com/boltdb/bolt#Cursor.Seek
copy(ie.k, k)
copy(ie.v, v)
writeBatch = append(writeBatch, ie)
if len(writeBatch) == cap(writeBatch) {
// batch is full, write the batch and create a new one.
err := t.writeBatch(writeBatch)
if err != nil {
return err
}
writeBatch = make([]indexEntry, 0, batchSize)
}
return nil
})
if err != nil {
return err
}
// write the remaining batch which might have been left unwritten due to it not being full yet.
return t.writeBatch(writeBatch)
})
}
// upload uploads the compacted db in compressed format.
func (t *table) upload() error {
compactedDBPath := t.compactedDB.Path()
// close the compactedDB to make sure all the writes are processed.
err := t.compactedDB.Close()
if err != nil {
return err
}
t.compactedDB = nil
// compress the compactedDB.
compressedDBPath := fmt.Sprintf("%s.gz", compactedDBPath)
err = shipper_util.CompressFile(compactedDBPath, compressedDBPath)
if err != nil {
return err
}
// open the file for reading.
compressedDB, err := os.Open(compressedDBPath)
if err != nil {
return err
}
defer func() {
if err := compressedDB.Close(); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to close file", "path", compactedDBPath, "err", err)
}
if err := os.Remove(compressedDBPath); err != nil {
level.Error(util_log.Logger).Log("msg", "failed to remove file", "path", compressedDBPath, "err", err)
}
}()
objectKey := fmt.Sprintf("%s.gz", shipper_util.BuildObjectKey(t.name, uploaderName, fmt.Sprint(time.Now().Unix())))
level.Info(util_log.Logger).Log("msg", "uploading the compacted file", "objectKey", objectKey)
return t.storageClient.PutObject(t.ctx, objectKey, compressedDB)
}
// removeObjectsFromStorage deletes objects from storage.
func (t *table) removeObjectsFromStorage(objects []chunk.StorageObject) error {
level.Info(util_log.Logger).Log("msg", "removing source db files from storage", "count", len(objects))
for _, object := range objects {
err := t.storageClient.DeleteObject(t.ctx, object.Key)
if err != nil {
return err
}
}
return nil
}