|
@@ -29,6 +29,8 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
|
|
|
import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.NumberReplicas;
|
|
|
import org.apache.commons.logging.impl.Log4JLogger;
|
|
|
import org.apache.log4j.Level;
|
|
|
+import org.apache.commons.logging.Log;
|
|
|
+import org.apache.commons.logging.LogFactory;
|
|
|
|
|
|
import junit.framework.TestCase;
|
|
|
|
|
@@ -38,6 +40,8 @@ public class TestBlocksWithNotEnoughRacks extends TestCase {
|
|
|
((Log4JLogger)NameNode.stateChangeLog).getLogger().setLevel(Level.ALL) ;
|
|
|
}
|
|
|
|
|
|
+ private static final Log LOG =
|
|
|
+ LogFactory.getLog(TestBlocksWithNotEnoughRacks.class.getName());
|
|
|
//Creates a block with all datanodes on same rack
|
|
|
//Adds additional datanode on a different rack
|
|
|
//The block should be replicated to the new rack
|
|
@@ -61,20 +65,29 @@ public class TestBlocksWithNotEnoughRacks extends TestCase {
|
|
|
Block b = DFSTestUtil.getFirstBlock(fs, FILE_PATH);
|
|
|
final FSNamesystem namesystem = cluster.getNamesystem();
|
|
|
int numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
+ NumberReplicas number = namesystem.blockManager.countNodes(b);
|
|
|
+ int curReplicas = number.liveReplicas();
|
|
|
+ int neededReplicationSize =
|
|
|
+ namesystem.blockManager.neededReplications.size();
|
|
|
|
|
|
//Add a new datanode on a different rack
|
|
|
String newRacks[] = {"/rack2"} ;
|
|
|
cluster.startDataNodes(conf, 1, true, null, newRacks);
|
|
|
-
|
|
|
- Thread.sleep(5000);
|
|
|
-
|
|
|
- numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
- NumberReplicas number = namesystem.blockManager.countNodes(b);
|
|
|
- int curReplicas = number.liveReplicas();
|
|
|
|
|
|
- System.out.println("curReplicas = " + curReplicas);
|
|
|
- System.out.println("numRacks = " + numRacks);
|
|
|
- System.out.println("Size = " + namesystem.blockManager.neededReplications.size());
|
|
|
+ while ( (numRacks < 2) || (curReplicas < REPLICATION_FACTOR) ||
|
|
|
+ (neededReplicationSize > 0) ) {
|
|
|
+ LOG.info("Waiting for replication");
|
|
|
+ Thread.sleep(600);
|
|
|
+ numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
+ number = namesystem.blockManager.countNodes(b);
|
|
|
+ curReplicas = number.liveReplicas();
|
|
|
+ neededReplicationSize =
|
|
|
+ namesystem.blockManager.neededReplications.size();
|
|
|
+ }
|
|
|
+
|
|
|
+ LOG.info("curReplicas = " + curReplicas);
|
|
|
+ LOG.info("numRacks = " + numRacks);
|
|
|
+ LOG.info("Size = " + namesystem.blockManager.neededReplications.size());
|
|
|
|
|
|
assertEquals(2,numRacks);
|
|
|
assertTrue(curReplicas == REPLICATION_FACTOR);
|
|
@@ -106,22 +119,31 @@ public class TestBlocksWithNotEnoughRacks extends TestCase {
|
|
|
Block b = DFSTestUtil.getFirstBlock(fs, FILE_PATH);
|
|
|
final FSNamesystem namesystem = cluster.getNamesystem();
|
|
|
int numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
+ NumberReplicas number = namesystem.blockManager.countNodes(b);
|
|
|
+ int curReplicas = number.liveReplicas();
|
|
|
+ int neededReplicationSize =
|
|
|
+ namesystem.blockManager.neededReplications.size();
|
|
|
|
|
|
//Add a new datanode on a different rack
|
|
|
- String newRacks[] = {"/rack2","/rack2"} ;
|
|
|
- cluster.startDataNodes(conf, 2, true, null, newRacks);
|
|
|
+ String newRacks[] = {"/rack2","/rack2","/rack2"} ;
|
|
|
+ cluster.startDataNodes(conf, 3, true, null, newRacks);
|
|
|
REPLICATION_FACTOR = 5;
|
|
|
namesystem.setReplication(FILE_NAME, REPLICATION_FACTOR);
|
|
|
- Thread.sleep(30000);
|
|
|
-
|
|
|
|
|
|
- numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
- NumberReplicas number = namesystem.blockManager.countNodes(b);
|
|
|
- int curReplicas = number.liveReplicas();
|
|
|
+ while ( (numRacks < 2) || (curReplicas < REPLICATION_FACTOR) ||
|
|
|
+ (neededReplicationSize > 0) ) {
|
|
|
+ LOG.info("Waiting for replication");
|
|
|
+ Thread.sleep(600);
|
|
|
+ numRacks = namesystem.blockManager.getNumberOfRacks(b);
|
|
|
+ number = namesystem.blockManager.countNodes(b);
|
|
|
+ curReplicas = number.liveReplicas();
|
|
|
+ neededReplicationSize =
|
|
|
+ namesystem.blockManager.neededReplications.size();
|
|
|
+ }
|
|
|
|
|
|
- System.out.println("curReplicas = " + curReplicas);
|
|
|
- System.out.println("numRacks = " + numRacks);
|
|
|
- System.out.println("Size = " + namesystem.blockManager.neededReplications.size());
|
|
|
+ LOG.info("curReplicas = " + curReplicas);
|
|
|
+ LOG.info("numRacks = " + numRacks);
|
|
|
+ LOG.info("Size = " + namesystem.blockManager.neededReplications.size());
|
|
|
|
|
|
assertEquals(2,numRacks);
|
|
|
assertTrue(curReplicas == REPLICATION_FACTOR);
|