|
@@ -521,6 +521,23 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
replication, blockSize, progress, buffersize);
|
|
replication, blockSize, progress, buffersize);
|
|
}
|
|
}
|
|
|
|
|
|
|
|
+ /**
|
|
|
|
+ * Call
|
|
|
|
+ * {@link #create(String,FsPermission,EnumSet,boolean,short,long,Progressable,int)}
|
|
|
|
+ * with createParent set to true.
|
|
|
|
+ */
|
|
|
|
+ public OutputStream create(String src,
|
|
|
|
+ FsPermission permission,
|
|
|
|
+ EnumSet<CreateFlag> flag,
|
|
|
|
+ short replication,
|
|
|
|
+ long blockSize,
|
|
|
|
+ Progressable progress,
|
|
|
|
+ int buffersize
|
|
|
|
+ ) throws IOException {
|
|
|
|
+ return create(src, permission, flag, true,
|
|
|
|
+ replication, blockSize, progress, buffersize);
|
|
|
|
+ }
|
|
|
|
+
|
|
/**
|
|
/**
|
|
* Create a new dfs file with the specified block replication
|
|
* Create a new dfs file with the specified block replication
|
|
* with write-progress reporting and return an output stream for writing
|
|
* with write-progress reporting and return an output stream for writing
|
|
@@ -530,14 +547,16 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
* @param permission The permission of the directory being created.
|
|
* @param permission The permission of the directory being created.
|
|
* If permission == null, use {@link FsPermission#getDefault()}.
|
|
* If permission == null, use {@link FsPermission#getDefault()}.
|
|
* @param flag do not check for file existence if true
|
|
* @param flag do not check for file existence if true
|
|
|
|
+ * @param createParent create missing parent directory if true
|
|
* @param replication block replication
|
|
* @param replication block replication
|
|
* @return output stream
|
|
* @return output stream
|
|
* @throws IOException
|
|
* @throws IOException
|
|
- * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, short, long)
|
|
|
|
|
|
+ * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, boolean, short, long)
|
|
*/
|
|
*/
|
|
public OutputStream create(String src,
|
|
public OutputStream create(String src,
|
|
FsPermission permission,
|
|
FsPermission permission,
|
|
EnumSet<CreateFlag> flag,
|
|
EnumSet<CreateFlag> flag,
|
|
|
|
+ boolean createParent,
|
|
short replication,
|
|
short replication,
|
|
long blockSize,
|
|
long blockSize,
|
|
Progressable progress,
|
|
Progressable progress,
|
|
@@ -550,7 +569,7 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
|
|
FsPermission masked = permission.applyUMask(FsPermission.getUMask(conf));
|
|
LOG.debug(src + ": masked=" + masked);
|
|
LOG.debug(src + ": masked=" + masked);
|
|
OutputStream result = new DFSOutputStream(src, masked,
|
|
OutputStream result = new DFSOutputStream(src, masked,
|
|
- flag, replication, blockSize, progress, buffersize,
|
|
|
|
|
|
+ flag, createParent, replication, blockSize, progress, buffersize,
|
|
conf.getInt("io.bytes.per.checksum", 512));
|
|
conf.getInt("io.bytes.per.checksum", 512));
|
|
leasechecker.put(src, result);
|
|
leasechecker.put(src, result);
|
|
return result;
|
|
return result;
|
|
@@ -3068,10 +3087,10 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
|
|
|
/**
|
|
/**
|
|
* Create a new output stream to the given DataNode.
|
|
* Create a new output stream to the given DataNode.
|
|
- * @see ClientProtocol#create(String, FsPermission, String, boolean, short, long)
|
|
|
|
|
|
+ * @see ClientProtocol#create(String, FsPermission, String, EnumSetWritable, boolean, short, long)
|
|
*/
|
|
*/
|
|
DFSOutputStream(String src, FsPermission masked, EnumSet<CreateFlag> flag,
|
|
DFSOutputStream(String src, FsPermission masked, EnumSet<CreateFlag> flag,
|
|
- short replication, long blockSize, Progressable progress,
|
|
|
|
|
|
+ boolean createParent, short replication, long blockSize, Progressable progress,
|
|
int buffersize, int bytesPerChecksum) throws IOException {
|
|
int buffersize, int bytesPerChecksum) throws IOException {
|
|
this(src, blockSize, progress, bytesPerChecksum);
|
|
this(src, blockSize, progress, bytesPerChecksum);
|
|
|
|
|
|
@@ -3079,9 +3098,11 @@ public class DFSClient implements FSConstants, java.io.Closeable {
|
|
|
|
|
|
try {
|
|
try {
|
|
namenode.create(
|
|
namenode.create(
|
|
- src, masked, clientName, new EnumSetWritable<CreateFlag>(flag), replication, blockSize);
|
|
|
|
|
|
+ src, masked, clientName, new EnumSetWritable<CreateFlag>(flag), createParent, replication, blockSize);
|
|
} catch(RemoteException re) {
|
|
} catch(RemoteException re) {
|
|
throw re.unwrapRemoteException(AccessControlException.class,
|
|
throw re.unwrapRemoteException(AccessControlException.class,
|
|
|
|
+ FileAlreadyExistsException.class,
|
|
|
|
+ FileNotFoundException.class,
|
|
NSQuotaExceededException.class,
|
|
NSQuotaExceededException.class,
|
|
DSQuotaExceededException.class);
|
|
DSQuotaExceededException.class);
|
|
}
|
|
}
|