lokeshj1703 commented on code in PR #3272:
URL: https://github.com/apache/ozone/pull/3272#discussion_r857467692
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java:
##########
@@ -603,25 +585,46 @@ private boolean checkIterationLimits() {
*/
private boolean moveContainer(DatanodeDetails source,
ContainerMoveSelection moveSelection) {
- ContainerID container = moveSelection.getContainerID();
+ ContainerID containerID = moveSelection.getContainerID();
CompletableFuture<ReplicationManager.MoveResult> future;
try {
+ ContainerInfo containerInfo = containerManager.getContainer(containerID);
future = replicationManager
- .move(container, source, moveSelection.getTargetNode());
+ .move(containerID, source, moveSelection.getTargetNode())
+ .whenComplete((result, ex) -> {
+ if (ex != null) {
+ LOG.info("Container move for container {} from source {} to " +
+ "target {} failed with exceptions",
Review Comment:
missing {} after exceptions
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java:
##########
@@ -603,25 +585,46 @@ private boolean checkIterationLimits() {
*/
private boolean moveContainer(DatanodeDetails source,
ContainerMoveSelection moveSelection) {
- ContainerID container = moveSelection.getContainerID();
+ ContainerID containerID = moveSelection.getContainerID();
CompletableFuture<ReplicationManager.MoveResult> future;
try {
+ ContainerInfo containerInfo = containerManager.getContainer(containerID);
future = replicationManager
- .move(container, source, moveSelection.getTargetNode());
+ .move(containerID, source, moveSelection.getTargetNode())
+ .whenComplete((result, ex) -> {
+ if (ex != null) {
+ LOG.info("Container move for container {} from source {} to " +
+ "target {} failed with exceptions",
+ containerID.toString(),
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString(), ex);
+ } else {
+ if (result == ReplicationManager.MoveResult.COMPLETED) {
+ metrics.incrementDataSizeMovedGBInLatestIteration(
+ containerInfo.getUsedBytes() / OzoneConsts.GB);
+ metrics.incrementNumContainerMovesInLatestIteration(1);
+ LOG.info(
+ "Container move completed for container {} to target {}",
+ containerID,
+ moveSelection.getTargetNode().getUuidString());
+ } else {
+ LOG.debug(
+ "Container move for container {} to target {} failed: {}",
+ moveSelection.getContainerID(),
+ moveSelection.getTargetNode().getUuidString(), result);
+ }
+ }
+ });
} catch (ContainerNotFoundException e) {
- LOG.warn("Could not find Container {} for container move", container, e);
+ LOG.warn("Could not find Container {} for container move",
+ containerID, e);
return false;
} catch (NodeNotFoundException e) {
- LOG.warn("Container move failed for container {}", container, e);
+ LOG.warn("Container move failed for container {}", containerID, e);
return false;
}
if (future.isDone()) {
Review Comment:
Let's add an empty line before this.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java:
##########
@@ -462,49 +462,80 @@ private void checkIterationResults(boolean
isMoveGenerated,
private void checkIterationMoveResults(Set<DatanodeDetails> selectedTargets)
{
this.countDatanodesInvolvedPerIteration = 0;
this.sizeMovedPerIteration = 0;
- for (Map.Entry<ContainerMoveSelection,
- CompletableFuture<ReplicationManager.MoveResult>>
- futureEntry : moveSelectionToFutureMap.entrySet()) {
- ContainerMoveSelection moveSelection = futureEntry.getKey();
- CompletableFuture<ReplicationManager.MoveResult> future =
- futureEntry.getValue();
+
+ CompletableFuture<Void> allFuturesResult = CompletableFuture.allOf(
+ moveSelectionToFutureMap.values()
+ .toArray(new CompletableFuture[moveSelectionToFutureMap.size()]));
+ try {
+ allFuturesResult.get(config.getMoveTimeout().toMillis(),
+ TimeUnit.MILLISECONDS);
+ } catch (InterruptedException e) {
+ Thread.currentThread().interrupt();
+ } catch (Exception e) {
+ // For exceptions other than InterruptedException, we ignore them.
+ e.printStackTrace();
Review Comment:
This still needs to be handled in the latest commit.
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java:
##########
@@ -44,6 +44,10 @@ public final class ContainerBalancerMetrics {
"in the latest iteration.")
private MutableCounterLong numContainerMovesInLatestIteration;
+ @Metric(about = "Number of timeout container moves performed by " +
+ "Container Balancer in the latest iteration.")
+ private MutableCounterLong numTimeoutContainerMovesInLatestIteration;
Review Comment:
Can we rename this to `numContainerMovesTimeoutInLatestIteration` and
`numContainerMovesCompletedInLatestIteration`?
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java:
##########
@@ -603,25 +585,46 @@ private boolean checkIterationLimits() {
*/
private boolean moveContainer(DatanodeDetails source,
ContainerMoveSelection moveSelection) {
- ContainerID container = moveSelection.getContainerID();
+ ContainerID containerID = moveSelection.getContainerID();
CompletableFuture<ReplicationManager.MoveResult> future;
try {
+ ContainerInfo containerInfo = containerManager.getContainer(containerID);
future = replicationManager
- .move(container, source, moveSelection.getTargetNode());
+ .move(containerID, source, moveSelection.getTargetNode())
+ .whenComplete((result, ex) -> {
+ if (ex != null) {
+ LOG.info("Container move for container {} from source {} to " +
+ "target {} failed with exceptions",
+ containerID.toString(),
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString(), ex);
+ } else {
+ if (result == ReplicationManager.MoveResult.COMPLETED) {
+ metrics.incrementDataSizeMovedGBInLatestIteration(
+ containerInfo.getUsedBytes() / OzoneConsts.GB);
+ metrics.incrementNumContainerMovesInLatestIteration(1);
+ LOG.info(
Review Comment:
We can make this debug and wrap it inside LOG.isDebugEnabled check.
```
if (LOG.isDebugEnabled) {
LOG.info...
}
```
##########
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java:
##########
@@ -603,25 +585,46 @@ private boolean checkIterationLimits() {
*/
private boolean moveContainer(DatanodeDetails source,
ContainerMoveSelection moveSelection) {
- ContainerID container = moveSelection.getContainerID();
+ ContainerID containerID = moveSelection.getContainerID();
CompletableFuture<ReplicationManager.MoveResult> future;
try {
+ ContainerInfo containerInfo = containerManager.getContainer(containerID);
future = replicationManager
- .move(container, source, moveSelection.getTargetNode());
+ .move(containerID, source, moveSelection.getTargetNode())
+ .whenComplete((result, ex) -> {
+ if (ex != null) {
+ LOG.info("Container move for container {} from source {} to " +
+ "target {} failed with exceptions",
+ containerID.toString(),
+ source.getUuidString(),
+ moveSelection.getTargetNode().getUuidString(), ex);
+ } else {
+ if (result == ReplicationManager.MoveResult.COMPLETED) {
+ metrics.incrementDataSizeMovedGBInLatestIteration(
+ containerInfo.getUsedBytes() / OzoneConsts.GB);
+ metrics.incrementNumContainerMovesInLatestIteration(1);
+ LOG.info(
+ "Container move completed for container {} to target {}",
+ containerID,
+ moveSelection.getTargetNode().getUuidString());
+ } else {
+ LOG.debug(
Review Comment:
We can make this info or warn.
##########
hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancer.java:
##########
@@ -642,6 +642,37 @@ public void checkIterationResult()
containerBalancer.stop();
}
+ @Test
+ public void checkIterationResultTimeout()
+ throws NodeNotFoundException, ContainerNotFoundException {
+
+ Mockito.when(replicationManager.move(Mockito.any(ContainerID.class),
+ Mockito.any(DatanodeDetails.class),
+ Mockito.any(DatanodeDetails.class)))
+ .thenReturn(genCompletableFuture(500), genCompletableFuture(2000));
+
+ balancerConfiguration.setThreshold(10);
+ balancerConfiguration.setIterations(1);
+ balancerConfiguration.setMaxSizeEnteringTarget(10 * OzoneConsts.GB);
+ balancerConfiguration.setMaxSizeToMovePerIteration(100 * OzoneConsts.GB);
+ balancerConfiguration.setMaxDatanodesPercentageToInvolvePerIteration(100);
+ balancerConfiguration.setMoveTimeout(Duration.ofMillis(1000));
+
+ startBalancer(balancerConfiguration);
+ sleepWhileBalancing(2000);
+
+ /*
+ According to the setup and configurations, this iteration's result should
+ be ITERATION_COMPLETED.
+ */
+ Assert.assertEquals(ContainerBalancer.IterationResult.ITERATION_COMPLETED,
+ containerBalancer.getIterationResult());
+ Assert.assertEquals(1,
+
containerBalancer.getMetrics().getNumContainerMovesInLatestIteration());
Review Comment:
Can we also add assertion for timed out metrics?
--
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.
To unsubscribe, e-mail: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]