HADOOP-16582. LocalFileSystem's mkdirs() does not work as expected under viewfs. Contributed by Kihwal Lee

This commit is contained in:
Kihwal Lee 2019-09-19 08:22:19 -05:00
parent c9900a01d1
commit d4205dce17
4 changed files with 24 additions and 3 deletions

View File

@ -334,6 +334,10 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException {
return fs.mkdirs(f, permission);
}
@Override
public boolean mkdirs(Path f) throws IOException {
return fs.mkdirs(f);
}
/**
* The src file is on the local disk. Add it to FS at

View File

@ -279,6 +279,11 @@ public boolean mkdirs(final Path f, final FsPermission permission)
return super.mkdirs(fullPath(f), permission);
}
@Override
public boolean mkdirs(final Path f) throws IOException {
return super.mkdirs(fullPath(f));
}
@Override
public FSDataInputStream open(final Path f, final int bufferSize)
throws IOException {

View File

@ -549,12 +549,19 @@ private Path getChrootedPath(InodeTree.ResolveResult<FileSystem> res,
suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix));
}
@Override
public boolean mkdirs(Path dir) throws IOException {
InodeTree.ResolveResult<FileSystem> res =
fsState.resolve(getUriPath(dir), false);
return res.targetFileSystem.mkdirs(res.remainingPath);
}
@Override
public boolean mkdirs(final Path dir, final FsPermission permission)
throws IOException {
InodeTree.ResolveResult<FileSystem> res =
fsState.resolve(getUriPath(dir), false);
return res.targetFileSystem.mkdirs(res.remainingPath, permission);
fsState.resolve(getUriPath(dir), false);
return res.targetFileSystem.mkdirs(res.remainingPath, permission);
}
@Override
@ -1171,6 +1178,12 @@ public boolean mkdirs(Path dir, FsPermission permission)
throw readOnlyMountTable("mkdirs", dir);
}
@Override
public boolean mkdirs(Path dir)
throws AccessControlException, FileAlreadyExistsException {
return mkdirs(dir, null);
}
@Override
public FSDataInputStream open(Path f, int bufferSize)
throws AccessControlException, FileNotFoundException, IOException {

View File

@ -77,7 +77,6 @@ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
boolean overwrite, int bufferSize, short replication, long blockSize,
Progressable progress) throws IOException;
public boolean mkdirs(Path f);
public FSDataInputStream open(Path f);
public FSDataInputStream open(PathHandle f);
public FSDataOutputStream create(Path f);