|
@@ -29,6 +29,7 @@ import java.util.EnumSet;
|
|
|
import java.util.List;
|
|
|
import java.util.Map;
|
|
|
|
|
|
+import org.apache.hadoop.HadoopIllegalArgumentException;
|
|
|
import org.apache.hadoop.classification.InterfaceAudience;
|
|
|
import org.apache.hadoop.classification.InterfaceStability;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
@@ -2652,7 +2653,7 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
*/
|
|
|
public static final class HdfsDataOutputStreamBuilder
|
|
|
extends FSDataOutputStreamBuilder<
|
|
|
- HdfsDataOutputStream, HdfsDataOutputStreamBuilder> {
|
|
|
+ FSDataOutputStream, HdfsDataOutputStreamBuilder> {
|
|
|
private final DistributedFileSystem dfs;
|
|
|
private InetSocketAddress[] favoredNodes = null;
|
|
|
|
|
@@ -2739,16 +2740,23 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
* @throws IOException on I/O errors.
|
|
|
*/
|
|
|
@Override
|
|
|
- public HdfsDataOutputStream build() throws IOException {
|
|
|
- if (isRecursive()) {
|
|
|
- return dfs.create(getPath(), getPermission(), getFlags(),
|
|
|
- getBufferSize(), getReplication(), getBlockSize(),
|
|
|
- getProgress(), getChecksumOpt(), getFavoredNodes());
|
|
|
- } else {
|
|
|
- return dfs.createNonRecursive(getPath(), getPermission(), getFlags(),
|
|
|
- getBufferSize(), getReplication(), getBlockSize(), getProgress(),
|
|
|
- getChecksumOpt(), getFavoredNodes());
|
|
|
+ public FSDataOutputStream build() throws IOException {
|
|
|
+ if (getFlags().contains(CreateFlag.CREATE)) {
|
|
|
+ if (isRecursive()) {
|
|
|
+ return dfs.create(getPath(), getPermission(), getFlags(),
|
|
|
+ getBufferSize(), getReplication(), getBlockSize(),
|
|
|
+ getProgress(), getChecksumOpt(), getFavoredNodes());
|
|
|
+ } else {
|
|
|
+ return dfs.createNonRecursive(getPath(), getPermission(), getFlags(),
|
|
|
+ getBufferSize(), getReplication(), getBlockSize(), getProgress(),
|
|
|
+ getChecksumOpt(), getFavoredNodes());
|
|
|
+ }
|
|
|
+ } else if (getFlags().contains(CreateFlag.APPEND)) {
|
|
|
+ return dfs.append(getPath(), getFlags(), getBufferSize(), getProgress(),
|
|
|
+ getFavoredNodes());
|
|
|
}
|
|
|
+ throw new HadoopIllegalArgumentException(
|
|
|
+ "Must specify either create or append");
|
|
|
}
|
|
|
}
|
|
|
|
|
@@ -2763,4 +2771,15 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
public HdfsDataOutputStreamBuilder createFile(Path path) {
|
|
|
return new HdfsDataOutputStreamBuilder(this, path).create().overwrite(true);
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create a {@link HdfsDataOutputStreamBuilder} to append a file on DFS.
|
|
|
+ *
|
|
|
+ * @param path file path.
|
|
|
+ * @return A {@link HdfsDataOutputStreamBuilder} for appending a file.
|
|
|
+ */
|
|
|
+ @Override
|
|
|
+ public HdfsDataOutputStreamBuilder appendFile(Path path) {
|
|
|
+ return new HdfsDataOutputStreamBuilder(this, path).append();
|
|
|
+ }
|
|
|
}
|