hadoop-hdfs-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From hair...@apache.org
Subject svn commit: r811486 - in /hadoop/hdfs/trunk: CHANGES.txt src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
Date Fri, 04 Sep 2009 16:50:53 GMT
Author: hairong
Date: Fri Sep  4 16:50:51 2009
New Revision: 811486

URL: http://svn.apache.org/viewvc?rev=811486&view=rev
Log:
HDFS-586. TestBlocksWithNotEnoughRacks fails. Contributed by Jitendra Nath Pandey.

Modified:
    hadoop/hdfs/trunk/CHANGES.txt
    hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
    hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java

Modified: hadoop/hdfs/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/CHANGES.txt?rev=811486&r1=811485&r2=811486&view=diff
==============================================================================
--- hadoop/hdfs/trunk/CHANGES.txt (original)
+++ hadoop/hdfs/trunk/CHANGES.txt Fri Sep  4 16:50:51 2009
@@ -193,6 +193,9 @@
  
     HDFS-15. All replicas end up on 1 rack. (Jitendra Nath Pandey via hairong)
  
+    HDFS-586. TestBlocksWithNotEnoughRacks sometimes fails.
+    (Jitendra Nath Pandey via hairong)
+
 Release 0.20.1 - Unreleased
 
   IMPROVEMENTS

Modified: hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java?rev=811486&r1=811485&r2=811486&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java (original)
+++ hadoop/hdfs/trunk/src/java/org/apache/hadoop/hdfs/server/namenode/BlockManager.java Fri
Sep  4 16:50:51 2009
@@ -1034,7 +1034,7 @@
 
     // handle underReplication/overReplication
     short fileReplication = fileINode.getReplication();
-    if (numCurrentReplica >= fileReplication) {
+    if (!isNeededReplication(storedBlock, fileReplication, numCurrentReplica)) {
       neededReplications.remove(storedBlock, numCurrentReplica,
           num.decommissionedReplicas, fileReplication);
     } else {

Modified: hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
URL: http://svn.apache.org/viewvc/hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java?rev=811486&r1=811485&r2=811486&view=diff
==============================================================================
--- hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
(original)
+++ hadoop/hdfs/trunk/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestBlocksWithNotEnoughRacks.java
Fri Sep  4 16:50:51 2009
@@ -29,6 +29,8 @@
 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 @@
     ((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 @@
       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 @@
       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();
 
-      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);



Mime
View raw message