|
@@ -167,6 +167,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
private Map<SubClusterId, List<AllocateResponse>> asyncResponseSink;
|
|
|
|
|
|
+ private Map<SubClusterId, RegisterApplicationMasterResponse> uamRegistrations;
|
|
|
+
|
|
|
+ // For unit test synchronization
|
|
|
+ private Map<SubClusterId, Future<?>> uamRegisterFutures;
|
|
|
+
|
|
|
/** Thread pool used for asynchronous operations. */
|
|
|
private ExecutorService threadpool;
|
|
|
|
|
@@ -227,6 +232,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
public FederationInterceptor() {
|
|
|
this.containerIdToSubClusterIdMap = new ConcurrentHashMap<>();
|
|
|
this.asyncResponseSink = new ConcurrentHashMap<>();
|
|
|
+ this.uamRegistrations = new ConcurrentHashMap<>();
|
|
|
+ this.uamRegisterFutures = new ConcurrentHashMap<>();
|
|
|
this.threadpool = Executors.newCachedThreadPool();
|
|
|
this.uamPool = createUnmanagedAMPoolManager(this.threadpool);
|
|
|
this.secondaryRelayers = new ConcurrentHashMap<>();
|
|
@@ -279,8 +286,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
ApplicationMasterProtocol.class, appOwner), appId,
|
|
|
this.homeSubClusterId.toString());
|
|
|
|
|
|
- this.homeHeartbeartHandler = createHomeHeartbeartHandler(conf, appId);
|
|
|
- this.homeHeartbeartHandler.setAMRMClientRelayer(this.homeRMRelayer);
|
|
|
+ this.homeHeartbeartHandler =
|
|
|
+ createHomeHeartbeartHandler(conf, appId, this.homeRMRelayer);
|
|
|
this.homeHeartbeartHandler.setUGI(appOwner);
|
|
|
this.homeHeartbeartHandler.setDaemon(true);
|
|
|
this.homeHeartbeartHandler.start();
|
|
@@ -615,10 +622,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
/**
|
|
|
* Send the requests to the all sub-cluster resource managers. All
|
|
|
* requests are synchronously triggered but sent asynchronously. Later the
|
|
|
- * responses will be collected and merged. In addition, it also returns
|
|
|
- * the newly registered UAMs.
|
|
|
+ * responses will be collected and merged.
|
|
|
*/
|
|
|
- Registrations newRegistrations = sendRequestsToResourceManagers(requests);
|
|
|
+ sendRequestsToResourceManagers(requests);
|
|
|
|
|
|
// Wait for the first async response to arrive
|
|
|
long startTime = this.clock.getTime();
|
|
@@ -646,9 +652,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
// Merge the containers and NMTokens from the new registrations into
|
|
|
// the response
|
|
|
- if (!isNullOrEmpty(newRegistrations.getSuccessfulRegistrations())) {
|
|
|
- mergeRegistrationResponses(response,
|
|
|
- newRegistrations.getSuccessfulRegistrations());
|
|
|
+
|
|
|
+ if (!isNullOrEmpty(this.uamRegistrations)) {
|
|
|
+ Map<SubClusterId, RegisterApplicationMasterResponse> newRegistrations;
|
|
|
+ synchronized (this.uamRegistrations) {
|
|
|
+ newRegistrations = new HashMap<>(this.uamRegistrations);
|
|
|
+ this.uamRegistrations.clear();
|
|
|
+ }
|
|
|
+ mergeRegistrationResponses(response, newRegistrations);
|
|
|
}
|
|
|
|
|
|
// update the responseId and return the final response to AM
|
|
@@ -850,8 +861,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
@VisibleForTesting
|
|
|
protected AMHeartbeatRequestHandler createHomeHeartbeartHandler(
|
|
|
- Configuration conf, ApplicationId appId) {
|
|
|
- return new AMHeartbeatRequestHandler(conf, appId);
|
|
|
+ Configuration conf, ApplicationId appId,
|
|
|
+ AMRMClientRelayer rmProxyRelayer) {
|
|
|
+ return new AMHeartbeatRequestHandler(conf, appId, rmProxyRelayer);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1107,18 +1119,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*
|
|
|
* @param requests contains the heart beat requests to send to the resource
|
|
|
* manager keyed by the sub-cluster id
|
|
|
- * @return the registration responses from the newly added sub-cluster
|
|
|
- * resource managers
|
|
|
* @throws YarnException
|
|
|
* @throws IOException
|
|
|
*/
|
|
|
- private Registrations sendRequestsToResourceManagers(
|
|
|
+ private void sendRequestsToResourceManagers(
|
|
|
Map<SubClusterId, AllocateRequest> requests)
|
|
|
throws YarnException, IOException {
|
|
|
|
|
|
- // Create new UAM instances for the sub-cluster that we have not seen
|
|
|
- // before
|
|
|
- Registrations registrations = registerWithNewSubClusters(requests.keySet());
|
|
|
+ // Create new UAM instances for the sub-cluster that we haven't seen before
|
|
|
+ List<SubClusterId> newSubClusters =
|
|
|
+ registerAndAllocateWithNewSubClusters(requests);
|
|
|
|
|
|
// Now that all the registrations are done, send the allocation request
|
|
|
// to the sub-cluster RMs asynchronously and don't wait for the response.
|
|
@@ -1126,6 +1136,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
// response sink, then merged and sent to the application master.
|
|
|
for (Entry<SubClusterId, AllocateRequest> entry : requests.entrySet()) {
|
|
|
SubClusterId subClusterId = entry.getKey();
|
|
|
+ if (newSubClusters.contains(subClusterId)) {
|
|
|
+ // For new sub-clusters, we have already sent the request right after
|
|
|
+ // register in the async thread
|
|
|
+ continue;
|
|
|
+ }
|
|
|
|
|
|
if (subClusterId.equals(this.homeSubClusterId)) {
|
|
|
// Request for the home sub-cluster resource manager
|
|
@@ -1133,131 +1148,100 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
new HeartbeatCallBack(this.homeSubClusterId, false));
|
|
|
} else {
|
|
|
if (!this.uamPool.hasUAMId(subClusterId.getId())) {
|
|
|
- // TODO: This means that the registration for this sub-cluster RM
|
|
|
- // failed. For now, we ignore the resource requests and continue
|
|
|
- // but we need to fix this and handle this situation. One way would
|
|
|
- // be to send the request to another RM by consulting the policy.
|
|
|
- LOG.warn("Unmanaged AM registration not found for sub-cluster {}",
|
|
|
- subClusterId);
|
|
|
- continue;
|
|
|
+ throw new YarnException("UAM not found for " + this.attemptId
|
|
|
+ + " in sub-cluster " + subClusterId);
|
|
|
}
|
|
|
this.uamPool.allocateAsync(subClusterId.getId(), entry.getValue(),
|
|
|
new HeartbeatCallBack(subClusterId, true));
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return registrations;
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * This method ensures that Unmanaged AMs are created for each of the
|
|
|
- * specified sub-cluster specified in the input and registers with the
|
|
|
- * corresponding resource managers.
|
|
|
+ * This method ensures that Unmanaged AMs are created for newly specified
|
|
|
+ * sub-clusters, registers with the corresponding resource managers and send
|
|
|
+ * the first allocate request async.
|
|
|
*/
|
|
|
- private Registrations registerWithNewSubClusters(
|
|
|
- Set<SubClusterId> subClusterSet) throws IOException {
|
|
|
-
|
|
|
- List<SubClusterId> failedRegistrations = new ArrayList<>();
|
|
|
- Map<SubClusterId, RegisterApplicationMasterResponse>
|
|
|
- successfulRegistrations = new HashMap<>();
|
|
|
+ private List<SubClusterId> registerAndAllocateWithNewSubClusters(
|
|
|
+ final Map<SubClusterId, AllocateRequest> requests) throws IOException {
|
|
|
|
|
|
// Check to see if there are any new sub-clusters in this request
|
|
|
// list and create and register Unmanaged AM instance for the new ones
|
|
|
- List<String> newSubClusters = new ArrayList<>();
|
|
|
- for (SubClusterId subClusterId : subClusterSet) {
|
|
|
+ List<SubClusterId> newSubClusters = new ArrayList<>();
|
|
|
+ for (SubClusterId subClusterId : requests.keySet()) {
|
|
|
if (!subClusterId.equals(this.homeSubClusterId)
|
|
|
&& !this.uamPool.hasUAMId(subClusterId.getId())) {
|
|
|
- newSubClusters.add(subClusterId.getId());
|
|
|
+ newSubClusters.add(subClusterId);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- if (newSubClusters.size() > 0) {
|
|
|
- final RegisterApplicationMasterRequest registerRequest =
|
|
|
- this.amRegistrationRequest;
|
|
|
- final AMRMProxyApplicationContext appContext = getApplicationContext();
|
|
|
- ExecutorCompletionService<RegisterApplicationMasterResponseInfo>
|
|
|
- completionService = new ExecutorCompletionService<>(this.threadpool);
|
|
|
+ this.uamRegisterFutures.clear();
|
|
|
+ for (final SubClusterId scId : newSubClusters) {
|
|
|
+ Future<?> future = this.threadpool.submit(new Runnable() {
|
|
|
+ @Override
|
|
|
+ public void run() {
|
|
|
+ String subClusterId = scId.getId();
|
|
|
|
|
|
- for (final String subClusterId : newSubClusters) {
|
|
|
- completionService
|
|
|
- .submit(new Callable<RegisterApplicationMasterResponseInfo>() {
|
|
|
- @Override
|
|
|
- public RegisterApplicationMasterResponseInfo call()
|
|
|
- throws Exception {
|
|
|
-
|
|
|
- // Create a config loaded with federation on and subclusterId
|
|
|
- // for each UAM
|
|
|
- YarnConfiguration config = new YarnConfiguration(getConf());
|
|
|
- FederationProxyProviderUtil.updateConfForFederation(config,
|
|
|
- subClusterId);
|
|
|
-
|
|
|
- RegisterApplicationMasterResponse uamResponse = null;
|
|
|
- Token<AMRMTokenIdentifier> token = null;
|
|
|
- try {
|
|
|
- // For appNameSuffix, use subClusterId of the home sub-cluster
|
|
|
- token = uamPool.launchUAM(subClusterId, config,
|
|
|
- attemptId.getApplicationId(),
|
|
|
- amRegistrationResponse.getQueue(), appContext.getUser(),
|
|
|
- homeSubClusterId.toString(), true, subClusterId);
|
|
|
-
|
|
|
- secondaryRelayers.put(subClusterId,
|
|
|
- uamPool.getAMRMClientRelayer(subClusterId));
|
|
|
-
|
|
|
- uamResponse = uamPool.registerApplicationMaster(subClusterId,
|
|
|
- registerRequest);
|
|
|
- } catch (Throwable e) {
|
|
|
- LOG.error("Failed to register application master: "
|
|
|
- + subClusterId + " Application: " + attemptId, e);
|
|
|
- }
|
|
|
- return new RegisterApplicationMasterResponseInfo(uamResponse,
|
|
|
- SubClusterId.newInstance(subClusterId), token);
|
|
|
- }
|
|
|
- });
|
|
|
- }
|
|
|
+ // Create a config loaded with federation on and subclusterId
|
|
|
+ // for each UAM
|
|
|
+ YarnConfiguration config = new YarnConfiguration(getConf());
|
|
|
+ FederationProxyProviderUtil.updateConfForFederation(config,
|
|
|
+ subClusterId);
|
|
|
|
|
|
- // Wait for other sub-cluster resource managers to return the
|
|
|
- // response and add it to the Map for returning to the caller
|
|
|
- for (int i = 0; i < newSubClusters.size(); ++i) {
|
|
|
- try {
|
|
|
- Future<RegisterApplicationMasterResponseInfo> future =
|
|
|
- completionService.take();
|
|
|
- RegisterApplicationMasterResponseInfo uamResponse = future.get();
|
|
|
- if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("Received register application response from RM: "
|
|
|
- + uamResponse.getSubClusterId());
|
|
|
+ RegisterApplicationMasterResponse uamResponse = null;
|
|
|
+ Token<AMRMTokenIdentifier> token = null;
|
|
|
+ try {
|
|
|
+ // For appNameSuffix, use subClusterId of the home sub-cluster
|
|
|
+ token = uamPool.launchUAM(subClusterId, config,
|
|
|
+ attemptId.getApplicationId(), amRegistrationResponse.getQueue(),
|
|
|
+ getApplicationContext().getUser(), homeSubClusterId.toString(),
|
|
|
+ true, subClusterId);
|
|
|
+
|
|
|
+ secondaryRelayers.put(subClusterId,
|
|
|
+ uamPool.getAMRMClientRelayer(subClusterId));
|
|
|
+
|
|
|
+ uamResponse = uamPool.registerApplicationMaster(subClusterId,
|
|
|
+ amRegistrationRequest);
|
|
|
+ } catch (Throwable e) {
|
|
|
+ LOG.error("Failed to register application master: " + subClusterId
|
|
|
+ + " Application: " + attemptId, e);
|
|
|
+ // TODO: UAM registration for this sub-cluster RM
|
|
|
+ // failed. For now, we ignore the resource requests and continue
|
|
|
+ // but we need to fix this and handle this situation. One way would
|
|
|
+ // be to send the request to another RM by consulting the policy.
|
|
|
+ return;
|
|
|
}
|
|
|
+ uamRegistrations.put(scId, uamResponse);
|
|
|
+ LOG.info("Successfully registered unmanaged application master: "
|
|
|
+ + subClusterId + " ApplicationId: " + attemptId);
|
|
|
|
|
|
- if (uamResponse.getResponse() == null) {
|
|
|
- failedRegistrations.add(uamResponse.getSubClusterId());
|
|
|
- } else {
|
|
|
- LOG.info("Successfully registered unmanaged application master: "
|
|
|
- + uamResponse.getSubClusterId() + " ApplicationId: "
|
|
|
- + this.attemptId);
|
|
|
- successfulRegistrations.put(uamResponse.getSubClusterId(),
|
|
|
- uamResponse.getResponse());
|
|
|
+ try {
|
|
|
+ uamPool.allocateAsync(subClusterId, requests.get(scId),
|
|
|
+ new HeartbeatCallBack(scId, true));
|
|
|
+ } catch (Throwable e) {
|
|
|
+ LOG.error("Failed to allocate async to " + subClusterId
|
|
|
+ + " Application: " + attemptId, e);
|
|
|
+ }
|
|
|
|
|
|
- // Save the UAM token in registry or NMSS
|
|
|
+ // Save the UAM token in registry or NMSS
|
|
|
+ try {
|
|
|
if (registryClient != null) {
|
|
|
- registryClient.writeAMRMTokenForUAM(
|
|
|
- this.attemptId.getApplicationId(),
|
|
|
- uamResponse.getSubClusterId().getId(),
|
|
|
- uamResponse.getUamToken());
|
|
|
+ registryClient.writeAMRMTokenForUAM(attemptId.getApplicationId(),
|
|
|
+ subClusterId, token);
|
|
|
} else if (getNMStateStore() != null) {
|
|
|
- getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
|
|
|
- NMSS_SECONDARY_SC_PREFIX
|
|
|
- + uamResponse.getSubClusterId().getId(),
|
|
|
- uamResponse.getUamToken().encodeToUrlString()
|
|
|
- .getBytes(STRING_TO_BYTE_FORMAT));
|
|
|
+ getNMStateStore().storeAMRMProxyAppContextEntry(attemptId,
|
|
|
+ NMSS_SECONDARY_SC_PREFIX + subClusterId,
|
|
|
+ token.encodeToUrlString().getBytes(STRING_TO_BYTE_FORMAT));
|
|
|
}
|
|
|
+ } catch (Throwable e) {
|
|
|
+ LOG.error("Failed to persist UAM token from " + subClusterId
|
|
|
+ + " Application: " + attemptId, e);
|
|
|
}
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn("Failed to register unmanaged application master: "
|
|
|
- + " ApplicationId: " + this.attemptId, e);
|
|
|
}
|
|
|
- }
|
|
|
+ });
|
|
|
+ this.uamRegisterFutures.put(scId, future);
|
|
|
}
|
|
|
-
|
|
|
- return new Registrations(successfulRegistrations, failedRegistrations);
|
|
|
+ return newSubClusters;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1573,7 +1557,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- public int getUnmanagedAMPoolSize() {
|
|
|
+ protected int getUnmanagedAMPoolSize() {
|
|
|
return this.uamPool.getAllUAMIds().size();
|
|
|
}
|
|
|
|
|
@@ -1582,6 +1566,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
return this.uamPool;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ protected Map<SubClusterId, Future<?>> getUamRegisterFutures() {
|
|
|
+ return this.uamRegisterFutures;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public Map<SubClusterId, List<AllocateResponse>> getAsyncResponseSink() {
|
|
|
return this.asyncResponseSink;
|
|
@@ -1614,7 +1603,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
asyncResponseSink.notifyAll();
|
|
|
}
|
|
|
|
|
|
+ // Notify policy of allocate response
|
|
|
+ try {
|
|
|
+ policyInterpreter.notifyOfResponse(subClusterId, response);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.warn("notifyOfResponse for policy failed for sub-cluster "
|
|
|
+ + subClusterId, e);
|
|
|
+ }
|
|
|
+
|
|
|
// Save the new AMRMToken for the UAM if present
|
|
|
+ // Do this last because it can be slow...
|
|
|
if (this.isUAM && response.getAMRMToken() != null) {
|
|
|
Token<AMRMTokenIdentifier> newToken = ConverterUtils
|
|
|
.convertFromYarn(response.getAMRMToken(), (Text) null);
|
|
@@ -1648,44 +1646,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- // Notify policy of allocate response
|
|
|
- try {
|
|
|
- policyInterpreter.notifyOfResponse(subClusterId, response);
|
|
|
- } catch (YarnException e) {
|
|
|
- LOG.warn("notifyOfResponse for policy failed for sub-cluster "
|
|
|
- + subClusterId, e);
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
- /**
|
|
|
- * Private structure for encapsulating SubClusterId and
|
|
|
- * RegisterApplicationMasterResponse instances.
|
|
|
- */
|
|
|
- private static class RegisterApplicationMasterResponseInfo {
|
|
|
- private RegisterApplicationMasterResponse response;
|
|
|
- private SubClusterId subClusterId;
|
|
|
- private Token<AMRMTokenIdentifier> uamToken;
|
|
|
-
|
|
|
- RegisterApplicationMasterResponseInfo(
|
|
|
- RegisterApplicationMasterResponse response, SubClusterId subClusterId,
|
|
|
- Token<AMRMTokenIdentifier> uamToken) {
|
|
|
- this.response = response;
|
|
|
- this.subClusterId = subClusterId;
|
|
|
- this.uamToken = uamToken;
|
|
|
- }
|
|
|
-
|
|
|
- public RegisterApplicationMasterResponse getResponse() {
|
|
|
- return response;
|
|
|
- }
|
|
|
-
|
|
|
- public SubClusterId getSubClusterId() {
|
|
|
- return subClusterId;
|
|
|
- }
|
|
|
-
|
|
|
- public Token<AMRMTokenIdentifier> getUamToken() {
|
|
|
- return uamToken;
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1712,33 +1672,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Private structure for encapsulating successful and failed application
|
|
|
- * master registration responses.
|
|
|
- */
|
|
|
- private static class Registrations {
|
|
|
- private Map<SubClusterId, RegisterApplicationMasterResponse>
|
|
|
- successfulRegistrations;
|
|
|
- private List<SubClusterId> failedRegistrations;
|
|
|
-
|
|
|
- Registrations(
|
|
|
- Map<SubClusterId, RegisterApplicationMasterResponse>
|
|
|
- successfulRegistrations,
|
|
|
- List<SubClusterId> failedRegistrations) {
|
|
|
- this.successfulRegistrations = successfulRegistrations;
|
|
|
- this.failedRegistrations = failedRegistrations;
|
|
|
- }
|
|
|
-
|
|
|
- public Map<SubClusterId, RegisterApplicationMasterResponse>
|
|
|
- getSuccessfulRegistrations() {
|
|
|
- return this.successfulRegistrations;
|
|
|
- }
|
|
|
-
|
|
|
- public List<SubClusterId> getFailedRegistrations() {
|
|
|
- return this.failedRegistrations;
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Utility method to check if the specified Collection is null or empty.
|
|
|
*
|