Undoing commit r1091613 for HADOOP-7223. FileContext createFlag combinations are not clearly defined.
git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1091916 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d358eb75b7
commit
a12ab4bc6f
|
@ -86,9 +86,6 @@ Trunk (unreleased changes)
|
|||
|
||||
HADOOP-7202. Improve shell Command base class. (Daryn Sharp via szetszwo)
|
||||
|
||||
HADOOP-7223. FileContext createFlag combinations are not clearly defined.
|
||||
(suresh)
|
||||
|
||||
HADOOP-7224. Add CommandFactory to shell. (Daryn Sharp via szetszwo)
|
||||
|
||||
OPTIMIZATIONS
|
||||
|
|
|
@ -337,9 +337,8 @@ public abstract class ChecksumFs extends FilterFs {
|
|||
int bytesPerSum = fs.getBytesPerSum();
|
||||
int sumBufferSize = fs.getSumBufferSize(bytesPerSum, bufferSize);
|
||||
this.sums = fs.getRawFs().createInternal(fs.getChecksumFile(file),
|
||||
EnumSet.of(CreateFlag.CREATE, CreateFlag.OVERWRITE),
|
||||
absolutePermission, sumBufferSize, replication, blockSize, progress,
|
||||
bytesPerChecksum, createParent);
|
||||
EnumSet.of(CreateFlag.OVERWRITE), absolutePermission, sumBufferSize,
|
||||
replication, blockSize, progress, bytesPerChecksum, createParent);
|
||||
sums.write(CHECKSUM_VERSION, 0, CHECKSUM_VERSION.length);
|
||||
sums.writeInt(bytesPerSum);
|
||||
}
|
||||
|
|
|
@ -17,63 +17,49 @@
|
|||
*/
|
||||
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;
|
||||
|
||||
/****************************************************************
|
||||
* CreateFlag specifies the file create semantic. Users can combine flags like: <br>
|
||||
* <code>
|
||||
*CreateFlag specifies the file create semantic. Users can combine flags like:<br>
|
||||
*<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>
|
||||
*
|
||||
* Use the CreateFlag as follows:
|
||||
* 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:
|
||||
* <ol>
|
||||
* <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>
|
||||
* <li> create the file if it does not exist;
|
||||
* <li> append the file if it already exists.
|
||||
* </ol>
|
||||
*****************************************************************/
|
||||
@InterfaceAudience.Public
|
||||
@InterfaceStability.Evolving
|
||||
@InterfaceStability.Stable
|
||||
public enum CreateFlag {
|
||||
|
||||
/**
|
||||
* Create a file. See javadoc for more description
|
||||
* create the file if it does not exist, and throw an IOException if it
|
||||
* already exists
|
||||
*/
|
||||
CREATE((short) 0x01),
|
||||
|
||||
/**
|
||||
* Truncate/overwrite a file. Same as POSIX O_TRUNC. See javadoc for description.
|
||||
* create the file if it does not exist, if it exists, overwrite it.
|
||||
*/
|
||||
OVERWRITE((short) 0x02),
|
||||
|
||||
/**
|
||||
* Append to a file. See javadoc for more description.
|
||||
* append to a file, and throw an IOException if it does not exist
|
||||
*/
|
||||
APPEND((short) 0x04);
|
||||
|
||||
private final short mode;
|
||||
private short mode;
|
||||
|
||||
private CreateFlag(short mode) {
|
||||
this.mode = mode;
|
||||
|
@ -82,38 +68,4 @@ public enum CreateFlag {
|
|||
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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -511,7 +511,7 @@ public final class FileContext {
|
|||
* writing into the file.
|
||||
*
|
||||
* @param f the file name to open
|
||||
* @param createFlag gives the semantics of create; see {@link CreateFlag}
|
||||
* @param createFlag gives the semantics of create: overwrite, append etc.
|
||||
* @param opts file creation options; see {@link Options.CreateOpts}.
|
||||
* <ul>
|
||||
* <li>Progress - to report progress on the operation - default null
|
||||
|
@ -2057,10 +2057,7 @@ public final class FileContext {
|
|||
OutputStream out = null;
|
||||
try {
|
||||
in = open(qSrc);
|
||||
EnumSet<CreateFlag> createFlag = overwrite ? EnumSet.of(
|
||||
CreateFlag.CREATE, CreateFlag.OVERWRITE) :
|
||||
EnumSet.of(CreateFlag.CREATE);
|
||||
out = create(qDst, createFlag);
|
||||
out = create(qDst, EnumSet.of(CreateFlag.OVERWRITE));
|
||||
IOUtils.copyBytes(in, out, conf, true);
|
||||
} catch (IOException e) {
|
||||
IOUtils.closeStream(out);
|
||||
|
|
|
@ -717,21 +717,24 @@ public abstract class FileSystem extends Configured implements Closeable {
|
|||
FsPermission absolutePermission, EnumSet<CreateFlag> flag, int bufferSize,
|
||||
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 (pathExists && flag.contains(CreateFlag.APPEND)) {
|
||||
return append(f, bufferSize, progress);
|
||||
if (exists(f)) {
|
||||
if (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);
|
||||
}
|
||||
|
||||
|
|
|
@ -264,6 +264,28 @@ public class RawLocalFileSystem extends FileSystem {
|
|||
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;
|
||||
|
|
|
@ -21,8 +21,8 @@ package org.apache.hadoop.fs;
|
|||
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,7 +32,6 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
|
||||
import static org.apache.hadoop.fs.FileContextTestHelper.*;
|
||||
import static org.apache.hadoop.fs.CreateFlag.*;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
|
@ -156,7 +155,7 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
|
||||
// Now open a file relative to the wd we just set above.
|
||||
Path absolutePath = new Path(absoluteDir, "foo");
|
||||
fc.create(absolutePath, EnumSet.of(CREATE)).close();
|
||||
fc.create(absolutePath, EnumSet.of(CreateFlag.CREATE)).close();
|
||||
fc.open(new Path("foo")).close();
|
||||
|
||||
|
||||
|
@ -646,7 +645,7 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
|
||||
fc.mkdir(path.getParent(), FsPermission.getDefault(), true);
|
||||
|
||||
FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
|
||||
FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
|
||||
CreateOpts.repFac((short) 1), CreateOpts
|
||||
.blockSize(getDefaultBlockSize()));
|
||||
out.write(data, 0, len);
|
||||
|
@ -671,93 +670,31 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
|
||||
}
|
||||
|
||||
@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");
|
||||
}
|
||||
|
||||
@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 {
|
||||
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
|
||||
}
|
||||
|
||||
FSDataOutputStream out = fc.create(path,EnumSet.of(CreateFlag.OVERWRITE));
|
||||
out.write(data, 0, data.length);
|
||||
out.close();
|
||||
Assert.assertTrue("Exists", exists(fc, p));
|
||||
Assert.assertEquals("Length", expectedLen, fc.getFileStatus(p).getLen());
|
||||
|
||||
Assert.assertTrue("Exists", exists(fc, path));
|
||||
Assert.assertEquals("Length", data.length, fc.getFileStatus(path).getLen());
|
||||
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -1120,7 +1057,7 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
//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(CREATE),
|
||||
FSDataOutputStream out = fc.create(src, EnumSet.of(CreateFlag.CREATE),
|
||||
Options.CreateOpts.createParent());
|
||||
|
||||
out.writeChar('H'); //write some data
|
||||
|
@ -1154,7 +1091,7 @@ public abstract class FileContextMainOperationsBaseTest {
|
|||
}
|
||||
|
||||
protected void createFile(Path path) throws IOException {
|
||||
FSDataOutputStream out = fc.create(path, EnumSet.of(CREATE),
|
||||
FSDataOutputStream out = fc.create(path, EnumSet.of(CreateFlag.CREATE),
|
||||
Options.CreateOpts.createParent());
|
||||
out.write(data, 0, data.length);
|
||||
out.close();
|
||||
|
|
Loading…
Reference in New Issue