You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@helix.apache.org by GitBox <gi...@apache.org> on 2020/10/13 02:35:47 UTC

[GitHub] [helix] jiajunwang commented on a change in pull request #1456: Fix Periodic rebalancer Timer leak

jiajunwang commented on a change in pull request #1456:
URL: https://github.com/apache/helix/pull/1456#discussion_r503627199



##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -172,11 +173,15 @@
   private boolean _inMaintenanceMode;
 
   /**
-   * The timer that can periodically run the rebalancing pipeline. The timer will start if there is
-   * one resource group has the config to use the timer.
+   * The executors that can periodically run the rebalancing pipeline. A
+   * SingleThreadScheduledExecutor will start if there is one resource group has the config to do
+   * periodically rebalance.
    */
-  Timer _periodicalRebalanceTimer = null;
+  private static final ScheduledExecutorService _periodicalRebalanceExecutor =
+      Executors.newSingleThreadScheduledExecutor();
+  private ScheduledFuture _periodicRebalancerFutureTasks = null;
   long _timerPeriod = Long.MAX_VALUE;
+  private final Object _lock = new Object();

Review comment:
       Just lock the _periodicalRebalanceExecutor?
   &
   Have you tried the atomic reference solution that we discussed? I think that one is easier.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -333,15 +338,19 @@ private void forceRebalance(HelixManager manager, ClusterEventType eventType) {
   void startPeriodRebalance(long period, HelixManager manager) {
     if (period != _timerPeriod) {

Review comment:
       I think this check needs to be included in the critical section too.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -333,15 +338,19 @@ private void forceRebalance(HelixManager manager, ClusterEventType eventType) {
   void startPeriodRebalance(long period, HelixManager manager) {
     if (period != _timerPeriod) {
       logger.info("Controller starting periodical rebalance timer at period " + period);
-      if (_periodicalRebalanceTimer != null) {
-        _periodicalRebalanceTimer.cancel();
+      ScheduledFuture lastScheduledFuture = null;
+      synchronized (_lock) {
+        if (_periodicRebalancerFutureTasks!=null && !_periodicRebalancerFutureTasks.isCancelled()) {

Review comment:
       Is this check necessary?

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -1299,6 +1308,10 @@ protected void checkLiveInstancesObservation(List<LiveInstance> liveInstances,
   }
 
   public void shutdown() throws InterruptedException {
+    if (_periodicRebalancerFutureTasks != null) {
+      _periodicRebalancerFutureTasks.cancel(false);
+    }

Review comment:
       Why need to cancel the task while stopPeriodRebalance is called here? 

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -333,15 +338,19 @@ private void forceRebalance(HelixManager manager, ClusterEventType eventType) {
   void startPeriodRebalance(long period, HelixManager manager) {
     if (period != _timerPeriod) {
       logger.info("Controller starting periodical rebalance timer at period " + period);
-      if (_periodicalRebalanceTimer != null) {
-        _periodicalRebalanceTimer.cancel();
+      ScheduledFuture lastScheduledFuture = null;
+      synchronized (_lock) {
+        if (_periodicRebalancerFutureTasks!=null && !_periodicRebalancerFutureTasks.isCancelled()) {
+          lastScheduledFuture = _periodicRebalancerFutureTasks;
+        }
+        _timerPeriod = period;
+        _periodicRebalancerFutureTasks = _periodicalRebalanceExecutor
+            .scheduleAtFixedRate(new RebalanceTask(manager, ClusterEventType.PeriodicalRebalance),
+                _timerPeriod, _timerPeriod, TimeUnit.MILLISECONDS);
+      }
+      if (lastScheduledFuture != null) {
+        lastScheduledFuture.cancel(true /* mayInterruptIfRunning */);

Review comment:
       This changes the logic. Keep it as it was, please.
   Interrupt it may cause some unexpected result. One example, the newly scheduled rebalance is scheduled with a delay. If you cancel the previous one by interrupt, then there might be a very long period that no rebalance is triggered.

##########
File path: helix-core/src/main/java/org/apache/helix/controller/GenericHelixController.java
##########
@@ -352,11 +361,11 @@ void startPeriodRebalance(long period, HelixManager manager) {
    */
   void stopPeriodRebalance() {
     logger.info("Controller stopping periodical rebalance timer at period " + _timerPeriod);
-    if (_periodicalRebalanceTimer != null) {
-      _periodicalRebalanceTimer.cancel();
-      _periodicalRebalanceTimer = null;
-      _timerPeriod = Long.MAX_VALUE;
-      logger.info("Controller stopped periodical rebalance timer at period " + _timerPeriod);
+    synchronized (_lock) {
+      if (_periodicRebalancerFutureTasks != null && !_periodicRebalancerFutureTasks.isCancelled()) {
+        _periodicRebalancerFutureTasks.cancel(true /* mayInterruptIfRunning */);

Review comment:
       Same here.




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

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



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@helix.apache.org
For additional commands, e-mail: reviews-help@helix.apache.org