HADOOP-12573. TestRPC.testClientBackOff failing. (Xiao Chen via stevel)
This commit is contained in:
parent
3905435d3d
commit
ee9611719f
|
@ -34,6 +34,9 @@ Release 2.9.0 - UNRELEASED
|
||||||
"hadoop.security.keystore.JavaKeyStoreProvider.password"
|
"hadoop.security.keystore.JavaKeyStoreProvider.password"
|
||||||
should be updated with new name. (Surendra Singh Lilhore via stevel)
|
should be updated with new name. (Surendra Singh Lilhore via stevel)
|
||||||
|
|
||||||
|
HADOOP-12573. TestRPC.testClientBackOff failing.
|
||||||
|
(Xiao Chen via stevel)
|
||||||
|
|
||||||
Release 2.8.0 - UNRELEASED
|
Release 2.8.0 - UNRELEASED
|
||||||
|
|
||||||
INCOMPATIBLE CHANGES
|
INCOMPATIBLE CHANGES
|
||||||
|
|
|
@ -26,6 +26,9 @@ import static org.junit.Assert.assertEquals;
|
||||||
import static org.junit.Assert.assertNotSame;
|
import static org.junit.Assert.assertNotSame;
|
||||||
import static org.junit.Assert.assertTrue;
|
import static org.junit.Assert.assertTrue;
|
||||||
import static org.junit.Assert.fail;
|
import static org.junit.Assert.fail;
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
|
import static org.mockito.Mockito.timeout;
|
||||||
|
import static org.mockito.Mockito.verify;
|
||||||
|
|
||||||
import java.io.Closeable;
|
import java.io.Closeable;
|
||||||
import java.io.InterruptedIOException;
|
import java.io.InterruptedIOException;
|
||||||
|
@ -67,6 +70,7 @@ import org.apache.hadoop.io.retry.RetryPolicy;
|
||||||
import org.apache.hadoop.io.retry.RetryProxy;
|
import org.apache.hadoop.io.retry.RetryProxy;
|
||||||
import org.apache.hadoop.ipc.Client.ConnectionId;
|
import org.apache.hadoop.ipc.Client.ConnectionId;
|
||||||
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
|
import org.apache.hadoop.ipc.protobuf.RpcHeaderProtos.RpcResponseHeaderProto.RpcErrorCodeProto;
|
||||||
|
import org.apache.hadoop.ipc.Server.Call;
|
||||||
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
import org.apache.hadoop.metrics2.MetricsRecordBuilder;
|
||||||
import org.apache.hadoop.net.NetUtils;
|
import org.apache.hadoop.net.NetUtils;
|
||||||
import org.apache.hadoop.security.AccessControlException;
|
import org.apache.hadoop.security.AccessControlException;
|
||||||
|
@ -82,6 +86,8 @@ import org.apache.hadoop.test.MetricsAsserts;
|
||||||
import org.apache.hadoop.test.MockitoUtil;
|
import org.apache.hadoop.test.MockitoUtil;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
import org.mockito.Mockito;
|
||||||
|
import org.mockito.internal.util.reflection.Whitebox;
|
||||||
|
|
||||||
import com.google.protobuf.DescriptorProtos;
|
import com.google.protobuf.DescriptorProtos;
|
||||||
import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
|
import com.google.protobuf.DescriptorProtos.EnumDescriptorProto;
|
||||||
|
@ -1104,8 +1110,13 @@ public class TestRPC {
|
||||||
.setBindAddress(ADDRESS).setPort(0)
|
.setBindAddress(ADDRESS).setPort(0)
|
||||||
.setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
|
.setQueueSizePerHandler(1).setNumHandlers(1).setVerbose(true)
|
||||||
.build();
|
.build();
|
||||||
|
@SuppressWarnings("unchecked")
|
||||||
|
CallQueueManager<Call> spy = spy((CallQueueManager<Call>) Whitebox
|
||||||
|
.getInternalState(server, "callQueue"));
|
||||||
|
Whitebox.setInternalState(server, "callQueue", spy);
|
||||||
server.start();
|
server.start();
|
||||||
|
|
||||||
|
Exception lastException = null;
|
||||||
final TestProtocol proxy =
|
final TestProtocol proxy =
|
||||||
RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
|
RPC.getProxy(TestProtocol.class, TestProtocol.versionID,
|
||||||
NetUtils.getConnectAddress(server), conf);
|
NetUtils.getConnectAddress(server), conf);
|
||||||
|
@ -1122,10 +1133,7 @@ public class TestRPC {
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
}));
|
}));
|
||||||
}
|
verify(spy, timeout(500).times(i + 1)).offer(Mockito.<Call>anyObject());
|
||||||
while (server.getCallQueueLen() != 1
|
|
||||||
&& countThreads(CallQueueManager.class.getName()) != 1) {
|
|
||||||
Thread.sleep(100);
|
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
proxy.sleep(100);
|
proxy.sleep(100);
|
||||||
|
@ -1133,6 +1141,8 @@ public class TestRPC {
|
||||||
IOException unwrapExeption = e.unwrapRemoteException();
|
IOException unwrapExeption = e.unwrapRemoteException();
|
||||||
if (unwrapExeption instanceof RetriableException) {
|
if (unwrapExeption instanceof RetriableException) {
|
||||||
succeeded = true;
|
succeeded = true;
|
||||||
|
} else {
|
||||||
|
lastException = unwrapExeption;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
|
@ -1140,6 +1150,9 @@ public class TestRPC {
|
||||||
RPC.stopProxy(proxy);
|
RPC.stopProxy(proxy);
|
||||||
executorService.shutdown();
|
executorService.shutdown();
|
||||||
}
|
}
|
||||||
|
if (lastException != null) {
|
||||||
|
LOG.error("Last received non-RetriableException:", lastException);
|
||||||
|
}
|
||||||
assertTrue("RetriableException not received", succeeded);
|
assertTrue("RetriableException not received", succeeded);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue