Browse Source

HDFS-2684. Fix up some failing unit tests on HA branch. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1623@1215241 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 13 năm trước cách đây
mục cha
commit
371f4228e8

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.HDFS-1623.txt

@@ -61,3 +61,5 @@ HDFS-2689. HA: BookKeeperEditLogInputStream doesn't implement isInProgress() (at
 HDFS-2602. NN should log newly-allocated blocks without losing BlockInfo (atm)
 
 HDFS-2667. Fix transition from active to standby (todd)
+
+HDFS-2684. Fix up some failing unit tests on HA branch (todd)

+ 2 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java

@@ -325,6 +325,8 @@ class BPOfferService {
     } else {
       bpRegistration = reg;
     }
+    
+    dn.bpRegistrationSucceeded(bpRegistration, getBlockPoolId());
   }
 
   /**

+ 5 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/BackupNode.java

@@ -199,6 +199,11 @@ public class BackupNode extends NameNode {
       checkpointManager.interrupt();
       checkpointManager = null;
     }
+
+    // Abort current log segment - otherwise the NN shutdown code
+    // will close it gracefully, which is incorrect.
+    getFSImage().getEditLog().abortCurrentLogSegment();
+
     // Stop name-node threads
     super.stop();
   }

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -916,6 +916,7 @@ public class FSEditLog  {
       if (editLogStream != null) {
         editLogStream.abort();
         editLogStream = null;
+        state = State.BETWEEN_LOG_SEGMENTS;
       }
     } catch (IOException e) {
       LOG.warn("All journals failed to abort", e);

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -495,7 +495,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
     try {
       FSEditLog editLog = dir.fsImage.getEditLog();
       
-      if (!editLog.isSegmentOpen()) {
+      if (!editLog.isOpenForWrite()) {
         // During startup, we're already open for write during initialization.
         // TODO(HA): consider adding a startup state?
         editLog.initJournalsForWrite();
@@ -2774,7 +2774,7 @@ public class FSNamesystem implements Namesystem, FSClusterStats,
       DatanodeCommand[] cmds = blockManager.getDatanodeManager().handleHeartbeat(
           nodeReg, blockPoolId, capacity, dfsUsed, remaining, blockPoolUsed,
           xceiverCount, maxTransfer, failedVolumes);
-      if (cmds == null) {
+      if (cmds == null || cmds.length == 0) {
         DatanodeCommand cmd = upgradeManager.getBroadcastCommand();
         if (cmd != null) {
           cmds = new DatanodeCommand[] {cmd};

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSUpgrade.java

@@ -52,7 +52,7 @@ import static org.junit.Assert.*;
 */
 public class TestDFSUpgrade {
  
-  private static final int EXPECTED_TXID = 33;
+  private static final int EXPECTED_TXID = 49;
   private static final Log LOG = LogFactory.getLog(TestDFSUpgrade.class.getName());
   private Configuration conf;
   private int testCounter = 0;

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDataTransferProtocol.java

@@ -302,7 +302,7 @@ public class TestDataTransferProtocol extends TestCase {
         testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_CREATE, 0L,
             "Cannot create a RBW block", true);
         // test PIPELINE_SETUP_APPEND on an existing block
-        newGS = newBlock.getGenerationStamp() + 1;
+        newGS = firstBlock.getGenerationStamp() + 1;
         testWrite(firstBlock, BlockConstructionStage.PIPELINE_SETUP_APPEND,
             newGS, "Cannot append to a RBW replica", true);
         // test PIPELINE_SETUP_APPEND on an existing block

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestHeartbeatHandling.java

@@ -110,7 +110,7 @@ public class TestHeartbeatHandling extends TestCase {
 
           cmds = NameNodeAdapter.sendHeartBeat(nodeReg, dd, namesystem)
               .getCommands();
-          assertEquals(null, cmds);
+          assertEquals(0, cmds.length);
         }
       } finally {
         namesystem.writeUnlock();

+ 7 - 5
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBackupNode.java

@@ -240,9 +240,9 @@ public class TestBackupNode extends TestCase {
   }  
 
   void testCheckpoint(StartupOption op) throws Exception {
-    Path file1 = new Path("checkpoint.dat");
-    Path file2 = new Path("checkpoint2.dat");
-    Path file3 = new Path("backup.dat");
+    Path file1 = new Path("/checkpoint.dat");
+    Path file2 = new Path("/checkpoint2.dat");
+    Path file3 = new Path("/backup.dat");
 
     Configuration conf = new HdfsConfiguration();
     short replication = (short)conf.getInt("dfs.replication", 3);
@@ -341,11 +341,13 @@ public class TestBackupNode extends TestCase {
       TestCheckpoint.checkFile(fileSys, file3, replication);
       // should also be on BN right away
       assertTrue("file3 does not exist on BackupNode",
-          op != StartupOption.BACKUP || bnFS.exists(file3));
+          op != StartupOption.BACKUP ||
+          backup.getNamesystem().getFileInfo(
+              file3.toUri().getPath(), false) != null);
 
     } catch(IOException e) {
       LOG.error("Error in TestBackupNode:", e);
-      assertTrue(e.getLocalizedMessage(), false);
+      throw new AssertionError(e);
     } finally {
       if(backup != null) backup.stop();
       if(fileSys != null) fileSys.close();

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestCheckpoint.java

@@ -923,10 +923,12 @@ public class TestCheckpoint extends TestCase {
         throw new IOException(e);
       }
       
+      final int EXPECTED_TXNS_FIRST_SEG = 12;
+      
       // the following steps should have happened:
-      //   edits_inprogress_1 -> edits_1-8  (finalized)
-      //   fsimage_8 created
-      //   edits_inprogress_9 created
+      //   edits_inprogress_1 -> edits_1-12  (finalized)
+      //   fsimage_12 created
+      //   edits_inprogress_13 created
       //
       for(URI uri : editsDirs) {
         File ed = new File(uri.getPath());
@@ -938,19 +940,21 @@ public class TestCheckpoint extends TestCase {
                                       NNStorage.getInProgressEditsFileName(1));
         assertFalse(originalEdits.exists());
         File finalizedEdits = new File(curDir,
-            NNStorage.getFinalizedEditsFileName(1,8));
-        assertTrue(finalizedEdits.exists());
+            NNStorage.getFinalizedEditsFileName(1, EXPECTED_TXNS_FIRST_SEG));
+        GenericTestUtils.assertExists(finalizedEdits);
         assertTrue(finalizedEdits.length() > Integer.SIZE/Byte.SIZE);
 
-        assertTrue(new File(ed, "current/"
-                       + NNStorage.getInProgressEditsFileName(9)).exists());
+        GenericTestUtils.assertExists(new File(ed, "current/"
+                       + NNStorage.getInProgressEditsFileName(
+                           EXPECTED_TXNS_FIRST_SEG + 1)));
       }
       
       Collection<URI> imageDirs = cluster.getNameDirs(0);
       for (URI uri : imageDirs) {
         File imageDir = new File(uri.getPath());
         File savedImage = new File(imageDir, "current/"
-                                   + NNStorage.getImageFileName(8));
+                                   + NNStorage.getImageFileName(
+                                       EXPECTED_TXNS_FIRST_SEG));
         assertTrue("Should have saved image at " + savedImage,
             savedImage.exists());        
       }