|
@@ -42,6 +42,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
|
|
|
+import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationAttemptReportResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationAttemptReport;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
@@ -147,10 +148,8 @@ public class UnmanagedApplicationManager {
|
|
|
this.registerRequest = null;
|
|
|
this.recordFactory = RecordFactoryProvider.getRecordFactory(conf);
|
|
|
this.asyncApiPollIntervalMillis = conf.getLong(
|
|
|
- YarnConfiguration.
|
|
|
- YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS,
|
|
|
- YarnConfiguration.
|
|
|
- DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
|
|
|
+ YarnConfiguration.YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS,
|
|
|
+ YarnConfiguration.DEFAULT_YARN_CLIENT_APPLICATION_CLIENT_PROTOCOL_POLL_INTERVAL_MS);
|
|
|
this.keepContainersAcrossApplicationAttempts =
|
|
|
keepContainersAcrossApplicationAttempts;
|
|
|
this.applicationSubmissionContext = originalApplicationSubmissionContext;
|
|
@@ -175,8 +174,7 @@ public class UnmanagedApplicationManager {
|
|
|
this.connectionInitiated = true;
|
|
|
|
|
|
// Blocking call to RM
|
|
|
- Token<AMRMTokenIdentifier> amrmToken =
|
|
|
- initializeUnmanagedAM(this.applicationId);
|
|
|
+ Token<AMRMTokenIdentifier> amrmToken = initializeUnmanagedAM(this.applicationId);
|
|
|
|
|
|
// Creates the UAM connection
|
|
|
createUAMProxy(amrmToken);
|
|
@@ -217,8 +215,8 @@ public class UnmanagedApplicationManager {
|
|
|
* @throws IOException if register fails
|
|
|
*/
|
|
|
public RegisterApplicationMasterResponse registerApplicationMaster(
|
|
|
- RegisterApplicationMasterRequest request)
|
|
|
- throws YarnException, IOException {
|
|
|
+ RegisterApplicationMasterRequest request) throws YarnException, IOException {
|
|
|
+
|
|
|
// Save the register request for re-register later
|
|
|
this.registerRequest = request;
|
|
|
|
|
@@ -228,16 +226,17 @@ public class UnmanagedApplicationManager {
|
|
|
this.rmProxyRelayer.registerApplicationMaster(this.registerRequest);
|
|
|
this.heartbeatHandler.resetLastResponseId();
|
|
|
|
|
|
- for (Container container : response.getContainersFromPreviousAttempts()) {
|
|
|
- LOG.debug("RegisterUAM returned existing running container {}",
|
|
|
- container.getId());
|
|
|
- }
|
|
|
- for (NMToken nmToken : response.getNMTokensFromPreviousAttempts()) {
|
|
|
- LOG.debug("RegisterUAM returned existing NM token for node {}",
|
|
|
- nmToken.getNodeId());
|
|
|
+ if (LOG.isDebugEnabled()) {
|
|
|
+ for (Container container : response.getContainersFromPreviousAttempts()) {
|
|
|
+ LOG.debug("RegisterUAM returned existing running container {}", container.getId());
|
|
|
+ }
|
|
|
+
|
|
|
+ for (NMToken nmToken : response.getNMTokensFromPreviousAttempts()) {
|
|
|
+ LOG.debug("RegisterUAM returned existing NM token for node {}", nmToken.getNodeId());
|
|
|
+ }
|
|
|
}
|
|
|
- LOG.info(
|
|
|
- "RegisterUAM returned {} existing running container and {} NM tokens",
|
|
|
+
|
|
|
+ LOG.info("RegisterUAM returned {} existing running container and {} NM tokens",
|
|
|
response.getContainersFromPreviousAttempts().size(),
|
|
|
response.getNMTokensFromPreviousAttempts().size());
|
|
|
|
|
@@ -257,8 +256,8 @@ public class UnmanagedApplicationManager {
|
|
|
* @throws IOException if finishAM call fails
|
|
|
*/
|
|
|
public FinishApplicationMasterResponse finishApplicationMaster(
|
|
|
- FinishApplicationMasterRequest request)
|
|
|
- throws YarnException, IOException {
|
|
|
+ FinishApplicationMasterRequest request) throws YarnException, IOException {
|
|
|
+
|
|
|
if (this.userUgi == null) {
|
|
|
if (this.connectionInitiated) {
|
|
|
// This is possible if the async launchUAM is still
|
|
@@ -322,8 +321,7 @@ public class UnmanagedApplicationManager {
|
|
|
LOG.info("Unmanaged AM still not successfully launched/registered yet."
|
|
|
+ " Saving the allocate request and send later.");
|
|
|
} else {
|
|
|
- throw new YarnException(
|
|
|
- "AllocateAsync should not be called before launchUAM");
|
|
|
+ throw new YarnException("AllocateAsync should not be called before launchUAM");
|
|
|
}
|
|
|
}
|
|
|
}
|
|
@@ -358,7 +356,7 @@ public class UnmanagedApplicationManager {
|
|
|
* Returns RM proxy for the specified protocol type. Unit test cases can
|
|
|
* override this method and return mock proxy instances.
|
|
|
*
|
|
|
- * @param protocol protocal of the proxy
|
|
|
+ * @param protocol protocol of the proxy
|
|
|
* @param config configuration
|
|
|
* @param user ugi for the proxy connection
|
|
|
* @param token token for the connection
|
|
@@ -411,8 +409,8 @@ public class UnmanagedApplicationManager {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- private void submitUnmanagedApp(ApplicationId appId)
|
|
|
- throws YarnException, IOException {
|
|
|
+ private void submitUnmanagedApp(ApplicationId appId) throws YarnException, IOException {
|
|
|
+
|
|
|
SubmitApplicationRequest submitRequest =
|
|
|
this.recordFactory.newRecordInstance(SubmitApplicationRequest.class);
|
|
|
|
|
@@ -422,8 +420,7 @@ public class UnmanagedApplicationManager {
|
|
|
context.setApplicationId(appId);
|
|
|
context.setApplicationName(APP_NAME + "-" + appNameSuffix);
|
|
|
if (StringUtils.isBlank(this.queueName)) {
|
|
|
- context.setQueue(this.conf.get(DEFAULT_QUEUE_CONFIG,
|
|
|
- YarnConfiguration.DEFAULT_QUEUE_NAME));
|
|
|
+ context.setQueue(this.conf.get(DEFAULT_QUEUE_CONFIG, YarnConfiguration.DEFAULT_QUEUE_NAME));
|
|
|
} else {
|
|
|
context.setQueue(this.queueName);
|
|
|
}
|
|
@@ -467,8 +464,7 @@ public class UnmanagedApplicationManager {
|
|
|
* @throws IOException if getApplicationReport fails
|
|
|
*/
|
|
|
private ApplicationAttemptReport monitorCurrentAppAttempt(ApplicationId appId,
|
|
|
- Set<YarnApplicationState> appStates,
|
|
|
- YarnApplicationAttemptState attemptState)
|
|
|
+ Set<YarnApplicationState> appStates, YarnApplicationAttemptState attemptState)
|
|
|
throws YarnException, IOException {
|
|
|
|
|
|
long startTime = System.currentTimeMillis();
|
|
@@ -495,25 +491,26 @@ public class UnmanagedApplicationManager {
|
|
|
}
|
|
|
|
|
|
if (appAttemptId != null) {
|
|
|
- GetApplicationAttemptReportRequest req = this.recordFactory
|
|
|
- .newRecordInstance(GetApplicationAttemptReportRequest.class);
|
|
|
+ GetApplicationAttemptReportRequest req =
|
|
|
+ this.recordFactory.newRecordInstance(GetApplicationAttemptReportRequest.class);
|
|
|
req.setApplicationAttemptId(appAttemptId);
|
|
|
- ApplicationAttemptReport attemptReport = this.rmClient
|
|
|
- .getApplicationAttemptReport(req).getApplicationAttemptReport();
|
|
|
- if (attemptState
|
|
|
- .equals(attemptReport.getYarnApplicationAttemptState())) {
|
|
|
+ GetApplicationAttemptReportResponse appAttemptReport =
|
|
|
+ this.rmClient.getApplicationAttemptReport(req);
|
|
|
+ ApplicationAttemptReport attemptReport = appAttemptReport.getApplicationAttemptReport();
|
|
|
+ YarnApplicationAttemptState appAttemptState =
|
|
|
+ attemptReport.getYarnApplicationAttemptState();
|
|
|
+ if (attemptState.equals(appAttemptState)) {
|
|
|
return attemptReport;
|
|
|
}
|
|
|
- LOG.info("Current attempt state of " + appAttemptId + " is "
|
|
|
- + attemptReport.getYarnApplicationAttemptState()
|
|
|
- + ", waiting for current attempt to reach " + attemptState);
|
|
|
+ LOG.info("Current attempt state of {} is {}, waiting for current attempt to reach {}.",
|
|
|
+ appAttemptId, appAttemptState, attemptState);
|
|
|
}
|
|
|
|
|
|
try {
|
|
|
Thread.sleep(this.asyncApiPollIntervalMillis);
|
|
|
} catch (InterruptedException e) {
|
|
|
- LOG.warn("Interrupted while waiting for current attempt of " + appId
|
|
|
- + " to reach " + attemptState);
|
|
|
+ LOG.warn("Interrupted while waiting for current attempt of {} to reach {}.",
|
|
|
+ appId, attemptState);
|
|
|
}
|
|
|
|
|
|
if (System.currentTimeMillis() - startTime > AM_STATE_WAIT_TIMEOUT_MS) {
|
|
@@ -538,8 +535,7 @@ public class UnmanagedApplicationManager {
|
|
|
if (amrmToken != null) {
|
|
|
token = ConverterUtils.convertFromYarn(amrmToken, (Text) null);
|
|
|
} else {
|
|
|
- LOG.warn(
|
|
|
- "AMRMToken not found in the application report for application: {}",
|
|
|
+ LOG.warn("AMRMToken not found in the application report for application: {}",
|
|
|
this.applicationId);
|
|
|
}
|
|
|
return token;
|