YARN-9562. Add Java changes for the new RuncContainerRuntime. Contributed by Eric Badger
This commit is contained in:
parent
289bbca870
commit
0e22e9ab83
|
@ -144,6 +144,7 @@
|
|||
<item name="YARN Application Security" href="hadoop-yarn/hadoop-yarn-site/YarnApplicationSecurity.html"/>
|
||||
<item name="NodeManager" href="hadoop-yarn/hadoop-yarn-site/NodeManager.html"/>
|
||||
<item name="Running Applications in Docker Containers" href="hadoop-yarn/hadoop-yarn-site/DockerContainers.html"/>
|
||||
<item name="Running Applications in runC Containers" href="hadoop-yarn/hadoop-yarn-site/RuncContainers.html"/>
|
||||
<item name="Using CGroups" href="hadoop-yarn/hadoop-yarn-site/NodeManagerCgroups.html"/>
|
||||
<item name="Secure Containers" href="hadoop-yarn/hadoop-yarn-site/SecureContainer.html"/>
|
||||
<item name="Reservation System" href="hadoop-yarn/hadoop-yarn-site/ReservationSystem.html"/>
|
||||
|
|
|
@ -2001,6 +2001,7 @@ public class YarnConfiguration extends Configuration {
|
|||
* <li>default</li>
|
||||
* <li>docker</li>
|
||||
* <li>javasandbox</li>
|
||||
* <li>runc</li>
|
||||
* </ul>
|
||||
*/
|
||||
public static final String LINUX_CONTAINER_RUNTIME_ALLOWED_RUNTIMES =
|
||||
|
@ -2017,6 +2018,200 @@ public class YarnConfiguration extends Configuration {
|
|||
public static final String LINUX_CONTAINER_RUNTIME_TYPE =
|
||||
LINUX_CONTAINER_RUNTIME_PREFIX + "type";
|
||||
|
||||
public static final String RUNC_CONTAINER_RUNTIME_PREFIX =
|
||||
LINUX_CONTAINER_RUNTIME_PREFIX + "runc.";
|
||||
|
||||
/**
|
||||
* The runc image tag to manifest plugin class that should be used.
|
||||
*/
|
||||
public static final String NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "image-tag-to-manifest-plugin";
|
||||
|
||||
/** Default runc image tag to manifest plugin class. */
|
||||
public static final String DEFAULT_NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN =
|
||||
"org.apache.hadoop.yarn.server.nodemanager.containermanager" +
|
||||
".linux.runtime.runc.ImageTagToManifestPlugin";
|
||||
|
||||
/**
|
||||
* The runc manifest to resources plugin class that should be used.
|
||||
*/
|
||||
public static final String NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "manifest-to-resources-plugin";
|
||||
|
||||
/** Default runc manifest to resources plugin plugin class. */
|
||||
public static final String DEFAULT_NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN =
|
||||
"org.apache.hadoop.yarn.server.nodemanager.containermanager" +
|
||||
".linux.runtime.runc.HdfsManifestToResourcesPlugin";
|
||||
|
||||
/**
|
||||
* The HDFS location under which the oci image manifests, layers,
|
||||
* and configs directories exist.
|
||||
*/
|
||||
public static final String NM_RUNC_IMAGE_TOPLEVEL_DIR =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "image-toplevel-dir";
|
||||
|
||||
/**
|
||||
* Default HDFS location under which the oci image manifests, layers,
|
||||
* and configs directories exist.
|
||||
*/
|
||||
public static final String DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR =
|
||||
"/runc-root";
|
||||
|
||||
/**
|
||||
* Target count of layer mounts that we should keep on disk at one time.
|
||||
*/
|
||||
public static final String NM_RUNC_LAYER_MOUNTS_TO_KEEP =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "layer-mounts-to-keep";
|
||||
|
||||
public static final int DEFAULT_NM_RUNC_LAYER_MOUNTS_TO_KEEP = 100;
|
||||
|
||||
/**
|
||||
* The interval in seconds between executions of reaping layer mounts.
|
||||
*/
|
||||
public static final String NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "layer-mounts-interval-secs";
|
||||
|
||||
public static final int DEFAULT_NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL = 600;
|
||||
|
||||
/** Default runc image to be used. */
|
||||
public static final String NM_RUNC_IMAGE_NAME =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "image-name";
|
||||
|
||||
/** Allow privileged containers. Use with extreme care. */
|
||||
public static final String NM_RUNC_ALLOW_PRIVILEGED_CONTAINERS =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "privileged-containers.allowed";
|
||||
|
||||
/** Privileged containers are disabled by default. */
|
||||
public static final boolean DEFAULT_NM_RUNC_ALLOW_PRIVILEGED_CONTAINERS =
|
||||
false;
|
||||
|
||||
/** The set of networks allowed when launching containers using the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
public static final String NM_RUNC_ALLOWED_CONTAINER_NETWORKS =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "allowed-container-networks";
|
||||
|
||||
/** The default set of networks allowed when launching containers using the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
public static final String[] DEFAULT_NM_RUNC_ALLOWED_CONTAINER_NETWORKS =
|
||||
{"host", "none", "bridge"};
|
||||
|
||||
/** The set of runtimes allowed when launching containers using the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
public static final String NM_RUNC_ALLOWED_CONTAINER_RUNTIMES =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "allowed-container-runtimes";
|
||||
|
||||
/** The default set of runtimes allowed when launching containers using the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
public static final String[] DEFAULT_NM_RUNC_ALLOWED_CONTAINER_RUNTIMES =
|
||||
{"runc"};
|
||||
|
||||
/** ACL list for users allowed to run privileged containers. */
|
||||
public static final String NM_RUNC_PRIVILEGED_CONTAINERS_ACL =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "privileged-containers.acl";
|
||||
|
||||
/** Default list for users allowed to run privileged containers is empty. */
|
||||
public static final String DEFAULT_NM_RUNC_PRIVILEGED_CONTAINERS_ACL = "";
|
||||
|
||||
/** Allow host pid namespace for containers. Use with care. */
|
||||
public static final String NM_RUNC_ALLOW_HOST_PID_NAMESPACE =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "host-pid-namespace.allowed";
|
||||
|
||||
/** Host pid namespace for containers is disabled by default. */
|
||||
public static final boolean DEFAULT_NM_RUNC_ALLOW_HOST_PID_NAMESPACE =
|
||||
false;
|
||||
|
||||
/** The default list of read-only mounts to be bind-mounted into all
|
||||
* runC containers that use RuncContainerRuntime.
|
||||
*/
|
||||
public static final String NM_RUNC_DEFAULT_RO_MOUNTS =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "default-ro-mounts";
|
||||
|
||||
/** The default list of read-write mounts to be bind-mounted into all
|
||||
* runC containers that use RuncContainerRuntime.
|
||||
*/
|
||||
public static final String NM_RUNC_DEFAULT_RW_MOUNTS =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "default-rw-mounts";
|
||||
|
||||
/** Path to the seccomp profile to use with Runc containers. */
|
||||
public static final String NM_RUNC_SECCOMP_PROFILE =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "seccomp-profile";
|
||||
|
||||
/** Prefix for image tag to manifest hash plugin used with the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
private static final String IMAGE_TAG_TO_MANIFEST_PLUGIN_PREFIX =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "image-tag-to-manifest-plugin.";
|
||||
|
||||
/**
|
||||
* The HDFS location where the runC image tag to hash file exists.
|
||||
*/
|
||||
public static final String NM_HDFS_RUNC_IMAGE_TAG_TO_HASH_FILE =
|
||||
IMAGE_TAG_TO_MANIFEST_PLUGIN_PREFIX + "hdfs-hash-file";
|
||||
|
||||
/**
|
||||
* The local file system location where the runC image tag to hash file exists.
|
||||
*/
|
||||
public static final String NM_LOCAL_RUNC_IMAGE_TAG_TO_HASH_FILE =
|
||||
IMAGE_TAG_TO_MANIFEST_PLUGIN_PREFIX + "local-hash-file";
|
||||
|
||||
/**
|
||||
* The interval in seconds between refreshing the hdfs image tag to
|
||||
* hash cache.
|
||||
*/
|
||||
public static final String NM_RUNC_CACHE_REFRESH_INTERVAL =
|
||||
IMAGE_TAG_TO_MANIFEST_PLUGIN_PREFIX + "cache-refresh-interval-secs";
|
||||
|
||||
/**
|
||||
* The default interval in seconds between refreshing the hdfs image tag to
|
||||
* hash cache.
|
||||
*/
|
||||
public static final int DEFAULT_NM_RUNC_CACHE_REFRESH_INTERVAL = 60;
|
||||
|
||||
/**
|
||||
* The number of manifests to cache in the image tag to hash cache.
|
||||
*/
|
||||
public static final String NM_RUNC_NUM_MANIFESTS_TO_CACHE =
|
||||
IMAGE_TAG_TO_MANIFEST_PLUGIN_PREFIX + "num-manifests-to-cache";
|
||||
|
||||
/**
|
||||
* The default number of manifests to cache in the image tag to hash cache.
|
||||
*/
|
||||
public static final int DEFAULT_NUM_MANIFESTS_TO_CACHE = 10;
|
||||
|
||||
/** Prefix for hdfs manifest hash to local resources plugin used with the
|
||||
* RuncContainerRuntime.
|
||||
*/
|
||||
private static final String HDFS_MANIFEST_TO_RESOURCES_PLUGIN_PREFIX =
|
||||
RUNC_CONTAINER_RUNTIME_PREFIX + "hdfs-manifest-to-resources-plugin.";
|
||||
|
||||
/**
|
||||
* The timeout value in seconds for the values in the stat cache.
|
||||
*/
|
||||
public static final String NM_RUNC_STAT_CACHE_TIMEOUT =
|
||||
HDFS_MANIFEST_TO_RESOURCES_PLUGIN_PREFIX
|
||||
+ "stat-cache-timeout-interval-secs";
|
||||
|
||||
/**
|
||||
* The default timeout value in seconds for the values in the stat cache.
|
||||
*/
|
||||
public static final int DEFAULT_NM_RUNC_STAT_CACHE_TIMEOUT = 60 * 60;
|
||||
|
||||
/**
|
||||
* The size of the stat cache which stores stats of the layers and config.
|
||||
*/
|
||||
public static final String NM_RUNC_STAT_CACHE_SIZE =
|
||||
HDFS_MANIFEST_TO_RESOURCES_PLUGIN_PREFIX + "stat-cache-size";
|
||||
|
||||
/**
|
||||
* The default size of the stat cache which stores stats of the
|
||||
* layers and config.
|
||||
*/
|
||||
public static final int DEFAULT_RUNC_STAT_CACHE_SIZE = 500;
|
||||
|
||||
public static final String DOCKER_CONTAINER_RUNTIME_PREFIX =
|
||||
LINUX_CONTAINER_RUNTIME_PREFIX + "docker.";
|
||||
|
||||
|
|
|
@ -1782,7 +1782,7 @@
|
|||
|
||||
<property>
|
||||
<description>Comma separated list of runtimes that are allowed when using
|
||||
LinuxContainerExecutor. The allowed values are default, docker, and
|
||||
LinuxContainerExecutor. The allowed values are default, docker, runc, and
|
||||
javasandbox.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.allowed-runtimes</name>
|
||||
<value>default</value>
|
||||
|
@ -1924,6 +1924,144 @@
|
|||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The runC image tag to manifest plugin
|
||||
class to be used.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin</name>
|
||||
<value>org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageTagToManifestPlugin</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The runC manifest to resources plugin class to
|
||||
be used.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.manifest-to-resources-plugin</name>
|
||||
<value>org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.HdfsManifestToResourcesPlugin</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The HDFS location under which the oci image manifests, layers,
|
||||
and configs directories exist.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-toplevel-dir</name>
|
||||
<value>/runc-root</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Target count of layer mounts that we should keep on disk
|
||||
at one time.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.layer-mounts-to-keep</name>
|
||||
<value>100</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The interval in seconds between executions of
|
||||
reaping layer mounts.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.layer-mounts-interval-secs</name>
|
||||
<value>600</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Image to be used if no other image is specified.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-name</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Allow or disallow privileged containers.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.privileged-containers.allowed</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The set of networks allowed when launching containers
|
||||
using the RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.allowed-container-networks</name>
|
||||
<value>host,none,bridge</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The set of runtimes allowed when launching containers
|
||||
using the RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.allowed-container-runtimes</name>
|
||||
<value>runc</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>ACL list for users allowed to run privileged
|
||||
containers.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.privileged-containers.acl</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Allow host pid namespace for runC containers.
|
||||
Use with care.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.host-pid-namespace.allowed</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The default list of read-only mounts to be bind-mounted
|
||||
into all runC containers that use RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.default-ro-mounts</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The default list of read-write mounts to be bind-mounted
|
||||
into all runC containers that use RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.default-rw-mounts</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Path to the seccomp profile to use with runC
|
||||
containers</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.seccomp-profile</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The HDFS location where the runC image tag to hash
|
||||
file exists.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.hdfs-hash-file</name>
|
||||
<value>/runc-root/image-tag-to-hash</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The local file system location where the runC image tag
|
||||
to hash file exists.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.local-hash-file</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The interval in seconds between refreshing the hdfs image tag
|
||||
to hash cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.cache-refresh-interval-secs</name>
|
||||
<value>60</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The number of manifests to cache in the image tag
|
||||
to hash cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.num-manifests-to-cache</name>
|
||||
<value>10</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The timeout value in seconds for the values in
|
||||
the stat cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-timeout-interval-secs</name>
|
||||
<value>360</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The size of the stat cache which stores stats of the
|
||||
layers and config.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-size</name>
|
||||
<value>500</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The mode in which the Java Container Sandbox should run detailed by
|
||||
the JavaSandboxLinuxContainerRuntime.</description>
|
||||
|
|
|
@ -48,6 +48,7 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.ConfigurationException;
|
||||
|
@ -127,6 +128,10 @@ public abstract class ContainerExecutor implements Configurable {
|
|||
*/
|
||||
public abstract void init(Context nmContext) throws IOException;
|
||||
|
||||
public void start() {}
|
||||
|
||||
public void stop() {}
|
||||
|
||||
/**
|
||||
* This function localizes the JAR file on-demand.
|
||||
* On Windows the ContainerLaunch creates a temporary special JAR manifest of
|
||||
|
@ -259,6 +264,12 @@ public abstract class ContainerExecutor implements Configurable {
|
|||
public abstract boolean isContainerAlive(ContainerLivenessContext ctx)
|
||||
throws IOException;
|
||||
|
||||
|
||||
public Map<String, LocalResource> getLocalResources(Container container)
|
||||
throws IOException {
|
||||
return container.getLaunchContext().getLocalResources();
|
||||
}
|
||||
|
||||
/**
|
||||
* Update cluster information inside container.
|
||||
*
|
||||
|
|
|
@ -31,6 +31,7 @@ import org.apache.hadoop.util.ReflectionUtils;
|
|||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.ApplicationConstants;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.ConfigurationException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
|
@ -70,6 +71,7 @@ import java.net.InetSocketAddress;
|
|||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.*;
|
||||
|
@ -343,6 +345,18 @@ public class LinuxContainerExecutor extends ContainerExecutor {
|
|||
resourcesHandler.init(this);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
super.start();
|
||||
linuxContainerRuntime.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
super.stop();
|
||||
linuxContainerRuntime.stop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void startLocalizer(LocalizerStartContext ctx)
|
||||
throws IOException, InterruptedException {
|
||||
|
@ -1042,4 +1056,10 @@ public class LinuxContainerExecutor extends ContainerExecutor {
|
|||
throws ContainerExecutionException {
|
||||
return linuxContainerRuntime.getExposedPorts(container);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, LocalResource> getLocalResources(Container container)
|
||||
throws IOException {
|
||||
return linuxContainerRuntime.getLocalResources(container);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -424,7 +424,7 @@ public class NodeManager extends CompositeService
|
|||
exec.init(context);
|
||||
} catch (IOException e) {
|
||||
throw new YarnRuntimeException("Failed to initialize container executor", e);
|
||||
}
|
||||
}
|
||||
DeletionService del = createDeletionService(exec);
|
||||
addService(del);
|
||||
|
||||
|
@ -514,6 +514,7 @@ public class NodeManager extends CompositeService
|
|||
|
||||
registerMXBean();
|
||||
|
||||
context.getContainerExecutor().start();
|
||||
super.serviceInit(conf);
|
||||
// TODO add local dirs to del
|
||||
}
|
||||
|
@ -527,8 +528,10 @@ public class NodeManager extends CompositeService
|
|||
super.serviceStop();
|
||||
DefaultMetricsSystem.shutdown();
|
||||
|
||||
// Cleanup ResourcePluginManager
|
||||
if (null != context) {
|
||||
context.getContainerExecutor().stop();
|
||||
|
||||
// Cleanup ResourcePluginManager
|
||||
ResourcePluginManager rpm = context.getResourcePluginManager();
|
||||
if (rpm != null) {
|
||||
rpm.cleanup();
|
||||
|
|
|
@ -24,6 +24,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|||
import org.apache.hadoop.yarn.server.api.records.ContainerQueuingLimit;
|
||||
import org.apache.hadoop.yarn.server.api.records.OpportunisticContainersStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ContainerManagerEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor
|
||||
.ContainersMonitor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler
|
||||
|
@ -45,4 +46,7 @@ public interface ContainerManager extends ServiceStateChangeListener,
|
|||
ContainerScheduler getContainerScheduler();
|
||||
|
||||
void handleCredentialUpdate();
|
||||
|
||||
ResourceLocalizationService getResourceLocalizationService();
|
||||
|
||||
}
|
||||
|
|
|
@ -2022,4 +2022,7 @@ public class ContainerManagerImpl extends CompositeService implements
|
|||
return container.getLocalizationStatuses();
|
||||
}
|
||||
|
||||
public ResourceLocalizationService getResourceLocalizationService() {
|
||||
return rsrcLocalizationSrvc;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.event.EventHandler;
|
|||
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.server.api.protocolrecords.NMContainerStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -112,6 +113,11 @@ public interface Container extends EventHandler<ContainerEvent> {
|
|||
|
||||
boolean isRecovering();
|
||||
|
||||
void setContainerRuntimeData(Object containerRuntimeData);
|
||||
|
||||
<T> T getContainerRuntimeData(Class<T> runtimeClazz)
|
||||
throws ContainerExecutionException;
|
||||
|
||||
/**
|
||||
* Get assigned resource mappings to the container.
|
||||
*
|
||||
|
|
|
@ -37,6 +37,7 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
|
|||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerSubState;
|
||||
import org.apache.hadoop.yarn.api.records.LocalizationStatus;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.scheduler.UpdateContainerSchedulerEvent;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
@ -180,6 +181,7 @@ public class ContainerImpl implements Container {
|
|||
private volatile ReInitializationContext reInitContext;
|
||||
private volatile boolean isReInitializing = false;
|
||||
private volatile boolean isMarkeForKilling = false;
|
||||
private Object containerRuntimeData;
|
||||
|
||||
/** The NM-wide configuration - not specific to this container */
|
||||
private final Configuration daemonConf;
|
||||
|
@ -1211,25 +1213,27 @@ 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 {
|
||||
Map<String, LocalResource> cntrRsrc;
|
||||
try {
|
||||
cntrRsrc = container.context
|
||||
.getContainerExecutor().getLocalResources(container);
|
||||
if (!cntrRsrc.isEmpty()) {
|
||||
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);
|
||||
container.cleanup();
|
||||
return ContainerState.LOCALIZING;
|
||||
} else {
|
||||
container.sendScheduleEvent();
|
||||
container.metrics.endInitingContainer();
|
||||
return ContainerState.LOCALIZATION_FAILED;
|
||||
return ContainerState.SCHEDULED;
|
||||
}
|
||||
return ContainerState.LOCALIZING;
|
||||
} else {
|
||||
container.sendScheduleEvent();
|
||||
} catch (URISyntaxException | IOException e) {
|
||||
// malformed resource; abort container launch
|
||||
LOG.warn("Failed to parse resource-request", e);
|
||||
container.cleanup();
|
||||
container.metrics.endInitingContainer();
|
||||
return ContainerState.SCHEDULED;
|
||||
return ContainerState.LOCALIZATION_FAILED;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -2283,4 +2287,18 @@ public class ContainerImpl implements Container {
|
|||
this.readLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public void setContainerRuntimeData(Object containerRuntimeData) {
|
||||
this.containerRuntimeData = containerRuntimeData;
|
||||
}
|
||||
|
||||
public <T> T getContainerRuntimeData(Class<T> runtimeClass)
|
||||
throws ContainerExecutionException {
|
||||
if (!runtimeClass.isInstance(containerRuntimeData)) {
|
||||
throw new ContainerExecutionException(
|
||||
"Runtime class " + containerRuntimeData.getClass().getCanonicalName()
|
||||
+ " is invalid. Expected class " + runtimeClass.getCanonicalName());
|
||||
}
|
||||
return runtimeClass.cast(containerRuntimeData);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -35,7 +35,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
|
|||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerEventType;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerExitEvent;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.deletion.task.DockerContainerDeletionTask;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerReapContext;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerSignalContext;
|
||||
import org.slf4j.Logger;
|
||||
|
@ -147,7 +147,7 @@ public class ContainerCleanup implements Runnable {
|
|||
}
|
||||
|
||||
// rm container in docker
|
||||
if (OCIContainerRuntime.isOCICompliantContainerRequested(conf,
|
||||
if (DockerLinuxContainerRuntime.isDockerContainerRequested(conf,
|
||||
container.getLaunchContext().getEnvironment())) {
|
||||
rmDockerContainerDelayed();
|
||||
}
|
||||
|
|
|
@ -59,7 +59,9 @@ public class PrivilegedOperation {
|
|||
ADD_NUMA_PARAMS(""), // no CLI switch supported yet.
|
||||
REMOVE_DOCKER_CONTAINER("--remove-docker-container"),
|
||||
INSPECT_DOCKER_CONTAINER("--inspect-docker-container"),
|
||||
SYNC_YARN_SYSFS("");
|
||||
SYNC_YARN_SYSFS(""),
|
||||
RUN_RUNC_CONTAINER("--run-runc-container"),
|
||||
REAP_RUNC_LAYER_MOUNTS("--reap-runc-layer-mounts");
|
||||
|
||||
private final String option;
|
||||
|
||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.util.ReflectionUtils;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
|
@ -37,6 +38,7 @@ import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerExecContext;
|
|||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
|
@ -60,6 +62,7 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
|
|||
LoggerFactory.getLogger(DelegatingLinuxContainerRuntime.class);
|
||||
private DefaultLinuxContainerRuntime defaultLinuxContainerRuntime;
|
||||
private DockerLinuxContainerRuntime dockerLinuxContainerRuntime;
|
||||
private RuncContainerRuntime runcContainerRuntime;
|
||||
private JavaSandboxLinuxContainerRuntime javaSandboxLinuxContainerRuntime;
|
||||
private Set<String> allowedRuntimes = new HashSet<>();
|
||||
private List<LinuxContainerRuntime> pluggableRuntimes = new ArrayList<>();
|
||||
|
@ -92,6 +95,12 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
|
|||
PrivilegedOperationExecutor.getInstance(conf));
|
||||
dockerLinuxContainerRuntime.initialize(conf, nmContext);
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.RUNC.name())) {
|
||||
runcContainerRuntime = new RuncContainerRuntime(
|
||||
PrivilegedOperationExecutor.getInstance(conf));
|
||||
runcContainerRuntime.initialize(conf, nmContext);
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.DEFAULT.name())) {
|
||||
defaultLinuxContainerRuntime = new DefaultLinuxContainerRuntime(
|
||||
|
@ -116,6 +125,9 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
|
|||
} else if (dockerLinuxContainerRuntime != null &&
|
||||
dockerLinuxContainerRuntime.isRuntimeRequested(environment)) {
|
||||
runtime = dockerLinuxContainerRuntime;
|
||||
} else if (runcContainerRuntime != null &&
|
||||
runcContainerRuntime.isRuntimeRequested(environment)) {
|
||||
runtime = runcContainerRuntime;
|
||||
} else {
|
||||
LinuxContainerRuntime pluggableRuntime = pickPluggableRuntime(
|
||||
environment);
|
||||
|
@ -244,4 +256,59 @@ public class DelegatingLinuxContainerRuntime implements LinuxContainerRuntime {
|
|||
LinuxContainerRuntime runtime = pickContainerRuntime(container);
|
||||
return runtime.execContainer(ctx);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public Map<String, LocalResource> getLocalResources(Container container)
|
||||
throws IOException {
|
||||
try {
|
||||
LinuxContainerRuntime runtime = pickContainerRuntime(container);
|
||||
return runtime.getLocalResources(container);
|
||||
} catch (ContainerExecutionException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX.name())) {
|
||||
javaSandboxLinuxContainerRuntime.start();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.DOCKER.name())) {
|
||||
dockerLinuxContainerRuntime.start();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.RUNC.name())) {
|
||||
runcContainerRuntime.start();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.DEFAULT.name())) {
|
||||
defaultLinuxContainerRuntime.start();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.JAVASANDBOX.name())) {
|
||||
javaSandboxLinuxContainerRuntime.stop();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.DOCKER.name())) {
|
||||
dockerLinuxContainerRuntime.stop();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.RUNC.name())) {
|
||||
runcContainerRuntime.stop();
|
||||
}
|
||||
if (isRuntimeAllowed(
|
||||
LinuxContainerRuntimeConstants.RuntimeType.DEFAULT.name())) {
|
||||
defaultLinuxContainerRuntime.stop();
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
}
|
|
@ -208,6 +208,9 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
|
|||
|
||||
private static final String DEFAULT_PROCFS = "/proc";
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static final String RUNTIME_TYPE = "DOCKER";
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_DOCKER_CONTAINER_IMAGE =
|
||||
"YARN_CONTAINER_RUNTIME_DOCKER_IMAGE";
|
||||
|
@ -240,12 +243,10 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
|
|||
"YARN_CONTAINER_RUNTIME_DOCKER_SERVICE_MODE";
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static final String RUNTIME_TYPE = "DOCKER";
|
||||
@InterfaceAudience.Private
|
||||
private final static String ENV_OCI_CONTAINER_PID_NAMESPACE =
|
||||
public final static String ENV_OCI_CONTAINER_PID_NAMESPACE =
|
||||
formatOciEnvKey(RUNTIME_TYPE, CONTAINER_PID_NAMESPACE_SUFFIX);
|
||||
@InterfaceAudience.Private
|
||||
private final static String ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER =
|
||||
public final static String ENV_OCI_CONTAINER_RUN_PRIVILEGED_CONTAINER =
|
||||
formatOciEnvKey(RUNTIME_TYPE, RUN_PRIVILEGED_CONTAINER_SUFFIX);
|
||||
|
||||
private Configuration conf;
|
||||
|
|
|
@ -1,5 +1,4 @@
|
|||
/*
|
||||
* *
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
|
@ -15,7 +14,6 @@
|
|||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
* /
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
|
||||
|
@ -23,10 +21,13 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime
|
|||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntime;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
|
@ -55,5 +56,14 @@ public interface LinuxContainerRuntime extends ContainerRuntime {
|
|||
* @return whether this runtime is requested
|
||||
*/
|
||||
boolean isRuntimeRequested(Map<String, String> env);
|
||||
|
||||
default void start() {}
|
||||
|
||||
default void stop() {}
|
||||
|
||||
default Map<String, LocalResource> getLocalResources(Container container)
|
||||
throws IOException {
|
||||
return container.getLaunchContext().getLocalResources();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -37,7 +37,8 @@ public final class LinuxContainerRuntimeConstants {
|
|||
public enum RuntimeType {
|
||||
DEFAULT,
|
||||
DOCKER,
|
||||
JAVASANDBOX;
|
||||
JAVASANDBOX,
|
||||
RUNC;
|
||||
}
|
||||
|
||||
public static final Attribute<Map> LOCALIZED_RESOURCES = Attribute
|
||||
|
|
|
@ -53,6 +53,7 @@ import java.util.Set;
|
|||
import java.util.regex.Pattern;
|
||||
|
||||
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime.isDockerContainerRequested;
|
||||
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.RuncContainerRuntime.isRuncContainerRequested;
|
||||
|
||||
/**
|
||||
* <p>This class is a {@link ContainerRuntime} implementation that uses the
|
||||
|
@ -117,7 +118,8 @@ public abstract class OCIContainerRuntime implements LinuxContainerRuntime {
|
|||
|
||||
public static boolean isOCICompliantContainerRequested(
|
||||
Configuration daemonConf, Map<String, String> env) {
|
||||
return isDockerContainerRequested(daemonConf, env);
|
||||
return isDockerContainerRequested(daemonConf, env) ||
|
||||
isRuncContainerRequested(daemonConf, env);
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
|
|
|
@ -0,0 +1,906 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
* /
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair;
|
||||
import org.apache.hadoop.security.authorize.AccessControlList;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationExecutor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.CGroupsHandler;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.resources.ResourceHandlerModule;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageManifest;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig.OCILayer;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig.OCIRuntimeConfig;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig.OCIRuntimeConfig.OCILinuxConfig;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig.OCIRuntimeConfig.OCIMount;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncContainerExecutorConfig.OCIRuntimeConfig.OCIProcessConfig;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncImageTagToManifestPlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.RuncManifestToResourcesPlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalResourceRequest;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeContext;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.volume.csi.ContainerVolumePublisher;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.executor.ContainerExecContext;
|
||||
import org.codehaus.jackson.JsonNode;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.net.URISyntaxException;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.nio.file.Files;
|
||||
import java.nio.file.Paths;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.regex.Matcher;
|
||||
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_LAYER_MOUNTS_TO_KEEP;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_LAYER_MOUNTS_TO_KEEP;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL;
|
||||
import static org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.LinuxContainerRuntimeConstants.*;
|
||||
/**
|
||||
* <p>This class is an extension of {@link OCIContainerRuntime} that uses the
|
||||
* native {@code container-executor} binary via a
|
||||
* {@link PrivilegedOperationExecutor} instance to launch processes inside
|
||||
* Runc containers.</p>
|
||||
*
|
||||
* <p>The following environment variables are used to configure the Runc
|
||||
* engine:</p>
|
||||
*
|
||||
* <ul>
|
||||
* <li>
|
||||
* {@code YARN_CONTAINER_RUNTIME_TYPE} ultimately determines whether a
|
||||
* runC container will be used. If the value is {@code runc}, a runC
|
||||
* container will be used. Otherwise a regular process tree container will
|
||||
* be used. This environment variable is checked by the
|
||||
* {@link #isRuncContainerRequested} method, which is called by the
|
||||
* {@link DelegatingLinuxContainerRuntime}.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@code YARN_CONTAINER_RUNTIME_RUNC_IMAGE} names which image
|
||||
* will be used to launch the Runc container.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@code YARN_CONTAINER_RUNTIME_RUNC_MOUNTS} allows users to specify
|
||||
* additional volume mounts for the runC container. The value of the
|
||||
* environment variable should be a comma-separated list of mounts.
|
||||
* All such mounts must be given as {@code source:dest[:mode]} and the mode
|
||||
* must be "ro" (read-only) or "rw" (read-write) to specify the type of
|
||||
* access being requested. If neither is specified, read-write will be
|
||||
* assumed. The requested mounts will be validated by
|
||||
* container-executor based on the values set in container-executor.cfg for
|
||||
* {@code runc.allowed.ro-mounts} and {@code runc.allowed.rw-mounts}.
|
||||
* </li>
|
||||
* <li>
|
||||
* {@code YARN_CONTAINER_RUNTIME_RUNC_CONTAINER_HOSTNAME} sets the
|
||||
* hostname to be used by the Runc container. If not specified, a
|
||||
* hostname will be derived from the container ID and set as default
|
||||
* hostname for networks other than 'host'.
|
||||
* </li>
|
||||
* </ul>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class RuncContainerRuntime extends OCIContainerRuntime {
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(
|
||||
RuncContainerRuntime.class);
|
||||
|
||||
@InterfaceAudience.Private
|
||||
private static final String RUNTIME_TYPE = "RUNC";
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_RUNC_CONTAINER_IMAGE =
|
||||
"YARN_CONTAINER_RUNTIME_RUNC_IMAGE";
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_RUNC_CONTAINER_MOUNTS =
|
||||
"YARN_CONTAINER_RUNTIME_RUNC_MOUNTS";
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_RUNC_CONTAINER_HOSTNAME =
|
||||
"YARN_CONTAINER_RUNTIME_RUNC_CONTAINER_HOSTNAME";
|
||||
|
||||
@InterfaceAudience.Private
|
||||
public final static String ENV_RUNC_CONTAINER_PID_NAMESPACE =
|
||||
formatOciEnvKey(RUNTIME_TYPE, CONTAINER_PID_NAMESPACE_SUFFIX);
|
||||
@InterfaceAudience.Private
|
||||
public final static String ENV_RUNC_CONTAINER_RUN_PRIVILEGED_CONTAINER =
|
||||
formatOciEnvKey(RUNTIME_TYPE, RUN_PRIVILEGED_CONTAINER_SUFFIX);
|
||||
|
||||
private Configuration conf;
|
||||
private Context nmContext;
|
||||
private PrivilegedOperationExecutor privilegedOperationExecutor;
|
||||
private CGroupsHandler cGroupsHandler;
|
||||
private RuncImageTagToManifestPlugin imageTagToManifestPlugin;
|
||||
private RuncManifestToResourcesPlugin manifestToResourcesPlugin;
|
||||
private ObjectMapper mapper;
|
||||
private String seccomp;
|
||||
private int layersToKeep;
|
||||
private String defaultRuncImage;
|
||||
private ScheduledExecutorService exec;
|
||||
private String seccompProfile;
|
||||
private Set<String> defaultROMounts = new HashSet<>();
|
||||
private Set<String> defaultRWMounts = new HashSet<>();
|
||||
private Set<String> allowedNetworks = new HashSet<>();
|
||||
private Set<String> allowedRuntimes = new HashSet<>();
|
||||
private AccessControlList privilegedContainersAcl;
|
||||
|
||||
public RuncContainerRuntime(PrivilegedOperationExecutor
|
||||
privilegedOperationExecutor) {
|
||||
this(privilegedOperationExecutor, ResourceHandlerModule
|
||||
.getCGroupsHandler());
|
||||
}
|
||||
|
||||
//A constructor with an injected cGroupsHandler primarily used for testing.
|
||||
@VisibleForTesting
|
||||
public RuncContainerRuntime(PrivilegedOperationExecutor
|
||||
privilegedOperationExecutor, CGroupsHandler cGroupsHandler) {
|
||||
super(privilegedOperationExecutor, cGroupsHandler);
|
||||
this.privilegedOperationExecutor = privilegedOperationExecutor;
|
||||
|
||||
if (cGroupsHandler == null) {
|
||||
LOG.info("cGroupsHandler is null - cgroups not in use.");
|
||||
} else {
|
||||
this.cGroupsHandler = cGroupsHandler;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(Configuration configuration, Context nmCtx)
|
||||
throws ContainerExecutionException {
|
||||
super.initialize(configuration, nmCtx);
|
||||
this.conf = configuration;
|
||||
this.nmContext = nmCtx;
|
||||
imageTagToManifestPlugin = chooseImageTagToManifestPlugin();
|
||||
imageTagToManifestPlugin.init(conf);
|
||||
manifestToResourcesPlugin = chooseManifestToResourcesPlugin();
|
||||
manifestToResourcesPlugin.init(conf);
|
||||
mapper = new ObjectMapper();
|
||||
defaultRuncImage = conf.get(YarnConfiguration.NM_RUNC_IMAGE_NAME);
|
||||
|
||||
allowedNetworks.clear();
|
||||
allowedRuntimes.clear();
|
||||
|
||||
allowedNetworks.addAll(Arrays.asList(
|
||||
conf.getTrimmedStrings(
|
||||
YarnConfiguration.NM_RUNC_ALLOWED_CONTAINER_NETWORKS,
|
||||
YarnConfiguration.DEFAULT_NM_RUNC_ALLOWED_CONTAINER_NETWORKS)));
|
||||
|
||||
allowedRuntimes.addAll(Arrays.asList(
|
||||
conf.getTrimmedStrings(
|
||||
YarnConfiguration.NM_RUNC_ALLOWED_CONTAINER_RUNTIMES,
|
||||
YarnConfiguration.DEFAULT_NM_RUNC_ALLOWED_CONTAINER_RUNTIMES)));
|
||||
|
||||
privilegedContainersAcl = new AccessControlList(conf.getTrimmed(
|
||||
YarnConfiguration.NM_RUNC_PRIVILEGED_CONTAINERS_ACL,
|
||||
YarnConfiguration.DEFAULT_NM_RUNC_PRIVILEGED_CONTAINERS_ACL));
|
||||
|
||||
seccompProfile = conf.get(YarnConfiguration.NM_RUNC_SECCOMP_PROFILE);
|
||||
|
||||
defaultROMounts.addAll(Arrays.asList(
|
||||
conf.getTrimmedStrings(
|
||||
YarnConfiguration.NM_RUNC_DEFAULT_RO_MOUNTS)));
|
||||
|
||||
defaultRWMounts.addAll(Arrays.asList(
|
||||
conf.getTrimmedStrings(
|
||||
YarnConfiguration.NM_RUNC_DEFAULT_RW_MOUNTS)));
|
||||
|
||||
try {
|
||||
//TODO Remove whitespace in seccomp that gets output to config.json
|
||||
if (seccompProfile != null) {
|
||||
seccomp = new String(Files.readAllBytes(Paths.get(seccompProfile)),
|
||||
StandardCharsets.UTF_8);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new ContainerExecutionException(ioe);
|
||||
}
|
||||
|
||||
layersToKeep = conf.getInt(NM_RUNC_LAYER_MOUNTS_TO_KEEP,
|
||||
DEFAULT_NM_RUNC_LAYER_MOUNTS_TO_KEEP);
|
||||
|
||||
}
|
||||
|
||||
@Override
|
||||
public void start() {
|
||||
int reapRuncLayerMountsInterval =
|
||||
conf.getInt(NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL,
|
||||
DEFAULT_NM_REAP_RUNC_LAYER_MOUNTS_INTERVAL);
|
||||
exec = HadoopExecutors.newScheduledThreadPool(1);
|
||||
exec.scheduleAtFixedRate(
|
||||
new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
PrivilegedOperation launchOp = new PrivilegedOperation(
|
||||
PrivilegedOperation.OperationType.REAP_RUNC_LAYER_MOUNTS);
|
||||
launchOp.appendArgs(Integer.toString(layersToKeep));
|
||||
try {
|
||||
String stdout = privilegedOperationExecutor
|
||||
.executePrivilegedOperation(null,
|
||||
launchOp, null, null, false, false);
|
||||
if(stdout != null) {
|
||||
LOG.info("Reap layer mounts thread: " + stdout);
|
||||
}
|
||||
} catch (PrivilegedOperationException e) {
|
||||
LOG.warn("Failed to reap old runc layer mounts", e);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
LOG.warn("Reap layer mount thread caught an exception: ", e);
|
||||
}
|
||||
}
|
||||
}, 0, reapRuncLayerMountsInterval, TimeUnit.SECONDS);
|
||||
imageTagToManifestPlugin.start();
|
||||
manifestToResourcesPlugin.start();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void stop() {
|
||||
exec.shutdownNow();
|
||||
imageTagToManifestPlugin.stop();
|
||||
manifestToResourcesPlugin.stop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void launchContainer(ContainerRuntimeContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
List<String> env = new ArrayList<>();
|
||||
Container container = ctx.getContainer();
|
||||
String runAsUser = ctx.getExecutionAttribute(RUN_AS_USER);
|
||||
String user = ctx.getExecutionAttribute(USER);
|
||||
ContainerId containerId = container.getContainerId();
|
||||
ApplicationId appId = containerId.getApplicationAttemptId()
|
||||
.getApplicationId();
|
||||
|
||||
Map<String, String> environment = container.getLaunchContext()
|
||||
.getEnvironment();
|
||||
ArrayList<OCIMount> mounts = new ArrayList<>();
|
||||
ArrayList<OCILayer> layers = new ArrayList<>();
|
||||
String hostname = environment.get(ENV_RUNC_CONTAINER_HOSTNAME);
|
||||
|
||||
validateHostname(hostname);
|
||||
|
||||
String containerIdStr = containerId.toString();
|
||||
String applicationId = appId.toString();
|
||||
Path containerWorkDir = ctx.getExecutionAttribute(CONTAINER_WORK_DIR);
|
||||
|
||||
RuncRuntimeObject runcRuntimeObject =
|
||||
container.getContainerRuntimeData(RuncRuntimeObject.class);
|
||||
List<LocalResource> layerResources = runcRuntimeObject.getOCILayers();
|
||||
|
||||
ResourceLocalizationService localizationService =
|
||||
nmContext.getContainerManager().getResourceLocalizationService();
|
||||
|
||||
List<String> args = new ArrayList<>();
|
||||
|
||||
try {
|
||||
try {
|
||||
LocalResource rsrc = runcRuntimeObject.getConfig();
|
||||
LocalResourceRequest req = new LocalResourceRequest(rsrc);
|
||||
LocalizedResource localRsrc = localizationService
|
||||
.getLocalizedResource(req, user, appId);
|
||||
if (localRsrc == null) {
|
||||
throw new ContainerExecutionException("Could not successfully " +
|
||||
"localize layers. rsrc: " + rsrc.getResource().getFile());
|
||||
}
|
||||
|
||||
File file = new File(localRsrc.getLocalPath().toString());
|
||||
List<String> imageEnv = extractImageEnv(file);
|
||||
if (imageEnv != null && !imageEnv.isEmpty()) {
|
||||
env.addAll(imageEnv);
|
||||
}
|
||||
List<String> entrypoint = extractImageEntrypoint(file);
|
||||
if (entrypoint != null && !entrypoint.isEmpty()) {
|
||||
args.addAll(entrypoint);
|
||||
}
|
||||
} catch (IOException ioe) {
|
||||
throw new ContainerExecutionException(ioe);
|
||||
}
|
||||
|
||||
for (LocalResource rsrc : layerResources) {
|
||||
LocalResourceRequest req = new LocalResourceRequest(rsrc);
|
||||
LocalizedResource localRsrc = localizationService
|
||||
.getLocalizedResource(req, user, appId);
|
||||
|
||||
OCILayer layer = new OCILayer("application/vnd.squashfs",
|
||||
localRsrc.getLocalPath().toString());
|
||||
layers.add(layer);
|
||||
}
|
||||
} catch (URISyntaxException e) {
|
||||
throw new ContainerExecutionException(e);
|
||||
}
|
||||
|
||||
setContainerMounts(mounts, ctx, containerWorkDir, environment);
|
||||
|
||||
String resourcesOpts = ctx.getExecutionAttribute(RESOURCES_OPTIONS);
|
||||
|
||||
Path nmPrivateContainerScriptPath = ctx.getExecutionAttribute(
|
||||
NM_PRIVATE_CONTAINER_SCRIPT_PATH);
|
||||
|
||||
Path nmPrivateTokensPath =
|
||||
ctx.getExecutionAttribute(NM_PRIVATE_TOKENS_PATH);
|
||||
|
||||
int cpuShares = container.getResource().getVirtualCores();
|
||||
|
||||
// Zero sets to default of 1024. 2 is the minimum value otherwise
|
||||
if (cpuShares < 2) {
|
||||
cpuShares = 2;
|
||||
}
|
||||
|
||||
Path launchDst =
|
||||
new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
|
||||
|
||||
args.add("bash");
|
||||
args.add(launchDst.toUri().getPath());
|
||||
|
||||
String cgroupPath = getCgroupPath(resourcesOpts, "runc-" + containerIdStr);
|
||||
|
||||
String pidFile = ctx.getExecutionAttribute(PID_FILE_PATH).toString();
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> localDirs = ctx.getExecutionAttribute(LOCAL_DIRS);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> logDirs = ctx.getExecutionAttribute(LOG_DIRS);
|
||||
|
||||
Path keystorePath = ctx.getExecutionAttribute(NM_PRIVATE_KEYSTORE_PATH);
|
||||
Path truststorePath = ctx.getExecutionAttribute(NM_PRIVATE_TRUSTSTORE_PATH);
|
||||
|
||||
int https = 0;
|
||||
String keystore = null;
|
||||
String truststore = null;
|
||||
|
||||
if (keystorePath != null && truststorePath != null) {
|
||||
https = 1;
|
||||
keystore = keystorePath.toUri().getPath();
|
||||
truststore = truststorePath.toUri().getPath();
|
||||
}
|
||||
|
||||
OCIProcessConfig processConfig = createOCIProcessConfig(
|
||||
containerWorkDir.toString(), env, args);
|
||||
OCILinuxConfig linuxConfig = createOCILinuxConfig(cpuShares,
|
||||
cgroupPath, seccomp);
|
||||
|
||||
OCIRuntimeConfig ociRuntimeConfig = new OCIRuntimeConfig(null, mounts,
|
||||
processConfig, hostname, null, null, linuxConfig);
|
||||
|
||||
RuncContainerExecutorConfig runcContainerExecutorConfig =
|
||||
createRuncContainerExecutorConfig(runAsUser, user, containerIdStr,
|
||||
applicationId, pidFile, nmPrivateContainerScriptPath.toString(),
|
||||
nmPrivateTokensPath.toString(), https, keystore, truststore,
|
||||
localDirs, logDirs, layers,
|
||||
ociRuntimeConfig);
|
||||
|
||||
String commandFile = writeCommandToFile(
|
||||
runcContainerExecutorConfig, container);
|
||||
PrivilegedOperation launchOp = new PrivilegedOperation(
|
||||
PrivilegedOperation.OperationType.RUN_RUNC_CONTAINER);
|
||||
|
||||
launchOp.appendArgs(commandFile);
|
||||
|
||||
try {
|
||||
privilegedOperationExecutor.executePrivilegedOperation(null,
|
||||
launchOp, null, null, false, false);
|
||||
} catch (PrivilegedOperationException e) {
|
||||
LOG.info("Launch container failed: ", e);
|
||||
try {
|
||||
LOG.debug("config.json used: " +
|
||||
mapper.writeValueAsString(runcContainerExecutorConfig));
|
||||
} catch (IOException ioe) {
|
||||
LOG.info("Json Generation Exception", ioe);
|
||||
}
|
||||
|
||||
throw new ContainerExecutionException("Launch container failed", e
|
||||
.getExitCode(), e.getOutput(), e.getErrorOutput());
|
||||
}
|
||||
}
|
||||
|
||||
private String getCgroupPath(String resourcesOptions, String containerIdStr) {
|
||||
if (cGroupsHandler == null) {
|
||||
LOG.debug("cGroupsHandler is null. cgroups are not in use. nothing to"
|
||||
+ " do.");
|
||||
return null;
|
||||
}
|
||||
|
||||
if (resourcesOptions.equals(
|
||||
(PrivilegedOperation.CGROUP_ARG_PREFIX + PrivilegedOperation
|
||||
.CGROUP_ARG_NO_TASKS))) {
|
||||
LOG.debug("no resource restrictions specified. not using runc's "
|
||||
+ "cgroup options");
|
||||
} else {
|
||||
LOG.debug("using runc's cgroups options");
|
||||
|
||||
String cGroupPath = "/" + cGroupsHandler.getRelativePathForCGroup(
|
||||
containerIdStr);
|
||||
|
||||
LOG.debug("using cgroup parent: " + cGroupPath);
|
||||
|
||||
return cGroupPath;
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
private void addUserMounts(List<OCIMount> mounts,
|
||||
Map<String, String> environment,
|
||||
Map<Path, List<String>> localizedResources)
|
||||
throws ContainerExecutionException {
|
||||
if (environment.containsKey(ENV_RUNC_CONTAINER_MOUNTS)) {
|
||||
Matcher parsedMounts = USER_MOUNT_PATTERN.matcher(
|
||||
environment.get(ENV_RUNC_CONTAINER_MOUNTS));
|
||||
if (!parsedMounts.find()) {
|
||||
throw new ContainerExecutionException(
|
||||
"Unable to parse user supplied mount list: "
|
||||
+ environment.get(ENV_RUNC_CONTAINER_MOUNTS));
|
||||
}
|
||||
parsedMounts.reset();
|
||||
long mountCount = 0;
|
||||
while (parsedMounts.find()) {
|
||||
mountCount++;
|
||||
String src = parsedMounts.group(1);
|
||||
java.nio.file.Path srcPath = java.nio.file.Paths.get(src);
|
||||
if (!srcPath.isAbsolute()) {
|
||||
src = mountReadOnlyPath(src, localizedResources);
|
||||
}
|
||||
String dst = parsedMounts.group(2);
|
||||
String mode = parsedMounts.group(4);
|
||||
boolean isReadWrite;
|
||||
if (mode == null) {
|
||||
isReadWrite = true;
|
||||
} else if (mode.equals("rw")) {
|
||||
isReadWrite = true;
|
||||
} else if (mode.equals("ro")) {
|
||||
isReadWrite = false;
|
||||
} else {
|
||||
throw new ContainerExecutionException(
|
||||
"Unable to parse mode of some mounts in user supplied "
|
||||
+ "mount list: "
|
||||
+ environment.get(ENV_RUNC_CONTAINER_MOUNTS));
|
||||
}
|
||||
addRuncMountLocation(mounts, src, dst, false, isReadWrite);
|
||||
}
|
||||
long commaCount = environment.get(ENV_RUNC_CONTAINER_MOUNTS).chars()
|
||||
.filter(c -> c == ',').count();
|
||||
if (mountCount != commaCount + 1) {
|
||||
// this means the matcher skipped an improperly formatted mount
|
||||
throw new ContainerExecutionException(
|
||||
"Unable to parse some mounts in user supplied mount list: "
|
||||
+ environment.get(ENV_RUNC_CONTAINER_MOUNTS));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addDefaultMountLocation(List<OCIMount> mounts,
|
||||
Set<String> defaultMounts, boolean createSource, boolean isReadWrite)
|
||||
throws ContainerExecutionException {
|
||||
if(defaultMounts != null && !defaultMounts.isEmpty()) {
|
||||
for (String mount : defaultMounts) {
|
||||
String[] dir = StringUtils.split(mount, ':');
|
||||
if (dir.length != 2) {
|
||||
throw new ContainerExecutionException("Invalid mount : " +
|
||||
mount);
|
||||
}
|
||||
String src = dir[0];
|
||||
String dst = dir[1];
|
||||
addRuncMountLocation(mounts, src, dst, createSource, isReadWrite);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void addRuncMountLocation(List<OCIMount> mounts, String srcPath,
|
||||
String dstPath, boolean createSource, boolean isReadWrite) {
|
||||
if (!createSource) {
|
||||
boolean sourceExists = new File(srcPath).exists();
|
||||
if (!sourceExists) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
|
||||
ArrayList<String> options = new ArrayList<>();
|
||||
if (isReadWrite) {
|
||||
options.add("rw");
|
||||
} else {
|
||||
options.add("ro");
|
||||
}
|
||||
options.add("rbind");
|
||||
options.add("rprivate");
|
||||
mounts.add(new OCIMount(dstPath, "bind", srcPath, options));
|
||||
}
|
||||
|
||||
private void addAllRuncMountLocations(List<OCIMount> mounts,
|
||||
List<String> paths, boolean createSource, boolean isReadWrite) {
|
||||
for (String dir: paths) {
|
||||
this.addRuncMountLocation(mounts, dir, dir, createSource, isReadWrite);
|
||||
}
|
||||
}
|
||||
|
||||
public Map<String, LocalResource> getLocalResources(
|
||||
Container container) throws IOException {
|
||||
Map<String, LocalResource> containerLocalRsrc =
|
||||
container.getLaunchContext().getLocalResources();
|
||||
long layerCount = 0;
|
||||
Map<String, String> environment =
|
||||
container.getLaunchContext().getEnvironment();
|
||||
String imageName = environment.get(ENV_RUNC_CONTAINER_IMAGE);
|
||||
if (imageName == null || imageName.isEmpty()) {
|
||||
environment.put(ENV_RUNC_CONTAINER_IMAGE,
|
||||
defaultRuncImage);
|
||||
imageName = defaultRuncImage;
|
||||
}
|
||||
|
||||
ImageManifest manifest =
|
||||
imageTagToManifestPlugin.getManifestFromImageTag(imageName);
|
||||
LocalResource config =
|
||||
manifestToResourcesPlugin.getConfigResource(manifest);
|
||||
List<LocalResource> layers =
|
||||
manifestToResourcesPlugin.getLayerResources(manifest);
|
||||
|
||||
RuncRuntimeObject runcRuntimeObject =
|
||||
new RuncRuntimeObject(config, layers);
|
||||
container.setContainerRuntimeData(runcRuntimeObject);
|
||||
|
||||
for (LocalResource localRsrc : layers) {
|
||||
while(containerLocalRsrc.putIfAbsent("runc-layer" +
|
||||
Long.toString(layerCount), localRsrc) != null) {
|
||||
layerCount++;
|
||||
}
|
||||
}
|
||||
|
||||
while(containerLocalRsrc.putIfAbsent("runc-config" +
|
||||
Long.toString(layerCount), config) != null) {
|
||||
layerCount++;
|
||||
}
|
||||
|
||||
return containerLocalRsrc;
|
||||
}
|
||||
|
||||
protected RuncImageTagToManifestPlugin chooseImageTagToManifestPlugin()
|
||||
throws ContainerExecutionException {
|
||||
String pluginName =
|
||||
conf.get(NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN,
|
||||
DEFAULT_NM_RUNC_IMAGE_TAG_TO_MANIFEST_PLUGIN);
|
||||
RuncImageTagToManifestPlugin runcImageTagToManifestPlugin;
|
||||
try {
|
||||
Class<?> clazz = Class.forName(pluginName);
|
||||
runcImageTagToManifestPlugin =
|
||||
(RuncImageTagToManifestPlugin) clazz.newInstance();
|
||||
} catch (Exception e) {
|
||||
throw new ContainerExecutionException(e);
|
||||
}
|
||||
return runcImageTagToManifestPlugin;
|
||||
}
|
||||
|
||||
protected RuncManifestToResourcesPlugin chooseManifestToResourcesPlugin()
|
||||
throws ContainerExecutionException {
|
||||
String pluginName =
|
||||
conf.get(NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN,
|
||||
DEFAULT_NM_RUNC_MANIFEST_TO_RESOURCES_PLUGIN);
|
||||
LOG.info("pluginName = " + pluginName);
|
||||
RuncManifestToResourcesPlugin runcManifestToResourcesPlugin;
|
||||
try {
|
||||
Class<?> clazz = Class.forName(pluginName);
|
||||
runcManifestToResourcesPlugin =
|
||||
(RuncManifestToResourcesPlugin) clazz.newInstance();
|
||||
} catch (Exception e) {
|
||||
throw new ContainerExecutionException(e);
|
||||
}
|
||||
return runcManifestToResourcesPlugin;
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected List<String> extractImageEnv(File config) throws IOException {
|
||||
JsonNode node = mapper.readTree(config);
|
||||
JsonNode envNode = node.path("config").path("Env");
|
||||
if (envNode.isMissingNode()) {
|
||||
return null;
|
||||
}
|
||||
return mapper.readValue(envNode, List.class);
|
||||
}
|
||||
|
||||
@SuppressWarnings("unchecked")
|
||||
protected List<String> extractImageEntrypoint(File config)
|
||||
throws IOException {
|
||||
JsonNode node = mapper.readTree(config);
|
||||
JsonNode entrypointNode = node.path("config").path("Entrypoint");
|
||||
if (entrypointNode.isMissingNode()) {
|
||||
return null;
|
||||
}
|
||||
return mapper.readValue(entrypointNode, List.class);
|
||||
}
|
||||
|
||||
private RuncContainerExecutorConfig createRuncContainerExecutorConfig(
|
||||
String runAsUser, String username, String containerId,
|
||||
String applicationId, String pidFile,
|
||||
String containerScriptPath, String containerCredentialsPath,
|
||||
int https, String keystorePath, String truststorePath,
|
||||
List<String> localDirs, List<String> logDirs,
|
||||
List<OCILayer> layers, OCIRuntimeConfig ociRuntimeConfig) {
|
||||
|
||||
return new RuncContainerExecutorConfig(runAsUser, username, containerId,
|
||||
applicationId, pidFile, containerScriptPath, containerCredentialsPath,
|
||||
https, keystorePath, truststorePath,
|
||||
localDirs, logDirs, layers, layersToKeep, ociRuntimeConfig);
|
||||
}
|
||||
|
||||
private OCIProcessConfig createOCIProcessConfig(String cwd,
|
||||
List<String> env, List<String> args) {
|
||||
return new OCIProcessConfig(false, null, cwd, env,
|
||||
args, null, null, null, false, 0, null, null);
|
||||
}
|
||||
|
||||
private OCILinuxConfig createOCILinuxConfig(long cpuShares,
|
||||
String cgroupsPath, String seccompProf) {
|
||||
OCILinuxConfig.Resources.CPU cgroupCPU =
|
||||
new OCILinuxConfig.Resources.CPU(cpuShares, 0, 0, 0, 0,
|
||||
null, null);
|
||||
OCILinuxConfig.Resources cgroupResources =
|
||||
new OCILinuxConfig.Resources(null, null, cgroupCPU, null, null, null,
|
||||
null, null);
|
||||
|
||||
return new OCILinuxConfig(null, null, null, null,
|
||||
cgroupsPath, cgroupResources, null, null, seccompProf, null, null,
|
||||
null, null);
|
||||
}
|
||||
|
||||
private void setContainerMounts(ArrayList<OCIMount> mounts,
|
||||
ContainerRuntimeContext ctx, Path containerWorkDir,
|
||||
Map<String, String> environment)
|
||||
throws ContainerExecutionException {
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> filecacheDirs = ctx.getExecutionAttribute(FILECACHE_DIRS);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> containerLogDirs = ctx.getExecutionAttribute(
|
||||
CONTAINER_LOG_DIRS);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> userFilecacheDirs =
|
||||
ctx.getExecutionAttribute(USER_FILECACHE_DIRS);
|
||||
@SuppressWarnings("unchecked")
|
||||
List<String> applicationLocalDirs =
|
||||
ctx.getExecutionAttribute(APPLICATION_LOCAL_DIRS);
|
||||
@SuppressWarnings("unchecked")
|
||||
Map<Path, List<String>> localizedResources = ctx.getExecutionAttribute(
|
||||
LOCALIZED_RESOURCES);
|
||||
|
||||
addRuncMountLocation(mounts, containerWorkDir.toString() +
|
||||
"/private_slash_tmp", "/tmp", true, true);
|
||||
addRuncMountLocation(mounts, containerWorkDir.toString() +
|
||||
"/private_var_slash_tmp", "/var/tmp", true, true);
|
||||
|
||||
addAllRuncMountLocations(mounts, containerLogDirs, true, true);
|
||||
addAllRuncMountLocations(mounts, applicationLocalDirs, true, true);
|
||||
addAllRuncMountLocations(mounts, filecacheDirs, false, false);
|
||||
addAllRuncMountLocations(mounts, userFilecacheDirs, false, false);
|
||||
addDefaultMountLocation(mounts, defaultROMounts, false, false);
|
||||
addDefaultMountLocation(mounts, defaultRWMounts, false, true);
|
||||
addUserMounts(mounts, environment, localizedResources);
|
||||
}
|
||||
|
||||
public String writeCommandToFile(
|
||||
RuncContainerExecutorConfig runcContainerExecutorConfig,
|
||||
Container container)
|
||||
throws ContainerExecutionException {
|
||||
ContainerId containerId = container.getContainerId();
|
||||
String filePrefix = containerId.toString();
|
||||
ApplicationId appId = containerId.getApplicationAttemptId()
|
||||
.getApplicationId();
|
||||
File commandFile;
|
||||
try {
|
||||
File cmdDir = null;
|
||||
|
||||
if(nmContext != null && nmContext.getLocalDirsHandler() != null) {
|
||||
String cmdDirStr = nmContext.getLocalDirsHandler().getLocalPathForWrite(
|
||||
ResourceLocalizationService.NM_PRIVATE_DIR + Path.SEPARATOR +
|
||||
appId + Path.SEPARATOR + filePrefix + Path.SEPARATOR).toString();
|
||||
cmdDir = new File(cmdDirStr);
|
||||
if (!cmdDir.mkdirs() && !cmdDir.exists()) {
|
||||
throw new IOException("Cannot create container private directory "
|
||||
+ cmdDir);
|
||||
}
|
||||
}
|
||||
commandFile = new File(cmdDir + "/runc-config.json");
|
||||
|
||||
try {
|
||||
mapper.writeValue(commandFile, runcContainerExecutorConfig);
|
||||
} catch (IOException ioe) {
|
||||
throw new ContainerExecutionException(ioe);
|
||||
}
|
||||
|
||||
return commandFile.getAbsolutePath();
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Unable to write runc config.json to temporary file!");
|
||||
throw new ContainerExecutionException(e);
|
||||
}
|
||||
}
|
||||
|
||||
public String getExposedPorts(Container container) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public String[] getIpAndHost(Container container) {
|
||||
return null;
|
||||
}
|
||||
|
||||
public IOStreamPair execContainer(ContainerExecContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
return null;
|
||||
}
|
||||
|
||||
public void reapContainer(ContainerRuntimeContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
}
|
||||
|
||||
public void relaunchContainer(ContainerRuntimeContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
}
|
||||
|
||||
/**
|
||||
* Return whether the given environment variables indicate that the operation
|
||||
* is requesting a Runc container. If the environment contains a key
|
||||
* called {@code YARN_CONTAINER_RUNTIME_TYPE} whose value is {@code runc},
|
||||
* this method will return true. Otherwise it will return false.
|
||||
*
|
||||
* @param daemonConf the NodeManager daemon configuration
|
||||
* @param env the environment variable settings for the operation
|
||||
* @return whether a Runc container is requested
|
||||
*/
|
||||
public static boolean isRuncContainerRequested(Configuration daemonConf,
|
||||
Map<String, String> env) {
|
||||
String type = (env == null)
|
||||
? null : env.get(ContainerRuntimeConstants.ENV_CONTAINER_TYPE);
|
||||
if (type == null) {
|
||||
type = daemonConf.get(YarnConfiguration.LINUX_CONTAINER_RUNTIME_TYPE);
|
||||
}
|
||||
return type != null && type.equals(
|
||||
ContainerRuntimeConstants.CONTAINER_RUNTIME_RUNC);
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public boolean isRuntimeRequested(Map<String, String> env) {
|
||||
return isRuncContainerRequested(conf, env);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void signalContainer(ContainerRuntimeContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
ContainerExecutor.Signal signal = ctx.getExecutionAttribute(SIGNAL);
|
||||
Container container = ctx.getContainer();
|
||||
|
||||
if (signal == ContainerExecutor.Signal.KILL ||
|
||||
signal == ContainerExecutor.Signal.TERM) {
|
||||
ContainerVolumePublisher publisher = new ContainerVolumePublisher(
|
||||
container, container.getCsiVolumesRootDir(), this);
|
||||
try {
|
||||
publisher.unpublishVolumes();
|
||||
} catch (YarnException | IOException e) {
|
||||
throw new ContainerExecutionException(e);
|
||||
}
|
||||
}
|
||||
|
||||
PrivilegedOperation signalOp = new PrivilegedOperation(
|
||||
PrivilegedOperation.OperationType.SIGNAL_CONTAINER);
|
||||
|
||||
signalOp.appendArgs(ctx.getExecutionAttribute(RUN_AS_USER),
|
||||
ctx.getExecutionAttribute(USER),
|
||||
Integer.toString(PrivilegedOperation.RunAsUserCommand
|
||||
.SIGNAL_CONTAINER.getValue()),
|
||||
ctx.getExecutionAttribute(PID),
|
||||
Integer.toString(signal.getValue()));
|
||||
|
||||
//Some failures here are acceptable. Let the calling executor decide.
|
||||
signalOp.disableFailureLogging();
|
||||
|
||||
try {
|
||||
PrivilegedOperationExecutor executor = PrivilegedOperationExecutor
|
||||
.getInstance(conf);
|
||||
|
||||
executor.executePrivilegedOperation(null,
|
||||
signalOp, null, null, false, false);
|
||||
} catch (PrivilegedOperationException e) {
|
||||
//Don't log the failure here. Some kinds of signaling failures are
|
||||
// acceptable. Let the calling executor decide what to do.
|
||||
throw new ContainerExecutionException("Signal container failed", e
|
||||
.getExitCode(), e.getOutput(), e.getErrorOutput());
|
||||
}
|
||||
}
|
||||
|
||||
@InterfaceStability.Unstable
|
||||
static class RuncRuntimeObject {
|
||||
private final List<LocalResource> layers;
|
||||
private final LocalResource config;
|
||||
|
||||
RuncRuntimeObject(LocalResource config,
|
||||
List<LocalResource> layers) {
|
||||
this.config = config;
|
||||
this.layers = layers;
|
||||
}
|
||||
|
||||
public LocalResource getConfig() {
|
||||
return this.config;
|
||||
}
|
||||
|
||||
public List<LocalResource> getOCILayers() {
|
||||
return this.layers;
|
||||
}
|
||||
}
|
||||
|
||||
boolean getHostPidNamespaceEnabled() {
|
||||
return conf.getBoolean(
|
||||
YarnConfiguration.NM_RUNC_ALLOW_HOST_PID_NAMESPACE,
|
||||
YarnConfiguration.DEFAULT_NM_RUNC_ALLOW_HOST_PID_NAMESPACE);
|
||||
}
|
||||
|
||||
boolean getPrivilegedContainersEnabledOnCluster() {
|
||||
return conf.getBoolean(
|
||||
YarnConfiguration.NM_RUNC_ALLOW_PRIVILEGED_CONTAINERS,
|
||||
YarnConfiguration.DEFAULT_NM_RUNC_ALLOW_PRIVILEGED_CONTAINERS);
|
||||
}
|
||||
|
||||
Set<String> getAllowedNetworks() {
|
||||
return allowedNetworks;
|
||||
}
|
||||
|
||||
Set<String> getAllowedRuntimes() {
|
||||
return allowedRuntimes;
|
||||
}
|
||||
|
||||
AccessControlList getPrivilegedContainersAcl() {
|
||||
return privilegedContainersAcl;
|
||||
}
|
||||
|
||||
String getEnvOciContainerPidNamespace() {
|
||||
return ENV_RUNC_CONTAINER_PID_NAMESPACE;
|
||||
}
|
||||
|
||||
String getEnvOciContainerRunPrivilegedContainer() {
|
||||
return ENV_RUNC_CONTAINER_RUN_PRIVILEGED_CONTAINER;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,179 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import com.google.common.cache.CacheBuilder;
|
||||
import com.google.common.cache.CacheLoader;
|
||||
import com.google.common.cache.LoadingCache;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.service.AbstractService;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_STAT_CACHE_TIMEOUT;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_RUNC_STAT_CACHE_SIZE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_STAT_CACHE_SIZE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_STAT_CACHE_TIMEOUT;
|
||||
|
||||
/**
|
||||
* This class is a plugin for the
|
||||
* {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.RuncContainerRuntime}
|
||||
* that maps runC image manifests into their associated config and
|
||||
* layers that are located in HDFS.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public class HdfsManifestToResourcesPlugin extends AbstractService implements
|
||||
RuncManifestToResourcesPlugin {
|
||||
private Configuration conf;
|
||||
private String layersDir;
|
||||
private String configDir;
|
||||
private FileSystem fs;
|
||||
private LoadingCache<Path, FileStatus> statCache;
|
||||
|
||||
|
||||
private static final String CONFIG_MEDIA_TYPE =
|
||||
"application/vnd.docker.container.image.v1+json";
|
||||
|
||||
private static final String LAYER_TAR_GZIP_MEDIA_TYPE =
|
||||
"application/vnd.docker.image.rootfs.diff.tar.gzip";
|
||||
|
||||
private static final String SHA_256 = "sha256";
|
||||
|
||||
private static final String CONFIG_HASH_ALGORITHM =
|
||||
SHA_256;
|
||||
|
||||
private static final String LAYER_HASH_ALGORITHM =
|
||||
SHA_256;
|
||||
|
||||
private static final int SHA256_HASH_LENGTH = 64;
|
||||
|
||||
private static final String ALPHA_NUMERIC = "[a-zA-Z0-9]+";
|
||||
|
||||
public HdfsManifestToResourcesPlugin() {
|
||||
super(HdfsManifestToResourcesPlugin.class.getName());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serviceInit(Configuration configuration) {
|
||||
this.conf = configuration;
|
||||
String toplevelDir = conf.get(NM_RUNC_IMAGE_TOPLEVEL_DIR,
|
||||
DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR);
|
||||
this.layersDir = toplevelDir + "/layers/";
|
||||
this.configDir = toplevelDir + "/config/";
|
||||
CacheLoader<Path, FileStatus> cacheLoader =
|
||||
new CacheLoader<Path, FileStatus>() {
|
||||
@Override
|
||||
public FileStatus load(@Nonnull Path path) throws Exception {
|
||||
return statBlob(path);
|
||||
}
|
||||
};
|
||||
int statCacheSize = conf.getInt(NM_RUNC_STAT_CACHE_SIZE,
|
||||
DEFAULT_RUNC_STAT_CACHE_SIZE);
|
||||
int statCacheTimeout = conf.getInt(NM_RUNC_STAT_CACHE_TIMEOUT,
|
||||
DEFAULT_NM_RUNC_STAT_CACHE_TIMEOUT);
|
||||
this.statCache = CacheBuilder.newBuilder().maximumSize(statCacheSize)
|
||||
.refreshAfterWrite(statCacheTimeout, TimeUnit.SECONDS)
|
||||
.build(cacheLoader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void serviceStart() throws IOException {
|
||||
Path path = new Path(layersDir);
|
||||
this.fs = path.getFileSystem(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<LocalResource> getLayerResources(ImageManifest manifest)
|
||||
throws IOException {
|
||||
List<LocalResource> localRsrcs = new ArrayList<>();
|
||||
|
||||
for(ImageManifest.Blob blob : manifest.getLayers()) {
|
||||
LocalResource rsrc = getResource(blob, layersDir,
|
||||
LAYER_TAR_GZIP_MEDIA_TYPE, LAYER_HASH_ALGORITHM, ".sqsh");
|
||||
localRsrcs.add(rsrc);
|
||||
}
|
||||
return localRsrcs;
|
||||
}
|
||||
|
||||
public LocalResource getConfigResource(ImageManifest manifest)
|
||||
throws IOException {
|
||||
ImageManifest.Blob config = manifest.getConfig();
|
||||
return getResource(config, configDir, CONFIG_MEDIA_TYPE,
|
||||
CONFIG_HASH_ALGORITHM, "");
|
||||
}
|
||||
|
||||
public LocalResource getResource(ImageManifest.Blob blob,
|
||||
String dir, String expectedMediaType,
|
||||
String expectedHashAlgorithm, String resourceSuffix) throws IOException {
|
||||
String mediaType = blob.getMediaType();
|
||||
if (!mediaType.equals(expectedMediaType)) {
|
||||
throw new IOException("Invalid blob mediaType: " + mediaType);
|
||||
}
|
||||
|
||||
String[] blobDigest = blob.getDigest().split(":", 2);
|
||||
|
||||
String algorithm = blobDigest[0];
|
||||
if (!algorithm.equals(expectedHashAlgorithm)) {
|
||||
throw new IOException("Invalid blob digest algorithm: " + algorithm);
|
||||
}
|
||||
|
||||
String hash = blobDigest[1];
|
||||
if (!hash.matches(ALPHA_NUMERIC) || hash.length() != SHA256_HASH_LENGTH) {
|
||||
throw new IOException("Malformed blob digest: " + hash);
|
||||
}
|
||||
|
||||
long size = blob.getSize();
|
||||
Path path = new Path(dir, hash + resourceSuffix);
|
||||
LocalResource rsrc;
|
||||
|
||||
try {
|
||||
FileStatus stat = statCache.get(path);
|
||||
long timestamp = stat.getModificationTime();
|
||||
URL url = URL.fromPath(path);
|
||||
|
||||
rsrc = LocalResource.newInstance(url,
|
||||
LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
|
||||
size, timestamp);
|
||||
} catch (ExecutionException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
|
||||
return rsrc;
|
||||
}
|
||||
|
||||
protected FileStatus statBlob(Path path) throws IOException {
|
||||
return fs.getFileStatus(path);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,134 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
import java.util.ArrayList;
|
||||
import java.util.Map;
|
||||
|
||||
/**
|
||||
* This class is a Java representation of the OCI Image Manifest Specification.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public class ImageManifest {
|
||||
final private int schemaVersion;
|
||||
final private String mediaType;
|
||||
final private Blob config;
|
||||
final private ArrayList<Blob> layers;
|
||||
final private Map<String, String> annotations;
|
||||
|
||||
public ImageManifest() {
|
||||
this(0, null, null, null, null);
|
||||
}
|
||||
|
||||
public ImageManifest(int schemaVersion, String mediaType, Blob config,
|
||||
ArrayList<Blob> layers, Map<String, String> annotations) {
|
||||
this.schemaVersion = schemaVersion;
|
||||
this.mediaType = mediaType;
|
||||
this.config = config;
|
||||
this.layers = layers;
|
||||
this.annotations = annotations;
|
||||
}
|
||||
|
||||
public int getSchemaVersion() {
|
||||
return schemaVersion;
|
||||
}
|
||||
|
||||
public String getMediaType() {
|
||||
return mediaType;
|
||||
}
|
||||
|
||||
public Blob getConfig() {
|
||||
return config;
|
||||
}
|
||||
|
||||
public ArrayList<Blob> getLayers() {
|
||||
return layers;
|
||||
}
|
||||
|
||||
public Map<String, String> getAnnotations() {
|
||||
return annotations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
StringBuilder sb = new StringBuilder();
|
||||
sb.append("schemaVersion: " + schemaVersion + "\n");
|
||||
sb.append("mediaType: " + mediaType + "\n");
|
||||
sb.append(config.toString());
|
||||
for(Blob b : layers) {
|
||||
sb.append(b.toString());
|
||||
}
|
||||
return sb.toString();
|
||||
}
|
||||
|
||||
/**
|
||||
* This class is a Java representation of an OCI Image Blob.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public static class Blob {
|
||||
final private String mediaType;
|
||||
final private String digest;
|
||||
final private long size;
|
||||
final private ArrayList<String> urls;
|
||||
final private Map<String, String> annotations;
|
||||
|
||||
public Blob() {
|
||||
this(null, null, 0, null, null);
|
||||
}
|
||||
|
||||
public Blob(String mediaType, String digest, long size,
|
||||
ArrayList<String> urls, Map<String, String> annotations) {
|
||||
this.mediaType = mediaType;
|
||||
this.digest = digest;
|
||||
this.size = size;
|
||||
this.urls = urls;
|
||||
this.annotations = annotations;
|
||||
}
|
||||
|
||||
public String getMediaType() {
|
||||
return mediaType;
|
||||
}
|
||||
|
||||
public String getDigest() {
|
||||
return digest;
|
||||
}
|
||||
|
||||
public long getSize() {
|
||||
return size;
|
||||
}
|
||||
|
||||
public ArrayList<String> getUrls() {
|
||||
return urls;
|
||||
}
|
||||
|
||||
public Map<String, String> getAnnotations() {
|
||||
return annotations;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "mediaType: " + mediaType + "\n" + "size: " + size + "\n"
|
||||
+ "digest: " + digest + "\n";
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
@ -0,0 +1,331 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.commons.logging.Log;
|
||||
import org.apache.commons.logging.LogFactory;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.service.AbstractService;
|
||||
import org.apache.hadoop.util.concurrent.HadoopExecutors;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStreamReader;
|
||||
import java.nio.charset.StandardCharsets;
|
||||
import java.util.Collections;
|
||||
import java.util.HashMap;
|
||||
import java.util.LinkedHashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.ScheduledExecutorService;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.atomic.AtomicReference;
|
||||
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_CACHE_REFRESH_INTERVAL;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.DEFAULT_NUM_MANIFESTS_TO_CACHE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_HDFS_RUNC_IMAGE_TAG_TO_HASH_FILE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_LOCAL_RUNC_IMAGE_TAG_TO_HASH_FILE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_CACHE_REFRESH_INTERVAL;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_NUM_MANIFESTS_TO_CACHE;
|
||||
|
||||
/**
|
||||
* This class is a plugin for the
|
||||
* {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.RuncContainerRuntime}
|
||||
* to convert image tags into runC image manifests.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public class ImageTagToManifestPlugin extends AbstractService
|
||||
implements RuncImageTagToManifestPlugin {
|
||||
|
||||
private Map<String, ImageManifest> manifestCache;
|
||||
private ObjectMapper objMapper;
|
||||
private AtomicReference<Map<String, String>> localImageToHashCache =
|
||||
new AtomicReference<>(new HashMap<>());
|
||||
private AtomicReference<Map<String, String>> hdfsImageToHashCache =
|
||||
new AtomicReference<>(new HashMap<>());
|
||||
private Configuration conf;
|
||||
private ScheduledExecutorService exec;
|
||||
private long hdfsModTime;
|
||||
private long localModTime;
|
||||
private String hdfsImageToHashFile;
|
||||
private String manifestDir;
|
||||
private String localImageTagToHashFile;
|
||||
|
||||
private static final Log LOG = LogFactory.getLog(
|
||||
ImageTagToManifestPlugin.class);
|
||||
|
||||
private static final int SHA256_HASH_LENGTH = 64;
|
||||
private static final String ALPHA_NUMERIC = "[a-zA-Z0-9]+";
|
||||
|
||||
public ImageTagToManifestPlugin() {
|
||||
super("ImageTagToManifestPluginService");
|
||||
}
|
||||
|
||||
@Override
|
||||
public ImageManifest getManifestFromImageTag(String imageTag)
|
||||
throws IOException {
|
||||
String hash = getHashFromImageTag(imageTag);
|
||||
ImageManifest manifest = manifestCache.get(hash);
|
||||
if (manifest != null) {
|
||||
return manifest;
|
||||
}
|
||||
|
||||
Path manifestPath = new Path(manifestDir + hash);
|
||||
FileSystem fs = manifestPath.getFileSystem(conf);
|
||||
FSDataInputStream input;
|
||||
try {
|
||||
input = fs.open(manifestPath);
|
||||
} catch (IllegalArgumentException iae) {
|
||||
throw new IOException("Manifest file is not a valid HDFS file: "
|
||||
+ manifestPath.toString(), iae);
|
||||
}
|
||||
|
||||
byte[] bytes = IOUtils.toByteArray(input);
|
||||
manifest = objMapper.readValue(bytes, ImageManifest.class);
|
||||
|
||||
manifestCache.put(hash, manifest);
|
||||
return manifest;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getHashFromImageTag(String imageTag) {
|
||||
String hash;
|
||||
Map<String, String> localImageToHashCacheMap = localImageToHashCache.get();
|
||||
Map<String, String> hdfsImageToHashCacheMap = hdfsImageToHashCache.get();
|
||||
|
||||
// 1) Go to local file
|
||||
// 2) Go to HDFS
|
||||
// 3) Use tag as is/Assume tag is the hash
|
||||
hash = localImageToHashCacheMap.get(imageTag);
|
||||
if (hash == null) {
|
||||
hash = hdfsImageToHashCacheMap.get(imageTag);
|
||||
if (hash == null) {
|
||||
hash = imageTag;
|
||||
}
|
||||
}
|
||||
return hash;
|
||||
}
|
||||
|
||||
protected BufferedReader getLocalImageToHashReader() throws IOException {
|
||||
if (localImageTagToHashFile == null) {
|
||||
LOG.debug("Did not load local image to hash file, " +
|
||||
"file is null");
|
||||
return null;
|
||||
}
|
||||
|
||||
File imageTagToHashFile = new File(localImageTagToHashFile);
|
||||
if(!imageTagToHashFile.exists()) {
|
||||
LOG.debug("Did not load local image to hash file, " +
|
||||
"file doesn't exist");
|
||||
return null;
|
||||
}
|
||||
|
||||
long newLocalModTime = imageTagToHashFile.lastModified();
|
||||
if (newLocalModTime == localModTime) {
|
||||
LOG.debug("Did not load local image to hash file, " +
|
||||
"file is unmodified");
|
||||
return null;
|
||||
}
|
||||
localModTime = newLocalModTime;
|
||||
|
||||
return new BufferedReader(new InputStreamReader(
|
||||
new FileInputStream(imageTagToHashFile), StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
protected BufferedReader getHdfsImageToHashReader() throws IOException {
|
||||
if (hdfsImageToHashFile == null) {
|
||||
LOG.debug("Did not load hdfs image to hash file, " +
|
||||
"file is null");
|
||||
return null;
|
||||
}
|
||||
|
||||
Path imageToHash = new Path(hdfsImageToHashFile);
|
||||
FileSystem fs = imageToHash.getFileSystem(conf);
|
||||
if (!fs.exists(imageToHash)) {
|
||||
LOG.debug("Did not load hdfs image to hash file, " +
|
||||
"file doesn't exist");
|
||||
return null;
|
||||
}
|
||||
|
||||
long newHdfsModTime = fs.getFileStatus(imageToHash).getModificationTime();
|
||||
if (newHdfsModTime == hdfsModTime) {
|
||||
LOG.debug("Did not load hdfs image to hash file, " +
|
||||
"file is unmodified");
|
||||
return null;
|
||||
}
|
||||
hdfsModTime = newHdfsModTime;
|
||||
|
||||
return new BufferedReader(new InputStreamReader(fs.open(imageToHash),
|
||||
StandardCharsets.UTF_8));
|
||||
}
|
||||
|
||||
/** You may specify multiple tags per hash all on the same line.
|
||||
* Comments are allowed using #. Anything after this character will not
|
||||
* be read
|
||||
* Example file:
|
||||
* foo/bar:current,fizz/gig:latest:123456789
|
||||
* #this/line:wont,be:parsed:2378590895
|
||||
|
||||
* This will map both foo/bar:current and fizz/gig:latest to 123456789
|
||||
*/
|
||||
protected static Map<String, String> readImageToHashFile(
|
||||
BufferedReader br) throws IOException {
|
||||
if (br == null) {
|
||||
return null;
|
||||
}
|
||||
|
||||
String line;
|
||||
Map<String, String> imageToHashCache = new HashMap<>();
|
||||
while ((line = br.readLine()) != null) {
|
||||
int index;
|
||||
index = line.indexOf("#");
|
||||
if (index == 0) {
|
||||
continue;
|
||||
} else if (index != -1) {
|
||||
line = line.substring(0, index);
|
||||
}
|
||||
|
||||
index = line.lastIndexOf(":");
|
||||
if (index == -1) {
|
||||
LOG.warn("Malformed imageTagToManifest entry: " + line);
|
||||
continue;
|
||||
}
|
||||
String imageTags = line.substring(0, index);
|
||||
String[] imageTagArray = imageTags.split(",");
|
||||
String hash = line.substring(index + 1);
|
||||
if (!hash.matches(ALPHA_NUMERIC) || hash.length() != SHA256_HASH_LENGTH) {
|
||||
LOG.warn("Malformed image hash: " + hash);
|
||||
continue;
|
||||
}
|
||||
|
||||
for (String imageTag : imageTagArray) {
|
||||
imageToHashCache.put(imageTag, hash);
|
||||
}
|
||||
}
|
||||
return imageToHashCache;
|
||||
}
|
||||
|
||||
public boolean loadImageToHashFiles() throws IOException {
|
||||
boolean ret = false;
|
||||
try (
|
||||
BufferedReader localBr = getLocalImageToHashReader();
|
||||
BufferedReader hdfsBr = getHdfsImageToHashReader()
|
||||
) {
|
||||
Map<String, String> localImageToHash = readImageToHashFile(localBr);
|
||||
Map<String, String> hdfsImageToHash = readImageToHashFile(hdfsBr);
|
||||
|
||||
Map<String, String> tmpLocalImageToHash = localImageToHashCache.get();
|
||||
Map<String, String> tmpHdfsImageToHash = hdfsImageToHashCache.get();
|
||||
|
||||
if (localImageToHash != null &&
|
||||
!localImageToHash.equals(tmpLocalImageToHash)) {
|
||||
localImageToHashCache.set(localImageToHash);
|
||||
LOG.info("Reloaded local image tag to hash cache");
|
||||
ret = true;
|
||||
}
|
||||
if (hdfsImageToHash != null &&
|
||||
!hdfsImageToHash.equals(tmpHdfsImageToHash)) {
|
||||
hdfsImageToHashCache.set(hdfsImageToHash);
|
||||
LOG.info("Reloaded hdfs image tag to hash cache");
|
||||
ret = true;
|
||||
}
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceInit(Configuration configuration) throws Exception {
|
||||
super.serviceInit(configuration);
|
||||
this.conf = configuration;
|
||||
localImageTagToHashFile = conf.get(NM_LOCAL_RUNC_IMAGE_TAG_TO_HASH_FILE);
|
||||
if (localImageTagToHashFile == null) {
|
||||
LOG.debug("Failed to load local runC image to hash file. " +
|
||||
"Config not set");
|
||||
}
|
||||
hdfsImageToHashFile = conf.get(NM_HDFS_RUNC_IMAGE_TAG_TO_HASH_FILE);
|
||||
if (hdfsImageToHashFile == null) {
|
||||
LOG.debug("Failed to load HDFS runC image to hash file. Config not set");
|
||||
}
|
||||
if(hdfsImageToHashFile == null && localImageTagToHashFile == null) {
|
||||
LOG.warn("No valid image-tag-to-hash files");
|
||||
}
|
||||
manifestDir = conf.get(NM_RUNC_IMAGE_TOPLEVEL_DIR,
|
||||
DEFAULT_NM_RUNC_IMAGE_TOPLEVEL_DIR) + "/manifests/";
|
||||
int numManifestsToCache = conf.getInt(NM_RUNC_NUM_MANIFESTS_TO_CACHE,
|
||||
DEFAULT_NUM_MANIFESTS_TO_CACHE);
|
||||
this.objMapper = new ObjectMapper();
|
||||
this.manifestCache = Collections.synchronizedMap(
|
||||
new LRUCache(numManifestsToCache, 0.75f));
|
||||
|
||||
exec = HadoopExecutors.newScheduledThreadPool(1);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStart() throws Exception {
|
||||
super.serviceStart();
|
||||
if(!loadImageToHashFiles()) {
|
||||
LOG.warn("Couldn't load any image-tag-to-hash-files");
|
||||
}
|
||||
int runcCacheRefreshInterval = conf.getInt(NM_RUNC_CACHE_REFRESH_INTERVAL,
|
||||
DEFAULT_NM_RUNC_CACHE_REFRESH_INTERVAL);
|
||||
exec = HadoopExecutors.newScheduledThreadPool(1);
|
||||
exec.scheduleWithFixedDelay(
|
||||
new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
loadImageToHashFiles();
|
||||
} catch (Exception e) {
|
||||
LOG.warn("runC cache refresh thread caught an exception: ", e);
|
||||
}
|
||||
}
|
||||
}, runcCacheRefreshInterval, runcCacheRefreshInterval, TimeUnit.SECONDS);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected void serviceStop() throws Exception {
|
||||
super.serviceStop();
|
||||
exec.shutdownNow();
|
||||
}
|
||||
|
||||
private static class LRUCache extends LinkedHashMap<String, ImageManifest> {
|
||||
private int cacheSize;
|
||||
|
||||
LRUCache(int initialCapacity, float loadFactor) {
|
||||
super(initialCapacity, loadFactor, true);
|
||||
this.cacheSize = initialCapacity;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected boolean removeEldestEntry(
|
||||
Map.Entry<String, ImageManifest> eldest) {
|
||||
return this.size() > cacheSize;
|
||||
}
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -0,0 +1,37 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.service.Service;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* This class is a plugin interface for the
|
||||
* {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.RuncContainerRuntime}
|
||||
* to convert image tags into OCI Image Manifests.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public interface RuncImageTagToManifestPlugin extends Service {
|
||||
ImageManifest getManifestFromImageTag(String imageTag) throws IOException;
|
||||
|
||||
String getHashFromImageTag(String imageTag);
|
||||
}
|
|
@ -0,0 +1,42 @@
|
|||
/*
|
||||
*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.service.Service;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* This class is a plugin interface for the
|
||||
* {@link org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.RuncContainerRuntime}
|
||||
* that maps OCI Image Manifests into associated config and layers.
|
||||
*/
|
||||
@InterfaceStability.Unstable
|
||||
public interface RuncManifestToResourcesPlugin extends Service {
|
||||
//The layers should be returned in the order in which they
|
||||
// appear in the manifest
|
||||
List<LocalResource> getLayerResources(ImageManifest manifest)
|
||||
throws IOException;
|
||||
|
||||
LocalResource getConfigResource(ImageManifest manifest) throws IOException;
|
||||
}
|
|
@ -0,0 +1,26 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
/**
|
||||
* Package containing classes related to runC commands and common operations
|
||||
* used within the @{link RuncContainerRuntime}.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
|
@ -1699,4 +1699,14 @@ public class ResourceLocalizationService extends CompositeService
|
|||
localDirPathFsPermissionsMap.put(sysDir, nmPrivatePermission);
|
||||
return localDirPathFsPermissionsMap;
|
||||
}
|
||||
|
||||
public LocalizedResource getLocalizedResource(LocalResourceRequest req,
|
||||
String user, ApplicationId appId) {
|
||||
LocalResourcesTracker tracker =
|
||||
getLocalResourcesTracker(req.getVisibility(), user, appId);
|
||||
if (tracker == null) {
|
||||
return null;
|
||||
}
|
||||
return tracker.getLocalizedResource(req);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -34,4 +34,8 @@ public class ContainerRuntimeConstants {
|
|||
@Private
|
||||
public static final String CONTAINER_RUNTIME_DOCKER =
|
||||
"docker";
|
||||
|
||||
@Private
|
||||
public static final String CONTAINER_RUNTIME_RUNC =
|
||||
"runc";
|
||||
}
|
||||
|
|
|
@ -27,7 +27,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
|
|||
import org.apache.hadoop.yarn.api.records.ResourceInformation;
|
||||
import org.apache.hadoop.yarn.exceptions.YarnException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.DockerLinuxContainerRuntime;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.OCIContainerRuntime;
|
||||
import org.apache.hadoop.yarn.server.volume.csi.CsiConstants;
|
||||
import org.apache.hadoop.yarn.server.volume.csi.VolumeMetaData;
|
||||
import org.apache.hadoop.yarn.server.volume.csi.exception.InvalidVolumeException;
|
||||
|
@ -51,10 +51,10 @@ public class ContainerVolumePublisher {
|
|||
|
||||
private final Container container;
|
||||
private final String localMountRoot;
|
||||
private final DockerLinuxContainerRuntime runtime;
|
||||
private final OCIContainerRuntime runtime;
|
||||
|
||||
public ContainerVolumePublisher(Container container, String localMountRoot,
|
||||
DockerLinuxContainerRuntime runtime) {
|
||||
OCIContainerRuntime runtime) {
|
||||
LOG.info("Initiate container volume publisher, containerID={},"
|
||||
+ " volume local mount rootDir={}",
|
||||
container.getContainerId().toString(), localMountRoot);
|
||||
|
|
|
@ -717,6 +717,14 @@ public class TestLinuxContainerExecutor {
|
|||
verify(lce, times(1)).updateYarnSysFS(ctx, user, appId, spec);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetLocalResources() throws Exception {
|
||||
Container container = mock(Container.class);
|
||||
LinuxContainerExecutor lce = mock(LinuxContainerExecutor.class);
|
||||
lce.getLocalResources(container);
|
||||
verify(lce, times(1)).getLocalResources(container);
|
||||
}
|
||||
|
||||
private static class TestResourceHandler implements LCEResourcesHandler {
|
||||
static Set<ContainerId> postExecContainers = new HashSet<ContainerId>();
|
||||
|
||||
|
|
|
@ -223,6 +223,7 @@ public abstract class BaseContainerManagerTest {
|
|||
nodeHealthChecker.init(conf);
|
||||
containerManager = createContainerManager(delSrvc);
|
||||
((NMContext)context).setContainerManager(containerManager);
|
||||
((NMContext)context).setContainerExecutor(exec);
|
||||
nodeStatusUpdater.init(conf);
|
||||
containerManager.init(conf);
|
||||
nodeStatusUpdater.start();
|
||||
|
|
|
@ -794,6 +794,7 @@ public class TestContainerManagerRecovery extends BaseContainerManagerTest {
|
|||
.byteValue() }));
|
||||
context.getContainerTokenSecretManager().setMasterKey(masterKey);
|
||||
context.getNMTokenSecretManager().setMasterKey(masterKey);
|
||||
context.setContainerExecutor(exec);
|
||||
return context;
|
||||
}
|
||||
|
||||
|
|
|
@ -1316,6 +1316,7 @@ public class TestContainer {
|
|||
new HashMap<>();
|
||||
private final Map<ContainerEventType, ContainerState> eventToFinalState =
|
||||
new HashMap<>();
|
||||
private final ContainerExecutor exec;
|
||||
|
||||
WrappedContainer(int appId, long timestamp, int id, String user)
|
||||
throws IOException {
|
||||
|
@ -1352,6 +1353,7 @@ public class TestContainer {
|
|||
container.sendLaunchEvent();
|
||||
}
|
||||
};
|
||||
exec = mock(ContainerExecutor.class);
|
||||
dispatcher.register(LocalizationEventType.class, localizerBus);
|
||||
dispatcher.register(ContainersLauncherEventType.class, launcherBus);
|
||||
dispatcher.register(ContainersMonitorEventType.class, monitorBus);
|
||||
|
@ -1412,6 +1414,9 @@ public class TestContainer {
|
|||
}
|
||||
when(ctxt.getLocalResources()).thenReturn(localResources);
|
||||
|
||||
when(exec.getLocalResources(any())).thenReturn(localResources);
|
||||
when(context.getContainerExecutor()).thenReturn(exec);
|
||||
|
||||
if (withServiceData) {
|
||||
Random r = new Random();
|
||||
long seed = r.nextLong();
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
|
@ -30,6 +31,7 @@ import org.apache.hadoop.security.Credentials;
|
|||
import org.apache.hadoop.util.Shell;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
|
||||
|
@ -38,6 +40,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|||
import org.apache.hadoop.yarn.security.TestDockerClientConfigHandler;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.Context;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperation;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.privileged.PrivilegedOperationException;
|
||||
|
@ -49,9 +52,13 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.
|
|||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.DockerCommandPlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePluginManager;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.LocalizedResource;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeConstants;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerRuntimeContext;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Rule;
|
||||
|
@ -136,6 +143,7 @@ public class TestDockerContainerRuntime {
|
|||
private Container container;
|
||||
private ContainerId cId;
|
||||
private ApplicationAttemptId appAttemptId;
|
||||
private ApplicationId mockApplicationId;
|
||||
private ContainerLaunchContext context;
|
||||
private Context nmContext;
|
||||
private HashMap<String, String> env;
|
||||
|
@ -165,7 +173,9 @@ public class TestDockerContainerRuntime {
|
|||
private final String whitelistedUser = "yoda";
|
||||
private String[] testCapabilities;
|
||||
private final String signalPid = "1234";
|
||||
private String runtimeTypeUpper = "DOCKER";
|
||||
private final String tmpPath =
|
||||
new StringBuffer(System.getProperty("test.build.data"))
|
||||
.append('/').append("hadoop.tmp.dir").toString();
|
||||
|
||||
private static final String RUNTIME_TYPE = "DOCKER";
|
||||
private final static String ENV_OCI_CONTAINER_PID_NAMESPACE =
|
||||
|
@ -188,8 +198,6 @@ public class TestDockerContainerRuntime {
|
|||
|
||||
@Before
|
||||
public void setup() {
|
||||
String tmpPath = new StringBuffer(System.getProperty("test.build.data"))
|
||||
.append('/').append("hadoop.tmp.dir").toString();
|
||||
|
||||
conf = new Configuration();
|
||||
conf.set("hadoop.tmp.dir", tmpPath);
|
||||
|
@ -201,6 +209,7 @@ public class TestDockerContainerRuntime {
|
|||
container = mock(Container.class);
|
||||
cId = mock(ContainerId.class);
|
||||
appAttemptId = mock(ApplicationAttemptId.class);
|
||||
mockApplicationId = mock(ApplicationId.class);
|
||||
context = mock(ContainerLaunchContext.class);
|
||||
env = new HashMap<String, String>();
|
||||
env.put("FROM_CLIENT", "1");
|
||||
|
@ -210,6 +219,8 @@ public class TestDockerContainerRuntime {
|
|||
env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_IMAGE, image);
|
||||
when(container.getContainerId()).thenReturn(cId);
|
||||
when(cId.toString()).thenReturn(containerId);
|
||||
when(mockApplicationId.toString()).thenReturn("applicationId");
|
||||
when(appAttemptId.getApplicationId()).thenReturn(mockApplicationId);
|
||||
when(cId.getApplicationAttemptId()).thenReturn(appAttemptId);
|
||||
when(container.getLaunchContext()).thenReturn(context);
|
||||
when(context.getEnvironment()).thenReturn(env);
|
||||
|
@ -280,6 +291,9 @@ public class TestDockerContainerRuntime {
|
|||
localizedResources.put(new Path("/test_local_dir/test_resource_file"),
|
||||
Collections.singletonList("test_dir/test_resource_file"));
|
||||
|
||||
File tmpDir = new File(tmpPath);
|
||||
tmpDir.mkdirs();
|
||||
|
||||
testCapabilities = new String[] {"NET_BIND_SERVICE", "SYS_CHROOT"};
|
||||
conf.setStrings(YarnConfiguration.NM_DOCKER_CONTAINER_CAPABILITIES,
|
||||
testCapabilities);
|
||||
|
@ -309,6 +323,12 @@ public class TestDockerContainerRuntime {
|
|||
.setExecutionAttribute(RESOURCES_OPTIONS, resourcesOptions);
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanUp() throws IOException {
|
||||
File tmpDir = new File(tmpPath);
|
||||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
|
||||
public Context createMockNMContext() {
|
||||
Context mockNMContext = mock(Context.class);
|
||||
LocalDirsHandlerService localDirsHandler =
|
||||
|
@ -316,9 +336,6 @@ public class TestDockerContainerRuntime {
|
|||
ResourcePluginManager resourcePluginManager =
|
||||
mock(ResourcePluginManager.class);
|
||||
|
||||
String tmpPath = new StringBuffer(System.getProperty("test.build.data"))
|
||||
.append('/').append("hadoop.tmp.dir").toString();
|
||||
|
||||
ConcurrentMap<ContainerId, Container> containerMap =
|
||||
mock(ConcurrentMap.class);
|
||||
|
||||
|
@ -328,6 +345,20 @@ public class TestDockerContainerRuntime {
|
|||
when(mockNMContext.getContainers()).thenReturn(containerMap);
|
||||
when(containerMap.get(any())).thenReturn(container);
|
||||
|
||||
ContainerManager mockContainerManager = mock(ContainerManager.class);
|
||||
ResourceLocalizationService mockLocalzationService =
|
||||
mock(ResourceLocalizationService.class);
|
||||
|
||||
LocalizedResource mockLocalizedResource = mock(LocalizedResource.class);
|
||||
|
||||
when(mockLocalizedResource.getLocalPath()).thenReturn(
|
||||
new Path("/local/layer1"));
|
||||
when(mockLocalzationService.getLocalizedResource(any(), anyString(), any()))
|
||||
.thenReturn(mockLocalizedResource);
|
||||
when(mockContainerManager.getResourceLocalizationService())
|
||||
.thenReturn(mockLocalzationService);
|
||||
when(mockNMContext.getContainerManager()).thenReturn(mockContainerManager);
|
||||
|
||||
try {
|
||||
when(localDirsHandler.getLocalPathForWrite(anyString()))
|
||||
.thenReturn(new Path(tmpPath));
|
||||
|
|
|
@ -0,0 +1,182 @@
|
|||
/*
|
||||
* *
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
* /
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResource;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceType;
|
||||
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
|
||||
import org.apache.hadoop.yarn.api.records.URL;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.HdfsManifestToResourcesPlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageManifest;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayList;
|
||||
import java.util.List;
|
||||
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* This class tests the hdfs manifest to resources plugin used by the
|
||||
* RuncContainerRuntime to map an image manifest into a list of local resources.
|
||||
*/
|
||||
public class TestHdfsManifestToResourcesPlugin {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestHdfsManifestToResourcesPlugin.class);
|
||||
private Configuration conf;
|
||||
private String tmpPath = new StringBuffer(
|
||||
System.getProperty("test.build.data"))
|
||||
.append('/').append("hadoop.tmp.dir").toString();
|
||||
private static final String LAYER_MEDIA_TYPE =
|
||||
"application/vnd.docker.image.rootfs.diff.tar.gzip";
|
||||
private static final String CONFIG_MEDIA_TYPE =
|
||||
"application/vnd.docker.container.image.v1+json";
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
conf = new Configuration();
|
||||
File tmpDir = new File(tmpPath);
|
||||
tmpDir.mkdirs();
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanUp() throws IOException {
|
||||
File tmpDir = new File(tmpPath);
|
||||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetLayerResources() throws IOException {
|
||||
ImageManifest mockManifest = mock(ImageManifest.class);
|
||||
ImageManifest.Blob mockLayer1 = mock(ImageManifest.Blob.class);
|
||||
ImageManifest.Blob mockLayer2 = mock(ImageManifest.Blob.class);
|
||||
String digest1Hash =
|
||||
"e060f9dd9e8cd9ec0e2814b661a96d78f7298120d7654ba9f83ebfb11ff1fb1e";
|
||||
String digest2Hash =
|
||||
"5af5ff88469c8473487bfbc2fe81b4e7d84644bd91f1ab9305de47ef5673637e";
|
||||
String digest1 =
|
||||
"sha256:" + digest1Hash;
|
||||
String digest2 =
|
||||
"sha256:" + digest2Hash;
|
||||
long size1 = 1234;
|
||||
long size2 = 5678;
|
||||
|
||||
when(mockLayer1.getMediaType()).thenReturn(LAYER_MEDIA_TYPE);
|
||||
when(mockLayer1.getDigest()).thenReturn(digest1);
|
||||
when(mockLayer1.getSize()).thenReturn(size1);
|
||||
|
||||
when(mockLayer2.getMediaType()).thenReturn(LAYER_MEDIA_TYPE);
|
||||
when(mockLayer2.getDigest()).thenReturn(digest2);
|
||||
when(mockLayer2.getSize()).thenReturn(size2);
|
||||
|
||||
ArrayList<ImageManifest.Blob> mockLayers = new ArrayList<>();
|
||||
mockLayers.add(mockLayer1);
|
||||
mockLayers.add(mockLayer2);
|
||||
|
||||
when(mockManifest.getLayers()).thenReturn(mockLayers);
|
||||
|
||||
conf.set(NM_RUNC_IMAGE_TOPLEVEL_DIR, tmpPath);
|
||||
long modTime = 123456789;
|
||||
|
||||
HdfsManifestToResourcesPlugin hdfsManifestToResourcesPlugin =
|
||||
new HdfsManifestToResourcesPlugin() {
|
||||
@Override
|
||||
protected FileStatus statBlob(Path path) throws IOException {
|
||||
FileStatus mockFileStatus = mock(FileStatus.class);
|
||||
when(mockFileStatus.getModificationTime()).thenReturn(modTime);
|
||||
return mockFileStatus;
|
||||
}
|
||||
};
|
||||
hdfsManifestToResourcesPlugin.init(conf);
|
||||
|
||||
List<LocalResource> returnedLayers =
|
||||
hdfsManifestToResourcesPlugin.getLayerResources(mockManifest);
|
||||
|
||||
URL url1 = URL.fromPath(new Path(tmpPath + "/layers",
|
||||
digest1Hash + ".sqsh"));
|
||||
URL url2 = URL.fromPath(new Path(tmpPath + "/layers",
|
||||
digest2Hash + ".sqsh"));
|
||||
|
||||
LocalResource rsrc1 = LocalResource.newInstance(url1,
|
||||
LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
|
||||
size1, modTime);
|
||||
LocalResource rsrc2 = LocalResource.newInstance(url2,
|
||||
LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
|
||||
size2, modTime);
|
||||
|
||||
Assert.assertEquals(rsrc1, returnedLayers.get(0));
|
||||
Assert.assertEquals(rsrc2, returnedLayers.get(1));
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetConfigResources() throws IOException {
|
||||
ImageManifest mockManifest = mock(ImageManifest.class);
|
||||
ImageManifest.Blob mockConfig = mock(ImageManifest.Blob.class);
|
||||
String digestHash =
|
||||
"e23cac476d0238f0f859c1e07e5faad85262bca490ef5c3a9da32a5b39c6b204";
|
||||
String digest =
|
||||
"sha256:" + digestHash;
|
||||
long size = 1234;
|
||||
|
||||
when(mockConfig.getMediaType()).thenReturn(CONFIG_MEDIA_TYPE);
|
||||
when(mockConfig.getDigest()).thenReturn(digest);
|
||||
when(mockConfig.getSize()).thenReturn(size);
|
||||
when(mockManifest.getConfig()).thenReturn(mockConfig);
|
||||
|
||||
conf.set(NM_RUNC_IMAGE_TOPLEVEL_DIR, tmpPath);
|
||||
long modTime = 123456789;
|
||||
|
||||
HdfsManifestToResourcesPlugin hdfsManifestToResourcesPlugin =
|
||||
new HdfsManifestToResourcesPlugin() {
|
||||
@Override
|
||||
protected FileStatus statBlob(Path path) throws IOException {
|
||||
FileStatus mockFileStatus = mock(FileStatus.class);
|
||||
when(mockFileStatus.getModificationTime()).thenReturn(modTime);
|
||||
return mockFileStatus;
|
||||
}
|
||||
};
|
||||
hdfsManifestToResourcesPlugin.init(conf);
|
||||
|
||||
LocalResource returnedLayer =
|
||||
hdfsManifestToResourcesPlugin.getConfigResource(mockManifest);
|
||||
|
||||
URL url1 = URL.fromPath(new Path(tmpPath + "/config", digestHash));
|
||||
|
||||
LocalResource rsrc = LocalResource.newInstance(url1,
|
||||
LocalResourceType.FILE, LocalResourceVisibility.PUBLIC,
|
||||
size, modTime);
|
||||
|
||||
Assert.assertEquals(rsrc, returnedLayer);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,247 @@
|
|||
/*
|
||||
* *
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
* /
|
||||
*/
|
||||
|
||||
package org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime;
|
||||
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageManifest;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageTagToManifestPlugin;
|
||||
import org.codehaus.jackson.map.ObjectMapper;
|
||||
import org.junit.After;
|
||||
import org.junit.Assert;
|
||||
import org.junit.Before;
|
||||
import org.junit.Test;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import java.io.BufferedReader;
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintWriter;
|
||||
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_HDFS_RUNC_IMAGE_TAG_TO_HASH_FILE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_LOCAL_RUNC_IMAGE_TAG_TO_HASH_FILE;
|
||||
import static org.apache.hadoop.yarn.conf.YarnConfiguration.NM_RUNC_IMAGE_TOPLEVEL_DIR;
|
||||
import static org.mockito.Mockito.mock;
|
||||
import static org.mockito.Mockito.when;
|
||||
|
||||
/**
|
||||
* This class tests the hdfs manifest to resources plugin used by the
|
||||
* RuncContainerRuntime to map an image manifest into a list of local resources.
|
||||
*/
|
||||
public class TestImageTagToManifestPlugin {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestImageTagToManifestPlugin.class);
|
||||
private MockImageTagToManifestPlugin mockImageTagToManifestPlugin;
|
||||
private Configuration conf;
|
||||
private String tmpPath =
|
||||
new StringBuffer(System.getProperty("test.build.data"))
|
||||
.append('/').append("hadoop.tmp.dir").toString();
|
||||
private ObjectMapper mapper;
|
||||
|
||||
private String manifestJson =
|
||||
"{\n" +
|
||||
" \"schemaVersion\": 2,\n" +
|
||||
" \"mediaType\": \"application/vnd.docker.distribution.manifest.v2+json\",\n" +
|
||||
" \"config\": {\n" +
|
||||
" \"mediaType\": \"application/vnd.docker.container.image.v1+json\",\n" +
|
||||
" \"size\": 2857,\n" +
|
||||
" \"digest\": \"sha256:e23cac476d0238f0f859c1e07e5faad85262bca490ef5c3a9da32a5b39c6b204\"\n" +
|
||||
" },\n" +
|
||||
" \"layers\": [\n" +
|
||||
" {\n" +
|
||||
" \"mediaType\": \"application/vnd.docker.image.rootfs.diff.tar.gzip\",\n" +
|
||||
" \"size\": 185784329,\n" +
|
||||
" \"digest\": \"sha256:e060f9dd9e8cd9ec0e2814b661a96d78f7298120d7654ba9f83ebfb11ff1fb1e\"\n" +
|
||||
" },\n" +
|
||||
" {\n" +
|
||||
" \"mediaType\": \"application/vnd.docker.image.rootfs.diff.tar.gzip\",\n" +
|
||||
" \"size\": 10852,\n" +
|
||||
" \"digest\": \"sha256:5af5ff88469c8473487bfbc2fe81b4e7d84644bd91f1ab9305de47ef5673637e\"\n" +
|
||||
" }\n" +
|
||||
" ]\n" +
|
||||
"}";
|
||||
|
||||
@Before
|
||||
public void setup() {
|
||||
conf = new Configuration();
|
||||
mapper = new ObjectMapper();
|
||||
File tmpDir = new File(tmpPath);
|
||||
tmpDir.mkdirs();
|
||||
}
|
||||
|
||||
@After
|
||||
public void cleanUp() throws IOException {
|
||||
File tmpDir = new File(tmpPath);
|
||||
FileUtils.deleteDirectory(tmpDir);
|
||||
}
|
||||
|
||||
/**
|
||||
* This class mocks the hdfs manifest to resources plugin used by the
|
||||
* RuncContainerRuntime to enable testing.
|
||||
*/
|
||||
public class MockImageTagToManifestPlugin extends ImageTagToManifestPlugin {
|
||||
private BufferedReader mockLocalBufferedReader;
|
||||
private BufferedReader mockHdfsBufferedReader;
|
||||
|
||||
MockImageTagToManifestPlugin(BufferedReader mockLocalBufferedReader,
|
||||
BufferedReader mockHdfsBufferedReader) {
|
||||
super();
|
||||
this.mockLocalBufferedReader = mockLocalBufferedReader;
|
||||
this.mockHdfsBufferedReader = mockHdfsBufferedReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BufferedReader getLocalImageToHashReader() throws IOException {
|
||||
return mockLocalBufferedReader;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected BufferedReader getHdfsImageToHashReader() throws IOException {
|
||||
return mockHdfsBufferedReader;
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testLocalGetHashFromImageTag() throws IOException {
|
||||
BufferedReader mockLocalBufferedReader = mock(BufferedReader.class);
|
||||
BufferedReader mockHdfsBufferedReader = mock(BufferedReader.class);
|
||||
|
||||
String commentImage = "commentimage:latest";
|
||||
String commentImageHash =
|
||||
"142fff813433c1faa8796388db3a1fa1e899ba08c9e42ad2e33c42696d0f15d2";
|
||||
|
||||
String fakeImageLatest = "fakeimage:latest";
|
||||
String fakeImageCurrent= "fakeimage:current";
|
||||
String fakeImageHash =
|
||||
"f75903872eb2963e158502ef07f2e56d3a2e90a012b4afe3440461b54142a567";
|
||||
|
||||
String busyboxImage = "repo/busybox:123";
|
||||
String busyboxHash =
|
||||
"c6912b9911deceec6c43ebb4c31c96374a8ebb3de4cd75f377dba6c07707de6e";
|
||||
|
||||
String commentLine = "#" + commentImage + commentImageHash + "#2nd comment";
|
||||
String busyboxLine = busyboxImage + ":" + busyboxHash + "#comment";
|
||||
String fakeImageLine = fakeImageLatest + "," + fakeImageCurrent + ":"
|
||||
+ fakeImageHash + "#fakeimage comment";
|
||||
|
||||
when(mockLocalBufferedReader.readLine()).thenReturn(commentLine,
|
||||
fakeImageLine, busyboxLine, null);
|
||||
|
||||
mockImageTagToManifestPlugin = new MockImageTagToManifestPlugin(
|
||||
mockLocalBufferedReader, mockHdfsBufferedReader);
|
||||
mockImageTagToManifestPlugin.loadImageToHashFiles();
|
||||
|
||||
String returnedFakeImageHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(fakeImageLatest);
|
||||
String returnedBusyboxHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(busyboxImage);
|
||||
String returnedCommentHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(commentImage);
|
||||
|
||||
Assert.assertEquals(fakeImageHash, returnedFakeImageHash);
|
||||
Assert.assertEquals(busyboxHash, returnedBusyboxHash);
|
||||
|
||||
//Image hash should not be found, so returned hash should be the tag
|
||||
Assert.assertEquals(commentImage, returnedCommentHash);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testHdfsGetHashFromImageTag() throws IOException {
|
||||
BufferedReader mockLocalBufferedReader = mock(BufferedReader.class);
|
||||
BufferedReader mockHdfsBufferedReader = mock(BufferedReader.class);
|
||||
|
||||
String commentImage = "commentimage:latest";
|
||||
String commentImageHash =
|
||||
"142fff813433c1faa8796388db3a1fa1e899ba08c9e42ad2e33c42696d0f15d2";
|
||||
|
||||
String fakeImageLatest = "fakeimage:latest";
|
||||
String fakeImageCurrent= "fakeimage:current";
|
||||
String fakeImageHash =
|
||||
"f75903872eb2963e158502ef07f2e56d3a2e90a012b4afe3440461b54142a567";
|
||||
|
||||
String busyboxImage = "repo/busybox:123";
|
||||
String busyboxHash =
|
||||
"c6912b9911deceec6c43ebb4c31c96374a8ebb3de4cd75f377dba6c07707de6e";
|
||||
|
||||
String commentLine = "#" + commentImage + commentImageHash + "#2nd comment";
|
||||
String busyboxLine = busyboxImage + ":" + busyboxHash + "#comment";
|
||||
String fakeImageLine = fakeImageLatest + "," + fakeImageCurrent + ":"
|
||||
+ fakeImageHash + "#fakeimage comment";
|
||||
|
||||
when(mockHdfsBufferedReader.readLine()).thenReturn(commentLine,
|
||||
fakeImageLine, busyboxLine, null);
|
||||
|
||||
mockImageTagToManifestPlugin = new MockImageTagToManifestPlugin(
|
||||
mockLocalBufferedReader, mockHdfsBufferedReader);
|
||||
mockImageTagToManifestPlugin.loadImageToHashFiles();
|
||||
|
||||
String returnedFakeImageHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(fakeImageLatest);
|
||||
String returnedBusyboxHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(busyboxImage);
|
||||
String returnedCommentHash = mockImageTagToManifestPlugin
|
||||
.getHashFromImageTag(commentImage);
|
||||
|
||||
Assert.assertEquals(fakeImageHash, returnedFakeImageHash);
|
||||
Assert.assertEquals(busyboxHash, returnedBusyboxHash);
|
||||
|
||||
//Image hash should not be found, so returned hash should be the tag
|
||||
Assert.assertEquals(commentImage, returnedCommentHash);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testGetManifestFromImageTag() throws IOException {
|
||||
String manifestPath = tmpPath + "/manifests";
|
||||
File manifestDir = new File(manifestPath);
|
||||
manifestDir.mkdirs();
|
||||
|
||||
conf.set(NM_LOCAL_RUNC_IMAGE_TAG_TO_HASH_FILE, "local-image-tag-to-hash");
|
||||
conf.set(NM_HDFS_RUNC_IMAGE_TAG_TO_HASH_FILE, "hdfs-image-tag-to-hash");
|
||||
conf.set(NM_RUNC_IMAGE_TOPLEVEL_DIR, tmpPath);
|
||||
String manifestHash =
|
||||
"d0e8c542d28e8e868848aeb58beecb31079eb7ada1293c4bc2eded08daed605a";
|
||||
|
||||
PrintWriter printWriter = new PrintWriter(
|
||||
manifestPath + "/" + manifestHash);
|
||||
printWriter.println(manifestJson);
|
||||
printWriter.close();
|
||||
|
||||
BufferedReader mockLocalBufferedReader = mock(BufferedReader.class);
|
||||
BufferedReader mockHdfsBufferedReader = mock(BufferedReader.class);
|
||||
|
||||
mockImageTagToManifestPlugin = new MockImageTagToManifestPlugin(
|
||||
mockLocalBufferedReader, mockHdfsBufferedReader) {
|
||||
@Override
|
||||
public String getHashFromImageTag(String imageTag) {
|
||||
return manifestHash;
|
||||
}
|
||||
};
|
||||
mockImageTagToManifestPlugin.init(conf);
|
||||
|
||||
ImageManifest manifest = mockImageTagToManifestPlugin
|
||||
.getManifestFromImageTag("image");
|
||||
ImageManifest expectedManifest =
|
||||
mapper.readValue(manifestJson, ImageManifest.class);
|
||||
Assert.assertEquals(expectedManifest.toString(), manifest.toString());
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
|
@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Cont
|
|||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ResourceMappings;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceSet;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.runtime.ContainerExecutionException;
|
||||
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
|
@ -278,4 +279,14 @@ public class MockContainer implements Container {
|
|||
public List<LocalizationStatus> getLocalizationStatuses() {
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void setContainerRuntimeData(Object containerRuntimeData) {
|
||||
}
|
||||
|
||||
@Override
|
||||
public <T> T getContainerRuntimeData(Class<T> runtimeClazz)
|
||||
throws ContainerExecutionException {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,853 @@
|
|||
<!---
|
||||
Licensed under the Apache License, Version 2.0 (the "License");
|
||||
you may not use this file except in compliance with the License.
|
||||
You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License. See accompanying LICENSE file.
|
||||
-->
|
||||
|
||||
Launching Applications Using runC Containers
|
||||
==============================================
|
||||
|
||||
<!-- MACRO{toc|fromDepth=0|toDepth=1} -->
|
||||
|
||||
Security Warning
|
||||
---------------
|
||||
**IMPORTANT**
|
||||
|
||||
This feature is UNSTABLE. As this feature continues to evolve, APIs may not
|
||||
be maintained and functionality may be changed or removed.
|
||||
|
||||
Enabling this feature and running runC containers in your
|
||||
cluster has security implications. Given runC's integration with many powerful
|
||||
kernel features, it is imperative that administrators understand
|
||||
runC security before enabling this feature.
|
||||
|
||||
Overview
|
||||
--------
|
||||
|
||||
[runC](https://github.com/opencontainers/runc) is a CLI tool for spawning and
|
||||
running containers according to the Open Container Initiative (OCI)
|
||||
specification. runC was originally [spun out](https://www.docker.com/blog/runc/)
|
||||
of the original Docker infrastructure. Together with a rootfs mountpoint that
|
||||
is created via squashFS images, runC enables users to bundle an application
|
||||
together with its preferred execution environment to be executed on a
|
||||
target machine. For more information about the OCI, see their
|
||||
[website](https://www.opencontainers.org/).
|
||||
|
||||
The Linux Container Executor (LCE) allows the YARN NodeManager to launch YARN
|
||||
containers to run either directly on the host machine, inside of Docker
|
||||
containers, and now inside of runC containers.
|
||||
The application requesting the resources can specify for each
|
||||
container how it should be executed. The LCE also provides enhanced security
|
||||
and is required when deploying a secure cluster. When the LCE launches a YARN
|
||||
container to execute in a runC container, the application can specify the
|
||||
runC image to be used. These runC images can be built from Docker images.
|
||||
|
||||
runC containers provide a custom execution environment in which the
|
||||
application's code runs, isolated from the execution environment of the
|
||||
NodeManager and other applications. These containers can include special
|
||||
libraries needed by the application, and they can have different versions of
|
||||
native tools and libraries including Perl, Python, and Java. runC
|
||||
containers can even run a different flavor of Linux than what is running on the
|
||||
NodeManager.
|
||||
|
||||
runC for YARN provides both consistency (all YARN containers will have the
|
||||
same software environment) and isolation (no interference with whatever is
|
||||
installed on the physical machine).
|
||||
|
||||
runC support in the LCE is still evolving. To track progress and take a look
|
||||
at the runC design document, check out
|
||||
[YARN-9014](https://issues.apache.org/jira/browse/YARN-9014), the umbrella JIRA
|
||||
for runC support improvements.
|
||||
|
||||
Cluster Configuration
|
||||
---------------------
|
||||
|
||||
The LCE requires that container-executor binary be owned by root:hadoop and have
|
||||
6050 permissions. In order to launch runC containers, runC must be installed
|
||||
on all NodeManager hosts where runC containers will be launched.
|
||||
|
||||
The following properties should be set in yarn-site.xml:
|
||||
|
||||
```xml
|
||||
<configuration>
|
||||
<property>
|
||||
<name>yarn.nodemanager.container-executor.class</name>
|
||||
<value>org.apache.hadoop.yarn.server.nodemanager.LinuxContainerExecutor</value>
|
||||
<description>
|
||||
This is the container executor setting that ensures that all applications
|
||||
are started with the LinuxContainerExecutor.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>yarn.nodemanager.linux-container-executor.group</name>
|
||||
<value>hadoop</value>
|
||||
<description>
|
||||
The POSIX group of the NodeManager. It should match the setting in
|
||||
"container-executor.cfg". This configuration is required for validating
|
||||
the secure access of the container-executor binary.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>yarn.nodemanager.linux-container-executor.nonsecure-mode.limit-users</name>
|
||||
<value>false</value>
|
||||
<description>
|
||||
Whether all applications should be run as the NodeManager process' owner.
|
||||
When false, applications are launched instead as the application owner.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>yarn.nodemanager.runtime.linux.allowed-runtimes</name>
|
||||
<value>default,runc</value>
|
||||
<description>
|
||||
Comma separated list of runtimes that are allowed when using
|
||||
LinuxContainerExecutor. The allowed values are default, docker, runc, and
|
||||
javasandbox.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>yarn.nodemanager.runtime.linux.type</name>
|
||||
<value></value>
|
||||
<description>
|
||||
Optional. Sets the default container runtime to use.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The runC image tag to manifest plugin
|
||||
class to be used.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin</name>
|
||||
<value>org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.ImageTagToManifestPlugin</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The runC manifest to resources plugin class to
|
||||
be used.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.manifest-to-resources-plugin</name>
|
||||
<value>org.apache.hadoop.yarn.server.nodemanager.containermanager.linux.runtime.runc.HdfsManifestToResourcesPlugin</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The HDFS location under which the oci image manifests, layers,
|
||||
and configs directories exist.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-toplevel-dir</name>
|
||||
<value>/runc-root</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Target count of layer mounts that we should keep on disk
|
||||
at one time.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.layer-mounts-to-keep</name>
|
||||
<value>100</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The interval in seconds between executions of
|
||||
reaping layer mounts.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.layer-mounts-interval-secs</name>
|
||||
<value>600</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Image to be used if no other image is specified.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-name</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Allow or disallow privileged containers.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.privileged-containers.allowed</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The set of networks allowed when launching containers
|
||||
using the RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.allowed-container-networks</name>
|
||||
<value>host,none,bridge</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The set of runtimes allowed when launching containers
|
||||
using the RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.allowed-container-runtimes</name>
|
||||
<value>runc</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>ACL list for users allowed to run privileged
|
||||
containers.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.privileged-containers.acl</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Allow host pid namespace for runC containers.
|
||||
Use with care.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.host-pid-namespace.allowed</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The default list of read-only mounts to be bind-mounted
|
||||
into all runC containers that use RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.default-ro-mounts</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The default list of read-write mounts to be bind-mounted
|
||||
into all runC containers that use RuncContainerRuntime.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.default-rw-mounts</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>Path to the seccomp profile to use with runC
|
||||
containers</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.seccomp-profile</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The HDFS location where the runC image tag to hash
|
||||
file exists.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.hdfs-hash-file</name>
|
||||
<value>/runc-root/image-tag-to-hash</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The local file system location where the runC image tag
|
||||
to hash file exists.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.local-hash-file</name>
|
||||
<value></value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The interval in seconds between refreshing the hdfs image tag
|
||||
to hash cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.cache-refresh-interval-secs</name>
|
||||
<value>60</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The number of manifests to cache in the image tag
|
||||
to hash cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.image-tag-to-manifest-plugin.num-manifests-to-cache</name>
|
||||
<value>10</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The timeout value in seconds for the values in
|
||||
the stat cache.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-timeout-interval-secs</name>
|
||||
<value>360</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<description>The size of the stat cache which stores stats of the
|
||||
layers and config.</description>
|
||||
<name>yarn.nodemanager.runtime.linux.runc.hdfs-manifest-to-resources-plugin.stat-cache-size</name>
|
||||
<value>500</value>
|
||||
</property>
|
||||
</configuration>
|
||||
```
|
||||
|
||||
In addition, a container-executor.cfg file must exist and contain settings for
|
||||
the container executor. The file must be owned by root with permissions 0400.
|
||||
The format of the file is the standard Java properties file format, for example
|
||||
|
||||
`key=value`
|
||||
|
||||
The following properties are required to enable runC support:
|
||||
|
||||
|Configuration Name | Description |
|
||||
|:---- |:---- |
|
||||
| `yarn.nodemanager.linux-container-executor.group` | The Unix group of the NodeManager. It should match the yarn.nodemanager.linux-container-executor.group in the yarn-site.xml file. |
|
||||
|
||||
The container-executor.cfg must contain a section to determine the capabilities that containers
|
||||
are allowed. It contains the following properties:
|
||||
|
||||
|Configuration Name | Description |
|
||||
|:---- |:---- |
|
||||
| `module.enabled` | Must be "true" or "false" to enable or disable launching runC containers respectively. Default value is 0. |
|
||||
| `runc.binary` | The binary used to launch runC containers. /usr/bin/runc by default. |
|
||||
| `runc.run-root` | The directory where all runtime mounts and overlay mounts will be placed. |
|
||||
| `runc.allowed.ro-mounts` | Comma separated directories that containers are allowed to mount in read-only mode. By default, no directories are allowed to mounted. |
|
||||
| `runc.allowed.rw-mounts` | Comma separated directories that containers are allowed to mount in read-write mode. By default, no directories are allowed to mounted. |
|
||||
|
||||
Please note that if you wish to run runC containers that require access to the YARN local directories, you must add them to the runc.allowed.rw-mounts list.
|
||||
|
||||
In addition, containers are not permitted to mount any parent of the container-executor.cfg directory in read-write mode.
|
||||
|
||||
The following properties are optional:
|
||||
|
||||
|Configuration Name | Description |
|
||||
|:---- |:---- |
|
||||
| `min.user.id` | The minimum UID that is allowed to launch applications. The default is no minimum |
|
||||
| `banned.users` | A comma-separated list of usernames who should not be allowed to launch applications. The default setting is: yarn, mapred, hdfs, and bin. |
|
||||
| `allowed.system.users` | A comma-separated list of usernames who should be allowed to launch applications even if their UIDs are below the configured minimum. If a user appears in allowed.system.users and banned.users, the user will be considered banned. |
|
||||
| `feature.tc.enabled` | Must be "true" or "false". "false" means traffic control commands are disabled. "true" means traffic control commands are allowed. |
|
||||
| `feature.yarn.sysfs.enabled` | Must be "true" or "false". See YARN sysfs support for detail. The default setting is disabled. |
|
||||
|
||||
Part of a container-executor.cfg which allows runC containers to be launched is below:
|
||||
|
||||
```
|
||||
yarn.nodemanager.linux-container-executor.group=yarn
|
||||
[runc]
|
||||
module.enabled=true
|
||||
runc.binary=/usr/bin/runc
|
||||
runc.run-root=/run/yarn-container-executor
|
||||
runc.allowed.ro-mounts=/sys/fs/cgroup
|
||||
runc.allowed.rw-mounts=/var/hadoop/yarn/local-dir,/var/hadoop/yarn/log-dir
|
||||
|
||||
```
|
||||
|
||||
Image Requirements
|
||||
-------------------------
|
||||
|
||||
runC containers are run inside of images that are derived from Docker images.
|
||||
The docker images are transformed into a set of squashFS file images and
|
||||
uploaded into HDFS.
|
||||
In order to work with YARN, there are a few requirements for these
|
||||
Docker images.
|
||||
|
||||
1. The runC container will be explicitly launched with the application
|
||||
owner as the container user. If the application owner is not a valid user
|
||||
in the Docker image, the application will fail. The container user is specified
|
||||
by the user's UID. If the user's UID is different between the NodeManager host
|
||||
and the Docker image, the container may be launched as the wrong user or may
|
||||
fail to launch because the UID does not exist. See
|
||||
[User Management in runC Container](#user-management) section for more details.
|
||||
|
||||
2. The Docker image must have whatever is expected by the application
|
||||
in order to execute. In the case of Hadoop (MapReduce or Spark), the Docker
|
||||
image must contain the JRE and Hadoop libraries and have the necessary
|
||||
environment variables set: JAVA_HOME, HADOOP_COMMON_PATH, HADOOP_HDFS_HOME,
|
||||
HADOOP_MAPRED_HOME, HADOOP_YARN_HOME, and HADOOP_CONF_DIR. Note that the
|
||||
Java and Hadoop component versions available in the Docker image must be
|
||||
compatible with what's installed on the cluster and in any other Docker images
|
||||
being used for other tasks of the same job. Otherwise the Hadoop components
|
||||
started in the runC container may be unable to communicate with external
|
||||
Hadoop components.
|
||||
|
||||
3. `/bin/bash` must be available inside the image. This is generally true,
|
||||
however, tiny Docker images (eg. ones which use busybox for shell commands)
|
||||
might not have bash installed. In this case, the following error is
|
||||
displayed:
|
||||
|
||||
```
|
||||
Container id: container_1561638268473_0015_01_000002
|
||||
Exit code: 7
|
||||
Exception message: Launch container failed
|
||||
Shell error output: /usr/bin/docker-current: Error response from daemon: oci runtime error: container_linux.go:235: starting container process caused "exec: \"bash\": executable file not found in $PATH".
|
||||
Shell output: main : command provided 4
|
||||
```
|
||||
|
||||
4. `find` command must also be available inside the image. Not having
|
||||
`find` causes this error:
|
||||
|
||||
```
|
||||
Container exited with a non-zero exit code 127. Error file: prelaunch.err.
|
||||
Last 4096 bytes of prelaunch.err :
|
||||
/tmp/hadoop-systest/nm-local-dir/usercache/hadoopuser/appcache/application_1561638268473_0017/container_1561638268473_0017_01_000002/launch_container.sh: line 44: find: command not found
|
||||
```
|
||||
|
||||
If a Docker image has an entry point set, the entry point will be executed
|
||||
with the launch command of the container as its arguments.
|
||||
|
||||
The runC images that are derived from Docker images are localized onto the
|
||||
hosts where the runC containers will execute just like any other localized
|
||||
resource would be. Both MapReduce and Spark assume that
|
||||
tasks which take more that 10 minutes to report progress have stalled, so
|
||||
specifying a large image may cause the application to fail if the localization
|
||||
takes too long.
|
||||
|
||||
<a href="#docker-to-squash"></a>Transforming a Docker Image into a runC Image
|
||||
-----------------------------------
|
||||
|
||||
Every Docker image is comprised of 3 things:
|
||||
- A set of layers that create the file system.
|
||||
- A config file that holds information relative to the environment of the
|
||||
image.
|
||||
- A manifest that describes what layers and config are needed for that image.
|
||||
|
||||
Together, these 3 pieces combine to create an Open Container Initiative (OCI)
|
||||
compliant image. runC runs on top of OCI-compliant containers, but with a small
|
||||
twist. Each layer that the runC runtime uses is compressed into squashFS
|
||||
file system. The squashFS layers, along with the config, and manifest are
|
||||
uploaded to HDFS along with an `image-tag-to-hash mapping` file that
|
||||
describes the mapping between image tags and the manifest associated with
|
||||
that image. Getting this all setup is a complicated and tedious process.
|
||||
There is a patch on
|
||||
[YARN-9564](https://issues.apache.org/jira/browse/YARN-9564) that contains
|
||||
an unofficial Python script named `docker-to-squash.py`
|
||||
to help out with the conversion process. This tool will
|
||||
take in a Docker image as input, convert all of its layers
|
||||
into squashFS file systems, and upload the squashFS layers, config, and manifest
|
||||
to HDFS underneath the runc-root. It will also create or update the
|
||||
`image-tag-to-hash` mapping file. Below is an example invocation of the script
|
||||
to upload an image named `centos:latest` to HDFS with the
|
||||
runC image name `centos`
|
||||
|
||||
```
|
||||
[user@foobar sbin]$ pwd
|
||||
/home/user/hadoop/hadoop-dist/target/hadoop-3.3.0-SNAPSHOT/sbin
|
||||
[user@foobar sbin]$ ls
|
||||
distribute-exclude.sh hadoop-daemons.sh refresh-namenodes.sh start-dfs.cmd start-yarn.sh stop-dfs.cmd stop-yarn.sh
|
||||
docker_to_squash.py httpfs.sh start-all.cmd start-dfs.sh stop-all.cmd stop-dfs.sh workers.sh
|
||||
FederationStateStore kms.sh start-all.sh start-secure-dns.sh stop-all.sh stop-secure-dns.sh yarn-daemon.sh
|
||||
hadoop-daemon.sh mr-jobhistory-daemon.sh start-balancer.sh start-yarn.cmd stop-balancer.sh stop-yarn.cmd yarn-daemons.sh
|
||||
[user@foobar sbin]$ hadoop fs -ls /
|
||||
Found 3 items
|
||||
drwxrwx--- - user supergroup 0 2019-08-07 19:35 /home
|
||||
drwx------ - user supergroup 0 2019-08-07 19:35 /tmp
|
||||
drwx------ - user supergroup 0 2019-08-07 19:35 /user
|
||||
[user@foobar sbin]$ ./docker_to_squash.py --working-dir /tmp --log=DEBUG pull-build-push-update centos:latest,centos
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'version']
|
||||
DEBUG: command: ['skopeo', '-v']
|
||||
DEBUG: command: ['mksquashfs', '-version']
|
||||
DEBUG: args: Namespace(LOG_LEVEL='DEBUG', check_magic_file=False, force=False, func=<function pull_build_push_update at 0x7fe6974cd9b0>, hadoop_prefix='/hadoop-2.8.6-SNAPSHOT', hdfs_root='/runc-root', image_tag_to_hash='image-tag-to-hash', images_and_tags=['centos:latest,centos'], magic_file='etc/dockerfile-version', pull_format='docker', replication=1, skopeo_format='dir', sub_command='pull-build-push-update', working_dir='/tmp')
|
||||
DEBUG: extra: []
|
||||
DEBUG: image-tag-to-hash: image-tag-to-hash
|
||||
DEBUG: LOG_LEVEL: DEBUG
|
||||
DEBUG: HADOOP_BIN_DIR: /hadoop-2.8.6-SNAPSHOT/bin
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root']
|
||||
ls: `/runc-root': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-mkdir', '/runc-root']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '755', '/runc-root']
|
||||
DEBUG: Setting up squashfs dirs: ['/runc-root/layers', '/runc-root/config', '/runc-root/manifests']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/layers']
|
||||
ls: `/runc-root/layers': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-mkdir', '/runc-root/layers']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/layers']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '755', '/runc-root/layers']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/config']
|
||||
ls: `/runc-root/config': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-mkdir', '/runc-root/config']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/config']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '755', '/runc-root/config']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/manifests']
|
||||
ls: `/runc-root/manifests': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-mkdir', '/runc-root/manifests']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/manifests']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '755', '/runc-root/manifests']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/image-tag-to-hash']
|
||||
ls: `/runc-root/image-tag-to-hash': No such file or directory
|
||||
INFO: Working on image centos:latest with tags ['centos']
|
||||
DEBUG: command: ['skopeo', 'inspect', '--raw', 'docker://centos:latest']
|
||||
DEBUG: skopeo inspect --raw returned a list of manifests
|
||||
DEBUG: amd64 manifest sha is: sha256:ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66
|
||||
DEBUG: command: ['skopeo', 'inspect', '--raw', u'docker://centos@sha256:ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
INFO: manifest: {u'layers': [{u'mediaType': u'application/vnd.docker.image.rootfs.diff.tar.gzip', u'digest': u'sha256:8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df', u'size': 75403831}], u'schemaVersion': 2, u'config': {u'mediaType': u'application/vnd.docker.container.image.v1+json', u'digest': u'sha256:9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1', u'size': 2182}, u'mediaType': u'application/vnd.docker.distribution.manifest.v2+json'}
|
||||
INFO: manifest: {u'layers': [{u'mediaType': u'application/vnd.docker.image.rootfs.diff.tar.gzip', u'digest': u'sha256:8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df', u'size': 75403831}], u'schemaVersion': 2, u'config': {u'mediaType': u'application/vnd.docker.container.image.v1+json', u'digest': u'sha256:9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1', u'size': 2182}, u'mediaType': u'application/vnd.docker.distribution.manifest.v2+json'}
|
||||
DEBUG: Layers: [u'8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df']
|
||||
DEBUG: Config: 9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1
|
||||
DEBUG: hash_to_tags is null. Not removing tag centos
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
ls: `/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', u'/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1']
|
||||
ls: `/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
ls: `/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh': No such file or directory
|
||||
DEBUG: skopeo_dir: /tmp/docker-to-squash/centos:latest
|
||||
INFO: Pulling image: centos:latest
|
||||
DEBUG: command: ['skopeo', 'copy', 'docker://centos:latest', 'dir:/tmp/docker-to-squash/centos:latest']
|
||||
INFO: Squashifying and uploading layer: 8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
ls: `/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh': No such file or directory
|
||||
DEBUG: command: ['sudo', 'tar', '-C', u'/tmp/docker-to-squash/expand_archive_8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df', '--xattrs', "--xattrs-include='*'", '-xzf', u'/tmp/docker-to-squash/centos:latest/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df']
|
||||
DEBUG: command: ['sudo', 'find', u'/tmp/docker-to-squash/expand_archive_8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df', '-name', '.wh.*']
|
||||
DEBUG: command: ['sudo', 'mksquashfs', u'/tmp/docker-to-squash/expand_archive_8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df', u'/tmp/docker-to-squash/centos:latest/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
DEBUG: command: ['sudo', 'rm', '-rf', u'/tmp/docker-to-squash/expand_archive_8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
ls: `/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-put', u'/tmp/docker-to-squash/centos:latest/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-setrep', '1', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '444', u'/runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh']
|
||||
INFO: Uploaded file /runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh with replication 1 and permissions 444
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', u'/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1']
|
||||
ls: `/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-put', u'/tmp/docker-to-squash/centos:latest/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1', u'/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-setrep', '1', u'/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '444', u'/runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1']
|
||||
INFO: Uploaded file /runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1 with replication 1 and permissions 444
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-ls', '/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
ls: `/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66': No such file or directory
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-put', '/tmp/docker-to-squash/centos:latest/manifest.json', '/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-setrep', '1', '/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '444', '/runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66']
|
||||
INFO: Uploaded file /runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66 with replication 1 and permissions 444
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-put', '-f', '/tmp/docker-to-squash/image-tag-to-hash', '/runc-root/image-tag-to-hash']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-setrep', '1', '/runc-root/image-tag-to-hash']
|
||||
DEBUG: command: ['/hadoop-2.8.6-SNAPSHOT/bin/hadoop', 'fs', '-chmod', '444', '/runc-root/image-tag-to-hash']
|
||||
DEBUG: command: ['sudo', 'rm', '-rf', '/tmp/docker-to-squash']
|
||||
[user@foobar sbin]$ hadoop fs -ls /
|
||||
Found 4 items
|
||||
drwxrwx--- - user supergroup 0 2019-08-07 19:35 /home
|
||||
drwxr-xr-x - user supergroup 0 2019-08-08 22:38 /runc-root
|
||||
drwx------ - user supergroup 0 2019-08-07 19:35 /tmp
|
||||
drwx------ - user supergroup 0 2019-08-07 19:35 /user
|
||||
[user@foobar sbin]$ hadoop fs -ls /runc-root/*
|
||||
Found 1 items
|
||||
-r--r--r-- 1 user supergroup 2182 2019-08-08 22:38 /runc-root/config/9f38484d220fa527b1fb19747638497179500a1bed8bf0498eb788229229e6e1
|
||||
-r--r--r-- 1 user supergroup 86 2019-08-08 22:38 /runc-root/image-tag-to-hash
|
||||
Found 1 items
|
||||
-r--r--r-- 1 user supergroup 73625600 2019-08-08 22:38 /runc-root/layers/8ba884070f611d31cb2c42eddb691319dc9facf5e0ec67672fcfa135181ab3df.sqsh
|
||||
Found 1 items
|
||||
-r--r--r-- 1 user supergroup 529 2019-08-08 22:38 /runc-root/manifests/ca58fe458b8d94bc6e3072f1cfbd334855858e05e1fd633aa07cf7f82b048e66
|
||||
```
|
||||
|
||||
|
||||
Application Submission
|
||||
----------------------
|
||||
|
||||
Before attempting to launch a runC container, make sure that the LCE
|
||||
configuration is working for applications requesting regular YARN containers.
|
||||
If after enabling the LCE one or more NodeManagers fail to start, the cause is
|
||||
most likely that the ownership and/or permissions on the container-executor
|
||||
binary are incorrect. Check the logs to confirm.
|
||||
|
||||
In order to run an application in a runC container, set the following
|
||||
environment variables in the application's environment:
|
||||
|
||||
| Environment Variable Name | Description |
|
||||
| :------------------------ | :---------- |
|
||||
| `YARN_CONTAINER_RUNTIME_TYPE` | Determines whether an application will be launched in a runC container. If the value is "runc", the application will be launched in a runC container. Otherwise a regular process tree container will be used. |
|
||||
| `YARN_CONTAINER_RUNTIME_RUNC_IMAGE` | Names which image will be used to launch the runC container. |
|
||||
| `YARN_CONTAINER_RUNTIME_RUNC_CONTAINER_HOSTNAME` | Sets the hostname to be used by the runC container. |
|
||||
| `YARN_CONTAINER_RUNTIME_RUNC_MOUNTS` | Adds additional volume mounts to the runC container. The value of the environment variable should be a comma-separated list of mounts. All such mounts must be given as "source:dest:mode" and the mode must be "ro" (read-only) or "rw" (read-write) to specify the type of access being requested. If neither is specified, read-write will be assumed. The requested mounts will be validated by container-executor based on the values set in container-executor.cfg for runc.allowed.ro-mounts and runc.allowed.rw-mounts. |
|
||||
|
||||
The first two are required. The remainder can be set as needed. While
|
||||
controlling the container type through environment variables is somewhat less
|
||||
than ideal, it allows applications with no awareness of YARN's runC support
|
||||
(such as MapReduce and Spark) to nonetheless take advantage of it through their
|
||||
support for configuring the application environment.
|
||||
|
||||
**Note** The runtime will not work if you mount anything onto /tmp or /var/tmp
|
||||
in the container.
|
||||
|
||||
Once an application has been submitted to be launched in a runC container,
|
||||
the application will behave exactly as any other YARN application. Logs will be
|
||||
aggregated and stored in the relevant history server. The application life cycle
|
||||
will be the same as for a non-runC application.
|
||||
|
||||
Using runC Bind Mounted Volumes
|
||||
---------------------------------
|
||||
|
||||
**WARNING** Care should be taken when enabling this feature. Enabling access to
|
||||
directories such as, but not limited to, /, /etc, /run, or /home is not
|
||||
advisable and can result in containers negatively impacting the host or leaking
|
||||
sensitive information. **WARNING**
|
||||
|
||||
Files and directories from the host are commonly needed within the runC
|
||||
containers, which runC provides through mounts into the container.
|
||||
Examples include localized resources, Apache Hadoop binaries, and sockets.
|
||||
|
||||
In order to mount anything into the container, the following must be configured.
|
||||
|
||||
* The administrator must define the volume whitelist in container-executor.cfg by setting `runc.allowed.ro-mounts` and `runc.allowed.rw-mounts` to the list of parent directories that are allowed to be mounted.
|
||||
|
||||
The administrator supplied whitelist is defined as a comma separated list of
|
||||
directories that are allowed to be mounted into containers. The source directory
|
||||
supplied by the user must either match or be a child of the specified
|
||||
directory.
|
||||
|
||||
The user supplied mount list is defined as a comma separated list in the form
|
||||
*source*:*destination* or *source*:*destination*:*mode*. The source is the file
|
||||
or directory on the host. The destination is the path within the container
|
||||
where the source will be bind mounted. The mode defines the mode the user
|
||||
expects for the mount, which can be ro (read-only) or rw (read-write). If not
|
||||
specified, rw is assumed. The mode may also include a bind propagation option
|
||||
(shared, rshared, slave, rslave, private, or rprivate). In that case, the
|
||||
mode should be of the form *option*, rw+*option*, or ro+*option*.
|
||||
|
||||
The following example outlines how to use this feature to mount the commonly
|
||||
needed /sys/fs/cgroup directory into the container running on YARN.
|
||||
|
||||
The administrator sets runc.allowed.ro-mounts in container-executor.cfg to
|
||||
"/sys/fs/cgroup". Applications can now request that "/sys/fs/cgroup" be mounted
|
||||
from the host into the container in read-only mode.
|
||||
|
||||
The Nodemanager has the option to setup a default list of read-only or
|
||||
read-write mounts to be added to the container via
|
||||
`yarn.nodemanager.runtime.linux.runc.default-ro-mount`"
|
||||
and `yarn.nodemanager.runtime.linux.runc.default-rw-mounts` in yarn-site.xml.
|
||||
In this example, `yarn.nodemanager.runtime.linux.runc.default-ro-mounts`
|
||||
would be set to `/sys/fs/cgroup:/sys/fs/cgroup`.
|
||||
|
||||
<a href="#user-management"></a>User Management in runC Container
|
||||
-----------------------------------
|
||||
|
||||
YARN's runC container support launches container processes using the uid:gid
|
||||
identity of the user, as defined on the NodeManager host. User and group name
|
||||
mismatches between the NodeManager host and container can lead to permission
|
||||
issues, failed container launches, or even security holes. Centralizing user and
|
||||
group management for both hosts and containers greatly reduces these risks. When
|
||||
running containerized applications on YARN, it is necessary to understand which
|
||||
uid:gid pair will be used to launch the container's process.
|
||||
|
||||
As an example of what is meant by uid:gid pair, consider the following. By
|
||||
default, in non-secure mode, YARN will launch processes as the user `nobody`
|
||||
(see the table at the bottom of
|
||||
[Using CGroups with YARN](./NodeManagerCgroups.html) for how the run as user is
|
||||
determined in non-secure mode). On CentOS based systems, the `nobody` user's uid
|
||||
is `99` and the `nobody` group is `99`. As a result, YARN will invoke runC
|
||||
with uid `99` and gid `99`. If the `nobody` user does not have the uid `99` in the
|
||||
container, the launch may fail or have unexpected results.
|
||||
|
||||
There are many ways to address user and group management. runC, by default,
|
||||
will authenticate users against `/etc/passwd` (and `/etc/shadow`) within the
|
||||
container. Using the default `/etc/passwd` supplied in the runC image is
|
||||
unlikely to contain the appropriate user entries and will result in launch
|
||||
failures. It is highly recommended to centralize user and group management.
|
||||
Several approaches to user and group management are outlined below.
|
||||
|
||||
### Static user management
|
||||
|
||||
The most basic approach to managing user and groups is to modify the user and
|
||||
group within the runC image. This approach is only viable in non-secure mode
|
||||
where all container processes will be launched as a single known user, for
|
||||
instance `nobody`. In this case, the only requirement is that the uid:gid pair
|
||||
of the nobody user and group must match between the host and container. On a
|
||||
CentOS based system, this means that the nobody user in the container needs the
|
||||
UID `99` and the nobody group in the container needs GID `99`.
|
||||
|
||||
One approach to change the UID and GID is by leveraging `usermod` and
|
||||
`groupmod`. The following sets the correct UID and GID for the nobody
|
||||
user/group.
|
||||
```
|
||||
usermod -u 99 nobody
|
||||
groupmod -g 99 nobody
|
||||
```
|
||||
|
||||
This approach is not recommended beyond testing given the inflexibility to add
|
||||
users.
|
||||
|
||||
### Bind mounting
|
||||
|
||||
When organizations already have automation in place to create local users on
|
||||
each system, it may be appropriate to bind mount /etc/passwd and /etc/group
|
||||
into the container as an alternative to modifying the container image directly.
|
||||
To enable the ability to bind mount /etc/passwd and /etc/group, update
|
||||
`runc.allowed.ro-mounts` in `container-executor.cfg` to include those paths.
|
||||
For this to work on runC,
|
||||
"yarn.nodemanager.runtime.linux.runc.default-ro-mounts" will need to include
|
||||
`/etc/passwd:/etc/passwd:ro` and `/etc/group:/etc/group:ro`.
|
||||
|
||||
There are several challenges with this bind mount approach that need to be
|
||||
considered.
|
||||
|
||||
1. Any users and groups defined in the image will be overwritten by the host's users and groups
|
||||
2. No users and groups can be added once the container is started, as /etc/passwd and /etc/group are immutible in the container. Do not mount these read-write as it can render the host inoperable.
|
||||
|
||||
This approach is not recommended beyond testing given the inflexibility to
|
||||
modify running containers.
|
||||
|
||||
### SSSD
|
||||
|
||||
An alternative approach that allows for centrally managing users and groups is
|
||||
SSSD. System Security Services Daemon (SSSD) provides access to different
|
||||
identity and authentication providers, such as LDAP or Active Directory.
|
||||
|
||||
The traditional schema for Linux authentication is as follows:
|
||||
```
|
||||
application -> libpam -> pam_authenticate -> pam_unix.so -> /etc/passwd
|
||||
```
|
||||
|
||||
If we use SSSD for user lookup, it becomes:
|
||||
```
|
||||
application -> libpam -> pam_authenticate -> pam_sss.so -> SSSD -> pam_unix.so -> /etc/passwd
|
||||
```
|
||||
|
||||
We can bind-mount the UNIX sockets SSSD communicates over into the container.
|
||||
This will allow the SSSD client side libraries to authenticate against the SSSD
|
||||
running on the host. As a result, user information does not need to exist in
|
||||
/etc/passwd of the docker image and will instead be serviced by SSSD.
|
||||
|
||||
Step by step configuration for host and container:
|
||||
|
||||
1. Host config
|
||||
|
||||
- Install packages
|
||||
```
|
||||
# yum -y install sssd-common sssd-proxy
|
||||
```
|
||||
- create a PAM service for the container.
|
||||
```
|
||||
# cat /etc/pam.d/sss_proxy
|
||||
auth required pam_unix.so
|
||||
account required pam_unix.so
|
||||
password required pam_unix.so
|
||||
session required pam_unix.so
|
||||
```
|
||||
- create SSSD config file, /etc/sssd/sssd.conf
|
||||
Please note that the permissions must be 0600 and the file must be owned by root:root.
|
||||
```
|
||||
# cat /etc/sssd/sssd/conf
|
||||
[sssd]
|
||||
services = nss,pam
|
||||
config_file_version = 2
|
||||
domains = proxy
|
||||
[nss]
|
||||
[pam]
|
||||
[domain/proxy]
|
||||
id_provider = proxy
|
||||
proxy_lib_name = files
|
||||
proxy_pam_target = sss_proxy
|
||||
```
|
||||
- start sssd
|
||||
```
|
||||
# systemctl start sssd
|
||||
```
|
||||
- verify a user can be retrieved with sssd
|
||||
```
|
||||
# getent passwd -s sss localuser
|
||||
```
|
||||
|
||||
2. Container setup
|
||||
|
||||
It's important to bind-mount the /var/lib/sss/pipes directory from the host to the container since SSSD UNIX sockets are located there.
|
||||
```
|
||||
-v /var/lib/sss/pipes:/var/lib/sss/pipes:rw
|
||||
```
|
||||
|
||||
3. Container config
|
||||
|
||||
All the steps below should be executed on the container itself.
|
||||
|
||||
- Install only the sss client libraries
|
||||
```
|
||||
# yum -y install sssd-client
|
||||
```
|
||||
|
||||
- make sure sss is configured for passwd and group databases in
|
||||
```
|
||||
/etc/nsswitch.conf
|
||||
```
|
||||
|
||||
- configure the PAM service that the application uses to call into SSSD
|
||||
```
|
||||
# cat /etc/pam.d/system-auth
|
||||
#%PAM-1.0
|
||||
# This file is auto-generated.
|
||||
# User changes will be destroyed the next time authconfig is run.
|
||||
auth required pam_env.so
|
||||
auth sufficient pam_unix.so try_first_pass nullok
|
||||
auth sufficient pam_sss.so forward_pass
|
||||
auth required pam_deny.so
|
||||
|
||||
account required pam_unix.so
|
||||
account [default=bad success=ok user_unknown=ignore] pam_sss.so
|
||||
account required pam_permit.so
|
||||
|
||||
password requisite pam_pwquality.so try_first_pass local_users_only retry=3 authtok_type=
|
||||
password sufficient pam_unix.so try_first_pass use_authtok nullok sha512 shadow
|
||||
password sufficient pam_sss.so use_authtok
|
||||
password required pam_deny.so
|
||||
|
||||
session optional pam_keyinit.so revoke
|
||||
session required pam_limits.so
|
||||
-session optional pam_systemd.so
|
||||
session [success=1 default=ignore] pam_succeed_if.so service in crond quiet use_uid
|
||||
session required pam_unix.so
|
||||
session optional pam_sss.so
|
||||
```
|
||||
|
||||
- Save the docker image and use the docker image as base image for your applications.
|
||||
|
||||
- test the docker image launched in YARN environment.
|
||||
```
|
||||
$ id
|
||||
uid=5000(localuser) gid=5000(localuser) groups=5000(localuser),1337(hadoop)
|
||||
```
|
||||
|
||||
Example: MapReduce
|
||||
------------------
|
||||
|
||||
This example assumes that Hadoop is installed to `/usr/local/hadoop`.
|
||||
|
||||
You will also need to squashify a Docker image and upload it to HDFS before
|
||||
you can run with that image.
|
||||
See [Transforming a Docker Image into a runC Image](#docker-to-squash)
|
||||
for instructions on how to transform a Docker image into a image that
|
||||
runC can use.
|
||||
For this example, we will assume that you have
|
||||
done with that an image named `hadoop-image`.
|
||||
|
||||
Additionally, `runc.allowed.ro-mounts` in `container-executor.cfg` has been
|
||||
updated to include the directories: `/usr/local/hadoop,/etc/passwd,/etc/group`.
|
||||
|
||||
To submit the pi job to run in runC containers, run the following commands:
|
||||
|
||||
```
|
||||
HADOOP_HOME=/usr/local/hadoop
|
||||
YARN_EXAMPLES_JAR=$HADOOP_HOME/share/hadoop/mapreduce/hadoop-mapreduce-examples-*.jar
|
||||
MOUNTS="$HADOOP_HOME:$HADOOP_HOME:ro,/etc/passwd:/etc/passwd:ro,/etc/group:/etc/group:ro"
|
||||
IMAGE_ID="hadoop-image"
|
||||
|
||||
export YARN_CONTAINER_RUNTIME_TYPE=runc
|
||||
export YARN_CONTAINER_RUNTIME_RUNC_IMAGE=$IMAGE_ID
|
||||
export YARN_CONTAINER_RUNTIME_RUNC_MOUNTS=$MOUNTS
|
||||
|
||||
yarn jar $YARN_EXAMPLES_JAR pi \
|
||||
-Dmapreduce.map.env.YARN_CONTAINER_RUNTIME_TYPE=runc \
|
||||
-Dmapreduce.map.env.YARN_CONTAINER_RUNTIME_RUNC_MOUNTS=$MOUNTS \
|
||||
-Dmapreduce.map.env.YARN_CONTAINER_RUNTIME_RUNC_IMAGE=$IMAGE_ID \
|
||||
-Dmapreduce.reduce.env.YARN_CONTAINER_RUNTIME_TYPE=runc \
|
||||
-Dmapreduce.reduce.env.YARN_CONTAINER_RUNTIME_RUNC_MOUNTS=$MOUNTS \
|
||||
-Dmapreduce.reduce.env.YARN_CONTAINER_RUNTIME_RUNC_IMAGE=$IMAGE_ID \
|
||||
1 40000
|
||||
```
|
||||
|
||||
Note that the application master, map tasks, and reduce tasks are configured
|
||||
independently. In this example, we are using the `hadoop-image` image for all three.
|
||||
|
||||
Example: Spark
|
||||
--------------
|
||||
|
||||
This example assumes that Hadoop is installed to `/usr/local/hadoop` and Spark
|
||||
is installed to `/usr/local/spark`.
|
||||
|
||||
You will also need to squashify a Docker image and upload it to HDFS before
|
||||
you can run with that image.
|
||||
See [Transforming a Docker Image into a runC Image](#docker-to-squash)
|
||||
for instructions on how to transform a Docker image into a image that
|
||||
runC can use.
|
||||
For this example, we will assume that you have
|
||||
done with that an image named `hadoop-image`.
|
||||
|
||||
Additionally, `runc.allowed.ro-mounts` in `container-executor.cfg` has been
|
||||
updated to include the directories: `/usr/local/hadoop,/etc/passwd,/etc/group`.
|
||||
|
||||
To run a Spark shell in runC containers, run the following command:
|
||||
|
||||
```
|
||||
HADOOP_HOME=/usr/local/hadoop
|
||||
SPARK_HOME=/usr/local/spark
|
||||
MOUNTS="$HADOOP_HOME:$HADOOP_HOME:ro,/etc/passwd:/etc/passwd:ro,/etc/group:/etc/group:ro"
|
||||
IMAGE_ID="hadoop-image"
|
||||
|
||||
$SPARK_HOME/bin/spark-shell --master yarn \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_TYPE=runc \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_RUNC_IMAGE=$IMAGE_ID \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_RUNC_MOUNTS=$MOUNTS \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_TYPE=runc \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_RUNC_IMAGE=$IMAGE_ID \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_RUNC_MOUNTS=$MOUNTS
|
||||
```
|
||||
|
||||
Note that the application master and executors are configured
|
||||
independently. In this example, we are using the `hadoop-image` image for both.
|
Loading…
Reference in New Issue