Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[#2181]improvement(server): Replace ShuffleBlockInfo with ShufflePartitionedBlock in netty #2182

Open
wants to merge 1 commit into
base: master
Choose a base branch
from

Conversation

lwllvyb
Copy link
Contributor

@lwllvyb lwllvyb commented Oct 15, 2024

What changes were proposed in this pull request?

On the server side of the netty mode, when the server receives the sendShuffleData message, it decodes it into an unnecessary ShuffleBlockInfo and then converts it to ShufflePartitionedBlock in handleSendShuffleDataRequest.

Why are the changes needed?

Fix: #2181

Does this PR introduce any user-facing change?

No.

How was this patch tested?

Locally

@lwllvyb
Copy link
Contributor Author

lwllvyb commented Oct 15, 2024

To ensure upgrade compatibility, only the server-side code was modified.

Copy link

github-actions bot commented Oct 15, 2024

Test Results

 2 926 files  +10   2 926 suites  +10   6h 14m 27s ⏱️ + 5m 19s
 1 088 tests ± 0   1 086 ✅ + 1   2 💤 ±0  0 ❌ ±0 
13 630 runs  +10  13 600 ✅ +11  30 💤 ±0  0 ❌ ±0 

Results for commit 17d4112. ± Comparison against base commit 6e18b3e.

♻️ This comment has been updated with latest results.

@jerqi
Copy link
Contributor

jerqi commented Oct 18, 2024

Is this a improvement?

@lwllvyb
Copy link
Contributor Author

lwllvyb commented Oct 18, 2024

Is this a improvement?

In our stress test environment, it will generates 80million blocks. Removing useless ShuffleBlockInfo will reduce the JVM GC pressure.

@jerqi jerqi changed the title fix[server] Replace ShuffleBlockInfo with ShufflePartitionedBlock in netty [#2181]improvement(server): Replace ShuffleBlockInfo with ShufflePartitionedBlock in netty Oct 18, 2024
@jerqi
Copy link
Contributor

jerqi commented Oct 18, 2024

@maobaolong Could you help me review this?

@maobaolong
Copy link
Member

@lwllvyb With pleasure!

Copy link
Member

@maobaolong maobaolong left a comment

Choose a reason for hiding this comment

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

@lwllvyb Thanks for this great work, I guess you could do the following changes for this PR.

  • Split the unrelated changes into another PR.
  • Create an issue to track the followup task and comment(TODO comment) to a place.
  • Calc the decodeLength by a simply and clear way rather than compute it everywhere. Maybe you can get it by ByteBuf position change.

long crc = byteBuf.readLong();
long taskAttemptId = byteBuf.readLong();
public static Pair<Integer, ShufflePartitionedBlock> decodeShuffleBlockInfo(ByteBuf byteBuf) {
// partId Int
Copy link
Member

Choose a reason for hiding this comment

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

It would be better to create another issue to track the next plan base on this PR, and it could be clear and easy to track it by adding a issue link here as a TODO comment.

Copy link
Member

Choose a reason for hiding this comment

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

image

Copy link
Member

Choose a reason for hiding this comment

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

We should deprecate the SEND_SHUFFLE_DATA_REQUEST and removed in the future when release a new major version

Copy link
Member

Choose a reason for hiding this comment

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

When we upgrade to V2, we could send the encoded length to server, to avoid re-compute the encodedLength in the server side again.

@@ -59,19 +59,27 @@ public static int encodeLengthOfShuffleServerInfo(ShuffleServerInfo shuffleServe
+ 2 * Integer.BYTES;
}

public static int encodeLengthShuffleBlockInfoCommon() {
Copy link
Member

Choose a reason for hiding this comment

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

This could be a Constant?

// freeMemory Long
byteBuf.skipBytes(8);

int shuffleBlockInfoLength =
Copy link
Member

Choose a reason for hiding this comment

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

You mean shuffleBlockInfoLength to shuffleBlockInfoEncodeLength?

}

public int getEncodedLength() {
return encodedLength;
Copy link
Member

Choose a reason for hiding this comment

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

This make me confusing, what is the different between getEncodedLength() and encodedLength?

@@ -912,11 +905,9 @@ private ServerRpcAuditContext createAuditContext(
auditLogger = AUDIT_LOGGER;
}
ServerRpcAuditContext auditContext = new ServerRpcAuditContext(auditLogger);
auditContext.withCreationTimeNs(System.nanoTime());
Copy link
Member

Choose a reason for hiding this comment

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

This could be another fix?

+ Encoders.encodeLengthOfShuffleServerInfos(serverInfos);

return Pair.of(
shuffleBlockInfoLength,
Copy link
Member

Choose a reason for hiding this comment

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

It could be better to calc the decodedLength by the byteBuf position

@codecov-commenter
Copy link

codecov-commenter commented Oct 18, 2024

Codecov Report

Attention: Patch coverage is 62.06897% with 33 lines in your changes missing coverage. Please review.

Project coverage is 53.26%. Comparing base (78fe934) to head (1a6b65d).
Report is 15 commits behind head on master.

Files with missing lines Patch % Lines
...ommon/netty/protocol/SendShuffleDataRequestV1.java 69.81% 16 Missing ⚠️
.../common/netty/protocol/SendShuffleDataRequest.java 0.00% 9 Missing ⚠️
...niffle/server/netty/ShuffleServerNettyHandler.java 0.00% 8 Missing ⚠️
Additional details and impacted files
@@             Coverage Diff              @@
##             master    #2182      +/-   ##
============================================
+ Coverage     52.86%   53.26%   +0.39%     
+ Complexity     3385     3221     -164     
============================================
  Files           517      481      -36     
  Lines         28208    26073    -2135     
  Branches       2633     2467     -166     
============================================
- Hits          14913    13887    -1026     
+ Misses        12332    11281    -1051     
+ Partials        963      905      -58     

☔ View full report in Codecov by Sentry.
📢 Have feedback on the report? Share it here.

@lwllvyb
Copy link
Contributor Author

lwllvyb commented Oct 18, 2024

@maobaolong Fixed

@lwllvyb lwllvyb force-pushed the serverBlockInfoOpt01 branch 2 times, most recently from 7951a9e to 64af380 Compare October 18, 2024 12:02
@lwllvyb lwllvyb closed this Oct 19, 2024
@lwllvyb lwllvyb reopened this Oct 19, 2024
@@ -69,6 +70,32 @@ public static ShuffleBlockInfo decodeShuffleBlockInfo(ByteBuf byteBuf) {
taskAttemptId);
}

public static ShufflePartitionedBlock decodeShuffleBlockInfoV1(ByteBuf byteBuf) {
Copy link
Member

Choose a reason for hiding this comment

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

How about decodeShufflePartitionedBlockV1?

Copy link
Member

Choose a reason for hiding this comment

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

@lwllvyb Maybe you miss this comment

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Thanks for your suggestion. I will change the function name.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

@maobaolong Fixed.

Copy link
Member

@maobaolong maobaolong left a comment

Choose a reason for hiding this comment

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

Thanks for your update, it is better enough for code.

I comment inline to show the place that I concern, could you show that this change can not introduce memory leak?


@Override
public String getOperationType() {
return "sendShuffleData";
Copy link
Member

Choose a reason for hiding this comment

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

sendShuffleDataV1

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Fixed it as you suggested.

@@ -274,7 +273,7 @@ public void handleSendShuffleDataRequest(TransportClient client, SendShuffleData
return;
}
final long start = System.currentTimeMillis();
shuffleBufferManager.releaseMemory(req.encodedLength(), false, true);
shuffleBufferManager.releaseMemory(req.getDecodedLength(), false, true);
Copy link
Member

Choose a reason for hiding this comment

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

Worried about req.getDecodedLength() is not equals to encodedLength since the encodedLength is calculated approximately, but req.getDecodedLength() is calculated accurately.

So It is necessary to have a test to ensure the releaseMemory is equals to requireMemory, so that there is not memory leak take place.

Copy link
Contributor Author

@lwllvyb lwllvyb Oct 21, 2024

Choose a reason for hiding this comment

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

There already has a test class NettyProtocolTest to make sure that the encodedLength sent from client is same as the getDecodedLength of the server received.
Does we need a new test case?

Copy link
Member

Choose a reason for hiding this comment

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

@lwllvyb It is enough, thanks for show me this

@lwllvyb lwllvyb closed this Oct 21, 2024
@lwllvyb lwllvyb reopened this Oct 21, 2024
@lwllvyb lwllvyb closed this Oct 21, 2024
@lwllvyb lwllvyb reopened this Oct 21, 2024
Copy link
Member

@maobaolong maobaolong left a comment

Choose a reason for hiding this comment

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

@lwllvyb Thanks for this improvement, left a minor comment before, please take a look, otherwise, LGTM.

And look forward the next step base on this

@lwllvyb
Copy link
Contributor Author

lwllvyb commented Oct 22, 2024

Ping @jerqi

import org.apache.uniffle.common.ShufflePartitionedBlock;
import org.apache.uniffle.common.util.ByteBufUtils;

public class SendShuffleDataRequestV1 extends RequestMessage {
Copy link
Contributor

Choose a reason for hiding this comment

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

Why do we use v1 here?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The earliest version was a modification of the original class and there will be a lot of intrusive code and it will not be easy to review and understand.

SendShuffleDataRequestV1 class will not decode useless information to ShuffleBlockInfo which is temporary.

Copy link
Contributor

Choose a reason for hiding this comment

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

Will we have v2? If we don't have v2, maybe we should remove v1.

Copy link
Contributor Author

@lwllvyb lwllvyb Oct 23, 2024

Choose a reason for hiding this comment

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

SendShuffleDataRequestV1 will be only used by Shuffle Server and SendShuffleDataRequest only be used by client. It is compatible with currently running cluster and serves as a transition.

Next stage, i will add SendShuffleDataRequestV2 which will be used by Shuffle Server and client. SendShuffleDataRequestV2 will not include useless information which is sent from client.

Copy link
Contributor

Choose a reason for hiding this comment

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

The request of the client is different from the one of the server. It's a little weird.

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.

[Improvement] Replace ShuffleBlockInfo with ShufflePartitionedBlock in the server with netty mode.
4 participants