otterc opened a new pull request #33477:
URL: https://github.com/apache/spark/pull/33477


   ### What changes were proposed in this pull request?
   Once a job in a Spark application completes, the shuffle files are cleaned 
up by the executors. This is because the driver instructs the executors to 
clean up the shuffle. In this case, the push of the shuffle data by the 
executors can throw `FileNotFoundException`s because the shuffle files are 
deleted. When this exception is thrown from the `shuffle-block-push-thread`, it 
causes the executor to exit. Both the `shuffle-block-push` threads and the 
netty event-loops will encounter `FileNotFoundException`s in this case.  The 
fix here stops these threads from pushing more blocks when they encounter 
`FileNotFoundException`. When the exception is from the 
`shuffle-block-push-thread`, it will get handled and logged as warning instead 
of failing the executor.
   
   ### Why are the changes needed?
   This fixes the bug which causes executor to exits when they are instructed 
to clean up shuffle data by the driver
   Below is the stacktrace of this exception:
   ```
   21/06/17 16:03:57 ERROR util.SparkUncaughtExceptionHandler: Uncaught 
exception in thread Thread[block-push-thread-1,5,main]
   java.lang.Error: java.io.IOException: Error in opening 
FileSegmentManagedBuffer
   
   
{file=********/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data,
 offset=10640, length=190}
   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1155)
   at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
   at java.lang.Thread.run(Thread.java:748)
   Caused by: java.io.IOException: Error in opening 
FileSegmentManagedBuffer\{file=*******/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data,
 offset=10640, length=190}
   
   at 
org.apache.spark.network.buffer.FileSegmentManagedBuffer.nioByteBuffer(FileSegmentManagedBuffer.java:89)
   at 
org.apache.spark.shuffle.ShuffleWriter.sliceReqBufferIntoBlockBuffers(ShuffleWriter.scala:294)
   at 
org.apache.spark.shuffle.ShuffleWriter.org$apache$spark$shuffle$ShuffleWriter$$sendRequest(ShuffleWriter.scala:270)
   at 
org.apache.spark.shuffle.ShuffleWriter.org$apache$spark$shuffle$ShuffleWriter$$pushUpToMax(ShuffleWriter.scala:191)
   at 
org.apache.spark.shuffle.ShuffleWriter$$anon$2$$anon$4.run(ShuffleWriter.scala:244)
   at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
   ... 2 more
   Caused by: java.io.FileNotFoundException: 
******/application_1619720975011_11057757/blockmgr-560cb4cf-9918-4ea7-a007-a16c5e3a35fe/0a/shuffle_1_690_0.data
 (No such file or directory)
   at java.io.RandomAccessFile.open0(Native Method)
   at java.io.RandomAccessFile.open(RandomAccessFile.java:316)
   at java.io.RandomAccessFile.<init>(RandomAccessFile.java:243)
   at 
org.apache.spark.network.buffer.FileSegmentManagedBuffer.nioByteBuffer(FileSegmentManagedBuffer.java:62)
   ``` 
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Added a unit to verify no more data is pushed when `FileNotFoundException` 
is encountered. Have also verified in our environment.
   


-- 
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]

Reply via email to