HBASE-12793 [hbck] closeRegionSilentlyAndWait() should log cause of IOException and retry until hbase.hbck.close.timeout expires (Esteban)

This commit is contained in:
tedyu 2015-01-05 15:05:35 -08:00
parent 37e1bb61f4
commit 5b53a187d2
2 changed files with 5 additions and 1 deletions

View File

@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
import org.apache.hadoop.hbase.HRegionInfo;
import org.apache.hadoop.hbase.HTableDescriptor;
import org.apache.hadoop.hbase.MetaTableAccessor;
import org.apache.hadoop.hbase.NotServingRegionException;
import org.apache.hadoop.hbase.ServerName;
import org.apache.hadoop.hbase.TableName;
import org.apache.hadoop.hbase.ZooKeeperConnectionException;
@ -167,7 +168,9 @@ public class HBaseFsckRepair {
ProtobufUtil.getRegionInfo(rs, region.getRegionName());
if (rsRegion == null) return;
} catch (IOException ioe) {
return;
if (ioe instanceof NotServingRegionException) // no need to retry again
return;
LOG.warn("Exception when retrieving regioninfo from: " + region.getRegionNameAsString(), ioe);
}
Thread.sleep(1000);
}

View File

@ -154,6 +154,7 @@ public class TestHBaseFsck {
conf.setInt("hbase.htable.threads.max", POOL_SIZE);
conf.setInt("hbase.hconnection.threads.max", 2 * POOL_SIZE);
conf.setInt("hbase.hconnection.threads.core", POOL_SIZE);
conf.setInt("hbase.hbck.close.timeout", 2 * REGION_ONLINE_TIMEOUT);
TEST_UTIL.startMiniCluster(3);
tableExecutorService = new ThreadPoolExecutor(1, POOL_SIZE, 60, TimeUnit.SECONDS,