|
@@ -156,10 +156,16 @@ public class BlockManager {
|
|
|
|
|
|
/** The maximum number of replicas allowed for a block */
|
|
|
public final short maxReplication;
|
|
|
- /** The maximum number of outgoing replication streams
|
|
|
- * a given node should have at one time
|
|
|
+ /**
|
|
|
+ * The maximum number of outgoing replication streams a given node should have
|
|
|
+ * at one time considering all but the highest priority replications needed.
|
|
|
*/
|
|
|
int maxReplicationStreams;
|
|
|
+ /**
|
|
|
+ * The maximum number of outgoing replication streams a given node should have
|
|
|
+ * at one time.
|
|
|
+ */
|
|
|
+ int replicationStreamsHardLimit;
|
|
|
/** Minimum copies needed or else write is disallowed */
|
|
|
public final short minReplication;
|
|
|
/** Default number of replicas */
|
|
@@ -219,10 +225,16 @@ public class BlockManager {
|
|
|
this.minReplication = (short)minR;
|
|
|
this.maxReplication = (short)maxR;
|
|
|
|
|
|
- this.maxReplicationStreams = conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
|
|
|
- DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
|
|
|
- this.shouldCheckForEnoughRacks = conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null ? false
|
|
|
- : true;
|
|
|
+ this.maxReplicationStreams =
|
|
|
+ conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY,
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT);
|
|
|
+ this.replicationStreamsHardLimit =
|
|
|
+ conf.getInt(
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_KEY,
|
|
|
+ DFSConfigKeys.DFS_NAMENODE_REPLICATION_STREAMS_HARD_LIMIT_DEFAULT);
|
|
|
+ this.shouldCheckForEnoughRacks =
|
|
|
+ conf.get(DFSConfigKeys.NET_TOPOLOGY_SCRIPT_FILE_NAME_KEY) == null
|
|
|
+ ? false : true;
|
|
|
|
|
|
this.replicationRecheckInterval =
|
|
|
conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY,
|
|
@@ -329,7 +341,8 @@ public class BlockManager {
|
|
|
NumberReplicas numReplicas = new NumberReplicas();
|
|
|
// source node returned is not used
|
|
|
chooseSourceDatanode(block, containingNodes,
|
|
|
- containingLiveReplicasNodes, numReplicas);
|
|
|
+ containingLiveReplicasNodes, numReplicas,
|
|
|
+ UnderReplicatedBlocks.LEVEL);
|
|
|
assert containingLiveReplicasNodes.size() == numReplicas.liveReplicas();
|
|
|
int usableReplicas = numReplicas.liveReplicas() +
|
|
|
numReplicas.decommissionedReplicas();
|
|
@@ -1047,9 +1060,11 @@ public class BlockManager {
|
|
|
liveReplicaNodes = new ArrayList<DatanodeDescriptor>();
|
|
|
NumberReplicas numReplicas = new NumberReplicas();
|
|
|
srcNode = chooseSourceDatanode(
|
|
|
- block, containingNodes, liveReplicaNodes, numReplicas);
|
|
|
+ block, containingNodes, liveReplicaNodes, numReplicas, priority);
|
|
|
if(srcNode == null) // block can not be replicated from any node
|
|
|
+ {
|
|
|
return false;
|
|
|
+ }
|
|
|
|
|
|
assert liveReplicaNodes.size() == numReplicas.liveReplicas();
|
|
|
// do not schedule more if enough replicas is already pending
|
|
@@ -1212,16 +1227,34 @@ public class BlockManager {
|
|
|
* since the former do not have write traffic and hence are less busy.
|
|
|
* We do not use already decommissioned nodes as a source.
|
|
|
* Otherwise we choose a random node among those that did not reach their
|
|
|
- * replication limit.
|
|
|
+ * replication limits. However, if the replication is of the highest priority
|
|
|
+ * and all nodes have reached their replication limits, we will choose a
|
|
|
+ * random node despite the replication limit.
|
|
|
*
|
|
|
* In addition form a list of all nodes containing the block
|
|
|
* and calculate its replication numbers.
|
|
|
+ *
|
|
|
+ * @param block Block for which a replication source is needed
|
|
|
+ * @param containingNodes List to be populated with nodes found to contain the
|
|
|
+ * given block
|
|
|
+ * @param nodesContainingLiveReplicas List to be populated with nodes found to
|
|
|
+ * contain live replicas of the given block
|
|
|
+ * @param numReplicas NumberReplicas instance to be initialized with the
|
|
|
+ * counts of live, corrupt, excess, and
|
|
|
+ * decommissioned replicas of the given
|
|
|
+ * block.
|
|
|
+ * @param priority integer representing replication priority of the given
|
|
|
+ * block
|
|
|
+ * @return the DatanodeDescriptor of the chosen node from which to replicate
|
|
|
+ * the given block
|
|
|
*/
|
|
|
- private DatanodeDescriptor chooseSourceDatanode(
|
|
|
+ @VisibleForTesting
|
|
|
+ DatanodeDescriptor chooseSourceDatanode(
|
|
|
Block block,
|
|
|
List<DatanodeDescriptor> containingNodes,
|
|
|
List<DatanodeDescriptor> nodesContainingLiveReplicas,
|
|
|
- NumberReplicas numReplicas) {
|
|
|
+ NumberReplicas numReplicas,
|
|
|
+ int priority) {
|
|
|
containingNodes.clear();
|
|
|
nodesContainingLiveReplicas.clear();
|
|
|
DatanodeDescriptor srcNode = null;
|
|
@@ -1250,8 +1283,15 @@ public class BlockManager {
|
|
|
// If so, do not select the node as src node
|
|
|
if ((nodesCorrupt != null) && nodesCorrupt.contains(node))
|
|
|
continue;
|
|
|
- if(node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
|
|
|
+ if(priority != UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY
|
|
|
+ && node.getNumberOfBlocksToBeReplicated() >= maxReplicationStreams)
|
|
|
+ {
|
|
|
continue; // already reached replication limit
|
|
|
+ }
|
|
|
+ if (node.getNumberOfBlocksToBeReplicated() >= replicationStreamsHardLimit)
|
|
|
+ {
|
|
|
+ continue;
|
|
|
+ }
|
|
|
// the block must not be scheduled for removal on srcNode
|
|
|
if(excessBlocks != null && excessBlocks.contains(block))
|
|
|
continue;
|