Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

HDDS-5927. Improve defaults in ContainerBalancerConfiguration #2892

Merged
merged 10 commits into from Jan 12, 2022
Expand Up @@ -312,8 +312,8 @@ Map<String, Pair<Boolean, String>> getSafeModeRuleStatuses()
* Start ContainerBalancer.
*/
boolean startContainerBalancer(Optional<Double> threshold,
Optional<Integer> idleiterations,
Optional<Double> maxDatanodesRatioToInvolvePerIteration,
Optional<Integer> iterations,
Optional<Integer> maxDatanodesPercentageToInvolvePerIteration,
Optional<Long> maxSizeToMovePerIterationInGB,
Optional<Long> maxSizeEnteringTargetInGB,
Optional<Long> maxSizeLeavingSourceInGB) throws IOException;
Expand Down
Expand Up @@ -312,8 +312,8 @@ Map<String, Pair<Boolean, String>> getSafeModeRuleStatuses()
* Start ContainerBalancer.
*/
boolean startContainerBalancer(Optional<Double> threshold,
Optional<Integer> idleiterations,
Optional<Double> maxDatanodesRatioToInvolvePerIteration,
Optional<Integer> iterations,
Optional<Integer> maxDatanodesPercentageToInvolvePerIteration,
Optional<Long> maxSizeToMovePerIterationInGB,
Optional<Long> maxSizeEnteringTargetInGB,
Optional<Long> maxSizeLeavingSourceInGB) throws IOException;
Expand Down
Expand Up @@ -737,8 +737,8 @@ public boolean getReplicationManagerStatus() throws IOException {

@Override
public boolean startContainerBalancer(
Optional<Double> threshold, Optional<Integer> idleiterations,
Optional<Double> maxDatanodesRatioToInvolvePerIteration,
Optional<Double> threshold, Optional<Integer> iterations,
Optional<Integer> maxDatanodesPercentageToInvolvePerIteration,
Optional<Long> maxSizeToMovePerIterationInGB,
Optional<Long> maxSizeEnteringTargetInGB,
Optional<Long> maxSizeLeavingSourceInGB) throws IOException{
Expand All @@ -759,22 +759,22 @@ public boolean startContainerBalancer(
"maxSizeToMovePerIterationInGB must be positive.");
builder.setMaxSizeToMovePerIterationInGB(mstm);
}
if (maxDatanodesRatioToInvolvePerIteration.isPresent()) {
double mdti = maxDatanodesRatioToInvolvePerIteration.get();
if (maxDatanodesPercentageToInvolvePerIteration.isPresent()) {
int mdti = maxDatanodesPercentageToInvolvePerIteration.get();
Preconditions.checkState(mdti >= 0,
"maxDatanodesRatioToInvolvePerIteration must be " +
"maxDatanodesPercentageToInvolvePerIteration must be " +
"greater than equal to zero.");
Preconditions.checkState(mdti <= 1,
"maxDatanodesRatioToInvolvePerIteration must be " +
"lesser than equal to one.");
builder.setMaxDatanodesRatioToInvolvePerIteration(mdti);
Preconditions.checkState(mdti <= 100,
"maxDatanodesPercentageToInvolvePerIteration must be " +
"lesser than equal to hundred.");
builder.setMaxDatanodesPercentageToInvolvePerIteration(mdti);
}
if (idleiterations.isPresent()) {
int idi = idleiterations.get();
Preconditions.checkState(idi > 0 || idi == -1,
"idleiterations must be positive or" +
" -1(infinitly run container balancer).");
builder.setIdleiterations(idi);
if (iterations.isPresent()) {
int i = iterations.get();
Preconditions.checkState(i > 0 || i == -1,
"number of iterations must be positive or" +
" -1 (for running container balancer infinitely).");
builder.setIterations(i);
}

if (maxSizeEnteringTargetInGB.isPresent()) {
Expand Down
Expand Up @@ -479,11 +479,14 @@ message GetContainerTokenResponseProto {
message StartContainerBalancerRequestProto {
optional string traceID = 1;
optional double threshold = 2;
optional int32 idleiterations = 3;
optional double maxDatanodesRatioToInvolvePerIteration = 4;
optional int32 idleiterations = 3 [deprecated = true];
optional double maxDatanodesRatioToInvolvePerIteration = 4 [deprecated =
Comment on lines +492 to +493
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

can we just replace them with the new. Compatibility may be not considered for now

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The backwards compatibility goal fails during build if I just replace them.

true];
optional int64 maxSizeToMovePerIterationInGB = 5;
optional int64 maxSizeEnteringTargetInGB = 6;
optional int64 maxSizeLeavingSourceInGB = 7;
optional int32 maxDatanodesPercentageToInvolvePerIteration = 8;
optional int32 iterations = 9;
}

message StartContainerBalancerResponseProto {
Expand Down
Expand Up @@ -20,6 +20,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.DatanodeDetails;
import org.apache.hadoop.hdds.scm.PlacementPolicy;
import org.apache.hadoop.hdds.scm.ScmConfigKeys;
Expand Down Expand Up @@ -73,7 +74,7 @@ public class ContainerBalancer {
private long maxSizeToMovePerIteration;
private int countDatanodesInvolvedPerIteration;
private long sizeMovedPerIteration;
private int idleIteration;
private int iterations;
private List<DatanodeUsageInfo> unBalancedNodes;
private List<DatanodeUsageInfo> overUtilizedNodes;
private List<DatanodeUsageInfo> underUtilizedNodes;
Expand Down Expand Up @@ -124,7 +125,7 @@ public ContainerBalancer(
this.containerManager = containerManager;
this.replicationManager = replicationManager;
this.ozoneConfiguration = ozoneConfiguration;
this.config = new ContainerBalancerConfiguration(ozoneConfiguration);
config = ozoneConfiguration.getObject(ContainerBalancerConfiguration.class);
this.metrics = ContainerBalancerMetrics.create();
this.scmContext = scmContext;

Expand Down Expand Up @@ -155,7 +156,8 @@ public boolean start(ContainerBalancerConfiguration balancerConfiguration) {

balancerRunning = true;
this.config = balancerConfiguration;
this.ozoneConfiguration = config.getOzoneConfiguration();
ozoneConfiguration.setFromObject(balancerConfiguration);
validateConfiguration(config);
LOG.info("Starting Container Balancer...{}", this);

//we should start a new balancer thread async
Expand All @@ -174,23 +176,12 @@ public boolean start(ContainerBalancerConfiguration balancerConfiguration) {
* Balances the cluster.
*/
private void balance() {
this.idleIteration = config.getIdleIteration();
if(this.idleIteration == -1) {
this.iterations = config.getIterations();
if(this.iterations == -1) {
//run balancer infinitely
this.idleIteration = Integer.MAX_VALUE;
this.iterations = Integer.MAX_VALUE;
}
this.threshold = config.getThreshold();
this.maxDatanodesRatioToInvolvePerIteration =
config.getMaxDatanodesRatioToInvolvePerIteration();
this.maxSizeToMovePerIteration = config.getMaxSizeToMovePerIteration();
if (config.getNetworkTopologyEnable()) {
findTargetStrategy = new FindTargetGreedyByNetworkTopology(
containerManager, placementPolicy, nodeManager, networkTopology);
} else {
findTargetStrategy = new FindTargetGreedyByUsageInfo(containerManager,
placementPolicy, nodeManager);
}
for (int i = 0; i < idleIteration && balancerRunning; i++) {
for (int i = 0; i < iterations && balancerRunning; i++) {
// stop balancing if iteration is not initialized
if (!initializeIteration()) {
stop();
Expand All @@ -213,7 +204,7 @@ private void balance() {

// wait for configured time before starting next iteration, unless
// this was the final iteration
if (i != idleIteration - 1) {
if (i != iterations - 1) {
synchronized (this) {
try {
wait(config.getBalancingInterval().toMillis());
Expand Down Expand Up @@ -255,6 +246,17 @@ private boolean initializeIteration() {
}
return false;
}
this.threshold = config.getThresholdAsRatio();
this.maxDatanodesRatioToInvolvePerIteration =
config.getMaxDatanodesRatioToInvolvePerIteration();
this.maxSizeToMovePerIteration = config.getMaxSizeToMovePerIteration();
if (config.getNetworkTopologyEnable()) {
findTargetStrategy = new FindTargetGreedyByNetworkTopology(
containerManager, placementPolicy, nodeManager, networkTopology);
} else {
findTargetStrategy = new FindTargetGreedyByUsageInfo(containerManager,
placementPolicy, nodeManager);
}
this.excludeNodes = config.getExcludeNodes();
this.includeNodes = config.getIncludeNodes();
// include/exclude nodes from balancing according to configs
Expand Down Expand Up @@ -518,7 +520,7 @@ private ContainerMoveSelection matchSourceWithTarget(DatanodeDetails source) {
}

/**
* Checks if limits maxDatanodesRatioToInvolvePerIteration and
* Checks if limits maxDatanodesPercentageToInvolvePerIteration and
* maxSizeToMovePerIteration have not been hit.
*
* @return {@link IterationResult#MAX_DATANODES_TO_INVOLVE_REACHED} if reached
Expand Down Expand Up @@ -764,6 +766,31 @@ public void stop() {
LOG.info("Container Balancer stopped successfully.");
}

private void validateConfiguration(ContainerBalancerConfiguration conf) {
// maxSizeEnteringTarget and maxSizeLeavingSource should by default be
// greater than container size
long size = (long) ozoneConfiguration.getStorageSize(
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE,
ScmConfigKeys.OZONE_SCM_CONTAINER_SIZE_DEFAULT, StorageUnit.BYTES);

if (conf.getMaxSizeEnteringTarget() <= size) {
LOG.info("MaxSizeEnteringTarget should be larger than " +
"ozone.scm.container.size");
}
if (conf.getMaxSizeLeavingSource() <= size) {
LOG.info("MaxSizeLeavingSource should be larger than " +
"ozone.scm.container.size");
}

// balancing interval should be greater than DUFactory refresh period
DUFactory.Conf duConf = ozoneConfiguration.getObject(DUFactory.Conf.class);
long balancingInterval = duConf.getRefreshPeriod().toMillis();
if (conf.getBalancingInterval().toMillis() <= balancingInterval) {
LOG.info("balancing.iteration.interval should be larger than " +
"hdds.datanode.du.refresh.period.");
}
}

public void setNodeManager(NodeManager nodeManager) {
this.nodeManager = nodeManager;
}
Expand Down