sodonnel commented on code in PR #6613:
URL: https://github.com/apache/ozone/pull/6613#discussion_r2462675777


##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java:
##########
@@ -504,6 +516,65 @@ private XceiverClientReply sendCommandWithRetry(
     }
   }
 
+  /**
+   * Starts a streaming read operation, intended to read entire blocks from 
the datanodes. This method expects a
+   * {@link StreamObserver} to be passed in, which will be used to receive the 
streamed data from the datanode.
+   * Upon successfully starting the streaming read, a {@link 
StreamingReadResponse} is returned, which contains
+   * information about the datanode used for the read, and the request 
observer that can be used to manage the stream
+   * (e.g., to cancel it if needed). A semaphore is acquired to limit the 
number of concurrent streaming reads so upon
+   * successful return of this method, the caller must ensure to call {@link 
#completeStreamRead(StreamingReadResponse)}
+   * to release the semaphore once the streaming read is complete.
+   * @param request The container command request to initiate the streaming 
read.
+   * @param streamObserver The observer that will handle the streamed 
responses.
+   * @return A {@link StreamingReadResponse} containing details of the 
streaming read operation.
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  @Override
+  public StreamingReadResponse streamRead(ContainerCommandRequestProto request,
+      StreamObserver<ContainerCommandResponseProto> streamObserver) throws 
IOException, InterruptedException {
+    List<DatanodeDetails> datanodeList = sortDatanodes(request);
+    IOException lastException = null;
+    for (DatanodeDetails dn : datanodeList) {
+      try {
+        checkOpen(dn);
+        semaphore.acquire();
+        XceiverClientProtocolServiceStub stub = asyncStubs.get(dn.getID());
+        if (stub == null) {
+          throw new IOException("Failed to get gRPC stub for DataNode: " + dn);
+        }
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Executing command {} on datanode {}", 
processForDebug(request), dn);
+        }
+        StreamObserver<ContainerCommandRequestProto> requestObserver = stub
+            .withDeadlineAfter(timeout, TimeUnit.SECONDS)
+            .send(streamObserver);
+        requestObserver.onNext(request);
+        requestObserver.onCompleted();
+        return new StreamingReadResponse(dn, 
(ClientCallStreamObserver<ContainerCommandRequestProto>) requestObserver);

Review Comment:
   The implementation is just streaming the whole block each time and removing 
the length part. The idea being that the stream blocks on the socket until more 
is read. This new block input stream just starts reading from whatever position 
you tell it to start and then blocks until the client consumes it. Then reads 
more. The length part of the read isn’t there. Except at the higher level 
streams.
   
   With a 10kb read the server will respond with a checksum sized read of 16kb 
or 1mb if that is the checksum size. Then from my experiments the server will 
keep sending consecutive chunks up to 1mb which will block on the socket until 
the client pulls them off. Then the server will block sending until it can get 
more out.  (edited) 



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