HADOOP-13249. RetryInvocationHandler need wrap InterruptedException in IOException when call Thread.sleep. Contributed by Zhihai Xu.
This commit is contained in:
parent
03fc6b1bb0
commit
0bbb4ddd79
@ -27,6 +27,7 @@
|
|||||||
import org.apache.hadoop.ipc.Client.ConnectionId;
|
import org.apache.hadoop.ipc.Client.ConnectionId;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
import java.lang.reflect.InvocationHandler;
|
import java.lang.reflect.InvocationHandler;
|
||||||
import java.lang.reflect.InvocationTargetException;
|
import java.lang.reflect.InvocationTargetException;
|
||||||
import java.lang.reflect.Method;
|
import java.lang.reflect.Method;
|
||||||
@ -297,7 +298,16 @@ private void handleException(final Method method, final RetryPolicy policy,
|
|||||||
log(method, isFailover, counters.failovers, retryInfo.delay, ex);
|
log(method, isFailover, counters.failovers, retryInfo.delay, ex);
|
||||||
|
|
||||||
if (retryInfo.delay > 0) {
|
if (retryInfo.delay > 0) {
|
||||||
Thread.sleep(retryInfo.delay);
|
try {
|
||||||
|
Thread.sleep(retryInfo.delay);
|
||||||
|
} catch (InterruptedException e) {
|
||||||
|
Thread.currentThread().interrupt();
|
||||||
|
LOG.warn("Interrupted while waiting to retry", e);
|
||||||
|
InterruptedIOException intIOE = new InterruptedIOException(
|
||||||
|
"Retry interrupted");
|
||||||
|
intIOE.initCause(e);
|
||||||
|
throw intIOE;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
if (isFailover) {
|
if (isFailover) {
|
||||||
|
@ -31,6 +31,7 @@
|
|||||||
import org.mockito.stubbing.Answer;
|
import org.mockito.stubbing.Answer;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
import java.io.InterruptedIOException;
|
||||||
import java.lang.reflect.UndeclaredThrowableException;
|
import java.lang.reflect.UndeclaredThrowableException;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
@ -320,7 +321,9 @@ public Throwable call() throws Exception {
|
|||||||
futureThread.get().interrupt();
|
futureThread.get().interrupt();
|
||||||
Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately
|
Throwable e = future.get(1, TimeUnit.SECONDS); // should return immediately
|
||||||
assertNotNull(e);
|
assertNotNull(e);
|
||||||
assertEquals(InterruptedException.class, e.getClass());
|
assertEquals(InterruptedIOException.class, e.getClass());
|
||||||
assertEquals("sleep interrupted", e.getMessage());
|
assertEquals("Retry interrupted", e.getMessage());
|
||||||
|
assertEquals(InterruptedException.class, e.getCause().getClass());
|
||||||
|
assertEquals("sleep interrupted", e.getCause().getMessage());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
Loading…
Reference in New Issue
Block a user