From 774fbbced6eff9f0f0ca9a38c45069fd58e999a8 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 30 Aug 2024 16:28:10 +0300 Subject: [PATCH 01/39] HDDS-11367. Improve ozone balancing status command output --- .../src/main/proto/ScmAdminProtocol.proto | 11 +-- .../balancer/ContainerBalancerMetrics.java | 22 ++++++ .../balancer/ContainerBalancerTask.java | 38 ++++++---- ...tainerBalancerTaskIterationStatusInfo.java | 47 ++++++------ .../scm/server/SCMClientProtocolServer.java | 17 ++--- .../TestContainerBalancerStatusInfo.java | 4 +- .../ContainerBalancerStatusSubcommand.java | 55 +++++++++++--- .../TestContainerBalancerSubCommand.java | 72 ++++++++++--------- .../smoketest/balancer/testBalancer.robot | 4 ++ 9 files changed, 178 insertions(+), 92 deletions(-) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index 039914369b5f..47344ba076bb 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -627,19 +627,20 @@ message ContainerBalancerStatusInfo { message ContainerBalancerTaskIterationStatusInfo { optional int32 iterationNumber = 1; optional string iterationResult = 2; - optional int64 sizeScheduledForMoveGB = 3; - optional int64 dataSizeMovedGB = 4; + optional int64 sizeScheduledForMove = 3; + optional int64 dataSizeMoved = 4; optional int64 containerMovesScheduled = 5; optional int64 containerMovesCompleted = 6; optional int64 containerMovesFailed = 7; optional int64 containerMovesTimeout = 8; - repeated NodeTransferInfo sizeEnteringNodesGB = 9; - repeated NodeTransferInfo sizeLeavingNodesGB = 10; + repeated NodeTransferInfo sizeEnteringNodes = 9; + repeated NodeTransferInfo sizeLeavingNodes = 10; + optional int64 iterationDuration = 11; } message NodeTransferInfo { optional string uuid = 1; - optional int64 dataVolumeGB = 2; + optional int64 dataVolume = 2; } message DecommissionScmRequestProto { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index 6446089db353..ea1ea7042ddb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -40,6 +40,10 @@ public final class ContainerBalancerMetrics { " in the latest iteration.") private MutableCounterLong dataSizeMovedGBInLatestIteration; + @Metric(about = "Amount of bytes that Container Balancer moved" + + " in the latest iteration.") + private MutableCounterLong dataSizeMovedInLatestIteration; + @Metric(about = "Number of completed container moves performed by " + "Container Balancer in the latest iteration.") private MutableCounterLong numContainerMovesCompletedInLatestIteration; @@ -154,6 +158,24 @@ public void resetDataSizeMovedGBInLatestIteration() { -getDataSizeMovedGBInLatestIteration()); } + /** + * Gets the amount of data moved by Container Balancer in the latest + * iteration. + * @return size in bytes + */ + public long getDataSizeMovedInLatestIteration() { + return dataSizeMovedInLatestIteration.value(); + } + + public void incrementDataSizeMovedInLatestIteration(long valueToAdd) { + this.dataSizeMovedInLatestIteration.incr(valueToAdd); + } + + public void resetDataSizeMovedInLatestIteration() { + dataSizeMovedInLatestIteration.incr( + -getDataSizeMovedInLatestIteration()); + } + /** * Gets the number of container moves performed by Container Balancer in the * latest iteration. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 7fea44671ffa..437050f873c1 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -43,6 +43,7 @@ import java.io.IOException; import java.time.Duration; +import java.time.OffsetDateTime; import java.util.ArrayList; import java.util.Collection; import java.util.Collections; @@ -60,6 +61,7 @@ import java.util.concurrent.TimeoutException; import java.util.stream.Collectors; +import static java.time.OffsetDateTime.now; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT; @@ -118,7 +120,7 @@ public class ContainerBalancerTask implements Runnable { private int nextIterationIndex; private boolean delayStart; private List iterationsStatistic; - + private OffsetDateTime currentIterationStarted; /** * Constructs ContainerBalancerTask with the specified arguments. * @@ -207,6 +209,7 @@ private void balance() { // leader change or restart int i = nextIterationIndex; for (; i < iterations && isBalancerRunning(); i++) { + currentIterationStarted = now(); // reset some variables and metrics for this iteration resetState(); if (config.getTriggerDuEnable()) { @@ -302,8 +305,9 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( iterationNumber, iR.name(), - getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, - metrics.getDataSizeMovedGBInLatestIteration(), + now().toEpochSecond() - currentIterationStarted.toEpochSecond(), + getSizeScheduledForMoveInLatestIteration(), + metrics.getDataSizeMovedInLatestIteration(), metrics.getNumContainerMovesScheduledInLatestIteration(), metrics.getNumContainerMovesCompletedInLatestIteration(), metrics.getNumContainerMovesFailedInLatestIteration(), @@ -316,7 +320,7 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) .collect( Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB + Map.Entry::getValue ) ), findSourceStrategy.getSizeLeavingNodes() @@ -327,7 +331,7 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) .collect( Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB + entry -> entry.getValue() ) ) ); @@ -340,12 +344,13 @@ public List getCurrentIterationsStatis .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) .max() .orElse(0); - + long iterationDuration = getCurrentIterationDuration(); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber + 1, null, - getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, - sizeActuallyMovedInLatestIteration / OzoneConsts.GB, + iterationDuration, + getSizeScheduledForMoveInLatestIteration(), + sizeActuallyMovedInLatestIteration, metrics.getNumContainerMovesScheduledInLatestIteration(), metrics.getNumContainerMovesCompletedInLatestIteration(), metrics.getNumContainerMovesFailedInLatestIteration(), @@ -357,7 +362,7 @@ public List getCurrentIterationsStatis .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) .collect(Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB + entry -> entry.getValue() ) ), findSourceStrategy.getSizeLeavingNodes() @@ -368,7 +373,7 @@ public List getCurrentIterationsStatis .collect( Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB + entry -> entry.getValue() ) ) ); @@ -377,6 +382,14 @@ public List getCurrentIterationsStatis return resultList; } + private long getCurrentIterationDuration() { + if (currentIterationStarted == null) { + return -1L; + } else { + return now().toEpochSecond() - currentIterationStarted.toEpochSecond(); + } + } + /** * Logs the reason for stop and save configuration and stop the task. * @@ -717,8 +730,8 @@ private void checkIterationMoveResults() { metrics.getNumContainerMovesCompletedInLatestIteration()); metrics.incrementNumContainerMovesTimeout( metrics.getNumContainerMovesTimeoutInLatestIteration()); - metrics.incrementDataSizeMovedGBInLatestIteration( - sizeActuallyMovedInLatestIteration / OzoneConsts.GB); + metrics.incrementDataSizeMovedInLatestIteration( + sizeActuallyMovedInLatestIteration); metrics.incrementDataSizeMovedGB( metrics.getDataSizeMovedGBInLatestIteration()); metrics.incrementNumContainerMovesFailed( @@ -1145,6 +1158,7 @@ private void resetState() { this.sizeScheduledForMoveInLatestIteration = 0; this.sizeActuallyMovedInLatestIteration = 0; metrics.resetDataSizeMovedGBInLatestIteration(); + metrics.resetDataSizeMovedInLatestIteration(); metrics.resetNumContainerMovesScheduledInLatestIteration(); metrics.resetNumContainerMovesCompletedInLatestIteration(); metrics.resetNumContainerMovesTimeoutInLatestIteration(); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 1d597b0ca273..71038fe4d90e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -27,37 +27,40 @@ public class ContainerBalancerTaskIterationStatusInfo { private final Integer iterationNumber; private final String iterationResult; - private final long sizeScheduledForMoveGB; - private final long dataSizeMovedGB; + private final long iterationDuration; + private final long sizeScheduledForMove; + private final long dataSizeMoved; private final long containerMovesScheduled; private final long containerMovesCompleted; private final long containerMovesFailed; private final long containerMovesTimeout; - private final Map sizeEnteringNodesGB; - private final Map sizeLeavingNodesGB; + private final Map sizeEnteringNodes; + private final Map sizeLeavingNodes; @SuppressWarnings("checkstyle:ParameterNumber") public ContainerBalancerTaskIterationStatusInfo( Integer iterationNumber, String iterationResult, - long sizeScheduledForMoveGB, - long dataSizeMovedGB, + long iterationDuration, + long sizeScheduledForMove, + long dataSizeMoved, long containerMovesScheduled, long containerMovesCompleted, long containerMovesFailed, long containerMovesTimeout, - Map sizeEnteringNodesGB, - Map sizeLeavingNodesGB) { + Map sizeEnteringNodes, + Map sizeLeavingNodes) { this.iterationNumber = iterationNumber; this.iterationResult = iterationResult; - this.sizeScheduledForMoveGB = sizeScheduledForMoveGB; - this.dataSizeMovedGB = dataSizeMovedGB; + this.iterationDuration = iterationDuration; + this.sizeScheduledForMove = sizeScheduledForMove; + this.dataSizeMoved = dataSizeMoved; this.containerMovesScheduled = containerMovesScheduled; this.containerMovesCompleted = containerMovesCompleted; this.containerMovesFailed = containerMovesFailed; this.containerMovesTimeout = containerMovesTimeout; - this.sizeEnteringNodesGB = sizeEnteringNodesGB; - this.sizeLeavingNodesGB = sizeLeavingNodesGB; + this.sizeEnteringNodes = sizeEnteringNodes; + this.sizeLeavingNodes = sizeLeavingNodes; } public Integer getIterationNumber() { @@ -68,12 +71,12 @@ public String getIterationResult() { return iterationResult; } - public long getSizeScheduledForMoveGB() { - return sizeScheduledForMoveGB; + public long getSizeScheduledForMove() { + return sizeScheduledForMove; } - public long getDataSizeMovedGB() { - return dataSizeMovedGB; + public long getDataSizeMoved() { + return dataSizeMoved; } public long getContainerMovesScheduled() { @@ -92,12 +95,16 @@ public long getContainerMovesTimeout() { return containerMovesTimeout; } - public Map getSizeEnteringNodesGB() { - return sizeEnteringNodesGB; + public Map getSizeEnteringNodes() { + return sizeEnteringNodes; } - public Map getSizeLeavingNodesGB() { - return sizeLeavingNodesGB; + public Map getSizeLeavingNodes() { + return sizeLeavingNodes; + } + + public long getIterationDuration() { + return iterationDuration; } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 40d153a6bb41..92e22b3b3ed8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -1237,28 +1237,29 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(info.getIterationNumber()) .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) - .setSizeScheduledForMoveGB(info.getSizeScheduledForMoveGB()) - .setDataSizeMovedGB(info.getDataSizeMovedGB()) + .setIterationDuration(info.getIterationDuration()) + .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setDataSizeMoved(info.getDataSizeMoved()) .setContainerMovesScheduled(info.getContainerMovesScheduled()) .setContainerMovesCompleted(info.getContainerMovesCompleted()) .setContainerMovesFailed(info.getContainerMovesFailed()) .setContainerMovesTimeout(info.getContainerMovesTimeout()) - .addAllSizeEnteringNodesGB( - info.getSizeEnteringNodesGB().entrySet() + .addAllSizeEnteringNodes( + info.getSizeEnteringNodes().entrySet() .stream() .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) - .setDataVolumeGB(entry.getValue()) + .setDataVolume(entry.getValue()) .build() ) .collect(Collectors.toList()) ) - .addAllSizeLeavingNodesGB( - info.getSizeLeavingNodesGB().entrySet() + .addAllSizeLeavingNodes( + info.getSizeLeavingNodes().entrySet() .stream() .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) - .setDataVolumeGB(entry.getValue()) + .setDataVolume(entry.getValue()) .build() ) .collect(Collectors.toList()) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index b8ac648e8442..1918db1e1fec 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -50,8 +50,8 @@ void testGetIterationStatistics() { assertTrue(is.getContainerMovesCompleted() > 0); assertEquals(0, is.getContainerMovesFailed()); assertEquals(0, is.getContainerMovesTimeout()); - assertFalse(is.getSizeEnteringNodesGB().isEmpty()); - assertFalse(is.getSizeLeavingNodesGB().isEmpty()); + assertFalse(is.getSizeEnteringNodes().isEmpty()); + assertFalse(is.getSizeLeavingNodes().isEmpty()); }); } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index e58074bf140a..31de7feaed0e 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -31,6 +31,7 @@ import java.time.Duration; import java.time.Instant; import java.time.LocalDateTime; +import java.time.OffsetDateTime; import java.time.ZoneId; import java.util.List; import java.util.stream.Collectors; @@ -60,12 +61,15 @@ public void execute(ScmClient scmClient) throws IOException { boolean isRunning = response.getIsRunning(); ContainerBalancerStatusInfo balancerStatusInfo = response.getContainerBalancerStatusInfo(); if (isRunning) { + Instant startedAtInstant = Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()); LocalDateTime dateTime = - LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); + LocalDateTime.ofInstant(startedAtInstant, ZoneId.systemDefault()); System.out.println("ContainerBalancer is Running."); if (verbose) { - System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(), dateTime.toLocalTime()); + System.out.printf("Started at: %s %s%n", dateTime.toLocalDate(), dateTime.toLocalTime()); + long balancingDuration = OffsetDateTime.now().toEpochSecond() - startedAtInstant.getEpochSecond(); + System.out.printf("Balancing duration: %s%n%n", getPrettyIterationStatusInfo(balancingDuration)); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); @@ -78,7 +82,9 @@ public void execute(ScmClient scmClient) throws IOException { if (verboseWithHistory) { System.out.println("Iteration history list:"); System.out.println( - iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo) + iterationsStatusInfoList.subList(0, iterationsStatusInfoList.size() - 1) + .stream() + .map(this::getPrettyIterationStatusInfo) .collect(Collectors.joining("\n")) ); } @@ -137,17 +143,18 @@ String getConfigurationPrettyString(HddsProtos.ContainerBalancerConfigurationPro private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { int iterationNumber = iterationStatusInfo.getIterationNumber(); String iterationResult = iterationStatusInfo.getIterationResult(); - long sizeScheduledForMove = iterationStatusInfo.getSizeScheduledForMoveGB(); - long dataSizeMovedGB = iterationStatusInfo.getDataSizeMovedGB(); + long iterationDuration = iterationStatusInfo.getIterationDuration(); + long sizeScheduledForMove = iterationStatusInfo.getSizeScheduledForMove(); + long dataSizeMoved = iterationStatusInfo.getDataSizeMoved(); long containerMovesScheduled = iterationStatusInfo.getContainerMovesScheduled(); long containerMovesCompleted = iterationStatusInfo.getContainerMovesCompleted(); long containerMovesFailed = iterationStatusInfo.getContainerMovesFailed(); long containerMovesTimeout = iterationStatusInfo.getContainerMovesTimeout(); - String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesGBList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolumeGB() + "\n") + String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + getPrettySize(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); - String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesGBList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolumeGB() + "\n") + String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + getPrettySize(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); return String.format( "%-50s %s%n" + @@ -159,14 +166,16 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "%-50s %s%n" + "%-50s %s%n" + "%-50s %s%n" + + "%-50s %s%n" + "%-50s %n%s" + "%-50s %n%s", "Key", "Value", "Iteration number", iterationNumber, + "Iteration duration", getPrettyIterationStatusInfo(iterationDuration), "Iteration result", iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, - "Size scheduled to move", sizeScheduledForMove, - "Moved data size", dataSizeMovedGB, + "Size scheduled to move", getPrettySize(sizeScheduledForMove), + "Moved data size", getPrettySize(dataSizeMoved), "Scheduled to move containers", containerMovesScheduled, "Already moved containers", containerMovesCompleted, "Failed to move containers", containerMovesFailed, @@ -174,5 +183,29 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "Entered data to nodes", enteringDataNodeList, "Exited data from nodes", leavingDataNodeList); } + + private String getPrettyIterationStatusInfo(long duration) { + String prettyDuration; + if (duration >= 0 && duration < 60) { + prettyDuration = duration + "s"; + } else if (duration >= 60 && duration < 3600) { + prettyDuration = (duration / 60 + "m " + duration % 60 + "s"); + } else if (duration >= 3600) { + prettyDuration = (duration / 60 / 60 + "h " + duration / 60 % 60 + "m " + duration % 60 + "s"); + } else { + throw new IllegalStateException("Incorrect duration exception" + duration); + } + return prettyDuration; + } + + public static String getPrettySize(long sizeInBytes) { + if (sizeInBytes / OzoneConsts.GB > 0) { + return sizeInBytes / OzoneConsts.GB + " Gb" + sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; + } else if (sizeInBytes == 0) { + return "0"; + } else { + return sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; + } + } } diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index 41b419d23263..b1b4ab4769af 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -74,7 +74,7 @@ public void tearDown() { } @Test - public void testContainerBalancerStatusInfoSubcommandRunning() + void testContainerBalancerStatusInfoSubcommandRunning() throws IOException { ScmClient scmClient = mock(ScmClient.class); @@ -98,34 +98,35 @@ public void testContainerBalancerStatusInfoSubcommandRunning() StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(0) .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMoveGB(48) - .setDataSizeMovedGB(48) + .setIterationDuration(400L) + .setSizeScheduledForMove(48) + .setDataSizeMoved(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolumeGB(27) + .setDataVolume(27) .build() ) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolumeGB(23L) + .setDataVolume(23L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolumeGB(24L) + .setDataVolume(24L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolumeGB(26L) + .setDataVolume(26L) .build() ) .build(); @@ -133,69 +134,71 @@ public void testContainerBalancerStatusInfoSubcommandRunning() StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(1) .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMoveGB(48) - .setDataSizeMovedGB(48) + .setIterationDuration(300L) + .setSizeScheduledForMove(48) + .setDataSizeMoved(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolumeGB(27L) + .setDataVolume(27L) .build() ) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolumeGB(23L) + .setDataVolume(23L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolumeGB(24L) + .setDataVolume(24L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolumeGB(26L) + .setDataVolume(26L) .build() ) .build(); StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(1) + .setIterationNumber(2) .setIterationResult("") - .setSizeScheduledForMoveGB(48) - .setDataSizeMovedGB(48) + .setIterationDuration(4000L) + .setSizeScheduledForMove(48) + .setDataSizeMoved(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolumeGB(27L) + .setDataVolume(27L) .build() ) - .addSizeEnteringNodesGB( + .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolumeGB(23L) + .setDataVolume(23L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolumeGB(24L) + .setDataVolume(24L) .build() ) - .addSizeLeavingNodesGB( + .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolumeGB(26L) + .setDataVolume(26L) .build() ) .build(); @@ -217,7 +220,8 @@ public void testContainerBalancerStatusInfoSubcommandRunning() statusCmd.execute(scmClient); Pattern p = Pattern.compile( "^ContainerBalancer\\sis\\sRunning."); - Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); + String output = outContent.toString(DEFAULT_ENCODING); + Matcher m = p.matcher(output); assertTrue(m.find()); } @@ -226,7 +230,7 @@ public void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() throws IOException { ScmClient scmClient = mock(ScmClient.class); - //test status is running + //test status is not running when(scmClient.getContainerBalancerStatusInfo()).thenReturn( ContainerBalancerStatusInfoResponseProto.newBuilder() .setIsRunning(false) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 4299afe5f2d1..d629f64c25d7 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -93,12 +93,16 @@ Run Balancer Verbose Status Verify Verbose Balancer Status ${result} Verify Balancer Iteration ${result} 1 IN_PROGRESS 3 Should Contain ${result} Current iteration info: + Should Contain ${result} Balancing duration: + Should Contain ${result} Iteration duration Run Balancer Verbose History Status ${result} = Execute ozone admin containerbalancer status -v --history Verify Verbose Balancer Status ${result} Verify Balancer Iteration ${result} 1 IN_PROGRESS 3 Should Contain ${result} Iteration history list: + Should Contain ${result} Balancing duration: + Should Contain ${result} Iteration duration ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status From bac579485149614d657cc25fe9e3ecc3adedfd5e Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 19 Sep 2024 10:50:17 +0300 Subject: [PATCH 02/39] HDDS-11367. Add space between Gb and Mb in getPrettySize --- .../hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index 31de7feaed0e..5889165682e9 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -200,7 +200,7 @@ private String getPrettyIterationStatusInfo(long duration) { public static String getPrettySize(long sizeInBytes) { if (sizeInBytes / OzoneConsts.GB > 0) { - return sizeInBytes / OzoneConsts.GB + " Gb" + sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; + return sizeInBytes / OzoneConsts.GB + " Gb " + sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; } else if (sizeInBytes == 0) { return "0"; } else { From 9dbd39c9644cf200e821fe497ecb549c4ef5ad17 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 3 Oct 2024 11:01:52 +0300 Subject: [PATCH 03/39] HDDS-11367. Fixe review notices + add unit tests --- .../balancer/ContainerBalancerMetrics.java | 8 +- .../balancer/ContainerBalancerStatusInfo.java | 48 +++ .../balancer/ContainerBalancerTask.java | 7 +- .../scm/server/SCMClientProtocolServer.java | 46 +-- .../ContainerBalancerStatusSubcommand.java | 38 +- .../apache/hadoop/hdds/util/DurationUtil.java | 25 ++ .../apache/hadoop/hdds/util/package-info.java | 23 ++ .../TestContainerBalancerSubCommand.java | 375 ++++++++++++++---- .../hadoop/hdds/util/DurationUtilTest.java | 78 ++++ 9 files changed, 498 insertions(+), 150 deletions(-) create mode 100644 hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java create mode 100644 hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java create mode 100644 hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index ea1ea7042ddb..5ff725f76198 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -42,7 +42,7 @@ public final class ContainerBalancerMetrics { @Metric(about = "Amount of bytes that Container Balancer moved" + " in the latest iteration.") - private MutableCounterLong dataSizeMovedInLatestIteration; + private MutableCounterLong dataSizeMovedBytesInLatestIteration; @Metric(about = "Number of completed container moves performed by " + "Container Balancer in the latest iteration.") @@ -164,15 +164,15 @@ public void resetDataSizeMovedGBInLatestIteration() { * @return size in bytes */ public long getDataSizeMovedInLatestIteration() { - return dataSizeMovedInLatestIteration.value(); + return dataSizeMovedBytesInLatestIteration.value(); } public void incrementDataSizeMovedInLatestIteration(long valueToAdd) { - this.dataSizeMovedInLatestIteration.incr(valueToAdd); + this.dataSizeMovedBytesInLatestIteration.incr(valueToAdd); } public void resetDataSizeMovedInLatestIteration() { - dataSizeMovedInLatestIteration.incr( + dataSizeMovedBytesInLatestIteration.incr( -getDataSizeMovedInLatestIteration()); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index cbe8385e53a7..f27acc19dcad 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -19,9 +19,12 @@ package org.apache.hadoop.hdds.scm.container.balancer; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; import java.time.OffsetDateTime; import java.util.List; +import java.util.Optional; +import java.util.stream.Collectors; /** * Info about balancer status. @@ -51,4 +54,49 @@ public HddsProtos.ContainerBalancerConfigurationProto getConfiguration() { public List getIterationsStatusInfo() { return iterationsStatusInfo; } + + public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProto() { + return StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo + .newBuilder() + .setStartedAt(getStartedAt().toEpochSecond()) + .setConfiguration(getConfiguration()) + .addAllIterationsStatusInfo( + getIterationsStatusInfo() + .stream() + .map( + info -> StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(info.getIterationNumber()) + .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) + .setIterationDuration(info.getIterationDuration()) + .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setDataSizeMoved(info.getDataSizeMoved()) + .setContainerMovesScheduled(info.getContainerMovesScheduled()) + .setContainerMovesCompleted(info.getContainerMovesCompleted()) + .setContainerMovesFailed(info.getContainerMovesFailed()) + .setContainerMovesTimeout(info.getContainerMovesTimeout()) + .addAllSizeEnteringNodes( + info.getSizeEnteringNodes().entrySet() + .stream() + .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid(entry.getKey().toString()) + .setDataVolume(entry.getValue()) + .build() + ) + .collect(Collectors.toList()) + ) + .addAllSizeLeavingNodes( + info.getSizeLeavingNodes().entrySet() + .stream() + .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid(entry.getKey().toString()) + .setDataVolume(entry.getValue()) + .build() + ) + .collect(Collectors.toList()) + ) + .build() + ) + .collect(Collectors.toList()) + ).build(); + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 2ec8cc18c299..b473e69f0e3c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -339,11 +339,8 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) } public List getCurrentIterationsStatistic() { - - int lastIterationNumber = iterationsStatistic.stream() - .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) - .max() - .orElse(0); + int lastIterationNumber = iterationsStatistic.isEmpty() ? 0 : + iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); long iterationDuration = getCurrentIterationDuration(); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber + 1, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java index 92e22b3b3ed8..12ba25d0a14b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java @@ -37,10 +37,8 @@ import org.apache.hadoop.hdds.protocol.proto.ReconfigureProtocolProtos.ReconfigureProtocolService; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto.Builder; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.NodeTransferInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.protocolPB.ReconfigureProtocolPB; import org.apache.hadoop.hdds.protocolPB.ReconfigureProtocolServerSideTranslatorPB; @@ -1226,49 +1224,7 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() return ContainerBalancerStatusInfoResponseProto .newBuilder() .setIsRunning(true) - .setContainerBalancerStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo - .newBuilder() - .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) - .setConfiguration(balancerStatusInfo.getConfiguration()) - .addAllIterationsStatusInfo( - balancerStatusInfo.getIterationsStatusInfo() - .stream() - .map( - info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(info.getIterationNumber()) - .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) - .setIterationDuration(info.getIterationDuration()) - .setSizeScheduledForMove(info.getSizeScheduledForMove()) - .setDataSizeMoved(info.getDataSizeMoved()) - .setContainerMovesScheduled(info.getContainerMovesScheduled()) - .setContainerMovesCompleted(info.getContainerMovesCompleted()) - .setContainerMovesFailed(info.getContainerMovesFailed()) - .setContainerMovesTimeout(info.getContainerMovesTimeout()) - .addAllSizeEnteringNodes( - info.getSizeEnteringNodes().entrySet() - .stream() - .map(entry -> NodeTransferInfo.newBuilder() - .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) - .build() - ) - .collect(Collectors.toList()) - ) - .addAllSizeLeavingNodes( - info.getSizeLeavingNodes().entrySet() - .stream() - .map(entry -> NodeTransferInfo.newBuilder() - .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) - .build() - ) - .collect(Collectors.toList()) - ) - .build() - ) - .collect(Collectors.toList()) - ) - ) + .setContainerBalancerStatusInfo(balancerStatusInfo.toProto()) .build(); } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index 5889165682e9..23937daccc13 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -36,6 +36,9 @@ import java.util.List; import java.util.stream.Collectors; +import static org.apache.hadoop.hdds.util.DurationUtil.getPrettyDuration; +import static org.apache.hadoop.util.StringUtils.byteDesc; + /** * Handler to query status of container balancer. */ @@ -69,7 +72,7 @@ public void execute(ScmClient scmClient) throws IOException { if (verbose) { System.out.printf("Started at: %s %s%n", dateTime.toLocalDate(), dateTime.toLocalTime()); long balancingDuration = OffsetDateTime.now().toEpochSecond() - startedAtInstant.getEpochSecond(); - System.out.printf("Balancing duration: %s%n%n", getPrettyIterationStatusInfo(balancingDuration)); + System.out.printf("Balancing duration: %s%n%n", getPrettyDuration(balancingDuration)); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); @@ -151,10 +154,10 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus long containerMovesFailed = iterationStatusInfo.getContainerMovesFailed(); long containerMovesTimeout = iterationStatusInfo.getContainerMovesTimeout(); String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + getPrettySize(nodeInfo.getDataVolume()) + "\n") + .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + byteDesc(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + getPrettySize(nodeInfo.getDataVolume()) + "\n") + .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + byteDesc(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); return String.format( "%-50s %s%n" + @@ -171,11 +174,11 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "%-50s %n%s", "Key", "Value", "Iteration number", iterationNumber, - "Iteration duration", getPrettyIterationStatusInfo(iterationDuration), + "Iteration duration", getPrettyDuration(iterationDuration), "Iteration result", iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, - "Size scheduled to move", getPrettySize(sizeScheduledForMove), - "Moved data size", getPrettySize(dataSizeMoved), + "Size scheduled to move", byteDesc(sizeScheduledForMove), + "Moved data size", byteDesc(dataSizeMoved), "Scheduled to move containers", containerMovesScheduled, "Already moved containers", containerMovesCompleted, "Failed to move containers", containerMovesFailed, @@ -184,28 +187,5 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "Exited data from nodes", leavingDataNodeList); } - private String getPrettyIterationStatusInfo(long duration) { - String prettyDuration; - if (duration >= 0 && duration < 60) { - prettyDuration = duration + "s"; - } else if (duration >= 60 && duration < 3600) { - prettyDuration = (duration / 60 + "m " + duration % 60 + "s"); - } else if (duration >= 3600) { - prettyDuration = (duration / 60 / 60 + "h " + duration / 60 % 60 + "m " + duration % 60 + "s"); - } else { - throw new IllegalStateException("Incorrect duration exception" + duration); - } - return prettyDuration; - } - - public static String getPrettySize(long sizeInBytes) { - if (sizeInBytes / OzoneConsts.GB > 0) { - return sizeInBytes / OzoneConsts.GB + " Gb " + sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; - } else if (sizeInBytes == 0) { - return "0"; - } else { - return sizeInBytes % OzoneConsts.GB / OzoneConsts.MB + " Mb"; - } - } } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java new file mode 100644 index 000000000000..392010b28cc4 --- /dev/null +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java @@ -0,0 +1,25 @@ +package org.apache.hadoop.hdds.util; + +/** + * Pretty duration string representation. + */ +public final class DurationUtil { + + private DurationUtil() { + } + + public static String getPrettyDuration(long durationSeconds) { + String prettyDuration; + if (durationSeconds >= 0 && durationSeconds < 60) { + prettyDuration = durationSeconds + "s"; + } else if (durationSeconds >= 60 && durationSeconds < 3600) { + prettyDuration = (durationSeconds / 60 + "m " + durationSeconds % 60 + "s"); + } else if (durationSeconds >= 3600) { + prettyDuration = + (durationSeconds / 60 / 60 + "h " + durationSeconds / 60 % 60 + "m " + durationSeconds % 60 + "s"); + } else { + throw new IllegalStateException("Incorrect duration exception" + durationSeconds); + } + return prettyDuration; + } +} diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java new file mode 100644 index 000000000000..fa0d3d464125 --- /dev/null +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java @@ -0,0 +1,23 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + *

+ * SCM related cli tools. + */ +/** + * SCM related cli utils. + */ +package org.apache.hadoop.hdds.util; diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index b1b4ab4769af..741c23d16e91 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -25,9 +25,11 @@ import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStopSubcommand; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; +import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; +import picocli.CommandLine; import java.io.ByteArrayOutputStream; import java.io.IOException; @@ -39,6 +41,8 @@ import java.util.regex.Matcher; import java.util.regex.Pattern; +import static org.apache.hadoop.ozone.OzoneConsts.GB; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.when; @@ -58,49 +62,16 @@ class TestContainerBalancerSubCommand { private ContainerBalancerStartSubcommand startCmd; private ContainerBalancerStatusSubcommand statusCmd; - @BeforeEach - public void setup() throws UnsupportedEncodingException { - stopCmd = new ContainerBalancerStopSubcommand(); - startCmd = new ContainerBalancerStartSubcommand(); - statusCmd = new ContainerBalancerStatusSubcommand(); - System.setOut(new PrintStream(outContent, false, DEFAULT_ENCODING)); - System.setErr(new PrintStream(errContent, false, DEFAULT_ENCODING)); - } - - @AfterEach - public void tearDown() { - System.setOut(originalOut); - System.setErr(originalErr); - } - - @Test - void testContainerBalancerStatusInfoSubcommandRunning() - throws IOException { - ScmClient scmClient = mock(ScmClient.class); - - ContainerBalancerConfiguration config = new ContainerBalancerConfiguration(); - config.setThreshold(10); - config.setMaxDatanodesPercentageToInvolvePerIteration(20); - config.setMaxSizeToMovePerIteration(53687091200L); - config.setMaxSizeEnteringTarget(27917287424L); - config.setMaxSizeLeavingSource(27917287424L); - config.setIterations(2); - config.setExcludeNodes(""); - config.setMoveTimeout(3900000); - config.setMoveReplicationTimeout(3000000); - config.setBalancingInterval(0); - config.setIncludeNodes(""); - config.setExcludeNodes(""); - config.setNetworkTopologyEnable(false); - config.setTriggerDuEnable(false); - - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration0StatusInfo = + @NotNull + private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfoResponseProto( + ContainerBalancerConfiguration config) { + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(0) + .setIterationNumber(1) .setIterationResult("ITERATION_COMPLETED") .setIterationDuration(400L) - .setSizeScheduledForMove(48) - .setDataSizeMoved(48) + .setSizeScheduledForMove(54 * GB) + .setDataSizeMoved(54 * GB) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) @@ -108,35 +79,35 @@ void testContainerBalancerStatusInfoSubcommandRunning() .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27) + .setDataVolume(28 * GB) .build() ) .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolume(26 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolume(25 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolume(29 * GB) .build() ) .build(); - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(1) + .setIterationNumber(2) .setIterationResult("ITERATION_COMPLETED") .setIterationDuration(300L) - .setSizeScheduledForMove(48) - .setDataSizeMoved(48) + .setSizeScheduledForMove(30 * GB) + .setDataSizeMoved(30 * GB) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) @@ -144,35 +115,35 @@ void testContainerBalancerStatusInfoSubcommandRunning() .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) + .setDataVolume(20 * GB) .build() ) .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolume(10 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolume(15 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolume(15 * GB) .build() ) .build(); - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration3StatusInfo = StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(2) + .setIterationNumber(3) .setIterationResult("") .setIterationDuration(4000L) - .setSizeScheduledForMove(48) - .setDataSizeMoved(48) + .setSizeScheduledForMove(48 * GB) + .setDataSizeMoved(48 * GB) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) @@ -180,25 +151,25 @@ void testContainerBalancerStatusInfoSubcommandRunning() .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) + .setDataVolume(20 * GB) .build() ) .addSizeEnteringNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolume(28 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolume(30 * GB) .build() ) .addSizeLeavingNodes( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolume(18 * GB) .build() ) .build(); @@ -209,20 +180,291 @@ void testContainerBalancerStatusInfoSubcommandRunning() .setStartedAt(OffsetDateTime.now().toEpochSecond()) .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) .addAllIterationsStatusInfo( - Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) + Arrays.asList(iteration1StatusInfo, iteration2StatusInfo, iteration3StatusInfo) ) ) .build(); + return statusInfoResponseProto; + } + + @NotNull + private static ContainerBalancerConfiguration getContainerBalancerConfiguration() { + ContainerBalancerConfiguration config = new ContainerBalancerConfiguration(); + config.setThreshold(10); + config.setMaxDatanodesPercentageToInvolvePerIteration(20); + config.setMaxSizeToMovePerIteration(53687091200L); + config.setMaxSizeEnteringTarget(27917287424L); + config.setMaxSizeLeavingSource(27917287424L); + config.setIterations(3); + config.setExcludeNodes(""); + config.setMoveTimeout(3900000); + config.setMoveReplicationTimeout(3000000); + config.setBalancingInterval(0); + config.setIncludeNodes(""); + config.setExcludeNodes(""); + config.setNetworkTopologyEnable(false); + config.setTriggerDuEnable(false); + return config; + } + + @BeforeEach + public void setup() throws UnsupportedEncodingException { + stopCmd = new ContainerBalancerStopSubcommand(); + startCmd = new ContainerBalancerStartSubcommand(); + statusCmd = new ContainerBalancerStatusSubcommand(); + System.setOut(new PrintStream(outContent, false, DEFAULT_ENCODING)); + System.setErr(new PrintStream(errContent, false, DEFAULT_ENCODING)); + } + + @AfterEach + public void tearDown() { + System.setOut(originalOut); + System.setErr(originalErr); + } + + @Test + void testContainerBalancerStatusInfoSubcommandRunningWithoutFlags() + throws IOException { + ScmClient scmClient = mock(ScmClient.class); + + ContainerBalancerConfiguration config = + getContainerBalancerConfiguration(); + + ContainerBalancerStatusInfoResponseProto + statusInfoResponseProto = getContainerBalancerStatusInfoResponseProto(config); //test status is running when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); - statusCmd.execute(scmClient); Pattern p = Pattern.compile( "^ContainerBalancer\\sis\\sRunning."); String output = outContent.toString(DEFAULT_ENCODING); Matcher m = p.matcher(output); assertTrue(m.find()); + + String balancerConfigOutput = + "Container Balancer Configuration values:\n" + + "Key Value\n" + + "Threshold 10.0\n" + + "Max Datanodes to Involve per Iteration(percent) 20\n" + + "Max Size to Move per Iteration 0GB\n" + + "Max Size Entering Target per Iteration 26GB\n" + + "Max Size Leaving Source per Iteration 26GB\n" + + "Number of Iterations 3\n" + + "Time Limit for Single Container's Movement 65min\n" + + "Time Limit for Single Container's Replication 50min\n" + + "Interval between each Iteration 0min\n" + + "Whether to Enable Network Topology false\n" + + "Whether to Trigger Refresh Datanode Usage Info false\n" + + "Container IDs to Exclude from Balancing None\n" + + "Datanodes Specified to be Balanced None\n" + + "Datanodes Excluded from Balancing None"; + assertFalse(output.contains(balancerConfigOutput)); + + String currentIterationOutput = + "Current iteration info:\n" + + "Key Value\n" + + "Iteration number 3\n" + + "Iteration duration 1h 6m 40s\n" + + "Iteration result IN_PROGRESS\n" + + "Size scheduled to move 48 GB\n" + + "Moved data size 48 GB\n" + + "Scheduled to move containers 11\n" + + "Already moved containers 11\n" + + "Failed to move containers 0\n" + + "Failed to move containers by timeout 0\n" + + "Entered data to nodes \n" + + "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + + "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 28 GB\n" + + "Exited data from nodes \n" + + "b8b9c511-c30f-4933-8938-2f272e307070 -> 30 GB\n" + + "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 18 GB"; + assertFalse(output.contains(currentIterationOutput)); + + assertFalse(output.contains("Iteration history list:")); + } + + @Test + void testContainerBalancerStatusInfoSubcommandVerboseHistory() + throws IOException { + ScmClient scmClient = mock(ScmClient.class); + + ContainerBalancerConfiguration config = + getContainerBalancerConfiguration(); + + ContainerBalancerStatusInfoResponseProto + statusInfoResponseProto = getContainerBalancerStatusInfoResponseProto(config); + //test status is running + when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); + CommandLine c = new CommandLine(statusCmd); + c.parseArgs("--verbose", "--history"); + statusCmd.execute(scmClient); + String output = outContent.toString(DEFAULT_ENCODING); + Pattern p = Pattern.compile( + "^ContainerBalancer\\sis\\sRunning.$", Pattern.MULTILINE); + Matcher m = p.matcher(output); + assertTrue(m.find()); + + p = Pattern.compile( + "^Started at: (\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2})$", Pattern.MULTILINE); + m = p.matcher(output); + assertTrue(m.find()); + + p = Pattern.compile( + "^Balancing duration: \\d{1}s$", Pattern.MULTILINE); + m = p.matcher(output); + assertTrue(m.find()); + + String balancerConfigOutput = + "Container Balancer Configuration values:\n" + + "Key Value\n" + + "Threshold 10.0\n" + + "Max Datanodes to Involve per Iteration(percent) 20\n" + + "Max Size to Move per Iteration 0GB\n" + + "Max Size Entering Target per Iteration 26GB\n" + + "Max Size Leaving Source per Iteration 26GB\n" + + "Number of Iterations 3\n" + + "Time Limit for Single Container's Movement 65min\n" + + "Time Limit for Single Container's Replication 50min\n" + + "Interval between each Iteration 0min\n" + + "Whether to Enable Network Topology false\n" + + "Whether to Trigger Refresh Datanode Usage Info false\n" + + "Container IDs to Exclude from Balancing None\n" + + "Datanodes Specified to be Balanced None\n" + + "Datanodes Excluded from Balancing None"; + assertTrue(output.contains(balancerConfigOutput)); + + String currentIterationOutput = + "Current iteration info:\n" + + "Key Value\n" + + "Iteration number 3\n" + + "Iteration duration 1h 6m 40s\n" + + "Iteration result IN_PROGRESS\n" + + "Size scheduled to move 48 GB\n" + + "Moved data size 48 GB\n" + + "Scheduled to move containers 11\n" + + "Already moved containers 11\n" + + "Failed to move containers 0\n" + + "Failed to move containers by timeout 0\n" + + "Entered data to nodes \n" + + "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + + "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 28 GB\n" + + "Exited data from nodes \n" + + "b8b9c511-c30f-4933-8938-2f272e307070 -> 30 GB\n" + + "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 18 GB"; + assertTrue(output.contains(currentIterationOutput)); + + assertTrue(output.contains("Iteration history list:")); + String firstHistoryIterationOutput = + "Key Value\n" + + "Iteration number 3\n" + + "Iteration duration 1h 6m 40s\n" + + "Iteration result IN_PROGRESS\n" + + "Size scheduled to move 48 GB\n" + + "Moved data size 48 GB\n" + + "Scheduled to move containers 11\n" + + "Already moved containers 11\n" + + "Failed to move containers 0\n" + + "Failed to move containers by timeout 0\n" + + "Entered data to nodes \n" + + "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + + "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 28 GB\n" + + "Exited data from nodes \n" + + "b8b9c511-c30f-4933-8938-2f272e307070 -> 30 GB\n" + + "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 18 GB"; + assertTrue(output.contains(firstHistoryIterationOutput)); + + String secondHistoryIterationOutput = + "Key Value\n" + + "Iteration number 2\n" + + "Iteration duration 5m 0s\n" + + "Iteration result ITERATION_COMPLETED\n" + + "Size scheduled to move 30 GB\n" + + "Moved data size 30 GB\n" + + "Scheduled to move containers 11\n" + + "Already moved containers 11\n" + + "Failed to move containers 0\n" + + "Failed to move containers by timeout 0\n" + + "Entered data to nodes \n" + + "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + + "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 10 GB\n" + + "Exited data from nodes \n" + + "b8b9c511-c30f-4933-8938-2f272e307070 -> 15 GB\n" + + "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 15 GB"; + assertTrue(output.contains(secondHistoryIterationOutput)); + } + + @Test + void testContainerBalancerStatusInfoSubcommandVerbose() + throws IOException { + ScmClient scmClient = mock(ScmClient.class); + + ContainerBalancerConfiguration config = + getContainerBalancerConfiguration(); + + ContainerBalancerStatusInfoResponseProto + statusInfoResponseProto = getContainerBalancerStatusInfoResponseProto(config); + //test status is running + when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); + CommandLine c = new CommandLine(statusCmd); + c.parseArgs("--verbose"); + statusCmd.execute(scmClient); + String output = outContent.toString(DEFAULT_ENCODING); + Pattern p = Pattern.compile( + "^ContainerBalancer\\sis\\sRunning.$", Pattern.MULTILINE); + Matcher m = p.matcher(output); + assertTrue(m.find()); + + p = Pattern.compile( + "^Started at: (\\d{4}-\\d{2}-\\d{2} \\d{2}:\\d{2}:\\d{2})$", Pattern.MULTILINE); + m = p.matcher(output); + assertTrue(m.find()); + + p = Pattern.compile( + "^Balancing duration: \\d{1}s$", Pattern.MULTILINE); + m = p.matcher(output); + assertTrue(m.find()); + + String balancerConfigOutput = + "Container Balancer Configuration values:\n" + + "Key Value\n" + + "Threshold 10.0\n" + + "Max Datanodes to Involve per Iteration(percent) 20\n" + + "Max Size to Move per Iteration 0GB\n" + + "Max Size Entering Target per Iteration 26GB\n" + + "Max Size Leaving Source per Iteration 26GB\n" + + "Number of Iterations 3\n" + + "Time Limit for Single Container's Movement 65min\n" + + "Time Limit for Single Container's Replication 50min\n" + + "Interval between each Iteration 0min\n" + + "Whether to Enable Network Topology false\n" + + "Whether to Trigger Refresh Datanode Usage Info false\n" + + "Container IDs to Exclude from Balancing None\n" + + "Datanodes Specified to be Balanced None\n" + + "Datanodes Excluded from Balancing None"; + assertTrue(output.contains(balancerConfigOutput)); + + String currentIterationOutput = + "Current iteration info:\n" + + "Key Value\n" + + "Iteration number 3\n" + + "Iteration duration 1h 6m 40s\n" + + "Iteration result IN_PROGRESS\n" + + "Size scheduled to move 48 GB\n" + + "Moved data size 48 GB\n" + + "Scheduled to move containers 11\n" + + "Already moved containers 11\n" + + "Failed to move containers 0\n" + + "Failed to move containers by timeout 0\n" + + "Entered data to nodes \n" + + "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + + "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 28 GB\n" + + "Exited data from nodes \n" + + "b8b9c511-c30f-4933-8938-2f272e307070 -> 30 GB\n" + + "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 18 GB"; + assertTrue(output.contains(currentIterationOutput)); + + assertFalse(output.contains("Iteration history list:")); } @Test @@ -267,8 +509,8 @@ public void testContainerBalancerStopSubcommand() throws IOException { stopCmd.execute(scmClient); Pattern p = Pattern.compile("^Sending\\sstop\\scommand." + - "\\sWaiting\\sfor\\sContainer\\sBalancer\\sto\\sstop...\\n" + - "Container\\sBalancer\\sstopped."); + "\\sWaiting\\sfor\\sContainer\\sBalancer\\sto\\sstop...\\n" + + "Container\\sBalancer\\sstopped."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); @@ -288,7 +530,7 @@ public void testContainerBalancerStartSubcommandWhenBalancerIsNotRunning() startCmd.execute(scmClient); Pattern p = Pattern.compile("^Container\\sBalancer\\sstarted" + - "\\ssuccessfully."); + "\\ssuccessfully."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); } @@ -306,8 +548,7 @@ public void testContainerBalancerStartSubcommandWhenBalancerIsRunning() .build()); startCmd.execute(scmClient); - Pattern p = Pattern.compile("^Failed\\sto\\sstart\\sContainer" + - "\\sBalancer."); + Pattern p = Pattern.compile("^Failed\\sto\\sstart\\sContainer\\sBalancer."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java new file mode 100644 index 000000000000..b04a76777a7c --- /dev/null +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java @@ -0,0 +1,78 @@ +package org.apache.hadoop.hdds.util; + +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import java.time.Duration; +import java.util.Arrays; +import java.util.Collection; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.params.provider.Arguments.arguments; + +class DurationUtilTest { + + private static Stream paramsForPositiveCases() { + return Stream.of( + arguments( + "0s", + DurationUtil.getPrettyDuration(0) + ), + arguments( + "2562047788015215h 30m 7s", + DurationUtil.getPrettyDuration(Long.MAX_VALUE) + ), + arguments( + "1s", + DurationUtil.getPrettyDuration(Duration.ofSeconds(1).getSeconds()) + ), + arguments( + "30s", + DurationUtil.getPrettyDuration(Duration.ofSeconds(30).getSeconds()) + ), + arguments( + "1m 0s", + DurationUtil.getPrettyDuration(Duration.ofMinutes(1).getSeconds()) + ), + arguments( + "2m 30s", + DurationUtil.getPrettyDuration(Duration.ofMinutes(2).getSeconds() + Duration.ofSeconds(30).getSeconds()) + ), + arguments( + "1h 30m 45s", + DurationUtil.getPrettyDuration( + Duration.ofHours(1).getSeconds() + + Duration.ofMinutes(30).getSeconds() + + Duration.ofSeconds(45).getSeconds()) + ), + arguments( + "24h 0m 0s", + DurationUtil.getPrettyDuration(Duration.ofDays(1).getSeconds()) + ), + arguments( + "48h 0m 0s", + DurationUtil.getPrettyDuration(Duration.ofDays(2).getSeconds()) + ) + ); + } + + private static Collection paramsForNegativeCases() { + return Arrays.asList(-1L, Long.MIN_VALUE); + } + + @ParameterizedTest + @MethodSource("paramsForPositiveCases") + void testDuration(String expected, String actual) { + assertEquals(expected, actual); + } + + @ParameterizedTest + @MethodSource("paramsForNegativeCases") + void testDuration(Long param) { + assertThrows(IllegalStateException.class, () -> DurationUtil.getPrettyDuration(param)); + } +} + From 570025fef97f9f8fd134a26bc62ced89f7f87fd5 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 8 Oct 2024 00:39:18 +0300 Subject: [PATCH 04/39] HDDS-11367. Small fixes + improve robot test --- .../balancer/ContainerBalancerTask.java | 4 +- .../ContainerBalancerStatusSubcommand.java | 2 +- .../TestContainerBalancerSubCommand.java | 3 -- .../main/compose/ozone-balancer/docker-config | 2 +- .../smoketest/balancer/testBalancer.robot | 48 +++++++++---------- 5 files changed, 26 insertions(+), 33 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 1fff6acc3326..ef05dafba68a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -265,7 +265,7 @@ private void balance() { } IterationResult iR = doIteration(); - saveIterationStatistic(i, iR); + saveIterationStatistic(i + 1, iR); metrics.incrementNumIterations(1); LOG.info("Result of this iteration of Container Balancer: {}", iR); @@ -351,7 +351,7 @@ public List getCurrentIterationsStatis iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); long iterationDuration = getCurrentIterationDuration(); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - lastIterationNumber + 1, + lastIterationNumber, null, iterationDuration, getSizeScheduledForMoveInLatestIteration(), diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index 23937daccc13..87d963f79bb3 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -173,7 +173,7 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "%-50s %n%s" + "%-50s %n%s", "Key", "Value", - "Iteration number", iterationNumber, + "Iteration number", iterationNumber == 0 ? "-" : iterationNumber, "Iteration duration", getPrettyDuration(iterationDuration), "Iteration result", iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index 741c23d16e91..4f765ea26c0a 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -25,7 +25,6 @@ import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStopSubcommand; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; -import org.jetbrains.annotations.NotNull; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -62,7 +61,6 @@ class TestContainerBalancerSubCommand { private ContainerBalancerStartSubcommand startCmd; private ContainerBalancerStatusSubcommand statusCmd; - @NotNull private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfoResponseProto( ContainerBalancerConfiguration config) { StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = @@ -188,7 +186,6 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat return statusInfoResponseProto; } - @NotNull private static ContainerBalancerConfiguration getContainerBalancerConfiguration() { ContainerBalancerConfiguration config = new ContainerBalancerConfiguration(); config.setThreshold(10); diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config index 29984d43662c..68ae62114c3a 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config @@ -53,7 +53,7 @@ OZONE-SITE.XML_ozone.om.s3.grpc.server_enabled=true OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon OZONE-SITE.XML_dfs.container.ratis.datastream.enabled=true OZONE-SITE.XML_ozone.http.basedir=/tmp/ozone_http - +OZONE-SITE.XML_hdds.container.balancer.balancing.iteration.interval=30s OZONE_CONF_DIR=/etc/hadoop OZONE_LOG_DIR=/var/log/hadoop diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index d629f64c25d7..393662c1d063 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -61,7 +61,7 @@ Datanode Recommission is Finished Should Not Contain ${result} ENTERING_MAINTENANCE Run Container Balancer - ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 + ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 3 Should Contain ${result} Container Balancer started successfully. Wait Finish Of Balancing @@ -70,19 +70,16 @@ Wait Finish Of Balancing Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running Sleep 60000ms -Verify Verbose Balancer Status - [arguments] ${output} - +Verify Balancer Iteration + [arguments] ${output} ${number} ${containers} Should Contain ${output} ContainerBalancer is Running. Should Contain ${output} Started at: Should Contain ${output} Container Balancer Configuration values: - -Verify Balancer Iteration - [arguments] ${output} ${number} ${status} ${containers} - - Should Contain ${output} Iteration number ${number} - Should Contain ${output} Iteration result ${status} - Should Contain ${output} Scheduled to move containers ${containers} + Should Contain ${output} Iteration number ${number} collapse_spaces=True + Should Contain ${output} Scheduled to move containers ${containers} collapse_spaces=True + Should Contain ${output} Balancing duration: + Should Contain ${output} Iteration duration + Should Contain ${output} Current iteration info: Run Balancer Status ${result} = Execute ozone admin containerbalancer status @@ -90,19 +87,21 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v - Verify Verbose Balancer Status ${result} - Verify Balancer Iteration ${result} 1 IN_PROGRESS 3 - Should Contain ${result} Current iteration info: - Should Contain ${result} Balancing duration: - Should Contain ${result} Iteration duration + Verify Balancer Iteration ${result} - 3 + Should Contain ${result} Iteration result IN_PROGRESS collapse_spaces=True + Run Balancer Verbose History Status ${result} = Execute ozone admin containerbalancer status -v --history - Verify Verbose Balancer Status ${result} - Verify Balancer Iteration ${result} 1 IN_PROGRESS 3 + Verify Balancer Iteration ${result} 1 3 Should Contain ${result} Iteration history list: - Should Contain ${result} Balancing duration: - Should Contain ${result} Iteration duration + Should Contain X Times ${result} Size scheduled to move 300 MB 2 collapse_spaces=True + Should Contain X Times ${result} Moved data size 300 MB 2 collapse_spaces=True + Should Contain X Times ${result} Scheduled to move containers 3 2 collapse_spaces=True + Should Contain X Times ${result} Already moved containers 3 2 collapse_spaces=True + Should Contain X Times ${result} Failed to move containers 0 2 collapse_spaces=True + Should Contain X Times ${result} Failed to move containers by timeout 0 2 collapse_spaces=True + Should Contain ${result} Iteration result ITERATION_COMPLETED collapse_spaces=True ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status @@ -173,6 +172,8 @@ Verify Container Balancer for RATIS containers Run Balancer Verbose Status + Sleep 30000ms + Run Balancer Verbose History Status Wait Finish Of Balancing @@ -180,9 +181,4 @@ Verify Container Balancer for RATIS containers ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} = Get Datanode Ozone Used Bytes Info ${uuid} Should Not Be Equal As Integers ${datanodeOzoneUsedBytesInfo} ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} < ${SIZE} * 3.5 - Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} > ${SIZE} * 3 - - - - - + Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} > ${SIZE} * 3 \ No newline at end of file From 0c04a2acf97c0947805f3f1ebc03dc2003ed59fb Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 8 Oct 2024 10:19:04 +0300 Subject: [PATCH 05/39] HDDS-11367. Add licences to files --- .../apache/hadoop/hdds/util/DurationUtil.java | 17 +++++++++++++++++ .../hadoop/hdds/util/DurationUtilTest.java | 17 +++++++++++++++++ 2 files changed, 34 insertions(+) diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java index 392010b28cc4..406d3d558981 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hdds.util; /** diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java index b04a76777a7c..e33352404e74 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java @@ -1,3 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.hadoop.hdds.util; import org.junit.jupiter.params.ParameterizedTest; From f1f11ec5c119055690cab6135baf1f08ce3607e9 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 8 Oct 2024 10:47:49 +0300 Subject: [PATCH 06/39] HDDS-11367. Fix review notice --- .../balancer/ContainerBalancerStatusInfo.java | 36 ++----------------- ...tainerBalancerTaskIterationStatusInfo.java | 36 +++++++++++++++++++ 2 files changed, 38 insertions(+), 34 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index f27acc19dcad..53ea254f9533 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -23,7 +23,6 @@ import java.time.OffsetDateTime; import java.util.List; -import java.util.Optional; import java.util.stream.Collectors; /** @@ -63,40 +62,9 @@ public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProt .addAllIterationsStatusInfo( getIterationsStatusInfo() .stream() - .map( - info -> StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(info.getIterationNumber()) - .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) - .setIterationDuration(info.getIterationDuration()) - .setSizeScheduledForMove(info.getSizeScheduledForMove()) - .setDataSizeMoved(info.getDataSizeMoved()) - .setContainerMovesScheduled(info.getContainerMovesScheduled()) - .setContainerMovesCompleted(info.getContainerMovesCompleted()) - .setContainerMovesFailed(info.getContainerMovesFailed()) - .setContainerMovesTimeout(info.getContainerMovesTimeout()) - .addAllSizeEnteringNodes( - info.getSizeEnteringNodes().entrySet() - .stream() - .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) - .build() - ) - .collect(Collectors.toList()) - ) - .addAllSizeLeavingNodes( - info.getSizeLeavingNodes().entrySet() - .stream() - .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) - .build() - ) - .collect(Collectors.toList()) - ) - .build() - ) + .map(ContainerBalancerTaskIterationStatusInfo::toProto) .collect(Collectors.toList()) ).build(); } + } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 71038fe4d90e..ee7f7284cc9f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -18,8 +18,13 @@ package org.apache.hadoop.hdds.scm.container.balancer; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; + +import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.UUID; +import java.util.stream.Collectors; /** * Information about balancer task iteration. @@ -106,6 +111,37 @@ public Map getSizeLeavingNodes() { public long getIterationDuration() { return iterationDuration; } + + public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo toProto() { + return StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(getIterationNumber()) + .setIterationResult(Optional.ofNullable(getIterationResult()).orElse("")) + .setIterationDuration(getIterationDuration()) + .setSizeScheduledForMove(getSizeScheduledForMove()) + .setDataSizeMoved(getDataSizeMoved()) + .setContainerMovesScheduled(getContainerMovesScheduled()) + .setContainerMovesCompleted(getContainerMovesCompleted()) + .setContainerMovesFailed(getContainerMovesFailed()) + .setContainerMovesTimeout(getContainerMovesTimeout()) + .addAllSizeEnteringNodes( + mapNodes(getSizeEnteringNodes()) + ) + .addAllSizeLeavingNodes( + mapNodes(getSizeLeavingNodes()) + ) + .build(); + } + + private List mapNodes(Map nodes) { + return nodes.entrySet() + .stream() + .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid(entry.getKey().toString()) + .setDataVolume(entry.getValue()) + .build() + ) + .collect(Collectors.toList()); + } } From 68766f9ea66ed62047a86e05e5e849b9e6f2248a Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 8 Oct 2024 11:11:01 +0300 Subject: [PATCH 07/39] HDDS-11367. Improve balancer robot test --- .../smoketest/balancer/testBalancer.robot | 22 +++++++++++-------- 1 file changed, 13 insertions(+), 9 deletions(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 393662c1d063..3da12d347b91 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -81,6 +81,17 @@ Verify Balancer Iteration Should Contain ${output} Iteration duration Should Contain ${output} Current iteration info: +Verify Balancer Iteration History + [arguments] ${output} + Should Contain ${output} Iteration history list: + Should Contain X Times ${output} Size scheduled to move 300 MB 2 collapse_spaces=True + Should Contain X Times ${output} Moved data size 300 MB 2 collapse_spaces=True + Should Contain X Times ${output} Scheduled to move containers 3 2 collapse_spaces=True + Should Contain X Times ${output} Already moved containers 3 2 collapse_spaces=True + Should Contain X Times ${output} Failed to move containers 0 2 collapse_spaces=True + Should Contain X Times ${output} Failed to move containers by timeout 0 2 collapse_spaces=True + Should Contain ${output} Iteration result ITERATION_COMPLETED collapse_spaces=True + Run Balancer Status ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. @@ -93,15 +104,8 @@ Run Balancer Verbose Status Run Balancer Verbose History Status ${result} = Execute ozone admin containerbalancer status -v --history - Verify Balancer Iteration ${result} 1 3 - Should Contain ${result} Iteration history list: - Should Contain X Times ${result} Size scheduled to move 300 MB 2 collapse_spaces=True - Should Contain X Times ${result} Moved data size 300 MB 2 collapse_spaces=True - Should Contain X Times ${result} Scheduled to move containers 3 2 collapse_spaces=True - Should Contain X Times ${result} Already moved containers 3 2 collapse_spaces=True - Should Contain X Times ${result} Failed to move containers 0 2 collapse_spaces=True - Should Contain X Times ${result} Failed to move containers by timeout 0 2 collapse_spaces=True - Should Contain ${result} Iteration result ITERATION_COMPLETED collapse_spaces=True + Verify Balancer Iteration ${result} 1 3 + Verify Balancer Iteration History ${result} ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status From 4138ad745e8506f03a769614b764151a32a9c2fd Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 10 Oct 2024 19:17:45 +0300 Subject: [PATCH 08/39] HDDS-11367. Fix review notices --- .../balancer/ContainerBalancerMetrics.java | 46 +++++--- .../balancer/ContainerBalancerStatusInfo.java | 5 +- .../balancer/ContainerBalancerTask.java | 110 +++++++++--------- ...tainerBalancerTaskIterationStatusInfo.java | 61 +++++++++- .../ContainerBalancerStatusSubcommand.java | 4 +- .../apache/hadoop/hdds/util/DurationUtil.java | 32 +++-- .../hadoop/hdds/util/DurationUtilTest.java | 26 ++--- 7 files changed, 180 insertions(+), 104 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index 5ff725f76198..da267c7e1813 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -40,8 +40,7 @@ public final class ContainerBalancerMetrics { " in the latest iteration.") private MutableCounterLong dataSizeMovedGBInLatestIteration; - @Metric(about = "Amount of bytes that Container Balancer moved" + - " in the latest iteration.") + @Metric(about = "Amount of bytes that Container Balancer moved in the latest iteration.") private MutableCounterLong dataSizeMovedBytesInLatestIteration; @Metric(about = "Number of completed container moves performed by " + @@ -135,14 +134,16 @@ void incrementNumContainerMovesScheduledInLatestIteration(long valueToAdd) { this.numContainerMovesScheduledInLatestIteration.incr(valueToAdd); } + /** + * Reset number of containers scheduled to move in last iteration. + */ public void resetNumContainerMovesScheduledInLatestIteration() { numContainerMovesScheduledInLatestIteration.incr( -getNumContainerMovesScheduledInLatestIteration()); } /** - * Gets the amount of data moved by Container Balancer in the latest - * iteration. + * Gets the amount of data moved by Container Balancer in the latest iteration. * @return size in GB */ public long getDataSizeMovedGBInLatestIteration() { @@ -159,21 +160,26 @@ public void resetDataSizeMovedGBInLatestIteration() { } /** - * Gets the amount of data moved by Container Balancer in the latest - * iteration. + * Gets the amount of data moved by Container Balancer in the latest iteration. * @return size in bytes */ public long getDataSizeMovedInLatestIteration() { return dataSizeMovedBytesInLatestIteration.value(); } - public void incrementDataSizeMovedInLatestIteration(long valueToAdd) { - this.dataSizeMovedBytesInLatestIteration.incr(valueToAdd); + /** + * Increment data size moved in last iteration. + * @param bytes bytes to add + */ + public void incrementDataSizeMovedInLatestIteration(long bytes) { + this.dataSizeMovedBytesInLatestIteration.incr(bytes); } + /** + * Reset data size moved in last iteration. + */ public void resetDataSizeMovedInLatestIteration() { - dataSizeMovedBytesInLatestIteration.incr( - -getDataSizeMovedInLatestIteration()); + dataSizeMovedBytesInLatestIteration.incr(-getDataSizeMovedInLatestIteration()); } /** @@ -185,11 +191,6 @@ public long getNumContainerMovesCompletedInLatestIteration() { return numContainerMovesCompletedInLatestIteration.value(); } - public void incrementNumContainerMovesCompletedInLatestIteration( - long valueToAdd) { - this.numContainerMovesCompletedInLatestIteration.incr(valueToAdd); - } - public void incrementCurrentIterationContainerMoveMetric( MoveManager.MoveResult result, long valueToAdd) { if (result == null) { @@ -226,9 +227,11 @@ public void incrementCurrentIterationContainerMoveMetric( } } + /** + * Moved containers in last iteration. + */ public void resetNumContainerMovesCompletedInLatestIteration() { - numContainerMovesCompletedInLatestIteration.incr( - -getNumContainerMovesCompletedInLatestIteration()); + numContainerMovesCompletedInLatestIteration.incr(-getNumContainerMovesCompletedInLatestIteration()); } /** @@ -240,14 +243,19 @@ public long getNumContainerMovesTimeoutInLatestIteration() { return numContainerMovesTimeoutInLatestIteration.value(); } + /** + * Increment number timeouted container moves. + */ public void incrementNumContainerMovesTimeoutInLatestIteration( long valueToAdd) { this.numContainerMovesTimeoutInLatestIteration.incr(valueToAdd); } + /** + * Reset number timeouted container moves. + */ public void resetNumContainerMovesTimeoutInLatestIteration() { - numContainerMovesTimeoutInLatestIteration.incr( - -getNumContainerMovesTimeoutInLatestIteration()); + numContainerMovesTimeoutInLatestIteration.incr(-getNumContainerMovesTimeoutInLatestIteration()); } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index 53ea254f9533..1ff6784f5d8f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -54,6 +54,10 @@ public List getIterationsStatusInfo() return iterationsStatusInfo; } + /** + * Map to proto. + * @return proto representation + */ public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProto() { return StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo .newBuilder() @@ -66,5 +70,4 @@ public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProt .collect(Collectors.toList()) ).build(); } - } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index ef05dafba68a..25024ec88f72 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -64,6 +64,7 @@ import static java.time.OffsetDateTime.now; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT; +import static org.apache.hadoop.util.StringUtils.byteDesc; /** * Container balancer task performs move of containers between over- and @@ -310,45 +311,46 @@ private void balance() { } private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) { + long iterationDuration = now().toEpochSecond() - currentIterationStarted.toEpochSecond(); ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - iterationNumber, - iR.name(), - now().toEpochSecond() - currentIterationStarted.toEpochSecond(), - getSizeScheduledForMoveInLatestIteration(), - metrics.getDataSizeMovedInLatestIteration(), - metrics.getNumContainerMovesScheduledInLatestIteration(), - metrics.getNumContainerMovesCompletedInLatestIteration(), - metrics.getNumContainerMovesFailedInLatestIteration(), - metrics.getNumContainerMovesTimeoutInLatestIteration(), - findTargetStrategy.getSizeEnteringNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ), - findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - entry -> entry.getValue() - ) - ) + iterationNumber, + iR.name(), + iterationDuration, + getSizeScheduledForMoveInLatestIteration(), + metrics.getDataSizeMovedInLatestIteration(), + metrics.getNumContainerMovesScheduledInLatestIteration(), + metrics.getNumContainerMovesCompletedInLatestIteration(), + metrics.getNumContainerMovesFailedInLatestIteration(), + metrics.getNumContainerMovesTimeoutInLatestIteration(), + findTargetStrategy.getSizeEnteringNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ), + findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ) ); iterationsStatistic.add(iterationStatistic); } public List getCurrentIterationsStatistic() { - int lastIterationNumber = iterationsStatistic.isEmpty() ? 0 : - iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); + int lastIterationNumber = iterationsStatistic.isEmpty() ? 0 + : iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); long iterationDuration = getCurrentIterationDuration(); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber, @@ -367,7 +369,7 @@ public List getCurrentIterationsStatis .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) .collect(Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() + Map.Entry::getValue ) ), findSourceStrategy.getSizeLeavingNodes() @@ -378,7 +380,7 @@ public List getCurrentIterationsStatis .collect( Collectors.toMap( entry -> entry.getKey().getUuid(), - entry -> entry.getValue() + Map.Entry::getValue ) ) ); @@ -718,28 +720,28 @@ private void checkIterationMoveResults() { } } - countDatanodesInvolvedPerIteration = - selectedSources.size() + selectedTargets.size(); - metrics.incrementNumDatanodesInvolvedInLatestIteration( - countDatanodesInvolvedPerIteration); - metrics.incrementNumContainerMovesScheduled( - metrics.getNumContainerMovesScheduledInLatestIteration()); - metrics.incrementNumContainerMovesCompleted( - metrics.getNumContainerMovesCompletedInLatestIteration()); - metrics.incrementNumContainerMovesTimeout( - metrics.getNumContainerMovesTimeoutInLatestIteration()); - metrics.incrementDataSizeMovedGBInLatestIteration( - sizeActuallyMovedInLatestIteration / OzoneConsts.GB); - metrics.incrementDataSizeMovedInLatestIteration( - sizeActuallyMovedInLatestIteration); - metrics.incrementDataSizeMovedGB( - metrics.getDataSizeMovedGBInLatestIteration()); - metrics.incrementNumContainerMovesFailed( - metrics.getNumContainerMovesFailedInLatestIteration()); + countDatanodesInvolvedPerIteration = selectedSources.size() + selectedTargets.size(); + + metrics.incrementNumDatanodesInvolvedInLatestIteration(countDatanodesInvolvedPerIteration); + + metrics.incrementNumContainerMovesScheduled(metrics.getNumContainerMovesScheduledInLatestIteration()); + + metrics.incrementNumContainerMovesCompleted(metrics.getNumContainerMovesCompletedInLatestIteration()); + + metrics.incrementNumContainerMovesTimeout(metrics.getNumContainerMovesTimeoutInLatestIteration()); + + metrics.incrementDataSizeMovedGBInLatestIteration(sizeActuallyMovedInLatestIteration / OzoneConsts.GB); + + metrics.incrementDataSizeMovedInLatestIteration(sizeActuallyMovedInLatestIteration); + + metrics.incrementDataSizeMovedGB(metrics.getDataSizeMovedGBInLatestIteration()); + + metrics.incrementNumContainerMovesFailed(metrics.getNumContainerMovesFailedInLatestIteration()); + LOG.info("Iteration Summary. Number of Datanodes involved: {}. Size " + "moved: {} ({} Bytes). Number of Container moves completed: {}.", countDatanodesInvolvedPerIteration, - StringUtils.byteDesc(sizeActuallyMovedInLatestIteration), + byteDesc(sizeActuallyMovedInLatestIteration), sizeActuallyMovedInLatestIteration, metrics.getNumContainerMovesCompletedInLatestIteration()); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index ee7f7284cc9f..7caf9b692e32 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -68,50 +68,98 @@ public ContainerBalancerTaskIterationStatusInfo( this.sizeLeavingNodes = sizeLeavingNodes; } + /** + * Get number of iteration. + * @return iteration number + */ public Integer getIterationNumber() { return iterationNumber; } + /** + * Get iteration result. + * @return iteration result + */ public String getIterationResult() { return iterationResult; } + /** + * Get size in bytes scheduled to move in iteration. + * @return size in bytes + */ public long getSizeScheduledForMove() { return sizeScheduledForMove; } + /** + * Get size in bytes moved in iteration. + * @return size in bytes + */ public long getDataSizeMoved() { return dataSizeMoved; } + /** + * Get number of scheduled containers to move. + * @return number of scheduled containers to move + */ public long getContainerMovesScheduled() { return containerMovesScheduled; } + /** + * Get number of successfully moved containers. + * @return number of successfully moved containers + */ public long getContainerMovesCompleted() { return containerMovesCompleted; } + /** + * Get number of unsuccessfully moved containers. + * @return number of unsuccessfully moved containers + */ public long getContainerMovesFailed() { return containerMovesFailed; } + /** + * Get number of moved with timeout containers. + * @return number of moved with timeout containers + */ public long getContainerMovesTimeout() { return containerMovesTimeout; } + /** + * Get nodeId to size entering from node map. + * @return nodeId to size entering from node map + */ public Map getSizeEnteringNodes() { return sizeEnteringNodes; } + /** + * Get nodeId to size leaving from node map. + * @return nodeId to size leaving from node map + */ public Map getSizeLeavingNodes() { return sizeLeavingNodes; } + /** + * Get iteration duration + * @return iteration duration + */ public long getIterationDuration() { return iterationDuration; } + /** + * Map to proto. + * @return proto representation + */ public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo toProto() { return StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(getIterationNumber()) @@ -124,15 +172,22 @@ public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStat .setContainerMovesFailed(getContainerMovesFailed()) .setContainerMovesTimeout(getContainerMovesTimeout()) .addAllSizeEnteringNodes( - mapNodes(getSizeEnteringNodes()) + mapToProtoNodeTransferInfo(getSizeEnteringNodes()) ) .addAllSizeLeavingNodes( - mapNodes(getSizeLeavingNodes()) + mapToProtoNodeTransferInfo(getSizeLeavingNodes()) ) .build(); } - private List mapNodes(Map nodes) { + /** + * Map to proto node transfer info. + * @param nodes node id to node traffic size + * @return node transfer info proto representation + */ + private List mapToProtoNodeTransferInfo( + Map nodes + ) { return nodes.entrySet() .stream() .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index 87d963f79bb3..beadef9e78d8 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -71,7 +71,7 @@ public void execute(ScmClient scmClient) throws IOException { if (verbose) { System.out.printf("Started at: %s %s%n", dateTime.toLocalDate(), dateTime.toLocalTime()); - long balancingDuration = OffsetDateTime.now().toEpochSecond() - startedAtInstant.getEpochSecond(); + Duration balancingDuration = Duration.between(startedAtInstant, OffsetDateTime.now()); System.out.printf("Balancing duration: %s%n%n", getPrettyDuration(balancingDuration)); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList @@ -174,7 +174,7 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "%-50s %n%s", "Key", "Value", "Iteration number", iterationNumber == 0 ? "-" : iterationNumber, - "Iteration duration", getPrettyDuration(iterationDuration), + "Iteration duration", getPrettyDuration(Duration.ofSeconds(iterationDuration)), "Iteration result", iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, "Size scheduled to move", byteDesc(sizeScheduledForMove), diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java index 406d3d558981..45f8eb6ab734 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java @@ -17,6 +17,10 @@ */ package org.apache.hadoop.hdds.util; +import java.time.Duration; + +import static java.lang.String.format; + /** * Pretty duration string representation. */ @@ -25,18 +29,24 @@ public final class DurationUtil { private DurationUtil() { } - public static String getPrettyDuration(long durationSeconds) { - String prettyDuration; - if (durationSeconds >= 0 && durationSeconds < 60) { - prettyDuration = durationSeconds + "s"; - } else if (durationSeconds >= 60 && durationSeconds < 3600) { - prettyDuration = (durationSeconds / 60 + "m " + durationSeconds % 60 + "s"); - } else if (durationSeconds >= 3600) { - prettyDuration = - (durationSeconds / 60 / 60 + "h " + durationSeconds / 60 % 60 + "m " + durationSeconds % 60 + "s"); + /** + * Modify duration to string view. E.x. 1h 30m 45s, 2m 30s, 30s. + * + * @param duration duration + * @return duration in string format + */ + public static String getPrettyDuration(Duration duration) { + long hours = duration.toHours(); + long minutes = duration.getSeconds() / 60 % 60; + long seconds = duration.getSeconds() % 60; + if (hours > 0) { + return format("%dh %dm %ds", hours, minutes, seconds); + } else if (minutes > 0) { + return format("%dm %ds", minutes, seconds); + } else if (seconds >= 0) { + return format("%ds", seconds); } else { - throw new IllegalStateException("Incorrect duration exception" + durationSeconds); + throw new IllegalStateException("Incorrect duration exception" + duration); } - return prettyDuration; } } diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java index e33352404e74..1ffb0b477793 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java @@ -36,48 +36,46 @@ private static Stream paramsForPositiveCases() { return Stream.of( arguments( "0s", - DurationUtil.getPrettyDuration(0) + DurationUtil.getPrettyDuration(Duration.ZERO) ), arguments( "2562047788015215h 30m 7s", - DurationUtil.getPrettyDuration(Long.MAX_VALUE) + DurationUtil.getPrettyDuration(Duration.ofSeconds(Long.MAX_VALUE)) ), arguments( "1s", - DurationUtil.getPrettyDuration(Duration.ofSeconds(1).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofSeconds(1)) ), arguments( "30s", - DurationUtil.getPrettyDuration(Duration.ofSeconds(30).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofSeconds(30)) ), arguments( "1m 0s", - DurationUtil.getPrettyDuration(Duration.ofMinutes(1).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofMinutes(1)) ), arguments( "2m 30s", - DurationUtil.getPrettyDuration(Duration.ofMinutes(2).getSeconds() + Duration.ofSeconds(30).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofMinutes(2).plusSeconds(30)) ), arguments( "1h 30m 45s", DurationUtil.getPrettyDuration( - Duration.ofHours(1).getSeconds() + - Duration.ofMinutes(30).getSeconds() + - Duration.ofSeconds(45).getSeconds()) + Duration.ofHours(1).plusMinutes(30).plusSeconds(45)) ), arguments( "24h 0m 0s", - DurationUtil.getPrettyDuration(Duration.ofDays(1).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofDays(1)) ), arguments( "48h 0m 0s", - DurationUtil.getPrettyDuration(Duration.ofDays(2).getSeconds()) + DurationUtil.getPrettyDuration(Duration.ofDays(2)) ) ); } - private static Collection paramsForNegativeCases() { - return Arrays.asList(-1L, Long.MIN_VALUE); + private static Collection paramsForNegativeCases() { + return Arrays.asList(Duration.ofSeconds(-1L), Duration.ofSeconds(Long.MIN_VALUE)); } @ParameterizedTest @@ -88,7 +86,7 @@ void testDuration(String expected, String actual) { @ParameterizedTest @MethodSource("paramsForNegativeCases") - void testDuration(Long param) { + void testDuration(Duration param) { assertThrows(IllegalStateException.class, () -> DurationUtil.getPrettyDuration(param)); } } From 4d916a53667aa75608ae3c5e7b5bef72c7c55884 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 15 Oct 2024 12:19:07 +0300 Subject: [PATCH 09/39] HDDS-11367. Fix review notices --- .../container/balancer/ContainerBalancerMetrics.java | 6 +++--- .../balancer/ContainerBalancerStatusInfo.java | 2 +- .../scm/container/balancer/ContainerBalancerTask.java | 6 +++--- .../ContainerBalancerTaskIterationStatusInfo.java | 10 +++++----- 4 files changed, 12 insertions(+), 12 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index da267c7e1813..270bdc6925b5 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -168,7 +168,7 @@ public long getDataSizeMovedInLatestIteration() { } /** - * Increment data size moved in last iteration. + * Increment data size moved in the last iteration. * @param bytes bytes to add */ public void incrementDataSizeMovedInLatestIteration(long bytes) { @@ -176,7 +176,7 @@ public void incrementDataSizeMovedInLatestIteration(long bytes) { } /** - * Reset data size moved in last iteration. + * Reset data size moved in the last iteration. */ public void resetDataSizeMovedInLatestIteration() { dataSizeMovedBytesInLatestIteration.incr(-getDataSizeMovedInLatestIteration()); @@ -228,7 +228,7 @@ public void incrementCurrentIterationContainerMoveMetric( } /** - * Moved containers in last iteration. + * Moved containers in the last iteration. */ public void resetNumContainerMovesCompletedInLatestIteration() { numContainerMovesCompletedInLatestIteration.incr(-getNumContainerMovesCompletedInLatestIteration()); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index 1ff6784f5d8f..742aa8c5719c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -55,7 +55,7 @@ public List getIterationsStatusInfo() } /** - * Map to proto. + * Converts an instance into the protobuf compatible object. * @return proto representation */ public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProto() { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 25024ec88f72..2ee939509fa0 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -37,7 +37,6 @@ 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; @@ -74,6 +73,7 @@ public class ContainerBalancerTask implements Runnable { public static final Logger LOG = LoggerFactory.getLogger(ContainerBalancerTask.class); + public static final long ABSENCE_OF_DURATION = -1L; private NodeManager nodeManager; private ContainerManager containerManager; @@ -266,7 +266,7 @@ private void balance() { } IterationResult iR = doIteration(); - saveIterationStatistic(i + 1, iR); + saveIterationStatistic(++i, iR); metrics.incrementNumIterations(1); LOG.info("Result of this iteration of Container Balancer: {}", iR); @@ -391,7 +391,7 @@ public List getCurrentIterationsStatis private long getCurrentIterationDuration() { if (currentIterationStarted == null) { - return -1L; + return ABSENCE_OF_DURATION; } else { return now().toEpochSecond() - currentIterationStarted.toEpochSecond(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 7caf9b692e32..288911127397 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -69,7 +69,7 @@ public ContainerBalancerTaskIterationStatusInfo( } /** - * Get number of iteration. + * Get number of iterations. * @return iteration number */ public Integer getIterationNumber() { @@ -85,7 +85,7 @@ public String getIterationResult() { } /** - * Get size in bytes scheduled to move in iteration. + * Get size in bytes scheduled to move in the iteration. * @return size in bytes */ public long getSizeScheduledForMove() { @@ -93,7 +93,7 @@ public long getSizeScheduledForMove() { } /** - * Get size in bytes moved in iteration. + * Get size in bytes moved in the iteration. * @return size in bytes */ public long getDataSizeMoved() { @@ -101,8 +101,8 @@ public long getDataSizeMoved() { } /** - * Get number of scheduled containers to move. - * @return number of scheduled containers to move + * Get number of containers scheduled to move. + * @return number of containers scheduled to move */ public long getContainerMovesScheduled() { return containerMovesScheduled; From 75a103cff2fda0290e5e1bb1d7053d1a12aa2ffd Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 16 Oct 2024 23:23:38 +0300 Subject: [PATCH 10/39] HDDS-11367. Fix review notices --- .../balancer/ContainerBalancerMetrics.java | 8 +- .../balancer/ContainerBalancerTask.java | 98 ++++++++++--------- ...tainerBalancerTaskIterationStatusInfo.java | 18 ++-- 3 files changed, 65 insertions(+), 59 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index 270bdc6925b5..c1d46ec67df7 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -135,7 +135,7 @@ void incrementNumContainerMovesScheduledInLatestIteration(long valueToAdd) { } /** - * Reset number of containers scheduled to move in last iteration. + * Reset the number of containers scheduled to move in the last iteration. */ public void resetNumContainerMovesScheduledInLatestIteration() { numContainerMovesScheduledInLatestIteration.incr( @@ -228,7 +228,7 @@ public void incrementCurrentIterationContainerMoveMetric( } /** - * Moved containers in the last iteration. + * Reset the number of containers moved in the last iteration. */ public void resetNumContainerMovesCompletedInLatestIteration() { numContainerMovesCompletedInLatestIteration.incr(-getNumContainerMovesCompletedInLatestIteration()); @@ -244,7 +244,7 @@ public long getNumContainerMovesTimeoutInLatestIteration() { } /** - * Increment number timeouted container moves. + * Increase the number of containers that are transferred and finished with the timeout. */ public void incrementNumContainerMovesTimeoutInLatestIteration( long valueToAdd) { @@ -252,7 +252,7 @@ public void incrementNumContainerMovesTimeoutInLatestIteration( } /** - * Reset number timeouted container moves. + * Reset the number of containers that are transferred and finished with the timeout. */ public void resetNumContainerMovesTimeoutInLatestIteration() { numContainerMovesTimeoutInLatestIteration.incr(-getNumContainerMovesTimeoutInLatestIteration()); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 2ee939509fa0..df40e8d9cdaa 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -53,6 +53,7 @@ import java.util.Map; import java.util.Objects; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; @@ -310,11 +311,33 @@ private void balance() { tryStopWithSaveConfiguration("Completed all iterations."); } - private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) { + private void saveIterationStatistic(Integer iterationNumber, IterationResult currentIterationResult) { long iterationDuration = now().toEpochSecond() - currentIterationStarted.toEpochSecond(); + Map sizeEnteringDataToNodes = findTargetStrategy.getSizeEnteringNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ); + Map sizeLeavingDataFromNodes = findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ); ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( iterationNumber, - iR.name(), + currentIterationResult.name(), iterationDuration, getSizeScheduledForMoveInLatestIteration(), metrics.getDataSizeMovedInLatestIteration(), @@ -322,36 +345,38 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) metrics.getNumContainerMovesCompletedInLatestIteration(), metrics.getNumContainerMovesFailedInLatestIteration(), metrics.getNumContainerMovesTimeoutInLatestIteration(), - findTargetStrategy.getSizeEnteringNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ), - findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ) + sizeEnteringDataToNodes, + sizeLeavingDataFromNodes ); iterationsStatistic.add(iterationStatistic); } public List getCurrentIterationsStatistic() { - int lastIterationNumber = iterationsStatistic.isEmpty() ? 0 + int lastIterationNumber = iterationsStatistic.isEmpty() + ? 0 : iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); long iterationDuration = getCurrentIterationDuration(); + Map sizeEnteringDataToNodes = findTargetStrategy.getSizeEnteringNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect(Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ); + Map sizeLeavingDataFromNodes = findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + Map.Entry::getValue + ) + ); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber, null, @@ -362,27 +387,8 @@ public List getCurrentIterationsStatis metrics.getNumContainerMovesCompletedInLatestIteration(), metrics.getNumContainerMovesFailedInLatestIteration(), metrics.getNumContainerMovesTimeoutInLatestIteration(), - findTargetStrategy.getSizeEnteringNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect(Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ), - findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ) + sizeEnteringDataToNodes, + sizeLeavingDataFromNodes ); List resultList = new ArrayList<>(iterationsStatistic); resultList.add(currentIterationStatistic); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 288911127397..122e74c2014d 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -69,7 +69,7 @@ public ContainerBalancerTaskIterationStatusInfo( } /** - * Get number of iterations. + * Get the number of iterations. * @return iteration number */ public Integer getIterationNumber() { @@ -77,7 +77,7 @@ public Integer getIterationNumber() { } /** - * Get iteration result. + * Get the iteration result. * @return iteration result */ public String getIterationResult() { @@ -101,7 +101,7 @@ public long getDataSizeMoved() { } /** - * Get number of containers scheduled to move. + * Get the number of containers scheduled to move. * @return number of containers scheduled to move */ public long getContainerMovesScheduled() { @@ -109,7 +109,7 @@ public long getContainerMovesScheduled() { } /** - * Get number of successfully moved containers. + * Get the number of successfully moved containers. * @return number of successfully moved containers */ public long getContainerMovesCompleted() { @@ -117,7 +117,7 @@ public long getContainerMovesCompleted() { } /** - * Get number of unsuccessfully moved containers. + * Get the number of unsuccessfully moved containers. * @return number of unsuccessfully moved containers */ public long getContainerMovesFailed() { @@ -125,7 +125,7 @@ public long getContainerMovesFailed() { } /** - * Get number of moved with timeout containers. + * Get the number of moved with timeout containers. * @return number of moved with timeout containers */ public long getContainerMovesTimeout() { @@ -133,7 +133,7 @@ public long getContainerMovesTimeout() { } /** - * Get nodeId to size entering from node map. + * Get a map of the id node and the data size moved to the node. * @return nodeId to size entering from node map */ public Map getSizeEnteringNodes() { @@ -141,7 +141,7 @@ public Map getSizeEnteringNodes() { } /** - * Get nodeId to size leaving from node map. + * Get a map of the id node and the data size moved from the node. * @return nodeId to size leaving from node map */ public Map getSizeLeavingNodes() { @@ -149,7 +149,7 @@ public Map getSizeLeavingNodes() { } /** - * Get iteration duration + * Get iteration duration. * @return iteration duration */ public long getIterationDuration() { From 33af9b13439c1b027187bfacdbd1e5b5fc5b7862 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 17 Oct 2024 11:11:31 +0300 Subject: [PATCH 11/39] HDDS-11367. Improve javadocs --- .../balancer/ContainerBalancerMetrics.java | 14 +++++++------- .../balancer/ContainerBalancerStatusInfo.java | 2 +- ...ntainerBalancerTaskIterationStatusInfo.java | 18 +++++++++--------- .../apache/hadoop/hdds/util/package-info.java | 1 - 4 files changed, 17 insertions(+), 18 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index c1d46ec67df7..2c290180466f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -40,7 +40,7 @@ public final class ContainerBalancerMetrics { " in the latest iteration.") private MutableCounterLong dataSizeMovedGBInLatestIteration; - @Metric(about = "Amount of bytes that Container Balancer moved in the latest iteration.") + @Metric(about = "Amount of bytes that the Container Balancer moved in the latest iteration.") private MutableCounterLong dataSizeMovedBytesInLatestIteration; @Metric(about = "Number of completed container moves performed by " + @@ -143,7 +143,7 @@ public void resetNumContainerMovesScheduledInLatestIteration() { } /** - * Gets the amount of data moved by Container Balancer in the latest iteration. + * Retrieves the amount of data moved by the Container Balancer in the latest iteration. * @return size in GB */ public long getDataSizeMovedGBInLatestIteration() { @@ -160,7 +160,7 @@ public void resetDataSizeMovedGBInLatestIteration() { } /** - * Gets the amount of data moved by Container Balancer in the latest iteration. + * Retrieves the amount of data moved by the Container Balancer in the latest iteration. * @return size in bytes */ public long getDataSizeMovedInLatestIteration() { @@ -168,7 +168,7 @@ public long getDataSizeMovedInLatestIteration() { } /** - * Increment data size moved in the last iteration. + * Increment the amount of data moved in the last iteration. * @param bytes bytes to add */ public void incrementDataSizeMovedInLatestIteration(long bytes) { @@ -176,7 +176,7 @@ public void incrementDataSizeMovedInLatestIteration(long bytes) { } /** - * Reset data size moved in the last iteration. + * Reset the amount of data moved in the last iteration. */ public void resetDataSizeMovedInLatestIteration() { dataSizeMovedBytesInLatestIteration.incr(-getDataSizeMovedInLatestIteration()); @@ -244,7 +244,7 @@ public long getNumContainerMovesTimeoutInLatestIteration() { } /** - * Increase the number of containers that are transferred and finished with the timeout. + * Increase the number of containers that are transferred and completed with the timeout. */ public void incrementNumContainerMovesTimeoutInLatestIteration( long valueToAdd) { @@ -252,7 +252,7 @@ public void incrementNumContainerMovesTimeoutInLatestIteration( } /** - * Reset the number of containers that are transferred and finished with the timeout. + * Reset the number of containers that are transferred and completed with the timeout. */ public void resetNumContainerMovesTimeoutInLatestIteration() { numContainerMovesTimeoutInLatestIteration.incr(-getNumContainerMovesTimeoutInLatestIteration()); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index 742aa8c5719c..bb908e67acc9 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -55,7 +55,7 @@ public List getIterationsStatusInfo() } /** - * Converts an instance into the protobuf compatible object. + * Converts an instance into a protobuf-compatible object. * @return proto representation */ public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProto() { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 122e74c2014d..ee8791ca31fd 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -85,7 +85,7 @@ public String getIterationResult() { } /** - * Get size in bytes scheduled to move in the iteration. + * Get the size of the bytes that are scheduled to move in the iteration. * @return size in bytes */ public long getSizeScheduledForMove() { @@ -93,7 +93,7 @@ public long getSizeScheduledForMove() { } /** - * Get size in bytes moved in the iteration. + * Get the size of the bytes that were moved in the iteration. * @return size in bytes */ public long getDataSizeMoved() { @@ -117,7 +117,7 @@ public long getContainerMovesCompleted() { } /** - * Get the number of unsuccessfully moved containers. + * Get the number of containers that were not moved successfully. * @return number of unsuccessfully moved containers */ public long getContainerMovesFailed() { @@ -125,7 +125,7 @@ public long getContainerMovesFailed() { } /** - * Get the number of moved with timeout containers. + * Get the number of containers moved with a timeout. * @return number of moved with timeout containers */ public long getContainerMovesTimeout() { @@ -133,7 +133,7 @@ public long getContainerMovesTimeout() { } /** - * Get a map of the id node and the data size moved to the node. + * Get a map of the node IDs and the corresponding data sizes moved to each node. * @return nodeId to size entering from node map */ public Map getSizeEnteringNodes() { @@ -141,7 +141,7 @@ public Map getSizeEnteringNodes() { } /** - * Get a map of the id node and the data size moved from the node. + * Get a map of the node IDs and the corresponding data sizes moved from each node. * @return nodeId to size leaving from node map */ public Map getSizeLeavingNodes() { @@ -149,7 +149,7 @@ public Map getSizeLeavingNodes() { } /** - * Get iteration duration. + * Get the iteration duration. * @return iteration duration */ public long getIterationDuration() { @@ -157,7 +157,7 @@ public long getIterationDuration() { } /** - * Map to proto. + * Converts an instance into the protobuf compatible object. * @return proto representation */ public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo toProto() { @@ -181,7 +181,7 @@ public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStat } /** - * Map to proto node transfer info. + * Converts an instance into the protobuf compatible object. * @param nodes node id to node traffic size * @return node transfer info proto representation */ diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java index fa0d3d464125..6dd25c12c536 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/package-info.java @@ -15,7 +15,6 @@ * See the License for the specific language governing permissions and * limitations under the License. *

- * SCM related cli tools. */ /** * SCM related cli utils. From 8ff20eebb89d272648039831677e86ab89de02f5 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 17 Oct 2024 20:53:31 +0300 Subject: [PATCH 12/39] HDDS-11367. fix timeouts in balancer test --- .../hdds/scm/container/balancer/ContainerBalancerTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index df40e8d9cdaa..c91636bcc782 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -267,7 +267,7 @@ private void balance() { } IterationResult iR = doIteration(); - saveIterationStatistic(++i, iR); + saveIterationStatistic(i + 1, iR); metrics.incrementNumIterations(1); LOG.info("Result of this iteration of Container Balancer: {}", iR); From 5284e6b23b7d77560942c0ef9adcf61befbdea76 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sat, 19 Oct 2024 00:12:00 +0300 Subject: [PATCH 13/39] HDDS-11367. fix robo test --- .../main/smoketest/balancer/testBalancer.robot | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index a66bdcaf8163..4fa6f50663a8 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -72,12 +72,12 @@ Wait Finish Of Balancing Sleep 60000ms Verify Balancer Iteration - [arguments] ${output} ${number} ${containers} + [arguments] ${output} ${number} Should Contain ${output} ContainerBalancer is Running. Should Contain ${output} Started at: Should Contain ${output} Container Balancer Configuration values: Should Contain ${output} Iteration number ${number} collapse_spaces=True - Should Contain ${output} Scheduled to move containers ${containers} collapse_spaces=True + Should Contain ${output} Scheduled to move containers collapse_spaces=True Should Contain ${output} Balancing duration: Should Contain ${output} Iteration duration Should Contain ${output} Current iteration info: @@ -85,10 +85,10 @@ Verify Balancer Iteration Verify Balancer Iteration History [arguments] ${output} Should Contain ${output} Iteration history list: - Should Contain X Times ${output} Size scheduled to move 300 MB 2 collapse_spaces=True - Should Contain X Times ${output} Moved data size 300 MB 2 collapse_spaces=True - Should Contain X Times ${output} Scheduled to move containers 3 2 collapse_spaces=True - Should Contain X Times ${output} Already moved containers 3 2 collapse_spaces=True + Should Contain X Times ${output} Size scheduled to move 2 collapse_spaces=True + Should Contain X Times ${output} Moved data size 2 collapse_spaces=True + Should Contain X Times ${output} Scheduled to move containers 2 collapse_spaces=True + Should Contain X Times ${output} Already moved containers 2 collapse_spaces=True Should Contain X Times ${output} Failed to move containers 0 2 collapse_spaces=True Should Contain X Times ${output} Failed to move containers by timeout 0 2 collapse_spaces=True Should Contain ${output} Iteration result ITERATION_COMPLETED collapse_spaces=True @@ -99,13 +99,13 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v - Verify Balancer Iteration ${result} - 3 + Verify Balancer Iteration ${result} - Should Contain ${result} Iteration result IN_PROGRESS collapse_spaces=True Run Balancer Verbose History Status ${result} = Execute ozone admin containerbalancer status -v --history - Verify Balancer Iteration ${result} 1 3 + Verify Balancer Iteration ${result} 1 Verify Balancer Iteration History ${result} ContainerBalancer is Not Running From 051ea1d9506d1a952ad2c9e0b90ccadb4902c056 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sat, 19 Oct 2024 01:58:08 +0300 Subject: [PATCH 14/39] HDDS-11367. fix robo test --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 4fa6f50663a8..6bfa37899ebd 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -177,7 +177,7 @@ Verify Container Balancer for RATIS/EC containers Run Balancer Verbose Status - Sleep 30000ms + Sleep 60000ms Run Balancer Verbose History Status From 49b6e146a490b0b83a098935be20dae6074ac9f4 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sat, 19 Oct 2024 12:02:58 +0300 Subject: [PATCH 15/39] HDDS-11367. fix robo test --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 6bfa37899ebd..4fa6f50663a8 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -177,7 +177,7 @@ Verify Container Balancer for RATIS/EC containers Run Balancer Verbose Status - Sleep 60000ms + Sleep 30000ms Run Balancer Verbose History Status From 1513b9c4a2bd22e75b74ffbe10f5978680fcac6c Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sat, 19 Oct 2024 13:34:50 +0300 Subject: [PATCH 16/39] HDDS-11367. fix robo test --- hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config index 68ae62114c3a..5d426d6bc6d9 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config @@ -53,7 +53,7 @@ OZONE-SITE.XML_ozone.om.s3.grpc.server_enabled=true OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon OZONE-SITE.XML_dfs.container.ratis.datastream.enabled=true OZONE-SITE.XML_ozone.http.basedir=/tmp/ozone_http -OZONE-SITE.XML_hdds.container.balancer.balancing.iteration.interval=30s +OZONE-SITE.XML_hdds.container.balancer.balancing.iteration.interval=25s OZONE_CONF_DIR=/etc/hadoop OZONE_LOG_DIR=/var/log/hadoop From 16683a28f133a440d63b26e3d0b980768badc23e Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sun, 20 Oct 2024 01:31:30 +0300 Subject: [PATCH 17/39] HDDS-11367. fix robo test --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 4fa6f50663a8..8cf64a596e20 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -177,7 +177,7 @@ Verify Container Balancer for RATIS/EC containers Run Balancer Verbose Status - Sleep 30000ms + Sleep 90000ms Run Balancer Verbose History Status From 899f314e878f65894461476a88c69d20f95aa0d3 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sun, 20 Oct 2024 12:08:15 +0300 Subject: [PATCH 18/39] HDDS-11367. fix robo test --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 8cf64a596e20..549d2133a5d2 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -177,7 +177,7 @@ Verify Container Balancer for RATIS/EC containers Run Balancer Verbose Status - Sleep 90000ms + Sleep 40000ms Run Balancer Verbose History Status From c8e21d978453e2ef331c3e66918dcdd270d7874e Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 24 Oct 2024 10:38:54 +0300 Subject: [PATCH 19/39] HDDS-11367. fix review notices --- .../balancer/ContainerBalancerMetrics.java | 4 +- .../balancer/ContainerBalancerTask.java | 60 +++++++------------ .../balancer/FindSourceStrategy.java | 4 ++ .../balancer/FindTargetStrategy.java | 4 ++ .../apache/hadoop/hdds/util/DurationUtil.java | 2 +- 5 files changed, 33 insertions(+), 41 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index 2c290180466f..e1748b25e0b2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -244,7 +244,7 @@ public long getNumContainerMovesTimeoutInLatestIteration() { } /** - * Increase the number of containers that are transferred and completed with the timeout. + * Increases the number of containers that transfer completed with a timeout. */ public void incrementNumContainerMovesTimeoutInLatestIteration( long valueToAdd) { @@ -252,7 +252,7 @@ public void incrementNumContainerMovesTimeoutInLatestIteration( } /** - * Reset the number of containers that are transferred and completed with the timeout. + * Resets the number of containers that transfer completed with the timeout. */ public void resetNumContainerMovesTimeoutInLatestIteration() { numContainerMovesTimeoutInLatestIteration.incr(-getNumContainerMovesTimeoutInLatestIteration()); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 532b11888393..67adfd216d0f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -123,6 +123,7 @@ public class ContainerBalancerTask implements Runnable { private boolean delayStart; private List iterationsStatistic; private OffsetDateTime currentIterationStarted; + /** * Constructs ContainerBalancerTask with the specified arguments. * @@ -313,28 +314,10 @@ private void balance() { private void saveIterationStatistic(Integer iterationNumber, IterationResult currentIterationResult) { long iterationDuration = now().toEpochSecond() - currentIterationStarted.toEpochSecond(); - Map sizeEnteringDataToNodes = findTargetStrategy.getSizeEnteringNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ); - Map sizeLeavingDataFromNodes = findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ); + Map sizeEnteringDataToNodes = + convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); + Map sizeLeavingDataFromNodes = + convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes()); ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( iterationNumber, currentIterationResult.name(), @@ -351,22 +334,8 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult cur iterationsStatistic.add(iterationStatistic); } - public List getCurrentIterationsStatistic() { - int lastIterationNumber = iterationsStatistic.isEmpty() - ? 0 - : iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); - long iterationDuration = getCurrentIterationDuration(); - Map sizeEnteringDataToNodes = findTargetStrategy.getSizeEnteringNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect(Collectors.toMap( - entry -> entry.getKey().getUuid(), - Map.Entry::getValue - ) - ); - Map sizeLeavingDataFromNodes = findSourceStrategy.getSizeLeavingNodes() + private Map convertToNodeIdToTrafficMap(Map nodeTrafficMap) { + return nodeTrafficMap .entrySet() .stream() .filter(Objects::nonNull) @@ -377,6 +346,21 @@ public List getCurrentIterationsStatis Map.Entry::getValue ) ); + } + + /** + * Get current iteration statistics. + * @return current iteration statistic + */ + public List getCurrentIterationsStatistic() { + int lastIterationNumber = iterationsStatistic.isEmpty() + ? 0 + : iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); + long iterationDuration = getCurrentIterationDuration(); + Map sizeEnteringDataToNodes = + convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); + Map sizeLeavingDataFromNodes = + convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes()); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber, null, diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java index 9e429aaa21d9..cd41abd6c0bb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java @@ -87,5 +87,9 @@ void reInitialize(List potentialDataNodes, */ void resetPotentialSources(@Nonnull Collection sources); + /** + * Get a map of the node IDs and the corresponding data sizes moved from each node. + * @return nodeId to size leaving from node map + */ Map getSizeLeavingNodes(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java index 389ea6e5192f..4190143d7399 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java @@ -70,5 +70,9 @@ void reInitialize(List potentialDataNodes, */ void resetPotentialTargets(@Nonnull Collection targets); + /** + * Get a map of the node IDs and the corresponding data sizes moved to each node. + * @return nodeId to size entering from node map + */ Map getSizeEnteringNodes(); } diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java index 45f8eb6ab734..7b2ded9b13da 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/util/DurationUtil.java @@ -46,7 +46,7 @@ public static String getPrettyDuration(Duration duration) { } else if (seconds >= 0) { return format("%ds", seconds); } else { - throw new IllegalStateException("Incorrect duration exception" + duration); + throw new IllegalStateException("Provided duration is incorrect: " + duration); } } } From b0cb7600b154c03e3446c7d79bf448a1dec5d644 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 25 Oct 2024 11:11:29 +0300 Subject: [PATCH 20/39] HDDS-11367. Fix review comments --- .../balancer/ContainerBalancerTask.java | 37 ++++++---- ...tainerBalancerTaskIterationStatusInfo.java | 68 ++++++------------- .../container/balancer/ContainerMoveInfo.java | 34 ++++++++++ .../scm/container/balancer/DataMoveInfo.java | 42 ++++++++++++ .../scm/container/balancer/IterationInfo.java | 29 ++++++++ 5 files changed, 150 insertions(+), 60 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 67adfd216d0f..fc39514d24ad 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -318,19 +318,23 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult cur convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); Map sizeLeavingDataFromNodes = convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes()); - ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + IterationInfo iterationInfo = new IterationInfo( iterationNumber, currentIterationResult.name(), - iterationDuration, + iterationDuration + ); + ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(metrics); + DataMoveInfo dataMoveInfo = new DataMoveInfo( getSizeScheduledForMoveInLatestIteration(), metrics.getDataSizeMovedInLatestIteration(), - metrics.getNumContainerMovesScheduledInLatestIteration(), - metrics.getNumContainerMovesCompletedInLatestIteration(), - metrics.getNumContainerMovesFailedInLatestIteration(), - metrics.getNumContainerMovesTimeoutInLatestIteration(), sizeEnteringDataToNodes, sizeLeavingDataFromNodes ); + ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + iterationInfo, + containerMoveInfo, + dataMoveInfo + ); iterationsStatistic.add(iterationStatistic); } @@ -361,19 +365,24 @@ public List getCurrentIterationsStatis convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); Map sizeLeavingDataFromNodes = convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes()); - ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - lastIterationNumber, - null, - iterationDuration, + + ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(metrics); + DataMoveInfo dataMoveInfo = new DataMoveInfo( getSizeScheduledForMoveInLatestIteration(), sizeActuallyMovedInLatestIteration, - metrics.getNumContainerMovesScheduledInLatestIteration(), - metrics.getNumContainerMovesCompletedInLatestIteration(), - metrics.getNumContainerMovesFailedInLatestIteration(), - metrics.getNumContainerMovesTimeoutInLatestIteration(), sizeEnteringDataToNodes, sizeLeavingDataFromNodes ); + IterationInfo iterationInfo = new IterationInfo( + lastIterationNumber, + null, + iterationDuration + ); + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + iterationInfo, + containerMoveInfo, + dataMoveInfo + ); List resultList = new ArrayList<>(iterationsStatistic); resultList.add(currentIterationStatistic); return resultList; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index ee8791ca31fd..bb58849286ae 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -30,42 +30,18 @@ * Information about balancer task iteration. */ public class ContainerBalancerTaskIterationStatusInfo { - private final Integer iterationNumber; - private final String iterationResult; - private final long iterationDuration; - private final long sizeScheduledForMove; - private final long dataSizeMoved; - private final long containerMovesScheduled; - private final long containerMovesCompleted; - private final long containerMovesFailed; - private final long containerMovesTimeout; - private final Map sizeEnteringNodes; - private final Map sizeLeavingNodes; - - @SuppressWarnings("checkstyle:ParameterNumber") + + private final IterationInfo iterationInfo; + private final ContainerMoveInfo containerMoveInfo; + private final DataMoveInfo dataMoveInfo; + public ContainerBalancerTaskIterationStatusInfo( - Integer iterationNumber, - String iterationResult, - long iterationDuration, - long sizeScheduledForMove, - long dataSizeMoved, - long containerMovesScheduled, - long containerMovesCompleted, - long containerMovesFailed, - long containerMovesTimeout, - Map sizeEnteringNodes, - Map sizeLeavingNodes) { - this.iterationNumber = iterationNumber; - this.iterationResult = iterationResult; - this.iterationDuration = iterationDuration; - this.sizeScheduledForMove = sizeScheduledForMove; - this.dataSizeMoved = dataSizeMoved; - this.containerMovesScheduled = containerMovesScheduled; - this.containerMovesCompleted = containerMovesCompleted; - this.containerMovesFailed = containerMovesFailed; - this.containerMovesTimeout = containerMovesTimeout; - this.sizeEnteringNodes = sizeEnteringNodes; - this.sizeLeavingNodes = sizeLeavingNodes; + IterationInfo iterationInfo, + ContainerMoveInfo containerMoveInfo, + DataMoveInfo dataMoveInfo) { + this.iterationInfo = iterationInfo; + this.containerMoveInfo = containerMoveInfo; + this.dataMoveInfo = dataMoveInfo; } /** @@ -73,7 +49,7 @@ public ContainerBalancerTaskIterationStatusInfo( * @return iteration number */ public Integer getIterationNumber() { - return iterationNumber; + return iterationInfo.getIterationNumber(); } /** @@ -81,7 +57,7 @@ public Integer getIterationNumber() { * @return iteration result */ public String getIterationResult() { - return iterationResult; + return iterationInfo.getIterationResult(); } /** @@ -89,7 +65,7 @@ public String getIterationResult() { * @return size in bytes */ public long getSizeScheduledForMove() { - return sizeScheduledForMove; + return dataMoveInfo.getSizeScheduledForMove(); } /** @@ -97,7 +73,7 @@ public long getSizeScheduledForMove() { * @return size in bytes */ public long getDataSizeMoved() { - return dataSizeMoved; + return dataMoveInfo.getDataSizeMoved(); } /** @@ -105,7 +81,7 @@ public long getDataSizeMoved() { * @return number of containers scheduled to move */ public long getContainerMovesScheduled() { - return containerMovesScheduled; + return containerMoveInfo.getContainerMovesScheduled(); } /** @@ -113,7 +89,7 @@ public long getContainerMovesScheduled() { * @return number of successfully moved containers */ public long getContainerMovesCompleted() { - return containerMovesCompleted; + return containerMoveInfo.getContainerMovesCompleted(); } /** @@ -121,7 +97,7 @@ public long getContainerMovesCompleted() { * @return number of unsuccessfully moved containers */ public long getContainerMovesFailed() { - return containerMovesFailed; + return containerMoveInfo.getContainerMovesFailed(); } /** @@ -129,7 +105,7 @@ public long getContainerMovesFailed() { * @return number of moved with timeout containers */ public long getContainerMovesTimeout() { - return containerMovesTimeout; + return containerMoveInfo.getContainerMovesTimeout(); } /** @@ -137,7 +113,7 @@ public long getContainerMovesTimeout() { * @return nodeId to size entering from node map */ public Map getSizeEnteringNodes() { - return sizeEnteringNodes; + return dataMoveInfo.getSizeEnteringNodes(); } /** @@ -145,7 +121,7 @@ public Map getSizeEnteringNodes() { * @return nodeId to size leaving from node map */ public Map getSizeLeavingNodes() { - return sizeLeavingNodes; + return dataMoveInfo.getSizeLeavingNodes(); } /** @@ -153,7 +129,7 @@ public Map getSizeLeavingNodes() { * @return iteration duration */ public long getIterationDuration() { - return iterationDuration; + return iterationInfo.getIterationDuration(); } /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java new file mode 100644 index 000000000000..f89b8d130a00 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java @@ -0,0 +1,34 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +/** + * Information about moving containers. + */ +public class ContainerMoveInfo { + private final long containerMovesScheduled; + private final long containerMovesCompleted; + private final long containerMovesFailed; + private final long containerMovesTimeout; + + public ContainerMoveInfo(ContainerBalancerMetrics metrics) { + this.containerMovesScheduled = metrics.getNumContainerMovesScheduledInLatestIteration(); + this.containerMovesCompleted = metrics.getNumContainerMovesCompletedInLatestIteration(); + this.containerMovesFailed = metrics.getNumContainerMovesFailedInLatestIteration(); + this.containerMovesTimeout = metrics.getNumContainerMovesTimeoutInLatestIteration(); + } + + public long getContainerMovesScheduled() { + return containerMovesScheduled; + } + + public long getContainerMovesCompleted() { + return containerMovesCompleted; + } + + public long getContainerMovesFailed() { + return containerMovesFailed; + } + + public long getContainerMovesTimeout() { + return containerMovesTimeout; + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java new file mode 100644 index 000000000000..b129ff4cdd72 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java @@ -0,0 +1,42 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +import java.util.Map; +import java.util.UUID; + +/** + * Information about the process of moving data. + */ +public class DataMoveInfo { + private final long sizeScheduledForMove; + private final long dataSizeMoved; + private final Map sizeEnteringNodes; + private final Map sizeLeavingNodes; + + + public DataMoveInfo( + long sizeScheduledForMove, + long dataSizeMoved, + Map sizeEnteringNodes, + Map sizeLeavingNodes) { + this.sizeScheduledForMove = sizeScheduledForMove; + this.dataSizeMoved = dataSizeMoved; + this.sizeEnteringNodes = sizeEnteringNodes; + this.sizeLeavingNodes = sizeLeavingNodes; + } + + public long getSizeScheduledForMove() { + return sizeScheduledForMove; + } + + public long getDataSizeMoved() { + return dataSizeMoved; + } + + public Map getSizeEnteringNodes() { + return sizeEnteringNodes; + } + + public Map getSizeLeavingNodes() { + return sizeLeavingNodes; + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java new file mode 100644 index 000000000000..a3cd6c81ec52 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java @@ -0,0 +1,29 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +/** + * Information about the process of moving data. + */ +public class IterationInfo { + + private final Integer iterationNumber; + private final String iterationResult; + private final long iterationDuration; + + public IterationInfo(Integer iterationNumber, String iterationResult, long iterationDuration) { + this.iterationNumber = iterationNumber; + this.iterationResult = iterationResult; + this.iterationDuration = iterationDuration; + } + + public Integer getIterationNumber() { + return iterationNumber; + } + + public String getIterationResult() { + return iterationResult; + } + + public long getIterationDuration() { + return iterationDuration; + } +} From d682cccb2d45c210a28719c99bdd0b83e7ef3457 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 25 Oct 2024 20:51:08 +0300 Subject: [PATCH 21/39] HDDS-11367. Add licence info to classes --- .../container/balancer/ContainerMoveInfo.java | 18 ++++++++++++++++++ .../scm/container/balancer/DataMoveInfo.java | 18 ++++++++++++++++++ .../scm/container/balancer/IterationInfo.java | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java index f89b8d130a00..5ee8026c3d7f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.hdds.scm.container.balancer; /** diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java index b129ff4cdd72..cd97011768da 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/DataMoveInfo.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.hdds.scm.container.balancer; import java.util.Map; diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java index a3cd6c81ec52..93542d1ea5cd 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java @@ -1,3 +1,21 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + package org.apache.hadoop.hdds.scm.container.balancer; /** From 2c4e1f45fdbdbb35d3fba07104a4774faa1b69e4 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 28 Oct 2024 19:36:40 +0300 Subject: [PATCH 22/39] HDDS-11367. Partly fix review notices --- .../src/main/proto/ScmAdminProtocol.proto | 14 +++---- .../balancer/ContainerBalancerMetrics.java | 4 +- .../balancer/ContainerBalancerStatusInfo.java | 4 +- ...tainerBalancerTaskIterationStatusInfo.java | 8 ++-- .../scm/container/balancer/IterationInfo.java | 2 +- .../ContainerBalancerStatusSubcommand.java | 10 ++--- .../TestContainerBalancerSubCommand.java | 40 +++++++++---------- ...ionUtilTest.java => TestDurationUtil.java} | 2 +- 8 files changed, 42 insertions(+), 42 deletions(-) rename hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/{DurationUtilTest.java => TestDurationUtil.java} (99%) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index b0cf52c1549d..b487c7c7cecb 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -617,15 +617,15 @@ message ContainerBalancerStatusInfoRequestProto { message ContainerBalancerStatusInfoResponseProto { optional bool isRunning = 1; - optional ContainerBalancerStatusInfo containerBalancerStatusInfo = 2; + optional ContainerBalancerStatusInfoProto containerBalancerStatusInfo = 2; } -message ContainerBalancerStatusInfo { +message ContainerBalancerStatusInfoProto { optional uint64 startedAt = 1; optional ContainerBalancerConfigurationProto configuration = 2; - repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; + repeated ContainerBalancerTaskIterationStatusInfoProto iterationsStatusInfo = 3; } -message ContainerBalancerTaskIterationStatusInfo { +message ContainerBalancerTaskIterationStatusInfoProto { optional int32 iterationNumber = 1; optional string iterationResult = 2; optional int64 sizeScheduledForMove = 3; @@ -634,12 +634,12 @@ message ContainerBalancerTaskIterationStatusInfo { optional int64 containerMovesCompleted = 6; optional int64 containerMovesFailed = 7; optional int64 containerMovesTimeout = 8; - repeated NodeTransferInfo sizeEnteringNodes = 9; - repeated NodeTransferInfo sizeLeavingNodes = 10; + repeated NodeTransferInfoProto sizeEnteringNodes = 9; + repeated NodeTransferInfoProto sizeLeavingNodes = 10; optional int64 iterationDuration = 11; } -message NodeTransferInfo { +message NodeTransferInfoProto { optional string uuid = 1; optional int64 dataVolume = 2; } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java index e1748b25e0b2..3e164cb0bba3 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerMetrics.java @@ -244,7 +244,7 @@ public long getNumContainerMovesTimeoutInLatestIteration() { } /** - * Increases the number of containers that transfer completed with a timeout. + * Increases the number of timeout container moves in the latest iteration. */ public void incrementNumContainerMovesTimeoutInLatestIteration( long valueToAdd) { @@ -252,7 +252,7 @@ public void incrementNumContainerMovesTimeoutInLatestIteration( } /** - * Resets the number of containers that transfer completed with the timeout. + * Reset the number of timeout container moves in the latest iteration. */ public void resetNumContainerMovesTimeoutInLatestIteration() { numContainerMovesTimeoutInLatestIteration.incr(-getNumContainerMovesTimeoutInLatestIteration()); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java index bb908e67acc9..a0552142b3be 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -58,8 +58,8 @@ public List getIterationsStatusInfo() * Converts an instance into a protobuf-compatible object. * @return proto representation */ - public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo toProto() { - return StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo + public StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoProto toProto() { + return StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoProto .newBuilder() .setStartedAt(getStartedAt().toEpochSecond()) .setConfiguration(getConfiguration()) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index bb58849286ae..09c1174a66c9 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -136,8 +136,8 @@ public long getIterationDuration() { * Converts an instance into the protobuf compatible object. * @return proto representation */ - public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo toProto() { - return StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto toProto() { + return StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto.newBuilder() .setIterationNumber(getIterationNumber()) .setIterationResult(Optional.ofNullable(getIterationResult()).orElse("")) .setIterationDuration(getIterationDuration()) @@ -161,12 +161,12 @@ public StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStat * @param nodes node id to node traffic size * @return node transfer info proto representation */ - private List mapToProtoNodeTransferInfo( + private List mapToProtoNodeTransferInfo( Map nodes ) { return nodes.entrySet() .stream() - .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid(entry.getKey().toString()) .setDataVolume(entry.getValue()) .build() diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java index 93542d1ea5cd..ad0445ef800a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java @@ -19,7 +19,7 @@ package org.apache.hadoop.hdds.scm.container.balancer; /** - * Information about the process of moving data. + * Information about the iteration. */ public class IterationInfo { diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index beadef9e78d8..ff61a0eb1bcc 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -19,9 +19,9 @@ import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.ozone.OzoneConsts; import picocli.CommandLine; @@ -62,7 +62,7 @@ public class ContainerBalancerStatusSubcommand extends ScmSubcommand { public void execute(ScmClient scmClient) throws IOException { ContainerBalancerStatusInfoResponseProto response = scmClient.getContainerBalancerStatusInfo(); boolean isRunning = response.getIsRunning(); - ContainerBalancerStatusInfo balancerStatusInfo = response.getContainerBalancerStatusInfo(); + ContainerBalancerStatusInfoProto balancerStatusInfo = response.getContainerBalancerStatusInfo(); if (isRunning) { Instant startedAtInstant = Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()); LocalDateTime dateTime = @@ -74,7 +74,7 @@ public void execute(ScmClient scmClient) throws IOException { Duration balancingDuration = Duration.between(startedAtInstant, OffsetDateTime.now()); System.out.printf("Balancing duration: %s%n%n", getPrettyDuration(balancingDuration)); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); - List iterationsStatusInfoList + List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); System.out.println("Current iteration info:"); @@ -143,7 +143,7 @@ String getConfigurationPrettyString(HddsProtos.ContainerBalancerConfigurationPro configuration.getExcludeDatanodes().isEmpty() ? "None" : configuration.getExcludeDatanodes()); } - private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { + private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatusInfoProto iterationStatusInfo) { int iterationNumber = iterationStatusInfo.getIterationNumber(); String iterationResult = iterationStatusInfo.getIterationResult(); long iterationDuration = iterationStatusInfo.getIterationDuration(); diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index 4f765ea26c0a..34dfa8203755 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -18,7 +18,7 @@ package org.apache.hadoop.hdds.scm.cli.datanode; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; -import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStartSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStatusSubcommand; @@ -63,8 +63,8 @@ class TestContainerBalancerSubCommand { private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfoResponseProto( ContainerBalancerConfiguration config) { - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto iteration1StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto.newBuilder() .setIterationNumber(1) .setIterationResult("ITERATION_COMPLETED") .setIterationDuration(400L) @@ -75,32 +75,32 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat .setContainerMovesFailed(0) .setContainerMovesTimeout(0) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") .setDataVolume(28 * GB) .build() ) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") .setDataVolume(26 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") .setDataVolume(25 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") .setDataVolume(29 * GB) .build() ) .build(); - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto iteration2StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto.newBuilder() .setIterationNumber(2) .setIterationResult("ITERATION_COMPLETED") .setIterationDuration(300L) @@ -111,32 +111,32 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat .setContainerMovesFailed(0) .setContainerMovesTimeout(0) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") .setDataVolume(20 * GB) .build() ) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") .setDataVolume(10 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") .setDataVolume(15 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") .setDataVolume(15 * GB) .build() ) .build(); - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration3StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto iteration3StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto.newBuilder() .setIterationNumber(3) .setIterationResult("") .setIterationDuration(4000L) @@ -147,25 +147,25 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat .setContainerMovesFailed(0) .setContainerMovesTimeout(0) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") .setDataVolume(20 * GB) .build() ) .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") .setDataVolume(28 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") .setDataVolume(30 * GB) .build() ) .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + StorageContainerLocationProtocolProtos.NodeTransferInfoProto.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") .setDataVolume(18 * GB) .build() @@ -174,7 +174,7 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = ContainerBalancerStatusInfoResponseProto.newBuilder() .setIsRunning(true) - .setContainerBalancerStatusInfo(ContainerBalancerStatusInfo.newBuilder() + .setContainerBalancerStatusInfo(ContainerBalancerStatusInfoProto.newBuilder() .setStartedAt(OffsetDateTime.now().toEpochSecond()) .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) .addAllIterationsStatusInfo( diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java similarity index 99% rename from hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java rename to hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java index 1ffb0b477793..3498f04416de 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/DurationUtilTest.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java @@ -30,7 +30,7 @@ import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.params.provider.Arguments.arguments; -class DurationUtilTest { +class TestDurationUtil { private static Stream paramsForPositiveCases() { return Stream.of( From ad0c95d048ca59960563111d53df955b671356b9 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 30 Oct 2024 13:52:52 +0300 Subject: [PATCH 23/39] HDDS-11367. Fix review notices --- .../balancer/AbstractFindTargetGreedy.java | 5 + .../balancer/ContainerBalancerTask.java | 7 +- ...tainerBalancerTaskIterationStatusInfo.java | 2 +- .../container/balancer/FindSourceGreedy.java | 5 + .../balancer/FindSourceStrategy.java | 5 + .../balancer/FindTargetStrategy.java | 5 + .../scm/container/balancer/IterationInfo.java | 4 +- .../scm/container/balancer/MockedSCM.java | 15 ++ .../TestContainerBalancerStatusInfo.java | 191 +++++++++++++++++- .../ContainerBalancerStatusSubcommand.java | 8 +- .../TestContainerBalancerSubCommand.java | 56 ++--- .../main/compose/ozone-balancer/docker-config | 1 + 12 files changed, 251 insertions(+), 53 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java index dd2d1c578940..38ef1c5b983a 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/AbstractFindTargetGreedy.java @@ -283,4 +283,9 @@ NodeManager getNodeManager() { public Map getSizeEnteringNodes() { return sizeEnteringNode; } + + @Override + public void clearSizeEnteringNodes() { + sizeEnteringNode.clear(); + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index fc39514d24ad..c29dcbef7212 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -269,6 +269,11 @@ private void balance() { IterationResult iR = doIteration(); saveIterationStatistic(i + 1, iR); + + resetState(); + findTargetStrategy.clearSizeEnteringNodes(); + findSourceStrategy.clearSizeLeavingNodes(); + metrics.incrementNumIterations(1); LOG.info("Result of this iteration of Container Balancer: {}", iR); @@ -374,7 +379,7 @@ public List getCurrentIterationsStatis sizeLeavingDataFromNodes ); IterationInfo iterationInfo = new IterationInfo( - lastIterationNumber, + lastIterationNumber + 1, null, iterationDuration ); diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java index 09c1174a66c9..a466d9fd4745 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -128,7 +128,7 @@ public Map getSizeLeavingNodes() { * Get the iteration duration. * @return iteration duration */ - public long getIterationDuration() { + public Long getIterationDuration() { return iterationInfo.getIterationDuration(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java index 435cc9859a94..ac2151dfdc31 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceGreedy.java @@ -206,4 +206,9 @@ public void reInitialize(List potentialDataNodes, public Map getSizeLeavingNodes() { return sizeLeavingNode; } + + @Override + public void clearSizeLeavingNodes() { + sizeLeavingNode.clear(); + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java index cd41abd6c0bb..0043d8509b0f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindSourceStrategy.java @@ -92,4 +92,9 @@ void reInitialize(List potentialDataNodes, * @return nodeId to size leaving from node map */ Map getSizeLeavingNodes(); + + /** + * Clear the map of node IDs and their corresponding data sizes that were moved from each node. + */ + void clearSizeLeavingNodes(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java index 4190143d7399..8959fc4ff23f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/FindTargetStrategy.java @@ -75,4 +75,9 @@ void reInitialize(List potentialDataNodes, * @return nodeId to size entering from node map */ Map getSizeEnteringNodes(); + + /** + * Clear the map of node IDs and their corresponding data sizes that were moved to each node. + */ + void clearSizeEnteringNodes(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java index ad0445ef800a..615848a097ae 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/IterationInfo.java @@ -25,7 +25,7 @@ public class IterationInfo { private final Integer iterationNumber; private final String iterationResult; - private final long iterationDuration; + private final Long iterationDuration; public IterationInfo(Integer iterationNumber, String iterationResult, long iterationDuration) { this.iterationNumber = iterationNumber; @@ -41,7 +41,7 @@ public String getIterationResult() { return iterationResult; } - public long getIterationDuration() { + public Long getIterationDuration() { return iterationDuration; } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java index 0972e57df64d..6c203c13099b 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java @@ -137,11 +137,26 @@ public String toString() { return task; } + public @Nonnull ContainerBalancerTask startBalancerTaskAsync( + @Nonnull ContainerBalancer containerBalancer, + @Nonnull ContainerBalancerConfiguration config, + Boolean withDelay) { + ContainerBalancerTask task = new ContainerBalancerTask(scm, 0, containerBalancer, + containerBalancer.getMetrics(), config, withDelay); + new Thread(task).start(); + return task; + } + public @Nonnull ContainerBalancerTask startBalancerTask(@Nonnull ContainerBalancerConfiguration config) { init(config, new OzoneConfiguration()); return startBalancerTask(new ContainerBalancer(scm), config); } + public @Nonnull ContainerBalancerTask startBalancerTaskAsync(@Nonnull ContainerBalancerConfiguration config, Boolean withDelay) { + init(config, new OzoneConfiguration()); + return startBalancerTaskAsync(new ContainerBalancer(scm), config, withDelay); + } + public void enableLegacyReplicationManager() { mockedReplicaManager.conf.setEnableLegacy(true); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index a534208902c7..9b620d0e7987 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdds.scm.container.balancer; +import org.apache.commons.math3.util.ArithmeticUtils; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.OzoneConsts; @@ -25,10 +26,10 @@ import org.junit.jupiter.api.Test; import java.util.List; +import java.util.Map; +import java.util.UUID; -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertTrue; +import static org.junit.jupiter.api.Assertions.*; /** * Tests for {@link ContainerBalancerStatusInfo}. @@ -48,14 +49,186 @@ void testGetIterationStatistics() { ContainerBalancerTask task = mockedScm.startBalancerTask(config); List iterationStatistics = task.getCurrentIterationsStatistic(); assertEquals(3, iterationStatistics.size()); - iterationStatistics.forEach(is -> { - assertTrue(is.getContainerMovesCompleted() > 0); - assertEquals(0, is.getContainerMovesFailed()); - assertEquals(0, is.getContainerMovesTimeout()); - assertFalse(is.getSizeEnteringNodes().isEmpty()); - assertFalse(is.getSizeLeavingNodes().isEmpty()); + + ContainerBalancerTaskIterationStatusInfo iterationHistory1 = iterationStatistics.get(0); + verifyCompletedIteration(iterationHistory1, 1, "ITERATION_COMPLETED"); + + ContainerBalancerTaskIterationStatusInfo iterationHistory2 = iterationStatistics.get(1); + verifyCompletedIteration(iterationHistory2, 2, "ITERATION_COMPLETED"); + + ContainerBalancerTaskIterationStatusInfo currentIteration = iterationStatistics.get(2); + verifyCompletedIteration1(currentIteration, 3, null); + } + + @Test + void testReRequestIterationStatistics() throws InterruptedException { + MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); + + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + + config.setIterations(2); + config.setBalancingInterval(0); + config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + + ContainerBalancerTask task = mockedScm.startBalancerTask(config); + List firstRequestIterationStatistics = task.getCurrentIterationsStatistic(); + Thread.sleep(1000L); + List secondRequestIterationStatistics = task.getCurrentIterationsStatistic(); + assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); + assertEquals(firstRequestIterationStatistics.get(1), secondRequestIterationStatistics.get(1)); + + assertEquals(firstRequestIterationStatistics.get(2).getIterationNumber(), secondRequestIterationStatistics.get(2).getIterationNumber()); + assertNotEquals(firstRequestIterationStatistics.get(2).getIterationDuration(), secondRequestIterationStatistics.get(2).getIterationDuration()); + } + + @Test + void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws InterruptedException { + MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); + + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + + config.setIterations(2); + config.setBalancingInterval(10000); + config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); + // Delay in finishing the first iteration + Thread.sleep(1000L); + List iterationsStatic = task.getCurrentIterationsStatistic(); + assertEquals(2, iterationsStatic.size()); + + ContainerBalancerTaskIterationStatusInfo firstIteration = iterationsStatic.get(0); + verifyCompletedIteration(firstIteration, 1, "ITERATION_COMPLETED"); + + assertEquals(2, iterationsStatic.get(1).getIterationNumber()); + assertTrue(iterationsStatic.get(1).getIterationDuration() > 0); + assertNull(iterationsStatic.get(1).getIterationResult()); + assertEquals(0, iterationsStatic.get(1).getContainerMovesScheduled()); + assertEquals(0, iterationsStatic.get(1).getContainerMovesCompleted()); + assertEquals(0, iterationsStatic.get(1).getContainerMovesFailed()); + assertEquals(0, iterationsStatic.get(1).getContainerMovesTimeout()); + assertEquals(0, iterationsStatic.get(1).getSizeScheduledForMove()); + assertEquals(0, iterationsStatic.get(1).getDataSizeMoved()); + assertEquals(0, iterationsStatic.get(1).getSizeEnteringNodes().size()); + assertEquals(0, iterationsStatic.get(1).getSizeLeavingNodes().size()); + } + + @Test + void testCurrentStatisticsDoesntChangeWhenReRequestInPeriodBetweenIterations() throws InterruptedException { + MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); + + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + + config.setIterations(2); + config.setBalancingInterval(10000); + config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); + // Delay in finishing the first iteration + Thread.sleep(1000L); + List firstRequestIterationStatistics = task.getCurrentIterationsStatistic(); + // Delay occurred for some time during the period between iterations. + Thread.sleep(1000L); + List secondRequestIterationStatistics = task.getCurrentIterationsStatistic(); + assertEquals(2, firstRequestIterationStatistics.size()); + assertEquals(2, secondRequestIterationStatistics.size()); + assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); + + assertEquals(firstRequestIterationStatistics.get(1).getIterationNumber(), secondRequestIterationStatistics.get(1).getIterationNumber()); + assertNotEquals(firstRequestIterationStatistics.get(1).getIterationDuration(), secondRequestIterationStatistics.get(1).getIterationDuration()); + assertEquals(firstRequestIterationStatistics.get(1).getIterationResult(), secondRequestIterationStatistics.get(1).getIterationResult()); + assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesScheduled(), secondRequestIterationStatistics.get(1).getContainerMovesScheduled()); + assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesCompleted(), secondRequestIterationStatistics.get(1).getContainerMovesCompleted()); + assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesFailed(), secondRequestIterationStatistics.get(1).getContainerMovesFailed()); + assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesTimeout(), secondRequestIterationStatistics.get(1).getContainerMovesTimeout()); + assertEquals(firstRequestIterationStatistics.get(1).getSizeScheduledForMove(), secondRequestIterationStatistics.get(1).getSizeScheduledForMove()); + assertEquals(firstRequestIterationStatistics.get(1).getDataSizeMoved(), secondRequestIterationStatistics.get(1).getDataSizeMoved()); + assertEquals(firstRequestIterationStatistics.get(1).getSizeEnteringNodes().size(), secondRequestIterationStatistics.get(1).getSizeEnteringNodes().size()); + assertEquals(firstRequestIterationStatistics.get(1).getSizeLeavingNodes().size(), secondRequestIterationStatistics.get(1).getSizeLeavingNodes().size()); + } + + @Test + void testGetCurrentStatisticsWithDelay() throws InterruptedException { + MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); + + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + + config.setIterations(2); + config.setBalancingInterval(0); + config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, true); + // Delay in finishing the first iteration + Thread.sleep(1000L); + List iterationsStatic = task.getCurrentIterationsStatistic(); + assertEquals(1, iterationsStatic.size()); + ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(0); + assertEmptyIteration(currentIteration); + } + + private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInfo iterationsStatic) { + + assertEquals(-1, iterationsStatic.getIterationDuration()); + assertNull(iterationsStatic.getIterationResult()); + assertEquals(0, iterationsStatic.getContainerMovesScheduled()); + assertEquals(0, iterationsStatic.getContainerMovesCompleted()); + assertEquals(0, iterationsStatic.getContainerMovesFailed()); + assertEquals(0, iterationsStatic.getContainerMovesTimeout()); + assertEquals(0, iterationsStatic.getSizeScheduledForMove()); + assertEquals(0, iterationsStatic.getDataSizeMoved()); + assertEquals(0, iterationsStatic.getSizeEnteringNodes().size()); + assertEquals(0, iterationsStatic.getSizeLeavingNodes().size()); + } + + private void verifyCompletedIteration( + ContainerBalancerTaskIterationStatusInfo iteration, + Integer expectedIterationNumber, + String iterationResult + ) { + assertEquals(expectedIterationNumber, iteration.getIterationNumber()); + assertEquals("ITERATION_COMPLETED", iteration.getIterationResult()); + assertNotNull(iteration.getIterationDuration()); + assertTrue(iteration.getContainerMovesScheduled() > 0); + assertTrue(iteration.getContainerMovesCompleted() > 0); + assertEquals(0, iteration.getContainerMovesFailed()); + assertEquals(0, iteration.getContainerMovesTimeout()); + assertTrue(iteration.getSizeScheduledForMove() > 0); + assertTrue(iteration.getDataSizeMoved() > 0); + assertFalse(iteration.getSizeEnteringNodes().isEmpty()); + assertFalse(iteration.getSizeLeavingNodes().isEmpty()); + iteration.getSizeEnteringNodes().forEach((id, size) -> { + assertNotNull(id); + assertTrue(size > 0); + }); + iteration.getSizeLeavingNodes().forEach((id, size) -> { + assertNotNull(id); + assertTrue(size > 0); }); + Long enteringDataSum = getTotalMovedData(iteration.getSizeEnteringNodes()); + Long leavingDataSum = getTotalMovedData(iteration.getSizeLeavingNodes()); + assertEquals(enteringDataSum, leavingDataSum); + } + + private void verifyCompletedIteration1( + ContainerBalancerTaskIterationStatusInfo iteration, + Integer expectedIterationNumber, + String iterationResult + ) { + assertEquals(expectedIterationNumber, iteration.getIterationNumber()); + assertEquals(iterationResult, iteration.getIterationResult()); + assertNotNull(iteration.getIterationDuration()); + assertEquals(0,iteration.getContainerMovesScheduled()); + assertEquals(0,iteration.getContainerMovesCompleted()); + assertEquals(0, iteration.getContainerMovesFailed()); + assertEquals(0, iteration.getContainerMovesTimeout()); + assertEquals(0, iteration.getSizeScheduledForMove()); + assertEquals(0, iteration.getDataSizeMoved()); + assertTrue(iteration.getSizeEnteringNodes().isEmpty()); + assertTrue(iteration.getSizeLeavingNodes().isEmpty()); + } + private static Long getTotalMovedData(Map iteration) { + return iteration.values().stream().reduce(0L, ArithmeticUtils::addAndCheck); } /** diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index ff61a0eb1bcc..c3800ae3c4f3 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -156,9 +156,15 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesList() .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + byteDesc(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); + if (enteringDataNodeList.isEmpty()) { + enteringDataNodeList = " -\n"; + } String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesList() .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + byteDesc(nodeInfo.getDataVolume()) + "\n") .collect(Collectors.joining()); + if (leavingDataNodeList.isEmpty()) { + leavingDataNodeList = " -\n"; + } return String.format( "%-50s %s%n" + "%-50s %s%n" + @@ -176,7 +182,7 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "Iteration number", iterationNumber == 0 ? "-" : iterationNumber, "Iteration duration", getPrettyDuration(Duration.ofSeconds(iterationDuration)), "Iteration result", - iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, + iterationResult.isEmpty() ? "-" : iterationResult, "Size scheduled to move", byteDesc(sizeScheduledForMove), "Moved data size", byteDesc(dataSizeMoved), "Scheduled to move containers", containerMovesScheduled, diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index 34dfa8203755..8fcd17eba356 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -106,8 +106,8 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat .setIterationDuration(300L) .setSizeScheduledForMove(30 * GB) .setDataSizeMoved(30 * GB) - .setContainerMovesScheduled(11) - .setContainerMovesCompleted(11) + .setContainerMovesScheduled(8) + .setContainerMovesCompleted(8) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) .addSizeEnteringNodes( @@ -139,11 +139,11 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfoProto.newBuilder() .setIterationNumber(3) .setIterationResult("") - .setIterationDuration(4000L) + .setIterationDuration(370L) .setSizeScheduledForMove(48 * GB) .setDataSizeMoved(48 * GB) - .setContainerMovesScheduled(11) - .setContainerMovesCompleted(11) + .setContainerMovesScheduled(5) + .setContainerMovesCompleted(5) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) .addSizeEnteringNodes( @@ -171,8 +171,7 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat .build() ) .build(); - ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = - ContainerBalancerStatusInfoResponseProto.newBuilder() + return ContainerBalancerStatusInfoResponseProto.newBuilder() .setIsRunning(true) .setContainerBalancerStatusInfo(ContainerBalancerStatusInfoProto.newBuilder() .setStartedAt(OffsetDateTime.now().toEpochSecond()) @@ -183,7 +182,6 @@ private static ContainerBalancerStatusInfoResponseProto getContainerBalancerStat ) .build(); - return statusInfoResponseProto; } private static ContainerBalancerConfiguration getContainerBalancerConfiguration() { @@ -331,36 +329,16 @@ void testContainerBalancerStatusInfoSubcommandVerboseHistory() "Datanodes Excluded from Balancing None"; assertTrue(output.contains(balancerConfigOutput)); - String currentIterationOutput = - "Current iteration info:\n" + - "Key Value\n" + - "Iteration number 3\n" + - "Iteration duration 1h 6m 40s\n" + - "Iteration result IN_PROGRESS\n" + - "Size scheduled to move 48 GB\n" + - "Moved data size 48 GB\n" + - "Scheduled to move containers 11\n" + - "Already moved containers 11\n" + - "Failed to move containers 0\n" + - "Failed to move containers by timeout 0\n" + - "Entered data to nodes \n" + - "80f6bc27-e6f3-493e-b1f4-25f810ad960d <- 20 GB\n" + - "701ca98e-aa1a-4b36-b817-e28ed634bba6 <- 28 GB\n" + - "Exited data from nodes \n" + - "b8b9c511-c30f-4933-8938-2f272e307070 -> 30 GB\n" + - "7bd99815-47e7-4015-bc61-ca6ef6dfd130 -> 18 GB"; - assertTrue(output.contains(currentIterationOutput)); - assertTrue(output.contains("Iteration history list:")); String firstHistoryIterationOutput = "Key Value\n" + "Iteration number 3\n" + - "Iteration duration 1h 6m 40s\n" + - "Iteration result IN_PROGRESS\n" + + "Iteration duration 6m 10s\n" + + "Iteration result -\n" + "Size scheduled to move 48 GB\n" + "Moved data size 48 GB\n" + - "Scheduled to move containers 11\n" + - "Already moved containers 11\n" + + "Scheduled to move containers 5\n" + + "Already moved containers 5\n" + "Failed to move containers 0\n" + "Failed to move containers by timeout 0\n" + "Entered data to nodes \n" + @@ -378,8 +356,8 @@ void testContainerBalancerStatusInfoSubcommandVerboseHistory() "Iteration result ITERATION_COMPLETED\n" + "Size scheduled to move 30 GB\n" + "Moved data size 30 GB\n" + - "Scheduled to move containers 11\n" + - "Already moved containers 11\n" + + "Scheduled to move containers 8\n" + + "Already moved containers 8\n" + "Failed to move containers 0\n" + "Failed to move containers by timeout 0\n" + "Entered data to nodes \n" + @@ -445,12 +423,12 @@ void testContainerBalancerStatusInfoSubcommandVerbose() "Current iteration info:\n" + "Key Value\n" + "Iteration number 3\n" + - "Iteration duration 1h 6m 40s\n" + - "Iteration result IN_PROGRESS\n" + + "Iteration duration 6m 10s\n" + + "Iteration result -\n" + "Size scheduled to move 48 GB\n" + "Moved data size 48 GB\n" + - "Scheduled to move containers 11\n" + - "Already moved containers 11\n" + + "Scheduled to move containers 5\n" + + "Already moved containers 5\n" + "Failed to move containers 0\n" + "Failed to move containers by timeout 0\n" + "Entered data to nodes \n" + @@ -465,7 +443,7 @@ void testContainerBalancerStatusInfoSubcommandVerbose() } @Test - public void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() + void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() throws IOException { ScmClient scmClient = mock(ScmClient.class); diff --git a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config index 5d426d6bc6d9..7b55c9a4fbe1 100644 --- a/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config +++ b/hadoop-ozone/dist/src/main/compose/ozone-balancer/docker-config @@ -54,6 +54,7 @@ OZONE-SITE.XML_ozone.recon.db.dir=/data/metadata/recon OZONE-SITE.XML_dfs.container.ratis.datastream.enabled=true OZONE-SITE.XML_ozone.http.basedir=/tmp/ozone_http OZONE-SITE.XML_hdds.container.balancer.balancing.iteration.interval=25s +OZONE-SITE.XML_hdds.container.balancer.trigger.du.before.move.enable=false OZONE_CONF_DIR=/etc/hadoop OZONE_LOG_DIR=/var/log/hadoop From 4243f987e220cfff2e797b7706ea40abae6ea97c Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 30 Oct 2024 14:31:25 +0300 Subject: [PATCH 24/39] HDDS-11367. Fix review notices --- .../balancer/TestContainerBalancerStatusInfo.java | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 9b620d0e7987..cee48f048601 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -29,7 +29,12 @@ import java.util.Map; import java.util.UUID; -import static org.junit.jupiter.api.Assertions.*; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests for {@link ContainerBalancerStatusInfo}. From e9080b3a60ec44a40714a04a175ed86a37f49071 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 30 Oct 2024 15:06:24 +0300 Subject: [PATCH 25/39] HDDS-11367. Fix review notices --- .../scm/container/balancer/MockedSCM.java | 3 +- .../TestContainerBalancerStatusInfo.java | 70 +++++++++++++------ 2 files changed, 49 insertions(+), 24 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java index 6c203c13099b..25a79ac88a01 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java @@ -152,7 +152,8 @@ public String toString() { return startBalancerTask(new ContainerBalancer(scm), config); } - public @Nonnull ContainerBalancerTask startBalancerTaskAsync(@Nonnull ContainerBalancerConfiguration config, Boolean withDelay) { + public @Nonnull ContainerBalancerTask startBalancerTaskAsync(@Nonnull ContainerBalancerConfiguration config, + Boolean withDelay) { init(config, new OzoneConfiguration()); return startBalancerTaskAsync(new ContainerBalancer(scm), config, withDelay); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index cee48f048601..7194273d1178 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -56,13 +56,13 @@ void testGetIterationStatistics() { assertEquals(3, iterationStatistics.size()); ContainerBalancerTaskIterationStatusInfo iterationHistory1 = iterationStatistics.get(0); - verifyCompletedIteration(iterationHistory1, 1, "ITERATION_COMPLETED"); + verifyCompletedIteration(iterationHistory1, 1); ContainerBalancerTaskIterationStatusInfo iterationHistory2 = iterationStatistics.get(1); - verifyCompletedIteration(iterationHistory2, 2, "ITERATION_COMPLETED"); + verifyCompletedIteration(iterationHistory2, 2); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationStatistics.get(2); - verifyCompletedIteration1(currentIteration, 3, null); + verifyEmptyIteration(currentIteration, 3); } @Test @@ -82,8 +82,10 @@ void testReRequestIterationStatistics() throws InterruptedException { assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); assertEquals(firstRequestIterationStatistics.get(1), secondRequestIterationStatistics.get(1)); - assertEquals(firstRequestIterationStatistics.get(2).getIterationNumber(), secondRequestIterationStatistics.get(2).getIterationNumber()); - assertNotEquals(firstRequestIterationStatistics.get(2).getIterationDuration(), secondRequestIterationStatistics.get(2).getIterationDuration()); + assertEquals(firstRequestIterationStatistics.get(2).getIterationNumber(), + secondRequestIterationStatistics.get(2).getIterationNumber()); + assertNotEquals(firstRequestIterationStatistics.get(2).getIterationDuration(), + secondRequestIterationStatistics.get(2).getIterationDuration()); } @Test @@ -103,7 +105,7 @@ void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Interrupt assertEquals(2, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo firstIteration = iterationsStatic.get(0); - verifyCompletedIteration(firstIteration, 1, "ITERATION_COMPLETED"); + verifyCompletedIteration(firstIteration, 1); assertEquals(2, iterationsStatic.get(1).getIterationNumber()); assertTrue(iterationsStatic.get(1).getIterationDuration() > 0); @@ -139,17 +141,41 @@ void testCurrentStatisticsDoesntChangeWhenReRequestInPeriodBetweenIterations() t assertEquals(2, secondRequestIterationStatistics.size()); assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); - assertEquals(firstRequestIterationStatistics.get(1).getIterationNumber(), secondRequestIterationStatistics.get(1).getIterationNumber()); - assertNotEquals(firstRequestIterationStatistics.get(1).getIterationDuration(), secondRequestIterationStatistics.get(1).getIterationDuration()); - assertEquals(firstRequestIterationStatistics.get(1).getIterationResult(), secondRequestIterationStatistics.get(1).getIterationResult()); - assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesScheduled(), secondRequestIterationStatistics.get(1).getContainerMovesScheduled()); - assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesCompleted(), secondRequestIterationStatistics.get(1).getContainerMovesCompleted()); - assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesFailed(), secondRequestIterationStatistics.get(1).getContainerMovesFailed()); - assertEquals(firstRequestIterationStatistics.get(1).getContainerMovesTimeout(), secondRequestIterationStatistics.get(1).getContainerMovesTimeout()); - assertEquals(firstRequestIterationStatistics.get(1).getSizeScheduledForMove(), secondRequestIterationStatistics.get(1).getSizeScheduledForMove()); - assertEquals(firstRequestIterationStatistics.get(1).getDataSizeMoved(), secondRequestIterationStatistics.get(1).getDataSizeMoved()); - assertEquals(firstRequestIterationStatistics.get(1).getSizeEnteringNodes().size(), secondRequestIterationStatistics.get(1).getSizeEnteringNodes().size()); - assertEquals(firstRequestIterationStatistics.get(1).getSizeLeavingNodes().size(), secondRequestIterationStatistics.get(1).getSizeLeavingNodes().size()); + ContainerBalancerTaskIterationStatusInfo currentIterationFromFirstRequest = + firstRequestIterationStatistics.get(1); + ContainerBalancerTaskIterationStatusInfo currentIterationFromSecondRequest = + secondRequestIterationStatistics.get(1); + assertEquals( + currentIterationFromFirstRequest.getIterationNumber(), + currentIterationFromSecondRequest.getIterationNumber()); + assertNotEquals( + currentIterationFromFirstRequest.getIterationDuration(), + currentIterationFromSecondRequest.getIterationDuration()); + assertEquals( + currentIterationFromFirstRequest.getIterationResult(), + currentIterationFromSecondRequest.getIterationResult()); + assertEquals( + currentIterationFromFirstRequest.getContainerMovesScheduled(), + currentIterationFromSecondRequest.getContainerMovesScheduled()); + assertEquals( + currentIterationFromFirstRequest.getContainerMovesCompleted(), + currentIterationFromSecondRequest.getContainerMovesCompleted()); + assertEquals( + currentIterationFromFirstRequest.getContainerMovesFailed(), + currentIterationFromSecondRequest.getContainerMovesFailed()); + assertEquals( + currentIterationFromFirstRequest.getContainerMovesTimeout(), + currentIterationFromSecondRequest.getContainerMovesTimeout()); + assertEquals( + currentIterationFromFirstRequest.getSizeScheduledForMove(), + currentIterationFromSecondRequest.getSizeScheduledForMove()); + assertEquals(currentIterationFromFirstRequest.getDataSizeMoved(), + currentIterationFromSecondRequest.getDataSizeMoved()); + assertEquals(currentIterationFromFirstRequest.getSizeEnteringNodes().size(), + currentIterationFromSecondRequest.getSizeEnteringNodes().size()); + assertEquals( + currentIterationFromFirstRequest.getSizeLeavingNodes().size(), + currentIterationFromSecondRequest.getSizeLeavingNodes().size()); } @Test @@ -187,8 +213,7 @@ private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInf private void verifyCompletedIteration( ContainerBalancerTaskIterationStatusInfo iteration, - Integer expectedIterationNumber, - String iterationResult + Integer expectedIterationNumber ) { assertEquals(expectedIterationNumber, iteration.getIterationNumber()); assertEquals("ITERATION_COMPLETED", iteration.getIterationResult()); @@ -214,13 +239,12 @@ private void verifyCompletedIteration( assertEquals(enteringDataSum, leavingDataSum); } - private void verifyCompletedIteration1( + private void verifyEmptyIteration( ContainerBalancerTaskIterationStatusInfo iteration, - Integer expectedIterationNumber, - String iterationResult + Integer expectedIterationNumber ) { assertEquals(expectedIterationNumber, iteration.getIterationNumber()); - assertEquals(iterationResult, iteration.getIterationResult()); + assertNull(iteration.getIterationResult()); assertNotNull(iteration.getIterationDuration()); assertEquals(0,iteration.getContainerMovesScheduled()); assertEquals(0,iteration.getContainerMovesCompleted()); From dabd4c716a0b6f69997f39f49b57d9e10495cec9 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 30 Oct 2024 15:48:42 +0300 Subject: [PATCH 26/39] HDDS-11367. Fix review notices --- .../TestContainerBalancerStatusInfo.java | 16 ++++++++++------ 1 file changed, 10 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 7194273d1178..c8146b70c59d 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -76,9 +76,11 @@ void testReRequestIterationStatistics() throws InterruptedException { config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); ContainerBalancerTask task = mockedScm.startBalancerTask(config); - List firstRequestIterationStatistics = task.getCurrentIterationsStatistic(); + List firstRequestIterationStatistics = + task.getCurrentIterationsStatistic(); Thread.sleep(1000L); - List secondRequestIterationStatistics = task.getCurrentIterationsStatistic(); + List secondRequestIterationStatistics = + task.getCurrentIterationsStatistic(); assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); assertEquals(firstRequestIterationStatistics.get(1), secondRequestIterationStatistics.get(1)); @@ -133,10 +135,12 @@ void testCurrentStatisticsDoesntChangeWhenReRequestInPeriodBetweenIterations() t ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); // Delay in finishing the first iteration Thread.sleep(1000L); - List firstRequestIterationStatistics = task.getCurrentIterationsStatistic(); + List firstRequestIterationStatistics = + task.getCurrentIterationsStatistic(); // Delay occurred for some time during the period between iterations. Thread.sleep(1000L); - List secondRequestIterationStatistics = task.getCurrentIterationsStatistic(); + List secondRequestIterationStatistics = + task.getCurrentIterationsStatistic(); assertEquals(2, firstRequestIterationStatistics.size()); assertEquals(2, secondRequestIterationStatistics.size()); assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); @@ -246,8 +250,8 @@ private void verifyEmptyIteration( assertEquals(expectedIterationNumber, iteration.getIterationNumber()); assertNull(iteration.getIterationResult()); assertNotNull(iteration.getIterationDuration()); - assertEquals(0,iteration.getContainerMovesScheduled()); - assertEquals(0,iteration.getContainerMovesCompleted()); + assertEquals(0, iteration.getContainerMovesScheduled()); + assertEquals(0, iteration.getContainerMovesCompleted()); assertEquals(0, iteration.getContainerMovesFailed()); assertEquals(0, iteration.getContainerMovesTimeout()); assertEquals(0, iteration.getSizeScheduledForMove()); From c6562fccaed236e566fe75ce99a409a931944cb2 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 1 Nov 2024 16:09:11 +0300 Subject: [PATCH 27/39] HDDS-11367. Fix tests --- .../balancer/ContainerBalancerTask.java | 53 ++++++++++++++++--- .../container/balancer/ContainerMoveInfo.java | 8 +++ 2 files changed, 55 insertions(+), 6 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index c29dcbef7212..84efecb65106 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -59,9 +59,11 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import static java.time.OffsetDateTime.now; +import static java.util.Collections.emptyMap; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT; import static org.apache.hadoop.util.StringUtils.byteDesc; @@ -104,7 +106,6 @@ public class ContainerBalancerTask implements Runnable { private double lowerLimit; private ContainerBalancerSelectionCriteria selectionCriteria; private volatile Status taskStatus = Status.RUNNING; - /* Since a container can be selected only once during an iteration, these maps use it as a primary key to track source to target pairings. @@ -123,6 +124,7 @@ public class ContainerBalancerTask implements Runnable { private boolean delayStart; private List iterationsStatistic; private OffsetDateTime currentIterationStarted; + private AtomicBoolean isCurrentIterationInProgress = new AtomicBoolean(false); /** * Constructs ContainerBalancerTask with the specified arguments. @@ -221,6 +223,9 @@ private void balance() { int i = nextIterationIndex; for (; i < iterations && isBalancerRunning(); i++) { currentIterationStarted = now(); + + isCurrentIterationInProgress.compareAndSet(false, true); + // reset some variables and metrics for this iteration resetState(); if (config.getTriggerDuEnable()) { @@ -270,7 +275,8 @@ private void balance() { IterationResult iR = doIteration(); saveIterationStatistic(i + 1, iR); - resetState(); + isCurrentIterationInProgress.compareAndSet(true, false); + findTargetStrategy.clearSizeEnteringNodes(); findSourceStrategy.clearSizeLeavingNodes(); @@ -362,10 +368,48 @@ private Map convertToNodeIdToTrafficMap(Map n * @return current iteration statistic */ public List getCurrentIterationsStatistic() { + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = createCurrentIterationStatistic(); + List resultList = new ArrayList<>(iterationsStatistic); + resultList.add(currentIterationStatistic); + return resultList; + } + + private ContainerBalancerTaskIterationStatusInfo createCurrentIterationStatistic() { int lastIterationNumber = iterationsStatistic.isEmpty() ? 0 : iterationsStatistic.get(iterationsStatistic.size() - 1).getIterationNumber(); long iterationDuration = getCurrentIterationDuration(); + + if (isCurrentIterationInProgress.get()) { + return getFilledCurrentIterationStatistic(lastIterationNumber, iterationDuration); + } else { + return getEmptyCurrentIterationStatistic(lastIterationNumber, iterationDuration); + } + } + + private static ContainerBalancerTaskIterationStatusInfo getEmptyCurrentIterationStatistic( + int lastIterationNumber, long iterationDuration) { + ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(0, 0, 0, 0); + DataMoveInfo dataMoveInfo = new DataMoveInfo( + 0, + 0, + emptyMap(), + emptyMap() + ); + IterationInfo iterationInfo = new IterationInfo( + lastIterationNumber + 1, + null, + iterationDuration + ); + return new ContainerBalancerTaskIterationStatusInfo( + iterationInfo, + containerMoveInfo, + dataMoveInfo + ); + } + + private ContainerBalancerTaskIterationStatusInfo getFilledCurrentIterationStatistic(int lastIterationNumber, + long iterationDuration) { Map sizeEnteringDataToNodes = convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); Map sizeLeavingDataFromNodes = @@ -383,14 +427,11 @@ public List getCurrentIterationsStatis null, iterationDuration ); - ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + return new ContainerBalancerTaskIterationStatusInfo( iterationInfo, containerMoveInfo, dataMoveInfo ); - List resultList = new ArrayList<>(iterationsStatistic); - resultList.add(currentIterationStatistic); - return resultList; } private long getCurrentIterationDuration() { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java index 5ee8026c3d7f..caed286480b1 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerMoveInfo.java @@ -27,6 +27,14 @@ public class ContainerMoveInfo { private final long containerMovesFailed; private final long containerMovesTimeout; + public ContainerMoveInfo(long containerMovesScheduled, long containerMovesCompleted, long containerMovesFailed, + long containerMovesTimeout) { + this.containerMovesScheduled = containerMovesScheduled; + this.containerMovesCompleted = containerMovesCompleted; + this.containerMovesFailed = containerMovesFailed; + this.containerMovesTimeout = containerMovesTimeout; + } + public ContainerMoveInfo(ContainerBalancerMetrics metrics) { this.containerMovesScheduled = metrics.getNumContainerMovesScheduledInLatestIteration(); this.containerMovesCompleted = metrics.getNumContainerMovesCompletedInLatestIteration(); From 361ba7414a8a1e7dab6826582076437a82676915 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 1 Nov 2024 22:24:21 +0300 Subject: [PATCH 28/39] HDDS-11367. Fix tests --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 549d2133a5d2..51c94df80211 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -99,7 +99,7 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v - Verify Balancer Iteration ${result} - + Verify Balancer Iteration ${result} 1 Should Contain ${result} Iteration result IN_PROGRESS collapse_spaces=True From 78caa9736eb8851f1d5a75b5fb7861f35858c0ea Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 1 Nov 2024 23:49:05 +0300 Subject: [PATCH 29/39] HDDS-11367. Fix tests --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 51c94df80211..c548dda9a756 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -100,7 +100,7 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v Verify Balancer Iteration ${result} 1 - Should Contain ${result} Iteration result IN_PROGRESS collapse_spaces=True + Should Contain ${result} Iteration result - collapse_spaces=True Run Balancer Verbose History Status From bae2a6b63b2a4f9cf6b9652972fee6d4b50a9cce Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sat, 2 Nov 2024 16:30:21 +0300 Subject: [PATCH 30/39] HDDS-11367. Fix tests --- .../scm/container/balancer/ContainerBalancerTask.java | 6 +++--- .../balancer/TestContainerBalancerStatusInfo.java | 9 ++++----- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 3 files changed, 8 insertions(+), 9 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 84efecb65106..0816ea3fcb8c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -383,12 +383,12 @@ private ContainerBalancerTaskIterationStatusInfo createCurrentIterationStatistic if (isCurrentIterationInProgress.get()) { return getFilledCurrentIterationStatistic(lastIterationNumber, iterationDuration); } else { - return getEmptyCurrentIterationStatistic(lastIterationNumber, iterationDuration); + return getEmptyCurrentIterationStatistic(iterationDuration); } } private static ContainerBalancerTaskIterationStatusInfo getEmptyCurrentIterationStatistic( - int lastIterationNumber, long iterationDuration) { + long iterationDuration) { ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(0, 0, 0, 0); DataMoveInfo dataMoveInfo = new DataMoveInfo( 0, @@ -397,7 +397,7 @@ private static ContainerBalancerTaskIterationStatusInfo getEmptyCurrentIteration emptyMap() ); IterationInfo iterationInfo = new IterationInfo( - lastIterationNumber + 1, + 0, null, iterationDuration ); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index c8146b70c59d..d4ec5e3ae8f0 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -62,7 +62,7 @@ void testGetIterationStatistics() { verifyCompletedIteration(iterationHistory2, 2); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationStatistics.get(2); - verifyEmptyIteration(currentIteration, 3); + verifyEmptyIteration(currentIteration); } @Test @@ -109,7 +109,7 @@ void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Interrupt ContainerBalancerTaskIterationStatusInfo firstIteration = iterationsStatic.get(0); verifyCompletedIteration(firstIteration, 1); - assertEquals(2, iterationsStatic.get(1).getIterationNumber()); + assertEquals(0, iterationsStatic.get(1).getIterationNumber()); assertTrue(iterationsStatic.get(1).getIterationDuration() > 0); assertNull(iterationsStatic.get(1).getIterationResult()); assertEquals(0, iterationsStatic.get(1).getContainerMovesScheduled()); @@ -244,10 +244,9 @@ private void verifyCompletedIteration( } private void verifyEmptyIteration( - ContainerBalancerTaskIterationStatusInfo iteration, - Integer expectedIterationNumber + ContainerBalancerTaskIterationStatusInfo iteration ) { - assertEquals(expectedIterationNumber, iteration.getIterationNumber()); + assertEquals(0, iteration.getIterationNumber()); assertNull(iteration.getIterationResult()); assertNotNull(iteration.getIterationDuration()); assertEquals(0, iteration.getContainerMovesScheduled()); diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index ece820e67f60..604c96935f52 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -99,7 +99,7 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v - Verify Balancer Iteration ${result} 1 + Verify Balancer Iteration ${result} - Should Contain ${result} Iteration result - collapse_spaces=True From bea5d60bd7e3969275183280dd69a9ee2e2ef03a Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 4 Nov 2024 22:27:16 +0300 Subject: [PATCH 31/39] HDDS-11367. Fix tests --- .../hdds/scm/container/balancer/ContainerBalancerTask.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 0816ea3fcb8c..d1e94ea9bad5 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -423,7 +423,7 @@ private ContainerBalancerTaskIterationStatusInfo getFilledCurrentIterationStatis sizeLeavingDataFromNodes ); IterationInfo iterationInfo = new IterationInfo( - lastIterationNumber + 1, + 0, null, iterationDuration ); From 9ae75273cff20a0bb7f6c1bcf74e9abaaa776df6 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 5 Nov 2024 00:07:43 +0300 Subject: [PATCH 32/39] HDDS-11367. Fix tests --- .../hdds/scm/container/balancer/ContainerBalancerTask.java | 2 +- .../dist/src/main/smoketest/balancer/testBalancer.robot | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index d1e94ea9bad5..0816ea3fcb8c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -423,7 +423,7 @@ private ContainerBalancerTaskIterationStatusInfo getFilledCurrentIterationStatis sizeLeavingDataFromNodes ); IterationInfo iterationInfo = new IterationInfo( - 0, + lastIterationNumber + 1, null, iterationDuration ); diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 604c96935f52..ece820e67f60 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -99,7 +99,7 @@ Run Balancer Status Run Balancer Verbose Status ${result} = Execute ozone admin containerbalancer status -v - Verify Balancer Iteration ${result} - + Verify Balancer Iteration ${result} 1 Should Contain ${result} Iteration result - collapse_spaces=True From 9e8213ba6bab710e0fb9d04f57c6f6cc82592ce9 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 5 Nov 2024 11:29:44 +0300 Subject: [PATCH 33/39] HDDS-11367. Add tests --- .../balancer/ContainerBalancerTask.java | 2 +- .../TestContainerBalancerStatusInfo.java | 58 +++++++++++++++++-- 2 files changed, 55 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 0816ea3fcb8c..4129c340eeac 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -368,8 +368,8 @@ private Map convertToNodeIdToTrafficMap(Map n * @return current iteration statistic */ public List getCurrentIterationsStatistic() { - ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = createCurrentIterationStatistic(); List resultList = new ArrayList<>(iterationsStatistic); + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = createCurrentIterationStatistic(); resultList.add(currentIterationStatistic); return resultList; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index d4ec5e3ae8f0..6ebe6cb618ab 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.OzoneConsts; +import org.apache.ozone.test.LambdaTestUtils; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; @@ -91,7 +92,7 @@ void testReRequestIterationStatistics() throws InterruptedException { } @Test - void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws InterruptedException { + void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Exception { MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); @@ -102,7 +103,7 @@ void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Interrupt ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); // Delay in finishing the first iteration - Thread.sleep(1000L); + LambdaTestUtils.await(1000, 500, () -> task.getCurrentIterationsStatistic().size() == 2); List iterationsStatic = task.getCurrentIterationsStatistic(); assertEquals(2, iterationsStatic.size()); @@ -183,7 +184,7 @@ void testCurrentStatisticsDoesntChangeWhenReRequestInPeriodBetweenIterations() t } @Test - void testGetCurrentStatisticsWithDelay() throws InterruptedException { + void testGetCurrentStatisticsWithDelay() throws Exception { MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); @@ -194,13 +195,34 @@ void testGetCurrentStatisticsWithDelay() throws InterruptedException { ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, true); // Delay in finishing the first iteration - Thread.sleep(1000L); + LambdaTestUtils.await(1000, 500, () -> task.getCurrentIterationsStatistic().size() == 1); List iterationsStatic = task.getCurrentIterationsStatistic(); assertEquals(1, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(0); assertEmptyIteration(currentIteration); } + @Test + void testGetCurrentStatisticsWhileBalancingInProgress() throws Exception { + MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); + + ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); + + config.setIterations(3); + config.setBalancingInterval(0); + config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); + + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); + // Get the current iteration statistics when it has information about the containers moving. + LambdaTestUtils.await(1000, 10, + () -> task.getCurrentIterationsStatistic().size() == 2 && + task.getCurrentIterationsStatistic().get(1).getContainerMovesScheduled() > 0); + List iterationsStatic = task.getCurrentIterationsStatistic(); + assertEquals(2, iterationsStatic.size()); + ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(1); + assertCurrentIterationStatisticWhileBalancingInProgress(currentIteration); + } + private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInfo iterationsStatic) { assertEquals(-1, iterationsStatic.getIterationDuration()); @@ -215,6 +237,34 @@ private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInf assertEquals(0, iterationsStatic.getSizeLeavingNodes().size()); } + private static void assertCurrentIterationStatisticWhileBalancingInProgress( + ContainerBalancerTaskIterationStatusInfo iterationsStatic + ) { + + assertEquals(2, iterationsStatic.getIterationNumber()); + assertEquals(0, iterationsStatic.getIterationDuration()); + assertNull(iterationsStatic.getIterationResult()); + assertTrue(iterationsStatic.getContainerMovesScheduled() > 0); + assertTrue(iterationsStatic.getContainerMovesCompleted() > 0); + assertEquals(0, iterationsStatic.getContainerMovesFailed()); + assertEquals(0, iterationsStatic.getContainerMovesTimeout()); + assertTrue(iterationsStatic.getSizeScheduledForMove() > 0); + assertTrue(iterationsStatic.getDataSizeMoved() > 0); + assertFalse(iterationsStatic.getSizeEnteringNodes().isEmpty()); + assertFalse(iterationsStatic.getSizeLeavingNodes().isEmpty()); + iterationsStatic.getSizeEnteringNodes().forEach((id, size) -> { + assertNotNull(id); + assertTrue(size > 0); + }); + iterationsStatic.getSizeLeavingNodes().forEach((id, size) -> { + assertNotNull(id); + assertTrue(size > 0); + }); + Long enteringDataSum = getTotalMovedData(iterationsStatic.getSizeEnteringNodes()); + Long leavingDataSum = getTotalMovedData(iterationsStatic.getSizeLeavingNodes()); + assertEquals(enteringDataSum, leavingDataSum); + } + private void verifyCompletedIteration( ContainerBalancerTaskIterationStatusInfo iteration, Integer expectedIterationNumber From 786fc4f18424067ed7b30155fb3faeca4e425211 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 20 Nov 2024 11:38:10 +0300 Subject: [PATCH 34/39] HDDS-11367. Fix review comments. --- .../TestContainerBalancerStatusInfo.java | 13 ++++++----- .../hadoop/hdds/util/TestDurationUtil.java | 23 +++++++++---------- 2 files changed, 18 insertions(+), 18 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 6ebe6cb618ab..0f9c89ba0a3b 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -63,7 +63,7 @@ void testGetIterationStatistics() { verifyCompletedIteration(iterationHistory2, 2); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationStatistics.get(2); - verifyEmptyIteration(currentIteration); + verifyStartedEmptyIteration(currentIteration); } @Test @@ -199,7 +199,7 @@ void testGetCurrentStatisticsWithDelay() throws Exception { List iterationsStatic = task.getCurrentIterationsStatistic(); assertEquals(1, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(0); - assertEmptyIteration(currentIteration); + verifyUnstartedIteration(currentIteration); } @Test @@ -223,8 +223,9 @@ void testGetCurrentStatisticsWhileBalancingInProgress() throws Exception { assertCurrentIterationStatisticWhileBalancingInProgress(currentIteration); } - private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInfo iterationsStatic) { + private static void verifyUnstartedIteration(ContainerBalancerTaskIterationStatusInfo iterationsStatic) { + assertEquals(0, iterationsStatic.getIterationNumber()); assertEquals(-1, iterationsStatic.getIterationDuration()); assertNull(iterationsStatic.getIterationResult()); assertEquals(0, iterationsStatic.getContainerMovesScheduled()); @@ -233,8 +234,8 @@ private static void assertEmptyIteration(ContainerBalancerTaskIterationStatusInf assertEquals(0, iterationsStatic.getContainerMovesTimeout()); assertEquals(0, iterationsStatic.getSizeScheduledForMove()); assertEquals(0, iterationsStatic.getDataSizeMoved()); - assertEquals(0, iterationsStatic.getSizeEnteringNodes().size()); - assertEquals(0, iterationsStatic.getSizeLeavingNodes().size()); + assertTrue(iterationsStatic.getSizeEnteringNodes().isEmpty()); + assertTrue(iterationsStatic.getSizeLeavingNodes().isEmpty()); } private static void assertCurrentIterationStatisticWhileBalancingInProgress( @@ -293,7 +294,7 @@ private void verifyCompletedIteration( assertEquals(enteringDataSum, leavingDataSum); } - private void verifyEmptyIteration( + private void verifyStartedEmptyIteration( ContainerBalancerTaskIterationStatusInfo iteration ) { assertEquals(0, iteration.getIterationNumber()); diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java index 3498f04416de..7b0a95486399 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/util/TestDurationUtil.java @@ -36,40 +36,39 @@ private static Stream paramsForPositiveCases() { return Stream.of( arguments( "0s", - DurationUtil.getPrettyDuration(Duration.ZERO) + Duration.ZERO ), arguments( "2562047788015215h 30m 7s", - DurationUtil.getPrettyDuration(Duration.ofSeconds(Long.MAX_VALUE)) + Duration.ofSeconds(Long.MAX_VALUE) ), arguments( "1s", - DurationUtil.getPrettyDuration(Duration.ofSeconds(1)) + Duration.ofSeconds(1) ), arguments( "30s", - DurationUtil.getPrettyDuration(Duration.ofSeconds(30)) + Duration.ofSeconds(30) ), arguments( "1m 0s", - DurationUtil.getPrettyDuration(Duration.ofMinutes(1)) + Duration.ofMinutes(1) ), arguments( "2m 30s", - DurationUtil.getPrettyDuration(Duration.ofMinutes(2).plusSeconds(30)) + Duration.ofMinutes(2).plusSeconds(30) ), arguments( "1h 30m 45s", - DurationUtil.getPrettyDuration( - Duration.ofHours(1).plusMinutes(30).plusSeconds(45)) + Duration.ofHours(1).plusMinutes(30).plusSeconds(45) ), arguments( "24h 0m 0s", - DurationUtil.getPrettyDuration(Duration.ofDays(1)) + Duration.ofDays(1) ), arguments( "48h 0m 0s", - DurationUtil.getPrettyDuration(Duration.ofDays(2)) + Duration.ofDays(2) ) ); } @@ -80,8 +79,8 @@ private static Collection paramsForNegativeCases() { @ParameterizedTest @MethodSource("paramsForPositiveCases") - void testDuration(String expected, String actual) { - assertEquals(expected, actual); + void testDuration(String expected, Duration actual) { + assertEquals(expected, DurationUtil.getPrettyDuration(actual)); } @ParameterizedTest From e77ea148a83e3999f3fed79c2fe3dae2fb2db31f Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 20 Nov 2024 15:57:59 +0300 Subject: [PATCH 35/39] HDDS-11367. Fix review comments. --- .../scm/container/balancer/ContainerBalancerTask.java | 2 +- .../cli/datanode/TestContainerBalancerSubCommand.java | 9 +++++---- 2 files changed, 6 insertions(+), 5 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 4129c340eeac..c33afc3f0bb8 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -397,7 +397,7 @@ private static ContainerBalancerTaskIterationStatusInfo getEmptyCurrentIteration emptyMap() ); IterationInfo iterationInfo = new IterationInfo( - 0, + 0, null, iterationDuration ); diff --git a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java index 8fcd17eba356..bdce0f5d7070 100644 --- a/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java +++ b/hadoop-hdds/tools/src/test/java/org/apache/hadoop/hdds/scm/cli/datanode/TestContainerBalancerSubCommand.java @@ -484,8 +484,8 @@ public void testContainerBalancerStopSubcommand() throws IOException { stopCmd.execute(scmClient); Pattern p = Pattern.compile("^Sending\\sstop\\scommand." + - "\\sWaiting\\sfor\\sContainer\\sBalancer\\sto\\sstop...\\n" + - "Container\\sBalancer\\sstopped."); + "\\sWaiting\\sfor\\sContainer\\sBalancer\\sto\\sstop...\\n" + + "Container\\sBalancer\\sstopped."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); @@ -505,7 +505,7 @@ public void testContainerBalancerStartSubcommandWhenBalancerIsNotRunning() startCmd.execute(scmClient); Pattern p = Pattern.compile("^Container\\sBalancer\\sstarted" + - "\\ssuccessfully."); + "\\ssuccessfully."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); } @@ -523,7 +523,8 @@ public void testContainerBalancerStartSubcommandWhenBalancerIsRunning() .build()); startCmd.execute(scmClient); - Pattern p = Pattern.compile("^Failed\\sto\\sstart\\sContainer\\sBalancer."); + Pattern p = Pattern.compile("^Failed\\sto\\sstart\\sContainer" + + "\\sBalancer."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); From 6ccd6581671ee8bdeed493c3d8f3240b474ef59d Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Fri, 22 Nov 2024 15:51:39 +0300 Subject: [PATCH 36/39] HDDS-11367. Fix flaky TestContainerBalancerStatusInfo.testGetCurrentStatisticsRequestInPeriodBetweenIterations --- .../scm/container/balancer/TestContainerBalancerStatusInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 0f9c89ba0a3b..4b9016baac53 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -111,7 +111,7 @@ void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Exception verifyCompletedIteration(firstIteration, 1); assertEquals(0, iterationsStatic.get(1).getIterationNumber()); - assertTrue(iterationsStatic.get(1).getIterationDuration() > 0); + assertTrue(iterationsStatic.get(1).getIterationDuration() >= 0); assertNull(iterationsStatic.get(1).getIterationResult()); assertEquals(0, iterationsStatic.get(1).getContainerMovesScheduled()); assertEquals(0, iterationsStatic.get(1).getContainerMovesCompleted()); From 15a8558afebfcae53304146e36b055fe2b5c4238 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 25 Nov 2024 22:30:16 +0300 Subject: [PATCH 37/39] HDDS-11367. Remove unstarted balancing iteration --- .../balancer/ContainerBalancerTask.java | 19 ++++- .../scm/container/balancer/MockedSCM.java | 7 ++ .../TestContainerBalancerStatusInfo.java | 84 ++++--------------- 3 files changed, 38 insertions(+), 72 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 9e386c544006..8f2775d44c16 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -43,7 +43,18 @@ import java.io.IOException; import java.time.Duration; import java.time.OffsetDateTime; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.Objects; +import java.util.Queue; +import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentLinkedQueue; @@ -361,7 +372,9 @@ private Map convertToNodeIdToTrafficMap(Map n public List getCurrentIterationsStatistic() { List resultList = new ArrayList<>(iterationsStatistic); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = createCurrentIterationStatistic(); - resultList.add(currentIterationStatistic); + if (currentIterationStatistic != null) { + resultList.add(currentIterationStatistic); + } return resultList; } @@ -377,7 +390,7 @@ private ContainerBalancerTaskIterationStatusInfo createCurrentIterationStatistic if (isCurrentIterationInProgress.get()) { return getFilledCurrentIterationStatistic(lastIterationNumber, iterationDuration); } else { - return getEmptyCurrentIterationStatistic(iterationDuration); + return null; } } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java index 25a79ac88a01..62a126da15c0 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/MockedSCM.java @@ -152,6 +152,13 @@ public String toString() { return startBalancerTask(new ContainerBalancer(scm), config); } + public @Nonnull ContainerBalancerTask startBalancerTaskAsync(@Nonnull ContainerBalancerConfiguration config, + OzoneConfiguration ozoneConfig, + Boolean withDelay) { + init(config, ozoneConfig); + return startBalancerTaskAsync(new ContainerBalancer(scm), config, withDelay); + } + public @Nonnull ContainerBalancerTask startBalancerTaskAsync(@Nonnull ContainerBalancerConfiguration config, Boolean withDelay) { init(config, new OzoneConfiguration()); diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 4b9016baac53..5110c93edec0 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdds.scm.container.balancer; import org.apache.commons.math3.util.ArithmeticUtils; +import org.apache.hadoop.hdds.HddsConfigKeys; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.scm.server.StorageContainerManager; import org.apache.hadoop.ozone.OzoneConsts; @@ -32,7 +33,6 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; -import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -54,20 +54,17 @@ void testGetIterationStatistics() { ContainerBalancerTask task = mockedScm.startBalancerTask(config); List iterationStatistics = task.getCurrentIterationsStatistic(); - assertEquals(3, iterationStatistics.size()); + assertEquals(2, iterationStatistics.size()); ContainerBalancerTaskIterationStatusInfo iterationHistory1 = iterationStatistics.get(0); verifyCompletedIteration(iterationHistory1, 1); ContainerBalancerTaskIterationStatusInfo iterationHistory2 = iterationStatistics.get(1); verifyCompletedIteration(iterationHistory2, 2); - - ContainerBalancerTaskIterationStatusInfo currentIteration = iterationStatistics.get(2); - verifyStartedEmptyIteration(currentIteration); } @Test - void testReRequestIterationStatistics() throws InterruptedException { + void testReRequestIterationStatistics() throws Exception { MockedSCM mockedScm = new MockedSCM(new TestableCluster(20, OzoneConsts.GB)); ContainerBalancerConfiguration config = new OzoneConfiguration().getObject(ContainerBalancerConfiguration.class); @@ -84,11 +81,6 @@ void testReRequestIterationStatistics() throws InterruptedException { task.getCurrentIterationsStatistic(); assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); assertEquals(firstRequestIterationStatistics.get(1), secondRequestIterationStatistics.get(1)); - - assertEquals(firstRequestIterationStatistics.get(2).getIterationNumber(), - secondRequestIterationStatistics.get(2).getIterationNumber()); - assertNotEquals(firstRequestIterationStatistics.get(2).getIterationDuration(), - secondRequestIterationStatistics.get(2).getIterationDuration()); } @Test @@ -102,25 +94,14 @@ void testGetCurrentStatisticsRequestInPeriodBetweenIterations() throws Exception config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, false); - // Delay in finishing the first iteration - LambdaTestUtils.await(1000, 500, () -> task.getCurrentIterationsStatistic().size() == 2); + LambdaTestUtils.await(1000, 500, + () -> task.getCurrentIterationsStatistic().size() == 1 && + task.getCurrentIterationsStatistic().get(0).getIterationResult().equals("ITERATION_COMPLETED")); List iterationsStatic = task.getCurrentIterationsStatistic(); - assertEquals(2, iterationsStatic.size()); + assertEquals(1, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo firstIteration = iterationsStatic.get(0); verifyCompletedIteration(firstIteration, 1); - - assertEquals(0, iterationsStatic.get(1).getIterationNumber()); - assertTrue(iterationsStatic.get(1).getIterationDuration() >= 0); - assertNull(iterationsStatic.get(1).getIterationResult()); - assertEquals(0, iterationsStatic.get(1).getContainerMovesScheduled()); - assertEquals(0, iterationsStatic.get(1).getContainerMovesCompleted()); - assertEquals(0, iterationsStatic.get(1).getContainerMovesFailed()); - assertEquals(0, iterationsStatic.get(1).getContainerMovesTimeout()); - assertEquals(0, iterationsStatic.get(1).getSizeScheduledForMove()); - assertEquals(0, iterationsStatic.get(1).getDataSizeMoved()); - assertEquals(0, iterationsStatic.get(1).getSizeEnteringNodes().size()); - assertEquals(0, iterationsStatic.get(1).getSizeLeavingNodes().size()); } @Test @@ -142,45 +123,9 @@ void testCurrentStatisticsDoesntChangeWhenReRequestInPeriodBetweenIterations() t Thread.sleep(1000L); List secondRequestIterationStatistics = task.getCurrentIterationsStatistic(); - assertEquals(2, firstRequestIterationStatistics.size()); - assertEquals(2, secondRequestIterationStatistics.size()); + assertEquals(1, firstRequestIterationStatistics.size()); + assertEquals(1, secondRequestIterationStatistics.size()); assertEquals(firstRequestIterationStatistics.get(0), secondRequestIterationStatistics.get(0)); - - ContainerBalancerTaskIterationStatusInfo currentIterationFromFirstRequest = - firstRequestIterationStatistics.get(1); - ContainerBalancerTaskIterationStatusInfo currentIterationFromSecondRequest = - secondRequestIterationStatistics.get(1); - assertEquals( - currentIterationFromFirstRequest.getIterationNumber(), - currentIterationFromSecondRequest.getIterationNumber()); - assertNotEquals( - currentIterationFromFirstRequest.getIterationDuration(), - currentIterationFromSecondRequest.getIterationDuration()); - assertEquals( - currentIterationFromFirstRequest.getIterationResult(), - currentIterationFromSecondRequest.getIterationResult()); - assertEquals( - currentIterationFromFirstRequest.getContainerMovesScheduled(), - currentIterationFromSecondRequest.getContainerMovesScheduled()); - assertEquals( - currentIterationFromFirstRequest.getContainerMovesCompleted(), - currentIterationFromSecondRequest.getContainerMovesCompleted()); - assertEquals( - currentIterationFromFirstRequest.getContainerMovesFailed(), - currentIterationFromSecondRequest.getContainerMovesFailed()); - assertEquals( - currentIterationFromFirstRequest.getContainerMovesTimeout(), - currentIterationFromSecondRequest.getContainerMovesTimeout()); - assertEquals( - currentIterationFromFirstRequest.getSizeScheduledForMove(), - currentIterationFromSecondRequest.getSizeScheduledForMove()); - assertEquals(currentIterationFromFirstRequest.getDataSizeMoved(), - currentIterationFromSecondRequest.getDataSizeMoved()); - assertEquals(currentIterationFromFirstRequest.getSizeEnteringNodes().size(), - currentIterationFromSecondRequest.getSizeEnteringNodes().size()); - assertEquals( - currentIterationFromFirstRequest.getSizeLeavingNodes().size(), - currentIterationFromSecondRequest.getSizeLeavingNodes().size()); } @Test @@ -192,14 +137,15 @@ void testGetCurrentStatisticsWithDelay() throws Exception { config.setIterations(2); config.setBalancingInterval(0); config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); - - ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, true); + OzoneConfiguration configuration = new OzoneConfiguration(); + configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, "1"); + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, configuration,true); // Delay in finishing the first iteration - LambdaTestUtils.await(1000, 500, () -> task.getCurrentIterationsStatistic().size() == 1); + LambdaTestUtils.await(1100, 50, () -> task.getCurrentIterationsStatistic().size() == 1); List iterationsStatic = task.getCurrentIterationsStatistic(); assertEquals(1, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(0); - verifyUnstartedIteration(currentIteration); + verifyStartedEmptyIteration(currentIteration); } @Test @@ -297,7 +243,7 @@ private void verifyCompletedIteration( private void verifyStartedEmptyIteration( ContainerBalancerTaskIterationStatusInfo iteration ) { - assertEquals(0, iteration.getIterationNumber()); + assertEquals(1, iteration.getIterationNumber()); assertNull(iteration.getIterationResult()); assertNotNull(iteration.getIterationDuration()); assertEquals(0, iteration.getContainerMovesScheduled()); From 2e60bf3a340438607dd706ef0cad052853ae56be Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 25 Nov 2024 22:55:23 +0300 Subject: [PATCH 38/39] HDDS-11367. Fix review notice. Refactor saving iteration statistic. --- .../balancer/ContainerBalancerTask.java | 61 ++++++++++++------- .../TestContainerBalancerStatusInfo.java | 2 +- .../ContainerBalancerStatusSubcommand.java | 18 ++++-- .../smoketest/balancer/testBalancer.robot | 12 ++-- 4 files changed, 60 insertions(+), 33 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java index 8f2775d44c16..f1eee8c6755b 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTask.java @@ -274,8 +274,10 @@ private void balance() { return; } - IterationResult iR = doIteration(); - saveIterationStatistic(i + 1, iR); + IterationResult currentIterationResult = doIteration(); + ContainerBalancerTaskIterationStatusInfo iterationStatistic = + getIterationStatistic(i + 1, currentIterationResult, getCurrentIterationDuration()); + iterationsStatistic.offer(iterationStatistic); isCurrentIterationInProgress.compareAndSet(true, false); @@ -284,17 +286,17 @@ private void balance() { metrics.incrementNumIterations(1); - LOG.info("Result of this iteration of Container Balancer: {}", iR); + LOG.info("Result of this iteration of Container Balancer: {}", currentIterationResult); // if no new move option is generated, it means the cluster cannot be // balanced anymore; so just stop balancer - if (iR == IterationResult.CAN_NOT_BALANCE_ANY_MORE) { - tryStopWithSaveConfiguration(iR.toString()); + if (currentIterationResult == IterationResult.CAN_NOT_BALANCE_ANY_MORE) { + tryStopWithSaveConfiguration(currentIterationResult.toString()); return; } // persist next iteration index - if (iR == IterationResult.ITERATION_COMPLETED) { + if (currentIterationResult == IterationResult.ITERATION_COMPLETED) { try { saveConfiguration(config, true, i + 1); } catch (IOException | TimeoutException e) { @@ -325,30 +327,45 @@ private void balance() { tryStopWithSaveConfiguration("Completed all iterations."); } - private void saveIterationStatistic(Integer iterationNumber, IterationResult currentIterationResult) { - long iterationDuration = now().toEpochSecond() - currentIterationStarted.toEpochSecond(); + private ContainerBalancerTaskIterationStatusInfo getIterationStatistic(Integer iterationNumber, + IterationResult currentIterationResult, + long iterationDuration) { + String currentIterationResultName = currentIterationResult == null ? null : currentIterationResult.name(); Map sizeEnteringDataToNodes = convertToNodeIdToTrafficMap(findTargetStrategy.getSizeEnteringNodes()); Map sizeLeavingDataFromNodes = convertToNodeIdToTrafficMap(findSourceStrategy.getSizeLeavingNodes()); IterationInfo iterationInfo = new IterationInfo( iterationNumber, - currentIterationResult.name(), + currentIterationResultName, iterationDuration ); ContainerMoveInfo containerMoveInfo = new ContainerMoveInfo(metrics); - DataMoveInfo dataMoveInfo = new DataMoveInfo( - getSizeScheduledForMoveInLatestIteration(), - metrics.getDataSizeMovedInLatestIteration(), - sizeEnteringDataToNodes, - sizeLeavingDataFromNodes - ); - ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - iterationInfo, - containerMoveInfo, - dataMoveInfo - ); - iterationsStatistic.offer(iterationStatistic); + + DataMoveInfo dataMoveInfo = + getDataMoveInfo(currentIterationResultName, sizeEnteringDataToNodes, sizeLeavingDataFromNodes); + return new ContainerBalancerTaskIterationStatusInfo(iterationInfo, containerMoveInfo, dataMoveInfo); + } + + private DataMoveInfo getDataMoveInfo(String currentIterationResultName, Map sizeEnteringDataToNodes, + Map sizeLeavingDataFromNodes) { + if (currentIterationResultName == null) { + // For unfinished iteration + return new DataMoveInfo( + getSizeScheduledForMoveInLatestIteration(), + sizeActuallyMovedInLatestIteration, + sizeEnteringDataToNodes, + sizeLeavingDataFromNodes + ); + } else { + // For finished iteration + return new DataMoveInfo( + getSizeScheduledForMoveInLatestIteration(), + metrics.getDataSizeMovedInLatestIteration(), + sizeEnteringDataToNodes, + sizeLeavingDataFromNodes + ); + } } private Map convertToNodeIdToTrafficMap(Map nodeTrafficMap) { @@ -388,7 +405,7 @@ private ContainerBalancerTaskIterationStatusInfo createCurrentIterationStatistic long iterationDuration = getCurrentIterationDuration(); if (isCurrentIterationInProgress.get()) { - return getFilledCurrentIterationStatistic(lastIterationNumber, iterationDuration); + return getIterationStatistic(lastIterationNumber + 1, null, iterationDuration); } else { return null; } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index 5110c93edec0..fafd43c69ff5 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -139,7 +139,7 @@ void testGetCurrentStatisticsWithDelay() throws Exception { config.setMaxSizeToMovePerIteration(50 * OzoneConsts.GB); OzoneConfiguration configuration = new OzoneConfiguration(); configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, "1"); - ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, configuration,true); + ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, configuration, true); // Delay in finishing the first iteration LambdaTestUtils.await(1100, 50, () -> task.getCurrentIterationsStatistic().size() == 1); List iterationsStatic = task.getCurrentIterationsStatistic(); diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java index c3800ae3c4f3..9d7c270c9620 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java @@ -78,15 +78,25 @@ public void execute(ScmClient scmClient) throws IOException { = balancerStatusInfo.getIterationsStatusInfoList(); System.out.println("Current iteration info:"); - System.out.println( - getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1)) - ); + ContainerBalancerTaskIterationStatusInfoProto currentIterationStatistic = iterationsStatusInfoList.stream() + .filter(it -> it.getIterationResult().isEmpty()) + .findFirst() + .orElse(null); + if (currentIterationStatistic == null) { + System.out.println("-\n"); + } else { + System.out.println( + getPrettyIterationStatusInfo(currentIterationStatistic) + ); + } + if (verboseWithHistory) { System.out.println("Iteration history list:"); System.out.println( - iterationsStatusInfoList.subList(0, iterationsStatusInfoList.size() - 1) + iterationsStatusInfoList .stream() + .filter(it -> !it.getIterationResult().isEmpty()) .map(this::getPrettyIterationStatusInfo) .collect(Collectors.joining("\n")) ); diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index ece820e67f60..608fd27f8e6d 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -85,12 +85,12 @@ Verify Balancer Iteration Verify Balancer Iteration History [arguments] ${output} Should Contain ${output} Iteration history list: - Should Contain X Times ${output} Size scheduled to move 2 collapse_spaces=True - Should Contain X Times ${output} Moved data size 2 collapse_spaces=True - Should Contain X Times ${output} Scheduled to move containers 2 collapse_spaces=True - Should Contain X Times ${output} Already moved containers 2 collapse_spaces=True - Should Contain X Times ${output} Failed to move containers 0 2 collapse_spaces=True - Should Contain X Times ${output} Failed to move containers by timeout 0 2 collapse_spaces=True + Should Contain X Times ${output} Size scheduled to move 1 collapse_spaces=True + Should Contain X Times ${output} Moved data size 1 collapse_spaces=True + Should Contain X Times ${output} Scheduled to move containers 1 collapse_spaces=True + Should Contain X Times ${output} Already moved containers 1 collapse_spaces=True + Should Contain X Times ${output} Failed to move containers 0 1 collapse_spaces=True + Should Contain X Times ${output} Failed to move containers by timeout 0 1 collapse_spaces=True Should Contain ${output} Iteration result ITERATION_COMPLETED collapse_spaces=True Run Balancer Status From 4ff34c70eca2cbb5353550aa7418ea9676e9204c Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 9 Dec 2024 17:22:36 +0300 Subject: [PATCH 39/39] HDDS-11367. Fix review flaky test. --- .../TestContainerBalancerStatusInfo.java | 17 +---------------- 1 file changed, 1 insertion(+), 16 deletions(-) diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java index fafd43c69ff5..bad63602544c 100644 --- a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -141,7 +141,7 @@ void testGetCurrentStatisticsWithDelay() throws Exception { configuration.set(HddsConfigKeys.HDDS_SCM_WAIT_TIME_AFTER_SAFE_MODE_EXIT, "1"); ContainerBalancerTask task = mockedScm.startBalancerTaskAsync(config, configuration, true); // Delay in finishing the first iteration - LambdaTestUtils.await(1100, 50, () -> task.getCurrentIterationsStatistic().size() == 1); + LambdaTestUtils.await(1100, 1, () -> task.getCurrentIterationsStatistic().size() == 1); List iterationsStatic = task.getCurrentIterationsStatistic(); assertEquals(1, iterationsStatic.size()); ContainerBalancerTaskIterationStatusInfo currentIteration = iterationsStatic.get(0); @@ -169,21 +169,6 @@ void testGetCurrentStatisticsWhileBalancingInProgress() throws Exception { assertCurrentIterationStatisticWhileBalancingInProgress(currentIteration); } - - private static void verifyUnstartedIteration(ContainerBalancerTaskIterationStatusInfo iterationsStatic) { - assertEquals(0, iterationsStatic.getIterationNumber()); - assertEquals(-1, iterationsStatic.getIterationDuration()); - assertNull(iterationsStatic.getIterationResult()); - assertEquals(0, iterationsStatic.getContainerMovesScheduled()); - assertEquals(0, iterationsStatic.getContainerMovesCompleted()); - assertEquals(0, iterationsStatic.getContainerMovesFailed()); - assertEquals(0, iterationsStatic.getContainerMovesTimeout()); - assertEquals(0, iterationsStatic.getSizeScheduledForMove()); - assertEquals(0, iterationsStatic.getDataSizeMoved()); - assertTrue(iterationsStatic.getSizeEnteringNodes().isEmpty()); - assertTrue(iterationsStatic.getSizeLeavingNodes().isEmpty()); - } - private static void assertCurrentIterationStatisticWhileBalancingInProgress( ContainerBalancerTaskIterationStatusInfo iterationsStatic ) {