You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hbase.apache.org by en...@apache.org on 2013/08/02 06:22:07 UTC

svn commit: r1509532 - in /hbase/branches/0.94/src: main/java/org/apache/hadoop/hbase/util/ main/resources/ test/java/org/apache/hadoop/hbase/regionserver/wal/ test/java/org/apache/hadoop/hbase/util/

Author: enis
Date: Fri Aug  2 04:22:07 2013
New Revision: 1509532

URL: http://svn.apache.org/r1509532
Log:
HBASE-8670 [0.94] Backport HBASE-8449,HBASE-8204 and HBASE-8699 to 0.94 (Refactor recoverLease retries and pauses)

Added:
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java
Modified:
    hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
    hbase/branches/0.94/src/main/resources/hbase-default.xml
    hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java

Modified: hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java?rev=1509532&r1=1509531&r2=1509532&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java (original)
+++ hbase/branches/0.94/src/main/java/org/apache/hadoop/hbase/util/FSHDFSUtils.java Fri Aug  2 04:22:07 2013
@@ -21,17 +21,14 @@ package org.apache.hadoop.hbase.util;
 import java.io.FileNotFoundException;
 import java.io.IOException;
 import java.io.InterruptedIOException;
-import java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.RemoteExceptionHandler;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException;
 import org.apache.hadoop.hdfs.server.namenode.LeaseExpiredException;
 
 
@@ -42,99 +39,175 @@ public class FSHDFSUtils extends FSUtils
   private static final Log LOG = LogFactory.getLog(FSHDFSUtils.class);
 
   /**
-   * Lease timeout constant, sourced from HDFS upstream.
-   * The upstream constant is defined in a private interface, so we
-   * can't reuse for compatibility reasons.
-   * NOTE: On versions earlier than Hadoop 0.23, the constant is in
-   * o.a.h.hdfs.protocol.FSConstants, while for 0.23 and above it is
-   * in o.a.h.hdfs.protocol.HdfsConstants cause of HDFS-1620.
+   * Recover the lease from HDFS, retrying multiple times.
    */
-  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)
-  throws IOException{
+  public void recoverFileLease(final FileSystem fs, final Path p,
+      Configuration conf)
+  throws IOException {
     if (!isAppendSupported(conf)) {
       LOG.warn("Running on HDFS without append enabled may result in data loss");
       return;
     }
     // lease recovery not needed for local file system case.
-    // currently, local file system doesn't implement append either.
-    if (!(fs instanceof DistributedFileSystem)) {
-      return;
-    }
-    LOG.info("Recovering file " + p);
-    long startWaiting = System.currentTimeMillis();
+    if (!(fs instanceof DistributedFileSystem)) return;
+    recoverDFSFileLease((DistributedFileSystem)fs, p, conf);
+  }
 
-    // Trying recovery
+  /*
+   * Run the dfs recover lease. recoverLease is asynchronous. It returns:
+   *    -false when it starts the lease recovery (i.e. lease recovery not *yet* done)
+   *    - true when the lease recovery has succeeded or the file is closed.
+   * But, we have to be careful.  Each time we call recoverLease, it starts the recover lease
+   * process over from the beginning.  We could put ourselves in a situation where we are
+   * doing nothing but starting a recovery, interrupting it to start again, and so on.
+   * The findings over in HBASE-8354 have it that the namenode will try to recover the lease
+   * on the file's primary node.  If all is well, it should return near immediately.  But,
+   * as is common, it is the very primary node that has crashed and so the namenode will be
+   * stuck waiting on a socket timeout before it will ask another datanode to start the
+   * recovery. It does not help if we call recoverLease in the meantime and in particular,
+   * subsequent to the socket timeout, a recoverLease invocation will cause us to start
+   * over from square one (possibly waiting on socket timeout against primary node).  So,
+   * in the below, we do the following:
+   * 1. Call recoverLease.
+   * 2. If it returns true, break.
+   * 3. If it returns false, wait a few seconds and then call it again.
+   * 4. If it returns true, break.
+   * 5. If it returns false, wait for what we think the datanode socket timeout is
+   * (configurable) and then try again.
+   * 6. If it returns true, break.
+   * 7. If it returns false, repeat starting at step 5. above.
+   *
+   * If HDFS-4525 is available, call it every second and we might be able to exit early.
+   */
+  boolean recoverDFSFileLease(final DistributedFileSystem dfs, final Path p,
+      final Configuration conf)
+  throws IOException {
+    LOG.info("Recovering lease on dfs file " + p);
+    long startWaiting = EnvironmentEdgeManager.currentTimeMillis();
+    // Default is 15 minutes. It's huge, but the idea is that if we have a major issue, HDFS
+    // usually needs 10 minutes before marking the nodes as dead. So we're putting ourselves
+    // beyond that limit 'to be safe'.
+    long recoveryTimeout = conf.getInt("hbase.lease.recovery.timeout", 900000) + startWaiting;
+    // This setting should be what the cluster dfs heartbeat is set to.
+    long firstPause = conf.getInt("hbase.lease.recovery.first.pause", 3000);
+    // This should be set to how long it'll take for us to timeout against primary datanode if it
+    // is dead.  We set it to 61 seconds, 1 second than the default READ_TIMEOUT in HDFS, the
+    // default value for DFS_CLIENT_SOCKET_TIMEOUT_KEY.
+    long subsequentPause = conf.getInt("hbase.lease.recovery.dfs.timeout", 61 * 1000);
+
+    Method isFileClosedMeth = null;
+    // whether we need to look for isFileClosed method
+    boolean findIsFileClosedMeth = true;
     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);
-    // retrying lease recovery may preempt pending lease recovery; default to waiting for 4 seconds
-    // after calling recoverLease
-    int waitingPeriod = conf.getInt("hbase.lease.recovery.waiting.period", 4000);
-    Exception ex = null;
-    while (!recovered) {
+    // We break the loop if we succeed the lease recovery, timeout, or we throw an exception.
+    for (int nbAttempt = 0; !recovered; nbAttempt++) {
+      recovered = recoverLease(dfs, nbAttempt, p, startWaiting);
+      if (recovered) break;
+      if (checkIfTimedout(conf, recoveryTimeout, nbAttempt, p, startWaiting)) break;
       try {
-        try {
-          DistributedFileSystem dfs = (DistributedFileSystem) fs;
-          if (triggerAppend) throw new IOException();
-          try {
-            recovered = (Boolean) DistributedFileSystem.class.getMethod(
-              "recoverLease", new Class[] { Path.class }).invoke(dfs, p);
-            if (!recovered) LOG.debug("recoverLease returned false");
-          } 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");
-          ex = e;
-        }
-        if (ex != null || System.currentTimeMillis() - startWaiting > recoveryTimeout) {
-          LOG.debug("trying fs.append for " + p + " with " + ex);
-          ex = null; // assume the following append() call would succeed
-          FSDataOutputStream out = fs.append(p);
-          out.close();
-          recovered = true;
-          LOG.debug("fs.append passed");
-        }
-        if (recovered) break;
-      } catch (IOException e) {
-        e = RemoteExceptionHandler.checkIOException(e);
-        if (e instanceof AlreadyBeingCreatedException) {
-          // We expect that we'll get this message while the lease is still
-          // within its soft limit, but if we get it past that, it means
-          // that the RS is holding onto the file even though it lost its
-          // znode. We could potentially abort after some time here.
-          long waitedFor = System.currentTimeMillis() - startWaiting;
-          if (waitedFor > LEASE_SOFTLIMIT_PERIOD) {
-            LOG.warn("Waited " + waitedFor + "ms for lease recovery on " + p +
-              ":" + e.getMessage());
-          }
-        } else if (e instanceof LeaseExpiredException &&
-            e.getMessage().contains("File does not exist")) {
-          // This exception comes out instead of FNFE, fix it
-          throw new FileNotFoundException(
-              "The given HLog wasn't found at " + p.toString());
+        // On the first time through wait the short 'firstPause'.
+        if (nbAttempt == 0) {
+          Thread.sleep(firstPause);
         } else {
-          throw new IOException("Failed to open " + p + " for append", e);
+          // Cycle here until subsequentPause elapses.  While spinning, check isFileClosed if
+          // available (should be in hadoop 2.0.5... not in hadoop 1 though.
+          long localStartWaiting = EnvironmentEdgeManager.currentTimeMillis();
+          while ((EnvironmentEdgeManager.currentTimeMillis() - localStartWaiting) <
+              subsequentPause) {
+            Thread.sleep(conf.getInt("hbase.lease.recovery.pause", 1000));
+            if (findIsFileClosedMeth) {
+               try {
+                 isFileClosedMeth = dfs.getClass().getMethod("isFileClosed",
+                   new Class[]{ Path.class });
+               } catch (NoSuchMethodException nsme) {
+                 LOG.debug("isFileClosed not available");
+               } finally {
+                 findIsFileClosedMeth = false;
+               }
+             }
+             if (isFileClosedMeth != null && isFileClosed(dfs, isFileClosedMeth, p)) {
+               recovered = true;
+               break;
+             }
+          }
         }
-      }
-      try {
-        Thread.sleep(waitingPeriod);
       } catch (InterruptedException ie) {
         InterruptedIOException iioe = new InterruptedIOException();
         iioe.initCause(ie);
         throw iioe;
       }
-      // we keep original behavior without retrying lease recovery
-      break;
     }
-    LOG.info("Finished lease recovery attempt for " + p);
+    return recovered;
+  }
+
+  boolean checkIfTimedout(final Configuration conf, final long recoveryTimeout,
+      final int nbAttempt, final Path p, final long startWaiting) {
+    if (recoveryTimeout < EnvironmentEdgeManager.currentTimeMillis()) {
+      LOG.warn("Cannot recoverLease after trying for " +
+        conf.getInt("hbase.lease.recovery.timeout", 900000) +
+        "ms (hbase.lease.recovery.timeout); continuing, but may be DATALOSS!!!; " +
+        getLogMessageDetail(nbAttempt, p, startWaiting));
+      return true;
+    }
+    return false;
+  }
+
+  /**
+   * Try to recover the lease.
+   * @param dfs
+   * @param nbAttempt
+   * @param p
+   * @param startWaiting
+   * @return True if dfs#recoverLease came by true.
+   * @throws FileNotFoundException
+   */
+  boolean recoverLease(final DistributedFileSystem dfs, final int nbAttempt, final Path p,
+      final long startWaiting)
+  throws FileNotFoundException {
+    boolean recovered = false;
+    try {
+      recovered = dfs.recoverLease(p);
+      LOG.info("recoverLease=" + recovered + ", " +
+        getLogMessageDetail(nbAttempt, p, startWaiting));
+    } catch (IOException e) {
+      if (e instanceof LeaseExpiredException && e.getMessage().contains("File does not exist")) {
+        // This exception comes out instead of FNFE, fix it
+        throw new FileNotFoundException("The given HLog wasn't found at " + p);
+      } else if (e instanceof FileNotFoundException) {
+        throw (FileNotFoundException)e;
+      }
+      LOG.warn(getLogMessageDetail(nbAttempt, p, startWaiting), e);
+    }
+    return recovered;
+  }
+
+  /**
+   * @param nbAttempt
+   * @param p
+   * @param startWaiting
+   * @return Detail to append to any log message around lease recovering.
+   */
+  private String getLogMessageDetail(final int nbAttempt, final Path p, final long startWaiting) {
+    return "attempt=" + nbAttempt + " on file=" + p + " after " +
+      (EnvironmentEdgeManager.currentTimeMillis() - startWaiting) + "ms";
+  }
+
+  /**
+   * Call HDFS-4525 isFileClosed if it is available.
+   * @param dfs
+   * @param m
+   * @param p
+   * @return True if file is closed.
+   */
+  private boolean isFileClosed(final DistributedFileSystem dfs, final Method m, final Path p) {
+    try {
+      return (Boolean) m.invoke(dfs, p);
+    } catch (SecurityException e) {
+      LOG.warn("No access", e);
+    } catch (Exception e) {
+      LOG.warn("Failed invocation for " + p.toString(), e);
+    }
+    return false;
   }
 }

Modified: hbase/branches/0.94/src/main/resources/hbase-default.xml
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/main/resources/hbase-default.xml?rev=1509532&r1=1509531&r2=1509532&view=diff
==============================================================================
--- hbase/branches/0.94/src/main/resources/hbase-default.xml (original)
+++ hbase/branches/0.94/src/main/resources/hbase-default.xml Fri Aug  2 04:22:07 2013
@@ -973,4 +973,21 @@
         is 2.
     </description>
   </property>
+  <property>
+    <name>hbase.lease.recovery.timeout</name>
+    <value>900000</value>
+    <description>
+      How long we wait on dfs lease recovery in total before giving up.
+    </description>
+  </property>
+  <property>
+    <name>hbase.lease.recovery.dfs.timeout</name>
+    <value>61000</value>
+    <description>
+        How long between dfs recover lease invocations. Should be just larger than how long
+        it takes the namenode to timeout trying to reach a datanode; usually
+        dfs.socket.timeout.  If HBase asked hdfs its cluster configs, we would not need
+        this config.  See the end of HBASE-8389 for more.
+    </description>
+  </property>
 </configuration>

Modified: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java?rev=1509532&r1=1509531&r2=1509532&view=diff
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java (original)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/regionserver/wal/TestHLog.java Fri Aug  2 04:22:07 2013
@@ -20,8 +20,8 @@
 package org.apache.hadoop.hbase.regionserver.wal;
 
 import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.fail;
 
 import java.io.IOException;
@@ -39,21 +39,25 @@ import org.apache.hadoop.fs.FSDataOutput
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hbase.*;
-import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
-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.HBaseTestingUtility;
+import org.apache.hadoop.hbase.HColumnDescriptor;
+import org.apache.hadoop.hbase.HConstants;
+import org.apache.hadoop.hbase.HRegionInfo;
+import org.apache.hadoop.hbase.HTableDescriptor;
+import org.apache.hadoop.hbase.KeyValue;
+import org.apache.hadoop.hbase.LargeTests;
 import org.apache.hadoop.hbase.coprocessor.CoprocessorHost;
 import org.apache.hadoop.hbase.coprocessor.SampleRegionWALObserver;
+import org.apache.hadoop.hbase.regionserver.wal.HLog.Reader;
+import org.apache.hadoop.hbase.util.Bytes;
+import org.apache.hadoop.hbase.util.FSUtils;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 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;
@@ -198,7 +202,7 @@ public class TestHLog  {
       }
       log.close();
       HLogSplitter logSplitter = HLogSplitter.createLogSplitter(conf,
-          hbaseDir, logdir, this.oldLogDir, this.fs);
+          hbaseDir, logdir, oldLogDir, fs);
       List<Path> splits =
         logSplitter.splitLog();
       verifySplits(splits, howmany);
@@ -371,10 +375,10 @@ public class TestHLog  {
       }
     }
   }
-  
+
   /*
    * 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
@@ -383,18 +387,10 @@ public class TestHLog  {
    */
   @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);
-    Path subdir = new Path(dir, "hlogdir" + triggerDirectAppend);
+    Path subdir = new Path(dir, "hlogdir");
     Path archdir = new Path(dir, "hlogdir_archive");
     HLog wal = new HLog(fs, subdir, archdir, conf);
     final int total = 20;
@@ -411,7 +407,7 @@ public class TestHLog  {
     wal.sync();
      int namenodePort = cluster.getNameNodePort();
     final Path walPath = wal.computeFilename();
-    
+
 
     // Stop the cluster.  (ensure restart since we're sharing MiniDFSCluster)
     try {
@@ -452,23 +448,21 @@ public class TestHLog  {
     Method setLeasePeriod = cluster.getClass()
       .getDeclaredMethod("setLeasePeriod", new Class[]{Long.TYPE, Long.TYPE});
     setLeasePeriod.setAccessible(true);
-    setLeasePeriod.invoke(cluster,
-                          new Object[]{new Long(1000), new Long(1000)});
+    setLeasePeriod.invoke(cluster, 1000L, 1000L);
     try {
       Thread.sleep(1000);
     } catch (InterruptedException e) {
       LOG.info(e);
     }
-    
+
     // Now try recovering the log, like the HMaster would do
     final FileSystem recoveredFs = fs;
     final Configuration rlConf = conf;
-    
+
     class RecoverLogThread extends Thread {
       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) {
@@ -805,7 +799,7 @@ public class TestHLog  {
     @Override
     public void logRollRequested() {
       // TODO Auto-generated method stub
-      
+
     }
 
     @Override

Added: hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java
URL: http://svn.apache.org/viewvc/hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java?rev=1509532&view=auto
==============================================================================
--- hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java (added)
+++ hbase/branches/0.94/src/test/java/org/apache/hadoop/hbase/util/TestFSHDFSUtils.java Fri Aug  2 04:22:07 2013
@@ -0,0 +1,105 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hbase.util;
+
+import static org.junit.Assert.assertTrue;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hbase.HBaseTestingUtility;
+import org.apache.hadoop.hbase.MediumTests;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.junit.Before;
+import org.junit.Test;
+import org.junit.experimental.categories.Category;
+import org.mockito.Mockito;
+
+/**
+ * Test our recoverLease loop against mocked up filesystem.
+ */
+@Category(MediumTests.class)
+public class TestFSHDFSUtils {
+  private static final HBaseTestingUtility HTU = new HBaseTestingUtility();
+  static {
+    Configuration conf = HTU.getConfiguration();
+    conf.setInt("hbase.lease.recovery.dfs.timeout", 1000);
+    conf.setInt("hbase.lease.recovery.first.pause", 10);
+    conf.setInt("hbase.lease.recovery.pause", 10);
+  };
+  private FSHDFSUtils fsHDFSUtils = new FSHDFSUtils();
+  private static Path FILE = new Path(HTU.getDataTestDir(), "file.txt");
+  long startTime = -1;
+
+  @Before
+  public void setup() {
+    this.startTime = EnvironmentEdgeManager.currentTimeMillis();
+  }
+
+  /**
+   * Test recover lease eventually succeeding.
+   * @throws IOException
+   */
+  @Test (timeout = 30000)
+  public void testRecoverLease() throws IOException {
+    DistributedFileSystem dfs = Mockito.mock(DistributedFileSystem.class);
+    // Fail four times and pass on the fifth.
+    Mockito.when(dfs.recoverLease(FILE)).
+      thenReturn(false).thenReturn(false).thenReturn(false).thenReturn(false).thenReturn(true);
+    assertTrue(this.fsHDFSUtils.recoverDFSFileLease(dfs, FILE, HTU.getConfiguration()));
+    Mockito.verify(dfs, Mockito.times(5)).recoverLease(FILE);
+    // Make sure we waited at least hbase.lease.recovery.dfs.timeout * 3 (the first two
+    // invocations will happen pretty fast... the we fall into the longer wait loop).
+    assertTrue((EnvironmentEdgeManager.currentTimeMillis() - this.startTime) >
+      (3 * HTU.getConfiguration().getInt("hbase.lease.recovery.dfs.timeout", 61000)));
+  }
+
+  /**
+   * Test that isFileClosed makes us recover lease faster.
+   * @throws IOException
+   */
+  @Test (timeout = 30000)
+  public void testIsFileClosed() throws IOException {
+    // Make this time long so it is plain we broke out because of the isFileClosed invocation.
+    HTU.getConfiguration().setInt("hbase.lease.recovery.dfs.timeout", 100000);
+    IsFileClosedDistributedFileSystem dfs = Mockito.mock(IsFileClosedDistributedFileSystem.class);
+    // Now make it so we fail the first two times -- the two fast invocations, then we fall into
+    // the long loop during which we will call isFileClosed.... the next invocation should
+    // therefore return true if we are to break the loop.
+    Mockito.when(dfs.recoverLease(FILE)).
+      thenReturn(false).thenReturn(false).thenReturn(true);
+    Mockito.when(dfs.isFileClosed(FILE)).thenReturn(true);
+    assertTrue(this.fsHDFSUtils.recoverDFSFileLease(dfs, FILE, HTU.getConfiguration()));
+    Mockito.verify(dfs, Mockito.times(2)).recoverLease(FILE);
+    Mockito.verify(dfs, Mockito.times(1)).isFileClosed(FILE);
+  }
+
+  /**
+   * Version of DFS that has HDFS-4525 in it.
+   */
+  class IsFileClosedDistributedFileSystem extends DistributedFileSystem {
+    /**
+     * Close status of a file. Copied over from HDFS-4525
+     * @return true if file is already closed
+     **/
+    public boolean isFileClosed(Path f) throws IOException{
+      return false;
+    }
+  }
+}
\ No newline at end of file