HBASE-20827 Use backoff on CallQueueTooBigException when reporting region state transition
Signed-off-by: Josh Elser <elserj@apache.org>
This commit is contained in:
parent
d9180ab6d7
commit
d5f6329fac
@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hbase.Abortable;
|
||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||
import org.apache.hadoop.hbase.CallQueueTooBigException;
|
||||
import org.apache.hadoop.hbase.ChoreService;
|
||||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||
@ -2336,8 +2337,9 @@ public class HRegionServer extends HasThread implements
|
||||
return true;
|
||||
} catch (ServiceException se) {
|
||||
IOException ioe = ProtobufUtil.getRemoteException(se);
|
||||
boolean pause = ioe instanceof ServerNotRunningYetException ||
|
||||
ioe instanceof PleaseHoldException;
|
||||
boolean pause =
|
||||
ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException
|
||||
|| ioe instanceof CallQueueTooBigException;
|
||||
if (pause) {
|
||||
// Do backoff else we flood the Master with requests.
|
||||
pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
|
||||
|
Loading…
x
Reference in New Issue
Block a user