HBASE-13732 TestHBaseFsck#testParallelWithRetriesHbck fails intermittently (Stephen Yuan Jiang, ADDENDUM for failing tests)

This commit is contained in:
Enis Soztutar 2015-06-02 17:26:35 -07:00
parent b7c0ac7cd7
commit de01553bc4
2 changed files with 11 additions and 3 deletions

View File

@ -201,7 +201,11 @@ public class HBaseFsck extends Configured implements Closeable {
private static final int DEFAULT_MAX_LOCK_FILE_ATTEMPTS = 5; 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_SLEEP_INTERVAL = 200; // milliseconds
private static final int DEFAULT_LOCK_FILE_ATTEMPT_MAX_SLEEP_TIME = 5000; // 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 * Internal resources

View File

@ -617,8 +617,12 @@ public class TestHBaseFsck {
final Future<HBaseFsck> hbck1,hbck2; final Future<HBaseFsck> hbck1,hbck2;
// With the ExponentialBackoffPolicyWithLimit (starting with 200 milliseconds sleep time, and // 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. // max sleep time of 5 seconds), we can retry around 15 times within 80 seconds before bail out.
final int timeoutInSeconds = 60; //
// 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 sleepIntervalInMilliseconds = 200;
final int maxSleepTimeInMilliseconds = 6000; final int maxSleepTimeInMilliseconds = 6000;
final int maxRetryAttempts = 15; final int maxRetryAttempts = 15;