// Set our own configuration (ToolRunner only sets it prior to calling
// run())
conf = getConf();
// Our own RM Handler
ResourceManagerHandler rmHandler = new ResourceManagerHandler(conf,
appAttemptId);
// Connect
rmHandler.getAMResourceManager();
// Register
try {
rmHandler.registerApplicationMaster(masterHost, masterPort);
} catch (YarnRemoteException ex) {
LOG.error(
"Error encountered while trying to register application master", ex);
return ReturnCode.MASTER_ERROR.getCode();
}
// Get file splits, configuration, etc.
Set<ConfigurationTuple> configTuples;
try {
configTuples = getConfigurationTuples();
} catch (IOException ex) {
LOG.error("Error encountered while trying to generate configurations", ex);
return ReturnCode.MASTER_ERROR.getCode();
}
// Needed for our master service later
Map<WorkerId, StartupConfiguration> startupConf = getMasterStartupConfiguration(configTuples);
// Initial containers we want, based off of the file splits
List<ResourceRequest> requestedContainers = getRequestedContainersList(
configTuples, rmHandler);
List<ContainerId> releasedContainers = new ArrayList<ContainerId>();
// Send an initial allocation request
List<Container> allocatedContainers = new ArrayList<Container>();
try {
int needed = configTuples.size();
int got = 0;
int maxAttempts = Integer.parseInt(props.getProperty(ConfigFields.APP_ALLOCATION_MAX_ATTEMPTS, "10"));;
int attempts = 0;
List<Container> acquiredContainers = null;
while (got < needed && attempts < maxAttempts) {
LOG.info("Requesting containers" + ", got=" + got + ", needed="
+ needed + ", attempts=" + attempts + ", maxAttempts="
+ maxAttempts);
acquiredContainers = rmHandler.allocateRequest(requestedContainers,
releasedContainers).getAllocatedContainers();
got += acquiredContainers.size();
attempts++;
allocatedContainers.addAll(acquiredContainers);
acquiredContainers.clear();
LOG.info("Got allocation response, allocatedContainers="
+ acquiredContainers.size());
Thread.sleep(2500);
}
} catch (YarnRemoteException ex) {
LOG.error("Encountered an error while trying to allocate containers", ex);
return ReturnCode.MASTER_ERROR.getCode();
}
final int numContainers = configTuples.size();
/*
*
*
* TODO: fix this so we try N times to get enough containers!
*
*
*
*
*/
// Make sure we got all our containers, or else bail
if (allocatedContainers.size() < numContainers) {
LOG.info("Unable to get requried number of containers, will not continue"
+ ", needed=" + numContainers + ", allocated="
+ allocatedContainers.size());
requestedContainers.clear(); // We don't want new containers!
// Add containers into released list
for (Container c : allocatedContainers) {
releasedContainers.add(c.getId());
}
// Release containers
try {
rmHandler.allocateRequest(requestedContainers, releasedContainers);
} catch (YarnRemoteException ex) {
LOG.warn(
"Encountered an error while trying to release unwanted containers",
ex);
}
// Notify our handlers that we got a problem
rmHandler.finishApplication("Unable to allocate containers, needed "
+ numContainers + ", but got " + allocatedContainers.size(),
FinalApplicationStatus.FAILED);
// bail
return ReturnCode.MASTER_ERROR.getCode();
}
/*
* public ApplicationMasterService(InetSocketAddress masterAddr,
* HashMap<WorkerId, StartupConfiguration> workers, ComputableMaster<T>
* computable, Class<T> updatable, Map<String, String> appConf,
* Configuration conf) {
*/
// Launch our worker process, as we now expect workers to actally do
// something
LOG.info("Starting master service");
ApplicationMasterService<T> masterService = new ApplicationMasterService<T>(
masterAddr, startupConf, masterComputable, masterUpdateable, appConfig,
conf);
ExecutorService executor = Executors.newSingleThreadExecutor();
Future<Integer> masterThread = executor.submit(masterService);
// We got the number of containers we wanted, let's launch them
LOG.info("Launching child containers");
List<Thread> launchThreads = launchContainers(configTuples,
allocatedContainers);
// Use an empty list for heartbeat purposes
requestedContainers.clear();
// Some local counters. Do we really need Atomic?
AtomicInteger numCompletedContainers = new AtomicInteger();
AtomicInteger numFailedContainers = new AtomicInteger();
LOG.info("Waiting for containers to complete...");
// Go into run-loop waiting for containers to finish, also our heartbeat
while (numCompletedContainers.get() < numContainers) {
// Don't pound the RM
try {
Thread.sleep(2000);
} catch (InterruptedException ex) {
LOG.warn("Interrupted while waiting on completed containers", ex);
return ReturnCode.MASTER_ERROR.getCode();
}
// Heartbeat, effectively
List<ContainerStatus> completedContainers;
try {
completedContainers = rmHandler.allocateRequest(requestedContainers,
releasedContainers).getCompletedContainersStatuses();
} catch (YarnRemoteException ex) {
LOG.warn(
"Encountered an error while trying to heartbeat to resource manager",
ex);
continue; // Nothing to report, probably an error / endless loop
}
for (ContainerStatus cs : completedContainers) {
int exitCode = cs.getExitStatus();
if (exitCode != 0) {
numCompletedContainers.incrementAndGet();
numFailedContainers.incrementAndGet();
masterService.fail();
executor.shutdown();
// Force kill our application, fail fast?
LOG.info("At least one container failed with a non-zero exit code ("
+ exitCode + "); killing application");
rmHandler
.finishApplication(
"Failing, due to at least container coming back with an non-zero exit code.",
FinalApplicationStatus.KILLED);
return -10;
} else {
numCompletedContainers.incrementAndGet();
}
}
}
// All containers have completed
// Wait for launch threads to complete (this shouldn't really happen)
LOG.info("Containers completed");
for (Thread launchThread : launchThreads) {
try {
launchThread.join(1000);
} catch (InterruptedException ex) {
LOG.warn("Interrupted while waiting for Launcher threads to complete",
ex);
}
}
// Ensure that our master service has completed as well
if (!masterThread.isDone()) {
masterService.fail();
}
int masterExit = masterThread.get();
LOG.info("Master service completed with exitCode=" + masterExit);
executor.shutdown();
if (masterExit == 0) {
/*
// Write results to file
Path out = new Path(props.getProperty(ConfigFields.APP_OUTPUT_PATH));
FileSystem fs = out.getFileSystem(conf);
FSDataOutputStream fos = fs.create(out);
LOG.info("Writing master results to " + out.toString());
masterComputable.complete(fos);
fos.flush();
fos.close();
*/
//Path out = new Path(props.getProperty(ConfigFields.APP_OUTPUT_PATH));
String impersonatedUser = System.getenv("USER");
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(impersonatedUser);
//UserGroupInformation.createProxyUser(impersonatedUser, UserGroupInformation.getLoginUser());
ugi.doAs(new PrivilegedExceptionAction<Void>() {
public Void run() {
Path out = new Path(props.getProperty(ConfigFields.APP_OUTPUT_PATH));
FileSystem fs;
try {
fs = out.getFileSystem(conf);
FSDataOutputStream fos = fs.create(out);
LOG.info("Writing master results to " + out.toString());
masterComputable.complete(fos);
fos.flush();
fos.close();
} catch (IOException e) {
// TODO Auto-generated catch block
e.printStackTrace();
}
return null;
//FileSystem fs = FileSystem.get(conf);
//fs.mkdir( out );
}
});
/*
System.out.println( "Here we would try to write to " + out.toString() );
System.out.println( "As current user: " + UserGroupInformation.getCurrentUser().getShortUserName() );
System.out.println( "As login user: " + UserGroupInformation.getLoginUser().getShortUserName() );
System.out.println( "Env Var User: " + System.getenv("USER") );
*/
//System.out.println( "Ideally we'd be user: " + this.props.getProperty( ) );
// for (Map.Entry<String, String> entry : this.conf) {
// System.out.println("ApplicationMaster->Conf: " + entry.getKey() + " = " + entry.getValue());
// }
} else {
LOG.warn("Not writing master results, as the master came back with errors!");
}
// Application finished
ReturnCode rc = (numFailedContainers.get() == 0) ? ReturnCode.OK
: ReturnCode.CONTAINER_ERROR;
try {
if (numFailedContainers.get() == 0) {
rmHandler.finishApplication("Completed succesfully",
FinalApplicationStatus.SUCCEEDED);
} else {
String diag = "Completed with " + numFailedContainers.get()
+ " failed cotainers";
rmHandler.finishApplication(diag, FinalApplicationStatus.FAILED);
}
} catch (YarnRemoteException ex) {
LOG.warn(
"Encounterd an error while trying to send final status to resource manager",
ex);