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

jackie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-pinot.git


The following commit(s) were added to refs/heads/master by this push:
     new 88081bd  In ClusterChangeMediator, add stop flag check inside for loop 
and wait block (#4121)
88081bd is described below

commit 88081bd2730499e2f4af4a7ee2927bda1986777f
Author: Xiaotian (Jackie) Jiang <[email protected]>
AuthorDate: Tue Apr 16 14:00:41 2019 -0700

    In ClusterChangeMediator, add stop flag check inside for loop and wait 
block (#4121)
    
    1. Add stop flag check inside for loop to not wait until all change types 
getting processed
    2. Add stop flag check into wait block to prevent thread wait without 
notify() called
---
 .../apache/pinot/broker/broker/helix/ClusterChangeMediator.java   | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)

diff --git 
a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/ClusterChangeMediator.java
 
b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/ClusterChangeMediator.java
index 25d5ff2..64b2a1e 100644
--- 
a/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/ClusterChangeMediator.java
+++ 
b/pinot-broker/src/main/java/org/apache/pinot/broker/broker/helix/ClusterChangeMediator.java
@@ -77,9 +77,12 @@ public class ClusterChangeMediator implements 
ExternalViewChangeListener, Instan
     _clusterChangeHandlingThread = new Thread("ClusterChangeHandlingThread") {
       @Override
       public void run() {
-        while (!_stopped) {
+        while (true) {
           try {
             for (Map.Entry<ChangeType, List<ClusterChangeHandler>> entry : 
_changeHandlersMap.entrySet()) {
+              if (_stopped) {
+                return;
+              }
               ChangeType changeType = entry.getKey();
               List<ClusterChangeHandler> changeHandlers = entry.getValue();
               long currentTime = System.currentTimeMillis();
@@ -101,6 +104,9 @@ public class ClusterChangeMediator implements 
ExternalViewChangeListener, Instan
               }
             }
             synchronized (_lastChangeTimeMap) {
+              if (_stopped) {
+                return;
+              }
               // Wait for at most 1/10 of proactive change check interval if 
no new event received. This can guarantee
               // that the proactive change check will not be delayed for more 
than 1/10 of the interval. In case of
               // spurious wakeup, execute the while loop again for the 
proactive change check.


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to