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

[#133] feat(netty): Implement ShuffleServer interface. #879

Merged
merged 12 commits into from
May 17, 2023

Conversation

leixm
Copy link
Contributor

@leixm leixm commented May 15, 2023

What changes were proposed in this pull request?

Implement ShuffleServer interface.

Why are the changes needed?

For #133.

Does this PR introduce any user-facing change?

No.

How was this patch tested?

existing UTs.

@codecov-commenter
Copy link

codecov-commenter commented May 15, 2023

Codecov Report

Merging #879 (3d197f3) into master (0125fe3) will increase coverage by 0.71%.
The diff coverage is 20.80%.

@@             Coverage Diff              @@
##             master     #879      +/-   ##
============================================
+ Coverage     56.57%   57.28%   +0.71%     
+ Complexity     2174     2173       -1     
============================================
  Files           327      310      -17     
  Lines         15979    13897    -2082     
  Branches       1263     1278      +15     
============================================
- Hits           9040     7961    -1079     
+ Misses         6430     5505     -925     
+ Partials        509      431      -78     
Impacted Files Coverage Δ
...e/uniffle/common/netty/client/TransportClient.java 17.74% <0.00%> (-0.60%) ⬇️
...niffle/server/netty/ShuffleServerNettyHandler.java 1.68% <1.68%> (ø)
.../common/netty/handle/TransportResponseHandler.java 25.00% <16.66%> (ø)
...apache/uniffle/common/ShufflePartitionedBlock.java 54.76% <30.00%> (-15.83%) ⬇️
...e/common/netty/handle/TransportChannelHandler.java 38.59% <38.59%> (ø)
...a/org/apache/uniffle/common/ShuffleDataResult.java 43.47% <50.00%> (-8.91%) ⬇️
...e/common/netty/handle/TransportRequestHandler.java 69.23% <69.23%> (ø)
.../org/apache/uniffle/server/netty/StreamServer.java 65.07% <75.00%> (+0.56%) ⬆️
...rg/apache/uniffle/server/buffer/ShuffleBuffer.java 92.56% <80.00%> (-2.48%) ⬇️
...le/common/netty/client/TransportClientFactory.java 75.51% <100.00%> (+0.51%) ⬆️
... and 5 more

... and 33 files with indirect coverage changes

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

@@ -80,4 +80,9 @@ public static final byte[] readBytes(ByteBuf buf) {
buf.resetReaderIndex();
return bytes;
}

public static void readBytes(ByteBuf from, byte[] to, int offset, int length) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add UT for this method?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Sure.

@jerqi jerqi requested a review from smallzhongfeng May 15, 2023 03:18
@jerqi
Copy link
Contributor

jerqi commented May 15, 2023

@smallzhongfeng Could you help me review this pr?

return data;
}

public ByteBuffer getDataBuffer() {
Copy link
Contributor

Choose a reason for hiding this comment

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

Do you need to consider nioBufferCount != 1?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

It was my overlooked, since the increaseWriterIndex parameter is true when we call CompositeByteBuf.addComponent, CompositeByteBuf can be used as a normal ByteBuf.

Copy link
Contributor

Choose a reason for hiding this comment

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

Do you mean that we don't need consider nioBufferCount != 1?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Yes, we don't need, I have made some tests.

Copy link
Contributor

Choose a reason for hiding this comment

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

Yes, we don't need, I have made some tests.

Could you add some UT for this case?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

See ByteBufUtilsTest#test, CompositeByteBuf can be read as ByteBuf.

final int expectedLength = byteBuf.readableBytes() + byteBuf1.readableBytes();
CompositeByteBuf compositeByteBuf = Unpooled.compositeBuffer();
compositeByteBuf.addComponent(true, byteBuf);
compositeByteBuf.addComponent(true, byteBuf1);

ByteBuf res = Unpooled.buffer(100);
ByteBufUtils.copyByteBuf(compositeByteBuf, res);
assertEquals(expectedLength, res.readableBytes() - Integer.BYTES);

res.clear();
ByteBufUtils.copyByteBuf(compositeByteBuf, res);
assertEquals(expectedLength, res.readableBytes()  - Integer.BYTES);


addDecoder(ctx, magicByte);
public void exceptionCaught(Throwable cause, TransportClient client) {

Copy link
Contributor

Choose a reason for hiding this comment

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

Could we add some logs for Exception?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Okay.

@leixm
Copy link
Contributor Author

leixm commented May 15, 2023

LocalFileServerReadHandler.getShuffleData() still uses byte[], we will not change it at present.

@leixm
Copy link
Contributor Author

leixm commented May 15, 2023

cc @smallzhongfeng @jerqi PTAL.

@@ -161,11 +163,10 @@ public synchronized ShuffleDataResult getShuffleData(
updateBufferSegmentsAndResultBlocks(
lastBlockId, readBufferSize, bufferSegments, readBlocks, expectedTaskIds);
if (!bufferSegments.isEmpty()) {
int length = calculateDataLength(bufferSegments);
byte[] data = new byte[length];
CompositeByteBuf byteBuf = Unpooled.compositeBuffer();
Copy link
Contributor

Choose a reason for hiding this comment

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

Will CompositeByteBuf cause some extra copy because we may trigger resize action?
Will we use direct memory for CompositeByteBuf?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

CompositeByteBuf does not need to apply for memory. It depends on child ByteBuf. ByteBuf comes from netty server, so it depends on ByteBufAllocator in the end. I will fix this problem when init StreamServer.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Actually PooledByteBufAllocator.DEFAULT use direct memory, we needn't fix any more.

Copy link
Contributor

@jerqi jerqi May 16, 2023

Choose a reason for hiding this comment

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

Actually PooledByteBufAllocator.DEFAULT use direct memory, we needn't fix any more.

OK.

Copy link
Contributor

Choose a reason for hiding this comment

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

CompositeByteBuf does not need to apply for memory. It depends on child ByteBuf. ByteBuf comes from netty server, so it depends on ByteBufAllocator in the end. I will fix this problem when init StreamServer.

Do you consider CompositeByteBuf#consolidateIfNeeded()? In this method, if we exceed the maxComponentNum, we will trigger the logic of merging component.

Copy link
Contributor

Choose a reason for hiding this comment

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

I'm not sure, but there is a problem which we need to discuss. For me, we could add some comments, 16 is enough if we only use it to read memory data in my opinion.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

16 is not enough, because read buffer is 16M.

Copy link
Contributor

Choose a reason for hiding this comment

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

16 is not enough, because read buffer is 16M.

We would better use a larger value to avoid data copy.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Suppose the size of a block is small enough to be 16k, readBuffer=16M, we can make maxNumComponents=1024 to avoid data copy.

Copy link
Contributor

Choose a reason for hiding this comment

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

OK.

import org.apache.uniffle.common.netty.client.TransportClient;
import org.apache.uniffle.common.netty.protocol.RequestMessage;

public class BaseMessageHandler {
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 this class have so many empty methods? Is it better to use abstract class?

import org.apache.uniffle.common.netty.client.TransportClient;
import org.apache.uniffle.common.netty.protocol.RequestMessage;

public abstract class BaseMessageHandler {
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we use interface here? Because we don't use any default implement of this class.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

done.

leixianming added 2 commits May 16, 2023 11:42
cachedClient.getChannel().pipeline().get(TransportResponseHandler.class);
TransportChannelHandler handler =
cachedClient.getChannel().pipeline().get(TransportChannelHandler.class);
synchronized (handler) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Could we instead optimistic lock of synchronized ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Spark also like this.

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, thanks @leixm @smallzhongfeng

@jerqi jerqi merged commit 3b8d6c8 into apache:master May 17, 2023
zuston pushed a commit that referenced this pull request Mar 28, 2024
… release readMemory (#1605)

### What changes were proposed in this pull request?

1. Add a `ChannelFutureListener` and use its callback mechanism to release `readMemory` only after the `writeAndFlush` method is truly completed.
2. Change the descriptions of configurations `rss.server.buffer.capacity.ratio` and `rss.server.read.buffer.capacity.ratio`. 

### Why are the changes needed?

This is actually a bug, which was introduced by PR #879. The issue has been present since the very beginning when the Netty feature was first integrated.
Fix #1596.

### Does this PR introduce _any_ user-facing change?

No.

### How was this patch tested?

I don't think we need new tests. Tested in our env.
The new log will be:
```
[2024-03-26 23:11:51.039] [epollEventLoopGroup-3-158] [INFO] ShuffleServerNettyHandler.operationComplete - Successfully executed getLocalShuffleData for appId[application_1703049085550_7359933_1711463990606], shuffleId[0], partitionId[1328], offset[0], length[14693742]. Took 1457 ms and retrieved 14693742 bytes of data
[2024-03-26 23:11:51.040] [epollEventLoopGroup-3-130] [INFO] ShuffleServerNettyHandler.operationComplete - Successfully executed getMemoryShuffleData for appId[application_1703049085550_7359933_1711463990606], shuffleId[0], partitionId[1262]. Took 1 ms and retrieved 0 bytes of data
[2024-03-26 23:11:51.068] [epollEventLoopGroup-3-177] [INFO] ShuffleServerNettyHandler.operationComplete - Successfully executed getLocalShuffleIndex for appId[application_1703049085550_7359933_1711463990606], shuffleId[0], partitionId[1366]. Took 918 ms and retrieved 1653600 bytes of data
```
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