Skip to content

Commit

Permalink
PARQUET-2422: Prevent unwrapping of Hadoop filestreams (apache#1256)
Browse files Browse the repository at this point in the history
  • Loading branch information
rathinb-db authored Feb 29, 2024
1 parent 86f90f5 commit 0eec215
Showing 1 changed file with 6 additions and 4 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import java.io.InputStream;
import java.util.Objects;
import java.util.function.Function;
import org.apache.hadoop.fs.ByteBufferReadable;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.FSDataOutputStream;
Expand Down Expand Up @@ -64,7 +65,7 @@ public static SeekableInputStream wrap(FSDataInputStream stream) {
}
}

return unwrapByteBufferReadableLegacy(stream);
return unwrapByteBufferReadableLegacy(stream).apply(stream);
}

/**
Expand All @@ -77,16 +78,17 @@ public static SeekableInputStream wrap(FSDataInputStream stream) {
* @param stream stream to probe
* @return A H2SeekableInputStream to access, or H1SeekableInputStream if the stream is not seekable
*/
private static SeekableInputStream unwrapByteBufferReadableLegacy(FSDataInputStream stream) {
private static Function<FSDataInputStream, SeekableInputStream> unwrapByteBufferReadableLegacy(
FSDataInputStream stream) {
InputStream wrapped = stream.getWrappedStream();
if (wrapped instanceof FSDataInputStream) {
LOG.debug("Checking on wrapped stream {} of {} whether is ByteBufferReadable", wrapped, stream);
return unwrapByteBufferReadableLegacy(((FSDataInputStream) wrapped));
}
if (stream.getWrappedStream() instanceof ByteBufferReadable) {
return new H2SeekableInputStream(stream);
return H2SeekableInputStream::new;
} else {
return new H1SeekableInputStream(stream);
return H1SeekableInputStream::new;
}
}

Expand Down

0 comments on commit 0eec215

Please sign in to comment.