|
@@ -27,14 +27,13 @@ import java.util.HashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
import java.util.Map.Entry;
|
|
|
-import java.util.Random;
|
|
|
import java.util.Set;
|
|
|
import java.util.concurrent.Callable;
|
|
|
import java.util.concurrent.CompletionService;
|
|
|
import java.util.concurrent.ExecutorCompletionService;
|
|
|
import java.util.concurrent.ExecutorService;
|
|
|
import java.util.concurrent.Future;
|
|
|
-import java.util.stream.Collectors;
|
|
|
+import java.util.concurrent.TimeUnit;
|
|
|
|
|
|
import javax.servlet.http.HttpServletRequest;
|
|
|
import javax.servlet.http.HttpServletRequestWrapper;
|
|
@@ -62,9 +61,10 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
|
|
|
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
|
|
|
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.FederationPolicyException;
|
|
|
import org.apache.hadoop.yarn.server.federation.policies.exceptions.FederationPolicyInitializationException;
|
|
|
import org.apache.hadoop.yarn.server.federation.resolver.SubClusterResolver;
|
|
|
-import org.apache.hadoop.yarn.server.federation.store.records.ApplicationHomeSubCluster;
|
|
|
+import org.apache.hadoop.yarn.server.federation.retry.FederationActionRetry;
|
|
|
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.FederationStateStoreFacade;
|
|
@@ -134,13 +134,13 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
|
|
|
|
|
|
private int numSubmitRetries;
|
|
|
private FederationStateStoreFacade federationFacade;
|
|
|
- private Random rand;
|
|
|
private RouterPolicyFacade policyFacade;
|
|
|
private RouterMetrics routerMetrics;
|
|
|
private final Clock clock = new MonotonicClock();
|
|
|
private boolean returnPartialReport;
|
|
|
private boolean appInfosCacheEnabled;
|
|
|
private int appInfosCacheCount;
|
|
|
+ private long submitIntervalTime;
|
|
|
|
|
|
private Map<SubClusterId, DefaultRequestInterceptorREST> interceptors;
|
|
|
private LRUCacheHashMap<RouterAppInfoCacheKey, AppsInfo> appInfosCaches;
|
|
@@ -156,7 +156,6 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
|
|
|
super.init(user);
|
|
|
|
|
|
federationFacade = FederationStateStoreFacade.getInstance();
|
|
|
- rand = new Random();
|
|
|
|
|
|
final Configuration conf = this.getConf();
|
|
|
|
|
@@ -194,24 +193,10 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
|
|
|
YarnConfiguration.DEFAULT_ROUTER_APPSINFO_CACHED_COUNT);
|
|
|
appInfosCaches = new LRUCacheHashMap<>(appInfosCacheCount, true);
|
|
|
}
|
|
|
- }
|
|
|
-
|
|
|
- private SubClusterId getRandomActiveSubCluster(
|
|
|
- Map<SubClusterId, SubClusterInfo> activeSubclusters,
|
|
|
- List<SubClusterId> blackListSubClusters) throws YarnException {
|
|
|
-
|
|
|
- if (activeSubclusters == null || activeSubclusters.size() < 1) {
|
|
|
- RouterServerUtil.logAndThrowException(
|
|
|
- FederationPolicyUtils.NO_ACTIVE_SUBCLUSTER_AVAILABLE, null);
|
|
|
- }
|
|
|
- Collection<SubClusterId> keySet = activeSubclusters.keySet();
|
|
|
- FederationPolicyUtils.validateSubClusterAvailability(keySet, blackListSubClusters);
|
|
|
- if (blackListSubClusters != null) {
|
|
|
- keySet.removeAll(blackListSubClusters);
|
|
|
- }
|
|
|
|
|
|
- List<SubClusterId> list = keySet.stream().collect(Collectors.toList());
|
|
|
- return list.get(rand.nextInt(list.size()));
|
|
|
+ submitIntervalTime = conf.getTimeDuration(
|
|
|
+ YarnConfiguration.ROUTER_CLIENTRM_SUBMIT_INTERVAL_TIME,
|
|
|
+ YarnConfiguration.DEFAULT_CLIENTRM_SUBMIT_INTERVAL_TIME, TimeUnit.MILLISECONDS);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|
|
@@ -301,62 +286,79 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
|
|
|
|
|
|
long startTime = clock.getTime();
|
|
|
|
|
|
- Map<SubClusterId, SubClusterInfo> subClustersActive;
|
|
|
try {
|
|
|
- subClustersActive = federationFacade.getSubClusters(true);
|
|
|
- } catch (YarnException e) {
|
|
|
+ Map<SubClusterId, SubClusterInfo> subClustersActive =
|
|
|
+ federationFacade.getSubClusters(true);
|
|
|
+
|
|
|
+ // We declare blackList and retries.
|
|
|
+ List<SubClusterId> blackList = new ArrayList<>();
|
|
|
+ int actualRetryNums = federationFacade.getRetryNumbers(numSubmitRetries);
|
|
|
+ Response response = ((FederationActionRetry<Response>) (retryCount) ->
|
|
|
+ invokeGetNewApplication(subClustersActive, blackList, hsr, retryCount)).
|
|
|
+ runWithRetries(actualRetryNums, submitIntervalTime);
|
|
|
+
|
|
|
+ // If the response is not empty and the status is SC_OK,
|
|
|
+ // this request can be returned directly.
|
|
|
+ if (response != null && response.getStatus() == HttpServletResponse.SC_OK) {
|
|
|
+ long stopTime = clock.getTime();
|
|
|
+ routerMetrics.succeededAppsCreated(stopTime - startTime);
|
|
|
+ return response;
|
|
|
+ }
|
|
|
+ } catch (FederationPolicyException e) {
|
|
|
+ // If a FederationPolicyException is thrown, the service is unavailable.
|
|
|
routerMetrics.incrAppsFailedCreated();
|
|
|
- return Response.status(Status.INTERNAL_SERVER_ERROR)
|
|
|
- .entity(e.getLocalizedMessage()).build();
|
|
|
+ return Response.status(Status.SERVICE_UNAVAILABLE).entity(e.getLocalizedMessage()).build();
|
|
|
+ } catch (Exception e) {
|
|
|
+ routerMetrics.incrAppsFailedCreated();
|
|
|
+ return Response.status(Status.INTERNAL_SERVER_ERROR).entity(e.getLocalizedMessage()).build();
|
|
|
}
|
|
|
|
|
|
- List<SubClusterId> blacklist = new ArrayList<>();
|
|
|
-
|
|
|
- for (int i = 0; i < numSubmitRetries; ++i) {
|
|
|
-
|
|
|
- SubClusterId subClusterId;
|
|
|
- try {
|
|
|
- subClusterId = getRandomActiveSubCluster(subClustersActive, blacklist);
|
|
|
- } catch (YarnException e) {
|
|
|
- routerMetrics.incrAppsFailedCreated();
|
|
|
- return Response.status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(e.getLocalizedMessage()).build();
|
|
|
- }
|
|
|
+ // return error message directly.
|
|
|
+ String errMsg = "Fail to create a new application.";
|
|
|
+ LOG.error(errMsg);
|
|
|
+ routerMetrics.incrAppsFailedCreated();
|
|
|
+ return Response.status(Status.INTERNAL_SERVER_ERROR).entity(errMsg).build();
|
|
|
+ }
|
|
|
|
|
|
- LOG.debug("getNewApplication try #{} on SubCluster {}.", i, subClusterId);
|
|
|
+ /**
|
|
|
+ * Invoke GetNewApplication to different subClusters.
|
|
|
+ *
|
|
|
+ * @param subClustersActive Active SubClusters.
|
|
|
+ * @param blackList Blacklist avoid repeated calls to unavailable subCluster.
|
|
|
+ * @param hsr HttpServletRequest.
|
|
|
+ * @param retryCount number of retries.
|
|
|
+ * @return Get response, If the response is empty or status not equal SC_OK, the request fails,
|
|
|
+ * if the response is not empty and status equal SC_OK, the request is successful.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ * @throws IOException io error.
|
|
|
+ * @throws InterruptedException interrupted exception.
|
|
|
+ */
|
|
|
+ private Response invokeGetNewApplication(Map<SubClusterId, SubClusterInfo> subClustersActive,
|
|
|
+ List<SubClusterId> blackList, HttpServletRequest hsr, int retryCount)
|
|
|
+ throws YarnException, IOException, InterruptedException {
|
|
|
|
|
|
- DefaultRequestInterceptorREST interceptor =
|
|
|
- getOrCreateInterceptorForSubCluster(subClusterId,
|
|
|
- subClustersActive.get(subClusterId).getRMWebServiceAddress());
|
|
|
- Response response = null;
|
|
|
- try {
|
|
|
- response = interceptor.createNewApplication(hsr);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn("Unable to create a new ApplicationId in SubCluster {}.",
|
|
|
- subClusterId.getId(), e);
|
|
|
- }
|
|
|
+ SubClusterId subClusterId =
|
|
|
+ federationFacade.getRandomActiveSubCluster(subClustersActive, blackList);
|
|
|
|
|
|
- if (response != null &&
|
|
|
- response.getStatus() == HttpServletResponse.SC_OK) {
|
|
|
+ LOG.info("getNewApplication try #{} on SubCluster {}.", retryCount, subClusterId);
|
|
|
|
|
|
- long stopTime = clock.getTime();
|
|
|
- routerMetrics.succeededAppsCreated(stopTime - startTime);
|
|
|
+ DefaultRequestInterceptorREST interceptor = getOrCreateInterceptorForSubCluster(subClusterId,
|
|
|
+ subClustersActive.get(subClusterId).getRMWebServiceAddress());
|
|
|
|
|
|
+ try {
|
|
|
+ Response response = interceptor.createNewApplication(hsr);
|
|
|
+ if (response != null && response.getStatus() == HttpServletResponse.SC_OK) {
|
|
|
return response;
|
|
|
- } else {
|
|
|
- // Empty response from the ResourceManager.
|
|
|
- // Blacklist this subcluster for this request.
|
|
|
- blacklist.add(subClusterId);
|
|
|
}
|
|
|
+ } catch (Exception e) {
|
|
|
+ blackList.add(subClusterId);
|
|
|
+ RouterServerUtil.logAndThrowException(e.getMessage(), e);
|
|
|
}
|
|
|
|
|
|
- String errMsg = "Fail to create a new application.";
|
|
|
- LOG.error(errMsg);
|
|
|
- routerMetrics.incrAppsFailedCreated();
|
|
|
- return Response
|
|
|
- .status(Status.INTERNAL_SERVER_ERROR)
|
|
|
- .entity(errMsg)
|
|
|
- .build();
|
|
|
+ // We need to throw the exception directly.
|
|
|
+ String msg = String.format("Unable to create a new ApplicationId in SubCluster %s.",
|
|
|
+ subClusterId.getId());
|
|
|
+ throw new YarnException(msg);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -431,142 +433,106 @@ public class FederationInterceptorREST extends AbstractRESTRequestInterceptor {
|
|
|
|
|
|
long startTime = clock.getTime();
|
|
|
|
|
|
+ // We verify the parameters to ensure that newApp is not empty and
|
|
|
+ // that the format of applicationId is correct.
|
|
|
if (newApp == null || newApp.getApplicationId() == null) {
|
|
|
routerMetrics.incrAppsFailedSubmitted();
|
|
|
String errMsg = "Missing ApplicationSubmissionContextInfo or "
|
|
|
+ "applicationSubmissionContext information.";
|
|
|
- return Response
|
|
|
- .status(Status.BAD_REQUEST)
|
|
|
- .entity(errMsg)
|
|
|
- .build();
|
|
|
+ return Response.status(Status.BAD_REQUEST).entity(errMsg).build();
|
|
|
}
|
|
|
|
|
|
- ApplicationId applicationId = null;
|
|
|
try {
|
|
|
- applicationId = ApplicationId.fromString(newApp.getApplicationId());
|
|
|
+ String applicationId = newApp.getApplicationId();
|
|
|
+ RouterServerUtil.validateApplicationId(applicationId);
|
|
|
} catch (IllegalArgumentException e) {
|
|
|
routerMetrics.incrAppsFailedSubmitted();
|
|
|
- return Response
|
|
|
- .status(Status.BAD_REQUEST)
|
|
|
- .entity(e.getLocalizedMessage())
|
|
|
- .build();
|
|
|
+ return Response.status(Status.BAD_REQUEST).entity(e.getLocalizedMessage()).build();
|
|
|
}
|
|
|
|
|
|
- List<SubClusterId> blacklist = new ArrayList<>();
|
|
|
-
|
|
|
- for (int i = 0; i < numSubmitRetries; ++i) {
|
|
|
-
|
|
|
- ApplicationSubmissionContext context =
|
|
|
- RMWebAppUtil.createAppSubmissionContext(newApp, this.getConf());
|
|
|
-
|
|
|
- SubClusterId subClusterId = null;
|
|
|
- try {
|
|
|
- subClusterId = policyFacade.getHomeSubcluster(context, blacklist);
|
|
|
- } catch (YarnException e) {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(e.getLocalizedMessage())
|
|
|
- .build();
|
|
|
+ List<SubClusterId> blackList = new ArrayList<>();
|
|
|
+ try {
|
|
|
+ int activeSubClustersCount = federationFacade.getActiveSubClustersCount();
|
|
|
+ int actualRetryNums = Math.min(activeSubClustersCount, numSubmitRetries);
|
|
|
+ Response response = ((FederationActionRetry<Response>) (retryCount) ->
|
|
|
+ invokeSubmitApplication(newApp, blackList, hsr, retryCount)).
|
|
|
+ runWithRetries(actualRetryNums, submitIntervalTime);
|
|
|
+ if (response != null) {
|
|
|
+ long stopTime = clock.getTime();
|
|
|
+ routerMetrics.succeededAppsSubmitted(stopTime - startTime);
|
|
|
+ return response;
|
|
|
}
|
|
|
- LOG.info("submitApplication appId {} try #{} on SubCluster {}.",
|
|
|
- applicationId, i, subClusterId);
|
|
|
+ } catch (Exception e) {
|
|
|
+ routerMetrics.incrAppsFailedSubmitted();
|
|
|
+ return Response.status(Status.SERVICE_UNAVAILABLE).entity(e.getLocalizedMessage()).build();
|
|
|
+ }
|
|
|
|
|
|
- ApplicationHomeSubCluster appHomeSubCluster =
|
|
|
- ApplicationHomeSubCluster.newInstance(applicationId, subClusterId);
|
|
|
+ routerMetrics.incrAppsFailedSubmitted();
|
|
|
+ String errMsg = String.format("Application %s with appId %s failed to be submitted.",
|
|
|
+ newApp.getApplicationName(), newApp.getApplicationId());
|
|
|
+ LOG.error(errMsg);
|
|
|
+ return Response.status(Status.SERVICE_UNAVAILABLE).entity(errMsg).build();
|
|
|
+ }
|
|
|
|
|
|
- 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 errMsg = "Unable to insert the ApplicationId " + applicationId
|
|
|
- + " into the FederationStateStore";
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(errMsg + " " + e.getLocalizedMessage())
|
|
|
- .build();
|
|
|
- }
|
|
|
- } 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 errMsg = "Unable to update the ApplicationId " + applicationId
|
|
|
- + " into the FederationStateStore";
|
|
|
- SubClusterId subClusterIdInStateStore;
|
|
|
- try {
|
|
|
- subClusterIdInStateStore =
|
|
|
- federationFacade.getApplicationHomeSubCluster(applicationId);
|
|
|
- } catch (YarnException e1) {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(e1.getLocalizedMessage())
|
|
|
- .build();
|
|
|
- }
|
|
|
- if (subClusterId == subClusterIdInStateStore) {
|
|
|
- LOG.info("Application {} already submitted on SubCluster {}.",
|
|
|
- applicationId, subClusterId);
|
|
|
- } else {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(errMsg)
|
|
|
- .build();
|
|
|
- }
|
|
|
- }
|
|
|
- }
|
|
|
+ /**
|
|
|
+ * Invoke SubmitApplication to different subClusters.
|
|
|
+ *
|
|
|
+ * @param submissionContext application submission context.
|
|
|
+ * @param blackList Blacklist avoid repeated calls to unavailable subCluster.
|
|
|
+ * @param hsr HttpServletRequest.
|
|
|
+ * @param retryCount number of retries.
|
|
|
+ * @return Get response, If the response is empty or status not equal SC_ACCEPTED,
|
|
|
+ * the request fails, if the response is not empty and status equal SC_OK,
|
|
|
+ * the request is successful.
|
|
|
+ * @throws YarnException yarn exception.
|
|
|
+ * @throws IOException io error.
|
|
|
+ */
|
|
|
+ private Response invokeSubmitApplication(ApplicationSubmissionContextInfo submissionContext,
|
|
|
+ List<SubClusterId> blackList, HttpServletRequest hsr, int retryCount)
|
|
|
+ throws YarnException, IOException, InterruptedException {
|
|
|
+
|
|
|
+ // Step1. We convert ApplicationSubmissionContextInfo to ApplicationSubmissionContext
|
|
|
+ // and Prepare parameters.
|
|
|
+ ApplicationSubmissionContext context =
|
|
|
+ RMWebAppUtil.createAppSubmissionContext(submissionContext, this.getConf());
|
|
|
+ ApplicationId applicationId = ApplicationId.fromString(submissionContext.getApplicationId());
|
|
|
+ SubClusterId subClusterId = null;
|
|
|
|
|
|
- SubClusterInfo subClusterInfo;
|
|
|
- try {
|
|
|
- subClusterInfo = federationFacade.getSubCluster(subClusterId);
|
|
|
- } catch (YarnException e) {
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(e.getLocalizedMessage())
|
|
|
- .build();
|
|
|
- }
|
|
|
+ try {
|
|
|
+ // Get subClusterId from policy.
|
|
|
+ subClusterId = policyFacade.getHomeSubcluster(context, blackList);
|
|
|
|
|
|
- Response response = null;
|
|
|
- try {
|
|
|
- response = getOrCreateInterceptorForSubCluster(subClusterId,
|
|
|
- subClusterInfo.getRMWebServiceAddress()).submitApplication(newApp,
|
|
|
- hsr);
|
|
|
- } catch (Exception e) {
|
|
|
- LOG.warn("Unable to submit the application {} to SubCluster {}",
|
|
|
- applicationId, subClusterId.getId(), e);
|
|
|
- }
|
|
|
+ // Print the log of submitting the submitApplication.
|
|
|
+ LOG.info("submitApplication appId {} try #{} on SubCluster {}.",
|
|
|
+ applicationId, retryCount, subClusterId);
|
|
|
|
|
|
- if (response != null &&
|
|
|
- response.getStatus() == HttpServletResponse.SC_ACCEPTED) {
|
|
|
- LOG.info("Application {} with appId {} submitted on {}",
|
|
|
- context.getApplicationName(), applicationId, subClusterId);
|
|
|
+ // Step2. We Store the mapping relationship
|
|
|
+ // between Application and HomeSubCluster in stateStore.
|
|
|
+ federationFacade.addOrUpdateApplicationHomeSubCluster(
|
|
|
+ applicationId, subClusterId, retryCount);
|
|
|
|
|
|
- long stopTime = clock.getTime();
|
|
|
- routerMetrics.succeededAppsSubmitted(stopTime - startTime);
|
|
|
+ // Step3. We get subClusterInfo based on subClusterId.
|
|
|
+ SubClusterInfo subClusterInfo = federationFacade.getSubCluster(subClusterId);
|
|
|
|
|
|
+ // Step4. Submit the request, if the response is HttpServletResponse.SC_ACCEPTED,
|
|
|
+ // We return the response, otherwise we throw an exception.
|
|
|
+ Response response = getOrCreateInterceptorForSubCluster(subClusterId,
|
|
|
+ subClusterInfo.getRMWebServiceAddress()).submitApplication(submissionContext, hsr);
|
|
|
+ if (response != null && response.getStatus() == HttpServletResponse.SC_ACCEPTED) {
|
|
|
+ LOG.info("Application {} with appId {} submitted on {}.",
|
|
|
+ context.getApplicationName(), applicationId, subClusterId);
|
|
|
return response;
|
|
|
- } else {
|
|
|
- // Empty response from the ResourceManager.
|
|
|
- // Blacklist this subcluster for this request.
|
|
|
- blacklist.add(subClusterId);
|
|
|
}
|
|
|
+ String msg = String.format("application %s failed to be submitted.", applicationId);
|
|
|
+ throw new YarnException(msg);
|
|
|
+ } catch (Exception e) {
|
|
|
+ LOG.warn("Unable to submit the application {} to SubCluster {}.", applicationId,
|
|
|
+ subClusterId, e);
|
|
|
+ if (subClusterId != null) {
|
|
|
+ blackList.add(subClusterId);
|
|
|
+ }
|
|
|
+ throw e;
|
|
|
}
|
|
|
-
|
|
|
- routerMetrics.incrAppsFailedSubmitted();
|
|
|
- String errMsg = "Application " + newApp.getApplicationName()
|
|
|
- + " with appId " + applicationId + " failed to be submitted.";
|
|
|
- LOG.error(errMsg);
|
|
|
- return Response
|
|
|
- .status(Status.SERVICE_UNAVAILABLE)
|
|
|
- .entity(errMsg)
|
|
|
- .build();
|
|
|
}
|
|
|
|
|
|
/**
|