You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/05/27 09:34:00 UTC

[GitHub] [ozone] ChenSammi opened a new pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

ChenSammi opened a new pull request #2203:
URL: https://github.com/apache/ozone/pull/2203


   https://issues.apache.org/jira/browse/HDDS-5151


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2203: Hdds 5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r623834924



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
##########
@@ -507,4 +509,73 @@ private void handleReadError(IOException cause) throws IOException {
   public synchronized List<ChunkInputStream> getChunkStreams() {
     return chunkStreams;
   }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    if (byteBuffer == null) {
+      throw new NullPointerException();
+    }
+    int bufferLen = byteBuffer.remaining();
+    if (bufferLen == 0) {
+      return 0;
+    }
+
+    if (!initialized) {
+      initialize();
+    }
+
+    checkOpen();
+    int totalReadLen = 0;
+    int bufferLimit = byteBuffer.limit();
+    while (bufferLen > 0) {
+      // if we are at the last chunk and have read the entire chunk, return
+      if (chunkStreams.size() == 0 ||
+          (chunkStreams.size() - 1 <= chunkIndex &&
+              chunkStreams.get(chunkIndex)
+                  .getRemaining() == 0)) {
+        return totalReadLen == 0 ? EOF : totalReadLen;
+      }
+
+      // Get the current chunkStream and read data from it
+      ChunkInputStream current = chunkStreams.get(chunkIndex);
+      int numBytesToRead = Math.min(bufferLen, (int)current.getRemaining());
+      // change buffer limit
+      if (numBytesToRead < bufferLen) {
+        byteBuffer.limit(byteBuffer.position() + numBytesToRead);
+      }
+      int numBytesRead;
+      try {
+        numBytesRead = current.read(byteBuffer);
+        retries = 0; // reset retries after successful read
+      } catch (StorageContainerException e) {
+        if (shouldRetryRead(e)) {
+          handleReadError(e);
+          continue;
+        } else {
+          throw e;
+        }
+      } finally {
+        // restore buffer limit
+        if (numBytesToRead < bufferLen) {
+          byteBuffer.limit(bufferLimit);
+        }
+      }
+      if (numBytesRead != numBytesToRead) {
+        // This implies that there is either data loss or corruption in the
+        // chunk entries. Even EOF in the current stream would be covered in
+        // this case.
+        throw new IOException(String.format(
+            "Inconsistent read for chunkName=%s length=%d numBytesToRead= %d " +
+                "numBytesRead=%d", current.getChunkName(), current.getLength(),
+            numBytesToRead, numBytesRead));
+      }
+      totalReadLen += numBytesRead;
+      bufferLen -= numBytesRead;
+      if (current.getRemaining() <= 0 &&
+          ((chunkIndex + 1) < chunkStreams.size())) {
+        chunkIndex += 1;
+      }
+    }
+    return totalReadLen;
+  }

Review comment:
       Can we avoid duplicating logic from `read(byte[], ...)`?  (Also in `ChunkInputStream` and `KeyInputStream`.)




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r640358011



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ReaderStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A Reader interface to work with InputStream.
+ */
+public interface ReaderStrategy {

Review comment:
       Sure. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hanishakoneru commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
hanishakoneru commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-852514546


   Thank you @ChenSammi for working on this. 
   LGTM. +1 for merge.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-847507372


   @hanishakoneru and @adoroszlai ,  would you help to take another look again? 


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r628184753



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
##########
@@ -507,4 +509,73 @@ private void handleReadError(IOException cause) throws IOException {
   public synchronized List<ChunkInputStream> getChunkStreams() {
     return chunkStreams;
   }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    if (byteBuffer == null) {
+      throw new NullPointerException();
+    }
+    int bufferLen = byteBuffer.remaining();
+    if (bufferLen == 0) {
+      return 0;
+    }
+
+    if (!initialized) {
+      initialize();
+    }
+
+    checkOpen();
+    int totalReadLen = 0;
+    int bufferLimit = byteBuffer.limit();
+    while (bufferLen > 0) {
+      // if we are at the last chunk and have read the entire chunk, return
+      if (chunkStreams.size() == 0 ||
+          (chunkStreams.size() - 1 <= chunkIndex &&
+              chunkStreams.get(chunkIndex)
+                  .getRemaining() == 0)) {
+        return totalReadLen == 0 ? EOF : totalReadLen;
+      }
+
+      // Get the current chunkStream and read data from it
+      ChunkInputStream current = chunkStreams.get(chunkIndex);
+      int numBytesToRead = Math.min(bufferLen, (int)current.getRemaining());
+      // change buffer limit
+      if (numBytesToRead < bufferLen) {
+        byteBuffer.limit(byteBuffer.position() + numBytesToRead);
+      }
+      int numBytesRead;
+      try {
+        numBytesRead = current.read(byteBuffer);
+        retries = 0; // reset retries after successful read
+      } catch (StorageContainerException e) {
+        if (shouldRetryRead(e)) {
+          handleReadError(e);
+          continue;
+        } else {
+          throw e;
+        }
+      } finally {
+        // restore buffer limit
+        if (numBytesToRead < bufferLen) {
+          byteBuffer.limit(bufferLimit);
+        }
+      }
+      if (numBytesRead != numBytesToRead) {
+        // This implies that there is either data loss or corruption in the
+        // chunk entries. Even EOF in the current stream would be covered in
+        // this case.
+        throw new IOException(String.format(
+            "Inconsistent read for chunkName=%s length=%d numBytesToRead= %d " +
+                "numBytesRead=%d", current.getChunkName(), current.getLength(),
+            numBytesToRead, numBytesRead));
+      }
+      totalReadLen += numBytesRead;
+      bufferLen -= numBytesRead;
+      if (current.getRemaining() <= 0 &&
+          ((chunkIndex + 1) < chunkStreams.size())) {
+        chunkIndex += 1;
+      }
+    }
+    return totalReadLen;
+  }

Review comment:
       Good point.  I have updated a new patch for this. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hanishakoneru commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
hanishakoneru commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r640087808



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ReaderStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A Reader interface to work with InputStream.
+ */
+public interface ReaderStrategy {

Review comment:
       Similar to implementations of this interface, do you think we can rename to something like ByteReader?

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
##########
@@ -200,6 +201,8 @@ public void testReadChunk() throws Exception {
       for (int bufferSize : bufferSizeList) {
         assertReadFully(data, keyInputStream, bufferSize, 0);
         keyInputStream.seek(0);
+        assertReadFullyUseByteBuffer(data, keyInputStream, bufferSize, 0);
+        keyInputStream.seek(0);

Review comment:
       I meant to say a separate unit test. This is so that the test does not time out in Jenkins. When there is a crunch in resources, tests might take longer to run and there is timeout for each individual test.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r640358011



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ReaderStrategy.java
##########
@@ -0,0 +1,41 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A Reader interface to work with InputStream.
+ */
+public interface ReaderStrategy {

Review comment:
       Sure.  @hanishakoneru H,  it has been addressed. 




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi merged pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi merged pull request #2203:
URL: https://github.com/apache/ozone/pull/2203


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] adoroszlai commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
adoroszlai commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r629298455



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
##########
@@ -507,4 +509,73 @@ private void handleReadError(IOException cause) throws IOException {
   public synchronized List<ChunkInputStream> getChunkStreams() {
     return chunkStreams;
   }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    if (byteBuffer == null) {
+      throw new NullPointerException();
+    }
+    int bufferLen = byteBuffer.remaining();
+    if (bufferLen == 0) {
+      return 0;
+    }
+
+    if (!initialized) {
+      initialize();
+    }
+
+    checkOpen();
+    int totalReadLen = 0;
+    int bufferLimit = byteBuffer.limit();
+    while (bufferLen > 0) {
+      // if we are at the last chunk and have read the entire chunk, return
+      if (chunkStreams.size() == 0 ||
+          (chunkStreams.size() - 1 <= chunkIndex &&
+              chunkStreams.get(chunkIndex)
+                  .getRemaining() == 0)) {
+        return totalReadLen == 0 ? EOF : totalReadLen;
+      }
+
+      // Get the current chunkStream and read data from it
+      ChunkInputStream current = chunkStreams.get(chunkIndex);
+      int numBytesToRead = Math.min(bufferLen, (int)current.getRemaining());
+      // change buffer limit
+      if (numBytesToRead < bufferLen) {
+        byteBuffer.limit(byteBuffer.position() + numBytesToRead);
+      }
+      int numBytesRead;
+      try {
+        numBytesRead = current.read(byteBuffer);
+        retries = 0; // reset retries after successful read
+      } catch (StorageContainerException e) {
+        if (shouldRetryRead(e)) {
+          handleReadError(e);
+          continue;
+        } else {
+          throw e;
+        }
+      } finally {
+        // restore buffer limit
+        if (numBytesToRead < bufferLen) {
+          byteBuffer.limit(bufferLimit);
+        }
+      }
+      if (numBytesRead != numBytesToRead) {
+        // This implies that there is either data loss or corruption in the
+        // chunk entries. Even EOF in the current stream would be covered in
+        // this case.
+        throw new IOException(String.format(
+            "Inconsistent read for chunkName=%s length=%d numBytesToRead= %d " +
+                "numBytesRead=%d", current.getChunkName(), current.getLength(),
+            numBytesToRead, numBytesRead));
+      }
+      totalReadLen += numBytesRead;
+      bufferLen -= numBytesRead;
+      if (current.getRemaining() <= 0 &&
+          ((chunkIndex + 1) < chunkStreams.size())) {
+        chunkIndex += 1;
+      }
+    }
+    return totalReadLen;
+  }

Review comment:
       Thanks @ChenSammi for the update.  The new implementation using strategy pattern looks good.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi closed pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi closed pull request #2203:
URL: https://github.com/apache/ozone/pull/2203


   


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] hanishakoneru commented on a change in pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
hanishakoneru commented on a change in pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#discussion_r632058083



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
##########
@@ -507,4 +457,68 @@ private void handleReadError(IOException cause) throws IOException {
   public synchronized List<ChunkInputStream> getChunkStreams() {
     return chunkStreams;
   }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {

Review comment:
       Can we move this and readWithStrategy() methods up and place it after the read(byte[], int, int) method. It would help improve code readability if we club similar methods together. 

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
##########
@@ -200,6 +201,8 @@ public void testReadChunk() throws Exception {
       for (int bufferSize : bufferSizeList) {
         assertReadFully(data, keyInputStream, bufferSize, 0);
         keyInputStream.seek(0);
+        assertReadFullyUseByteBuffer(data, keyInputStream, bufferSize, 0);
+        keyInputStream.seek(0);

Review comment:
       Can we create a new test for ByteBuffer read?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ChunkInputStream.java
##########
@@ -191,6 +192,41 @@ public synchronized int read(byte[] b, int off, int len) throws IOException {
     return total;
   }
 
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    if (byteBuffer == null) {
+      throw new NullPointerException();
+    }
+    int bufferLen = byteBuffer.remaining();

Review comment:
       Can we rename "bufferLen" to something to specify it is the remaining number of bytes in the buffer (like "bufferLenRemaining" or just "len")

##########
File path: hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/client/io/LengthInputStream.java
##########
@@ -46,4 +51,14 @@ public long getLength() {
   public InputStream getWrappedStream() {
     return in;
   }
+
+  @Override
+  public int read(ByteBuffer byteBuffer) throws IOException {
+    if (in instanceof ByteBufferReadable) {
+      return ((ByteBufferReadable)in).read(byteBuffer);
+    } else {
+      throw new UnsupportedOperationException("Read with ByteBuffer is not " +
+          " supported by this input stream");

Review comment:
       Instead of "this" can we specify the class of input stream?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteArrayStrategy.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.io.InputStream;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class ByteArrayStrategy implements ReaderStrategy {

Review comment:
       Can you please add a Javadoc for this class

##########
File path: hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/read/TestKeyInputStream.java
##########
@@ -351,4 +362,25 @@ private void assertReadFully(byte[] data, InputStream in,
     }
     Assert.assertEquals(data.length, totalRead);
   }
+
+  private void assertReadFullyUseByteBuffer(byte[] data, KeyInputStream in,

Review comment:
       NIT: assertReadFullyUsingByteBuffer

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockInputStream.java
##########
@@ -507,4 +457,68 @@ private void handleReadError(IOException cause) throws IOException {
   public synchronized List<ChunkInputStream> getChunkStreams() {
     return chunkStreams;
   }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    ReaderStrategy strategy = new ByteBufferStrategy(byteBuffer);
+    int bufferLen = strategy.getTargetLength();
+    if (bufferLen == 0) {
+      return 0;
+    }
+    return readWithStrategy(strategy);
+  }
+
+  synchronized int readWithStrategy(ReaderStrategy strategy) throws
+      IOException {
+    Preconditions.checkArgument(strategy != null);
+    if (!initialized) {
+      initialize();
+    }
+
+    checkOpen();
+    int totalReadLen = 0;
+    int bufferLen = strategy.getTargetLength();

Review comment:
       `bufferLen` here represents the number of bytes remaining to be read into byteArray/ byteBuffer. Can we keep the variable name as `len` to avoid confusion with total length of buffer.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneInputStream.java
##########
@@ -52,6 +55,16 @@ public int read(byte[] b, int off, int len) throws IOException {
     return inputStream.read(b, off, len);
   }
 
+  @Override
+  public int read(ByteBuffer byteBuffer) throws IOException {
+    if (inputStream instanceof ByteBufferReadable) {
+      return ((ByteBufferReadable)inputStream).read(byteBuffer);
+    } else {
+      throw new UnsupportedOperationException("Read with ByteBuffer is not " +
+          " supported by this detail input stream");

Review comment:
       Instead of "this" can we specify the class of input stream?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteArrayStrategy.java
##########
@@ -0,0 +1,60 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import com.google.common.base.Preconditions;
+import java.io.IOException;
+import java.io.InputStream;
+import edu.umd.cs.findbugs.annotations.SuppressFBWarnings;
+
+public class ByteArrayStrategy implements ReaderStrategy {

Review comment:
       Can we rename this class to something to specify that it is a Reader (for example ByteArrayReader like other Readers - FileReader etc).

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStrategy.java
##########
@@ -0,0 +1,67 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.fs.ByteBufferReadable;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+public class ByteBufferStrategy implements ReaderStrategy {

Review comment:
       Same as for ByteArrayStrategy - It would be good if classname denotes its a Reader class, and class needs Javadoc.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ReaderStrategy.java
##########
@@ -0,0 +1,38 @@
+/*
+ * 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.hadoop.hdds.scm.storage;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+public interface ReaderStrategy {

Review comment:
       Javadoc missing.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyInputStream.java
##########
@@ -216,18 +222,32 @@ public synchronized int read() throws IOException {
    */
   @Override
   public synchronized int read(byte[] b, int off, int len) throws IOException {
-    checkOpen();
-    if (b == null) {
-      throw new NullPointerException();
-    }
-    if (off < 0 || len < 0 || len > b.length - off) {
-      throw new IndexOutOfBoundsException();
+    ReaderStrategy strategy = new ByteArrayStrategy(b, off, len);
+    int bufferLen = strategy.getTargetLength();
+    if (bufferLen == 0) {
+      return 0;
     }
-    if (len == 0) {
+    return readWithStrategy(strategy);
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer byteBuffer) throws IOException {
+    ReaderStrategy strategy = new ByteBufferStrategy(byteBuffer);
+    int bufferLen = strategy.getTargetLength();

Review comment:
       Same comment for `bufferLen`. It can be confused with length of the buffer instead of remaining bytes in the buffer/ array.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-842008131


   @hanishakoneru thanks the detail code review,  I have addressed all the comments in the new commit. 


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-842008131


   @hanishakoneru thanks the detail code review,  I have addressed all the comments in the new commit. 


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] jojochuang commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
jojochuang commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-836591754


   @hanishakoneru or @bshashikant should probably check this out too. Changes the client side inputstream implementation


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ChenSammi commented on pull request #2203: HDDS-5151. Support ByteBuffer read in OzoneInputStream

Posted by GitBox <gi...@apache.org>.
ChenSammi commented on pull request #2203:
URL: https://github.com/apache/ozone/pull/2203#issuecomment-852664952


   Thanks @hanishakoneru @adoroszlai @jojochuang for the code review.


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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org