From 68604b4cd02e8cadc57552032d117f85ec89ca6b Mon Sep 17 00:00:00 2001 From: Viraj Jasani Date: Wed, 23 Jun 2021 08:51:34 +0530 Subject: [PATCH] HDFS-16082. Atomic operations on exceptionsSinceLastBalance and failedTimesSinceLastSuccessfulBalance in Balancer (#3127) (cherry picked from commit d9fbb3c5082baf301b082f51eea2f8a2e25e8715) --- .../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 8d97d2e1ab..998598ec12 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 @@ -35,6 +35,7 @@ import java.util.List; import java.util.Set; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdfs.DFSUtilClient; @@ -204,8 +205,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; @@ -266,11 +269,11 @@ static int getInt(Configuration conf, String key, int defaultValue) { } static int getExceptionsSinceLastBalance() { - return exceptionsSinceLastBalance; + return EXCEPTIONS_SINCE_LAST_BALANCE.get(); } static int getFailedTimesSinceLastSuccessfulBalance() { - return failedTimesSinceLastSuccessfulBalance; + return FAILED_TIMES_SINCE_LAST_SUCCESSFUL_BALANCE.get(); } /** @@ -816,20 +819,21 @@ static int run(Collection namenodes, Collection nsIds, 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