Skip to content

Conversation

@yabola
Copy link
Contributor

@yabola yabola commented Nov 8, 2022

No description provided.

@github-actions github-actions bot added the CORE label Nov 8, 2022
@mridulm
Copy link
Contributor

mridulm commented Nov 8, 2022

+CC @otterc

@yabola
Copy link
Contributor Author

yabola commented Nov 8, 2022

One things that I know need to be addressed are:
Some merge data infos are not saved on the driver because they are too small ( controlled by spark.shuffle.push.minShuffleSizeToWait)
please see https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2295

@yabola
Copy link
Contributor Author

yabola commented Nov 9, 2022

I am wondering whether the driver needs to pass the merged reduceId to the external shuffle service (but now the driver cannot fully record merged info), or the shuffle service records the merged reduceIds, and driver only need to pass the shuffleId and other information

@yabola
Copy link
Contributor Author

yabola commented Nov 10, 2022

I am wondering whether the driver needs to pass the merged reduceId to the external shuffle service (but now the driver cannot fully record merged info), or the shuffle service records the merged reduceIds, and driver only need to pass the shuffleId and other information

I decided to change the driver to not send reduceIds ( only send shuffleId and appId), because only the shuffle service finally understands which shuffle data is stored, no matter how the driver processes or processes the message

@mridulm
Copy link
Contributor

mridulm commented Nov 10, 2022

This is related quite a lot to #37922 by @wankunde
That PR seems to be having build issues, and so has not made progress.

@AmplabJenkins
Copy link

Can one of the admins verify this patch?

@yabola
Copy link
Contributor Author

yabola commented Nov 11, 2022

@mridulm Yes...These two issues are the similar. @wankunde Can I continue editing my PR in this Issue?

@yabola
Copy link
Contributor Author

yabola commented Nov 16, 2022

@mridulm as your comment #37922 (comment) , I want to Improve this part of the deletion logic

Copy link
Contributor Author

@yabola yabola left a comment

Choose a reason for hiding this comment

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

~~@mridulm @wankunde @otterc I'm not sure if I missed any logic, please help review my code , thanks~ I will improve my code style later.
Now I don't change my code in BlockManagerMasterEndpoint as #37922 do . Can it be split into two PRs, I implement the code of the shuffle service part first and @wankunde finish the rest part since he has done?~~

Copy link
Contributor Author

Choose a reason for hiding this comment

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

Unified push based shuffle identification variables here, which will be used in yarn external shuffle service and spark core module.

Copy link
Contributor Author

@yabola yabola Nov 16, 2022

Choose a reason for hiding this comment

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

I checked the appAttemptShuffleMergeId in the code before.
I think if we want to delete partitions merged data, then we should delete the corresponding ShuffleMergeId in DB (Otherwise, inconsistency will occur when restoring shuffle info from db)

Copy link
Contributor Author

@yabola yabola Nov 16, 2022

Choose a reason for hiding this comment

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

And I think when deleting partitions, we shouldn't store shuffleMergeID in DB anymore

Copy link
Contributor Author

Choose a reason for hiding this comment

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

The previous code is difficult to understand because of the scala syntax. For example, mapOutputTracker.shuffleStatuses.get(shuffleId).**foreach** The foreach here is not actually an iterator.
externalBlockStoreClient.map the externalBlockStoreClient is not actually an iterator.

I didn't change the code logic, just changed the style

Copy link
Contributor

Choose a reason for hiding this comment

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

What if the shuffle statuses are not exists ?

Copy link
Contributor Author

@yabola yabola Nov 24, 2022

Choose a reason for hiding this comment

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

What if the shuffle statuses are not exists ?

I think it will not, please see case match codes

@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 though.

@mridulm
Copy link
Contributor

mridulm commented Nov 21, 2022

One things that I know need to be addressed are:
Some merge data infos are not saved on the driver because they are too small ( controlled by spark.shuffle.push.minShuffleSizeToWait)
please see https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2295

In this case, we should fire a remove immediately - we are not going to use it for this app anyway ...

@mridulm
Copy link
Contributor

mridulm commented Nov 21, 2022

@yabola, there is quite a lot of nontrivial overlap between this PR and @wankunde's PR is trying doing.
Would be great if you both can coordinate on this - I would love to get this functionality merged before we start getting closer to code freeze for 3.4

@yabola
Copy link
Contributor Author

yabola commented Nov 21, 2022

@mridulm
I will speed up to finish the unfinished parts of the previous PR together in this PR.
From your comments in the previous PR #37922 (comment) and #37922 (comment) .
I had addressed these in my PR , please help review if it is suitable: codes to remove shuffle merge
and codes to clean shuffle files

Yes , there are nontrivial overlap between #37922 , I can cherry pick some codes and fix the comments in this PR.

@yabola
Copy link
Contributor Author

yabola commented Nov 21, 2022

@mridulm @wankunde @otterc I'm not sure if I missed any logic, please help review my code , thanks~ I will improve my code style later. Now I don't change my code in BlockManagerMasterEndpoint as #37922 do . Can it be split into two PRs, I implement the code of the shuffle service part first and @wankunde finish the rest part since he has done?

Some thoughts on these changes I wrote here

@mridulm
Copy link
Contributor

mridulm commented Nov 21, 2022

Thanks for taking over the PR @yabola !
I am heading out on a vacation soon, not sure if @otterc will have bandwidth to take a look in meantime.
I will definitely circle back to this once you are done, and I am back.

@mridulm
Copy link
Contributor

mridulm commented Nov 21, 2022

To add, @wankunde's PR is very close to being done.
One approach would be:
a) We get the protocol changes and the immediate impl from that PR merged (once pending comments are addressed) once you take over the PR and complete that.
b) We can follow it up with other changes to make it more robust. For example, this issue you identified. There might be others as well

Thoughts ?

@yabola
Copy link
Contributor Author

yabola commented Nov 21, 2022

One things that I know need to be addressed are: Some merge data infos are not saved on the driver because they are too small ( controlled by spark.shuffle.push.minShuffleSizeToWait) please see https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala#L2295

@mridulm sorry, in my previous implementation, I needed to pass the reduceid to the external shuffle service, but I found a problem, the driver cannot record the complete merged reduceId (see my comment for the reason)...
But I had changed my implementation, so it may not be a problem (we can save merged reduceIds in shuffle service, please see codes.
But it is also better if we can clean up these useless merged data early.

Copy link
Contributor

Choose a reason for hiding this comment

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

What if the shuffle statuses are not exists ?

mergeStatuses = new MergeStatuses(msg.shuffleId, msg.shuffleMergeId,
bitmaps.toArray(new RoaringBitmap[bitmaps.size()]), Ints.toArray(reduceIds),
Longs.toArray(sizes));
appShuffleInfo.shuffles.get(msg.shuffleId).setFinalizedPartitions(Ints.toArray(reduceIds));
Copy link
Contributor

Choose a reason for hiding this comment

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

The FinalizedPartitions will be empty after the Shuffle service restart which will cause the merged shuffle files leak.

Copy link
Contributor Author

@yabola yabola Nov 23, 2022

Choose a reason for hiding this comment

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

Thanks for your review @wankunde !
Yes, if we need to solve this situation completely, we need to store finalized partitions in db.
But on the other hand, when Application removed, all the merged data will be cleaned up finally.
I'm not sure if we can just ignore this case to simplify the logic since it will finally be cleaned up.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

What if the shuffle statuses are not exists ?
I think it will not, please see case match codes

Comment on lines 541 to 553
try {
File metaFile =
shuffleInfo.getMergedShuffleMetaFile(shuffleId, mergeId, partition);
File indexFile = new File(
shuffleInfo.getMergedShuffleIndexFilePath(shuffleId, mergeId, partition));
File dataFile =
shuffleInfo.getMergedShuffleDataFile(shuffleId, mergeId, partition);
metaFile.delete();
indexFile.delete();
dataFile.delete();
} catch (Exception e) {
logger.error("Error delete shuffle files for {}", shuffleMergeId, e);
}
Copy link
Contributor

Choose a reason for hiding this comment

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

Just like closeAllFilesAndDeleteIfNeeded method, can we continue delete the other files if one delete() failed?

Copy link
Contributor Author

@yabola yabola Nov 23, 2022

Choose a reason for hiding this comment

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

done

@github-actions github-actions bot added the YARN label Nov 23, 2022
val removeShuffleFromExecutorsFutures = blockManagerInfo.values.map { bm =>
bm.storageEndpoint.ask[Boolean](removeMsg).recover {
// use false as default value means no shuffle data were removed
handleBlockRemovalFailure("shuffle", shuffleId.toString, bm.blockManagerId, false)
Copy link
Contributor Author

@yabola yabola Nov 24, 2022

Choose a reason for hiding this comment

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

I just move removeShuffleFromExecutorsFutures to the last.
It needs to be invoked at last to avoid cleaning up shuffleStatuses in mapOutputTracker too early. Otherwise mapOutputTracker.shuffleStatuses.get(shuffleId) may be none sometimes
Please refer to unregisterShuffle codes

mergeManager2, mergeManager2DB) == 1)
assert(ShuffleTestAccessor.getOutdatedFinalizedShuffleCountDuringDBReload(
mergeManager2, mergeManager2DB) == 2)
mergeManager2, mergeManager2DB) == 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.

This is as expected, because we delete the current merge partitions and the current outdated merge status in db (not cleaned before this PR)
Please refer to codes

@yabola yabola changed the title [WIP][SPARK-38005][core] Support cleaning up merged shuffle files and state from external shuffle service [SPARK-38005][core] Support cleaning up merged shuffle files and state from external shuffle service Nov 30, 2022
@yabola
Copy link
Contributor Author

yabola commented Dec 1, 2022

@mridulm If you are back and have time, please review my PR, I think the function is almost done. let me know if there is something inappropriate, I will modify it soon, thanks!

@mridulm
Copy link
Contributor

mridulm commented Dec 3, 2022

Apologies for the delay in getting to this @yabola - will try to get to this next week.
Thanks for your patience.

My recommandation would be to keep the change as close to @wankunde's PR as possible, and fix the pending issues there to expedite the reviews (since that change was already reviewed quite a lot). If there are additional functional gaps in that PR, we can do it in follow up PR's.

@yabola
Copy link
Contributor Author

yabola commented Dec 13, 2022

Apologies for the delay in getting to this @yabola - will try to get to this next week. Thanks for your patience.

My recommandation would be to keep the change as close to @wankunde's PR as possible, and fix the pending issues there to expedite the reviews (since that change was already reviewed quite a lot). If there are additional functional gaps in that PR, we can do it in follow up PR's.

@wankunde according to comments, could you fix the remaining comments in your PR?

@wankunde
Copy link
Contributor

Hi, @yabola @mridulm , I will update SPARK-40480 this weekend.

@yabola
Copy link
Contributor Author

yabola commented Dec 23, 2022

close on #37922

@yabola yabola closed this Dec 23, 2022
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Projects

None yet

Development

Successfully merging this pull request may close these issues.

4 participants