forked from apache/datafusion-comet
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathNativeBatchReader.java
More file actions
1093 lines (985 loc) · 41.1 KB
/
NativeBatchReader.java
File metadata and controls
1093 lines (985 loc) · 41.1 KB
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
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.comet.parquet;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
import java.io.IOException;
import java.lang.reflect.InvocationTargetException;
import java.lang.reflect.Method;
import java.net.URI;
import java.net.URISyntaxException;
import java.nio.channels.Channels;
import java.util.*;
import java.util.stream.Collectors;
import scala.Option;
import scala.collection.Seq;
import scala.collection.mutable.Buffer;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
import org.apache.arrow.vector.ipc.WriteChannel;
import org.apache.arrow.vector.ipc.message.MessageSerializer;
import org.apache.arrow.vector.types.pojo.Schema;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.mapreduce.InputSplit;
import org.apache.hadoop.mapreduce.RecordReader;
import org.apache.hadoop.mapreduce.TaskAttemptContext;
import org.apache.parquet.HadoopReadOptions;
import org.apache.parquet.ParquetReadOptions;
import org.apache.parquet.Preconditions;
import org.apache.parquet.column.ColumnDescriptor;
import org.apache.parquet.hadoop.metadata.BlockMetaData;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
import org.apache.parquet.schema.GroupType;
import org.apache.parquet.schema.MessageType;
import org.apache.parquet.schema.Type;
import org.apache.spark.TaskContext;
import org.apache.spark.TaskContext$;
import org.apache.spark.executor.TaskMetrics;
import org.apache.spark.sql.catalyst.InternalRow;
import org.apache.spark.sql.comet.parquet.CometParquetReadSupport;
import org.apache.spark.sql.comet.util.Utils$;
import org.apache.spark.sql.errors.QueryExecutionErrors;
import org.apache.spark.sql.execution.datasources.PartitionedFile;
import org.apache.spark.sql.execution.datasources.parquet.ParquetColumn;
import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils;
import org.apache.spark.sql.execution.metric.SQLMetric;
import org.apache.spark.sql.internal.SQLConf;
import org.apache.spark.sql.types.*;
import org.apache.spark.sql.vectorized.ColumnarBatch;
import org.apache.spark.util.AccumulatorV2;
import org.apache.comet.CometConf;
import org.apache.comet.CometSchemaImporter;
import org.apache.comet.objectstore.NativeConfig;
import org.apache.comet.shims.ShimBatchReader;
import org.apache.comet.shims.ShimFileFormat;
import org.apache.comet.vector.CometVector;
import org.apache.comet.vector.NativeUtil;
import static scala.jdk.javaapi.CollectionConverters.asJava;
/**
* A vectorized Parquet reader that reads a Parquet file in a batched fashion.
*
* <p>Example of how to use this:
*
* <pre>
* BatchReader reader = new BatchReader(parquetFile, batchSize);
* try {
* reader.init();
* while (reader.readBatch()) {
* ColumnarBatch batch = reader.currentBatch();
* // consume the batch
* }
* } finally { // resources associated with the reader should be released
* reader.close();
* }
* </pre>
*/
public class NativeBatchReader extends RecordReader<Void, ColumnarBatch> implements Closeable {
/**
* A class that contains the necessary file information for reading a Parquet file. This class
* provides an abstraction over PartitionedFile properties.
*/
public static class FileInfo {
private final long start;
private final long length;
private final String filePath;
private final long fileSize;
public FileInfo(long start, long length, String filePath, long fileSize)
throws URISyntaxException {
this.start = start;
this.length = length;
URI uri = new Path(filePath).toUri();
if (uri.getScheme() == null) {
uri = new Path("file://" + filePath).toUri();
}
this.filePath = uri.toString();
this.fileSize = fileSize;
}
public static FileInfo fromPartitionedFile(PartitionedFile file) throws URISyntaxException {
return new FileInfo(file.start(), file.length(), file.filePath().toString(), file.fileSize());
}
public long start() {
return start;
}
public long length() {
return length;
}
public String filePath() {
return filePath;
}
public long fileSize() {
return fileSize;
}
public URI pathUri() throws URISyntaxException {
return new URI(filePath);
}
}
private static final Logger LOG = LoggerFactory.getLogger(NativeBatchReader.class);
protected static final BufferAllocator ALLOCATOR = new RootAllocator();
private NativeUtil nativeUtil = new NativeUtil();
protected Configuration conf;
protected int capacity;
protected boolean isCaseSensitive;
protected boolean useFieldId;
protected boolean ignoreMissingIds;
protected StructType partitionSchema;
protected InternalRow partitionValues;
protected PartitionedFile file;
protected FileInfo fileInfo;
protected final Map<String, SQLMetric> metrics;
// Unfortunately CometMetricNode is from the "spark" package and cannot be used directly here
// TODO: Move it to common package?
protected Object metricsNode = null;
protected StructType sparkSchema;
protected StructType dataSchema;
MessageType fileSchema;
protected MessageType requestedSchema;
protected CometVector[] vectors;
protected AbstractColumnReader[] columnReaders;
protected CometSchemaImporter importer;
protected ColumnarBatch currentBatch;
// private FileReader fileReader;
protected boolean[] missingColumns;
protected boolean isInitialized;
protected ParquetMetadata footer;
protected byte[] nativeFilter;
protected AbstractColumnReader[] preInitializedReaders;
private ParquetColumn parquetColumn;
/**
* Map from field name to spark schema index for efficient lookups during batch loading. Built
* once during initialization and reused across all batch loads.
*/
private Map<String, Integer> sparkFieldIndexMap;
/**
* Whether the native scan should always return decimal represented by 128 bits, regardless of its
* precision. Normally, this should be true if native execution is enabled, since Arrow compute
* kernels doesn't support 32 and 64 bit decimals yet.
*/
// TODO: (ARROW NATIVE)
private boolean useDecimal128;
/**
* Whether to return dates/timestamps that were written with legacy hybrid (Julian + Gregorian)
* calendar as it is. If this is true, Comet will return them as it is, instead of rebasing them
* to the new Proleptic Gregorian calendar. If this is false, Comet will throw exceptions when
* seeing these dates/timestamps.
*/
// TODO: (ARROW NATIVE)
protected boolean useLegacyDateTimestamp;
/** The TaskContext object for executing this task. */
private final TaskContext taskContext;
private long totalRowCount = 0;
private long handle;
// Protected no-arg constructor for subclasses
protected NativeBatchReader() {
this.taskContext = TaskContext$.MODULE$.get();
this.metrics = new HashMap<>();
}
// Only for testing
public NativeBatchReader(String file, int capacity) {
this(file, capacity, null, null);
}
// Only for testing
public NativeBatchReader(
String file, int capacity, StructType partitionSchema, InternalRow partitionValues) {
this(new Configuration(), file, capacity, partitionSchema, partitionValues);
}
// Only for testing
public NativeBatchReader(
Configuration conf,
String file,
int capacity,
StructType partitionSchema,
InternalRow partitionValues) {
this.conf = conf;
this.capacity = capacity;
this.isCaseSensitive = false;
this.useFieldId = false;
this.ignoreMissingIds = false;
this.partitionSchema = partitionSchema;
this.partitionValues = partitionValues;
this.file = ShimBatchReader.newPartitionedFile(partitionValues, file);
this.metrics = new HashMap<>();
this.taskContext = TaskContext$.MODULE$.get();
}
private NativeBatchReader(AbstractColumnReader[] columnReaders) {
// Todo: set useDecimal128 and useLazyMaterialization
int numColumns = columnReaders.length;
this.columnReaders = new AbstractColumnReader[numColumns];
vectors = new CometVector[numColumns];
currentBatch = new ColumnarBatch(vectors);
// This constructor is used by Iceberg only. The columnReaders are
// initialized in Iceberg, so no need to call the init()
isInitialized = true;
this.taskContext = TaskContext$.MODULE$.get();
this.metrics = new HashMap<>();
}
NativeBatchReader(
Configuration conf,
PartitionedFile inputSplit,
ParquetMetadata footer,
byte[] nativeFilter,
int capacity,
StructType sparkSchema,
StructType dataSchema,
boolean isCaseSensitive,
boolean useFieldId,
boolean ignoreMissingIds,
boolean useLegacyDateTimestamp,
StructType partitionSchema,
InternalRow partitionValues,
Map<String, SQLMetric> metrics,
Object metricsNode) {
this.conf = conf;
this.capacity = capacity;
this.sparkSchema = sparkSchema;
this.dataSchema = dataSchema;
this.isCaseSensitive = isCaseSensitive;
this.useFieldId = useFieldId;
this.ignoreMissingIds = ignoreMissingIds;
this.useLegacyDateTimestamp = useLegacyDateTimestamp;
this.partitionSchema = partitionSchema;
this.partitionValues = partitionValues;
this.file = inputSplit;
this.footer = footer;
this.nativeFilter = nativeFilter;
this.metrics = metrics;
this.metricsNode = metricsNode;
this.taskContext = TaskContext$.MODULE$.get();
}
/** Alternate constructor that accepts FileInfo instead of PartitionedFile. */
NativeBatchReader(
Configuration conf,
FileInfo fileInfo,
ParquetMetadata footer,
byte[] nativeFilter,
int capacity,
StructType sparkSchema,
StructType dataSchema,
boolean isCaseSensitive,
boolean useFieldId,
boolean ignoreMissingIds,
boolean useLegacyDateTimestamp,
StructType partitionSchema,
InternalRow partitionValues,
Map<String, SQLMetric> metrics,
Object metricsNode) {
this.conf = conf;
this.capacity = capacity;
this.sparkSchema = sparkSchema;
this.dataSchema = dataSchema;
this.isCaseSensitive = isCaseSensitive;
this.useFieldId = useFieldId;
this.ignoreMissingIds = ignoreMissingIds;
this.useLegacyDateTimestamp = useLegacyDateTimestamp;
this.partitionSchema = partitionSchema;
this.partitionValues = partitionValues;
this.fileInfo = fileInfo;
this.footer = footer;
this.nativeFilter = nativeFilter;
this.metrics = metrics;
this.metricsNode = metricsNode;
this.taskContext = TaskContext$.MODULE$.get();
}
/**
* Initialize this reader. The reason we don't do it in the constructor is that we want to close
* any resource hold by this reader when error happens during the initialization.
*/
public void init() throws Throwable {
useDecimal128 =
conf.getBoolean(
CometConf.COMET_USE_DECIMAL_128().key(),
(Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get());
// Use fileInfo if available, otherwise fall back to file
long start = fileInfo != null ? fileInfo.start() : file.start();
long length = fileInfo != null ? fileInfo.length() : file.length();
String filePath = fileInfo != null ? fileInfo.filePath() : file.filePath().toString();
long fileSize = fileInfo != null ? fileInfo.fileSize() : file.fileSize();
URI pathUri = fileInfo != null ? fileInfo.pathUri() : file.pathUri();
ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath));
if (start >= 0 && length >= 0) {
builder = builder.withRange(start, start + length);
}
ParquetReadOptions readOptions = builder.build();
Map<String, String> objectStoreOptions =
asJava(NativeConfig.extractObjectStoreOptions(conf, pathUri));
// TODO: enable off-heap buffer when they are ready
ReadOptions cometReadOptions = ReadOptions.builder(conf).build();
Path path = new Path(new URI(filePath));
try (FileReader fileReader =
new FileReader(
CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics)) {
requestedSchema = footer.getFileMetaData().getSchema();
fileSchema = requestedSchema;
if (sparkSchema == null) {
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(conf);
sparkSchema = converter.convert(requestedSchema);
} else {
requestedSchema =
CometParquetReadSupport.clipParquetSchema(
requestedSchema, sparkSchema, isCaseSensitive, useFieldId, ignoreMissingIds);
if (requestedSchema.getFieldCount() != sparkSchema.size()) {
throw new IllegalArgumentException(
String.format(
"Spark schema has %d columns while " + "Parquet schema has %d columns",
sparkSchema.size(), requestedSchema.getFieldCount()));
}
}
boolean caseSensitive =
conf.getBoolean(
SQLConf.CASE_SENSITIVE().key(),
(boolean) SQLConf.CASE_SENSITIVE().defaultValue().get());
// rename spark fields based on field_id so name of spark schema field matches the parquet
// field name
if (useFieldId && ParquetUtils.hasFieldIds(sparkSchema)) {
sparkSchema =
getSparkSchemaByFieldId(sparkSchema, requestedSchema.asGroupType(), caseSensitive);
}
this.parquetColumn = getParquetColumn(requestedSchema, this.sparkSchema);
// Create Column readers
List<Type> fields = requestedSchema.getFields();
List<Type> fileFields = fileSchema.getFields();
ParquetColumn[] parquetFields =
asJava(parquetColumn.children()).toArray(new ParquetColumn[0]);
int numColumns = fields.size();
if (partitionSchema != null) numColumns += partitionSchema.size();
columnReaders = new AbstractColumnReader[numColumns];
// Initialize missing columns and use null vectors for them
missingColumns = new boolean[numColumns];
// We do not need the column index of the row index; but this method has the
// side effect of throwing an exception if a column with the same name is
// found which we do want (spark unit tests explicitly test for that).
ShimFileFormat.findRowIndexColumnIndexInSchema(sparkSchema);
StructField[] nonPartitionFields = sparkSchema.fields();
boolean hasRowIndexColumn = false;
// Ranges of rows to read (needed iff row indexes are being read)
List<BlockMetaData> blocks =
FileReader.filterRowGroups(readOptions, footer.getBlocks(), fileReader);
totalRowCount = fileReader.getFilteredRecordCount();
if (totalRowCount == 0) {
// all the data is filtered out.
isInitialized = true;
return;
}
long[] starts = new long[blocks.size()];
long[] lengths = new long[blocks.size()];
int blockIndex = 0;
for (BlockMetaData block : blocks) {
long blockStart = block.getStartingPos();
long blockLength = block.getCompressedSize();
starts[blockIndex] = blockStart;
lengths[blockIndex] = blockLength;
blockIndex++;
}
for (int i = 0; i < fields.size(); i++) {
Type field = fields.get(i);
Optional<Type> optFileField =
fileFields.stream().filter(f -> f.getName().equals(field.getName())).findFirst();
if (nonPartitionFields[i].name().equals(ShimFileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME())) {
// Values of ROW_INDEX_TEMPORARY_COLUMN_NAME column are always populated with
// generated row indexes, rather than read from the file.
// TODO(SPARK-40059): Allow users to include columns named
// FileFormat.ROW_INDEX_TEMPORARY_COLUMN_NAME in their schemas.
long[] rowIndices = FileReader.getRowIndices(blocks);
columnReaders[i] =
new ArrowRowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices);
hasRowIndexColumn = true;
missingColumns[i] = true;
} else if (optFileField.isPresent()) {
// The column we are reading may be a complex type in which case we check if each field in
// the requested type is in the file type (and the same data type)
// This makes the same check as Spark's VectorizedParquetReader
checkColumn(parquetFields[i]);
missingColumns[i] = false;
} else {
if (preInitializedReaders != null
&& i < preInitializedReaders.length
&& preInitializedReaders[i] != null) {
columnReaders[i] = preInitializedReaders[i];
missingColumns[i] = true;
} else {
if (field.getRepetition() == Type.Repetition.REQUIRED) {
throw new IOException(
"Required column '"
+ field.getName()
+ "' is missing"
+ " in data file "
+ filePath);
}
if (field.isPrimitive()) {
ArrowConstantColumnReader reader =
new ArrowConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128);
columnReaders[i] = reader;
missingColumns[i] = true;
} else {
// the column requested is not in the file, but the native reader can handle that
// and will return nulls for all rows requested
missingColumns[i] = false;
}
}
}
}
// Initialize constant readers for partition columns
if (partitionSchema != null) {
StructField[] partitionFields = partitionSchema.fields();
for (int i = fields.size(); i < columnReaders.length; i++) {
int fieldIndex = i - fields.size();
StructField field = partitionFields[fieldIndex];
ArrowConstantColumnReader reader =
new ArrowConstantColumnReader(
field, capacity, partitionValues, fieldIndex, useDecimal128);
columnReaders[i] = reader;
}
}
vectors = new CometVector[numColumns];
currentBatch = new ColumnarBatch(vectors);
// For test purpose only
// If the last external accumulator is `NumRowGroupsAccumulator`, the row group number to read
// will be updated to the accumulator. So we can check if the row groups are filtered or not
// in test case.
// Note that this tries to get thread local TaskContext object, if this is called at other
// thread, it won't update the accumulator.
if (taskContext != null) {
Option<AccumulatorV2<?, ?>> accu = getTaskAccumulator(taskContext.taskMetrics());
if (accu.isDefined() && accu.get().getClass().getSimpleName().equals("NumRowGroupsAcc")) {
@SuppressWarnings("unchecked")
AccumulatorV2<Integer, Integer> intAccum = (AccumulatorV2<Integer, Integer>) accu.get();
intAccum.add(blocks.size());
}
}
boolean encryptionEnabled = CometParquetUtils.encryptionEnabled(conf);
// Create keyUnwrapper if encryption is enabled
CometFileKeyUnwrapper keyUnwrapper = null;
if (encryptionEnabled) {
keyUnwrapper = new CometFileKeyUnwrapper();
keyUnwrapper.storeDecryptionKeyRetriever(filePath, conf);
}
// Filter out columns with preinitialized readers from sparkSchema before making the
// call to native
if (preInitializedReaders != null) {
StructType filteredSchema = new StructType();
StructField[] sparkFields = sparkSchema.fields();
// Build name map for efficient lookups
Map<String, Type> fileFieldNameMap =
caseSensitive
? buildCaseSensitiveNameMap(fileFields)
: buildCaseInsensitiveNameMap(fileFields);
for (int i = 0; i < sparkFields.length; i++) {
// Keep the column if:
// 1. It doesn't have a preinitialized reader, OR
// 2. It has a preinitialized reader but exists in fileSchema
boolean hasPreInitializedReader =
i < preInitializedReaders.length && preInitializedReaders[i] != null;
String fieldName =
caseSensitive
? sparkFields[i].name()
: sparkFields[i].name().toLowerCase(Locale.ROOT);
boolean existsInFileSchema = fileFieldNameMap.containsKey(fieldName);
if (!hasPreInitializedReader || existsInFileSchema) {
filteredSchema = filteredSchema.add(sparkFields[i]);
}
}
sparkSchema = filteredSchema;
}
// Native code uses "UTC" always as the timeZoneId when converting from spark to arrow schema.
String timeZoneId = "UTC";
Schema arrowSchema = Utils$.MODULE$.toArrowSchema(sparkSchema, timeZoneId);
byte[] serializedRequestedArrowSchema = serializeArrowSchema(arrowSchema);
Schema dataArrowSchema = Utils$.MODULE$.toArrowSchema(dataSchema, timeZoneId);
byte[] serializedDataArrowSchema = serializeArrowSchema(dataArrowSchema);
int batchSize =
conf.getInt(
CometConf.COMET_BATCH_SIZE().key(),
(Integer) CometConf.COMET_BATCH_SIZE().defaultValue().get());
this.handle =
Native.initRecordBatchReader(
filePath,
fileSize,
starts,
lengths,
hasRowIndexColumn ? null : nativeFilter,
serializedRequestedArrowSchema,
serializedDataArrowSchema,
timeZoneId,
batchSize,
caseSensitive,
objectStoreOptions,
keyUnwrapper,
metricsNode);
// Build spark field index map for efficient lookups during batch loading
StructField[] sparkFields = sparkSchema.fields();
sparkFieldIndexMap = new HashMap<>();
for (int j = 0; j < sparkFields.length; j++) {
String fieldName =
caseSensitive ? sparkFields[j].name() : sparkFields[j].name().toLowerCase(Locale.ROOT);
sparkFieldIndexMap.put(fieldName, j);
}
}
isInitialized = true;
}
private ParquetColumn getParquetColumn(MessageType schema, StructType sparkSchema) {
// We use a different config from the config that is passed in.
// This follows the setting used in Spark's SpecificParquetRecordReaderBase
Configuration config = new Configuration(conf);
config.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING().key(), false);
config.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP().key(), false);
config.setBoolean(SQLConf.CASE_SENSITIVE().key(), false);
config.setBoolean(SQLConf.PARQUET_INFER_TIMESTAMP_NTZ_ENABLED().key(), false);
config.setBoolean(SQLConf.LEGACY_PARQUET_NANOS_AS_LONG().key(), false);
ParquetToSparkSchemaConverter converter = new ParquetToSparkSchemaConverter(config);
return converter.convertParquetColumn(schema, Option.apply(sparkSchema));
}
private Map<Integer, List<Type>> getIdToParquetFieldMap(GroupType type) {
return type.getFields().stream()
.filter(f -> f.getId() != null)
.collect(Collectors.groupingBy(f -> f.getId().intValue()));
}
private Map<String, List<Type>> getCaseSensitiveParquetFieldMap(GroupType schema) {
return schema.getFields().stream().collect(Collectors.toMap(Type::getName, Arrays::asList));
}
private Map<String, List<Type>> getCaseInsensitiveParquetFieldMap(GroupType schema) {
return schema.getFields().stream()
.collect(Collectors.groupingBy(f -> f.getName().toLowerCase(Locale.ROOT)));
}
private Map<String, Type> buildCaseSensitiveNameMap(List<Type> types) {
return types.stream().collect(Collectors.toMap(Type::getName, t -> t));
}
private Map<String, Type> buildCaseInsensitiveNameMap(List<Type> types) {
return types.stream()
.collect(Collectors.toMap(t -> t.getName().toLowerCase(Locale.ROOT), t -> t));
}
private Type getMatchingParquetFieldById(
StructField f,
Map<Integer, List<Type>> idToParquetFieldMap,
Map<String, List<Type>> nameToParquetFieldMap,
boolean isCaseSensitive) {
List<Type> matched = null;
int fieldId = 0;
if (ParquetUtils.hasFieldId(f)) {
fieldId = ParquetUtils.getFieldId(f);
matched = idToParquetFieldMap.get(fieldId);
} else {
String fieldName = isCaseSensitive ? f.name() : f.name().toLowerCase(Locale.ROOT);
matched = nameToParquetFieldMap.get(fieldName);
}
if (matched == null || matched.isEmpty()) {
return null;
}
if (matched.size() > 1) {
// Need to fail if there is ambiguity, i.e. more than one field is matched
String parquetTypesString =
matched.stream().map(Type::getName).collect(Collectors.joining("[", ", ", "]"));
throw QueryExecutionErrors.foundDuplicateFieldInFieldIdLookupModeError(
fieldId, parquetTypesString);
} else {
return matched.get(0);
}
}
// Derived from CometParquetReadSupport.matchFieldId
private String getMatchingNameById(
StructField f,
Map<Integer, List<Type>> idToParquetFieldMap,
Map<String, List<Type>> nameToParquetFieldMap,
boolean isCaseSensitive) {
Type matched =
getMatchingParquetFieldById(f, idToParquetFieldMap, nameToParquetFieldMap, isCaseSensitive);
// When there is no ID match, we use a fake name to avoid a name match by accident
// We need this name to be unique as well, otherwise there will be type conflicts
if (matched == null) {
return CometParquetReadSupport.generateFakeColumnName();
} else {
return matched.getName();
}
}
// clip ParquetGroup Type
private StructType getSparkSchemaByFieldId(
StructType schema, GroupType parquetSchema, boolean caseSensitive) {
StructType newSchema = new StructType();
Map<Integer, List<Type>> idToParquetFieldMap = getIdToParquetFieldMap(parquetSchema);
Map<String, List<Type>> nameToParquetFieldMap =
caseSensitive
? getCaseSensitiveParquetFieldMap(parquetSchema)
: getCaseInsensitiveParquetFieldMap(parquetSchema);
for (StructField f : schema.fields()) {
DataType newDataType;
String fieldName = isCaseSensitive ? f.name() : f.name().toLowerCase(Locale.ROOT);
List<Type> parquetFieldList = nameToParquetFieldMap.get(fieldName);
if (parquetFieldList == null) {
newDataType = f.dataType();
} else {
Type fieldType = parquetFieldList.get(0);
if (f.dataType() instanceof StructType) {
newDataType =
getSparkSchemaByFieldId(
(StructType) f.dataType(), fieldType.asGroupType(), caseSensitive);
} else {
newDataType = getSparkTypeByFieldId(f.dataType(), fieldType, caseSensitive);
}
}
String matchedName =
getMatchingNameById(f, idToParquetFieldMap, nameToParquetFieldMap, isCaseSensitive);
StructField newField = f.copy(matchedName, newDataType, f.nullable(), f.metadata());
newSchema = newSchema.add(newField);
}
return newSchema;
}
private static boolean isPrimitiveCatalystType(DataType dataType) {
return !(dataType instanceof ArrayType)
&& !(dataType instanceof MapType)
&& !(dataType instanceof StructType);
}
private DataType getSparkTypeByFieldId(
DataType dataType, Type parquetType, boolean caseSensitive) {
DataType newDataType;
if (dataType instanceof StructType) {
newDataType =
getSparkSchemaByFieldId((StructType) dataType, parquetType.asGroupType(), caseSensitive);
} else if (dataType instanceof ArrayType
&& !isPrimitiveCatalystType(((ArrayType) dataType).elementType())) {
newDataType =
getSparkArrayTypeByFieldId(
(ArrayType) dataType, parquetType.asGroupType(), caseSensitive);
} else if (dataType instanceof MapType) {
MapType mapType = (MapType) dataType;
DataType keyType = mapType.keyType();
DataType valueType = mapType.valueType();
DataType newKeyType;
DataType newValueType;
Type parquetMapType = parquetType.asGroupType().getFields().get(0);
Type parquetKeyType = parquetMapType.asGroupType().getType("key");
Type parquetValueType = parquetMapType.asGroupType().getType("value");
if (keyType instanceof StructType) {
newKeyType =
getSparkSchemaByFieldId(
(StructType) keyType, parquetKeyType.asGroupType(), caseSensitive);
} else {
newKeyType = keyType;
}
if (valueType instanceof StructType) {
newValueType =
getSparkSchemaByFieldId(
(StructType) valueType, parquetValueType.asGroupType(), caseSensitive);
} else {
newValueType = valueType;
}
newDataType = new MapType(newKeyType, newValueType, mapType.valueContainsNull());
} else {
newDataType = dataType;
}
return newDataType;
}
private DataType getSparkArrayTypeByFieldId(
ArrayType arrayType, GroupType parquetList, boolean caseSensitive) {
DataType newDataType;
DataType elementType = arrayType.elementType();
DataType newElementType;
Type parquetElementType;
if (parquetList.getLogicalTypeAnnotation() == null
&& parquetList.isRepetition(Type.Repetition.REPEATED)) {
parquetElementType = parquetList;
} else {
// we expect only non-primitive types here (see clipParquetListTypes for related logic)
GroupType repeatedGroup = parquetList.asGroupType().getType(0).asGroupType();
if (repeatedGroup.getFieldCount() > 1
|| Objects.equals(repeatedGroup.getName(), "array")
|| Objects.equals(repeatedGroup.getName(), parquetList.getName() + "_tuple")) {
parquetElementType = repeatedGroup;
} else {
parquetElementType = repeatedGroup.getType(0);
}
}
if (elementType instanceof StructType) {
newElementType =
getSparkSchemaByFieldId(
(StructType) elementType, parquetElementType.asGroupType(), caseSensitive);
} else {
newElementType = getSparkTypeByFieldId(elementType, parquetElementType, caseSensitive);
}
newDataType = new ArrayType(newElementType, arrayType.containsNull());
return newDataType;
}
private void checkParquetType(ParquetColumn column) throws IOException {
String[] path = asJava(column.path()).toArray(new String[0]);
if (containsPath(fileSchema, path)) {
if (column.isPrimitive()) {
ColumnDescriptor desc = column.descriptor().get();
ColumnDescriptor fd = fileSchema.getColumnDescription(desc.getPath());
TypeUtil.checkParquetType(fd, column.sparkType());
} else {
for (ParquetColumn childColumn : asJava(column.children())) {
checkColumn(childColumn);
}
}
} else { // A missing column which is either primitive or complex
if (column.required()) {
// check if we have a preinitialized column reader for this column.
int columnIndex = getColumnIndexFromParquetColumn(column);
if (columnIndex == -1
|| preInitializedReaders == null
|| columnIndex >= preInitializedReaders.length
|| preInitializedReaders[columnIndex] == null) {
// Column is missing in data but the required data is non-nullable. This file is invalid.
throw new IOException(
"Required column is missing in data file. Col: " + Arrays.toString(path));
}
}
}
}
/**
* Get the column index in the requested schema for a given ParquetColumn. Returns -1 if not
* found.
*/
private int getColumnIndexFromParquetColumn(ParquetColumn column) {
String[] targetPath = asJava(column.path()).toArray(new String[0]);
if (targetPath.length == 0) {
return -1;
}
// For top-level columns, match by name
String columnName = targetPath[0];
ParquetColumn[] parquetFields = asJava(parquetColumn.children()).toArray(new ParquetColumn[0]);
for (int i = 0; i < parquetFields.length; i++) {
String[] fieldPath = asJava(parquetFields[i].path()).toArray(new String[0]);
if (fieldPath.length > 0 && fieldPath[0].equals(columnName)) {
return i;
}
}
return -1;
}
/**
* Checks whether the given 'path' exists in 'parquetType'. The difference between this and {@link
* MessageType#containsPath(String[])} is that the latter only support paths to leaf From Spark:
* VectorizedParquetRecordReader Check whether a column from requested schema is missing from the
* file schema, or whether it conforms to the type of the file schema.
*/
private void checkColumn(ParquetColumn column) throws IOException {
String[] path = asJava(column.path()).toArray(new String[0]);
if (containsPath(fileSchema, path)) {
if (column.isPrimitive()) {
ColumnDescriptor desc = column.descriptor().get();
ColumnDescriptor fd = fileSchema.getColumnDescription(desc.getPath());
if (!fd.equals(desc)) {
throw new UnsupportedOperationException("Schema evolution not supported.");
}
} else {
for (ParquetColumn childColumn : asJava(column.children())) {
checkColumn(childColumn);
}
}
} else { // A missing column which is either primitive or complex
if (column.required()) {
// Column is missing in data but the required data is non-nullable. This file is invalid.
throw new IOException(
"Required column is missing in data file. Col: " + Arrays.toString(path));
}
}
}
/**
* Checks whether the given 'path' exists in 'parquetType'. The difference between this and {@link
* MessageType#containsPath(String[])} is that the latter only support paths to leaf nodes, while
* this support paths both to leaf and non-leaf nodes.
*/
private boolean containsPath(Type parquetType, String[] path) {
return containsPath(parquetType, path, 0);
}
private boolean containsPath(Type parquetType, String[] path, int depth) {
if (path.length == depth) return true;
if (parquetType instanceof GroupType) {
String fieldName = path[depth];
GroupType parquetGroupType = (GroupType) parquetType;
if (parquetGroupType.containsField(fieldName)) {
return containsPath(parquetGroupType.getType(fieldName), path, depth + 1);
}
}
return false;
}
public void setSparkSchema(StructType schema) {
this.sparkSchema = schema;
}
public AbstractColumnReader[] getColumnReaders() {
return columnReaders;
}
@Override
public void initialize(InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
throws IOException, InterruptedException {
// Do nothing. The initialization work is done in 'init' already.
}
@Override
public boolean nextKeyValue() throws IOException {
return nextBatch();
}
@Override
public Void getCurrentKey() {
return null;
}
@Override
public ColumnarBatch getCurrentValue() {
return currentBatch();
}
@Override
public float getProgress() {
return 0;
}
/**
* Returns the current columnar batch being read.
*
* <p>Note that this must be called AFTER {@link NativeBatchReader#nextBatch()}.
*/
public ColumnarBatch currentBatch() {
return currentBatch;
}
/**
* Loads the next batch of rows. This is called by Spark _and_ Iceberg
*
* @return true if there are no more rows to read, false otherwise.
*/
public boolean nextBatch() throws IOException {
Preconditions.checkState(isInitialized, "init() should be called first!");
// if (rowsRead >= totalRowCount) return false;
if (totalRowCount == 0) return false;
int batchSize;
try {
batchSize = loadNextBatch();
} catch (RuntimeException e) {
// Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`.
throw e;
} catch (Throwable e) {
throw new IOException(e);
}
if (batchSize == 0) return false;
long totalDecodeTime = 0, totalLoadTime = 0;
for (int i = 0; i < columnReaders.length; i++) {
AbstractColumnReader reader = columnReaders[i];
long startNs = System.nanoTime();
// TODO: read from native reader
reader.readBatch(batchSize);
// totalDecodeTime += System.nanoTime() - startNs;
// startNs = System.nanoTime();
vectors[i] = reader.currentBatch();
totalLoadTime += System.nanoTime() - startNs;
}
// TODO: (ARROW NATIVE) Add Metrics
// SQLMetric decodeMetric = metrics.get("ParquetNativeDecodeTime");
// if (decodeMetric != null) {
// decodeMetric.add(totalDecodeTime);
// }
SQLMetric loadMetric = metrics.get("ParquetNativeLoadTime");
if (loadMetric != null) {
loadMetric.add(totalLoadTime);
}
currentBatch.setNumRows(batchSize);
return true;
}
@Override
public void close() throws IOException {
if (columnReaders != null) {
for (AbstractColumnReader reader : columnReaders) {
if (reader != null) {
reader.close();
}
}
}
if (importer != null) {
importer.close();
importer = null;
}
nativeUtil.close();