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:
parent
dc72b2e65a
commit
ca93156cc5
|
@ -320,6 +320,10 @@ public class FilterFileSystem extends FileSystem {
|
||||||
return fs.mkdirs(f, permission);
|
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
|
* The src file is on the local disk. Add it to FS at
|
||||||
|
|
|
@ -255,6 +255,11 @@ class ChRootedFileSystem extends FilterFileSystem {
|
||||||
return super.mkdirs(fullPath(f), permission);
|
return super.mkdirs(fullPath(f), permission);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean mkdirs(final Path f) throws IOException {
|
||||||
|
return super.mkdirs(fullPath(f));
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FSDataInputStream open(final Path f, final int bufferSize)
|
public FSDataInputStream open(final Path f, final int bufferSize)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
|
@ -447,12 +447,19 @@ public class ViewFileSystem extends FileSystem {
|
||||||
suffix.length() == 0 ? f : new Path(res.resolvedPath, suffix));
|
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
|
@Override
|
||||||
public boolean mkdirs(final Path dir, final FsPermission permission)
|
public boolean mkdirs(final Path dir, final FsPermission permission)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
InodeTree.ResolveResult<FileSystem> res =
|
InodeTree.ResolveResult<FileSystem> res =
|
||||||
fsState.resolve(getUriPath(dir), false);
|
fsState.resolve(getUriPath(dir), false);
|
||||||
return res.targetFileSystem.mkdirs(res.remainingPath, permission);
|
return res.targetFileSystem.mkdirs(res.remainingPath, permission);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -943,6 +950,12 @@ public class ViewFileSystem extends FileSystem {
|
||||||
throw readOnlyMountTable("mkdirs", dir);
|
throw readOnlyMountTable("mkdirs", dir);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public boolean mkdirs(Path dir)
|
||||||
|
throws AccessControlException, FileAlreadyExistsException {
|
||||||
|
return mkdirs(dir, null);
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FSDataInputStream open(Path f, int bufferSize)
|
public FSDataInputStream open(Path f, int bufferSize)
|
||||||
throws AccessControlException, FileNotFoundException, IOException {
|
throws AccessControlException, FileNotFoundException, IOException {
|
||||||
|
|
|
@ -77,7 +77,6 @@ public class TestFilterFileSystem {
|
||||||
boolean overwrite, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException;
|
Progressable progress) throws IOException;
|
||||||
|
|
||||||
public boolean mkdirs(Path f);
|
|
||||||
public FSDataInputStream open(Path f);
|
public FSDataInputStream open(Path f);
|
||||||
public FSDataOutputStream create(Path f);
|
public FSDataOutputStream create(Path f);
|
||||||
public FSDataOutputStream create(Path f, boolean overwrite);
|
public FSDataOutputStream create(Path f, boolean overwrite);
|
||||||
|
|
Loading…
Reference in New Issue