HADOOP-6826. Revert FileSystem create method that takes CreateFlags.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@956710 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
4b8e1bda2d
commit
35a4624771
|
@ -1651,6 +1651,9 @@ Release 0.21.0 - Unreleased
|
||||||
HADOOP-6461. Webapps aren't located correctly post-split.
|
HADOOP-6461. Webapps aren't located correctly post-split.
|
||||||
(Todd Lipcon and Steve Loughran via tomwhite)
|
(Todd Lipcon and Steve Loughran via tomwhite)
|
||||||
|
|
||||||
|
HADOOP-6826. Revert FileSystem create method that takes CreateFlags.
|
||||||
|
(tomwhite)
|
||||||
|
|
||||||
Release 0.20.3 - Unreleased
|
Release 0.20.3 - Unreleased
|
||||||
|
|
||||||
NEW FEATURES
|
NEW FEATURES
|
||||||
|
|
|
@ -20,7 +20,6 @@ package org.apache.hadoop.fs;
|
||||||
|
|
||||||
import java.io.*;
|
import java.io.*;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.EnumSet;
|
|
||||||
import java.util.zip.CRC32;
|
import java.util.zip.CRC32;
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
|
@ -388,14 +387,14 @@ public abstract class ChecksumFileSystem extends FilterFileSystem {
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path f, FsPermission permission,
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
Path parent = f.getParent();
|
Path parent = f.getParent();
|
||||||
if (parent != null && !mkdirs(parent)) {
|
if (parent != null && !mkdirs(parent)) {
|
||||||
throw new IOException("Mkdirs failed to create " + parent);
|
throw new IOException("Mkdirs failed to create " + parent);
|
||||||
}
|
}
|
||||||
final FSDataOutputStream out = new FSDataOutputStream(
|
final FSDataOutputStream out = new FSDataOutputStream(
|
||||||
new ChecksumFSOutputSummer(this, f, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
|
new ChecksumFSOutputSummer(this, f, overwrite, bufferSize, replication,
|
||||||
blockSize, progress), null);
|
blockSize, progress), null);
|
||||||
if (permission != null) {
|
if (permission != null) {
|
||||||
setPermission(f, permission);
|
setPermission(f, permission);
|
||||||
|
|
|
@ -607,15 +607,13 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
long blockSize,
|
long blockSize,
|
||||||
Progressable progress
|
Progressable progress
|
||||||
) throws IOException {
|
) throws IOException {
|
||||||
return this.create(f, FsPermission.getDefault(), overwrite ? EnumSet
|
return this.create(f, FsPermission.getDefault(), overwrite, bufferSize,
|
||||||
.of(CreateFlag.OVERWRITE) : EnumSet.of(CreateFlag.CREATE), bufferSize,
|
|
||||||
replication, blockSize, progress);
|
replication, blockSize, progress);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Opens an FSDataOutputStream at the indicated Path with write-progress
|
* Opens an FSDataOutputStream at the indicated Path with write-progress
|
||||||
* reporting.
|
* reporting.
|
||||||
* @deprecated Consider using {@link #create(Path, FsPermission, EnumSet, int, short, long, Progressable)} instead.
|
|
||||||
* @param f the file name to open
|
* @param f the file name to open
|
||||||
* @param permission
|
* @param permission
|
||||||
* @param overwrite if a file with this name already exists, then if true,
|
* @param overwrite if a file with this name already exists, then if true,
|
||||||
|
@ -627,35 +625,14 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* @see #setPermission(Path, FsPermission)
|
* @see #setPermission(Path, FsPermission)
|
||||||
*/
|
*/
|
||||||
public FSDataOutputStream create(Path f,
|
public abstract FSDataOutputStream create(Path f,
|
||||||
FsPermission permission,
|
FsPermission permission,
|
||||||
boolean overwrite,
|
boolean overwrite,
|
||||||
int bufferSize,
|
int bufferSize,
|
||||||
short replication,
|
short replication,
|
||||||
long blockSize,
|
long blockSize,
|
||||||
Progressable progress) throws IOException{
|
Progressable progress) throws IOException;
|
||||||
return create(f, permission, overwrite ? EnumSet.of(CreateFlag.OVERWRITE)
|
|
||||||
: EnumSet.of(CreateFlag.CREATE), bufferSize, replication, blockSize,
|
|
||||||
progress);
|
|
||||||
}
|
|
||||||
|
|
||||||
/**
|
|
||||||
* Opens an FSDataOutputStream at the indicated Path with write-progress
|
|
||||||
* reporting.
|
|
||||||
* @param f the file name to open.
|
|
||||||
* @param permission - applied against umask
|
|
||||||
* @param flag determines the semantic of this create.
|
|
||||||
* @param bufferSize the size of the buffer to be used.
|
|
||||||
* @param replication required block replication for the file.
|
|
||||||
* @param blockSize
|
|
||||||
* @param progress
|
|
||||||
* @throws IOException
|
|
||||||
* @see #setPermission(Path, FsPermission)
|
|
||||||
* @see CreateFlag
|
|
||||||
*/
|
|
||||||
public abstract FSDataOutputStream create(Path f, FsPermission permission,
|
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
|
||||||
Progressable progress) throws IOException ;
|
|
||||||
|
|
||||||
/*.
|
/*.
|
||||||
* This create has been added to support the FileContext that processes
|
* This create has been added to support the FileContext that processes
|
||||||
|
@ -675,116 +652,21 @@ public abstract class FileSystem extends Configured implements Closeable {
|
||||||
// calling the regular create is good enough.
|
// calling the regular create is good enough.
|
||||||
// FSs that implement permissions should override this.
|
// FSs that implement permissions should override this.
|
||||||
|
|
||||||
return this.create(f, absolutePermission, flag, bufferSize, replication,
|
if (exists(f)) {
|
||||||
blockSize, progress);
|
if (flag.contains(CreateFlag.APPEND)) {
|
||||||
|
return append(f, bufferSize, progress);
|
||||||
|
} else if (!flag.contains(CreateFlag.OVERWRITE)) {
|
||||||
|
throw new IOException("File already exists: " + f);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/*.
|
|
||||||
* This create has been added to support the FileContext that passes
|
|
||||||
* an absolute permission with (ie umask was already applied)
|
|
||||||
* This a temporary method added to support the transition from FileSystem
|
|
||||||
* to FileContext for user applications.
|
|
||||||
*/
|
|
||||||
@Deprecated
|
|
||||||
protected FSDataOutputStream primitiveCreate(final Path f,
|
|
||||||
final EnumSet<CreateFlag> createFlag,
|
|
||||||
CreateOpts... opts) throws IOException {
|
|
||||||
checkPath(f);
|
|
||||||
int bufferSize = -1;
|
|
||||||
short replication = -1;
|
|
||||||
long blockSize = -1;
|
|
||||||
int bytesPerChecksum = -1;
|
|
||||||
FsPermission permission = null;
|
|
||||||
Progressable progress = null;
|
|
||||||
Boolean createParent = null;
|
|
||||||
|
|
||||||
for (CreateOpts iOpt : opts) {
|
|
||||||
if (CreateOpts.BlockSize.class.isInstance(iOpt)) {
|
|
||||||
if (blockSize != -1) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
blockSize = ((CreateOpts.BlockSize) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.BufferSize.class.isInstance(iOpt)) {
|
|
||||||
if (bufferSize != -1) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
bufferSize = ((CreateOpts.BufferSize) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.ReplicationFactor.class.isInstance(iOpt)) {
|
|
||||||
if (replication != -1) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
replication = ((CreateOpts.ReplicationFactor) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.BytesPerChecksum.class.isInstance(iOpt)) {
|
|
||||||
if (bytesPerChecksum != -1) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
bytesPerChecksum = ((CreateOpts.BytesPerChecksum) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.Perms.class.isInstance(iOpt)) {
|
|
||||||
if (permission != null) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
permission = ((CreateOpts.Perms) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.Progress.class.isInstance(iOpt)) {
|
|
||||||
if (progress != null) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
progress = ((CreateOpts.Progress) iOpt).getValue();
|
|
||||||
} else if (CreateOpts.CreateParent.class.isInstance(iOpt)) {
|
|
||||||
if (createParent != null) {
|
|
||||||
throw new IllegalArgumentException("multiple varargs of same kind");
|
|
||||||
}
|
|
||||||
createParent = ((CreateOpts.CreateParent) iOpt).getValue();
|
|
||||||
} else {
|
} else {
|
||||||
throw new IllegalArgumentException("Unkown CreateOpts of type " +
|
if (flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
|
||||||
iOpt.getClass().getName());
|
throw new IOException("File already exists: " + f.toString());
|
||||||
}
|
|
||||||
}
|
|
||||||
if (blockSize % bytesPerChecksum != 0) {
|
|
||||||
throw new IllegalArgumentException(
|
|
||||||
"blockSize should be a multiple of checksumsize");
|
|
||||||
}
|
}
|
||||||
|
|
||||||
FsServerDefaults ssDef = getServerDefaults();
|
return this.create(f, absolutePermission, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
|
||||||
|
|
||||||
if (blockSize == -1) {
|
|
||||||
blockSize = ssDef.getBlockSize();
|
|
||||||
}
|
|
||||||
if (bufferSize == -1) {
|
|
||||||
bufferSize = ssDef.getFileBufferSize();
|
|
||||||
}
|
|
||||||
if (replication == -1) {
|
|
||||||
replication = ssDef.getReplication();
|
|
||||||
}
|
|
||||||
if (permission == null) {
|
|
||||||
permission = FsPermission.getDefault();
|
|
||||||
}
|
|
||||||
if (createParent == null) {
|
|
||||||
createParent = false;
|
|
||||||
}
|
|
||||||
|
|
||||||
// Default impl assumes that permissions do not matter and
|
|
||||||
// nor does the bytesPerChecksum hence
|
|
||||||
// calling the regular create is good enough.
|
|
||||||
// FSs that implement permissions should override this.
|
|
||||||
|
|
||||||
if (!createParent) { // parent must exist.
|
|
||||||
// since this.create makes parent dirs automatically
|
|
||||||
// we must throw exception if parent does not exist.
|
|
||||||
final FileStatus stat = getFileStatus(f.getParent());
|
|
||||||
if (stat == null) {
|
|
||||||
throw new FileNotFoundException("Missing parent:" + f);
|
|
||||||
}
|
|
||||||
if (!stat.isDirectory()) {
|
|
||||||
throw new ParentNotDirectoryException("parent is not a dir:" + f);
|
|
||||||
}
|
|
||||||
// parent does exist - go ahead with create of file.
|
|
||||||
}
|
|
||||||
return this.create(f, permission, createFlag, bufferSize, replication,
|
|
||||||
blockSize, progress);
|
blockSize, progress);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* This version of the mkdirs method assumes that the permission is absolute.
|
* This version of the mkdirs method assumes that the permission is absolute.
|
||||||
* It has been added to support the FileContext that processes the permission
|
* It has been added to support the FileContext that processes the permission
|
||||||
|
|
|
@ -111,10 +111,10 @@ public class FilterFileSystem extends FileSystem {
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path f, FsPermission permission,
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
return fs.create(f, permission,
|
return fs.create(f, permission,
|
||||||
flag, bufferSize, replication, blockSize, progress);
|
overwrite, bufferSize, replication, blockSize, progress);
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
|
|
@ -244,14 +244,13 @@ public class RawLocalFileSystem extends FileSystem {
|
||||||
/** {@inheritDoc} */
|
/** {@inheritDoc} */
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path f, FsPermission permission,
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
return primitiveCreate(f,
|
|
||||||
permission.applyUMask(FsPermission.getUMask(getConf())), flag,
|
|
||||||
bufferSize, replication, blockSize, progress, -1);
|
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
|
FSDataOutputStream out = create(f,
|
||||||
|
overwrite, bufferSize, replication, blockSize, progress);
|
||||||
|
setPermission(f, permission);
|
||||||
|
return out;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
|
@ -21,7 +21,6 @@ import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.InputStream;
|
import java.io.InputStream;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.EnumSet;
|
|
||||||
|
|
||||||
import org.apache.commons.logging.Log;
|
import org.apache.commons.logging.Log;
|
||||||
import org.apache.commons.logging.LogFactory;
|
import org.apache.commons.logging.LogFactory;
|
||||||
|
@ -32,7 +31,6 @@ import org.apache.commons.net.ftp.FTPReply;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CreateFlag;
|
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
@ -200,28 +198,18 @@ public class FTPFileSystem extends FileSystem {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path file, FsPermission permission,
|
public FSDataOutputStream create(Path file, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
final FTPClient client = connect();
|
final FTPClient client = connect();
|
||||||
Path workDir = new Path(client.printWorkingDirectory());
|
Path workDir = new Path(client.printWorkingDirectory());
|
||||||
Path absolute = makeAbsolute(workDir, file);
|
Path absolute = makeAbsolute(workDir, file);
|
||||||
|
|
||||||
boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
|
|
||||||
boolean create = flag.contains(CreateFlag.CREATE);
|
|
||||||
boolean append= flag.contains(CreateFlag.APPEND);
|
|
||||||
|
|
||||||
if (exists(client, file)) {
|
if (exists(client, file)) {
|
||||||
if (overwrite) {
|
if (overwrite) {
|
||||||
delete(client, file);
|
delete(client, file);
|
||||||
} else if(append){
|
|
||||||
return append(file, bufferSize, progress);
|
|
||||||
} else {
|
} else {
|
||||||
disconnect(client);
|
disconnect(client);
|
||||||
throw new IOException("File already exists: " + file);
|
throw new IOException("File already exists: " + file);
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
if(append && !create)
|
|
||||||
throw new FileNotFoundException("File does not exist: "+ file);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Path parent = absolute.getParent();
|
Path parent = absolute.getParent();
|
||||||
|
|
|
@ -22,13 +22,11 @@ package org.apache.hadoop.fs.kfs;
|
||||||
import java.io.FileNotFoundException;
|
import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.EnumSet;
|
|
||||||
|
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.BlockLocation;
|
import org.apache.hadoop.fs.BlockLocation;
|
||||||
import org.apache.hadoop.fs.CreateFlag;
|
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
@ -191,25 +189,16 @@ public class KosmosFileSystem extends FileSystem {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path file, FsPermission permission,
|
public FSDataOutputStream create(Path file, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize,
|
boolean overwrite, int bufferSize,
|
||||||
short replication, long blockSize, Progressable progress)
|
short replication, long blockSize, Progressable progress)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
|
|
||||||
boolean create = flag.contains(CreateFlag.CREATE);
|
|
||||||
boolean append= flag.contains(CreateFlag.APPEND);
|
|
||||||
|
|
||||||
if (exists(file)) {
|
if (exists(file)) {
|
||||||
if (overwrite) {
|
if (overwrite) {
|
||||||
delete(file, true);
|
delete(file, true);
|
||||||
} else if (append){
|
|
||||||
return append(file, bufferSize, progress);
|
|
||||||
} else {
|
} else {
|
||||||
throw new IOException("File already exists: " + file);
|
throw new IOException("File already exists: " + file);
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
if(append && !create)
|
|
||||||
throw new FileNotFoundException("File does not exist: "+ file);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
Path parent = file.getParent();
|
Path parent = file.getParent();
|
||||||
|
|
|
@ -22,7 +22,6 @@ import java.io.FileNotFoundException;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.net.URI;
|
import java.net.URI;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.EnumSet;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -31,7 +30,6 @@ import java.util.concurrent.TimeUnit;
|
||||||
import org.apache.hadoop.classification.InterfaceAudience;
|
import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.CreateFlag;
|
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FileStatus;
|
import org.apache.hadoop.fs.FileStatus;
|
||||||
|
@ -207,24 +205,18 @@ public class S3FileSystem extends FileSystem {
|
||||||
*/
|
*/
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path file, FsPermission permission,
|
public FSDataOutputStream create(Path file, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize,
|
boolean overwrite, int bufferSize,
|
||||||
short replication, long blockSize, Progressable progress)
|
short replication, long blockSize, Progressable progress)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
|
|
||||||
INode inode = store.retrieveINode(makeAbsolute(file));
|
INode inode = store.retrieveINode(makeAbsolute(file));
|
||||||
if (inode != null) {
|
if (inode != null) {
|
||||||
if (flag.contains(CreateFlag.OVERWRITE)) {
|
if (overwrite) {
|
||||||
delete(file, true);
|
delete(file, true);
|
||||||
} else if (flag.contains(CreateFlag.APPEND)){
|
|
||||||
return append(file, bufferSize, progress);
|
|
||||||
} else {
|
} else {
|
||||||
throw new IOException("File already exists: " + file);
|
throw new IOException("File already exists: " + file);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
|
||||||
if(flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
|
|
||||||
throw new FileNotFoundException("File does not exist: "+ file);
|
|
||||||
|
|
||||||
Path parent = file.getParent();
|
Path parent = file.getParent();
|
||||||
if (parent != null) {
|
if (parent != null) {
|
||||||
if (!mkdirs(parent)) {
|
if (!mkdirs(parent)) {
|
||||||
|
|
|
@ -30,7 +30,6 @@ import java.security.DigestOutputStream;
|
||||||
import java.security.MessageDigest;
|
import java.security.MessageDigest;
|
||||||
import java.security.NoSuchAlgorithmException;
|
import java.security.NoSuchAlgorithmException;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.EnumSet;
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -44,7 +43,6 @@ import org.apache.hadoop.classification.InterfaceAudience;
|
||||||
import org.apache.hadoop.classification.InterfaceStability;
|
import org.apache.hadoop.classification.InterfaceStability;
|
||||||
import org.apache.hadoop.conf.Configuration;
|
import org.apache.hadoop.conf.Configuration;
|
||||||
import org.apache.hadoop.fs.BufferedFSInputStream;
|
import org.apache.hadoop.fs.BufferedFSInputStream;
|
||||||
import org.apache.hadoop.fs.CreateFlag;
|
|
||||||
import org.apache.hadoop.fs.FSDataInputStream;
|
import org.apache.hadoop.fs.FSDataInputStream;
|
||||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||||
import org.apache.hadoop.fs.FSInputStream;
|
import org.apache.hadoop.fs.FSInputStream;
|
||||||
|
@ -326,18 +324,11 @@ public class NativeS3FileSystem extends FileSystem {
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public FSDataOutputStream create(Path f, FsPermission permission,
|
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||||
EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
|
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||||
Progressable progress) throws IOException {
|
Progressable progress) throws IOException {
|
||||||
|
|
||||||
if(exists(f)) {
|
if (exists(f) && !overwrite) {
|
||||||
if(flag.contains(CreateFlag.APPEND)){
|
throw new IOException("File already exists:"+f);
|
||||||
return append(f, bufferSize, progress);
|
|
||||||
} else if(!flag.contains(CreateFlag.OVERWRITE)) {
|
|
||||||
throw new IOException("File already exists: "+f);
|
|
||||||
}
|
|
||||||
} else {
|
|
||||||
if (flag.contains(CreateFlag.APPEND) && !flag.contains(CreateFlag.CREATE))
|
|
||||||
throw new IOException("File already exists: " + f.toString());
|
|
||||||
}
|
}
|
||||||
|
|
||||||
LOG.debug("Creating new file '" + f + "' in S3");
|
LOG.debug("Creating new file '" + f + "' in S3");
|
||||||
|
|
Loading…
Reference in New Issue