attilapiros commented on code in PR #36512:
URL: https://github.com/apache/spark/pull/36512#discussion_r878644162


##########
core/src/main/scala/org/apache/spark/storage/BlockManager.scala:
##########
@@ -933,46 +935,56 @@ private[spark] class BlockManager(
           })
           Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
         } else if (level.useDisk && diskStore.contains(blockId)) {
+          var diskData: BlockData = null
           try {
-            val diskData = diskStore.getBytes(blockId)
-            val iterToReturn: Iterator[Any] = {
-              if (level.deserialized) {
-                val diskValues = serializerManager.dataDeserializeStream(
-                  blockId,
-                  diskData.toInputStream())(info.classTag)
-                maybeCacheDiskValuesInMemory(info, blockId, level, diskValues)
-              } else {
-                val stream = maybeCacheDiskBytesInMemory(info, blockId, level, 
diskData)
-                  .map { _.toInputStream(dispose = false) }
-                  .getOrElse { diskData.toInputStream() }
-                serializerManager.dataDeserializeStream(blockId, 
stream)(info.classTag)
-              }
+            diskData = diskStore.getBytes(blockId)
+            val iterToReturn = if (level.deserialized) {
+              val diskValues = serializerManager.dataDeserializeStream(
+                blockId,
+                diskData.toInputStream())(info.classTag)
+              maybeCacheDiskValuesInMemory(info, blockId, level, diskValues)
+            } else {
+              val stream = maybeCacheDiskBytesInMemory(info, blockId, level, 
diskData)
+                .map { _.toInputStream(dispose = false) }
+                .getOrElse { diskData.toInputStream() }
+              serializerManager.dataDeserializeStream(blockId, 
stream)(info.classTag)
             }
             val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
               releaseLockAndDispose(blockId, diskData, taskContext)
             })
             Some(new BlockResult(ci, DataReadMethod.Disk, info.size))
           } catch {
-            case ex: KryoException if ex.getCause.isInstanceOf[IOException] =>
-              // We need to have detailed log message to catch environmental 
problems easily.
-              // Further details: 
https://issues.apache.org/jira/browse/SPARK-37710
-              processKryoException(ex, blockId)
-              throw ex
+            case t: Throwable =>
+              if (diskData != null) {
+                diskData.dispose()
+                diskData = null
+              }
+              releaseLock(blockId, taskContext)
+              if (isIORelatedException(t)) {
+                logInfo(extendMessageWithBlockDetails(t.getMessage, blockId))
+                // Remove the block so that its unavailability is reported to 
the driver
+                removeBlock(blockId)

Review Comment:
   This should only affect RDD blocks (`removeBlock` is also called via 
`handleLocalReadFailure` from line 971 when the block info manager knows the 
block but the and the stores memory/disk is does not contain it). 



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