This is an automated email from the ASF dual-hosted git repository.

zhouky pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/incubator-celeborn.git


The following commit(s) were added to refs/heads/main by this push:
     new 785a035ec [CELEBORN-1017] Make checkPushTimeout and checkFetchTimeout 
conditions independent
785a035ec is described below

commit 785a035eca94d55b8694c2f607efdd78cb33c417
Author: jiaoqingbo <[email protected]>
AuthorDate: Sat Oct 7 16:23:44 2023 +0800

    [CELEBORN-1017] Make checkPushTimeout and checkFetchTimeout conditions 
independent
    
    ### What changes were proposed in this pull request?
    
    Make checkPushTimeout and checkFetchTimeout conditions independent
    
    ### Why are the changes needed?
    
    https://github.com/apache/incubator-celeborn/pull/1940 introduced a bug
    
    
![image](https://github.com/apache/incubator-celeborn/assets/14961757/8e8764d4-235d-49cf-8a87-1052ffbe4697)
    
    The judgment logic of checkFetchTimeout is added to the judgment logic of 
checkPushTimeout. The two of them should be independent.
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    PASS GA
    
    Closes #1954 from jiaoqingbo/1017.
    
    Authored-by: jiaoqingbo <[email protected]>
    Signed-off-by: zky.zhoukeyong <[email protected]>
---
 .../common/network/client/TransportResponseHandler.java    | 14 ++++++++------
 1 file changed, 8 insertions(+), 6 deletions(-)

diff --git 
a/common/src/main/java/org/apache/celeborn/common/network/client/TransportResponseHandler.java
 
b/common/src/main/java/org/apache/celeborn/common/network/client/TransportResponseHandler.java
index 78120a2c1..de1bd7379 100644
--- 
a/common/src/main/java/org/apache/celeborn/common/network/client/TransportResponseHandler.java
+++ 
b/common/src/main/java/org/apache/celeborn/common/network/client/TransportResponseHandler.java
@@ -96,15 +96,17 @@ public class TransportResponseHandler extends 
MessageHandler<ResponseMessage> {
               ThreadUtils.newDaemonThreadPoolScheduledExecutor(
                   "push-timeout-checker", 
conf.pushDataTimeoutCheckerThreads());
         }
-        if (checkFetchTimeout) {
-          if (fetchTimeoutChecker == null) {
-            fetchTimeoutChecker =
-                ThreadUtils.newDaemonThreadPoolScheduledExecutor(
-                    "fetch-timeout-checker", 
conf.fetchDataTimeoutCheckerThreads());
-          }
+      }
+
+      if (checkFetchTimeout) {
+        if (fetchTimeoutChecker == null) {
+          fetchTimeoutChecker =
+              ThreadUtils.newDaemonThreadPoolScheduledExecutor(
+                  "fetch-timeout-checker", 
conf.fetchDataTimeoutCheckerThreads());
         }
       }
     }
+
     if (checkPushTimeout) {
       pushCheckerScheduleFuture =
           pushTimeoutChecker.scheduleAtFixedRate(

Reply via email to