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

[#590][part-1] ManagedBuffer instead ByteBuf to hold ShuffleData #906

Merged
merged 3 commits into from
Jun 14, 2023

Conversation

xumanbu
Copy link
Contributor

@xumanbu xumanbu commented May 26, 2023

What changes were proposed in this pull request?

part-1:

  1. add a ManagedBuffer instead ByteBuf to hold ShuffleData.
  2. ShuffleResultData & GetLocalShuffleDataResponse use ManagedBuffer instead of ByteBuf
  3. MessageEncoder write support sendfile

Why are the changes needed?

Fix: #590

Does this PR introduce any user-facing change?

(Please list the user-facing changes introduced by your change, including
No.

How was this patch tested?

Integration Testing

@codecov-commenter
Copy link

codecov-commenter commented May 26, 2023

Codecov Report

Merging #906 (26c6d0c) into master (9c62784) will increase coverage by 1.77%.
The diff coverage is 22.97%.

@@             Coverage Diff              @@
##             master     #906      +/-   ##
============================================
+ Coverage     55.22%   56.99%   +1.77%     
- Complexity     2197     2280      +83     
============================================
  Files           333      327       -6     
  Lines         16444    14675    -1769     
  Branches       1306     1354      +48     
============================================
- Hits           9081     8364     -717     
+ Misses         6851     5834    -1017     
+ Partials        512      477      -35     
Impacted Files Coverage Δ
...rg/apache/uniffle/common/netty/MessageEncoder.java 68.75% <0.00%> (-9.83%) ⬇️
.../common/netty/buffer/FileSegmentManagedBuffer.java 0.00% <0.00%> (ø)
...client/impl/grpc/ShuffleServerGrpcNettyClient.java 0.00% <0.00%> (ø)
...niffle/server/netty/ShuffleServerNettyHandler.java 1.68% <0.00%> (ø)
...a/org/apache/uniffle/common/ShuffleDataResult.java 35.71% <7.14%> (-7.77%) ⬇️
...on/netty/protocol/GetLocalShuffleDataResponse.java 66.66% <54.54%> (-7.02%) ⬇️
...che/uniffle/common/netty/buffer/ManagedBuffer.java 100.00% <100.00%> (ø)
...niffle/common/netty/buffer/NettyManagedBuffer.java 100.00% <100.00%> (ø)

... and 93 files with indirect coverage changes

📣 We’re building smart automated test selection to slash your CI/CD build times. Learn more

@jerqi jerqi requested a review from leixm May 26, 2023 06:19
@jerqi
Copy link
Contributor

jerqi commented May 26, 2023

@leixm Could you help me review this pr?

@leixm
Copy link
Contributor

leixm commented May 26, 2023

@leixm Could you help me review this pr?

Sure.

ByteBufUtils.copyByteBuf(data, buf);
data.release();
if (buffer instanceof FileSegmentManagedBuffer) {
buf.writeInt(buffer.size());
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 only write the size of buffer?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

FileSegmentManagedBuffer.convertToNetty() returned DefaultFileRegion type object , in order to achieve sendfile the DefaultFileRegion type object should writed by channel.write(object).
could you give same other suggest of realization for sendfile in netty?

Copy link
Contributor

Choose a reason for hiding this comment

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

How do the other systems like Spark, Celeborn implement this?

Copy link
Contributor Author

@xumanbu xumanbu May 31, 2023

Choose a reason for hiding this comment

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

In order to keep the uniffle interface design as much as possible, reference Celeborn&Spark add ManagedBuffer to hold different type ShuffleData(bytebuf or FileRegion).

Copy link
Contributor

Choose a reason for hiding this comment

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

In order to keep the uniffle interface design as much as possible, reference Celeborn&Spark add ManagedBuffer to hold different type ShuffleData(bytebuf or FileRegion).

Ok for me.

@@ -59,5 +59,9 @@ public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise)
byteBuf.release();
}
ctx.writeAndFlush(byteBuf);
// do transferTo send data after encode buffer send.
if (message instanceof Transferable) {
((Transferable) message).transferTo(ctx.channel());
Copy link
Contributor Author

Choose a reason for hiding this comment

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

@jerqi the line is write the FileSegmentManageBuffer ShuffleData.

Copy link
Contributor

Choose a reason for hiding this comment

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

OK.

}
return ByteBufUtils.readBytes(data);
return ByteBufUtils.readBytes(Unpooled.wrappedBuffer(buffer.nioByteBuffer()));
Copy link
Contributor

Choose a reason for hiding this comment

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

Maybe we can return ByteBufUtils.readBytes(buffer.byteBuf());

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok,I' ll fix it

Copy link
Contributor

Choose a reason for hiding this comment

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

@xumanbu remind to fix this.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

ok

}

public ByteBuf getDataBuf() {
return data;
return Unpooled.wrappedBuffer(buffer.nioByteBuffer());
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto.

@leixm
Copy link
Contributor

leixm commented May 31, 2023

Maybe we can also make SendShuffleDataRequest hold ManagedBuffer instead of ByteBuf.

@jerqi
Copy link
Contributor

jerqi commented Jun 1, 2023

Maybe we can also make SendShuffleDataRequest hold ManagedBuffer instead of ByteBuf.

What's the benefit?

@jerqi jerqi requested a review from leixm June 8, 2023 09:11
Copy link
Contributor

@leixm leixm left a comment

Choose a reason for hiding this comment

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

+1 LGTM.

Copy link
Contributor

@jerqi jerqi left a comment

Choose a reason for hiding this comment

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

LGTM, merged to master.

@jerqi jerqi merged commit b8f17bf into apache:master Jun 14, 2023
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.

[Subtask] [Netty] Support zero copy to read local file
4 participants