You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/12/23 14:15:40 UTC

[GitHub] [spark] srowen commented on a change in pull request #34980: [SPARK-37710][CORE] Add clear error message for java.io.IOException: Input/output error

srowen commented on a change in pull request #34980:
URL: https://github.com/apache/spark/pull/34980#discussion_r774597380



##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -926,30 +936,62 @@ private[spark] class BlockManager(
           })
           Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
         } else if (level.useDisk && diskStore.contains(blockId)) {
-          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)
-            }
-          }
-          val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
-            releaseLockAndDispose(blockId, diskData, taskContext)
-          })
-          Some(new BlockResult(ci, DataReadMethod.Disk, info.size))
+          getLocalValuesFromDisk(blockId, info, taskContext)
         } else {
           handleLocalReadFailure(blockId)
         }
     }
   }
 
+  private def getLocalValuesFromDisk(blockId: BlockId, info: BlockInfo,

Review comment:
       I missed why this needed to be refactored

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -342,6 +343,15 @@ private[spark] class BlockManager(
             iter.close()
             false
         }
+      } catch {
+        case ex: KryoException
+          // We need to have clear error message to catch environmental problems easily.
+          // Further details: https://issues.apache.org/jira/browse/SPARK-37710
+          if ex.getMessage.toLowerCase(Locale.ROOT)

Review comment:
       Can we match on the cause of the KryoException to detect IOException? would that catch it more cleanly, or are only some IOExceptions of interest?

##########
File path: core/src/main/scala/org/apache/spark/storage/BlockManager.scala
##########
@@ -926,30 +936,62 @@ private[spark] class BlockManager(
           })
           Some(new BlockResult(ci, DataReadMethod.Memory, info.size))
         } else if (level.useDisk && diskStore.contains(blockId)) {
-          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)
-            }
-          }
-          val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
-            releaseLockAndDispose(blockId, diskData, taskContext)
-          })
-          Some(new BlockResult(ci, DataReadMethod.Disk, info.size))
+          getLocalValuesFromDisk(blockId, info, taskContext)
         } else {
           handleLocalReadFailure(blockId)
         }
     }
   }
 
+  private def getLocalValuesFromDisk(blockId: BlockId, info: BlockInfo,
+    taskContext: Option[TaskContext]): Option[BlockResult] = {
+    try {
+      val level = info.level
+      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)
+        }
+      }
+      val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, {
+        releaseLockAndDispose(blockId, diskData, taskContext)
+      })
+      Some(new BlockResult(ci, DataReadMethod.Disk, info.size))
+    } catch {
+      case ex: KryoException
+        // We need to have clear error message to catch environmental problems easily.
+        // Further details: https://jira2.workday.com/browse/PRISM-102331
+        if ex.getMessage.toLowerCase(Locale.ROOT)
+          .contains("java.io.ioexception: input/output error") => {
+        processKryoException(ex, blockId)
+        throw ex
+      }
+    }
+  }
+
+  private def processKryoException(ex: KryoException, blockId: BlockId): Unit = {
+    val errorMessage =
+      new StringBuffer(s"${ex.getMessage} usually occurs due to environmental problems " +

Review comment:
       Does this really provide more info?




-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org