From 5bf295e2e3ffe18321e40fe1b4f9354a6caa56ee Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 8 Jul 2024 19:18:52 +0300 Subject: [PATCH 01/27] Init commit --- .../hadoop/hdds/scm/client/ScmClient.java | 3 + .../StorageContainerLocationProtocol.java | 3 + ...ocationProtocolClientSideTranslatorPB.java | 15 ++ .../src/main/proto/ScmAdminProtocol.proto | 31 ++++ .../src/main/resources/proto.lock | 50 ++++++ .../balancer/AbstractFindTargetGreedy.java | 4 + .../container/balancer/ContainerBalancer.java | 14 ++ .../ContainerBalancerConfiguration.java | 2 +- .../balancer/ContainerBalancerMetrics.java | 5 + .../balancer/ContainerBalancerTask.java | 84 ++++++++++ ...tainerBalancerTaskIterationStatusInfo.java | 82 +++++++++ .../ContainerBalancerTaskStatusInfo.java | 28 ++++ .../container/balancer/FindSourceGreedy.java | 5 + .../balancer/FindSourceStrategy.java | 3 + .../balancer/FindTargetStrategy.java | 3 + ...ocationProtocolServerSideTranslatorPB.java | 18 ++ .../scm/server/SCMClientProtocolServer.java | 61 ++++++- .../apache/hadoop/ozone/audit/SCMAction.java | 1 + .../TestContainerBalancerStatusInfo.java | 42 +++++ .../ContainerBalancerStatusSubcommand.java | 109 +++++++++++- .../scm/cli/ContainerOperationClient.java | 6 + .../TestContainerBalancerSubCommand.java | 156 +++++++++++++++++- 22 files changed, 710 insertions(+), 15 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java create mode 100644 hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index 14fb0a40cd00..c481468ad854 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -24,6 +24,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerReplicaInfo; @@ -384,6 +385,8 @@ StartContainerBalancerResponseProto startContainerBalancer( */ boolean getContainerBalancerStatus() throws IOException; + ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo(); + /** * returns the list of ratis peer roles. Currently only include peer address. */ diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index df8ed02cf7f0..2935e88b2d36 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -25,6 +25,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.ScmConfig; import org.apache.hadoop.hdds.scm.ScmInfo; @@ -429,6 +430,8 @@ StartContainerBalancerResponseProto startContainerBalancer( */ boolean getContainerBalancerStatus() throws IOException; + ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo(); + /** * Get Datanode usage information by ip or hostname or uuid. * diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java index 3570257b5855..c058cb4d444e 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/scm/protocolPB/StorageContainerLocationProtocolClientSideTranslatorPB.java @@ -104,6 +104,8 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StopContainerBalancerRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ResetDeletedBlockRetryCountRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.Type; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.container.ContainerID; @@ -1025,6 +1027,19 @@ public boolean getContainerBalancerStatus() throws IOException { } + @Override + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() throws IOException { + + ContainerBalancerStatusInfoRequestProto request = + ContainerBalancerStatusInfoRequestProto.getDefaultInstance(); + ContainerBalancerStatusInfoResponseProto response = + submitRequest(Type.GetContainerBalancerStatusInfo, + builder -> builder.setContainerBalancerStatusInfoRequest(request)) + .getContainerBalancerStatusInfoResponse(); + return response; + + } + /** * Builds request for datanode usage information and receives response. * diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index c190dc3f4517..84c4295fb229 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -84,6 +84,7 @@ message ScmContainerLocationRequest { optional SingleNodeQueryRequestProto singleNodeQueryRequest = 45; optional GetContainersOnDecomNodeRequestProto getContainersOnDecomNodeRequest = 46; optional GetMetricsRequestProto getMetricsRequest = 47; + optional ContainerBalancerStatusInfoRequestProto containerBalancerStatusInfoRequest = 48; } message ScmContainerLocationResponse { @@ -139,6 +140,7 @@ message ScmContainerLocationResponse { optional SingleNodeQueryResponseProto singleNodeQueryResponse = 45; optional GetContainersOnDecomNodeResponseProto getContainersOnDecomNodeResponse = 46; optional GetMetricsResponseProto getMetricsResponse = 47; + optional ContainerBalancerStatusInfoResponseProto containerBalancerStatusInfoResponse = 48; enum Status { OK = 1; @@ -193,6 +195,7 @@ enum Type { SingleNodeQuery = 41; GetContainersOnDecomNode = 42; GetMetrics = 43; + GetContainerBalancerStatusInfo = 44; } /** @@ -607,6 +610,34 @@ message ContainerBalancerStatusResponseProto { required bool isRunning = 1; } +message ContainerBalancerStatusInfoRequestProto { + optional string traceID = 1; +} + +message ContainerBalancerStatusInfoResponseProto { + required uint64 startedAt = 1; + required ContainerBalancerConfigurationProto configuration = 2; + repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; +} + +message ContainerBalancerTaskIterationStatusInfo { + required int32 iterationNumber = 1; + optional string iterationResult = 2; + required int64 sizeScheduledForMove = 3; + required int64 dataSizeMovedGB = 4; + required int64 containerMovesScheduled = 5; + required int64 containerMovesCompleted = 6; + required int64 containerMovesFailed = 7; + required int64 containerMovesTimeout = 8; + repeated NodeTransferInfo sizeEnteringNodes = 9; + repeated NodeTransferInfo sizeLeavingNodes = 10; +} + +message NodeTransferInfo { + required string uuid = 1; + required int64 dataVolume = 2; +} + message DecommissionScmRequestProto { required string scmId = 1; } diff --git a/hadoop-hdds/interface-admin/src/main/resources/proto.lock b/hadoop-hdds/interface-admin/src/main/resources/proto.lock index 8e898afc5c11..81368e3f60fd 100644 --- a/hadoop-hdds/interface-admin/src/main/resources/proto.lock +++ b/hadoop-hdds/interface-admin/src/main/resources/proto.lock @@ -187,6 +187,10 @@ { "name": "DecommissionScm", "integer": 40 + }, + { + "name": "GetContainerBalancerStatusInfo", + "integer": 44 } ] }, @@ -497,6 +501,12 @@ "name": "decommissionScmRequest", "type": "DecommissionScmRequestProto", "optional": true + }, + { + "id": 48, + "name": "containerBalancerStatusInfoRequest", + "type": "ContainerBalancerStatusInfoRequestProto", + "optional": true } ] }, @@ -772,6 +782,12 @@ "name": "decommissionScmResponse", "type": "DecommissionScmResponseProto", "optional": true + }, + { + "id": 48, + "name": "containerBalancerStatusInfoResponse", + "type": "ContainerBalancerStatusInfoResponseProto", + "optional": true } ] }, @@ -1915,6 +1931,40 @@ "optional": true } ] + }, + { + "name": "ContainerBalancerStatusInfoRequestProto", + "fields": [ + { + "id": 1, + "name": "traceID", + "type": "string", + "optional": true + } + ] + }, + { + "name": "ContainerBalancerStatusInfoResponseProto", + "fields": [ + { + "id": 1, + "name": "startedAt", + "type": "uint64", + "required": true + }, + { + "id": 2, + "name": "configuration", + "type": "ContainerBalancerConfigurationProto", + "required": true + }, + { + "id": 3, + "name": "iterationsStatusInfo", + "type": "ContainerBalancerTaskIterationStatusInfo", + "is_repeated": true + } + ] } ], "services": [ 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 5416a9ff1c38..dd2d1c578940 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 @@ -279,4 +279,8 @@ NodeManager getNodeManager() { return nodeManager; } + @Override + public Map getSizeEnteringNodes() { + return sizeEnteringNode; + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index f47abe65befd..0f0f9e0a598c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -31,6 +31,7 @@ import org.slf4j.LoggerFactory; import java.io.IOException; +import java.time.OffsetDateTime; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.locks.ReentrantLock; @@ -53,6 +54,7 @@ public class ContainerBalancer extends StatefulService { private volatile Thread currentBalancingThread; private volatile ContainerBalancerTask task = null; private ReentrantLock lock; + private OffsetDateTime startedAt; /** * Constructs ContainerBalancer with the specified arguments. Initializes @@ -175,6 +177,17 @@ public ContainerBalancerTask.Status getBalancerStatus() { : ContainerBalancerTask.Status.STOPPED; } + /** + * Get balancer status info + * @return balancer status info if balancer started + */ + public ContainerBalancerTaskStatusInfo getBalancerStatusInfo() { + return new ContainerBalancerTaskStatusInfo( + this.startedAt, + task.getConfig(), + task.getCurrentIterationsStatistic() + ); + } /** * Checks if ContainerBalancer is in valid state to call stop. * @@ -204,6 +217,7 @@ public String getServiceName() { @Override public void start() throws IllegalContainerBalancerStateException, InvalidContainerBalancerConfigurationException { + startedAt = OffsetDateTime.now(); lock.lock(); try { // should be leader-ready, out of safe mode, and not running already diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java index e275d345a5a7..644e362b3bf2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java @@ -453,7 +453,7 @@ public String toString() { excludeNodes.equals("") ? "None" : excludeNodes); } - ContainerBalancerConfigurationProto.Builder toProtobufBuilder() { + public ContainerBalancerConfigurationProto.Builder toProtobufBuilder() { ContainerBalancerConfigurationProto.Builder builder = ContainerBalancerConfigurationProto.newBuilder(); 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 09558d3a6d4f..6446089db353 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 @@ -131,6 +131,11 @@ void incrementNumContainerMovesScheduledInLatestIteration(long valueToAdd) { this.numContainerMovesScheduledInLatestIteration.incr(valueToAdd); } + public void resetNumContainerMovesScheduledInLatestIteration() { + numContainerMovesScheduledInLatestIteration.incr( + -getNumContainerMovesScheduledInLatestIteration()); + } + /** * Gets the amount of data moved 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 8daf764057ef..d47924e6808c 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 @@ -51,12 +51,14 @@ import java.util.Iterator; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ExecutionException; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.stream.Collectors; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL; import static org.apache.hadoop.hdds.HddsConfigKeys.HDDS_NODE_REPORT_INTERVAL_DEFAULT; @@ -115,6 +117,7 @@ public class ContainerBalancerTask implements Runnable { private IterationResult iterationResult; private int nextIterationIndex; private boolean delayStart; + private List iterationsStatistic; /** * Constructs ContainerBalancerTask with the specified arguments. @@ -155,6 +158,7 @@ public ContainerBalancerTask(StorageContainerManager scm, this.selectedSources = new HashSet<>(); this.selectedTargets = new HashSet<>(); findSourceStrategy = new FindSourceGreedy(nodeManager); + this.iterationsStatistic = new ArrayList<>(); } /** @@ -250,7 +254,9 @@ private void balance() { } IterationResult iR = doIteration(); + saveIterationStatistic(i, iR); metrics.incrementNumIterations(1); + LOG.info("Result of this iteration of Container Balancer: {}", iR); // if no new move option is generated, it means the cluster cannot be @@ -292,6 +298,79 @@ private void balance() { tryStopWithSaveConfiguration("Completed all iterations."); } + private void saveIterationStatistic(Integer iterationNumber, IterationResult iterationResult) { + ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + iterationNumber, + iterationResult.name(), + getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, + metrics.getDataSizeMovedGBInLatestIteration(), + 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(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ), + findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ) + ); + iterationsStatistic.add(iterationStatistic); + } + + public List getCurrentIterationsStatistic() { + int lastIterationNumber = iterationsStatistic.stream().mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber).max().orElse(0); + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + lastIterationNumber + 1, + null, + getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, + sizeActuallyMovedInLatestIteration / OzoneConsts.GB, + 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(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ), + findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ) + ); + iterationsStatistic.add(currentIterationStatistic); + return iterationsStatistic; + } + /** * Logs the reason for stop and save configuration and stop the task. * @@ -1060,6 +1139,7 @@ private void resetState() { this.sizeScheduledForMoveInLatestIteration = 0; this.sizeActuallyMovedInLatestIteration = 0; metrics.resetDataSizeMovedGBInLatestIteration(); + metrics.resetNumContainerMovesScheduledInLatestIteration(); metrics.resetNumContainerMovesCompletedInLatestIteration(); metrics.resetNumContainerMovesTimeoutInLatestIteration(); metrics.resetNumDatanodesInvolvedInLatestIteration(); @@ -1136,6 +1216,10 @@ IterationResult getIterationResult() { return iterationResult; } + ContainerBalancerConfiguration getConfig() { + return config; + } + @VisibleForTesting void setConfig(ContainerBalancerConfiguration config) { this.config = config; 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 new file mode 100644 index 000000000000..778d5766ea18 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java @@ -0,0 +1,82 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +import java.util.Map; +import java.util.UUID; + +public class ContainerBalancerTaskIterationStatusInfo { + private final Integer iterationNumber; + private final String iterationResult; + private final long sizeScheduledForMove; + private final long dataSizeMovedGB; + private final long containerMovesScheduled; + private final long containerMovesCompleted; + private final long containerMovesFailed; + private final long containerMovesTimeout; + private final Map sizeEnteringNodes; + private final Map sizeLeavingNodes; + + public ContainerBalancerTaskIterationStatusInfo( + Integer iterationNumber, + String iterationResult, + long sizeScheduledForMove, + long dataSizeMovedGB, + long containerMovesScheduled, + long containerMovesCompleted, + long containerMovesFailed, + long containerMovesTimeout, + Map sizeEnteringNodes, + Map sizeLeavingNodes) { + this.iterationNumber = iterationNumber; + this.iterationResult = iterationResult; + this.sizeScheduledForMove = sizeScheduledForMove; + this.dataSizeMovedGB = dataSizeMovedGB; + this.containerMovesScheduled = containerMovesScheduled; + this.containerMovesCompleted = containerMovesCompleted; + this.containerMovesFailed = containerMovesFailed; + this.containerMovesTimeout = containerMovesTimeout; + this.sizeEnteringNodes = sizeEnteringNodes; + this.sizeLeavingNodes = sizeLeavingNodes; + } + + public Integer getIterationNumber() { + return iterationNumber; + } + + public String getIterationResult() { + return iterationResult; + } + + public long getSizeScheduledForMove() { + return sizeScheduledForMove; + } + + public long getDataSizeMovedGB() { + return dataSizeMovedGB; + } + + public long getContainerMovesScheduled() { + return containerMovesScheduled; + } + + public long getContainerMovesCompleted() { + return containerMovesCompleted; + } + + public long getContainerMovesFailed() { + return containerMovesFailed; + } + + public long getContainerMovesTimeout() { + return containerMovesTimeout; + } + + 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/ContainerBalancerTaskStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java new file mode 100644 index 000000000000..21d3d1964b48 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java @@ -0,0 +1,28 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +import java.time.OffsetDateTime; +import java.util.List; + +public class ContainerBalancerTaskStatusInfo { + private final OffsetDateTime startedAt; + private final ContainerBalancerConfiguration configuration; + private final List iterationsStatusInfo; + + public ContainerBalancerTaskStatusInfo(OffsetDateTime startedAt, ContainerBalancerConfiguration configuration, List iterationsStatusInfo) { + this.startedAt = startedAt; + this.configuration = configuration; + this.iterationsStatusInfo = iterationsStatusInfo; + } + + public OffsetDateTime getStartedAt() { + return startedAt; + } + + public ContainerBalancerConfiguration getConfiguration() { + return configuration; + } + + public List getIterationsStatusInfo() { + return iterationsStatusInfo; + } +} 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 8306d8e1e1ff..435cc9859a94 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 @@ -201,4 +201,9 @@ public void reInitialize(List potentialDataNodes, sizeLeavingNode.clear(); resetSources(potentialDataNodes); } + + @Override + public Map getSizeLeavingNodes() { + return sizeLeavingNode; + } } 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 f9eb24bd3cc6..9e429aaa21d9 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 @@ -24,6 +24,7 @@ import jakarta.annotation.Nonnull; import java.util.Collection; import java.util.List; +import java.util.Map; /** * This interface can be used to implement strategies to get a @@ -85,4 +86,6 @@ void reInitialize(List potentialDataNodes, * {@link DatanodeDetails} that containers can move from */ void resetPotentialSources(@Nonnull Collection sources); + + 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 a9f2ee00a2d3..389ea6e5192f 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 @@ -25,6 +25,7 @@ import jakarta.annotation.Nonnull; import java.util.Collection; import java.util.List; +import java.util.Map; /** * This interface can be used to implement strategies to find a target for a @@ -68,4 +69,6 @@ void reInitialize(List potentialDataNodes, * that containers can be moved to */ void resetPotentialTargets(@Nonnull Collection targets); + + Map getSizeEnteringNodes(); } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 3d7cff358fe4..684a81ab5530 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -600,6 +600,13 @@ public ScmContainerLocationResponse processRequest( .setContainerBalancerStatusResponse(getContainerBalancerStatus( request.getContainerBalancerStatusRequest())) .build(); + case GetContainerBalancerStatusInfo: + return ScmContainerLocationResponse.newBuilder() + .setCmdType(request.getCmdType()) + .setStatus(Status.OK) + .setContainerBalancerStatusInfoResponse(getContainerBalancerStatusInfo( + request.getContainerBalancerStatusInfoRequest())) + .build(); case GetPipeline: return ScmContainerLocationResponse.newBuilder() .setCmdType(request.getCmdType()) @@ -1196,6 +1203,17 @@ public ContainerBalancerStatusResponseProto getContainerBalancerStatus( .setIsRunning(impl.getContainerBalancerStatus()).build(); } + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo( + ContainerBalancerStatusInfoRequestProto request) + throws IOException { + ContainerBalancerStatusInfoResponseProto containerBalancerStatusInfo = impl.getContainerBalancerStatusInfo(); + return ContainerBalancerStatusInfoResponseProto.newBuilder() + .setStartedAt(containerBalancerStatusInfo.getStartedAt()) + .setConfiguration(containerBalancerStatusInfo.getConfiguration()) + .addAllIterationsStatusInfo(containerBalancerStatusInfo.getIterationsStatusInfoList()) + .build(); + } + public DecommissionNodesResponseProto decommissionNodes( DecommissionNodesRequestProto request) throws IOException { List errors = 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 47bc66d8331a..4e16548bf1ca 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 @@ -33,9 +33,10 @@ import org.apache.hadoop.hdds.conf.ReconfigurationHandler; import org.apache.hadoop.hdds.protocol.DatanodeDetails; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -import org.apache.hadoop.hdds.protocol.proto.ReconfigureProtocolProtos.ReconfigureProtocolService; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo; +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.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto.Builder; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; @@ -43,18 +44,20 @@ import org.apache.hadoop.hdds.protocolPB.ReconfigureProtocolServerSideTranslatorPB; import org.apache.hadoop.hdds.ratis.RatisHelper; import org.apache.hadoop.hdds.scm.DatanodeAdminError; +import org.apache.hadoop.hdds.scm.FetchMetrics; import org.apache.hadoop.hdds.scm.ScmInfo; import org.apache.hadoop.hdds.scm.container.ContainerID; import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException; import org.apache.hadoop.hdds.scm.container.ContainerReplica; -import org.apache.hadoop.hdds.scm.container.common.helpers.DeletedBlocksTransactionInfoWrapper; import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; +import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerTaskStatusInfo; import org.apache.hadoop.hdds.scm.container.balancer.IllegalContainerBalancerStateException; import org.apache.hadoop.hdds.scm.container.balancer.InvalidContainerBalancerConfigurationException; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdds.scm.container.common.helpers.DeletedBlocksTransactionInfoWrapper; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.exceptions.SCMException; import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes; @@ -62,7 +65,6 @@ import org.apache.hadoop.hdds.scm.ha.SCMRatisServer; import org.apache.hadoop.hdds.scm.ha.SCMRatisServerImpl; import org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo; -import org.apache.hadoop.hdds.scm.FetchMetrics; import org.apache.hadoop.hdds.scm.node.NodeStatus; import org.apache.hadoop.hdds.scm.node.states.NodeNotFoundException; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; @@ -108,14 +110,14 @@ import java.util.Optional; import java.util.Set; import java.util.TreeSet; +import java.util.UUID; import java.util.stream.Collectors; import java.util.stream.Stream; -import java.util.UUID; import static org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StorageContainerLocationProtocolService.newReflectiveBlockingService; +import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_DEFAULT; import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_HANDLER_COUNT_KEY; -import static org.apache.hadoop.hdds.scm.ScmConfigKeys.OZONE_SCM_CLIENT_HANDLER_COUNT_KEY; import static org.apache.hadoop.hdds.scm.ScmUtils.checkIfCertSignRequestAllowed; import static org.apache.hadoop.hdds.scm.ha.HASecurityUtils.createSCMRatisTLSConfig; import static org.apache.hadoop.hdds.scm.server.StorageContainerManager.startRpcServer; @@ -1202,6 +1204,55 @@ public boolean getContainerBalancerStatus() { return scm.getContainerBalancer().isBalancerRunning(); } + @Override + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { + AUDIT.logReadSuccess(buildAuditMessageForSuccess( + SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); + ContainerBalancerTaskStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); + return ContainerBalancerStatusInfoResponseProto + .newBuilder() + .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) + .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) + .addAllIterationsStatusInfo( + balancerStatusInfo.getIterationsStatusInfo() + .stream() + .map( + info -> StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(info.getIterationNumber()) + .setIterationResult(info.getIterationResult()) + .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setDataSizeMovedGB(info.getDataSizeMovedGB()) + .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(); + } + /** * Get Datanode usage info such as capacity, SCMUsed, and remaining by ip * or hostname or uuid. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java index 4e1fe234ff01..128e3401c370 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/ozone/audit/SCMAction.java @@ -47,6 +47,7 @@ public enum SCMAction implements AuditAction { START_CONTAINER_BALANCER, STOP_CONTAINER_BALANCER, GET_CONTAINER_BALANCER_STATUS, + GET_CONTAINER_BALANCER_STATUS_INFO, GET_CONTAINER_WITH_PIPELINE_BATCH, ADD_SCM, GET_REPLICATION_MANAGER_REPORT, 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 new file mode 100644 index 000000000000..a9ce6b13798c --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java @@ -0,0 +1,42 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.ozone.OzoneConsts; +import org.junit.jupiter.api.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + + +public class TestContainerBalancerStatusInfo { + private static final Logger LOG = + LoggerFactory.getLogger(TestContainerBalancerStatusInfo.class); + + @Test + void testGetIterationStatistics() { + 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 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()); + }); + + } +} 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 44e4d4c9c50b..f74cc31aedbb 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 @@ -18,10 +18,21 @@ package org.apache.hadoop.hdds.scm.cli; import org.apache.hadoop.hdds.cli.HddsVersionProvider; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.client.ScmClient; +import org.apache.hadoop.ozone.OzoneConsts; +import picocli.CommandLine; import picocli.CommandLine.Command; import java.io.IOException; +import java.time.Duration; +import java.time.Instant; +import java.time.LocalDateTime; +import java.time.ZoneId; +import java.util.List; +import java.util.stream.Collectors; /** * Handler to query status of container balancer. @@ -33,13 +44,103 @@ versionProvider = HddsVersionProvider.class) public class ContainerBalancerStatusSubcommand extends ScmSubcommand { + @CommandLine.Option(names = {"-v", "--verbose"}, + description = "More verbose output. Show balance iteration history.") + private boolean verbose; + @Override public void execute(ScmClient scmClient) throws IOException { - boolean execReturn = scmClient.getContainerBalancerStatus(); - if (execReturn) { - System.out.println("ContainerBalancer is Running."); - } else { + ContainerBalancerStatusInfoResponseProto balancerStatusInfo = scmClient.getContainerBalancerStatusInfo(); + if (balancerStatusInfo == null) { System.out.println("ContainerBalancer is Not Running."); + } else { + LocalDateTime dateTime = + LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); + System.out.println("ContainerBalancer is Running."); + System.out.printf("Started at: %s %s\n\n", dateTime.toLocalDate(), dateTime.toLocalTime()); + System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); + List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); + if (verbose) { + System.out.println("Iteration history list:"); + System.out.println(iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo).collect(Collectors.joining("\n"))); + } else { + System.out.println("Current iteration info:"); + System.out.println(getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1))); + } } } + + String getConfigurationPrettyString(HddsProtos.ContainerBalancerConfigurationProto configuration) { + return String.format("Container Balancer Configuration values:%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %d%n" + + "%-50s %dGB%n" + + "%-50s %dGB%n" + + "%-50s %dGB%n" + + "%-50s %d%n" + + "%-50s %dmin%n" + + "%-50s %dmin%n" + + "%-50s %dmin%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n", "Key", "Value", "Threshold", + configuration.getUtilizationThreshold(), "Max Datanodes to Involve per Iteration(percent)", + configuration.getDatanodesInvolvedMaxPercentagePerIteration(), + "Max Size to Move per Iteration", + configuration.getDatanodesInvolvedMaxPercentagePerIteration() / OzoneConsts.GB, + "Max Size Entering Target per Iteration", + configuration.getSizeEnteringTargetMax() / OzoneConsts.GB, + "Max Size Leaving Source per Iteration", + configuration.getSizeLeavingSourceMax() / OzoneConsts.GB, + "Number of Iterations", + configuration.getIterations(), + "Time Limit for Single Container's Movement", + Duration.ofMillis(configuration.getMoveTimeout()).toMinutes(), + "Time Limit for Single Container's Replication", + Duration.ofMillis(configuration.getMoveReplicationTimeout()).toMinutes(), + "Interval between each Iteration", + Duration.ofMillis(configuration.getBalancingIterationInterval()).toMinutes(), + "Whether to Enable Network Topology", + configuration.getMoveNetworkTopologyEnable(), + "Whether to Trigger Refresh Datanode Usage Info", + configuration.getTriggerDuBeforeMoveEnable(), + "Container IDs to Exclude from Balancing", + configuration.getExcludeContainers().isEmpty() ? "None" : configuration.getExcludeContainers(), + "Datanodes Specified to be Balanced", + configuration.getIncludeDatanodes().isEmpty() ? "None" : configuration.getIncludeDatanodes(), + "Datanodes Excluded from Balancing", + configuration.getExcludeDatanodes().isEmpty() ? "None" : configuration.getExcludeDatanodes()); + } + + private String getPrettyIterationStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { + return String.format( + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s %s%n" + + "%-50s \n%s" + + "%-50s \n%s", + "Key", "Value", + "Iteration number", iterationStatusInfo.getIterationNumber(), + "Iteration result", iterationStatusInfo.getIterationResult().isEmpty() ? "IN_PROGRESS" : iterationStatusInfo.getIterationResult(), + "Size scheduled to move", iterationStatusInfo.getSizeScheduledForMove(), + "Moved data size", iterationStatusInfo.getDataSizeMovedGB(), + "Scheduled to move containers", iterationStatusInfo.getContainerMovesScheduled(), + "Already moved containers", iterationStatusInfo.getContainerMovesCompleted(), + "Failed to move containers", iterationStatusInfo.getContainerMovesFailed(), + "Failed to move containers by timeout", iterationStatusInfo.getContainerMovesTimeout(), + "Entered data to nodes", iterationStatusInfo.getSizeEnteringNodesList().stream().map( + nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolume() + "\n") + .collect(Collectors.joining()), + "Exited data from nodes", iterationStatusInfo.getSizeLeavingNodesList().stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolume() + "\n").collect(Collectors.joining())); + } } + diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java index 0dd52cd291ab..ddb552f06e06 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java @@ -31,6 +31,7 @@ import org.apache.hadoop.hdds.protocol.proto.HddsProtos.DeletedBlocksTransactionInfo; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.DecommissionScmResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.StartContainerBalancerResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.DatanodeAdminError; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.XceiverClientManager; @@ -508,6 +509,11 @@ public boolean getContainerBalancerStatus() throws IOException { return storageContainerLocationClient.getContainerBalancerStatus(); } + @Override + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { + return storageContainerLocationClient.getContainerBalancerStatusInfo(); + } + @Override public List getScmRatisRoles() throws IOException { return storageContainerLocationClient.getScmInfo().getRatisPeerRoles(); 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 27c360e72743..0d2e1e14828a 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,10 +18,11 @@ package org.apache.hadoop.hdds.scm.cli.datanode; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos; -import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStopSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStartSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStatusSubcommand; +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.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; @@ -31,6 +32,8 @@ import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.nio.charset.StandardCharsets; +import java.time.OffsetDateTime; +import java.util.Arrays; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -69,17 +72,160 @@ public void tearDown() { } @Test - public void testContainerBalancerStatusSubcommandRunning() - throws IOException { + public 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 = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(0) + .setIterationResult("ITERATION_COMPLETED") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(1) + .setIterationResult("ITERATION_COMPLETED") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27L) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(1) + .setIterationResult("") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27L) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = + StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto.newBuilder() + .setStartedAt(OffsetDateTime.now().toEpochSecond()) + .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) + .addAllIterationsStatusInfo( + Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) + ) + .build(); //test status is running - when(scmClient.getContainerBalancerStatus()).thenAnswer(invocation -> true); + when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); statusCmd.execute(scmClient); + Pattern p = Pattern.compile( + "^ContainerBalancer\\sis\\sRunning."); + Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); + assertTrue(m.find()); + } + + @Test + public void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() + throws IOException { + ScmClient scmClient = mock(ScmClient.class); + //test status is running + when(scmClient.getContainerBalancerStatusInfo()).thenReturn(null); + + statusCmd.execute(scmClient); Pattern p = Pattern.compile( - "^ContainerBalancer\\sis\\sRunning."); + "^ContainerBalancer\\sis\\sNot\\sRunning."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); } From 9dafda8cd195f82e451954bc7caa03adb5c5537a Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 09:56:39 +0300 Subject: [PATCH 02/27] HDDS-11120. Fix compilation --- .../main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java | 2 +- .../hdds/scm/protocol/StorageContainerLocationProtocol.java | 2 +- ...StorageContainerLocationProtocolServerSideTranslatorPB.java | 3 ++- .../apache/hadoop/hdds/scm/cli/ContainerOperationClient.java | 2 +- 4 files changed, 5 insertions(+), 4 deletions(-) diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java index c481468ad854..34b2680b301b 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/client/ScmClient.java @@ -385,7 +385,7 @@ StartContainerBalancerResponseProto startContainerBalancer( */ boolean getContainerBalancerStatus() throws IOException; - ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo(); + ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() throws IOException; /** * returns the list of ratis peer roles. Currently only include peer address. diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java index 2935e88b2d36..45825cb2b621 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocol.java @@ -430,7 +430,7 @@ StartContainerBalancerResponseProto startContainerBalancer( */ boolean getContainerBalancerStatus() throws IOException; - ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo(); + ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() throws IOException; /** * Get Datanode usage information by ip or hostname or uuid. diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 684a81ab5530..5eb847ea36f2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -34,6 +34,7 @@ import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ActivatePipelineResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ClosePipelineRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ClosePipelineResponseProto; +import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusRequestProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerStatusResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerRequestProto; @@ -1204,7 +1205,7 @@ public ContainerBalancerStatusResponseProto getContainerBalancerStatus( } public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo( - ContainerBalancerStatusInfoRequestProto request) + StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto request) throws IOException { ContainerBalancerStatusInfoResponseProto containerBalancerStatusInfo = impl.getContainerBalancerStatusInfo(); return ContainerBalancerStatusInfoResponseProto.newBuilder() diff --git a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java index ddb552f06e06..ba556bf24e98 100644 --- a/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java +++ b/hadoop-hdds/tools/src/main/java/org/apache/hadoop/hdds/scm/cli/ContainerOperationClient.java @@ -510,7 +510,7 @@ public boolean getContainerBalancerStatus() throws IOException { } @Override - public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() throws IOException { return storageContainerLocationClient.getContainerBalancerStatusInfo(); } From 8d78b361286ae11f74b7d4edc2ee327c3818883f Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 13:45:27 +0300 Subject: [PATCH 03/27] HDDS-11120. Fix checkstyle errors --- .../container/balancer/ContainerBalancer.java | 7 +- .../balancer/ContainerBalancerStatusInfo.java | 34 +++++ .../balancer/ContainerBalancerTask.java | 9 +- ...tainerBalancerTaskIterationStatusInfo.java | 128 +++++++++--------- .../ContainerBalancerTaskStatusInfo.java | 28 ---- .../scm/server/SCMClientProtocolServer.java | 12 +- .../TestContainerBalancerStatusInfo.java | 56 ++++---- .../ContainerBalancerStatusSubcommand.java | 54 +++++--- 8 files changed, 180 insertions(+), 148 deletions(-) create mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java delete mode 100644 hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 0f0f9e0a598c..2c46801b6607 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -178,11 +178,12 @@ public ContainerBalancerTask.Status getBalancerStatus() { } /** - * Get balancer status info + * Get balancer status info. + * * @return balancer status info if balancer started */ - public ContainerBalancerTaskStatusInfo getBalancerStatusInfo() { - return new ContainerBalancerTaskStatusInfo( + public ContainerBalancerStatusInfo getBalancerStatusInfo() { + return new ContainerBalancerStatusInfo( this.startedAt, task.getConfig(), task.getCurrentIterationsStatistic() 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 new file mode 100644 index 000000000000..7aa3ee3b4b8a --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java @@ -0,0 +1,34 @@ +package org.apache.hadoop.hdds.scm.container.balancer; + +import java.time.OffsetDateTime; +import java.util.List; + +/** + * Info about balancer status. + */ +public class ContainerBalancerStatusInfo { + private final OffsetDateTime startedAt; + private final ContainerBalancerConfiguration configuration; + private final List iterationsStatusInfo; + + public ContainerBalancerStatusInfo( + OffsetDateTime startedAt, + ContainerBalancerConfiguration configuration, + List iterationsStatusInfo) { + this.startedAt = startedAt; + this.configuration = configuration; + this.iterationsStatusInfo = iterationsStatusInfo; + } + + public OffsetDateTime getStartedAt() { + return startedAt; + } + + public ContainerBalancerConfiguration getConfiguration() { + return configuration; + } + + public List getIterationsStatusInfo() { + return iterationsStatusInfo; + } +} 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 d47924e6808c..744d8c166a4c 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 @@ -298,10 +298,10 @@ private void balance() { tryStopWithSaveConfiguration("Completed all iterations."); } - private void saveIterationStatistic(Integer iterationNumber, IterationResult iterationResult) { + private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) { ContainerBalancerTaskIterationStatusInfo iterationStatistic = new ContainerBalancerTaskIterationStatusInfo( iterationNumber, - iterationResult.name(), + iR.name(), getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, metrics.getDataSizeMovedGBInLatestIteration(), metrics.getNumContainerMovesScheduledInLatestIteration(), @@ -335,7 +335,10 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult ite } public List getCurrentIterationsStatistic() { - int lastIterationNumber = iterationsStatistic.stream().mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber).max().orElse(0); + int lastIterationNumber = iterationsStatistic.stream() + .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) + .max() + .orElse(0); ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( lastIterationNumber + 1, null, 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 778d5766ea18..7449ccf7a3bf 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 @@ -3,80 +3,84 @@ import java.util.Map; import java.util.UUID; +/** + * Information about balancer task iteration. + */ public class ContainerBalancerTaskIterationStatusInfo { - private final Integer iterationNumber; - private final String iterationResult; - private final long sizeScheduledForMove; - private final long dataSizeMovedGB; - private final long containerMovesScheduled; - private final long containerMovesCompleted; - private final long containerMovesFailed; - private final long containerMovesTimeout; - private final Map sizeEnteringNodes; - private final Map sizeLeavingNodes; + private final Integer iterationNumber; + private final String iterationResult; + private final long sizeScheduledForMove; + private final long dataSizeMovedGB; + private final long containerMovesScheduled; + private final long containerMovesCompleted; + private final long containerMovesFailed; + private final long containerMovesTimeout; + private final Map sizeEnteringNodes; + private final Map sizeLeavingNodes; - public ContainerBalancerTaskIterationStatusInfo( - Integer iterationNumber, - String iterationResult, - long sizeScheduledForMove, - long dataSizeMovedGB, - long containerMovesScheduled, - long containerMovesCompleted, - long containerMovesFailed, - long containerMovesTimeout, - Map sizeEnteringNodes, - Map sizeLeavingNodes) { - this.iterationNumber = iterationNumber; - this.iterationResult = iterationResult; - this.sizeScheduledForMove = sizeScheduledForMove; - this.dataSizeMovedGB = dataSizeMovedGB; - this.containerMovesScheduled = containerMovesScheduled; - this.containerMovesCompleted = containerMovesCompleted; - this.containerMovesFailed = containerMovesFailed; - this.containerMovesTimeout = containerMovesTimeout; - this.sizeEnteringNodes = sizeEnteringNodes; - this.sizeLeavingNodes = sizeLeavingNodes; - } + @SuppressWarnings("checkstyle:ParameterNumber") + public ContainerBalancerTaskIterationStatusInfo( + Integer iterationNumber, + String iterationResult, + long sizeScheduledForMove, + long dataSizeMovedGB, + long containerMovesScheduled, + long containerMovesCompleted, + long containerMovesFailed, + long containerMovesTimeout, + Map sizeEnteringNodes, + Map sizeLeavingNodes) { + this.iterationNumber = iterationNumber; + this.iterationResult = iterationResult; + this.sizeScheduledForMove = sizeScheduledForMove; + this.dataSizeMovedGB = dataSizeMovedGB; + this.containerMovesScheduled = containerMovesScheduled; + this.containerMovesCompleted = containerMovesCompleted; + this.containerMovesFailed = containerMovesFailed; + this.containerMovesTimeout = containerMovesTimeout; + this.sizeEnteringNodes = sizeEnteringNodes; + this.sizeLeavingNodes = sizeLeavingNodes; + } - public Integer getIterationNumber() { - return iterationNumber; - } + public Integer getIterationNumber() { + return iterationNumber; + } - public String getIterationResult() { - return iterationResult; - } + public String getIterationResult() { + return iterationResult; + } - public long getSizeScheduledForMove() { - return sizeScheduledForMove; - } + public long getSizeScheduledForMove() { + return sizeScheduledForMove; + } - public long getDataSizeMovedGB() { - return dataSizeMovedGB; - } + public long getDataSizeMovedGB() { + return dataSizeMovedGB; + } - public long getContainerMovesScheduled() { - return containerMovesScheduled; - } + public long getContainerMovesScheduled() { + return containerMovesScheduled; + } - public long getContainerMovesCompleted() { - return containerMovesCompleted; - } + public long getContainerMovesCompleted() { + return containerMovesCompleted; + } - public long getContainerMovesFailed() { - return containerMovesFailed; - } + public long getContainerMovesFailed() { + return containerMovesFailed; + } - public long getContainerMovesTimeout() { - return containerMovesTimeout; - } + public long getContainerMovesTimeout() { + return containerMovesTimeout; + } - public Map getSizeEnteringNodes() { - return sizeEnteringNodes; - } + public Map getSizeEnteringNodes() { + return sizeEnteringNodes; + } - public Map getSizeLeavingNodes() { - return sizeLeavingNodes; - } + public Map getSizeLeavingNodes() { + return sizeLeavingNodes; + } } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java deleted file mode 100644 index 21d3d1964b48..000000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskStatusInfo.java +++ /dev/null @@ -1,28 +0,0 @@ -package org.apache.hadoop.hdds.scm.container.balancer; - -import java.time.OffsetDateTime; -import java.util.List; - -public class ContainerBalancerTaskStatusInfo { - private final OffsetDateTime startedAt; - private final ContainerBalancerConfiguration configuration; - private final List iterationsStatusInfo; - - public ContainerBalancerTaskStatusInfo(OffsetDateTime startedAt, ContainerBalancerConfiguration configuration, List iterationsStatusInfo) { - this.startedAt = startedAt; - this.configuration = configuration; - this.iterationsStatusInfo = iterationsStatusInfo; - } - - public OffsetDateTime getStartedAt() { - return startedAt; - } - - public ContainerBalancerConfiguration getConfiguration() { - return configuration; - } - - public List getIterationsStatusInfo() { - return iterationsStatusInfo; - } -} 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 4e16548bf1ca..11ad182377dc 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,8 +37,10 @@ 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; @@ -53,7 +55,7 @@ import org.apache.hadoop.hdds.scm.container.ReplicationManagerReport; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; -import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerTaskStatusInfo; +import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerStatusInfo; import org.apache.hadoop.hdds.scm.container.balancer.IllegalContainerBalancerStateException; import org.apache.hadoop.hdds.scm.container.balancer.InvalidContainerBalancerConfigurationException; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; @@ -1208,7 +1210,7 @@ public boolean getContainerBalancerStatus() { public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { AUDIT.logReadSuccess(buildAuditMessageForSuccess( SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); - ContainerBalancerTaskStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); + ContainerBalancerStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); return ContainerBalancerStatusInfoResponseProto .newBuilder() .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) @@ -1217,7 +1219,7 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() balancerStatusInfo.getIterationsStatusInfo() .stream() .map( - info -> StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(info.getIterationNumber()) .setIterationResult(info.getIterationResult()) .setSizeScheduledForMove(info.getSizeScheduledForMove()) @@ -1229,7 +1231,7 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() .addAllSizeEnteringNodes( info.getSizeEnteringNodes().entrySet() .stream() - .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) .setDataVolume(entry.getValue()) .build() @@ -1239,7 +1241,7 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() .addAllSizeLeavingNodes( info.getSizeLeavingNodes().entrySet() .stream() - .map(entry -> StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) .setDataVolume(entry.getValue()) .build() 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 a9ce6b13798c..3fee5cb859d0 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 @@ -3,8 +3,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.ozone.OzoneConsts; import org.junit.jupiter.api.Test; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.util.List; @@ -12,31 +10,31 @@ import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertTrue; - -public class TestContainerBalancerStatusInfo { - private static final Logger LOG = - LoggerFactory.getLogger(TestContainerBalancerStatusInfo.class); - - @Test - void testGetIterationStatistics() { - 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 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()); - }); - - } +/** + * Tests for {@link ContainerBalancerStatusInfo}. + */ +class TestContainerBalancerStatusInfo { + + @Test + void testGetIterationStatistics() { + 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 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()); + }); + + } } 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 f74cc31aedbb..859ceec88d59 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,8 +19,8 @@ import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.protocol.proto.HddsProtos; -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.scm.client.ScmClient; import org.apache.hadoop.ozone.OzoneConsts; import picocli.CommandLine; @@ -59,13 +59,18 @@ public void execute(ScmClient scmClient) throws IOException { System.out.println("ContainerBalancer is Running."); System.out.printf("Started at: %s %s\n\n", dateTime.toLocalDate(), dateTime.toLocalTime()); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); - List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); + List iterationsStatusInfoList + = balancerStatusInfo.getIterationsStatusInfoList(); if (verbose) { System.out.println("Iteration history list:"); - System.out.println(iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo).collect(Collectors.joining("\n"))); + System.out.println( + iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo) + .collect(Collectors.joining("\n")) + ); } else { System.out.println("Current iteration info:"); - System.out.println(getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1))); + System.out.println( + getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1))); } } } @@ -115,9 +120,23 @@ String getConfigurationPrettyString(HddsProtos.ContainerBalancerConfigurationPro configuration.getExcludeDatanodes().isEmpty() ? "None" : configuration.getExcludeDatanodes()); } - private String getPrettyIterationStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { + private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { + int iterationNumber = iterationStatusInfo.getIterationNumber(); + String iterationResult = iterationStatusInfo.getIterationResult(); + long sizeScheduledForMove = iterationStatusInfo.getSizeScheduledForMove(); + long dataSizeMovedGB = iterationStatusInfo.getDataSizeMovedGB(); + long containerMovesScheduled = iterationStatusInfo.getContainerMovesScheduled(); + long containerMovesCompleted = iterationStatusInfo.getContainerMovesCompleted(); + long containerMovesFailed = iterationStatusInfo.getContainerMovesFailed(); + long containerMovesTimeout = iterationStatusInfo.getContainerMovesTimeout(); + String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolume() + "\n") + .collect(Collectors.joining()); + String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolume() + "\n") + .collect(Collectors.joining()); return String.format( - "%-50s %s%n" + + "%-50s %s%n" + "%-50s %s%n" + "%-50s %s%n" + "%-50s %s%n" + @@ -129,18 +148,17 @@ private String getPrettyIterationStatusInfo(StorageContainerLocationProtocolProt "%-50s \n%s" + "%-50s \n%s", "Key", "Value", - "Iteration number", iterationStatusInfo.getIterationNumber(), - "Iteration result", iterationStatusInfo.getIterationResult().isEmpty() ? "IN_PROGRESS" : iterationStatusInfo.getIterationResult(), - "Size scheduled to move", iterationStatusInfo.getSizeScheduledForMove(), - "Moved data size", iterationStatusInfo.getDataSizeMovedGB(), - "Scheduled to move containers", iterationStatusInfo.getContainerMovesScheduled(), - "Already moved containers", iterationStatusInfo.getContainerMovesCompleted(), - "Failed to move containers", iterationStatusInfo.getContainerMovesFailed(), - "Failed to move containers by timeout", iterationStatusInfo.getContainerMovesTimeout(), - "Entered data to nodes", iterationStatusInfo.getSizeEnteringNodesList().stream().map( - nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolume() + "\n") - .collect(Collectors.joining()), - "Exited data from nodes", iterationStatusInfo.getSizeLeavingNodesList().stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolume() + "\n").collect(Collectors.joining())); + "Iteration number", iterationNumber, + "Iteration result", + iterationResult.isEmpty() ? "IN_PROGRESS" : iterationResult, + "Size scheduled to move", sizeScheduledForMove, + "Moved data size", dataSizeMovedGB, + "Scheduled to move containers", containerMovesScheduled, + "Already moved containers", containerMovesCompleted, + "Failed to move containers", containerMovesFailed, + "Failed to move containers by timeout", containerMovesTimeout, + "Entered data to nodes", enteringDataNodeList, + "Exited data from nodes", leavingDataNodeList); } } From 04fb8719ef9c675890f1af8f3c12ff34487a7615 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 16:43:06 +0300 Subject: [PATCH 04/27] HDDS-11120. Fix findbugs errors --- .../hdds/scm/cli/ContainerBalancerStatusSubcommand.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) 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 859ceec88d59..36d052a291c9 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 @@ -145,8 +145,8 @@ private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatus "%-50s %s%n" + "%-50s %s%n" + "%-50s %s%n" + - "%-50s \n%s" + - "%-50s \n%s", + "%-50s %n%s" + + "%-50s %n%s", "Key", "Value", "Iteration number", iterationNumber, "Iteration result", From b25c6c0520bf64a2200f93c2509b441897a6282e Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 17:15:22 +0300 Subject: [PATCH 05/27] HDDS-11120. Fix findbugs errors --- .../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 36d052a291c9..1fdac95d46db 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 @@ -57,7 +57,7 @@ public void execute(ScmClient scmClient) throws IOException { LocalDateTime dateTime = LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); System.out.println("ContainerBalancer is Running."); - System.out.printf("Started at: %s %s\n\n", dateTime.toLocalDate(), dateTime.toLocalTime()); + System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(), dateTime.toLocalTime()); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); From 648dfde06083a2f7700f540147c5b2a5dd24b2b9 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 17:30:14 +0300 Subject: [PATCH 06/27] HDDS-11120. Fix rat check --- .../balancer/ContainerBalancerStatusInfo.java | 18 ++++++++++++++++++ ...ntainerBalancerTaskIterationStatusInfo.java | 18 ++++++++++++++++++ .../TestContainerBalancerStatusInfo.java | 18 ++++++++++++++++++ 3 files changed, 54 insertions(+) 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 7aa3ee3b4b8a..b5edf894e624 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 @@ -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.time.OffsetDateTime; 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 7449ccf7a3bf..11bc85074440 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 @@ -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/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 3fee5cb859d0..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 @@ -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 org.apache.hadoop.hdds.conf.OzoneConfiguration; From 7b3eb0c5205e2ce687d6e4546ae65ab0cd9a953d Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Tue, 9 Jul 2024 19:23:09 +0300 Subject: [PATCH 07/27] HDDS-11120. Try to fix robot balancer test --- .../container/balancer/ContainerBalancer.java | 3 + .../scm/server/SCMClientProtocolServer.java | 87 ++++++++++--------- 2 files changed, 48 insertions(+), 42 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 2c46801b6607..e592e641e4fb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -183,6 +183,9 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() { + if (this.startedAt == null || task == null) { + return null; + } return new ContainerBalancerStatusInfo( this.startedAt, task.getConfig(), 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 11ad182377dc..1f1cc783f103 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 @@ -1209,50 +1209,53 @@ public boolean getContainerBalancerStatus() { @Override public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { AUDIT.logReadSuccess(buildAuditMessageForSuccess( - SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); + SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); ContainerBalancerStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); + if (balancerStatusInfo == null) { + return null; + } return ContainerBalancerStatusInfoResponseProto - .newBuilder() - .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) - .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) - .addAllIterationsStatusInfo( - balancerStatusInfo.getIterationsStatusInfo() - .stream() - .map( - info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(info.getIterationNumber()) - .setIterationResult(info.getIterationResult()) - .setSizeScheduledForMove(info.getSizeScheduledForMove()) - .setDataSizeMovedGB(info.getDataSizeMovedGB()) - .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()) - ) - .build(); + .newBuilder() + .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) + .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) + .addAllIterationsStatusInfo( + balancerStatusInfo.getIterationsStatusInfo() + .stream() + .map( + info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(info.getIterationNumber()) + .setIterationResult(info.getIterationResult()) + .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setDataSizeMovedGB(info.getDataSizeMovedGB()) + .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()) + ) + .build(); } /** From a0e92f138d0e313eaa0967eb43c96b83e926d04e Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 01:39:03 +0300 Subject: [PATCH 08/27] HDDS-11120. Try to fix robot balancer test --- ...StorageContainerLocationProtocolServerSideTranslatorPB.java | 3 +++ 1 file changed, 3 insertions(+) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 5eb847ea36f2..347886389aeb 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -1208,6 +1208,9 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo( StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto request) throws IOException { ContainerBalancerStatusInfoResponseProto containerBalancerStatusInfo = impl.getContainerBalancerStatusInfo(); + if (containerBalancerStatusInfo == null) { + return ContainerBalancerStatusInfoResponseProto.newBuilder().getDefaultInstanceForType(); + } return ContainerBalancerStatusInfoResponseProto.newBuilder() .setStartedAt(containerBalancerStatusInfo.getStartedAt()) .setConfiguration(containerBalancerStatusInfo.getConfiguration()) From 2f0b05e5eea1a483d7a1aa4d88c119623b676725 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 15:27:08 +0300 Subject: [PATCH 09/27] HDDS-11120. Try to fix robot balancer test --- .../src/main/proto/ScmAdminProtocol.proto | 4 + .../src/main/resources/proto.lock | 14 +-- ...ocationProtocolServerSideTranslatorPB.java | 10 +- .../scm/server/SCMClientProtocolServer.java | 94 ++++++++++--------- .../ContainerBalancerStatusSubcommand.java | 52 ++++++---- .../TestContainerBalancerSubCommand.java | 7 +- .../smoketest/balancer/testBalancer.robot | 2 +- 7 files changed, 99 insertions(+), 84 deletions(-) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index 84c4295fb229..53b4334e0034 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -615,6 +615,10 @@ message ContainerBalancerStatusInfoRequestProto { } message ContainerBalancerStatusInfoResponseProto { + required bool isRunning = 1; + optional ContainerBalancerStatusInfo containerBalancerStatusInfo = 2; +} +message ContainerBalancerStatusInfo { required uint64 startedAt = 1; required ContainerBalancerConfigurationProto configuration = 2; repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; diff --git a/hadoop-hdds/interface-admin/src/main/resources/proto.lock b/hadoop-hdds/interface-admin/src/main/resources/proto.lock index 81368e3f60fd..798b77abf546 100644 --- a/hadoop-hdds/interface-admin/src/main/resources/proto.lock +++ b/hadoop-hdds/interface-admin/src/main/resources/proto.lock @@ -1948,21 +1948,15 @@ "fields": [ { "id": 1, - "name": "startedAt", - "type": "uint64", + "name": "isRunning", + "type": "bool", "required": true }, { "id": 2, - "name": "configuration", - "type": "ContainerBalancerConfigurationProto", + "name": "containerBalancerStatusInfo", + "type": "ContainerBalancerStatusInfo", "required": true - }, - { - "id": 3, - "name": "iterationsStatusInfo", - "type": "ContainerBalancerTaskIterationStatusInfo", - "is_repeated": true } ] } diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java index 347886389aeb..f2efae959872 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/protocol/StorageContainerLocationProtocolServerSideTranslatorPB.java @@ -1207,15 +1207,7 @@ public ContainerBalancerStatusResponseProto getContainerBalancerStatus( public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo( StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto request) throws IOException { - ContainerBalancerStatusInfoResponseProto containerBalancerStatusInfo = impl.getContainerBalancerStatusInfo(); - if (containerBalancerStatusInfo == null) { - return ContainerBalancerStatusInfoResponseProto.newBuilder().getDefaultInstanceForType(); - } - return ContainerBalancerStatusInfoResponseProto.newBuilder() - .setStartedAt(containerBalancerStatusInfo.getStartedAt()) - .setConfiguration(containerBalancerStatusInfo.getConfiguration()) - .addAllIterationsStatusInfo(containerBalancerStatusInfo.getIterationsStatusInfoList()) - .build(); + return impl.getContainerBalancerStatusInfo(); } public DecommissionNodesResponseProto decommissionNodes( 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 1f1cc783f103..eeed5cdd0e6d 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 @@ -1212,50 +1212,58 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); ContainerBalancerStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); if (balancerStatusInfo == null) { - return null; + return ContainerBalancerStatusInfoResponseProto + .newBuilder() + .setIsRunning(false) + .build(); + } else { + return ContainerBalancerStatusInfoResponseProto + .newBuilder() + .setIsRunning(true) + .setContainerBalancerStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo + .newBuilder() + .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) + .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) + .addAllIterationsStatusInfo( + balancerStatusInfo.getIterationsStatusInfo() + .stream() + .map( + info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(info.getIterationNumber()) + .setIterationResult(info.getIterationResult()) + .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setDataSizeMovedGB(info.getDataSizeMovedGB()) + .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()) + ) + ) + .build(); } - return ContainerBalancerStatusInfoResponseProto - .newBuilder() - .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) - .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) - .addAllIterationsStatusInfo( - balancerStatusInfo.getIterationsStatusInfo() - .stream() - .map( - info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(info.getIterationNumber()) - .setIterationResult(info.getIterationResult()) - .setSizeScheduledForMove(info.getSizeScheduledForMove()) - .setDataSizeMovedGB(info.getDataSizeMovedGB()) - .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()) - ) - .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 1fdac95d46db..4d0852fac297 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,6 +19,7 @@ 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.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.protocol.proto.StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo; import org.apache.hadoop.hdds.scm.client.ScmClient; @@ -45,33 +46,44 @@ public class ContainerBalancerStatusSubcommand extends ScmSubcommand { @CommandLine.Option(names = {"-v", "--verbose"}, - description = "More verbose output. Show balance iteration history.") + description = "Verbose output. Show current iteration info.") private boolean verbose; + @CommandLine.Option(names = {"-vh", "--verbose-with-history"}, + description = "More verbose output. Show current and history iteration info.") + private boolean verboseWithHistory; + @Override public void execute(ScmClient scmClient) throws IOException { - ContainerBalancerStatusInfoResponseProto balancerStatusInfo = scmClient.getContainerBalancerStatusInfo(); - if (balancerStatusInfo == null) { - System.out.println("ContainerBalancer is Not Running."); - } else { + ContainerBalancerStatusInfoResponseProto response = scmClient.getContainerBalancerStatusInfo(); + boolean isRunning = response.getIsRunning(); + ContainerBalancerStatusInfo balancerStatusInfo = response.getContainerBalancerStatusInfo(); + if (isRunning) { LocalDateTime dateTime = - LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); + LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); System.out.println("ContainerBalancer is Running."); - System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(), dateTime.toLocalTime()); - System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); - List iterationsStatusInfoList - = balancerStatusInfo.getIterationsStatusInfoList(); - if (verbose) { - System.out.println("Iteration history list:"); - System.out.println( - iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo) - .collect(Collectors.joining("\n")) - ); - } else { - System.out.println("Current iteration info:"); - System.out.println( - getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1))); + + if (verbose || verboseWithHistory) { + System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(), dateTime.toLocalTime()); + System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); + List iterationsStatusInfoList + = balancerStatusInfo.getIterationsStatusInfoList(); + if (verbose) { + System.out.println("Current iteration info:"); + System.out.println( + getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1)) + ); + } else if (verboseWithHistory) { + System.out.println("Iteration history list:"); + System.out.println( + iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo) + .collect(Collectors.joining("\n")) + ); + } } + + } else { + System.out.println("ContainerBalancer is Not Running."); } } 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 0d2e1e14828a..d3d6207d7d0b 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,6 +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.scm.cli.ContainerBalancerStartSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStatusSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStopSubcommand; @@ -199,11 +200,15 @@ public void testContainerBalancerStatusInfoSubcommandRunning() .build(); StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(true) + .setContainerBalancerStatusInfo(ContainerBalancerStatusInfo.newBuilder() .setStartedAt(OffsetDateTime.now().toEpochSecond()) .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) .addAllIterationsStatusInfo( - Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) + Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) ) + ) + .build(); //test status is running when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 49679587be94..979ac02c99f7 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -66,7 +66,7 @@ Run Container Balancer ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running - Sleep 60000ms + ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status From a9b9864fafe87c2a5271a7a14ed7feac7c43dc7d Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 17:54:15 +0300 Subject: [PATCH 10/27] HDDS-11120. Try to fix robot balancer test --- .../scm/container/balancer/ContainerBalancer.java | 8 ++++++-- .../balancer/ContainerBalancerStatusInfo.java | 12 +++++++----- .../hdds/scm/server/SCMClientProtocolServer.java | 8 +++++--- hadoop-hdds/tools/pom.xml | 6 ++++++ .../src/main/smoketest/balancer/testBalancer.robot | 2 +- 5 files changed, 25 insertions(+), 11 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index e592e641e4fb..f11713373e30 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -22,6 +22,7 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.fs.DUFactory; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerBalancerConfigurationProto; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ha.SCMContext; @@ -182,13 +183,15 @@ public ContainerBalancerTask.Status getBalancerStatus() { * * @return balancer status info if balancer started */ - public ContainerBalancerStatusInfo getBalancerStatusInfo() { + public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { if (this.startedAt == null || task == null) { return null; } + HddsProtos.ContainerBalancerConfigurationProto configProto = + readConfiguration(HddsProtos.ContainerBalancerConfigurationProto.class); return new ContainerBalancerStatusInfo( this.startedAt, - task.getConfig(), + configProto, task.getCurrentIterationsStatistic() ); } @@ -269,6 +272,7 @@ public void start() throws IllegalContainerBalancerStateException, public void startBalancer(ContainerBalancerConfiguration configuration) throws IllegalContainerBalancerStateException, InvalidContainerBalancerConfigurationException, IOException { + startedAt = OffsetDateTime.now(); lock.lock(); try { // validates state, config, and then saves config 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 b5edf894e624..cbe8385e53a7 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 @@ -18,6 +18,8 @@ package org.apache.hadoop.hdds.scm.container.balancer; +import org.apache.hadoop.hdds.protocol.proto.HddsProtos; + import java.time.OffsetDateTime; import java.util.List; @@ -26,13 +28,13 @@ */ public class ContainerBalancerStatusInfo { private final OffsetDateTime startedAt; - private final ContainerBalancerConfiguration configuration; + private final HddsProtos.ContainerBalancerConfigurationProto configuration; private final List iterationsStatusInfo; public ContainerBalancerStatusInfo( - OffsetDateTime startedAt, - ContainerBalancerConfiguration configuration, - List iterationsStatusInfo) { + OffsetDateTime startedAt, + HddsProtos.ContainerBalancerConfigurationProto configuration, + List iterationsStatusInfo) { this.startedAt = startedAt; this.configuration = configuration; this.iterationsStatusInfo = iterationsStatusInfo; @@ -42,7 +44,7 @@ public OffsetDateTime getStartedAt() { return startedAt; } - public ContainerBalancerConfiguration getConfiguration() { + public HddsProtos.ContainerBalancerConfigurationProto getConfiguration() { return configuration; } 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 eeed5cdd0e6d..95efd70a3b06 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 @@ -56,6 +56,7 @@ import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerStatusInfo; +import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerTask; import org.apache.hadoop.hdds.scm.container.balancer.IllegalContainerBalancerStateException; import org.apache.hadoop.hdds.scm.container.balancer.InvalidContainerBalancerConfigurationException; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; @@ -1207,7 +1208,7 @@ public boolean getContainerBalancerStatus() { } @Override - public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() { + public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() throws IOException { AUDIT.logReadSuccess(buildAuditMessageForSuccess( SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null)); ContainerBalancerStatusInfo balancerStatusInfo = scm.getContainerBalancer().getBalancerStatusInfo(); @@ -1217,20 +1218,21 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() .setIsRunning(false) .build(); } else { + return ContainerBalancerStatusInfoResponseProto .newBuilder() .setIsRunning(true) .setContainerBalancerStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo .newBuilder() .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond()) - .setConfiguration(balancerStatusInfo.getConfiguration().toProtobufBuilder()) + .setConfiguration(balancerStatusInfo.getConfiguration()) .addAllIterationsStatusInfo( balancerStatusInfo.getIterationsStatusInfo() .stream() .map( info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(info.getIterationNumber()) - .setIterationResult(info.getIterationResult()) + .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) .setSizeScheduledForMove(info.getSizeScheduledForMove()) .setDataSizeMovedGB(info.getDataSizeMovedGB()) .setContainerMovesScheduled(info.getContainerMovesScheduled()) diff --git a/hadoop-hdds/tools/pom.xml b/hadoop-hdds/tools/pom.xml index 5122f1d4a450..d9df0813ac13 100644 --- a/hadoop-hdds/tools/pom.xml +++ b/hadoop-hdds/tools/pom.xml @@ -109,5 +109,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-test-utils test + + + org.apache.ozone + hdds-interface-admin + ${hdds.version} + diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 979ac02c99f7..49679587be94 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -66,7 +66,7 @@ Run Container Balancer ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running - + Sleep 60000ms ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status From 1dda8b3fdcf43b58a25cbd7bcbe4e7aa463ba7ba Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 19:21:54 +0300 Subject: [PATCH 11/27] HDDS-11120. Try to fix robot balancer test --- .../hadoop/hdds/scm/container/balancer/ContainerBalancer.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/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index f11713373e30..c435602e5a7c 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -184,7 +184,7 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - if (this.startedAt == null || task == null) { + if (task == null && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING) { return null; } HddsProtos.ContainerBalancerConfigurationProto configProto = From 0bbe031c400f95e93bf8a02c856e7638378c6534 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 19:34:30 +0300 Subject: [PATCH 12/27] HDDS-11120. Fix checkstyle --- .../apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java | 1 - 1 file changed, 1 deletion(-) 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 95efd70a3b06..1489233f0c3f 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 @@ -56,7 +56,6 @@ import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancer; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerConfiguration; import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerStatusInfo; -import org.apache.hadoop.hdds.scm.container.balancer.ContainerBalancerTask; import org.apache.hadoop.hdds.scm.container.balancer.IllegalContainerBalancerStateException; import org.apache.hadoop.hdds.scm.container.balancer.InvalidContainerBalancerConfigurationException; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; From 20cf4962e4fc0191a1ec72888a455be0a1514774 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 19:55:40 +0300 Subject: [PATCH 13/27] HDDS-11120. Try to fix robot balancer test --- .../hadoop/hdds/scm/container/balancer/ContainerBalancer.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/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index c435602e5a7c..465059ea2b74 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -184,7 +184,7 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - if (task == null && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING) { + if (task == null && task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING) { return null; } HddsProtos.ContainerBalancerConfigurationProto configProto = From 947ae3b69d6fcde51cb4c90d7bd1d94cf44eb804 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 10 Jul 2024 20:27:04 +0300 Subject: [PATCH 14/27] HDDS-11120. Try to fix robot balancer test --- .../hadoop/hdds/scm/container/balancer/ContainerBalancer.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/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 465059ea2b74..ec0eeb63f3b1 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -184,7 +184,7 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - if (task == null && task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING) { + if (task == null || task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING) { return null; } HddsProtos.ContainerBalancerConfigurationProto configProto = From e7cd45b035cdb66c97733d4ed2b20d44d6d1d978 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 11 Jul 2024 00:21:32 +0300 Subject: [PATCH 15/27] HDDS-11120. TFix test + refactoring + modify balancer robot test --- .../container/balancer/ContainerBalancer.java | 19 +++-- .../balancer/ContainerBalancerTask.java | 80 ++++++++++--------- .../TestContainerBalancerSubCommand.java | 16 +++- .../smoketest/balancer/testBalancer.robot | 16 ++++ 4 files changed, 83 insertions(+), 48 deletions(-) diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index ec0eeb63f3b1..98eeb2bf826f 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -184,16 +184,19 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - if (task == null || task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING) { + boolean isTaskRunning = task != null && task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING; + if (isTaskRunning) { + HddsProtos.ContainerBalancerConfigurationProto configProto = + readConfiguration(HddsProtos.ContainerBalancerConfigurationProto.class); + return new ContainerBalancerStatusInfo( + this.startedAt, + configProto, + task.getCurrentIterationsStatistic() + ); + } else { return null; } - HddsProtos.ContainerBalancerConfigurationProto configProto = - readConfiguration(HddsProtos.ContainerBalancerConfigurationProto.class); - return new ContainerBalancerStatusInfo( - this.startedAt, - configProto, - task.getCurrentIterationsStatistic() - ); + } /** * Checks if ContainerBalancer is in valid state to call stop. 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 744d8c166a4c..733eb3359525 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 @@ -335,42 +335,50 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) } public List getCurrentIterationsStatistic() { - int lastIterationNumber = iterationsStatistic.stream() - .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) - .max() - .orElse(0); - ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - lastIterationNumber + 1, - null, - getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, - sizeActuallyMovedInLatestIteration / OzoneConsts.GB, - 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(), - entry -> entry.getValue() / OzoneConsts.GB - ) - ), - findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB - ) - ) - ); - iterationsStatistic.add(currentIterationStatistic); + + boolean isActiveIterationPresent = iterationsStatistic.stream() + .anyMatch(iterationStatusInfo -> iterationStatusInfo.getIterationResult() == null); + + if (!isActiveIterationPresent) { + + int lastIterationNumber = iterationsStatistic.stream() + .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) + .max() + .orElse(0); + + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + lastIterationNumber + 1, + null, + getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, + sizeActuallyMovedInLatestIteration / OzoneConsts.GB, + 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(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ), + findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ) + ); + iterationsStatistic.add(currentIterationStatistic); + } return iterationsStatistic; } 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 d3d6207d7d0b..c5048d5d7ddb 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 @@ -19,6 +19,7 @@ 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.ContainerBalancerStatusInfoResponseProto; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStartSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStatusSubcommand; import org.apache.hadoop.hdds.scm.cli.ContainerBalancerStopSubcommand; @@ -198,8 +199,8 @@ public void testContainerBalancerStatusInfoSubcommandRunning() .build() ) .build(); - StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = - StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoResponseProto.newBuilder() + ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = + ContainerBalancerStatusInfoResponseProto.newBuilder() .setIsRunning(true) .setContainerBalancerStatusInfo(ContainerBalancerStatusInfo.newBuilder() .setStartedAt(OffsetDateTime.now().toEpochSecond()) @@ -226,7 +227,11 @@ public void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() ScmClient scmClient = mock(ScmClient.class); //test status is running - when(scmClient.getContainerBalancerStatusInfo()).thenReturn(null); + when(scmClient.getContainerBalancerStatusInfo()).thenReturn( + ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(false) + .build()) + ; statusCmd.execute(scmClient); Pattern p = Pattern.compile( @@ -240,7 +245,10 @@ public void testContainerBalancerStatusSubcommandNotRunning() throws IOException { ScmClient scmClient = mock(ScmClient.class); - when(scmClient.getContainerBalancerStatus()).thenAnswer(invocation -> false); + when(scmClient.getContainerBalancerStatusInfo()).thenReturn( + ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(false) + .build()); statusCmd.execute(scmClient); diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 49679587be94..74bda99e240d 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -63,6 +63,22 @@ Datanode Recommission is Finished Run Container Balancer ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 Should Contain ${result} Container Balancer started successfully. + ${result} = Execute ozone admin containerbalancer status -v + Should Contain ${result} ContainerBalancer is Running. + Should Contain ${result} Started at: + Should Contain ${result} Container Balancer Configuration values: + Should Contain ${result} Current iteration info: + Should Contain ${result} Iteration number 1 + Should Contain ${result} Iteration result IN_PROGRESS + Should Contain ${result} Scheduled to move containers 3 + ${result} = Execute ozone admin containerbalancer status -vh + Should Contain ${result} ContainerBalancer is Running. + Should Contain ${result} Started at: + Should Contain ${result} Container Balancer Configuration values: + Should Contain ${result} Iteration history list: + Should Contain ${result} Iteration number 1 + Should Contain ${result} Iteration result IN_PROGRESS + Should Contain ${result} Scheduled to move containers 3 ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running From 7c035be16b4eb995ba02fe442aa1b5cff9aa4aa3 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 11 Jul 2024 00:35:26 +0300 Subject: [PATCH 16/27] HDDS-11120. Fix formatting. --- .../TestContainerBalancerSubCommand.java | 271 +++++++++--------- 1 file changed, 135 insertions(+), 136 deletions(-) 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 c5048d5d7ddb..ce5748d1b0c2 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 @@ -47,16 +47,16 @@ * Unit tests to validate the ContainerBalancerSubCommand class includes the * correct output when executed against a mock client. */ -public class TestContainerBalancerSubCommand { +class TestContainerBalancerSubCommand { - private ContainerBalancerStopSubcommand stopCmd; - private ContainerBalancerStartSubcommand startCmd; - private ContainerBalancerStatusSubcommand statusCmd; + private static final String DEFAULT_ENCODING = StandardCharsets.UTF_8.name(); private final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); private final ByteArrayOutputStream errContent = new ByteArrayOutputStream(); private final PrintStream originalOut = System.out; private final PrintStream originalErr = System.err; - private static final String DEFAULT_ENCODING = StandardCharsets.UTF_8.name(); + private ContainerBalancerStopSubcommand stopCmd; + private ContainerBalancerStartSubcommand startCmd; + private ContainerBalancerStatusSubcommand statusCmd; @BeforeEach public void setup() throws UnsupportedEncodingException { @@ -75,7 +75,7 @@ public void tearDown() { @Test public void testContainerBalancerStatusInfoSubcommandRunning() - throws IOException { + throws IOException { ScmClient scmClient = mock(ScmClient.class); ContainerBalancerConfiguration config = new ContainerBalancerConfiguration(); @@ -95,160 +95,159 @@ public void testContainerBalancerStatusInfoSubcommandRunning() config.setTriggerDuEnable(false); StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration0StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(0) - .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMove(48) - .setDataSizeMovedGB(48) - .setContainerMovesScheduled(11) - .setContainerMovesCompleted(11) - .setContainerMovesFailed(0) - .setContainerMovesTimeout(0) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27) - .build() - ) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) - .build() - ) - .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(0) + .setIterationResult("ITERATION_COMPLETED") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration1StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(1) - .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMove(48) - .setDataSizeMovedGB(48) - .setContainerMovesScheduled(11) - .setContainerMovesCompleted(11) - .setContainerMovesFailed(0) - .setContainerMovesTimeout(0) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) - .build() - ) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) - .build() - ) - .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(1) + .setIterationResult("ITERATION_COMPLETED") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27L) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo iteration2StatusInfo = - StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() - .setIterationNumber(1) - .setIterationResult("") - .setSizeScheduledForMove(48) - .setDataSizeMovedGB(48) - .setContainerMovesScheduled(11) - .setContainerMovesCompleted(11) - .setContainerMovesFailed(0) - .setContainerMovesTimeout(0) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) - .build() - ) - .addSizeEnteringNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) - .build() - ) - .addSizeLeavingNodes( - StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() - .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) - .build() - ) - .build(); + StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() + .setIterationNumber(1) + .setIterationResult("") + .setSizeScheduledForMove(48) + .setDataSizeMovedGB(48) + .setContainerMovesScheduled(11) + .setContainerMovesCompleted(11) + .setContainerMovesFailed(0) + .setContainerMovesTimeout(0) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") + .setDataVolume(27L) + .build() + ) + .addSizeEnteringNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") + .setDataVolume(23L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") + .setDataVolume(24L) + .build() + ) + .addSizeLeavingNodes( + StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() + .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") + .setDataVolume(26L) + .build() + ) + .build(); ContainerBalancerStatusInfoResponseProto statusInfoResponseProto = - ContainerBalancerStatusInfoResponseProto.newBuilder() - .setIsRunning(true) - .setContainerBalancerStatusInfo(ContainerBalancerStatusInfo.newBuilder() - .setStartedAt(OffsetDateTime.now().toEpochSecond()) - .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) - .addAllIterationsStatusInfo( - Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) - ) + ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(true) + .setContainerBalancerStatusInfo(ContainerBalancerStatusInfo.newBuilder() + .setStartedAt(OffsetDateTime.now().toEpochSecond()) + .setConfiguration(config.toProtobufBuilder().setShouldRun(true)) + .addAllIterationsStatusInfo( + Arrays.asList(iteration0StatusInfo, iteration1StatusInfo, iteration2StatusInfo) ) + ) - .build(); + .build(); //test status is running when(scmClient.getContainerBalancerStatusInfo()).thenReturn(statusInfoResponseProto); statusCmd.execute(scmClient); Pattern p = Pattern.compile( - "^ContainerBalancer\\sis\\sRunning."); + "^ContainerBalancer\\sis\\sRunning."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); } @Test public void testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer() - throws IOException { + throws IOException { ScmClient scmClient = mock(ScmClient.class); //test status is running when(scmClient.getContainerBalancerStatusInfo()).thenReturn( - ContainerBalancerStatusInfoResponseProto.newBuilder() - .setIsRunning(false) - .build()) - ; + ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(false) + .build()); statusCmd.execute(scmClient); Pattern p = Pattern.compile( - "^ContainerBalancer\\sis\\sNot\\sRunning."); + "^ContainerBalancer\\sis\\sNot\\sRunning."); Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING)); assertTrue(m.find()); } @Test - public void testContainerBalancerStatusSubcommandNotRunning() - throws IOException { + void testContainerBalancerStatusSubcommandNotRunning() + throws IOException { ScmClient scmClient = mock(ScmClient.class); when(scmClient.getContainerBalancerStatusInfo()).thenReturn( - ContainerBalancerStatusInfoResponseProto.newBuilder() - .setIsRunning(false) - .build()); + ContainerBalancerStatusInfoResponseProto.newBuilder() + .setIsRunning(false) + .build()); statusCmd.execute(scmClient); @@ -259,13 +258,13 @@ public void testContainerBalancerStatusSubcommandNotRunning() } @Test - public void testContainerBalancerStopSubcommand() throws IOException { + public void testContainerBalancerStopSubcommand() throws IOException { ScmClient scmClient = mock(ScmClient.class); 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()); @@ -273,10 +272,10 @@ public void testContainerBalancerStopSubcommand() throws IOException { @Test public void testContainerBalancerStartSubcommandWhenBalancerIsNotRunning() - throws IOException { + throws IOException { ScmClient scmClient = mock(ScmClient.class); when(scmClient.startContainerBalancer( - null, null, null, null, null, null, null, null, null, null, null, null)) + null, null, null, null, null, null, null, null, null, null, null, null)) .thenReturn( StorageContainerLocationProtocolProtos .StartContainerBalancerResponseProto.newBuilder() @@ -292,10 +291,10 @@ public void testContainerBalancerStartSubcommandWhenBalancerIsNotRunning() @Test public void testContainerBalancerStartSubcommandWhenBalancerIsRunning() - throws IOException { + throws IOException { ScmClient scmClient = mock(ScmClient.class); when(scmClient.startContainerBalancer( - null, null, null, null, null, null, null, null, null, null, null, null)) + null, null, null, null, null, null, null, null, null, null, null, null)) .thenReturn(StorageContainerLocationProtocolProtos .StartContainerBalancerResponseProto.newBuilder() .setStart(false) From b6ea9a5e49b848b79f81f648ba74f973735863da Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 11 Jul 2024 10:10:22 +0300 Subject: [PATCH 17/27] HDDS-11120. Fix balancing test. --- .../hadoop/hdds/scm/container/balancer/ContainerBalancer.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/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 98eeb2bf826f..2c6836daa815 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -184,7 +184,7 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - boolean isTaskRunning = task != null && task.getBalancerStatus() != ContainerBalancerTask.Status.RUNNING; + boolean isTaskRunning = task != null && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING; if (isTaskRunning) { HddsProtos.ContainerBalancerConfigurationProto configProto = readConfiguration(HddsProtos.ContainerBalancerConfigurationProto.class); From 68b12b11c5576acb0428eeda9b18dfafaee6ef54 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sun, 14 Jul 2024 11:52:49 +0300 Subject: [PATCH 18/27] HDDS-11120. Fix review notices. --- .../src/main/proto/ScmAdminProtocol.proto | 32 +++++++------- .../src/main/resources/proto.lock | 44 ------------------- hadoop-hdds/tools/pom.xml | 6 --- .../ContainerBalancerStatusSubcommand.java | 8 ++-- .../smoketest/balancer/testBalancer.robot | 33 +++++++------- 5 files changed, 37 insertions(+), 86 deletions(-) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index 53b4334e0034..c88dfb1b1f2b 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -615,31 +615,31 @@ message ContainerBalancerStatusInfoRequestProto { } message ContainerBalancerStatusInfoResponseProto { - required bool isRunning = 1; + optional bool isRunning = 1; optional ContainerBalancerStatusInfo containerBalancerStatusInfo = 2; } message ContainerBalancerStatusInfo { - required uint64 startedAt = 1; - required ContainerBalancerConfigurationProto configuration = 2; - repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; + optional uint64 startedAt = 1; + optional ContainerBalancerConfigurationProto configuration = 2; + optional ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; } message ContainerBalancerTaskIterationStatusInfo { - required int32 iterationNumber = 1; - optional string iterationResult = 2; - required int64 sizeScheduledForMove = 3; - required int64 dataSizeMovedGB = 4; - required int64 containerMovesScheduled = 5; - required int64 containerMovesCompleted = 6; - required int64 containerMovesFailed = 7; - required int64 containerMovesTimeout = 8; - repeated NodeTransferInfo sizeEnteringNodes = 9; - repeated NodeTransferInfo sizeLeavingNodes = 10; + optional int32 iterationNumber = 1; + optional string iterationResult = 2; + optional int64 sizeScheduledForMove = 3; + optional int64 dataSizeMovedGB = 4; + optional int64 containerMovesScheduled = 5; + optional int64 containerMovesCompleted = 6; + optional int64 containerMovesFailed = 7; + optional int64 containerMovesTimeout = 8; + repeated NodeTransferInfo sizeEnteringNodes = 9; + repeated NodeTransferInfo sizeLeavingNodes = 10; } message NodeTransferInfo { - required string uuid = 1; - required int64 dataVolume = 2; + optional string uuid = 1; + optional int64 dataVolume = 2; } message DecommissionScmRequestProto { diff --git a/hadoop-hdds/interface-admin/src/main/resources/proto.lock b/hadoop-hdds/interface-admin/src/main/resources/proto.lock index 798b77abf546..8e898afc5c11 100644 --- a/hadoop-hdds/interface-admin/src/main/resources/proto.lock +++ b/hadoop-hdds/interface-admin/src/main/resources/proto.lock @@ -187,10 +187,6 @@ { "name": "DecommissionScm", "integer": 40 - }, - { - "name": "GetContainerBalancerStatusInfo", - "integer": 44 } ] }, @@ -501,12 +497,6 @@ "name": "decommissionScmRequest", "type": "DecommissionScmRequestProto", "optional": true - }, - { - "id": 48, - "name": "containerBalancerStatusInfoRequest", - "type": "ContainerBalancerStatusInfoRequestProto", - "optional": true } ] }, @@ -782,12 +772,6 @@ "name": "decommissionScmResponse", "type": "DecommissionScmResponseProto", "optional": true - }, - { - "id": 48, - "name": "containerBalancerStatusInfoResponse", - "type": "ContainerBalancerStatusInfoResponseProto", - "optional": true } ] }, @@ -1931,34 +1915,6 @@ "optional": true } ] - }, - { - "name": "ContainerBalancerStatusInfoRequestProto", - "fields": [ - { - "id": 1, - "name": "traceID", - "type": "string", - "optional": true - } - ] - }, - { - "name": "ContainerBalancerStatusInfoResponseProto", - "fields": [ - { - "id": 1, - "name": "isRunning", - "type": "bool", - "required": true - }, - { - "id": 2, - "name": "containerBalancerStatusInfo", - "type": "ContainerBalancerStatusInfo", - "required": true - } - ] } ], "services": [ diff --git a/hadoop-hdds/tools/pom.xml b/hadoop-hdds/tools/pom.xml index d9df0813ac13..5122f1d4a450 100644 --- a/hadoop-hdds/tools/pom.xml +++ b/hadoop-hdds/tools/pom.xml @@ -109,11 +109,5 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hdds-test-utils test - - - org.apache.ozone - hdds-interface-admin - ${hdds.version} - 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 4d0852fac297..5a573de0dd17 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 @@ -49,8 +49,8 @@ public class ContainerBalancerStatusSubcommand extends ScmSubcommand { description = "Verbose output. Show current iteration info.") private boolean verbose; - @CommandLine.Option(names = {"-vh", "--verbose-with-history"}, - description = "More verbose output. Show current and history iteration info.") + @CommandLine.Option(names = {"-h", "--history"}, + description = "Verbose output with history. Show current and history iteration info. Works only with -v.") private boolean verboseWithHistory; @Override @@ -63,12 +63,12 @@ public void execute(ScmClient scmClient) throws IOException { LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()), ZoneId.systemDefault()); System.out.println("ContainerBalancer is Running."); - if (verbose || verboseWithHistory) { + if (verbose) { System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(), dateTime.toLocalTime()); System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); - if (verbose) { + if (!verboseWithHistory) { System.out.println("Current iteration info:"); System.out.println( getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1)) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 74bda99e240d..bdb9e88b0824 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -63,22 +63,23 @@ Datanode Recommission is Finished Run Container Balancer ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 Should Contain ${result} Container Balancer started successfully. - ${result} = Execute ozone admin containerbalancer status -v - Should Contain ${result} ContainerBalancer is Running. - Should Contain ${result} Started at: - Should Contain ${result} Container Balancer Configuration values: - Should Contain ${result} Current iteration info: - Should Contain ${result} Iteration number 1 - Should Contain ${result} Iteration result IN_PROGRESS - Should Contain ${result} Scheduled to move containers 3 - ${result} = Execute ozone admin containerbalancer status -vh - Should Contain ${result} ContainerBalancer is Running. - Should Contain ${result} Started at: - Should Contain ${result} Container Balancer Configuration values: - Should Contain ${result} Iteration history list: - Should Contain ${result} Iteration number 1 - Should Contain ${result} Iteration result IN_PROGRESS - Should Contain ${result} Scheduled to move containers 3 + ${verboseStatusResult} = Execute ozone admin containerbalancer status -v + Should Contain ${verboseStatusResult} ContainerBalancer is Running. + Should Contain ${verboseStatusResult} Started at: + Should Contain ${verboseStatusResult} Container Balancer Configuration values: + Should Contain ${verboseStatusResult} Current iteration info: + Should Contain ${verboseStatusResult} Iteration number 1 + Should Contain ${verboseStatusResult} Iteration result IN_PROGRESS + Should Contain ${verboseStatusResult} Scheduled to move containers 3 + ${statusHistoryResult} = + Execute ozone admin containerbalancer status -v -h + Should Contain ${statusHistoryResult} ContainerBalancer is Running. + Should Contain ${statusHistoryResult} Started at: + Should Contain ${statusHistoryResult} Container Balancer Configuration values: + Should Contain ${statusHistoryResult} Iteration history list: + Should Contain ${statusHistoryResult} Iteration number 1 + Should Contain ${statusHistoryResult} Iteration result IN_PROGRESS + Should Contain ${statusHistoryResult} Scheduled to move containers 3 ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running From 1538ee937225e3668da63fa008812f013e926e6d Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sun, 14 Jul 2024 21:22:08 +0300 Subject: [PATCH 19/27] HDDS-11120. Fix review notices. --- .../interface-admin/src/main/proto/ScmAdminProtocol.proto | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index c88dfb1b1f2b..5bb0b572eafb 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -621,7 +621,7 @@ message ContainerBalancerStatusInfoResponseProto { message ContainerBalancerStatusInfo { optional uint64 startedAt = 1; optional ContainerBalancerConfigurationProto configuration = 2; - optional ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; + repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3; } message ContainerBalancerTaskIterationStatusInfo { From 04bb57ec8d62df557046fe26ea6bd2d7958f9884 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Sun, 14 Jul 2024 23:35:54 +0300 Subject: [PATCH 20/27] HDDS-11120. Fix testBalancer. --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index bdb9e88b0824..ae8b9e2bd1de 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -71,8 +71,7 @@ Run Container Balancer Should Contain ${verboseStatusResult} Iteration number 1 Should Contain ${verboseStatusResult} Iteration result IN_PROGRESS Should Contain ${verboseStatusResult} Scheduled to move containers 3 - ${statusHistoryResult} = - Execute ozone admin containerbalancer status -v -h + ${statusHistoryResult} = Execute ozone admin containerbalancer status -v -h Should Contain ${statusHistoryResult} ContainerBalancer is Running. Should Contain ${statusHistoryResult} Started at: Should Contain ${statusHistoryResult} Container Balancer Configuration values: From 4a39571a343d453ea8f66c8bee5d0f634468333d Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 15 Jul 2024 09:26:23 +0300 Subject: [PATCH 21/27] HDDS-11120. Change -h command description. --- .../hadoop/hdds/scm/cli/ContainerBalancerStatusSubcommand.java | 3 ++- 1 file changed, 2 insertions(+), 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 5a573de0dd17..e73af1df78e8 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 @@ -50,7 +50,8 @@ public class ContainerBalancerStatusSubcommand extends ScmSubcommand { private boolean verbose; @CommandLine.Option(names = {"-h", "--history"}, - description = "Verbose output with history. Show current and history iteration info. Works only with -v.") + description = "Verbose output with history. Show current iteration info and history of iterations. " + + "Works only with -v.") private boolean verboseWithHistory; @Override From 2207557893dda245550f90b07c97fff91dbed3e1 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 17 Jul 2024 10:32:09 +0300 Subject: [PATCH 22/27] HDDS-11120. Fix review notices. --- .../ContainerBalancerStatusSubcommand.java | 15 +- .../smoketest/balancer/testBalancer.robot | 16 -- .../balancer/testBalancerStatus.robot | 149 ++++++++++++++++++ 3 files changed, 157 insertions(+), 23 deletions(-) create mode 100644 hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot 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 e73af1df78e8..d3d2ed2d54b5 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 @@ -49,7 +49,7 @@ public class ContainerBalancerStatusSubcommand extends ScmSubcommand { description = "Verbose output. Show current iteration info.") private boolean verbose; - @CommandLine.Option(names = {"-h", "--history"}, + @CommandLine.Option(names = {"-H", "--history"}, description = "Verbose output with history. Show current iteration info and history of iterations. " + "Works only with -v.") private boolean verboseWithHistory; @@ -69,12 +69,13 @@ public void execute(ScmClient scmClient) throws IOException { System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration())); List iterationsStatusInfoList = balancerStatusInfo.getIterationsStatusInfoList(); - if (!verboseWithHistory) { - System.out.println("Current iteration info:"); - System.out.println( - getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1)) - ); - } else if (verboseWithHistory) { + + System.out.println("Current iteration info:"); + System.out.println( + getPrettyIterationStatusInfo(iterationsStatusInfoList.get(iterationsStatusInfoList.size() - 1)) + ); + + if (verboseWithHistory) { System.out.println("Iteration history list:"); System.out.println( iterationsStatusInfoList.stream().map(this::getPrettyIterationStatusInfo) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index ae8b9e2bd1de..49679587be94 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -63,22 +63,6 @@ Datanode Recommission is Finished Run Container Balancer ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 Should Contain ${result} Container Balancer started successfully. - ${verboseStatusResult} = Execute ozone admin containerbalancer status -v - Should Contain ${verboseStatusResult} ContainerBalancer is Running. - Should Contain ${verboseStatusResult} Started at: - Should Contain ${verboseStatusResult} Container Balancer Configuration values: - Should Contain ${verboseStatusResult} Current iteration info: - Should Contain ${verboseStatusResult} Iteration number 1 - Should Contain ${verboseStatusResult} Iteration result IN_PROGRESS - Should Contain ${verboseStatusResult} Scheduled to move containers 3 - ${statusHistoryResult} = Execute ozone admin containerbalancer status -v -h - Should Contain ${statusHistoryResult} ContainerBalancer is Running. - Should Contain ${statusHistoryResult} Started at: - Should Contain ${statusHistoryResult} Container Balancer Configuration values: - Should Contain ${statusHistoryResult} Iteration history list: - Should Contain ${statusHistoryResult} Iteration number 1 - Should Contain ${statusHistoryResult} Iteration result IN_PROGRESS - Should Contain ${statusHistoryResult} Scheduled to move containers 3 ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot new file mode 100644 index 000000000000..450b69ad58c2 --- /dev/null +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot @@ -0,0 +1,149 @@ +# 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. + +*** Settings *** +Documentation Smoketest ozone cluster startup +Library OperatingSystem +Library Collections +Resource ../commonlib.robot +Resource ../ozone-lib/shell.robot + +Test Timeout 20 minutes + +*** Variables *** +${SECURITY_ENABLED} false +${HOST} datanode1 +${VOLUME} volume1 +${BUCKET} bucket1 +${SIZE} 104857600 + + +** Keywords *** +Prepare For Tests + Execute dd if=/dev/urandom of=/tmp/100mb bs=1048576 count=100 + Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab + Execute ozone sh volume create /${VOLUME} + Execute ozone sh bucket create /${VOLUME}/${BUCKET} + + +Datanode In Maintenance Mode + ${result} = Execute ozone admin datanode maintenance ${HOST} + Should Contain ${result} Entering maintenance mode on datanode + ${result} = Execute ozone admin datanode list | grep "Operational State:*" + Wait Until Keyword Succeeds 30sec 5sec Should contain ${result} ENTERING_MAINTENANCE + Wait Until Keyword Succeeds 3min 10sec Related pipelines are closed + Sleep 60000ms + +Create Multiple Keys + [arguments] ${NUM_KEYS} + ${file} = Set Variable /tmp/100mb + FOR ${INDEX} IN RANGE ${NUM_KEYS} + ${fileName} = Set Variable file-${INDEX}.txt + ${key} = Set Variable /${VOLUME}/${BUCKET}/${fileName} + LOG ${fileName} + Create Key ${key} ${file} + Key Should Match Local File ${key} ${file} + END + +Close All Containers + FOR ${INDEX} IN RANGE 15 + ${container} = Execute ozone admin container list --state OPEN | jq -r 'select(.replicationConfig.replicationFactor == "THREE") | .containerID' | head -1 + EXIT FOR LOOP IF "${container}" == "${EMPTY}" + ${message} = Execute And Ignore Error ozone admin container close "${container}" + Run Keyword If '${message}' != '${EMPTY}' Should Contain ${message} is in closing state + ${output} = Execute ozone admin container info "${container}" + Should contain ${output} CLOS + END + Wait until keyword succeeds 3min 10sec All container is closed + +All container is closed + ${output} = Execute ozone admin container list --state OPEN + Should Be Empty ${output} + +Related pipelines are closed + ${result} = Execute ozone admin datanode list | awk -v RS= '{$1=$1}1'|grep MAINT | sed -e 's/^.*pipelines: \\(.*\\)$/\\1/' + Should Contain Any ${result} CLOSED No related pipelines or the node is not in Healthy state. + +Datanode Recommission + ${result} = Execute ozone admin datanode recommission ${HOST} + Should Contain ${result} Started recommissioning datanode + Wait Until Keyword Succeeds 1min 10sec Datanode Recommission is Finished + Sleep 300000ms + +Datanode Recommission is Finished + ${result} = Execute ozone admin datanode list | grep "Operational State:*" + Should Not Contain ${result} ENTERING_MAINTENANCE +Run Container Balancer + ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 + Should Contain ${result} Container Balancer started successfully. + +Verify Verbose Balancer Status + [arguments] ${output} + + 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} + +Run Balancer Status + ${result} = Execute ozone admin containerbalancer status + Should Contain ${result} ContainerBalancer is Running. + +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: + +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: +Get Datanode Ozone Used Bytes Info + [arguments] ${uuid} + ${output} = Execute export DATANODES=$(ozone admin datanode list --json) && for datanode in $(echo "$\{DATANODES\}" | jq -r '.[].datanodeDetails.uuid'); do ozone admin datanode usageinfo --uuid=$\{datanode\} --json | jq '{(.[0].datanodeDetails.uuid) : .[0].ozoneUsed}'; done | jq -s add + ${result} = Execute echo '${output}' | jq '. | to_entries | .[] | select(.key == "${uuid}") | .value' + [return] ${result} + +** Test Cases *** +Verify Container Balancer for RATIS containers + Prepare For Tests + + Datanode In Maintenance Mode + + Create Multiple Keys 3 + + Close All Containers + + Datanode Recommission + + Run Container Balancer + + Run Balancer Status + + Run Balancer Verbose Status + + Run Balancer Verbose History Status + + + + From daa7da448f6ce3416bcac1b999966e69461c5ebe Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 17 Jul 2024 12:54:07 +0300 Subject: [PATCH 23/27] HDDS-11120. Change test name. --- .../dist/src/main/smoketest/balancer/testBalancerStatus.robot | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot index 450b69ad58c2..d529ce7ba663 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot @@ -125,7 +125,7 @@ Get Datanode Ozone Used Bytes Info [return] ${result} ** Test Cases *** -Verify Container Balancer for RATIS containers +Verify Container Balancer Status Output Prepare For Tests Datanode In Maintenance Mode From 9b51a85aefcf6bff1b6016166d75c3025d2c0f6c Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Wed, 17 Jul 2024 18:58:06 +0300 Subject: [PATCH 24/27] HDDS-11120. Remove testBalancerStatus test. --- .../smoketest/balancer/testBalancer.robot | 36 +++++ .../balancer/testBalancerStatus.robot | 149 ------------------ 2 files changed, 36 insertions(+), 149 deletions(-) delete mode 100644 hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 49679587be94..7977cead7cbd 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -68,6 +68,36 @@ Run Container Balancer Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running Sleep 60000ms +Verify Verbose Balancer Status + [arguments] ${output} + + 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} + +Run Balancer Status + ${result} = Execute ozone admin containerbalancer status + Should Contain ${result} ContainerBalancer is Running. + +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: + +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: + ContainerBalancer is Not Running ${result} = Execute ozone admin containerbalancer status Should contain ${result} ContainerBalancer is Not Running. @@ -133,6 +163,12 @@ Verify Container Balancer for RATIS containers Run Container Balancer + Run Balancer Status + + Run Balancer Verbose Status + + Run Balancer Verbose History Status + ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} = Get Datanode Ozone Used Bytes Info ${uuid} Should Not Be Equal As Integers ${datanodeOzoneUsedBytesInfo} ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} < ${SIZE} * 3.5 diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot deleted file mode 100644 index d529ce7ba663..000000000000 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancerStatus.robot +++ /dev/null @@ -1,149 +0,0 @@ -# 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. - -*** Settings *** -Documentation Smoketest ozone cluster startup -Library OperatingSystem -Library Collections -Resource ../commonlib.robot -Resource ../ozone-lib/shell.robot - -Test Timeout 20 minutes - -*** Variables *** -${SECURITY_ENABLED} false -${HOST} datanode1 -${VOLUME} volume1 -${BUCKET} bucket1 -${SIZE} 104857600 - - -** Keywords *** -Prepare For Tests - Execute dd if=/dev/urandom of=/tmp/100mb bs=1048576 count=100 - Run Keyword if '${SECURITY_ENABLED}' == 'true' Kinit test user testuser testuser.keytab - Execute ozone sh volume create /${VOLUME} - Execute ozone sh bucket create /${VOLUME}/${BUCKET} - - -Datanode In Maintenance Mode - ${result} = Execute ozone admin datanode maintenance ${HOST} - Should Contain ${result} Entering maintenance mode on datanode - ${result} = Execute ozone admin datanode list | grep "Operational State:*" - Wait Until Keyword Succeeds 30sec 5sec Should contain ${result} ENTERING_MAINTENANCE - Wait Until Keyword Succeeds 3min 10sec Related pipelines are closed - Sleep 60000ms - -Create Multiple Keys - [arguments] ${NUM_KEYS} - ${file} = Set Variable /tmp/100mb - FOR ${INDEX} IN RANGE ${NUM_KEYS} - ${fileName} = Set Variable file-${INDEX}.txt - ${key} = Set Variable /${VOLUME}/${BUCKET}/${fileName} - LOG ${fileName} - Create Key ${key} ${file} - Key Should Match Local File ${key} ${file} - END - -Close All Containers - FOR ${INDEX} IN RANGE 15 - ${container} = Execute ozone admin container list --state OPEN | jq -r 'select(.replicationConfig.replicationFactor == "THREE") | .containerID' | head -1 - EXIT FOR LOOP IF "${container}" == "${EMPTY}" - ${message} = Execute And Ignore Error ozone admin container close "${container}" - Run Keyword If '${message}' != '${EMPTY}' Should Contain ${message} is in closing state - ${output} = Execute ozone admin container info "${container}" - Should contain ${output} CLOS - END - Wait until keyword succeeds 3min 10sec All container is closed - -All container is closed - ${output} = Execute ozone admin container list --state OPEN - Should Be Empty ${output} - -Related pipelines are closed - ${result} = Execute ozone admin datanode list | awk -v RS= '{$1=$1}1'|grep MAINT | sed -e 's/^.*pipelines: \\(.*\\)$/\\1/' - Should Contain Any ${result} CLOSED No related pipelines or the node is not in Healthy state. - -Datanode Recommission - ${result} = Execute ozone admin datanode recommission ${HOST} - Should Contain ${result} Started recommissioning datanode - Wait Until Keyword Succeeds 1min 10sec Datanode Recommission is Finished - Sleep 300000ms - -Datanode Recommission is Finished - ${result} = Execute ozone admin datanode list | grep "Operational State:*" - Should Not Contain ${result} ENTERING_MAINTENANCE -Run Container Balancer - ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 - Should Contain ${result} Container Balancer started successfully. - -Verify Verbose Balancer Status - [arguments] ${output} - - 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} - -Run Balancer Status - ${result} = Execute ozone admin containerbalancer status - Should Contain ${result} ContainerBalancer is Running. - -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: - -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: -Get Datanode Ozone Used Bytes Info - [arguments] ${uuid} - ${output} = Execute export DATANODES=$(ozone admin datanode list --json) && for datanode in $(echo "$\{DATANODES\}" | jq -r '.[].datanodeDetails.uuid'); do ozone admin datanode usageinfo --uuid=$\{datanode\} --json | jq '{(.[0].datanodeDetails.uuid) : .[0].ozoneUsed}'; done | jq -s add - ${result} = Execute echo '${output}' | jq '. | to_entries | .[] | select(.key == "${uuid}") | .value' - [return] ${result} - -** Test Cases *** -Verify Container Balancer Status Output - Prepare For Tests - - Datanode In Maintenance Mode - - Create Multiple Keys 3 - - Close All Containers - - Datanode Recommission - - Run Container Balancer - - Run Balancer Status - - Run Balancer Verbose Status - - Run Balancer Verbose History Status - - - - From ffafee0535a23966504e238d830f4c205407fc06 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 18 Jul 2024 00:27:26 +0300 Subject: [PATCH 25/27] HDDS-11120. Fix testBalancer test. --- .../dist/src/main/smoketest/balancer/testBalancer.robot | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot index 7977cead7cbd..4299afe5f2d1 100644 --- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot +++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot @@ -63,6 +63,8 @@ Datanode Recommission is Finished Run Container Balancer ${result} = Execute ozone admin containerbalancer start -t 1 -d 100 -i 1 Should Contain ${result} Container Balancer started successfully. + +Wait Finish Of Balancing ${result} = Execute ozone admin containerbalancer status Should Contain ${result} ContainerBalancer is Running. Wait Until Keyword Succeeds 3min 10sec ContainerBalancer is Not Running @@ -169,6 +171,8 @@ Verify Container Balancer for RATIS containers Run Balancer Verbose History Status + Wait Finish Of Balancing + ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} = Get Datanode Ozone Used Bytes Info ${uuid} Should Not Be Equal As Integers ${datanodeOzoneUsedBytesInfo} ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} Should Be True ${datanodeOzoneUsedBytesInfoAfterContainerBalancing} < ${SIZE} * 3.5 From fe663c1d3d30d2360b31aeb6d787740f17451fba Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Thu, 18 Jul 2024 16:40:15 +0300 Subject: [PATCH 26/27] Empty commit message for restart pr checker From b59431be0c9b2bbda763e863f7bcda319c2decc5 Mon Sep 17 00:00:00 2001 From: Alexandr Juncevich Date: Mon, 22 Jul 2024 10:22:36 +0300 Subject: [PATCH 27/27] HDDS-11120. Fix review notices. --- .../src/main/proto/ScmAdminProtocol.proto | 8 +- .../container/balancer/ContainerBalancer.java | 7 +- .../balancer/ContainerBalancerTask.java | 83 +++++++++---------- ...tainerBalancerTaskIterationStatusInfo.java | 30 +++---- .../scm/server/SCMClientProtocolServer.java | 14 ++-- .../TestContainerBalancerStatusInfo.java | 4 +- .../ContainerBalancerStatusSubcommand.java | 10 +-- .../TestContainerBalancerSubCommand.java | 54 ++++++------ 8 files changed, 101 insertions(+), 109 deletions(-) diff --git a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto index 5bb0b572eafb..039914369b5f 100644 --- a/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto +++ b/hadoop-hdds/interface-admin/src/main/proto/ScmAdminProtocol.proto @@ -627,19 +627,19 @@ message ContainerBalancerStatusInfo { message ContainerBalancerTaskIterationStatusInfo { optional int32 iterationNumber = 1; optional string iterationResult = 2; - optional int64 sizeScheduledForMove = 3; + optional int64 sizeScheduledForMoveGB = 3; optional int64 dataSizeMovedGB = 4; optional int64 containerMovesScheduled = 5; optional int64 containerMovesCompleted = 6; optional int64 containerMovesFailed = 7; optional int64 containerMovesTimeout = 8; - repeated NodeTransferInfo sizeEnteringNodes = 9; - repeated NodeTransferInfo sizeLeavingNodes = 10; + repeated NodeTransferInfo sizeEnteringNodesGB = 9; + repeated NodeTransferInfo sizeLeavingNodesGB = 10; } message NodeTransferInfo { optional string uuid = 1; - optional int64 dataVolume = 2; + optional int64 dataVolumeGB = 2; } message DecommissionScmRequestProto { diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java index 2c6836daa815..3dddd67bd8ad 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancer.java @@ -22,7 +22,6 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.conf.StorageUnit; import org.apache.hadoop.hdds.fs.DUFactory; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos; import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ContainerBalancerConfigurationProto; import org.apache.hadoop.hdds.scm.ScmConfigKeys; import org.apache.hadoop.hdds.scm.ha.SCMContext; @@ -184,10 +183,8 @@ public ContainerBalancerTask.Status getBalancerStatus() { * @return balancer status info if balancer started */ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws IOException { - boolean isTaskRunning = task != null && task.getBalancerStatus() == ContainerBalancerTask.Status.RUNNING; - if (isTaskRunning) { - HddsProtos.ContainerBalancerConfigurationProto configProto = - readConfiguration(HddsProtos.ContainerBalancerConfigurationProto.class); + if (isBalancerRunning()) { + ContainerBalancerConfigurationProto configProto = readConfiguration(ContainerBalancerConfigurationProto.class); return new ContainerBalancerStatusInfo( this.startedAt, configProto, 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 733eb3359525..7fea44671ffa 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 @@ -336,50 +336,45 @@ private void saveIterationStatistic(Integer iterationNumber, IterationResult iR) public List getCurrentIterationsStatistic() { - boolean isActiveIterationPresent = iterationsStatistic.stream() - .anyMatch(iterationStatusInfo -> iterationStatusInfo.getIterationResult() == null); - - if (!isActiveIterationPresent) { - - int lastIterationNumber = iterationsStatistic.stream() - .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) - .max() - .orElse(0); - - ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( - lastIterationNumber + 1, - null, - getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, - sizeActuallyMovedInLatestIteration / OzoneConsts.GB, - 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(), - entry -> entry.getValue() / OzoneConsts.GB - ) - ), - findSourceStrategy.getSizeLeavingNodes() - .entrySet() - .stream() - .filter(Objects::nonNull) - .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) - .collect( - Collectors.toMap( - entry -> entry.getKey().getUuid(), - entry -> entry.getValue() / OzoneConsts.GB - ) - ) - ); - iterationsStatistic.add(currentIterationStatistic); - } - return iterationsStatistic; + int lastIterationNumber = iterationsStatistic.stream() + .mapToInt(ContainerBalancerTaskIterationStatusInfo::getIterationNumber) + .max() + .orElse(0); + + ContainerBalancerTaskIterationStatusInfo currentIterationStatistic = new ContainerBalancerTaskIterationStatusInfo( + lastIterationNumber + 1, + null, + getSizeScheduledForMoveInLatestIteration() / OzoneConsts.GB, + sizeActuallyMovedInLatestIteration / OzoneConsts.GB, + 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(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ), + findSourceStrategy.getSizeLeavingNodes() + .entrySet() + .stream() + .filter(Objects::nonNull) + .filter(datanodeDetailsLongEntry -> datanodeDetailsLongEntry.getValue() > 0) + .collect( + Collectors.toMap( + entry -> entry.getKey().getUuid(), + entry -> entry.getValue() / OzoneConsts.GB + ) + ) + ); + 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 11bc85074440..1d597b0ca273 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,37 @@ public class ContainerBalancerTaskIterationStatusInfo { private final Integer iterationNumber; private final String iterationResult; - private final long sizeScheduledForMove; + private final long sizeScheduledForMoveGB; private final long dataSizeMovedGB; private final long containerMovesScheduled; private final long containerMovesCompleted; private final long containerMovesFailed; private final long containerMovesTimeout; - private final Map sizeEnteringNodes; - private final Map sizeLeavingNodes; + private final Map sizeEnteringNodesGB; + private final Map sizeLeavingNodesGB; @SuppressWarnings("checkstyle:ParameterNumber") public ContainerBalancerTaskIterationStatusInfo( Integer iterationNumber, String iterationResult, - long sizeScheduledForMove, + long sizeScheduledForMoveGB, long dataSizeMovedGB, long containerMovesScheduled, long containerMovesCompleted, long containerMovesFailed, long containerMovesTimeout, - Map sizeEnteringNodes, - Map sizeLeavingNodes) { + Map sizeEnteringNodesGB, + Map sizeLeavingNodesGB) { this.iterationNumber = iterationNumber; this.iterationResult = iterationResult; - this.sizeScheduledForMove = sizeScheduledForMove; + this.sizeScheduledForMoveGB = sizeScheduledForMoveGB; this.dataSizeMovedGB = dataSizeMovedGB; this.containerMovesScheduled = containerMovesScheduled; this.containerMovesCompleted = containerMovesCompleted; this.containerMovesFailed = containerMovesFailed; this.containerMovesTimeout = containerMovesTimeout; - this.sizeEnteringNodes = sizeEnteringNodes; - this.sizeLeavingNodes = sizeLeavingNodes; + this.sizeEnteringNodesGB = sizeEnteringNodesGB; + this.sizeLeavingNodesGB = sizeLeavingNodesGB; } public Integer getIterationNumber() { @@ -68,8 +68,8 @@ public String getIterationResult() { return iterationResult; } - public long getSizeScheduledForMove() { - return sizeScheduledForMove; + public long getSizeScheduledForMoveGB() { + return sizeScheduledForMoveGB; } public long getDataSizeMovedGB() { @@ -92,12 +92,12 @@ public long getContainerMovesTimeout() { return containerMovesTimeout; } - public Map getSizeEnteringNodes() { - return sizeEnteringNodes; + public Map getSizeEnteringNodesGB() { + return sizeEnteringNodesGB; } - public Map getSizeLeavingNodes() { - return sizeLeavingNodes; + public Map getSizeLeavingNodesGB() { + return sizeLeavingNodesGB; } } 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 1489233f0c3f..828b452d3000 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 @@ -1232,28 +1232,28 @@ public ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo() info -> ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(info.getIterationNumber()) .setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse("")) - .setSizeScheduledForMove(info.getSizeScheduledForMove()) + .setSizeScheduledForMoveGB(info.getSizeScheduledForMoveGB()) .setDataSizeMovedGB(info.getDataSizeMovedGB()) .setContainerMovesScheduled(info.getContainerMovesScheduled()) .setContainerMovesCompleted(info.getContainerMovesCompleted()) .setContainerMovesFailed(info.getContainerMovesFailed()) .setContainerMovesTimeout(info.getContainerMovesTimeout()) - .addAllSizeEnteringNodes( - info.getSizeEnteringNodes().entrySet() + .addAllSizeEnteringNodesGB( + info.getSizeEnteringNodesGB().entrySet() .stream() .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) + .setDataVolumeGB(entry.getValue()) .build() ) .collect(Collectors.toList()) ) - .addAllSizeLeavingNodes( - info.getSizeLeavingNodes().entrySet() + .addAllSizeLeavingNodesGB( + info.getSizeLeavingNodesGB().entrySet() .stream() .map(entry -> NodeTransferInfo.newBuilder() .setUuid(entry.getKey().toString()) - .setDataVolume(entry.getValue()) + .setDataVolumeGB(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 1918db1e1fec..b8ac648e8442 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.getSizeEnteringNodes().isEmpty()); - assertFalse(is.getSizeLeavingNodes().isEmpty()); + assertFalse(is.getSizeEnteringNodesGB().isEmpty()); + assertFalse(is.getSizeLeavingNodesGB().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 d3d2ed2d54b5..e58074bf140a 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 @@ -137,17 +137,17 @@ String getConfigurationPrettyString(HddsProtos.ContainerBalancerConfigurationPro private String getPrettyIterationStatusInfo(ContainerBalancerTaskIterationStatusInfo iterationStatusInfo) { int iterationNumber = iterationStatusInfo.getIterationNumber(); String iterationResult = iterationStatusInfo.getIterationResult(); - long sizeScheduledForMove = iterationStatusInfo.getSizeScheduledForMove(); + long sizeScheduledForMove = iterationStatusInfo.getSizeScheduledForMoveGB(); long dataSizeMovedGB = iterationStatusInfo.getDataSizeMovedGB(); long containerMovesScheduled = iterationStatusInfo.getContainerMovesScheduled(); long containerMovesCompleted = iterationStatusInfo.getContainerMovesCompleted(); long containerMovesFailed = iterationStatusInfo.getContainerMovesFailed(); long containerMovesTimeout = iterationStatusInfo.getContainerMovesTimeout(); - String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolume() + "\n") + String enteringDataNodeList = iterationStatusInfo.getSizeEnteringNodesGBList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " + nodeInfo.getDataVolumeGB() + "\n") .collect(Collectors.joining()); - String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesList() - .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolume() + "\n") + String leavingDataNodeList = iterationStatusInfo.getSizeLeavingNodesGBList() + .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " + nodeInfo.getDataVolumeGB() + "\n") .collect(Collectors.joining()); return String.format( "%-50s %s%n" + 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 ce5748d1b0c2..41b419d23263 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 @@ -98,34 +98,34 @@ public void testContainerBalancerStatusInfoSubcommandRunning() StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(0) .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMove(48) + .setSizeScheduledForMoveGB(48) .setDataSizeMovedGB(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27) + .setDataVolumeGB(27) .build() ) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolumeGB(23L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolumeGB(24L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolumeGB(26L) .build() ) .build(); @@ -133,34 +133,34 @@ public void testContainerBalancerStatusInfoSubcommandRunning() StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(1) .setIterationResult("ITERATION_COMPLETED") - .setSizeScheduledForMove(48) + .setSizeScheduledForMoveGB(48) .setDataSizeMovedGB(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) + .setDataVolumeGB(27L) .build() ) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolumeGB(23L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolumeGB(24L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolumeGB(26L) .build() ) .build(); @@ -168,34 +168,34 @@ public void testContainerBalancerStatusInfoSubcommandRunning() StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder() .setIterationNumber(1) .setIterationResult("") - .setSizeScheduledForMove(48) + .setSizeScheduledForMoveGB(48) .setDataSizeMovedGB(48) .setContainerMovesScheduled(11) .setContainerMovesCompleted(11) .setContainerMovesFailed(0) .setContainerMovesTimeout(0) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d") - .setDataVolume(27L) + .setDataVolumeGB(27L) .build() ) - .addSizeEnteringNodes( + .addSizeEnteringNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6") - .setDataVolume(23L) + .setDataVolumeGB(23L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("b8b9c511-c30f-4933-8938-2f272e307070") - .setDataVolume(24L) + .setDataVolumeGB(24L) .build() ) - .addSizeLeavingNodes( + .addSizeLeavingNodesGB( StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder() .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130") - .setDataVolume(26L) + .setDataVolumeGB(26L) .build() ) .build();