|
@@ -22,6 +22,7 @@ import java.io.IOException;
|
|
|
import java.net.InetSocketAddress;
|
|
|
|
|
|
import org.apache.hadoop.fs.UnsupportedFileSystemException;
|
|
|
+import org.apache.hadoop.io.retry.UnreliableInterface;
|
|
|
import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
@@ -58,8 +59,8 @@ public class TestNMProxy extends BaseContainerManagerTest {
|
|
|
|
|
|
@Before
|
|
|
public void setUp() throws Exception {
|
|
|
- conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 10000);
|
|
|
- conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_RETRY_INTERVAL_MS, 100);
|
|
|
+ containerManager.start();
|
|
|
+ containerManager.setBlockNewContainerRequests(false);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -77,7 +78,13 @@ public class TestNMProxy extends BaseContainerManagerTest {
|
|
|
// This causes super to throw an NMNotYetReadyException
|
|
|
containerManager.setBlockNewContainerRequests(true);
|
|
|
} else {
|
|
|
- throw new java.net.ConnectException("start container exception");
|
|
|
+ if (isRetryPolicyRetryForEver()) {
|
|
|
+ // Throw non network exception
|
|
|
+ throw new IOException(
|
|
|
+ new UnreliableInterface.UnreliableException());
|
|
|
+ } else {
|
|
|
+ throw new java.net.ConnectException("start container exception");
|
|
|
+ }
|
|
|
}
|
|
|
} else {
|
|
|
// This stops super from throwing an NMNotYetReadyException
|
|
@@ -86,6 +93,11 @@ public class TestNMProxy extends BaseContainerManagerTest {
|
|
|
return super.startContainers(requests);
|
|
|
}
|
|
|
|
|
|
+ private boolean isRetryPolicyRetryForEver() {
|
|
|
+ return conf.getLong(
|
|
|
+ YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 1000) == -1;
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public StopContainersResponse stopContainers(
|
|
|
StopContainersRequest requests) throws YarnException, IOException {
|
|
@@ -110,30 +122,13 @@ public class TestNMProxy extends BaseContainerManagerTest {
|
|
|
}
|
|
|
|
|
|
@Test(timeout = 20000)
|
|
|
- public void testNMProxyRetry() throws Exception {
|
|
|
- containerManager.start();
|
|
|
- containerManager.setBlockNewContainerRequests(false);
|
|
|
- StartContainersRequest allRequests =
|
|
|
- Records.newRecord(StartContainersRequest.class);
|
|
|
- ApplicationId appId = ApplicationId.newInstance(1, 1);
|
|
|
- ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
|
|
|
+ public void testNMProxyRetry() throws Exception {
|
|
|
+ conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, 10000);
|
|
|
+ conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_RETRY_INTERVAL_MS, 100);
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ Records.newRecord(StartContainersRequest.class);
|
|
|
|
|
|
- org.apache.hadoop.yarn.api.records.Token nmToken =
|
|
|
- context.getNMTokenSecretManager().createNMToken(attemptId,
|
|
|
- context.getNodeId(), user);
|
|
|
- final InetSocketAddress address =
|
|
|
- conf.getSocketAddr(YarnConfiguration.NM_BIND_HOST,
|
|
|
- YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS,
|
|
|
- YarnConfiguration.DEFAULT_NM_PORT);
|
|
|
- Token<NMTokenIdentifier> token =
|
|
|
- ConverterUtils.convertFromYarn(nmToken,
|
|
|
- SecurityUtil.buildTokenService(address));
|
|
|
- UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
|
|
|
- ugi.addToken(token);
|
|
|
-
|
|
|
- ContainerManagementProtocol proxy =
|
|
|
- NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
|
|
|
- YarnRPC.create(conf), address);
|
|
|
+ ContainerManagementProtocol proxy = getNMProxy();
|
|
|
|
|
|
retryCount = 0;
|
|
|
shouldThrowNMNotYetReadyException = false;
|
|
@@ -156,4 +151,38 @@ public class TestNMProxy extends BaseContainerManagerTest {
|
|
|
proxy.startContainers(allRequests);
|
|
|
Assert.assertEquals(5, retryCount);
|
|
|
}
|
|
|
+
|
|
|
+ @Test(timeout = 20000, expected = IOException.class)
|
|
|
+ public void testShouldNotRetryForeverForNonNetworkExceptionsOnNMConnections()
|
|
|
+ throws Exception {
|
|
|
+ conf.setLong(YarnConfiguration.CLIENT_NM_CONNECT_MAX_WAIT_MS, -1);
|
|
|
+ StartContainersRequest allRequests =
|
|
|
+ Records.newRecord(StartContainersRequest.class);
|
|
|
+
|
|
|
+ ContainerManagementProtocol proxy = getNMProxy();
|
|
|
+
|
|
|
+ shouldThrowNMNotYetReadyException = false;
|
|
|
+ retryCount = 0;
|
|
|
+ proxy.startContainers(allRequests);
|
|
|
+ }
|
|
|
+
|
|
|
+ private ContainerManagementProtocol getNMProxy() {
|
|
|
+ ApplicationId appId = ApplicationId.newInstance(1, 1);
|
|
|
+ ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
|
|
|
+
|
|
|
+ org.apache.hadoop.yarn.api.records.Token nmToken =
|
|
|
+ context.getNMTokenSecretManager().createNMToken(attemptId,
|
|
|
+ context.getNodeId(), user);
|
|
|
+ final InetSocketAddress address =
|
|
|
+ conf.getSocketAddr(YarnConfiguration.NM_BIND_HOST,
|
|
|
+ YarnConfiguration.NM_ADDRESS, YarnConfiguration.DEFAULT_NM_ADDRESS,
|
|
|
+ YarnConfiguration.DEFAULT_NM_PORT);
|
|
|
+ Token<NMTokenIdentifier> token =
|
|
|
+ ConverterUtils.convertFromYarn(nmToken,
|
|
|
+ SecurityUtil.buildTokenService(address));
|
|
|
+ UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
|
|
|
+ ugi.addToken(token);
|
|
|
+ return NMProxy.createNMProxy(conf, ContainerManagementProtocol.class, ugi,
|
|
|
+ YarnRPC.create(conf), address);
|
|
|
+ }
|
|
|
}
|