waitinfuture commented on code in PR #2349:
URL: 
https://github.com/apache/incubator-celeborn/pull/2349#discussion_r1509043800


##########
client/src/main/java/org/apache/celeborn/client/read/CelebornInputStream.java:
##########
@@ -532,66 +559,78 @@ private boolean moveToNextChunk() throws IOException {
     }
 
     private boolean fillBuffer() throws IOException {
-      if (currentChunk == null) {
-        return false;
-      }
-
-      boolean hasData = false;
-      while (currentChunk.isReadable() || moveToNextChunk()) {
-        currentChunk.readBytes(sizeBuf);
-        int mapId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET);
-        int attemptId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET + 
4);
-        int batchId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET + 8);
-        int size = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET + 12);
-
-        if (shuffleCompressionEnabled) {
-          if (size > compressedBuf.length) {
-            compressedBuf = new byte[size];
-          }
+      try {
+        if (currentChunk == null) {
+          return false;
+        }
 
-          currentChunk.readBytes(compressedBuf, 0, size);
-        } else {
-          if (size > rawDataBuf.length) {
-            rawDataBuf = new byte[size];
-          }
+        boolean hasData = false;
+        while (currentChunk.isReadable() || moveToNextChunk()) {
+          currentChunk.readBytes(sizeBuf);
+          int mapId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET);
+          int attemptId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET 
+ 4);
+          int batchId = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET + 
8);
+          int size = Platform.getInt(sizeBuf, Platform.BYTE_ARRAY_OFFSET + 12);
+
+          if (shuffleCompressionEnabled) {
+            if (size > compressedBuf.length) {
+              compressedBuf = new byte[size];
+            }
 
-          currentChunk.readBytes(rawDataBuf, 0, size);
-        }
+            currentChunk.readBytes(compressedBuf, 0, size);
+          } else {
+            if (size > rawDataBuf.length) {
+              rawDataBuf = new byte[size];
+            }
 
-        // de-duplicate
-        if (attemptId == attempts[mapId]) {
-          if (!batchesRead.containsKey(mapId)) {
-            Set<Integer> batchSet = new HashSet<>();
-            batchesRead.put(mapId, batchSet);
+            currentChunk.readBytes(rawDataBuf, 0, size);
           }
-          Set<Integer> batchSet = batchesRead.get(mapId);
-          if (!batchSet.contains(batchId)) {
-            batchSet.add(batchId);
-            callback.incBytesRead(BATCH_HEADER_SIZE + size);
-            if (shuffleCompressionEnabled) {
-              // decompress data
-              int originalLength = decompressor.getOriginalLen(compressedBuf);
-              if (rawDataBuf.length < originalLength) {
-                rawDataBuf = new byte[originalLength];
+
+          // de-duplicate
+          if (attemptId == attempts[mapId]) {
+            if (!batchesRead.containsKey(mapId)) {
+              Set<Integer> batchSet = new HashSet<>();
+              batchesRead.put(mapId, batchSet);
+            }
+            Set<Integer> batchSet = batchesRead.get(mapId);
+            if (!batchSet.contains(batchId)) {
+              batchSet.add(batchId);
+              callback.incBytesRead(BATCH_HEADER_SIZE + size);
+              if (shuffleCompressionEnabled) {
+                // decompress data
+                int originalLength = 
decompressor.getOriginalLen(compressedBuf);
+                if (rawDataBuf.length < originalLength) {
+                  rawDataBuf = new byte[originalLength];
+                }
+                limit = decompressor.decompress(compressedBuf, rawDataBuf, 0);
+              } else {
+                limit = size;
               }
-              limit = decompressor.decompress(compressedBuf, rawDataBuf, 0);
+              position = 0;
+              hasData = true;
+              break;
             } else {
-              limit = size;
+              logger.debug(
+                  "Skip duplicated batch: mapId {}, attemptId {}, batchId {}.",
+                  mapId,
+                  attemptId,
+                  batchId);
             }
-            position = 0;
-            hasData = true;
-            break;
-          } else {
-            logger.debug(
-                "Skip duplicated batch: mapId {}, attemptId {}, batchId {}.",
-                mapId,
-                attemptId,
-                batchId);
           }
         }
-      }
 
-      return hasData;
+        return hasData;
+      } catch (IOException e) {
+        IOException ioe = e;
+        if (exceptionMaker != null) {
+          if (shuffleClient.reportShuffleFetchFailure(appShuffleId, 
shuffleId)) {
+            ioe =
+                new CelebornIOException(

Review Comment:
   Not necessarily to throw FetchFailedException here, because 
FetchFailedException constructor automatically set fetchFailed
   ```
     // SPARK-19276. We set the fetch failure in the task context, so that even 
if there is user-code
     // which intercepts this exception (possibly wrapping it), the Executor 
can still tell there was
     // a fetch failure, and send the correct error msg back to the driver.  We 
wrap with an Option
     // because the TaskContext is not defined in some test cases.
     Option(TaskContext.get()).map(_.setFetchFailed(this))
   ```
   , and Spark checks it even if the Exception doesn't match
   ```
           case t: Throwable if hasFetchFailure && !Utils.isFatalError(t) =>
             val reason = task.context.fetchFailed.get.toTaskFailedReason
             if (!t.isInstanceOf[FetchFailedException]) {
               // there was a fetch failure in the task, but some user code 
wrapped that exception
               // and threw something else.  Regardless, we treat it as a fetch 
failure.
               val fetchFailedCls = classOf[FetchFailedException].getName
               logWarning(s"TID ${taskId} encountered a ${fetchFailedCls} and " 
+
                 s"failed, but the ${fetchFailedCls} was hidden by another " +
                 s"exception.  Spark is handling this like a fetch failure and 
ignoring the " +
                 s"other exception: $t")
             }
             setTaskFinishedAndClearInterruptStatus()
             execBackend.statusUpdate(taskId, TaskState.FAILED, 
ser.serialize(reason))
   ```



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

Reply via email to