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

[#378] feat: introduce storage manager selector #621

Merged
merged 1 commit into from
Mar 1, 2023

Conversation

zuston
Copy link
Member

@zuston zuston commented Feb 17, 2023

What changes were proposed in this pull request?

  1. introduce storage manager selector to support more selector strategy for MultiStorageManager
  2. Introduce the conf of rss.server.multistorage.cold.storage.preferred.factor to support different flush strategy, like I hope huge partition directly flushed to HDFS and normal partition could be flushed to DISK when single buffer flush is enabled.

Why are the changes needed?

Solving the problem mentioned in #378 (comment).

In current codebase, when encountering huge partition, if single buffer flush is enabled, the normal partition data will be flush to HDFS(I don't hope so, because the local disk is free and the flushing speed is faster than HDFS). But if disable single flush buffer, the huge partition event before marking as huge partition may be big, which cause the slow flushing and then cause requiring allocated buffer failed.

Based on above problems, this PR is to make single event carrying with 100 mb flushed into HDFS or local file leveraging the conf of rss.server.multistorage.cold.storage.preferred.factor

Does this PR introduce any user-facing change?

Yes. Doc will be updated later.

How was this patch tested?

  1. UTs

@codecov-commenter
Copy link

codecov-commenter commented Feb 17, 2023

Codecov Report

Merging #621 (b3e217f) into master (e20fb62) will increase coverage by 2.11%.
The diff coverage is 98.07%.

@@             Coverage Diff              @@
##             master     #621      +/-   ##
============================================
+ Coverage     60.90%   63.02%   +2.11%     
- Complexity     1799     1810      +11     
============================================
  Files           214      204      -10     
  Lines         12381    10528    -1853     
  Branches       1042     1056      +14     
============================================
- Hits           7541     6635     -906     
+ Misses         4437     3547     -890     
+ Partials        403      346      -57     
Impacted Files Coverage Δ
...r/storage/multi/DefaultStorageManagerSelector.java 93.75% <93.75%> (ø)
...g/apache/uniffle/server/ShuffleDataFlushEvent.java 84.90% <100.00%> (+1.23%) ⬆️
...a/org/apache/uniffle/server/ShuffleServerConf.java 99.35% <100.00%> (+0.03%) ⬆️
.../org/apache/uniffle/server/ShuffleTaskManager.java 76.76% <100.00%> (+0.13%) ⬆️
...he/uniffle/server/buffer/ShuffleBufferManager.java 83.50% <100.00%> (+0.40%) ⬆️
...he/uniffle/server/storage/MultiStorageManager.java 59.61% <100.00%> (-1.10%) ⬇️
...e/server/storage/multi/StorageManagerSelector.java 100.00% <100.00%> (ø)
...a/org/apache/uniffle/server/RegisterHeartBeat.java 43.85% <0.00%> (-43.86%) ⬇️
.../java/org/apache/uniffle/server/ShuffleServer.java 62.06% <0.00%> (-2.32%) ⬇️
... and 24 more

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

@zuston zuston changed the title [#378] feat: introduce storage manager selector to support more selector strategy [#378] feat: introduce storage manager selector Feb 17, 2023
@zuston
Copy link
Member Author

zuston commented Feb 17, 2023

PTAL @jerqi @xianjingfeng . This is a remaining improvement for #378

@@ -671,7 +671,7 @@ public ShuffleTaskInfo getShuffleTaskInfo(String appId) {

private void triggerFlush() {
synchronized (this.shuffleBufferManager) {
this.shuffleBufferManager.flushIfNecessary();
this.shuffleBufferManager.flushIfNecessary(this::getPartitionDataSize);
Copy link
Contributor

Choose a reason for hiding this comment

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

I can get you intention.. But I believe passing the getPartitionDataSize method ref around is adding a lot of interface change and it's adding maintenance overhead.

How about make ShuffleTaskManager a private field of ShuffleBufferManager? Then all this interface change is unnecessary.

Copy link
Member Author

Choose a reason for hiding this comment

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

Got it

Copy link
Member Author

Choose a reason for hiding this comment

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

The shuffleTaskManager is initialized after creating the instance of ShuffleBufferManager, emm, we'd better to pass the ShuffleServer into ShuffleBufferManager. But it still looks unclear, especially for the dependency of different managers. I have no idea on this.

Copy link
Contributor

Choose a reason for hiding this comment

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

The shuffleTaskManager is initialized after creating the instance of ShuffleBufferManager, emm, we'd better to pass the ShuffleServer into ShuffleBufferManager. But it still looks unclear, especially for the dependency of different managers. I have no idea on this.

For impl purpose:
ShuffleBufferManager should expose a setTaskManager method, and in the construction of ShuffleTaskManager, it could call shuffleBufferManager.setTaskManager(this)

But I do think the BufferManager, 'TaskManagerandFlushManager`'s dependency is unclear and should be refactored in later PRs.

@advancedxy
Copy link
Contributor

Hi @zuston could you rebase/merge you branch with the latest master branch?

I'd like to check whether the CI workflow for operator is worked as expected or not.

@zuston
Copy link
Member Author

zuston commented Feb 21, 2023

I'd like to check whether the CI workflow for operator is worked as expected or not.

Done

@advancedxy
Copy link
Contributor

I'd like to check whether the CI workflow for operator is worked as expected or not.

Done

Thanks. The ci workflow of operator is skipped as expected.

@@ -76,7 +78,12 @@ public class ShuffleBufferManager {
// appId -> shuffleId -> shuffle size in buffer
protected Map<String, Map<Integer, AtomicLong>> shuffleSizeMap = Maps.newConcurrentMap();

public ShuffleBufferManager(ShuffleServerConf conf, ShuffleFlushManager shuffleFlushManager) {
public ShuffleBufferManager(ShuffleServerConf conf, ShuffleServer shuffleServer) {
this(conf, shuffleServer.getShuffleFlushManager(), shuffleServer.getShuffleTaskManager());
Copy link
Contributor

Choose a reason for hiding this comment

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

Copy link
Member Author

Choose a reason for hiding this comment

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

Yes. Fixed

Copy link
Contributor

@advancedxy advancedxy left a comment

Choose a reason for hiding this comment

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

LGTM, except one minor comment

.enumType(StorageManagerSelector.ColdStoragePreferredFactor.class)
.defaultValue(StorageManagerSelector.ColdStoragePreferredFactor.HUGE_EVENT)
.withDescription("The cold storage preferred factor for multiple storage manager. Only the value is "
+ StorageManagerSelector.ColdStoragePreferredFactor.HUGE_EVENT + ", the conf of "
Copy link
Contributor

Choose a reason for hiding this comment

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

This description is a bit hard to read. Would you mind do some rewording here.

I believe we should also update the doc about this new configuration.

Copy link
Member Author

Choose a reason for hiding this comment

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

Got it.

@@ -263,6 +264,15 @@ public class ShuffleServerConf extends RssBaseConf {
.defaultValue(64L * 1024L * 1024L)
.withDescription("For multistorage, the event size exceed this value, flush data to cold storage");

public static final ConfigOption<StorageManagerSelector.ColdStoragePreferredFactor>
MULTISTORAGE_SELECTOR_COLD_STORAGE_PREFER = ConfigOptions
.key("rss.server.multistorage.cold.storage.preferred.factor")
Copy link
Contributor

Choose a reason for hiding this comment

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

Could you give this config option a better name? It is more like a strategy in our system.

Copy link
Member Author

Choose a reason for hiding this comment

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

WDYT?

Copy link
Contributor

Choose a reason for hiding this comment

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

rss.server.multistorage.selector.strategy

Copy link
Member Author

Choose a reason for hiding this comment

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

This has been discussed in above conversation. #621 (comment). If you prefer, I will refactor it.

Copy link
Contributor

Choose a reason for hiding this comment

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

It's better to refactor ... it's more clear.

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

@zuston zuston merged commit 1fbdfe5 into apache:master Mar 1, 2023
advancedxy pushed a commit to advancedxy/incubator-uniffle that referenced this pull request Mar 21, 2023
…ategy (apache#621)

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

1. Introduce storage manager selector to support more selector strategy for `MultiStorageManager`
2. Introduce the conf of `rss.server.multistorage.manager.selector.class` to support different flush strategy, like I hope huge partition directly flushed to HDFS and normal partition could be flushed to DISK when single buffer flush is enabled. 

### Why are the changes needed?
Solving the problem mentioned in apache#378 (comment). 

In current codebase, when encountering huge partition, if single buffer flush is enabled, the normal partition data will be flush to HDFS(I don't hope so, because the local disk is free and the flushing speed is faster than HDFS). But if disable single flush buffer, the huge partition event before marking as huge partition may be big, which cause the slow flushing and then cause requiring allocated buffer failed.

Based on above problems, this PR is to make single event carrying with 100 mb flushed into HDFS or local file leveraging the conf of `rss.server.multistorage.manager.selector.class`

### Does this PR introduce _any_ user-facing change?
Yes. Doc will be updated later.

### How was this patch tested?
1. UTs
xianjingfeng pushed a commit to xianjingfeng/incubator-uniffle that referenced this pull request Apr 5, 2023
…ategy (apache#621)

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

1. Introduce storage manager selector to support more selector strategy for `MultiStorageManager`
2. Introduce the conf of `rss.server.multistorage.manager.selector.class` to support different flush strategy, like I hope huge partition directly flushed to HDFS and normal partition could be flushed to DISK when single buffer flush is enabled. 

### Why are the changes needed?
Solving the problem mentioned in apache#378 (comment). 

In current codebase, when encountering huge partition, if single buffer flush is enabled, the normal partition data will be flush to HDFS(I don't hope so, because the local disk is free and the flushing speed is faster than HDFS). But if disable single flush buffer, the huge partition event before marking as huge partition may be big, which cause the slow flushing and then cause requiring allocated buffer failed.

Based on above problems, this PR is to make single event carrying with 100 mb flushed into HDFS or local file leveraging the conf of `rss.server.multistorage.manager.selector.class`

### Does this PR introduce _any_ user-facing change?
Yes. Doc will be updated later.

### How was this patch tested?
1. UTs
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.

None yet

5 participants