|
@@ -18,6 +18,8 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.amrmproxy;
|
|
|
|
|
|
+import java.io.ByteArrayInputStream;
|
|
|
+import java.io.DataInputStream;
|
|
|
import java.io.IOException;
|
|
|
import java.util.ArrayList;
|
|
|
import java.util.Collection;
|
|
@@ -62,14 +64,18 @@ import org.apache.hadoop.yarn.api.records.ResourceBlacklistRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.StrictPreemptionContract;
|
|
|
import org.apache.hadoop.yarn.api.records.UpdateContainerRequest;
|
|
|
+import org.apache.hadoop.yarn.client.AMRMClientUtils;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.ApplicationMasterNotRegisteredException;
|
|
|
import org.apache.hadoop.yarn.exceptions.InvalidApplicationMasterRequestException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
+import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
|
+import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
|
|
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
+import org.apache.hadoop.yarn.server.AMHeartbeatRequestHandler;
|
|
|
import org.apache.hadoop.yarn.server.AMRMClientRelayer;
|
|
|
import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProviderUtil;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
|
|
@@ -80,9 +86,9 @@ import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
|
|
|
import org.apache.hadoop.yarn.server.federation.utils.FederationRegistryClient;
|
|
|
import org.apache.hadoop.yarn.server.federation.utils.FederationStateStoreFacade;
|
|
|
import org.apache.hadoop.yarn.server.uam.UnmanagedAMPoolManager;
|
|
|
-import org.apache.hadoop.yarn.server.utils.YarnServerSecurityUtils;
|
|
|
import org.apache.hadoop.yarn.util.AsyncCallback;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
+import org.apache.hadoop.yarn.util.MonotonicClock;
|
|
|
import org.apache.hadoop.yarn.util.resource.Resources;
|
|
|
import org.slf4j.Logger;
|
|
|
import org.slf4j.LoggerFactory;
|
|
@@ -116,6 +122,17 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
NMSS_CLASS_PREFIX + "secondarySC/";
|
|
|
public static final String STRING_TO_BYTE_FORMAT = "UTF-8";
|
|
|
|
|
|
+ private static final RecordFactory RECORD_FACTORY =
|
|
|
+ RecordFactoryProvider.getRecordFactory(null);
|
|
|
+
|
|
|
+ /**
|
|
|
+ * From AM's perspective, FederationInterceptor behaves exactly the same as
|
|
|
+ * YarnRM (ApplicationMasterService). This is to remember the last heart beat
|
|
|
+ * response, used to handle duplicate heart beat and responseId from AM.
|
|
|
+ */
|
|
|
+ private AllocateResponse lastAllocateResponse;
|
|
|
+ private final Object lastAllocateResponseLock = new Object();
|
|
|
+
|
|
|
private ApplicationAttemptId attemptId;
|
|
|
|
|
|
/**
|
|
@@ -124,7 +141,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
private AMRMClientRelayer homeRMRelayer;
|
|
|
private SubClusterId homeSubClusterId;
|
|
|
- private volatile int lastHomeResponseId;
|
|
|
+ private AMHeartbeatRequestHandler homeHeartbeartHandler;
|
|
|
|
|
|
/**
|
|
|
* UAM pool for secondary sub-clusters (ones other than home sub-cluster),
|
|
@@ -146,7 +163,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
/**
|
|
|
* Stores the AllocateResponses that are received asynchronously from all the
|
|
|
- * sub-cluster resource managers except the home RM.
|
|
|
+ * sub-cluster resource managers, including home RM.
|
|
|
*/
|
|
|
private Map<SubClusterId, List<AllocateResponse>> asyncResponseSink;
|
|
|
|
|
@@ -194,14 +211,13 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
/** The policy used to split requests among sub-clusters. */
|
|
|
private FederationAMRMProxyPolicy policyInterpreter;
|
|
|
|
|
|
- /**
|
|
|
- * The proxy ugi used to talk to home RM, loaded with the up-to-date AMRMToken
|
|
|
- * issued by home RM.
|
|
|
- */
|
|
|
- private UserGroupInformation appOwner;
|
|
|
-
|
|
|
private FederationRegistryClient registryClient;
|
|
|
|
|
|
+ // the maximum wait time for the first async heart beat response
|
|
|
+ private long heartbeatMaxWaitTimeMs;
|
|
|
+
|
|
|
+ private MonotonicClock clock = new MonotonicClock();
|
|
|
+
|
|
|
/**
|
|
|
* Creates an instance of the FederationInterceptor class.
|
|
|
*/
|
|
@@ -213,7 +229,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.secondaryRelayers = new ConcurrentHashMap<>();
|
|
|
this.amRegistrationRequest = null;
|
|
|
this.amRegistrationResponse = null;
|
|
|
- this.lastHomeResponseId = Integer.MAX_VALUE;
|
|
|
this.justRecovered = false;
|
|
|
}
|
|
|
|
|
@@ -233,8 +248,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
setConf(conf);
|
|
|
}
|
|
|
|
|
|
+ // The proxy ugi used to talk to home RM as well as Yarn Registry, loaded
|
|
|
+ // with the up-to-date AMRMToken issued by home RM.
|
|
|
+ UserGroupInformation appOwner;
|
|
|
try {
|
|
|
- this.appOwner = UserGroupInformation.createProxyUser(appContext.getUser(),
|
|
|
+ appOwner = UserGroupInformation.createProxyUser(appContext.getUser(),
|
|
|
UserGroupInformation.getCurrentUser());
|
|
|
} catch (Exception ex) {
|
|
|
throw new YarnRuntimeException(ex);
|
|
@@ -242,10 +260,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
if (appContext.getRegistryClient() != null) {
|
|
|
this.registryClient = new FederationRegistryClient(conf,
|
|
|
- appContext.getRegistryClient(), this.appOwner);
|
|
|
+ appContext.getRegistryClient(), appOwner);
|
|
|
// Add all app tokens for Yarn Registry access
|
|
|
if (appContext.getCredentials() != null) {
|
|
|
- this.appOwner.addCredentials(appContext.getCredentials());
|
|
|
+ appOwner.addCredentials(appContext.getCredentials());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -254,9 +272,21 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.homeSubClusterId =
|
|
|
SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
|
|
|
this.homeRMRelayer = new AMRMClientRelayer(createHomeRMProxy(appContext,
|
|
|
- ApplicationMasterProtocol.class, this.appOwner), appId,
|
|
|
+ ApplicationMasterProtocol.class, appOwner), appId,
|
|
|
this.homeSubClusterId.toString());
|
|
|
|
|
|
+ this.homeHeartbeartHandler = createHomeHeartbeartHandler(conf, appId);
|
|
|
+ this.homeHeartbeartHandler.setAMRMClientRelayer(this.homeRMRelayer);
|
|
|
+ this.homeHeartbeartHandler.setUGI(appOwner);
|
|
|
+ this.homeHeartbeartHandler.setDaemon(true);
|
|
|
+ this.homeHeartbeartHandler.start();
|
|
|
+
|
|
|
+ // set lastResponseId to -1 before application master registers
|
|
|
+ this.lastAllocateResponse =
|
|
|
+ RECORD_FACTORY.newRecordInstance(AllocateResponse.class);
|
|
|
+ this.lastAllocateResponse
|
|
|
+ .setResponseId(AMRMClientUtils.PRE_REGISTER_RESPONSE_ID);
|
|
|
+
|
|
|
this.federationFacade = FederationStateStoreFacade.getInstance();
|
|
|
this.subClusterResolver = this.federationFacade.getSubClusterResolver();
|
|
|
|
|
@@ -265,6 +295,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
this.uamPool.init(conf);
|
|
|
this.uamPool.start();
|
|
|
+
|
|
|
+ this.heartbeatMaxWaitTimeMs =
|
|
|
+ conf.getLong(YarnConfiguration.FEDERATION_AMRMPROXY_HB_MAX_WAIT_MS,
|
|
|
+ YarnConfiguration.DEFAULT_FEDERATION_AMRMPROXY_HB_MAX_WAIT_MS);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -272,6 +306,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
super.recover(recoveredDataMap);
|
|
|
LOG.info("Recovering data for FederationInterceptor for {}",
|
|
|
this.attemptId);
|
|
|
+ this.justRecovered = true;
|
|
|
+
|
|
|
if (recoveredDataMap == null) {
|
|
|
return;
|
|
|
}
|
|
@@ -294,9 +330,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.amRegistrationResponse =
|
|
|
new RegisterApplicationMasterResponsePBImpl(pb);
|
|
|
LOG.info("amRegistrationResponse recovered for {}", this.attemptId);
|
|
|
- // Trigger re-register and full pending re-send only if we have a
|
|
|
- // saved register response. This should always be true though.
|
|
|
- this.justRecovered = true;
|
|
|
}
|
|
|
|
|
|
// Recover UAM amrmTokens from registry or NMSS
|
|
@@ -355,6 +388,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
.getContainersFromPreviousAttempts()) {
|
|
|
containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
|
|
|
containers++;
|
|
|
+ LOG.debug(" From subcluster " + subClusterId
|
|
|
+ + " running container " + container.getId());
|
|
|
}
|
|
|
LOG.info("Recovered {} running containers from UAM in {}",
|
|
|
response.getContainersFromPreviousAttempts().size(),
|
|
@@ -384,7 +419,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
LOG.debug(" From home RM " + this.homeSubClusterId
|
|
|
+ " running container " + container.getContainerId());
|
|
|
}
|
|
|
- LOG.info("{} running containers including AM recovered from home RM ",
|
|
|
+ LOG.info("{} running containers including AM recovered from home RM {}",
|
|
|
response.getContainerList().size(), this.homeSubClusterId);
|
|
|
|
|
|
LOG.info(
|
|
@@ -411,8 +446,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
* so that when AM registers more than once, it returns the same register
|
|
|
* success response instead of throwing
|
|
|
* {@link InvalidApplicationMasterRequestException}. Furthermore, we present
|
|
|
- * to AM as if we are the RM that never fails over. When actual RM fails over,
|
|
|
- * we always re-register automatically.
|
|
|
+ * to AM as if we are the RM that never fails over (except when AMRMProxy
|
|
|
+ * restarts). When actual RM fails over, we always re-register automatically.
|
|
|
*
|
|
|
* We did this because FederationInterceptor can receive concurrent register
|
|
|
* requests from AM because of timeout between AM and AMRMProxy, which is
|
|
@@ -425,6 +460,13 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
public synchronized RegisterApplicationMasterResponse
|
|
|
registerApplicationMaster(RegisterApplicationMasterRequest request)
|
|
|
throws YarnException, IOException {
|
|
|
+
|
|
|
+ // Reset the heartbeat responseId to zero upon register
|
|
|
+ synchronized (this.lastAllocateResponseLock) {
|
|
|
+ this.lastAllocateResponse.setResponseId(0);
|
|
|
+ }
|
|
|
+ this.justRecovered = false;
|
|
|
+
|
|
|
// If AM is calling with a different request, complain
|
|
|
if (this.amRegistrationRequest != null) {
|
|
|
if (!this.amRegistrationRequest.equals(request)) {
|
|
@@ -524,34 +566,34 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
@Override
|
|
|
public AllocateResponse allocate(AllocateRequest request)
|
|
|
- throws YarnException {
|
|
|
+ throws YarnException, IOException {
|
|
|
Preconditions.checkArgument(this.policyInterpreter != null,
|
|
|
"Allocate should be called after registerApplicationMaster");
|
|
|
|
|
|
- if (this.justRecovered && this.lastHomeResponseId == Integer.MAX_VALUE) {
|
|
|
- // Save the responseId home RM is expecting
|
|
|
- this.lastHomeResponseId = request.getResponseId();
|
|
|
-
|
|
|
+ if (this.justRecovered) {
|
|
|
throw new ApplicationMasterNotRegisteredException(
|
|
|
"AMRMProxy just restarted and recovered for " + this.attemptId
|
|
|
+ ". AM should re-register and full re-send pending requests.");
|
|
|
}
|
|
|
|
|
|
- // Override responseId in the request in two cases:
|
|
|
- //
|
|
|
- // 1. After we just recovered after an NM restart and AM's responseId is
|
|
|
- // reset due to the exception we generate. We need to override the
|
|
|
- // responseId to the one homeRM expects.
|
|
|
- //
|
|
|
- // 2. After homeRM fail-over, the allocate response with reseted responseId
|
|
|
- // might not be returned successfully back to AM because of RPC connection
|
|
|
- // timeout between AM and AMRMProxy. In this case, we remember and reset the
|
|
|
- // responseId for AM.
|
|
|
- if (this.justRecovered
|
|
|
- || request.getResponseId() > this.lastHomeResponseId) {
|
|
|
- LOG.warn("Setting allocate responseId for {} from {} to {}",
|
|
|
- this.attemptId, request.getResponseId(), this.lastHomeResponseId);
|
|
|
- request.setResponseId(this.lastHomeResponseId);
|
|
|
+ // Check responseId and handle duplicate heartbeat exactly same as RM
|
|
|
+ synchronized (this.lastAllocateResponseLock) {
|
|
|
+ LOG.info("Heartbeat from " + this.attemptId + " with responseId "
|
|
|
+ + request.getResponseId() + " when we are expecting "
|
|
|
+ + this.lastAllocateResponse.getResponseId());
|
|
|
+ // Normally request.getResponseId() == lastResponse.getResponseId()
|
|
|
+ if (AMRMClientUtils.getNextResponseId(
|
|
|
+ request.getResponseId()) == this.lastAllocateResponse
|
|
|
+ .getResponseId()) {
|
|
|
+ // heartbeat one step old, simply return lastReponse
|
|
|
+ return this.lastAllocateResponse;
|
|
|
+ } else if (request.getResponseId() != this.lastAllocateResponse
|
|
|
+ .getResponseId()) {
|
|
|
+ throw new InvalidApplicationMasterRequestException(
|
|
|
+ AMRMClientUtils.assembleInvalidResponseIdExceptionMessage(attemptId,
|
|
|
+ this.lastAllocateResponse.getResponseId(),
|
|
|
+ request.getResponseId()));
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
try {
|
|
@@ -560,71 +602,55 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
Map<SubClusterId, AllocateRequest> requests =
|
|
|
splitAllocateRequest(request);
|
|
|
|
|
|
- // Send the requests to the secondary sub-cluster resource managers.
|
|
|
- // These secondary requests are send asynchronously and the responses will
|
|
|
- // be collected and merged with the home response. In addition, it also
|
|
|
- // return the newly registered Unmanaged AMs.
|
|
|
- Registrations newRegistrations =
|
|
|
- sendRequestsToSecondaryResourceManagers(requests);
|
|
|
-
|
|
|
- // Send the request to the home RM and get the response
|
|
|
- AllocateRequest homeRequest = requests.get(this.homeSubClusterId);
|
|
|
- LOG.info("{} heartbeating to home RM with responseId {}", this.attemptId,
|
|
|
- homeRequest.getResponseId());
|
|
|
-
|
|
|
- AllocateResponse homeResponse = this.homeRMRelayer.allocate(homeRequest);
|
|
|
-
|
|
|
- // Reset the flag after the first successful homeRM allocate response,
|
|
|
- // otherwise keep overriding the responseId of new allocate request
|
|
|
- if (this.justRecovered) {
|
|
|
- this.justRecovered = false;
|
|
|
+ /**
|
|
|
+ * 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.
|
|
|
+ */
|
|
|
+ Registrations newRegistrations = sendRequestsToResourceManagers(requests);
|
|
|
+
|
|
|
+ // Wait for the first async response to arrive
|
|
|
+ long startTime = this.clock.getTime();
|
|
|
+ synchronized (this.asyncResponseSink) {
|
|
|
+ try {
|
|
|
+ this.asyncResponseSink.wait(this.heartbeatMaxWaitTimeMs);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
+ }
|
|
|
}
|
|
|
+ long firstResponseTime = this.clock.getTime() - startTime;
|
|
|
|
|
|
- // Notify policy of home response
|
|
|
+ // An extra brief wait for other async heart beats, so that most of their
|
|
|
+ // responses can make it back to AM in the same heart beat round trip.
|
|
|
try {
|
|
|
- this.policyInterpreter.notifyOfResponse(this.homeSubClusterId,
|
|
|
- homeResponse);
|
|
|
- } catch (YarnException e) {
|
|
|
- LOG.warn("notifyOfResponse for policy failed for home sub-cluster "
|
|
|
- + this.homeSubClusterId, e);
|
|
|
+ Thread.sleep(firstResponseTime);
|
|
|
+ } catch (InterruptedException e) {
|
|
|
}
|
|
|
|
|
|
- // If the resource manager sent us a new token, add to the current user
|
|
|
- if (homeResponse.getAMRMToken() != null) {
|
|
|
- LOG.debug("Received new AMRMToken");
|
|
|
- YarnServerSecurityUtils.updateAMRMToken(homeResponse.getAMRMToken(),
|
|
|
- this.appOwner, getConf());
|
|
|
- }
|
|
|
+ // Prepare the response to AM
|
|
|
+ AllocateResponse response =
|
|
|
+ RECORD_FACTORY.newRecordInstance(AllocateResponse.class);
|
|
|
|
|
|
- // Merge the responses from home and secondary sub-cluster RMs
|
|
|
- homeResponse = mergeAllocateResponses(homeResponse);
|
|
|
+ // Merge all responses from response sink
|
|
|
+ mergeAllocateResponses(response);
|
|
|
|
|
|
// Merge the containers and NMTokens from the new registrations into
|
|
|
- // the homeResponse.
|
|
|
+ // the response
|
|
|
if (!isNullOrEmpty(newRegistrations.getSuccessfulRegistrations())) {
|
|
|
- homeResponse = mergeRegistrationResponses(homeResponse,
|
|
|
+ mergeRegistrationResponses(response,
|
|
|
newRegistrations.getSuccessfulRegistrations());
|
|
|
}
|
|
|
|
|
|
- LOG.info("{} heartbeat response from home RM with responseId {}",
|
|
|
- this.attemptId, homeResponse.getResponseId());
|
|
|
-
|
|
|
- // Update lastHomeResponseId in three cases:
|
|
|
- // 1. The normal responseId increments
|
|
|
- // 2. homeResponse.getResponseId() == 1. This happens when homeRM fails
|
|
|
- // over, AMRMClientRelayer auto re-register and full re-send for homeRM.
|
|
|
- // 3. lastHomeResponseId == MAX_INT. This is the initial case or
|
|
|
- // responseId about to overflow and wrap around
|
|
|
- if (homeResponse.getResponseId() == this.lastHomeResponseId + 1
|
|
|
- || homeResponse.getResponseId() == 1
|
|
|
- || this.lastHomeResponseId == Integer.MAX_VALUE) {
|
|
|
- this.lastHomeResponseId = homeResponse.getResponseId();
|
|
|
+ // update the responseId and return the final response to AM
|
|
|
+ synchronized (this.lastAllocateResponseLock) {
|
|
|
+ response.setResponseId(AMRMClientUtils
|
|
|
+ .getNextResponseId(this.lastAllocateResponse.getResponseId()));
|
|
|
+ this.lastAllocateResponse = response;
|
|
|
}
|
|
|
-
|
|
|
- // return the final response to the application master.
|
|
|
- return homeResponse;
|
|
|
- } catch (IOException ex) {
|
|
|
- LOG.error("Exception encountered while processing heart beat", ex);
|
|
|
+ return response;
|
|
|
+ } catch (Throwable ex) {
|
|
|
+ LOG.error("Exception encountered while processing heart beat for "
|
|
|
+ + this.attemptId, ex);
|
|
|
throw new YarnException(ex);
|
|
|
}
|
|
|
}
|
|
@@ -696,6 +722,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
FinishApplicationMasterResponse homeResponse =
|
|
|
this.homeRMRelayer.finishApplicationMaster(request);
|
|
|
|
|
|
+ // Stop the home heartbeat thread
|
|
|
+ this.homeHeartbeartHandler.shutdown();
|
|
|
+
|
|
|
if (subClusterIds.size() > 0) {
|
|
|
// Wait for other sub-cluster resource managers to return the
|
|
|
// response and merge it with the home response
|
|
@@ -758,10 +787,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
this.threadpool = null;
|
|
|
}
|
|
|
- homeRMRelayer.shutdown();
|
|
|
- for(AMRMClientRelayer relayer : secondaryRelayers.values()){
|
|
|
+
|
|
|
+ // Stop the home heartbeat thread
|
|
|
+ this.homeHeartbeartHandler.shutdown();
|
|
|
+ this.homeRMRelayer.shutdown();
|
|
|
+ for (AMRMClientRelayer relayer : this.secondaryRelayers.values()) {
|
|
|
relayer.shutdown();
|
|
|
}
|
|
|
+
|
|
|
super.shutdown();
|
|
|
}
|
|
|
|
|
@@ -781,8 +814,13 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
|
- protected int getLastHomeResponseId() {
|
|
|
- return this.lastHomeResponseId;
|
|
|
+ protected ApplicationAttemptId getAttemptId() {
|
|
|
+ return this.attemptId;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ protected AMHeartbeatRequestHandler getHomeHeartbeartHandler() {
|
|
|
+ return this.homeHeartbeartHandler;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -798,6 +836,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
return new UnmanagedAMPoolManager(threadPool);
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ protected AMHeartbeatRequestHandler createHomeHeartbeartHandler(
|
|
|
+ Configuration conf, ApplicationId appId) {
|
|
|
+ return new AMHeartbeatRequestHandler(conf, appId);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Create a proxy instance that is used to connect to the Home resource
|
|
|
* manager.
|
|
@@ -872,7 +916,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
+ "Reattaching in parallel", uamMap.size(), appId);
|
|
|
|
|
|
ExecutorCompletionService<RegisterApplicationMasterResponse>
|
|
|
- completionService = new ExecutorCompletionService<>(threadpool);
|
|
|
+ completionService = new ExecutorCompletionService<>(this.threadpool);
|
|
|
|
|
|
for (Entry<String, Token<AMRMTokenIdentifier>> entry : uamMap.entrySet()) {
|
|
|
final SubClusterId subClusterId =
|
|
@@ -1047,16 +1091,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
/**
|
|
|
* This methods sends the specified AllocateRequests to the appropriate
|
|
|
- * sub-cluster resource managers.
|
|
|
+ * sub-cluster resource managers asynchronously.
|
|
|
*
|
|
|
* @param requests contains the heart beat requests to send to the resource
|
|
|
- * manager keyed by the resource manager address
|
|
|
+ * 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 sendRequestsToSecondaryResourceManagers(
|
|
|
+ private Registrations sendRequestsToResourceManagers(
|
|
|
Map<SubClusterId, AllocateRequest> requests)
|
|
|
throws YarnException, IOException {
|
|
|
|
|
@@ -1065,32 +1109,29 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
Registrations registrations = registerWithNewSubClusters(requests.keySet());
|
|
|
|
|
|
// Now that all the registrations are done, send the allocation request
|
|
|
- // to the sub-cluster RMs using the Unmanaged application masters
|
|
|
- // asynchronously and don't wait for the response. The responses will
|
|
|
- // arrive asynchronously and will be added to the response sink. These
|
|
|
- // responses will be sent to the application master in some future heart
|
|
|
- // beat response.
|
|
|
+ // to the sub-cluster RMs asynchronously and don't wait for the response.
|
|
|
+ // The responses will arrive asynchronously and will be added to the
|
|
|
+ // response sink, then merged and sent to the application master.
|
|
|
for (Entry<SubClusterId, AllocateRequest> entry : requests.entrySet()) {
|
|
|
- final SubClusterId subClusterId = entry.getKey();
|
|
|
+ SubClusterId subClusterId = entry.getKey();
|
|
|
|
|
|
if (subClusterId.equals(this.homeSubClusterId)) {
|
|
|
- // Skip the request for the home sub-cluster resource manager.
|
|
|
- // It will be handled separately in the allocate() method
|
|
|
- continue;
|
|
|
- }
|
|
|
-
|
|
|
- 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;
|
|
|
+ // Request for the home sub-cluster resource manager
|
|
|
+ this.homeHeartbeartHandler.allocateAsync(entry.getValue(),
|
|
|
+ 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;
|
|
|
+ }
|
|
|
+ this.uamPool.allocateAsync(subClusterId.getId(), entry.getValue(),
|
|
|
+ new HeartbeatCallBack(subClusterId, true));
|
|
|
}
|
|
|
-
|
|
|
- this.uamPool.allocateAsync(subClusterId.getId(), entry.getValue(),
|
|
|
- new HeartbeatCallBack(subClusterId));
|
|
|
}
|
|
|
|
|
|
return registrations;
|
|
@@ -1123,7 +1164,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.amRegistrationRequest;
|
|
|
final AMRMProxyApplicationContext appContext = getApplicationContext();
|
|
|
ExecutorCompletionService<RegisterApplicationMasterResponseInfo>
|
|
|
- completionService = new ExecutorCompletionService<>(threadpool);
|
|
|
+ completionService = new ExecutorCompletionService<>(this.threadpool);
|
|
|
|
|
|
for (final String subClusterId : newSubClusters) {
|
|
|
completionService
|
|
@@ -1208,21 +1249,14 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Merges the responses from other sub-clusters that we received
|
|
|
- * asynchronously with the specified home cluster response and keeps track of
|
|
|
- * the containers received from each sub-cluster resource managers.
|
|
|
+ * Merge the responses from all sub-clusters that we received asynchronously
|
|
|
+ * and keeps track of the containers received from each sub-cluster resource
|
|
|
+ * managers.
|
|
|
*/
|
|
|
- private AllocateResponse mergeAllocateResponses(
|
|
|
- AllocateResponse homeResponse) {
|
|
|
- // Timing issue, we need to remove the completed and then save the new ones.
|
|
|
- removeFinishedContainersFromCache(
|
|
|
- homeResponse.getCompletedContainersStatuses());
|
|
|
- cacheAllocatedContainers(homeResponse.getAllocatedContainers(),
|
|
|
- this.homeSubClusterId);
|
|
|
-
|
|
|
+ private void mergeAllocateResponses(AllocateResponse mergedResponse) {
|
|
|
synchronized (this.asyncResponseSink) {
|
|
|
- for (Entry<SubClusterId, List<AllocateResponse>> entry : asyncResponseSink
|
|
|
- .entrySet()) {
|
|
|
+ for (Entry<SubClusterId, List<AllocateResponse>> entry :
|
|
|
+ this.asyncResponseSink.entrySet()) {
|
|
|
SubClusterId subClusterId = entry.getKey();
|
|
|
List<AllocateResponse> responses = entry.getValue();
|
|
|
if (responses.size() > 0) {
|
|
@@ -1231,14 +1265,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
response.getCompletedContainersStatuses());
|
|
|
cacheAllocatedContainers(response.getAllocatedContainers(),
|
|
|
subClusterId);
|
|
|
- mergeAllocateResponse(homeResponse, response, subClusterId);
|
|
|
+ mergeAllocateResponse(mergedResponse, response, subClusterId);
|
|
|
}
|
|
|
responses.clear();
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return homeResponse;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -1256,11 +1288,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Helper method for merging the responses from the secondary sub cluster RMs
|
|
|
- * with the home response to return to the AM.
|
|
|
+ * Helper method for merging the registration responses from the secondary sub
|
|
|
+ * cluster RMs into the allocate response to return to the AM.
|
|
|
*/
|
|
|
- private AllocateResponse mergeRegistrationResponses(
|
|
|
- AllocateResponse homeResponse,
|
|
|
+ private void mergeRegistrationResponses(AllocateResponse homeResponse,
|
|
|
Map<SubClusterId, RegisterApplicationMasterResponse> registrations) {
|
|
|
|
|
|
for (Entry<SubClusterId, RegisterApplicationMasterResponse> entry :
|
|
@@ -1292,13 +1323,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- return homeResponse;
|
|
|
}
|
|
|
|
|
|
private void mergeAllocateResponse(AllocateResponse homeResponse,
|
|
|
AllocateResponse otherResponse, SubClusterId otherRMAddress) {
|
|
|
|
|
|
+ if (otherResponse.getAMRMToken() != null) {
|
|
|
+ // Propagate only the new amrmToken from home sub-cluster back to
|
|
|
+ // AMRMProxyService
|
|
|
+ if (otherRMAddress.equals(this.homeSubClusterId)) {
|
|
|
+ homeResponse.setAMRMToken(otherResponse.getAMRMToken());
|
|
|
+ } else {
|
|
|
+ throw new YarnRuntimeException(
|
|
|
+ "amrmToken from UAM " + otherRMAddress + " should be null here");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
if (!isNullOrEmpty(otherResponse.getAllocatedContainers())) {
|
|
|
if (!isNullOrEmpty(homeResponse.getAllocatedContainers())) {
|
|
|
homeResponse.getAllocatedContainers()
|
|
@@ -1406,9 +1446,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
SubClusterId subClusterId) {
|
|
|
for (Container container : containers) {
|
|
|
LOG.debug("Adding container {}", container);
|
|
|
- if (containerIdToSubClusterIdMap.containsKey(container.getId())) {
|
|
|
+
|
|
|
+ if (this.containerIdToSubClusterIdMap.containsKey(container.getId())) {
|
|
|
SubClusterId existingSubClusterId =
|
|
|
- containerIdToSubClusterIdMap.get(container.getId());
|
|
|
+ this.containerIdToSubClusterIdMap.get(container.getId());
|
|
|
if (existingSubClusterId.equals(subClusterId)) {
|
|
|
/*
|
|
|
* When RM fails over, the new RM master might send out the same
|
|
@@ -1441,7 +1482,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
|
|
|
+ this.containerIdToSubClusterIdMap.put(container.getId(), subClusterId);
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1463,7 +1504,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
newRequest.setProgress(originalAMRequest.getProgress());
|
|
|
requestMap.put(subClusterId, newRequest);
|
|
|
}
|
|
|
-
|
|
|
return newRequest;
|
|
|
}
|
|
|
|
|
@@ -1472,7 +1512,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
private static AllocateRequest createAllocateRequest() {
|
|
|
AllocateRequest request =
|
|
|
- AllocateRequest.newInstance(0, 0, null, null, null);
|
|
|
+ RECORD_FACTORY.newRecordInstance(AllocateRequest.class);
|
|
|
request.setAskList(new ArrayList<ResourceRequest>());
|
|
|
request.setReleaseList(new ArrayList<ContainerId>());
|
|
|
ResourceBlacklistRequest blackList =
|
|
@@ -1525,6 +1565,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
return this.uamPool.getAllUAMIds().size();
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ protected UnmanagedAMPoolManager getUnmanagedAMPool() {
|
|
|
+ return this.uamPool;
|
|
|
+ }
|
|
|
+
|
|
|
@VisibleForTesting
|
|
|
public Map<SubClusterId, List<AllocateResponse>> getAsyncResponseSink() {
|
|
|
return this.asyncResponseSink;
|
|
@@ -1535,9 +1580,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
private class HeartbeatCallBack implements AsyncCallback<AllocateResponse> {
|
|
|
private SubClusterId subClusterId;
|
|
|
+ private boolean isUAM;
|
|
|
|
|
|
- HeartbeatCallBack(SubClusterId subClusterId) {
|
|
|
+ HeartbeatCallBack(SubClusterId subClusterId, boolean isUAM) {
|
|
|
this.subClusterId = subClusterId;
|
|
|
+ this.isUAM = isUAM;
|
|
|
}
|
|
|
|
|
|
@Override
|
|
@@ -1551,16 +1598,33 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
asyncResponseSink.put(subClusterId, responses);
|
|
|
}
|
|
|
responses.add(response);
|
|
|
+ // Notify main thread about the response arrival
|
|
|
+ asyncResponseSink.notifyAll();
|
|
|
}
|
|
|
|
|
|
// Save the new AMRMToken for the UAM if present
|
|
|
- if (response.getAMRMToken() != null) {
|
|
|
+ if (this.isUAM && response.getAMRMToken() != null) {
|
|
|
Token<AMRMTokenIdentifier> newToken = ConverterUtils
|
|
|
.convertFromYarn(response.getAMRMToken(), (Text) null);
|
|
|
+ // Do not further propagate the new amrmToken for UAM
|
|
|
+ response.setAMRMToken(null);
|
|
|
+
|
|
|
// Update the token in registry or NMSS
|
|
|
if (registryClient != null) {
|
|
|
- registryClient.writeAMRMTokenForUAM(attemptId.getApplicationId(),
|
|
|
- subClusterId.getId(), newToken);
|
|
|
+ if (registryClient.writeAMRMTokenForUAM(attemptId.getApplicationId(),
|
|
|
+ subClusterId.getId(), newToken)) {
|
|
|
+ try {
|
|
|
+ AMRMTokenIdentifier identifier = new AMRMTokenIdentifier();
|
|
|
+ identifier.readFields(new DataInputStream(
|
|
|
+ new ByteArrayInputStream(newToken.getIdentifier())));
|
|
|
+ LOG.info(
|
|
|
+ "Received new UAM amrmToken with keyId {} and "
|
|
|
+ + "service {} from {} for {}, written to Registry",
|
|
|
+ identifier.getKeyId(), newToken.getService(), subClusterId,
|
|
|
+ attemptId);
|
|
|
+ } catch (IOException e) {
|
|
|
+ }
|
|
|
+ }
|
|
|
} else if (getNMStateStore() != null) {
|
|
|
try {
|
|
|
getNMStateStore().storeAMRMProxyAppContextEntry(attemptId,
|
|
@@ -1573,11 +1637,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- // Notify policy of secondary sub-cluster responses
|
|
|
+ // Notify policy of allocate response
|
|
|
try {
|
|
|
policyInterpreter.notifyOfResponse(subClusterId, response);
|
|
|
} catch (YarnException e) {
|
|
|
- LOG.warn("notifyOfResponse for policy failed for home sub-cluster "
|
|
|
+ LOG.warn("notifyOfResponse for policy failed for sub-cluster "
|
|
|
+ subClusterId, e);
|
|
|
}
|
|
|
}
|