forked from apache/datafusion-comet
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathBatchReader.java
More file actions
534 lines (474 loc) · 18.7 KB
/
BatchReader.java
File metadata and controls
534 lines (474 loc) · 18.7 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
/*
* 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.Closeable;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.*;
import scala.Option;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
import org.apache.arrow.memory.BufferAllocator;
import org.apache.arrow.memory.RootAllocator;
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.column.page.PageReadStore;
import org.apache.parquet.hadoop.metadata.ParquetMetadata;
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.sql.catalyst.InternalRow;
import org.apache.spark.sql.comet.parquet.CometParquetReadSupport;
import org.apache.spark.sql.comet.shims.ShimTaskMetrics;
import org.apache.spark.sql.execution.datasources.PartitionedFile;
import org.apache.spark.sql.execution.datasources.parquet.ParquetToSparkSchemaConverter;
import org.apache.spark.sql.execution.metric.SQLMetric;
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.IcebergApi;
import org.apache.comet.shims.ShimBatchReader;
import org.apache.comet.shims.ShimFileFormat;
import org.apache.comet.vector.CometVector;
/**
* 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>
*
* @deprecated since 0.14.0. This class is kept for Iceberg compatibility only.
*/
@Deprecated
@IcebergApi
public class BatchReader extends RecordReader<Void, ColumnarBatch> implements Closeable {
private static final Logger LOG = LoggerFactory.getLogger(FileReader.class);
protected static final BufferAllocator ALLOCATOR = new RootAllocator();
private Configuration conf;
private int capacity;
private boolean isCaseSensitive;
private boolean useFieldId;
private boolean ignoreMissingIds;
private StructType partitionSchema;
private InternalRow partitionValues;
private PartitionedFile file;
protected Map<String, SQLMetric> metrics;
private long rowsRead;
protected StructType sparkSchema;
private MessageType requestedSchema;
protected CometVector[] vectors;
protected AbstractColumnReader[] columnReaders;
private CometSchemaImporter importer;
protected ColumnarBatch currentBatch;
private FileReader fileReader;
private boolean[] missingColumns;
protected boolean isInitialized;
private ParquetMetadata footer;
/** The total number of rows across all row groups of the input split. */
private long totalRowCount;
/**
* The total number of rows loaded so far, including all the rows from row groups that we've
* processed and the current row group.
*/
private long totalRowsLoaded;
/**
* 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.
*/
private boolean useDecimal128;
/** Whether to use the lazy materialization reader for reading columns. */
private boolean useLazyMaterialization;
/**
* 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.
*/
private boolean useLegacyDateTimestamp;
/** The TaskContext object for executing this task. */
private TaskContext taskContext;
public BatchReader() {}
// Only for testing
public BatchReader(String file, int capacity) {
this(file, capacity, null, null);
}
// Only for testing
public BatchReader(
String file, int capacity, StructType partitionSchema, InternalRow partitionValues) {
this(new Configuration(), file, capacity, partitionSchema, partitionValues);
}
// Only for testing
public BatchReader(
Configuration conf,
String file,
int capacity,
StructType partitionSchema,
InternalRow partitionValues) {
conf.set("spark.sql.parquet.binaryAsString", "false");
conf.set("spark.sql.parquet.int96AsTimestamp", "false");
conf.set("spark.sql.caseSensitive", "false");
conf.set("spark.sql.parquet.inferTimestampNTZ.enabled", "true");
conf.set("spark.sql.legacy.parquet.nanosAsLong", "false");
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();
}
/**
* @see <a href="https://github.com/apache/datafusion-comet/issues/2079">Comet Issue #2079</a>
*/
@IcebergApi
public BatchReader(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<>();
}
BatchReader(
Configuration conf,
PartitionedFile inputSplit,
ParquetMetadata footer,
int capacity,
StructType sparkSchema,
boolean isCaseSensitive,
boolean useFieldId,
boolean ignoreMissingIds,
boolean useLegacyDateTimestamp,
StructType partitionSchema,
InternalRow partitionValues,
Map<String, SQLMetric> metrics) {
this.conf = conf;
this.capacity = capacity;
this.sparkSchema = sparkSchema;
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.metrics = metrics;
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 URISyntaxException, IOException {
useDecimal128 =
conf.getBoolean(
CometConf.COMET_USE_DECIMAL_128().key(),
(Boolean) CometConf.COMET_USE_DECIMAL_128().defaultValue().get());
useLazyMaterialization =
conf.getBoolean(
CometConf.COMET_USE_LAZY_MATERIALIZATION().key(),
(Boolean) CometConf.COMET_USE_LAZY_MATERIALIZATION().defaultValue().get());
long start = file.start();
long length = file.length();
String filePath = file.filePath().toString();
ParquetReadOptions.Builder builder = HadoopReadOptions.builder(conf, new Path(filePath));
if (start >= 0 && length >= 0) {
builder = builder.withRange(start, start + length);
}
ParquetReadOptions readOptions = builder.build();
// TODO: enable off-heap buffer when they are ready
ReadOptions cometReadOptions = ReadOptions.builder(conf).build();
Path path = new Path(new URI(filePath));
fileReader =
new FileReader(
CometInputFile.fromPath(path, conf), footer, readOptions, cometReadOptions, metrics);
requestedSchema = fileReader.getFileMetaData().getSchema();
MessageType fileSchema = requestedSchema;
if (sparkSchema == null) {
sparkSchema = new ParquetToSparkSchemaConverter(conf).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.getColumns().size()));
}
}
totalRowCount = fileReader.getRecordCount();
List<ColumnDescriptor> columns = requestedSchema.getColumns();
int numColumns = columns.size();
if (partitionSchema != null) numColumns += partitionSchema.size();
columnReaders = new AbstractColumnReader[numColumns];
// Initialize missing columns and use null vectors for them
missingColumns = new boolean[columns.size()];
List<String[]> paths = requestedSchema.getPaths();
// 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();
for (int i = 0; i < requestedSchema.getFieldCount(); i++) {
Type t = requestedSchema.getFields().get(i);
Preconditions.checkState(
t.isPrimitive() && !t.isRepetition(Type.Repetition.REPEATED),
"Complex type is not supported");
String[] colPath = paths.get(i);
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();
columnReaders[i] = new RowIndexColumnReader(nonPartitionFields[i], capacity, rowIndices);
missingColumns[i] = true;
} else if (fileSchema.containsPath(colPath)) {
ColumnDescriptor fd = fileSchema.getColumnDescription(colPath);
if (!fd.equals(columns.get(i))) {
throw new UnsupportedOperationException("Schema evolution is not supported");
}
missingColumns[i] = false;
} else {
if (columns.get(i).getMaxDefinitionLevel() == 0) {
throw new IOException(
"Required column '"
+ Arrays.toString(colPath)
+ "' is missing"
+ " in data file "
+ filePath);
}
ConstantColumnReader reader =
new ConstantColumnReader(nonPartitionFields[i], capacity, useDecimal128);
columnReaders[i] = reader;
missingColumns[i] = true;
}
}
// Initialize constant readers for partition columns
if (partitionSchema != null) {
StructField[] partitionFields = partitionSchema.fields();
for (int i = columns.size(); i < columnReaders.length; i++) {
int fieldIndex = i - columns.size();
StructField field = partitionFields[fieldIndex];
ConstantColumnReader reader =
new ConstantColumnReader(field, capacity, partitionValues, fieldIndex, useDecimal128);
columnReaders[i] = reader;
}
}
vectors = new CometVector[numColumns];
currentBatch = new ColumnarBatch(vectors);
fileReader.setRequestedSchema(requestedSchema.getColumns());
// 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 =
ShimTaskMetrics.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(fileReader.getRowGroups().size());
}
}
isInitialized = true;
}
/**
* @see <a href="https://github.com/apache/datafusion-comet/issues/2079">Comet Issue #2079</a>
*/
@IcebergApi
public void setSparkSchema(StructType schema) {
this.sparkSchema = schema;
}
/**
* @see <a href="https://github.com/apache/datafusion-comet/issues/2079">Comet Issue #2079</a>
*/
@IcebergApi
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 (float) rowsRead / totalRowCount;
}
/**
* Returns the current columnar batch being read.
*
* <p>Note that this must be called AFTER {@link BatchReader#nextBatch()}.
*/
public ColumnarBatch currentBatch() {
return currentBatch;
}
/**
* Loads the next batch of rows.
*
* @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;
boolean hasMore;
try {
hasMore = loadNextRowGroupIfNecessary();
} catch (RuntimeException e) {
// Spark will check certain exception e.g. `SchemaColumnConvertNotSupportedException`.
throw e;
} catch (Throwable e) {
throw new IOException(e);
}
if (!hasMore) return false;
int batchSize = (int) Math.min(capacity, totalRowsLoaded - rowsRead);
return nextBatch(batchSize);
}
@IcebergApi
public boolean nextBatch(int batchSize) {
long totalDecodeTime = 0, totalLoadTime = 0;
for (int i = 0; i < columnReaders.length; i++) {
AbstractColumnReader reader = columnReaders[i];
long startNs = System.nanoTime();
reader.readBatch(batchSize);
totalDecodeTime += System.nanoTime() - startNs;
startNs = System.nanoTime();
vectors[i] = reader.currentBatch();
totalLoadTime += System.nanoTime() - startNs;
}
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);
rowsRead += batchSize;
return true;
}
@IcebergApi
@Override
public void close() throws IOException {
if (columnReaders != null) {
for (AbstractColumnReader reader : columnReaders) {
if (reader != null) {
reader.close();
}
}
}
if (fileReader != null) {
fileReader.close();
fileReader = null;
}
if (importer != null) {
importer.close();
importer = null;
}
}
private boolean loadNextRowGroupIfNecessary() throws Throwable {
// More rows can be read from loaded row group. No need to load next one.
if (rowsRead != totalRowsLoaded) return true;
SQLMetric rowGroupTimeMetric = metrics.get("ParquetLoadRowGroupTime");
SQLMetric numRowGroupsMetric = metrics.get("ParquetRowGroups");
long startNs = System.nanoTime();
PageReadStore rowGroupReader = fileReader.readNextRowGroup();
if (rowGroupTimeMetric != null) {
rowGroupTimeMetric.add(System.nanoTime() - startNs);
}
if (rowGroupReader == null) {
return false;
}
if (numRowGroupsMetric != null) {
numRowGroupsMetric.add(1);
}
if (importer != null) importer.close();
importer = new CometSchemaImporter(ALLOCATOR);
List<ColumnDescriptor> columns = requestedSchema.getColumns();
for (int i = 0; i < columns.size(); i++) {
if (missingColumns[i]) continue;
if (columnReaders[i] != null) columnReaders[i].close();
// TODO: handle tz, datetime & int96 rebase
// TODO: consider passing page reader via ctor - however we need to fix the shading issue
// from Iceberg side.
DataType dataType = sparkSchema.fields()[i].dataType();
ColumnReader reader =
Utils.getColumnReader(
dataType,
columns.get(i),
importer,
capacity,
useDecimal128,
useLazyMaterialization,
useLegacyDateTimestamp);
reader.setPageReader(rowGroupReader.getPageReader(columns.get(i)));
columnReaders[i] = reader;
}
totalRowsLoaded += rowGroupReader.getRowCount();
return true;
}
}