YARN-11255. Support loading alternative docker client config from system environment (#4884)
Co-authored-by: Ashutosh Gupta <ashugpt@amazon.com>
This commit is contained in:
parent
4c5a7cc6fc
commit
917aef75fc
@ -40,8 +40,10 @@
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.resourceplugin.ResourcePlugin;
|
||||
import org.apache.hadoop.yarn.server.nodemanager.containermanager.volume.csi.ContainerVolumePublisher;
|
||||
import org.apache.hadoop.yarn.util.DockerClientConfigHandler;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
@ -215,6 +217,9 @@ public class DockerLinuxContainerRuntime extends OCIContainerRuntime {
|
||||
public static final String ENV_DOCKER_CONTAINER_IMAGE =
|
||||
"YARN_CONTAINER_RUNTIME_DOCKER_IMAGE";
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_DOCKER_CONTAINER_CLIENT_CONFIG =
|
||||
"YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG";
|
||||
@InterfaceAudience.Private
|
||||
public static final String ENV_DOCKER_CONTAINER_NETWORK =
|
||||
"YARN_CONTAINER_RUNTIME_DOCKER_CONTAINER_NETWORK";
|
||||
@InterfaceAudience.Private
|
||||
@ -595,6 +600,7 @@ public void launchContainer(ContainerRuntimeContext ctx)
|
||||
boolean serviceMode = Boolean.parseBoolean(environment.get(
|
||||
ENV_DOCKER_CONTAINER_DOCKER_SERVICE_MODE));
|
||||
boolean useEntryPoint = serviceMode || checkUseEntryPoint(environment);
|
||||
String clientConfig = environment.get(ENV_DOCKER_CONTAINER_CLIENT_CONFIG);
|
||||
|
||||
if (imageName == null || imageName.isEmpty()) {
|
||||
imageName = defaultImageName;
|
||||
@ -796,7 +802,8 @@ public void launchContainer(ContainerRuntimeContext ctx)
|
||||
runCommand.setPrivileged();
|
||||
}
|
||||
|
||||
addDockerClientConfigToRunCommand(ctx, runCommand);
|
||||
addDockerClientConfigToRunCommand(ctx, runCommand,
|
||||
getAdditionalDockerClientCredentials(clientConfig, containerIdStr));
|
||||
|
||||
String resourcesOpts = ctx.getExecutionAttribute(RESOURCES_OPTIONS);
|
||||
|
||||
@ -891,6 +898,22 @@ public void launchContainer(ContainerRuntimeContext ctx)
|
||||
}
|
||||
}
|
||||
|
||||
private Credentials getAdditionalDockerClientCredentials(String clientConfig,
|
||||
String containerIdStr) {
|
||||
Credentials additionalDockerCredentials = null;
|
||||
if (clientConfig != null && !clientConfig.isEmpty()) {
|
||||
try {
|
||||
additionalDockerCredentials =
|
||||
DockerClientConfigHandler.readCredentialsFromConfigFile(new Path(clientConfig), conf,
|
||||
containerIdStr);
|
||||
} catch (IOException e) {
|
||||
throw new RuntimeException(
|
||||
"Fail to read additional docker client config file from " + clientConfig);
|
||||
}
|
||||
}
|
||||
return additionalDockerCredentials;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void relaunchContainer(ContainerRuntimeContext ctx)
|
||||
throws ContainerExecutionException {
|
||||
@ -1366,36 +1389,41 @@ private void handleContainerRemove(String containerId,
|
||||
}
|
||||
|
||||
private void addDockerClientConfigToRunCommand(ContainerRuntimeContext ctx,
|
||||
DockerRunCommand dockerRunCommand) throws ContainerExecutionException {
|
||||
DockerRunCommand dockerRunCommand, Credentials additionDockerCredentials)
|
||||
throws ContainerExecutionException {
|
||||
ByteBuffer tokens = ctx.getContainer().getLaunchContext().getTokens();
|
||||
Credentials credentials;
|
||||
Credentials credentials = new Credentials();
|
||||
if (tokens != null) {
|
||||
tokens.rewind();
|
||||
if (tokens.hasRemaining()) {
|
||||
try {
|
||||
credentials = DockerClientConfigHandler
|
||||
.getCredentialsFromTokensByteBuffer(tokens);
|
||||
credentials.addAll(DockerClientConfigHandler
|
||||
.getCredentialsFromTokensByteBuffer(tokens));
|
||||
} catch (IOException e) {
|
||||
throw new ContainerExecutionException("Unable to read tokens.");
|
||||
}
|
||||
if (credentials.numberOfTokens() > 0) {
|
||||
Path nmPrivateDir =
|
||||
ctx.getExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH)
|
||||
.getParent();
|
||||
File dockerConfigPath = new File(nmPrivateDir + "/config.json");
|
||||
try {
|
||||
if (DockerClientConfigHandler
|
||||
.writeDockerCredentialsToPath(dockerConfigPath, credentials)) {
|
||||
dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
|
||||
}
|
||||
} catch (IOException e) {
|
||||
throw new ContainerExecutionException(
|
||||
"Unable to write Docker client credentials to "
|
||||
+ dockerConfigPath);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (additionDockerCredentials != null) {
|
||||
credentials.addAll(additionDockerCredentials);
|
||||
}
|
||||
|
||||
if (credentials.numberOfTokens() > 0) {
|
||||
Path nmPrivateDir =
|
||||
ctx.getExecutionAttribute(NM_PRIVATE_CONTAINER_SCRIPT_PATH)
|
||||
.getParent();
|
||||
File dockerConfigPath = new File(nmPrivateDir + "/config.json");
|
||||
try {
|
||||
DockerClientConfigHandler
|
||||
.writeDockerCredentialsToPath(dockerConfigPath, credentials);
|
||||
} catch (IOException e) {
|
||||
throw new ContainerExecutionException(
|
||||
"Unable to write Docker client credentials to "
|
||||
+ dockerConfigPath);
|
||||
}
|
||||
dockerRunCommand.setClientConfigDir(dockerConfigPath.getParent());
|
||||
}
|
||||
}
|
||||
|
||||
boolean getHostPidNamespaceEnabled() {
|
||||
|
@ -2383,24 +2383,37 @@ public void testDockerCapabilities() throws ContainerExecutionException {
|
||||
|
||||
@Test
|
||||
public void testLaunchContainerWithDockerTokens()
|
||||
throws ContainerExecutionException, PrivilegedOperationException,
|
||||
IOException {
|
||||
// Write the JSOn to a temp file.
|
||||
File file = File.createTempFile("docker-client-config", "runtime-test");
|
||||
file.deleteOnExit();
|
||||
BufferedWriter bw = new BufferedWriter(new FileWriter(file));
|
||||
bw.write(TestDockerClientConfigHandler.JSON);
|
||||
bw.close();
|
||||
throws ContainerExecutionException, PrivilegedOperationException, IOException {
|
||||
|
||||
// Get the credentials object with the Tokens.
|
||||
Credentials credentials = DockerClientConfigHandler
|
||||
.readCredentialsFromConfigFile(new Path(file.toURI()), conf, appId);
|
||||
Credentials credentials = DockerClientConfigHandler.readCredentialsFromConfigFile(
|
||||
new Path(getDockerClientConfigFile().toURI()), conf, appId);
|
||||
DataOutputBuffer dob = new DataOutputBuffer();
|
||||
credentials.writeTokenStorageToStream(dob);
|
||||
ByteBuffer tokens = ByteBuffer.wrap(dob.getData(), 0, dob.getLength());
|
||||
|
||||
// Configure the runtime and launch the container
|
||||
when(context.getTokens()).thenReturn(tokens);
|
||||
testLaunchContainer(tokens, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLaunchContainerWithAdditionalDockerClientConfig()
|
||||
throws ContainerExecutionException, PrivilegedOperationException, IOException {
|
||||
testLaunchContainer(null, getDockerClientConfigFile());
|
||||
}
|
||||
|
||||
public void testLaunchContainer(ByteBuffer tokens, File dockerConfigFile)
|
||||
throws ContainerExecutionException, PrivilegedOperationException,
|
||||
IOException {
|
||||
if (dockerConfigFile != null) {
|
||||
// load the docker client config file from system environment
|
||||
env.put(DockerLinuxContainerRuntime.ENV_DOCKER_CONTAINER_CLIENT_CONFIG,
|
||||
dockerConfigFile.getPath());
|
||||
}
|
||||
|
||||
if (tokens != null) {
|
||||
// Configure the runtime and launch the container
|
||||
when(context.getTokens()).thenReturn(tokens);
|
||||
}
|
||||
DockerLinuxContainerRuntime runtime =
|
||||
new DockerLinuxContainerRuntime(mockExecutor, mockCGroupsHandler);
|
||||
runtime.initialize(conf, nmContext);
|
||||
@ -2486,6 +2499,16 @@ public void testLaunchContainerWithDockerTokens()
|
||||
dockerCommands.get(counter++));
|
||||
}
|
||||
|
||||
private File getDockerClientConfigFile() throws IOException {
|
||||
// Write the JSOn to a temp file.
|
||||
File file = File.createTempFile("docker-client-config", "runtime-test");
|
||||
file.deleteOnExit();
|
||||
BufferedWriter bw = new BufferedWriter(new FileWriter(file));
|
||||
bw.write(TestDockerClientConfigHandler.JSON);
|
||||
bw.close();
|
||||
return file;
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDockerContainerRelaunch()
|
||||
throws ContainerExecutionException, PrivilegedOperationException,
|
||||
|
@ -438,7 +438,7 @@ environment variables in the application's environment:
|
||||
| `YARN_CONTAINER_RUNTIME_DOCKER_DELAYED_REMOVAL` | Allows a user to request delayed deletion of the Docker container on a per container basis. If true, Docker containers will not be removed until the duration defined by yarn.nodemanager.delete.debug-delay-sec has elapsed. Administrators can disable this feature through the yarn-site property yarn.nodemanager.runtime.linux.docker.delayed-removal.allowed. This feature is disabled by default. When this feature is disabled or set to false, the container will be removed as soon as it exits. |
|
||||
| `YARN_CONTAINER_RUNTIME_YARN_SYSFS_ENABLE` | Enable mounting of container working directory sysfs sub-directory into Docker container /hadoop/yarn/sysfs. This is useful for populating cluster information into container. |
|
||||
| `YARN_CONTAINER_RUNTIME_DOCKER_SERVICE_MODE` | Enable Service Mode which runs the docker container as defined by the image but does not set the user (--user and --group-add). |
|
||||
|
||||
| `YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG` | Sets the docker client config using which docker container executor can access the remote docker image. |
|
||||
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 Docker support
|
||||
@ -1016,6 +1016,24 @@ To run a Spark shell in Docker containers, run the following command:
|
||||
Note that the application master and executors are configured
|
||||
independently. In this example, we are using the `openjdk:8` image for both.
|
||||
|
||||
When using remote container registry,
|
||||
the YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG must reference the config.json
|
||||
file containing the credentials used to authenticate.
|
||||
|
||||
```
|
||||
DOCKER_IMAGE_NAME=hadoop-docker
|
||||
DOCKER_CLIENT_CONFIG=hdfs:///user/hadoop/config.json
|
||||
spark-submit --master yarn \
|
||||
--deploy-mode cluster \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_TYPE=docker \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_DOCKER_IMAGE=$DOCKER_IMAGE_NAME \
|
||||
--conf spark.executorEnv.YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG=$DOCKER_CLIENT_CONFIG \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_TYPE=docker \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_DOCKER_IMAGE=$DOCKER_IMAGE_NAME \
|
||||
--conf spark.yarn.appMasterEnv.YARN_CONTAINER_RUNTIME_DOCKER_CLIENT_CONFIG=$DOCKER_CLIENT_CONFIG \
|
||||
sparkR.R
|
||||
```
|
||||
|
||||
Docker Container ENTRYPOINT Support
|
||||
------------------------------------
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user