otterc commented on a change in pull request #33078:
URL: https://github.com/apache/spark/pull/33078#discussion_r660843950
##########
File path:
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -567,7 +598,8 @@ public void onData(String streamId, ByteBuffer buf) throws
IOException {
// memory, while still providing the necessary guarantee.
synchronized (partitionInfo) {
Map<Integer, AppShufflePartitionInfo> shufflePartitions =
- mergeManager.partitions.get(partitionInfo.appShuffleId);
+ mergeManager.appsShuffleInfo.get(partitionInfo.appId).partitions
Review comment:
There are couple of challenges that I see with doing it in finalization.
1. Right now cleanup is being done in the executor registration only when
the registration involves a new attempt. This is a low frequency event. So
event though cleanup right now is being done synchronously, it happens rarely
and increasing the time it takes for the first executor of the new attempt to
register doesn't have a big impact.
If we move this to finalization, then again we need to filter the
partitionInfos of the previous attempts and try closing them. If we increase
the time to finalize a shuffle, then it will have a negative impact. The driver
will move on to scheduling the reduce stage and not consider this reduce
partition merged.
2. For another attempt, there is still a possibility that finalizeMerge for
a shuffleId is not invoked. Maybe there weren't enough number of mergers
available in the new attempt and the stage wasn't push-enabled. In this case,
the partitionInfos of the old attempts will linger until the app finishes. An
app can be very long running so all the files that are not being use will say
open by then.
I think it's fine to do this in registration. We just need to make sure that
we are using the relevant `AppShuffleInfo` instance
--
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]