|
@@ -91,6 +91,7 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterInfo;
|
|
|
import org.apache.hadoop.yarn.server.federation.utils.FederationRegistryClient;
|
|
|
import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.recovery.NMStateStoreService;
|
|
|
import org.apache.hadoop.yarn.server.uam.UnmanagedAMPoolManager;
|
|
|
import org.apache.hadoop.yarn.util.AsyncCallback;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
@@ -570,9 +571,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
* For the same reason, this method needs to be synchronized.
|
|
|
*/
|
|
|
@Override
|
|
|
- public synchronized RegisterApplicationMasterResponse
|
|
|
- registerApplicationMaster(RegisterApplicationMasterRequest request)
|
|
|
- throws YarnException, IOException {
|
|
|
+ public synchronized RegisterApplicationMasterResponse registerApplicationMaster(
|
|
|
+ RegisterApplicationMasterRequest request) throws YarnException, IOException {
|
|
|
+
|
|
|
+ if (request == null) {
|
|
|
+ throw new YarnException("RegisterApplicationMasterRequest can't be null!");
|
|
|
+ }
|
|
|
|
|
|
// Reset the heartbeat responseId to zero upon register
|
|
|
synchronized (this.lastAllocateResponseLock) {
|
|
@@ -590,18 +594,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
// Save the registration request. This will be used for registering with
|
|
|
// secondary sub-clusters using UAMs, as well as re-register later
|
|
|
this.amRegistrationRequest = request;
|
|
|
- if (getNMStateStore() != null) {
|
|
|
- try {
|
|
|
- RegisterApplicationMasterRequestPBImpl pb =
|
|
|
- (RegisterApplicationMasterRequestPBImpl)
|
|
|
- this.amRegistrationRequest;
|
|
|
- getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
|
|
|
- NMSS_REG_REQUEST_KEY, pb.getProto().toByteArray());
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Error storing AMRMProxy application context entry for "
|
|
|
- + this.attemptId, e);
|
|
|
- }
|
|
|
- }
|
|
|
+ RegisterApplicationMasterRequestPBImpl requestPB = (RegisterApplicationMasterRequestPBImpl)
|
|
|
+ this.amRegistrationRequest;
|
|
|
+ storeAMRMProxyAppContextEntry(NMSS_REG_REQUEST_KEY, requestPB.getProto().toByteArray());
|
|
|
}
|
|
|
|
|
|
/*
|
|
@@ -625,54 +620,64 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
* is running and will breaks the elasticity feature. The registration with
|
|
|
* the other sub-cluster RM will be done lazily as needed later.
|
|
|
*/
|
|
|
- this.amRegistrationResponse =
|
|
|
- this.homeRMRelayer.registerApplicationMaster(request);
|
|
|
- if (this.amRegistrationResponse
|
|
|
- .getContainersFromPreviousAttempts() != null) {
|
|
|
- cacheAllocatedContainers(
|
|
|
- this.amRegistrationResponse.getContainersFromPreviousAttempts(),
|
|
|
- this.homeSubClusterId);
|
|
|
+ this.amRegistrationResponse = this.homeRMRelayer.registerApplicationMaster(request);
|
|
|
+
|
|
|
+ if (this.amRegistrationResponse == null) {
|
|
|
+ throw new YarnException("RegisterApplicationMasterResponse can't be null!");
|
|
|
+ }
|
|
|
+
|
|
|
+ List<Container> containersFromPreviousAttempts =
|
|
|
+ this.amRegistrationResponse.getContainersFromPreviousAttempts();
|
|
|
+ if (containersFromPreviousAttempts != null) {
|
|
|
+ cacheAllocatedContainers(containersFromPreviousAttempts, this.homeSubClusterId);
|
|
|
}
|
|
|
|
|
|
ApplicationId appId = this.attemptId.getApplicationId();
|
|
|
reAttachUAMAndMergeRegisterResponse(this.amRegistrationResponse, appId);
|
|
|
|
|
|
- if (getNMStateStore() != null) {
|
|
|
- try {
|
|
|
- RegisterApplicationMasterResponsePBImpl pb =
|
|
|
- (RegisterApplicationMasterResponsePBImpl)
|
|
|
- this.amRegistrationResponse;
|
|
|
- getNMStateStore().storeAMRMProxyAppContextEntry(this.attemptId,
|
|
|
- NMSS_REG_RESPONSE_KEY, pb.getProto().toByteArray());
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.error("Error storing AMRMProxy application context entry for "
|
|
|
- + this.attemptId, e);
|
|
|
- }
|
|
|
- }
|
|
|
+ RegisterApplicationMasterResponsePBImpl responsePB = (RegisterApplicationMasterResponsePBImpl)
|
|
|
+ this.amRegistrationResponse;
|
|
|
+ storeAMRMProxyAppContextEntry(NMSS_REG_RESPONSE_KEY, responsePB.getProto().toByteArray());
|
|
|
|
|
|
// the queue this application belongs will be used for getting
|
|
|
// AMRMProxy policy from state store.
|
|
|
String queue = this.amRegistrationResponse.getQueue();
|
|
|
if (queue == null) {
|
|
|
- LOG.warn("Received null queue for application " + appId
|
|
|
- + " from home subcluster. Will use default queue name "
|
|
|
- + YarnConfiguration.DEFAULT_QUEUE_NAME
|
|
|
- + " for getting AMRMProxyPolicy");
|
|
|
+ LOG.warn("Received null queue for application {} from home subcluster. " +
|
|
|
+ " Will use default queue name {} for getting AMRMProxyPolicy.", appId,
|
|
|
+ YarnConfiguration.DEFAULT_QUEUE_NAME);
|
|
|
} else {
|
|
|
- LOG.info("Application " + appId + " belongs to queue " + queue);
|
|
|
+ LOG.info("Application {} belongs to queue {}.", appId, queue);
|
|
|
}
|
|
|
|
|
|
// Initialize the AMRMProxyPolicy
|
|
|
try {
|
|
|
- this.policyInterpreter =
|
|
|
- FederationPolicyUtils.loadAMRMPolicy(queue, this.policyInterpreter,
|
|
|
- getConf(), this.federationFacade, this.homeSubClusterId);
|
|
|
+ this.policyInterpreter = FederationPolicyUtils.loadAMRMPolicy(queue, this.policyInterpreter,
|
|
|
+ getConf(), this.federationFacade, this.homeSubClusterId);
|
|
|
} catch (FederationPolicyInitializationException e) {
|
|
|
throw new YarnRuntimeException(e);
|
|
|
}
|
|
|
return this.amRegistrationResponse;
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Add a context entry for an application attempt in AMRMProxyService.
|
|
|
+ *
|
|
|
+ * @param key key string
|
|
|
+ * @param data state data
|
|
|
+ */
|
|
|
+ private void storeAMRMProxyAppContextEntry(String key, byte[] data) {
|
|
|
+ NMStateStoreService nmStateStore = getNMStateStore();
|
|
|
+ if (nmStateStore != null) {
|
|
|
+ try {
|
|
|
+ nmStateStore.storeAMRMProxyAppContextEntry(this.attemptId, key, data);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.error("Error storing AMRMProxy application context entry[{}] for {}.",
|
|
|
+ key, this.attemptId, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Sends the heart beats to the home RM and the secondary sub-cluster RMs that
|
|
|
* are being used by the application.
|