|
@@ -24,6 +24,7 @@ import java.io.IOException;
|
|
|
import java.net.URI;
|
|
|
import java.net.URISyntaxException;
|
|
|
import java.util.Arrays;
|
|
|
+import java.util.EnumSet;
|
|
|
import java.util.HashSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Set;
|
|
@@ -35,6 +36,7 @@ import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.fs.BlockLocation;
|
|
|
import org.apache.hadoop.fs.ContentSummary;
|
|
|
+import org.apache.hadoop.fs.CreateFlag;
|
|
|
import org.apache.hadoop.fs.FSDataInputStream;
|
|
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
@@ -264,6 +266,21 @@ public class ViewFileSystem extends FileSystem {
|
|
|
return res.targetFileSystem.append(res.remainingPath, bufferSize, progress);
|
|
|
}
|
|
|
|
|
|
+ @Override
|
|
|
+ public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
|
|
|
+ EnumSet<CreateFlag> flags, int bufferSize, short replication, long blockSize,
|
|
|
+ Progressable progress) throws IOException {
|
|
|
+ InodeTree.ResolveResult<FileSystem> res;
|
|
|
+ try {
|
|
|
+ res = fsState.resolve(getUriPath(f), false);
|
|
|
+ } catch (FileNotFoundException e) {
|
|
|
+ throw readOnlyMountTable("create", f);
|
|
|
+ }
|
|
|
+ assert(res.remainingPath != null);
|
|
|
+ return res.targetFileSystem.createNonRecursive(res.remainingPath, permission,
|
|
|
+ flags, bufferSize, replication, blockSize, progress);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
public FSDataOutputStream create(final Path f, final FsPermission permission,
|
|
|
final boolean overwrite, final int bufferSize, final short replication,
|