HBASE-1386 NPE in housekeeping -- part 2
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@776067 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
c40632b210
commit
16c92b7ce8
|
@ -1388,11 +1388,10 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
|||
* Data structure to hold a HMsg and retries count.
|
||||
*/
|
||||
private static final class ToDoEntry {
|
||||
protected volatile int tries;
|
||||
protected final AtomicInteger tries = new AtomicInteger(0);
|
||||
protected final HMsg msg;
|
||||
|
||||
ToDoEntry(final HMsg msg) {
|
||||
this.tries = 0;
|
||||
this.msg = msg;
|
||||
}
|
||||
}
|
||||
|
@ -1487,9 +1486,9 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
|||
if (ex instanceof IOException) {
|
||||
ex = RemoteExceptionHandler.checkIOException((IOException) ex);
|
||||
}
|
||||
if(e != null && e.tries < numRetries) {
|
||||
if(e != null && e.tries.get() < numRetries) {
|
||||
LOG.warn(ex);
|
||||
e.tries++;
|
||||
e.tries.incrementAndGet();
|
||||
try {
|
||||
toDo.put(e);
|
||||
} catch (InterruptedException ie) {
|
||||
|
|
Loading…
Reference in New Issue