|
@@ -116,6 +116,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationPriorityRespo
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsRequest;
|
|
|
import org.apache.hadoop.yarn.api.protocolrecords.UpdateApplicationTimeoutsResponse;
|
|
|
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
|
|
+import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.ReservationId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
@@ -124,6 +125,7 @@ import org.apache.hadoop.yarn.server.federation.failover.FederationProxyProvider
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.FederationPolicyUtils;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.RouterPolicyFacade;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
|
|
|
+import org.apache.hadoop.yarn.server.federation.retry.FederationActionRetry;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.ReservationHomeSubCluster;
|
|
|
import org.apache.hadoop.yarn.server.federation.store.records.SubClusterId;
|
|
@@ -176,6 +178,7 @@ public class FederationClientInterceptor
|
|
|
private ThreadPoolExecutor executorService;
|
|
|
private final Clock clock = new MonotonicClock();
|
|
|
private boolean returnPartialReport;
|
|
|
+ private long submitIntervalTime;
|
|
|
|
|
|
@Override
|
|
|
public void init(String userName) {
|
|
@@ -207,6 +210,10 @@ public class FederationClientInterceptor
|
|
|
YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_RETRY,
|
|
|
YarnConfiguration.DEFAULT_ROUTER_CLIENTRM_SUBMIT_RETRY);
|
|
|
|
|
|
+ submitIntervalTime = conf.getTimeDuration(
|
|
|
+ YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_INTERVAL_TIME,
|
|
|
+ YarnConfiguration.DEFAULT_CLIENTRM_SUBMIT_INTERVAL_TIME, TimeUnit.MILLISECONDS);
|
|
|
+
|
|
|
clientRMProxies = new ConcurrentHashMap<>();
|
|
|
routerMetrics = RouterMetrics.getMetrics();
|
|
|
|
|
@@ -260,6 +267,17 @@ public class FederationClientInterceptor
|
|
|
return list.get(rand.nextInt(list.size()));
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ private int getActiveSubClustersCount() throws YarnException {
|
|
|
+ Map<SubClusterId, SubClusterInfo> activeSubClusters =
|
|
|
+ federationFacade.getSubClusters(true);
|
|
|
+ if (activeSubClusters == null || activeSubClusters.isEmpty()) {
|
|
|
+ return 0;
|
|
|
+ } else {
|
|
|
+ return activeSubClusters.size();
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* YARN Router forwards every getNewApplication requests to any RM. During
|
|
|
* this operation there will be no communication with the State Store. The
|
|
@@ -293,25 +311,25 @@ public class FederationClientInterceptor
|
|
|
Map<SubClusterId, SubClusterInfo> subClustersActive =
|
|
|
federationFacade.getSubClusters(true);
|
|
|
|
|
|
- for (int i = 0; i < numSubmitRetries; ++i) {
|
|
|
- SubClusterId subClusterId = getRandomActiveSubCluster(subClustersActive);
|
|
|
- LOG.info("getNewApplication try #{} on SubCluster {}.", i, subClusterId);
|
|
|
- ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId);
|
|
|
- GetNewApplicationResponse response = null;
|
|
|
- try {
|
|
|
- response = clientRMProxy.getNewApplication(request);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn("Unable to create a new ApplicationId in SubCluster {}.", subClusterId.getId(), e);
|
|
|
- subClustersActive.remove(subClusterId);
|
|
|
- }
|
|
|
+ // Try calling the getNewApplication method
|
|
|
+ List<SubClusterId> blacklist = new ArrayList<>();
|
|
|
+ int activeSubClustersCount = getActiveSubClustersCount();
|
|
|
+ int actualRetryNums = Math.min(activeSubClustersCount, numSubmitRetries) + 1;
|
|
|
+
|
|
|
+ try {
|
|
|
+ GetNewApplicationResponse response =
|
|
|
+ ((FederationActionRetry<GetNewApplicationResponse>) (retryCount) ->
|
|
|
+ invokeGetNewApplication(subClustersActive, blacklist, request, retryCount)).
|
|
|
+ runWithRetries(actualRetryNums, submitIntervalTime);
|
|
|
|
|
|
if (response != null) {
|
|
|
long stopTime = clock.getTime();
|
|
|
routerMetrics.succeededAppsCreated(stopTime - startTime);
|
|
|
- RouterAuditLogger.logSuccess(user.getShortUserName(), GET_NEW_APP,
|
|
|
- TARGET_CLIENT_RM_SERVICE, response.getApplicationId());
|
|
|
return response;
|
|
|
}
|
|
|
+ } catch (Exception e) {
|
|
|
+ routerMetrics.incrAppsFailedCreated();
|
|
|
+ RouterServerUtil.logAndThrowException(e.getMessage(), e);
|
|
|
}
|
|
|
|
|
|
routerMetrics.incrAppsFailedCreated();
|
|
@@ -321,6 +339,46 @@ public class FederationClientInterceptor
|
|
|
throw new YarnException(errMsg);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Invoke GetNewApplication to different subClusters.
|
|
|
+ *
|
|
|
+ * @param subClustersActive Active SubClusters
|
|
|
+ * @param blackList Blacklist avoid repeated calls to unavailable subCluster.
|
|
|
+ * @param request getNewApplicationRequest.
|
|
|
+ * @param retryCount number of retries.
|
|
|
+ * @return Get NewApplicationResponse response, If the response is empty, the request fails,
|
|
|
+ * if the response is not empty, the request is successful.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ * @throws IOException io error.
|
|
|
+ */
|
|
|
+ private GetNewApplicationResponse invokeGetNewApplication(
|
|
|
+ Map<SubClusterId, SubClusterInfo> subClustersActive,
|
|
|
+ List<SubClusterId> blackList, GetNewApplicationRequest request, int retryCount)
|
|
|
+ throws YarnException, IOException {
|
|
|
+ SubClusterId subClusterId =
|
|
|
+ RouterServerUtil.getRandomActiveSubCluster(subClustersActive, blackList);
|
|
|
+ LOG.info("getNewApplication try #{} on SubCluster {}.", retryCount, subClusterId);
|
|
|
+ ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId);
|
|
|
+ try {
|
|
|
+ GetNewApplicationResponse response = clientRMProxy.getNewApplication(request);
|
|
|
+ if (response != null) {
|
|
|
+ RouterAuditLogger.logSuccess(user.getShortUserName(), GET_NEW_APP,
|
|
|
+ TARGET_CLIENT_RM_SERVICE, response.getApplicationId(), subClusterId);
|
|
|
+ return response;
|
|
|
+ }
|
|
|
+ } catch (Exception e) {
|
|
|
+ RouterAuditLogger.logFailure(user.getShortUserName(), GET_NEW_APP, UNKNOWN,
|
|
|
+ TARGET_CLIENT_RM_SERVICE, e.getMessage(), subClusterId);
|
|
|
+ LOG.warn("Unable to create a new ApplicationId in SubCluster {}.", subClusterId.getId(), e);
|
|
|
+ blackList.add(subClusterId);
|
|
|
+ throw e;
|
|
|
+ }
|
|
|
+ // If SubmitApplicationResponse is empty, the request fails.
|
|
|
+ String msg = String.format("Unable to create a new ApplicationId in SubCluster %s.",
|
|
|
+ subClusterId.getId());
|
|
|
+ throw new YarnException(msg);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Today, in YARN there are no checks of any applicationId submitted.
|
|
|
*
|
|
@@ -400,98 +458,188 @@ public class FederationClientInterceptor
|
|
|
RouterServerUtil.logAndThrowException(errMsg, null);
|
|
|
}
|
|
|
|
|
|
- SubmitApplicationResponse response = null;
|
|
|
-
|
|
|
long startTime = clock.getTime();
|
|
|
-
|
|
|
ApplicationId applicationId =
|
|
|
request.getApplicationSubmissionContext().getApplicationId();
|
|
|
-
|
|
|
List<SubClusterId> blacklist = new ArrayList<>();
|
|
|
|
|
|
- for (int i = 0; i < numSubmitRetries; ++i) {
|
|
|
+ try {
|
|
|
+
|
|
|
+ // We need to handle this situation,
|
|
|
+ // the user will provide us with an expected submitRetries,
|
|
|
+ // but if the number of Active SubClusters is less than this number at this time,
|
|
|
+ // we should provide a high number of retry according to the number of Active SubClusters.
|
|
|
+ int activeSubClustersCount = getActiveSubClustersCount();
|
|
|
+ int actualRetryNums = Math.min(activeSubClustersCount, numSubmitRetries) + 1;
|
|
|
+
|
|
|
+ // Try calling the SubmitApplication method
|
|
|
+ SubmitApplicationResponse response =
|
|
|
+ ((FederationActionRetry<SubmitApplicationResponse>) (retryCount) ->
|
|
|
+ invokeSubmitApplication(blacklist, request, retryCount)).
|
|
|
+ runWithRetries(actualRetryNums, submitIntervalTime);
|
|
|
+
|
|
|
+ if (response != null) {
|
|
|
+ long stopTime = clock.getTime();
|
|
|
+ routerMetrics.succeededAppsSubmitted(stopTime - startTime);
|
|
|
+ return response;
|
|
|
+ }
|
|
|
+
|
|
|
+ } catch (Exception e){
|
|
|
+ routerMetrics.incrAppsFailedSubmitted();
|
|
|
+ RouterServerUtil.logAndThrowException(e.getMessage(), e);
|
|
|
+ }
|
|
|
+
|
|
|
+ routerMetrics.incrAppsFailedSubmitted();
|
|
|
+ String msg = String.format("Application %s with appId %s failed to be submitted.",
|
|
|
+ request.getApplicationSubmissionContext().getApplicationName(), applicationId);
|
|
|
+ RouterAuditLogger.logFailure(user.getShortUserName(), SUBMIT_NEW_APP, UNKNOWN,
|
|
|
+ TARGET_CLIENT_RM_SERVICE, msg, applicationId);
|
|
|
+ throw new YarnException(msg);
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Invoke SubmitApplication to different subClusters.
|
|
|
+ *
|
|
|
+ * Step1. Select homeSubCluster for Application according to Policy.
|
|
|
+ *
|
|
|
+ * Step2. Query homeSubCluster according to ApplicationId,
|
|
|
+ * if homeSubCluster does not exist or first attempt(consider repeated submissions), write;
|
|
|
+ * if homeSubCluster exists, update.
|
|
|
+ *
|
|
|
+ * Step3. Find the clientRMProxy of the corresponding cluster according to homeSubCluster,
|
|
|
+ * and then call the SubmitApplication method.
|
|
|
+ *
|
|
|
+ * Step4. If SubmitApplicationResponse is empty, the request fails,
|
|
|
+ * if SubmitApplicationResponse is not empty, the request is successful.
|
|
|
+ *
|
|
|
+ * @param blackList Blacklist avoid repeated calls to unavailable subCluster.
|
|
|
+ * @param request submitApplicationRequest.
|
|
|
+ * @param retryCount number of retries.
|
|
|
+ * @return submitApplication response, If the response is empty, the request fails,
|
|
|
+ * if the response is not empty, the request is successful.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ */
|
|
|
+ private SubmitApplicationResponse invokeSubmitApplication(
|
|
|
+ List<SubClusterId> blackList, SubmitApplicationRequest request, int retryCount)
|
|
|
+ throws YarnException, IOException {
|
|
|
|
|
|
- SubClusterId subClusterId = policyFacade.getHomeSubcluster(
|
|
|
- request.getApplicationSubmissionContext(), blacklist);
|
|
|
+ // The request is not checked here,
|
|
|
+ // because the request has been checked before the method is called.
|
|
|
+ // We get applicationId and subClusterId from context.
|
|
|
+ ApplicationSubmissionContext appSubmissionContext = request.getApplicationSubmissionContext();
|
|
|
+ ApplicationId applicationId = appSubmissionContext.getApplicationId();
|
|
|
+ SubClusterId subClusterId = null;
|
|
|
|
|
|
+ try {
|
|
|
+
|
|
|
+ // Step1. Select homeSubCluster for Application according to Policy.
|
|
|
+ subClusterId = policyFacade.getHomeSubcluster(appSubmissionContext, blackList);
|
|
|
LOG.info("submitApplication appId {} try #{} on SubCluster {}.",
|
|
|
- applicationId, i, subClusterId);
|
|
|
+ applicationId, retryCount, subClusterId);
|
|
|
+
|
|
|
+ // Step2. Query homeSubCluster according to ApplicationId.
|
|
|
+ Boolean exists = existsApplicationHomeSubCluster(applicationId);
|
|
|
|
|
|
ApplicationHomeSubCluster appHomeSubCluster =
|
|
|
ApplicationHomeSubCluster.newInstance(applicationId, subClusterId);
|
|
|
|
|
|
- if (i == 0) {
|
|
|
- try {
|
|
|
- // persist the mapping of applicationId and the subClusterId which has
|
|
|
- // been selected as its home
|
|
|
- subClusterId =
|
|
|
- federationFacade.addApplicationHomeSubCluster(appHomeSubCluster);
|
|
|
- } catch (YarnException e) {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- String message =
|
|
|
- String.format("Unable to insert the ApplicationId %s into the FederationStateStore.",
|
|
|
- applicationId);
|
|
|
- RouterAuditLogger.logFailure(user.getShortUserName(), SUBMIT_NEW_APP, UNKNOWN,
|
|
|
- TARGET_CLIENT_RM_SERVICE, message, applicationId, subClusterId);
|
|
|
- RouterServerUtil.logAndThrowException(message, e);
|
|
|
- }
|
|
|
+ if (exists || retryCount == 0) {
|
|
|
+ addApplicationHomeSubCluster(applicationId, appHomeSubCluster);
|
|
|
} else {
|
|
|
- try {
|
|
|
- // update the mapping of applicationId and the home subClusterId to
|
|
|
- // the new subClusterId we have selected
|
|
|
- federationFacade.updateApplicationHomeSubCluster(appHomeSubCluster);
|
|
|
- } catch (YarnException e) {
|
|
|
- String message =
|
|
|
- String.format("Unable to update the ApplicationId %s into the FederationStateStore.",
|
|
|
- applicationId);
|
|
|
- SubClusterId subClusterIdInStateStore =
|
|
|
- federationFacade.getApplicationHomeSubCluster(applicationId);
|
|
|
- if (subClusterId == subClusterIdInStateStore) {
|
|
|
- LOG.info("Application {} already submitted on SubCluster {}.",
|
|
|
- applicationId, subClusterId);
|
|
|
- } else {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- RouterAuditLogger.logFailure(user.getShortUserName(), SUBMIT_NEW_APP, UNKNOWN,
|
|
|
- TARGET_CLIENT_RM_SERVICE, message, applicationId, subClusterId);
|
|
|
- RouterServerUtil.logAndThrowException(message, e);
|
|
|
- }
|
|
|
- }
|
|
|
+ updateApplicationHomeSubCluster(subClusterId, applicationId, appHomeSubCluster);
|
|
|
}
|
|
|
|
|
|
- ApplicationClientProtocol clientRMProxy =
|
|
|
- getClientRMProxyForSubCluster(subClusterId);
|
|
|
-
|
|
|
- try {
|
|
|
- response = clientRMProxy.submitApplication(request);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn("Unable to submit the application {} to SubCluster {} error = {}.",
|
|
|
- applicationId, subClusterId.getId(), e);
|
|
|
- }
|
|
|
+ // Step3. SubmitApplication to the subCluster
|
|
|
+ ApplicationClientProtocol clientRMProxy = getClientRMProxyForSubCluster(subClusterId);
|
|
|
+ SubmitApplicationResponse response = clientRMProxy.submitApplication(request);
|
|
|
|
|
|
+ // Step4. if SubmitApplicationResponse is not empty, the request is successful.
|
|
|
if (response != null) {
|
|
|
- LOG.info("Application {} with appId {} submitted on {}.",
|
|
|
- request.getApplicationSubmissionContext().getApplicationName(),
|
|
|
- applicationId, subClusterId);
|
|
|
- long stopTime = clock.getTime();
|
|
|
- routerMetrics.succeededAppsSubmitted(stopTime - startTime);
|
|
|
+ LOG.info("Application {} submitted on subCluster {}.", applicationId, subClusterId);
|
|
|
RouterAuditLogger.logSuccess(user.getShortUserName(), SUBMIT_NEW_APP,
|
|
|
TARGET_CLIENT_RM_SERVICE, applicationId, subClusterId);
|
|
|
return response;
|
|
|
- } else {
|
|
|
- // Empty response from the ResourceManager.
|
|
|
- // Blacklist this subcluster for this request.
|
|
|
- blacklist.add(subClusterId);
|
|
|
}
|
|
|
+ } catch (Exception e) {
|
|
|
+ RouterAuditLogger.logFailure(user.getShortUserName(), SUBMIT_NEW_APP, UNKNOWN,
|
|
|
+ TARGET_CLIENT_RM_SERVICE, e.getMessage(), applicationId, subClusterId);
|
|
|
+ LOG.warn("Unable to submitApplication appId {} try #{} on SubCluster {} error = {}.",
|
|
|
+ applicationId, subClusterId, e);
|
|
|
+ if (subClusterId != null) {
|
|
|
+ blackList.add(subClusterId);
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
}
|
|
|
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- String msg = String.format("Application %s with appId %s failed to be submitted.",
|
|
|
- request.getApplicationSubmissionContext().getApplicationName(), applicationId);
|
|
|
- RouterAuditLogger.logFailure(user.getShortUserName(), SUBMIT_NEW_APP, UNKNOWN,
|
|
|
- TARGET_CLIENT_RM_SERVICE, msg, applicationId);
|
|
|
+ // If SubmitApplicationResponse is empty, the request fails.
|
|
|
+ String msg = String.format("Application %s failed to be submitted.", applicationId);
|
|
|
throw new YarnException(msg);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Add ApplicationHomeSubCluster to FederationStateStore.
|
|
|
+ *
|
|
|
+ * @param applicationId applicationId.
|
|
|
+ * @param homeSubCluster homeSubCluster, homeSubCluster selected according to policy.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ */
|
|
|
+ private void addApplicationHomeSubCluster(ApplicationId applicationId,
|
|
|
+ ApplicationHomeSubCluster homeSubCluster) throws YarnException {
|
|
|
+ try {
|
|
|
+ federationFacade.addApplicationHomeSubCluster(homeSubCluster);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ RouterServerUtil.logAndThrowException(e,
|
|
|
+ "Unable to insert the ApplicationId %s into the FederationStateStore.", applicationId);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Update ApplicationHomeSubCluster to FederationStateStore.
|
|
|
+ *
|
|
|
+ * @param subClusterId homeSubClusterId
|
|
|
+ * @param applicationId applicationId.
|
|
|
+ * @param homeSubCluster homeSubCluster, homeSubCluster selected according to policy.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ */
|
|
|
+ private void updateApplicationHomeSubCluster(SubClusterId subClusterId,
|
|
|
+ ApplicationId applicationId, ApplicationHomeSubCluster homeSubCluster) throws YarnException {
|
|
|
+ try {
|
|
|
+ federationFacade.updateApplicationHomeSubCluster(homeSubCluster);
|
|
|
+ } catch (YarnException e) {
|
|
|
+ SubClusterId subClusterIdInStateStore =
|
|
|
+ federationFacade.getApplicationHomeSubCluster(applicationId);
|
|
|
+ if (subClusterId == subClusterIdInStateStore) {
|
|
|
+ LOG.info("Application {} already submitted on SubCluster {}.",
|
|
|
+ applicationId, subClusterId);
|
|
|
+ } else {
|
|
|
+ RouterServerUtil.logAndThrowException(e,
|
|
|
+ "Unable to update the ApplicationId %s into the FederationStateStore.",
|
|
|
+ applicationId);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Query SubClusterId By applicationId.
|
|
|
+ *
|
|
|
+ * If SubClusterId is not empty, it means it exists and returns true;
|
|
|
+ * if SubClusterId is empty, it means it does not exist and returns false.
|
|
|
+ *
|
|
|
+ * @param applicationId applicationId
|
|
|
+ * @return true, SubClusterId exists; false, SubClusterId not exists.
|
|
|
+ */
|
|
|
+ private boolean existsApplicationHomeSubCluster(ApplicationId applicationId) {
|
|
|
+ try {
|
|
|
+ SubClusterId subClusterId = federationFacade.getApplicationHomeSubCluster(applicationId);
|
|
|
+ if (subClusterId != null) {
|
|
|
+ return true;
|
|
|
+ }
|
|
|
+ } catch (YarnException e) {
|
|
|
+ LOG.warn("get homeSubCluster by applicationId = {} error.", applicationId, e);
|
|
|
+ }
|
|
|
+ return false;
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* The YARN Router will forward to the respective YARN RM in which the AM is
|
|
|
* running.
|