|
@@ -24,6 +24,7 @@ import java.io.IOException;
|
|
|
import org.apache.commons.logging.Log;
|
|
|
import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.NodeHealthCheckerService;
|
|
|
+import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.FileContext;
|
|
|
import org.apache.hadoop.fs.Path;
|
|
|
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
|
|
@@ -50,6 +51,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.recovery.StoreFactory;
|
|
|
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
|
|
|
import org.apache.hadoop.yarn.service.AbstractService;
|
|
|
import org.apache.hadoop.yarn.service.CompositeService;
|
|
|
+import org.apache.hadoop.yarn.service.Service.STATE;
|
|
|
|
|
|
public class MiniYARNCluster extends CompositeService {
|
|
|
|
|
@@ -60,15 +62,19 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
DefaultMetricsSystem.setMiniClusterMode(true);
|
|
|
}
|
|
|
|
|
|
- private NodeManager nodeManager;
|
|
|
+ private NodeManager[] nodeManagers;
|
|
|
private ResourceManager resourceManager;
|
|
|
|
|
|
private ResourceManagerWrapper resourceManagerWrapper;
|
|
|
- private NodeManagerWrapper nodeManagerWrapper;
|
|
|
|
|
|
private File testWorkDir;
|
|
|
|
|
|
public MiniYARNCluster(String testName) {
|
|
|
+ //default number of nodeManagers = 1
|
|
|
+ this(testName, 1);
|
|
|
+ }
|
|
|
+
|
|
|
+ public MiniYARNCluster(String testName, int noOfNodeManagers) {
|
|
|
super(testName);
|
|
|
this.testWorkDir = new File("target", testName);
|
|
|
try {
|
|
@@ -80,8 +86,11 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
}
|
|
|
resourceManagerWrapper = new ResourceManagerWrapper();
|
|
|
addService(resourceManagerWrapper);
|
|
|
- nodeManagerWrapper = new NodeManagerWrapper();
|
|
|
- addService(nodeManagerWrapper);
|
|
|
+ nodeManagers = new CustomNodeManager[noOfNodeManagers];
|
|
|
+ for(int index = 0; index < noOfNodeManagers; index++) {
|
|
|
+ addService(new NodeManagerWrapper(index));
|
|
|
+ nodeManagers[index] = new CustomNodeManager();
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
public File getTestWorkDir() {
|
|
@@ -92,10 +101,10 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
return this.resourceManager;
|
|
|
}
|
|
|
|
|
|
- public NodeManager getNodeManager() {
|
|
|
- return this.nodeManager;
|
|
|
+ public NodeManager getNodeManager(int i) {
|
|
|
+ return this.nodeManagers[i];
|
|
|
}
|
|
|
-
|
|
|
+
|
|
|
private class ResourceManagerWrapper extends AbstractService {
|
|
|
public ResourceManagerWrapper() {
|
|
|
super(ResourceManagerWrapper.class.getName());
|
|
@@ -145,106 +154,60 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
}
|
|
|
|
|
|
private class NodeManagerWrapper extends AbstractService {
|
|
|
- public NodeManagerWrapper() {
|
|
|
- super(NodeManagerWrapper.class.getName());
|
|
|
+ int index = 0;
|
|
|
+
|
|
|
+ public NodeManagerWrapper(int i) {
|
|
|
+ super(NodeManagerWrapper.class.getName() + "_" + i);
|
|
|
+ index = i;
|
|
|
}
|
|
|
|
|
|
+ public synchronized void init(Configuration conf) {
|
|
|
+ Configuration config = new Configuration(conf);
|
|
|
+ super.init(config);
|
|
|
+ }
|
|
|
+
|
|
|
public synchronized void start() {
|
|
|
try {
|
|
|
- File localDir =
|
|
|
- new File(testWorkDir, MiniYARNCluster.this.getName() + "-localDir");
|
|
|
+ File localDir = new File(testWorkDir, MiniYARNCluster.this.getName()
|
|
|
+ + "-localDir-nm-" + index);
|
|
|
localDir.mkdir();
|
|
|
LOG.info("Created localDir in " + localDir.getAbsolutePath());
|
|
|
- getConfig().set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
|
|
|
+ getConfig().set(YarnConfiguration.NM_LOCAL_DIRS,
|
|
|
+ localDir.getAbsolutePath());
|
|
|
File logDir =
|
|
|
new File(testWorkDir, MiniYARNCluster.this.getName()
|
|
|
- + "-logDir");
|
|
|
+ + "-logDir-nm-" + index);
|
|
|
File remoteLogDir =
|
|
|
- new File(testWorkDir, MiniYARNCluster.this.getName()
|
|
|
- + "-remoteLogDir");
|
|
|
+ new File(testWorkDir, MiniYARNCluster.this.getName()
|
|
|
+ + "-remoteLogDir-nm-" + index);
|
|
|
logDir.mkdir();
|
|
|
remoteLogDir.mkdir();
|
|
|
LOG.info("Created logDir in " + logDir.getAbsolutePath());
|
|
|
- getConfig().set(YarnConfiguration.NM_LOG_DIRS, logDir.getAbsolutePath());
|
|
|
+ getConfig().set(YarnConfiguration.NM_LOG_DIRS,
|
|
|
+ logDir.getAbsolutePath());
|
|
|
getConfig().set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR,
|
|
|
- remoteLogDir.getAbsolutePath());
|
|
|
- getConfig().setInt(YarnConfiguration.NM_PMEM_MB, 4*1024); // By default AM + 2 containers
|
|
|
- nodeManager = new NodeManager() {
|
|
|
-
|
|
|
- @Override
|
|
|
- protected void doSecureLogin() throws IOException {
|
|
|
- // Don't try to login using keytab in the testcase.
|
|
|
- };
|
|
|
-
|
|
|
- @Override
|
|
|
- protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
- Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
|
|
|
- ContainerTokenSecretManager containerTokenSecretManager) {
|
|
|
- return new NodeStatusUpdaterImpl(context, dispatcher,
|
|
|
- healthChecker, metrics, containerTokenSecretManager) {
|
|
|
- @Override
|
|
|
- protected ResourceTracker getRMClient() {
|
|
|
- final ResourceTrackerService rt = resourceManager
|
|
|
- .getResourceTrackerService();
|
|
|
- final RecordFactory recordFactory =
|
|
|
- RecordFactoryProvider.getRecordFactory(null);
|
|
|
-
|
|
|
- // For in-process communication without RPC
|
|
|
- return new ResourceTracker() {
|
|
|
-
|
|
|
- @Override
|
|
|
- public NodeHeartbeatResponse nodeHeartbeat(
|
|
|
- NodeHeartbeatRequest request) throws YarnRemoteException {
|
|
|
- NodeHeartbeatResponse response = recordFactory.newRecordInstance(
|
|
|
- NodeHeartbeatResponse.class);
|
|
|
- try {
|
|
|
- response.setHeartbeatResponse(rt.nodeHeartbeat(request)
|
|
|
- .getHeartbeatResponse());
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.info("Exception in heartbeat from node " +
|
|
|
- request.getNodeStatus().getNodeId(), ioe);
|
|
|
- throw RPCUtil.getRemoteException(ioe);
|
|
|
- }
|
|
|
- return response;
|
|
|
- }
|
|
|
-
|
|
|
- @Override
|
|
|
- public RegisterNodeManagerResponse registerNodeManager(
|
|
|
- RegisterNodeManagerRequest request)
|
|
|
- throws YarnRemoteException {
|
|
|
- RegisterNodeManagerResponse response = recordFactory.newRecordInstance(
|
|
|
- RegisterNodeManagerResponse.class);
|
|
|
- try {
|
|
|
- response.setRegistrationResponse(rt
|
|
|
- .registerNodeManager(request)
|
|
|
- .getRegistrationResponse());
|
|
|
- } catch (IOException ioe) {
|
|
|
- LOG.info("Exception in node registration from "
|
|
|
- + request.getNodeId().toString(), ioe);
|
|
|
- throw RPCUtil.getRemoteException(ioe);
|
|
|
- }
|
|
|
- return response;
|
|
|
- }
|
|
|
- };
|
|
|
- };
|
|
|
- };
|
|
|
- };
|
|
|
- };
|
|
|
- nodeManager.init(getConfig());
|
|
|
+ remoteLogDir.getAbsolutePath());
|
|
|
+ // By default AM + 2 containers
|
|
|
+ getConfig().setInt(YarnConfiguration.NM_PMEM_MB, 4*1024);
|
|
|
+ getConfig().set(YarnConfiguration.NM_ADDRESS, "0.0.0.0:0");
|
|
|
+ getConfig().set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "0.0.0.0:0");
|
|
|
+ getConfig().set(YarnConfiguration.NM_WEBAPP_ADDRESS, "0.0.0.0:0");
|
|
|
+ LOG.info("Starting NM: " + index);
|
|
|
+ nodeManagers[index].init(getConfig());
|
|
|
new Thread() {
|
|
|
public void run() {
|
|
|
- nodeManager.start();
|
|
|
+ nodeManagers[index].start();
|
|
|
};
|
|
|
}.start();
|
|
|
int waitCount = 0;
|
|
|
- while (nodeManager.getServiceState() == STATE.INITED
|
|
|
+ while (nodeManagers[index].getServiceState() == STATE.INITED
|
|
|
&& waitCount++ < 60) {
|
|
|
- LOG.info("Waiting for NM to start...");
|
|
|
+ LOG.info("Waiting for NM " + index + " to start...");
|
|
|
Thread.sleep(1000);
|
|
|
}
|
|
|
- if (nodeManager.getServiceState() != STATE.STARTED) {
|
|
|
+ if (nodeManagers[index].getServiceState() != STATE.STARTED) {
|
|
|
// RM could have failed.
|
|
|
- throw new IOException("NodeManager failed to start");
|
|
|
+ throw new IOException("NodeManager " + index + " failed to start");
|
|
|
}
|
|
|
super.start();
|
|
|
} catch (Throwable t) {
|
|
@@ -254,10 +217,71 @@ public class MiniYARNCluster extends CompositeService {
|
|
|
|
|
|
@Override
|
|
|
public synchronized void stop() {
|
|
|
- if (nodeManager != null) {
|
|
|
- nodeManager.stop();
|
|
|
+ if (nodeManagers[index] != null) {
|
|
|
+ nodeManagers[index].stop();
|
|
|
}
|
|
|
super.stop();
|
|
|
}
|
|
|
}
|
|
|
+
|
|
|
+ private class CustomNodeManager extends NodeManager {
|
|
|
+ @Override
|
|
|
+ protected void doSecureLogin() throws IOException {
|
|
|
+ // Don't try to login using keytab in the testcase.
|
|
|
+ };
|
|
|
+
|
|
|
+ @Override
|
|
|
+ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
|
|
|
+ Dispatcher dispatcher, NodeHealthCheckerService healthChecker,
|
|
|
+ ContainerTokenSecretManager containerTokenSecretManager) {
|
|
|
+ return new NodeStatusUpdaterImpl(context, dispatcher,
|
|
|
+ healthChecker, metrics, containerTokenSecretManager) {
|
|
|
+ @Override
|
|
|
+ protected ResourceTracker getRMClient() {
|
|
|
+ final ResourceTrackerService rt = resourceManager
|
|
|
+ .getResourceTrackerService();
|
|
|
+ final RecordFactory recordFactory =
|
|
|
+ RecordFactoryProvider.getRecordFactory(null);
|
|
|
+
|
|
|
+ // For in-process communication without RPC
|
|
|
+ return new ResourceTracker() {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public NodeHeartbeatResponse nodeHeartbeat(
|
|
|
+ NodeHeartbeatRequest request) throws YarnRemoteException {
|
|
|
+ NodeHeartbeatResponse response = recordFactory.newRecordInstance(
|
|
|
+ NodeHeartbeatResponse.class);
|
|
|
+ try {
|
|
|
+ response.setHeartbeatResponse(rt.nodeHeartbeat(request)
|
|
|
+ .getHeartbeatResponse());
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("Exception in heartbeat from node " +
|
|
|
+ request.getNodeStatus().getNodeId(), ioe);
|
|
|
+ throw RPCUtil.getRemoteException(ioe);
|
|
|
+ }
|
|
|
+ return response;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public RegisterNodeManagerResponse registerNodeManager(
|
|
|
+ RegisterNodeManagerRequest request)
|
|
|
+ throws YarnRemoteException {
|
|
|
+ RegisterNodeManagerResponse response = recordFactory.
|
|
|
+ newRecordInstance(RegisterNodeManagerResponse.class);
|
|
|
+ try {
|
|
|
+ response.setRegistrationResponse(rt
|
|
|
+ .registerNodeManager(request)
|
|
|
+ .getRegistrationResponse());
|
|
|
+ } catch (IOException ioe) {
|
|
|
+ LOG.info("Exception in node registration from "
|
|
|
+ + request.getNodeId().toString(), ioe);
|
|
|
+ throw RPCUtil.getRemoteException(ioe);
|
|
|
+ }
|
|
|
+ return response;
|
|
|
+ }
|
|
|
+ };
|
|
|
+ };
|
|
|
+ };
|
|
|
+ };
|
|
|
+ }
|
|
|
}
|