|
@@ -33,7 +33,6 @@ 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.FileStatus;
|
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
import org.apache.hadoop.fs.FsServerDefaults;
|
|
@@ -47,6 +46,7 @@ import org.apache.hadoop.fs.RemoteIterator;
|
|
|
import org.apache.hadoop.fs.permission.FsPermission;
|
|
|
import org.apache.hadoop.hdfs.DFSClient.DFSDataInputStream;
|
|
|
import org.apache.hadoop.hdfs.client.HdfsDataInputStream;
|
|
|
+import org.apache.hadoop.hdfs.client.HdfsDataOutputStream;
|
|
|
import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
|
|
|
import org.apache.hadoop.hdfs.protocol.DirectoryListing;
|
|
|
import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
|
|
@@ -216,31 +216,33 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
|
|
|
/** This optional operation is not yet supported. */
|
|
|
@Override
|
|
|
- public FSDataOutputStream append(Path f, int bufferSize,
|
|
|
+ public HdfsDataOutputStream append(Path f, int bufferSize,
|
|
|
Progressable progress) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
return dfs.append(getPathName(f), bufferSize, progress, statistics);
|
|
|
}
|
|
|
|
|
|
@Override
|
|
|
- public FSDataOutputStream create(Path f, FsPermission permission,
|
|
|
+ public HdfsDataOutputStream create(Path f, FsPermission permission,
|
|
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
|
|
Progressable progress) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
- return new FSDataOutputStream(dfs.create(getPathName(f), permission,
|
|
|
- overwrite ? EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
|
|
|
- : EnumSet.of(CreateFlag.CREATE), replication, blockSize, progress,
|
|
|
- bufferSize), statistics);
|
|
|
+ final EnumSet<CreateFlag> cflags = overwrite?
|
|
|
+ EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE)
|
|
|
+ : EnumSet.of(CreateFlag.CREATE);
|
|
|
+ final DFSOutputStream out = dfs.create(getPathName(f), permission, cflags,
|
|
|
+ replication, blockSize, progress, bufferSize);
|
|
|
+ return new HdfsDataOutputStream(out, statistics);
|
|
|
}
|
|
|
|
|
|
@SuppressWarnings("deprecation")
|
|
|
@Override
|
|
|
- protected FSDataOutputStream primitiveCreate(Path f,
|
|
|
+ protected HdfsDataOutputStream primitiveCreate(Path f,
|
|
|
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
|
|
|
short replication, long blockSize, Progressable progress,
|
|
|
int bytesPerChecksum) throws IOException {
|
|
|
statistics.incrementReadOps(1);
|
|
|
- return new FSDataOutputStream(dfs.primitiveCreate(getPathName(f),
|
|
|
+ return new HdfsDataOutputStream(dfs.primitiveCreate(getPathName(f),
|
|
|
absolutePermission, flag, true, replication, blockSize,
|
|
|
progress, bufferSize, bytesPerChecksum),statistics);
|
|
|
}
|
|
@@ -248,14 +250,14 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
/**
|
|
|
* Same as create(), except fails if parent directory doesn't already exist.
|
|
|
*/
|
|
|
- public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
|
|
|
+ public HdfsDataOutputStream createNonRecursive(Path f, FsPermission permission,
|
|
|
EnumSet<CreateFlag> flag, int bufferSize, short replication,
|
|
|
long blockSize, Progressable progress) throws IOException {
|
|
|
statistics.incrementWriteOps(1);
|
|
|
if (flag.contains(CreateFlag.OVERWRITE)) {
|
|
|
flag.add(CreateFlag.CREATE);
|
|
|
}
|
|
|
- return new FSDataOutputStream(dfs.create(getPathName(f), permission, flag,
|
|
|
+ return new HdfsDataOutputStream(dfs.create(getPathName(f), permission, flag,
|
|
|
false, replication, blockSize, progress, bufferSize), statistics);
|
|
|
}
|
|
|
|