This is an automated email from the ASF dual-hosted git repository.
gangwu pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/parquet-java.git
The following commit(s) were added to refs/heads/master by this push:
new 97321b831 GH-3141: Add constructor to `ParquetFileReader` to pass
parquet footer and expose setRequestedSchema that accepts
`List<ColumnDescriptor>` (#3262)
97321b831 is described below
commit 97321b83110d12b689d72c6f214627c20343925d
Author: Cheng Pan <[email protected]>
AuthorDate: Fri Aug 22 13:50:21 2025 +0800
GH-3141: Add constructor to `ParquetFileReader` to pass parquet footer and
expose setRequestedSchema that accepts `List<ColumnDescriptor>` (#3262)
---
.../apache/parquet/hadoop/ParquetFileReader.java | 69 ++++++++++++++++++----
.../parquet/hadoop/TestDataPageChecksums.java | 11 +++-
2 files changed, 67 insertions(+), 13 deletions(-)
diff --git
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
index ae2de87cb..2ef39f780 100644
---
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
+++
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java
@@ -567,8 +567,23 @@ public class ParquetFileReader implements Closeable {
public static final ParquetMetadata readFooter(InputFile file,
ParquetReadOptions options, SeekableInputStream f)
throws IOException {
+ return readFooter(file, options, f, /*closeStreamOnFailure*/ false);
+ }
+
+ private static final ParquetMetadata readFooter(
+ InputFile file, ParquetReadOptions options, SeekableInputStream f,
boolean closeStreamOnFailure)
+ throws IOException {
ParquetMetadataConverter converter = new ParquetMetadataConverter(options);
- return readFooter(file, options, f, converter);
+ try {
+ return readFooter(file, options, f, converter);
+ } catch (Exception e) {
+ // In case that readFooter throws an exception in the constructor, the
new stream
+ // should be closed. Otherwise, there's no way to close this outside.
+ if (closeStreamOnFailure) {
+ f.close();
+ }
+ throw e;
+ }
}
private static final ParquetMetadata readFooter(
@@ -729,6 +744,22 @@ public class ParquetFileReader implements Closeable {
return new ParquetFileReader(file, options, f);
}
+ /**
+ * Open a {@link InputFile file} with {@link ParquetMetadata footer} and
{@link ParquetReadOptions options}.
+ *
+ * @param file an input file
+ * @param footer a {@link ParquetMetadata} footer already read from the file
+ * @param options parquet read options
+ * @param f the input stream for the file
+ * @return an open ParquetFileReader
+ * @throws IOException if there is an error while opening the file
+ */
+ public static ParquetFileReader open(
+ InputFile file, ParquetMetadata footer, ParquetReadOptions options,
SeekableInputStream f)
+ throws IOException {
+ return new ParquetFileReader(file, footer, options, f);
+ }
+
protected SeekableInputStream f;
private final InputFile file;
private final ParquetReadOptions options;
@@ -930,19 +961,31 @@ public class ParquetFileReader implements Closeable {
this(file, options, file.newStream());
}
+ /**
+ * @param file Path to a parquet file
+ * @param options {@link ParquetReadOptions}
+ * @param f a {@link SeekableInputStream} for the parquet file
+ * @throws IOException if the file can not be opened
+ */
public ParquetFileReader(InputFile file, ParquetReadOptions options,
SeekableInputStream f) throws IOException {
+ this(file, readFooter(file, options, f, /*closeStreamOnFailure*/ true),
options, f);
+ }
+
+ /**
+ * @param file Path to a parquet file
+ * @param footer a {@link ParquetMetadata} footer already read from the file
+ * @param options {@link ParquetReadOptions}
+ * @param f a {@link SeekableInputStream} for the parquet file
+ * @throws IOException if the file can not be opened
+ */
+ public ParquetFileReader(InputFile file, ParquetMetadata footer,
ParquetReadOptions options, SeekableInputStream f)
+ throws IOException {
this.converter = new ParquetMetadataConverter(options);
this.file = file;
this.f = f;
this.options = options;
- try {
- this.footer = readFooter(file, options, f, converter);
- } catch (Exception e) {
- // In case that reading footer throws an exception in the constructor,
the new stream
- // should be closed. Otherwise, there's no way to close this outside.
- f.close();
- throw e;
- }
+ this.footer = footer;
+
this.fileMetaData = footer.getFileMetaData();
this.fileDecryptor = fileMetaData.getFileDecryptor(); // must be called
before filterRowGroups!
if (null != fileDecryptor && fileDecryptor.plaintextFile()) {
@@ -1054,13 +1097,17 @@ public class ParquetFileReader implements Closeable {
return blocks;
}
- public void setRequestedSchema(MessageType projection) {
+ public void setRequestedSchema(List<ColumnDescriptor> columns) {
paths.clear();
- for (ColumnDescriptor col : projection.getColumns()) {
+ for (ColumnDescriptor col : columns) {
paths.put(ColumnPath.get(col.getPath()), col);
}
}
+ public void setRequestedSchema(MessageType projection) {
+ setRequestedSchema(projection.getColumns());
+ }
+
public void appendTo(ParquetFileWriter writer) throws IOException {
writer.appendRowGroups(f, blocks, true);
}
diff --git
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDataPageChecksums.java
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDataPageChecksums.java
index 79a81a5e9..013498c2b 100644
---
a/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDataPageChecksums.java
+++
b/parquet-hadoop/src/test/java/org/apache/parquet/hadoop/TestDataPageChecksums.java
@@ -36,6 +36,8 @@ import java.util.Random;
import java.util.zip.CRC32;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path;
+import org.apache.parquet.HadoopReadOptions;
+import org.apache.parquet.ParquetReadOptions;
import org.apache.parquet.bytes.BytesInput;
import org.apache.parquet.bytes.HeapByteBufferAllocator;
import org.apache.parquet.bytes.TrackingByteBufferAllocator;
@@ -688,8 +690,13 @@ public class TestDataPageChecksums {
*/
private ParquetFileReader getParquetFileReader(Path path, Configuration
conf, List<ColumnDescriptor> columns)
throws IOException {
- ParquetMetadata footer = ParquetFileReader.readFooter(conf, path);
- return new ParquetFileReader(conf, footer.getFileMetaData(), path,
footer.getBlocks(), columns);
+ HadoopInputFile inputFile = HadoopInputFile.fromPath(path, conf);
+ SeekableInputStream inputStream = inputFile.newStream();
+ ParquetReadOptions readOptions = HadoopReadOptions.builder(conf).build();
+ ParquetMetadata footer = ParquetFileReader.readFooter(inputFile,
readOptions, inputStream);
+ ParquetFileReader reader = ParquetFileReader.open(inputFile, footer,
readOptions, inputStream);
+ reader.setRequestedSchema(columns);
+ return reader;
}
/**