|
@@ -42,6 +42,7 @@ import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.ConcurrentMap;
|
|
|
import java.util.concurrent.atomic.AtomicInteger;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.concurrent.atomic.AtomicLong;
|
|
|
|
|
|
import org.apache.commons.cli.CommandLine;
|
|
|
import org.apache.commons.cli.GnuParser;
|
|
@@ -87,8 +88,11 @@ import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
|
|
import org.apache.hadoop.yarn.api.records.NodeReport;
|
|
|
import org.apache.hadoop.yarn.api.records.Priority;
|
|
|
import org.apache.hadoop.yarn.api.records.ProfileCapability;
|
|
|
+import org.apache.hadoop.yarn.api.records.RejectedSchedulingRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.Resource;
|
|
|
import org.apache.hadoop.yarn.api.records.ResourceRequest;
|
|
|
+import org.apache.hadoop.yarn.api.records.ResourceSizing;
|
|
|
+import org.apache.hadoop.yarn.api.records.SchedulingRequest;
|
|
|
import org.apache.hadoop.yarn.api.records.URL;
|
|
|
import org.apache.hadoop.yarn.api.records.UpdatedContainer;
|
|
|
import org.apache.hadoop.yarn.api.records.ExecutionType;
|
|
@@ -99,6 +103,7 @@ import org.apache.hadoop.yarn.api.records.timeline.TimelineEntity;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEntityGroupId;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelineEvent;
|
|
|
import org.apache.hadoop.yarn.api.records.timeline.TimelinePutResponse;
|
|
|
+import org.apache.hadoop.yarn.api.resource.PlacementConstraint;
|
|
|
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
|
|
|
import org.apache.hadoop.yarn.client.api.TimelineClient;
|
|
|
import org.apache.hadoop.yarn.client.api.TimelineV2Client;
|
|
@@ -274,6 +279,10 @@ public class ApplicationMaster {
|
|
|
@VisibleForTesting
|
|
|
protected AtomicInteger numRequestedContainers = new AtomicInteger();
|
|
|
|
|
|
+ protected AtomicInteger numIgnore = new AtomicInteger();
|
|
|
+
|
|
|
+ protected AtomicInteger totalRetries = new AtomicInteger(10);
|
|
|
+
|
|
|
// Shell command to be executed
|
|
|
private String shellCommand = "";
|
|
|
// Args to be passed to the shell command
|
|
@@ -289,6 +298,9 @@ public class ApplicationMaster {
|
|
|
// File length needed for local resource
|
|
|
private long shellScriptPathLen = 0;
|
|
|
|
|
|
+ // Placement Specifications
|
|
|
+ private Map<String, PlacementSpec> placementSpecs = null;
|
|
|
+
|
|
|
// Container retry options
|
|
|
private ContainerRetryPolicy containerRetryPolicy =
|
|
|
ContainerRetryPolicy.NEVER_RETRY;
|
|
@@ -334,6 +346,7 @@ public class ApplicationMaster {
|
|
|
private final String windows_command = "cmd /c";
|
|
|
|
|
|
private int yarnShellIdCounter = 1;
|
|
|
+ private final AtomicLong allocIdCounter = new AtomicLong(1);
|
|
|
|
|
|
@VisibleForTesting
|
|
|
protected final Set<ContainerId> launchedContainers =
|
|
@@ -457,6 +470,7 @@ public class ApplicationMaster {
|
|
|
"If container could retry, it specifies max retires");
|
|
|
opts.addOption("container_retry_interval", true,
|
|
|
"Interval between each retry, unit is milliseconds");
|
|
|
+ opts.addOption("placement_spec", true, "Placement specification");
|
|
|
opts.addOption("debug", false, "Dump out debug information");
|
|
|
|
|
|
opts.addOption("help", false, "Print usage");
|
|
@@ -487,6 +501,17 @@ public class ApplicationMaster {
|
|
|
dumpOutDebugInfo();
|
|
|
}
|
|
|
|
|
|
+ if (cliParser.hasOption("placement_spec")) {
|
|
|
+ String placementSpec = cliParser.getOptionValue("placement_spec");
|
|
|
+ LOG.info("Placement Spec received [{}]", placementSpec);
|
|
|
+ parsePlacementSpecs(placementSpec);
|
|
|
+ LOG.info("Total num containers requested [{}]", numTotalContainers);
|
|
|
+ if (numTotalContainers == 0) {
|
|
|
+ throw new IllegalArgumentException(
|
|
|
+ "Cannot run distributed shell with no containers");
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
Map<String, String> envs = System.getenv();
|
|
|
|
|
|
if (!envs.containsKey(Environment.CONTAINER_ID.name())) {
|
|
@@ -609,8 +634,11 @@ public class ApplicationMaster {
|
|
|
}
|
|
|
containerResourceProfile =
|
|
|
cliParser.getOptionValue("container_resource_profile", "");
|
|
|
- numTotalContainers = Integer.parseInt(cliParser.getOptionValue(
|
|
|
- "num_containers", "1"));
|
|
|
+
|
|
|
+ if (this.placementSpecs == null) {
|
|
|
+ numTotalContainers = Integer.parseInt(cliParser.getOptionValue(
|
|
|
+ "num_containers", "1"));
|
|
|
+ }
|
|
|
if (numTotalContainers == 0) {
|
|
|
throw new IllegalArgumentException(
|
|
|
"Cannot run distributed shell with no containers");
|
|
@@ -642,6 +670,17 @@ public class ApplicationMaster {
|
|
|
return true;
|
|
|
}
|
|
|
|
|
|
+ private void parsePlacementSpecs(String placementSpecifications) {
|
|
|
+ Map<String, PlacementSpec> pSpecs =
|
|
|
+ PlacementSpec.parse(placementSpecifications);
|
|
|
+ this.placementSpecs = new HashMap<>();
|
|
|
+ this.numTotalContainers = 0;
|
|
|
+ for (PlacementSpec pSpec : pSpecs.values()) {
|
|
|
+ this.numTotalContainers += pSpec.numContainers;
|
|
|
+ this.placementSpecs.put(pSpec.sourceTag, pSpec);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Helper function to print usage
|
|
|
*
|
|
@@ -719,9 +758,19 @@ public class ApplicationMaster {
|
|
|
// Register self with ResourceManager
|
|
|
// This will start heartbeating to the RM
|
|
|
appMasterHostname = NetUtils.getHostname();
|
|
|
+ Map<Set<String>, PlacementConstraint> placementConstraintMap = null;
|
|
|
+ if (this.placementSpecs != null) {
|
|
|
+ placementConstraintMap = new HashMap<>();
|
|
|
+ for (PlacementSpec spec : this.placementSpecs.values()) {
|
|
|
+ if (spec.constraint != null) {
|
|
|
+ placementConstraintMap.put(
|
|
|
+ Collections.singleton(spec.sourceTag), spec.constraint);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
RegisterApplicationMasterResponse response = amRMClient
|
|
|
.registerApplicationMaster(appMasterHostname, appMasterRpcPort,
|
|
|
- appMasterTrackingUrl);
|
|
|
+ appMasterTrackingUrl, placementConstraintMap);
|
|
|
resourceProfiles = response.getResourceProfiles();
|
|
|
ResourceUtils.reinitializeResources(response.getResourceTypes());
|
|
|
// Dump out information about cluster capability as seen by the
|
|
@@ -765,9 +814,20 @@ public class ApplicationMaster {
|
|
|
// containers
|
|
|
// Keep looping until all the containers are launched and shell script
|
|
|
// executed on them ( regardless of success/failure).
|
|
|
- for (int i = 0; i < numTotalContainersToRequest; ++i) {
|
|
|
- ContainerRequest containerAsk = setupContainerAskForRM();
|
|
|
- amRMClient.addContainerRequest(containerAsk);
|
|
|
+ if (this.placementSpecs == null) {
|
|
|
+ for (int i = 0; i < numTotalContainersToRequest; ++i) {
|
|
|
+ ContainerRequest containerAsk = setupContainerAskForRM();
|
|
|
+ amRMClient.addContainerRequest(containerAsk);
|
|
|
+ }
|
|
|
+ } else {
|
|
|
+ List<SchedulingRequest> schedReqs = new ArrayList<>();
|
|
|
+ for (PlacementSpec pSpec : this.placementSpecs.values()) {
|
|
|
+ for (int i = 0; i < pSpec.numContainers; i++) {
|
|
|
+ SchedulingRequest sr = setupSchedulingRequest(pSpec);
|
|
|
+ schedReqs.add(sr);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ amRMClient.addSchedulingRequests(schedReqs);
|
|
|
}
|
|
|
numRequestedContainers.set(numTotalContainers);
|
|
|
}
|
|
@@ -933,6 +993,12 @@ public class ApplicationMaster {
|
|
|
numRequestedContainers.decrementAndGet();
|
|
|
// we do not need to release the container as it would be done
|
|
|
// by the RM
|
|
|
+
|
|
|
+ // Ignore these containers if placementspec is enabled
|
|
|
+ // for the time being.
|
|
|
+ if (placementSpecs != null) {
|
|
|
+ numIgnore.incrementAndGet();
|
|
|
+ }
|
|
|
}
|
|
|
} else {
|
|
|
// nothing to do
|
|
@@ -962,14 +1028,18 @@ public class ApplicationMaster {
|
|
|
int askCount = numTotalContainers - numRequestedContainers.get();
|
|
|
numRequestedContainers.addAndGet(askCount);
|
|
|
|
|
|
- if (askCount > 0) {
|
|
|
- for (int i = 0; i < askCount; ++i) {
|
|
|
- ContainerRequest containerAsk = setupContainerAskForRM();
|
|
|
- amRMClient.addContainerRequest(containerAsk);
|
|
|
+ // Dont bother re-asking if we are using placementSpecs
|
|
|
+ if (placementSpecs == null) {
|
|
|
+ if (askCount > 0) {
|
|
|
+ for (int i = 0; i < askCount; ++i) {
|
|
|
+ ContainerRequest containerAsk = setupContainerAskForRM();
|
|
|
+ amRMClient.addContainerRequest(containerAsk);
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
-
|
|
|
- if (numCompletedContainers.get() == numTotalContainers) {
|
|
|
+
|
|
|
+ if (numCompletedContainers.get() + numIgnore.get() >=
|
|
|
+ numTotalContainers) {
|
|
|
done = true;
|
|
|
}
|
|
|
}
|
|
@@ -1028,6 +1098,23 @@ public class ApplicationMaster {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public void onRequestsRejected(List<RejectedSchedulingRequest> rejReqs) {
|
|
|
+ List<SchedulingRequest> reqsToRetry = new ArrayList<>();
|
|
|
+ for (RejectedSchedulingRequest rejReq : rejReqs) {
|
|
|
+ LOG.info("Scheduling Request {} has been rejected. Reason {}",
|
|
|
+ rejReq.getRequest(), rejReq.getReason());
|
|
|
+ reqsToRetry.add(rejReq.getRequest());
|
|
|
+ }
|
|
|
+ totalRetries.addAndGet(-1 * reqsToRetry.size());
|
|
|
+ if (totalRetries.get() <= 0) {
|
|
|
+ LOG.info("Exiting, since retries are exhausted !!");
|
|
|
+ done = true;
|
|
|
+ } else {
|
|
|
+ amRMClient.addSchedulingRequests(reqsToRetry);
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public void onShutdownRequest() {
|
|
|
done = true;
|
|
@@ -1335,6 +1422,19 @@ public class ApplicationMaster {
|
|
|
return request;
|
|
|
}
|
|
|
|
|
|
+ private SchedulingRequest setupSchedulingRequest(PlacementSpec spec) {
|
|
|
+ long allocId = allocIdCounter.incrementAndGet();
|
|
|
+ SchedulingRequest sReq = SchedulingRequest.newInstance(
|
|
|
+ allocId, Priority.newInstance(requestPriority),
|
|
|
+ ExecutionTypeRequest.newInstance(),
|
|
|
+ Collections.singleton(spec.sourceTag),
|
|
|
+ ResourceSizing.newInstance(
|
|
|
+ createProfileCapability().getProfileCapabilityOverride()), null);
|
|
|
+ sReq.setPlacementConstraint(spec.constraint);
|
|
|
+ LOG.info("Scheduling Request made: " + sReq.toString());
|
|
|
+ return sReq;
|
|
|
+ }
|
|
|
+
|
|
|
private boolean fileExist(String filePath) {
|
|
|
return new File(filePath).exists();
|
|
|
}
|