|
@@ -62,14 +62,15 @@ 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.proto.YarnServiceProtos.RegisterApplicationMasterRequestProto;
|
|
|
import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProto;
|
|
|
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
|
|
|
+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;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.amrmproxy.FederationAMRMProxyPolicy;
|
|
@@ -106,9 +107,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
public static final String NMSS_REG_RESPONSE_KEY =
|
|
|
NMSS_CLASS_PREFIX + "registerResponse";
|
|
|
|
|
|
- /*
|
|
|
+ /**
|
|
|
* When AMRMProxy HA is enabled, secondary AMRMTokens will be stored in Yarn
|
|
|
- * Registry. Otherwise if NM recovery is enabled, the UAM token are store in
|
|
|
+ * Registry. Otherwise if NM recovery is enabled, the UAM token are stored in
|
|
|
* local NMSS instead under this directory name.
|
|
|
*/
|
|
|
public static final String NMSS_SECONDARY_SC_PREFIX =
|
|
@@ -119,8 +120,23 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
* The home sub-cluster is the sub-cluster where the AM container is running
|
|
|
* in.
|
|
|
*/
|
|
|
- private ApplicationMasterProtocol homeRM;
|
|
|
+ private AMRMClientRelayer homeRMRelayer;
|
|
|
private SubClusterId homeSubClusterId;
|
|
|
+ private volatile int lastHomeResponseId;
|
|
|
+
|
|
|
+ /**
|
|
|
+ * A flag for work preserving NM restart. If we just recovered, we need to
|
|
|
+ * generate an {@link ApplicationMasterNotRegisteredException} exception back
|
|
|
+ * to AM (similar to what RM will do after its restart/fail-over) in its next
|
|
|
+ * allocate to trigger AM re-register (which we will shield from RM and just
|
|
|
+ * return our saved register response) and a full pending requests re-send, so
|
|
|
+ * that all the {@link AMRMClientRelayer} will be re-populated with all
|
|
|
+ * pending requests.
|
|
|
+ *
|
|
|
+ * TODO: When split-merge is not idempotent, this can lead to some
|
|
|
+ * over-allocation without a full cancel to RM.
|
|
|
+ */
|
|
|
+ private volatile boolean justRecovered;
|
|
|
|
|
|
/**
|
|
|
* UAM pool for secondary sub-clusters (ones other than home sub-cluster),
|
|
@@ -134,6 +150,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
*/
|
|
|
private UnmanagedAMPoolManager uamPool;
|
|
|
|
|
|
+ /**
|
|
|
+ * The rmProxy relayers for secondary sub-clusters that keep track of all
|
|
|
+ * pending requests.
|
|
|
+ */
|
|
|
+ private Map<String, AMRMClientRelayer> secondaryRelayers;
|
|
|
+
|
|
|
/** Thread pool used for asynchronous operations. */
|
|
|
private ExecutorService threadpool;
|
|
|
|
|
@@ -186,8 +208,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.asyncResponseSink = new ConcurrentHashMap<>();
|
|
|
this.threadpool = Executors.newCachedThreadPool();
|
|
|
this.uamPool = createUnmanagedAMPoolManager(this.threadpool);
|
|
|
+ this.secondaryRelayers = new ConcurrentHashMap<>();
|
|
|
this.amRegistrationRequest = null;
|
|
|
this.amRegistrationResponse = null;
|
|
|
+ this.lastHomeResponseId = Integer.MAX_VALUE;
|
|
|
+ this.justRecovered = false;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -224,8 +249,8 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
|
|
|
this.homeSubClusterId =
|
|
|
SubClusterId.newInstance(YarnConfiguration.getClusterId(conf));
|
|
|
- this.homeRM = createHomeRMProxy(appContext, ApplicationMasterProtocol.class,
|
|
|
- this.appOwner);
|
|
|
+ this.homeRMRelayer = new AMRMClientRelayer(createHomeRMProxy(appContext,
|
|
|
+ ApplicationMasterProtocol.class, this.appOwner));
|
|
|
|
|
|
this.federationFacade = FederationStateStoreFacade.getInstance();
|
|
|
this.subClusterResolver = this.federationFacade.getSubClusterResolver();
|
|
@@ -240,13 +265,12 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
@Override
|
|
|
public void recover(Map<String, byte[]> recoveredDataMap) {
|
|
|
super.recover(recoveredDataMap);
|
|
|
- LOG.info("Recovering data for FederationInterceptor");
|
|
|
+ ApplicationAttemptId attemptId =
|
|
|
+ getApplicationContext().getApplicationAttemptId();
|
|
|
+ LOG.info("Recovering data for FederationInterceptor for {}", attemptId);
|
|
|
if (recoveredDataMap == null) {
|
|
|
return;
|
|
|
}
|
|
|
-
|
|
|
- ApplicationAttemptId attemptId =
|
|
|
- getApplicationContext().getApplicationAttemptId();
|
|
|
try {
|
|
|
if (recoveredDataMap.containsKey(NMSS_REG_REQUEST_KEY)) {
|
|
|
RegisterApplicationMasterRequestProto pb =
|
|
@@ -255,6 +279,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.amRegistrationRequest =
|
|
|
new RegisterApplicationMasterRequestPBImpl(pb);
|
|
|
LOG.info("amRegistrationRequest recovered for {}", attemptId);
|
|
|
+
|
|
|
+ // Give the register request to homeRMRelayer for future re-registration
|
|
|
+ this.homeRMRelayer.setAMRegistrationRequest(this.amRegistrationRequest);
|
|
|
}
|
|
|
if (recoveredDataMap.containsKey(NMSS_REG_RESPONSE_KEY)) {
|
|
|
RegisterApplicationMasterResponseProto pb =
|
|
@@ -263,6 +290,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
this.amRegistrationResponse =
|
|
|
new RegisterApplicationMasterResponsePBImpl(pb);
|
|
|
LOG.info("amRegistrationResponse recovered for {}", 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
|
|
@@ -309,6 +339,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
getApplicationContext().getUser(), this.homeSubClusterId.getId(),
|
|
|
entry.getValue());
|
|
|
|
|
|
+ this.secondaryRelayers.put(subClusterId.getId(),
|
|
|
+ this.uamPool.getAMRMClientRelayer(subClusterId.getId()));
|
|
|
+
|
|
|
RegisterApplicationMasterResponse response =
|
|
|
this.uamPool.registerApplicationMaster(subClusterId.getId(),
|
|
|
this.amRegistrationRequest);
|
|
@@ -436,7 +469,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
* the other sub-cluster RM will be done lazily as needed later.
|
|
|
*/
|
|
|
this.amRegistrationResponse =
|
|
|
- this.homeRM.registerApplicationMaster(request);
|
|
|
+ this.homeRMRelayer.registerApplicationMaster(request);
|
|
|
if (this.amRegistrationResponse
|
|
|
.getContainersFromPreviousAttempts() != null) {
|
|
|
cacheAllocatedContainers(
|
|
@@ -495,6 +528,34 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
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();
|
|
|
+
|
|
|
+ throw new ApplicationMasterNotRegisteredException(
|
|
|
+ "AMRMProxy just restarted and recovered for "
|
|
|
+ + getApplicationContext().getApplicationAttemptId()
|
|
|
+ + ". 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 {}",
|
|
|
+ getApplicationContext().getApplicationAttemptId(),
|
|
|
+ request.getResponseId(), this.lastHomeResponseId);
|
|
|
+ request.setResponseId(this.lastHomeResponseId);
|
|
|
+ }
|
|
|
+
|
|
|
try {
|
|
|
// Split the heart beat request into multiple requests, one for each
|
|
|
// sub-cluster RM that is used by this application.
|
|
@@ -509,10 +570,18 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
sendRequestsToSecondaryResourceManagers(requests);
|
|
|
|
|
|
// Send the request to the home RM and get the response
|
|
|
- AllocateResponse homeResponse = AMRMClientUtils.allocateWithReRegister(
|
|
|
- requests.get(this.homeSubClusterId), this.homeRM,
|
|
|
- this.amRegistrationRequest,
|
|
|
- getApplicationContext().getApplicationAttemptId().getApplicationId());
|
|
|
+ AllocateRequest homeRequest = requests.get(this.homeSubClusterId);
|
|
|
+ LOG.info("{} heartbeating to home RM with responseId {}",
|
|
|
+ getApplicationContext().getApplicationAttemptId(),
|
|
|
+ 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;
|
|
|
+ }
|
|
|
|
|
|
// Notify policy of home response
|
|
|
try {
|
|
@@ -540,6 +609,22 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
newRegistrations.getSuccessfulRegistrations());
|
|
|
}
|
|
|
|
|
|
+ LOG.info("{} heartbeat response from home RM with responseId {}",
|
|
|
+ getApplicationContext().getApplicationAttemptId(),
|
|
|
+ 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();
|
|
|
+ }
|
|
|
+
|
|
|
// return the final response to the application master.
|
|
|
return homeResponse;
|
|
|
} catch (IOException ex) {
|
|
@@ -584,6 +669,16 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
try {
|
|
|
uamResponse =
|
|
|
uamPool.finishApplicationMaster(subClusterId, finishRequest);
|
|
|
+
|
|
|
+ if (uamResponse.getIsUnregistered()) {
|
|
|
+ secondaryRelayers.remove(subClusterId);
|
|
|
+
|
|
|
+ if (getNMStateStore() != null) {
|
|
|
+ getNMStateStore().removeAMRMProxyAppContextEntry(
|
|
|
+ getApplicationContext().getApplicationAttemptId(),
|
|
|
+ NMSS_SECONDARY_SC_PREFIX + subClusterId);
|
|
|
+ }
|
|
|
+ }
|
|
|
} catch (Throwable e) {
|
|
|
LOG.warn("Failed to finish unmanaged application master: "
|
|
|
+ "RM address: " + subClusterId + " ApplicationId: "
|
|
@@ -600,9 +695,7 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
// asynchronously by other sub-cluster resource managers, send the same
|
|
|
// request to the home resource manager on this thread.
|
|
|
FinishApplicationMasterResponse homeResponse =
|
|
|
- AMRMClientUtils.finishAMWithReRegister(request, this.homeRM,
|
|
|
- this.amRegistrationRequest, getApplicationContext()
|
|
|
- .getApplicationAttemptId().getApplicationId());
|
|
|
+ this.homeRMRelayer.finishApplicationMaster(request);
|
|
|
|
|
|
if (subClusterIds.size() > 0) {
|
|
|
// Wait for other sub-cluster resource managers to return the
|
|
@@ -621,10 +714,6 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
if (uamResponse.getResponse() == null
|
|
|
|| !uamResponse.getResponse().getIsUnregistered()) {
|
|
|
failedToUnRegister = true;
|
|
|
- } else if (getNMStateStore() != null) {
|
|
|
- getNMStateStore().removeAMRMProxyAppContextEntry(
|
|
|
- getApplicationContext().getApplicationAttemptId(),
|
|
|
- NMSS_SECONDARY_SC_PREFIX + uamResponse.getSubClusterId());
|
|
|
}
|
|
|
} catch (Throwable e) {
|
|
|
failedToUnRegister = true;
|
|
@@ -689,6 +778,11 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
return this.registryClient;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ protected int getLastHomeResponseId() {
|
|
|
+ return this.lastHomeResponseId;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Create the UAM pool manager for secondary sub-clsuters. For unit test to
|
|
|
* override.
|
|
@@ -800,6 +894,9 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
getApplicationContext().getUser(), homeSubClusterId.getId(),
|
|
|
amrmToken);
|
|
|
|
|
|
+ secondaryRelayers.put(subClusterId.getId(),
|
|
|
+ uamPool.getAMRMClientRelayer(subClusterId.getId()));
|
|
|
+
|
|
|
response = uamPool.registerApplicationMaster(
|
|
|
subClusterId.getId(), amRegistrationRequest);
|
|
|
|
|
@@ -1098,7 +1195,10 @@ public class FederationInterceptor extends AbstractRequestInterceptor {
|
|
|
token = uamPool.launchUAM(subClusterId, config,
|
|
|
appContext.getApplicationAttemptId().getApplicationId(),
|
|
|
amRegistrationResponse.getQueue(), appContext.getUser(),
|
|
|
- homeSubClusterId.toString(), registryClient != null);
|
|
|
+ homeSubClusterId.toString(), true);
|
|
|
+
|
|
|
+ secondaryRelayers.put(subClusterId,
|
|
|
+ uamPool.getAMRMClientRelayer(subClusterId));
|
|
|
|
|
|
uamResponse = uamPool.registerApplicationMaster(subClusterId,
|
|
|
registerRequest);
|