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

(cherry picked from commit d4205dce17)
This commit is contained in:
Kihwal Lee 2019-09-19 08:26:36 -05:00
parent d25b3339ed
commit 56562b9117
4 changed files with 24 additions and 3 deletions

View File

@ -332,6 +332,10 @@ public class FilterFileSystem extends FileSystem {
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

@ -267,6 +267,11 @@ class ChRootedFileSystem extends FilterFileSystem {
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

@ -461,12 +461,19 @@ public class ViewFileSystem extends FileSystem {
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
@ -1076,6 +1083,12 @@ public class ViewFileSystem extends FileSystem {
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

@ -78,7 +78,6 @@ public class TestFilterFileSystem {
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);