You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@celeborn.apache.org by "AngersZhuuuu (via GitHub)" <gi...@apache.org> on 2023/02/24 08:59:13 UTC

[GitHub] [incubator-celeborn] AngersZhuuuu opened a new pull request, #1274: [CELEBORN-338] Clean duplicated exception message of ShuffleClientImpl

AngersZhuuuu opened a new pull request, #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274

   ### What changes were proposed in this pull request?
   In the current ShuffleClientImple code, the delivery of exceptions is messy, repetitive, and redundant.
   In this pr, do below things:
   
   1. Keep passing `RuntimeException` from `TransportResponseHandler` to `wrappedCallback.onFailure`
   2. In the `ShuffleClientImpl` side, we only pass `CelebornIOException` with a message only contain cause name to `wrappedCallback.onFailure`
   3. Only add detail info of pushing data into the final exception of pushState
   4. Keep error stack of `ShuffleClientImpl` side exception 
   
   
   ### Why are the changes needed?
   
   
   
   ### Does this PR introduce _any_ user-facing change?
   
   
   
   ### How was this patch tested?
   
   


-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118343642


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -698,17 +704,17 @@ public void onSuccess(ByteBuffer response) {
 
             @Override
             public void onFailure(Throwable e) {
-              pushState.exception.compareAndSet(
-                  null, new CelebornIOException("Revived PushData failed!", e));
-              logger.error(
-                  "Push data to {} failed for shuffle {} map {} attempt {} partition {} batch {}.",
-                  loc.hostAndPushPort(),
-                  shuffleId,
-                  mapId,
-                  attemptId,
-                  partitionId,
-                  nextBatchId,
-                  e);
+              String errorMsg =
+                  String.format(
+                      "Push data to %s failed for shuffle $d map %d attempt %d partition %d batch %d.",
+                      loc,

Review Comment:
   Here log the old partition location,  and other log can find the revived new location by partition id in executor's log.



##########
common/src/main/java/org/apache/celeborn/common/protocol/message/StatusCode.java:
##########
@@ -41,8 +41,8 @@ public enum StatusCode {
 
   // push data fail causes
   PUSH_DATA_FAIL_NON_CRITICAL_CAUSE(17),
-  PUSH_DATA_FAIL_SLAVE(18),
-  PUSH_DATA_FAIL_MASTER(19),
+  PUSH_DATA_WRITE_FAIL_SLAVE(18),
+  PUSH_DATA_WRITE_FAIL_MASTER(19),

Review Comment:
   Change here since origin code not so specify, make it more clear



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118342506


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -229,12 +237,7 @@ private void submitRetryPushData(
             newLoc,

Review Comment:
   For revive then push failed, here will log the new loc



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] waitinfuture merged pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "waitinfuture (via GitHub)" <gi...@apache.org>.
waitinfuture merged PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274


-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] codecov[bot] commented on pull request #1274: [CELEBORN-338] Clean duplicated exception message of ShuffleClientImpl

Posted by "codecov[bot] (via GitHub)" <gi...@apache.org>.
codecov[bot] commented on PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#issuecomment-1443256577

   # [Codecov](https://codecov.io/gh/apache/incubator-celeborn/pull/1274?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#1274](https://codecov.io/gh/apache/incubator-celeborn/pull/1274?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (c837ad1) into [main](https://codecov.io/gh/apache/incubator-celeborn/commit/a7587c3fe7dc1f9cbb1182e5f6f684ef25c94b04?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (a7587c3) will **decrease** coverage by `0.01%`.
   > The diff coverage is `10.21%`.
   
   > :exclamation: Current head c837ad1 differs from pull request most recent head e6e55ad. Consider uploading reports for the commit e6e55ad to get more accurate results
   
   ```diff
   @@             Coverage Diff              @@
   ##               main    #1274      +/-   ##
   ============================================
   - Coverage     26.99%   26.97%   -0.01%     
   - Complexity      809      810       +1     
   ============================================
     Files           215      215              
     Lines         18418    18437      +19     
     Branches       1982     1984       +2     
   ============================================
   + Hits           4970     4972       +2     
   - Misses        13119    13137      +18     
   + Partials        329      328       -1     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/incubator-celeborn/pull/1274?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [.../org/apache/celeborn/client/ShuffleClientImpl.java](https://codecov.io/gh/apache/incubator-celeborn/pull/1274?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-Y2xpZW50L3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9jZWxlYm9ybi9jbGllbnQvU2h1ZmZsZUNsaWVudEltcGwuamF2YQ==) | `16.02% <10.21%> (-0.22%)` | :arrow_down: |
   | [...oy/worker/congestcontrol/CongestionController.java](https://codecov.io/gh/apache/incubator-celeborn/pull/1274?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-d29ya2VyL3NyYy9tYWluL2phdmEvb3JnL2FwYWNoZS9jZWxlYm9ybi9zZXJ2aWNlL2RlcGxveS93b3JrZXIvY29uZ2VzdGNvbnRyb2wvQ29uZ2VzdGlvbkNvbnRyb2xsZXIuamF2YQ==) | `77.78% <0.00%> (+0.93%)` | :arrow_up: |
   
   :mega: We’re building smart automated test selection to slash your CI/CD build times. [Learn more](https://about.codecov.io/iterative-testing/?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   


-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] waitinfuture commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "waitinfuture (via GitHub)" <gi...@apache.org>.
waitinfuture commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118457681


##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala:
##########
@@ -561,19 +554,18 @@ class PushDataHandler extends BaseMessageHandler with Logging {
             }
 
             override def onFailure(e: Throwable): Unit = {
+              logError(s"PushMergedData replicate failed for partitionLocation: $location", e)
               workerSource.incCounter(WorkerSource.PushDataFailCount)
-              // Throw by slave peer worker
-              if (e.getMessage.startsWith(StatusCode.PUSH_DATA_FAIL_SLAVE.name())) {
+              // 1. Throw PUSH_DATA_FAIL_SLAVE by slave peer worker
+              // 2. Throw PUSH_DATA_TIMEOUT_SLAVE by TransportResponseHandler
+              // 3. Throw IOException by channel, convert to PUSH_DATA_CONNECTION_EXCEPTION_SLAVE
+              if (e.getMessage.startsWith(StatusCode.PUSH_DATA_WRITE_FAIL_SLAVE.name())) {
                 callbackWithTimer.onFailure(e)
               } else if (e.getMessage.startsWith(StatusCode.PUSH_DATA_TIMEOUT_SLAVE.name())) {
-                callbackWithTimer.onFailure(new CelebornIOException(
-                  s"${StatusCode.PUSH_DATA_TIMEOUT_SLAVE}! Push data to peer of ${partitionIdToLocations.head._2} timeout: ${e.getMessage}",
-                  e))
+                callbackWithTimer.onFailure(e)

Review Comment:
   ditto



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala:
##########
@@ -303,20 +300,18 @@ class PushDataHandler extends BaseMessageHandler with Logging {
             }
 
             override def onFailure(e: Throwable): Unit = {
-              logError(s"[handlePushData.onFailure] partitionLocation: $location", e)
+              logError(s"PushData replication failed for partitionLocation: $location", e)
               workerSource.incCounter(WorkerSource.PushDataFailCount)
-              // Throw by slave peer worker
-              if (e.getMessage.startsWith(StatusCode.PUSH_DATA_FAIL_SLAVE.name())) {
+              // 1. Throw PUSH_DATA_FAIL_SLAVE by slave peer worker
+              // 2. Throw PUSH_DATA_TIMEOUT_SLAVE by TransportResponseHandler
+              // 3. Throw IOException by channel, convert to PUSH_DATA_CONNECTION_EXCEPTION_SLAVE
+              if (e.getMessage.startsWith(StatusCode.PUSH_DATA_WRITE_FAIL_SLAVE.name())) {
                 callbackWithTimer.onFailure(e)
               } else if (e.getMessage.startsWith(StatusCode.PUSH_DATA_TIMEOUT_SLAVE.name())) {
-                callbackWithTimer.onFailure(new CelebornIOException(
-                  s"${StatusCode.PUSH_DATA_TIMEOUT_SLAVE}! Push data to peer of $location timeout: ${e.getMessage}",
-                  e))
+                callbackWithTimer.onFailure(e)

Review Comment:
   can merge the first two branches



##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala:
##########
@@ -978,7 +964,7 @@ class PushDataHandler extends BaseMessageHandler with Logging {
           callback.onFailure(new CelebornIOException(StatusCode.REGION_FINISH_FAIL_SLAVE, e))
         case _ =>
           workerSource.incCounter(WorkerSource.PushDataFailCount)
-          callback.onFailure(new CelebornIOException(StatusCode.PUSH_DATA_FAIL_SLAVE, e))
+          callback.onFailure(new CelebornIOException(StatusCode.PUSH_DATA_WRITE_FAIL_SLAVE, e))

Review Comment:
   I think we should check if e is CelebornIOException, if yes just use it, if not, use PUSH_DATA_FAIL_SLAVE, we can't be sure that it's PUSH_DATA_WRITE_FAIL_SLAVE here



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -183,16 +183,24 @@ private void submitRetryPushData(
     int partitionId = loc.getId();
     if (!revive(
         applicationId, shuffleId, mapId, attemptId, partitionId, loc.getEpoch(), loc, cause)) {
-      wrappedCallback.onFailure(
-          new CelebornIOException("Revive Failed, remain revive times " + remainReviveTimes));
+      logger.error(

Review Comment:
   RpcResponseCallback.onFailure will log
   ```
                 logger.error(
                     "Push data to {} failed for shuffle {} map {} attempt {} partition {} batch {}.",
                     loc.hostAndPushPort(),
                     shuffleId,
                     mapId,
                     attemptId,
                     partitionId,
                     nextBatchId,
                     e);
   ```
   will this be duplicate?



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -266,10 +269,12 @@ private void submitRetryPushMergedData(
         if (remainReviveTimes > 0) {
           reviveFailedBatchesMap.add(batch);
         } else {
-          pushState.exception.compareAndSet(
-              null,
-              new CelebornIOException(
-                  "Revive Failed in retry push merged data for location: " + batch.loc));
+          String errorMsg =
+              String.format(
+                  "Revive failed while pushing merged for shuffle %d map %d attempt %d partition %d batch %d location %s.",
+                  shuffleId, mapId, attemptId, partitionId, oldGroupedBatchId, batch.loc);
+          logger.debug(errorMsg);
+          pushState.exception.compareAndSet(null, new CelebornIOException(errorMsg));

Review Comment:
   When pushState.exception is thrown, the errorMsg will be print, so will it be duplicate?



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -698,17 +699,12 @@ public void onSuccess(ByteBuffer response) {
 
             @Override
             public void onFailure(Throwable e) {
-              pushState.exception.compareAndSet(
-                  null, new CelebornIOException("Revived PushData failed!", e));
-              logger.error(
-                  "Push data to {} failed for shuffle {} map {} attempt {} partition {} batch {}.",
-                  loc.hostAndPushPort(),
-                  shuffleId,
-                  mapId,
-                  attemptId,
-                  partitionId,
-                  nextBatchId,
-                  e);
+              String errorMsg =
+                  String.format(
+                      "Push data to %s failed for shuffle $d map %d attempt %d partition %d batch %d.",
+                      loc, shuffleId, mapId, attemptId, partitionId, nextBatchId);
+              pushState.exception.compareAndSet(null, new CelebornIOException(errorMsg, e));
+              logger.error(errorMsg, e);

Review Comment:
   ditto



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1183,23 +1166,21 @@ public void onSuccess(ByteBuffer response) {
           @Override
           public void onFailure(Throwable e) {
             StatusCode cause = getPushDataFailCause(e.getMessage());
+
             if (pushState.exception.get() != null) {
               return;
             }
             if (remainReviveTimes <= 0) {
-              callback.onFailure(
-                  new Exception(
-                      cause
-                          + "! Push data to master worker of "
-                          + hostPort
-                          + " failed: "
-                          + e.getMessage(),
-                      e));
+              if (e instanceof CelebornIOException) {
+                callback.onFailure(e);
+              } else {
+                callback.onFailure(new CelebornIOException(cause, e));
+              }
               return;
             }
             logger.error(
                 "Push merged data to {} failed for shuffle {} map {} attempt {} partition {} groupedBatch {} batch {}.",
-                hostPort,
+                addressPair,

Review Comment:
   Better to print ```remainReviveTimes``` here, and should log in else branch below that mapper has ended



##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1601,16 +1577,12 @@ public void onFailure(Throwable e) {
               return;
             }
             if (!mapperEnded(shuffleId, mapId, attemptId)) {
-              pushState.exception.compareAndSet(
-                  null, new CelebornIOException("PushData byteBuf failed!", e));
-              logger.error(
-                  "Push data byteBuf to {} failed for shuffle {} map {} attempt {} batch {}.",
-                  location.hostAndPushPort(),
-                  shuffleId,
-                  mapId,
-                  attemptId,
-                  nextBatchId,
-                  e);
+              String errorMsg =
+                  String.format(
+                      "Push data byteBuf to %s failed for shuffle %d map %d attempt %d batch %d.",
+                      location.hostAndPushPort(), shuffleId, mapId, attemptId, nextBatchId);
+              pushState.exception.compareAndSet(null, new CelebornIOException(errorMsg, e));
+              logger.error(errorMsg, e);

Review Comment:
   ditto



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118516971


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1183,23 +1166,21 @@ public void onSuccess(ByteBuffer response) {
           @Override
           public void onFailure(Throwable e) {
             StatusCode cause = getPushDataFailCause(e.getMessage());
+
             if (pushState.exception.get() != null) {
               return;
             }
             if (remainReviveTimes <= 0) {
-              callback.onFailure(
-                  new Exception(
-                      cause
-                          + "! Push data to master worker of "
-                          + hostPort
-                          + " failed: "
-                          + e.getMessage(),
-                      e));
+              if (e instanceof CelebornIOException) {
+                callback.onFailure(e);
+              } else {
+                callback.onFailure(new CelebornIOException(cause, e));
+              }
               return;
             }
             logger.error(
                 "Push merged data to {} failed for shuffle {} map {} attempt {} partition {} groupedBatch {} batch {}.",
-                hostPort,
+                addressPair,

Review Comment:
   DOne



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118520161


##########
worker/src/main/scala/org/apache/celeborn/service/deploy/worker/PushDataHandler.scala:
##########
@@ -978,7 +964,7 @@ class PushDataHandler extends BaseMessageHandler with Logging {
           callback.onFailure(new CelebornIOException(StatusCode.REGION_FINISH_FAIL_SLAVE, e))
         case _ =>
           workerSource.incCounter(WorkerSource.PushDataFailCount)
-          callback.onFailure(new CelebornIOException(StatusCode.PUSH_DATA_FAIL_SLAVE, e))
+          callback.onFailure(new CelebornIOException(StatusCode.PUSH_DATA_WRITE_FAIL_SLAVE, e))

Review Comment:
   Done



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118504079


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -183,16 +183,24 @@ private void submitRetryPushData(
     int partitionId = loc.getId();
     if (!revive(
         applicationId, shuffleId, mapId, attemptId, partitionId, loc.getEpoch(), loc, cause)) {
-      wrappedCallback.onFailure(
-          new CelebornIOException("Revive Failed, remain revive times " + remainReviveTimes));
+      logger.error(

Review Comment:
   Yea, should remove this



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118508263


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -266,10 +269,12 @@ private void submitRetryPushMergedData(
         if (remainReviveTimes > 0) {
           reviveFailedBatchesMap.add(batch);
         } else {
-          pushState.exception.compareAndSet(
-              null,
-              new CelebornIOException(
-                  "Revive Failed in retry push merged data for location: " + batch.loc));
+          String errorMsg =
+              String.format(
+                  "Revive failed while pushing merged for shuffle %d map %d attempt %d partition %d batch %d location %s.",
+                  shuffleId, mapId, attemptId, partitionId, oldGroupedBatchId, batch.loc);
+          logger.debug(errorMsg);
+          pushState.exception.compareAndSet(null, new CelebornIOException(errorMsg));

Review Comment:
   > When pushState.exception is thrown, the errorMsg will be print, so will it be duplicate?
   
   Yea, don't need to print the error.



-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#issuecomment-1445635727

   ping @FMX @waitinfuture @pan3793 


-- 
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: issues-unsubscribe@celeborn.apache.org

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


[GitHub] [incubator-celeborn] AngersZhuuuu commented on a diff in pull request #1274: [CELEBORN-338] Clean duplicated exception message of handling push data

Posted by "AngersZhuuuu (via GitHub)" <gi...@apache.org>.
AngersZhuuuu commented on code in PR #1274:
URL: https://github.com/apache/incubator-celeborn/pull/1274#discussion_r1118265749


##########
client/src/main/java/org/apache/celeborn/client/ShuffleClientImpl.java:
##########
@@ -1187,19 +1180,16 @@ public void onFailure(Throwable e) {
               return;
             }
             if (remainReviveTimes <= 0) {
-              callback.onFailure(
-                  new Exception(
-                      cause
-                          + "! Push data to master worker of "
-                          + hostPort
-                          + " failed: "
-                          + e.getMessage(),
-                      e));
+              if (e instanceof CelebornIOException) {
+                callback.onFailure(e);

Review Comment:
   Channel may throw IOException, so here add this judgement.



-- 
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: issues-unsubscribe@celeborn.apache.org

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