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


##########
hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/StreamBlockInputStream.java:
##########
@@ -0,0 +1,443 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hdds.scm.storage;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.Function;
+import org.apache.commons.lang3.NotImplementedException;
+import org.apache.hadoop.fs.ByteBufferReadable;
+import org.apache.hadoop.fs.CanUnbuffer;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.Seekable;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import 
org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ReadBlockResponseProto;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.StreamingReadResponse;
+import org.apache.hadoop.hdds.scm.StreamingReaderSpi;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+import 
org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.ratis.thirdparty.io.grpc.StatusRuntimeException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An {@link java.io.InputStream} called from KeyInputStream to read a block 
from the
+ * container.
+ */
+public class StreamBlockInputStream extends BlockExtendedInputStream
+    implements Seekable, CanUnbuffer, ByteBufferReadable {
+  private static final Logger LOG = 
LoggerFactory.getLogger(StreamBlockInputStream.class);
+  private static final int EOF = -1;
+  private static final Throwable CANCELLED_EXCEPTION = new 
Throwable("Cancelled by client");
+
+  private final BlockID blockID;
+  private final long blockLength;
+  private final AtomicReference<Pipeline> pipelineRef = new 
AtomicReference<>();
+  private final AtomicReference<Token<OzoneBlockTokenIdentifier>> tokenRef = 
new AtomicReference<>();
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientSpi xceiverClient;
+
+  private ByteBuffer buffer;
+  private long position = 0;
+  private boolean initialized = false;
+  private StreamingReader streamingReader;
+
+  private final boolean verifyChecksum;
+  private final Function<BlockID, BlockLocationInfo> refreshFunction;
+  private final RetryPolicy retryPolicy;
+  private int retries = 0;
+
+  public StreamBlockInputStream(
+      BlockID blockID, long length, Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token,
+      XceiverClientFactory xceiverClientFactory,
+      Function<BlockID, BlockLocationInfo> refreshFunction,
+      OzoneClientConfig config) throws IOException {
+    this.blockID = blockID;
+    this.blockLength = length;
+    pipelineRef.set(setPipeline(pipeline));
+    tokenRef.set(token);
+    this.xceiverClientFactory = xceiverClientFactory;
+    this.verifyChecksum = config.isChecksumVerify();
+    this.retryPolicy = getReadRetryPolicy(config);
+    this.refreshFunction = refreshFunction;
+  }
+
+  @Override
+  public BlockID getBlockID() {
+    return blockID;
+  }
+
+  @Override
+  public long getLength() {
+    return blockLength;
+  }
+
+  @Override
+  public synchronized long getPos() {
+    return position;
+  }
+
+  @Override
+  public synchronized int read() throws IOException {
+    checkOpen();
+    if (!dataAvailableToRead()) {
+      return EOF;
+    }
+    position++;
+    return buffer.get();
+  }
+
+  @Override
+  public synchronized int read(byte[] b, int off, int len) throws IOException {
+    ByteBuffer tmpBuffer = ByteBuffer.wrap(b, off, len);
+    return read(tmpBuffer);
+  }
+
+  @Override
+  public synchronized int read(ByteBuffer targetBuf) throws IOException {
+    checkOpen();
+    int read = 0;
+    while (targetBuf.hasRemaining()) {
+      if (!dataAvailableToRead()) {
+        break;
+      }
+      int toCopy = Math.min(buffer.remaining(), targetBuf.remaining());
+      ByteBuffer tmpBuf = buffer.duplicate();
+      tmpBuf.limit(tmpBuf.position() + toCopy);
+      targetBuf.put(tmpBuf);
+      buffer.position(tmpBuf.position());
+      position += toCopy;
+      read += toCopy;
+    }
+    return read > 0 ? read : EOF;
+  }
+
+  private boolean dataAvailableToRead() throws IOException {
+    if (position >= blockLength) {
+      return false;
+    }
+    initialize();
+    if (buffer == null || buffer.remaining() == 0) {
+      int loaded = fillBuffer();
+      return loaded != EOF;
+    }
+    return true;
+  }
+
+  @Override
+  protected int readWithStrategy(ByteReaderStrategy strategy) throws 
IOException {
+    throw new NotImplementedException("readWithStrategy is not implemented.");
+  }
+
+  @Override
+  public synchronized void seek(long pos) throws IOException {
+    checkOpen();
+    if (pos < 0) {
+      throw new IOException("Cannot seek to negative offset");
+    }
+    if (pos > blockLength) {
+      throw new IOException("Cannot seek after the end of the block");
+    }
+    if (pos == position) {
+      return;
+    }
+    closeStream();
+    position = pos;
+  }
+
+  @Override
+  // The seekable interface indicates that seekToNewSource should seek to a 
new source of the data,
+  // ie a different datanode. This is not supported for now.
+  public synchronized boolean seekToNewSource(long l) throws IOException {
+    return false;
+  }
+
+  @Override
+  public synchronized void unbuffer() {
+    releaseClient();
+  }
+
+  private void closeStream() {
+    if (streamingReader != null) {
+      streamingReader.cancel();
+      streamingReader = null;
+    }
+    initialized = false;
+    buffer = null;
+  }
+
+  protected synchronized void checkOpen() throws IOException {
+    if (xceiverClientFactory == null) {
+      throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED + " Block: " 
+ blockID);
+    }
+  }
+
+  protected synchronized void acquireClient() throws IOException {
+    checkOpen();
+    if (xceiverClient == null) {
+      final Pipeline pipeline = pipelineRef.get();
+      try {
+        xceiverClient = 
xceiverClientFactory.acquireClientForReadData(pipeline);
+      } catch (IOException ioe) {
+        LOG.warn("Failed to acquire client for pipeline {}, block {}", 
pipeline, blockID);
+        throw ioe;
+      }
+    }
+  }
+
+  private void initialize() throws IOException {
+    if (initialized) {
+      return;
+    }
+    while (true) {
+      try {
+        acquireClient();
+        streamingReader = new StreamingReader();
+        ContainerProtocolCalls.readBlock(xceiverClient, position, blockID, 
tokenRef.get(),
+            pipelineRef.get().getReplicaIndexes(), streamingReader);
+        initialized = true;
+        return;
+      } catch (InterruptedException ie) {
+        Thread.currentThread().interrupt();
+        handleExceptions(new IOException("Interrupted", ie));
+      } catch (IOException ioe) {
+        handleExceptions(ioe);
+      }
+    }
+  }
+
+  private void handleExceptions(IOException cause) throws IOException {
+    if (cause instanceof StorageContainerException || 
isConnectivityIssue(cause)) {
+      if (shouldRetryRead(cause, retryPolicy, retries++)) {
+        releaseClient();
+        refreshBlockInfo(cause);
+        LOG.warn("Refreshing block data to read block {} due to {}", blockID, 
cause.getMessage());
+      } else {
+        throw cause;
+      }
+    } else {
+      throw cause;
+    }
+  }
+
+  private int fillBuffer() throws IOException {
+    if (!streamingReader.hasNext()) {
+      return EOF;
+    }
+    buffer = streamingReader.readNext();
+    return buffer == null ? EOF : buffer.limit();
+  }
+
+  protected synchronized void releaseClient() {
+    if (xceiverClientFactory != null && xceiverClient != null) {
+      closeStream();
+      xceiverClientFactory.releaseClientForReadData(xceiverClient, false);
+      xceiverClient = null;
+    }
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    releaseClient();
+    xceiverClientFactory = null;
+  }
+
+  private void refreshBlockInfo(IOException cause) throws IOException {
+    refreshBlockInfo(cause, blockID, pipelineRef, tokenRef, refreshFunction);
+  }
+
+  private synchronized void releaseStreamResources(StreamingReadResponse 
response) {
+    if (xceiverClient != null) {
+      xceiverClient.completeStreamRead(response);
+    }
+  }
+
+  /**
+   * Implementation of a StreamObserver used to received and buffer streaming 
GRPC reads.
+   */
+  public class StreamingReader implements StreamingReaderSpi {
+
+    private final BlockingQueue<ContainerProtos.ReadBlockResponseProto> 
responseQueue = new LinkedBlockingQueue<>(1);

Review Comment:
   Added `client side`  to 
https://github.com/apache/ozone/pull/6613#discussion_r2462932439
   
   > ...  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`.
   
   



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

Reply via email to