-
Notifications
You must be signed in to change notification settings - Fork 21
Expand file tree
/
Copy pathconvert.go
More file actions
698 lines (599 loc) · 17.8 KB
/
convert.go
File metadata and controls
698 lines (599 loc) · 17.8 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
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
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
// Copyright (c) The Thanos Authors.
// Licensed under the Apache 2.0 license found in the LICENSE file or at:
// https://opensource.org/licenses/Apache-2.0
package convert
import (
"bufio"
"bytes"
"context"
"errors"
"fmt"
"io"
"time"
"golang.org/x/sync/errgroup"
"github.com/alecthomas/units"
"github.com/efficientgo/core/errcapture"
jsoniter "github.com/json-iterator/go"
"github.com/parquet-go/parquet-go"
"github.com/prometheus/prometheus/model/labels"
"github.com/prometheus/prometheus/tsdb"
"github.com/prometheus/prometheus/tsdb/tombstones"
"github.com/prometheus/prometheus/util/zeropool"
"github.com/thanos-io/objstore"
"github.com/thanos-io/thanos/pkg/block/metadata"
"google.golang.org/protobuf/proto"
"github.com/thanos-io/thanos-parquet-gateway/internal/encoding"
"github.com/thanos-io/thanos-parquet-gateway/internal/util"
"github.com/thanos-io/thanos-parquet-gateway/proto/metapb"
"github.com/thanos-io/thanos-parquet-gateway/schema"
)
const (
// ParquetMigratedExtensionKey is the key used in TSDB block metadata Extensions
// to indicate that the block has been migrated to Parquet format.
// TODO: Import this constant from Thanos when it becomes available in their metadata package.
ParquetMigratedExtensionKey = "parquet_migrated"
)
type Convertible interface {
Index() (tsdb.IndexReader, error)
Chunks() (tsdb.ChunkReader, error)
Tombstones() (tombstones.Reader, error)
Meta() tsdb.BlockMeta
}
type convertOpts struct {
numRowGroups int
rowGroupSize int
encodingConcurrency int
sortLabels []string
sortingColumns [][]string
bloomfilterColumns [][]string
labelBufferPool parquet.BufferPool
chunkbufferPool parquet.BufferPool
labelPageBufferSize int
chunkPageBufferSize int
updateTSDBMeta bool
tsdbBucket objstore.Bucket
tsdbMetas []metadata.Meta
}
func (cfg convertOpts) buildBloomfilterColumns() []parquet.BloomFilterColumn {
cols := make([]parquet.BloomFilterColumn, 0, len(cfg.bloomfilterColumns))
for i := range cfg.bloomfilterColumns {
cols = append(cols,
parquet.SplitBlockFilter(10, cfg.bloomfilterColumns[i]...))
}
return cols
}
func (cfg convertOpts) buildSortingColumns() []parquet.SortingColumn {
cols := make([]parquet.SortingColumn, 0, len(cfg.bloomfilterColumns))
for i := range cfg.sortingColumns {
cols = append(cols,
parquet.Ascending(cfg.sortingColumns[i]...))
}
return cols
}
type ConvertOption func(*convertOpts)
func RowGroupSize(rbs int) ConvertOption {
return func(opts *convertOpts) {
opts.rowGroupSize = rbs
}
}
func RowGroupCount(rc int) ConvertOption {
return func(opts *convertOpts) {
opts.numRowGroups = rc
}
}
func SortBy(labels ...string) ConvertOption {
return func(opts *convertOpts) {
sortingColumns := make([][]string, len(labels))
for i := range labels {
sortingColumns[i] = []string{schema.LabelNameToColumn(labels[i])}
}
opts.sortingColumns = sortingColumns
opts.sortLabels = labels
}
}
func LabelPageBufferSize(pb units.Base2Bytes) ConvertOption {
return func(opts *convertOpts) {
opts.labelPageBufferSize = int(pb)
}
}
func ChunkPageBufferSize(pb units.Base2Bytes) ConvertOption {
return func(opts *convertOpts) {
opts.chunkPageBufferSize = int(pb)
}
}
func LabelBufferPool(p parquet.BufferPool) ConvertOption {
return func(opts *convertOpts) {
opts.labelBufferPool = p
}
}
func ChunkBufferPool(p parquet.BufferPool) ConvertOption {
return func(opts *convertOpts) {
opts.chunkbufferPool = p
}
}
func EncodingConcurrency(c int) ConvertOption {
return func(opts *convertOpts) {
opts.encodingConcurrency = c
}
}
// UpdateTSDBMeta enables updating the original TSDB block metadata after successful conversion
func UpdateTSDBMeta(tsdbBucket objstore.Bucket, tsdbMetas []metadata.Meta) ConvertOption {
return func(opts *convertOpts) {
opts.updateTSDBMeta = true
opts.tsdbBucket = tsdbBucket
opts.tsdbMetas = tsdbMetas
}
}
func ConvertTSDBBlock(
ctx context.Context,
bkt objstore.Bucket,
day time.Time,
blks []Convertible,
opts ...ConvertOption,
) (rerr error) {
cfg := &convertOpts{
rowGroupSize: 1_000_000,
numRowGroups: 6,
sortingColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}},
bloomfilterColumns: [][]string{{schema.LabelNameToColumn(labels.MetricName)}},
labelBufferPool: parquet.NewBufferPool(),
chunkbufferPool: parquet.NewBufferPool(),
encodingConcurrency: 1,
labelPageBufferSize: int(256 * units.KiB),
chunkPageBufferSize: int(2 * units.MiB),
}
for i := range opts {
opts[i](cfg)
}
start, end := util.BeginOfDay(day), util.EndOfDay(day)
name, err := schema.BlockNameForDay(start)
if err != nil {
return fmt.Errorf("unable to get block name: %w", err)
}
rr, err := newIndexRowReader(ctx, start.UnixMilli(), end.UnixMilli(), blks, indexReaderOpts{
sortLabels: cfg.sortLabels,
concurrency: cfg.encodingConcurrency,
})
if err != nil {
return fmt.Errorf("unable to create index rowreader: %w", err)
}
defer errcapture.Do(&rerr, rr.Close, "index row reader close")
converter := newConverter(
name,
start.UnixMilli(),
end.UnixMilli(),
rr,
bkt,
cfg.rowGroupSize,
cfg.numRowGroups,
cfg.buildSortingColumns(),
cfg.buildBloomfilterColumns(),
cfg.labelBufferPool,
cfg.chunkbufferPool,
cfg.labelPageBufferSize,
cfg.chunkPageBufferSize,
)
if err := converter.convert(ctx); err != nil {
return fmt.Errorf("unable to convert block: %w", err)
}
if cfg.updateTSDBMeta {
for _, meta := range cfg.tsdbMetas {
if err := markBlockAsMigrated(ctx, cfg.tsdbBucket, meta); err != nil {
return fmt.Errorf("unable to mark block %s as migrated: %w", meta.ULID, err)
}
}
}
lastSuccessfulConvertTime.SetToCurrentTime()
return nil
}
type converter struct {
name string
mint, maxt int64
currentShard int
seriesPerShard int
rowGroupSize int
numRowGroups int
bkt objstore.Bucket
rr *indexRowReader
sortingColumns []parquet.SortingColumn
bloomfilterColumns []parquet.BloomFilterColumn
labelBufferPool parquet.BufferPool
chunkBufferPool parquet.BufferPool
labelPageBufferSize int
chunkPageBufferSize int
}
func newConverter(
name string,
mint int64,
maxt int64,
rr *indexRowReader,
bkt objstore.Bucket,
rowGroupSize int,
numRowGroups int,
sortingColumns []parquet.SortingColumn,
bloomfilterColumns []parquet.BloomFilterColumn,
labelBufferPool parquet.BufferPool,
chunkBufferPool parquet.BufferPool,
labelPageBufferSize int,
chunkPageBufferSize int,
) *converter {
return &converter{
name: name,
mint: mint,
maxt: maxt,
bkt: bkt,
rr: rr,
rowGroupSize: rowGroupSize,
numRowGroups: numRowGroups,
sortingColumns: sortingColumns,
bloomfilterColumns: bloomfilterColumns,
labelBufferPool: labelBufferPool,
chunkBufferPool: chunkBufferPool,
labelPageBufferSize: labelPageBufferSize,
chunkPageBufferSize: chunkPageBufferSize,
}
}
func (c *converter) labelWriterOptions() []parquet.WriterOption {
return []parquet.WriterOption{
parquet.MaxRowsPerRowGroup(int64(c.rowGroupSize)),
parquet.SortingWriterConfig(parquet.SortingColumns(c.sortingColumns...)),
parquet.BloomFilters(c.bloomfilterColumns...),
parquet.SkipPageBounds(schema.LabelIndexColumn),
parquet.ColumnPageBuffers(c.labelBufferPool),
parquet.PageBufferSize(c.labelPageBufferSize),
}
}
func (c *converter) chunkWriterOptions() []parquet.WriterOption {
return []parquet.WriterOption{
parquet.MaxRowsPerRowGroup(int64(c.rowGroupSize)),
parquet.SkipPageBounds(schema.LabelHashColumn),
parquet.SkipPageBounds(schema.ChunksColumn0),
parquet.SkipPageBounds(schema.ChunksColumn1),
parquet.SkipPageBounds(schema.ChunksColumn2),
parquet.ColumnPageBuffers(c.chunkBufferPool),
parquet.PageBufferSize(c.chunkPageBufferSize),
}
}
func (c *converter) convert(ctx context.Context) error {
if err := c.convertShards(ctx); err != nil {
return fmt.Errorf("unable to convert shards: %w", err)
}
if err := c.optimizeShards(ctx); err != nil {
return fmt.Errorf("unable to optimize shards: %w", err)
}
if err := c.writeMetaFile(ctx); err != nil {
return fmt.Errorf("unable to write meta file: %w", err)
}
return nil
}
func (c *converter) writeMetaFile(ctx context.Context) error {
meta := &metapb.Metadata{
Version: schema.V2,
Mint: c.mint,
Maxt: c.maxt,
Shards: int64(c.currentShard) + 1,
}
metaBytes, err := proto.Marshal(meta)
if err != nil {
return fmt.Errorf("unable to marshal meta bytes: %w", err)
}
if err := c.bkt.Upload(ctx, schema.MetaFileNameForBlock(c.name), bytes.NewReader(metaBytes)); err != nil {
return fmt.Errorf("unable to upload meta file: %w", err)
}
return nil
}
func (c *converter) convertShards(ctx context.Context) error {
for {
if ok, err := c.convertShard(ctx); err != nil {
return fmt.Errorf("unable to convert shard: %w", err)
} else if !ok {
break
}
}
return nil
}
func (c *converter) optimizeShards(ctx context.Context) error {
g, ctx := errgroup.WithContext(ctx)
for i := range c.currentShard + 1 {
g.Go(func() error {
return c.optimizeShard(ctx, i)
})
}
return g.Wait()
}
// Since we have to compute the schema from the whole TSDB Block it is highly likely that the
// labels parquet file we wrote just now contains many empty columns - we project them away again
func (c *converter) optimizeShard(ctx context.Context, i int) (rerr error) {
rc, err := c.bkt.Get(ctx, schema.LabelsPfileNameForShard(c.name, i))
if err != nil {
return fmt.Errorf("unable to fetch labels parquet file: %w", err)
}
defer errcapture.Do(&rerr, rc.Close, "labels parquet file close")
rbuf := bytes.NewBuffer(nil)
if _, err := io.Copy(rbuf, rc); err != nil {
return fmt.Errorf("unable to copy labels parquet file: %w", err)
}
pf, err := parquet.OpenFile(bytes.NewReader(rbuf.Bytes()), int64(rbuf.Len()))
if err != nil {
return fmt.Errorf("unable to open labels parquet file: %w", err)
}
s := pf.Schema()
ns := schema.WithCompression(schema.RemoveNullColumns(pf))
buf := bytes.NewBuffer(nil)
w := parquet.NewGenericWriter[any](buf, append(c.labelWriterOptions(), ns)...)
rb := parquet.NewRowBuilder(ns)
rowBuf := make([]parquet.Row, 128)
colIdxSlice := make([]int, 0)
labelIndexColumn := columnIDForKnownColumn(ns, schema.LabelIndexColumn)
for _, rg := range pf.RowGroups() {
rows := rg.Rows()
defer errcapture.Do(&rerr, rows.Close, "labels parquet file row group close")
for {
n, err := rows.ReadRows(rowBuf)
if err != nil && !errors.Is(err, io.EOF) {
return fmt.Errorf("unable to read rows: %w", err)
}
for i, row := range rowBuf[:n] {
rb.Reset()
colIdxSlice = colIdxSlice[:0]
for j, v := range row {
if !v.IsNull() {
if lc, ok := ns.Lookup(s.Columns()[j]...); ok && lc.ColumnIndex != labelIndexColumn {
colIdxSlice = append(colIdxSlice, lc.ColumnIndex)
rb.Add(lc.ColumnIndex, v)
}
}
}
rb.Add(labelIndexColumn, parquet.ValueOf(encoding.EncodeLabelColumnIndex(colIdxSlice)))
rowBuf[i] = rb.AppendRow(rowBuf[i][:0])
}
if m, err := w.WriteRows(rowBuf[:n]); err != nil {
return fmt.Errorf("unable to write transformed rows: %w", err)
} else if m != n {
return fmt.Errorf("unable to write rows: %d != %d", n, m)
}
if errors.Is(err, io.EOF) {
break
}
}
}
if err := w.Close(); err != nil {
return fmt.Errorf("unable to close writer: %w", err)
}
if err := c.bkt.Upload(ctx, schema.LabelsPfileNameForShard(c.name, i), buf); err != nil {
return fmt.Errorf("unable to override optimized labels parquet file: %w", err)
}
return nil
}
func (c *converter) convertShard(ctx context.Context) (_ bool, rerr error) {
s := c.rr.Schema()
rowsToWrite := c.numRowGroups * c.rowGroupSize
w, err := newSplitFileWriter(ctx, c.bkt, s, map[string]writerConfig{
schema.LabelsPfileNameForShard(c.name, c.currentShard): {
s: schema.WithCompression(schema.LabelsProjection(s)),
opts: c.labelWriterOptions(),
},
schema.ChunksPfileNameForShard(c.name, c.currentShard): {
s: schema.WithCompression(schema.ChunkProjection(s)),
opts: c.chunkWriterOptions(),
},
},
)
if err != nil {
return false, fmt.Errorf("unable to build multifile writer: %w", err)
}
defer errcapture.Do(&rerr, w.Close, "multifile writer close")
n, err := parquet.CopyRows(w, newBufferedReader(ctx, newLimitReader(c.rr, rowsToWrite)))
if err != nil {
return false, fmt.Errorf("unable to copy rows to writer: %w", err)
}
if n < int64(rowsToWrite) {
return false, nil
}
c.currentShard++
return true, nil
}
type limitReader struct {
parquet.RowReader
limit int
cur int
}
func newLimitReader(r parquet.RowReader, limit int) parquet.RowReader {
return &limitReader{RowReader: r, limit: limit}
}
func (lr *limitReader) ReadRows(buf []parquet.Row) (int, error) {
n, err := lr.RowReader.ReadRows(buf)
if err != nil {
return n, err
}
lr.cur += n
if lr.cur > lr.limit {
return n, io.EOF
}
return n, nil
}
type fileWriter struct {
pw *parquet.GenericWriter[any]
conv parquet.Conversion
w io.WriteCloser
r io.ReadCloser
}
type splitPipeFileWriter struct {
fileWriters map[string]*fileWriter
g *errgroup.Group
}
type writerConfig struct {
s *parquet.Schema
opts []parquet.WriterOption
}
func newSplitFileWriter(ctx context.Context, bkt objstore.Bucket, inSchema *parquet.Schema, files map[string]writerConfig) (*splitPipeFileWriter, error) {
fileWriters := make(map[string]*fileWriter)
g, ctx := errgroup.WithContext(ctx)
for file, cfg := range files {
conv, err := parquet.Convert(cfg.s, inSchema)
if err != nil {
return nil, fmt.Errorf("unable to convert schemas")
}
r, w := io.Pipe()
bw := bufio.NewWriterSize(w, 32_000_000)
br := bufio.NewReaderSize(r, 32_000_000)
fileWriters[file] = &fileWriter{
pw: parquet.NewGenericWriter[any](bw, append(cfg.opts, cfg.s)...),
w: w,
conv: conv,
}
g.Go(func() (rerr error) {
defer errcapture.Do(&rerr, r.Close, "buffered writer flush")
return bkt.Upload(ctx, file, br)
})
}
return &splitPipeFileWriter{
fileWriters: fileWriters,
g: g,
}, nil
}
func (s *splitPipeFileWriter) WriteRows(rows []parquet.Row) (int, error) {
var g errgroup.Group
for _, writer := range s.fileWriters {
g.Go(func() error {
rr := make([]parquet.Row, len(rows))
for i, row := range rows {
rr[i] = row.Clone()
}
_, err := writer.conv.Convert(rr)
if err != nil {
return fmt.Errorf("unable to convert rows: %w", err)
}
n, err := writer.pw.WriteRows(rr)
if err != nil {
return fmt.Errorf("unable to write rows: %w", err)
}
if n != len(rows) {
return fmt.Errorf("unable to write rows: %d != %d", n, len(rows))
}
return nil
})
}
return len(rows), g.Wait()
}
func (s *splitPipeFileWriter) Close() error {
errs := make([]error, 0)
for _, fw := range s.fileWriters {
if err := fw.pw.Close(); err != nil {
errs = append(errs, fmt.Errorf("unable to close pipewriter: %w", err))
}
if err := fw.w.Close(); err != nil {
errs = append(errs, fmt.Errorf("unable to close writer: %w", err))
}
}
if err := s.g.Wait(); err != nil {
errs = append(errs, fmt.Errorf("unable to wait for group: %w", err))
}
return errors.Join(errs...)
}
type bufferedReader struct {
rr parquet.RowReader
ctx context.Context
c chan []parquet.Row
errCh chan error
rowPool zeropool.Pool[[]parquet.Row]
current []parquet.Row
currentIndex int
}
func newBufferedReader(ctx context.Context, rr parquet.RowReader) *bufferedReader {
br := &bufferedReader{
rr: rr,
ctx: ctx,
c: make(chan []parquet.Row, 128),
errCh: make(chan error, 1),
rowPool: zeropool.New(func() []parquet.Row {
return make([]parquet.Row, 128)
}),
}
go br.readRows()
return br
}
func (b *bufferedReader) ReadRows(rows []parquet.Row) (int, error) {
if b.current == nil {
select {
case next, ok := <-b.c:
if !ok {
return 0, io.EOF
}
b.current = next
b.currentIndex = 0
case err := <-b.errCh:
return 0, err
}
}
current := b.current[b.currentIndex:]
i := min(len(current), len(rows))
copy(rows[:i], current[:i])
b.currentIndex += i
if b.currentIndex >= len(b.current) {
b.rowPool.Put(b.current[0:cap(b.current)])
b.current = nil
}
return i, nil
}
func (b *bufferedReader) Close() {
close(b.c)
close(b.errCh)
}
func (b *bufferedReader) readRows() {
for {
select {
case <-b.ctx.Done():
b.errCh <- b.ctx.Err()
return
default:
rows := b.rowPool.Get()
n, err := b.rr.ReadRows(rows)
if n > 0 {
b.c <- rows[:n]
}
if err != nil {
if err == io.EOF {
close(b.c)
return
}
b.errCh <- err
return
}
}
}
}
// markBlockAsMigrated updates a single TSDB block's metadata to indicate it has been migrated to Parquet
func markBlockAsMigrated(ctx context.Context, bkt objstore.Bucket, meta metadata.Meta) (rerr error) {
metaPath := meta.ULID.String() + "/meta.json"
rc, err := bkt.Get(ctx, metaPath)
if err != nil {
return fmt.Errorf("unable to get meta.json: %w", err)
}
defer errcapture.ExhaustClose(&rerr, rc, "meta.json close")
var existingMeta metadata.Meta
if err := jsoniter.ConfigCompatibleWithStandardLibrary.NewDecoder(rc).Decode(&existingMeta); err != nil {
return fmt.Errorf("unable to decode meta.json: %w", err)
}
if existingMeta.Thanos.Extensions == nil {
existingMeta.Thanos.Extensions = make(map[string]any)
}
var extensionsMap map[string]any
if ext, ok := existingMeta.Thanos.Extensions.(map[string]any); ok {
extensionsMap = ext
} else {
extensionsMap = make(map[string]any)
existingMeta.Thanos.Extensions = extensionsMap
}
extensionsMap[ParquetMigratedExtensionKey] = true
var buf bytes.Buffer
encoder := jsoniter.ConfigCompatibleWithStandardLibrary.NewEncoder(&buf)
encoder.SetIndent("", "\t")
if err := encoder.Encode(&existingMeta); err != nil {
return fmt.Errorf("unable to encode meta.json: %w", err)
}
if err := bkt.Upload(ctx, metaPath, &buf); err != nil {
return fmt.Errorf("unable to upload meta.json: %w", err)
}
return nil
}