This is an automated email from the ASF dual-hosted git repository.
gabor 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 73a4430af PARQUET-3031: Support to transfer input stream when building
ParquetFileReader (#3030)
73a4430af is described below
commit 73a4430af6c40f8eb246ad4911eb6d103c9a2abe
Author: Fei Wang <[email protected]>
AuthorDate: Thu Oct 31 01:50:22 2024 -0700
PARQUET-3031: Support to transfer input stream when building
ParquetFileReader (#3030)
---
.../apache/parquet/hadoop/ParquetFileReader.java | 36 ++++++++++++++++++++--
1 file changed, 34 insertions(+), 2 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 1d8cce3d8..785f145b2 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
@@ -710,6 +710,20 @@ public class ParquetFileReader implements Closeable {
return new ParquetFileReader(file, options);
}
+ /**
+ * Open a {@link InputFile file} with {@link ParquetReadOptions options}.
+ *
+ * @param file an input 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, ParquetReadOptions
options, SeekableInputStream f)
+ throws IOException {
+ return new ParquetFileReader(file, options, f);
+ }
+
protected final SeekableInputStream f;
private final InputFile file;
private final ParquetReadOptions options;
@@ -863,9 +877,23 @@ public class ParquetFileReader implements Closeable {
*/
public ParquetFileReader(Configuration conf, Path file, ParquetMetadata
footer, ParquetReadOptions options)
throws IOException {
+ this(conf, file, footer, options, HadoopInputFile.fromPath(file,
conf).newStream());
+ }
+
+ /**
+ * @param conf the Hadoop Configuration
+ * @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(
+ Configuration conf, Path file, ParquetMetadata footer,
ParquetReadOptions options, SeekableInputStream f)
+ throws IOException {
this.converter = new ParquetMetadataConverter(conf);
this.file = HadoopInputFile.fromPath(file, conf);
- this.f = this.file.newStream();
+ this.f = f;
this.fileMetaData = footer.getFileMetaData();
this.fileDecryptor = fileMetaData.getFileDecryptor();
this.options = options;
@@ -894,9 +922,13 @@ public class ParquetFileReader implements Closeable {
}
public ParquetFileReader(InputFile file, ParquetReadOptions options) throws
IOException {
+ this(file, options, file.newStream());
+ }
+
+ public ParquetFileReader(InputFile file, ParquetReadOptions options,
SeekableInputStream f) throws IOException {
this.converter = new ParquetMetadataConverter(options);
this.file = file;
- this.f = file.newStream();
+ this.f = f;
this.options = options;
try {
this.footer = readFooter(file, options, f, converter);