You are viewing a plain text version of this content. The canonical link for it is here.
Posted to codereview@trafodion.apache.org by mashengchen <gi...@git.apache.org> on 2018/01/31 10:53:11 UTC

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

GitHub user mashengchen opened a pull request:

    https://github.com/apache/trafodion/pull/1427

    TRAFODION-2940 In HA env, one node lose network, when recover, trafci can't use

    when there loses network for a long time ,and then network recover, there will trigger zookeeper session expired, at this time ,check whether current dcsmaster is leader, if not unbind this node's floating ip, and make dcsmaster init status, then rerun dcs master.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mashengchen/trafodion TRAFODION-2940

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/trafodion/pull/1427.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #1427
    
----
commit c92bd619a310f6b1b13d4164250ee8c8cb93f1e6
Author: aven <sh...@...>
Date:   2018-01-31T10:33:48Z

    TRAFODION-2940 In HA env, one node lose network, when recover, trafci can't use

----


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736393
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/listener/ListenerService.java ---
    @@ -263,14 +264,20 @@ else if ((key.interestOps() & SelectionKey.OP_WRITE) == SelectionKey.OP_WRITE) {
                             }
                         }
                     }
    +                if (this.isInterrupted()) {
    +                    throw new InterruptedException();
    +                }
                     //gc();
                 }
    +        } catch (InterruptedException e) {
    --- End diff --
    
    Log what happens by info level at least.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736968
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/zookeeper/ZkClient.java ---
    @@ -151,8 +163,28 @@ public void connect() throws IOException, InterruptedException {
     				this.zk=null;
     				throw new IOException("Cannot connect to Zookeeper");
     			}
    -			
    -			LOG.debug("Zookeeper.State=" + this.zk.getState());
    +
    +            // Solve the forcible reconnection
    +            // When zk reconn, the backup-master may take over the master,
    +            // so current master should restart, and queues in /dcs/master/leader
    +            if (LOG.isDebugEnabled()) {
    +                LOG.debug("force = [" + force + "]. checkPath = [" + checkPath + "]");
    +            }
    +            if (force && checkPath != null) {
    +                try {
    +                    Stat stat = zk.exists(checkPath, false);
    +                    if (LOG.isDebugEnabled()) {
    +                        LOG.debug("stat = [" + stat + "].");
    +                    }
    +                    if (stat == null) {
    +                        // this means master has change.
    +                        setSessionRecoverSuccessful(false);
    +                    }
    +                } catch (KeeperException e) {
    +                    e.printStackTrace();
    --- End diff --
    
    Log messages in file.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736451
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/listener/ListenerService.java ---
    @@ -263,14 +264,20 @@ else if ((key.interestOps() & SelectionKey.OP_WRITE) == SelectionKey.OP_WRITE) {
                             }
                         }
                     }
    +                if (this.isInterrupted()) {
    +                    throw new InterruptedException();
    +                }
                     //gc();
                 }
    +        } catch (InterruptedException e) {
             } catch (IOException e) {
                 LOG.error(e);
    --- End diff --
    
    it's better to use 2 parameters API for LOG.error


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172734228
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -203,8 +244,8 @@ public void run() {
             } catch (KeeperException.NodeExistsException e) {
                 // do nothing...some other server has created znodes
    --- End diff --
    
    it should be better to add warning for this exception that we could what happened. Any events happened should be reflected in log files.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172733924
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -129,19 +170,19 @@ public void run() {
                 instance = "1";
             } catch (NullPointerException e) {
                 LOG.error("No args found: ", e);
    -            System.exit(1);
    +            return -1;
             } catch (ParseException e) {
                 LOG.error("Could not parse: ", e);
    -            System.exit(1);
    +            return -1;
             }
     
             try {
                 zkc = new ZkClient();
                 zkc.connect();
                 LOG.info("Connected to ZooKeeper");
             } catch (Exception e) {
    -            LOG.error(e);
    -            System.exit(1);
    +            LOG.error(e.getMessage(), e);
    +            return 1;
    --- End diff --
    
    Close zk connection while any connection exception happens. it's better to use specific Exception for zookeeper.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736344
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/listener/ListenerService.java ---
    @@ -263,14 +264,20 @@ else if ((key.interestOps() & SelectionKey.OP_WRITE) == SelectionKey.OP_WRITE) {
                             }
                         }
                     }
    +                if (this.isInterrupted()) {
    +                    throw new InterruptedException();
    --- End diff --
    
    Add some specific message for this exception.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by svarnau <gi...@git.apache.org>.
Github user svarnau commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r168625988
  
    --- Diff: dcs/bin/scripts/dcsunbind.sh ---
    @@ -51,6 +51,31 @@ function check_node {
         done
     }
     
    +function check_self_node {
    +    for myinterface in `/sbin/ip link show|cut -d: -f1- | cut -c1- | awk -F': ' '/^[0-9]+:.*/ {print $2;}'`; do
    +        ip_output=$(/sbin/ip addr show $myinterface | cut -d: -f1- | cut -c1-)
    +
    +        myifport=`echo "$ip_output" | grep -w $gv_float_external_ip`
    +        status=$?
    +        if [ $status -eq 0 ]; then
    +            tempinterface=`echo $gv_float_interface:$gv_port`
    +            # check if another interface is bound to this virtual ip address
    +            echo "$myifport" | grep "$tempinterface"  > /dev/null
    +            if [ $? -eq 0 ]; then
    +                unbindip=`echo "$myifport" | awk '{print $2}'`
    +                unbindlb=`echo "$myifport"|awk '{print $NF}'`
    +                echo "Virtual ip $gv_float_external_ip is in use on node $HOSTNAME bound to interface $myinterface($unbindlb) - unbinding..."
    +                sudo /sbin/ip addr del $unbindip dev $myinterface
    +                status=$?
    --- End diff --
    
    It seems unnecessary to duplicate entire function just to change whether commands run locally or remotely. That can be done with a variable, so that we don't have two copies of this code to maintain.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172737025
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/zookeeper/ZkClient.java ---
    @@ -176,7 +208,19 @@ public ZooKeeper getZk() {
     	public void process(WatchedEvent event) {
     		if(event.getState() == Watcher.Event.KeeperState.SyncConnected) {
     			connectedSignal.countDown();
    -		}
    +		} else if (event.getState() == Watcher.Event.KeeperState.Expired) {
    +            LOG.info("session expired. now rebuilding");
    +            // session expired, may be never happending. but if it happen there
    +            // need to close old client and rebuild new client
    +            try {
    +                connect(true);
    +            } catch (IOException e) {
    +                setSessionRecoverSuccessful(false);
    +                LOG.error("session expired and throw IOException while do reconnect: " + e.getMessage());
    --- End diff --
    
    Use 2 params API for log. If it doesn't matter for whatever the message is, it should be LOG.warn.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by mashengchen <gi...@git.apache.org>.
Github user mashengchen commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172462100
  
    --- Diff: dcs/bin/scripts/dcsunbind.sh ---
    @@ -51,6 +51,31 @@ function check_node {
         done
     }
     
    +function check_self_node {
    +    for myinterface in `/sbin/ip link show|cut -d: -f1- | cut -c1- | awk -F': ' '/^[0-9]+:.*/ {print $2;}'`; do
    +        ip_output=$(/sbin/ip addr show $myinterface | cut -d: -f1- | cut -c1-)
    +
    +        myifport=`echo "$ip_output" | grep -w $gv_float_external_ip`
    +        status=$?
    +        if [ $status -eq 0 ]; then
    +            tempinterface=`echo $gv_float_interface:$gv_port`
    +            # check if another interface is bound to this virtual ip address
    +            echo "$myifport" | grep "$tempinterface"  > /dev/null
    +            if [ $? -eq 0 ]; then
    +                unbindip=`echo "$myifport" | awk '{print $2}'`
    +                unbindlb=`echo "$myifport"|awk '{print $NF}'`
    +                echo "Virtual ip $gv_float_external_ip is in use on node $HOSTNAME bound to interface $myinterface($unbindlb) - unbinding..."
    +                sudo /sbin/ip addr del $unbindip dev $myinterface
    +                status=$?
    --- End diff --
    
    this function is a little different from the one using PDSH,  the ouput of `$SQ_PDSH $MY_NODES /sbin/ip addr show` is different from `/sbin/ip link show` , so the cut step after the " | " (pipe) is different. 


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736542
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/listener/ListenerService.java ---
    @@ -263,14 +264,20 @@ else if ((key.interestOps() & SelectionKey.OP_WRITE) == SelectionKey.OP_WRITE) {
                             }
                         }
                     }
    +                if (this.isInterrupted()) {
    +                    throw new InterruptedException();
    +                }
                     //gc();
                 }
    +        } catch (InterruptedException e) {
             } catch (IOException e) {
                 LOG.error(e);
                 System.exit(1);
             } finally {
    +            LOG.info("close ServerSocketChannel...");
                 if (server != null) {
                     try {
    +                    server.socket().close();
                         server.close();
                     } catch (IOException e) {
                         e.printStackTrace();
    --- End diff --
    
    it's better to print the message into log file.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by DaveBirdsall <gi...@git.apache.org>.
Github user DaveBirdsall commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r166434382
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -111,11 +104,59 @@ public DcsMaster(String[] args) {
             trafodionHome = System.getProperty(Constants.DCS_TRAFODION_HOME);
             jvmShutdownHook = new JVMShutdownHook();
             Runtime.getRuntime().addShutdownHook(jvmShutdownHook);
    -        thrd = new Thread(this);
    -        thrd.start();
    +
    +        ExecutorService executorService = Executors.newFixedThreadPool(1);
    +        CompletionService<Integer> completionService = new ExecutorCompletionService<Integer>(executorService);
    +
    +        while (true) {
    +            completionService.submit(this);
    +            Future<Integer> f = null;
    +            try {
    +                f = completionService.take();
    +                if (f != null) {
    +                    Integer status = f.get();
    +                    if (status <= 0) {
    +                        System.exit(status);
    +                    } else {
    +                        // 35000 * 15mins ~= 1 years
    +                        RetryCounter retryCounter = RetryCounterFactory.create(35000, 15, TimeUnit.MINUTES);
    +                        while (true) {
    +                            try {
    +                                ZkClient tmpZkc = new ZkClient();
    +                                tmpZkc.connect();
    +                                tmpZkc.close();
    +                                tmpZkc = null;
    +                                LOG.info("Connected to ZooKeeper successful, restart DCS Master.");
    +                                // reset lock
    +                                isLeader = new CountDownLatch(1);
    +                                break;
    --- End diff --
    
    I'm not sure I understand this logic. Do we sit inside one of these method calls during normal processing? Or does tmpZkc.connect() and close() complete immediately? If so, it looks like we just loop around the while loop and do it again, over and over.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172735971
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -262,12 +308,38 @@ public void run() {
                 future.get();// block
     
             } catch (Exception e) {
    -            LOG.error(e);
    -            e.printStackTrace();
    +            LOG.error(e.getMessage(), e);
    +            try {
    +                FloatingIp floatingIp = FloatingIp.getInstance(this);
    +                floatingIp.unbindScript();
    +            } catch (Exception e1) {
    +                if (LOG.isErrorEnabled()) {
    +                    LOG.error("Error creating class FloatingIp [" + e.getMessage() + "]", e1);
    +                }
    +            }
                 if (pool != null)
                     pool.shutdown();
    -            System.exit(0);
    +            if (ls != null) {
    +                ListenerWorker lw = ls.getWorker();
    +                if (lw != null) {
    +                    lw.interrupt();
    +                    LOG.info("Interrupt listenerWorker.");
    +                }
    +                ls.interrupt();
    +                LOG.info("Interrupt listenerService.");
    +            }
    +            if (infoServer != null) {
    +                try {
    +                    infoServer.stop();
    +                    LOG.info("Stop infoServer.");
    +                } catch (Exception e1) {
    +                    LOG.error(e1.getMessage(), e1);
    +                }
    +            }
    +            return 1;
    +
    --- End diff --
    
    Add try-catch for each progress.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by mashengchen <gi...@git.apache.org>.
Github user mashengchen commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r166504107
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -111,11 +104,59 @@ public DcsMaster(String[] args) {
             trafodionHome = System.getProperty(Constants.DCS_TRAFODION_HOME);
             jvmShutdownHook = new JVMShutdownHook();
             Runtime.getRuntime().addShutdownHook(jvmShutdownHook);
    -        thrd = new Thread(this);
    -        thrd.start();
    +
    +        ExecutorService executorService = Executors.newFixedThreadPool(1);
    +        CompletionService<Integer> completionService = new ExecutorCompletionService<Integer>(executorService);
    +
    +        while (true) {
    +            completionService.submit(this);
    +            Future<Integer> f = null;
    +            try {
    +                f = completionService.take();
    +                if (f != null) {
    +                    Integer status = f.get();
    +                    if (status <= 0) {
    +                        System.exit(status);
    +                    } else {
    +                        // 35000 * 15mins ~= 1 years
    +                        RetryCounter retryCounter = RetryCounterFactory.create(35000, 15, TimeUnit.MINUTES);
    +                        while (true) {
    +                            try {
    +                                ZkClient tmpZkc = new ZkClient();
    +                                tmpZkc.connect();
    +                                tmpZkc.close();
    +                                tmpZkc = null;
    +                                LOG.info("Connected to ZooKeeper successful, restart DCS Master.");
    +                                // reset lock
    +                                isLeader = new CountDownLatch(1);
    +                                break;
    --- End diff --
    
    this logic is for when dcsmaster return with network erro situation.
    in the logic , it will try to connect to zk
    if it can't conn ( tmpZkc.connect(); ) , there will in catch block and do retry
    if it connect to zk, then dcsmaster will run call() method again, in the time dcsmaster rework ,there must hava another backup-master working ( because there must one dcs master work and current master lose network ,then backupmaster take over the role) ,  so when dcsmaster rework , it will set value in zk /rootpath/dcs/leader/ then hang by lock "isLeader = new CountDownLatch(1);"


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172729543
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/DcsMaster.java ---
    @@ -111,11 +104,59 @@ public DcsMaster(String[] args) {
             trafodionHome = System.getProperty(Constants.DCS_TRAFODION_HOME);
             jvmShutdownHook = new JVMShutdownHook();
             Runtime.getRuntime().addShutdownHook(jvmShutdownHook);
    -        thrd = new Thread(this);
    -        thrd.start();
    +
    +        ExecutorService executorService = Executors.newFixedThreadPool(1);
    +        CompletionService<Integer> completionService = new ExecutorCompletionService<Integer>(executorService);
    +
    +        while (true) {
    +            completionService.submit(this);
    +            Future<Integer> f = null;
    +            try {
    +                f = completionService.take();
    +                if (f != null) {
    +                    Integer status = f.get();
    +                    if (status <= 0) {
    +                        System.exit(status);
    +                    } else {
    +                        // 35000 * 15mins ~= 1 years
    +                        RetryCounter retryCounter = RetryCounterFactory.create(35000, 15, TimeUnit.MINUTES);
    +                        while (true) {
    +                            try {
    +                                ZkClient tmpZkc = new ZkClient();
    +                                tmpZkc.connect();
    +                                tmpZkc.close();
    +                                tmpZkc = null;
    +                                LOG.info("Connected to ZooKeeper successful, restart DCS Master.");
    +                                // reset lock
    +                                isLeader = new CountDownLatch(1);
    +                                break;
    --- End diff --
    
    As we discussed, Zookeeper connection lost has been covered by session expired event, so this loop is useless.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172737047
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/zookeeper/ZkClient.java ---
    @@ -176,7 +208,19 @@ public ZooKeeper getZk() {
     	public void process(WatchedEvent event) {
     		if(event.getState() == Watcher.Event.KeeperState.SyncConnected) {
     			connectedSignal.countDown();
    -		}
    +		} else if (event.getState() == Watcher.Event.KeeperState.Expired) {
    +            LOG.info("session expired. now rebuilding");
    +            // session expired, may be never happending. but if it happen there
    +            // need to close old client and rebuild new client
    +            try {
    +                connect(true);
    +            } catch (IOException e) {
    +                setSessionRecoverSuccessful(false);
    +                LOG.error("session expired and throw IOException while do reconnect: " + e.getMessage());
    +            } catch (InterruptedException e) {
    +                LOG.error("session expired and throw InterruptedException while do reconnect: " + e.getMessage());
    --- End diff --
    
    The same as above.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172737467
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/zookeeper/ZkClient.java ---
    @@ -176,7 +208,19 @@ public ZooKeeper getZk() {
     	public void process(WatchedEvent event) {
     		if(event.getState() == Watcher.Event.KeeperState.SyncConnected) {
     			connectedSignal.countDown();
    -		}
    +		} else if (event.getState() == Watcher.Event.KeeperState.Expired) {
    +            LOG.info("session expired. now rebuilding");
    +            // session expired, may be never happending. but if it happen there
    +            // need to close old client and rebuild new client
    +            try {
    +                connect(true);
    --- End diff --
    
    In connect(...), I saw it was catch for zookeeper exception, how can get exception here?


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736094
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/FloatingIp.java ---
    @@ -61,6 +61,43 @@ public boolean isEnabled() {
             return isEnabled;
         }
     
    +    public synchronized int unbindScript() throws Exception {
    +        if (isEnabled)
    +            LOG.info("Floating IP is enabled");
    +        else {
    +            LOG.info("Floating IP is disabled");
    +            return 0;
    +        }
    +
    +        ScriptContext scriptContext = new ScriptContext();
    +        scriptContext.setScriptName(Constants.SYS_SHELL_SCRIPT_NAME);
    +        scriptContext.setStripStdOut(false);
    +        scriptContext.setStripStdErr(false);
    +
    +        String command = master.getConfiguration().get(Constants.DCS_MASTER_FLOATING_IP_COMMAND_UNBIND,
    +                Constants.DEFAULT_DCS_MASTER_FLOATING_IP_COMMAND_UNBIND);
    +
    +        scriptContext.setCommand(command);
    +        LOG.info("Unbind Floating IP [" + scriptContext.getCommand() + "]");
    +        ScriptManager.getInstance().runScript(scriptContext);// Blocking call
    +
    +        StringBuilder sb = new StringBuilder();
    +        sb.append("exit code [" + scriptContext.getExitCode() + "]");
    +        if (!scriptContext.getStdOut().toString().isEmpty())
    +            sb.append(", stdout [" + scriptContext.getStdOut().toString() + "]");
    +        if (!scriptContext.getStdErr().toString().isEmpty())
    +            sb.append(", stderr [" + scriptContext.getStdErr().toString() + "]");
    +        if (LOG.isErrorEnabled())
    +            LOG.info(sb.toString());
    +
    +        if (scriptContext.getExitCode() == 0)
    +            LOG.info("Unbind Floating IP successful");
    +        else
    +            LOG.error("Unbind Floating IP failed");
    --- End diff --
    
    It's better to print exit code.


---

[GitHub] trafodion pull request #1427: TRAFODION-2940 In HA env, one node lose networ...

Posted by kevinxu021 <gi...@git.apache.org>.
Github user kevinxu021 commented on a diff in the pull request:

    https://github.com/apache/trafodion/pull/1427#discussion_r172736627
  
    --- Diff: dcs/src/main/java/org/trafodion/dcs/master/listener/ListenerWorker.java ---
    @@ -93,6 +93,7 @@ public void run() {
                             try {
                                 queue.wait();
                             } catch (InterruptedException e) {
    +                            return;
    --- End diff --
    
    Log message for what happened by debug level.


---