diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java index 149c8de..5244900 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/util/HBaseFsck.java @@ -205,7 +205,11 @@ public class HBaseFsck extends Configured implements Closeable { private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5; private static final int DEFAULT_LOCK_FILE_ATTEMPT_SLEEP_INTERVAL = 200; // milliseconds private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000; // milliseconds - private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT = 30; // seconds + // We have to set the timeout value > HdfsConstants.LEASE_SOFTLIMIT_PERIOD. + // In HADOOP-2.6 and later, the Namenode proxy now created with custom RetryPolicy for + // AlreadyBeingCreatedException which is implies timeout on this operations up to + // HdfsConstants.LEASE_SOFTLIMIT_PERIOD (60 seconds). + private static final int DEFAULT_WAIT_FOR_LOCK_TIMEOUT = 80; // seconds /********************** * Internal resources @@ -432,7 +436,7 @@ public class HBaseFsck extends Configured implements Closeable { "hbase.hbck.lockfile.maxwaittime", DEFAULT_WAIT_FOR_LOCK_TIMEOUT); FSDataOutputStream stream = null; try { - stream = futureTask.get(30, TimeUnit.SECONDS); + stream = futureTask.get(timeoutInSeconds, TimeUnit.SECONDS); } catch (ExecutionException ee) { LOG.warn("Encountered exception when opening lock file", ee); } catch (InterruptedException ie) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java index ccca4d2..c6314fd 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/TestHBaseFsck.java @@ -625,8 +625,12 @@ public class TestHBaseFsck { final Future hbck1,hbck2; // With the ExponentialBackoffPolicyWithLimit (starting with 200 milliseconds sleep time, and - // max sleep time of 5 seconds), we can retry around 15 times within 60 seconds before bail out. - final int timeoutInSeconds = 60; + // max sleep time of 5 seconds), we can retry around 15 times within 80 seconds before bail out. + // + // Note: the reason to use 80 seconds is that in HADOOP-2.6 and later, the create file would + // retry up to HdfsConstants.LEASE_SOFTLIMIT_PERIOD (60 seconds). See HBASE-13574 for more + // details. + final int timeoutInSeconds = 80; final int sleepIntervalInMilliseconds = 200; final int maxSleepTimeInMilliseconds = 6000; final int maxRetryAttempts = 15;