-
Notifications
You must be signed in to change notification settings - Fork 693
Expand file tree
/
Copy pathcompactor.go
More file actions
297 lines (249 loc) · 8.73 KB
/
compactor.go
File metadata and controls
297 lines (249 loc) · 8.73 KB
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
package tempodb
import (
"bytes"
"context"
"fmt"
"io"
"math/rand"
"strconv"
"time"
"github.com/pkg/errors"
"github.com/go-kit/kit/log/level"
"github.com/google/uuid"
"github.com/grafana/tempo/tempodb/backend"
"github.com/grafana/tempo/tempodb/encoding"
"github.com/grafana/tempo/tempodb/wal"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
)
var (
metricCompactionDuration = promauto.NewHistogramVec(prometheus.HistogramOpts{
Namespace: "tempodb",
Name: "compaction_duration_seconds",
Help: "Records the amount of time to compact a set of blocks.",
Buckets: prometheus.ExponentialBuckets(30, 2, 10),
}, []string{"level"})
metricCompactionErrors = promauto.NewCounter(prometheus.CounterOpts{
Namespace: "tempodb",
Name: "compaction_errors_total",
Help: "Total number of errors occurring during compaction.",
})
metricCompactionObjectsCombined = promauto.NewCounter(prometheus.CounterOpts{
Namespace: "tempodb",
Name: "compaction_objects_combined_total",
Help: "Total number of objects combined during compaction.",
})
)
const (
inputBlocks = 2
outputBlocks = 1
compactionCycle = 30 * time.Second
)
// todo: pass a context/chan in to cancel this cleanly
func (rw *readerWriter) compactionLoop() {
ticker := time.NewTicker(compactionCycle)
for range ticker.C {
rw.doCompaction()
}
}
func (rw *readerWriter) doCompaction() {
tenants := rw.blocklistTenants()
if len(tenants) == 0 {
return
}
// pick a random tenant and find some blocks to compact
rand.Seed(time.Now().Unix())
tenantID := tenants[rand.Intn(len(tenants))].(string)
blocklist := rw.blocklist(tenantID)
blockSelector := newTimeWindowBlockSelector(blocklist, rw.compactorCfg.MaxCompactionRange, rw.compactorCfg.MaxCompactionObjects, defaultMinInputBlocks, defaultMaxInputBlocks)
start := time.Now()
level.Info(rw.logger).Log("msg", "starting compaction cycle", "tenantID", tenantID)
for {
toBeCompacted, hashString := blockSelector.BlocksToCompact()
if len(toBeCompacted) == 0 {
level.Info(rw.logger).Log("msg", "compaction cycle complete. No more blocks to compact", "tenantID", tenantID)
break
}
if !rw.compactorSharder.Owns(hashString) {
// continue on this tenant until we find something we own
continue
}
level.Info(rw.logger).Log("msg", "Compacting hash", "hashString", hashString)
err := rw.compact(toBeCompacted, tenantID)
if err == backend.ErrMetaDoesNotExist {
level.Warn(rw.logger).Log("msg", "unable to find meta during compaction. trying again on this block list", "err", err)
} else if err != nil {
level.Error(rw.logger).Log("msg", "error during compaction cycle", "err", err)
metricCompactionErrors.Inc()
}
// after a maintenance cycle bail out
if start.Add(rw.cfg.BlocklistPoll).Before(time.Now()) {
level.Info(rw.logger).Log("msg", "compacted blocks for a maintenance cycle, bailing out", "tenantID", tenantID)
break
}
}
}
// todo : this method is brittle and has weird failure conditions. if it fails after it has written a new block then it will not clean up the old
// in these cases it's possible that the compact method actually will start making more blocks.
func (rw *readerWriter) compact(blockMetas []*encoding.BlockMeta, tenantID string) error {
level.Debug(rw.logger).Log("msg", "beginning compaction", "num blocks compacting", len(blockMetas))
if len(blockMetas) == 0 {
return nil
}
compactionLevel := compactionLevelForBlocks(blockMetas)
nextCompactionLevel := compactionLevel + 1
start := time.Now()
defer func() {
level.Info(rw.logger).Log("msg", "compaction complete")
metricCompactionDuration.WithLabelValues(strconv.Itoa(int(compactionLevel))).Observe(time.Since(start).Seconds())
}()
var err error
bookmarks := make([]*bookmark, 0, len(blockMetas))
var totalRecords int
for _, blockMeta := range blockMetas {
level.Info(rw.logger).Log("msg", "compacting block", "block", fmt.Sprintf("%+v", blockMeta))
totalRecords += blockMeta.TotalObjects
iter, err := encoding.NewBackendIterator(tenantID, blockMeta.BlockID, rw.compactorCfg.ChunkSizeBytes, rw.r)
if err != nil {
return err
}
bookmarks = append(bookmarks, newBookmark(iter))
_, err = rw.r.BlockMeta(context.TODO(), blockMeta.BlockID, tenantID)
if err != nil {
return err
}
}
recordsPerBlock := (totalRecords / outputBlocks)
var newCompactedBlocks []*encoding.BlockMeta
var currentBlock *wal.CompactorBlock
var tracker backend.AppendTracker
for !allDone(bookmarks) {
var lowestID []byte
var lowestObject []byte
var lowestBookmark *bookmark
// find lowest ID of the new object
for _, b := range bookmarks {
currentID, currentObject, err := b.current()
if err == io.EOF {
continue
} else if err != nil {
return err
}
if bytes.Equal(currentID, lowestID) {
lowestObject = rw.compactorSharder.Combine(currentObject, lowestObject)
b.clear()
metricCompactionObjectsCombined.Inc()
} else if len(lowestID) == 0 || bytes.Compare(currentID, lowestID) == -1 {
lowestID = currentID
lowestObject = currentObject
lowestBookmark = b
}
}
if len(lowestID) == 0 || len(lowestObject) == 0 || lowestBookmark == nil {
return fmt.Errorf("failed to find a lowest object in compaction")
}
// make a new block if necessary
if currentBlock == nil {
currentBlock, err = rw.wal.NewCompactorBlock(uuid.New(), tenantID, blockMetas, recordsPerBlock)
if err != nil {
return errors.Wrap(err, "error making new compacted block")
}
currentBlock.BlockMeta().CompactionLevel = nextCompactionLevel
newCompactedBlocks = append(newCompactedBlocks, currentBlock.BlockMeta())
}
// writing to the current block will cause the id to escape the iterator so we need to make a copy of it
writeID := append([]byte(nil), lowestID...)
err = currentBlock.Write(writeID, lowestObject)
if err != nil {
return err
}
lowestBookmark.clear()
// write partial block
if currentBlock.CurrentBufferLength() >= int(rw.compactorCfg.FlushSizeBytes) {
tracker, err = appendBlock(rw, tracker, currentBlock)
if err != nil {
return errors.Wrap(err, "error writing partial block")
}
}
// ship block to backend if done
if currentBlock.Length() >= recordsPerBlock {
err = finishBlock(rw, tracker, currentBlock)
if err != nil {
return errors.Wrap(err, "error shipping block to backend")
}
currentBlock = nil
tracker = nil
}
}
// ship final block to backend
if currentBlock != nil {
err = finishBlock(rw, tracker, currentBlock)
if err != nil {
return errors.Wrap(err, "error shipping block to backend")
}
}
// mark old blocks compacted so they don't show up in polling
markCompacted(rw, tenantID, blockMetas, newCompactedBlocks)
return nil
}
func appendBlock(rw *readerWriter, tracker backend.AppendTracker, block *wal.CompactorBlock) (backend.AppendTracker, error) {
tracker, err := rw.w.AppendObject(context.TODO(), tracker, block.BlockMeta(), block.CurrentBuffer())
if err != nil {
return nil, err
}
block.ResetBuffer()
return tracker, nil
}
func finishBlock(rw *readerWriter, tracker backend.AppendTracker, block *wal.CompactorBlock) error {
level.Info(rw.logger).Log("msg", "writing compacted block", "block", fmt.Sprintf("%+v", block.BlockMeta()))
tracker, err := appendBlock(rw, tracker, block)
if err != nil {
return err
}
block.Complete()
err = rw.WriteBlockMeta(context.TODO(), tracker, block) // todo: add timeout
if err != nil {
return err
}
err = block.Clear()
if err != nil {
level.Error(rw.logger).Log("msg", "error cleaning up currentBlock in compaction", "err", err)
}
return nil
}
func allDone(bookmarks []*bookmark) bool {
for _, b := range bookmarks {
if !b.done() {
return false
}
}
return true
}
func compactionLevelForBlocks(blockMetas []*encoding.BlockMeta) uint8 {
level := uint8(0)
for _, m := range blockMetas {
if m.CompactionLevel > level {
level = m.CompactionLevel
}
}
return level
}
func markCompacted(rw *readerWriter, tenantID string, oldBlocks []*encoding.BlockMeta, newBlocks []*encoding.BlockMeta) {
for _, meta := range oldBlocks {
// Mark in the backend
if err := rw.c.MarkBlockCompacted(meta.BlockID, tenantID); err != nil {
level.Error(rw.logger).Log("msg", "unable to mark block compacted", "blockID", meta.BlockID, "tenantID", tenantID, "err", err)
metricCompactionErrors.Inc()
}
}
// Converted outgoing blocks into compacted entries.
newCompactions := make([]*encoding.CompactedBlockMeta, 0, len(oldBlocks))
for _, newBlock := range oldBlocks {
newCompactions = append(newCompactions, &encoding.CompactedBlockMeta{
BlockMeta: *newBlock,
CompactedTime: time.Now(),
})
}
// Update blocklist in memory
rw.updateBlocklist(tenantID, newBlocks, oldBlocks, newCompactions)
}