You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2021/06/23 03:22:19 UTC

[hadoop] branch trunk updated: HDFS-16082. Atomic operations on exceptionsSinceLastBalance and failedTimesSinceLastSuccessfulBalance in Balancer (#3127)

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

weichiu pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new d9fbb3c  HDFS-16082. Atomic operations on exceptionsSinceLastBalance and failedTimesSinceLastSuccessfulBalance in Balancer (#3127)
d9fbb3c is described below

commit d9fbb3c5082baf301b082f51eea2f8a2e25e8715
Author: Viraj Jasani <vj...@apache.org>
AuthorDate: Wed Jun 23 08:51:34 2021 +0530

    HDFS-16082. Atomic operations on exceptionsSinceLastBalance and failedTimesSinceLastSuccessfulBalance in Balancer (#3127)
---
 .../hadoop/hdfs/server/balancer/Balancer.java      | 22 +++++++++++++---------
 1 file changed, 13 insertions(+), 9 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
index 33650ea..73a8e68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java
@@ -37,6 +37,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.lang3.builder.ToStringBuilder;
 import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting;
@@ -211,8 +212,10 @@ public class Balancer {
   @VisibleForTesting
   private static volatile boolean serviceRunning = false;
 
-  private static volatile int exceptionsSinceLastBalance = 0;
-  private static volatile int failedTimesSinceLastSuccessfulBalance = 0;
+  private static final AtomicInteger EXCEPTIONS_SINCE_LAST_BALANCE =
+      new AtomicInteger(0);
+  private static final AtomicInteger
+      FAILED_TIMES_SINCE_LAST_SUCCESSFUL_BALANCE = new AtomicInteger(0);
 
   private final Dispatcher dispatcher;
   private final NameNodeConnector nnc;
@@ -274,11 +277,11 @@ public class Balancer {
   }
 
   static int getExceptionsSinceLastBalance() {
-    return exceptionsSinceLastBalance;
+    return EXCEPTIONS_SINCE_LAST_BALANCE.get();
   }
 
   static int getFailedTimesSinceLastSuccessfulBalance() {
-    return failedTimesSinceLastSuccessfulBalance;
+    return FAILED_TIMES_SINCE_LAST_SUCCESSFUL_BALANCE.get();
   }
 
   /**
@@ -866,20 +869,21 @@ public class Balancer {
         int retCode = doBalance(namenodes, nsIds, p, conf);
         if (retCode < 0) {
           LOG.info("Balance failed, error code: " + retCode);
-          failedTimesSinceLastSuccessfulBalance++;
+          FAILED_TIMES_SINCE_LAST_SUCCESSFUL_BALANCE.incrementAndGet();
         } else {
           LOG.info("Balance succeed!");
-          failedTimesSinceLastSuccessfulBalance = 0;
+          FAILED_TIMES_SINCE_LAST_SUCCESSFUL_BALANCE.set(0);
         }
-        exceptionsSinceLastBalance = 0;
+        EXCEPTIONS_SINCE_LAST_BALANCE.set(0);
       } catch (Exception e) {
-        if (++exceptionsSinceLastBalance > retryOnException) {
+        if (EXCEPTIONS_SINCE_LAST_BALANCE.incrementAndGet()
+            > retryOnException) {
           // The caller will process and log the exception
           throw e;
         }
         LOG.warn(
             "Encounter exception while do balance work. Already tried {} times",
-            exceptionsSinceLastBalance, e);
+            EXCEPTIONS_SINCE_LAST_BALANCE, e);
       }
 
       // sleep for next round, will retry for next round when it's interrupted

---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org