|
@@ -19,13 +19,7 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import static org.apache.hadoop.util.Time.now;
|
|
|
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Collection;
|
|
|
-import java.util.EnumSet;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.TreeSet;
|
|
|
+import java.util.*;
|
|
|
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -142,8 +136,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
|
|
|
Set<Node> favoriteAndExcludedNodes = excludedNodes == null ?
|
|
|
new HashSet<Node>() : new HashSet<Node>(excludedNodes);
|
|
|
- final List<StorageType> storageTypes = storagePolicy.chooseStorageTypes(
|
|
|
- (short)numOfReplicas);
|
|
|
+ final List<StorageType> requiredStorageTypes = storagePolicy
|
|
|
+ .chooseStorageTypes((short)numOfReplicas);
|
|
|
+ final EnumMap<StorageType, Integer> storageTypes =
|
|
|
+ getRequiredStorageTypes(requiredStorageTypes);
|
|
|
|
|
|
// Choose favored nodes
|
|
|
List<DatanodeStorageInfo> results = new ArrayList<DatanodeStorageInfo>();
|
|
@@ -156,13 +152,12 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
final DatanodeStorageInfo target = chooseLocalStorage(favoredNode,
|
|
|
favoriteAndExcludedNodes, blocksize,
|
|
|
getMaxNodesPerRack(results.size(), numOfReplicas)[1],
|
|
|
- results, avoidStaleNodes, storageTypes.get(0), false);
|
|
|
+ results, avoidStaleNodes, storageTypes, false);
|
|
|
if (target == null) {
|
|
|
LOG.warn("Could not find a target for file " + src
|
|
|
+ " with favored node " + favoredNode);
|
|
|
continue;
|
|
|
}
|
|
|
- storageTypes.remove(0);
|
|
|
favoriteAndExcludedNodes.add(target.getDatanodeDescriptor());
|
|
|
}
|
|
|
|
|
@@ -241,6 +236,21 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
return new int[] {numOfReplicas, maxNodesPerRack};
|
|
|
}
|
|
|
|
|
|
+ private EnumMap<StorageType, Integer> getRequiredStorageTypes(
|
|
|
+ List<StorageType> types) {
|
|
|
+ EnumMap<StorageType, Integer> map = new EnumMap<StorageType,
|
|
|
+ Integer>(StorageType.class);
|
|
|
+ for (StorageType type : types) {
|
|
|
+ if (!map.containsKey(type)) {
|
|
|
+ map.put(type, 1);
|
|
|
+ } else {
|
|
|
+ int num = map.get(type);
|
|
|
+ map.put(type, num + 1);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return map;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* choose <i>numOfReplicas</i> from all data nodes
|
|
|
* @param numOfReplicas additional number of replicas wanted
|
|
@@ -272,17 +282,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
}
|
|
|
|
|
|
// Keep a copy of original excludedNodes
|
|
|
- final Set<Node> oldExcludedNodes = avoidStaleNodes ?
|
|
|
- new HashSet<Node>(excludedNodes) : null;
|
|
|
+ final Set<Node> oldExcludedNodes = new HashSet<Node>(excludedNodes);
|
|
|
|
|
|
// choose storage types; use fallbacks for unavailable storages
|
|
|
- final List<StorageType> storageTypes = storagePolicy.chooseStorageTypes(
|
|
|
- (short)totalReplicasExpected, DatanodeStorageInfo.toStorageTypes(results),
|
|
|
- unavailableStorages, newBlock);
|
|
|
+ final List<StorageType> requiredStorageTypes = storagePolicy
|
|
|
+ .chooseStorageTypes((short) totalReplicasExpected,
|
|
|
+ DatanodeStorageInfo.toStorageTypes(results),
|
|
|
+ unavailableStorages, newBlock);
|
|
|
+ final EnumMap<StorageType, Integer> storageTypes =
|
|
|
+ getRequiredStorageTypes(requiredStorageTypes);
|
|
|
|
|
|
- StorageType curStorageType = null;
|
|
|
try {
|
|
|
- if ((numOfReplicas = storageTypes.size()) == 0) {
|
|
|
+ if ((numOfReplicas = requiredStorageTypes.size()) == 0) {
|
|
|
throw new NotEnoughReplicasException(
|
|
|
"All required storage types are unavailable: "
|
|
|
+ " unavailableStorages=" + unavailableStorages
|
|
@@ -290,9 +301,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
}
|
|
|
|
|
|
if (numOfResults == 0) {
|
|
|
- curStorageType = storageTypes.remove(0);
|
|
|
writer = chooseLocalStorage(writer, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, curStorageType, true)
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes, true)
|
|
|
.getDatanodeDescriptor();
|
|
|
if (--numOfReplicas == 0) {
|
|
|
return writer;
|
|
@@ -300,33 +310,30 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
}
|
|
|
final DatanodeDescriptor dn0 = results.get(0).getDatanodeDescriptor();
|
|
|
if (numOfResults <= 1) {
|
|
|
- curStorageType = storageTypes.remove(0);
|
|
|
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
|
|
|
- results, avoidStaleNodes, curStorageType);
|
|
|
+ results, avoidStaleNodes, storageTypes);
|
|
|
if (--numOfReplicas == 0) {
|
|
|
return writer;
|
|
|
}
|
|
|
}
|
|
|
if (numOfResults <= 2) {
|
|
|
final DatanodeDescriptor dn1 = results.get(1).getDatanodeDescriptor();
|
|
|
- curStorageType = storageTypes.remove(0);
|
|
|
if (clusterMap.isOnSameRack(dn0, dn1)) {
|
|
|
chooseRemoteRack(1, dn0, excludedNodes, blocksize, maxNodesPerRack,
|
|
|
- results, avoidStaleNodes, curStorageType);
|
|
|
+ results, avoidStaleNodes, storageTypes);
|
|
|
} else if (newBlock){
|
|
|
chooseLocalRack(dn1, excludedNodes, blocksize, maxNodesPerRack,
|
|
|
- results, avoidStaleNodes, curStorageType);
|
|
|
+ results, avoidStaleNodes, storageTypes);
|
|
|
} else {
|
|
|
chooseLocalRack(writer, excludedNodes, blocksize, maxNodesPerRack,
|
|
|
- results, avoidStaleNodes, curStorageType);
|
|
|
+ results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
if (--numOfReplicas == 0) {
|
|
|
return writer;
|
|
|
}
|
|
|
}
|
|
|
- curStorageType = storageTypes.remove(0);
|
|
|
chooseRandom(numOfReplicas, NodeBase.ROOT, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, curStorageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
} catch (NotEnoughReplicasException e) {
|
|
|
final String message = "Failed to place enough replicas, still in need of "
|
|
|
+ (totalReplicasExpected - results.size()) + " to reach "
|
|
@@ -355,10 +362,22 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
newBlock);
|
|
|
}
|
|
|
|
|
|
- if (storageTypes.size() > 0) {
|
|
|
- // Retry chooseTarget with fallback storage types
|
|
|
- unavailableStorages.add(curStorageType);
|
|
|
- return chooseTarget(numOfReplicas, writer, excludedNodes, blocksize,
|
|
|
+ boolean retry = false;
|
|
|
+ // simply add all the remaining types into unavailableStorages and give
|
|
|
+ // another try. No best effort is guaranteed here.
|
|
|
+ for (StorageType type : storageTypes.keySet()) {
|
|
|
+ if (!unavailableStorages.contains(type)) {
|
|
|
+ unavailableStorages.add(type);
|
|
|
+ retry = true;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ if (retry) {
|
|
|
+ for (DatanodeStorageInfo resultStorage : results) {
|
|
|
+ addToExcludedNodes(resultStorage.getDatanodeDescriptor(),
|
|
|
+ oldExcludedNodes);
|
|
|
+ }
|
|
|
+ numOfReplicas = totalReplicasExpected - results.size();
|
|
|
+ return chooseTarget(numOfReplicas, writer, oldExcludedNodes, blocksize,
|
|
|
maxNodesPerRack, results, false, storagePolicy, unavailableStorages,
|
|
|
newBlock);
|
|
|
}
|
|
@@ -373,28 +392,35 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
* @return the chosen storage
|
|
|
*/
|
|
|
protected DatanodeStorageInfo chooseLocalStorage(Node localMachine,
|
|
|
- Set<Node> excludedNodes,
|
|
|
- long blocksize,
|
|
|
- int maxNodesPerRack,
|
|
|
- List<DatanodeStorageInfo> results,
|
|
|
- boolean avoidStaleNodes,
|
|
|
- StorageType storageType,
|
|
|
- boolean fallbackToLocalRack)
|
|
|
+ Set<Node> excludedNodes, long blocksize, int maxNodesPerRack,
|
|
|
+ List<DatanodeStorageInfo> results, boolean avoidStaleNodes,
|
|
|
+ EnumMap<StorageType, Integer> storageTypes, boolean fallbackToLocalRack)
|
|
|
throws NotEnoughReplicasException {
|
|
|
// if no local machine, randomly choose one node
|
|
|
if (localMachine == null) {
|
|
|
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
if (preferLocalNode && localMachine instanceof DatanodeDescriptor) {
|
|
|
DatanodeDescriptor localDatanode = (DatanodeDescriptor) localMachine;
|
|
|
// otherwise try local machine first
|
|
|
if (excludedNodes.add(localMachine)) { // was not in the excluded list
|
|
|
- for(DatanodeStorageInfo localStorage : DFSUtil.shuffle(
|
|
|
- localDatanode.getStorageInfos())) {
|
|
|
- if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, false, results, avoidStaleNodes, storageType) >= 0) {
|
|
|
- return localStorage;
|
|
|
+ for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
|
|
|
+ .entrySet().iterator(); iter.hasNext(); ) {
|
|
|
+ Map.Entry<StorageType, Integer> entry = iter.next();
|
|
|
+ for (DatanodeStorageInfo localStorage : DFSUtil.shuffle(
|
|
|
+ localDatanode.getStorageInfos())) {
|
|
|
+ StorageType type = entry.getKey();
|
|
|
+ if (addIfIsGoodTarget(localStorage, excludedNodes, blocksize,
|
|
|
+ maxNodesPerRack, false, results, avoidStaleNodes, type) >= 0) {
|
|
|
+ int num = entry.getValue();
|
|
|
+ if (num == 1) {
|
|
|
+ iter.remove();
|
|
|
+ } else {
|
|
|
+ entry.setValue(num - 1);
|
|
|
+ }
|
|
|
+ return localStorage;
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -405,7 +431,7 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
}
|
|
|
// try a node on local rack
|
|
|
return chooseLocalRack(localMachine, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -428,23 +454,23 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
* @return the chosen node
|
|
|
*/
|
|
|
protected DatanodeStorageInfo chooseLocalRack(Node localMachine,
|
|
|
- Set<Node> excludedNodes,
|
|
|
- long blocksize,
|
|
|
- int maxNodesPerRack,
|
|
|
- List<DatanodeStorageInfo> results,
|
|
|
- boolean avoidStaleNodes,
|
|
|
- StorageType storageType)
|
|
|
+ Set<Node> excludedNodes,
|
|
|
+ long blocksize,
|
|
|
+ int maxNodesPerRack,
|
|
|
+ List<DatanodeStorageInfo> results,
|
|
|
+ boolean avoidStaleNodes,
|
|
|
+ EnumMap<StorageType, Integer> storageTypes)
|
|
|
throws NotEnoughReplicasException {
|
|
|
// no local machine, so choose a random machine
|
|
|
if (localMachine == null) {
|
|
|
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
|
|
|
// choose one from the local rack
|
|
|
try {
|
|
|
return chooseRandom(localMachine.getNetworkLocation(), excludedNodes,
|
|
|
- blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ blocksize, maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
} catch (NotEnoughReplicasException e1) {
|
|
|
// find the second replica
|
|
|
DatanodeDescriptor newLocal=null;
|
|
@@ -458,16 +484,17 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
if (newLocal != null) {
|
|
|
try {
|
|
|
return chooseRandom(newLocal.getNetworkLocation(), excludedNodes,
|
|
|
- blocksize, maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ blocksize, maxNodesPerRack, results, avoidStaleNodes,
|
|
|
+ storageTypes);
|
|
|
} catch(NotEnoughReplicasException e2) {
|
|
|
//otherwise randomly choose one from the network
|
|
|
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
} else {
|
|
|
//otherwise randomly choose one from the network
|
|
|
return chooseRandom(NodeBase.ROOT, excludedNodes, blocksize,
|
|
|
- maxNodesPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxNodesPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -486,18 +513,18 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
int maxReplicasPerRack,
|
|
|
List<DatanodeStorageInfo> results,
|
|
|
boolean avoidStaleNodes,
|
|
|
- StorageType storageType)
|
|
|
+ EnumMap<StorageType, Integer> storageTypes)
|
|
|
throws NotEnoughReplicasException {
|
|
|
int oldNumOfReplicas = results.size();
|
|
|
// randomly choose one node from remote racks
|
|
|
try {
|
|
|
chooseRandom(numOfReplicas, "~" + localMachine.getNetworkLocation(),
|
|
|
excludedNodes, blocksize, maxReplicasPerRack, results,
|
|
|
- avoidStaleNodes, storageType);
|
|
|
+ avoidStaleNodes, storageTypes);
|
|
|
} catch (NotEnoughReplicasException e) {
|
|
|
chooseRandom(numOfReplicas-(results.size()-oldNumOfReplicas),
|
|
|
localMachine.getNetworkLocation(), excludedNodes, blocksize,
|
|
|
- maxReplicasPerRack, results, avoidStaleNodes, storageType);
|
|
|
+ maxReplicasPerRack, results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -511,10 +538,10 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
int maxNodesPerRack,
|
|
|
List<DatanodeStorageInfo> results,
|
|
|
boolean avoidStaleNodes,
|
|
|
- StorageType storageType)
|
|
|
+ EnumMap<StorageType, Integer> storageTypes)
|
|
|
throws NotEnoughReplicasException {
|
|
|
return chooseRandom(1, scope, excludedNodes, blocksize, maxNodesPerRack,
|
|
|
- results, avoidStaleNodes, storageType);
|
|
|
+ results, avoidStaleNodes, storageTypes);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -528,8 +555,8 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
int maxNodesPerRack,
|
|
|
List<DatanodeStorageInfo> results,
|
|
|
boolean avoidStaleNodes,
|
|
|
- StorageType storageType)
|
|
|
- throws NotEnoughReplicasException {
|
|
|
+ EnumMap<StorageType, Integer> storageTypes)
|
|
|
+ throws NotEnoughReplicasException {
|
|
|
|
|
|
int numOfAvailableNodes = clusterMap.countNumOfAvailableNodes(
|
|
|
scope, excludedNodes);
|
|
@@ -549,18 +576,31 @@ public class BlockPlacementPolicyDefault extends BlockPlacementPolicy {
|
|
|
|
|
|
final DatanodeStorageInfo[] storages = DFSUtil.shuffle(
|
|
|
chosenNode.getStorageInfos());
|
|
|
- int i;
|
|
|
- for(i = 0; i < storages.length; i++) {
|
|
|
- final int newExcludedNodes = addIfIsGoodTarget(storages[i],
|
|
|
- excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
|
|
|
- avoidStaleNodes, storageType);
|
|
|
- if (newExcludedNodes >= 0) {
|
|
|
- numOfReplicas--;
|
|
|
- if (firstChosen == null) {
|
|
|
- firstChosen = storages[i];
|
|
|
+ int i = 0;
|
|
|
+ boolean search = true;
|
|
|
+ for (Iterator<Map.Entry<StorageType, Integer>> iter = storageTypes
|
|
|
+ .entrySet().iterator(); search && iter.hasNext(); ) {
|
|
|
+ Map.Entry<StorageType, Integer> entry = iter.next();
|
|
|
+ for (i = 0; i < storages.length; i++) {
|
|
|
+ StorageType type = entry.getKey();
|
|
|
+ final int newExcludedNodes = addIfIsGoodTarget(storages[i],
|
|
|
+ excludedNodes, blocksize, maxNodesPerRack, considerLoad, results,
|
|
|
+ avoidStaleNodes, type);
|
|
|
+ if (newExcludedNodes >= 0) {
|
|
|
+ numOfReplicas--;
|
|
|
+ if (firstChosen == null) {
|
|
|
+ firstChosen = storages[i];
|
|
|
+ }
|
|
|
+ numOfAvailableNodes -= newExcludedNodes;
|
|
|
+ int num = entry.getValue();
|
|
|
+ if (num == 1) {
|
|
|
+ iter.remove();
|
|
|
+ } else {
|
|
|
+ entry.setValue(num - 1);
|
|
|
+ }
|
|
|
+ search = false;
|
|
|
+ break;
|
|
|
}
|
|
|
- numOfAvailableNodes -= newExcludedNodes;
|
|
|
- break;
|
|
|
}
|
|
|
}
|
|
|
|