From 7467c402c06b65d717474a90b469409c7cc0dc19 Mon Sep 17 00:00:00 2001 From: Siddhant Sangwan Date: Mon, 18 Jul 2022 19:08:50 +0530 Subject: [PATCH 1/3] HDDS-6976. 0GB data moved by container balancer after successful iteration --- .../container/balancer/ContainerBalancer.java | 26 +++++---- .../balancer/TestContainerBalancer.java | 58 ++++++++++--------- 2 files changed, 46 insertions(+), 38 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index a29be341b08e..63dcc64c667e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -82,7 +82,9 @@ public class ContainerBalancer extends StatefulService { private double maxDatanodesRatioToInvolvePerIteration; private long maxSizeToMovePerIteration; private int countDatanodesInvolvedPerIteration; - private long sizeMovedPerIteration; + private long sizeScheduledForMoveInLatestIteration; + // count actual size moved in bytes + private long sizeActuallyMovedInLatestIteration; private int iterations; private List unBalancedNodes; private List overUtilizedNodes; @@ -536,12 +538,14 @@ private void checkIterationMoveResults(Set selectedTargets) { metrics.getNumContainerMovesCompletedInLatestIteration()); metrics.incrementNumContainerMovesTimeout( metrics.getNumContainerMovesTimeoutInLatestIteration()); + metrics.incrementDataSizeMovedGBInLatestIteration( + sizeActuallyMovedInLatestIteration / OzoneConsts.GB); metrics.incrementDataSizeMovedGB( metrics.getDataSizeMovedGBInLatestIteration()); LOG.info("Number of datanodes involved in this iteration: {}. Size moved " + "in this iteration: {}GB.", countDatanodesInvolvedPerIteration, - metrics.getDataSizeMovedGBInLatestIteration()); + sizeActuallyMovedInLatestIteration / (double) OzoneConsts.GB); } /** @@ -601,14 +605,15 @@ private boolean checkIterationLimits() { } return true; } - if (sizeMovedPerIteration + (long) ozoneConfiguration.getStorageSize( + if (sizeScheduledForMoveInLatestIteration + + (long) ozoneConfiguration.getStorageSize( ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE, ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES) > maxSizeToMovePerIteration) { if (LOG.isDebugEnabled()) { LOG.debug("Hit max size to move limit. {} bytes have already been " + "scheduled for balancing and the limit is {} bytes.", - sizeMovedPerIteration, + sizeScheduledForMoveInLatestIteration, maxSizeToMovePerIteration); } return true; @@ -645,8 +650,8 @@ private boolean moveContainer(DatanodeDetails source, moveSelection.getTargetNode().getUuidString(), ex); } else { if (result == LegacyReplicationManager.MoveResult.COMPLETED) { - metrics.incrementDataSizeMovedGBInLatestIteration( - containerInfo.getUsedBytes() / OzoneConsts.GB); + sizeActuallyMovedInLatestIteration += + containerInfo.getUsedBytes(); if (LOG.isDebugEnabled()) { LOG.debug( "Container move completed for container {} to target {}", @@ -812,7 +817,7 @@ private void incSizeSelectedForMoving(DatanodeDetails source, return; } long size = container.getUsedBytes(); - sizeMovedPerIteration += size; + sizeScheduledForMoveInLatestIteration += size; // update sizeLeavingNode map with the recent moveSelection findSourceStrategy.increaseSizeLeaving(source, size); @@ -833,7 +838,8 @@ private void resetState() { this.underUtilizedNodes.clear(); this.unBalancedNodes.clear(); this.countDatanodesInvolvedPerIteration = 0; - this.sizeMovedPerIteration = 0; + this.sizeScheduledForMoveInLatestIteration = 0; + this.sizeActuallyMovedInLatestIteration = 0; metrics.resetDataSizeMovedGBInLatestIteration(); metrics.resetNumContainerMovesCompletedInLatestIteration(); metrics.resetNumContainerMovesTimeoutInLatestIteration(); @@ -1262,8 +1268,8 @@ int getCountDatanodesInvolvedPerIteration() { } @VisibleForTesting - public long getSizeMovedPerIteration() { - return sizeMovedPerIteration; + public long getSizeScheduledForMoveInLatestIteration() { + return sizeScheduledForMoveInLatestIteration; } public ContainerBalancerMetrics getMetrics() { diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java index a6a11d8f440e..59ea9d2bd810 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java @@ -103,6 +103,7 @@ public class TestContainerBalancer { private static final ThreadLocalRandom RANDOM = ThreadLocalRandom.current(); private StatefulServiceStateManager serviceStateManager; + private static final long STORAGE_UNIT = OzoneConsts.GB; /** * Sets up configuration values and creates a mock cluster. @@ -123,8 +124,8 @@ public void setup() throws IOException, NodeNotFoundException, balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeEnteringTarget(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); conf.setFromObject(balancerConfiguration); GenericTestUtils.setLogLevel(ContainerBalancer.LOG, Level.DEBUG); @@ -282,7 +283,7 @@ public void containerBalancerShouldObeyMaxDatanodesToInvolveLimit() int percent = 20; balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration( percent); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setThreshold(1); balancerConfiguration.setIterations(1); startBalancer(balancerConfiguration); @@ -347,15 +348,16 @@ public void containerBalancerShouldObeyMaxSizeToMoveLimit() throws IllegalContainerBalancerStateException, IOException, InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(1); - balancerConfiguration.setMaxSizeToMovePerIteration(10 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT); balancerConfiguration.setIterations(1); startBalancer(balancerConfiguration); sleepWhileBalancing(500); // balancer should not have moved more size than the limit - Assertions.assertFalse(containerBalancer.getSizeMovedPerIteration() > - 10 * OzoneConsts.GB); + Assertions.assertFalse( + containerBalancer.getSizeScheduledForMoveInLatestIteration() > + 10 * STORAGE_UNIT); long size = containerBalancer.getMetrics() .getDataSizeMovedGBInLatestIteration(); @@ -369,7 +371,7 @@ public void targetDatanodeShouldNotAlreadyContainSelectedContainer() throws IllegalContainerBalancerStateException, IOException, InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(10); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); startBalancer(balancerConfiguration); @@ -398,7 +400,7 @@ public void containerMoveSelectionShouldFollowPlacementPolicy() throws IllegalContainerBalancerStateException, IOException, InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(10); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); startBalancer(balancerConfiguration); @@ -443,8 +445,8 @@ public void targetDatanodeShouldBeInServiceHealthy() TimeoutException { balancerConfiguration.setThreshold(10); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeEnteringTarget(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); startBalancer(balancerConfiguration); // waiting for balance completed. @@ -472,8 +474,8 @@ public void selectedContainerShouldNotAlreadyHaveBeenSelected() InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(10); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeEnteringTarget(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); startBalancer(balancerConfiguration); @@ -500,8 +502,8 @@ public void balancerShouldNotSelectConfiguredExcludeContainers() InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(10); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeEnteringTarget(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeEnteringTarget(50 * STORAGE_UNIT); balancerConfiguration.setExcludeContainers("1, 4, 5"); startBalancer(balancerConfiguration); @@ -532,7 +534,7 @@ public void balancerShouldObeyMaxSizeEnteringTargetLimit() conf.getObject(ContainerBalancerConfiguration.class); balancerConfiguration.setThreshold(10); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); - balancerConfiguration.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(50 * STORAGE_UNIT); // no containers should be selected when the limit is just 2 MB balancerConfiguration.setMaxSizeEnteringTarget(2 * OzoneConsts.MB); @@ -565,9 +567,9 @@ public void testMetrics() balancerConfiguration.setBalancingInterval(Duration.ofMillis(2)); balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(6 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeEnteringTarget(6 * STORAGE_UNIT); // deliberately set max size per iteration to a low value, 6 GB - balancerConfiguration.setMaxSizeToMovePerIteration(6 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(6 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); startBalancer(balancerConfiguration); @@ -596,8 +598,8 @@ public void balancerShouldFollowExcludeAndIncludeDatanodesConfigurations() InvalidContainerBalancerConfigurationException, TimeoutException { balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); // only these nodes should be included @@ -670,8 +672,8 @@ public void checkIterationResult() TimeoutException { balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); startBalancer(balancerConfiguration); @@ -695,7 +697,7 @@ public void checkIterationResult() Mockito.any(DatanodeDetails.class))) .thenReturn(CompletableFuture.completedFuture( MoveResult.REPLICATION_FAIL_NODE_UNHEALTHY)); - balancerConfiguration.setMaxSizeToMovePerIteration(10 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeToMovePerIteration(10 * STORAGE_UNIT); startBalancer(balancerConfiguration); sleepWhileBalancing(1000); @@ -720,8 +722,8 @@ public void checkIterationResultTimeout() balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); balancerConfiguration.setMoveTimeout(Duration.ofMillis(1000)); @@ -762,8 +764,8 @@ public void checkIterationResultTimeoutFromReplicationManager() balancerConfiguration.setThreshold(10); balancerConfiguration.setIterations(1); - balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB); - balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB); + balancerConfiguration.setMaxSizeEnteringTarget(10 * STORAGE_UNIT); + balancerConfiguration.setMaxSizeToMovePerIteration(100 * STORAGE_UNIT); balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100); balancerConfiguration.setMoveTimeout(Duration.ofMillis(1000)); @@ -857,7 +859,7 @@ private double createCluster() { // use node utilization and used space to determine node capacity if (nodeUtilizations.get(i) == 0) { - datanodeCapacity = OzoneConsts.GB * RANDOM.nextInt(10, 60); + datanodeCapacity = STORAGE_UNIT * RANDOM.nextInt(10, 60); } else { datanodeCapacity = (long) (datanodeUsedSpace / nodeUtilizations.get(i)); } @@ -914,7 +916,7 @@ private ContainerInfo createContainer(long id, int multiple) { .getInstance(HddsProtos.ReplicationFactor.THREE)) .setState(HddsProtos.LifeCycleState.CLOSED) .setOwner("TestContainerBalancer") - .setUsedBytes(OzoneConsts.GB * multiple) + .setUsedBytes(STORAGE_UNIT * multiple) .build(); } From cb5981d49edc44a725b15be738e9450e55fac934 Mon Sep 17 00:00:00 2001 From: Siddhant Sangwan Date: Tue, 19 Jul 2022 15:58:48 +0530 Subject: [PATCH 2/3] log number of container moves and represent size in bytes at the end of iteration --- .../hdds/scm/container/balancer/ContainerBalancer.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 63dcc64c667e..cb522812533e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -542,10 +542,12 @@ private void checkIterationMoveResults(Set selectedTargets) { sizeActuallyMovedInLatestIteration / OzoneConsts.GB); metrics.incrementDataSizeMovedGB( metrics.getDataSizeMovedGBInLatestIteration()); - LOG.info("Number of datanodes involved in this iteration: {}. Size moved " + - "in this iteration: {}GB.", + LOG.info("Iteration Summary. Number of Datanodes involved: {}. Size " + + "moved: {}GB ({} Bytes). Number of Container moves completed: {}.", countDatanodesInvolvedPerIteration, - sizeActuallyMovedInLatestIteration / (double) OzoneConsts.GB); + sizeActuallyMovedInLatestIteration / (double) OzoneConsts.GB, + sizeActuallyMovedInLatestIteration, + metrics.getNumContainerMovesCompletedInLatestIteration()); } /** From 97359a2377be4b7d200f71876601ae7480239159 Mon Sep 17 00:00:00 2001 From: Siddhant Sangwan Date: Wed, 20 Jul 2022 12:10:10 +0530 Subject: [PATCH 3/3] do appropriate size unit conversion in logs --- .../hdds/scm/container/balancer/ContainerBalancer.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index cb522812533e..61da6052d115 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -41,6 +41,7 @@ import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.hadoop.util.StringUtils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -543,9 +544,9 @@ private void checkIterationMoveResults(Set selectedTargets) { metrics.incrementDataSizeMovedGB( metrics.getDataSizeMovedGBInLatestIteration()); LOG.info("Iteration Summary. Number of Datanodes involved: {}. Size " + - "moved: {}GB ({} Bytes). Number of Container moves completed: {}.", + "moved: {} ({} Bytes). Number of Container moves completed: {}.", countDatanodesInvolvedPerIteration, - sizeActuallyMovedInLatestIteration / (double) OzoneConsts.GB, + StringUtils.byteDesc(sizeActuallyMovedInLatestIteration), sizeActuallyMovedInLatestIteration, metrics.getNumContainerMovesCompletedInLatestIteration()); }