Skip to content

Commit 0eec215

Browse files
authoredFeb 29, 2024
PARQUET-2422: Prevent unwrapping of Hadoop filestreams (apache#1256)
1 parent 86f90f5 commit 0eec215

File tree

1 file changed

+6
-4
lines changed

1 file changed

+6
-4
lines changed
 

‎parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java

+6-4
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121

2222
import java.io.InputStream;
2323
import java.util.Objects;
24+
import java.util.function.Function;
2425
import org.apache.hadoop.fs.ByteBufferReadable;
2526
import org.apache.hadoop.fs.FSDataInputStream;
2627
import org.apache.hadoop.fs.FSDataOutputStream;
@@ -64,7 +65,7 @@ public static SeekableInputStream wrap(FSDataInputStream stream) {
6465
}
6566
}
6667

67-
return unwrapByteBufferReadableLegacy(stream);
68+
return unwrapByteBufferReadableLegacy(stream).apply(stream);
6869
}
6970

7071
/**
@@ -77,16 +78,17 @@ public static SeekableInputStream wrap(FSDataInputStream stream) {
7778
* @param stream stream to probe
7879
* @return A H2SeekableInputStream to access, or H1SeekableInputStream if the stream is not seekable
7980
*/
80-
private static SeekableInputStream unwrapByteBufferReadableLegacy(FSDataInputStream stream) {
81+
private static Function<FSDataInputStream, SeekableInputStream> unwrapByteBufferReadableLegacy(
82+
FSDataInputStream stream) {
8183
InputStream wrapped = stream.getWrappedStream();
8284
if (wrapped instanceof FSDataInputStream) {
8385
LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream);
8486
return unwrapByteBufferReadableLegacy(((FSDataInputStream) wrapped));
8587
}
8688
if (stream.getWrappedStream() instanceof ByteBufferReadable) {
87-
return new H2SeekableInputStream(stream);
89+
return H2SeekableInputStream::new;
8890
} else {
89-
return new H1SeekableInputStream(stream);
91+
return H1SeekableInputStream::new;
9092
}
9193
}
9294

0 commit comments

Comments
 (0)
Please sign in to comment.