YARN-3839. Quit throwing NMNotYetReadyException. Contributed by Manikandan R
This commit is contained in:
parent
cef2815cf4
commit
424887ecb7
@ -24,10 +24,10 @@
|
|||||||
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
import org.apache.hadoop.classification.InterfaceStability.Stable;
|
||||||
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
import org.apache.hadoop.classification.InterfaceStability.Unstable;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.CommitResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
|
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceRequest;
|
||||||
|
import org.apache.hadoop.yarn.api.protocolrecords.IncreaseContainersResourceResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerRequest;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerResponse;
|
import org.apache.hadoop.yarn.api.protocolrecords.ReInitializeContainerResponse;
|
||||||
import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
|
import org.apache.hadoop.yarn.api.protocolrecords.ResourceLocalizationRequest;
|
||||||
@ -45,7 +45,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||||
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
import org.apache.hadoop.yarn.api.records.ContainerStatus;
|
||||||
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -101,9 +100,6 @@ public interface ContainerManagementProtocol {
|
|||||||
* a allServicesMetaData map.
|
* a allServicesMetaData map.
|
||||||
* @throws YarnException
|
* @throws YarnException
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @throws NMNotYetReadyException
|
|
||||||
* This exception is thrown when NM starts from scratch but has not
|
|
||||||
* yet connected with RM.
|
|
||||||
*/
|
*/
|
||||||
@Public
|
@Public
|
||||||
@Stable
|
@Stable
|
||||||
|
@ -80,6 +80,11 @@ protected static RetryPolicy createRetryPolicy(Configuration conf,
|
|||||||
exceptionToPolicyMap.put(ConnectTimeoutException.class, retryPolicy);
|
exceptionToPolicyMap.put(ConnectTimeoutException.class, retryPolicy);
|
||||||
exceptionToPolicyMap.put(RetriableException.class, retryPolicy);
|
exceptionToPolicyMap.put(RetriableException.class, retryPolicy);
|
||||||
exceptionToPolicyMap.put(SocketException.class, retryPolicy);
|
exceptionToPolicyMap.put(SocketException.class, retryPolicy);
|
||||||
|
|
||||||
|
/*
|
||||||
|
* Still keeping this to cover case like newer client talking
|
||||||
|
* to an older version of server
|
||||||
|
*/
|
||||||
exceptionToPolicyMap.put(NMNotYetReadyException.class, retryPolicy);
|
exceptionToPolicyMap.put(NMNotYetReadyException.class, retryPolicy);
|
||||||
|
|
||||||
return RetryPolicies.retryByException(RetryPolicies.TRY_ONCE_THEN_FAIL,
|
return RetryPolicies.retryByException(RetryPolicies.TRY_ONCE_THEN_FAIL,
|
||||||
|
@ -460,8 +460,6 @@ protected void resyncWithRM() {
|
|||||||
@Override
|
@Override
|
||||||
public void run() {
|
public void run() {
|
||||||
try {
|
try {
|
||||||
LOG.info("Notifying ContainerManager to block new container-requests");
|
|
||||||
containerManager.setBlockNewContainerRequests(true);
|
|
||||||
if (!rmWorkPreservingRestartEnabled) {
|
if (!rmWorkPreservingRestartEnabled) {
|
||||||
LOG.info("Cleaning up running containers on resync");
|
LOG.info("Cleaning up running containers on resync");
|
||||||
containerManager.cleanupContainersOnNMResync();
|
containerManager.cleanupContainersOnNMResync();
|
||||||
|
@ -428,8 +428,6 @@ nodeManagerVersionId, containerReports, getRunningApplications(),
|
|||||||
.verifyRMRegistrationResponseForNodeLabels(regNMResponse));
|
.verifyRMRegistrationResponseForNodeLabels(regNMResponse));
|
||||||
|
|
||||||
LOG.info(successfullRegistrationMsg);
|
LOG.info(successfullRegistrationMsg);
|
||||||
LOG.info("Notifying ContainerManager to unblock new container-requests");
|
|
||||||
this.context.getContainerManager().setBlockNewContainerRequests(false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private List<ApplicationId> createKeepAliveApplicationList() {
|
private List<ApplicationId> createKeepAliveApplicationList() {
|
||||||
|
@ -42,8 +42,6 @@ public interface ContainerManager extends ServiceStateChangeListener,
|
|||||||
|
|
||||||
void updateQueuingLimit(ContainerQueuingLimit queuingLimit);
|
void updateQueuingLimit(ContainerQueuingLimit queuingLimit);
|
||||||
|
|
||||||
void setBlockNewContainerRequests(boolean blockNewContainerRequests);
|
|
||||||
|
|
||||||
ContainerScheduler getContainerScheduler();
|
ContainerScheduler getContainerScheduler();
|
||||||
|
|
||||||
}
|
}
|
||||||
|
@ -166,7 +166,6 @@
|
|||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Map.Entry;
|
import java.util.Map.Entry;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.atomic.AtomicBoolean;
|
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock.ReadLock;
|
||||||
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
import java.util.concurrent.locks.ReentrantReadWriteLock.WriteLock;
|
||||||
@ -204,7 +203,6 @@ private enum ReInitOp {
|
|||||||
protected final AsyncDispatcher dispatcher;
|
protected final AsyncDispatcher dispatcher;
|
||||||
|
|
||||||
private final DeletionService deletionService;
|
private final DeletionService deletionService;
|
||||||
private AtomicBoolean blockNewContainerRequests = new AtomicBoolean(false);
|
|
||||||
private boolean serviceStopped = false;
|
private boolean serviceStopped = false;
|
||||||
private final ReadLock readLock;
|
private final ReadLock readLock;
|
||||||
private final WriteLock writeLock;
|
private final WriteLock writeLock;
|
||||||
@ -550,10 +548,6 @@ protected void serviceStart() throws Exception {
|
|||||||
refreshServiceAcls(conf, new NMPolicyProvider());
|
refreshServiceAcls(conf, new NMPolicyProvider());
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.info("Blocking new container-requests as container manager rpc" +
|
|
||||||
" server is still starting.");
|
|
||||||
this.setBlockNewContainerRequests(true);
|
|
||||||
|
|
||||||
String bindHost = conf.get(YarnConfiguration.NM_BIND_HOST);
|
String bindHost = conf.get(YarnConfiguration.NM_BIND_HOST);
|
||||||
String nmAddress = conf.getTrimmed(YarnConfiguration.NM_ADDRESS);
|
String nmAddress = conf.getTrimmed(YarnConfiguration.NM_ADDRESS);
|
||||||
String hostOverride = null;
|
String hostOverride = null;
|
||||||
@ -617,7 +611,6 @@ void refreshServiceAcls(Configuration configuration,
|
|||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void serviceStop() throws Exception {
|
public void serviceStop() throws Exception {
|
||||||
setBlockNewContainerRequests(true);
|
|
||||||
this.writeLock.lock();
|
this.writeLock.lock();
|
||||||
try {
|
try {
|
||||||
serviceStopped = true;
|
serviceStopped = true;
|
||||||
@ -852,11 +845,6 @@ protected void authorizeStartAndResourceIncreaseRequest(
|
|||||||
@Override
|
@Override
|
||||||
public StartContainersResponse startContainers(
|
public StartContainersResponse startContainers(
|
||||||
StartContainersRequest requests) throws YarnException, IOException {
|
StartContainersRequest requests) throws YarnException, IOException {
|
||||||
if (blockNewContainerRequests.get()) {
|
|
||||||
throw new NMNotYetReadyException(
|
|
||||||
"Rejecting new containers as NodeManager has not"
|
|
||||||
+ " yet connected with ResourceManager");
|
|
||||||
}
|
|
||||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||||
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
||||||
authorizeUser(remoteUgi, nmTokenIdentifier);
|
authorizeUser(remoteUgi, nmTokenIdentifier);
|
||||||
@ -1113,11 +1101,6 @@ protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier(
|
|||||||
public IncreaseContainersResourceResponse increaseContainersResource(
|
public IncreaseContainersResourceResponse increaseContainersResource(
|
||||||
IncreaseContainersResourceRequest requests)
|
IncreaseContainersResourceRequest requests)
|
||||||
throws YarnException, IOException {
|
throws YarnException, IOException {
|
||||||
if (blockNewContainerRequests.get()) {
|
|
||||||
throw new NMNotYetReadyException(
|
|
||||||
"Rejecting container resource increase as NodeManager has not"
|
|
||||||
+ " yet connected with ResourceManager");
|
|
||||||
}
|
|
||||||
UserGroupInformation remoteUgi = getRemoteUgi();
|
UserGroupInformation remoteUgi = getRemoteUgi();
|
||||||
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
|
||||||
authorizeUser(remoteUgi, nmTokenIdentifier);
|
authorizeUser(remoteUgi, nmTokenIdentifier);
|
||||||
@ -1559,17 +1542,6 @@ public void handle(ContainerManagerEvent event) {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
this.blockNewContainerRequests.set(blockNewContainerRequests);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Private
|
|
||||||
@VisibleForTesting
|
|
||||||
public boolean getBlockNewContainerRequestsStatus() {
|
|
||||||
return this.blockNewContainerRequests.get();
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void stateChanged(Service service) {
|
public void stateChanged(Service service) {
|
||||||
// TODO Auto-generated method stub
|
// TODO Auto-generated method stub
|
||||||
|
@ -190,11 +190,6 @@ public void handle(LogHandlerEvent event) {
|
|||||||
};
|
};
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void authorizeStartAndResourceIncreaseRequest(
|
protected void authorizeStartAndResourceIncreaseRequest(
|
||||||
NMTokenIdentifier nmTokenIdentifier,
|
NMTokenIdentifier nmTokenIdentifier,
|
||||||
|
@ -65,7 +65,6 @@
|
|||||||
import org.apache.hadoop.yarn.api.records.URL;
|
import org.apache.hadoop.yarn.api.records.URL;
|
||||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||||
import org.apache.hadoop.yarn.event.Dispatcher;
|
import org.apache.hadoop.yarn.event.Dispatcher;
|
||||||
import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
|
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||||
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
||||||
import org.apache.hadoop.yarn.factories.RecordFactory;
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
||||||
@ -87,7 +86,6 @@
|
|||||||
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
||||||
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
||||||
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
|
||||||
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
@ -188,34 +186,6 @@ protected void testContainerPreservationOnResyncImpl(TestNodeManager1 nm,
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// This test tests new container requests are blocked when NM starts from
|
|
||||||
// scratch until it register with RM AND while NM is resyncing with RM
|
|
||||||
@SuppressWarnings("unchecked")
|
|
||||||
@Test(timeout=60000)
|
|
||||||
public void testBlockNewContainerRequestsOnStartAndResync()
|
|
||||||
throws IOException, InterruptedException, YarnException {
|
|
||||||
NodeManager nm = new TestNodeManager2();
|
|
||||||
int port = ServerSocketUtil.getPort(49154, 10);
|
|
||||||
YarnConfiguration conf = createNMConfig(port);
|
|
||||||
conf.setBoolean(YarnConfiguration.RM_WORK_PRESERVING_RECOVERY_ENABLED, false);
|
|
||||||
nm.init(conf);
|
|
||||||
nm.start();
|
|
||||||
|
|
||||||
// Start the container in running state
|
|
||||||
ContainerId cId = TestNodeManagerShutdown.createContainerId();
|
|
||||||
TestNodeManagerShutdown.startContainer(nm, cId, localFS, tmpDir,
|
|
||||||
processStartFile, port);
|
|
||||||
|
|
||||||
nm.getNMDispatcher().getEventHandler()
|
|
||||||
.handle(new NodeManagerEvent(NodeManagerEventType.RESYNC));
|
|
||||||
try {
|
|
||||||
syncBarrier.await();
|
|
||||||
} catch (BrokenBarrierException e) {
|
|
||||||
}
|
|
||||||
Assert.assertFalse(assertionFailedInThread.get());
|
|
||||||
nm.stop();
|
|
||||||
}
|
|
||||||
|
|
||||||
@SuppressWarnings("unchecked")
|
@SuppressWarnings("unchecked")
|
||||||
@Test(timeout=10000)
|
@Test(timeout=10000)
|
||||||
public void testNMshutdownWhenResyncThrowException() throws IOException,
|
public void testNMshutdownWhenResyncThrowException() throws IOException,
|
||||||
@ -493,135 +463,6 @@ protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
class TestNodeManager2 extends NodeManager {
|
|
||||||
|
|
||||||
Thread launchContainersThread = null;
|
|
||||||
@Override
|
|
||||||
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
||||||
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
|
||||||
return new TestNodeStatusUpdaterImpl2(context, dispatcher,
|
|
||||||
healthChecker, metrics);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected ContainerManagerImpl createContainerManager(Context context,
|
|
||||||
ContainerExecutor exec, DeletionService del,
|
|
||||||
NodeStatusUpdater nodeStatusUpdater, ApplicationACLsManager aclsManager,
|
|
||||||
LocalDirsHandlerService dirsHandler) {
|
|
||||||
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
|
|
||||||
metrics, dirsHandler){
|
|
||||||
@Override
|
|
||||||
public void setBlockNewContainerRequests(
|
|
||||||
boolean blockNewContainerRequests) {
|
|
||||||
if (blockNewContainerRequests) {
|
|
||||||
// start test thread right after blockNewContainerRequests is set
|
|
||||||
// true
|
|
||||||
super.setBlockNewContainerRequests(blockNewContainerRequests);
|
|
||||||
launchContainersThread = new RejectedContainersLauncherThread();
|
|
||||||
launchContainersThread.start();
|
|
||||||
} else {
|
|
||||||
// join the test thread right before blockNewContainerRequests is
|
|
||||||
// reset
|
|
||||||
try {
|
|
||||||
// stop the test thread
|
|
||||||
((RejectedContainersLauncherThread) launchContainersThread)
|
|
||||||
.setStopThreadFlag(true);
|
|
||||||
launchContainersThread.join();
|
|
||||||
((RejectedContainersLauncherThread) launchContainersThread)
|
|
||||||
.setStopThreadFlag(false);
|
|
||||||
super.setBlockNewContainerRequests(blockNewContainerRequests);
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
|
|
||||||
class TestNodeStatusUpdaterImpl2 extends MockNodeStatusUpdater {
|
|
||||||
|
|
||||||
public TestNodeStatusUpdaterImpl2(Context context, Dispatcher dispatcher,
|
|
||||||
NodeHealthCheckerService healthChecker, NodeManagerMetrics metrics) {
|
|
||||||
super(context, dispatcher, healthChecker, metrics);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
protected void rebootNodeStatusUpdaterAndRegisterWithRM() {
|
|
||||||
ConcurrentMap<ContainerId, org.apache.hadoop.yarn.server.nodemanager
|
|
||||||
.containermanager.container.Container> containers =
|
|
||||||
getNMContext().getContainers();
|
|
||||||
|
|
||||||
try {
|
|
||||||
// ensure that containers are empty before restart nodeStatusUpdater
|
|
||||||
if (!containers.isEmpty()) {
|
|
||||||
for (Container container: containers.values()) {
|
|
||||||
Assert.assertEquals(ContainerState.COMPLETE,
|
|
||||||
container.cloneAndGetContainerStatus().getState());
|
|
||||||
}
|
|
||||||
}
|
|
||||||
super.rebootNodeStatusUpdaterAndRegisterWithRM();
|
|
||||||
// After this point new containers are free to be launched, except
|
|
||||||
// containers from previous RM
|
|
||||||
// Wait here so as to sync with the main test thread.
|
|
||||||
syncBarrier.await();
|
|
||||||
} catch (InterruptedException e) {
|
|
||||||
} catch (BrokenBarrierException e) {
|
|
||||||
} catch (AssertionError ae) {
|
|
||||||
ae.printStackTrace();
|
|
||||||
assertionFailedInThread.set(true);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
class RejectedContainersLauncherThread extends Thread {
|
|
||||||
|
|
||||||
boolean isStopped = false;
|
|
||||||
public void setStopThreadFlag(boolean isStopped) {
|
|
||||||
this.isStopped = isStopped;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void run() {
|
|
||||||
int numContainers = 0;
|
|
||||||
int numContainersRejected = 0;
|
|
||||||
ContainerLaunchContext containerLaunchContext =
|
|
||||||
recordFactory.newRecordInstance(ContainerLaunchContext.class);
|
|
||||||
try {
|
|
||||||
while (!isStopped && numContainers < 10) {
|
|
||||||
StartContainerRequest scRequest =
|
|
||||||
StartContainerRequest.newInstance(containerLaunchContext,
|
|
||||||
null);
|
|
||||||
List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
|
|
||||||
list.add(scRequest);
|
|
||||||
StartContainersRequest allRequests =
|
|
||||||
StartContainersRequest.newInstance(list);
|
|
||||||
System.out.println("no. of containers to be launched: "
|
|
||||||
+ numContainers);
|
|
||||||
numContainers++;
|
|
||||||
try {
|
|
||||||
getContainerManager().startContainers(allRequests);
|
|
||||||
} catch (YarnException e) {
|
|
||||||
numContainersRejected++;
|
|
||||||
Assert.assertTrue(e.getMessage().contains(
|
|
||||||
"Rejecting new containers as NodeManager has not" +
|
|
||||||
" yet connected with ResourceManager"));
|
|
||||||
Assert.assertEquals(NMNotYetReadyException.class.getName(), e
|
|
||||||
.getClass().getName());
|
|
||||||
} catch (IOException e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
assertionFailedInThread.set(true);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
// no. of containers to be launched should equal to no. of
|
|
||||||
// containers rejected
|
|
||||||
Assert.assertEquals(numContainers, numContainersRejected);
|
|
||||||
} catch (AssertionError ae) {
|
|
||||||
assertionFailedInThread.set(true);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
class TestNodeManager3 extends NodeManager {
|
class TestNodeManager3 extends NodeManager {
|
||||||
|
|
||||||
private int registrationCount = 0;
|
private int registrationCount = 0;
|
||||||
@ -681,11 +522,6 @@ protected ContainerManagerImpl createContainerManager(Context context,
|
|||||||
LocalDirsHandlerService dirsHandler) {
|
LocalDirsHandlerService dirsHandler) {
|
||||||
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
|
return new ContainerManagerImpl(context, exec, del, nodeStatusUpdater,
|
||||||
metrics, dirsHandler){
|
metrics, dirsHandler){
|
||||||
@Override
|
|
||||||
public void
|
|
||||||
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void authorizeGetAndStopContainerRequest(
|
protected void authorizeGetAndStopContainerRequest(
|
||||||
|
@ -18,6 +18,8 @@
|
|||||||
|
|
||||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager;
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager;
|
||||||
|
|
||||||
|
import static org.mockito.Mockito.spy;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
@ -28,8 +30,6 @@
|
|||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
|
|
||||||
import org.junit.Assert;
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
@ -75,6 +75,7 @@
|
|||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||||
|
import org.apache.hadoop.yarn.server.nodemanager.executor.DeletionAsUserContext;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
|
import org.apache.hadoop.yarn.server.nodemanager.recovery.NMNullStateStoreService;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
|
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
|
||||||
@ -82,10 +83,9 @@
|
|||||||
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
||||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||||
import org.junit.After;
|
import org.junit.After;
|
||||||
|
import org.junit.Assert;
|
||||||
import org.junit.Before;
|
import org.junit.Before;
|
||||||
|
|
||||||
import static org.mockito.Mockito.spy;
|
|
||||||
|
|
||||||
public abstract class BaseContainerManagerTest {
|
public abstract class BaseContainerManagerTest {
|
||||||
|
|
||||||
protected static RecordFactory recordFactory = RecordFactoryProvider
|
protected static RecordFactory recordFactory = RecordFactoryProvider
|
||||||
@ -214,11 +214,6 @@ public void setup() throws IOException {
|
|||||||
|
|
||||||
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
|
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
|
||||||
metrics, dirsHandler) {
|
metrics, dirsHandler) {
|
||||||
@Override
|
|
||||||
public void
|
|
||||||
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
|
||||||
|
@ -127,11 +127,6 @@ public void setup() throws IOException {
|
|||||||
createContainerManager(DeletionService delSrvc) {
|
createContainerManager(DeletionService delSrvc) {
|
||||||
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
|
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
|
||||||
metrics, dirsHandler) {
|
metrics, dirsHandler) {
|
||||||
@Override
|
|
||||||
public void
|
|
||||||
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected UserGroupInformation getRemoteUgi() throws YarnException {
|
protected UserGroupInformation getRemoteUgi() throws YarnException {
|
||||||
|
@ -545,11 +545,6 @@ private ContainerManagerImpl createContainerManager(Context context,
|
|||||||
return new ContainerManagerImpl(context, exec, delSrvc,
|
return new ContainerManagerImpl(context, exec, delSrvc,
|
||||||
mock(NodeStatusUpdater.class), metrics, dirsHandler) {
|
mock(NodeStatusUpdater.class), metrics, dirsHandler) {
|
||||||
@Override
|
@Override
|
||||||
public void
|
|
||||||
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
@Override
|
|
||||||
protected void authorizeGetAndStopContainerRequest(
|
protected void authorizeGetAndStopContainerRequest(
|
||||||
ContainerId containerId, Container container,
|
ContainerId containerId, Container container,
|
||||||
boolean stopRequest, NMTokenIdentifier identifier)
|
boolean stopRequest, NMTokenIdentifier identifier)
|
||||||
@ -756,12 +751,6 @@ protected ContainersLauncher createContainersLauncher(
|
|||||||
return launcher;
|
return launcher;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public void setBlockNewContainerRequests(
|
|
||||||
boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public NMTimelinePublisher
|
public NMTimelinePublisher
|
||||||
createNMTimelinePublisher(Context context) {
|
createNMTimelinePublisher(Context context) {
|
||||||
|
@ -56,12 +56,10 @@ public TestNMProxy() throws UnsupportedFileSystemException {
|
|||||||
}
|
}
|
||||||
|
|
||||||
int retryCount = 0;
|
int retryCount = 0;
|
||||||
boolean shouldThrowNMNotYetReadyException = false;
|
|
||||||
|
|
||||||
@Before
|
@Before
|
||||||
public void setUp() throws Exception {
|
public void setUp() throws Exception {
|
||||||
containerManager.start();
|
containerManager.start();
|
||||||
containerManager.setBlockNewContainerRequests(false);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
@ -75,10 +73,6 @@ public StartContainersResponse startContainers(
|
|||||||
StartContainersRequest requests) throws YarnException, IOException {
|
StartContainersRequest requests) throws YarnException, IOException {
|
||||||
if (retryCount < 5) {
|
if (retryCount < 5) {
|
||||||
retryCount++;
|
retryCount++;
|
||||||
if (shouldThrowNMNotYetReadyException) {
|
|
||||||
// This causes super to throw an NMNotYetReadyException
|
|
||||||
containerManager.setBlockNewContainerRequests(true);
|
|
||||||
} else {
|
|
||||||
if (isRetryPolicyRetryForEver()) {
|
if (isRetryPolicyRetryForEver()) {
|
||||||
// Throw non network exception
|
// Throw non network exception
|
||||||
throw new IOException(
|
throw new IOException(
|
||||||
@ -87,10 +81,6 @@ public StartContainersResponse startContainers(
|
|||||||
throw new java.net.ConnectException("start container exception");
|
throw new java.net.ConnectException("start container exception");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
// This stops super from throwing an NMNotYetReadyException
|
|
||||||
containerManager.setBlockNewContainerRequests(false);
|
|
||||||
}
|
|
||||||
return super.startContainers(requests);
|
return super.startContainers(requests);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -131,26 +121,17 @@ public void testNMProxyRetry() throws Exception {
|
|||||||
|
|
||||||
ContainerManagementProtocol proxy = getNMProxy(conf);
|
ContainerManagementProtocol proxy = getNMProxy(conf);
|
||||||
|
|
||||||
retryCount = 0;
|
|
||||||
shouldThrowNMNotYetReadyException = false;
|
|
||||||
proxy.startContainers(allRequests);
|
proxy.startContainers(allRequests);
|
||||||
Assert.assertEquals(5, retryCount);
|
Assert.assertEquals(5, retryCount);
|
||||||
|
|
||||||
retryCount = 0;
|
retryCount = 0;
|
||||||
shouldThrowNMNotYetReadyException = false;
|
|
||||||
proxy.stopContainers(Records.newRecord(StopContainersRequest.class));
|
proxy.stopContainers(Records.newRecord(StopContainersRequest.class));
|
||||||
Assert.assertEquals(5, retryCount);
|
Assert.assertEquals(5, retryCount);
|
||||||
|
|
||||||
retryCount = 0;
|
retryCount = 0;
|
||||||
shouldThrowNMNotYetReadyException = false;
|
|
||||||
proxy.getContainerStatuses(Records
|
proxy.getContainerStatuses(Records
|
||||||
.newRecord(GetContainerStatusesRequest.class));
|
.newRecord(GetContainerStatusesRequest.class));
|
||||||
Assert.assertEquals(5, retryCount);
|
Assert.assertEquals(5, retryCount);
|
||||||
|
|
||||||
retryCount = 0;
|
|
||||||
shouldThrowNMNotYetReadyException = true;
|
|
||||||
proxy.startContainers(allRequests);
|
|
||||||
Assert.assertEquals(5, retryCount);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Test(timeout = 20000, expected = IOException.class)
|
@Test(timeout = 20000, expected = IOException.class)
|
||||||
@ -162,7 +143,6 @@ public void testShouldNotRetryForeverForNonNetworkExceptionsOnNMConnections()
|
|||||||
|
|
||||||
ContainerManagementProtocol proxy = getNMProxy(conf);
|
ContainerManagementProtocol proxy = getNMProxy(conf);
|
||||||
|
|
||||||
shouldThrowNMNotYetReadyException = false;
|
|
||||||
retryCount = 0;
|
retryCount = 0;
|
||||||
proxy.startContainers(allRequests);
|
proxy.startContainers(allRequests);
|
||||||
}
|
}
|
||||||
|
@ -77,11 +77,6 @@ protected ContainerManagerImpl createContainerManager(
|
|||||||
DeletionService delSrvc) {
|
DeletionService delSrvc) {
|
||||||
return new ContainerManagerImpl(context, exec, delSrvc,
|
return new ContainerManagerImpl(context, exec, delSrvc,
|
||||||
nodeStatusUpdater, metrics, dirsHandler) {
|
nodeStatusUpdater, metrics, dirsHandler) {
|
||||||
@Override
|
|
||||||
public void
|
|
||||||
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
|
|
||||||
// do nothing
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected UserGroupInformation getRemoteUgi() throws YarnException {
|
protected UserGroupInformation getRemoteUgi() throws YarnException {
|
||||||
|
@ -66,7 +66,6 @@
|
|||||||
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
|
|
||||||
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
|
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.MockRMApp;
|
||||||
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMAppState;
|
||||||
@ -178,7 +177,7 @@ private void testNMTokens(Configuration conf) throws Exception {
|
|||||||
|
|
||||||
NodeManager nm = yarnCluster.getNodeManager(0);
|
NodeManager nm = yarnCluster.getNodeManager(0);
|
||||||
|
|
||||||
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerNM, nm);
|
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerNM);
|
||||||
|
|
||||||
// Both id should be equal.
|
// Both id should be equal.
|
||||||
Assert.assertEquals(nmTokenSecretManagerNM.getCurrentKey().getKeyId(),
|
Assert.assertEquals(nmTokenSecretManagerNM.getCurrentKey().getKeyId(),
|
||||||
@ -412,13 +411,10 @@ private void waitForContainerToFinishOnNM(ContainerId containerId) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
protected void waitForNMToReceiveNMTokenKey(
|
protected void waitForNMToReceiveNMTokenKey(
|
||||||
NMTokenSecretManagerInNM nmTokenSecretManagerNM, NodeManager nm)
|
NMTokenSecretManagerInNM nmTokenSecretManagerNM)
|
||||||
throws InterruptedException {
|
throws InterruptedException {
|
||||||
int attempt = 60;
|
int attempt = 60;
|
||||||
ContainerManagerImpl cm =
|
while (nmTokenSecretManagerNM.getNodeId() == null && attempt-- > 0) {
|
||||||
((ContainerManagerImpl) nm.getNMContext().getContainerManager());
|
|
||||||
while ((cm.getBlockNewContainerRequestsStatus() || nmTokenSecretManagerNM
|
|
||||||
.getNodeId() == null) && attempt-- > 0) {
|
|
||||||
Thread.sleep(2000);
|
Thread.sleep(2000);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -627,7 +623,7 @@ private void testContainerToken(Configuration conf) throws IOException,
|
|||||||
nm.getNMContext().getNMTokenSecretManager();
|
nm.getNMContext().getNMTokenSecretManager();
|
||||||
String user = "test";
|
String user = "test";
|
||||||
|
|
||||||
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM, nm);
|
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM);
|
||||||
|
|
||||||
NodeId nodeId = nm.getNMContext().getNodeId();
|
NodeId nodeId = nm.getNMContext().getNodeId();
|
||||||
|
|
||||||
@ -722,7 +718,7 @@ private void testContainerTokenWithEpoch(Configuration conf)
|
|||||||
nm.getNMContext().getNMTokenSecretManager();
|
nm.getNMContext().getNMTokenSecretManager();
|
||||||
String user = "test";
|
String user = "test";
|
||||||
|
|
||||||
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM, nm);
|
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM);
|
||||||
|
|
||||||
NodeId nodeId = nm.getNMContext().getNodeId();
|
NodeId nodeId = nm.getNMContext().getNodeId();
|
||||||
|
|
||||||
|
Loading…
Reference in New Issue
Block a user