|
@@ -22,11 +22,11 @@ import org.apache.hadoop.fs.FSDataOutputStream;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileStatus;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
import org.apache.hadoop.fs.FileSystem;
|
|
|
|
+import org.apache.hadoop.fs.ParentNotDirectoryException;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.apache.hadoop.fs.Path;
|
|
import org.junit.Test;
|
|
import org.junit.Test;
|
|
-import org.junit.internal.AssumptionViolatedException;
|
|
|
|
|
|
+import org.junit.AssumptionViolatedException;
|
|
|
|
|
|
-import java.io.FileNotFoundException;
|
|
|
|
import java.io.IOException;
|
|
import java.io.IOException;
|
|
|
|
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
|
|
import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset;
|
|
@@ -40,7 +40,7 @@ import static org.apache.hadoop.fs.contract.ContractTestUtils.writeTextFile;
|
|
* Test creating files, overwrite options etc.
|
|
* Test creating files, overwrite options etc.
|
|
*/
|
|
*/
|
|
public abstract class AbstractContractCreateTest extends
|
|
public abstract class AbstractContractCreateTest extends
|
|
- AbstractFSContractTestBase {
|
|
|
|
|
|
+ AbstractFSContractTestBase {
|
|
|
|
|
|
/**
|
|
/**
|
|
* How long to wait for a path to become visible.
|
|
* How long to wait for a path to become visible.
|
|
@@ -113,7 +113,6 @@ public abstract class AbstractContractCreateTest extends
|
|
* This test catches some eventual consistency problems that blobstores exhibit,
|
|
* This test catches some eventual consistency problems that blobstores exhibit,
|
|
* as we are implicitly verifying that updates are consistent. This
|
|
* as we are implicitly verifying that updates are consistent. This
|
|
* is why different file lengths and datasets are used
|
|
* is why different file lengths and datasets are used
|
|
- * @throws Throwable
|
|
|
|
*/
|
|
*/
|
|
@Test
|
|
@Test
|
|
public void testOverwriteExistingFile() throws Throwable {
|
|
public void testOverwriteExistingFile() throws Throwable {
|
|
@@ -137,10 +136,6 @@ public abstract class AbstractContractCreateTest extends
|
|
} catch (FileAlreadyExistsException expected) {
|
|
} catch (FileAlreadyExistsException expected) {
|
|
//expected
|
|
//expected
|
|
handleExpectedException(expected);
|
|
handleExpectedException(expected);
|
|
- } catch (FileNotFoundException e) {
|
|
|
|
- handleRelaxedException("overwriting a dir with a file ",
|
|
|
|
- "FileAlreadyExistsException",
|
|
|
|
- e);
|
|
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
handleRelaxedException("overwriting a dir with a file ",
|
|
handleRelaxedException("overwriting a dir with a file ",
|
|
"FileAlreadyExistsException",
|
|
"FileAlreadyExistsException",
|
|
@@ -189,10 +184,6 @@ public abstract class AbstractContractCreateTest extends
|
|
} catch (FileAlreadyExistsException expected) {
|
|
} catch (FileAlreadyExistsException expected) {
|
|
//expected
|
|
//expected
|
|
handleExpectedException(expected);
|
|
handleExpectedException(expected);
|
|
- } catch (FileNotFoundException e) {
|
|
|
|
- handleRelaxedException("overwriting a dir with a file ",
|
|
|
|
- "FileAlreadyExistsException",
|
|
|
|
- e);
|
|
|
|
} catch (IOException e) {
|
|
} catch (IOException e) {
|
|
handleRelaxedException("overwriting a dir with a file ",
|
|
handleRelaxedException("overwriting a dir with a file ",
|
|
"FileAlreadyExistsException",
|
|
"FileAlreadyExistsException",
|
|
@@ -332,4 +323,117 @@ public abstract class AbstractContractCreateTest extends
|
|
assertTrue("Grandparent directory does not appear to be a directory",
|
|
assertTrue("Grandparent directory does not appear to be a directory",
|
|
fs.getFileStatus(grandparent).isDirectory());
|
|
fs.getFileStatus(grandparent).isDirectory());
|
|
}
|
|
}
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testCreateFileUnderFile() throws Throwable {
|
|
|
|
+ describe("Verify that it is forbidden to create file/file");
|
|
|
|
+ if (isSupported(CREATE_FILE_UNDER_FILE_ALLOWED)) {
|
|
|
|
+ // object store or some file systems: downgrade to a skip so that the
|
|
|
|
+ // failure is visible in test results
|
|
|
|
+ skip("This filesystem supports creating files under files");
|
|
|
|
+ }
|
|
|
|
+ Path grandparent = methodPath();
|
|
|
|
+ Path parent = new Path(grandparent, "parent");
|
|
|
|
+ expectCreateUnderFileFails(
|
|
|
|
+ "creating a file under a file",
|
|
|
|
+ grandparent,
|
|
|
|
+ parent);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testCreateUnderFileSubdir() throws Throwable {
|
|
|
|
+ describe("Verify that it is forbidden to create file/dir/file");
|
|
|
|
+ if (isSupported(CREATE_FILE_UNDER_FILE_ALLOWED)) {
|
|
|
|
+ // object store or some file systems: downgrade to a skip so that the
|
|
|
|
+ // failure is visible in test results
|
|
|
|
+ skip("This filesystem supports creating files under files");
|
|
|
|
+ }
|
|
|
|
+ Path grandparent = methodPath();
|
|
|
|
+ Path parent = new Path(grandparent, "parent");
|
|
|
|
+ Path child = new Path(parent, "child");
|
|
|
|
+ expectCreateUnderFileFails(
|
|
|
|
+ "creating a file under a subdirectory of a file",
|
|
|
|
+ grandparent,
|
|
|
|
+ child);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMkdirUnderFile() throws Throwable {
|
|
|
|
+ describe("Verify that it is forbidden to create file/dir");
|
|
|
|
+ Path grandparent = methodPath();
|
|
|
|
+ Path parent = new Path(grandparent, "parent");
|
|
|
|
+ expectMkdirsUnderFileFails("mkdirs() under a file",
|
|
|
|
+ grandparent, parent);
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ @Test
|
|
|
|
+ public void testMkdirUnderFileSubdir() throws Throwable {
|
|
|
|
+ describe("Verify that it is forbidden to create file/dir/dir");
|
|
|
|
+ Path grandparent = methodPath();
|
|
|
|
+ Path parent = new Path(grandparent, "parent");
|
|
|
|
+ Path child = new Path(parent, "child");
|
|
|
|
+ expectMkdirsUnderFileFails("mkdirs() file/dir",
|
|
|
|
+ grandparent, child);
|
|
|
|
+
|
|
|
|
+ try {
|
|
|
|
+ // create the child
|
|
|
|
+ mkdirs(child);
|
|
|
|
+ } catch (FileAlreadyExistsException | ParentNotDirectoryException ex) {
|
|
|
|
+ // either of these may be raised.
|
|
|
|
+ handleExpectedException(ex);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ handleRelaxedException("creating a file under a subdirectory of a file ",
|
|
|
|
+ "FileAlreadyExistsException",
|
|
|
|
+ e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ /**
|
|
|
|
+ * Expect that touch() will fail because the parent is a file.
|
|
|
|
+ * @param action action for message
|
|
|
|
+ * @param file filename to create
|
|
|
|
+ * @param descendant path under file
|
|
|
|
+ * @throws Exception failure
|
|
|
|
+ */
|
|
|
|
+ protected void expectCreateUnderFileFails(String action,
|
|
|
|
+ Path file, Path descendant)
|
|
|
|
+ throws Exception {
|
|
|
|
+ createFile(file);
|
|
|
|
+ try {
|
|
|
|
+ // create the child
|
|
|
|
+ createFile(descendant);
|
|
|
|
+ } catch (FileAlreadyExistsException | ParentNotDirectoryException ex) {
|
|
|
|
+ //expected
|
|
|
|
+ handleExpectedException(ex);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ handleRelaxedException(action,
|
|
|
|
+ "ParentNotDirectoryException",
|
|
|
|
+ e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ protected void expectMkdirsUnderFileFails(String action,
|
|
|
|
+ Path file, Path descendant)
|
|
|
|
+ throws Exception {
|
|
|
|
+ createFile(file);
|
|
|
|
+ try {
|
|
|
|
+ // now mkdirs
|
|
|
|
+ mkdirs(descendant);
|
|
|
|
+ } catch (FileAlreadyExistsException | ParentNotDirectoryException ex) {
|
|
|
|
+ //expected
|
|
|
|
+ handleExpectedException(ex);
|
|
|
|
+ } catch (IOException e) {
|
|
|
|
+ handleRelaxedException(action,
|
|
|
|
+ "ParentNotDirectoryException",
|
|
|
|
+ e);
|
|
|
|
+ }
|
|
|
|
+ }
|
|
|
|
+
|
|
|
|
+ private void createFile(Path path) throws IOException {
|
|
|
|
+ byte[] data = dataset(256, 'a', 'z');
|
|
|
|
+ FileSystem fs = getFileSystem();
|
|
|
|
+ writeDataset(fs, path, data, data.length, 1024 * 1024,
|
|
|
|
+ true);
|
|
|
|
+ }
|
|
}
|
|
}
|