This is an automated email from the ASF dual-hosted git repository.
ivandika pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/ozone.git
The following commit(s) were added to refs/heads/master by this push:
new a5e420cf90 HDDS-11120. Rich rebalancing status info (#6911)
a5e420cf90 is described below
commit a5e420cf90c67728a47f9350340a059bed1ed1c2
Author: Alexandr Juncevich <[email protected]>
AuthorDate: Mon Jul 22 13:46:02 2024 +0300
HDDS-11120. Rich rebalancing status info (#6911)
---
.../apache/hadoop/hdds/scm/client/ScmClient.java | 3 +
.../protocol/StorageContainerLocationProtocol.java | 3 +
...inerLocationProtocolClientSideTranslatorPB.java | 15 ++
.../src/main/proto/ScmAdminProtocol.proto | 35 ++++
.../balancer/AbstractFindTargetGreedy.java | 4 +
.../scm/container/balancer/ContainerBalancer.java | 22 +++
.../balancer/ContainerBalancerConfiguration.java | 2 +-
.../balancer/ContainerBalancerMetrics.java | 5 +
.../balancer/ContainerBalancerStatusInfo.java | 54 ++++++
.../container/balancer/ContainerBalancerTask.java | 90 ++++++++++
.../ContainerBalancerTaskIterationStatusInfo.java | 104 +++++++++++
.../scm/container/balancer/FindSourceGreedy.java | 5 +
.../scm/container/balancer/FindSourceStrategy.java | 3 +
.../scm/container/balancer/FindTargetStrategy.java | 3 +
...inerLocationProtocolServerSideTranslatorPB.java | 14 ++
.../hdds/scm/server/SCMClientProtocolServer.java | 75 +++++++-
.../org/apache/hadoop/ozone/audit/SCMAction.java | 1 +
.../balancer/TestContainerBalancerStatusInfo.java | 58 ++++++
.../scm/cli/ContainerBalancerStatusSubcommand.java | 137 +++++++++++++-
.../hdds/scm/cli/ContainerOperationClient.java | 6 +
.../datanode/TestContainerBalancerSubCommand.java | 198 ++++++++++++++++++---
.../src/main/smoketest/balancer/testBalancer.robot | 40 +++++
22 files changed, 849 insertions(+), 28 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 14fb0a40cd..34b2680b30 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.DatanodeDetails;
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 @@ public interface ScmClient extends Closeable {
*/
boolean getContainerBalancerStatus() throws IOException;
+ 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 df8ed02cf7..45825cb2b6 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.HddsProtos.DeletedBlocksTransaction
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 @@ public interface StorageContainerLocationProtocol extends
Closeable {
*/
boolean getContainerBalancerStatus() throws IOException;
+ ContainerBalancerStatusInfoResponseProto getContainerBalancerStatusInfo()
throws IOException;
+
/**
* 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 3570257b58..c058cb4d44 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.StorageContainerLocationProtocolPro
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 final class
StorageContainerLocationProtocolClientSideTranslatorPB
}
+ @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 c190dc3f45..039914369b 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,38 @@ message ContainerBalancerStatusResponseProto {
required bool isRunning = 1;
}
+message ContainerBalancerStatusInfoRequestProto {
+ optional string traceID = 1;
+}
+
+message ContainerBalancerStatusInfoResponseProto {
+ optional bool isRunning = 1;
+ optional ContainerBalancerStatusInfo containerBalancerStatusInfo = 2;
+}
+message ContainerBalancerStatusInfo {
+ optional uint64 startedAt = 1;
+ optional ContainerBalancerConfigurationProto configuration = 2;
+ repeated ContainerBalancerTaskIterationStatusInfo iterationsStatusInfo = 3;
+}
+
+message ContainerBalancerTaskIterationStatusInfo {
+ optional int32 iterationNumber = 1;
+ optional string iterationResult = 2;
+ 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 sizeEnteringNodesGB = 9;
+ repeated NodeTransferInfo sizeLeavingNodesGB = 10;
+}
+
+message NodeTransferInfo {
+ optional string uuid = 1;
+ optional int64 dataVolumeGB = 2;
+}
+
message DecommissionScmRequestProto {
required string scmId = 1;
}
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 5416a9ff1c..dd2d1c5789 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 @@ public abstract class AbstractFindTargetGreedy implements
FindTargetStrategy {
return nodeManager;
}
+ @Override
+ public Map<DatanodeDetails, Long> 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 f47abe65be..3dddd67bd8 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.Logger;
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,24 @@ public class ContainerBalancer extends StatefulService {
: ContainerBalancerTask.Status.STOPPED;
}
+ /**
+ * Get balancer status info.
+ *
+ * @return balancer status info if balancer started
+ */
+ public ContainerBalancerStatusInfo getBalancerStatusInfo() throws
IOException {
+ if (isBalancerRunning()) {
+ ContainerBalancerConfigurationProto configProto =
readConfiguration(ContainerBalancerConfigurationProto.class);
+ return new ContainerBalancerStatusInfo(
+ this.startedAt,
+ configProto,
+ task.getCurrentIterationsStatistic()
+ );
+ } else {
+ return null;
+ }
+
+ }
/**
* Checks if ContainerBalancer is in valid state to call stop.
*
@@ -204,6 +224,7 @@ public class ContainerBalancer extends StatefulService {
@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
@@ -251,6 +272,7 @@ public class ContainerBalancer extends StatefulService {
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/ContainerBalancerConfiguration.java
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerConfiguration.java
index e275d345a5..644e362b3b 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 final class ContainerBalancerConfiguration {
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 09558d3a6d..6446089db3 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 @@ public final class ContainerBalancerMetrics {
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/ContainerBalancerStatusInfo.java
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java
new file mode 100644
index 0000000000..cbe8385e53
--- /dev/null
+++
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerStatusInfo.java
@@ -0,0 +1,54 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.protocol.proto.HddsProtos;
+
+import java.time.OffsetDateTime;
+import java.util.List;
+
+/**
+ * Info about balancer status.
+ */
+public class ContainerBalancerStatusInfo {
+ private final OffsetDateTime startedAt;
+ private final HddsProtos.ContainerBalancerConfigurationProto configuration;
+ private final List<ContainerBalancerTaskIterationStatusInfo>
iterationsStatusInfo;
+
+ public ContainerBalancerStatusInfo(
+ OffsetDateTime startedAt,
+ HddsProtos.ContainerBalancerConfigurationProto configuration,
+ List<ContainerBalancerTaskIterationStatusInfo> iterationsStatusInfo)
{
+ this.startedAt = startedAt;
+ this.configuration = configuration;
+ this.iterationsStatusInfo = iterationsStatusInfo;
+ }
+
+ public OffsetDateTime getStartedAt() {
+ return startedAt;
+ }
+
+ public HddsProtos.ContainerBalancerConfigurationProto getConfiguration() {
+ return configuration;
+ }
+
+ public List<ContainerBalancerTaskIterationStatusInfo>
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 8daf764057..7fea44671f 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.HashSet;
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<ContainerBalancerTaskIterationStatusInfo> iterationsStatistic;
/**
* Constructs ContainerBalancerTask with the specified arguments.
@@ -155,6 +158,7 @@ public class ContainerBalancerTask implements Runnable {
this.selectedSources = new HashSet<>();
this.selectedTargets = new HashSet<>();
findSourceStrategy = new FindSourceGreedy(nodeManager);
+ this.iterationsStatistic = new ArrayList<>();
}
/**
@@ -250,7 +254,9 @@ public class ContainerBalancerTask implements Runnable {
}
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,85 @@ public class ContainerBalancerTask implements Runnable {
tryStopWithSaveConfiguration("Completed all iterations.");
}
+ private void saveIterationStatistic(Integer iterationNumber, IterationResult
iR) {
+ ContainerBalancerTaskIterationStatusInfo iterationStatistic = new
ContainerBalancerTaskIterationStatusInfo(
+ iterationNumber,
+ iR.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<ContainerBalancerTaskIterationStatusInfo>
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
+ )
+ )
+ );
+ List<ContainerBalancerTaskIterationStatusInfo> resultList = new
ArrayList<>(iterationsStatistic);
+ resultList.add(currentIterationStatistic);
+ return resultList;
+ }
+
/**
* Logs the reason for stop and save configuration and stop the task.
*
@@ -1060,6 +1145,7 @@ public class ContainerBalancerTask implements Runnable {
this.sizeScheduledForMoveInLatestIteration = 0;
this.sizeActuallyMovedInLatestIteration = 0;
metrics.resetDataSizeMovedGBInLatestIteration();
+ metrics.resetNumContainerMovesScheduledInLatestIteration();
metrics.resetNumContainerMovesCompletedInLatestIteration();
metrics.resetNumContainerMovesTimeoutInLatestIteration();
metrics.resetNumDatanodesInvolvedInLatestIteration();
@@ -1136,6 +1222,10 @@ public class ContainerBalancerTask implements Runnable {
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 0000000000..1d597b0ca2
--- /dev/null
+++
b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/container/balancer/ContainerBalancerTaskIterationStatusInfo.java
@@ -0,0 +1,104 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+import java.util.UUID;
+
+/**
+ * Information about balancer task iteration.
+ */
+public class ContainerBalancerTaskIterationStatusInfo {
+ private final Integer iterationNumber;
+ private final String iterationResult;
+ 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<UUID, Long> sizeEnteringNodesGB;
+ private final Map<UUID, Long> sizeLeavingNodesGB;
+
+ @SuppressWarnings("checkstyle:ParameterNumber")
+ public ContainerBalancerTaskIterationStatusInfo(
+ Integer iterationNumber,
+ String iterationResult,
+ long sizeScheduledForMoveGB,
+ long dataSizeMovedGB,
+ long containerMovesScheduled,
+ long containerMovesCompleted,
+ long containerMovesFailed,
+ long containerMovesTimeout,
+ Map<UUID, Long> sizeEnteringNodesGB,
+ Map<UUID, Long> sizeLeavingNodesGB) {
+ this.iterationNumber = iterationNumber;
+ this.iterationResult = iterationResult;
+ this.sizeScheduledForMoveGB = sizeScheduledForMoveGB;
+ this.dataSizeMovedGB = dataSizeMovedGB;
+ this.containerMovesScheduled = containerMovesScheduled;
+ this.containerMovesCompleted = containerMovesCompleted;
+ this.containerMovesFailed = containerMovesFailed;
+ this.containerMovesTimeout = containerMovesTimeout;
+ this.sizeEnteringNodesGB = sizeEnteringNodesGB;
+ this.sizeLeavingNodesGB = sizeLeavingNodesGB;
+ }
+
+ public Integer getIterationNumber() {
+ return iterationNumber;
+ }
+
+ public String getIterationResult() {
+ return iterationResult;
+ }
+
+ public long getSizeScheduledForMoveGB() {
+ return sizeScheduledForMoveGB;
+ }
+
+ 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<UUID, Long> getSizeEnteringNodesGB() {
+ return sizeEnteringNodesGB;
+ }
+
+ public Map<UUID, Long> getSizeLeavingNodesGB() {
+ return sizeLeavingNodesGB;
+ }
+}
+
+
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 8306d8e1e1..435cc9859a 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 class FindSourceGreedy implements FindSourceStrategy
{
sizeLeavingNode.clear();
resetSources(potentialDataNodes);
}
+
+ @Override
+ public Map<DatanodeDetails, Long> 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 f9eb24bd3c..9e429aaa21 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 org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
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 @@ public interface FindSourceStrategy {
* {@link DatanodeDetails} that containers can move from
*/
void resetPotentialSources(@Nonnull Collection<DatanodeDetails> sources);
+
+ Map<DatanodeDetails, Long> 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 a9f2ee00a2..389ea6e519 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 org.apache.hadoop.hdds.scm.node.DatanodeUsageInfo;
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 @@ public interface FindTargetStrategy {
* that containers can be moved to
*/
void resetPotentialTargets(@Nonnull Collection<DatanodeDetails> targets);
+
+ Map<DatanodeDetails, Long> 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 3d7cff358f..f2efae9598 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.StorageContainerLocationProtocolPro
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;
@@ -600,6 +601,13 @@ public final class
StorageContainerLocationProtocolServerSideTranslatorPB
.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 +1204,12 @@ public final class
StorageContainerLocationProtocolServerSideTranslatorPB
.setIsRunning(impl.getContainerBalancerStatus()).build();
}
+ public ContainerBalancerStatusInfoResponseProto
getContainerBalancerStatusInfo(
+
StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfoRequestProto
request)
+ throws IOException {
+ return impl.getContainerBalancerStatusInfo();
+ }
+
public DecommissionNodesResponseProto decommissionNodes(
DecommissionNodesRequestProto request) throws IOException {
List<DatanodeAdminError> 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 47bc66d833..828b452d30 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,28 +33,33 @@ import org.apache.hadoop.hdds.conf.OzoneConfiguration;
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.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;
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.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;
+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 +67,6 @@ import org.apache.hadoop.hdds.scm.ha.SCMHAUtils;
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 +112,14 @@ import java.util.Map;
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 +1206,67 @@ public class SCMClientProtocolServer implements
return scm.getContainerBalancer().isBalancerRunning();
}
+ @Override
+ public ContainerBalancerStatusInfoResponseProto
getContainerBalancerStatusInfo() throws IOException {
+ AUDIT.logReadSuccess(buildAuditMessageForSuccess(
+ SCMAction.GET_CONTAINER_BALANCER_STATUS_INFO, null));
+ ContainerBalancerStatusInfo balancerStatusInfo =
scm.getContainerBalancer().getBalancerStatusInfo();
+ if (balancerStatusInfo == null) {
+ return ContainerBalancerStatusInfoResponseProto
+ .newBuilder()
+ .setIsRunning(false)
+ .build();
+ } else {
+
+ return ContainerBalancerStatusInfoResponseProto
+ .newBuilder()
+ .setIsRunning(true)
+
.setContainerBalancerStatusInfo(StorageContainerLocationProtocolProtos.ContainerBalancerStatusInfo
+ .newBuilder()
+ .setStartedAt(balancerStatusInfo.getStartedAt().toEpochSecond())
+ .setConfiguration(balancerStatusInfo.getConfiguration())
+ .addAllIterationsStatusInfo(
+ balancerStatusInfo.getIterationsStatusInfo()
+ .stream()
+ .map(
+ info ->
ContainerBalancerTaskIterationStatusInfo.newBuilder()
+ .setIterationNumber(info.getIterationNumber())
+
.setIterationResult(Optional.ofNullable(info.getIterationResult()).orElse(""))
+
.setSizeScheduledForMoveGB(info.getSizeScheduledForMoveGB())
+ .setDataSizeMovedGB(info.getDataSizeMovedGB())
+
.setContainerMovesScheduled(info.getContainerMovesScheduled())
+
.setContainerMovesCompleted(info.getContainerMovesCompleted())
+
.setContainerMovesFailed(info.getContainerMovesFailed())
+
.setContainerMovesTimeout(info.getContainerMovesTimeout())
+ .addAllSizeEnteringNodesGB(
+ info.getSizeEnteringNodesGB().entrySet()
+ .stream()
+ .map(entry ->
NodeTransferInfo.newBuilder()
+ .setUuid(entry.getKey().toString())
+ .setDataVolumeGB(entry.getValue())
+ .build()
+ )
+ .collect(Collectors.toList())
+ )
+ .addAllSizeLeavingNodesGB(
+ info.getSizeLeavingNodesGB().entrySet()
+ .stream()
+ .map(entry ->
NodeTransferInfo.newBuilder()
+ .setUuid(entry.getKey().toString())
+ .setDataVolumeGB(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 4e1fe234ff..128e3401c3 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 0000000000..b8ac648e84
--- /dev/null
+++
b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/container/balancer/TestContainerBalancerStatusInfo.java
@@ -0,0 +1,58 @@
+/*
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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;
+import org.apache.hadoop.ozone.OzoneConsts;
+import org.junit.jupiter.api.Test;
+
+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;
+
+/**
+ * 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<ContainerBalancerTaskIterationStatusInfo> iterationStatistics =
task.getCurrentIterationsStatistic();
+ assertEquals(3, iterationStatistics.size());
+ iterationStatistics.forEach(is -> {
+ assertTrue(is.getContainerMovesCompleted() > 0);
+ assertEquals(0, is.getContainerMovesFailed());
+ assertEquals(0, is.getContainerMovesTimeout());
+ 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 44e4d4c9c5..e58074bf14 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,22 @@
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.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;
+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 +45,134 @@ import java.io.IOException;
versionProvider = HddsVersionProvider.class)
public class ContainerBalancerStatusSubcommand extends ScmSubcommand {
+ @CommandLine.Option(names = {"-v", "--verbose"},
+ description = "Verbose output. Show current iteration info.")
+ private boolean verbose;
+
+ @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;
+
@Override
public void execute(ScmClient scmClient) throws IOException {
- boolean execReturn = scmClient.getContainerBalancerStatus();
- if (execReturn) {
+ ContainerBalancerStatusInfoResponseProto response =
scmClient.getContainerBalancerStatusInfo();
+ boolean isRunning = response.getIsRunning();
+ ContainerBalancerStatusInfo balancerStatusInfo =
response.getContainerBalancerStatusInfo();
+ if (isRunning) {
+ LocalDateTime dateTime =
+
LocalDateTime.ofInstant(Instant.ofEpochSecond(balancerStatusInfo.getStartedAt()),
ZoneId.systemDefault());
System.out.println("ContainerBalancer is Running.");
+
+ if (verbose) {
+ System.out.printf("Started at: %s %s%n%n", dateTime.toLocalDate(),
dateTime.toLocalTime());
+
System.out.println(getConfigurationPrettyString(balancerStatusInfo.getConfiguration()));
+ List<ContainerBalancerTaskIterationStatusInfo> iterationsStatusInfoList
+ = balancerStatusInfo.getIterationsStatusInfoList();
+
+ 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)
+ .collect(Collectors.joining("\n"))
+ );
+ }
+ }
+
} else {
System.out.println("ContainerBalancer is Not Running.");
}
}
+
+ 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(ContainerBalancerTaskIterationStatusInfo
iterationStatusInfo) {
+ int iterationNumber = iterationStatusInfo.getIterationNumber();
+ String iterationResult = iterationStatusInfo.getIterationResult();
+ 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.getSizeEnteringNodesGBList()
+ .stream().map(nodeInfo -> nodeInfo.getUuid() + " <- " +
nodeInfo.getDataVolumeGB() + "\n")
+ .collect(Collectors.joining());
+ String leavingDataNodeList =
iterationStatusInfo.getSizeLeavingNodesGBList()
+ .stream().map(nodeInfo -> nodeInfo.getUuid() + " -> " +
nodeInfo.getDataVolumeGB() + "\n")
+ .collect(Collectors.joining());
+ 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", 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);
+ }
}
+
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 0dd52cd291..ba556bf24e 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;
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 class ContainerOperationClient implements ScmClient
{
return storageContainerLocationClient.getContainerBalancerStatus();
}
+ @Override
+ public ContainerBalancerStatusInfoResponseProto
getContainerBalancerStatusInfo() throws IOException {
+ return storageContainerLocationClient.getContainerBalancerStatusInfo();
+ }
+
@Override
public List<String> 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 27c360e727..41b419d232 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,13 @@
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.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;
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 +34,8 @@ import java.io.IOException;
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;
@@ -42,16 +47,16 @@ import static org.mockito.Mockito.when;
* 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 {
@@ -69,15 +74,147 @@ public class TestContainerBalancerSubCommand {
}
@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")
+ .setSizeScheduledForMoveGB(48)
+ .setDataSizeMovedGB(48)
+ .setContainerMovesScheduled(11)
+ .setContainerMovesCompleted(11)
+ .setContainerMovesFailed(0)
+ .setContainerMovesTimeout(0)
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d")
+ .setDataVolumeGB(27)
+ .build()
+ )
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6")
+ .setDataVolumeGB(23L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("b8b9c511-c30f-4933-8938-2f272e307070")
+ .setDataVolumeGB(24L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130")
+ .setDataVolumeGB(26L)
+ .build()
+ )
+ .build();
+
StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo
iteration1StatusInfo =
+
StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder()
+ .setIterationNumber(1)
+ .setIterationResult("ITERATION_COMPLETED")
+ .setSizeScheduledForMoveGB(48)
+ .setDataSizeMovedGB(48)
+ .setContainerMovesScheduled(11)
+ .setContainerMovesCompleted(11)
+ .setContainerMovesFailed(0)
+ .setContainerMovesTimeout(0)
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d")
+ .setDataVolumeGB(27L)
+ .build()
+ )
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6")
+ .setDataVolumeGB(23L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("b8b9c511-c30f-4933-8938-2f272e307070")
+ .setDataVolumeGB(24L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130")
+ .setDataVolumeGB(26L)
+ .build()
+ )
+ .build();
+
StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo
iteration2StatusInfo =
+
StorageContainerLocationProtocolProtos.ContainerBalancerTaskIterationStatusInfo.newBuilder()
+ .setIterationNumber(1)
+ .setIterationResult("")
+ .setSizeScheduledForMoveGB(48)
+ .setDataSizeMovedGB(48)
+ .setContainerMovesScheduled(11)
+ .setContainerMovesCompleted(11)
+ .setContainerMovesFailed(0)
+ .setContainerMovesTimeout(0)
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("80f6bc27-e6f3-493e-b1f4-25f810ad960d")
+ .setDataVolumeGB(27L)
+ .build()
+ )
+ .addSizeEnteringNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("701ca98e-aa1a-4b36-b817-e28ed634bba6")
+ .setDataVolumeGB(23L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("b8b9c511-c30f-4933-8938-2f272e307070")
+ .setDataVolumeGB(24L)
+ .build()
+ )
+ .addSizeLeavingNodesGB(
+
StorageContainerLocationProtocolProtos.NodeTransferInfo.newBuilder()
+ .setUuid("7bd99815-47e7-4015-bc61-ca6ef6dfd130")
+ .setDataVolumeGB(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)
+ )
+ )
+
+ .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));
@@ -85,11 +222,32 @@ public class TestContainerBalancerSubCommand {
}
@Test
- public void testContainerBalancerStatusSubcommandNotRunning()
- throws IOException {
+ public void
testContainerBalancerStatusInfoSubcommandRunningOnStoppedBalancer()
+ throws IOException {
ScmClient scmClient = mock(ScmClient.class);
- when(scmClient.getContainerBalancerStatus()).thenAnswer(invocation ->
false);
+ //test status is running
+ when(scmClient.getContainerBalancerStatusInfo()).thenReturn(
+ ContainerBalancerStatusInfoResponseProto.newBuilder()
+ .setIsRunning(false)
+ .build());
+
+ statusCmd.execute(scmClient);
+ Pattern p = Pattern.compile(
+ "^ContainerBalancer\\sis\\sNot\\sRunning.");
+ Matcher m = p.matcher(outContent.toString(DEFAULT_ENCODING));
+ assertTrue(m.find());
+ }
+
+ @Test
+ void testContainerBalancerStatusSubcommandNotRunning()
+ throws IOException {
+ ScmClient scmClient = mock(ScmClient.class);
+
+ when(scmClient.getContainerBalancerStatusInfo()).thenReturn(
+ ContainerBalancerStatusInfoResponseProto.newBuilder()
+ .setIsRunning(false)
+ .build());
statusCmd.execute(scmClient);
@@ -100,13 +258,13 @@ public class TestContainerBalancerSubCommand {
}
@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());
@@ -114,10 +272,10 @@ public class TestContainerBalancerSubCommand {
@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()
@@ -133,10 +291,10 @@ public class TestContainerBalancerSubCommand {
@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)
diff --git a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot
b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot
index 49679587be..4299afe5f2 100644
--- a/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot
+++ b/hadoop-ozone/dist/src/main/smoketest/balancer/testBalancer.robot
@@ -63,11 +63,43 @@ 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
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 +165,14 @@ Verify Container Balancer for RATIS containers
Run Container Balancer
+ Run Balancer Status
+
+ Run Balancer Verbose Status
+
+ 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
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]