forked from apache/iceberg
/
Parquet.java
1201 lines (1029 loc) · 42.8 KB
/
Parquet.java
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
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.iceberg.parquet;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_COMPRESSION_LEVEL;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_DICT_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_PAGE_ROW_LIMIT;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_PAGE_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.DELETE_PARQUET_ROW_GROUP_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL;
import static org.apache.iceberg.TableProperties.PARQUET_COMPRESSION_LEVEL_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_DICT_SIZE_BYTES_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT;
import static org.apache.iceberg.TableProperties.PARQUET_PAGE_ROW_LIMIT_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_PAGE_SIZE_BYTES_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT_DEFAULT;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES;
import static org.apache.iceberg.TableProperties.PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT;
import java.io.File;
import java.io.IOException;
import java.util.Collection;
import java.util.List;
import java.util.Locale;
import java.util.Map;
import java.util.Objects;
import java.util.function.Function;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.Files;
import org.apache.iceberg.MetricsConfig;
import org.apache.iceberg.PartitionSpec;
import org.apache.iceberg.Schema;
import org.apache.iceberg.SchemaParser;
import org.apache.iceberg.SortOrder;
import org.apache.iceberg.StructLike;
import org.apache.iceberg.Table;
import org.apache.iceberg.avro.AvroSchemaUtil;
import org.apache.iceberg.data.parquet.GenericParquetWriter;
import org.apache.iceberg.deletes.EqualityDeleteWriter;
import org.apache.iceberg.deletes.PositionDeleteWriter;
import org.apache.iceberg.encryption.EncryptionKeyMetadata;
import org.apache.iceberg.exceptions.RuntimeIOException;
import org.apache.iceberg.expressions.Expression;
import org.apache.iceberg.hadoop.HadoopInputFile;
import org.apache.iceberg.hadoop.HadoopOutputFile;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.DataWriter;
import org.apache.iceberg.io.DeleteSchemaUtil;
import org.apache.iceberg.io.FileAppender;
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.OutputFile;
import org.apache.iceberg.mapping.NameMapping;
import org.apache.iceberg.parquet.ParquetValueWriters.PositionDeleteStructWriter;
import org.apache.iceberg.parquet.ParquetValueWriters.StructWriter;
import org.apache.iceberg.relocated.com.google.common.annotations.VisibleForTesting;
import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.relocated.com.google.common.collect.Sets;
import org.apache.iceberg.util.ArrayUtil;
import org.apache.iceberg.util.PropertyUtil;
import org.apache.parquet.HadoopReadOptions;
import org.apache.parquet.ParquetReadOptions;
import org.apache.parquet.avro.AvroReadSupport;
import org.apache.parquet.avro.AvroWriteSupport;
import org.apache.parquet.column.ParquetProperties;
import org.apache.parquet.column.ParquetProperties.WriterVersion;
import org.apache.parquet.hadoop.ParquetFileReader;
import org.apache.parquet.hadoop.ParquetFileWriter;
import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.parquet.hadoop.ParquetReader;
import org.apache.parquet.hadoop.ParquetWriter;
import org.apache.parquet.hadoop.api.ReadSupport;
import org.apache.parquet.hadoop.api.WriteSupport;
import org.apache.parquet.hadoop.metadata.CompressionCodecName;
import org.apache.parquet.schema.MessageType;
public class Parquet {
private Parquet() {}
private static final Collection<String> READ_PROPERTIES_TO_REMOVE =
Sets.newHashSet(
"parquet.read.filter",
"parquet.private.read.filter.predicate",
"parquet.read.support.class");
public static WriteBuilder write(OutputFile file) {
return new WriteBuilder(file);
}
public static class WriteBuilder {
private final OutputFile file;
private final Configuration conf;
private final Map<String, String> metadata = Maps.newLinkedHashMap();
private final Map<String, String> config = Maps.newLinkedHashMap();
private Schema schema = null;
private String name = "table";
private WriteSupport<?> writeSupport = null;
private Function<MessageType, ParquetValueWriter<?>> createWriterFunc = null;
private MetricsConfig metricsConfig = MetricsConfig.getDefault();
private ParquetFileWriter.Mode writeMode = ParquetFileWriter.Mode.CREATE;
private WriterVersion writerVersion = WriterVersion.PARQUET_1_0;
private Function<Map<String, String>, Context> createContextFunc = Context::dataContext;
private WriteBuilder(OutputFile file) {
this.file = file;
if (file instanceof HadoopOutputFile) {
this.conf = new Configuration(((HadoopOutputFile) file).getConf());
} else {
this.conf = new Configuration();
}
}
public WriteBuilder forTable(Table table) {
schema(table.schema());
setAll(table.properties());
metricsConfig(MetricsConfig.forTable(table));
return this;
}
public WriteBuilder schema(Schema newSchema) {
this.schema = newSchema;
return this;
}
public WriteBuilder named(String newName) {
this.name = newName;
return this;
}
public WriteBuilder writeSupport(WriteSupport<?> newWriteSupport) {
this.writeSupport = newWriteSupport;
return this;
}
public WriteBuilder set(String property, String value) {
config.put(property, value);
return this;
}
public WriteBuilder setAll(Map<String, String> properties) {
config.putAll(properties);
return this;
}
public WriteBuilder meta(String property, String value) {
metadata.put(property, value);
return this;
}
public WriteBuilder createWriterFunc(
Function<MessageType, ParquetValueWriter<?>> newCreateWriterFunc) {
this.createWriterFunc = newCreateWriterFunc;
return this;
}
public WriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
this.metricsConfig = newMetricsConfig;
return this;
}
public WriteBuilder overwrite() {
return overwrite(true);
}
public WriteBuilder overwrite(boolean enabled) {
this.writeMode = enabled ? ParquetFileWriter.Mode.OVERWRITE : ParquetFileWriter.Mode.CREATE;
return this;
}
public WriteBuilder writerVersion(WriterVersion version) {
this.writerVersion = version;
return this;
}
@SuppressWarnings("unchecked")
private <T> WriteSupport<T> getWriteSupport(MessageType type) {
if (writeSupport != null) {
return (WriteSupport<T>) writeSupport;
} else {
return new AvroWriteSupport<>(
type,
ParquetAvro.parquetAvroSchema(AvroSchemaUtil.convert(schema, name)),
ParquetAvro.DEFAULT_MODEL);
}
}
/*
* Sets the writer version. Default value is PARQUET_1_0 (v1).
*/
@VisibleForTesting
WriteBuilder withWriterVersion(WriterVersion version) {
this.writerVersion = version;
return this;
}
// supposed to always be a private method used strictly by data and delete write builders
private WriteBuilder createContextFunc(
Function<Map<String, String>, Context> newCreateContextFunc) {
this.createContextFunc = newCreateContextFunc;
return this;
}
public <D> FileAppender<D> build() throws IOException {
Preconditions.checkNotNull(schema, "Schema is required");
Preconditions.checkNotNull(name, "Table name is required and cannot be null");
// add the Iceberg schema to keyValueMetadata
meta("iceberg.schema", SchemaParser.toJson(schema));
// Map Iceberg properties to pass down to the Parquet writer
Context context = createContextFunc.apply(config);
int rowGroupSize = context.rowGroupSize();
int pageSize = context.pageSize();
int pageRowLimit = context.pageRowLimit();
int dictionaryPageSize = context.dictionaryPageSize();
String compressionLevel = context.compressionLevel();
CompressionCodecName codec = context.codec();
int rowGroupCheckMinRecordCount = context.rowGroupCheckMinRecordCount();
int rowGroupCheckMaxRecordCount = context.rowGroupCheckMaxRecordCount();
int bloomFilterMaxBytes = context.bloomFilterMaxBytes();
Map<String, String> columnBloomFilterEnabled = context.columnBloomFilterEnabled();
boolean dictionaryEnabled = context.dictionaryEnabled();
if (compressionLevel != null) {
switch (codec) {
case GZIP:
config.put("zlib.compress.level", compressionLevel);
break;
case BROTLI:
config.put("compression.brotli.quality", compressionLevel);
break;
case ZSTD:
// keep "io.compression.codec.zstd.level" for backwards compatibility
config.put("io.compression.codec.zstd.level", compressionLevel);
config.put("parquet.compression.codec.zstd.level", compressionLevel);
break;
default:
// compression level is not supported; ignore it
}
}
set("parquet.avro.write-old-list-structure", "false");
MessageType type = ParquetSchemaUtil.convert(schema, name);
if (createWriterFunc != null) {
Preconditions.checkArgument(
writeSupport == null, "Cannot write with both write support and Parquet value writer");
for (Map.Entry<String, String> entry : config.entrySet()) {
conf.set(entry.getKey(), entry.getValue());
}
ParquetProperties.Builder propsBuilder =
ParquetProperties.builder()
.withWriterVersion(writerVersion)
.withPageSize(pageSize)
.withPageRowCountLimit(pageRowLimit)
.withDictionaryEncoding(dictionaryEnabled)
.withDictionaryPageSize(dictionaryPageSize)
.withMinRowCountForPageSizeCheck(rowGroupCheckMinRecordCount)
.withMaxRowCountForPageSizeCheck(rowGroupCheckMaxRecordCount)
.withMaxBloomFilterBytes(bloomFilterMaxBytes);
for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) {
String colPath = entry.getKey();
String bloomEnabled = entry.getValue();
propsBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled));
}
ParquetProperties parquetProperties = propsBuilder.build();
return new org.apache.iceberg.parquet.ParquetWriter<>(
conf,
file,
schema,
rowGroupSize,
metadata,
createWriterFunc,
codec,
parquetProperties,
metricsConfig,
writeMode);
} else {
ParquetWriteBuilder<D> parquetWriteBuilder =
new ParquetWriteBuilder<D>(ParquetIO.file(file))
.withWriterVersion(writerVersion)
.setType(type)
.setConfig(config)
.setKeyValueMetadata(metadata)
.setWriteSupport(getWriteSupport(type))
.withCompressionCodec(codec)
.withWriteMode(writeMode)
.withRowGroupSize(rowGroupSize)
.withPageSize(pageSize)
.withPageRowCountLimit(pageRowLimit)
.withDictionaryEncoding(dictionaryEnabled)
.withDictionaryPageSize(dictionaryPageSize);
for (Map.Entry<String, String> entry : columnBloomFilterEnabled.entrySet()) {
String colPath = entry.getKey();
String bloomEnabled = entry.getValue();
parquetWriteBuilder.withBloomFilterEnabled(colPath, Boolean.valueOf(bloomEnabled));
}
return new ParquetWriteAdapter<>(parquetWriteBuilder.build(), metricsConfig);
}
}
private static class Context {
private final int rowGroupSize;
private final int pageSize;
private final int pageRowLimit;
private final int dictionaryPageSize;
private final CompressionCodecName codec;
private final String compressionLevel;
private final int rowGroupCheckMinRecordCount;
private final int rowGroupCheckMaxRecordCount;
private final int bloomFilterMaxBytes;
private final Map<String, String> columnBloomFilterEnabled;
private final boolean dictionaryEnabled;
private Context(
int rowGroupSize,
int pageSize,
int pageRowLimit,
int dictionaryPageSize,
CompressionCodecName codec,
String compressionLevel,
int rowGroupCheckMinRecordCount,
int rowGroupCheckMaxRecordCount,
int bloomFilterMaxBytes,
Map<String, String> columnBloomFilterEnabled,
boolean dictionaryEnabled) {
this.rowGroupSize = rowGroupSize;
this.pageSize = pageSize;
this.pageRowLimit = pageRowLimit;
this.dictionaryPageSize = dictionaryPageSize;
this.codec = codec;
this.compressionLevel = compressionLevel;
this.rowGroupCheckMinRecordCount = rowGroupCheckMinRecordCount;
this.rowGroupCheckMaxRecordCount = rowGroupCheckMaxRecordCount;
this.bloomFilterMaxBytes = bloomFilterMaxBytes;
this.columnBloomFilterEnabled = columnBloomFilterEnabled;
this.dictionaryEnabled = dictionaryEnabled;
}
static Context dataContext(Map<String, String> config) {
int rowGroupSize =
PropertyUtil.propertyAsInt(
config, PARQUET_ROW_GROUP_SIZE_BYTES, PARQUET_ROW_GROUP_SIZE_BYTES_DEFAULT);
Preconditions.checkArgument(rowGroupSize > 0, "Row group size must be > 0");
int pageSize =
PropertyUtil.propertyAsInt(
config, PARQUET_PAGE_SIZE_BYTES, PARQUET_PAGE_SIZE_BYTES_DEFAULT);
Preconditions.checkArgument(pageSize > 0, "Page size must be > 0");
int pageRowLimit =
PropertyUtil.propertyAsInt(
config, PARQUET_PAGE_ROW_LIMIT, PARQUET_PAGE_ROW_LIMIT_DEFAULT);
Preconditions.checkArgument(pageRowLimit > 0, "Page row count limit must be > 0");
int dictionaryPageSize =
PropertyUtil.propertyAsInt(
config, PARQUET_DICT_SIZE_BYTES, PARQUET_DICT_SIZE_BYTES_DEFAULT);
Preconditions.checkArgument(dictionaryPageSize > 0, "Dictionary page size must be > 0");
String codecAsString =
config.getOrDefault(PARQUET_COMPRESSION, PARQUET_COMPRESSION_DEFAULT);
CompressionCodecName codec = toCodec(codecAsString);
String compressionLevel =
config.getOrDefault(PARQUET_COMPRESSION_LEVEL, PARQUET_COMPRESSION_LEVEL_DEFAULT);
int rowGroupCheckMinRecordCount =
PropertyUtil.propertyAsInt(
config,
PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT,
PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT_DEFAULT);
Preconditions.checkArgument(
rowGroupCheckMinRecordCount > 0, "Row group check minimal record count must be > 0");
int rowGroupCheckMaxRecordCount =
PropertyUtil.propertyAsInt(
config,
PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT,
PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT_DEFAULT);
Preconditions.checkArgument(
rowGroupCheckMaxRecordCount > 0, "Row group check maximum record count must be > 0");
Preconditions.checkArgument(
rowGroupCheckMaxRecordCount >= rowGroupCheckMinRecordCount,
"Row group check maximum record count must be >= minimal record count");
int bloomFilterMaxBytes =
PropertyUtil.propertyAsInt(
config, PARQUET_BLOOM_FILTER_MAX_BYTES, PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT);
Preconditions.checkArgument(bloomFilterMaxBytes > 0, "bloom Filter Max Bytes must be > 0");
Map<String, String> columnBloomFilterEnabled =
PropertyUtil.propertiesWithPrefix(config, PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX);
boolean dictionaryEnabled =
PropertyUtil.propertyAsBoolean(config, ParquetOutputFormat.ENABLE_DICTIONARY, true);
return new Context(
rowGroupSize,
pageSize,
pageRowLimit,
dictionaryPageSize,
codec,
compressionLevel,
rowGroupCheckMinRecordCount,
rowGroupCheckMaxRecordCount,
bloomFilterMaxBytes,
columnBloomFilterEnabled,
dictionaryEnabled);
}
static Context deleteContext(Map<String, String> config) {
// default delete config using data config
Context dataContext = dataContext(config);
int rowGroupSize =
PropertyUtil.propertyAsInt(
config, DELETE_PARQUET_ROW_GROUP_SIZE_BYTES, dataContext.rowGroupSize());
Preconditions.checkArgument(rowGroupSize > 0, "Row group size must be > 0");
int pageSize =
PropertyUtil.propertyAsInt(
config, DELETE_PARQUET_PAGE_SIZE_BYTES, dataContext.pageSize());
Preconditions.checkArgument(pageSize > 0, "Page size must be > 0");
int pageRowLimit =
PropertyUtil.propertyAsInt(
config, DELETE_PARQUET_PAGE_ROW_LIMIT, dataContext.pageRowLimit());
Preconditions.checkArgument(pageRowLimit > 0, "Page row count limit must be > 0");
int dictionaryPageSize =
PropertyUtil.propertyAsInt(
config, DELETE_PARQUET_DICT_SIZE_BYTES, dataContext.dictionaryPageSize());
Preconditions.checkArgument(dictionaryPageSize > 0, "Dictionary page size must be > 0");
String codecAsString = config.get(DELETE_PARQUET_COMPRESSION);
CompressionCodecName codec =
codecAsString != null ? toCodec(codecAsString) : dataContext.codec();
String compressionLevel =
config.getOrDefault(DELETE_PARQUET_COMPRESSION_LEVEL, dataContext.compressionLevel());
int rowGroupCheckMinRecordCount =
PropertyUtil.propertyAsInt(
config,
DELETE_PARQUET_ROW_GROUP_CHECK_MIN_RECORD_COUNT,
dataContext.rowGroupCheckMinRecordCount());
Preconditions.checkArgument(
rowGroupCheckMinRecordCount > 0, "Row group check minimal record count must be > 0");
int rowGroupCheckMaxRecordCount =
PropertyUtil.propertyAsInt(
config,
DELETE_PARQUET_ROW_GROUP_CHECK_MAX_RECORD_COUNT,
dataContext.rowGroupCheckMaxRecordCount());
Preconditions.checkArgument(
rowGroupCheckMaxRecordCount > 0, "Row group check maximum record count must be > 0");
Preconditions.checkArgument(
rowGroupCheckMaxRecordCount >= rowGroupCheckMinRecordCount,
"Row group check maximum record count must be >= minimal record count");
int bloomFilterMaxBytes =
PropertyUtil.propertyAsInt(
config, PARQUET_BLOOM_FILTER_MAX_BYTES, PARQUET_BLOOM_FILTER_MAX_BYTES_DEFAULT);
Preconditions.checkArgument(bloomFilterMaxBytes > 0, "bloom Filter Max Bytes must be > 0");
Map<String, String> columnBloomFilterEnabled =
PropertyUtil.propertiesWithPrefix(config, PARQUET_BLOOM_FILTER_COLUMN_ENABLED_PREFIX);
boolean dictionaryEnabled =
PropertyUtil.propertyAsBoolean(config, ParquetOutputFormat.ENABLE_DICTIONARY, true);
return new Context(
rowGroupSize,
pageSize,
pageRowLimit,
dictionaryPageSize,
codec,
compressionLevel,
rowGroupCheckMinRecordCount,
rowGroupCheckMaxRecordCount,
bloomFilterMaxBytes,
columnBloomFilterEnabled,
dictionaryEnabled);
}
private static CompressionCodecName toCodec(String codecAsString) {
try {
return CompressionCodecName.valueOf(codecAsString.toUpperCase(Locale.ENGLISH));
} catch (IllegalArgumentException e) {
throw new IllegalArgumentException("Unsupported compression codec: " + codecAsString);
}
}
int rowGroupSize() {
return rowGroupSize;
}
int pageSize() {
return pageSize;
}
int pageRowLimit() {
return pageRowLimit;
}
int dictionaryPageSize() {
return dictionaryPageSize;
}
CompressionCodecName codec() {
return codec;
}
String compressionLevel() {
return compressionLevel;
}
int rowGroupCheckMinRecordCount() {
return rowGroupCheckMinRecordCount;
}
int rowGroupCheckMaxRecordCount() {
return rowGroupCheckMaxRecordCount;
}
int bloomFilterMaxBytes() {
return bloomFilterMaxBytes;
}
Map<String, String> columnBloomFilterEnabled() {
return columnBloomFilterEnabled;
}
boolean dictionaryEnabled() {
return dictionaryEnabled;
}
}
}
public static DataWriteBuilder writeData(OutputFile file) {
return new DataWriteBuilder(file);
}
public static class DataWriteBuilder {
private final WriteBuilder appenderBuilder;
private final String location;
private PartitionSpec spec = null;
private StructLike partition = null;
private EncryptionKeyMetadata keyMetadata = null;
private SortOrder sortOrder = null;
private DataWriteBuilder(OutputFile file) {
this.appenderBuilder = write(file);
this.location = file.location();
}
public DataWriteBuilder forTable(Table table) {
schema(table.schema());
withSpec(table.spec());
setAll(table.properties());
metricsConfig(MetricsConfig.forTable(table));
return this;
}
public DataWriteBuilder schema(Schema newSchema) {
appenderBuilder.schema(newSchema);
return this;
}
public DataWriteBuilder set(String property, String value) {
appenderBuilder.set(property, value);
return this;
}
public DataWriteBuilder setAll(Map<String, String> properties) {
appenderBuilder.setAll(properties);
return this;
}
public DataWriteBuilder meta(String property, String value) {
appenderBuilder.meta(property, value);
return this;
}
public DataWriteBuilder overwrite() {
return overwrite(true);
}
public DataWriteBuilder overwrite(boolean enabled) {
appenderBuilder.overwrite(enabled);
return this;
}
public DataWriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
appenderBuilder.metricsConfig(newMetricsConfig);
return this;
}
public DataWriteBuilder createWriterFunc(
Function<MessageType, ParquetValueWriter<?>> newCreateWriterFunc) {
appenderBuilder.createWriterFunc(newCreateWriterFunc);
return this;
}
public DataWriteBuilder withSpec(PartitionSpec newSpec) {
this.spec = newSpec;
return this;
}
public DataWriteBuilder withPartition(StructLike newPartition) {
this.partition = newPartition;
return this;
}
public DataWriteBuilder withKeyMetadata(EncryptionKeyMetadata metadata) {
this.keyMetadata = metadata;
return this;
}
public DataWriteBuilder withSortOrder(SortOrder newSortOrder) {
this.sortOrder = newSortOrder;
return this;
}
public <T> DataWriter<T> build() throws IOException {
Preconditions.checkArgument(spec != null, "Cannot create data writer without spec");
Preconditions.checkArgument(
spec.isUnpartitioned() || partition != null,
"Partition must not be null when creating data writer for partitioned spec");
FileAppender<T> fileAppender = appenderBuilder.build();
return new DataWriter<>(
fileAppender, FileFormat.PARQUET, location, spec, partition, keyMetadata, sortOrder);
}
}
public static DeleteWriteBuilder writeDeletes(OutputFile file) {
return new DeleteWriteBuilder(file);
}
public static class DeleteWriteBuilder {
private final WriteBuilder appenderBuilder;
private final String location;
private Function<MessageType, ParquetValueWriter<?>> createWriterFunc = null;
private Schema rowSchema = null;
private PartitionSpec spec = null;
private StructLike partition = null;
private EncryptionKeyMetadata keyMetadata = null;
private int[] equalityFieldIds = null;
private SortOrder sortOrder;
private Function<CharSequence, ?> pathTransformFunc = Function.identity();
private DeleteWriteBuilder(OutputFile file) {
this.appenderBuilder = write(file);
this.location = file.location();
}
public DeleteWriteBuilder forTable(Table table) {
rowSchema(table.schema());
withSpec(table.spec());
setAll(table.properties());
metricsConfig(MetricsConfig.forTable(table));
return this;
}
public DeleteWriteBuilder set(String property, String value) {
appenderBuilder.set(property, value);
return this;
}
public DeleteWriteBuilder setAll(Map<String, String> properties) {
appenderBuilder.setAll(properties);
return this;
}
public DeleteWriteBuilder meta(String property, String value) {
appenderBuilder.meta(property, value);
return this;
}
public DeleteWriteBuilder overwrite() {
return overwrite(true);
}
public DeleteWriteBuilder overwrite(boolean enabled) {
appenderBuilder.overwrite(enabled);
return this;
}
public DeleteWriteBuilder metricsConfig(MetricsConfig newMetricsConfig) {
appenderBuilder.metricsConfig(newMetricsConfig);
return this;
}
public DeleteWriteBuilder createWriterFunc(
Function<MessageType, ParquetValueWriter<?>> newCreateWriterFunc) {
this.createWriterFunc = newCreateWriterFunc;
return this;
}
public DeleteWriteBuilder rowSchema(Schema newSchema) {
this.rowSchema = newSchema;
return this;
}
public DeleteWriteBuilder withSpec(PartitionSpec newSpec) {
this.spec = newSpec;
return this;
}
public DeleteWriteBuilder withPartition(StructLike key) {
this.partition = key;
return this;
}
public DeleteWriteBuilder withKeyMetadata(EncryptionKeyMetadata metadata) {
this.keyMetadata = metadata;
return this;
}
public DeleteWriteBuilder equalityFieldIds(List<Integer> fieldIds) {
this.equalityFieldIds = ArrayUtil.toIntArray(fieldIds);
return this;
}
public DeleteWriteBuilder equalityFieldIds(int... fieldIds) {
this.equalityFieldIds = fieldIds;
return this;
}
public DeleteWriteBuilder transformPaths(Function<CharSequence, ?> newPathTransformFunc) {
this.pathTransformFunc = newPathTransformFunc;
return this;
}
public DeleteWriteBuilder withSortOrder(SortOrder newSortOrder) {
this.sortOrder = newSortOrder;
return this;
}
public <T> EqualityDeleteWriter<T> buildEqualityWriter() throws IOException {
Preconditions.checkState(
rowSchema != null, "Cannot create equality delete file without a schema");
Preconditions.checkState(
equalityFieldIds != null, "Cannot create equality delete file without delete field ids");
Preconditions.checkState(
createWriterFunc != null,
"Cannot create equality delete file unless createWriterFunc is set");
Preconditions.checkArgument(
spec != null, "Spec must not be null when creating equality delete writer");
Preconditions.checkArgument(
spec.isUnpartitioned() || partition != null,
"Partition must not be null for partitioned writes");
meta("delete-type", "equality");
meta(
"delete-field-ids",
IntStream.of(equalityFieldIds)
.mapToObj(Objects::toString)
.collect(Collectors.joining(", ")));
// the appender uses the row schema without extra columns
appenderBuilder.schema(rowSchema);
appenderBuilder.createWriterFunc(createWriterFunc);
appenderBuilder.createContextFunc(WriteBuilder.Context::deleteContext);
return new EqualityDeleteWriter<>(
appenderBuilder.build(),
FileFormat.PARQUET,
location,
spec,
partition,
keyMetadata,
sortOrder,
equalityFieldIds);
}
public <T> PositionDeleteWriter<T> buildPositionWriter() throws IOException {
Preconditions.checkState(
equalityFieldIds == null, "Cannot create position delete file using delete field ids");
Preconditions.checkArgument(
spec != null, "Spec must not be null when creating position delete writer");
Preconditions.checkArgument(
spec.isUnpartitioned() || partition != null,
"Partition must not be null for partitioned writes");
Preconditions.checkArgument(
rowSchema == null || createWriterFunc != null,
"Create function should be provided if we write row data");
meta("delete-type", "position");
if (rowSchema != null && createWriterFunc != null) {
// the appender uses the row schema wrapped with position fields
appenderBuilder.schema(DeleteSchemaUtil.posDeleteSchema(rowSchema));
appenderBuilder.createWriterFunc(
parquetSchema -> {
ParquetValueWriter<?> writer = createWriterFunc.apply(parquetSchema);
if (writer instanceof StructWriter) {
return new PositionDeleteStructWriter<T>(
(StructWriter<?>) writer, pathTransformFunc);
} else {
throw new UnsupportedOperationException(
"Cannot wrap writer for position deletes: " + writer.getClass());
}
});
} else {
appenderBuilder.schema(DeleteSchemaUtil.pathPosSchema());
// We ignore the 'createWriterFunc' and 'rowSchema' even if is provided, since we do not
// write row data itself
appenderBuilder.createWriterFunc(
parquetSchema ->
new PositionDeleteStructWriter<T>(
(StructWriter<?>) GenericParquetWriter.buildWriter(parquetSchema),
Function.identity()));
}
appenderBuilder.createContextFunc(WriteBuilder.Context::deleteContext);
return new PositionDeleteWriter<>(
appenderBuilder.build(), FileFormat.PARQUET, location, spec, partition, keyMetadata);
}
}
private static class ParquetWriteBuilder<T>
extends ParquetWriter.Builder<T, ParquetWriteBuilder<T>> {
private Map<String, String> keyValueMetadata = Maps.newHashMap();
private Map<String, String> config = Maps.newHashMap();
private MessageType type;
private WriteSupport<T> writeSupport;
private ParquetWriteBuilder(org.apache.parquet.io.OutputFile path) {
super(path);
}
@Override
protected ParquetWriteBuilder<T> self() {
return this;
}
public ParquetWriteBuilder<T> setKeyValueMetadata(Map<String, String> keyValueMetadata) {
this.keyValueMetadata = keyValueMetadata;
return self();
}
public ParquetWriteBuilder<T> setConfig(Map<String, String> config) {
this.config = config;
return self();
}
public ParquetWriteBuilder<T> setType(MessageType type) {
this.type = type;
return self();
}
public ParquetWriteBuilder<T> setWriteSupport(WriteSupport<T> writeSupport) {
this.writeSupport = writeSupport;
return self();
}
@Override
protected WriteSupport<T> getWriteSupport(Configuration configuration) {
for (Map.Entry<String, String> entry : config.entrySet()) {
configuration.set(entry.getKey(), entry.getValue());
}
return new ParquetWriteSupport<>(type, keyValueMetadata, writeSupport);
}
}
public static ReadBuilder read(InputFile file) {
return new ReadBuilder(file);
}
public static class ReadBuilder {
private final InputFile file;
private final Map<String, String> properties = Maps.newHashMap();
private Long start = null;
private Long length = null;
private Schema schema = null;
private Expression filter = null;
private ReadSupport<?> readSupport = null;
private Function<MessageType, VectorizedReader<?>> batchedReaderFunc = null;
private Function<MessageType, ParquetValueReader<?>> readerFunc = null;
private boolean filterRecords = true;
private boolean caseSensitive = true;
private boolean callInit = false;
private boolean reuseContainers = false;
private int maxRecordsPerBatch = 10000;
private NameMapping nameMapping = null;
private ReadBuilder(InputFile file) {
this.file = file;
}
/**
* Restricts the read to the given range: [start, start + length).
*
* @param newStart the start position for this read
* @param newLength the length of the range this read should scan
* @return this builder for method chaining
*/
public ReadBuilder split(long newStart, long newLength) {
this.start = newStart;
this.length = newLength;
return this;
}
public ReadBuilder project(Schema newSchema) {
this.schema = newSchema;
return this;
}
public ReadBuilder caseInsensitive() {
return caseSensitive(false);
}
public ReadBuilder caseSensitive(boolean newCaseSensitive) {
this.caseSensitive = newCaseSensitive;
return this;
}
public ReadBuilder filterRecords(boolean newFilterRecords) {
this.filterRecords = newFilterRecords;
return this;
}
public ReadBuilder filter(Expression newFilter) {
this.filter = newFilter;
return this;
}
public ReadBuilder readSupport(ReadSupport<?> newFilterSupport) {
this.readSupport = newFilterSupport;
return this;
}
public ReadBuilder createReaderFunc(
Function<MessageType, ParquetValueReader<?>> newReaderFunction) {
Preconditions.checkArgument(
this.batchedReaderFunc == null,
"Reader function cannot be set since the batched version is already set");
this.readerFunc = newReaderFunction;
return this;
}
public ReadBuilder createBatchedReaderFunc(Function<MessageType, VectorizedReader<?>> func) {
Preconditions.checkArgument(
this.readerFunc == null,
"Batched reader function cannot be set since the non-batched version is already set");
this.batchedReaderFunc = func;
return this;
}
public ReadBuilder set(String key, String value) {
properties.put(key, value);
return this;
}