You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ratis.apache.org by "szetszwo (via GitHub)" <gi...@apache.org> on 2023/04/26 15:52:54 UTC

[GitHub] [ratis] szetszwo commented on a diff in pull request #877: RATIS-1837. Restrict reading maxChunkSize bytes each installSnapshot RPC

szetszwo commented on code in PR #877:
URL: https://github.com/apache/ratis/pull/877#discussion_r1178072913


##########
ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java:
##########
@@ -65,14 +72,16 @@ public FileChunkReader(FileInfo info, Path relativePath) throws IOException {
   /**
    * Read the next chunk.
    *
-   * @param chunkMaxSize maximum chunk size
    * @return the chunk read from the file.
    * @throws IOException if it failed to read the file.
    */
-  public FileChunkProto readFileChunk(int chunkMaxSize) throws IOException {
+  public FileChunkProto readFileChunk() throws IOException {
     final long remaining = info.getFileSize() - offset;
     final int chunkLength = remaining < chunkMaxSize ? (int) remaining : chunkMaxSize;
-    final ByteString data = ByteString.readFrom(in, chunkLength);
+    final int bytesRead = in.read(chunkBuffer, 0, chunkLength);

Review Comment:
   Let's use `IOUtils.readFully`.  It will read in a loop.



##########
ratis-server/src/main/java/org/apache/ratis/server/storage/FileChunkReader.java:
##########
@@ -47,11 +51,14 @@ public class FileChunkReader implements Closeable {
    *
    * @param info the information of the file.
    * @param relativePath the relative path of the file.
+   * @param chunkMaxSize maximum chunk size for each {@link #readFileChunk()} call
    * @throws IOException if it failed to open the file.
    */
-  public FileChunkReader(FileInfo info, Path relativePath) throws IOException {
+  public FileChunkReader(FileInfo info, Path relativePath, int chunkMaxSize) throws IOException {
     this.info = info;
     this.relativePath = relativePath;
+    this.chunkMaxSize = Math.min(chunkMaxSize, (int) info.getFileSize());
+    this.chunkBuffer = new byte[chunkMaxSize];

Review Comment:
   Since `ByteString` is supposed to be immutable, `FileChunkProto.Builder` won't copy the `data` in its `build()` method.  If we only have a single buffer, we can only have a single outstanding `FileChunkProto` (and a single outstanding `InstallSnapshotRequestProto`).  Otherwise, if we read a second `FileChunkProto`, the `data` in first `FileChunkProto` will be changed silently.  (That's why it is named `UnsafeByteOperations`.)
   



-- 
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-unsubscribe@ratis.apache.org

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