-
Notifications
You must be signed in to change notification settings - Fork 3.8k
/
exportparquet.go
773 lines (718 loc) · 27.4 KB
/
exportparquet.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
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.
package importer
import (
"bytes"
"context"
"fmt"
"math"
"strings"
"time"
"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/geo"
"github.com/cockroachdb/cockroach/pkg/geo/geopb"
"github.com/cockroachdb/cockroach/pkg/roachpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/colinfo"
"github.com/cockroachdb/cockroach/pkg/sql/execinfra"
"github.com/cockroachdb/cockroach/pkg/sql/execinfrapb"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/rowenc"
"github.com/cockroachdb/cockroach/pkg/sql/rowexec"
"github.com/cockroachdb/cockroach/pkg/sql/sem/builtins"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/sql/types"
"github.com/cockroachdb/cockroach/pkg/util/bitarray"
"github.com/cockroachdb/cockroach/pkg/util/duration"
crlparquet "github.com/cockroachdb/cockroach/pkg/util/parquet"
"github.com/cockroachdb/cockroach/pkg/util/timeofday"
"github.com/cockroachdb/cockroach/pkg/util/tracing"
"github.com/cockroachdb/cockroach/pkg/util/uuid"
"github.com/cockroachdb/errors"
"github.com/fraugster/parquet-go/parquet"
"github.com/fraugster/parquet-go/parquetschema"
"github.com/lib/pq/oid"
)
const exportParquetFilePatternDefault = exportFilePatternPart + ".parquet"
func fileName(spec execinfrapb.ExportSpec, part string) string {
pattern := exportParquetFilePatternDefault
if spec.NamePattern != "" {
pattern = spec.NamePattern
}
fileName := strings.Replace(pattern, exportFilePatternPart, part, -1)
suffix := ""
switch spec.Format.Compression {
case roachpb.IOFileFormat_Gzip:
suffix = ".gz"
case roachpb.IOFileFormat_Snappy:
suffix = ".snappy"
}
fileName += suffix
return fileName
}
// ParquetColumn contains the relevant data to map a crdb table column to a parquet table column.
type ParquetColumn struct {
name string
crbdType *types.T
// definition contains all relevant information around the parquet type for the table column
definition *parquetschema.ColumnDefinition
// encodeFn converts crdb table column value to a native go type that the
// parquet vendor can ingest.
encodeFn func(datum tree.Datum) (interface{}, error)
// DecodeFn converts a native go type, created by the parquet vendor while
// reading a parquet file, into a crdb column value
DecodeFn func(interface{}) (tree.Datum, error)
}
// populateLogicalStringCol is a helper function for populating parquet schema
// info for a column that will get encoded as a string
func populateLogicalStringCol(schemaEl *parquet.SchemaElement) {
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.STRING = parquet.NewStringType()
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_UTF8)
}
// RoundtripStringer pretty prints the datum's value as string, allowing the
// parser in certain decoders to work.
func RoundtripStringer(d tree.Datum) string {
fmtCtx := tree.NewFmtCtx(tree.FmtBareStrings)
d.Format(fmtCtx)
return fmtCtx.CloseAndGetString()
}
// NewParquetColumn populates a ParquetColumn by finding the right parquet type
// and defining the encoder and decoder.
//
// TODO(#104278): delete this function and dependencies once it is not required
// by tests.
func NewParquetColumn(typ *types.T, name string, nullable bool) (ParquetColumn, error) {
col := ParquetColumn{}
col.definition = new(parquetschema.ColumnDefinition)
col.definition.SchemaElement = parquet.NewSchemaElement()
col.name = name
col.crbdType = typ
schemaEl := col.definition.SchemaElement
/*
The type of a parquet column is either a group (i.e.
an array in crdb) or a primitive type (e.g., int, float, boolean,
string) and the repetition can be one of the three following cases:
- required: exactly one occurrence (i.e. the column value is a scalar, and
cannot have null values). A column is set to required if the user
specified the CRDB column as NOT NULL.
- optional: 0 or 1 occurrence (i.e. same as above, but can have values)
- repeated: 0 or more occurrences (the column value will be an array. A
value within the array will have its own repetition type)
See this blog post for more on parquet type specification:
https://blog.twitter.com/engineering/en_us/a/2013/dremel-made-simple-with-parquet
*/
schemaEl.RepetitionType = parquet.FieldRepetitionTypePtr(parquet.FieldRepetitionType_OPTIONAL)
if !nullable {
schemaEl.RepetitionType = parquet.FieldRepetitionTypePtr(parquet.FieldRepetitionType_REQUIRED)
}
schemaEl.Name = col.name
// MB figured out the low level properties of the encoding by running the goland debugger on
// the following vendor example:
// https://github.com/fraugster/parquet-go/blob/master/examples/write-low-level/main.go
switch typ.Family() {
case types.BoolFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BOOLEAN)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return bool(*d.(*tree.DBool)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.MakeDBool(tree.DBool(x.(bool))), nil
}
case types.StringFamily:
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(*d.(*tree.DString)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDString(string(x.([]byte))), nil
}
case types.CollatedStringFamily:
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DCollatedString).Contents), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDCollatedString(string(x.([]byte)), typ.Locale(), &tree.CollationEnvironment{})
}
case types.INetFamily:
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DIPAddr).IPAddr.String()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.ParseDIPAddrFromINetString(string(x.([]byte)))
}
case types.JsonFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.JSON = parquet.NewJsonType()
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_JSON)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DJSON).JSON.String()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
jsonStr := string(x.([]byte))
return tree.ParseDJSON(jsonStr)
}
case types.IntFamily:
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.INTEGER = parquet.NewIntType()
schemaEl.LogicalType.INTEGER.IsSigned = true
if typ.Oid() == oid.T_int8 {
schemaEl.Type = parquet.TypePtr(parquet.Type_INT64)
schemaEl.LogicalType.INTEGER.BitWidth = int8(64)
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_INT_64)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return int64(*d.(*tree.DInt)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDInt(tree.DInt(x.(int64))), nil
}
} else {
schemaEl.Type = parquet.TypePtr(parquet.Type_INT32)
schemaEl.LogicalType.INTEGER.BitWidth = int8(32)
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_INT_32)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return int32(*d.(*tree.DInt)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDInt(tree.DInt(x.(int32))), nil
}
}
case types.FloatFamily:
if typ.Oid() == oid.T_float4 {
schemaEl.Type = parquet.TypePtr(parquet.Type_FLOAT)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
h := float32(*d.(*tree.DFloat))
return h, nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
// must convert float32 to string before converting to float64 (the
// underlying data type of a tree.Dfloat) because directly converting
// a float32 to a float64 will add on trailing significant digits,
// causing the round trip tests to fail.
hS := fmt.Sprintf("%f", x.(float32))
return tree.ParseDFloat(hS)
}
} else {
schemaEl.Type = parquet.TypePtr(parquet.Type_DOUBLE)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return float64(*d.(*tree.DFloat)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDFloat(tree.DFloat(x.(float64))), nil
}
}
case types.DecimalFamily:
// TODO (MB): Investigate if the parquet vendor should enforce precision and
// scale requirements. In a toy example, the parquet vendor was able to
// write/read roundtrip the string "3235.5432" as a Decimal with Scale = 1,
// Precision = 1, even though this decimal has a larger scale and precision.
// I guess it's the responsibility of CRDB to enforce the Scale and
// Precision conditions, and for the parquet vendor to NOT lose data, even if
// the data doesn't follow the scale and precision conditions.
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.DECIMAL = parquet.NewDecimalType()
schemaEl.LogicalType.DECIMAL.Scale = typ.Scale()
schemaEl.LogicalType.DECIMAL.Precision = typ.Precision()
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_DECIMAL)
// According to PostgresSQL docs, scale or precision of 0 implies max
// precision and scale. I assume this is what CRDB does, but this isn't
// explicit in the docs https://www.postgresql.org/docs/10/datatype-numeric.html
if typ.Scale() == 0 {
schemaEl.LogicalType.DECIMAL.Scale = math.MaxInt32
}
if typ.Precision() == 0 {
schemaEl.LogicalType.DECIMAL.Precision = math.MaxInt32
}
schemaEl.Scale = &schemaEl.LogicalType.DECIMAL.Scale
schemaEl.Precision = &schemaEl.LogicalType.DECIMAL.Precision
col.encodeFn = func(d tree.Datum) (interface{}, error) {
dec := d.(*tree.DDecimal).Decimal
return []byte(dec.String()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
// TODO (MB): investigative if crdb should gather decimal metadata from
// parquet file during IMPORT PARQUET.
return tree.ParseDDecimal(string(x.([]byte)))
}
case types.UuidFamily:
// Vendor parquet documentation suggests that UUID maps to the [16]byte go type
// https://github.com/fraugster/parquet-go#supported-logical-types
schemaEl.Type = parquet.TypePtr(parquet.Type_FIXED_LEN_BYTE_ARRAY)
byteArraySize := int32(uuid.Size)
schemaEl.TypeLength = &byteArraySize
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.UUID = parquet.NewUUIDType()
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return d.(*tree.DUuid).UUID.GetBytes(), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.ParseDUuidFromBytes(x.([]byte))
}
case types.BytesFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(*d.(*tree.DBytes)), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.NewDBytes(tree.DBytes(x.([]byte))), nil
}
case types.BitFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
// TODO(MB): investigate whether bit arrays should be encoded as an array of longs,
// like in avro changefeeds
baS := RoundtripStringer(d.(*tree.DBitArray))
return []byte(baS), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
ba, err := bitarray.Parse(string(x.([]byte)))
return &tree.DBitArray{BitArray: ba}, err
}
case types.EnumFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.ENUM = parquet.NewEnumType()
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_ENUM)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DEnum).LogicalRep), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
e, err := tree.MakeDEnumFromLogicalRepresentation(typ, string(x.([]byte)))
if err != nil {
return nil, err
}
return tree.NewDEnum(e), nil
}
case types.Box2DFamily:
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DBox2D).CartesianBoundingBox.Repr()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
b, err := geo.ParseCartesianBoundingBox(string(x.([]byte)))
if err != nil {
return nil, err
}
return tree.NewDBox2D(b), nil
}
case types.GeographyFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DGeography).EWKB()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
g, err := geo.ParseGeographyFromEWKB(geopb.EWKB(x.([]byte)))
if err != nil {
return nil, err
}
return &tree.DGeography{Geography: g}, nil
}
case types.GeometryFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_BYTE_ARRAY)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DGeometry).EWKB()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
g, err := geo.ParseGeometryFromEWKBUnsafe(geopb.EWKB(x.([]byte)))
if err != nil {
return nil, err
}
return &tree.DGeometry{Geometry: g}, nil
}
case types.DateFamily:
// Even though the parquet vendor supports Dates, we export Dates as strings
// because the vendor only supports encoding them as an int32, the Days
// since the Unix epoch, which according CRDB's `date.UnixEpochDays( )` (in
// pgdate package) is vulnerable to overflow.
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
date := d.(*tree.DDate)
ds := RoundtripStringer(date)
return []byte(ds), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
dStr := string(x.([]byte))
d, dependCtx, err := tree.ParseDDate(nil, dStr)
if dependCtx {
return nil, errors.Newf("decoding date %s failed. depends on context", string(x.([]byte)))
}
return d, err
}
case types.TimeFamily:
schemaEl.Type = parquet.TypePtr(parquet.Type_INT64)
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.TIME = parquet.NewTimeType()
t := parquet.NewTimeUnit()
t.MICROS = parquet.NewMicroSeconds()
schemaEl.LogicalType.TIME.Unit = t
schemaEl.LogicalType.TIME.IsAdjustedToUTC = true // per crdb docs
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_TIME_MICROS)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
// Time of day is stored in microseconds since midnight,
// which is also how parquet stores time
time := d.(*tree.DTime)
m := int64(*time)
return m, nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.MakeDTime(timeofday.TimeOfDay(x.(int64))), nil
}
case types.TimeTZFamily:
// The parquet vendor does not support an efficient encoding of TimeTZ
// (i.e. a datetime field and a timezone field), so we must fall back to
// encoding the whole TimeTZ as a string.
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DTimeTZ).TimeTZ.String()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
d, dependsOnCtx, err := tree.ParseDTimeTZ(nil, string(x.([]byte)), time.Microsecond)
if dependsOnCtx {
return nil, errors.New("parsed time depends on context")
}
return d, err
}
case types.IntervalFamily:
// The parquet vendor only supports intervals as a parquet converted type,
// but converted types have been deprecated in the Apache Parquet format.
// https://github.com/fraugster/parquet-go#supported-converted-types
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
return []byte(d.(*tree.DInterval).ValueAsISO8601String()), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
return tree.ParseDInterval(duration.IntervalStyle_ISO_8601, string(x.([]byte)))
}
case types.TimestampFamily:
// Didn't encode this as Microseconds since the unix epoch because of threat
// of overflow. See comment associated with time.Time.UnixMicro().
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
ts := RoundtripStringer(d.(*tree.DTimestamp))
return []byte(ts), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
// return tree.MakeDTimestamp(time.UnixMicro(x.(int64)).UTC(), time.Microsecond)
dtStr := string(x.([]byte))
d, dependsOnCtx, err := tree.ParseDTimestamp(nil, dtStr, time.Microsecond)
if dependsOnCtx {
return nil, errors.New("TimestampTZ depends on context")
}
if err != nil {
return nil, err
}
// Converts the timezone from "loc(+0000)" to "UTC", which are equivalent,
// allowing roundtrip tests to pass.
d.Time = d.Time.UTC()
return d, nil
}
case types.TimestampTZFamily:
// Didn't encode this as Microseconds since the unix epoch because of threat
// of overflow. See comment associated with time.Time.UnixMicro().
populateLogicalStringCol(schemaEl)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
ts := RoundtripStringer(d.(*tree.DTimestampTZ))
return []byte(ts), nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
dtStr := string(x.([]byte))
d, dependsOnCtx, err := tree.ParseDTimestampTZ(nil, dtStr, time.Microsecond)
if dependsOnCtx {
return nil, errors.New("TimestampTZ depends on context")
}
if err != nil {
return nil, err
}
// Converts the timezone from "loc(+0000)" to "UTC", which are equivalent,
// allowing tests to pass.
d.Time = d.Time.UTC()
return d, nil
}
case types.ArrayFamily:
// Define a list such that the parquet schema in json is:
/*
required group colName (LIST){ // parent
repeated group list { // child
required colType element; //grandChild
}
}
*/
// MB figured this out by running toy examples of the fraugster-parquet
// vendor repository for added context, checkout this issue
// https://github.com/fraugster/parquet-go/issues/18
// First, define the grandChild definition, the schema for the array value.
grandChild, err := NewParquetColumn(typ.ArrayContents(), "element", true)
if err != nil {
return col, err
}
// Next define the child definition, required by fraugster-parquet vendor library. Again,
// there's little documentation on this. MB figured this out using a debugger.
child := &parquetschema.ColumnDefinition{}
child.SchemaElement = parquet.NewSchemaElement()
child.SchemaElement.RepetitionType = parquet.FieldRepetitionTypePtr(parquet.
FieldRepetitionType_REPEATED)
child.SchemaElement.Name = "list"
child.Children = []*parquetschema.ColumnDefinition{grandChild.definition}
ngc := int32(len(child.Children))
child.SchemaElement.NumChildren = &ngc
// Finally, define the parent definition.
col.definition.Children = []*parquetschema.ColumnDefinition{child}
nc := int32(len(col.definition.Children))
child.SchemaElement.NumChildren = &nc
schemaEl.LogicalType = parquet.NewLogicalType()
schemaEl.LogicalType.LIST = parquet.NewListType()
schemaEl.ConvertedType = parquet.ConvertedTypePtr(parquet.ConvertedType_LIST)
col.encodeFn = func(d tree.Datum) (interface{}, error) {
datumArr := d.(*tree.DArray)
els := make([]map[string]interface{}, datumArr.Len())
for i, elt := range datumArr.Array {
var el interface{}
if elt.ResolvedType().Family() == types.UnknownFamily {
// skip encoding the datum
} else {
el, err = grandChild.encodeFn(tree.UnwrapDOidWrapper(elt))
if err != nil {
return col, err
}
}
els[i] = map[string]interface{}{"element": el}
}
encEl := map[string]interface{}{"list": els}
return encEl, nil
}
col.DecodeFn = func(x interface{}) (tree.Datum, error) {
// The parquet vendor decodes an array into the native go type
// map[string]interface{}, and the values of the array are stored in the
// "list" key of the map. "list" maps to an array of maps
// []map[string]interface{}, where the ith map contains a single key value
// pair. The key is always "element" and the value is the ith value in the
// array.
// If the array of maps only contains an empty map, the array is empty. This
// occurs IFF "element" is not in the map.
// NB: there's a bug in the fraugster-parquet vendor library around
// reading an ARRAY[NULL],
// https://github.com/fraugster/parquet-go/issues/60 I already verified
// that the vendor's parquet writer can write arrays with null values just
// fine, so EXPORT PARQUET is bug free; however this roundtrip test would
// fail. Ideally, once the bug gets fixed, ARRAY[NULL] will get read as
// the kvp {"element":interface{}} while ARRAY[] will continue to get read
// as an empty map.
datumArr := tree.NewDArray(typ.ArrayContents())
datumArr.Array = []tree.Datum{}
intermediate := x.(map[string]interface{})
vals := intermediate["list"].([]map[string]interface{})
if _, nonEmpty := vals[0]["element"]; !nonEmpty {
if len(vals) > 1 {
return nil, errors.New("array is empty, it shouldn't have a length greater than 1")
}
} else {
for _, elMap := range vals {
itemDatum, err := grandChild.DecodeFn(elMap["element"])
if err != nil {
return nil, err
}
err = datumArr.Append(itemDatum)
if err != nil {
return nil, err
}
}
}
return datumArr, nil
}
default:
return col, errors.Errorf("parquet export does not support the %v type yet", typ.Family())
}
return col, nil
}
func newParquetWriterProcessor(
ctx context.Context,
flowCtx *execinfra.FlowCtx,
processorID int32,
spec execinfrapb.ExportSpec,
post *execinfrapb.PostProcessSpec,
input execinfra.RowSource,
) (execinfra.Processor, error) {
c := &parquetWriterProcessor{
flowCtx: flowCtx,
processorID: processorID,
spec: spec,
input: input,
}
semaCtx := tree.MakeSemaContext()
if err := c.out.Init(ctx, post, colinfo.ExportColumnTypes, &semaCtx, flowCtx.NewEvalCtx()); err != nil {
return nil, err
}
return c, nil
}
type parquetWriterProcessor struct {
flowCtx *execinfra.FlowCtx
processorID int32
spec execinfrapb.ExportSpec
input execinfra.RowSource
out execinfra.ProcOutputHelper
}
var _ execinfra.Processor = &parquetWriterProcessor{}
func (sp *parquetWriterProcessor) OutputTypes() []*types.T {
return sp.out.OutputTypes
}
// MustBeStreaming currently never gets called by the parquetWriterProcessor as
// the function only applies to implementation.
func (sp *parquetWriterProcessor) MustBeStreaming() bool {
return false
}
func (sp *parquetWriterProcessor) Run(ctx context.Context, output execinfra.RowReceiver) {
ctx, span := tracing.ChildSpan(ctx, "parquetWriter")
defer span.Finish()
instanceID := sp.flowCtx.EvalCtx.NodeID.SQLInstanceID()
uniqueID := builtins.GenerateUniqueInt(builtins.ProcessUniqueID(instanceID))
err := func() error {
typs := sp.input.OutputTypes()
sp.input.Start(ctx)
input := execinfra.MakeNoMetadataRowSource(sp.input, output)
alloc := &tree.DatumAlloc{}
datumRowAlloc := make([]tree.Datum, len(sp.spec.ColNames))
var buf bytes.Buffer
sch, err := crlparquet.NewSchema(sp.spec.ColNames, typs)
if err != nil {
return err
}
var compression crlparquet.CompressionCodec
switch sp.spec.Format.Compression {
case roachpb.IOFileFormat_Snappy:
compression = crlparquet.CompressionSnappy
case roachpb.IOFileFormat_Gzip:
compression = crlparquet.CompressionGZIP
case roachpb.IOFileFormat_Auto, roachpb.IOFileFormat_None:
compression = crlparquet.CompressionNone
default:
return pgerror.Newf(pgcode.FeatureNotSupported,
"parquet writer does not support compression format %s", sp.spec.Format.Compression)
}
chunk := 0
done := false
for {
var rows int64
buf.Reset()
var writer *crlparquet.Writer
if sp.flowCtx.TestingKnobs().Export != nil &&
sp.flowCtx.TestingKnobs().Export.(*ExportTestingKnobs).EnableParquetTestMetadata {
writer, err = crlparquet.NewWriterWithReaderMeta(sch, &buf, crlparquet.WithCompressionCodec(compression))
if err != nil {
return err
}
} else {
writer, err = crlparquet.NewWriter(sch, &buf, crlparquet.WithCompressionCodec(compression))
if err != nil {
return err
}
}
for {
// If the bytes.Buffer sink exceeds the target size of a Parquet file, we
// flush before exporting any additional rows.
if int64(buf.Len()) >= sp.spec.ChunkSize {
break
}
if sp.spec.ChunkRows > 0 && rows >= sp.spec.ChunkRows {
break
}
row, err := input.NextRow()
if err != nil {
return err
}
if row == nil {
done = true
break
}
rows++
datumRowAlloc = datumRowAlloc[:0]
for i, ed := range row {
if !ed.IsNull() {
if err := ed.EnsureDecoded(typs[i], alloc); err != nil {
return err
}
}
// If we're encoding a DOidWrapper, then we want to cast
// the wrapped datum. Note that we don't use
// eval.UnwrapDatum since we're not interested in
// evaluating the placeholders.
datumRowAlloc = append(datumRowAlloc, tree.UnwrapDOidWrapper(ed.Datum))
}
if err := writer.AddRow(datumRowAlloc); err != nil {
return err
}
}
if rows < 1 {
break
}
// Flushes data to the buffer.
if err := writer.Close(); err != nil {
return errors.Wrap(err, "failed to close parquet writer")
}
conf, err := cloud.ExternalStorageConfFromURI(sp.spec.Destination, sp.spec.User())
if err != nil {
return err
}
es, err := sp.flowCtx.Cfg.ExternalStorage(ctx, conf)
if err != nil {
return err
}
defer es.Close()
part := fmt.Sprintf("n%d.%d", uniqueID, chunk)
chunk++
filename := fileName(sp.spec, part)
size := buf.Len()
if err := cloud.WriteFile(ctx, es, filename, &buf); err != nil {
return err
}
res := rowenc.EncDatumRow{
rowenc.DatumToEncDatum(
types.String,
tree.NewDString(filename),
),
rowenc.DatumToEncDatum(
types.Int,
tree.NewDInt(tree.DInt(rows)),
),
rowenc.DatumToEncDatum(
types.Int,
tree.NewDInt(tree.DInt(size)),
),
}
cs, err := sp.out.EmitRow(ctx, res, output)
if err != nil {
return err
}
if cs != execinfra.NeedMoreRows {
// We don't return an error here because we want the error (if any) that
// actually caused the consumer to enter a closed/draining state to take precendence.
return nil
}
if done {
break
}
}
return nil
}()
// TODO(dt): pick up tracing info in trailing meta
execinfra.DrainAndClose(
ctx, output, err, func(context.Context, execinfra.RowReceiver) {} /* pushTrailingMeta */, sp.input)
}
// Resume is part of the execinfra.Processor interface.
func (sp *parquetWriterProcessor) Resume(output execinfra.RowReceiver) {
panic("not implemented")
}
func init() {
rowexec.NewParquetWriterProcessor = newParquetWriterProcessor
}