|
@@ -39,6 +39,7 @@ 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.FSDataOutputStreamBuilder;
|
|
|
import org.apache.hadoop.fs.FSLinkResolver;
|
|
|
import org.apache.hadoop.fs.FileChecksum;
|
|
|
import org.apache.hadoop.fs.FileEncryptionInfo;
|
|
@@ -446,6 +447,48 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
}.resolve(this, absF);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Same as
|
|
|
+ * {@link #create(Path, FsPermission, EnumSet<CreateFlag>, int, short, long,
|
|
|
+ * Progressable, ChecksumOpt)} with the addition of favoredNodes that is a
|
|
|
+ * hint to where the namenode should place the file blocks.
|
|
|
+ * The favored nodes hint is not persisted in HDFS. Hence it may be honored
|
|
|
+ * at the creation time only. And with favored nodes, blocks will be pinned
|
|
|
+ * on the datanodes to prevent balancing move the block. HDFS could move the
|
|
|
+ * blocks during replication, to move the blocks from favored nodes. A value
|
|
|
+ * of null means no favored nodes for this create
|
|
|
+ */
|
|
|
+ private HdfsDataOutputStream create(final Path f,
|
|
|
+ final FsPermission permission, EnumSet<CreateFlag> flag,
|
|
|
+ final int bufferSize, final short replication, final long blockSize,
|
|
|
+ final Progressable progress, final ChecksumOpt checksumOpt,
|
|
|
+ final InetSocketAddress[] favoredNodes) throws IOException {
|
|
|
+ statistics.incrementWriteOps(1);
|
|
|
+ storageStatistics.incrementOpCounter(OpType.CREATE);
|
|
|
+ Path absF = fixRelativePart(f);
|
|
|
+ return new FileSystemLinkResolver<HdfsDataOutputStream>() {
|
|
|
+ @Override
|
|
|
+ public HdfsDataOutputStream doCall(final Path p) throws IOException {
|
|
|
+ final DFSOutputStream out = dfs.create(getPathName(f), permission,
|
|
|
+ flag, true, replication, blockSize, progress, bufferSize,
|
|
|
+ checksumOpt, favoredNodes);
|
|
|
+ return dfs.createWrappedOutputStream(out, statistics);
|
|
|
+ }
|
|
|
+ @Override
|
|
|
+ public HdfsDataOutputStream next(final FileSystem fs, final Path p)
|
|
|
+ throws IOException {
|
|
|
+ if (fs instanceof DistributedFileSystem) {
|
|
|
+ DistributedFileSystem myDfs = (DistributedFileSystem)fs;
|
|
|
+ return myDfs.create(p, permission, flag, bufferSize, replication,
|
|
|
+ blockSize, progress, checksumOpt, favoredNodes);
|
|
|
+ }
|
|
|
+ throw new UnsupportedOperationException("Cannot create with" +
|
|
|
+ " favoredNodes through a symlink to a non-DistributedFileSystem: "
|
|
|
+ + f + " -> " + p);
|
|
|
+ }
|
|
|
+ }.resolve(this, absF);
|
|
|
+ }
|
|
|
+
|
|
|
@Override
|
|
|
protected HdfsDataOutputStream primitiveCreate(Path f,
|
|
|
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
|
|
@@ -2584,4 +2627,42 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
DFSOpsCountStatistics getDFSOpsCountStatistics() {
|
|
|
return storageStatistics;
|
|
|
}
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Extends FSDataOutputStreamBuilder to support special requirements
|
|
|
+ * of DistributedFileSystem.
|
|
|
+ */
|
|
|
+ public static class HdfsDataOutputStreamBuilder
|
|
|
+ extends FSDataOutputStreamBuilder {
|
|
|
+ private final DistributedFileSystem dfs;
|
|
|
+ private InetSocketAddress[] favoredNodes = null;
|
|
|
+
|
|
|
+ public HdfsDataOutputStreamBuilder(DistributedFileSystem dfs, Path path) {
|
|
|
+ super(dfs, path);
|
|
|
+ this.dfs = dfs;
|
|
|
+ }
|
|
|
+
|
|
|
+ protected InetSocketAddress[] getFavoredNodes() {
|
|
|
+ return favoredNodes;
|
|
|
+ }
|
|
|
+
|
|
|
+ public HdfsDataOutputStreamBuilder setFavoredNodes(
|
|
|
+ final InetSocketAddress[] nodes) {
|
|
|
+ Preconditions.checkNotNull(nodes);
|
|
|
+ favoredNodes = nodes.clone();
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public HdfsDataOutputStream build() throws IOException {
|
|
|
+ return dfs.create(getPath(), getPermission(), getFlags(),
|
|
|
+ getBufferSize(), getReplication(), getBlockSize(),
|
|
|
+ getProgress(), getChecksumOpt(), getFavoredNodes());
|
|
|
+ }
|
|
|
+ }
|
|
|
+
|
|
|
+ @Override
|
|
|
+ public HdfsDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
|
|
|
+ return new HdfsDataOutputStreamBuilder(this, path);
|
|
|
+ }
|
|
|
}
|