otterc commented on a change in pull request #30433:
URL: https://github.com/apache/spark/pull/30433#discussion_r533694433
##########
File path:
common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RemoteBlockPushResolver.java
##########
@@ -827,13 +833,16 @@ void resetChunkTracker() {
void updateChunkInfo(long chunkOffset, int mapIndex) throws IOException {
long idxStartPos = -1;
try {
- // update the chunk tracker to meta file before index file
- writeChunkTracker(mapIndex);
idxStartPos = indexFile.getFilePointer();
logger.trace("{} shuffleId {} reduceId {} updated index current {}
updated {}",
appShuffleId.appId, appShuffleId.shuffleId, reduceId,
this.lastChunkOffset,
chunkOffset);
- indexFile.writeLong(chunkOffset);
+ indexFile.write(Longs.toByteArray(chunkOffset));
+ // Chunk bitmap should be written to the meta file after the index
file because if there are
+ // any exceptions during writing the offset to the index file, meta
file should not be
+ // updated. If the update to the index file is successful but the
update to meta file isn't
+ // then the index file position is reset in the catch clause.
+ writeChunkTracker(mapIndex);
Review comment:
Just want to reiterate the change I am working on:
- Since seek is not recoverable, we let the clients know to stop pushing for
a particular shuffle partition.
- Have a threshold on number of IOExceptions from writes and when this
threshold is reached for a single partition, inform the client to stop pushing
and stop merging this partition.
- When the update to metadata fails, not propagate this exception back to
client so that they push the block again. The size of the current chunk may
grow but with (2) in place it will still be of a manageable size.
These changes will impact https://github.com/apache/spark/pull/30312. Right
now the client stops pushing for the entire shuffle not a particular shuffle
partition. The support needs to be extended to stop pushing for a particular
shuffle partition.
----------------------------------------------------------------
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.
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]