Skip to content

Conversation

@szetszwo
Copy link
Contributor

What changes were proposed in this pull request?

Retain the buffers (using RATIS-1556) so that the PutBlockRequest can be deserialized when the stream is closed.

What is the link to the Apache JIRA

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

How was this patch tested?

New unit tests.

@guohao-rosicky
Copy link
Contributor

guohao-rosicky commented Mar 24, 2022

Of course, Ozone may choose to add a header for each call but it will add more overhead and need more code changes.

This header is actually included in every ratis Stream request.

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

Can we use it?

@szetszwo
Copy link
Contributor Author

This header is actually included in every ratis Stream request. ...

@guohao-rosicky , that is Ratis header but not Ozone header. Later on, Ratis can buffer the small requests at the client side before sending it out for performance improvement; see https://issues.apache.org/jira/browse/RATIS-1157 . That's the reason that we cannot depend on individual packet sizes or empty packets.

As an analogy, we use TCP in streaming. We may not be able to use TCP header in our code.

@captainzmc
Copy link
Member

Thanks @szetszwo update this PR. Let me test this PR first and then we can continue the review.

@szetszwo
Copy link
Contributor Author

@captainzmc , sure, please test this. Thanks a lot!

Copy link
Member

@captainzmc captainzmc Mar 27, 2022

Choose a reason for hiding this comment

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

Hi @szetszwo. I just tested the PR and found that the previous test case did not run successfully. There are a lot of AbstractMethodError found. Here is datanode log. Can you confirm that?
image

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@captainzmc , I guess you somehow were using some old version of Ratis/Ozone since you got AbstractMethodError. Please try deleting the maven cache and then rebuild everything.

This change should work since it already has passed all the checks in https://github.com/apache/ozone/runs/5687893931

Copy link
Member

Choose a reason for hiding this comment

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

Oops, my mistake, I didn't replace all the jars. Let me recompile and retest.

Copy link
Member

Choose a reason for hiding this comment

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

I recompiled the code and the tests were fine.

@guohao-rosicky
Copy link
Contributor

Hi, @szetszwo and @captainzmc

The current code does not handle small files separately:

Current process:

  1. init stream
  2. write data
  3. put block with stream
  4. Close and send put block proto's length

Is it necessary to continue #2860 after this PR is merged?

Because there are only two steps in its transmission:

  1. init stream
  2. close and put small file proto (data + put block)

@captainzmc
Copy link
Member

captainzmc commented Mar 28, 2022

Is it necessary to continue #2860 after this PR is merged?

Hi @guohao-rosicky. I think writing small file using Streaming needs to continue separately in #2860. @szetszwo Any suggestion here?

@szetszwo
Copy link
Contributor Author

  1. init stream
  2. write data
  3. put block with stream
  4. Close and send put block proto's length

If we buffer everything before sending it out, this is actually is a single step for small files in the transmission. Indeed, write small file should only have one transmission. Why it needs to take two transmissions to send a small file?

@captainzmc , @guohao-rosicky , I am okay to continue working on #2860. As mentioned above, we should just buffer everything in the client side.

@captainzmc
Copy link
Member

Thanks @szetszwo. There are some conflicts in the current code. I think after resolving the conflict, we can merge this PR first.

@guohao-rosicky
Copy link
Contributor

I am okay to continue working on #2860. As mentioned above, we should just buffer everything in the client side.

Thanks @szetszwo Can You take over the job?

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 @szetszwo, Let merge this.

@captainzmc captainzmc merged commit 30c9cc8 into apache:HDDS-4454 Mar 28, 2022
@szetszwo
Copy link
Contributor Author

@captainzmc , thanks a lot for reviewing and merging this!

@szetszwo
Copy link
Contributor Author

Thanks @szetszwo Can You take over the job?

Sure.

@szetszwo
Copy link
Contributor Author

@captainzmc , I actually like to run some benchmarks. Could you show me the details steps (including setting up a cluster)?

@captainzmc
Copy link
Member

@captainzmc , I actually like to run some benchmarks. Could you show me the details steps (including setting up a cluster)?

@szetszwo Sure, I will prepare a test document and update it here.

@captainzmc
Copy link
Member

captainzmc commented Mar 28, 2022

@captainzmc , I actually like to run some benchmarks. Could you show me the details steps (including setting up a cluster)?

Hi @szetszwo here is my test doc. It contains my test results and procedures. I recommend using Freon instead of my test code, because it's easier. Please leave me a message if you have any questions.

https://docs.google.com/document/d/1byRoqHJXMIVaJWVpIX_vtwMzK8PO6CQlNlanlanffXQ/edit#heading=h.tflxcwmtnab

@szetszwo
Copy link
Contributor Author

@captainzmc , thanks a lot for sharing the test details! Definitely will try it.

@captainzmc
Copy link
Member

@captainzmc , thanks a lot for sharing the test details! Definitely will try it.

Hi @szetszwo, Any progress on the streaming tests? Are there performance and stability issues? We can keep discuss if there are any questions.

@szetszwo
Copy link
Contributor Author

@captainzmc , I tried running some single client benchmarks. However, the ASYNC performance is better than STREAM Found that we have not called RaftServerConfigKeys.DataStream.setAsyncWriteThreadPoolSize. I temporarily have the following change:

+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -237,6 +237,8 @@ private void setUpRatisStream(RaftProperties properties) {
             .getStreamRequestThreads();
     RaftServerConfigKeys.DataStream.setAsyncRequestThreadPoolSize(properties,
         dataStreamAsyncRequestThreadPoolSize);
+    RaftServerConfigKeys.DataStream.setAsyncWriteThreadPoolSize(properties,
+        dataStreamAsyncRequestThreadPoolSize);
     int dataStreamClientPoolSize =
         conf.getObject(DatanodeRatisServerConfig.class)
             .getClientPoolSize();

@captainzmc
Copy link
Member

@captainzmc , I tried running some single client benchmarks. However, the ASYNC performance is better than STREAM Found that we have not called RaftServerConfigKeys.DataStream.setAsyncWriteThreadPoolSize. I temporarily have the following change:

+++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java
@@ -237,6 +237,8 @@ private void setUpRatisStream(RaftProperties properties) {
             .getStreamRequestThreads();
     RaftServerConfigKeys.DataStream.setAsyncRequestThreadPoolSize(properties,
         dataStreamAsyncRequestThreadPoolSize);
+    RaftServerConfigKeys.DataStream.setAsyncWriteThreadPoolSize(properties,
+        dataStreamAsyncRequestThreadPoolSize);
     int dataStreamClientPoolSize =
         conf.getObject(DatanodeRatisServerConfig.class)
             .getClientPoolSize();

@szetszwo, Our previous test results are similar, streaming performance is not as good as async. However, the bottleneck here is probably not WriteThreadPool. In our tests we looked at the VM monitor, and WriteThreadPool and RequestThreadPool were mostly idle. So we think the problem might be Netty.
image

@szetszwo
Copy link
Contributor Author

... So we think the problem might be Netty.

@captainzmc , ASYNC also uses Netty (gRPC + Netty). If the slowness in STREAM is due to Netty, then it should just be a configuration problem.

BTW, we did not see similar problem in FileStore benchmarks we did earlier.

@captainzmc
Copy link
Member

If the slowness in STREAM is due to Netty, then it should just be a configuration problem.

Agree,Maybe there are some Netty configurations that we haven't noticed.

BTW, we did not see similar problem in FileStore benchmarks we did earlier.

@szetszwo Yes, I remember that. runzhiwang and I did the FileStore test together. FileStore performed well at the time. But recently I noticed that runzhiwang commented a line of code in the Filestore test code. There was no logic for startTransaction in the previous Filestore test. This line is required in Ozone code. That's as different as I can think of so far.

@szetszwo
Copy link
Contributor Author

There was no logic for startTransaction in the previous Filestore test. This line is required in Ozone code.

I see. Then, if we move the startTransaction to the client slide, it may help a lot. Let me try.

szetszwo added a commit to szetszwo/ozone that referenced this pull request May 6, 2022
@szetszwo
Copy link
Contributor Author

@captainzmc , @guohao-rosicky , after some tests, I found that the current client Stream code is slow. The Stream code was mainly copied from the Async code. The flush and watch handling in Async seems slowing down Streaming since they call wait().

@captainzmc
Copy link
Member

captainzmc commented May 13, 2022

@captainzmc , @guohao-rosicky , after some tests, I found that the current client Stream code is slow. The Stream code was mainly copied from the Async code. The flush and watch handling in Async seems slowing down Streaming since they call wait().

Thanks @szetszwo for the test, the performance problem may be not easy to solve. I wonder if we can continue with HDDS-4474 (WriteSmallFile) and HDDS-5869 (S3Gateway)? The streaming will be almost complete after these two are completed.
After finish HDDS-4474 and HDDS-5869, we can start the Merge Master. More work can continue on the Master, and there will have more people help optimize. It would be best if streaming can add into ozone 1.3.0 release.

@szetszwo
Copy link
Contributor Author

@captainzmc , I am fine to continue working to these two JIRAs.

For HDDS-4474 (WriteSmallFile), I thought @guohao-rosicky was going to work on RATIS-1157 ?

For HDDS-5869 (S3Gateway), it seems that the pull request was not able to pass all the tests yet.

szetszwo added a commit that referenced this pull request May 13, 2022
@guohao-rosicky
Copy link
Contributor

guohao-rosicky commented May 13, 2022

Hi, @szetszwo

For HDDS-5869 (S3Gateway), it seems that the pull request was not able to pass all the tests yet.

For now, I will focus on HDDS-5869 (S3Gateway), which will help us merge HDDS-4454 into master and add ozone 1.3.0 release.

For HDDS-4474 (WriteSmallFile), I thought @guohao-rosicky was going to work on RATIS-1157 ?

Ratis-1157 and HDDS-4474 (WriteSmallFile), I think can continue to optimize after HDDS-4454 merged into master.

@captainzmc
Copy link
Member

Ratis-1157 and HDDS-4474 (WriteSmallFile), I think can continue to optimize after HDDS-4454 merged into master.

Thanks @guohao-rosicky. Agree. Ratis2.3.0 is currently in release and will not contain ratis-1157. Hdds-4474 WriteSmallFile optimization we can improve after merged into master. @szetszwo, Any suggestions here?

szetszwo added a commit that referenced this pull request May 24, 2022
@szetszwo
Copy link
Contributor Author

szetszwo commented Jun 7, 2022

@captainzmc , @guohao-rosicky , I find out why Ozone-Streaming is slow -- the watch requests slows it down. Will submit a pr.

szetszwo added a commit that referenced this pull request Jun 9, 2022
captainzmc pushed a commit to captainzmc/hadoop-ozone that referenced this pull request Jul 4, 2022
szetszwo added a commit that referenced this pull request Oct 25, 2022
… the stream. (#3229)

(cherry picked from commit d72a8a0)
(cherry picked from commit 1dddbec2524ca3463aa6140f33c6f16f0468fce8)
szetszwo added a commit that referenced this pull request Nov 7, 2022
… the stream. (#3229)

(cherry picked from commit d72a8a0)
(cherry picked from commit 1dddbec2524ca3463aa6140f33c6f16f0468fce8)
(cherry picked from commit f2d574c)
szetszwo added a commit that referenced this pull request Dec 1, 2022
szetszwo added 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
@guohao-rosicky
Copy link
Contributor

@szetszwo
Copy link
Contributor Author

szetszwo commented Jun 6, 2023

@guohao-rosicky , thanks a lot for pointing it out! Would you like to submit a pull request for fixing it? Or I can do it.

@guohao-rosicky
Copy link
Contributor

@guohao-rosicky , thanks a lot for pointing it out! Would you like to submit a pull request for fixing it? Or I can do it.
@szetszwo
Thank you for your attention. You can fix it, that's very important to me.
Can you submit a jira? I will continue to follow this jira.

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.

4 participants