Skip to content

Conversation

@guohao-rosicky
Copy link
Contributor

What changes were proposed in this pull request?

Optimized the putBlock method of BlockDataStreamOutput

  1. PutBlock does not use RaftRpc, but through the stream close() method
// putBlock metadata

ByteBuffer buf =
    ContainerCommandRequestMessage.toMessage(putBlockRequestProto, null)
        .getContent().asReadOnlyByteBuffer();

out.writeAsync(buf, StandardWriteOption.CLOSE); 

2.Mark the stream data boundary by sending an empty stream packet

// Mark the stream data boundary

out.writeAsync(ByteBuffer.allocateDirect(0).asReadOnlyBuffer()); 
  1. Flush is also in the form of async RPC, because putBlock needs to fetch bcsId (Raft log index)

What is the link to the Apache JIRA

https://issues.apache.org/jira/browse/HDDS-6137

How was this patch tested?

Use the existing UT

@guohao-rosicky guohao-rosicky changed the title Guohao hdds 6137 dev HDDS-6137. [Ozone-Streaming] Optimized the putBlock method of BlockDataStreamOutput Mar 15, 2022
Copy link
Member

@captainzmc captainzmc left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks @guohao-rosicky Improve this. LGTM over all. TestBlockDataStreamOutput has some error, can you confirm?

Copy link
Contributor

@szetszwo szetszwo Mar 16, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@guohao-rosicky, Writing an empty buffer may not work as expected. In the streaming model, data is continuous but not block-by-block. Both sender and receiver must consider that the data is a stream of bytes.

For example, the sender may write 3 times with lengths 3, 4, 5. Since there could be buffers/encoder/decoder in between, the receiver may receive a single buffer with length 12.

Let me think about how to make it work.

Copy link
Contributor Author

@guohao-rosicky guohao-rosicky Mar 16, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks for your comment @szetszwo . I wrote a piece of pseudo-code, Ideas about the modification of org.apache.ratis.netty.server.DataStreamManagement#writeTo.

image

What do you think of this idea? There is no need to send this empty buffer rpc.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@guohao-rosicky , this is a standard problem in streaming. The standard solution is to write the length with the data. Let's don't change the API for it.

Copy link
Contributor Author

@guohao-rosicky guohao-rosicky Mar 24, 2022

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hi, @szetszwo
I have a question, about

For example, the sender may write 3 times with lengths 3, 4, 5. Since there could be buffers/encoder/decoder in between, the receiver may receive a single buffer with length 12.

I noticed that in HDFS, marking the end of stream is also represented by an empty package.
In Ratis Stream, sticky packets have already been dealt with.

org.apache.ratis.netty.NettyDataStreamUtils#encodeDataStreamRequestHeader
https://github.com/apache/ratis/blob/master/ratis-netty/src/main/java/org/apache/ratis/netty/NettyDataStreamUtils.java#L90

@captainzmc tested the code from some time ago and did not find this problem

@szetszwo
Copy link
Contributor

@guohao-rosicky , the idea is to write the proto length at the end as shown below; see also https://issues.apache.org/jira/secure/attachment/13041230/3195_review.patch

    //   readerIndex   protoIndex   lengthIndex    readerIndex+readableBytes
    //         V            V             V                              V
    // format: |--- data ---|--- proto ---|--- proto length (4 bytes) ---|

@guohao-rosicky
Copy link
Contributor Author

@guohao-rosicky , the idea is to write the proto length at the end as shown below; see also https://issues.apache.org/jira/secure/attachment/13041230/3195_review.patch

    //   readerIndex   protoIndex   lengthIndex    readerIndex+readableBytes
    //         V            V             V                              V
    // format: |--- data ---|--- proto ---|--- proto length (4 bytes) ---|

Thanks @szetszwo, Patch has been applied

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@guohao-rosicky , please fix the checkstyle warnings and test the change manually. I have not tested it. Thanks.

@captainzmc
Copy link
Member

Thanks @guohao-rosicky @szetszwo update this. I used this PR test and found that the file failed to write. In DataNode IndexOutOfBoundsException will appear.
image

@szetszwo
Copy link
Contributor

@captainzmc , thanks. Let me test the serialization/deserialization code.

@szetszwo
Copy link
Contributor

Found the bug: ByteBu.nioBuffers(index, length) somehow is not working. Let me fix it.

@szetszwo
Copy link
Contributor

@captainzmc
Copy link
Member

captainzmc commented Mar 17, 2022

@captainzmc , @guohao-rosicky, Fixed the bug; see https://issues.apache.org/jira/secure/attachment/13041255/3195_TestStreamDataChannel.patch

Thanks @szetszwo for the update. Using new patch I found IndexOutOfBoundsException still appear.
I found that the ERROR only appeared when writing with high concurrency. I was successful with 30 concurrent writes, but I get the exception with 600 concurrent writes.
image

@szetszwo
Copy link
Contributor

@captainzmc , thanks a lot for testing it!

I added one more test for the serialization and buffers. It worked fine. See https://issues.apache.org/jira/secure/attachment/13041275/3195_testBuffers.patch

I found that the ERROR only appeared when writing with high concurrency. I was successful with 30 concurrent writes, but I get the exception with 600 concurrent writes.

It seems there are some synchronization problems. Let me check.

@szetszwo
Copy link
Contributor

@captainzmc , could you post the test code?

@captainzmc
Copy link
Member

@szetszwo yes here is my test code. Is written according to the ratis Filestore test.

Actually there a problem with my previous description. I noticed that the when I use 30 threads writing, we’ll get same error.

@captainzmc
Copy link
Member

captainzmc commented Mar 17, 2022

@szetszwo Here is my datanode log. https://issues.apache.org/jira/secure/attachment/13041276/ozone-root-datanode-9-29-173-57.log
In the test, I wrote 30 files concurrently. Each file is 128MB and the chunk size is 1MB

@captainzmc captainzmc closed this Mar 17, 2022
@captainzmc captainzmc reopened this Mar 17, 2022
@szetszwo
Copy link
Contributor

@captainzmc , @guohao-rosicky , just found a bug in BlockDataStreamOutput. Could you test https://issues.apache.org/jira/secure/attachment/13041279/3195_bugfix.patch to see if it can fix the bug?

@szetszwo
Copy link
Contributor

@szetszwo yes here is my test code. Is written according to the ratis Filestore test.

Thanks for sharing the test code. It looks good.

@captainzmc
Copy link
Member

@captainzmc , @guohao-rosicky , just found a bug in BlockDataStreamOutput. Could you test https://issues.apache.org/jira/secure/attachment/13041279/3195_bugfix.patch to see if it can fix the bug?

Thanks @szetszwo for update this. I had just test this patch, we still get same error. Just as the same as this.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's add an abstract class instead static methods. Then, KeyDataStreamOutput can extends it. If we need to add SmallFileDataStreamOutput, it can extends it too.

package org.apache.hadoop.ozone.client.io;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
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.ratis.protocol.exceptions.AlreadyClosedException;
import org.apache.ratis.protocol.exceptions.RaftRetryFailureException;

import java.io.IOException;
import java.io.InterruptedIOException;
import java.util.Map;
import java.util.function.Function;
import java.util.stream.Collectors;

public abstract class AbstractDataStreamOutput implements ByteBufferStreamOutput {
  private final Map<Class<? extends Throwable>, RetryPolicy> retryPolicyMap;
  private int retryCount;
  private boolean isException;

  protected AbstractDataStreamOutput() {
    this.retryPolicyMap = HddsClientUtils.getExceptionList()
        .stream()
        .collect(Collectors.toMap(Function.identity(),
            e -> RetryPolicies.TRY_ONCE_THEN_FAIL));
  }

  @VisibleForTesting
  public int getRetryCount() {
    return retryCount;
  }

  void resetRetryCount() {
    retryCount = 0;
  }

  boolean isException() {
    return isException;
  }

  /**
   * Checks if the provided exception signifies retry failure in ratis client.
   * In case of retry failure, ratis client throws RaftRetryFailureException
   * and all succeeding operations are failed with AlreadyClosedException.
   */
  boolean checkForRetryFailure(Throwable t) {
    return t instanceof RaftRetryFailureException
        || t instanceof AlreadyClosedException;
  }

  // Every container specific exception from datatnode will be seen as
  // StorageContainerException
  boolean checkIfContainerToExclude(Throwable t) {
    return t instanceof StorageContainerException;
  }

  private void setExceptionAndThrow(IOException ioe) throws IOException {
    isException = true;
    throw ioe;
  }

  void handleRetry(IOException exception) throws IOException {
    RetryPolicy retryPolicy = retryPolicyMap
        .get(HddsClientUtils.checkForException(exception).getClass());
    if (retryPolicy == null) {
      retryPolicy = retryPolicyMap.get(Exception.class);
    }
    handleRetry(exception, retryPolicy);
  }

  private void handleRetry(IOException exception, RetryPolicy retryPolicy)
      throws IOException {
    RetryPolicy.RetryAction action = null;
    try {
      action = retryPolicy.shouldRetry(exception, retryCount, 0, true);
    } catch (Exception e) {
      setExceptionAndThrow(new IOException(e));
    }
    if (action.action == RetryPolicy.RetryAction.RetryDecision.FAIL) {
      String msg = "";
      if (action.reason != null) {
        msg = "Retry request failed. " + action.reason;
        //LOG.error(msg, exception);
      }
      setExceptionAndThrow(new IOException(msg, exception));
    }

    // Throw the exception if the thread is interrupted
    if (Thread.currentThread().isInterrupted()) {
      //LOG.warn("Interrupted while trying for retry");
      setExceptionAndThrow(exception);
    }
    Preconditions.checkArgument(
        action.action == RetryPolicy.RetryAction.RetryDecision.RETRY);
    if (action.delayMillis > 0) {
      try {
        Thread.sleep(action.delayMillis);
      } catch (InterruptedException e) {
        Thread.currentThread().interrupt();
        IOException ioe = (IOException) new InterruptedIOException(
            "Interrupted: action=" + action + ", retry policy=" + retryPolicy)
            .initCause(e);
        setExceptionAndThrow(ioe);
      }
    }
    retryCount++;
  }
}

@guohao-rosicky
Copy link
Contributor Author

Ok, I'll change it

@guohao-rosicky
Copy link
Contributor Author

Has been modified. @szetszwo Please take a look.Thanks.

@szetszwo
Copy link
Contributor

@guohao-rosicky , please review it yourself -- go through the change and remove the unused code (e.g. ConsumerWithIOException). Thanks.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@guohao-rosicky , thanks for the update. Some comments inlined.

});
}

private CompletableFuture<ContainerCommandResponseProto> runCommandAsync(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

runCommandAsync is not used. Let's add it later.

Comment on lines 194 to 197
LOG.error(
"Get stream data channel error Malformed request " +
"containerID: {} msg: {}",
container.getContainerData().getContainerID(), msg);
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Let's remove LOG.error(..) since it already throws an exception.

ContainerCommandRequestProto request, KeyValueContainer kvContainer,
DispatcherContext dispatcherContext) {
if (!request.hasWriteChunk()) {
BlockID blockID;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Add final.

* A constructor for testing purpose only.
*/
@VisibleForTesting
public KeyDataStreamOutput() {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This method is not used anywhere. Let's remove it.

Comment on lines -161 to -162
this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
config.getMaxRetryCount(), config.getRetryInterval());
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We should pass the retryPolicyMap to the super constructor.

Comment on lines 116 to 119
super(HddsClientUtils.getExceptionList()
.stream()
.collect(Collectors.toMap(Function.identity(),
e -> RetryPolicies.TRY_ONCE_THEN_FAIL)));
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The map is passed incorrectly. Below is the original map.

-    this.retryPolicyMap = HddsClientUtils.getRetryPolicyByException(
-        config.getMaxRetryCount(), config.getRetryInterval());

@guohao-rosicky guohao-rosicky requested a review from szetszwo March 28, 2022 02:04
Copy link
Member

@captainzmc captainzmc left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 Thanks @guohao-rosicky. The change looks good.

Copy link
Contributor

@szetszwo szetszwo left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

+1 the change looks good.

@szetszwo szetszwo merged commit 9546d4c into apache:HDDS-4454 Mar 28, 2022
@szetszwo szetszwo changed the title HDDS-6137. [Ozone-Streaming] Optimized the putBlock method of BlockDataStreamOutput HDDS-6137. [Ozone-Streaming] Refactor KeyDataStreamOutput. Mar 28, 2022
szetszwo pushed a commit to szetszwo/ozone that referenced this pull request May 6, 2022
captainzmc pushed a commit to captainzmc/hadoop-ozone that referenced this pull request Jul 4, 2022
szetszwo pushed a commit that referenced this pull request Oct 25, 2022
(cherry picked from commit 88c4d59)
(cherry picked from commit 0a432aa0eb57bd25d63e656d0ec6e96a05689ac7)
szetszwo pushed a commit that referenced this pull request Nov 7, 2022
(cherry picked from commit 88c4d59)
(cherry picked from commit 0a432aa0eb57bd25d63e656d0ec6e96a05689ac7)
(cherry picked from commit 99b414d)
szetszwo pushed a commit that referenced this pull request Dec 1, 2022
szetszwo pushed a commit that referenced this pull request Dec 16, 2022
nishitpatira pushed a commit to nishitpatira/ozone that referenced this pull request Dec 16, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants