|
@@ -189,7 +189,6 @@ public class Balancer {
|
|
|
/** The maximum number of concurrent blocks moves for
|
|
|
* balancing purpose at a datanode
|
|
|
*/
|
|
|
- public static final int MAX_NUM_CONCURRENT_MOVES = 5;
|
|
|
private static final int MAX_NO_PENDING_BLOCK_ITERATIONS = 5;
|
|
|
public static final long DELAY_AFTER_ERROR = 10 * 1000L; //10 seconds
|
|
|
public static final int BLOCK_MOVE_READ_TIMEOUT=20*60*1000; // 20 minutes
|
|
@@ -231,6 +230,7 @@ public class Balancer {
|
|
|
|
|
|
private final ExecutorService moverExecutor;
|
|
|
private final ExecutorService dispatcherExecutor;
|
|
|
+ private final int maxConcurrentMovesPerNode;
|
|
|
|
|
|
/* This class keeps track of a scheduled block move */
|
|
|
private class PendingBlockMove {
|
|
@@ -516,8 +516,8 @@ public class Balancer {
|
|
|
private long scheduledSize = 0L;
|
|
|
protected long delayUntil = 0L;
|
|
|
// blocks being moved but not confirmed yet
|
|
|
- private final List<PendingBlockMove> pendingBlocks =
|
|
|
- new ArrayList<PendingBlockMove>(MAX_NUM_CONCURRENT_MOVES);
|
|
|
+ private final List<PendingBlockMove> pendingBlocks;
|
|
|
+ private final int maxConcurrentMoves;
|
|
|
|
|
|
@Override
|
|
|
public String toString() {
|
|
@@ -528,7 +528,8 @@ public class Balancer {
|
|
|
/* Constructor
|
|
|
* Depending on avgutil & threshold, calculate maximum bytes to move
|
|
|
*/
|
|
|
- private BalancerDatanode(DatanodeInfo node, BalancingPolicy policy, double threshold) {
|
|
|
+ private BalancerDatanode(DatanodeInfo node, BalancingPolicy policy, double threshold,
|
|
|
+ int maxConcurrentMoves) {
|
|
|
datanode = node;
|
|
|
utilization = policy.getUtilization(node);
|
|
|
final double avgUtil = policy.getAvgUtilization();
|
|
@@ -545,6 +546,8 @@ public class Balancer {
|
|
|
maxSizeToMove = Math.min(datanode.getRemaining(), maxSizeToMove);
|
|
|
}
|
|
|
this.maxSize2Move = Math.min(MAX_SIZE_TO_MOVE, maxSizeToMove);
|
|
|
+ this.maxConcurrentMoves = maxConcurrentMoves;
|
|
|
+ this.pendingBlocks = new ArrayList<PendingBlockMove>(maxConcurrentMoves);
|
|
|
}
|
|
|
|
|
|
/** Get the datanode */
|
|
@@ -606,7 +609,7 @@ public class Balancer {
|
|
|
|
|
|
/* Check if the node can schedule more blocks to move */
|
|
|
synchronized private boolean isPendingQNotFull() {
|
|
|
- if ( pendingBlocks.size() < MAX_NUM_CONCURRENT_MOVES ) {
|
|
|
+ if ( pendingBlocks.size() < this.maxConcurrentMoves ) {
|
|
|
return true;
|
|
|
}
|
|
|
return false;
|
|
@@ -655,8 +658,9 @@ public class Balancer {
|
|
|
= new ArrayList<BalancerBlock>();
|
|
|
|
|
|
/* constructor */
|
|
|
- private Source(DatanodeInfo node, BalancingPolicy policy, double threshold) {
|
|
|
- super(node, policy, threshold);
|
|
|
+ private Source(DatanodeInfo node, BalancingPolicy policy, double threshold,
|
|
|
+ int maxConcurrentMoves) {
|
|
|
+ super(node, policy, threshold, maxConcurrentMoves);
|
|
|
}
|
|
|
|
|
|
/** Add a node task */
|
|
@@ -869,6 +873,9 @@ public class Balancer {
|
|
|
this.dispatcherExecutor = Executors.newFixedThreadPool(
|
|
|
conf.getInt(DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_KEY,
|
|
|
DFSConfigKeys.DFS_BALANCER_DISPATCHERTHREADS_DEFAULT));
|
|
|
+ this.maxConcurrentMovesPerNode =
|
|
|
+ conf.getInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY,
|
|
|
+ DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_DEFAULT);
|
|
|
}
|
|
|
|
|
|
/* Given a data node set, build a network topology and decide
|
|
@@ -908,7 +915,7 @@ public class Balancer {
|
|
|
BalancerDatanode datanodeS;
|
|
|
final double avg = policy.getAvgUtilization();
|
|
|
if (policy.getUtilization(datanode) > avg) {
|
|
|
- datanodeS = new Source(datanode, policy, threshold);
|
|
|
+ datanodeS = new Source(datanode, policy, threshold, maxConcurrentMovesPerNode);
|
|
|
if (isAboveAvgUtilized(datanodeS)) {
|
|
|
this.aboveAvgUtilizedDatanodes.add((Source)datanodeS);
|
|
|
} else {
|
|
@@ -919,7 +926,8 @@ public class Balancer {
|
|
|
-threshold)*datanodeS.datanode.getCapacity()/100.0);
|
|
|
}
|
|
|
} else {
|
|
|
- datanodeS = new BalancerDatanode(datanode, policy, threshold);
|
|
|
+ datanodeS = new BalancerDatanode(datanode, policy, threshold,
|
|
|
+ maxConcurrentMovesPerNode);
|
|
|
if ( isBelowOrEqualAvgUtilized(datanodeS)) {
|
|
|
this.belowAvgUtilizedDatanodes.add(datanodeS);
|
|
|
} else {
|