|
@@ -18,18 +18,15 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager.containermanager.container;
|
|
|
|
|
|
+import java.io.File;
|
|
|
import java.io.IOException;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.nio.ByteBuffer;
|
|
|
-import java.util.ArrayList;
|
|
|
import java.util.Arrays;
|
|
|
import java.util.Collection;
|
|
|
import java.util.EnumSet;
|
|
|
-import java.util.HashMap;
|
|
|
-import java.util.LinkedHashMap;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
-import java.util.concurrent.ConcurrentHashMap;
|
|
|
import java.util.concurrent.locks.Lock;
|
|
|
import java.util.concurrent.locks.ReadWriteLock;
|
|
|
import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|
@@ -65,6 +62,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainersLauncherEventType;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
|
|
|
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationCleanupEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationEvent;
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.event.ContainerLocalizationRequestEvent;
|
|
@@ -124,20 +122,7 @@ public class ContainerImpl implements Container {
|
|
|
private final Configuration daemonConf;
|
|
|
|
|
|
private static final Log LOG = LogFactory.getLog(ContainerImpl.class);
|
|
|
- private final Map<LocalResourceRequest,List<String>> pendingResources =
|
|
|
- new HashMap<LocalResourceRequest,List<String>>();
|
|
|
- private final Map<Path,List<String>> localizedResources =
|
|
|
- new HashMap<Path,List<String>>();
|
|
|
- private final List<LocalResourceRequest> publicRsrcs =
|
|
|
- new ArrayList<LocalResourceRequest>();
|
|
|
- private final List<LocalResourceRequest> privateRsrcs =
|
|
|
- new ArrayList<LocalResourceRequest>();
|
|
|
- private final List<LocalResourceRequest> appRsrcs =
|
|
|
- new ArrayList<LocalResourceRequest>();
|
|
|
- private final Map<LocalResourceRequest, Path> resourcesToBeUploaded =
|
|
|
- new ConcurrentHashMap<LocalResourceRequest, Path>();
|
|
|
- private final Map<LocalResourceRequest, Boolean> resourcesUploadPolicies =
|
|
|
- new ConcurrentHashMap<LocalResourceRequest, Boolean>();
|
|
|
+
|
|
|
|
|
|
// whether container has been recovered after a restart
|
|
|
private RecoveredContainerStatus recoveredStatus =
|
|
@@ -145,6 +130,7 @@ public class ContainerImpl implements Container {
|
|
|
// whether container was marked as killed after recovery
|
|
|
private boolean recoveredAsKilled = false;
|
|
|
private Context context;
|
|
|
+ private ResourceSet resourceSet;
|
|
|
|
|
|
public ContainerImpl(Configuration conf, Dispatcher dispatcher,
|
|
|
ContainerLaunchContext launchContext, Credentials creds,
|
|
@@ -204,6 +190,7 @@ public class ContainerImpl implements Container {
|
|
|
|
|
|
stateMachine = stateMachineFactory.make(this);
|
|
|
this.context = context;
|
|
|
+ this.resourceSet = new ResourceSet();
|
|
|
}
|
|
|
|
|
|
// constructor for a recovered container
|
|
@@ -312,6 +299,12 @@ public class ContainerImpl implements Container {
|
|
|
ContainerState.EXITED_WITH_FAILURE),
|
|
|
ContainerEventType.CONTAINER_EXITED_WITH_FAILURE,
|
|
|
new RetryFailureTransition())
|
|
|
+ .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
|
|
|
+ ContainerEventType.RESOURCE_LOCALIZED,
|
|
|
+ new ResourceLocalizedWhileRunningTransition())
|
|
|
+ .addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
|
|
|
+ ContainerEventType.RESOURCE_FAILED,
|
|
|
+ new ResourceLocalizationFailedWhileRunningTransition())
|
|
|
.addTransition(ContainerState.RUNNING, ContainerState.RUNNING,
|
|
|
ContainerEventType.UPDATE_DIAGNOSTICS_MSG,
|
|
|
UPDATE_DIAGNOSTICS_TRANSITION)
|
|
@@ -470,7 +463,7 @@ public class ContainerImpl implements Container {
|
|
|
try {
|
|
|
if (ContainerState.LOCALIZED == getContainerState()
|
|
|
|| ContainerState.RELAUNCHING == getContainerState()) {
|
|
|
- return localizedResources;
|
|
|
+ return resourceSet.getLocalizedResources();
|
|
|
} else {
|
|
|
return null;
|
|
|
}
|
|
@@ -591,6 +584,11 @@ public class ContainerImpl implements Container {
|
|
|
this.logDir = logDir;
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public ResourceSet getResourceSet() {
|
|
|
+ return this.resourceSet;
|
|
|
+ }
|
|
|
+
|
|
|
@SuppressWarnings("unchecked")
|
|
|
private void sendFinishedEvents() {
|
|
|
// Inform the application
|
|
@@ -653,7 +651,7 @@ public class ContainerImpl implements Container {
|
|
|
for (String s : diags) {
|
|
|
this.diagnostics.append(s);
|
|
|
}
|
|
|
- if (isRetryContextSet() && diagnostics.length() > diagnosticsMaxSize) {
|
|
|
+ if (diagnostics.length() > diagnosticsMaxSize) {
|
|
|
diagnostics.delete(0, diagnostics.length() - diagnosticsMaxSize);
|
|
|
}
|
|
|
try {
|
|
@@ -667,17 +665,7 @@ public class ContainerImpl implements Container {
|
|
|
@SuppressWarnings("unchecked") // dispatcher not typed
|
|
|
public void cleanup() {
|
|
|
Map<LocalResourceVisibility, Collection<LocalResourceRequest>> rsrc =
|
|
|
- new HashMap<LocalResourceVisibility,
|
|
|
- Collection<LocalResourceRequest>>();
|
|
|
- if (!publicRsrcs.isEmpty()) {
|
|
|
- rsrc.put(LocalResourceVisibility.PUBLIC, publicRsrcs);
|
|
|
- }
|
|
|
- if (!privateRsrcs.isEmpty()) {
|
|
|
- rsrc.put(LocalResourceVisibility.PRIVATE, privateRsrcs);
|
|
|
- }
|
|
|
- if (!appRsrcs.isEmpty()) {
|
|
|
- rsrc.put(LocalResourceVisibility.APPLICATION, appRsrcs);
|
|
|
- }
|
|
|
+ resourceSet.getAllResourcesByVisibility();
|
|
|
dispatcher.getEventHandler().handle(
|
|
|
new ContainerLocalizationCleanupEvent(this, rsrc));
|
|
|
}
|
|
@@ -697,7 +685,7 @@ public class ContainerImpl implements Container {
|
|
|
* message.
|
|
|
*
|
|
|
* If there are resources to localize, sends a
|
|
|
- * ContainerLocalizationRequest (INIT_CONTAINER_RESOURCES)
|
|
|
+ * ContainerLocalizationRequest (LOCALIZE_CONTAINER_RESOURCES)
|
|
|
* to the ResourceLocalizationManager and enters LOCALIZING state.
|
|
|
*
|
|
|
* If there are no resources to localize, sends LAUNCH_CONTAINER event
|
|
@@ -749,39 +737,15 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
|
|
|
container.containerLocalizationStartTime = clock.getTime();
|
|
|
+
|
|
|
// Send requests for public, private resources
|
|
|
Map<String,LocalResource> cntrRsrc = ctxt.getLocalResources();
|
|
|
if (!cntrRsrc.isEmpty()) {
|
|
|
try {
|
|
|
- for (Map.Entry<String,LocalResource> rsrc : cntrRsrc.entrySet()) {
|
|
|
- try {
|
|
|
- LocalResourceRequest req =
|
|
|
- new LocalResourceRequest(rsrc.getValue());
|
|
|
- List<String> links = container.pendingResources.get(req);
|
|
|
- if (links == null) {
|
|
|
- links = new ArrayList<String>();
|
|
|
- container.pendingResources.put(req, links);
|
|
|
- }
|
|
|
- links.add(rsrc.getKey());
|
|
|
- storeSharedCacheUploadPolicy(container, req, rsrc.getValue()
|
|
|
- .getShouldBeUploadedToSharedCache());
|
|
|
- switch (rsrc.getValue().getVisibility()) {
|
|
|
- case PUBLIC:
|
|
|
- container.publicRsrcs.add(req);
|
|
|
- break;
|
|
|
- case PRIVATE:
|
|
|
- container.privateRsrcs.add(req);
|
|
|
- break;
|
|
|
- case APPLICATION:
|
|
|
- container.appRsrcs.add(req);
|
|
|
- break;
|
|
|
- }
|
|
|
- } catch (URISyntaxException e) {
|
|
|
- LOG.info("Got exception parsing " + rsrc.getKey()
|
|
|
- + " and value " + rsrc.getValue());
|
|
|
- throw e;
|
|
|
- }
|
|
|
- }
|
|
|
+ Map<LocalResourceVisibility, Collection<LocalResourceRequest>> req =
|
|
|
+ container.resourceSet.addResources(ctxt.getLocalResources());
|
|
|
+ container.dispatcher.getEventHandler().handle(
|
|
|
+ new ContainerLocalizationRequestEvent(container, req));
|
|
|
} catch (URISyntaxException e) {
|
|
|
// malformed resource; abort container launch
|
|
|
LOG.warn("Failed to parse resource-request", e);
|
|
@@ -789,21 +753,6 @@ public class ContainerImpl implements Container {
|
|
|
container.metrics.endInitingContainer();
|
|
|
return ContainerState.LOCALIZATION_FAILED;
|
|
|
}
|
|
|
- Map<LocalResourceVisibility, Collection<LocalResourceRequest>> req =
|
|
|
- new LinkedHashMap<LocalResourceVisibility,
|
|
|
- Collection<LocalResourceRequest>>();
|
|
|
- if (!container.publicRsrcs.isEmpty()) {
|
|
|
- req.put(LocalResourceVisibility.PUBLIC, container.publicRsrcs);
|
|
|
- }
|
|
|
- if (!container.privateRsrcs.isEmpty()) {
|
|
|
- req.put(LocalResourceVisibility.PRIVATE, container.privateRsrcs);
|
|
|
- }
|
|
|
- if (!container.appRsrcs.isEmpty()) {
|
|
|
- req.put(LocalResourceVisibility.APPLICATION, container.appRsrcs);
|
|
|
- }
|
|
|
-
|
|
|
- container.dispatcher.getEventHandler().handle(
|
|
|
- new ContainerLocalizationRequestEvent(container, req));
|
|
|
return ContainerState.LOCALIZING;
|
|
|
} else {
|
|
|
container.sendLaunchEvent();
|
|
@@ -813,27 +762,6 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- /**
|
|
|
- * Store the resource's shared cache upload policies
|
|
|
- * Given LocalResourceRequest can be shared across containers in
|
|
|
- * LocalResourcesTrackerImpl, we preserve the upload policies here.
|
|
|
- * In addition, it is possible for the application to create several
|
|
|
- * "identical" LocalResources as part of
|
|
|
- * ContainerLaunchContext.setLocalResources with different symlinks.
|
|
|
- * There is a corner case where these "identical" local resources have
|
|
|
- * different upload policies. For that scenario, upload policy will be set to
|
|
|
- * true as long as there is at least one LocalResource entry with
|
|
|
- * upload policy set to true.
|
|
|
- */
|
|
|
- private static void storeSharedCacheUploadPolicy(ContainerImpl container,
|
|
|
- LocalResourceRequest resourceRequest, Boolean uploadPolicy) {
|
|
|
- Boolean storedUploadPolicy =
|
|
|
- container.resourcesUploadPolicies.get(resourceRequest);
|
|
|
- if (storedUploadPolicy == null || (!storedUploadPolicy && uploadPolicy)) {
|
|
|
- container.resourcesUploadPolicies.put(resourceRequest, uploadPolicy);
|
|
|
- }
|
|
|
- }
|
|
|
-
|
|
|
/**
|
|
|
* Transition when one of the requested resources for this container
|
|
|
* has been successfully localized.
|
|
@@ -847,22 +775,21 @@ public class ContainerImpl implements Container {
|
|
|
ContainerResourceLocalizedEvent rsrcEvent = (ContainerResourceLocalizedEvent) event;
|
|
|
LocalResourceRequest resourceRequest = rsrcEvent.getResource();
|
|
|
Path location = rsrcEvent.getLocation();
|
|
|
- List<String> syms = container.pendingResources.remove(resourceRequest);
|
|
|
+ List<String> syms =
|
|
|
+ container.resourceSet.resourceLocalized(resourceRequest, location);
|
|
|
if (null == syms) {
|
|
|
- LOG.warn("Localized unknown resource " + resourceRequest +
|
|
|
- " for container " + container.containerId);
|
|
|
- assert false;
|
|
|
- // fail container?
|
|
|
+ LOG.info("Localized resource " + resourceRequest +
|
|
|
+ " for container " + container.containerId);
|
|
|
return ContainerState.LOCALIZING;
|
|
|
}
|
|
|
- container.localizedResources.put(location, syms);
|
|
|
|
|
|
// check to see if this resource should be uploaded to the shared cache
|
|
|
// as well
|
|
|
if (shouldBeUploadedToSharedCache(container, resourceRequest)) {
|
|
|
- container.resourcesToBeUploaded.put(resourceRequest, location);
|
|
|
+ container.resourceSet.getResourcesToBeUploaded()
|
|
|
+ .put(resourceRequest, location);
|
|
|
}
|
|
|
- if (!container.pendingResources.isEmpty()) {
|
|
|
+ if (!container.resourceSet.getPendingResources().isEmpty()) {
|
|
|
return ContainerState.LOCALIZING;
|
|
|
}
|
|
|
|
|
@@ -884,7 +811,8 @@ public class ContainerImpl implements Container {
|
|
|
&& container.recoveredStatus != RecoveredContainerStatus.COMPLETED) {
|
|
|
// kick off uploads to the shared cache
|
|
|
container.dispatcher.getEventHandler().handle(
|
|
|
- new SharedCacheUploadEvent(container.resourcesToBeUploaded, container
|
|
|
+ new SharedCacheUploadEvent(
|
|
|
+ container.resourceSet.getResourcesToBeUploaded(), container
|
|
|
.getLaunchContext(), container.getUser(),
|
|
|
SharedCacheUploadEventType.UPLOAD));
|
|
|
}
|
|
@@ -893,6 +821,56 @@ public class ContainerImpl implements Container {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Resource is localized while the container is running - create symlinks
|
|
|
+ */
|
|
|
+ static class ResourceLocalizedWhileRunningTransition
|
|
|
+ extends ContainerTransition {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ ContainerResourceLocalizedEvent rsrcEvent =
|
|
|
+ (ContainerResourceLocalizedEvent) event;
|
|
|
+ List<String> links = container.resourceSet
|
|
|
+ .resourceLocalized(rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
+ // creating symlinks.
|
|
|
+ for (String link : links) {
|
|
|
+ try {
|
|
|
+ String linkFile = new Path(container.workDir, link).toString();
|
|
|
+ if (new File(linkFile).exists()) {
|
|
|
+ LOG.info("Symlink file already exists: " + linkFile);
|
|
|
+ } else {
|
|
|
+ container.context.getContainerExecutor()
|
|
|
+ .symLink(rsrcEvent.getLocation().toString(), linkFile);
|
|
|
+ LOG.info("Created symlink: " + linkFile + " -> " + rsrcEvent
|
|
|
+ .getLocation());
|
|
|
+ }
|
|
|
+ } catch (IOException e) {
|
|
|
+ String message = String
|
|
|
+ .format("Error when creating symlink %s -> %s", link,
|
|
|
+ rsrcEvent.getLocation());
|
|
|
+ LOG.error(message, e);
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Resource localization failed while the container is running.
|
|
|
+ */
|
|
|
+ static class ResourceLocalizationFailedWhileRunningTransition
|
|
|
+ extends ContainerTransition {
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
+ ContainerResourceFailedEvent failedEvent =
|
|
|
+ (ContainerResourceFailedEvent) event;
|
|
|
+ container.resourceSet
|
|
|
+ .resourceLocalizationFailed(failedEvent.getResource());
|
|
|
+ container.addDiagnostics(failedEvent.getDiagnosticMessage());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Transition from LOCALIZED state to RUNNING state upon receiving
|
|
|
* a CONTAINER_LAUNCHED event
|
|
@@ -1136,17 +1114,10 @@ public class ContainerImpl implements Container {
|
|
|
SingleArcTransition<ContainerImpl, ContainerEvent> {
|
|
|
@Override
|
|
|
public void transition(ContainerImpl container, ContainerEvent event) {
|
|
|
- ContainerResourceLocalizedEvent rsrcEvent = (ContainerResourceLocalizedEvent) event;
|
|
|
- List<String> syms =
|
|
|
- container.pendingResources.remove(rsrcEvent.getResource());
|
|
|
- if (null == syms) {
|
|
|
- LOG.warn("Localized unknown resource " + rsrcEvent.getResource() +
|
|
|
- " for container " + container.containerId);
|
|
|
- assert false;
|
|
|
- // fail container?
|
|
|
- return;
|
|
|
- }
|
|
|
- container.localizedResources.put(rsrcEvent.getLocation(), syms);
|
|
|
+ ContainerResourceLocalizedEvent rsrcEvent =
|
|
|
+ (ContainerResourceLocalizedEvent) event;
|
|
|
+ container.resourceSet
|
|
|
+ .resourceLocalized(rsrcEvent.getResource(), rsrcEvent.getLocation());
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -1402,7 +1373,7 @@ public class ContainerImpl implements Container {
|
|
|
*/
|
|
|
private static boolean shouldBeUploadedToSharedCache(ContainerImpl container,
|
|
|
LocalResourceRequest resource) {
|
|
|
- return container.resourcesUploadPolicies.get(resource);
|
|
|
+ return container.resourceSet.getResourcesUploadPolicies().get(resource);
|
|
|
}
|
|
|
|
|
|
@VisibleForTesting
|