HDFS-8204. Mover/Balancer should not schedule two replicas to the same datanode. Contributed by Walter Su

This commit is contained in:
Tsz-Wo Nicholas Sze 2015-04-28 13:05:33 -07:00
parent d149af0539
commit 9b62a8ed77
3 changed files with 81 additions and 2 deletions

View File

@ -266,6 +266,9 @@ Release 2.7.1 - UNRELEASED
HDFS-7931. DistributedFileSystem should not look for keyProvider in HDFS-7931. DistributedFileSystem should not look for keyProvider in
cache if Encryption is disabled (asuresh) cache if Encryption is disabled (asuresh)
HDFS-8204. Mover/Balancer should not schedule two replicas to the same
datanode. (Walter Su via szetszwo)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -973,6 +973,9 @@ public class Dispatcher {
*/ */
private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target,
StorageType targetStorageType, DBlock block) { StorageType targetStorageType, DBlock block) {
if (source.equals(target)) {
return false;
}
if (target.storageType != targetStorageType) { if (target.storageType != targetStorageType) {
return false; return false;
} }
@ -980,9 +983,19 @@ public class Dispatcher {
if (movedBlocks.contains(block.getBlock())) { if (movedBlocks.contains(block.getBlock())) {
return false; return false;
} }
if (block.isLocatedOn(target)) { final DatanodeInfo targetDatanode = target.getDatanodeInfo();
if (source.getDatanodeInfo().equals(targetDatanode)) {
// the block is moved inside same DN
return true;
}
// check if block has replica in target node
for (StorageGroup blockLocation : block.getLocations()) {
if (blockLocation.getDatanodeInfo().equals(targetDatanode)) {
return false; return false;
} }
}
if (cluster.isNodeGroupAware() if (cluster.isNodeGroupAware()
&& isOnSameNodeGroupWithReplicas(source, target, block)) { && isOnSameNodeGroupWithReplicas(source, target, block)) {
return false; return false;

View File

@ -48,6 +48,7 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.FileSystem;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.StorageType;
import org.apache.hadoop.fs.permission.FsPermission;
import org.apache.hadoop.hdfs.DFSClient; import org.apache.hadoop.hdfs.DFSClient;
import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSConfigKeys;
import org.apache.hadoop.hdfs.DFSTestUtil; import org.apache.hadoop.hdfs.DFSTestUtil;
@ -1309,6 +1310,68 @@ public class TestBalancer {
} }
} }
/**
* Test special case. Two replicas belong to same block should not in same node.
* We have 2 nodes.
* We have a block in (DN0,SSD) and (DN1,DISK).
* Replica in (DN0,SSD) should not be moved to (DN1,SSD).
* Otherwise DN1 has 2 replicas.
*/
@Test(timeout=100000)
public void testTwoReplicaShouldNotInSameDN() throws Exception {
final Configuration conf = new HdfsConfiguration();
int blockSize = 5 * 1024 * 1024 ;
conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize);
conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L);
conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L);
int numOfDatanodes =2;
final MiniDFSCluster cluster = new MiniDFSCluster.Builder(conf)
.numDataNodes(2)
.racks(new String[]{"/default/rack0", "/default/rack0"})
.storagesPerDatanode(2)
.storageTypes(new StorageType[][]{
{StorageType.SSD, StorageType.DISK},
{StorageType.SSD, StorageType.DISK}})
.storageCapacities(new long[][]{
{100 * blockSize, 20 * blockSize},
{20 * blockSize, 100 * blockSize}})
.build();
try {
cluster.waitActive();
//set "/bar" directory with ONE_SSD storage policy.
DistributedFileSystem fs = cluster.getFileSystem();
Path barDir = new Path("/bar");
fs.mkdir(barDir,new FsPermission((short)777));
fs.setStoragePolicy(barDir, HdfsConstants.ONESSD_STORAGE_POLICY_NAME);
// Insert 30 blocks. So (DN0,SSD) and (DN1,DISK) are about half full,
// and (DN0,SSD) and (DN1,DISK) are about 15% full.
long fileLen = 30 * blockSize;
// fooFile has ONE_SSD policy. So
// (DN0,SSD) and (DN1,DISK) have 2 replicas belong to same block.
// (DN0,DISK) and (DN1,SSD) have 2 replicas belong to same block.
Path fooFile = new Path(barDir, "foo");
createFile(cluster, fooFile, fileLen, (short) numOfDatanodes, 0);
// update space info
cluster.triggerHeartbeats();
Balancer.Parameters p = Balancer.Parameters.DEFAULT;
Collection<URI> namenodes = DFSUtil.getNsServiceRpcUris(conf);
final int r = Balancer.run(namenodes, p, conf);
// Replica in (DN0,SSD) was not moved to (DN1,SSD), because (DN1,DISK)
// already has one. Otherwise DN1 will have 2 replicas.
// For same reason, no replicas were moved.
assertEquals(ExitStatus.NO_MOVE_PROGRESS.getExitCode(), r);
} finally {
cluster.shutdown();
}
}
/** /**
* @param args * @param args
*/ */