From eed9bc24c4dd0215c7178287120c9c929f736ab0 Mon Sep 17 00:00:00 2001 From: Hu Maobo Date: Mon, 11 Oct 2021 20:40:44 +0800 Subject: [PATCH] Parquet: avoid premature closing of FSDataInputStream object --- .../src/main/java/org/apache/iceberg/parquet/ParquetIO.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java index d65b8d638edf..19d845e97bd0 100644 --- a/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java +++ b/parquet/src/main/java/org/apache/iceberg/parquet/ParquetIO.java @@ -166,9 +166,11 @@ public long defaultBlockSize() { private static class ParquetInputFile implements InputFile { private final org.apache.iceberg.io.InputFile file; + private org.apache.iceberg.io.SeekableInputStream icebergHadoopStream; private ParquetInputFile(org.apache.iceberg.io.InputFile file) { this.file = file; + this.icebergHadoopStream = null; } @Override @@ -178,7 +180,8 @@ public long getLength() throws IOException { @Override public SeekableInputStream newStream() throws IOException { - return stream(file.newStream()); + icebergHadoopStream = file.newStream(); + return stream(icebergHadoopStream); } } }