wgtmac commented on code in PR #3096:
URL: https://github.com/apache/parquet-java/pull/3096#discussion_r1884049013


##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##########
@@ -111,14 +79,8 @@ private static Function<FSDataInputStream, 
SeekableInputStream> unwrapByteBuffer
    * the data, null when it cannot be determined because of missing 
hasCapabilities
    */
   private static Boolean isWrappedStreamByteBufferReadable(FSDataInputStream 
stream) {
-    if (hasCapabilitiesMethod.isNoop()) {
-      // When the method is not available, just return a null
-      return null;
-    }
-
-    boolean isByteBufferReadable = hasCapabilitiesMethod.invoke(stream, 
"in:readbytebuffer");
 
-    if (isByteBufferReadable) {
+    if (stream.hasCapability(READBYTEBUFFER)) {

Review Comment:
   We can avoid any reflection here because of the Hadoop version bump?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/HadoopStreams.java:
##########
@@ -53,42 +49,14 @@ public class HadoopStreams {
   public static SeekableInputStream wrap(FSDataInputStream stream) {
     Objects.requireNonNull(stream, "Cannot wrap a null input stream");
 
-    // Try to check using hasCapabilities(str)
-    Boolean hasCapabilitiesResult = isWrappedStreamByteBufferReadable(stream);
-
-    // If it is null, then fall back to the old method
-    if (hasCapabilitiesResult != null) {
-      if (hasCapabilitiesResult) {
-        return new H2SeekableInputStream(stream);
-      } else {
-        return new H1SeekableInputStream(stream);
-      }
-    }
-
-    return unwrapByteBufferReadableLegacy(stream).apply(stream);
-  }
-
-  /**
-   * Is the inner stream byte buffer readable?
-   * The test is 'the stream is not FSDataInputStream
-   * and implements ByteBufferReadable'
-   * <p>
-   * This logic is only used for Hadoop <2.9.x, and <3.x.x
-   *
-   * @param stream stream to probe
-   * @return A H2SeekableInputStream to access, or H1SeekableInputStream if 
the stream is not seekable
-   */
-  private static Function<FSDataInputStream, SeekableInputStream> 
unwrapByteBufferReadableLegacy(

Review Comment:
   Is there any behavior change of a wrapped stream after removing 
`unwrapByteBufferReadableLegacy`?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/H3ByteBufferInputStream.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.parquet.hadoop.util;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+/**
+ * Class which implements {@link #readFully(ByteBuffer)} through
+ * {@code ByteBufferPositionedReadable.readFully()}.
+ * <p>This is implemented by HDFS and possibly other clients,
+ */
+class H3ByteBufferInputStream extends H2SeekableInputStream {
+  public H3ByteBufferInputStream(final FSDataInputStream stream) {
+    super(stream);
+  }
+
+  @Override
+  public FSDataInputStream getStream() {
+    return (FSDataInputStream) super.getStream();
+  }
+
+  /**
+   * Read the buffer fully through use of {@code 
ByteBufferPositionedReadable.readFully()}
+   * at the current location.
+   * <p>That operation is designed to not use the current reading position, 
rather
+   * an absolute position is passed in.
+   * In the use here the original read position is saved, and
+   * after the read is finished a {@code seek()} call made to move the
+   * cursor on.
+   *
+   * @param buf a byte buffer to fill with data from the stream
+   *
+   * @throws EOFException the buffer length is greater than the file length
+   * @throws IOException other IO problems.
+   */
+  @Override
+  public void readFully(final ByteBuffer buf) throws EOFException, IOException 
{
+    performRead(getStream(), buf);
+  }
+
+  /**
+   * Read the buffer fully through use of {@code 
ByteBufferPositionedReadable.readFully()}
+   * from the current location.
+   * That is it reads from stream[pos] to stream[pos + buf.remaining() -1]
+   *
+   * @param buf a byte buffer to fill with data from the stream
+   * @return number of bytes read.
+   *
+   * @throws EOFException the buffer length is greater than the file length

Review Comment:
   Add `EOFException` to the method signature?



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/H3ByteBufferInputStream.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.parquet.hadoop.util;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import org.apache.hadoop.fs.FSDataInputStream;
+
+/**
+ * Class which implements {@link #readFully(ByteBuffer)} through
+ * {@code ByteBufferPositionedReadable.readFully()}.
+ * <p>This is implemented by HDFS and possibly other clients,
+ */
+class H3ByteBufferInputStream extends H2SeekableInputStream {

Review Comment:
   I'm fine with the inheritance to reduce code duplication.



##########
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/util/H2SeekableInputStream.java:
##########
@@ -100,6 +100,10 @@ public void readVectored(List<ParquetFileRange> ranges, 
ByteBufferAllocator allo
     VectorIoBridge.instance().readVectoredRanges(stream, ranges, allocator);
   }
 
+  protected Reader getReader() {

Review Comment:
   Why adding this but not used elsewhere?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: issues-unsubscr...@parquet.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscr...@parquet.apache.org
For additional commands, e-mail: issues-h...@parquet.apache.org

Reply via email to