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

[SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished #37922

Closed
wants to merge 27 commits into from

Conversation

wankunde
Copy link
Contributor

@wankunde wankunde commented Sep 18, 2022

What changes were proposed in this pull request?

Cleanup merged shuffle data files after query finished.

The main changes are:

  • After push merge service received FinalizeShuffleMerge RPC from driver, it will mark the MergePartitionsInfo as finilized instead of remove it.
  • Delete the shuffle merge files if partition.mapTracker is empty.
  • When spark driver begin to cleanup shuffle, if push-based shuffle is enabled, spark driver will send RemoveShuffleMerge RPC to all merger locations to delete all merge data files.

Why are the changes needed?

There will be too many merged shuffle data files for long running spark applications.

Does this PR introduce any user-facing change?

No

How was this patch tested?

Local cluster test.

@github-actions github-actions bot added the CORE label Sep 18, 2022
@mridulm
Copy link
Contributor

mridulm commented Sep 18, 2022

+CC @akpatnam25, @otterc - since you were planning to work on this already.

Copy link
Contributor

@mridulm mridulm 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 working on this @wankunde !

@@ -393,6 +393,20 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) {
}
}

@Override
public void removeShuffleMerge(String appId, int shuffleId) {
Copy link
Contributor

Choose a reason for hiding this comment

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

When adding support for shuffleMergeId, follow the same pattern as finalizeShuffleMerge - there are a few corner cases here, and that method handles them.

@HyukjinKwon HyukjinKwon changed the title [SPARK-40480][SHUFFLE]Remove push-based shuffle data after query finished [SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished Sep 19, 2022
@wankunde
Copy link
Contributor Author

Hi, @mridulm Thanks for your review.

In my opinion, I do not think we should add ShuffleMergeId in RemoveShuffleMerge.
The push-based shuffle service will only keep the latest shuffleMergeId data for each shuffle, and RemoveShuffleMerge will cleanup its data.
Just like cleaning up normal shuffle data written by mapTasks, these methods only need shuffleId: def removeShuffle(shuffleId: Int)

What do you think?

@mridulm
Copy link
Contributor

mridulm commented Sep 19, 2022

We should decouple current implementation details when making protocol changes, and make it extensible for future evolution.

In this case though, it is much more straightforward - there is an existing usecase which requires shuffle merge id.
When retrying an indeterminate stage, we should cleanup merged shuffle data for previous stage attempt (in submitMissingTasks, before unregisterAllMapAndMergeOutput) - and given the potential race conditions there, we dont want RemoveShuffleMerge to clean up for the next attempt (when we add support for this).

This specific change can be done in a follow up PR though - I want to get the basic mechanics working in this PR, and ensure the cleanup usecase is handled - before looking at further enhancements.

@wankunde
Copy link
Contributor Author

wankunde commented Sep 19, 2022

We should decouple current implementation details when making protocol changes, and make it extensible for future evolution.

In this case though, it is much more straightforward - there is an existing usecase which requires shuffle merge id. When retrying an indeterminate stage, we should cleanup merged shuffle data for previous stage attempt (in submitMissingTasks, before unregisterAllMapAndMergeOutput) - and given the potential race conditions there, we dont want RemoveShuffleMerge to clean up for the next attempt (when we add support for this).

This specific change can be done in a follow up PR though - I want to get the basic mechanics working in this PR, and ensure the cleanup usecase is handled - before looking at further enhancements.

The push-based shuffle service will auto clean up the old shuffle merge data, and the following stage will read the new stage's output, so we don't need send RemoveShuffleMerge RPC for a new ShuffleMerge?
The only scenario I can think of now where a cleanup RPC needs is the spark job completes. Could we think of other scenarios?

@wankunde
Copy link
Contributor Author

retest this please

@wankunde wankunde changed the title [SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished [WIP][SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished Sep 19, 2022
@mridulm
Copy link
Contributor

mridulm commented Sep 19, 2022

The push-based shuffle service will auto clean up the old shuffle merge data

Consider the case I mentioned above - stage retry for an INDETERMINATE stage.
We cleanup previous attempts only if it happens to use the same merger in new attempt as well.

The previous attempt's mergers are not reused for the next attempt - and so the previous mergers will continue to hold stale data without cleaning them up - until application terminates.
Note - any merger which happens to be reused in new stage attempt will cleanup - I am referring to those which are not used: old attempt mergerLocs -- new attempts mergerLocs.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@wankunde
Copy link
Contributor Author

Thanks @mridulm , I will the the shuffleMergeId into the protocol later.
This PR also has some other issues, I will fix it soon.

} else if (msgObj instanceof RemoveShuffleMerge) {
RemoveShuffleMerge msg = (RemoveShuffleMerge) msgObj;
checkAuth(client, msg.appId);
logger.info("Remove shuffle merge data for application %s shuffle %d",

Choose a reason for hiding this comment

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

Nit: remove --> removing

@otterc
Copy link
Contributor

otterc commented Oct 6, 2022

Gentle ping @wankunde. Do you think you can update the PR soon? Please let us know if you need any help.

@wankunde
Copy link
Contributor Author

wankunde commented Oct 9, 2022

Gentle ping @wankunde. Do you think you can update the PR soon? Please let us know if you need any help.

Sorry for the late reply, I will update this PR today.

@wankunde wankunde changed the title [WIP][SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished [SPARK-40480][SHUFFLE] Remove push-based shuffle data after query finished Oct 9, 2022
@wankunde
Copy link
Contributor Author

wankunde commented Oct 9, 2022

Hi, @otterc @mridulm If the spark application stops unexpectedly, there will be some leaked merge files. In our production cluster, yarn nodeManager will cleanup these leaked files if the disk is not full, but not if the disk is full.
Could we change the blockHandler.applicationRemoved(appId, false /* clean up local dirs */) to blockHandler.applicationRemoved(appId, true) in YarnShuffleService to clean up them ?

blockHandler.applicationRemoved(appId, false /* clean up local dirs */);

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Added a few comments.
While going over this change, I think I might have hit upon an existing bug as well - so thanks for raising this PR for two reasons now !

submitCleanupTask(() ->
closeAndDeleteOutdatedPartitions(
appAttemptShuffleMergeId, mergePartitionsInfo.shuffleMergePartitions));
closeAndDeleteOutdatedPartitions(appAttemptShuffleMergeId, shuffleMergePartitions));
} else {
Copy link
Contributor

Choose a reason for hiding this comment

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

revert 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.

Now in finalizeShuffleMerge() method we will update the AppShuffleMergePartitionsInfo to an empty object, so we can not cleanup the leaked files?

Copy link
Contributor

Choose a reason for hiding this comment

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

See my note above on how to handle this.

// Update the DB for the finalized shuffle
writeAppAttemptShuffleMergeInfoToDB(appAttemptShuffleMergeId);
// Even when the mergePartitionsInfo is null, we mark the shuffle as finalized but the results
// sent to the driver will be empty. This can happen when the service didn't receive any
// blocks for the shuffle yet and the driver didn't wait for enough time to finalize the
// shuffle.
return new AppShuffleMergePartitionsInfo(msg.shuffleMergeId, true);
return mergePartitionsInfo;
Copy link
Contributor

@mridulm mridulm Oct 9, 2022

Choose a reason for hiding this comment

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

revert changes to this method ? (assuming the changes I sketched above are fine)

@@ -702,7 +722,8 @@ public MergeStatuses finalizeShuffleMerge(FinalizeShuffleMerge msg) {
"finalizing shuffle partition {}", msg.appId, msg.appAttemptId, msg.shuffleId,
msg.shuffleMergeId, partition.reduceId);
} finally {
partition.closeAllFilesAndDeleteIfNeeded(false);
Boolean deleteFile = partition.mapTracker.getCardinality() == 0;
partition.closeAllFilesAndDeleteIfNeeded(deleteFile);
Copy link
Contributor

Choose a reason for hiding this comment

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

Revert ?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Won't return the reduceId which partition.mapTracker.getCardinality() == 0, so we can closeAllFilesAndDeleteIfNeeded now.

Copy link
Contributor

Choose a reason for hiding this comment

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

The actual deletion will be handled by either application termination, or during shuffle cleanup (this PR).
Making this change here might have other issues we will need to think through.

If we do want to do this, can you file a follow up jira and we can investigate it there ?
I want to limit this PR specifically to changes required for this feature.

Copy link
Contributor

Choose a reason for hiding this comment

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

Please revert this for the time being, and revisit it as a follow up as required.
There are a number of edge cases which needs to be handled in this important PR, and I dont want to add to them with optional changes with minor utility.

@@ -1410,26 +1431,27 @@ public String toString() {
* required for the shuffles of indeterminate stages.
*/
public static class AppShuffleMergePartitionsInfo {
Copy link
Contributor

@mridulm mridulm Oct 9, 2022

Choose a reason for hiding this comment

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

Revert changes to this class ? (assuming the changes I sketched above are fine)

@mridulm
Copy link
Contributor

mridulm commented Nov 17, 2022

I will try to get to this later this week, do let me know if you are still working on it/have pending comments to address. Thanks

@wankunde
Copy link
Contributor Author

Hi, @mridulm , I've been working on some other issues recently. If @yabola can do all or part of this task in #38560, please go ahead.

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Took a quick look, and some of the comments from the previous review are not yet addressed.
Do let me know if you have queries or comments on this @wankunde .

Thanks

/**
* Remove shuffle merge data files.
*
* @param removeShuffleMerge Remove shuffle merge RPC
Copy link
Contributor

Choose a reason for hiding this comment

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

Suggested change
* @param removeShuffleMerge Remove shuffle merge RPC
* @param removeShuffleMerge contains shuffle details (appId, shuffleId, etc) to uniquely identify a shuffle to be removed


@Override
public void removeShuffleMerge(RemoveShuffleMerge removeShuffleMerge) {
throw new UnsupportedOperationException("Cannot handle shuffle block merge");
Copy link
Contributor

Choose a reason for hiding this comment

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

nit:

Suggested change
throw new UnsupportedOperationException("Cannot handle shuffle block merge");
throw new UnsupportedOperationException("Cannot handle merged shuffle remove");

}

appShuffleInfo.shuffles.compute(shuffleId, (shuffleIdKey, partitionsInfo) -> {
if (null != partitionsInfo) {
Copy link
Contributor

Choose a reason for hiding this comment

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

The validation related to shuffleMergeId, from finalizeShuffleMerge, is applicable here depending on shuffleMergeId.
I have given the details here - please refer to it.

@wankunde
Copy link
Contributor Author

Hi, @mridulm @yabola I have updated this PR, could you help to this PR again? Thanks.

@wankunde
Copy link
Contributor Author

wankunde commented Jan 1, 2023

@mridulm I rebase the code, could you help to review this PR again? Thanks

@mridulm
Copy link
Contributor

mridulm commented Jan 1, 2023

I just merged #39316 to unblock this PR, can you update to latest please ? Thx

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Just couple of minor comments - rest looks fine.
Please ping me when done, I want to try to get this in for 3.4, thanks !

File dataFile =
appShuffleInfo.getMergedShuffleDataFile(shuffleId, shuffleMergeId, reduceId);
dataFile.delete();
} catch (Exception e) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we fix this @wankunde ?
Essentially, the changes are:

a) There is no exception thrown in this block - we dont need the try/catch.
b) When the delete fails, we dont need to do the warn (here and below) : this can happen if application exit is racing against remove shuffle - the info at the end of the method will suffice.

assertTrue(shuffleInfo.getMergedShuffleMetaFile(3, 1, 0).exists());
pushResolver.finalizeShuffleMerge(new FinalizeShuffleMerge(testApp, NO_ATTEMPT_ID, 3, 2));
closed.tryAcquire(10, TimeUnit.SECONDS);
assertFalse(shuffleInfo.getMergedShuffleMetaFile(3, 1, 0).exists());
Copy link
Contributor

Choose a reason for hiding this comment

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

Can we also add tests for the additional cases we have ?

  • removeShuffleMerge for DELETE_ALL_MERGED_SHUFFLE - we can just do what we have for streamCallback1 (Case 1) above, but use DELETE_ALL_MERGED_SHUFFLE for merge id.
  • additional cases here to make sure of the negative cases as well ? namely:
    • msg.shuffleMergeId < mergePartitionsInfo.shuffleMergeId
    • delete for a merge id which does not exist.

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Looks good to me, thanks for the changes @wankunde !

+CC @otterc, @Ngone51

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Can we add a test to check whether the message goes to the expected mergers from driver for RemoteShuffleMerge when a shuffle is getting cleaned up ?
While trying to test the proposal here, I realized we are not testing this behavior - even replacing it with Nil always causes all tests to pass !

sc.cleaner.get.doCleanupShuffle(SHUFFLE_ID, blocking = true)
assert(foundHosts.asScala == mergerLocs.map(_.host).toSet)
}
}
Copy link
Contributor

Choose a reason for hiding this comment

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

There is a bug in this test - sorry about that.
Please take a look here: 26b2d18

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Backport the code, much thanks

Copy link
Contributor

@otterc otterc left a comment

Choose a reason for hiding this comment

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

Just have some minor comments

msg.appId, msg.appAttemptId, msg.shuffleId, mergePartitionsInfo.shuffleMergeId);
AppAttemptShuffleMergeId appAttemptShuffleMergeId = new AppAttemptShuffleMergeId(
msg.appId, msg.appAttemptId, msg.shuffleId, shuffleMergeId);
if(deleteCurrentMergedShuffle || shuffleMergeId > mergePartitionsInfo.shuffleMergeId) {
Copy link
Contributor

Choose a reason for hiding this comment

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

Nit: missing a space after if

deleteMergedFiles(currentAppAttemptShuffleMergeId, appShuffleInfo,
mergePartitionsInfo.getReduceIds(), false));
}
} else if(shuffleMergeId < mergePartitionsInfo.shuffleMergeId) {
Copy link
Contributor

Choose a reason for hiding this comment

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

nit: space missing after if. Also the condition here is not necessary. It will always be true (IDE shows that as well).

wankunde and others added 2 commits January 13, 2023 11:40
…shuffle/RemoteBlockPushResolver.java

Co-authored-by: otterc <singh.chandni@gmail.com>
@wankunde
Copy link
Contributor Author

@otterc Thanks for your review. Format the code.

Copy link
Contributor

@mridulm mridulm left a comment

Choose a reason for hiding this comment

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

Added one comment.
Rest looks good to me.

Will it be possible to change that and push an update soon @wankunde ? I will merge it once the tests pass successfully.

@mridulm
Copy link
Contributor

mridulm commented Jan 14, 2023

The tests are failing @wankunde, though I dont think it is due to your PR.
Can you please take a look ? And retrigger it if it is unrelated ? Thanks !

@asfgit asfgit closed this in e424499 Jan 14, 2023
@mridulm
Copy link
Contributor

mridulm commented Jan 14, 2023

Merged to master.
Thanks for fixing this @wankunde !
Thanks for the reviews @otterc, @akpatnam25, @yabola :-)

Copy link
Member

@dongjoon-hyun dongjoon-hyun left a comment

Choose a reason for hiding this comment

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

.

Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
Projects
None yet
8 participants