Переглянути джерело

HDFS-2168. Reenable TestEditLog.testFailedOpen and fix exposed bug. Contributed by Todd Lipcon.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/HDFS-1073@1148580 13f79535-47bb-0310-9956-ffa450edef68
Todd Lipcon 14 роки тому
батько
коміт
3bcbe07d70

+ 1 - 0
hdfs/CHANGES.HDFS-1073.txt

@@ -76,3 +76,4 @@ HDFS-2104. Add a flag to the 2NN to format its checkpoint dirs on startup.
            (todd)
 HDFS-2135. Fix regression of HDFS-1955 in HDFS-1073 branch. (todd)
 HDFS-2160. Fix CreateEditsLog test tool in HDFS-1073 branch. (todd)
+HDFS-2168. Reenable TestEditLog.testFailedOpen and fix exposed bug. (todd)

+ 2 - 1
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/BackupImage.java

@@ -310,7 +310,8 @@ public class BackupImage extends FSImage {
    * This causes the BN to also start the new edit log in its local
    * directories.
    */
-  synchronized void namenodeStartedLogSegment(long txid) {
+  synchronized void namenodeStartedLogSegment(long txid)
+      throws IOException {
     LOG.info("NameNode started a new log segment at txid " + txid);
     if (editLog.isOpen()) {
       if (editLog.getLastWrittenTxId() == txid - 1) {

+ 20 - 2
hdfs/src/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java

@@ -725,6 +725,19 @@ public class FSEditLog  {
     return new DeprecatedUTF8(Long.toString(timestamp));
   }
 
+  /**
+   * @return the number of active (non-failed) journals
+   */
+  private int countActiveJournals() {
+    int count = 0;
+    for (JournalAndStream jas : journals) {
+      if (jas.isActive()) {
+        count++;
+      }
+    }
+    return count;
+  }
+  
   /**
    * Used only by unit tests.
    */
@@ -777,7 +790,7 @@ public class FSEditLog  {
    * Transitions from BETWEEN_LOG_SEGMENTS state to IN_LOG_SEGMENT state. 
    */
   synchronized void startLogSegment(final long segmentTxId,
-      boolean writeHeaderTxn) {
+      boolean writeHeaderTxn) throws IOException {
     LOG.info("Starting log segment at " + segmentTxId);
     Preconditions.checkArgument(segmentTxId > 0,
         "Bad txid: %s", segmentTxId);
@@ -789,7 +802,6 @@ public class FSEditLog  {
     Preconditions.checkArgument(segmentTxId == txid + 1,
         "Cannot start log segment at txid %s when next expected " +
         "txid is %s", segmentTxId, txid + 1);
-    curSegmentTxId = segmentTxId;
     
     numTransactions = totalTimeTransactions = numTransactionsBatchedInSync = 0;
 
@@ -803,6 +815,12 @@ public class FSEditLog  {
       }
     }, "starting log segment " + segmentTxId);
 
+    if (countActiveJournals() == 0) {
+      throw new IOException("Unable to start log segment " +
+          segmentTxId + ": no journals successfully started.");
+    }
+    
+    curSegmentTxId = segmentTxId;
     state = State.IN_SEGMENT;
 
     if (writeHeaderTxn) {

+ 11 - 46
hdfs/src/test/hdfs/org/apache/hadoop/hdfs/server/namenode/TestEditLog.java

@@ -726,55 +726,20 @@ public class TestEditLog extends TestCase {
     }
   }
 
-  /*
-   * TODO: need to re-enable this test in HDFS-1073 branch!
   public void testFailedOpen() throws Exception {
-    Configuration conf = new HdfsConfiguration();
-    MiniDFSCluster cluster = null;
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATA_NODES).build();
-    cluster.waitActive();
-    final FSNamesystem fsn = cluster.getNamesystem();
-
-    // Set up spys
-    final FSImage originalImage = fsn.getFSImage();
-    NNStorage storage = originalImage.getStorage();
-    NNStorage spyStorage = spy(storage);
-    originalImage.storage = spyStorage;
-    
-    final FSEditLog editLog = originalImage.getEditLog();
-    FSEditLog spyLog = spy(editLog);
-
-    FSImage spyImage = spy(originalImage);
-    fsn.dir.fsImage = spyImage;
-    spyImage.storage.setStorageDirectories(
-        FSNamesystem.getNamespaceDirs(conf), 
-        FSNamesystem.getNamespaceEditsDirs(conf));
-        
-    // Fail every attempt to open a new edit file
-    doThrow(new IOException("Injected fault: open")).
-      when(spyLog).addNewEditLogStream((File)anyObject());
-    
+    File logDir = new File(TEST_DIR, "testFailedOpen");
+    logDir.mkdirs();
+    FSEditLog log = FSImageTestUtil.createStandaloneEditLog(logDir);
     try {
-      spyLog.close();
-      spyLog.open();
-      fail("open did not fail even when all directories failed!");
-    } catch(IOException ioe) {
-      LOG.info("Got expected exception", ioe);
+      logDir.setWritable(false);
+      log.open();
+      fail("Did no throw exception on only having a bad dir");
+    } catch (IOException ioe) {
+      GenericTestUtils.assertExceptionContains(
+          "no journals successfully started", ioe);
     } finally {
-      spyLog.close();
+      logDir.setWritable(true);
+      log.close();
     }
-    
-    // Reset and try it with a working open
-    Mockito.reset(spyLog);
-    spyImage.storage.setStorageDirectories(
-        FSNamesystem.getNamespaceDirs(conf), 
-        FSNamesystem.getNamespaceEditsDirs(conf));
-    spyLog.open();
-    
-    // Close everything off
-    spyLog.close();
-    originalImage.close();
-    fsn.close();
   }
-  */
 }