yandrey321 commented on code in PR #9402:
URL: https://github.com/apache/ozone/pull/9402#discussion_r2589598245


##########
hadoop-ozone/ozonefs-common/src/main/java/org/apache/hadoop/fs/ozone/OzoneFSInputStream.java:
##########
@@ -215,42 +215,29 @@ public synchronized void readFully(long position, 
ByteBuffer buf) throws IOExcep
    * @throws IOException if there is an error performing the reads
    */
   @Override
-  public void readVectored(List<? extends FileRange> ranges,
-                           IntFunction<ByteBuffer> allocate) throws 
IOException {
+  public void readVectored(
+      List<? extends FileRange> ranges,
+      IntFunction<ByteBuffer> allocate
+  ) throws IOException {
     TracingUtil.executeInNewSpan("OzoneFSInputStream.readVectored", () -> {
-      // Perform vectored read using positioned read operations
-      for (FileRange range : ranges) {
-        CompletableFuture<ByteBuffer> result = range.getData();
-        if (result == null) {
-          result = new CompletableFuture<>();
-          range.setData(result);
-        }
-
-        final CompletableFuture<ByteBuffer> finalResult = result;
-        final long offset = range.getOffset();
-        final int length = range.getLength();
+      // Save the initial position
+      final long initialPosition = getPos();
 
-        // Submit async read task for this range
-        CompletableFuture.runAsync(() -> {
-          try {
-            ByteBuffer buffer = allocate.apply(length);
-            int bytesRead = read(offset, buffer);
+      try {
+        VectoredReadUtils.performVectoredRead(ranges, allocate, (offset, 
buffer) -> {
+          int length = buffer.remaining();
+          readFully(offset, buffer);
 
-            if (bytesRead < length) {
-              finalResult.completeExceptionally(
-                  new EOFException("Requested " + length +
-                      " bytes but only read " + bytesRead));
-            } else {
-              buffer.flip();
-              if (statistics != null) {
-                statistics.incrementBytesRead(bytesRead);
-              }
-              finalResult.complete(buffer);
-            }
-          } catch (Exception e) {
-            finalResult.completeExceptionally(e);
+          // Update statistics
+          if (statistics != null) {
+            statistics.incrementBytesRead(length);
           }
         });
+      } finally {
+        // Restore position
+        synchronized (this) {

Review Comment:
   what would happen in case of concurrent readVectored() calls?



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to