|
@@ -101,7 +101,6 @@ import org.apache.hadoop.crypto.key.KeyProviderDelegationTokenExtension;
|
|
|
|
|
|
import com.google.common.annotations.VisibleForTesting;
|
|
|
import com.google.common.base.Preconditions;
|
|
|
-import org.apache.commons.lang.StringUtils;
|
|
|
|
|
|
import javax.annotation.Nonnull;
|
|
|
|
|
@@ -525,6 +524,49 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
return dfs.createWrappedOutputStream(dfsos, statistics);
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Similar to {@link #create(Path, FsPermission, EnumSet, int, short, long,
|
|
|
+ * Progressable, ChecksumOpt, InetSocketAddress[], String)}, it provides a
|
|
|
+ * HDFS-specific version of {@link #createNonRecursive(Path, FsPermission,
|
|
|
+ * EnumSet, int, short, long, Progressable)} with a few additions.
|
|
|
+ *
|
|
|
+ * @see #create(Path, FsPermission, EnumSet, int, short, long, Progressable,
|
|
|
+ * ChecksumOpt, InetSocketAddress[], String) for the descriptions of
|
|
|
+ * additional parameters, i.e., favoredNodes and ecPolicyName.
|
|
|
+ */
|
|
|
+ private HdfsDataOutputStream createNonRecursive(final Path f,
|
|
|
+ final FsPermission permission, final EnumSet<CreateFlag> flag,
|
|
|
+ final int bufferSize, final short replication, final long blockSize,
|
|
|
+ final Progressable progress, final ChecksumOpt checksumOpt,
|
|
|
+ final InetSocketAddress[] favoredNodes, final String ecPolicyName)
|
|
|
+ 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, false, replication, blockSize, progress, bufferSize,
|
|
|
+ checksumOpt, favoredNodes, ecPolicyName);
|
|
|
+ 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.createNonRecursive(p, permission, flag, bufferSize,
|
|
|
+ replication, blockSize, progress, checksumOpt, favoredNodes,
|
|
|
+ ecPolicyName);
|
|
|
+ }
|
|
|
+ throw new UnsupportedOperationException("Cannot create with" +
|
|
|
+ " favoredNodes through a symlink to a non-DistributedFileSystem: "
|
|
|
+ + f + " -> " + p);
|
|
|
+ }
|
|
|
+ }.resolve(this, absF);
|
|
|
+ }
|
|
|
+
|
|
|
/**
|
|
|
* Same as create(), except fails if parent directory doesn't already exist.
|
|
|
*/
|
|
@@ -2686,33 +2728,88 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * Extends FSDataOutputStreamBuilder to support special requirements
|
|
|
- * of DistributedFileSystem.
|
|
|
+ * HdfsDataOutputStreamBuilder provides the HDFS-specific capabilities to
|
|
|
+ * write file on HDFS.
|
|
|
*/
|
|
|
- public static class HdfsDataOutputStreamBuilder
|
|
|
- extends FSDataOutputStreamBuilder {
|
|
|
+ public static final class HdfsDataOutputStreamBuilder
|
|
|
+ extends FSDataOutputStreamBuilder<
|
|
|
+ HdfsDataOutputStream, HdfsDataOutputStreamBuilder> {
|
|
|
private final DistributedFileSystem dfs;
|
|
|
private InetSocketAddress[] favoredNodes = null;
|
|
|
private String ecPolicyName = null;
|
|
|
- private boolean shouldReplicate = false;
|
|
|
|
|
|
- public HdfsDataOutputStreamBuilder(DistributedFileSystem dfs, Path path) {
|
|
|
+ /**
|
|
|
+ * Construct a HdfsDataOutputStream builder for a file.
|
|
|
+ * @param dfs the {@link DistributedFileSystem} instance.
|
|
|
+ * @param path the path of the file to create / append.
|
|
|
+ */
|
|
|
+ private HdfsDataOutputStreamBuilder(DistributedFileSystem dfs, Path path) {
|
|
|
super(dfs, path);
|
|
|
this.dfs = dfs;
|
|
|
}
|
|
|
|
|
|
- protected InetSocketAddress[] getFavoredNodes() {
|
|
|
+ @Override
|
|
|
+ protected HdfsDataOutputStreamBuilder getThisBuilder() {
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ private InetSocketAddress[] getFavoredNodes() {
|
|
|
return favoredNodes;
|
|
|
}
|
|
|
|
|
|
- public HdfsDataOutputStreamBuilder setFavoredNodes(
|
|
|
+ /**
|
|
|
+ * Set favored DataNodes.
|
|
|
+ * @param nodes the addresses of the favored DataNodes.
|
|
|
+ */
|
|
|
+ public HdfsDataOutputStreamBuilder favoredNodes(
|
|
|
@Nonnull final InetSocketAddress[] nodes) {
|
|
|
Preconditions.checkNotNull(nodes);
|
|
|
favoredNodes = nodes.clone();
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
- protected String getEcPolicyName() {
|
|
|
+ /**
|
|
|
+ * Force closed blocks to disk.
|
|
|
+ *
|
|
|
+ * @see CreateFlag for the details.
|
|
|
+ */
|
|
|
+ public HdfsDataOutputStreamBuilder syncBlock() {
|
|
|
+ getFlags().add(CreateFlag.SYNC_BLOCK);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Create the block on transient storage if possible.
|
|
|
+ *
|
|
|
+ * @see CreateFlag for the details.
|
|
|
+ */
|
|
|
+ public HdfsDataOutputStreamBuilder lazyPersist() {
|
|
|
+ getFlags().add(CreateFlag.LAZY_PERSIST);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Append data to a new block instead of the end of the last partial block.
|
|
|
+ *
|
|
|
+ * @see CreateFlag for the details.
|
|
|
+ */
|
|
|
+ public HdfsDataOutputStreamBuilder newBlock() {
|
|
|
+ getFlags().add(CreateFlag.NEW_BLOCK);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Advise that a block replica NOT be written to the local DataNode.
|
|
|
+ *
|
|
|
+ * @see CreateFlag for the details.
|
|
|
+ */
|
|
|
+ public HdfsDataOutputStreamBuilder noLocalWrite() {
|
|
|
+ getFlags().add(CreateFlag.NO_LOCAL_WRITE);
|
|
|
+ return this;
|
|
|
+ }
|
|
|
+
|
|
|
+ @VisibleForTesting
|
|
|
+ String getEcPolicyName() {
|
|
|
return ecPolicyName;
|
|
|
}
|
|
|
|
|
@@ -2722,17 +2819,17 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
* or erasure coding policy is. Don't call this function and
|
|
|
* enforceReplicate() in the same builder since they have conflict
|
|
|
* of interest.
|
|
|
- *
|
|
|
*/
|
|
|
- public HdfsDataOutputStreamBuilder setEcPolicyName(
|
|
|
+ public HdfsDataOutputStreamBuilder ecPolicyName(
|
|
|
@Nonnull final String policyName) {
|
|
|
Preconditions.checkNotNull(policyName);
|
|
|
ecPolicyName = policyName;
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
- public boolean shouldReplicate() {
|
|
|
- return shouldReplicate;
|
|
|
+ @VisibleForTesting
|
|
|
+ boolean shouldReplicate() {
|
|
|
+ return getFlags().contains(CreateFlag.SHOULD_REPLICATE);
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -2742,30 +2839,46 @@ public class DistributedFileSystem extends FileSystem {
|
|
|
* conflict of interest.
|
|
|
*/
|
|
|
public HdfsDataOutputStreamBuilder replicate() {
|
|
|
- shouldReplicate = true;
|
|
|
+ getFlags().add(CreateFlag.SHOULD_REPLICATE);
|
|
|
return this;
|
|
|
}
|
|
|
|
|
|
+ @VisibleForTesting
|
|
|
+ @Override
|
|
|
+ protected EnumSet<CreateFlag> getFlags() {
|
|
|
+ return super.getFlags();
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Build HdfsDataOutputStream to write.
|
|
|
+ *
|
|
|
+ * @return a fully-initialized OutputStream.
|
|
|
+ * @throws IOException on I/O errors.
|
|
|
+ */
|
|
|
@Override
|
|
|
public HdfsDataOutputStream build() throws IOException {
|
|
|
- Preconditions.checkState(
|
|
|
- !(shouldReplicate() && (!StringUtils.isEmpty(getEcPolicyName()))),
|
|
|
- "shouldReplicate and ecPolicyName are " +
|
|
|
- "exclusive parameters. Set both is not allowed!");
|
|
|
-
|
|
|
- EnumSet<CreateFlag> createFlags = getFlags();
|
|
|
- if (shouldReplicate()) {
|
|
|
- createFlags.add(CreateFlag.SHOULD_REPLICATE);
|
|
|
- }
|
|
|
- return dfs.create(getPath(), getPermission(), createFlags,
|
|
|
- getBufferSize(), getReplication(), getBlockSize(),
|
|
|
- getProgress(), getChecksumOpt(), getFavoredNodes(),
|
|
|
- getEcPolicyName());
|
|
|
+ if (isRecursive()) {
|
|
|
+ return dfs.create(getPath(), getPermission(), getFlags(),
|
|
|
+ getBufferSize(), getReplication(), getBlockSize(),
|
|
|
+ getProgress(), getChecksumOpt(), getFavoredNodes(),
|
|
|
+ getEcPolicyName());
|
|
|
+ } else {
|
|
|
+ return dfs.createNonRecursive(getPath(), getPermission(), getFlags(),
|
|
|
+ getBufferSize(), getReplication(), getBlockSize(), getProgress(),
|
|
|
+ getChecksumOpt(), getFavoredNodes(), getEcPolicyName());
|
|
|
+ }
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ /**
|
|
|
+ * Create a HdfsDataOutputStreamBuilder to create a file on DFS.
|
|
|
+ * Similar to {@link #create(Path)}, file is overwritten by default.
|
|
|
+ *
|
|
|
+ * @param path the path of the file to create.
|
|
|
+ * @return A HdfsDataOutputStreamBuilder for creating a file.
|
|
|
+ */
|
|
|
@Override
|
|
|
- public HdfsDataOutputStreamBuilder newFSDataOutputStreamBuilder(Path path) {
|
|
|
- return new HdfsDataOutputStreamBuilder(this, path);
|
|
|
+ public HdfsDataOutputStreamBuilder createFile(Path path) {
|
|
|
+ return new HdfsDataOutputStreamBuilder(this, path).create().overwrite(true);
|
|
|
}
|
|
|
}
|