Skip to content

[#436] feat(client,server): Introduce multi-part LocalStorageManager#2253

Merged
maobaolong merged 8 commits intoapache:masterfrom
maobaolong:raidStorage.gh
Nov 26, 2024
Merged

[#436] feat(client,server): Introduce multi-part LocalStorageManager#2253
maobaolong merged 8 commits intoapache:masterfrom
maobaolong:raidStorage.gh

Conversation

@maobaolong
Copy link
Member

@maobaolong maobaolong commented Nov 16, 2024

What changes were proposed in this pull request?

  • Introduce a factory to create specific LocalStorageManager by config.
  • Introduce multiply disk LocalStorageManager.

Why are the changes needed?

Fix: #436

Does this PR introduce any user-facing change?

No.

How was this patch tested?

  • Existing UTs and new added UT
  • Tested on our pressure test cluster.
    • new client -> old server ✓
    • new client -> new server ✓
    • old client -> new server ❌, so we have to upgraded client first, than upgrade the servers

@github-actions
Copy link

github-actions bot commented Nov 16, 2024

Test Results

 2 966 files  +26   2 966 suites  +26   6h 28m 59s ⏱️ + 18m 27s
 1 096 tests + 4   1 094 ✅ + 6   2 💤 ±0  0 ❌  - 1 
13 735 runs  +72  13 705 ✅ +74  30 💤 ±0  0 ❌  - 1 

Results for commit e2254f7. ± Comparison against base commit 54611f3.

♻️ This comment has been updated with latest results.

@maobaolong maobaolong force-pushed the raidStorage.gh branch 4 times, most recently from d372b3d to 58f4644 Compare November 17, 2024 01:07
@maobaolong
Copy link
Member Author

@zuston Thank you advance for your previous work on #436 , would you like to take a look at this PR?

@maobaolong maobaolong requested a review from zuston November 17, 2024 02:20
@roryqi roryqi changed the title [#436] feat(client/server): Introduce multiply disk LocalStorageManager [#436] feat(client,server): Introduce multiply disk LocalStorageManager Nov 18, 2024
@zuston
Copy link
Member

zuston commented Nov 18, 2024

@zuston Thank you advance for your previous work on #436 , would you like to take a look at this PR?

Yes. I will take a look in the next 3 days.

@codecov-commenter
Copy link

codecov-commenter commented Nov 18, 2024

Codecov Report

Attention: Patch coverage is 24.40318% with 285 lines in your changes missing coverage. Please review.

Project coverage is 51.82%. Comparing base (477bc30) to head (b09f92d).
Report is 6 commits behind head on master.

Files with missing lines Patch % Lines
...fle/common/netty/protocol/CompositeFileRegion.java 0.00% 50 Missing ⚠️
...netty/protocol/GetLocalShuffleIndexV2Response.java 0.00% 27 Missing ⚠️
...iffle/server/storage/Raid0LocalStorageManager.java 45.23% 19 Missing and 4 partials ⚠️
...dler/impl/CompositeLocalFileServerReadHandler.java 0.00% 23 Missing ⚠️
...on/netty/buffer/MultiFileSegmentManagedBuffer.java 0.00% 22 Missing ⚠️
...client/impl/grpc/ShuffleServerGrpcNettyClient.java 0.00% 22 Missing ⚠️
...pache/uniffle/storage/common/CompositeStorage.java 0.00% 21 Missing ⚠️
...n/netty/protocol/GetLocalShuffleDataV2Request.java 0.00% 19 Missing ⚠️
...ffle/common/segment/LocalOrderSegmentSplitter.java 54.54% 6 Missing and 4 partials ⚠️
...iffle/common/segment/FixedSizeSegmentSplitter.java 62.50% 5 Missing and 4 partials ⚠️
... and 19 more
Additional details and impacted files
@@             Coverage Diff              @@
##             master    #2253      +/-   ##
============================================
- Coverage     52.65%   51.82%   -0.83%     
+ Complexity     3376     2977     -399     
============================================
  Files           514      475      -39     
  Lines         27565    22645    -4920     
  Branches       2582     2084     -498     
============================================
- Hits          14513    11735    -2778     
+ Misses        12092    10150    -1942     
+ Partials        960      760     -200     

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


🚨 Try these New Features:

long crc = block.getCrc();
long startOffset = dataWriter.nextOffset();
// put storage id to high 8 bits
long startOffset = ShuffleDataSegment.wrapOffset(dataWriter.nextOffset(), storageId);
Copy link
Member

Choose a reason for hiding this comment

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

This is too rough. If the offset's high 8 bits has used, this will make the data inconsistent.
And this is also not compatible with the previous shuffle-server/client version.

@zuston
Copy link
Member

zuston commented Nov 18, 2024

Thanks for proposing this. But I'm afraid of the implicit storage index behind the offset, which is dangerous. Why not introducing the extra storage index into the protobuf.

@maobaolong
Copy link
Member Author

@zuston Thanks for your review, in fact I have done another approach with the proto change, which introduce storageId into the shuffleIndex file and changed the netty encode/decode methods in the client/server side, it can be more graceful than this PR and it can works as expected too. But I cannot give a gray online plan, I cannot upgrade client and server at this same time.

For this PR, although both client and server are need to upgrade, but I can upgrade all the clients first, then I upgrade the server.

@zuston
Copy link
Member

zuston commented Nov 19, 2024

Thanks for your review, in fact I have done another approach with the proto change, which introduce storageId into the shuffleIndex file and changed the netty encode/decode methods in the client/server side

I don't think the storage id should be recorded into the index file, this could be retrieved from the local storage index. If so, the client could do some compatible change for this.

@maobaolong maobaolong force-pushed the raidStorage.gh branch 2 times, most recently from ae1c289 to b09f92d Compare November 20, 2024 14:55
@maobaolong
Copy link
Member Author

@zuston Thanks for the discussion and suggestion, I change to add storageIds as the response of getShuffleIndex, it looks clear than the previous tricky approach which use high 8 bits of offset as storageId, and the newer approach did not modify the layout of index file in server, this is a good idea, PTAL.

Copy link
Member

@zuston zuston left a comment

Choose a reason for hiding this comment

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

Overall LGTM now. Thanks for your effort @maobaolong

bufferSegments.add(
new BufferSegment(blockId, bufferOffset, length, uncompressLength, crc, taskAttemptId));
bufferOffset += length;
boolean storageChanged = preStorageId != -1 && currentStorageId != preStorageId;
Copy link
Member

Choose a reason for hiding this comment

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

This class should be fully tested by unit test cases to cover the storage changed situation

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

* ShuffleDataSegment size should < readBufferSize 3. Single shuffleDataSegment's blocks should
* be continuous
*/
int[] storageIds = shuffleIndexResult.getStorageIds();
Copy link
Member

Choose a reason for hiding this comment

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

For the fixedSizeSegments and localOrderSegmentSpilter, the storage changing logic may be unified into the underlying abstract segment spiltter.

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

@zuston
Copy link
Member

zuston commented Nov 22, 2024

BTW, SHUFFLE_SERVER_VERSION should be updated, because the new server is not compatible with the old version client. @maobaolong

@maobaolong
Copy link
Member Author

maobaolong commented Nov 22, 2024

@zuston Thanks for your review, rename related minor comments has been addressed, others will updated it these weekend.

BTW, SHUFFLE_SERVER_VERSION should be updated, because the new server is not compatible with the old version client. @maobaolong

This cannot work for gray upgrade.

  • New Client -> Old Server ❌


import io.netty.channel.FileRegion;

public class CompositeFileRegion extends AbstractFileRegion {
Copy link
Contributor

Choose a reason for hiding this comment

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

@rickyma Could you help review the Netty part?


@Override
public boolean release() {
super.release();
Copy link
Contributor

Choose a reason for hiding this comment

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

The return value of super.release() is never checked.

Maybe use the return value to initialize released:

boolean released = super.release();

Copy link
Member Author

Choose a reason for hiding this comment

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

Done

}

@Override
protected void deallocate() {}
Copy link
Contributor

Choose a reason for hiding this comment

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

This is empty. Do we need to release extra resources here?

Copy link
Member Author

Choose a reason for hiding this comment

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

Added

position -= region.count();
} else {
long transferredNow = region.transferTo(target, position);
totalTransferred += transferredNow;
Copy link
Contributor

Choose a reason for hiding this comment

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

totalTranferred -> totalBytesTransferred
transferredNow -> currentBytesTransferred
transferred -> bytesTransferred

Copy link
Member Author

Choose a reason for hiding this comment

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

done


@Override
public boolean release(int decrement) {
super.release(decrement);
Copy link
Contributor

Choose a reason for hiding this comment

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

ditto.

The return value of super.release(decrement) is never checked.

Copy link
Member Author

Choose a reason for hiding this comment

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

done

@maobaolong
Copy link
Member Author

@rickyma Thanks for your review and suggestion. It helps to make this PR better.

I updated the code and addressed your comments, PTAL.

rickyma
rickyma previously approved these changes Nov 25, 2024
Copy link
Contributor

@rickyma rickyma left a comment

Choose a reason for hiding this comment

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

Netty part LGTM

roryqi
roryqi previously approved these changes Nov 25, 2024
@maobaolong
Copy link
Member Author

@jerqi @rickyma @zuston Thanks for your review, will merge after our final test pass.

@maobaolong maobaolong dismissed stale reviews from roryqi and rickyma via fee8c4e November 25, 2024 10:41
roryqi
roryqi previously approved these changes Nov 25, 2024
@rickyma
Copy link
Contributor

rickyma commented Nov 25, 2024

The title and description of this PR should be improved.

multiply is not right.

@maobaolong maobaolong changed the title [#436] feat(client,server): Introduce multiply disk LocalStorageManager [#436] feat(client,server): Introduce multi-part LocalStorageManager Nov 25, 2024
@maobaolong
Copy link
Member Author

@rickyma The title has been updated.

@maobaolong maobaolong merged commit b7d391c into apache:master Nov 26, 2024
@rickyma
Copy link
Contributor

rickyma commented Nov 26, 2024

The description of this PR was not changed.

roryqi pushed a commit that referenced this pull request Dec 4, 2024
…2253)

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

- Introduce a factory to create specific LocalStorageManager by config.
- Introduce multiply disk LocalStorageManager.

### Why are the changes needed?

Fix: #436 

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

No.

### How was this patch tested?

- Existing UTs and new added UT
- Tested on our pressure test cluster.
  - new client -> old server ✓
  - new client -> new server ✓
  - old client -> new server ❌, so we have to upgraded client first, than upgrade the servers
zuston added a commit to zuston/riffle that referenced this pull request Dec 4, 2024
zuston added a commit to zuston/riffle that referenced this pull request Dec 4, 2024
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.

[FEATURE] Support concurrent write to multiple local disks for one partition

5 participants