|
@@ -19,11 +19,12 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
|
|
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
import java.util.ArrayList;
|
|
import java.util.ArrayList;
|
|
|
|
+import java.util.Collections;
|
|
import java.util.HashSet;
|
|
import java.util.HashSet;
|
|
import java.util.Iterator;
|
|
import java.util.Iterator;
|
|
import java.util.List;
|
|
import java.util.List;
|
|
-import java.util.Map;
|
|
|
|
import java.util.NavigableMap;
|
|
import java.util.NavigableMap;
|
|
|
|
+import java.util.Random;
|
|
import java.util.Set;
|
|
import java.util.Set;
|
|
import java.util.UUID;
|
|
import java.util.UUID;
|
|
import java.util.concurrent.ConcurrentSkipListMap;
|
|
import java.util.concurrent.ConcurrentSkipListMap;
|
|
@@ -229,11 +230,8 @@ public class ProvidedStorageMap {
|
|
sids.add(currInfo.getStorageID());
|
|
sids.add(currInfo.getStorageID());
|
|
types.add(storageType);
|
|
types.add(storageType);
|
|
if (StorageType.PROVIDED.equals(storageType)) {
|
|
if (StorageType.PROVIDED.equals(storageType)) {
|
|
- DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
|
|
|
|
- locs.add(
|
|
|
|
- new DatanodeInfoWithStorage(
|
|
|
|
- dn, currInfo.getStorageID(), currInfo.getStorageType()));
|
|
|
|
- excludedUUids.add(dn.getDatanodeUuid());
|
|
|
|
|
|
+ // Provided location will be added to the list of locations after
|
|
|
|
+ // examining all local locations.
|
|
isProvidedBlock = true;
|
|
isProvidedBlock = true;
|
|
} else {
|
|
} else {
|
|
locs.add(new DatanodeInfoWithStorage(
|
|
locs.add(new DatanodeInfoWithStorage(
|
|
@@ -245,11 +243,17 @@ public class ProvidedStorageMap {
|
|
|
|
|
|
int numLocations = locs.size();
|
|
int numLocations = locs.size();
|
|
if (isProvidedBlock) {
|
|
if (isProvidedBlock) {
|
|
|
|
+ // add the first datanode here
|
|
|
|
+ DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
|
|
|
|
+ locs.add(
|
|
|
|
+ new DatanodeInfoWithStorage(dn, storageId, StorageType.PROVIDED));
|
|
|
|
+ excludedUUids.add(dn.getDatanodeUuid());
|
|
|
|
+ numLocations++;
|
|
// add more replicas until we reach the defaultReplication
|
|
// add more replicas until we reach the defaultReplication
|
|
for (int count = numLocations + 1;
|
|
for (int count = numLocations + 1;
|
|
count <= defaultReplication && count <= providedDescriptor
|
|
count <= defaultReplication && count <= providedDescriptor
|
|
.activeProvidedDatanodes(); count++) {
|
|
.activeProvidedDatanodes(); count++) {
|
|
- DatanodeDescriptor dn = chooseProvidedDatanode(excludedUUids);
|
|
|
|
|
|
+ dn = chooseProvidedDatanode(excludedUUids);
|
|
locs.add(new DatanodeInfoWithStorage(
|
|
locs.add(new DatanodeInfoWithStorage(
|
|
dn, storageId, StorageType.PROVIDED));
|
|
dn, storageId, StorageType.PROVIDED));
|
|
sids.add(storageId);
|
|
sids.add(storageId);
|
|
@@ -284,6 +288,9 @@ public class ProvidedStorageMap {
|
|
|
|
|
|
private final NavigableMap<String, DatanodeDescriptor> dns =
|
|
private final NavigableMap<String, DatanodeDescriptor> dns =
|
|
new ConcurrentSkipListMap<>();
|
|
new ConcurrentSkipListMap<>();
|
|
|
|
+ // maintain a separate list of the datanodes with provided storage
|
|
|
|
+ // to efficiently choose Datanodes when required.
|
|
|
|
+ private final List<DatanodeDescriptor> dnR = new ArrayList<>();
|
|
public final static String NETWORK_LOCATION = "/REMOTE";
|
|
public final static String NETWORK_LOCATION = "/REMOTE";
|
|
public final static String NAME = "PROVIDED";
|
|
public final static String NAME = "PROVIDED";
|
|
|
|
|
|
@@ -300,8 +307,8 @@ public class ProvidedStorageMap {
|
|
|
|
|
|
DatanodeStorageInfo getProvidedStorage(
|
|
DatanodeStorageInfo getProvidedStorage(
|
|
DatanodeDescriptor dn, DatanodeStorage s) {
|
|
DatanodeDescriptor dn, DatanodeStorage s) {
|
|
- LOG.info("XXXXX adding Datanode " + dn.getDatanodeUuid());
|
|
|
|
dns.put(dn.getDatanodeUuid(), dn);
|
|
dns.put(dn.getDatanodeUuid(), dn);
|
|
|
|
+ dnR.add(dn);
|
|
// TODO: maintain separate RPC ident per dn
|
|
// TODO: maintain separate RPC ident per dn
|
|
return storageMap.get(s.getStorageID());
|
|
return storageMap.get(s.getStorageID());
|
|
}
|
|
}
|
|
@@ -315,84 +322,42 @@ public class ProvidedStorageMap {
|
|
}
|
|
}
|
|
|
|
|
|
DatanodeDescriptor choose(DatanodeDescriptor client) {
|
|
DatanodeDescriptor choose(DatanodeDescriptor client) {
|
|
- // exact match for now
|
|
|
|
- DatanodeDescriptor dn = client != null ?
|
|
|
|
- dns.get(client.getDatanodeUuid()) : null;
|
|
|
|
- if (null == dn) {
|
|
|
|
- dn = chooseRandom();
|
|
|
|
- }
|
|
|
|
- return dn;
|
|
|
|
|
|
+ return choose(client, Collections.<String>emptySet());
|
|
}
|
|
}
|
|
|
|
|
|
DatanodeDescriptor choose(DatanodeDescriptor client,
|
|
DatanodeDescriptor choose(DatanodeDescriptor client,
|
|
Set<String> excludedUUids) {
|
|
Set<String> excludedUUids) {
|
|
// exact match for now
|
|
// exact match for now
|
|
- DatanodeDescriptor dn = client != null ?
|
|
|
|
- dns.get(client.getDatanodeUuid()) : null;
|
|
|
|
-
|
|
|
|
- if (null == dn || excludedUUids.contains(client.getDatanodeUuid())) {
|
|
|
|
- dn = null;
|
|
|
|
- Set<String> exploredUUids = new HashSet<String>();
|
|
|
|
-
|
|
|
|
- while(exploredUUids.size() < dns.size()) {
|
|
|
|
- Map.Entry<String, DatanodeDescriptor> d =
|
|
|
|
- dns.ceilingEntry(UUID.randomUUID().toString());
|
|
|
|
- if (null == d) {
|
|
|
|
- d = dns.firstEntry();
|
|
|
|
- }
|
|
|
|
- String uuid = d.getValue().getDatanodeUuid();
|
|
|
|
- //this node has already been explored, and was not selected earlier
|
|
|
|
- if (exploredUUids.contains(uuid)) {
|
|
|
|
- continue;
|
|
|
|
- }
|
|
|
|
- exploredUUids.add(uuid);
|
|
|
|
- //this node has been excluded
|
|
|
|
- if (excludedUUids.contains(uuid)) {
|
|
|
|
- continue;
|
|
|
|
- }
|
|
|
|
- return dns.get(uuid);
|
|
|
|
- }
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- return dn;
|
|
|
|
- }
|
|
|
|
-
|
|
|
|
- DatanodeDescriptor chooseRandom(DatanodeStorageInfo[] excludedStorages) {
|
|
|
|
- // TODO: Currently this is not uniformly random;
|
|
|
|
- // skewed toward sparse sections of the ids
|
|
|
|
- Set<DatanodeDescriptor> excludedNodes =
|
|
|
|
- new HashSet<DatanodeDescriptor>();
|
|
|
|
- if (excludedStorages != null) {
|
|
|
|
- for (int i= 0; i < excludedStorages.length; i++) {
|
|
|
|
- LOG.info("Excluded: " + excludedStorages[i].getDatanodeDescriptor());
|
|
|
|
- excludedNodes.add(excludedStorages[i].getDatanodeDescriptor());
|
|
|
|
|
|
+ if (client != null && !excludedUUids.contains(client.getDatanodeUuid())) {
|
|
|
|
+ DatanodeDescriptor dn = dns.get(client.getDatanodeUuid());
|
|
|
|
+ if (dn != null) {
|
|
|
|
+ return dn;
|
|
}
|
|
}
|
|
}
|
|
}
|
|
- Set<DatanodeDescriptor> exploredNodes = new HashSet<DatanodeDescriptor>();
|
|
|
|
|
|
|
|
- while(exploredNodes.size() < dns.size()) {
|
|
|
|
- Map.Entry<String, DatanodeDescriptor> d =
|
|
|
|
- dns.ceilingEntry(UUID.randomUUID().toString());
|
|
|
|
- if (null == d) {
|
|
|
|
- d = dns.firstEntry();
|
|
|
|
- }
|
|
|
|
- DatanodeDescriptor node = d.getValue();
|
|
|
|
- //this node has already been explored, and was not selected earlier
|
|
|
|
- if (exploredNodes.contains(node)) {
|
|
|
|
- continue;
|
|
|
|
|
|
+ Random r = new Random();
|
|
|
|
+ for (int i = dnR.size() - 1; i >= 0; --i) {
|
|
|
|
+ int pos = r.nextInt(i + 1);
|
|
|
|
+ DatanodeDescriptor node = dnR.get(pos);
|
|
|
|
+ String uuid = node.getDatanodeUuid();
|
|
|
|
+ if (!excludedUUids.contains(uuid)) {
|
|
|
|
+ return node;
|
|
}
|
|
}
|
|
- exploredNodes.add(node);
|
|
|
|
- //this node has been excluded
|
|
|
|
- if (excludedNodes.contains(node)) {
|
|
|
|
- continue;
|
|
|
|
- }
|
|
|
|
- return node;
|
|
|
|
|
|
+ Collections.swap(dnR, i, pos);
|
|
}
|
|
}
|
|
return null;
|
|
return null;
|
|
}
|
|
}
|
|
|
|
|
|
- DatanodeDescriptor chooseRandom() {
|
|
|
|
- return chooseRandom(null);
|
|
|
|
|
|
+ DatanodeDescriptor chooseRandom(DatanodeStorageInfo... excludedStorages) {
|
|
|
|
+ Set<String> excludedNodes = new HashSet<>();
|
|
|
|
+ if (excludedStorages != null) {
|
|
|
|
+ for (int i = 0; i < excludedStorages.length; i++) {
|
|
|
|
+ DatanodeDescriptor dn = excludedStorages[i].getDatanodeDescriptor();
|
|
|
|
+ String uuid = dn.getDatanodeUuid();
|
|
|
|
+ excludedNodes.add(uuid);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+ return choose(null, excludedNodes);
|
|
}
|
|
}
|
|
|
|
|
|
@Override
|
|
@Override
|
|
@@ -414,6 +379,7 @@ public class ProvidedStorageMap {
|
|
DatanodeDescriptor storedDN = dns.get(dnToRemove.getDatanodeUuid());
|
|
DatanodeDescriptor storedDN = dns.get(dnToRemove.getDatanodeUuid());
|
|
if (storedDN != null) {
|
|
if (storedDN != null) {
|
|
dns.remove(dnToRemove.getDatanodeUuid());
|
|
dns.remove(dnToRemove.getDatanodeUuid());
|
|
|
|
+ dnR.remove(dnToRemove);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
return dns.size();
|
|
return dns.size();
|