HBASE-1815 HBaseClient can get stuck in an infinite loop while attempting to contact a failed regionserver
git-svn-id: https://svn.apache.org/repos/asf/hadoop/hbase/trunk@817719 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
ad29fed7d5
commit
f90645dd77
|
@ -36,6 +36,8 @@ Release 0.21.0 - Unreleased
|
|||
HBASE-1850 src/examples/mapred do not compile after HBASE-1822
|
||||
HBASE-1853 Each time around the regionserver core loop, we clear the
|
||||
messages to pass master, even if we failed to deliver them
|
||||
HBASE-1815 HBaseClient can get stuck in an infinite loop while attempting
|
||||
to contact a failed regionserver
|
||||
|
||||
IMPROVEMENTS
|
||||
HBASE-1760 Cleanup TODOs in HTable
|
||||
|
|
|
@ -329,6 +329,8 @@ public class HConnectionManager implements HConstants {
|
|||
} catch (IOException e) {
|
||||
if (tries == numRetries - 1) {
|
||||
// This was our last chance - don't bother sleeping
|
||||
LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
|
||||
" failed; no more retrying.", e);
|
||||
break;
|
||||
}
|
||||
LOG.info("getMaster attempt " + tries + " of " + this.numRetries +
|
||||
|
@ -847,7 +849,7 @@ public class HConnectionManager implements HConstants {
|
|||
public HRegionInterface getHRegionConnection(
|
||||
HServerAddress regionServer, boolean getMaster)
|
||||
throws IOException {
|
||||
if(getMaster) {
|
||||
if (getMaster) {
|
||||
getMaster();
|
||||
}
|
||||
HRegionInterface server;
|
||||
|
@ -925,9 +927,9 @@ public class HConnectionManager implements HConstants {
|
|||
"Timed out trying to locate root region");
|
||||
}
|
||||
|
||||
// get a connection to the region server
|
||||
HRegionInterface server = getHRegionConnection(rootRegionAddress);
|
||||
try {
|
||||
// Get a connection to the region server
|
||||
HRegionInterface server = getHRegionConnection(rootRegionAddress);
|
||||
// if this works, then we're good, and we have an acceptable address,
|
||||
// so we can stop doing retries and return the result.
|
||||
server.getRegionInfo(HRegionInfo.ROOT_REGIONINFO.getRegionName());
|
||||
|
|
|
@ -26,6 +26,11 @@ import org.apache.hadoop.hbase.util.Bytes;
|
|||
*/
|
||||
public class RetriesExhaustedException extends IOException {
|
||||
private static final long serialVersionUID = 1876775844L;
|
||||
|
||||
public RetriesExhaustedException(final String msg) {
|
||||
super(msg);
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a new RetriesExhaustedException from the list of prior failures.
|
||||
* @param serverName name of HRegionServer
|
||||
|
@ -35,11 +40,9 @@ public class RetriesExhaustedException extends IOException {
|
|||
* @param exceptions List of exceptions that failed before giving up
|
||||
*/
|
||||
public RetriesExhaustedException(String serverName, final byte [] regionName,
|
||||
final byte [] row,
|
||||
int numTries, List<Throwable> exceptions) {
|
||||
final byte [] row, int numTries, List<Throwable> exceptions) {
|
||||
super(getMessage(serverName, regionName, row, numTries, exceptions));
|
||||
}
|
||||
|
||||
|
||||
private static String getMessage(String serverName, final byte [] regionName,
|
||||
final byte [] row,
|
||||
|
@ -59,4 +62,4 @@ public class RetriesExhaustedException extends IOException {
|
|||
}
|
||||
return buffer.toString();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -76,6 +76,7 @@ public class HBaseClient {
|
|||
final protected int maxIdleTime; //connections will be culled if it was idle for
|
||||
//maxIdleTime msecs
|
||||
final protected int maxRetries; //the max. no. of retries for socket connections
|
||||
final protected long failureSleep; // Time to sleep before retry on failure.
|
||||
protected boolean tcpNoDelay; // if T then disable Nagle's Algorithm
|
||||
protected boolean tcpKeepAlive; // if T then use keepalives
|
||||
protected int pingInterval; // how often sends ping to the server in msecs
|
||||
|
@ -308,10 +309,7 @@ public class HBaseClient {
|
|||
this.socket.setSoTimeout(pingInterval);
|
||||
break;
|
||||
} catch (SocketTimeoutException toe) {
|
||||
/* The max number of retries is 45,
|
||||
* which amounts to 20s*45 = 15 minutes retries.
|
||||
*/
|
||||
handleConnectionFailure(timeoutFailures++, 45, toe);
|
||||
handleConnectionFailure(timeoutFailures++, maxRetries, toe);
|
||||
} catch (IOException ie) {
|
||||
handleConnectionFailure(ioFailures++, maxRetries, ie);
|
||||
}
|
||||
|
@ -338,7 +336,7 @@ public class HBaseClient {
|
|||
/* Handle connection failures
|
||||
*
|
||||
* If the current number of retries is equal to the max number of retries,
|
||||
* stop retrying and throw the exception; Otherwise backoff 1 second and
|
||||
* stop retrying and throw the exception; Otherwise backoff N seconds and
|
||||
* try connecting again.
|
||||
*
|
||||
* This Method is only called from inside setupIOstreams(), which is
|
||||
|
@ -368,11 +366,12 @@ public class HBaseClient {
|
|||
|
||||
// otherwise back off and retry
|
||||
try {
|
||||
Thread.sleep(1000);
|
||||
Thread.sleep(failureSleep);
|
||||
} catch (InterruptedException ignored) {}
|
||||
|
||||
LOG.info("Retrying connect to server: " + remoteId.getAddress() +
|
||||
". Already tried " + curRetries + " time(s).");
|
||||
" after sleeping " + failureSleep + "ms. Already tried " + curRetries +
|
||||
" time(s).");
|
||||
}
|
||||
|
||||
/* Write the header for each connection
|
||||
|
@ -636,10 +635,11 @@ public class HBaseClient {
|
|||
SocketFactory factory) {
|
||||
this.valueClass = valueClass;
|
||||
this.maxIdleTime =
|
||||
conf.getInt("ipc.client.connection.maxidletime", 10000); //10s
|
||||
this.maxRetries = conf.getInt("ipc.client.connect.max.retries", 10);
|
||||
this.tcpNoDelay = conf.getBoolean("ipc.client.tcpnodelay", false);
|
||||
this.tcpKeepAlive = conf.getBoolean("ipc.client.tcpkeepalive", true);
|
||||
conf.getInt("hbase.ipc.client.connection.maxidletime", 10000); //10s
|
||||
this.maxRetries = conf.getInt("hbase.ipc.client.connect.max.retries", 0);
|
||||
this.failureSleep = conf.getInt("hbase.client.pause", 2000);
|
||||
this.tcpNoDelay = conf.getBoolean("hbase.ipc.client.tcpnodelay", false);
|
||||
this.tcpKeepAlive = conf.getBoolean("hbase.ipc.client.tcpkeepalive", true);
|
||||
this.pingInterval = getPingInterval(conf);
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("The ping interval is" + this.pingInterval + "ms.");
|
||||
|
|
|
@ -417,14 +417,12 @@ public class HBaseRPC {
|
|||
try {
|
||||
return getProxy(protocol, clientVersion, addr, conf);
|
||||
} catch(ConnectException se) { // namenode has not been started
|
||||
LOG.info("Server at " + addr + " not available yet, Zzzzz...");
|
||||
ioe = se;
|
||||
if (maxAttempts >= 0 && ++reconnectAttempts >= maxAttempts) {
|
||||
LOG.info("Server at " + addr + " could not be reached after " +
|
||||
reconnectAttempts + " tries, giving up.");
|
||||
throw new RetriesExhaustedException(addr.toString(), "unknown".getBytes(),
|
||||
"unknown".getBytes(), reconnectAttempts - 1,
|
||||
new ArrayList<Throwable>());
|
||||
reconnectAttempts + " tries, giving up.");
|
||||
throw new RetriesExhaustedException("Failed setting up proxy to " +
|
||||
addr.toString() + " after attempts=" + reconnectAttempts);
|
||||
}
|
||||
} catch(SocketTimeoutException te) { // namenode is busy
|
||||
LOG.info("Problem connecting to server: " + addr);
|
||||
|
|
|
@ -567,24 +567,20 @@ public class HRegionServer implements HConstants, HRegionInterface,
|
|||
if (e instanceof IOException) {
|
||||
e = RemoteExceptionHandler.checkIOException((IOException) e);
|
||||
}
|
||||
if (tries < this.numRetries) {
|
||||
LOG.warn("Processing message (Retry: " + tries + ")", e);
|
||||
tries++;
|
||||
} else {
|
||||
LOG.error("Exceeded max retries: " + this.numRetries, e);
|
||||
if (checkFileSystem()) {
|
||||
// Filesystem is OK. Something is up w/ ZK or master. Sleep
|
||||
// a little while if only to stop our logging many times a
|
||||
// millisecond.
|
||||
Thread.sleep(1000);
|
||||
}
|
||||
tries++;
|
||||
if (tries > 0 && (tries % this.numRetries) == 0) {
|
||||
// Check filesystem every so often.
|
||||
checkFileSystem();
|
||||
}
|
||||
if (this.stopRequested.get()) {
|
||||
LOG.info("Stop was requested, clearing the toDo " +
|
||||
"despite of the exception");
|
||||
toDo.clear();
|
||||
continue;
|
||||
LOG.info("Stop requested, clearing toDo despite exception");
|
||||
toDo.clear();
|
||||
continue;
|
||||
}
|
||||
LOG.warn("Attempt=" + tries, e);
|
||||
// No point retrying immediately; this is probably connection to
|
||||
// master issue. Doing below will cause us to sleep.
|
||||
lastMsg = System.currentTimeMillis();
|
||||
}
|
||||
}
|
||||
// Do some housekeeping before going to sleep
|
||||
|
|
Loading…
Reference in New Issue