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
2a969e1d8b
commit
41bec4b8b1
|
@ -65,6 +65,7 @@ import org.apache.hadoop.fs.FileSystem;
|
||||||
import org.apache.hadoop.fs.Path;
|
import org.apache.hadoop.fs.Path;
|
||||||
import org.apache.hadoop.hbase.Abortable;
|
import org.apache.hadoop.hbase.Abortable;
|
||||||
import org.apache.hadoop.hbase.CacheEvictionStats;
|
import org.apache.hadoop.hbase.CacheEvictionStats;
|
||||||
|
import org.apache.hadoop.hbase.CallQueueTooBigException;
|
||||||
import org.apache.hadoop.hbase.ChoreService;
|
import org.apache.hadoop.hbase.ChoreService;
|
||||||
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
import org.apache.hadoop.hbase.ClockOutOfSyncException;
|
||||||
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
import org.apache.hadoop.hbase.CoordinatedStateManager;
|
||||||
|
@ -2361,8 +2362,9 @@ public class HRegionServer extends HasThread implements
|
||||||
return true;
|
return true;
|
||||||
} catch (ServiceException se) {
|
} catch (ServiceException se) {
|
||||||
IOException ioe = ProtobufUtil.getRemoteException(se);
|
IOException ioe = ProtobufUtil.getRemoteException(se);
|
||||||
boolean pause = ioe instanceof ServerNotRunningYetException ||
|
boolean pause =
|
||||||
ioe instanceof PleaseHoldException;
|
ioe instanceof ServerNotRunningYetException || ioe instanceof PleaseHoldException
|
||||||
|
|| ioe instanceof CallQueueTooBigException;
|
||||||
if (pause) {
|
if (pause) {
|
||||||
// Do backoff else we flood the Master with requests.
|
// Do backoff else we flood the Master with requests.
|
||||||
pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
|
pauseTime = ConnectionUtils.getPauseTime(INIT_PAUSE_TIME_MS, tries);
|
||||||
|
|
Loading…
Reference in New Issue