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-mr.git


The following commit(s) were added to refs/heads/master by this push:
     new 0eec215e1 PARQUET-2422: Prevent unwrapping of Hadoop filestreams 
(#1256)
0eec215e1 is described below

commit 0eec215e13e4a90d2c9ba73f44d56ff481c3ada9
Author: Rathin Bhargava <[email protected]>
AuthorDate: Thu Feb 29 08:48:44 2024 +0100

    PARQUET-2422: Prevent unwrapping of Hadoop filestreams (#1256)
---
 .../java/org/apache/parquet/hadoop/util/HadoopStreams.java     | 10 ++++++----
 1 file changed, 6 insertions(+), 4 deletions(-)

diff --git 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
index 22fa9727b..eebc59987 100644
--- 
a/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
+++ 
b/parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java
@@ -21,6 +21,7 @@ package org.apache.parquet.hadoop.util;
 
 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;
@@ -64,7 +65,7 @@ public class HadoopStreams {
       }
     }
 
-    return unwrapByteBufferReadableLegacy(stream);
+    return unwrapByteBufferReadableLegacy(stream).apply(stream);
   }
 
   /**
@@ -77,16 +78,17 @@ public class HadoopStreams {
    * @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;
     }
   }
 

Reply via email to