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


##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientGrpc.java:
##########
@@ -618,6 +624,69 @@ private void decreasePendingMetricsAndReleaseSemaphore() {
     return new XceiverClientReply(replyFuture);
   }
 
+  public XceiverClientReply sendCommandReadBlock(
+      ContainerCommandRequestProto request, DatanodeDetails dn)
+      throws IOException, InterruptedException {
+
+    CompletableFuture<ContainerCommandResponseProto> future =
+        new CompletableFuture<>();
+    ContainerCommandResponseProto.Builder response =
+        ContainerCommandResponseProto.newBuilder();
+    ContainerProtos.ReadBlockResponseProto.Builder readBlock =
+        ContainerProtos.ReadBlockResponseProto.newBuilder();
+    checkOpen(dn);
+    UUID dnID = dn.getUuid();
+    Type cmdType = request.getCmdType();
+    semaphore.acquire();
+    long requestTime = System.currentTimeMillis();
+    metrics.incrPendingContainerOpsMetrics(cmdType);
+
+    final StreamObserver<ContainerCommandRequestProto> requestObserver =
+        asyncStubs.get(dnID).withDeadlineAfter(timeout, TimeUnit.SECONDS)
+            .send(new StreamObserver<ContainerCommandResponseProto>() {
+              @Override
+              public void onNext(
+                  ContainerCommandResponseProto responseProto) {
+                if (responseProto.getResult() == Result.SUCCESS) {
+                  readBlock.addReadChunk(responseProto.getReadChunk());
+                } else {
+                  future.complete(
+                      ContainerCommandResponseProto.newBuilder(responseProto)
+                          .setCmdType(Type.ReadBlock).build());
+                }
+              }
+
+              @Override
+              public void onError(Throwable t) {
+                future.completeExceptionally(t);
+                metrics.decrPendingContainerOpsMetrics(cmdType);
+                metrics.addContainerOpsLatency(
+                    cmdType, System.currentTimeMillis() - requestTime);
+
+              }
+
+              @Override
+              public void onCompleted() {
+                if (readBlock.getReadChunkCount() > 0) {
+                  future.complete(response.setReadBlock(readBlock)
+                      
.setCmdType(Type.ReadBlock).setResult(Result.SUCCESS).build());
+                }
+                if (!future.isDone()) {
+                  future.completeExceptionally(new IOException(
+                      "Stream completed but no reply for request " +
+                          processForDebug(request)));
+                }
+                metrics.decrPendingContainerOpsMetrics(cmdType);
+                metrics.addContainerOpsLatency(
+                    cmdType, System.currentTimeMillis() - requestTime);
+              }
+            });
+    requestObserver.onNext(request);
+    requestObserver.onCompleted();
+    semaphore.release();

Review Comment:
   We cannot release the semaphore directly here.  Release the semaphore after 
the response is returned. 



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