HADOOP-7223. FileContext createFlag combinations are not clearly defined. Contributed by Suresh Srinivas.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1091613 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Suresh Srinivas 2011-04-13 00:09:12 +00:00
parent 6e5bd7b622
commit f5efc187e5
7 changed files with 173 additions and 80 deletions

View File

@ -84,6 +84,9 @@ Trunk (unreleased changes)
HADOOP-7202. Improve shell Command base class. (Daryn Sharp via szetszwo)
HADOOP-7223. FileContext createFlag combinations are not clearly defined.
(suresh)
OPTIMIZATIONS
BUG FIXES

View File

@ -337,8 +337,9 @@ public ChecksumFSOutputSummer(final ChecksumFs fs, final Path file,
int bytesPerSum = fs.getBytesPerSum();
int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
this.sums = fs.getRawFs().createInternal(fs.getChecksumFile(file),
EnumSet.of(CreateFlag.OVERWRITE), absolutePermission, sumBufferSize,
replication, blockSize, progress, bytesPerChecksum, createParent);
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
absolutePermission, sumBufferSize, replication, blockSize, progress,
bytesPerChecksum, createParent);
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
sums.writeInt(bytesPerSum);
}

View File

@ -17,6 +17,11 @@
*/
package org.apache.hadoop.fs;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.EnumSet;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceStability;
@ -25,41 +30,50 @@
* <code>
* EnumSet.of(CreateFlag.CREATE, CreateFlag.APPEND)
* <code>
* and pass it to {@link org.apache.hadoop.fs.FileSystem #create(Path f, FsPermission permission,
* EnumSet<CreateFlag> flag, int bufferSize, short replication, long blockSize,
* Progressable progress)}.
*
* <p>
* Combine {@link #OVERWRITE} with either {@link #CREATE}
* or {@link #APPEND} does the same as only use
* {@link #OVERWRITE}. <br>
* Combine {@link #CREATE} with {@link #APPEND} has the semantic:
*
* Use the CreateFlag as follows:
* <ol>
* <li> create the file if it does not exist;
* <li> append the file if it already exists.
* <li> CREATE - to create a file if it does not exist,
* else throw FileAlreadyExists.</li>
* <li> APPEND - to append to a file if it exists,
* else throw FileNotFoundException.</li>
* <li> OVERWRITE - to truncate a file if it exists,
* else throw FileNotFoundException.</li>
* <li> CREATE|APPEND - to create a file if it does not exist,
* else append to an existing file.</li>
* <li> CREATE|OVERWRITE - to create a file if it does not exist,
* else overwrite an existing file.</li>
* </ol>
*
* Following combination is not valid and will result in
* {@link HadoopIllegalArgumentException}:
* <ol>
* <li> APPEND|OVERWRITE</li>
* <li> CREATE|APPEND|OVERWRITE</li>
* </ol>
*****************************************************************/
@InterfaceAudience.Public
@InterfaceStability.Stable
@InterfaceStability.Evolving
public enum CreateFlag {
/**
* create the file if it does not exist, and throw an IOException if it
* Create a file. See javadoc for more description
* already exists
*/
CREATE((short) 0x01),
/**
* create the file if it does not exist, if it exists, overwrite it.
* Truncate/overwrite a file. Same as POSIX O_TRUNC. See javadoc for description.
*/
OVERWRITE((short) 0x02),
/**
* append to a file, and throw an IOException if it does not exist
* Append to a file. See javadoc for more description.
*/
APPEND((short) 0x04);
private short mode;
private final short mode;
private CreateFlag(short mode) {
this.mode = mode;
@ -68,4 +82,38 @@ private CreateFlag(short mode) {
short getMode() {
return mode;
}
/**
* Validate the CreateFlag for create operation
* @param path Object representing the path; usually String or {@link Path}
* @param pathExists pass true if the path exists in the file system
* @param flag set of CreateFlag
* @throws IOException on error
* @throws HadoopIllegalArgumentException if the CreateFlag is invalid
*/
public static void validate(Object path, boolean pathExists,
EnumSet<CreateFlag> flag) throws IOException {
if (flag == null || flag.isEmpty()) {
throw new HadoopIllegalArgumentException(flag
+ " does not specify any options");
}
final boolean append = flag.contains(APPEND);
final boolean overwrite = flag.contains(OVERWRITE);
// Both append and overwrite is an error
if (append && overwrite) {
throw new HadoopIllegalArgumentException(
flag + "Both append and overwrite options cannot be enabled.");
}
if (pathExists) {
if (!(append || overwrite)) {
throw new FileAlreadyExistsException("File already exists: "
+ path.toString()
+ ". Append or overwrite option must be specified in " + flag);
}
} else if (!flag.contains(CREATE)) {
throw new FileNotFoundException("Non existing file: " + path.toString()
+ ". Create option is not specified in " + flag);
}
}
}

View File

@ -509,7 +509,7 @@ public Path makeQualified(final Path path) {
* writing into the file.
*
* @param f the file name to open
* @param createFlag gives the semantics of create: overwrite, append etc.
* @param createFlag gives the semantics of create; see {@link CreateFlag}
* @param opts file creation options; see {@link Options.CreateOpts}.
* <ul>
* <li>Progress - to report progress on the operation - default null
@ -2055,7 +2055,10 @@ public boolean copy(final Path src, final Path dst, boolean deleteSource,
OutputStream out = null;
try {
in = open(qSrc);
out = create(qDst, EnumSet.of(CreateFlag.OVERWRITE));
EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
CreateFlag.CREATE, CreateFlag.OVERWRITE) :
EnumSet.of(CreateFlag.CREATE);
out = create(qDst, createFlag);
IOUtils.copyBytes(in, out, conf, true);
} catch (IOException e) {
IOUtils.closeStream(out);

View File

@ -700,23 +700,20 @@ protected FSDataOutputStream primitiveCreate(Path f,
short replication, long blockSize, Progressable progress,
int bytesPerChecksum) throws IOException {
boolean pathExists = exists(f);
CreateFlag.validate(f, pathExists, flag);
// 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 (exists(f)) {
if (flag.contains(CreateFlag.APPEND)) {
if (pathExists && flag.contains(CreateFlag.APPEND)) {
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());
}
return this.create(f, absolutePermission, flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
return this.create(f, absolutePermission,
flag.contains(CreateFlag.OVERWRITE), bufferSize, replication,
blockSize, progress);
}

View File

@ -264,28 +264,6 @@ public FSDataOutputStream create(Path f, FsPermission permission,
return out;
}
@Override
protected FSDataOutputStream primitiveCreate(Path f,
FsPermission absolutePermission, EnumSet<CreateFlag> flag,
int bufferSize, short replication, long blockSize, Progressable progress,
int bytesPerChecksum) throws IOException {
if(flag.contains(CreateFlag.APPEND)){
if (!exists(f)){
if(flag.contains(CreateFlag.CREATE)) {
return create(f, false, bufferSize, replication, blockSize, null);
}
}
return append(f, bufferSize, null);
}
FSDataOutputStream out = create(f, flag.contains(CreateFlag.OVERWRITE),
bufferSize, replication, blockSize, progress);
setPermission(f, absolutePermission);
return out;
}
public boolean rename(Path src, Path dst) throws IOException {
if (pathToFile(src).renameTo(pathToFile(dst))) {
return true;

View File

@ -21,8 +21,8 @@
import java.io.FileNotFoundException;
import java.io.IOException;
import java.util.EnumSet;
import java.util.Iterator;
import org.apache.hadoop.HadoopIllegalArgumentException;
import org.apache.hadoop.fs.Options.CreateOpts;
import org.apache.hadoop.fs.Options.Rename;
import org.apache.hadoop.fs.permission.FsPermission;
@ -32,6 +32,7 @@
import org.junit.Test;
import static org.apache.hadoop.fs.FileContextTestHelper.*;
import static org.apache.hadoop.fs.CreateFlag.*;
/**
* <p>
@ -155,7 +156,7 @@ public void testWorkingDirectory() throws Exception {
// Now open a file relative to the wd we just set above.
Path absolutePath = new Path(absoluteDir, "foo");
fc.create(absolutePath, EnumSet.of(CreateFlag.CREATE)).close();
fc.create(absolutePath, EnumSet.of(CREATE)).close();
fc.open(new Path("foo")).close();
@ -645,7 +646,7 @@ private void writeReadAndDelete(int len) throws IOException {
fc.mkdir(path.getParent(), FsPermission.getDefault(), true);
FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
CreateOpts.repFac((short) 1), CreateOpts
.blockSize(getDefaultBlockSize()));
out.write(data, 0, len);
@ -670,31 +671,93 @@ private void writeReadAndDelete(int len) throws IOException {
}
@Test
public void testOverwrite() throws IOException {
Path path = getTestRootPath(fc, "test/hadoop/file");
fc.mkdir(path.getParent(), FsPermission.getDefault(), true);
createFile(path);
Assert.assertTrue("Exists", exists(fc, path));
Assert.assertEquals("Length", data.length, fc.getFileStatus(path).getLen());
try {
fc.create(path, EnumSet.of(CreateFlag.CREATE));
Assert.fail("Should throw IOException.");
} catch (IOException e) {
// Expected
@Test(expected=HadoopIllegalArgumentException.class)
public void testNullCreateFlag() throws IOException {
Path p = getTestRootPath(fc, "test/file");
fc.create(p, null);
Assert.fail("Excepted exception not thrown");
}
FSDataOutputStream out = fc.create(path,EnumSet.of(CreateFlag.OVERWRITE));
@Test(expected=HadoopIllegalArgumentException.class)
public void testEmptyCreateFlag() throws IOException {
Path p = getTestRootPath(fc, "test/file");
fc.create(p, EnumSet.noneOf(CreateFlag.class));
Assert.fail("Excepted exception not thrown");
}
@Test(expected=FileAlreadyExistsException.class)
public void testCreateFlagCreateExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/file");
createFile(p);
fc.create(p, EnumSet.of(CREATE));
Assert.fail("Excepted exception not thrown");
}
@Test(expected=FileNotFoundException.class)
public void testCreateFlagOverwriteNonExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
fc.create(p, EnumSet.of(OVERWRITE));
Assert.fail("Excepted exception not thrown");
}
@Test
public void testCreateFlagOverwriteExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/file");
createFile(p);
FSDataOutputStream out = fc.create(p, EnumSet.of(OVERWRITE));
writeData(fc, p, out, data, data.length);
}
@Test(expected=FileNotFoundException.class)
public void testCreateFlagAppendNonExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
fc.create(p, EnumSet.of(APPEND));
Assert.fail("Excepted exception not thrown");
}
@Test
public void testCreateFlagAppendExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/file");
createFile(p);
FSDataOutputStream out = fc.create(p, EnumSet.of(APPEND));
writeData(fc, p, out, data, 2 * data.length);
}
@Test
public void testCreateFlagCreateAppendNonExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
FSDataOutputStream out = fc.create(p, EnumSet.of(CREATE, APPEND));
writeData(fc, p, out, data, data.length);
}
@Test
public void testCreateFlagCreateAppendExistingFile() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
createFile(p);
FSDataOutputStream out = fc.create(p, EnumSet.of(CREATE, APPEND));
writeData(fc, p, out, data, 2*data.length);
}
@Test(expected=HadoopIllegalArgumentException.class)
public void testCreateFlagAppendOverwrite() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
fc.create(p, EnumSet.of(APPEND, OVERWRITE));
Assert.fail("Excepted exception not thrown");
}
@Test(expected=HadoopIllegalArgumentException.class)
public void testCreateFlagAppendCreateOverwrite() throws IOException {
Path p = getTestRootPath(fc, "test/nonExistent");
fc.create(p, EnumSet.of(CREATE, APPEND, OVERWRITE));
Assert.fail("Excepted exception not thrown");
}
private static void writeData(FileContext fc, Path p, FSDataOutputStream out,
byte[] data, long expectedLen) throws IOException {
out.write(data, 0, data.length);
out.close();
Assert.assertTrue("Exists", exists(fc, path));
Assert.assertEquals("Length", data.length, fc.getFileStatus(path).getLen());
Assert.assertTrue("Exists", exists(fc, p));
Assert.assertEquals("Length", expectedLen, fc.getFileStatus(p).getLen());
}
@Test
@ -1057,7 +1120,7 @@ public void testOutputStreamClosedTwice() throws IOException {
//HADOOP-4760 according to Closeable#close() closing already-closed
//streams should have no effect.
Path src = getTestRootPath(fc, "test/hadoop/file");
FSDataOutputStream out = fc.create(src, EnumSet.of(CreateFlag.CREATE),
FSDataOutputStream out = fc.create(src, EnumSet.of(CREATE),
Options.CreateOpts.createParent());
out.writeChar('H'); //write some data
@ -1091,7 +1154,7 @@ public void testUnsupportedSymlink() throws IOException {
}
protected void createFile(Path path) throws IOException {
FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
Options.CreateOpts.createParent());
out.write(data, 0, data.length);
out.close();