-
Notifications
You must be signed in to change notification settings - Fork 53
/
file_writer.go
358 lines (298 loc) · 10.3 KB
/
file_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
package goparquet
import (
"context"
"encoding/binary"
"errors"
"io"
"github.com/fraugster/parquet-go/parquet"
"github.com/fraugster/parquet-go/parquetschema"
)
// FileWriter is used to write data to a parquet file. Always use NewFileWriter
// to create such an object.
type FileWriter struct {
w writePos
version int32
//SchemaWriter
schemaWriter *schema
totalNumRecords int64
kvStore map[string]string
createdBy string
rowGroupFlushSize int64
rowGroups []*parquet.RowGroup
codec parquet.CompressionCodec
newPageFunc newDataPageFunc
ctx context.Context
schemaDef *parquetschema.SchemaDefinition
}
// FileWriterOption describes an option function that is applied to a FileWriter when it is created.
type FileWriterOption func(fw *FileWriter)
// NewFileWriter creates a new FileWriter. You can provide FileWriterOptions to influence the
// file writer's behaviour.
func NewFileWriter(w io.Writer, options ...FileWriterOption) *FileWriter {
fw := &FileWriter{
w: &writePosStruct{
w: w,
pos: 0,
},
version: 1,
schemaWriter: &schema{},
kvStore: make(map[string]string),
rowGroups: []*parquet.RowGroup{},
createdBy: "parquet-go",
newPageFunc: newDataPageV1Writer,
ctx: context.Background(),
}
for _, opt := range options {
opt(fw)
}
// if a WithSchemaDefinition option was provided, the schema needs to be set after everything else
// as other options can change settings on the schemaWriter (such as the maximum page size).
if fw.schemaDef != nil {
if err := fw.schemaWriter.SetSchemaDefinition(fw.schemaDef); err != nil {
panic(err) // TODO: this shouldn't happen, but still isn't great. We messed up the API design for options and NewFileWriter.
}
}
return fw
}
// FileVersion sets the version of the file itself.
func FileVersion(version int32) FileWriterOption {
return func(fw *FileWriter) {
fw.version = version
}
}
// WithCreator sets the creator in the meta data of the file.
func WithCreator(createdBy string) FileWriterOption {
return func(fw *FileWriter) {
fw.createdBy = createdBy
}
}
// WithCompressionCodec sets the compression codec used when writing the file.
func WithCompressionCodec(codec parquet.CompressionCodec) FileWriterOption {
return func(fw *FileWriter) {
fw.codec = codec
}
}
// WithMetaData sets the key-value meta data on the file.
func WithMetaData(data map[string]string) FileWriterOption {
return func(fw *FileWriter) {
if data != nil {
fw.kvStore = data
return
}
fw.kvStore = make(map[string]string)
}
}
// WithMaxRowGroupSize sets the rough maximum size of a row group before it shall
// be flushed automatically. Please note that enabling auto-flush will not allow
// you to set per-column-chunk meta-data upon calling FlushRowGroup. If you
// require this feature, you need to flush your rowgroups manually.
func WithMaxRowGroupSize(size int64) FileWriterOption {
return func(fw *FileWriter) {
fw.rowGroupFlushSize = size
}
}
func WithMaxPageSize(size int64) FileWriterOption {
return func(fw *FileWriter) {
fw.schemaWriter.maxPageSize = size
}
}
// WithSchemaDefinition sets the schema definition to use for this parquet file.
func WithSchemaDefinition(sd *parquetschema.SchemaDefinition) FileWriterOption {
return func(fw *FileWriter) {
fw.schemaDef = sd
}
}
// WithDataPageV2 enables the writer to write pages in the new V2 format. By default,
// the library is using the V1 format. Please be aware that this may cause compatibility
// issues with older implementations of parquet.
func WithDataPageV2() FileWriterOption {
return func(fw *FileWriter) {
fw.newPageFunc = newDataPageV2Writer
}
}
func WithCRC(enableCRC bool) FileWriterOption {
return func(fw *FileWriter) {
fw.schemaWriter.enableCRC = enableCRC
}
}
// WithWriterContext overrides the default context (which is a context.Background())
// in the FileWriter with the provided context.Context object.
func WithWriterContext(ctx context.Context) FileWriterOption {
return func(fw *FileWriter) {
fw.ctx = ctx
}
}
type flushRowGroupOptionHandle struct {
cols map[string]map[string]string
global map[string]string
}
func newFlushRowGroupOptionHandle() *flushRowGroupOptionHandle {
return &flushRowGroupOptionHandle{
cols: make(map[string]map[string]string),
global: make(map[string]string),
}
}
func (h *flushRowGroupOptionHandle) getMetaData(col string) map[string]string {
data := make(map[string]string)
for k, v := range h.global {
data[k] = v
}
for k, v := range h.cols[col] {
data[k] = v
}
if len(data) > 0 {
return data
}
return nil
}
// FlushRowGroupOption is an option to pass additiona configuration to FlushRowGroup.
type FlushRowGroupOption func(h *flushRowGroupOptionHandle)
// WithRowGroupMetaDataForColumn adds key-value metadata to a particular column that is identified
// by its full dotted-notation name.
func WithRowGroupMetaDataForColumn(col string, kv map[string]string) FlushRowGroupOption {
return func(h *flushRowGroupOptionHandle) {
colKV := h.cols[col]
if colKV == nil {
colKV = make(map[string]string)
}
for k, v := range kv {
colKV[k] = v
}
h.cols[col] = colKV
}
}
// WithRowGroupMetaData adds key-value metadata to all columns. Please note that if you use the same
// key both in the meta data for all columns as well as in column-specific meta data
// (using MetaDataForColumn), the column-specific meta data has preference.
func WithRowGroupMetaData(kv map[string]string) FlushRowGroupOption {
return func(h *flushRowGroupOptionHandle) {
for k, v := range kv {
h.global[k] = v
}
}
}
// FlushRowGroup writes the current row group to the parquet file.
func (fw *FileWriter) FlushRowGroup(opts ...FlushRowGroupOption) error {
return fw.FlushRowGroupWithContext(fw.ctx, opts...)
}
// FlushRowGroupWithContext writes the current row group to the parquet file.
func (fw *FileWriter) FlushRowGroupWithContext(ctx context.Context, opts ...FlushRowGroupOption) error {
// Write the entire row group
if fw.schemaWriter.rowGroupNumRecords() == 0 {
return errors.New("nothing to write")
}
if fw.w.Pos() == 0 {
if err := writeFull(fw.w, magic); err != nil {
return err
}
}
h := newFlushRowGroupOptionHandle()
for _, o := range opts {
o(h)
}
cc, err := writeRowGroup(ctx, fw.w, fw.schemaWriter, fw.codec, fw.newPageFunc, h)
if err != nil {
return err
}
var totalCompressedSize, totalUncompressedSize int64
for _, c := range cc {
totalCompressedSize += c.MetaData.TotalCompressedSize
totalUncompressedSize += c.MetaData.TotalUncompressedSize
}
fw.rowGroups = append(fw.rowGroups, &parquet.RowGroup{
Columns: cc,
TotalByteSize: totalUncompressedSize,
TotalCompressedSize: &totalCompressedSize,
NumRows: fw.schemaWriter.rowGroupNumRecords(),
SortingColumns: nil,
})
fw.totalNumRecords += fw.schemaWriter.rowGroupNumRecords()
// flush the schema
fw.schemaWriter.resetData()
return nil
}
// AddData adds a new record to the current row group and flushes it if auto-flush is enabled and the size
// is equal to or greater than the configured maximum row group size.
func (fw *FileWriter) AddData(m map[string]interface{}) error {
if err := fw.schemaWriter.AddData(m); err != nil {
return err
}
if fw.rowGroupFlushSize > 0 && fw.schemaWriter.DataSize() >= fw.rowGroupFlushSize {
return fw.FlushRowGroup()
}
return nil
}
// Close flushes the current row group if necessary, taking the provided
// options into account, and writes the meta data footer to the file.
// Please be aware that this only finalizes the writing process. If you
// provided a file as io.Writer when creating the FileWriter, you still need
// to Close that file handle separately.
func (fw *FileWriter) Close(opts ...FlushRowGroupOption) error {
return fw.CloseWithContext(fw.ctx, opts...)
}
// CloseWithContext flushes the current row group if necessary, taking the provided
// options into account, and writes the meta data footer to the file.
// Please be aware that this only finalizes the writing process. If you
// provided a file as io.Writer when creating the FileWriter, you still need
// to Close that file handle separately.
func (fw *FileWriter) CloseWithContext(ctx context.Context, opts ...FlushRowGroupOption) error {
if len(fw.rowGroups) == 0 || fw.schemaWriter.rowGroupNumRecords() > 0 {
if err := fw.FlushRowGroup(opts...); err != nil {
return err
}
}
kv := make([]*parquet.KeyValue, 0, len(fw.kvStore))
for i := range fw.kvStore {
v := fw.kvStore[i]
addr := &v
if v == "" {
addr = nil
}
kv = append(kv, &parquet.KeyValue{
Key: i,
Value: addr,
})
}
meta := &parquet.FileMetaData{
Version: fw.version,
Schema: fw.schemaWriter.getSchemaArray(),
NumRows: fw.totalNumRecords,
RowGroups: fw.rowGroups,
KeyValueMetadata: kv,
CreatedBy: &fw.createdBy,
ColumnOrders: nil,
}
pos := fw.w.Pos()
if err := writeThrift(ctx, meta, fw.w); err != nil {
return err
}
ln := int32(fw.w.Pos() - pos)
if err := binary.Write(fw.w, binary.LittleEndian, &ln); err != nil {
return err
}
return writeFull(fw.w, magic)
}
// CurrentRowGroupSize returns a rough estimation of the uncompressed size of the current row group data. If you selected
// a compression format other than UNCOMPRESSED, the final size will most likely be smaller and will dpeend on how well
// your data can be compressed.
func (fw *FileWriter) CurrentRowGroupSize() int64 {
return fw.schemaWriter.DataSize()
}
// CurrentFileSize returns the amount of data written to the file so far. This does not include data that is in the
// current row group and has not been flushed yet. After closing the file, the size will be even larger since the
// footer is appended to the file upon closing.
func (fw *FileWriter) CurrentFileSize() int64 {
return fw.w.Pos()
}
// AddColumn adds a new data column to the schema of this file writer.
func (fw *FileWriter) AddColumn(path string, col *Column) error {
return fw.schemaWriter.AddColumn(path, col)
}
// AddGroup adds a new group to the schema of this file writer.
func (fw *FileWriter) AddGroup(path string, rep parquet.FieldRepetitionType) error {
return fw.schemaWriter.AddGroup(path, rep)
}
// GetSchemaDefinition returns the schema definition that has been set in this file writer.
func (fw *FileWriter) GetSchemaDefinition() *parquetschema.SchemaDefinition {
return fw.schemaWriter.GetSchemaDefinition()
}