|
@@ -49,6 +49,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.conf.Configuration;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
import org.apache.hadoop.hdfs.DFSUtil;
|
|
|
|
+import org.apache.hadoop.hdfs.DistributedFileSystem;
|
|
import org.apache.hadoop.hdfs.StorageType;
|
|
import org.apache.hadoop.hdfs.StorageType;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.Block;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
@@ -103,7 +104,8 @@ public class Dispatcher {
|
|
private final MovedBlocks<StorageGroup> movedBlocks;
|
|
private final MovedBlocks<StorageGroup> movedBlocks;
|
|
|
|
|
|
/** Map (datanodeUuid,storageType -> StorageGroup) */
|
|
/** Map (datanodeUuid,storageType -> StorageGroup) */
|
|
- private final StorageGroupMap storageGroupMap = new StorageGroupMap();
|
|
|
|
|
|
+ private final StorageGroupMap<StorageGroup> storageGroupMap
|
|
|
|
+ = new StorageGroupMap<StorageGroup>();
|
|
|
|
|
|
private NetworkTopology cluster;
|
|
private NetworkTopology cluster;
|
|
|
|
|
|
@@ -140,18 +142,18 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
- static class StorageGroupMap {
|
|
|
|
|
|
+ public static class StorageGroupMap<G extends StorageGroup> {
|
|
private static String toKey(String datanodeUuid, StorageType storageType) {
|
|
private static String toKey(String datanodeUuid, StorageType storageType) {
|
|
return datanodeUuid + ":" + storageType;
|
|
return datanodeUuid + ":" + storageType;
|
|
}
|
|
}
|
|
|
|
|
|
- private final Map<String, StorageGroup> map = new HashMap<String, StorageGroup>();
|
|
|
|
|
|
+ private final Map<String, G> map = new HashMap<String, G>();
|
|
|
|
|
|
- StorageGroup get(String datanodeUuid, StorageType storageType) {
|
|
|
|
|
|
+ public G get(String datanodeUuid, StorageType storageType) {
|
|
return map.get(toKey(datanodeUuid, storageType));
|
|
return map.get(toKey(datanodeUuid, storageType));
|
|
}
|
|
}
|
|
|
|
|
|
- void put(StorageGroup g) {
|
|
|
|
|
|
+ public void put(G g) {
|
|
final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType);
|
|
final String key = toKey(g.getDatanodeInfo().getDatanodeUuid(), g.storageType);
|
|
final StorageGroup existing = map.put(key, g);
|
|
final StorageGroup existing = map.put(key, g);
|
|
Preconditions.checkState(existing == null);
|
|
Preconditions.checkState(existing == null);
|
|
@@ -167,7 +169,7 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** This class keeps track of a scheduled block move */
|
|
/** This class keeps track of a scheduled block move */
|
|
- private class PendingMove {
|
|
|
|
|
|
+ public class PendingMove {
|
|
private DBlock block;
|
|
private DBlock block;
|
|
private Source source;
|
|
private Source source;
|
|
private DDatanode proxySource;
|
|
private DDatanode proxySource;
|
|
@@ -176,6 +178,12 @@ public class Dispatcher {
|
|
private PendingMove() {
|
|
private PendingMove() {
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ public PendingMove(DBlock block, Source source, StorageGroup target) {
|
|
|
|
+ this.block = block;
|
|
|
|
+ this.source = source;
|
|
|
|
+ this.target = target;
|
|
|
|
+ }
|
|
|
|
+
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
final Block b = block.getBlock();
|
|
final Block b = block.getBlock();
|
|
@@ -227,7 +235,7 @@ public class Dispatcher {
|
|
*
|
|
*
|
|
* @return true if a proxy is found; otherwise false
|
|
* @return true if a proxy is found; otherwise false
|
|
*/
|
|
*/
|
|
- private boolean chooseProxySource() {
|
|
|
|
|
|
+ public boolean chooseProxySource() {
|
|
final DatanodeInfo targetDN = target.getDatanodeInfo();
|
|
final DatanodeInfo targetDN = target.getDatanodeInfo();
|
|
// if node group is supported, first try add nodes in the same node group
|
|
// if node group is supported, first try add nodes in the same node group
|
|
if (cluster.isNodeGroupAware()) {
|
|
if (cluster.isNodeGroupAware()) {
|
|
@@ -356,8 +364,8 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** A class for keeping track of block locations in the dispatcher. */
|
|
/** A class for keeping track of block locations in the dispatcher. */
|
|
- private static class DBlock extends MovedBlocks.Locations<StorageGroup> {
|
|
|
|
- DBlock(Block block) {
|
|
|
|
|
|
+ public static class DBlock extends MovedBlocks.Locations<StorageGroup> {
|
|
|
|
+ public DBlock(Block block) {
|
|
super(block);
|
|
super(block);
|
|
}
|
|
}
|
|
}
|
|
}
|
|
@@ -378,10 +386,10 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** A class that keeps track of a datanode. */
|
|
/** A class that keeps track of a datanode. */
|
|
- static class DDatanode {
|
|
|
|
|
|
+ public static class DDatanode {
|
|
|
|
|
|
/** A group of storages in a datanode with the same storage type. */
|
|
/** A group of storages in a datanode with the same storage type. */
|
|
- class StorageGroup {
|
|
|
|
|
|
+ public class StorageGroup {
|
|
final StorageType storageType;
|
|
final StorageType storageType;
|
|
final long maxSize2Move;
|
|
final long maxSize2Move;
|
|
private long scheduledSize = 0L;
|
|
private long scheduledSize = 0L;
|
|
@@ -390,18 +398,26 @@ public class Dispatcher {
|
|
this.storageType = storageType;
|
|
this.storageType = storageType;
|
|
this.maxSize2Move = maxSize2Move;
|
|
this.maxSize2Move = maxSize2Move;
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ public StorageType getStorageType() {
|
|
|
|
+ return storageType;
|
|
|
|
+ }
|
|
|
|
|
|
private DDatanode getDDatanode() {
|
|
private DDatanode getDDatanode() {
|
|
return DDatanode.this;
|
|
return DDatanode.this;
|
|
}
|
|
}
|
|
|
|
|
|
- DatanodeInfo getDatanodeInfo() {
|
|
|
|
|
|
+ public DatanodeInfo getDatanodeInfo() {
|
|
return DDatanode.this.datanode;
|
|
return DDatanode.this.datanode;
|
|
}
|
|
}
|
|
|
|
|
|
/** Decide if still need to move more bytes */
|
|
/** Decide if still need to move more bytes */
|
|
- synchronized boolean hasSpaceForScheduling() {
|
|
|
|
- return availableSizeToMove() > 0L;
|
|
|
|
|
|
+ boolean hasSpaceForScheduling() {
|
|
|
|
+ return hasSpaceForScheduling(0L);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ synchronized boolean hasSpaceForScheduling(long size) {
|
|
|
|
+ return availableSizeToMove() > size;
|
|
}
|
|
}
|
|
|
|
|
|
/** @return the total number of bytes that need to be moved */
|
|
/** @return the total number of bytes that need to be moved */
|
|
@@ -410,7 +426,7 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** increment scheduled size */
|
|
/** increment scheduled size */
|
|
- synchronized void incScheduledSize(long size) {
|
|
|
|
|
|
+ public synchronized void incScheduledSize(long size) {
|
|
scheduledSize += size;
|
|
scheduledSize += size;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -436,7 +452,9 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
final DatanodeInfo datanode;
|
|
final DatanodeInfo datanode;
|
|
- final EnumMap<StorageType, StorageGroup> storageMap
|
|
|
|
|
|
+ private final EnumMap<StorageType, Source> sourceMap
|
|
|
|
+ = new EnumMap<StorageType, Source>(StorageType.class);
|
|
|
|
+ private final EnumMap<StorageType, StorageGroup> targetMap
|
|
= new EnumMap<StorageType, StorageGroup>(StorageType.class);
|
|
= new EnumMap<StorageType, StorageGroup>(StorageType.class);
|
|
protected long delayUntil = 0L;
|
|
protected long delayUntil = 0L;
|
|
/** blocks being moved but not confirmed yet */
|
|
/** blocks being moved but not confirmed yet */
|
|
@@ -445,29 +463,34 @@ public class Dispatcher {
|
|
|
|
|
|
@Override
|
|
@Override
|
|
public String toString() {
|
|
public String toString() {
|
|
- return getClass().getSimpleName() + ":" + datanode + ":" + storageMap.values();
|
|
|
|
|
|
+ return getClass().getSimpleName() + ":" + datanode;
|
|
}
|
|
}
|
|
|
|
|
|
- private DDatanode(DatanodeStorageReport r, int maxConcurrentMoves) {
|
|
|
|
- this.datanode = r.getDatanodeInfo();
|
|
|
|
|
|
+ private DDatanode(DatanodeInfo datanode, int maxConcurrentMoves) {
|
|
|
|
+ this.datanode = datanode;
|
|
this.maxConcurrentMoves = maxConcurrentMoves;
|
|
this.maxConcurrentMoves = maxConcurrentMoves;
|
|
this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves);
|
|
this.pendings = new ArrayList<PendingMove>(maxConcurrentMoves);
|
|
}
|
|
}
|
|
|
|
|
|
- private void put(StorageType storageType, StorageGroup g) {
|
|
|
|
- final StorageGroup existing = storageMap.put(storageType, g);
|
|
|
|
|
|
+ public DatanodeInfo getDatanodeInfo() {
|
|
|
|
+ return datanode;
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private static <G extends StorageGroup> void put(StorageType storageType,
|
|
|
|
+ G g, EnumMap<StorageType, G> map) {
|
|
|
|
+ final StorageGroup existing = map.put(storageType, g);
|
|
Preconditions.checkState(existing == null);
|
|
Preconditions.checkState(existing == null);
|
|
}
|
|
}
|
|
|
|
|
|
- StorageGroup addStorageGroup(StorageType storageType, long maxSize2Move) {
|
|
|
|
|
|
+ public StorageGroup addTarget(StorageType storageType, long maxSize2Move) {
|
|
final StorageGroup g = new StorageGroup(storageType, maxSize2Move);
|
|
final StorageGroup g = new StorageGroup(storageType, maxSize2Move);
|
|
- put(storageType, g);
|
|
|
|
|
|
+ put(storageType, g, targetMap);
|
|
return g;
|
|
return g;
|
|
}
|
|
}
|
|
|
|
|
|
- Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
|
|
|
|
|
|
+ public Source addSource(StorageType storageType, long maxSize2Move, Dispatcher d) {
|
|
final Source s = d.new Source(storageType, maxSize2Move, this);
|
|
final Source s = d.new Source(storageType, maxSize2Move, this);
|
|
- put(storageType, s);
|
|
|
|
|
|
+ put(storageType, s, sourceMap);
|
|
return s;
|
|
return s;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -508,7 +531,7 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** A node that can be the sources of a block move */
|
|
/** A node that can be the sources of a block move */
|
|
- class Source extends DDatanode.StorageGroup {
|
|
|
|
|
|
+ public class Source extends DDatanode.StorageGroup {
|
|
|
|
|
|
private final List<Task> tasks = new ArrayList<Task>(2);
|
|
private final List<Task> tasks = new ArrayList<Task>(2);
|
|
private long blocksToReceive = 0L;
|
|
private long blocksToReceive = 0L;
|
|
@@ -654,13 +677,7 @@ public class Dispatcher {
|
|
&& (!srcBlocks.isEmpty() || blocksToReceive > 0)) {
|
|
&& (!srcBlocks.isEmpty() || blocksToReceive > 0)) {
|
|
final PendingMove p = chooseNextMove();
|
|
final PendingMove p = chooseNextMove();
|
|
if (p != null) {
|
|
if (p != null) {
|
|
- // move the block
|
|
|
|
- moveExecutor.execute(new Runnable() {
|
|
|
|
- @Override
|
|
|
|
- public void run() {
|
|
|
|
- p.dispatch();
|
|
|
|
- }
|
|
|
|
- });
|
|
|
|
|
|
+ executePendingMove(p);
|
|
continue;
|
|
continue;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -716,7 +733,8 @@ public class Dispatcher {
|
|
this.cluster = NetworkTopology.getInstance(conf);
|
|
this.cluster = NetworkTopology.getInstance(conf);
|
|
|
|
|
|
this.moveExecutor = Executors.newFixedThreadPool(moverThreads);
|
|
this.moveExecutor = Executors.newFixedThreadPool(moverThreads);
|
|
- this.dispatchExecutor = Executors.newFixedThreadPool(dispatcherThreads);
|
|
|
|
|
|
+ this.dispatchExecutor = dispatcherThreads == 0? null
|
|
|
|
+ : Executors.newFixedThreadPool(dispatcherThreads);
|
|
this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
|
|
this.maxConcurrentMovesPerNode = maxConcurrentMovesPerNode;
|
|
|
|
|
|
final boolean fallbackToSimpleAuthAllowed = conf.getBoolean(
|
|
final boolean fallbackToSimpleAuthAllowed = conf.getBoolean(
|
|
@@ -727,11 +745,15 @@ public class Dispatcher {
|
|
TrustedChannelResolver.getInstance(conf), fallbackToSimpleAuthAllowed);
|
|
TrustedChannelResolver.getInstance(conf), fallbackToSimpleAuthAllowed);
|
|
}
|
|
}
|
|
|
|
|
|
- StorageGroupMap getStorageGroupMap() {
|
|
|
|
|
|
+ public DistributedFileSystem getDistributedFileSystem() {
|
|
|
|
+ return nnc.getDistributedFileSystem();
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public StorageGroupMap<StorageGroup> getStorageGroupMap() {
|
|
return storageGroupMap;
|
|
return storageGroupMap;
|
|
}
|
|
}
|
|
|
|
|
|
- NetworkTopology getCluster() {
|
|
|
|
|
|
+ public NetworkTopology getCluster() {
|
|
return cluster;
|
|
return cluster;
|
|
}
|
|
}
|
|
|
|
|
|
@@ -779,7 +801,7 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** Get live datanode storage reports and then build the network topology. */
|
|
/** Get live datanode storage reports and then build the network topology. */
|
|
- List<DatanodeStorageReport> init() throws IOException {
|
|
|
|
|
|
+ public List<DatanodeStorageReport> init() throws IOException {
|
|
final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport();
|
|
final DatanodeStorageReport[] reports = nnc.getLiveDatanodeStorageReport();
|
|
final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>();
|
|
final List<DatanodeStorageReport> trimmed = new ArrayList<DatanodeStorageReport>();
|
|
// create network topology and classify utilization collections:
|
|
// create network topology and classify utilization collections:
|
|
@@ -795,8 +817,18 @@ public class Dispatcher {
|
|
return trimmed;
|
|
return trimmed;
|
|
}
|
|
}
|
|
|
|
|
|
- public DDatanode newDatanode(DatanodeStorageReport r) {
|
|
|
|
- return new DDatanode(r, maxConcurrentMovesPerNode);
|
|
|
|
|
|
+ public DDatanode newDatanode(DatanodeInfo datanode) {
|
|
|
|
+ return new DDatanode(datanode, maxConcurrentMovesPerNode);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ public void executePendingMove(final PendingMove p) {
|
|
|
|
+ // move the block
|
|
|
|
+ moveExecutor.execute(new Runnable() {
|
|
|
|
+ @Override
|
|
|
|
+ public void run() {
|
|
|
|
+ p.dispatch();
|
|
|
|
+ }
|
|
|
|
+ });
|
|
}
|
|
}
|
|
|
|
|
|
public boolean dispatchAndCheckContinue() throws InterruptedException {
|
|
public boolean dispatchAndCheckContinue() throws InterruptedException {
|
|
@@ -869,6 +901,12 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
|
|
|
|
+ DBlock block) {
|
|
|
|
+ // match source and target storage type
|
|
|
|
+ return isGoodBlockCandidate(source, target, source.getStorageType(), block);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Decide if the block is a good candidate to be moved from source to target.
|
|
* Decide if the block is a good candidate to be moved from source to target.
|
|
* A block is a good candidate if
|
|
* A block is a good candidate if
|
|
@@ -876,9 +914,12 @@ public class Dispatcher {
|
|
* 2. the block does not have a replica on the target;
|
|
* 2. the block does not have a replica on the target;
|
|
* 3. doing the move does not reduce the number of racks that the block has
|
|
* 3. doing the move does not reduce the number of racks that the block has
|
|
*/
|
|
*/
|
|
- private boolean isGoodBlockCandidate(Source source, StorageGroup target,
|
|
|
|
- DBlock block) {
|
|
|
|
- if (source.storageType != target.storageType) {
|
|
|
|
|
|
+ public boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
|
|
|
|
+ StorageType targetStorageType, DBlock block) {
|
|
|
|
+ if (target.storageType != targetStorageType) {
|
|
|
|
+ return false;
|
|
|
|
+ }
|
|
|
|
+ if (!target.hasSpaceForScheduling(block.getNumBytes())) {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
// check if the block is moved or not
|
|
// check if the block is moved or not
|
|
@@ -889,7 +930,7 @@ public class Dispatcher {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
if (cluster.isNodeGroupAware()
|
|
if (cluster.isNodeGroupAware()
|
|
- && isOnSameNodeGroupWithReplicas(target, block, source)) {
|
|
|
|
|
|
+ && isOnSameNodeGroupWithReplicas(source, target, block)) {
|
|
return false;
|
|
return false;
|
|
}
|
|
}
|
|
if (reduceNumOfRacks(source, target, block)) {
|
|
if (reduceNumOfRacks(source, target, block)) {
|
|
@@ -902,7 +943,7 @@ public class Dispatcher {
|
|
* Determine whether moving the given block replica from source to target
|
|
* Determine whether moving the given block replica from source to target
|
|
* would reduce the number of racks of the block replicas.
|
|
* would reduce the number of racks of the block replicas.
|
|
*/
|
|
*/
|
|
- private boolean reduceNumOfRacks(Source source, StorageGroup target,
|
|
|
|
|
|
+ private boolean reduceNumOfRacks(StorageGroup source, StorageGroup target,
|
|
DBlock block) {
|
|
DBlock block) {
|
|
final DatanodeInfo sourceDn = source.getDatanodeInfo();
|
|
final DatanodeInfo sourceDn = source.getDatanodeInfo();
|
|
if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) {
|
|
if (cluster.isOnSameRack(sourceDn, target.getDatanodeInfo())) {
|
|
@@ -939,8 +980,8 @@ public class Dispatcher {
|
|
* @return true if there are any replica (other than source) on the same node
|
|
* @return true if there are any replica (other than source) on the same node
|
|
* group with target
|
|
* group with target
|
|
*/
|
|
*/
|
|
- private boolean isOnSameNodeGroupWithReplicas(
|
|
|
|
- StorageGroup target, DBlock block, Source source) {
|
|
|
|
|
|
+ private boolean isOnSameNodeGroupWithReplicas(StorageGroup source,
|
|
|
|
+ StorageGroup target, DBlock block) {
|
|
final DatanodeInfo targetDn = target.getDatanodeInfo();
|
|
final DatanodeInfo targetDn = target.getDatanodeInfo();
|
|
for (StorageGroup g : block.getLocations()) {
|
|
for (StorageGroup g : block.getLocations()) {
|
|
if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) {
|
|
if (g != source && cluster.isOnSameNodeGroup(g.getDatanodeInfo(), targetDn)) {
|
|
@@ -961,7 +1002,7 @@ public class Dispatcher {
|
|
}
|
|
}
|
|
|
|
|
|
/** shutdown thread pools */
|
|
/** shutdown thread pools */
|
|
- void shutdownNow() {
|
|
|
|
|
|
+ public void shutdownNow() {
|
|
dispatchExecutor.shutdownNow();
|
|
dispatchExecutor.shutdownNow();
|
|
moveExecutor.shutdownNow();
|
|
moveExecutor.shutdownNow();
|
|
}
|
|
}
|