szetszwo commented on code in PR #6613:
URL: https://github.com/apache/ozone/pull/6613#discussion_r2462932439
##########
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:
1. A client application opens the file.
2. It calls read() with 10KB.
3. It leaves the file open for 1 minute. `// a gRPC thread is blocked here`
4. It closes the file.
I see how it works now -- the implementation uses a `BlockingQueue
responseQueue`. When the responses are not removed from the queue, the `client
side` gRPC thread is blocked in `StreamingReader.onNext(..)` and the gRPC
streaming mechanism will throttle and slow down the server side.
However, the number open files will be limited by the number of gRPC
threads. Also, when all the gRPC threads are blocked, any gRPC calls will be
blocked. In an asynchronous model, threads should not blocked. When the
connection is idle at Step 3 above, it should release the thread so it can work
on other 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]