You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@ozone.apache.org by GitBox <gi...@apache.org> on 2021/08/04 07:34:40 UTC

[GitHub] [ozone] ckj996 opened a new pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

ckj996 opened a new pull request #2495:
URL: https://github.com/apache/ozone/pull/2495


   ## What changes were proposed in this pull request?
   
   **Request for Comments**
   This is our streaming client implementation. It’s still in the early stages, but at least it works.
   This might be a good start to discuss the design, comments are welcomed.
   
   New DataStreamOutput were added, compared with OutputStream, there are some outstanding changes:
   
   * `write(ByteBuf)` methods were added, which can be used for zero buffer copy later.
   * `write(byte[])` methods were kept, so these DataStreamOutput still extends OutputStream for compatibility.
   * `write(ByteBuf)` is perferred over `write(byte[])`, the latter is implemented by calling the former.
   * Zero buffer copy is not implemented yet, there is still buffer copy in `BlockDataStreamOutput#write`.
   * OzoneDataStreamOutput only supports KeyDataStreamOutput for now, Crypto and Cipher were dropped.
   
   Also, we have modified the ozone shell to use the streaming api for putting large keys (length >= chunkSize).
   
   ## What is the link to the Apache JIRA
   
   https://issues.apache.org/jira/browse/HDDS-5488
   
   ## How was this patch tested?
   
   If this design is ok, unit tests should be added.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893243191






-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683903010



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       However, most fields and methods we are trying to reuse are private. Is it a good idea to make them protected?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683892677



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,767 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    if (len == 0) {
+      return;
+    }
+    ChunkBuffer chunk = ChunkBuffer.wrap(b.nioBuffer());
+    writeChunk(chunk);
+    writtenDataLength += len;
+    updateFlushLength();
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf);
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));

Review comment:
       There is still buffer copy here :(




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684578491



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;

Review comment:
       fixed in f98b3ee




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684642531



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;

Review comment:
       Actually, using ByteBuf here is okay.  DataStreamOutput is not really a client API.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893329582


   @ckj996 and @captainzmc , thanks a lot for working on this!  Will review this tomorrow.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-897498624


   > I think merge will also work. It's easier to deal with conflicts if we are doing merge.
   
   That's true.
   
   In our case, we don't expect a lot of conflicts since we are mainly adding new code.  How about we try rebase first?  If there are too many conflicts, we may switch to merge.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-896563753


   At present, CI is not stable, the [CI on private branch is successful.  ](https://github.com/kaijchen/ozone/runs/3296734309)


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894049337


   There is still some issues in my attempts to avoid the buffer copy into bufferPool.
   I have reverted it. And I will try it in [another branch](/kaijchen/ozone/tree/zero-copy-streaming-client).
   
   Let's focus on the overall design first.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894086434


   In BlockOutputStream, we want to write chunk in a fixed size (e.g. 4MB).
   
   However, in Streaming, we do not have this restriction.  We can write any size to the underlying stream.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682428512



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       make sure writes completed before PutBlock




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684004522



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An DataStreamOutput accepts netty.buffer.ByteBuf and sends them to some sink.
+*/
+public interface DataStreamOutput extends Closeable {

Review comment:
       Let's call this ByteBufferStreamOutput to avoid the name conflict.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;

Review comment:
       For zero buffer copying, BufferPool and ChunkBuffer should not be used anymore.  We get he ByteBuf from the API and then pass it down directly.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream

Review comment:
       Let's do not extend OutputStream in here so that we can simply the code.  We may add it later if there is a need.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;

Review comment:
       Let use ByteBuffer instead of ByteBuf here since this is a client API.  We don't want to force client to use ByteBuf.
   For our internal code, we may use ByteBuf for its efficiency.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;

Review comment:
       It should be
   ```
       int off = b.readerIndex();
   ```




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683894813



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,767 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    if (len == 0) {
+      return;
+    }
+    ChunkBuffer chunk = ChunkBuffer.wrap(b.nioBuffer());
+    writeChunk(chunk);
+    writtenDataLength += len;
+    updateFlushLength();
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf);
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));

Review comment:
       What's the difference between `duplicate` and `slice`?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682433101



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {

Review comment:
       interface name `org.apache.hadoop.hdds.scm.storage.DataStreamOutput` conflicts with `org.apache.ratis.client.api.DataStreamOutput`




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682429362



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());

Review comment:
       open stream




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893239885


   @szetszwo , can you have a look as well?
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894120792


   PutBlock is used outside the streaming pat and there will be log entry created for it. CommitWatcher will still be needed to ensure the associated log is replicated to all.
   
   The writeChunkToContainer can be executed as soon as there is data in the buffer. This should resolve the bursty IO pattern.
   The full buffer condition will never be hit in this case, bcoz before putBlock call we ensure all writes are acknowledged.
   
   The full buffer condition will make sense only if we send putBlock in the streaming path itself and once the buffer hits limit , it should just wait for all the acknowledgement. 


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-895955262


   > Two main comments:
   > 
   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo,  Currently this PR already supports not extends OutputStream. Regarding the second problem mentioned above "not use BufferPool and ChunkBuffer to avoid buffer copying", As we discussed  at the meeting on Monday, we are going to fix this in the next PR HDDS-5599.   We will continue to optimize this client,  for now let's make our client  supports  write a key  use streaming first.
   So can you take another look at this PR.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-895694302


   Can anyone rebase the HDDS-4454 branch to see if the CI issue got fixed?
   
   As said in meeting.
   
   Thanks.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683894813



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,767 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    if (len == 0) {
+      return;
+    }
+    ChunkBuffer chunk = ChunkBuffer.wrap(b.nioBuffer());
+    writeChunk(chunk);
+    writtenDataLength += len;
+    updateFlushLength();
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf);
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));

Review comment:
       What's the difference between `duplicate` and `slice`?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682426003



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();

Review comment:
       close stream asynchronously




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893925919


   I think we should probably stop using the `ChunkBuffer` pool in `BlockDataStreamOutput` because the chunks written through streaming does not appear in Raft log, there is nothing to `watchForCommit` for.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682434796



##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
##########
@@ -111,10 +117,36 @@ protected void execute(OzoneClient client, OzoneAddress address)
 
     int chunkSize = (int) getConf().getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY,
         OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES);
-    try (InputStream input = new FileInputStream(dataFile);
-        OutputStream output = bucket.createKey(keyName, dataFile.length(),
-            replicationConfig, keyMetadata)) {
-      IOUtils.copyBytes(input, output, chunkSize);
+
+    if (dataFile.length() <= chunkSize) {
+      if (isVerbose()) {
+        out().println("API: async");
+      }
+      try (InputStream input = new FileInputStream(dataFile);
+           OutputStream output = bucket.createKey(keyName, dataFile.length(),
+               replicationConfig, keyMetadata)) {
+        IOUtils.copyBytes(input, output, chunkSize);
+      }
+    } else {
+      if (isVerbose()) {
+        out().println("API: streaming");
+      }
+      try (RandomAccessFile raf = new RandomAccessFile(dataFile, "r");
+           OzoneDataStreamOutput out = bucket.createStreamKey(keyName,
+               dataFile.length(), replicationConfig, keyMetadata)) {
+        FileChannel ch = raf.getChannel();
+        long len = raf.length();
+        long off = 0;
+        while (len > 0) {
+          long writeLen = Math.min(len, chunkSize);
+          ByteBuffer segment =
+              ch.map(FileChannel.MapMode.READ_ONLY, off, writeLen);
+          ByteBuf buf = Unpooled.wrappedBuffer(segment);
+          out.write(buf);

Review comment:
       write `netty.buffer.ByteBuf` instead of write `byte[]`




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682436701



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/OzoneDataStreamOutput.java
##########
@@ -0,0 +1,90 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.ozone.client.io;
+
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.scm.storage.DataStreamOutput;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+
+import java.io.IOException;
+import java.io.OutputStream;
+
+/**
+ * OzoneOutputStream is used to write data into Ozone.
+ * It uses SCM's {@link KeyOutputStream} for writing the data.
+ */
+public class OzoneDataStreamOutput extends OutputStream
+    implements DataStreamOutput {
+
+  private final DataStreamOutput dataStreamOutput;
+
+  /**
+   * Constructs OzoneOutputStream with KeyOutputStream.
+   *
+   * @param outputStream
+   */
+  public OzoneDataStreamOutput(DataStreamOutput dataStreamOutput) {
+    this.dataStreamOutput = dataStreamOutput;
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    dataStreamOutput.write(b);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf, 0, 1);
+  }
+
+  @Override
+  public void write(byte[] b) throws IOException {
+    write(b, 0, b.length);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b, off, len));
+  }
+
+  @Override
+  public synchronized void flush() throws IOException {
+    dataStreamOutput.flush();
+  }
+
+  @Override
+  public synchronized void close() throws IOException {
+    //commitKey can be done here, if needed.
+    dataStreamOutput.close();
+  }
+
+  public OmMultipartCommitUploadPartInfo getCommitUploadPartInfo() {
+    if (dataStreamOutput instanceof KeyDataStreamOutput) {
+      return ((KeyDataStreamOutput) dataStreamOutput).getCommitUploadPartInfo();
+    }

Review comment:
       support for `CipherDataStreamOutput` and `CryptoDataStreamOutput` is dropped




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683180876



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       > Why the stream is being setup here?
   
   It seems natural to setup the stream on creation of the BlockDataStreamOutput.
   Alternatively, we can delay it until just before writing the first chunk.
   
   > Why its called with write chunk data?
   
   Execute a dummy write request just for getting the path of the target file, but not writing any data to it.
   The data write happens in `writeChunkToContainer()`.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       Yeah, we can do it if it shares many methods in the final design.
   
   Actually the HDDS-4454 design says the `BlockDataStreamOutput` should no longer extend `OutputStream`.
   So does @szetszwo said in #2442.
   
   However, I'm keeping it because it seems easy to translate `write(byte[])` to `write(ByteBuf)`.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable

Review comment:
       > Can't we send putblock command in the streaming path itself with close?
   
   This is the "Approach 1: Separate Data and Metadata path" vs "Approach 2 : Streaming Path for both Data and MetaData" in HDDS-4454.
   
   > We don't have hsync feature build in yet in Ozone. Can we make it zero for now?
   
   Yes.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));

Review comment:
       We need to eliminate buffer copy here. 
   For now, we just want to get some comments about the design.
   We will look into this later.
   
   The `currentBuffer` is managed by `BufferPool`, who allocates and reuses `ChunkBuffer`.
   Currently we just copy data from our `ByteBuf` into that `ChunkBuffer`.
   If we want to give it a `ByteBuffer` and say "hey, just wrap this buffer as `ChunkBuffer`", extra care should be taken.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Sure. Maybe we should move this into a new method for clarity.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       > Can't we send putblock command in the streaming path itself with close?
   
   This is the "Approach 1: Separate Data and Metadata path" vs "Approach 2 : Streaming Path for both Data and MetaData" in HDDS-4454.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable

Review comment:
       > We don't have hsync feature build in yet in Ozone. Can we make it zero for now?
   
   Yes.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-895758526


   > Can anyone rebase the [HDDS-4454](https://issues.apache.org/jira/browse/HDDS-4454) branch to see if the CI issue got fixed?
   
   I have merged the latest master into the branch of HDDS-4454.
   
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-897513452


   Since there is a discussion going on in the dev mailing list.  Let's merge this first. We can do the branch cleanup afterward.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894120792


   PutBlock is used outside the streaming pat and there will be log entry created for it. CommitWatcher will still be needed to ensure the associated log is replicated to all.
   
   The writeChunkToContainer can be executed as soon as there is data in the buffer. This should resolve the bursty IO pattern.
   The full buffer condition will never be hit in this case, bcoz before putBlock call we ensure all writes are acknowledged.
   
   The full buffer condition will make sense only if we send putBlock in the streaming path itself and once the buffer hits limit , it should just wait for all the acknowledgement. 
   
   I am still in process of reviewing this.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683170775



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       In my opinion, this class needs to extend the abstract BlockOutputStream/RatisBlockOutputClass  in master class and implement and ovverride methods as necessary methods?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686467261



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -528,6 +529,43 @@ public OzoneOutputStream createKey(String key, long size,
         .createKey(volumeName, name, key, size, replicationConfig, keyMetadata);
   }
 
+  /**
+   * Creates a new key in the bucket.
+   * @param key Name of the key to be created.
+   * @param size Size of the data the key will point to.
+   * @param type Replication type to be used.
+   * @param factor Replication factor of the key.
+   * @return OzoneDataStreamOutput to which the data has to be written.
+   * @throws IOException
+   */
+  @Deprecated
+  public OzoneDataStreamOutput createStreamKey(String key, long size,

Review comment:
       fixed in 7115b48 and 5f5cdac

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -801,6 +803,59 @@ public OzoneOutputStream createKey(
     return createOutputStream(openKey, requestId, replicationConfig);
   }
 
+  @Override
+  @Deprecated
+  public OzoneDataStreamOutput createStreamKey(String volumeName,

Review comment:
       fixed in 7115b48 and 5f5cdac




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894618954


   > BlockDataStreamOutput and the other new classes are internal classes. They are not user facing classes. Users will use OzoneOutputStream. If a byte[] is passed from a user, say via OzoneOutputStream.write(byte[]), we may wrap the array to a ByteBuffer internally.
   > 
   > BTW, we should add ByteBuffer methods to OzoneOutputStream later. 
   
   Thanks @szetszwo,I get you point. Yes, we can wrap byte[] to ByteBuffer in OzoneOutputStream. The BlockDataStreamOutput does not need to extends OutputStream.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-896143846


   Sure, we can fix the second problem later.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-897248710


   > I think we should keep rebasing the changes from the master onto the HDDS-4454 branch.  Otherwise, it is hard to merge HDDS-4454 back to the master in the future. Any thought?
   
   +1, sounds good


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc removed a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc removed a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-896563753


   At present, CI is not stable, the [CI on private branch is successful.  ](https://github.com/kaijchen/ozone/runs/3296734309)


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683180876



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       > Why the stream is being setup here?
   
   It seems natural to setup the stream on creation of the BlockDataStreamOutput.
   Alternatively, we can delay it until just before writing the first chunk.
   
   > Why its called with write chunk data?
   
   Execute a dummy write request just for getting the path of the target file, but not writing any data to it.
   The data write happens in `writeChunkToContainer()`.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-897387110


   > I think we should keep rebasing the changes from the master onto the HDDS-4454 branch. Otherwise, it is hard to merge HDDS-4454 back to the master in the future. Any thought?
   
   I think merge will also work. It's easier to deal with conflicts if we are doing merge.
   
   Rebase will give a clearer history but requires more work to deal with conflicts.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682426488



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));
+  }
+
+  /**
+   * @param close whether the flush is happening as part of closing the stream
+   */
+  private void handleFlush(boolean close)
+      throws IOException, InterruptedException, ExecutionException {
+    checkOpen();
+    // flush the last chunk data residing on the currentBuffer
+    if (totalDataFlushedLength < writtenDataLength) {
+      refreshCurrentBuffer(bufferPool);
+      Preconditions.checkArgument(currentBuffer.position() > 0);
+      if (currentBuffer.hasRemaining()) {
+        writeChunk(currentBuffer);
+      }
+      // This can be a partially filled chunk. Since we are flushing the buffer
+      // here, we just limit this buffer to the current position. So that next
+      // write will happen in new buffer
+      updateFlushLength();
+      executePutBlock(close, false);
+    } else if (close) {
+      // forcing an "empty" putBlock if stream is being closed without new
+      // data since latest flush - we need to send the "EOF" flag
+      executePutBlock(true, true);
+    }
+    waitOnFlushFutures();
+    watchForCommit(false);
+    // just check again if the exception is hit while waiting for the
+    // futures to ensure flush has indeed succeeded
+
+    // irrespective of whether the commitIndex2flushedDataMap is empty
+    // or not, ensure there is no exception set
+    checkOpen();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0) {
+      try {
+        handleFlush(true);
+        dataStreamCloseReply.get();

Review comment:
       confirm stream closed

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();

Review comment:
       close stream asynchronously




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684730073



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       The DataStreamOutputs no longer extends OutputStream for simplicity.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893329582


   @ckj996 and @captainzmc , thanks a lot for working on this!  Will review this tomorrow.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894085044


   Since we are going to use Streaming in BlockDataStreamOutput, the write logic is different from BlockOutputStream.
   - In BlockOutputStream, we copy the data to a ChunkBuffer in write(byte[] b, int off, int len).  We won't push the data out to the network until the ChunkBuffer is full.  When the ChunkBuffer is full, we push it out to the network in writeChunkToContainer(ChunkBuffer).  We also convert the header to proto and the data to ByteString.  Then, we use commitWatcher to wait for the commit.
   - For Streaming (i.e. BlockDataStreamOutput), we should write the header (using proto) when creating the stream or right before the first write.  In write(ByteBuffer), we will pass the ByteBuffer (without converting it to ByteString) to the underlying stream immediately since we don't need to wait for any buffer becoming full.  We also don't need commitWatcher anymore.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-895955262


   > Two main comments:
   > 
   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo,  Currently this PR already supports not extends OutputStream. Regarding the second problem mentioned above "not use BufferPool and ChunkBuffer to avoid buffer copying", As we discussed  at the meeting on Monday, we are going to fix this in the next PR HDDS-5599.   We will continue to optimize this client,  for now let's make our client  supports  write a key  use streaming first.
   So can you take another look at this PR?
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894114695


   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo, I have a question about compatibility here.  User often use HDFS APi, using ofs:// or o3fs:// to access Ozone. The old app used like this:
   `FSDataOutputStream fsOut = fs.create(new Path(pathStr), true);`
   `fsOut.write(str.getBytes());`
   Old applications, such as MapReduce or Spark Tasks, pass in a byte array, which is difficult to change.
   So can we still keep the OutputStream? Because we need its write(byte b[]).  Although using this API will increase the buffer copy of the client, it will resolve compatibility issues.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684578544



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An DataStreamOutput accepts netty.buffer.ByteBuf and sends them to some sink.
+*/
+public interface DataStreamOutput extends Closeable {

Review comment:
       changed in d91de0d




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686138677



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/OzoneBucket.java
##########
@@ -528,6 +529,43 @@ public OzoneOutputStream createKey(String key, long size,
         .createKey(volumeName, name, key, size, replicationConfig, keyMetadata);
   }
 
+  /**
+   * Creates a new key in the bucket.
+   * @param key Name of the key to be created.
+   * @param size Size of the data the key will point to.
+   * @param type Replication type to be used.
+   * @param factor Replication factor of the key.
+   * @return OzoneDataStreamOutput to which the data has to be written.
+   * @throws IOException
+   */
+  @Deprecated
+  public OzoneDataStreamOutput createStreamKey(String key, long size,

Review comment:
       Since this is already deprecated, let's don't add it.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686467905



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;

Review comment:
       We are going to fix this issue in HDDS-5599.
   
   I'll mark this as resolved here.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683208867



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable

Review comment:
       > Can't we send putblock command in the streaming path itself with close?
   
   This is the "Approach 1: Separate Data and Metadata path" vs "Approach 2 : Streaming Path for both Data and MetaData" in HDDS-4454.
   
   > We don't have hsync feature build in yet in Ozone. Can we make it zero for now?
   
   Yes.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894114695


   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo, I have a question about compatibility here.  We are using HDFS APi, using OFS or o3FS to access Ozone. The old app used like this:
   `FSDataOutputStream fsOut = fs.create(new Path(dst), true);`
   `fsOut.write(str.getBytes());`
   Old applications, such as MapReduce or Spark Tasks, pass in a byte array, which is difficult to change.
   So can we still keep the OutputStream because we need its write(byte b[])?  Although using this API will increase the buffer copy of the client, it will resolve compatibility issues.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894410844


   > Hi @szetszwo, I have a question about compatibility here. User often use HDFS APi, using ofs:// or o3fs:// to access Ozone. 
   
   BlockDataStreamOutput and the other new classes are internal classes.  They are not user facing classes.  Users will use  OzoneOutputStream.  If a byte[] is passed from a user, say via OzoneOutputStream.write(byte[]), we may wrap the array to a ByteBuffer internally.
   
   BTW, we should add ByteBuffer methods to OzoneOutputStream later.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683171171



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Can you please add a comment for why the stream is being setup here, and why its called With write chunk data?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682426488



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));
+  }
+
+  /**
+   * @param close whether the flush is happening as part of closing the stream
+   */
+  private void handleFlush(boolean close)
+      throws IOException, InterruptedException, ExecutionException {
+    checkOpen();
+    // flush the last chunk data residing on the currentBuffer
+    if (totalDataFlushedLength < writtenDataLength) {
+      refreshCurrentBuffer(bufferPool);
+      Preconditions.checkArgument(currentBuffer.position() > 0);
+      if (currentBuffer.hasRemaining()) {
+        writeChunk(currentBuffer);
+      }
+      // This can be a partially filled chunk. Since we are flushing the buffer
+      // here, we just limit this buffer to the current position. So that next
+      // write will happen in new buffer
+      updateFlushLength();
+      executePutBlock(close, false);
+    } else if (close) {
+      // forcing an "empty" putBlock if stream is being closed without new
+      // data since latest flush - we need to send the "EOF" flag
+      executePutBlock(true, true);
+    }
+    waitOnFlushFutures();
+    watchForCommit(false);
+    // just check again if the exception is hit while waiting for the
+    // futures to ensure flush has indeed succeeded
+
+    // irrespective of whether the commitIndex2flushedDataMap is empty
+    // or not, ensure there is no exception set
+    checkOpen();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0) {
+      try {
+        handleFlush(true);
+        dataStreamCloseReply.get();

Review comment:
       confirm stream closed




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682447376



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An DataStreamOutput accepts netty.buffer.ByteBuf and sends them to some sink.
+*/
+public interface DataStreamOutput extends Closeable {
+  /**
+   * Try to write all the bytes in ByteBuf b to DataStream.
+   *
+   * @param b the data.
+   * @exception IOException if an I/O error occurs.
+   */
+  void write(ByteBuf b) throws IOException;
+
+  /**
+   * Try to write the [off:off + len) slice in ByteBuf b to DataStream.
+   *
+   * @param b the data.
+   * @param off the start offset in the data.
+   * @param len the number of bytes to write.
+   * @exception  IOException  if an I/O error occurs.
+   */
+  default void write(ByteBuf b, int off, int len) throws IOException {
+    write(b.slice(off, len));

Review comment:
       `slice(int, int)` does not copy the underlying buffer.
   maybe we should use `retainedSlice(int, int)` here to increase the reference count to the underlying buffer.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682424324



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));

Review comment:
       buffer copy here




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894114695


   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo, I have a question about compatibility here.  We are using HDFS APi, using OFS or o3FS to access Ozone. The old app used like this:
   `FSDataOutputStream fsOut = fs.create(new Path(dst), true);`
   `fsOut.write(str.getBytes());`
   Old applications, such as MapReduce or Spark Tasks, pass in a byte array, which is difficult to change.
   So can we still keep the OutputStream? Because we need its write(byte b[]).  Although using this API will increase the buffer copy of the client, it will resolve compatibility issues.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683208309



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Yes, i understand that. Can you this explicit by adding comments in the code?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Yes, i understand that. Can you make this explicit by adding comments in the code?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683173759



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       Isn't waiting for putBlock impliclitly mean all the pending write chunks are completed? We don't need to wait for all the write chunk futures to complete before issuing a putblock call.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683185342



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable

Review comment:
       We don't have hsync feature build in yet in Ozone. Can we make it zero for now?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       We need to implicitly wait for write chunks to complete before sending out the putblock command only if putBllock is happening outside the streaming path.   Can't we send putblock command in the streaming path itself with close?

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
##########
@@ -111,10 +117,36 @@ protected void execute(OzoneClient client, OzoneAddress address)
 
     int chunkSize = (int) getConf().getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY,
         OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES);
-    try (InputStream input = new FileInputStream(dataFile);
-        OutputStream output = bucket.createKey(keyName, dataFile.length(),
-            replicationConfig, keyMetadata)) {
-      IOUtils.copyBytes(input, output, chunkSize);
+
+    if (dataFile.length() <= chunkSize) {
+      if (isVerbose()) {
+        out().println("API: async");
+      }
+      try (InputStream input = new FileInputStream(dataFile);
+           OutputStream output = bucket.createKey(keyName, dataFile.length(),
+               replicationConfig, keyMetadata)) {
+        IOUtils.copyBytes(input, output, chunkSize);
+      }
+    } else {
+      if (isVerbose()) {
+        out().println("API: streaming");
+      }
+      try (RandomAccessFile raf = new RandomAccessFile(dataFile, "r");
+           OzoneDataStreamOutput out = bucket.createStreamKey(keyName,
+               dataFile.length(), replicationConfig, keyMetadata)) {
+        FileChannel ch = raf.getChannel();
+        long len = raf.length();
+        long off = 0;
+        while (len > 0) {
+          long writeLen = Math.min(len, chunkSize);
+          ByteBuffer segment =
+              ch.map(FileChannel.MapMode.READ_ONLY, off, writeLen);
+          ByteBuf buf = Unpooled.wrappedBuffer(segment);
+          out.write(buf);

Review comment:
       yes, we should use netty constructs.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894114695


   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo, I have a question about compatibility here.  We are using HDFS APi, using ofs:// or o3fs:// to access Ozone. The old app used like this:
   `FSDataOutputStream fsOut = fs.create(new Path("o3fs://xxxx"), true);`
   `fsOut.write(str.getBytes());`
   Old applications, such as MapReduce or Spark Tasks, pass in a byte array, which is difficult to change.
   So can we still keep the OutputStream? Because we need its write(byte b[]).  Although using this API will increase the buffer copy of the client, it will resolve compatibility issues.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894114695


   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo, I have a question about compatibility here.  We are using HDFS APi, using ofs:// or o3fs:// to access Ozone. The old app used like this:
   `FSDataOutputStream fsOut = fs.create(new Path(pathStr), true);`
   `fsOut.write(str.getBytes());`
   Old applications, such as MapReduce or Spark Tasks, pass in a byte array, which is difficult to change.
   So can we still keep the OutputStream? Because we need its write(byte b[]).  Although using this API will increase the buffer copy of the client, it will resolve compatibility issues.
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683173759



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       Isn't waiting for putBlock impliclitly mean all the pending write chunks are completed? We don't need to wait for all the write chunk futures to complete before issuing a putblock call.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686467075



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An ByteBufferStreamOutput accepts Netty ByteBuf and sends them to some sink.
+*/
+public interface ByteBufferStreamOutput extends Closeable {

Review comment:
       fixed in efe65ab




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683186028



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       Yeah, we can do it if it shares many methods in the final design.
   
   Actually the HDDS-4454 design says the `BlockDataStreamOutput` should no longer extend `OutputStream`.
   So does @szetszwo said in #2442.
   
   However, I'm keeping it because it seems easy to translate `write(byte[])` to `write(ByteBuf)`.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686146415



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
+import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Maintaining a list of BlockInputStream. Write based on offset.
+ *
+ * Note that this may write to multiple containers in one write call. In case
+ * that first container succeeded but later ones failed, the succeeded writes
+ * are not rolled back.
+ *
+ * TODO : currently not support multi-thread access.
+ */
+public class KeyDataStreamOutput implements ByteBufferStreamOutput {
+
+  private OzoneClientConfig config;
+
+  /**
+   * Defines stream action while calling handleFlushOrClose.
+   */
+  enum StreamAction {
+    FLUSH, CLOSE, FULL
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KeyDataStreamOutput.class);
+
+  private boolean closed;
+  private FileEncryptionInfo feInfo;
+  private final Map<Class<? extends Throwable>, RetryPolicy> retryPolicyMap;
+  private int retryCount;
+  // how much of data is actually written yet to underlying stream
+  private long offset;
+  // how much data has been ingested into the stream
+  private long writeOffset;
+  // whether an exception is encountered while write and whole write could
+  // not succeed
+  private boolean isException;
+  private final BlockDataStreamOutputEntryPool blockDataStreamOutputEntryPool;
+
+  private long clientID;
+
+  /**
+   * A constructor for testing purpose only.
+   */
+  @VisibleForTesting
+  public KeyDataStreamOutput() {
+    closed = false;
+    this.retryPolicyMap = HddsClientUtils.getExceptionList()
+        .stream()
+        .collect(Collectors.toMap(Function.identity(),
+            e -> RetryPolicies.TRY_ONCE_THEN_FAIL));
+    retryCount = 0;
+    offset = 0;
+    blockDataStreamOutputEntryPool = new BlockDataStreamOutputEntryPool();
+  }
+
+  @VisibleForTesting
+  public List<BlockDataStreamOutputEntry> getStreamEntries() {
+    return blockDataStreamOutputEntryPool.getStreamEntries();
+  }
+
+  @VisibleForTesting
+  public XceiverClientFactory getXceiverClientFactory() {
+    return blockDataStreamOutputEntryPool.getXceiverClientFactory();
+  }
+
+  @VisibleForTesting
+  public List<OmKeyLocationInfo> getLocationInfoList() {
+    return blockDataStreamOutputEntryPool.getLocationInfoList();
+  }
+
+  @VisibleForTesting
+  public int getRetryCount() {
+    return retryCount;
+  }
+
+  @VisibleForTesting
+  public long getClientID() {
+    return clientID;
+  }
+
+  @SuppressWarnings({"parameternumber", "squid:S00107"})
+  public KeyDataStreamOutput(
+      OzoneClientConfig config,
+      OpenKeySession handler,
+      XceiverClientFactory xceiverClientManager,
+      OzoneManagerProtocol omClient, int chunkSize,
+      String requestId, ReplicationConfig replicationConfig,
+      String uploadID, int partNumber, boolean isMultipart,
+      boolean unsafeByteBufferConversion
+  ) {
+    this.config = config;
+    OmKeyInfo info = handler.getKeyInfo();
+    blockDataStreamOutputEntryPool =
+        new BlockDataStreamOutputEntryPool(
+            config,
+            omClient,
+            requestId, replicationConfig,
+            uploadID, partNumber,
+            isMultipart, info,
+            unsafeByteBufferConversion,
+            xceiverClientManager,
+            handler.getId());
+
+    // Retrieve the file encryption key info, null if file is not in
+    // encrypted bucket.
+    this.feInfo = info.getFileEncryptionInfo();
+    this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
+        config.getMaxRetryCount(), config.getRetryInterval());
+    this.retryCount = 0;
+    this.isException = false;
+    this.writeOffset = 0;
+    this.clientID = handler.getId();
+  }
+
+  /**
+   * When a key is opened, it is possible that there are some blocks already
+   * allocated to it for this open session. In this case, to make use of these
+   * blocks, we need to add these blocks to stream entries. But, a key's version
+   * also includes blocks from previous versions, we need to avoid adding these
+   * old blocks to stream entries, because these old blocks should not be picked
+   * for write. To do this, the following method checks that, only those
+   * blocks created in this particular open version are added to stream entries.
+   *
+   * @param version the set of blocks that are pre-allocated.
+   * @param openVersion the version corresponding to the pre-allocation.
+   * @throws IOException
+   */
+  public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
+      long openVersion) throws IOException {
+    blockDataStreamOutputEntryPool.addPreallocateBlocks(version, openVersion);
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkNotClosed();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    handleWrite(b, 0, len, false);

Review comment:
       Should the offset be b.readerIndex() instead of 0?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/ByteBufferStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+/**
+* This interface is for writing an output stream of ByteBuffers.
+* An ByteBufferStreamOutput accepts Netty ByteBuf and sends them to some sink.
+*/
+public interface ByteBufferStreamOutput extends Closeable {

Review comment:
       If we use ByteBuf, please rename this to ByteBufStreamOutput.

##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -801,6 +803,59 @@ public OzoneOutputStream createKey(
     return createOutputStream(openKey, requestId, replicationConfig);
   }
 
+  @Override
+  @Deprecated
+  public OzoneDataStreamOutput createStreamKey(String volumeName,

Review comment:
       Let's don't add Deprecated method.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686453951



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
+import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Maintaining a list of BlockInputStream. Write based on offset.
+ *
+ * Note that this may write to multiple containers in one write call. In case
+ * that first container succeeded but later ones failed, the succeeded writes
+ * are not rolled back.
+ *
+ * TODO : currently not support multi-thread access.
+ */
+public class KeyDataStreamOutput implements ByteBufferStreamOutput {
+
+  private OzoneClientConfig config;
+
+  /**
+   * Defines stream action while calling handleFlushOrClose.
+   */
+  enum StreamAction {
+    FLUSH, CLOSE, FULL
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KeyDataStreamOutput.class);
+
+  private boolean closed;
+  private FileEncryptionInfo feInfo;
+  private final Map<Class<? extends Throwable>, RetryPolicy> retryPolicyMap;
+  private int retryCount;
+  // how much of data is actually written yet to underlying stream
+  private long offset;
+  // how much data has been ingested into the stream
+  private long writeOffset;
+  // whether an exception is encountered while write and whole write could
+  // not succeed
+  private boolean isException;
+  private final BlockDataStreamOutputEntryPool blockDataStreamOutputEntryPool;
+
+  private long clientID;
+
+  /**
+   * A constructor for testing purpose only.
+   */
+  @VisibleForTesting
+  public KeyDataStreamOutput() {
+    closed = false;
+    this.retryPolicyMap = HddsClientUtils.getExceptionList()
+        .stream()
+        .collect(Collectors.toMap(Function.identity(),
+            e -> RetryPolicies.TRY_ONCE_THEN_FAIL));
+    retryCount = 0;
+    offset = 0;
+    blockDataStreamOutputEntryPool = new BlockDataStreamOutputEntryPool();
+  }
+
+  @VisibleForTesting
+  public List<BlockDataStreamOutputEntry> getStreamEntries() {
+    return blockDataStreamOutputEntryPool.getStreamEntries();
+  }
+
+  @VisibleForTesting
+  public XceiverClientFactory getXceiverClientFactory() {
+    return blockDataStreamOutputEntryPool.getXceiverClientFactory();
+  }
+
+  @VisibleForTesting
+  public List<OmKeyLocationInfo> getLocationInfoList() {
+    return blockDataStreamOutputEntryPool.getLocationInfoList();
+  }
+
+  @VisibleForTesting
+  public int getRetryCount() {
+    return retryCount;
+  }
+
+  @VisibleForTesting
+  public long getClientID() {
+    return clientID;
+  }
+
+  @SuppressWarnings({"parameternumber", "squid:S00107"})
+  public KeyDataStreamOutput(
+      OzoneClientConfig config,
+      OpenKeySession handler,
+      XceiverClientFactory xceiverClientManager,
+      OzoneManagerProtocol omClient, int chunkSize,
+      String requestId, ReplicationConfig replicationConfig,
+      String uploadID, int partNumber, boolean isMultipart,
+      boolean unsafeByteBufferConversion
+  ) {
+    this.config = config;
+    OmKeyInfo info = handler.getKeyInfo();
+    blockDataStreamOutputEntryPool =
+        new BlockDataStreamOutputEntryPool(
+            config,
+            omClient,
+            requestId, replicationConfig,
+            uploadID, partNumber,
+            isMultipart, info,
+            unsafeByteBufferConversion,
+            xceiverClientManager,
+            handler.getId());
+
+    // Retrieve the file encryption key info, null if file is not in
+    // encrypted bucket.
+    this.feInfo = info.getFileEncryptionInfo();
+    this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
+        config.getMaxRetryCount(), config.getRetryInterval());
+    this.retryCount = 0;
+    this.isException = false;
+    this.writeOffset = 0;
+    this.clientID = handler.getId();
+  }
+
+  /**
+   * When a key is opened, it is possible that there are some blocks already
+   * allocated to it for this open session. In this case, to make use of these
+   * blocks, we need to add these blocks to stream entries. But, a key's version
+   * also includes blocks from previous versions, we need to avoid adding these
+   * old blocks to stream entries, because these old blocks should not be picked
+   * for write. To do this, the following method checks that, only those
+   * blocks created in this particular open version are added to stream entries.
+   *
+   * @param version the set of blocks that are pre-allocated.
+   * @param openVersion the version corresponding to the pre-allocation.
+   * @throws IOException
+   */
+  public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
+      long openVersion) throws IOException {
+    blockDataStreamOutputEntryPool.addPreallocateBlocks(version, openVersion);
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkNotClosed();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    handleWrite(b, 0, len, false);

Review comment:
       Yes, you're correct.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682430709



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));
+  }
+
+  /**
+   * @param close whether the flush is happening as part of closing the stream
+   */
+  private void handleFlush(boolean close)
+      throws IOException, InterruptedException, ExecutionException {
+    checkOpen();
+    // flush the last chunk data residing on the currentBuffer
+    if (totalDataFlushedLength < writtenDataLength) {
+      refreshCurrentBuffer(bufferPool);
+      Preconditions.checkArgument(currentBuffer.position() > 0);
+      if (currentBuffer.hasRemaining()) {
+        writeChunk(currentBuffer);
+      }
+      // This can be a partially filled chunk. Since we are flushing the buffer
+      // here, we just limit this buffer to the current position. So that next
+      // write will happen in new buffer
+      updateFlushLength();
+      executePutBlock(close, false);
+    } else if (close) {
+      // forcing an "empty" putBlock if stream is being closed without new
+      // data since latest flush - we need to send the "EOF" flag
+      executePutBlock(true, true);
+    }
+    waitOnFlushFutures();
+    watchForCommit(false);
+    // just check again if the exception is hit while waiting for the
+    // futures to ensure flush has indeed succeeded
+
+    // irrespective of whether the commitIndex2flushedDataMap is empty
+    // or not, ensure there is no exception set
+    checkOpen();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0) {
+      try {
+        handleFlush(true);
+        dataStreamCloseReply.get();
+      } catch (ExecutionException e) {
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      } finally {
+        cleanup(false);
+      }
+      // TODO: Turn the below buffer empty check on when Standalone pipeline
+      // is removed in the write path in tests
+      // Preconditions.checkArgument(buffer.position() == 0);
+      // bufferPool.checkBufferPoolEmpty();
+
+    }
+  }
+
+  private void waitOnFlushFutures()
+      throws InterruptedException, ExecutionException {
+    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
+        commitWatcher.getFutureMap().values().toArray(
+            new CompletableFuture[commitWatcher.getFutureMap().size()]));
+    // wait for all the transactions to complete
+    combinedFuture.get();
+  }
+
+  private void validateResponse(
+      ContainerProtos.ContainerCommandResponseProto responseProto)
+      throws IOException {
+    try {
+      // if the ioException is already set, it means a prev request has failed
+      // just throw the exception. The current operation will fail with the
+      // original error
+      IOException exception = getIoException();
+      if (exception != null) {
+        throw exception;
+      }
+      ContainerProtocolCalls.validateContainerResponse(responseProto);
+    } catch (StorageContainerException sce) {
+      setIoException(sce);
+      throw sce;
+    }
+  }
+
+
+  private void setIoException(Exception e) {
+    IOException ioe = getIoException();
+    if (ioe == null) {
+      IOException exception =  new IOException(EXCEPTION_MSG + e.toString(), e);
+      ioException.compareAndSet(null, exception);
+    } else {
+      LOG.debug("Previous request had already failed with " + ioe.toString()
+          + " so subsequent request also encounters"
+          + " Storage Container Exception ", e);
+    }
+  }
+
+  public void cleanup(boolean invalidateClient) {
+    if (xceiverClientFactory != null) {
+      xceiverClientFactory.releaseClient(xceiverClient, invalidateClient);
+    }
+    xceiverClientFactory = null;
+    xceiverClient = null;
+    commitWatcher.cleanup();
+    if (bufferList !=  null) {
+      bufferList.clear();
+    }
+    bufferList = null;
+    responseExecutor.shutdown();
+  }
+
+  /**
+   * Checks if the stream is open or exception has occurred.
+   * If not, throws an exception.
+   *
+   * @throws IOException if stream is closed
+   */
+  private void checkOpen() throws IOException {
+    if (isClosed()) {
+      throw new IOException("BlockOutputStream has been closed.");
+    } else if (getIoException() != null) {
+      adjustBuffersOnException();
+      throw getIoException();
+    }
+  }
+
+  public boolean isClosed() {
+    return xceiverClient == null;
+  }
+
+  private boolean needSync(long position) {
+    if (syncSize > 0) {
+      // TODO: or position >= fileLength
+      if (position - syncPosition >= syncSize) {
+        syncPosition = position;
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Writes buffered data as a new chunk to the container and saves chunk
+   * information to be used later in putKey call.
+   *
+   * @throws IOException if there is an I/O error while performing the call
+   * @throws OzoneChecksumException if there is an error while computing
+   * checksum
+   */
+  private void writeChunkToContainer(ChunkBuffer chunk) throws IOException {
+    int effectiveChunkSize = chunk.remaining();
+    final long offset = chunkOffset.getAndAdd(effectiveChunkSize);
+    final ByteString data = chunk.toByteString(
+        bufferPool.byteStringConversion());
+    ChecksumData checksumData = checksum.computeChecksum(chunk);
+    ChunkInfo chunkInfo = ChunkInfo.newBuilder()
+        .setChunkName(blockID.get().getLocalID() + "_chunk_" + ++chunkIndex)
+        .setOffset(offset)
+        .setLen(effectiveChunkSize)
+        .setChecksumData(checksumData.getProtoBufMessage())
+        .build();
+
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Writing chunk {} length {} at offset {}",
+          chunkInfo.getChunkName(), effectiveChunkSize, offset);
+    }
+
+    CompletableFuture<DataStreamReply> future =
+        (needSync(offset + effectiveChunkSize) ?
+        out.writeAsync(data.asReadOnlyByteBuffer(), StandardWriteOption.SYNC) :
+        out.writeAsync(data.asReadOnlyByteBuffer()))

Review comment:
       write data through stream, SYNC flag here means `sync(2)`




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682437389



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/rpc/RpcClient.java
##########
@@ -1361,6 +1405,24 @@ private OzoneInputStream createInputStream(
           cryptoInputStreams);
     }
   }
+  private OzoneDataStreamOutput createDataStreamOutput(OpenKeySession openKey,
+      String requestId, ReplicationConfig replicationConfig)
+      throws IOException {
+    KeyDataStreamOutput keyOutputStream =
+        new KeyDataStreamOutput.Builder()
+            .setHandler(openKey)
+            .setXceiverClientManager(xceiverClientManager)
+            .setOmClient(ozoneManagerClient)
+            .setRequestID(requestId)
+            .setReplicationConfig(replicationConfig)
+            .enableUnsafeByteBufferConversion(unsafeByteBufferConversion)
+            .setConfig(clientConfig)
+            .build();
+    keyOutputStream
+        .addPreallocateBlocks(openKey.getKeyInfo().getLatestVersionLocations(),
+            openKey.getOpenVersion());
+    return new OzoneDataStreamOutput(keyOutputStream);

Review comment:
       support for `CipherDataStreamOutput` and `CryptoDataStreamOutput` is dropped




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684578639



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,775 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream

Review comment:
       changed in 787b630




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893239885


   @szetszwo , can you have a look as well?
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-893243191


   > Still reviewing this. Thanks @ckj996 for the work.
   
   Haha, thanks for reviewing. It's a co-work with @captainzmc.


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r686466965



##########
File path: hadoop-ozone/client/src/main/java/org/apache/hadoop/ozone/client/io/KeyDataStreamOutput.java
##########
@@ -0,0 +1,629 @@
+/*
+ * 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.ozone.client.io;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import org.apache.hadoop.fs.FSExceptionMessages;
+import org.apache.hadoop.fs.FileEncryptionInfo;
+import org.apache.hadoop.hdds.client.ReplicationConfig;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.container.ContainerID;
+import org.apache.hadoop.hdds.scm.container.common.helpers.ExcludeList;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.scm.storage.ByteBufferStreamOutput;
+import org.apache.hadoop.io.retry.RetryPolicies;
+import org.apache.hadoop.io.retry.RetryPolicy;
+import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
+import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfoGroup;
+import org.apache.hadoop.ozone.om.helpers.OmMultipartCommitUploadPartInfo;
+import org.apache.hadoop.ozone.om.helpers.OpenKeySession;
+import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
+import org.apache.ratis.protocol.exceptions.AlreadyClosedException;
+import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+import java.util.stream.Collectors;
+
+/**
+ * Maintaining a list of BlockInputStream. Write based on offset.
+ *
+ * Note that this may write to multiple containers in one write call. In case
+ * that first container succeeded but later ones failed, the succeeded writes
+ * are not rolled back.
+ *
+ * TODO : currently not support multi-thread access.
+ */
+public class KeyDataStreamOutput implements ByteBufferStreamOutput {
+
+  private OzoneClientConfig config;
+
+  /**
+   * Defines stream action while calling handleFlushOrClose.
+   */
+  enum StreamAction {
+    FLUSH, CLOSE, FULL
+  }
+
+  public static final Logger LOG =
+      LoggerFactory.getLogger(KeyDataStreamOutput.class);
+
+  private boolean closed;
+  private FileEncryptionInfo feInfo;
+  private final Map<Class<? extends Throwable>, RetryPolicy> retryPolicyMap;
+  private int retryCount;
+  // how much of data is actually written yet to underlying stream
+  private long offset;
+  // how much data has been ingested into the stream
+  private long writeOffset;
+  // whether an exception is encountered while write and whole write could
+  // not succeed
+  private boolean isException;
+  private final BlockDataStreamOutputEntryPool blockDataStreamOutputEntryPool;
+
+  private long clientID;
+
+  /**
+   * A constructor for testing purpose only.
+   */
+  @VisibleForTesting
+  public KeyDataStreamOutput() {
+    closed = false;
+    this.retryPolicyMap = HddsClientUtils.getExceptionList()
+        .stream()
+        .collect(Collectors.toMap(Function.identity(),
+            e -> RetryPolicies.TRY_ONCE_THEN_FAIL));
+    retryCount = 0;
+    offset = 0;
+    blockDataStreamOutputEntryPool = new BlockDataStreamOutputEntryPool();
+  }
+
+  @VisibleForTesting
+  public List<BlockDataStreamOutputEntry> getStreamEntries() {
+    return blockDataStreamOutputEntryPool.getStreamEntries();
+  }
+
+  @VisibleForTesting
+  public XceiverClientFactory getXceiverClientFactory() {
+    return blockDataStreamOutputEntryPool.getXceiverClientFactory();
+  }
+
+  @VisibleForTesting
+  public List<OmKeyLocationInfo> getLocationInfoList() {
+    return blockDataStreamOutputEntryPool.getLocationInfoList();
+  }
+
+  @VisibleForTesting
+  public int getRetryCount() {
+    return retryCount;
+  }
+
+  @VisibleForTesting
+  public long getClientID() {
+    return clientID;
+  }
+
+  @SuppressWarnings({"parameternumber", "squid:S00107"})
+  public KeyDataStreamOutput(
+      OzoneClientConfig config,
+      OpenKeySession handler,
+      XceiverClientFactory xceiverClientManager,
+      OzoneManagerProtocol omClient, int chunkSize,
+      String requestId, ReplicationConfig replicationConfig,
+      String uploadID, int partNumber, boolean isMultipart,
+      boolean unsafeByteBufferConversion
+  ) {
+    this.config = config;
+    OmKeyInfo info = handler.getKeyInfo();
+    blockDataStreamOutputEntryPool =
+        new BlockDataStreamOutputEntryPool(
+            config,
+            omClient,
+            requestId, replicationConfig,
+            uploadID, partNumber,
+            isMultipart, info,
+            unsafeByteBufferConversion,
+            xceiverClientManager,
+            handler.getId());
+
+    // Retrieve the file encryption key info, null if file is not in
+    // encrypted bucket.
+    this.feInfo = info.getFileEncryptionInfo();
+    this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
+        config.getMaxRetryCount(), config.getRetryInterval());
+    this.retryCount = 0;
+    this.isException = false;
+    this.writeOffset = 0;
+    this.clientID = handler.getId();
+  }
+
+  /**
+   * When a key is opened, it is possible that there are some blocks already
+   * allocated to it for this open session. In this case, to make use of these
+   * blocks, we need to add these blocks to stream entries. But, a key's version
+   * also includes blocks from previous versions, we need to avoid adding these
+   * old blocks to stream entries, because these old blocks should not be picked
+   * for write. To do this, the following method checks that, only those
+   * blocks created in this particular open version are added to stream entries.
+   *
+   * @param version the set of blocks that are pre-allocated.
+   * @param openVersion the version corresponding to the pre-allocation.
+   * @throws IOException
+   */
+  public void addPreallocateBlocks(OmKeyLocationInfoGroup version,
+      long openVersion) throws IOException {
+    blockDataStreamOutputEntryPool.addPreallocateBlocks(version, openVersion);
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkNotClosed();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    handleWrite(b, 0, len, false);

Review comment:
       fixed in f84936e




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r682429362



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());

Review comment:
       open stream




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-897042898


   I would like to clean up the HDDS-4454 branch before merging this.
   
   <img width="540" alt="image" src="https://user-images.githubusercontent.com/907380/129080065-d7ca2239-3201-4b10-beb3-e0ccebe9aac5.png">
   
   Currently, the HDDS-4454 branch diverges with the master (8 commits ahead, 10 commits behind).  I think we should keep rebasing the changes from the master onto the HDDS-4454 branch.  Otherwise, it is hard to merge HDDS-4454 back to the master in the future. Any thought?


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] ckj996 commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
ckj996 commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683896599



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,767 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 0; // TODO: disk sync is disabled for now
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    // Alternatively, stream setup can be delayed till the first chunk write.
+    this.out = setupStream();
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private DataStreamOutput setupStream() throws IOException {
+    // Execute a dummy WriteChunk request to get the path of the target file,
+    // but does NOT write any data to it.
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.get().getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.get().getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    return Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+        .stream(message.getContent().asReadOnlyByteBuffer());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    final int len = b.readableBytes();
+    if (len == 0) {
+      return;
+    }
+    ChunkBuffer chunk = ChunkBuffer.wrap(b.nioBuffer());
+    writeChunk(chunk);
+    writtenDataLength += len;
+    updateFlushLength();
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte) b;
+    write(buf);
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();
+    } catch (Exception e) {
+      LOG.warn("Failed to write all chunks through stream: " + e);
+      throw new IOException(e);
+    }
+    if (close) {
+      dataStreamCloseReply = out.closeAsync();
+    }
+
+    CompletableFuture<ContainerProtos.
+        ContainerCommandResponseProto> flushFuture = null;
+    try {
+      BlockData blockData = containerBlockData.build();
+      XceiverClientReply asyncReply =
+          putBlockAsync(xceiverClient, blockData, close, token);
+      CompletableFuture<ContainerProtos.ContainerCommandResponseProto> future =
+          asyncReply.getResponse();
+      flushFuture = future.thenApplyAsync(e -> {
+        try {
+          validateResponse(e);
+        } catch (IOException sce) {
+          throw new CompletionException(sce);
+        }
+        // if the ioException is not set, putBlock is successful
+        if (getIoException() == null && !force) {
+          BlockID responseBlockID = BlockID.getFromProtobuf(
+              e.getPutBlock().getCommittedBlockLength().getBlockID());
+          Preconditions.checkState(blockID.get().getContainerBlockID()
+              .equals(responseBlockID.getContainerBlockID()));
+          // updates the bcsId of the block
+          blockID.set(responseBlockID);
+          if (LOG.isDebugEnabled()) {
+            LOG.debug(
+                "Adding index " + asyncReply.getLogIndex() + " commitMap size "
+                    + commitWatcher.getCommitInfoMapSize() + " flushLength "
+                    + flushPos + " numBuffers " + byteBufferList.size()
+                    + " blockID " + blockID + " bufferPool size" + bufferPool
+                    .getSize() + " currentBufferIndex " + bufferPool
+                    .getCurrentBufferIndex());
+          }
+          // for standalone protocol, logIndex will always be 0.
+          commitWatcher
+              .updateCommitInfoMap(asyncReply.getLogIndex(), byteBufferList);
+        }
+        return e;
+      }, responseExecutor).exceptionally(e -> {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("putBlock failed for blockID {} with exception {}",
+              blockID, e.getLocalizedMessage());
+        }
+        CompletionException ce = new CompletionException(e);
+        setIoException(ce);
+        throw ce;
+      });
+    } catch (IOException | ExecutionException e) {
+      throw new IOException(EXCEPTION_MSG + e.toString(), e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, false);
+    }
+    commitWatcher.getFutureMap().put(flushPos, flushFuture);
+    return flushFuture;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null
+        && bufferPool != null && bufferPool.getSize() > 0
+        && (!config.isStreamBufferFlushDelay() ||
+            writtenDataLength - totalDataFlushedLength
+                >= config.getStreamBufferSize())) {
+      try {
+        handleFlush(false);
+      } catch (ExecutionException e) {
+        // just set the exception here as well in order to maintain sanctity of
+        // ioException field
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      }
+    }
+  }
+
+  private void writeChunk(ChunkBuffer buffer)
+      throws IOException {
+    // This data in the buffer will be pushed to datanode and a reference will
+    // be added to the bufferList. Once putBlock gets executed, this list will
+    // be marked null. Hence, during first writeChunk call after every putBlock
+    // call or during the first call to writeChunk here, the list will be null.
+
+    if (bufferList == null) {
+      bufferList = new ArrayList<>();
+    }
+    bufferList.add(buffer);
+    writeChunkToContainer(buffer.duplicate(0, buffer.position()));
+  }
+
+  /**
+   * @param close whether the flush is happening as part of closing the stream
+   */
+  private void handleFlush(boolean close)
+      throws IOException, InterruptedException, ExecutionException {
+    checkOpen();
+    // flush the last chunk data residing on the currentBuffer
+    if (totalDataFlushedLength < writtenDataLength) {
+      refreshCurrentBuffer(bufferPool);
+      Preconditions.checkArgument(currentBuffer.position() > 0);
+      if (currentBuffer.hasRemaining()) {
+        writeChunk(currentBuffer);
+      }
+      // This can be a partially filled chunk. Since we are flushing the buffer
+      // here, we just limit this buffer to the current position. So that next
+      // write will happen in new buffer
+      updateFlushLength();
+      executePutBlock(close, false);
+    } else if (close) {
+      // forcing an "empty" putBlock if stream is being closed without new
+      // data since latest flush - we need to send the "EOF" flag
+      executePutBlock(true, true);
+    }
+    waitOnFlushFutures();
+    watchForCommit(false);
+    // just check again if the exception is hit while waiting for the
+    // futures to ensure flush has indeed succeeded
+
+    // irrespective of whether the commitIndex2flushedDataMap is empty
+    // or not, ensure there is no exception set
+    checkOpen();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (xceiverClientFactory != null && xceiverClient != null) {
+      try {
+        handleFlush(true);
+        dataStreamCloseReply.get();
+      } catch (ExecutionException e) {
+        handleExecutionException(e);
+      } catch (InterruptedException ex) {
+        Thread.currentThread().interrupt();
+        handleInterruptedException(ex, true);
+      } finally {
+        cleanup(false);
+      }
+      // TODO: Turn the below buffer empty check on when Standalone pipeline
+      // is removed in the write path in tests
+      // Preconditions.checkArgument(buffer.position() == 0);
+      // bufferPool.checkBufferPoolEmpty();
+
+    }
+  }
+
+  private void waitOnFlushFutures()
+      throws InterruptedException, ExecutionException {
+    CompletableFuture<Void> combinedFuture = CompletableFuture.allOf(
+        commitWatcher.getFutureMap().values().toArray(
+            new CompletableFuture[commitWatcher.getFutureMap().size()]));
+    // wait for all the transactions to complete
+    combinedFuture.get();
+  }
+
+  private void validateResponse(
+      ContainerProtos.ContainerCommandResponseProto responseProto)
+      throws IOException {
+    try {
+      // if the ioException is already set, it means a prev request has failed
+      // just throw the exception. The current operation will fail with the
+      // original error
+      IOException exception = getIoException();
+      if (exception != null) {
+        throw exception;
+      }
+      ContainerProtocolCalls.validateContainerResponse(responseProto);
+    } catch (StorageContainerException sce) {
+      setIoException(sce);
+      throw sce;
+    }
+  }
+
+
+  private void setIoException(Exception e) {
+    IOException ioe = getIoException();
+    if (ioe == null) {
+      IOException exception =  new IOException(EXCEPTION_MSG + e.toString(), e);
+      ioException.compareAndSet(null, exception);
+    } else {
+      LOG.debug("Previous request had already failed with " + ioe.toString()
+          + " so subsequent request also encounters"
+          + " Storage Container Exception ", e);
+    }
+  }
+
+  public void cleanup(boolean invalidateClient) {
+    if (xceiverClientFactory != null) {
+      xceiverClientFactory.releaseClient(xceiverClient, invalidateClient);
+    }
+    xceiverClientFactory = null;
+    xceiverClient = null;
+    commitWatcher.cleanup();
+    if (bufferList !=  null) {
+      bufferList.clear();
+    }
+    bufferList = null;
+    responseExecutor.shutdown();
+  }
+
+  /**
+   * Checks if the stream is open or exception has occurred.
+   * If not, throws an exception.
+   *
+   * @throws IOException if stream is closed
+   */
+  private void checkOpen() throws IOException {
+    if (isClosed()) {
+      throw new IOException("BlockOutputStream has been closed.");
+    } else if (getIoException() != null) {
+      adjustBuffersOnException();
+      throw getIoException();
+    }
+  }
+
+  public boolean isClosed() {
+    return xceiverClient == null;
+  }
+
+  private boolean needSync(long position) {
+    if (syncSize > 0) {
+      // TODO: or position >= fileLength
+      if (position - syncPosition >= syncSize) {
+        syncPosition = position;
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Writes buffered data as a new chunk to the container and saves chunk
+   * information to be used later in putKey call.
+   *
+   * @throws IOException if there is an I/O error while performing the call
+   * @throws OzoneChecksumException if there is an error while computing
+   * checksum
+   */
+  private void writeChunkToContainer(ChunkBuffer chunk) throws IOException {
+    int effectiveChunkSize = chunk.remaining();
+    final long offset = chunkOffset.getAndAdd(effectiveChunkSize);
+    final ByteString data = chunk.toByteString(
+        bufferPool.byteStringConversion());
+    ChecksumData checksumData = checksum.computeChecksum(chunk);

Review comment:
       Is there buffer copy when computing checksum?
   
   i.e. Mapped/DirectByteBuffer -> in heap memory




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683170775



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,

Review comment:
       In my opinion, this class needs to extend the abstract BlockOutputStream/RatisBlockOutputClass  in master class and implement and ovverride methods as necessary methods?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Can you please add a comment for why the stream is being setup here, and why its called With write chunk data?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));

Review comment:
       Why do we need a buffer copy here? what's the contract of this write() API here? is the buffer passed via application supposed to mutate?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       Isn't waiting for putBlock impliclitly mean all the pending write chunks are completed? We don't need to wait for all the write chunk futures to complete before issuing a putblock call.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       Isn't waiting for putBlock impliclitly mean all the pending write chunks are completed? We don't need to wait for all the write chunk futures to complete before issuing a putblock call.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable

Review comment:
       We don't have hsync feature build in yet in Ozone. Can we make it zero for now?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));
+      currentBufferRemaining -= writeLen;
+      writeChunkIfNeeded();
+      off += writeLen;
+      len -= writeLen;
+      writtenDataLength += writeLen;
+      doFlushOrWatchIfNeeded();
+    }
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    byte[] buf = new byte[1];
+    buf[0] = (byte)b;
+    write(Unpooled.wrappedBuffer(buf));
+  }
+
+  private void writeChunkIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      writeChunk(currentBuffer);
+    }
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    write(Unpooled.wrappedBuffer(b), off, len);
+  }
+
+  private void doFlushOrWatchIfNeeded() throws IOException {
+    if (currentBufferRemaining == 0) {
+      if (bufferPool.getNumberOfUsedBuffers() % flushPeriod == 0) {
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      // Data in the bufferPool can not exceed streamBufferMaxSize
+      if (bufferPool.getNumberOfUsedBuffers() == bufferPool.getCapacity()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  private void allocateNewBufferIfNeeded() {
+    if (currentBufferRemaining == 0) {
+      currentBuffer = bufferPool.allocateBuffer(config.getBufferIncrement());
+      currentBufferRemaining = currentBuffer.remaining();
+    }
+  }
+
+  private void updateFlushLength() {
+    totalDataFlushedLength = writtenDataLength;
+  }
+
+  private boolean isBufferPoolFull() {
+    return bufferPool.computeBufferData() == config.getStreamBufferMaxSize();
+  }
+
+  /**
+   * Will be called on the retryPath in case closedContainerException/
+   * TimeoutException.
+   * @param len length of data to write
+   * @throws IOException if error occurred
+   */
+
+  // In this case, the data is already cached in the currentBuffer.
+  public void writeOnRetry(long len) throws IOException {
+    if (len == 0) {
+      return;
+    }
+    if (LOG.isDebugEnabled()) {
+      LOG.debug("Retrying write length {} for blockID {}", len, blockID);
+    }
+    Preconditions.checkArgument(len <= config.getStreamBufferMaxSize());
+    int count = 0;
+    while (len > 0) {
+      ChunkBuffer buffer = bufferPool.getBuffer(count);
+      long writeLen = Math.min(buffer.position(), len);
+      if (!buffer.hasRemaining()) {
+        writeChunk(buffer);
+      }
+      len -= writeLen;
+      count++;
+      writtenDataLength += writeLen;
+      // we should not call isBufferFull/shouldFlush here.
+      // The buffer might already be full as whole data is already cached in
+      // the buffer. We should just validate
+      // if we wrote data of size streamBufferMaxSize/streamBufferFlushSize to
+      // call for handling full buffer/flush buffer condition.
+      if (writtenDataLength % config.getStreamBufferFlushSize() == 0) {
+        // reset the position to zero as now we will be reading the
+        // next buffer in the list
+        updateFlushLength();
+        executePutBlock(false, false);
+      }
+      if (writtenDataLength == config.getStreamBufferMaxSize()) {
+        handleFullBuffer();
+      }
+    }
+  }
+
+  /**
+   * This is a blocking call. It will wait for the flush till the commit index
+   * at the head of the commitIndex2flushedDataMap gets replicated to all or
+   * majority.
+   * @throws IOException
+   */
+  private void handleFullBuffer() throws IOException {
+    try {
+      checkOpen();
+      if (!commitWatcher.getFutureMap().isEmpty()) {
+        waitOnFlushFutures();
+      }
+    } catch (ExecutionException e) {
+      handleExecutionException(e);
+    } catch (InterruptedException ex) {
+      Thread.currentThread().interrupt();
+      handleInterruptedException(ex, true);
+    }
+    watchForCommit(true);
+  }
+
+
+  // It may happen that once the exception is encountered , we still might
+  // have successfully flushed up to a certain index. Make sure the buffers
+  // only contain data which have not been sufficiently replicated
+  private void adjustBuffersOnException() {
+    commitWatcher.releaseBuffersOnException();
+    refreshCurrentBuffer(bufferPool);
+  }
+
+  /**
+   * calls watchForCommit API of the Ratis Client. For Standalone client,
+   * it is a no op.
+   * @param bufferFull flag indicating whether bufferFull condition is hit or
+   *              its called as part flush/close
+   * @return minimum commit index replicated to all nodes
+   * @throws IOException IOException in case watch gets timed out
+   */
+  private void watchForCommit(boolean bufferFull) throws IOException {
+    checkOpen();
+    try {
+      XceiverClientReply reply = bufferFull ?
+          commitWatcher.watchOnFirstIndex() : commitWatcher.watchOnLastIndex();
+      if (reply != null) {
+        List<DatanodeDetails> dnList = reply.getDatanodes();
+        if (!dnList.isEmpty()) {
+          Pipeline pipe = xceiverClient.getPipeline();
+
+          LOG.warn("Failed to commit BlockId {} on {}. Failed nodes: {}",
+              blockID, pipe, dnList);
+          failedServers.addAll(dnList);
+        }
+      }
+    } catch (IOException ioe) {
+      setIoException(ioe);
+      throw getIoException();
+    }
+    refreshCurrentBuffer(bufferPool);
+
+  }
+
+  /**
+   * @param close whether putBlock is happening as part of closing the stream
+   * @param force true if no data was written since most recent putBlock and
+   *            stream is being closed
+   */
+  private CompletableFuture<ContainerProtos.
+      ContainerCommandResponseProto> executePutBlock(boolean close,
+      boolean force) throws IOException {
+    checkOpen();
+    long flushPos = totalDataFlushedLength;
+    final List<ChunkBuffer> byteBufferList;
+    if (!force) {
+      Preconditions.checkNotNull(bufferList);
+      byteBufferList = bufferList;
+      bufferList = null;
+      Preconditions.checkNotNull(byteBufferList);
+    } else {
+      byteBufferList = null;
+    }
+
+    try {
+      CompletableFuture.allOf(futures.toArray(EMPTY_FUTURE_ARRAY)).get();

Review comment:
       We need to implicitly wait for write chunks to complete before sending out the putblock command only if putBllock is happening outside the streaming path.   Can't we send putblock command in the streaming path itself with close?

##########
File path: hadoop-ozone/tools/src/main/java/org/apache/hadoop/ozone/shell/keys/PutKeyHandler.java
##########
@@ -111,10 +117,36 @@ protected void execute(OzoneClient client, OzoneAddress address)
 
     int chunkSize = (int) getConf().getStorageSize(OZONE_SCM_CHUNK_SIZE_KEY,
         OZONE_SCM_CHUNK_SIZE_DEFAULT, StorageUnit.BYTES);
-    try (InputStream input = new FileInputStream(dataFile);
-        OutputStream output = bucket.createKey(keyName, dataFile.length(),
-            replicationConfig, keyMetadata)) {
-      IOUtils.copyBytes(input, output, chunkSize);
+
+    if (dataFile.length() <= chunkSize) {
+      if (isVerbose()) {
+        out().println("API: async");
+      }
+      try (InputStream input = new FileInputStream(dataFile);
+           OutputStream output = bucket.createKey(keyName, dataFile.length(),
+               replicationConfig, keyMetadata)) {
+        IOUtils.copyBytes(input, output, chunkSize);
+      }
+    } else {
+      if (isVerbose()) {
+        out().println("API: streaming");
+      }
+      try (RandomAccessFile raf = new RandomAccessFile(dataFile, "r");
+           OzoneDataStreamOutput out = bucket.createStreamKey(keyName,
+               dataFile.length(), replicationConfig, keyMetadata)) {
+        FileChannel ch = raf.getChannel();
+        long len = raf.length();
+        long off = 0;
+        while (len > 0) {
+          long writeLen = Math.min(len, chunkSize);
+          ByteBuffer segment =
+              ch.map(FileChannel.MapMode.READ_ONLY, off, writeLen);
+          ByteBuf buf = Unpooled.wrappedBuffer(segment);
+          out.write(buf);

Review comment:
       yes, we should use netty constructs.

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Yes, i understand that. Can you this explicit by adding comments in the code?

##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())

Review comment:
       Yes, i understand that. Can you make this explicit by adding comments in the code?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r683172208



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/BlockDataStreamOutput.java
##########
@@ -0,0 +1,768 @@
+/*
+ * 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 com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.BlockData;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
+import org.apache.hadoop.hdds.ratis.ContainerCommandRequestMessage;
+import org.apache.hadoop.hdds.scm.OzoneClientConfig;
+import org.apache.hadoop.hdds.scm.XceiverClientFactory;
+import org.apache.hadoop.hdds.scm.XceiverClientRatis;
+import org.apache.hadoop.hdds.scm.XceiverClientReply;
+import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.ozone.common.Checksum;
+import org.apache.hadoop.ozone.common.ChecksumData;
+import org.apache.hadoop.ozone.common.ChunkBuffer;
+import org.apache.hadoop.ozone.common.OzoneChecksumException;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.ratis.client.api.DataStreamOutput;
+import org.apache.ratis.io.StandardWriteOption;
+import org.apache.ratis.protocol.DataStreamReply;
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.OutputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.CompletionException;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.hadoop.hdds.scm.storage.ContainerProtocolCalls.putBlockAsync;
+
+/**
+ * An {@link OutputStream} used by the REST service in combination with the
+ * SCMClient to write the value of a key to a sequence
+ * of container chunks.  Writes are buffered locally and periodically written to
+ * the container as a new chunk.  In order to preserve the semantics that
+ * replacement of a pre-existing key is atomic, each instance of the stream has
+ * an internal unique identifier.  This unique identifier and a monotonically
+ * increasing chunk index form a composite key that is used as the chunk name.
+ * After all data is written, a putKey call creates or updates the corresponding
+ * container key, and this call includes the full list of chunks that make up
+ * the key data.  The list of chunks is updated all at once.  Therefore, a
+ * concurrent reader never can see an intermediate state in which different
+ * chunks of data from different versions of the key data are interleaved.
+ * This class encapsulates all state management for buffering and writing
+ * through to the container.
+ */
+public class BlockDataStreamOutput extends OutputStream
+    implements org.apache.hadoop.hdds.scm.storage.DataStreamOutput {
+  public static final Logger LOG =
+      LoggerFactory.getLogger(BlockDataStreamOutput.class);
+  public static final String EXCEPTION_MSG =
+      "Unexpected Storage Container Exception: ";
+  private static final CompletableFuture[] EMPTY_FUTURE_ARRAY = {};
+
+  private AtomicReference<BlockID> blockID;
+
+  private final BlockData.Builder containerBlockData;
+  private XceiverClientFactory xceiverClientFactory;
+  private XceiverClientRatis xceiverClient;
+  private OzoneClientConfig config;
+
+  private int chunkIndex;
+  private final AtomicLong chunkOffset = new AtomicLong();
+  private final BufferPool bufferPool;
+  // The IOException will be set by response handling thread in case there is an
+  // exception received in the response. If the exception is set, the next
+  // request will fail upfront.
+  private final AtomicReference<IOException> ioException;
+  private final ExecutorService responseExecutor;
+
+  // the effective length of data flushed so far
+  private long totalDataFlushedLength;
+
+  // effective data write attempted so far for the block
+  private long writtenDataLength;
+
+  // List containing buffers for which the putBlock call will
+  // update the length in the datanodes. This list will just maintain
+  // references to the buffers in the BufferPool which will be cleared
+  // when the watchForCommit acknowledges a putBlock logIndex has been
+  // committed on all datanodes. This list will be a  place holder for buffers
+  // which got written between successive putBlock calls.
+  private List<ChunkBuffer> bufferList;
+
+  // This object will maintain the commitIndexes and byteBufferList in order
+  // Also, corresponding to the logIndex, the corresponding list of buffers will
+  // be released from the buffer pool.
+  private final CommitWatcher commitWatcher;
+
+  private final List<DatanodeDetails> failedServers;
+  private final Checksum checksum;
+
+  //number of buffers used before doing a flush/putBlock.
+  private int flushPeriod;
+  //bytes remaining to write in the current buffer.
+  private int currentBufferRemaining;
+  //current buffer allocated to write
+  private ChunkBuffer currentBuffer;
+  private final Token<? extends TokenIdentifier> token;
+  private final DataStreamOutput out;
+  private CompletableFuture<DataStreamReply> dataStreamCloseReply;
+  private List<CompletableFuture<DataStreamReply>> futures = new ArrayList<>();
+  private final long syncSize = 1L << 24; // TODO: make it configurable
+  private long syncPosition = 0;
+
+  /**
+   * Creates a new BlockOutputStream.
+   *
+   * @param blockID              block ID
+   * @param xceiverClientManager client manager that controls client
+   * @param pipeline             pipeline where block will be written
+   * @param bufferPool           pool of buffers
+   */
+  public BlockDataStreamOutput(
+      BlockID blockID,
+      XceiverClientFactory xceiverClientManager,
+      Pipeline pipeline,
+      BufferPool bufferPool,
+      OzoneClientConfig config,
+      Token<? extends TokenIdentifier> token
+  ) throws IOException {
+    this.xceiverClientFactory = xceiverClientManager;
+    this.config = config;
+    this.blockID = new AtomicReference<>(blockID);
+    KeyValue keyValue =
+        KeyValue.newBuilder().setKey("TYPE").setValue("KEY").build();
+    this.containerBlockData =
+        BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
+            .addMetadata(keyValue);
+    this.xceiverClient =
+        (XceiverClientRatis)xceiverClientManager.acquireClient(pipeline);
+    ContainerProtos.WriteChunkRequestProto.Builder writeChunkRequest =
+        ContainerProtos.WriteChunkRequestProto.newBuilder()
+            .setBlockID(blockID.getDatanodeBlockIDProtobuf());
+
+    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
+    ContainerProtos.ContainerCommandRequestProto.Builder builder =
+        ContainerProtos.ContainerCommandRequestProto.newBuilder()
+            .setCmdType(ContainerProtos.Type.StreamInit)
+            .setContainerID(blockID.getContainerID())
+            .setDatanodeUuid(id).setWriteChunk(writeChunkRequest);
+
+    ContainerCommandRequestMessage message =
+        ContainerCommandRequestMessage.toMessage(builder.build(), null);
+
+    out = Preconditions.checkNotNull(xceiverClient.getDataStreamApi())
+            .stream(message.getContent().asReadOnlyByteBuffer());
+    this.bufferPool = bufferPool;
+    this.token = token;
+
+    //number of buffers used before doing a flush
+    refreshCurrentBuffer(bufferPool);
+    flushPeriod = (int) (config.getStreamBufferFlushSize() / config
+        .getStreamBufferSize());
+
+    Preconditions
+        .checkArgument(
+            (long) flushPeriod * config.getStreamBufferSize() == config
+                .getStreamBufferFlushSize());
+
+    // A single thread executor handle the responses of async requests
+    responseExecutor = Executors.newSingleThreadExecutor();
+    commitWatcher = new CommitWatcher(bufferPool, xceiverClient);
+    bufferList = null;
+    totalDataFlushedLength = 0;
+    writtenDataLength = 0;
+    failedServers = new ArrayList<>(0);
+    ioException = new AtomicReference<>(null);
+    checksum = new Checksum(config.getChecksumType(),
+        config.getBytesPerChecksum());
+  }
+
+  private void refreshCurrentBuffer(BufferPool pool) {
+    currentBuffer = pool.getCurrentBuffer();
+    currentBufferRemaining =
+        currentBuffer != null ? currentBuffer.remaining() : 0;
+  }
+
+  public BlockID getBlockID() {
+    return blockID.get();
+  }
+
+  public long getTotalAckDataLength() {
+    return commitWatcher.getTotalAckDataLength();
+  }
+
+  public long getWrittenDataLength() {
+    return writtenDataLength;
+  }
+
+  public List<DatanodeDetails> getFailedServers() {
+    return failedServers;
+  }
+
+  @VisibleForTesting
+  public XceiverClientRatis getXceiverClient() {
+    return xceiverClient;
+  }
+
+  @VisibleForTesting
+  public long getTotalDataFlushedLength() {
+    return totalDataFlushedLength;
+  }
+
+  @VisibleForTesting
+  public BufferPool getBufferPool() {
+    return bufferPool;
+  }
+
+  public IOException getIoException() {
+    return ioException.get();
+  }
+
+  @VisibleForTesting
+  public Map<Long, List<ChunkBuffer>> getCommitIndex2flushedDataMap() {
+    return commitWatcher.getCommitIndex2flushedDataMap();
+  }
+
+  @Override
+  public void write(ByteBuf b) throws IOException {
+    checkOpen();
+    if (b == null) {
+      throw new NullPointerException();
+    }
+    int off = 0;
+    int len = b.readableBytes();
+
+    while (len > 0) {
+      allocateNewBufferIfNeeded();
+      final int writeLen = Math.min(currentBufferRemaining, len);
+      // TODO: avoid buffer copy here
+      currentBuffer.put(b.nioBuffer(off, writeLen));

Review comment:
       Why do we need a buffer copy here? what's the contract of this write() API here? is the buffer passed via application supposed to mutate?




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] bshashikant commented on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
bshashikant commented on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-894120792


   PutBlock is used outside the streaming pat and there will be log entry created for it. CommitWatcher will still be needed to ensure the associated log is replicated to all.
   
   The writeChunkToContainer can be executed as soon as there is data in the buffer. This should resolve the bursty IO pattern.
   The full buffer condition will never be hit in this case, bcoz before putBlock call we ensure all writes are done.
   
   The full buffer condition will make sense only if we send putBlock in the streaming path itself and once the buffer hits limit , it should just wait for all the acknowledgement. 


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] captainzmc edited a comment on pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
captainzmc edited a comment on pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#issuecomment-895955262


   > Two main comments:
   > 
   > * Let's do not extend OutputStream in all the new classes in order to keep the code simple.  If there is a need, we may add it later.
   > * In BlockDataStreamOutput, we should write directly to the underlying stream. We should not use BufferPool and ChunkBuffer in BlockDataStreamOutput anymore since we want to avoid buffer copying.
   
   Hi @szetszwo,  Currently this PR already supports not extends OutputStream. Regarding the second problem mentioned above "not use BufferPool and ChunkBuffer to avoid buffer copying", As we discussed  at the meeting on Monday, we are going to fix this in the next PR HDDS-5599.   We will continue to optimize this client. So can you take another look at this PR?
   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] kaijchen commented on a change in pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
kaijchen commented on a change in pull request #2495:
URL: https://github.com/apache/ozone/pull/2495#discussion_r684643164



##########
File path: hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/storage/DataStreamOutput.java
##########
@@ -0,0 +1,58 @@
+/*
+ * 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 io.netty.buffer.ByteBuf;

Review comment:
       Okay, let's keep it as is.




-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org


[GitHub] [ozone] szetszwo merged pull request #2495: HDDS-5488. [Ozone-Streaming] Add a new BlockOutputStream/KeyOutputStream to support streaming api

Posted by GitBox <gi...@apache.org>.
szetszwo merged pull request #2495:
URL: https://github.com/apache/ozone/pull/2495


   


-- 
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: issues-unsubscribe@ozone.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@ozone.apache.org
For additional commands, e-mail: issues-help@ozone.apache.org