Skip to content

Conversation

@guohao-rosicky
Copy link
Contributor

  1. On the client side implement SmallFileDataStreamOutput, used to process the upload request less than ChunkSize:
  2. SmallFileDataStreamOutput use ContainerProtos.PutSmallFileRequestProto write data and metadata;
  3. Implement new SmallFileStreamDataChannel on Datanode, used to process Smallfile requests

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

@guohao-rosicky
Copy link
Contributor Author

@szetszwo @captainzmc please take a look,

PutSmallFile contains writeChunk and putBlock, which is transmitted to Datanode through stream write.

There is no applyTransaction that goes through containerStateMachine.

Please check my PR,I don't know how to get the BCSID.

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. The change look mostly good. Some comments:

  • Let's move the common code KeyValueStreamDataChannel and SmallFileStreamDataChannel to a common base class, say StreamDataChannelBase.
  • Please add some tests.

See also https://issues.apache.org/jira/secure/attachment/13036532/2860_review.patch .

Comment on lines 51 to 56
Copy link
Contributor

Choose a reason for hiding this comment

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

Move randomAccessFile, file, containerData and metrics and the shared code to a new base class.

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 call it StreamDataChannel.

Copy link
Contributor

Choose a reason for hiding this comment

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

Why changing data to optional?

Copy link
Contributor Author

@guohao-rosicky guohao-rosicky Nov 24, 2021

Choose a reason for hiding this comment

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

Data in PutSmallFileRequestProto is null in setupStream

ContainerProtos.PutSmallFileRequestProto putSmallFileRequest =
        ContainerProtos.PutSmallFileRequestProto.newBuilder()
            .setChunkInfo(chunk)
            .setBlock(createBlockRequest)  // data is empty
            .build();

    String id = xceiverClient.getPipeline().getFirstNode().getUuidString();
    ContainerProtos.ContainerCommandRequestProto.Builder builder =
        ContainerProtos.ContainerCommandRequestProto.newBuilder()
            .setCmdType(ContainerProtos.Type.StreamInit)
            .setContainerID(blockID.get().getContainerID())
            .setDatanodeUuid(id)
            .setPutSmallFile(putSmallFileRequest);

Copy link
Contributor

Choose a reason for hiding this comment

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

I see. Thanks.

Copy link
Contributor

Choose a reason for hiding this comment

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

is this backward compatible?

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, changing required to optional is backward compatible since the old code always provides the value and the new code can handle it.

However, it is not forward compatible since the old code may not be able to handle the case that the value is missing.

@szetszwo
Copy link
Contributor

@guohao-rosicky, The change looks good but the new TestSmallFileDataStreamOutput failed. Please take a look. Thanks.

@captainzmc
Copy link
Member

Thanks @guohao-rosicky for the contribution,
I want to know why we don't use async write API to write small files? The async write API should use fewer RPC calls when writing small files.

@szetszwo
Copy link
Contributor

@captainzmc , this change is an optimization of createStreamKey(..) and createStreamFile(..) in RpcClient for the case that the given size is smaller than the chunk size.

Using the Async API will have fewer RPC calls and we should implement it. However, the data have to go through the leader so that the network path won't be optimal and the leader may become a hotspot.

@captainzmc
Copy link
Member

captainzmc commented Nov 26, 2021

However, the data have to go through the leader so that the network path won't be optimal and the leader may become a hotspot.

Thanks @szetszwo for your explanation, agree with you. let's using Streaming instead of async write.

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 , we need support zero buffer copying so that we should not put the data inside PutSmallFileRequestProto. We should send the header and then send the raw data in the stream.

Comment on lines 79 to 104
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 copies buffer twice so that it is not zero buffer copying.

Copy link
Contributor

Choose a reason for hiding this comment

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

We cannot call array() since the ByteBuffer may not have an array.

Also, using an array means there must be buffer copying inside the code. This is the reason that we use ByteBuffer but not byte[].

Copy link
Contributor

Choose a reason for hiding this comment

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

Do not set the data inside the proto. We should send only the header in the proto and then send the raw data to the stream.

@bshashikant
Copy link
Contributor

@guohao-rosicky , can you please rebase?

@guohao-rosicky
Copy link
Contributor Author

@guohao-rosicky , can you please rebase?

done

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 for the update. The change looks good.

@guohao-rosicky
Copy link
Contributor Author

@szetszwo @captainzmc please take a look.

@szetszwo
Copy link
Contributor

@guohao-rosicky , the change grew from 13kB to 65kB so that it becomes hard to review. The SmallFileDataStreamOutput class is really long.

How about we move the refactoring to a separated JIRA?

@guohao-rosicky
Copy link
Contributor Author

guohao-rosicky commented Jan 24, 2022

@guohao-rosicky , the change grew from 13kB to 65kB so that it becomes hard to review. The SmallFileDataStreamOutput class is really long.

How about we move the refactoring to a separated JIRA?

How about splitting it into two parts @szetszwo

  1. Data Channle abstraction on the server
  2. write small file (HDDS-4474)

@szetszwo
Copy link
Contributor

@guohao-rosicky , sure, please do it. Thanks.

@captainzmc captainzmc requested review from szetszwo and removed request for bshashikant March 21, 2022 10:40
@guohao-rosicky
Copy link
Contributor Author

@szetszwo please take a look. Thanks.

@captainzmc
Copy link
Member

The CI run was successful. @szetszwo Could you help take another look?

@szetszwo
Copy link
Contributor

@captainzmc , @guohao-rosicky , sure, I am reviewing 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.

With HDDS-6137, we may not need to add much code to implement WriteSmallFile. The data is buffered at the client side. When the data size is small, all the data can be sent in a single write call with close. What do you think?

Comment on lines 570 to 576
public static byte[] getFixedLengthBytes(int length) {
byte[] bytes = new byte[length];
Random random = new Random();
random.nextBytes(bytes);
return bytes;
}

Copy link
Contributor

Choose a reason for hiding this comment

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

Use ThreadLocalRandom and support non-random data as below:

  public static byte[] generateData(int length, boolean random) {
    final byte[] data = new byte[length];
    if (random) {
      ThreadLocalRandom.current().nextBytes(data);
    } else {
      for (int i = 0; i < length; i++) {
        data[i] = (byte) i;
      }
    }
    return data;
  }

Comment on lines 581 to 592
private CompletableFuture<ContainerCommandResponseProto> link(
LogEntryProto entry, SmallFileStreamDataChannel smallFileChannel) {
return CompletableFuture.supplyAsync(() -> {
final DispatcherContext context = new DispatcherContext.Builder()
.setTerm(entry.getTerm())
.setLogIndex(entry.getIndex())
.setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA)
.setContainer2BCSIDMap(container2BCSIDMap)
.build();

return runCommand(smallFileChannel.getPutBlockRequest(), context);
}, executor);
Copy link
Contributor

Choose a reason for hiding this comment

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

Pass ContainerCommandRequestProto instead and rename it to runCommandAsync(..)

  private CompletableFuture<ContainerCommandResponseProto> runCommandAsync(
      ContainerCommandRequestProto requestProto, LogEntryProto entry) {
    return CompletableFuture.supplyAsync(() -> {
      final DispatcherContext context = new DispatcherContext.Builder()
          .setTerm(entry.getTerm())
          .setLogIndex(entry.getIndex())
          .setStage(DispatcherContext.WriteChunkStage.COMMIT_DATA)
          .setContainer2BCSIDMap(container2BCSIDMap)
          .build();

      return runCommand(requestProto, context);
    }, executor);
  }

* <p>
* TODO : currently not support multi-thread access.
*/
public class SmallFileDataStreamOutput implements ByteBufferStreamOutput {
Copy link
Contributor

Choose a reason for hiding this comment

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

It seems that the code in this class is copied from BlockDataStreamOutputEntryPool and KeyDataStreamOutput. We should reuse the code but not copy them. Otherwise, it is very hard to maintain.

@guohao-rosicky guohao-rosicky requested a review from szetszwo March 24, 2022 07:21
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 , have you seen this comment #2860 (review) ?

@guohao-rosicky
Copy link
Contributor Author

@guohao-rosicky , have you seen this comment #2860 (review) ?

@szetszwo
KeyDataStreamOutput can optionally not specify the data size, SmallFileStreamOutput can be retained, what do you think

@szetszwo
Copy link
Contributor

szetszwo commented Mar 24, 2022

KeyDataStreamOutput can optionally not specify the data size, SmallFileStreamOutput can be retained, ...

That's why KeyDataStreamOutput is more powerful than SmallFileStreamOutput since it works even if the data size is unknown.

@guohao-rosicky
Copy link
Contributor Author

guohao-rosicky commented Mar 24, 2022

KeyDataStreamOutput can optionally not specify the data size, SmallFileStreamOutput can be retained, ...

That's why KeyDataStreamOutput is more powerful than SmallFileStreamOutput since it works even if the data size is unknown.

@szetszwo Ok, how can we do the following work better? Does this PR code help us to achieve this function? I can split it.

@szetszwo
Copy link
Contributor

..., how can we do the following work better? Does this PR code help us to achieve this function? I can split it.

Yes, I actually suggest you to split and move the common code to #3195 in this comment #3195 (comment)

@captainzmc
Copy link
Member

Hi @szetszwo , will you take this task? If so, I will close the PR.

@szetszwo
Copy link
Contributor

@captainzmc , I was thinking to support a BufferedDataStreamOutput in Ratis (similar to java.io.BufferedOutputStream). Then, Ozone could use it. It will require only a small change in Ozone. We may close this pull request and one a new pull request later.

@captainzmc
Copy link
Member

@captainzmc , I was thinking to support a BufferedDataStreamOutput in Ratis (similar to java.io.BufferedOutputStream). Then, Ozone could use it. It will require only a small change in Ozone. We may close this pull request and one a new pull request later.

OK, let's close this PR.

@captainzmc captainzmc closed this Apr 11, 2022
@guohao-rosicky
Copy link
Contributor Author

guohao-rosicky commented Apr 12, 2022

@captainzmc , I was thinking to support a BufferedDataStreamOutput in Ratis (similar to java.io.BufferedOutputStream). Then, Ozone could use it. It will require only a small change in Ozone. We may close this pull request and one a new pull request later.

Hi @szetszwo , Is it this ratis Jira? https://issues.apache.org/jira/browse/RATIS-1157

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.

6 participants