|
@@ -18,10 +18,24 @@
|
|
|
|
|
|
package org.apache.hadoop.yarn.server.nodemanager;
|
|
|
|
|
|
-import com.google.common.annotations.VisibleForTesting;
|
|
|
-import com.google.common.base.Joiner;
|
|
|
-import com.google.common.base.Preconditions;
|
|
|
-import com.google.common.base.Strings;
|
|
|
+import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
|
+import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
|
|
|
+
|
|
|
+import java.io.ByteArrayOutputStream;
|
|
|
+import java.io.DataOutputStream;
|
|
|
+import java.io.File;
|
|
|
+import java.io.IOException;
|
|
|
+import java.io.OutputStream;
|
|
|
+import java.io.PrintStream;
|
|
|
+import java.net.InetSocketAddress;
|
|
|
+import java.util.ArrayList;
|
|
|
+import java.util.Collections;
|
|
|
+import java.util.EnumSet;
|
|
|
+import java.util.HashSet;
|
|
|
+import java.util.List;
|
|
|
+import java.util.Map;
|
|
|
+import java.util.Set;
|
|
|
+import java.util.regex.Pattern;
|
|
|
|
|
|
import org.apache.commons.lang.math.RandomUtils;
|
|
|
import org.apache.commons.logging.Log;
|
|
@@ -45,38 +59,35 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.Conta
|
|
|
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
|
|
|
import org.apache.hadoop.yarn.util.ConverterUtils;
|
|
|
|
|
|
-import java.io.ByteArrayOutputStream;
|
|
|
-import java.io.DataOutputStream;
|
|
|
-import java.io.File;
|
|
|
-import java.io.IOException;
|
|
|
-import java.io.OutputStream;
|
|
|
-import java.io.PrintStream;
|
|
|
-import java.util.ArrayList;
|
|
|
-import java.util.Collections;
|
|
|
-import java.util.EnumSet;
|
|
|
-import java.util.HashSet;
|
|
|
-import java.util.List;
|
|
|
-import java.util.Map;
|
|
|
-import java.util.Random;
|
|
|
-import java.util.Set;
|
|
|
-import java.util.regex.Pattern;
|
|
|
-import java.net.InetSocketAddress;
|
|
|
-import static org.apache.hadoop.fs.CreateFlag.CREATE;
|
|
|
-import static org.apache.hadoop.fs.CreateFlag.OVERWRITE;
|
|
|
+import com.google.common.annotations.VisibleForTesting;
|
|
|
+import com.google.common.base.Joiner;
|
|
|
+import com.google.common.base.Preconditions;
|
|
|
+import com.google.common.base.Strings;
|
|
|
|
|
|
/**
|
|
|
- * This executor will launch a docker container and run the task inside the container.
|
|
|
+ * This executor will launch and run tasks inside Docker containers. It
|
|
|
+ * currently only supports simple authentication mode. It shares a lot of code
|
|
|
+ * with the DefaultContainerExecutor (and it may make sense to pull out those
|
|
|
+ * common pieces later).
|
|
|
*/
|
|
|
public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
-
|
|
|
private static final Log LOG = LogFactory
|
|
|
- .getLog(DockerContainerExecutor.class);
|
|
|
- public static final String DOCKER_CONTAINER_EXECUTOR_SCRIPT = "docker_container_executor";
|
|
|
- public static final String DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT = "docker_container_executor_session";
|
|
|
-
|
|
|
- // This validates that the image is a proper docker image and would not crash docker.
|
|
|
- public static final String DOCKER_IMAGE_PATTERN = "^(([\\w\\.-]+)(:\\d+)*\\/)?[\\w\\.:-]+$";
|
|
|
-
|
|
|
+ .getLog(DockerContainerExecutor.class);
|
|
|
+ //The name of the script file that will launch the Docker containers
|
|
|
+ public static final String DOCKER_CONTAINER_EXECUTOR_SCRIPT =
|
|
|
+ "docker_container_executor";
|
|
|
+ //The name of the session script that the DOCKER_CONTAINER_EXECUTOR_SCRIPT
|
|
|
+ //launches in turn
|
|
|
+ public static final String DOCKER_CONTAINER_EXECUTOR_SESSION_SCRIPT =
|
|
|
+ "docker_container_executor_session";
|
|
|
+
|
|
|
+ //This validates that the image is a proper docker image and would not crash
|
|
|
+ //docker. The image name is not allowed to contain spaces. e.g.
|
|
|
+ //registry.somecompany.com:9999/containername:0.1 or
|
|
|
+ //containername:0.1 or
|
|
|
+ //containername
|
|
|
+ public static final String DOCKER_IMAGE_PATTERN =
|
|
|
+ "^(([\\w\\.-]+)(:\\d+)*\\/)?[\\w\\.:-]+$";
|
|
|
|
|
|
private final FileContext lfs;
|
|
|
private final Pattern dockerImagePattern;
|
|
@@ -96,23 +107,26 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
|
|
|
@Override
|
|
|
public void init() throws IOException {
|
|
|
- String auth = getConf().get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
|
|
|
+ String auth =
|
|
|
+ getConf().get(CommonConfigurationKeys.HADOOP_SECURITY_AUTHENTICATION);
|
|
|
if (auth != null && !auth.equals("simple")) {
|
|
|
- throw new IllegalStateException("DockerContainerExecutor only works with simple authentication mode");
|
|
|
+ throw new IllegalStateException(
|
|
|
+ "DockerContainerExecutor only works with simple authentication mode");
|
|
|
}
|
|
|
- String dockerExecutor = getConf().get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
|
|
|
+ String dockerExecutor = getConf().get(
|
|
|
+ YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
|
|
|
YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
|
|
|
if (!new File(dockerExecutor).exists()) {
|
|
|
- throw new IllegalStateException("Invalid docker exec path: " + dockerExecutor);
|
|
|
+ throw new IllegalStateException(
|
|
|
+ "Invalid docker exec path: " + dockerExecutor);
|
|
|
}
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
public synchronized void startLocalizer(Path nmPrivateContainerTokensPath,
|
|
|
- InetSocketAddress nmAddr, String user, String appId, String locId,
|
|
|
- LocalDirsHandlerService dirsHandler)
|
|
|
+ InetSocketAddress nmAddr, String user, String appId, String locId,
|
|
|
+ LocalDirsHandlerService dirsHandler)
|
|
|
throws IOException, InterruptedException {
|
|
|
-
|
|
|
List<String> localDirs = dirsHandler.getLocalDirs();
|
|
|
List<String> logDirs = dirsHandler.getLogDirs();
|
|
|
|
|
@@ -128,7 +142,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
// randomly choose the local directory
|
|
|
Path appStorageDir = getWorkingDir(localDirs, user, appId);
|
|
|
|
|
|
- String tokenFn = String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
|
|
|
+ String tokenFn =
|
|
|
+ String.format(ContainerLocalizer.TOKEN_FILE_NAME_FMT, locId);
|
|
|
Path tokenDst = new Path(appStorageDir, tokenFn);
|
|
|
copyFile(nmPrivateContainerTokensPath, tokenDst, user);
|
|
|
LOG.info("Copying from " + nmPrivateContainerTokensPath + " to " + tokenDst);
|
|
@@ -140,31 +155,34 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
|
|
|
|
|
|
@Override
|
|
|
- public int launchContainer(Container container,
|
|
|
- Path nmPrivateContainerScriptPath, Path nmPrivateTokensPath,
|
|
|
- String userName, String appId, Path containerWorkDir,
|
|
|
- List<String> localDirs, List<String> logDirs) throws IOException {
|
|
|
+ public int launchContainer(Container container, Path
|
|
|
+ nmPrivateContainerScriptPath, Path nmPrivateTokensPath, String userName,
|
|
|
+ String appId, Path containerWorkDir, List<String> localDirs, List<String>
|
|
|
+ logDirs) throws IOException {
|
|
|
+ //Variables for the launch environment can be injected from the command-line
|
|
|
+ //while submitting the application
|
|
|
String containerImageName = container.getLaunchContext().getEnvironment()
|
|
|
- .get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
|
|
|
+ .get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
LOG.debug("containerImageName from launchContext: " + containerImageName);
|
|
|
}
|
|
|
- Preconditions.checkArgument(!Strings.isNullOrEmpty(containerImageName), "Container image must not be null");
|
|
|
+ Preconditions.checkArgument(!Strings.isNullOrEmpty(containerImageName),
|
|
|
+ "Container image must not be null");
|
|
|
containerImageName = containerImageName.replaceAll("['\"]", "");
|
|
|
|
|
|
- Preconditions.checkArgument(saneDockerImage(containerImageName), "Image: " + containerImageName + " is not a proper docker image");
|
|
|
- String dockerExecutor = getConf().get(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
|
|
|
- YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
|
|
|
+ Preconditions.checkArgument(saneDockerImage(containerImageName), "Image: "
|
|
|
+ + containerImageName + " is not a proper docker image");
|
|
|
+ String dockerExecutor = getConf().get(
|
|
|
+ YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME,
|
|
|
+ YarnConfiguration.NM_DEFAULT_DOCKER_CONTAINER_EXECUTOR_EXEC_NAME);
|
|
|
|
|
|
FsPermission dirPerm = new FsPermission(APPDIR_PERM);
|
|
|
ContainerId containerId = container.getContainerId();
|
|
|
|
|
|
// create container dirs on all disks
|
|
|
String containerIdStr = ConverterUtils.toString(containerId);
|
|
|
- String appIdStr =
|
|
|
- ConverterUtils.toString(
|
|
|
- containerId.getApplicationAttemptId().
|
|
|
- getApplicationId());
|
|
|
+ String appIdStr = ConverterUtils.toString(
|
|
|
+ containerId.getApplicationAttemptId().getApplicationId());
|
|
|
for (String sLocalDir : localDirs) {
|
|
|
Path usersdir = new Path(sLocalDir, ContainerLocalizer.USERCACHE);
|
|
|
Path userdir = new Path(usersdir, userName);
|
|
@@ -178,46 +196,57 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
createContainerLogDirs(appIdStr, containerIdStr, logDirs, userName);
|
|
|
|
|
|
Path tmpDir = new Path(containerWorkDir,
|
|
|
- YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
|
|
|
+ YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR);
|
|
|
createDir(tmpDir, dirPerm, false, userName);
|
|
|
|
|
|
// copy launch script to work dir
|
|
|
Path launchDst =
|
|
|
- new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
|
|
|
+ new Path(containerWorkDir, ContainerLaunch.CONTAINER_SCRIPT);
|
|
|
lfs.util().copy(nmPrivateContainerScriptPath, launchDst);
|
|
|
|
|
|
// copy container tokens to work dir
|
|
|
Path tokenDst =
|
|
|
- new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
|
|
|
+ new Path(containerWorkDir, ContainerLaunch.FINAL_CONTAINER_TOKENS_FILE);
|
|
|
lfs.util().copy(nmPrivateTokensPath, tokenDst);
|
|
|
|
|
|
-
|
|
|
-
|
|
|
String localDirMount = toMount(localDirs);
|
|
|
String logDirMount = toMount(logDirs);
|
|
|
- String containerWorkDirMount = toMount(Collections.singletonList(containerWorkDir.toUri().getPath()));
|
|
|
+ String containerWorkDirMount = toMount(Collections.singletonList(
|
|
|
+ containerWorkDir.toUri().getPath()));
|
|
|
StringBuilder commands = new StringBuilder();
|
|
|
+ //Use docker run to launch the docker container. See man pages for
|
|
|
+ //docker-run
|
|
|
+ //--rm removes the container automatically once the container finishes
|
|
|
+ //--net=host allows the container to take on the host's network stack
|
|
|
+ //--name sets the Docker Container name to the YARN containerId string
|
|
|
+ //-v is used to bind mount volumes for local, log and work dirs.
|
|
|
String commandStr = commands.append(dockerExecutor)
|
|
|
- .append(" ")
|
|
|
- .append("run")
|
|
|
- .append(" ")
|
|
|
- .append("--rm --net=host")
|
|
|
- .append(" ")
|
|
|
- .append(" --name " + containerIdStr)
|
|
|
- .append(localDirMount)
|
|
|
- .append(logDirMount)
|
|
|
- .append(containerWorkDirMount)
|
|
|
- .append(" ")
|
|
|
- .append(containerImageName)
|
|
|
- .toString();
|
|
|
- String dockerPidScript = "`" + dockerExecutor + " inspect --format {{.State.Pid}} " + containerIdStr + "`";
|
|
|
+ .append(" ")
|
|
|
+ .append("run")
|
|
|
+ .append(" ")
|
|
|
+ .append("--rm --net=host")
|
|
|
+ .append(" ")
|
|
|
+ .append(" --name " + containerIdStr)
|
|
|
+ .append(localDirMount)
|
|
|
+ .append(logDirMount)
|
|
|
+ .append(containerWorkDirMount)
|
|
|
+ .append(" ")
|
|
|
+ .append(containerImageName)
|
|
|
+ .toString();
|
|
|
+ //Get the pid of the process which has been launched as a docker container
|
|
|
+ //using docker inspect
|
|
|
+ String dockerPidScript = "`" + dockerExecutor +
|
|
|
+ " inspect --format {{.State.Pid}} " + containerIdStr + "`";
|
|
|
+
|
|
|
// Create new local launch wrapper script
|
|
|
- LocalWrapperScriptBuilder sb =
|
|
|
- new UnixLocalWrapperScriptBuilder(containerWorkDir, commandStr, dockerPidScript);
|
|
|
+ LocalWrapperScriptBuilder sb = new UnixLocalWrapperScriptBuilder(
|
|
|
+ containerWorkDir, commandStr, dockerPidScript);
|
|
|
Path pidFile = getPidFilePath(containerId);
|
|
|
if (pidFile != null) {
|
|
|
sb.writeLocalWrapperScript(launchDst, pidFile);
|
|
|
} else {
|
|
|
+ //Although the container was activated by ContainerLaunch before exec()
|
|
|
+ //was called, since then deactivateContainer() has been called.
|
|
|
LOG.info("Container " + containerIdStr
|
|
|
+ " was marked as inactive. Returning terminated error");
|
|
|
return ExitCode.TERMINATED.getExitCode();
|
|
@@ -234,12 +263,13 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
String[] command = getRunCommand(sb.getWrapperScriptPath().toString(),
|
|
|
containerIdStr, userName, pidFile, this.getConf());
|
|
|
if (LOG.isDebugEnabled()) {
|
|
|
- LOG.debug("launchContainer: " + commandStr + " " + Joiner.on(" ").join(command));
|
|
|
+ LOG.debug("launchContainer: " + commandStr + " " +
|
|
|
+ Joiner.on(" ").join(command));
|
|
|
}
|
|
|
shExec = new ShellCommandExecutor(
|
|
|
- command,
|
|
|
- new File(containerWorkDir.toUri().getPath()),
|
|
|
- container.getLaunchContext().getEnvironment()); // sanitized env
|
|
|
+ command,
|
|
|
+ new File(containerWorkDir.toUri().getPath()),
|
|
|
+ container.getLaunchContext().getEnvironment()); // sanitized env
|
|
|
if (isContainerActive(containerId)) {
|
|
|
shExec.execute();
|
|
|
} else {
|
|
@@ -279,9 +309,17 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public void writeLaunchEnv(OutputStream out, Map<String, String> environment, Map<Path, List<String>> resources, List<String> command) throws IOException {
|
|
|
- ContainerLaunch.ShellScriptBuilder sb = ContainerLaunch.ShellScriptBuilder.create();
|
|
|
+ /**
|
|
|
+ * Filter the environment variables that may conflict with the ones set in
|
|
|
+ * the docker image and write them out to an OutputStream.
|
|
|
+ */
|
|
|
+ public void writeLaunchEnv(OutputStream out, Map<String, String> environment,
|
|
|
+ Map<Path, List<String>> resources, List<String> command)
|
|
|
+ throws IOException {
|
|
|
+ ContainerLaunch.ShellScriptBuilder sb =
|
|
|
+ ContainerLaunch.ShellScriptBuilder.create();
|
|
|
|
|
|
+ //Remove environments that may conflict with the ones in Docker image.
|
|
|
Set<String> exclusionSet = new HashSet<String>();
|
|
|
exclusionSet.add(YarnConfiguration.NM_DOCKER_CONTAINER_EXECUTOR_IMAGE_NAME);
|
|
|
exclusionSet.add(ApplicationConstants.Environment.HADOOP_YARN_HOME.name());
|
|
@@ -427,6 +465,9 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
return builder.toString();
|
|
|
}
|
|
|
|
|
|
+ //This class facilitates (only) the creation of platform-specific scripts that
|
|
|
+ //will be used to launch the containers
|
|
|
+ //TODO: This should be re-used from the DefaultContainerExecutor.
|
|
|
private abstract class LocalWrapperScriptBuilder {
|
|
|
|
|
|
private final Path wrapperScriptPath;
|
|
@@ -435,7 +476,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
return wrapperScriptPath;
|
|
|
}
|
|
|
|
|
|
- public void writeLocalWrapperScript(Path launchDst, Path pidFile) throws IOException {
|
|
|
+ public void writeLocalWrapperScript(Path launchDst, Path pidFile)
|
|
|
+ throws IOException {
|
|
|
DataOutputStream out = null;
|
|
|
PrintStream pout = null;
|
|
|
|
|
@@ -448,8 +490,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
- protected abstract void writeLocalWrapperScript(Path launchDst, Path pidFile,
|
|
|
- PrintStream pout);
|
|
|
+ protected abstract void writeLocalWrapperScript(Path launchDst,
|
|
|
+ Path pidFile, PrintStream pout);
|
|
|
|
|
|
protected LocalWrapperScriptBuilder(Path containerWorkDir) {
|
|
|
this.wrapperScriptPath = new Path(containerWorkDir,
|
|
@@ -457,13 +499,15 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ //TODO: This class too should be used from DefaultContainerExecutor.
|
|
|
private final class UnixLocalWrapperScriptBuilder
|
|
|
- extends LocalWrapperScriptBuilder {
|
|
|
+ extends LocalWrapperScriptBuilder {
|
|
|
private final Path sessionScriptPath;
|
|
|
private final String dockerCommand;
|
|
|
private final String dockerPidScript;
|
|
|
|
|
|
- public UnixLocalWrapperScriptBuilder(Path containerWorkDir, String dockerCommand, String dockerPidScript) {
|
|
|
+ public UnixLocalWrapperScriptBuilder(Path containerWorkDir,
|
|
|
+ String dockerCommand, String dockerPidScript) {
|
|
|
super(containerWorkDir);
|
|
|
this.dockerCommand = dockerCommand;
|
|
|
this.dockerPidScript = dockerPidScript;
|
|
@@ -480,8 +524,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
|
|
|
@Override
|
|
|
public void writeLocalWrapperScript(Path launchDst, Path pidFile,
|
|
|
- PrintStream pout) {
|
|
|
-
|
|
|
+ PrintStream pout) {
|
|
|
String exitCodeFile = ContainerLaunch.getExitCodeFile(
|
|
|
pidFile.toString());
|
|
|
String tmpFile = exitCodeFile + ".tmp";
|
|
@@ -505,7 +548,8 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
// hence write pid to tmp file first followed by a mv
|
|
|
pout.println("#!/usr/bin/env bash");
|
|
|
pout.println();
|
|
|
- pout.println("echo "+ dockerPidScript +" > " + pidFile.toString() + ".tmp");
|
|
|
+ pout.println("echo "+ dockerPidScript +" > " + pidFile.toString()
|
|
|
+ + ".tmp");
|
|
|
pout.println("/bin/mv -f " + pidFile.toString() + ".tmp " + pidFile);
|
|
|
pout.println(dockerCommand + " bash \"" +
|
|
|
launchDst.toUri().getPath().toString() + "\"");
|
|
@@ -518,7 +562,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
}
|
|
|
|
|
|
protected void createDir(Path dirPath, FsPermission perms,
|
|
|
- boolean createParent, String user) throws IOException {
|
|
|
+ boolean createParent, String user) throws IOException {
|
|
|
lfs.mkdir(dirPath, perms, createParent);
|
|
|
if (!perms.equals(perms.applyUMask(lfs.getUMask()))) {
|
|
|
lfs.setPermission(dirPath, perms);
|
|
@@ -532,13 +576,14 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
* </ul>
|
|
|
*/
|
|
|
void createUserLocalDirs(List<String> localDirs, String user)
|
|
|
- throws IOException {
|
|
|
+ throws IOException {
|
|
|
boolean userDirStatus = false;
|
|
|
FsPermission userperms = new FsPermission(USER_PERM);
|
|
|
for (String localDir : localDirs) {
|
|
|
// create $local.dir/usercache/$user and its immediate parent
|
|
|
try {
|
|
|
- createDir(getUserCacheDir(new Path(localDir), user), userperms, true, user);
|
|
|
+ createDir(getUserCacheDir(new Path(localDir), user), userperms, true,
|
|
|
+ user);
|
|
|
} catch (IOException e) {
|
|
|
LOG.warn("Unable to create the user directory : " + localDir, e);
|
|
|
continue;
|
|
@@ -633,7 +678,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
* Create application log directories on all disks.
|
|
|
*/
|
|
|
void createContainerLogDirs(String appId, String containerId,
|
|
|
- List<String> logDirs, String user) throws IOException {
|
|
|
+ List<String> logDirs, String user) throws IOException {
|
|
|
|
|
|
boolean containerLogDirStatus = false;
|
|
|
FsPermission containerLogDirPerms = new FsPermission(LOGDIR_PERM);
|
|
@@ -707,7 +752,7 @@ public class DockerContainerExecutor extends ContainerExecutor {
|
|
|
}
|
|
|
|
|
|
protected Path getWorkingDir(List<String> localDirs, String user,
|
|
|
- String appId) throws IOException {
|
|
|
+ String appId) throws IOException {
|
|
|
Path appStorageDir = null;
|
|
|
long totalAvailable = 0L;
|
|
|
long[] availableOnDisk = new long[localDirs.size()];
|