|
@@ -40,10 +40,6 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileg
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandler;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerChain;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerException;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.NodeResourceUpdaterPlugin;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
|
|
|
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
|
|
|
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
|
|
|
import org.junit.After;
|
|
|
import org.junit.Assert;
|
|
@@ -55,10 +51,11 @@ import java.util.Map;
|
|
|
|
|
|
import static org.mockito.Matchers.any;
|
|
|
import static org.mockito.Mockito.mock;
|
|
|
-import static org.mockito.Mockito.times;
|
|
|
import static org.mockito.Mockito.verify;
|
|
|
import static org.mockito.Mockito.when;
|
|
|
|
|
|
+import java.io.IOException;
|
|
|
+
|
|
|
public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
private NodeManager nm;
|
|
|
|
|
@@ -99,38 +96,33 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
private class CustomizedResourceHandler implements ResourceHandler {
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> bootstrap(Configuration configuration)
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> bootstrap(Configuration configuration) {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> preStart(Container container)
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> preStart(Container container) {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> reacquireContainer(ContainerId containerId)
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> reacquireContainer(
|
|
|
+ ContainerId containerId) {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> updateContainer(Container container)
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> updateContainer(Container container) {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> postComplete(ContainerId containerId)
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> postComplete(ContainerId containerId) {
|
|
|
return null;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public List<PrivilegedOperation> teardown()
|
|
|
- throws ResourceHandlerException {
|
|
|
+ public List<PrivilegedOperation> teardown() {
|
|
|
return null;
|
|
|
}
|
|
|
}
|
|
@@ -155,9 +147,9 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
ContainerExecutor exec, DeletionService del,
|
|
|
NodeStatusUpdater nodeStatusUpdater,
|
|
|
ApplicationACLsManager aclsManager,
|
|
|
- LocalDirsHandlerService diskhandler) {
|
|
|
+ LocalDirsHandlerService dirsHandler) {
|
|
|
return new MyContainerManager(context, exec, del, nodeStatusUpdater,
|
|
|
- metrics, diskhandler);
|
|
|
+ metrics, dirsHandler);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -185,7 +177,7 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
|
|
|
YarnConfiguration conf = createNMConfig();
|
|
|
nm.init(conf);
|
|
|
- verify(rpm, times(1)).initialize(
|
|
|
+ verify(rpm).initialize(
|
|
|
any(Context.class));
|
|
|
}
|
|
|
|
|
@@ -206,15 +198,17 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
rpm.getNameToPlugins().get("resource1")
|
|
|
.getNodeResourceHandlerInstance();
|
|
|
|
|
|
- verify(nodeResourceUpdaterPlugin, times(1)).updateConfiguredResource(
|
|
|
- any(Resource.class));
|
|
|
+
|
|
|
+ verify(nodeResourceUpdaterPlugin)
|
|
|
+ .updateConfiguredResource(any(Resource.class));
|
|
|
}
|
|
|
|
|
|
/*
|
|
|
* Make sure ResourcePluginManager is used to initialize ResourceHandlerChain
|
|
|
*/
|
|
|
@Test(timeout = 30000)
|
|
|
- public void testLinuxContainerExecutorWithResourcePluginsEnabled() throws Exception {
|
|
|
+ public void testLinuxContainerExecutorWithResourcePluginsEnabled()
|
|
|
+ throws IOException {
|
|
|
final ResourcePluginManager rpm = stubResourcePluginmanager();
|
|
|
final LinuxContainerExecutor lce = new MyLCE();
|
|
|
|
|
@@ -223,8 +217,8 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
|
|
|
((NMContext)context).setResourcePluginManager(rpm);
|
|
|
- return new BaseNodeStatusUpdaterForTest(context, dispatcher, healthChecker,
|
|
|
- metrics, new BaseResourceTrackerForTest());
|
|
|
+ return new BaseNodeStatusUpdaterForTest(context, dispatcher,
|
|
|
+ healthChecker, metrics, new BaseResourceTrackerForTest());
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -232,9 +226,9 @@ public class TestResourcePluginManager extends NodeManagerTestBase {
|
|
|
ContainerExecutor exec, DeletionService del,
|
|
|
NodeStatusUpdater nodeStatusUpdater,
|
|
|
ApplicationACLsManager aclsManager,
|
|
|
- LocalDirsHandlerService diskhandler) {
|
|
|
+ LocalDirsHandlerService dirsHandler) {
|
|
|
return new MyContainerManager(context, exec, del, nodeStatusUpdater,
|
|
|
- metrics, diskhandler);
|
|
|
+ metrics, dirsHandler);
|
|
|
}
|
|
|
|
|
|
@Override
|