forked from segmentio/parquet-go
/
writer.go
1711 lines (1509 loc) · 51.6 KB
/
writer.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
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
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
package parquet
import (
"bufio"
"bytes"
"encoding/binary"
"fmt"
"hash/crc32"
"io"
"math"
"math/bits"
"os"
"reflect"
"sort"
"github.com/parquet-go/parquet-go/compress"
"github.com/parquet-go/parquet-go/encoding"
"github.com/parquet-go/parquet-go/encoding/plain"
"github.com/parquet-go/parquet-go/format"
"github.com/segmentio/encoding/thrift"
)
const (
// The uncompressed page size is stored as int32 and must not be larger than the
// maximum int32 value (see format.PageHeader).
maxUncompressedPageSize = math.MaxInt32
)
// GenericWriter is similar to a Writer but uses a type parameter to define the
// Go type representing the schema of rows being written.
//
// Using this type over Writer has multiple advantages:
//
// - By leveraging type information, the Go compiler can provide greater
// guarantees that the code is correct. For example, the parquet.Writer.Write
// method accepts an argument of type interface{}, which delays type checking
// until runtime. The parquet.GenericWriter[T].Write method ensures at
// compile time that the values it receives will be of type T, reducing the
// risk of introducing errors.
//
// - Since type information is known at compile time, the implementation of
// parquet.GenericWriter[T] can make safe assumptions, removing the need for
// runtime validation of how the parameters are passed to its methods.
// Optimizations relying on type information are more effective, some of the
// writer's state can be precomputed at initialization, which was not possible
// with parquet.Writer.
//
// - The parquet.GenericWriter[T].Write method uses a data-oriented design,
// accepting an slice of T instead of a single value, creating more
// opportunities to amortize the runtime cost of abstractions.
// This optimization is not available for parquet.Writer because its Write
// method's argument would be of type []interface{}, which would require
// conversions back and forth from concrete types to empty interfaces (since
// a []T cannot be interpreted as []interface{} in Go), would make the API
// more difficult to use and waste compute resources in the type conversions,
// defeating the purpose of the optimization in the first place.
//
// Note that this type is only available when compiling with Go 1.18 or later.
type GenericWriter[T any] struct {
// At this time GenericWriter is expressed in terms of Writer to reuse the
// underlying logic. In the future, and if we accepted to break backward
// compatibility on the Write method, we could modify Writer to be an alias
// to GenericWriter with:
//
// type Writer = GenericWriter[any]
//
base Writer
// This function writes rows of type T to the writer, it gets generated by
// the NewGenericWriter function based on the type T and the underlying
// schema of the parquet file.
write writeFunc[T]
// This field is used to leverage the optimized writeRowsFunc algorithms.
columns []ColumnBuffer
}
// NewGenericWriter is like NewWriter but returns a GenericWriter[T] suited to
// write rows of Go type T.
//
// The type parameter T should be a map, struct, or any. Any other types will
// cause a panic at runtime. Type checking is a lot more effective when the
// generic parameter is a struct type, using map and interface types is somewhat
// similar to using a Writer.
//
// If the option list may explicitly declare a schema, it must be compatible
// with the schema generated from T.
//
// Sorting columns may be set on the writer to configure the generated row
// groups metadata. However, rows are always written in the order they were
// seen, no reordering is performed, the writer expects the application to
// ensure proper correlation between the order of rows and the list of sorting
// columns. See SortingWriter[T] for a writer which handles reordering rows
// based on the configured sorting columns.
func NewGenericWriter[T any](output io.Writer, options ...WriterOption) *GenericWriter[T] {
config, err := NewWriterConfig(options...)
if err != nil {
panic(err)
}
schema := config.Schema
t := typeOf[T]()
if schema == nil && t != nil {
schema = schemaOf(dereference(t))
config.Schema = schema
}
if config.Schema == nil {
panic("generic writer must be instantiated with schema or concrete type.")
}
return &GenericWriter[T]{
base: Writer{
output: output,
config: config,
schema: schema,
writer: newWriter(output, config),
},
write: writeFuncOf[T](t, config.Schema),
}
}
type writeFunc[T any] func(*GenericWriter[T], []T) (int, error)
func writeFuncOf[T any](t reflect.Type, schema *Schema) writeFunc[T] {
if t == nil {
return (*GenericWriter[T]).writeAny
}
switch t.Kind() {
case reflect.Interface, reflect.Map:
return (*GenericWriter[T]).writeRows
case reflect.Struct:
return makeWriteFunc[T](t, schema)
case reflect.Pointer:
if e := t.Elem(); e.Kind() == reflect.Struct {
return makeWriteFunc[T](t, schema)
}
}
panic("cannot create writer for values of type " + t.String())
}
func makeWriteFunc[T any](t reflect.Type, schema *Schema) writeFunc[T] {
writeRows := writeRowsFuncOf(t, schema, nil)
return func(w *GenericWriter[T], rows []T) (n int, err error) {
if w.columns == nil {
w.columns = make([]ColumnBuffer, len(w.base.writer.columns))
for i, c := range w.base.writer.columns {
// These fields are usually lazily initialized when writing rows,
// we need them to exist now tho.
c.columnBuffer = c.newColumnBuffer()
w.columns[i] = c.columnBuffer
}
}
err = writeRows(w.columns, makeArrayOf(rows), columnLevels{})
if err == nil {
n = len(rows)
}
return n, err
}
}
func (w *GenericWriter[T]) Close() error {
return w.base.Close()
}
func (w *GenericWriter[T]) Flush() error {
return w.base.Flush()
}
func (w *GenericWriter[T]) Reset(output io.Writer) {
w.base.Reset(output)
}
func (w *GenericWriter[T]) Write(rows []T) (int, error) {
return w.base.writer.writeRows(len(rows), func(i, j int) (int, error) {
n, err := w.write(w, rows[i:j:j])
if err != nil {
return n, err
}
for _, c := range w.base.writer.columns {
if c.columnBuffer.Size() >= int64(c.bufferSize) {
if err := c.flush(); err != nil {
return n, err
}
}
}
return n, nil
})
}
func (w *GenericWriter[T]) WriteRows(rows []Row) (int, error) {
return w.base.WriteRows(rows)
}
func (w *GenericWriter[T]) WriteRowGroup(rowGroup RowGroup) (int64, error) {
return w.base.WriteRowGroup(rowGroup)
}
// SetKeyValueMetadata sets a key/value pair in the Parquet file metadata.
//
// Keys are assumed to be unique, if the same key is repeated multiple times the
// last value is retained. While the parquet format does not require unique keys,
// this design decision was made to optimize for the most common use case where
// applications leverage this extension mechanism to associate single values to
// keys. This may create incompatibilities with other parquet libraries, or may
// cause some key/value pairs to be lost when open parquet files written with
// repeated keys. We can revisit this decision if it ever becomes a blocker.
func (w *GenericWriter[T]) SetKeyValueMetadata(key, value string) {
w.base.SetKeyValueMetadata(key, value)
}
func (w *GenericWriter[T]) ReadRowsFrom(rows RowReader) (int64, error) {
return w.base.ReadRowsFrom(rows)
}
func (w *GenericWriter[T]) Schema() *Schema {
return w.base.Schema()
}
func (w *GenericWriter[T]) writeRows(rows []T) (int, error) {
if cap(w.base.rowbuf) < len(rows) {
w.base.rowbuf = make([]Row, len(rows))
} else {
w.base.rowbuf = w.base.rowbuf[:len(rows)]
}
defer clearRows(w.base.rowbuf)
schema := w.base.Schema()
for i := range rows {
w.base.rowbuf[i] = schema.Deconstruct(w.base.rowbuf[i], &rows[i])
}
return w.base.WriteRows(w.base.rowbuf)
}
func (w *GenericWriter[T]) writeAny(rows []T) (n int, err error) {
for i := range rows {
if err = w.base.Write(rows[i]); err != nil {
return n, err
}
n++
}
return n, nil
}
var (
_ RowWriterWithSchema = (*GenericWriter[any])(nil)
_ RowReaderFrom = (*GenericWriter[any])(nil)
_ RowGroupWriter = (*GenericWriter[any])(nil)
_ RowWriterWithSchema = (*GenericWriter[struct{}])(nil)
_ RowReaderFrom = (*GenericWriter[struct{}])(nil)
_ RowGroupWriter = (*GenericWriter[struct{}])(nil)
_ RowWriterWithSchema = (*GenericWriter[map[struct{}]struct{}])(nil)
_ RowReaderFrom = (*GenericWriter[map[struct{}]struct{}])(nil)
_ RowGroupWriter = (*GenericWriter[map[struct{}]struct{}])(nil)
)
// Deprecated: A Writer uses a parquet schema and sequence of Go values to
// produce a parquet file to an io.Writer.
//
// This example showcases a typical use of parquet writers:
//
// writer := parquet.NewWriter(output)
//
// for _, row := range rows {
// if err := writer.Write(row); err != nil {
// ...
// }
// }
//
// if err := writer.Close(); err != nil {
// ...
// }
//
// The Writer type optimizes for minimal memory usage, each page is written as
// soon as it has been filled so only a single page per column needs to be held
// in memory and as a result, there are no opportunities to sort rows within an
// entire row group. Programs that need to produce parquet files with sorted
// row groups should use the Buffer type to buffer and sort the rows prior to
// writing them to a Writer.
//
// For programs building with Go 1.18 or later, the GenericWriter[T] type
// supersedes this one.
type Writer struct {
output io.Writer
config *WriterConfig
schema *Schema
writer *writer
rowbuf []Row
}
// NewWriter constructs a parquet writer writing a file to the given io.Writer.
//
// The function panics if the writer configuration is invalid. Programs that
// cannot guarantee the validity of the options passed to NewWriter should
// construct the writer configuration independently prior to calling this
// function:
//
// config, err := parquet.NewWriterConfig(options...)
// if err != nil {
// // handle the configuration error
// ...
// } else {
// // this call to create a writer is guaranteed not to panic
// writer := parquet.NewWriter(output, config)
// ...
// }
func NewWriter(output io.Writer, options ...WriterOption) *Writer {
config, err := NewWriterConfig(options...)
if err != nil {
panic(err)
}
w := &Writer{
output: output,
config: config,
}
if config.Schema != nil {
w.configure(config.Schema)
}
return w
}
func (w *Writer) configure(schema *Schema) {
if schema != nil {
w.config.Schema = schema
w.schema = schema
w.writer = newWriter(w.output, w.config)
}
}
// Close must be called after all values were produced to the writer in order to
// flush all buffers and write the parquet footer.
func (w *Writer) Close() error {
if w.writer != nil {
return w.writer.close()
}
return nil
}
// Flush flushes all buffers into a row group to the underlying io.Writer.
//
// Flush is called automatically on Close, it is only useful to call explicitly
// if the application needs to limit the size of row groups or wants to produce
// multiple row groups per file.
//
// If the writer attempts to create more than MaxRowGroups row groups the method
// returns ErrTooManyRowGroups.
func (w *Writer) Flush() error {
if w.writer != nil {
return w.writer.flush()
}
return nil
}
// Reset clears the state of the writer without flushing any of the buffers,
// and setting the output to the io.Writer passed as argument, allowing the
// writer to be reused to produce another parquet file.
//
// Reset may be called at any time, including after a writer was closed.
func (w *Writer) Reset(output io.Writer) {
if w.output = output; w.writer != nil {
w.writer.reset(w.output)
}
}
// Write is called to write another row to the parquet file.
//
// The method uses the parquet schema configured on w to traverse the Go value
// and decompose it into a set of columns and values. If no schema were passed
// to NewWriter, it is deducted from the Go type of the row, which then have to
// be a struct or pointer to struct.
func (w *Writer) Write(row interface{}) error {
if w.schema == nil {
w.configure(SchemaOf(row))
}
if cap(w.rowbuf) == 0 {
w.rowbuf = make([]Row, 1)
} else {
w.rowbuf = w.rowbuf[:1]
}
defer clearRows(w.rowbuf)
w.rowbuf[0] = w.schema.Deconstruct(w.rowbuf[0][:0], row)
_, err := w.WriteRows(w.rowbuf)
return err
}
// WriteRows is called to write rows to the parquet file.
//
// The Writer must have been given a schema when NewWriter was called, otherwise
// the structure of the parquet file cannot be determined from the row only.
//
// The row is expected to contain values for each column of the writer's schema,
// in the order produced by the parquet.(*Schema).Deconstruct method.
func (w *Writer) WriteRows(rows []Row) (int, error) {
return w.writer.WriteRows(rows)
}
// WriteRowGroup writes a row group to the parquet file.
//
// Buffered rows will be flushed prior to writing rows from the group, unless
// the row group was empty in which case nothing is written to the file.
//
// The content of the row group is flushed to the writer; after the method
// returns successfully, the row group will be empty and in ready to be reused.
func (w *Writer) WriteRowGroup(rowGroup RowGroup) (int64, error) {
rowGroupSchema := rowGroup.Schema()
switch {
case rowGroupSchema == nil:
return 0, ErrRowGroupSchemaMissing
case w.schema == nil:
w.configure(rowGroupSchema)
case !nodesAreEqual(w.schema, rowGroupSchema):
return 0, ErrRowGroupSchemaMismatch
}
if err := w.writer.flush(); err != nil {
return 0, err
}
w.writer.configureBloomFilters(rowGroup.ColumnChunks())
rows := rowGroup.Rows()
defer rows.Close()
n, err := CopyRows(w.writer, rows)
if err != nil {
return n, err
}
return w.writer.writeRowGroup(rowGroup.Schema(), rowGroup.SortingColumns())
}
// ReadRowsFrom reads rows from the reader passed as arguments and writes them
// to w.
//
// This is similar to calling WriteRow repeatedly, but will be more efficient
// if optimizations are supported by the reader.
func (w *Writer) ReadRowsFrom(rows RowReader) (written int64, err error) {
if w.schema == nil {
if r, ok := rows.(RowReaderWithSchema); ok {
w.configure(r.Schema())
}
}
if cap(w.rowbuf) < defaultRowBufferSize {
w.rowbuf = make([]Row, defaultRowBufferSize)
} else {
w.rowbuf = w.rowbuf[:cap(w.rowbuf)]
}
return copyRows(w.writer, rows, w.rowbuf)
}
// Schema returns the schema of rows written by w.
//
// The returned value will be nil if no schema has yet been configured on w.
func (w *Writer) Schema() *Schema { return w.schema }
// SetKeyValueMetadata sets a key/value pair in the Parquet file metadata.
//
// Keys are assumed to be unique, if the same key is repeated multiple times the
// last value is retained. While the parquet format does not require unique keys,
// this design decision was made to optimize for the most common use case where
// applications leverage this extension mechanism to associate single values to
// keys. This may create incompatibilities with other parquet libraries, or may
// cause some key/value pairs to be lost when open parquet files written with
// repeated keys. We can revisit this decision if it ever becomes a blocker.
func (w *Writer) SetKeyValueMetadata(key, value string) {
for i, kv := range w.writer.metadata {
if kv.Key == key {
kv.Value = value
w.writer.metadata[i] = kv
return
}
}
w.writer.metadata = append(w.writer.metadata, format.KeyValue{
Key: key,
Value: value,
})
}
type writer struct {
buffer *bufio.Writer
writer offsetTrackingWriter
values [][]Value
numRows int64
maxRows int64
createdBy string
metadata []format.KeyValue
columns []*writerColumn
columnChunk []format.ColumnChunk
columnIndex []format.ColumnIndex
offsetIndex []format.OffsetIndex
columnOrders []format.ColumnOrder
schemaElements []format.SchemaElement
rowGroups []format.RowGroup
columnIndexes [][]format.ColumnIndex
offsetIndexes [][]format.OffsetIndex
sortingColumns []format.SortingColumn
}
func newWriter(output io.Writer, config *WriterConfig) *writer {
w := new(writer)
if config.WriteBufferSize <= 0 {
w.writer.Reset(output)
} else {
w.buffer = bufio.NewWriterSize(output, config.WriteBufferSize)
w.writer.Reset(w.buffer)
}
w.maxRows = config.MaxRowsPerRowGroup
w.createdBy = config.CreatedBy
w.metadata = make([]format.KeyValue, 0, len(config.KeyValueMetadata))
for k, v := range config.KeyValueMetadata {
w.metadata = append(w.metadata, format.KeyValue{Key: k, Value: v})
}
sortKeyValueMetadata(w.metadata)
w.sortingColumns = make([]format.SortingColumn, len(config.Sorting.SortingColumns))
config.Schema.forEachNode(func(name string, node Node) {
nodeType := node.Type()
repetitionType := (*format.FieldRepetitionType)(nil)
if node != config.Schema { // the root has no repetition type
repetitionType = fieldRepetitionTypePtrOf(node)
}
// For backward compatibility with older readers, the parquet specification
// recommends to set the scale and precision on schema elements when the
// column is of logical type decimal.
logicalType := nodeType.LogicalType()
scale, precision := (*int32)(nil), (*int32)(nil)
if logicalType != nil && logicalType.Decimal != nil {
scale = &logicalType.Decimal.Scale
precision = &logicalType.Decimal.Precision
}
typeLength := (*int32)(nil)
if n := int32(nodeType.Length()); n > 0 {
typeLength = &n
}
w.schemaElements = append(w.schemaElements, format.SchemaElement{
Type: nodeType.PhysicalType(),
TypeLength: typeLength,
RepetitionType: repetitionType,
Name: name,
NumChildren: int32(len(node.Fields())),
ConvertedType: nodeType.ConvertedType(),
Scale: scale,
Precision: precision,
FieldID: int32(node.ID()),
LogicalType: logicalType,
})
})
dataPageType := format.DataPage
if config.DataPageVersion == 2 {
dataPageType = format.DataPageV2
}
defaultCompression := config.Compression
if defaultCompression == nil {
defaultCompression = &Uncompressed
}
// Those buffers are scratch space used to generate the page header and
// content, they are shared by all column chunks because they are only
// used during calls to writeDictionaryPage or writeDataPage, which are
// not done concurrently.
buffers := new(writerBuffers)
forEachLeafColumnOf(config.Schema, func(leaf leafColumn) {
encoding := encodingOf(leaf.node)
dictionary := Dictionary(nil)
columnType := leaf.node.Type()
columnIndex := int(leaf.columnIndex)
compression := leaf.node.Compression()
if compression == nil {
compression = defaultCompression
}
if isDictionaryEncoding(encoding) {
dictBuffer := columnType.NewValues(
make([]byte, 0, defaultDictBufferSize),
nil,
)
dictionary = columnType.NewDictionary(columnIndex, 0, dictBuffer)
columnType = dictionary.Type()
}
c := &writerColumn{
buffers: buffers,
pool: config.ColumnPageBuffers,
columnPath: leaf.path,
columnType: columnType,
columnIndex: columnType.NewColumnIndexer(config.ColumnIndexSizeLimit),
columnFilter: searchBloomFilterColumn(config.BloomFilters, leaf.path),
compression: compression,
dictionary: dictionary,
dataPageType: dataPageType,
maxRepetitionLevel: leaf.maxRepetitionLevel,
maxDefinitionLevel: leaf.maxDefinitionLevel,
bufferIndex: int32(leaf.columnIndex),
bufferSize: int32(float64(config.PageBufferSize) * 0.98),
writePageStats: config.DataPageStatistics,
encodings: make([]format.Encoding, 0, 3),
// Data pages in version 2 can omit compression when dictionary
// encoding is employed; only the dictionary page needs to be
// compressed, the data pages are encoded with the hybrid
// RLE/Bit-Pack encoding which doesn't benefit from an extra
// compression layer.
isCompressed: isCompressed(compression) && (dataPageType != format.DataPageV2 || dictionary == nil),
}
c.header.encoder.Reset(c.header.protocol.NewWriter(&buffers.header))
if leaf.maxDefinitionLevel > 0 {
c.encodings = addEncoding(c.encodings, format.RLE)
}
if isDictionaryEncoding(encoding) {
c.encodings = addEncoding(c.encodings, format.Plain)
}
c.encoding = encoding
c.encodings = addEncoding(c.encodings, c.encoding.Encoding())
sortPageEncodings(c.encodings)
w.columns = append(w.columns, c)
if sortingIndex := searchSortingColumn(config.Sorting.SortingColumns, leaf.path); sortingIndex < len(w.sortingColumns) {
w.sortingColumns[sortingIndex] = format.SortingColumn{
ColumnIdx: int32(leaf.columnIndex),
Descending: config.Sorting.SortingColumns[sortingIndex].Descending(),
NullsFirst: config.Sorting.SortingColumns[sortingIndex].NullsFirst(),
}
}
})
// Pre-allocate the backing array so that in most cases where the rows
// contain a single value we will hit collocated memory areas when writing
// rows to the writer. This won't benefit repeated columns much but in that
// case we would just waste a bit of memory which we can afford.
values := make([]Value, len(w.columns))
w.values = make([][]Value, len(w.columns))
for i := range values {
w.values[i] = values[i : i : i+1]
}
w.columnChunk = make([]format.ColumnChunk, len(w.columns))
w.columnIndex = make([]format.ColumnIndex, len(w.columns))
w.offsetIndex = make([]format.OffsetIndex, len(w.columns))
w.columnOrders = make([]format.ColumnOrder, len(w.columns))
for i, c := range w.columns {
w.columnChunk[i] = format.ColumnChunk{
MetaData: format.ColumnMetaData{
Type: format.Type(c.columnType.Kind()),
Encoding: c.encodings,
PathInSchema: c.columnPath,
Codec: c.compression.CompressionCodec(),
KeyValueMetadata: nil, // TODO
},
}
}
for i, c := range w.columns {
c.columnChunk = &w.columnChunk[i]
c.offsetIndex = &w.offsetIndex[i]
}
for i, c := range w.columns {
w.columnOrders[i] = *c.columnType.ColumnOrder()
}
return w
}
func (w *writer) reset(writer io.Writer) {
if w.buffer == nil {
w.writer.Reset(writer)
} else {
w.buffer.Reset(writer)
w.writer.Reset(w.buffer)
}
for _, c := range w.columns {
c.reset()
}
for i := range w.rowGroups {
w.rowGroups[i] = format.RowGroup{}
}
for i := range w.columnIndexes {
w.columnIndexes[i] = nil
}
for i := range w.offsetIndexes {
w.offsetIndexes[i] = nil
}
w.rowGroups = w.rowGroups[:0]
w.columnIndexes = w.columnIndexes[:0]
w.offsetIndexes = w.offsetIndexes[:0]
}
func (w *writer) close() error {
if err := w.writeFileHeader(); err != nil {
return err
}
if err := w.flush(); err != nil {
return err
}
if err := w.writeFileFooter(); err != nil {
return err
}
if w.buffer != nil {
return w.buffer.Flush()
}
return nil
}
func (w *writer) flush() error {
_, err := w.writeRowGroup(nil, nil)
return err
}
func (w *writer) writeFileHeader() error {
if w.writer.writer == nil {
return io.ErrClosedPipe
}
if w.writer.offset == 0 {
_, err := w.writer.WriteString("PAR1")
return err
}
return nil
}
func (w *writer) configureBloomFilters(columnChunks []ColumnChunk) {
for i, c := range w.columns {
if c.columnFilter != nil {
c.resizeBloomFilter(columnChunks[i].NumValues())
}
}
}
func (w *writer) writeFileFooter() error {
// The page index is composed of two sections: column and offset indexes.
// They are written after the row groups, right before the footer (which
// is written by the parent Writer.Close call).
//
// This section both writes the page index and generates the values of
// ColumnIndexOffset, ColumnIndexLength, OffsetIndexOffset, and
// OffsetIndexLength in the corresponding columns of the file metadata.
//
// Note: the page index is always written, even if we created data pages v1
// because the parquet format is backward compatible in this case. Older
// readers will simply ignore this section since they do not know how to
// decode its content, nor have loaded any metadata to reference it.
protocol := new(thrift.CompactProtocol)
encoder := thrift.NewEncoder(protocol.NewWriter(&w.writer))
for i, columnIndexes := range w.columnIndexes {
rowGroup := &w.rowGroups[i]
for j := range columnIndexes {
column := &rowGroup.Columns[j]
column.ColumnIndexOffset = w.writer.offset
if err := encoder.Encode(&columnIndexes[j]); err != nil {
return err
}
column.ColumnIndexLength = int32(w.writer.offset - column.ColumnIndexOffset)
}
}
for i, offsetIndexes := range w.offsetIndexes {
rowGroup := &w.rowGroups[i]
for j := range offsetIndexes {
column := &rowGroup.Columns[j]
column.OffsetIndexOffset = w.writer.offset
if err := encoder.Encode(&offsetIndexes[j]); err != nil {
return err
}
column.OffsetIndexLength = int32(w.writer.offset - column.OffsetIndexOffset)
}
}
numRows := int64(0)
for rowGroupIndex := range w.rowGroups {
numRows += w.rowGroups[rowGroupIndex].NumRows
}
footer, err := thrift.Marshal(new(thrift.CompactProtocol), &format.FileMetaData{
Version: 1,
Schema: w.schemaElements,
NumRows: numRows,
RowGroups: w.rowGroups,
KeyValueMetadata: w.metadata,
CreatedBy: w.createdBy,
ColumnOrders: w.columnOrders,
})
if err != nil {
return err
}
length := len(footer)
footer = append(footer, 0, 0, 0, 0)
footer = append(footer, "PAR1"...)
binary.LittleEndian.PutUint32(footer[length:], uint32(length))
_, err = w.writer.Write(footer)
return err
}
func (w *writer) writeRowGroup(rowGroupSchema *Schema, rowGroupSortingColumns []SortingColumn) (int64, error) {
numRows := w.columns[0].totalRowCount()
if numRows == 0 {
return 0, nil
}
if len(w.rowGroups) == MaxRowGroups {
return 0, ErrTooManyRowGroups
}
defer func() {
w.numRows = 0
for _, c := range w.columns {
c.reset()
}
for i := range w.columnIndex {
w.columnIndex[i] = format.ColumnIndex{}
}
}()
for _, c := range w.columns {
if err := c.flush(); err != nil {
return 0, err
}
if err := c.flushFilterPages(); err != nil {
return 0, err
}
}
if err := w.writeFileHeader(); err != nil {
return 0, err
}
fileOffset := w.writer.offset
for _, c := range w.columns {
if len(c.filter) > 0 {
c.columnChunk.MetaData.BloomFilterOffset = w.writer.offset
if err := c.writeBloomFilter(&w.writer); err != nil {
return 0, err
}
}
}
for i, c := range w.columns {
w.columnIndex[i] = format.ColumnIndex(c.columnIndex.ColumnIndex())
if c.dictionary != nil {
c.columnChunk.MetaData.DictionaryPageOffset = w.writer.offset
if err := c.writeDictionaryPage(&w.writer, c.dictionary); err != nil {
return 0, fmt.Errorf("writing dictionary page of row group colum %d: %w", i, err)
}
}
dataPageOffset := w.writer.offset
c.columnChunk.MetaData.DataPageOffset = dataPageOffset
for j := range c.offsetIndex.PageLocations {
c.offsetIndex.PageLocations[j].Offset += dataPageOffset
}
if offset, err := c.pageBuffer.Seek(0, io.SeekStart); err != nil {
return 0, err
} else if offset != 0 {
return 0, fmt.Errorf("resetting parquet page buffer to the start expected offset zero but got %d", offset)
}
if _, err := io.Copy(&w.writer, c.pageBuffer); err != nil {
return 0, fmt.Errorf("writing buffered pages of row group column %d: %w", i, err)
}
}
totalByteSize := int64(0)
totalCompressedSize := int64(0)
for i := range w.columnChunk {
c := &w.columnChunk[i].MetaData
sortPageEncodingStats(c.EncodingStats)
totalByteSize += int64(c.TotalUncompressedSize)
totalCompressedSize += int64(c.TotalCompressedSize)
}
sortingColumns := w.sortingColumns
if len(sortingColumns) == 0 && len(rowGroupSortingColumns) > 0 {
sortingColumns = make([]format.SortingColumn, 0, len(rowGroupSortingColumns))
forEachLeafColumnOf(rowGroupSchema, func(leaf leafColumn) {
if sortingIndex := searchSortingColumn(rowGroupSortingColumns, leaf.path); sortingIndex < len(sortingColumns) {
sortingColumns[sortingIndex] = format.SortingColumn{
ColumnIdx: int32(leaf.columnIndex),
Descending: rowGroupSortingColumns[sortingIndex].Descending(),
NullsFirst: rowGroupSortingColumns[sortingIndex].NullsFirst(),
}
}
})
}
columns := make([]format.ColumnChunk, len(w.columnChunk))
copy(columns, w.columnChunk)
columnIndex := make([]format.ColumnIndex, len(w.columnIndex))
copy(columnIndex, w.columnIndex)
offsetIndex := make([]format.OffsetIndex, len(w.offsetIndex))
copy(offsetIndex, w.offsetIndex)
for i := range columns {
c := &columns[i]
c.MetaData.EncodingStats = make([]format.PageEncodingStats, len(c.MetaData.EncodingStats))
copy(c.MetaData.EncodingStats, w.columnChunk[i].MetaData.EncodingStats)
}
for i := range offsetIndex {
c := &offsetIndex[i]
c.PageLocations = make([]format.PageLocation, len(c.PageLocations))
copy(c.PageLocations, w.offsetIndex[i].PageLocations)
}
w.rowGroups = append(w.rowGroups, format.RowGroup{
Columns: columns,
TotalByteSize: totalByteSize,
NumRows: numRows,
SortingColumns: sortingColumns,
FileOffset: fileOffset,
TotalCompressedSize: totalCompressedSize,
Ordinal: int16(len(w.rowGroups)),
})
w.columnIndexes = append(w.columnIndexes, columnIndex)
w.offsetIndexes = append(w.offsetIndexes, offsetIndex)
return numRows, nil
}
func (w *writer) WriteRows(rows []Row) (int, error) {
return w.writeRows(len(rows), func(start, end int) (int, error) {
defer func() {
for i, values := range w.values {
clearValues(values)
w.values[i] = values[:0]
}
}()
// TODO: if an error occurs in this method the writer may be left in an
// partially functional state. Applications are not expected to continue
// using the writer after getting an error, but maybe we could ensure that
// we are preventing further use as well?
for _, row := range rows[start:end] {
row.Range(func(columnIndex int, columnValues []Value) bool {
w.values[columnIndex] = append(w.values[columnIndex], columnValues...)
return true
})
}
for i, values := range w.values {
if len(values) > 0 {
if err := w.columns[i].writeRows(values); err != nil {
return 0, err
}
}
}
return end - start, nil
})
}
func (w *writer) writeRows(numRows int, write func(i, j int) (int, error)) (int, error) {
written := 0
for written < numRows {
remain := w.maxRows - w.numRows
length := numRows - written
if remain == 0 {
remain = w.maxRows
if err := w.flush(); err != nil {
return written, err
}
}
if remain < int64(length) {
length = int(remain)
}
// Since the writer cannot flush pages across row boundaries, calls to
// WriteRows with very large slices can result in greatly exceeding the
// target page size. To set a limit to the impact of these large writes
// we chunk the input in slices of 64 rows.
//
// Note that this mechanism isn't perfect; for example, values may hold
// large byte slices which could still cause the column buffers to grow
// beyond the target page size.
const maxRowsPerWrite = 64
if length > maxRowsPerWrite {