You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by te...@apache.org on 2013/02/27 16:00:09 UTC

svn commit: r1450792 - in /hbase/branches/0.95/hbase-server/src: main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java

Author: tedyu
Date: Wed Feb 27 15:00:08 2013
New Revision: 1450792

URL: http://svn.apache.org/r1450792
Log:
HBASE-7878 recoverFileLease does not check return value of recoverLease (Ted Yu)


Modified:
    hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
    hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java

Modified: hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java?rev=1450792&r1=1450791&r2=1450792&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java (original)
+++ hbase/branches/0.95/hbase-server/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java Wed Feb 27 15:00:08 2013
@@ -54,6 +54,8 @@ public class FSHDFSUtils extends FSUtils
    * in o.a.h.hdfs.protocol.HdfsConstants cause of HDFS-1620.
    */
   public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
+  
+  public static final String TEST_TRIGGER_DFS_APPEND = "hbase.test.trigger.dfs.append";
 
   @Override
   public void recoverFileLease(final FileSystem fs, final Path p, Configuration conf)
@@ -72,22 +74,37 @@ public class FSHDFSUtils extends FSUtils
 
     // Trying recovery
     boolean recovered = false;
+    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 300000);
+    // conf parameter passed from unit test, indicating whether fs.append() should be triggered
+    boolean triggerAppend = conf.getBoolean(TEST_TRIGGER_DFS_APPEND, false);
+    Exception ex = null;
     while (!recovered) {
       try {
         try {
           DistributedFileSystem dfs = (DistributedFileSystem) fs;
-          DistributedFileSystem.class.getMethod("recoverLease", new Class[] { Path.class }).invoke(
-              dfs, p);
-        } catch (InvocationTargetException ite) {
-          // function was properly called, but threw it's own exception
-          throw (IOException) ite.getCause();
+          if (triggerAppend) throw new IOException();
+          try {
+            recovered = (Boolean) DistributedFileSystem.class.getMethod(
+              "recoverLease", new Class[] { Path.class }).invoke(dfs, p);
+          } catch (InvocationTargetException ite) {
+            // function was properly called, but threw it's own exception
+            throw (IOException) ite.getCause();
+          }
         } catch (Exception e) {
-          LOG.debug("Failed fs.recoverLease invocation, " + e.toString() +
-            ", trying fs.append instead");
+          // hdfs 2.0 may throw RecoveryInProgressException
+          if (!e.getClass().getName().contains("RecoveryInProgressException")) {
+            LOG.debug("Failed fs.recoverLease invocation, " + e.toString() +
+                ", trying fs.append instead");
+            ex = e;
+          }
+        }
+        if (ex != null || System.currentTimeMillis() - startWaiting > recoveryTimeout) {
+          ex = null; // assume the following append() call would succeed
           FSDataOutputStream out = fs.append(p);
           out.close();
+          recovered = true;
         }
-        recovered = true;
+        if (recovered) break;
       } catch (IOException e) {
         e = RemoteExceptionHandler.checkIOException(e);
         if (e instanceof AlreadyBeingCreatedException) {
@@ -111,9 +128,9 @@ public class FSHDFSUtils extends FSUtils
       }
       try {
         Thread.sleep(1000);
-      } catch (InterruptedException ex) {
+      } catch (InterruptedException ie) {
         InterruptedIOException iioe = new InterruptedIOException();
-        iioe.initCause(ex);
+        iioe.initCause(ie);
         throw iioe;
       }
     }

Modified: hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1450792&r1=1450791&r2=1450792&view=diff
==============================================================================
--- hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.95/hbase-server/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Wed Feb 27 15:00:08 2013
@@ -39,6 +39,7 @@ import org.apache.hadoop.hbase.*;
 import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
 import org.apache.hadoop.hbase.regionserver.wal.HLogUtil;
 import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSHDFSUtils;
 import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hbase.Coprocessor;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
@@ -49,7 +50,6 @@ import org.apache.hadoop.hdfs.MiniDFSClu
 import org.apache.hadoop.hdfs.protocol.FSConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
-import org.apache.hadoop.io.SequenceFile;
 import org.apache.log4j.Level;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -79,7 +79,7 @@ public class TestHLog  {
   private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility();
   private static Path hbaseDir;
   private static Path oldLogDir;
-
+  
   @Before
   public void setUp() throws Exception {
 
@@ -99,6 +99,7 @@ public class TestHLog  {
     // Make block sizes small.
     TEST_UTIL.getConfiguration().setInt("dfs.blocksize", 1024 * 1024);
     // needed for testAppendClose()
+    TEST_UTIL.getConfiguration().setBoolean("dfs.support.broken.append", true);
     TEST_UTIL.getConfiguration().setBoolean("dfs.support.append", true);
     // quicker heartbeat interval for faster DN death notification
     TEST_UTIL.getConfiguration().setInt("heartbeat.recheck.interval", 5000);
@@ -370,18 +371,30 @@ public class TestHLog  {
     }
   }
   
-  // For this test to pass, requires:
-  // 1. HDFS-200 (append support)
-  // 2. HDFS-988 (SafeMode should freeze file operations
-  //              [FSNamesystem.nextGenerationStampForBlock])
-  // 3. HDFS-142 (on restart, maintain pendingCreates)
+  /*
+   * We pass different values to recoverFileLease() so that different code paths are covered
+   * 
+   * For this test to pass, requires:
+   * 1. HDFS-200 (append support)
+   * 2. HDFS-988 (SafeMode should freeze file operations
+   *              [FSNamesystem.nextGenerationStampForBlock])
+   * 3. HDFS-142 (on restart, maintain pendingCreates)
+   */
   @Test
   public void testAppendClose() throws Exception {
+    testAppendClose(true);
+    testAppendClose(false);
+  }
+  
+  /*
+   * @param triggerDirectAppend whether to trigger direct call of fs.append()
+   */
+  public void testAppendClose(final boolean triggerDirectAppend) throws Exception {
     byte [] tableName = Bytes.toBytes(getName());
     HRegionInfo regioninfo = new HRegionInfo(tableName,
              HConstants.EMPTY_START_ROW, HConstants.EMPTY_END_ROW, false);
 
-    HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir", 
+    HLog wal = HLogFactory.createHLog(fs, dir, "hlogdir" + triggerDirectAppend, 
         "hlogdir_archive", conf);
     final int total = 20;
 
@@ -456,6 +469,7 @@ public class TestHLog  {
       public Exception exception = null;
       public void run() {
           try {
+            rlConf.setBoolean(FSHDFSUtils.TEST_TRIGGER_DFS_APPEND, triggerDirectAppend);
             FSUtils.getInstance(fs, rlConf)
               .recoverFileLease(recoveredFs, walPath, rlConf);
           } catch (IOException e) {